diff --git a/.gitignore b/.gitignore
index 3d178992123da..cd9f90d55932c 100644
--- a/.gitignore
+++ b/.gitignore
@@ -7,6 +7,7 @@
sbt/*.jar
.settings
.cache
+.mima-excludes
/build/
work/
out/
@@ -45,3 +46,5 @@ dist/
spark-*-bin.tar.gz
unit-tests.log
/lib/
+rat-results.txt
+scalastyle.txt
diff --git a/.rat-excludes b/.rat-excludes
new file mode 100644
index 0000000000000..a2b5665a0be26
--- /dev/null
+++ b/.rat-excludes
@@ -0,0 +1,42 @@
+target
+.gitignore
+.project
+.classpath
+.mima-excludes
+.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
+test.out/*
+.*iml
\ No newline at end of file
diff --git a/.travis.yml b/.travis.yml
new file mode 100644
index 0000000000000..8ebd0d68429fc
--- /dev/null
+++ b/.travis.yml
@@ -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.
+
+ language: scala
+ scala:
+ - "2.10.3"
+ jdk:
+ - oraclejdk7
+ env:
+ matrix:
+ - TEST="scalastyle assembly/assembly"
+ - TEST="catalyst/test sql/test streaming/test mllib/test graphx/test bagel/test"
+ - TEST=hive/test
+ cache:
+ directories:
+ - $HOME/.m2
+ - $HOME/.ivy2
+ - $HOME/.sbt
+ script:
+ - "sbt ++$TRAVIS_SCALA_VERSION $TEST"
diff --git a/NOTICE b/NOTICE
index 7cbb114b2ae2d..42f6c3a835725 100644
--- a/NOTICE
+++ b/NOTICE
@@ -1,5 +1,14 @@
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/).
+
+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/assembly/pom.xml b/assembly/pom.xml
index 82a5985504b4e..923bf47f7076a 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.py4jpy4j
@@ -158,6 +163,26 @@
+
+ hive
+
+
+ org.apache.spark
+ spark-hive_${scala.binary.version}
+ ${project.version}
+
+
+
+
+ spark-ganglia-lgpl
+
+
+ org.apache.spark
+ spark-ganglia-lgpl_${scala.binary.version}
+ ${project.version}
+
+
+ bigtop-dist
+
+
+ yarn-alpha
+
+
+ org.apache.avro
+ avro
+
+
+
+
@@ -105,12 +117,10 @@
com.twitterchill_${scala.binary.version}
- 0.3.1com.twitterchill-java
- 0.3.1commons-net
@@ -138,7 +148,7 @@
json4s-jackson_${scala.binary.version}3.2.6
@@ -179,10 +189,6 @@
com.codahale.metricsmetrics-json
-
- com.codahale.metrics
- metrics-ganglia
- com.codahale.metricsmetrics-graphite
@@ -193,9 +199,51 @@
test
- commons-io
- commons-io
- 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
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/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala
index c4579cf6ad560..59fdf659c9e11 100644
--- a/core/src/main/scala/org/apache/spark/Aggregator.scala
+++ b/core/src/main/scala/org/apache/spark/Aggregator.scala
@@ -17,17 +17,18 @@
package org.apache.spark
-import scala.{Option, deprecated}
-
+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/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/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 cc30105940d1a..2c31cc20211ff 100644
--- a/core/src/main/scala/org/apache/spark/Dependency.scala
+++ b/core/src/main/scala/org/apache/spark/Dependency.scala
@@ -17,18 +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.
@@ -40,36 +46,46 @@ 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
- * @param serializerClass class name of the serializer to use
+ * @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.
*/
+@DeveloperApi
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()
+
+ rdd.sparkContext.cleaner.foreach(_.registerShuffleForCleanup(this))
}
/**
+ * :: 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 f2decd14ef6d9..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] {
@@ -141,17 +146,19 @@ 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)
}
}
}
/**
+ * :: 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..9d429dceeb858 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
@@ -53,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)
}
@@ -110,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")
}
}
@@ -128,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/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
index 5968973132942..ee82d9fa7874b 100644
--- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
+++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala
@@ -20,28 +20,43 @@ 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
-private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster)
+/** Actor class for 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!")
@@ -50,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)
@@ -79,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) {
@@ -137,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))
}
@@ -149,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) {
@@ -179,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")
}
}
@@ -201,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) {
@@ -208,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) {
@@ -223,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
@@ -259,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/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/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/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/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index cdc0e5a34240e..28923a1d8c340 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}
@@ -35,7 +34,10 @@ 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
import org.apache.spark.partial.{ApproximateEvaluator, PartialResult}
import org.apache.spark.rdd._
import org.apache.spark.scheduler._
@@ -44,25 +46,38 @@ 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}
/**
+ * :: 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
@@ -92,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()
/**
@@ -128,10 +178,18 @@ 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")
+ // 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 +197,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
@@ -147,14 +206,27 @@ 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)
- // Initialize the Spark UI
+ // Initialize the Spark UI, registering all associated listeners
private[spark] val ui = new SparkUI(this)
ui.bind()
+ // Optionally log Spark events
+ 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
+ }
+
+ // At this point, all relevant SparkListeners have been registered, so begin releasing events
+ listenerBus.start()
+
val startTime = System.currentTimeMillis()
// Add each JAR given through the constructor
@@ -162,7 +234,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
@@ -200,16 +272,26 @@ 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()
+ private[spark] val cleaner: Option[ContextCleaner] = {
+ if (conf.getBoolean("spark.cleaner.referenceTracking", true)) {
+ Some(new ContextCleaner(this))
+ } else {
+ None
+ }
+ }
+ cleaner.foreach(_.start())
+
+ postEnvironmentUpdate()
+ postApplicationStart()
/** 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
@@ -349,6 +431,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 preferred, as each file will be loaded fully in memory.
+ */
+ 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),
@@ -571,7 +686,6 @@ class SparkContext(
.flatMap(x => Utils.deserialize[Array[T]](x._2.getBytes))
}
-
protected[spark] def checkpointFile[T: ClassTag](
path: String
): RDD[T] = {
@@ -585,6 +699,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
/**
@@ -610,7 +727,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)
}
@@ -620,7 +737,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) = 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.
@@ -641,12 +762,21 @@ class SparkContext(
Utils.fetchFile(path, new File(SparkFiles.getRootDirectory), conf, env.securityManager)
logInfo("Added file " + path + " at " + key + " with timestamp " + addedFiles(key))
+ postEnvironmentUpdate()
}
+ /**
+ * :: DeveloperApi ::
+ * Register a listener to receive up-calls from events that happen during execution.
+ */
+ @DeveloperApi
def addSparkListener(listener: SparkListener) {
- dagScheduler.addSparkListener(listener)
+ 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.
@@ -671,10 +801,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
*/
@@ -698,7 +824,7 @@ class SparkContext(
}
/**
- * Return current scheduling mode
+ * Return current scheduling mode
*/
def getSchedulingMode: SchedulingMode.SchedulingMode = {
taskScheduler.schedulingMode
@@ -708,6 +834,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 +849,22 @@ 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(rddId: Int, blocking: Boolean = true) {
+ 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 +887,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 +895,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,18 +916,21 @@ 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()
}
/** 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.
@@ -792,16 +938,16 @@ class SparkContext(
dagScheduler = null
if (dagSchedulerCopy != null) {
metadataCleaner.cancel()
+ cleaner.foreach(_.stop())
dagSchedulerCopy.stop()
taskScheduler = null
// TODO: Cache.stop()?
env.stop()
- // Clean up locally linked files
- clearFiles()
- clearJars()
SparkEnv.set(null)
ShuffleMapTask.clearCache()
ResultTask.clearCache()
+ listenerBus.stop()
+ eventLogger.foreach(_.stop())
logInfo("Successfully stopped SparkContext")
} else {
logInfo("SparkContext already stopped")
@@ -832,13 +978,13 @@ 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
- }
+ val defaultCallSite = Utils.getCallSiteInfo
+ Option(getLocalProperty("externalCallSite")).getOrElse(defaultCallSite.toString)
}
/**
@@ -853,6 +999,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)
@@ -930,8 +1079,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,
@@ -949,6 +1100,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,
@@ -956,6 +1108,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(
@@ -1021,6 +1176,33 @@ class SparkContext(
/** 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) {
+ 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)
@@ -1033,6 +1215,8 @@ class SparkContext(
*/
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"
@@ -1067,7 +1251,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](
@@ -1104,27 +1288,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])
@@ -1184,11 +1374,9 @@ 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 =
- {
- // Regular expression used for local[N] master format
- val LOCAL_N_REGEX = """local\[([0-9]+)\]""".r
+ private def createTaskScheduler(sc: SparkContext, master: String): TaskScheduler = {
+ // 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
@@ -1211,8 +1399,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
@@ -1225,7 +1416,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
@@ -1242,7 +1433,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()
@@ -1302,9 +1493,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 5e43b5198422c..915315ed74436 100644
--- a/core/src/main/scala/org/apache/spark/SparkEnv.scala
+++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala
@@ -17,31 +17,39 @@
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
+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
-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.scheduler.LiveListenerBus
+import org.apache.spark.serializer.Serializer
+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 serializerManager: SerializerManager,
val serializer: Serializer,
val closureSerializer: Serializer,
val cacheManager: CacheManager,
@@ -50,11 +58,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
@@ -79,7 +87,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]
@@ -121,9 +129,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)
@@ -139,16 +154,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) {
@@ -165,11 +186,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)), conf)
- val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster,
- serializer, conf, securityManager)
+ new BlockManagerMasterActor(isLocal, conf, listenerBus)), conf)
+
+ val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster,
+ serializer, conf, securityManager, mapOutputTracker)
val connectionManager = blockManager.connectionManager
@@ -177,17 +211,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]))
-
val shuffleFetcher = instantiateClass[ShuffleFetcher](
"spark.shuffle.fetcher", "org.apache.spark.BlockStoreShuffleFetcher")
@@ -220,7 +243,6 @@ object SparkEnv extends Logging {
new SparkEnv(
executorId,
actorSystem,
- serializerManager,
serializer,
closureSerializer,
cacheManager,
@@ -229,10 +251,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/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/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala
index cae983ed4c652..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,
@@ -46,6 +52,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/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala
index 3fd6f5eb472f4..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,22 +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/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala
new file mode 100644
index 0000000000000..4597595a838e3
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/TestUtils.scala
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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
+
+/**
+ * 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.
+ *
+ * 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], value: String = ""): URL = {
+ val tempDir = Files.createTempDir()
+ 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.
+ */
+ 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, value: String = ""): File = {
+ val compiler = ToolProvider.getSystemJavaCompiler
+ 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.
+ 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
+ }
+}
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 0ff428c120353..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
@@ -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
@@ -31,6 +32,7 @@ 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}
@@ -200,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)
@@ -250,14 +256,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 +373,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 +468,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 +476,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 +491,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 +499,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 +507,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)))
/**
@@ -558,6 +564,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,
@@ -687,21 +701,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/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..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
@@ -17,9 +17,9 @@
package org.apache.spark.api.java
-import java.util.{Comparator, List => JList}
+import java.util.{Comparator, List => JList, Iterator => JIterator}
+import java.lang.{Iterable => JIterable}
-import scala.Tuple2
import scala.collection.JavaConversions._
import scala.reflect.ClassTag
@@ -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, _}
@@ -204,7 +205,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 +215,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])))
@@ -281,9 +282,22 @@ 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
*/
+ @Deprecated
def toArray(): JList[T] = collect()
/**
@@ -330,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)
@@ -390,19 +408,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`.
@@ -419,7 +442,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
@@ -477,6 +502,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].
+ * @param 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].
+ * @param 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.
@@ -500,8 +545,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/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
index 8e0eab56a3dcf..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
@@ -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 preferred, as each file will be loaded fully in memory.
+ */
+ 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
@@ -434,6 +462,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 +471,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()
}
@@ -461,7 +491,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/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/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
index e4d0285710e84..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._
@@ -100,6 +101,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]] {
@@ -198,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]] = {
@@ -258,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)
}
@@ -278,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/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..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,18 +16,22 @@
*/
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 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 3cd71213769b7..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,18 +204,18 @@ 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
+ TorrentBroadcast.conf = conf // TODO: we might have to fix it in tests
synchronized {
if (!initialized) {
initialized = true
@@ -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/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala
index 449b953530ff9..86305d2ea8a09 100644
--- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala
@@ -23,7 +23,8 @@ private[spark] class ApplicationDescription(
val memoryPerSlave: Int,
val command: Command,
val sparkHome: Option[String],
- val appUiUrl: String)
+ var appUiUrl: String,
+ val eventLogDir: Option[String] = None)
extends Serializable {
val user = System.getProperty("user.name", "")
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/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/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/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/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/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/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
new file mode 100644
index 0000000000000..e05fbfe321495
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
@@ -0,0 +1,234 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.{PrintStream, 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 {
+ 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
+
+ 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, 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
+ */
+ private[spark] def createLaunchEnv(appArgs: SparkSubmitArguments): (ArrayBuffer[String],
+ ArrayBuffer[String], Map[String, String], String) = {
+ 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 {
+ printErrorAndExit("master must start with yarn, mesos, spark, or local")
+ }
+
+ // 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" || 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" || 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-cluster"
+ }
+ 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) {
+ printErrorAndExit("Mesos does not support running the driver on the cluster")
+ }
+
+ 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)
+ }
+
+ 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)
+
+ 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)
+ }
+
+ private def addJarToClasspath(localJar: String, loader: ExecutorURLClassLoader) {
+ val localJarFile = new File(localJar)
+ if (!localJarFile.exists()) {
+ printWarning(s"Jar $localJar does not exist, 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..834b3df2f164b
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala
@@ -0,0 +1,206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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
+ var verbose: Boolean = false
+
+ loadEnvVars()
+ 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
+ }
+
+ 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
+ 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") {
+ 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 ("--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 (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 =>
+ }
+
+ private def printUsageAndExit(exitCode: Int, unknownParam: Any = null) {
+ val outStream = SparkSubmit.printStream
+ if (unknownParam != null) {
+ outStream.println("Unknown/unsupported param " + unknownParam)
+ }
+ outStream.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
+ )
+ SparkSubmit.exitFn()
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/deploy/WebUI.scala b/core/src/main/scala/org/apache/spark/deploy/WebUI.scala
deleted file mode 100644
index ae258b58b9cc5..0000000000000
--- a/core/src/main/scala/org/apache/spark/deploy/WebUI.scala
+++ /dev/null
@@ -1,47 +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 java.text.SimpleDateFormat
-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")
-
- def formatDate(date: Date): String = DATE_FORMAT.format(date)
-
- def formatDate(timestamp: Long): String = DATE_FORMAT.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
- return "%.1f h".format(hours)
- }
-}
diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala
new file mode 100644
index 0000000000000..7f7372746f92e
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.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.{WebUIPage, UIUtils}
+
+private[spark] class IndexPage(parent: HistoryServer) extends WebUIPage("") {
+
+ 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 =
+
+ }
+}
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..f495edcf1c6af
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
@@ -0,0 +1,278 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 scala.collection.mutable
+
+import org.apache.hadoop.fs.{FileStatus, Path}
+
+import org.apache.spark.{Logging, SecurityManager, SparkConf}
+import org.apache.spark.scheduler._
+import org.apache.spark.ui.{WebUI, 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,
+ securityManager: SecurityManager,
+ conf: SparkConf)
+ 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)
+
+ // 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)
+ }
+ }
+ }
+ }
+
+ // A mapping of application ID to its history information, which includes the rendered UI
+ val appIdToInfo = mutable.HashMap[String, ApplicationHistoryInfo]()
+
+ initialize()
+
+ /**
+ * 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 initialize() {
+ attachPage(new IndexPage(this))
+ attachHandler(createStaticHandler(STATIC_RESOURCE_DIR, "/static"))
+ logCheckingThread.start()
+ }
+
+ /**
+ * 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)) {
+ detachSparkUI(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 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
+ replayBus.replay()
+ if (appListener.applicationStarted) {
+ attachSparkUI(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()
+ }
+
+ /** 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
+
+ /** 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 securityManager = new SecurityManager(conf)
+ val server = new HistoryServer(args.logDir, securityManager, conf)
+ server.bind()
+
+ // 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/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/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/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala
index b8dfa44102583..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
@@ -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,15 +38,22 @@ 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.{EventLoggingListener, ReplayListenerBus}
+import org.apache.spark.ui.SparkUI
import org.apache.spark.util.{AkkaUtils, Utils}
-private[spark] class Master(host: String, port: Int, webUiPort: Int,
- val securityMgr: SecurityManager) 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
- 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)
@@ -64,6 +72,9 @@ 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 fileSystemsUsed = new HashSet[FileSystem]
+
val drivers = new HashSet[DriverInfo]
val completedDrivers = new ArrayBuffer[DriverInfo]
val waitingDrivers = new ArrayBuffer[DriverInfo] // Drivers currently spooled for scheduling
@@ -107,8 +118,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 +152,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int,
override def postStop() {
webUi.stop()
+ fileSystemsUsed.foreach(_.close())
masterMetricsSystem.stop()
applicationMetricsSystem.stop()
persistenceEngine.close()
@@ -373,7 +385,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 +593,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 +625,20 @@ 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 => webUi.detachSparkUI(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
+ if (!rebuildSparkUI(app)) {
+ // 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,9 +653,40 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int,
}
}
+ /**
+ * Rebuild a new SparkUI from the given application's event logs.
+ * Return whether this is successful.
+ */
+ def rebuildSparkUI(app: ApplicationInfo): Boolean = {
+ val appName = app.desc.name
+ 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(
+ new SparkConf, replayBus, appName + " (completed)", "/history/" + app.id)
+ replayBus.replay()
+ app.desc.appUiUrl = ui.basePath
+ appIdToUI(app.id) = ui
+ webUi.attachSparkUI(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 */
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
}
@@ -660,7 +710,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int,
}
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
}
@@ -717,9 +767,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/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/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala
index 90cad3c37fda6..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
@@ -23,21 +23,21 @@ 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
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)
@@ -83,7 +83,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) {
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
deleted file mode 100644
index 3233cd97f7bd0..0000000000000
--- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala
+++ /dev/null
@@ -1,196 +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.ui
-
-import javax.servlet.http.HttpServletRequest
-
-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}
-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.util.Utils
-
-private[spark] class IndexPage(parent: MasterWebUI) {
- val master = parent.masterActorRef
- val timeout = parent.timeout
-
- def renderJson(request: HttpServletRequest): JValue = {
- val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse]
- val state = Await.result(stateFuture, timeout)
- JsonProtocol.writeMasterState(state)
- }
-
- /** Index view listing applications and executors */
- def render(request: HttpServletRequest): Seq[Node] = {
- val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse]
- val state = Await.result(stateFuture, timeout)
-
- val workerHeaders = Seq("Id", "Address", "State", "Cores", "Memory")
- val workers = state.workers.sortBy(_.id)
- val workerTable = UIUtils.listingTable(workerHeaders, workerRow, workers)
-
- val appHeaders = Seq("ID", "Name", "Cores", "Memory per Node", "Submitted Time", "User",
- "State", "Duration")
- val activeApps = state.activeApps.sortBy(_.startTime).reverse
- val activeAppsTable = UIUtils.listingTable(appHeaders, appRow, activeApps)
- val completedApps = state.completedApps.sortBy(_.endTime).reverse
- val completedAppsTable = UIUtils.listingTable(appHeaders, appRow, completedApps)
-
- val driverHeaders = Seq("ID", "Submitted Time", "Worker", "State", "Cores", "Memory",
- "Main Class")
- val activeDrivers = state.activeDrivers.sortBy(_.startTime).reverse
- val activeDriversTable = UIUtils.listingTable(driverHeaders, driverRow, activeDrivers)
- val completedDrivers = state.completedDrivers.sortBy(_.startTime).reverse
- val completedDriversTable = UIUtils.listingTable(driverHeaders, driverRow, completedDrivers)
-
- // For now we only show driver information if the user has submitted drivers to the cluster.
- // This is until we integrate the notion of drivers and applications in the UI.
- def hasDrivers = activeDrivers.length > 0 || completedDrivers.length > 0
-
- val content =
-
-
-
-
URL: {state.uri}
-
Workers: {state.workers.size}
-
Cores: {state.workers.map(_.cores).sum} Total,
- {state.workers.map(_.coresUsed).sum} Used
-
Memory:
- {Utils.megabytesToString(state.workers.map(_.memory).sum)} Total,
- {Utils.megabytesToString(state.workers.map(_.memoryUsed).sum)} Used
- }
-}
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
new file mode 100644
index 0000000000000..30c2e4b1563d8
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.ui
+
+import javax.servlet.http.HttpServletRequest
+
+import scala.concurrent.Await
+import scala.xml.Node
+
+import akka.pattern.ask
+import org.json4s.JValue
+
+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.{WebUIPage, UIUtils}
+import org.apache.spark.util.Utils
+
+private[spark] class IndexPage(parent: MasterWebUI) extends WebUIPage("") {
+ private val master = parent.masterActorRef
+ private val timeout = parent.timeout
+
+ override def renderJson(request: HttpServletRequest): JValue = {
+ val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse]
+ val state = Await.result(stateFuture, timeout)
+ JsonProtocol.writeMasterState(state)
+ }
+
+ /** Index view listing applications and executors */
+ def render(request: HttpServletRequest): Seq[Node] = {
+ val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse]
+ val state = Await.result(stateFuture, timeout)
+
+ val workerHeaders = Seq("Id", "Address", "State", "Cores", "Memory")
+ val workers = state.workers.sortBy(_.id)
+ val workerTable = UIUtils.listingTable(workerHeaders, workerRow, workers)
+
+ val appHeaders = Seq("ID", "Name", "Cores", "Memory per Node", "Submitted Time", "User",
+ "State", "Duration")
+ val activeApps = state.activeApps.sortBy(_.startTime).reverse
+ val activeAppsTable = UIUtils.listingTable(appHeaders, appRow, activeApps)
+ val completedApps = state.completedApps.sortBy(_.endTime).reverse
+ val completedAppsTable = UIUtils.listingTable(appHeaders, appRow, completedApps)
+
+ val driverHeaders = Seq("ID", "Submitted Time", "Worker", "State", "Cores", "Memory",
+ "Main Class")
+ val activeDrivers = state.activeDrivers.sortBy(_.startTime).reverse
+ val activeDriversTable = UIUtils.listingTable(driverHeaders, driverRow, activeDrivers)
+ val completedDrivers = state.completedDrivers.sortBy(_.startTime).reverse
+ val completedDriversTable = UIUtils.listingTable(driverHeaders, driverRow, completedDrivers)
+
+ // For now we only show driver information if the user has submitted drivers to the cluster.
+ // This is until we integrate the notion of drivers and applications in the UI.
+ def hasDrivers = activeDrivers.length > 0 || completedDrivers.length > 0
+
+ val content =
+
+
+
+
URL: {state.uri}
+
Workers: {state.workers.size}
+
Cores: {state.workers.map(_.cores).sum} Total,
+ {state.workers.map(_.coresUsed).sum} Used
+
Memory:
+ {Utils.megabytesToString(state.workers.map(_.memory).sum)} Total,
+ {Utils.megabytesToString(state.workers.map(_.memoryUsed).sum)} Used
- }
-}
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 =
+ 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/WorkerPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala
new file mode 100644
index 0000000000000..15b79872bc556
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.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.deploy.worker.ui
+
+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
+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.{WebUIPage, UIUtils}
+import org.apache.spark.util.Utils
+
+private[spark] class IndexPage(parent: WorkerWebUI) extends WebUIPage("") {
+ val workerActor = parent.worker.self
+ val worker = parent.worker
+ val timeout = parent.timeout
+
+ override def renderJson(request: HttpServletRequest): JValue = {
+ val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerStateResponse]
+ val workerState = Await.result(stateFuture, timeout)
+ JsonProtocol.writeWorkerState(workerState)
+ }
+
+ def render(request: HttpServletRequest): Seq[Node] = {
+ val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerStateResponse]
+ val workerState = Await.result(stateFuture, timeout)
+
+ val executorHeaders = Seq("ExecutorID", "Cores", "Memory", "Job Details", "Logs")
+ val runningExecutorTable =
+ UIUtils.listingTable(executorHeaders, executorRow, workerState.executors)
+ val finishedExecutorTable =
+ UIUtils.listingTable(executorHeaders, executorRow, workerState.finishedExecutors)
+
+ val driverHeaders = Seq("DriverID", "Main Class", "State", "Cores", "Memory", "Logs", "Notes")
+ val runningDrivers = workerState.drivers.sortBy(_.driverId).reverse
+ val runningDriverTable = UIUtils.listingTable(driverHeaders, driverRow, runningDrivers)
+ val finishedDrivers = workerState.finishedDrivers.sortBy(_.driverId).reverse
+ def finishedDriverTable = UIUtils.listingTable(driverHeaders, driverRow, finishedDrivers)
+
+ // For now we only show driver information if the user has submitted drivers to the cluster.
+ // This is until we integrate the notion of drivers and applications in the UI.
+ def hasDrivers = runningDrivers.length > 0 || finishedDrivers.length > 0
+
+ val content =
+
-
- 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. */
- 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)
- }
+ val timeout = AkkaUtils.askTimeout(worker.conf)
- def stop() {
- server.foreach(_.stop())
+ initialize()
+
+ /** Initialize all components of the server. */
+ def initialize() {
+ val logPage = new LogPage(this)
+ attachPage(logPage)
+ attachPage(new IndexPage(this))
+ attachHandler(createStaticHandler(WorkerWebUI.STATIC_RESOURCE_BASE, "/static"))
+ attachHandler(createServletHandler("/log",
+ (request: HttpServletRequest) => logPage.renderLog(request), worker.securityMgr))
+ worker.metricsSystem.getServletHandlers.foreach(attachHandler)
}
}
private[spark] object WorkerWebUI {
- val STATIC_RESOURCE_BASE = "org/apache/spark/ui"
- 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/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/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala
index e69f6f72d3275..c12bd922d40e4 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.
@@ -112,17 +112,14 @@ 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.
- 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")
@@ -226,10 +223,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
@@ -265,7 +262,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
@@ -278,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
@@ -295,7 +291,7 @@ 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 = {
+ private def createClassLoader(): MutableURLClassLoader = {
val loader = this.getClass.getClassLoader
// For each of the jars in the jarSet, add them to the class loader.
@@ -303,7 +299,11 @@ private[spark] class Executor(
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)
+ }
}
/**
@@ -314,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/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/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/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/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala
index 760458cb02a9b..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,21 +17,29 @@
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
+ * 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,13 +76,23 @@ 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 {
- 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
@@ -109,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/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/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala
index 848b5c439bb5b..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
@@ -38,8 +45,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,12 +53,20 @@ private[spark] object CompressionCodec {
.getConstructor(classOf[SparkConf])
ctor.newInstance(conf).asInstanceOf[CompressionCodec]
}
+
+ val DEFAULT_COMPRESSION_CODEC = classOf[LZFCompressionCodec].getName
}
/**
+ * :: 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 = {
@@ -64,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/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/metrics/sink/ConsoleSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala
index 4d2ffc54d8983..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"
@@ -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..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"
@@ -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..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"
@@ -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/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 3110eccdee4fc..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
@@ -28,9 +28,9 @@ 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,
+private[spark] class MetricsServlet(val property: Properties, val registry: MetricRegistry,
securityMgr: SecurityManager) extends Sink {
val SERVLET_KEY_PATH = "path"
val SERVLET_KEY_SAMPLE = "sample"
@@ -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/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/network/Connection.scala b/core/src/main/scala/org/apache/spark/network/Connection.scala
index 8fd9c2b87d256..2f7576c53b482 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..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._
@@ -505,7 +504,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf,
}
}
handleMessageExecutor.execute(runnable)
- /*handleMessage(connection, message)*/
+ /* handleMessage(connection, message) */
}
private def handleClientAuthentication(
@@ -733,7 +732,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 +858,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 +947,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..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 3c09a713c6fe0..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 aac2c24a46faa..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/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/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 699a10c96c227..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,7 +23,9 @@ 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
private[spark] sealed trait CoGroupSplitDep extends Serializable
@@ -50,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) {
@@ -66,10 +73,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 +87,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)
}
}
}
@@ -102,7 +109,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
@@ -113,18 +120,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/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 100ddb360732a..6811e1abb8b70 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,9 +29,13 @@ 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._
+import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.util.NextIterator
@@ -65,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.
@@ -79,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]],
@@ -111,6 +122,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
@@ -163,14 +177,16 @@ 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]
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 +238,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/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala
index 8df8718f3b65b..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.
@@ -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..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]],
@@ -80,7 +86,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/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
index 2384c8f2b6fd4..343e4325c0ef0 100644
--- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala
@@ -30,20 +30,22 @@ 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.
-import org.apache.hadoop.mapred.SparkHadoopWriter
-
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
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 +75,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 +95,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)
@@ -200,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)
@@ -260,7 +264,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.
@@ -269,14 +273,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))
}
@@ -297,7 +301,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)
}
}
@@ -310,9 +314,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))
}
}
}
@@ -327,9 +331,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)
}
}
}
@@ -357,7 +361,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))
}
@@ -423,7 +427,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) }
@@ -452,7 +456,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.")
}
@@ -467,13 +472,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]])
}
}
@@ -481,7 +488,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))
}
@@ -490,7 +497,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))
}
@@ -498,7 +505,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))
}
@@ -507,18 +514,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))
}
@@ -602,50 +609,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)
}
/**
@@ -691,6 +657,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
@@ -710,6 +729,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 + ")")
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 4250a9d02f764..e441d4a40ccd2 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,18 +30,20 @@ import scala.io.Source
import scala.reflect.ClassTag
import org.apache.spark.{Partition, SparkEnv, TaskContext}
+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],
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 3fe56963e0008..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. */
@@ -121,19 +134,11 @@ 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
}
- /** 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
@@ -145,9 +150,10 @@ 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)
+ // Register the RDD with the ContextCleaner for automatic GC-based cleanup
+ sc.cleaner.foreach(_.registerRDDForCleanup(this))
storageLevel = newLevel
- // Register the RDD with the SparkContext
- sc.persistentRdds(id) = this
this
}
@@ -165,8 +171,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(id, blocking)
storageLevel = StorageLevel.NONE
this
}
@@ -318,6 +323,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 {
@@ -352,6 +358,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 {
@@ -370,7 +380,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()
}
@@ -441,20 +451,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)
}
@@ -486,16 +496,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)
}
/**
@@ -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] = {
@@ -663,9 +678,22 @@ 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.
*/
+ @deprecated("use collect", "1.0.0")
def toArray(): Array[T] = collect()
/**
@@ -779,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
@@ -825,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
@@ -847,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
@@ -854,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()
@@ -931,32 +965,61 @@ 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] = takeOrdered(num)(ord.reverse)
+
+ /**
+ * 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 top(num: Int)(implicit ord: Ordering[T]): Array[T] = {
+ def takeOrdered(num: Int)(implicit ord: Ordering[T]): Array[T] = {
mapPartitions { items =>
- val queue = new BoundedPriorityQueue[T](num)
- queue ++= 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.reverse)
+ }.toArray.sorted(ord)
}
/**
- * Returns the first K elements from this RDD as defined by
- * the specified implicit Ordering[T] and maintains the
- * ordering.
- * @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)
+ * 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.
@@ -1031,8 +1094,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: String = creationSiteInfo.toString
private[spark] def elementClassTag: ClassTag[T] = classTag[T]
@@ -1095,13 +1159,9 @@ 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/rdd/SampledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala
index b50307cfa49b7..b097c30f8c231 100644
--- a/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala
@@ -26,14 +26,14 @@ 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")
-class SampledRDD[T: ClassTag](
+@deprecated("Replaced by PartitionwiseSampledRDD", "1.0.0")
+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 0bbda25a905cd..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,8 @@ 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 {
override val index = idx
@@ -27,26 +29,28 @@ 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)
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 +61,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/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/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/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala
index dc5b25d845dc2..c41d6d75a1d49 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.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)
-
- // 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 executorGained(execId: String, host: String) {
- eventProcessActor ! ExecutorGained(execId, host)
- }
+ import DAGScheduler._
- // 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)
+ def this(sc: SparkContext, taskScheduler: TaskScheduler) = {
+ this(
+ taskScheduler,
+ sc.listenerBus,
+ sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster],
+ sc.env.blockManager.master,
+ sc.env)
}
- // 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 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 HashMap[Stage, StageInfo]
- private[scheduler] val stageIdToJobIds = new TimeStampedHashMap[Int, HashSet[Int]]
+ // Stages we need to run whose parents aren't done
+ private[scheduler] val waitingStages = new HashSet[Stage]
- private[scheduler] val stageIdToStage = new TimeStampedHashMap[Int, Stage]
+ // Stages we are running right now
+ private[scheduler] val runningStages = new HashSet[Stage]
- private[scheduler] val shuffleToMapStage = new TimeStampedHashMap[Int, Stage]
+ // Stages that must be resubmitted due to fetch failures
+ private[scheduler] val failedStages = new HashSet[Stage]
- private[spark] val stageToInfos = new TimeStampedHashMap[Stage, StageInfo]
+ // Missing tasks from each stage
+ private[scheduler] val pendingTasks = new HashMap[Stage, HashSet[Task[_]]]
- // An async scheduler event bus. The bus should be stopped when DAGSCheduler is stopped.
- private[spark] val listenerBus = new SparkListenerBus
+ 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,9 @@ 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]
-
- // stage id to the active job
- val idToActiveJob = new HashMap[Int, ActiveJob]
+ private val failedEpoch = new HashMap[String, Long]
- 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 +149,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 +236,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
}
@@ -269,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) {
@@ -279,7 +265,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
@@ -356,29 +342,31 @@ 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
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,40 +376,48 @@ 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
stageIdToStage -= stageId
stageIdToJobIds -= stageId
+ ShuffleMapTask.removeStage(stageId)
+ ResultTask.removeStage(stageId)
+
logDebug("After removal of stage %d, remaining stages = %d"
.format(stageId, stageIdToStage.size))
}
- jobSet -= jobId
+ jobSet -= job.jobId
if (jobSet.isEmpty) { // no other job needs this stage
- independentStages += stageId
removeStage(stageId)
}
}
}
}
- independentStages.toSet
- }
-
- private def jobIdToStageIdsRemove(jobId: Int) {
- if (!jobIdToStageIds.contains(jobId)) {
- logDebug("Trying to remove unregistered job " + jobId)
+ jobIdToStageIds -= job.jobId
+ jobIdToActiveJob -= job.jobId
+ activeJobs -= job
+
+ 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
}
}
@@ -471,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
}
@@ -544,13 +540,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
+ jobIdToActiveJob(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 +560,22 @@ 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...
+ jobIdToActiveJob.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);
stage <- stageIdToStage.get(task.stageId);
stageInfo <- stageToInfos.get(stage)
) {
@@ -591,20 +587,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 +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, JobFailed(error, None)))
+ // 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
}
@@ -629,9 +636,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 +651,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 +692,7 @@ class DAGScheduler(
}
} catch {
case e: Exception =>
- jobResult = JobFailed(e, Some(job.finalStage))
+ jobResult = JobFailed(e)
job.listener.jobFailed(e)
} finally {
val s = job.finalStage
@@ -693,7 +700,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 +712,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(jobIdToActiveJob.contains)
} else {
None
}
@@ -716,18 +723,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,10 +765,10 @@ class DAGScheduler(
}
}
- val properties = if (idToActiveJob.contains(jobId)) {
- idToActiveJob(stage.jobId).properties
+ val properties = if (jobIdToActiveJob.contains(jobId)) {
+ jobIdToActiveJob(stage.jobId).properties
} else {
- //this stage will be assigned to "default" pool
+ // this stage will be assigned to "default" pool
null
}
@@ -779,20 +786,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 +824,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 +833,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 +842,9 @@ class DAGScheduler(
job.numFinished += 1
// If the whole job has finished, remove it
if (job.numFinished == job.numPartitions) {
- idToActiveJob -= stage.jobId
- activeJobs -= job
- resultStageToJob -= stage
markStageAsFinished(stage)
- jobIdToStageIdsRemove(job.jobId)
- listenerBus.post(SparkListenerJobEnd(job, JobSucceeded))
+ cleanupStateForJobAndIndependentStages(job, Some(stage))
+ listenerBus.post(SparkListenerJobEnd(job.jobId, JobSucceeded))
}
job.listener.taskSucceeded(rt.outputId, event.result)
}
@@ -858,12 +861,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 +889,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 +915,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 +927,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 +935,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 +983,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
}
}
@@ -992,15 +995,7 @@ class DAGScheduler(
if (!jobIdToStageIds.contains(jobId)) {
logDebug("Trying to cancel unregistered job " + jobId)
} else {
- val independentStages = removeJobAndIndependentStages(jobId)
- independentStages.foreach { taskSched.cancelTasks }
- val error = new SparkException("Job %d cancelled".format(jobId))
- val job = idToActiveJob(jobId)
- job.listener.jobFailed(error)
- jobIdToStageIds -= jobId
- activeJobs -= job
- idToActiveJob -= jobId
- listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, Some(job.finalStage))))
+ failJobAndIndependentStages(jobIdToActiveJob(jobId), s"Job $jobId cancelled", None)
}
}
@@ -1017,19 +1012,57 @@ 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)
- idToActiveJob -= resultStage.jobId
- activeJobs -= job
- resultStageToJob -= resultStage
- listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, Some(failedStage))))
+ 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")
}
}
+ /**
+ * 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,
+ resultStage: Option[Stage]) {
+ val error = new SparkException(failureReason)
+ job.listener.jobFailed(error)
+
+ // Cancel all independent, running stages.
+ val stages = jobIdToStageIds(job.jobId)
+ if (stages.isEmpty) {
+ logError("No stages registered for job " + job.jobId)
+ }
+ 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)))
+ }
+
/**
* Return true if one of stage's ancestors is target.
*/
@@ -1094,27 +1127,24 @@ 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()
- 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..b983c16af14f4
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala
@@ -0,0 +1,216 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 org.apache.hadoop.fs.{FileSystem, Path}
+import org.json4s.jackson.JsonMethods._
+
+import org.apache.spark.{Logging, SparkConf, SparkContext}
+import org.apache.spark.io.CompressionCodec
+import org.apache.spark.util.{FileLogger, JsonProtocol}
+
+/**
+ * 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 {
+
+ 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
+ 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)
+
+ /**
+ * 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)
+ }
+
+ /** 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)
+ 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
+ }
+ }
+
+ /**
+ * 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)
+ }
+}
+
+
+/**
+ * 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/InputFormatInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala
index 5555585c8b4cd..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 {
@@ -164,8 +167,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 +180,6 @@ object InputFormatInfo {
}
}
- nodeToSplit
+ nodeToSplit.mapValues(_.toSet).toMap
}
}
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..713aebfa3ce00 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,27 @@ 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.annotation.DeveloperApi
import org.apache.spark.executor.TaskMetrics
-import org.apache.spark.rdd.RDD
-import org.apache.spark.storage.StorageLevel
/**
+ * :: DeveloperApi ::
* 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 {
+@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 {
def this() = this(System.getProperty("user.name", ""),
String.valueOf(System.currentTimeMillis()))
@@ -51,19 +54,21 @@ 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 DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss")
- private val eventQueue = new LinkedBlockingQueue[SparkListenerEvents]
+ private val jobIdToPrintWriter = new HashMap[Int, PrintWriter]
+ private val stageIdToJobId = new HashMap[Int, Int]
+ private val jobIdToStageIds = new HashMap[Int, Seq[Int]]
+ private val dateFormat = new ThreadLocal[SimpleDateFormat]() {
+ override def initialValue() = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss")
+ }
+ 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,191 +85,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
+ writeInfo = dateFormat.get.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 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
- }
- 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
@@ -282,7 +174,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)
}
/**
@@ -290,8 +182,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))
}
/**
@@ -299,36 +192,34 @@ 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
+ if (stageCompleted.stageInfo.failureReason.isEmpty) {
+ stageLogInfo(stageId, s"STAGE_ID=$stageId STATUS=COMPLETED")
+ } else {
+ stageLogInfo(stageId, s"STAGE_ID=$stageId STATUS=FAILED")
+ }
}
- 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 _ =>
}
}
@@ -338,28 +229,28 @@ 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, _) =>
+ case JobFailed(exception) =>
info += " STATUS=FAILED REASON="
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)
}
}
@@ -368,14 +259,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..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
+
+@DeveloperApi
+case object JobSucceeded extends JobResult
-private[spark] case object JobSucceeded extends JobResult
-private[spark] case class JobFailed(exception: Exception, failedStage: Option[Stage])
- extends JobResult
+@DeveloperApi
+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 b026f860a8cd8..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, None)
+ jobResult = JobFailed(exception)
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..76f3e327d60b8
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.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.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/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..b03665fd56d33
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 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
+import org.apache.spark.io.CompressionCodec
+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.
+ */
+private[spark] class ReplayListenerBus(
+ logPaths: Seq[Path],
+ fileSystem: FileSystem,
+ compressionCodec: Option[CompressionCodec])
+ extends SparkListenerBus with Logging {
+
+ private var replayed = false
+
+ 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() {
+ 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 = ""
+ try {
+ fileStream = Some(fileSystem.open(path))
+ bufferedStream = Some(new FastBufferedInputStream(fileStream.get))
+ compressStream = Some(wrapForCompression(bufferedStream.get))
+
+ // 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
+ 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())
+ }
+ }
+ 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/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 77789031f464a..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,21 +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 {
@@ -79,6 +74,10 @@ private[spark] object ShuffleMapTask {
HashMap(set.toSeq: _*)
}
+ def removeStage(stageId: Int) {
+ serializedInfoCache.remove(stageId)
+ }
+
def clearCache() {
synchronized {
serializedInfoCache.clear()
@@ -153,7 +152,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/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
index 9590c03f10632..378cf1aaebe7b 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala
@@ -19,40 +19,80 @@ 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.annotation.DeveloperApi
import org.apache.spark.executor.TaskMetrics
+import org.apache.spark.storage.BlockManagerId
import org.apache.spark.util.{Distribution, Utils}
-sealed trait SparkListenerEvents
+@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
-case class SparkListenerStageSubmitted(stage: StageInfo, properties: Properties)
- extends SparkListenerEvents
+@DeveloperApi
+case class SparkListenerTaskEnd(
+ stageId: Int,
+ taskType: String,
+ reason: TaskEndReason,
+ taskInfo: TaskInfo,
+ taskMetrics: TaskMetrics)
+ extends SparkListenerEvent
-case class SparkListenerStageCompleted(stage: StageInfo) extends SparkListenerEvents
+@DeveloperApi
+case class SparkListenerJobStart(jobId: Int, stageIds: Seq[Int], properties: Properties = null)
+ extends SparkListenerEvent
-case class SparkListenerTaskStart(task: Task[_], taskInfo: TaskInfo) extends SparkListenerEvents
+@DeveloperApi
+case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkListenerEvent
-case class SparkListenerTaskGettingResult(
- task: Task[_], taskInfo: TaskInfo) extends SparkListenerEvents
+@DeveloperApi
+case class SparkListenerEnvironmentUpdate(environmentDetails: Map[String, Seq[(String, String)]])
+ extends SparkListenerEvent
-case class SparkListenerTaskEnd(task: Task[_], reason: TaskEndReason, taskInfo: TaskInfo,
- taskMetrics: TaskMetrics) extends SparkListenerEvents
+@DeveloperApi
+case class SparkListenerBlockManagerAdded(blockManagerId: BlockManagerId, maxMem: Long)
+ extends SparkListenerEvent
-case class SparkListenerJobStart(job: ActiveJob, stageIds: Array[Int],
- properties: Properties = null) extends SparkListenerEvents
+@DeveloperApi
+case class SparkListenerBlockManagerRemoved(blockManagerId: BlockManagerId)
+ extends SparkListenerEvent
-case class SparkListenerJobEnd(job: ActiveJob, jobResult: JobResult)
- extends SparkListenerEvents
+@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[scheduler] case object SparkListenerShutdown extends SparkListenerEvents
+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 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) { }
@@ -87,97 +127,146 @@ 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) { }
+
+ /**
+ * Called when the application starts
+ */
+ def onApplicationStart(applicationStart: SparkListenerApplicationStart) { }
+
+ /**
+ * Called when the application ends
+ */
+ def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) { }
}
/**
+ * :: 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._
+
+ 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))
+ (_, metric) => metric.shuffleReadMetrics.map(_.remoteBytesRead), taskInfoMetrics)
+ showBytesDistribution("task result size:",
+ (_, metric) => Some(metric.resultSize), taskInfoMetrics)
- //runtime breakdown
-
- 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 +278,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 +290,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 +312,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..d6df193d9bcf8 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,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.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 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/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/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/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala
index 8f320e5c7a74b..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,28 +17,36 @@
package org.apache.spark.scheduler
-import scala.collection._
-
-import org.apache.spark.executor.TaskMetrics
+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.
- *
- * 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
+@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
+ /** Time when all tasks in the stage completed or when the stage was cancelled. */
var completionTime: Option[Long] = None
- val rddName = stage.rdd.name
- val name = stage.name
- val numPartitions = stage.numPartitions
- val numTasks = stage.numTasks
+ /** 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]
+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/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 ea3229b75be36..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,8 +17,11 @@
package org.apache.spark.scheduler
-private[spark] object TaskLocality extends Enumeration {
- // process local is expected to be used ONLY within tasksetmanager for now.
+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
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 23b06612fd7ab..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.
*/
@@ -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]
@@ -204,7 +207,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)
}
}
@@ -228,15 +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) -= 1
- 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)
@@ -399,8 +405,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/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
index 5ea4557bbf56a..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,8 +56,14 @@ 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,
+ * 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)
@@ -71,7 +77,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 +236,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 +258,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 +283,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 +300,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 +312,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 +324,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 +341,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 +354,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))
}
}
@@ -349,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)
@@ -460,6 +491,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 +512,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 +520,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 +538,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 +568,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/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
index fad03731572e7..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
@@ -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)
}
}
@@ -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/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala
index ee4b65e312abc..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
@@ -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.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/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala
index 28b019d9fd495..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
@@ -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()
@@ -195,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 bcf0ce19a54cd..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
@@ -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()
@@ -224,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 {
@@ -247,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/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala
index bfa647f7f0516..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,13 +21,13 @@ 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, 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 +51,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 +60,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 +82,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 = {
@@ -95,8 +95,24 @@ private[spark] class JavaSerializerInstance(conf: SparkConf) extends SerializerI
}
/**
+ * :: 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.
*/
-class JavaSerializer(conf: SparkConf) extends Serializer {
- def newInstance(): SerializerInstance = new JavaSerializerInstance(conf)
+@DeveloperApi
+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..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,11 +33,19 @@ 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 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 +56,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 +66,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]
@@ -103,7 +111,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
}
}
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..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,25 +22,44 @@ 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}
/**
+ * :: 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 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.
+ *
+ * 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
}
+object Serializer {
+ def getSerializer(serializer: Serializer): Serializer = {
+ if (serializer == null) SparkEnv.get.serializer else serializer
+ }
+}
+
+
/**
+ * :: DeveloperApi ::
* An instance of a serializer, for use by one thread at a time.
*/
+@DeveloperApi
trait SerializerInstance {
def serialize[T](t: T): ByteBuffer
@@ -71,8 +90,10 @@ trait SerializerInstance {
/**
+ * :: DeveloperApi ::
* A stream for writing serialized objects.
*/
+@DeveloperApi
trait SerializationStream {
def writeObject[T](t: T): SerializationStream
def flush(): Unit
@@ -88,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/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/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/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 1bf3f4db32ea7..a2a729130091f 100644
--- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
+++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala
@@ -29,17 +29,17 @@ 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, 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
+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,
@@ -48,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)
@@ -57,8 +58,19 @@ 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 = {
+ 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 = {
@@ -89,10 +101,10 @@ 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 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 +134,16 @@ 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,
+ mapOutputTracker: MapOutputTracker) = {
+ this(execId, actorSystem, master, serializer, BlockManager.getMaxMemory(conf),
+ conf, securityManager, mapOutputTracker)
}
/**
@@ -148,14 +166,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 +182,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 +211,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
@@ -202,24 +221,45 @@ 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 getLevel(blockId: BlockId): StorageLevel = blockInfo.get(blockId).map(_.level).orNull
+ 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 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
* 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 +270,45 @@ 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 inTachyonSize = status.tachyonSize
+ val onDiskSize = status.diskSize
+ master.updateBlockInfo(
+ blockManagerId, blockId, storageLevel, inMemSize, onDiskSize, inTachyonSize)
+ } 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, inTachyonSize) = info.synchronized {
info.level match {
case null =>
- (StorageLevel.NONE, 0L, 0L, false)
+ (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 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 || 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, info.tellMaster)
+ (storageLevel, memSize, diskSize, tachyonSize)
}
}
-
- if (tellMaster) {
- master.updateBlockInfo(blockManagerId, blockId, curLevel, inMemSize, onDiskSize)
- } else {
- true
- }
+ BlockStatus(newLevel, inMemSize, onDiskSize, inTachyonSize)
}
/**
@@ -330,6 +388,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) {
@@ -398,10 +474,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 +523,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,52 +536,71 @@ 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)
}
/**
* 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, 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 +608,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 +630,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 +645,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 +660,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 +669,25 @@ 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 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.
@@ -586,7 +699,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 +710,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, tachyon, 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 +767,7 @@ private[spark] class BlockManager(
Utils.getUsedTimeMs(startTimeMs))
}
- size
+ updatedBlocks
}
/**
@@ -658,7 +775,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)
}
@@ -687,28 +805,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 +849,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
}
/**
@@ -742,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
}
@@ -760,13 +912,15 @@ 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) {
- reportBlockStatus(blockId, info)
+ val status = getCurrentBlockStatus(blockId, info)
+ reportBlockStatus(blockId, info, status)
}
} else {
// The block has already been removed; do nothing.
@@ -785,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
@@ -798,17 +952,21 @@ private[spark] class BlockManager(
if (level.useDisk) {
diskStore.remove(id)
}
+ if (level.useOffHeap) {
+ tachyonStore.remove(id)
+ }
iterator.remove()
logInfo("Dropped block " + id)
}
- reportBlockStatus(id, info)
+ val status = getCurrentBlockStatus(id, info)
+ reportBlockStatus(id, info, status)
}
}
}
def shouldCompress(blockId: BlockId): Boolean = blockId match {
case ShuffleBlockId(_, _, _) => compressShuffle
- case BroadcastBlockId(_) => compressBroadcast
+ case BroadcastBlockId(_, _) => compressBroadcast
case RDDBlockId(_, _) => compressRdds
case TempBlockId(_) => compressShuffleSpill
case _ => false
@@ -872,6 +1030,9 @@ private[spark] class BlockManager(
blockInfo.clear()
memoryStore.clear()
diskStore.clear()
+ if (tachyonInitialized) {
+ tachyonStore.clear()
+ }
metadataCleaner.cancel()
broadcastCleaner.cancel()
logInfo("BlockManager stopped")
@@ -911,9 +1072,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 +1092,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..7897fade2df2b 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")
@@ -65,9 +63,10 @@ class BlockManagerMaster(var driverActor : ActorRef, conf: SparkConf) extends Lo
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
}
@@ -82,6 +81,14 @@ class BlockManagerMaster(var driverActor : ActorRef, conf: SparkConf) extends Lo
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))
@@ -100,12 +107,10 @@ class BlockManagerMaster(var driverActor : ActorRef, conf: SparkConf) extends Lo
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) {
@@ -113,6 +118,31 @@ class BlockManagerMaster(var driverActor : ActorRef, conf: SparkConf) extends Lo
}
}
+ /** 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
@@ -127,6 +157,51 @@ class BlockManagerMaster(var driverActor : ActorRef, conf: SparkConf) extends Lo
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 a999d76a326a6..c57b6e8391b13 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]
@@ -72,10 +73,11 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act
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)
@@ -92,9 +94,21 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act
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
@@ -138,9 +152,41 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act
// 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) {
@@ -160,6 +206,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act
blockLocations.remove(locations)
}
}
+ listenerBus.post(SparkListenerBlockManagerRemoved(blockManagerId))
}
private def expireDeadHosts() {
@@ -217,11 +264,66 @@ 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
}
+ /**
+ * 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 {
@@ -233,9 +335,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(
@@ -243,7 +346,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act
blockId: BlockId,
storageLevel: StorageLevel,
memSize: Long,
- diskSize: Long) {
+ diskSize: Long,
+ tachyonSize: Long) {
if (!blockManagerInfo.contains(blockManagerId)) {
if (blockManagerId.executorId == "" && !isLocal) {
@@ -262,7 +366,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act
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)) {
@@ -306,98 +411,113 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act
}
}
+private[spark] case class BlockStatus(
+ storageLevel: StorageLevel,
+ memSize: Long,
+ diskSize: Long,
+ tachyonSize: Long)
-private[spark]
-object BlockManagerMasterActor {
-
- case class BlockStatus(storageLevel: StorageLevel, memSize: Long, diskSize: Long)
+private[spark] class BlockManagerInfo(
+ val blockManagerId: BlockManagerId,
+ timeMs: Long,
+ val maxMem: Long,
+ val slaveActor: ActorRef)
+ extends Logging {
- 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 getStatus(blockId: BlockId) = Option(_blocks.get(blockId))
- 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,
+ tachyonSize: 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, 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, 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, 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)
+ _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 (blockStatus.storageLevel.useOffHeap) {
+ logInfo("Removed %s on %s on tachyon (size: %s)".format(
+ blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.tachyonSize)))
}
}
+ }
- 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/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala
index bbb9529b5a0ca..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.
@@ -53,11 +60,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 +73,7 @@ private[storage] object BlockManagerMessages {
storageLevel.writeExternal(out)
out.writeLong(memSize)
out.writeLong(diskSize)
+ out.writeLong(tachyonSize)
}
override def readExternal(in: ObjectInput) {
@@ -73,21 +82,25 @@ private[storage] object BlockManagerMessages {
storageLevel = StorageLevel(in)
memSize = in.readLong()
diskSize = in.readLong()
+ tachyonSize = in.readLong()
}
}
object UpdateBlockInfo {
- def apply(blockManagerId: BlockManagerId,
+ def apply(
+ blockManagerId: BlockManagerId,
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))
}
}
@@ -103,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/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/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/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/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala
index 1b7934d59fa1d..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,28 +129,35 @@ 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) =
- 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) =
+ 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/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala
new file mode 100644
index 0000000000000..7a174959037be
--- /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, 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..07255aa366a6d 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,18 @@
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.annotation.DeveloperApi
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 +48,25 @@ case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long,
}
}
-case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel,
- numCachedPartitions: Int, numPartitions: Int, memSize: Long, diskSize: Long)
- extends Ordered[RDDInfo] {
+@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; " +
- "DiskSize: %s").format(name, id, storageLevel.toString, numCachedPartitions,
- numPartitions, bytesToString(memSize), bytesToString(diskSize))
+ ("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) = {
@@ -62,55 +78,79 @@ 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] = {
-
- // Group by rddId, ignore the partition name
- val groupedRddBlocks = infos.groupBy { case(k, v) => k.rddId }.mapValues(_.values.toArray)
-
- // For each RDD, generate an RDDInfo object
- val rddInfos = groupedRddBlocks.map { case (rddId, rddBlocks) =>
- // 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)
+ /** 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)
+
+ // Mapping from RDD ID -> the associated RDDInfo (with potentially outdated storage information)
+ val rddInfoMap = rddInfos.map { info => (info.id, info) }.toMap
+
+ val rddStorageInfos = blockStatusMap.flatMap { case (rddId, blocks) =>
+ // 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
}
- }.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/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/storage/ThreadingTest.scala b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala
index 36f2a0fd02724..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,9 +22,9 @@ import java.util.concurrent.ArrayBlockingQueue
import akka.actor._
import util.Random
-import org.apache.spark.SparkConf
+import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf}
+import org.apache.spark.scheduler.LiveListenerBus
import org.apache.spark.serializer.KryoSerializer
-import org.apache.spark.{SecurityManager, SparkConf}
/**
* This class tests the BlockManager and MemoryStore for thread safety and
@@ -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))
}
@@ -97,10 +97,11 @@ 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))
+ 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/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala
index 7c35cd165ad7c..e1a1f209c9282 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,30 @@
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.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.Server
+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.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, 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,55 +57,68 @@ 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 {
@@ -114,10 +127,12 @@ private[spark] object JettyUtils extends Logging {
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
}
+ /** 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 {
@@ -126,7 +141,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 {
@@ -144,18 +159,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) = {
@@ -163,7 +181,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()
@@ -178,6 +196,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/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 ca82c3da2fc24..bca4c3c42d27f 100644
--- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
@@ -17,74 +17,78 @@
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}
+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 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(securityManager, SparkUI.getUIPort(conf), conf, basePath)
+ with Logging {
-/** 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
+ 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)
- val handlers = Seq[ServletContextHandler] (
- createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static/*"),
- createRedirectHandler("/stages", "/")
- )
- val storage = new BlockManagerUI(sc)
- val jobs = new JobProgressUI(sc)
- val env = new EnvironmentUI(sc)
- val exec = new ExecutorsUI(sc)
+ // If SparkContext is not provided, assume the associated application is not live
+ val live = sc != null
- // Add MetricsServlet handlers by default
- val metricsServletHandlers = SparkEnv.get.metricsSystem.getServletHandlers
+ // Maintain executor storage status through Spark events
+ val storageStatusListener = new StorageStatusListener
- val allHandlers = storage.getHandlers ++ jobs.getHandlers ++ env.getHandlers ++
- exec.getHandlers ++ metricsServletHandlers ++ handlers
+ initialize()
- /** 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)
- } catch {
- case e: Exception =>
- logError("Failed to create Spark JettyUtils", e)
- System.exit(1)
+ /** Initialize all components of the server. */
+ def initialize() {
+ listenerBus.addListener(storageStatusListener)
+ attachTab(new JobProgressTab(this))
+ attachTab(new StorageTab(this))
+ attachTab(new EnvironmentTab(this))
+ attachTab(new ExecutorsTab(this))
+ attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static"))
+ attachHandler(createRedirectHandler("/", "/stages", basePath))
+ if (live) {
+ sc.env.metricsSystem.getServletHandlers.foreach(attachHandler)
}
}
- /** 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
- jobs.start()
- exec.start()
+ /** Set the app name for this UI. */
+ def setAppName(name: String) {
+ appName = name
}
- def stop() {
- server.foreach(_.stop())
+ /** Register the given listener with the listener bus. */
+ def registerListener(listener: SparkListener) {
+ listenerBus.addListener(listener)
}
- private[spark] def appUIAddress = host + ":" + boundPort.getOrElse("-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))
+ }
+ private[spark] def appUIAddress = "http://" + publicHostName + ":" + boundPort
}
private[spark] object SparkUI {
- val DEFAULT_PORT = "4040"
- val STATIC_RESOURCE_DIR = "org/apache/spark/ui"
+ 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 547a194d58a5c..ecdb9c6385657 100644
--- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
+++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
@@ -17,40 +17,132 @@
package org.apache.spark.ui
-import scala.xml.Node
+import java.text.SimpleDateFormat
+import java.util.{Locale, Date}
-import org.apache.spark.SparkContext
+import scala.xml.Node
+import org.apache.spark.Logging
/** Utility functions for generating XML pages with spark content. */
-private[spark] object UIUtils {
- import Page._
+private[spark] object UIUtils extends Logging {
- // 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("")
+ // 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 prependBaseUri(resource: String = "") = uiRoot + resource
+ def formatDate(date: Date): String = dateFormat.get.format(date)
- /** Returns a spark page with correctly formatted headers */
- def headerSparkPage(content: => Seq[Node], sc: SparkContext, title: String, page: Page.Value)
- : Seq[Node] = {
- val jobs = page match {
- case Stages =>
@@ -123,21 +220,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 =>
+
}
diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala
new file mode 100644
index 0000000000000..b08f308fda1dd
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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
+
+import javax.servlet.http.HttpServletRequest
+
+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()
+ }
+ }
+ }
+
+ /** 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(className))
+ serverInfo.get.server.stop()
+ }
+}
+
+
+/**
+ * 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] 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
+ }
+
+ /** Get a list of header tabs from the parent UI. */
+ def headerTabs: Seq[WebUITab] = parent.getTabs
+}
+
+
+/**
+ * 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/EnvironmentPage.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala
new file mode 100644
index 0000000000000..70578c3eb87c8
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.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.ui.env
+
+import javax.servlet.http.HttpServletRequest
+
+import scala.xml.Node
+
+import org.apache.spark.ui.{UIUtils, WebUIPage}
+
+private[ui] class IndexPage(parent: EnvironmentTab) extends WebUIPage("") {
+ private val appName = parent.appName
+ private val basePath = parent.basePath
+ private val listener = parent.listener
+
+ 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 =
+
+
+}
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..7797057fa1aa9
--- /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 IndexPage(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/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala
deleted file mode 100644
index 14333476c0e31..0000000000000
--- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.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.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.ui.JettyUtils._
-import org.apache.spark.ui.Page.Environment
-import org.apache.spark.ui.UIUtils
-
-private[spark] class EnvironmentUI(sc: SparkContext) {
-
- def getHandlers = Seq[ServletContextHandler](
- createServletHandler("/environment",
- createServlet((request: HttpServletRequest) => envDetails(request), sc.env.securityManager))
- )
-
- 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)
-
- val content =
-
-
Runtime Information
{jvmTable}
-
Spark Properties
- {sparkPropertyTable}
-
System Properties
- {otherPropertyTable}
-
Classpath Entries
- {classPathTable}
-
-
- UIUtils.headerSparkPage(content, sc, "Environment", Environment)
- }
-}
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
new file mode 100644
index 0000000000000..56c3887923758
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 javax.servlet.http.HttpServletRequest
+
+import scala.xml.Node
+
+import org.apache.spark.ui.{WebUIPage, UIUtils}
+import org.apache.spark.util.Utils
+
+private[ui] class IndexPage(parent: ExecutorsTab) extends WebUIPage("") {
+ private val appName = parent.appName
+ private val basePath = parent.basePath
+ private val listener = parent.listener
+
+ 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 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", ""))
+ val execTable = UIUtils.listingTable(execHeader, execRow, execInfoSorted)
+
+ val content =
+
+
+
+
Memory:
+ {Utils.bytesToString(memUsed)} Used
+ ({Utils.bytesToString(maxMem)} Total)
+ }
+
+ /** 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
+ 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
+ val totalDuration = listener.executorToDuration.getOrElse(execId, 0)
+ val totalShuffleRead = listener.executorToShuffleRead.getOrElse(execId, 0)
+ val totalShuffleWrite = listener.executorToShuffleWrite.getOrElse(execId, 0)
+
+ // Also include fields not in the header
+ val execFields = execHeader ++ Seq("Maximum Memory")
+
+ val execValues = Seq(
+ execId,
+ hostPort,
+ rddBlocks,
+ memUsed,
+ diskUsed,
+ activeTasks,
+ failedTasks,
+ completedTasks,
+ totalTasks,
+ totalDuration,
+ totalShuffleRead,
+ totalShuffleWrite,
+ maxMem
+ ).map(_.toString)
+
+ execFields.zip(execValues).toMap
+ }
+}
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..e9ec18a3e74af
--- /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 IndexPage(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/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala
deleted file mode 100644
index 4235cfeff9fa2..0000000000000
--- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala
+++ /dev/null
@@ -1,182 +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.exec
-
-import javax.servlet.http.HttpServletRequest
-
-import scala.collection.mutable.{HashMap, HashSet}
-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.ui.JettyUtils._
-import org.apache.spark.ui.Page.Executors
-import org.apache.spark.ui.UIUtils
-import org.apache.spark.util.Utils
-
-private[spark] class ExecutorsUI(val sc: SparkContext) {
-
- private var _listener: Option[ExecutorsListener] = None
- def listener = _listener.get
-
- def start() {
- _listener = Some(new ExecutorsListener)
- sc.addSparkListener(listener)
- }
-
- def getHandlers = Seq[ServletContextHandler](
- createServletHandler("/executors", createServlet((request: HttpServletRequest) => render
- (request), sc.env.securityManager))
- )
-
- def render(request: HttpServletRequest): Seq[Node] = {
- val storageStatusList = sc.getExecutorStorageStatus
-
- 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]) = {
-
}
}
- 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
deleted file mode 100644
index 81713edcf5db2..0000000000000
--- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala
+++ /dev/null
@@ -1,90 +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.jobs
-
-import javax.servlet.http.HttpServletRequest
-
-import scala.xml.{NodeSeq, Node}
-
-import org.apache.spark.scheduler.SchedulingMode
-import org.apache.spark.ui.Page._
-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
-
- def render(request: HttpServletRequest): Seq[Node] = {
- listener.synchronized {
- val activeStages = listener.activeStages.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 failedStagesTable = new StageTable(failedStages.sortBy(_.submissionTime).reverse, parent)
-
- val pools = listener.sc.getAllPools
- val poolTable = new PoolTable(pools, listener)
- val summary: NodeSeq =
-
-
-
- Total Duration:
- {parent.formatDuration(now - listener.sc.startTime)}
-
++
+ failedStagesTable.toNodeSeq
+
+ UIUtils.headerSparkPage(content, basePath, appName, "Spark Stages", parent.headerTabs, parent)
+ }
+ }
+}
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala
new file mode 100644
index 0000000000000..da9de035f89f1
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.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.ui.jobs
+
+import org.apache.spark.SparkConf
+import org.apache.spark.scheduler.SchedulingMode
+import org.apache.spark.ui.{SparkUI, WebUITab}
+
+/** Web UI showing progress status of all jobs in the given SparkContext. */
+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 conf = if (live) sc.conf else new SparkConf
+ val listener = new JobProgressListener(conf)
+
+ attachPage(new IndexPage(this))
+ attachPage(new StagePage(this))
+ attachPage(new PoolPage(this))
+ parent.registerListener(listener)
+
+ def isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR)
+}
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
deleted file mode 100644
index 2d95d47e154cd..0000000000000
--- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala
+++ /dev/null
@@ -1,60 +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.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.ui.JettyUtils._
-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
- val dateFmt = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss")
-
- private val indexPage = new IndexPage(this)
- private val stagePage = new StagePage(this)
- private val poolPage = new PoolPage(this)
-
- def start() {
- _listener = Some(new JobProgressListener(sc))
- sc.addSparkListener(listener)
- }
-
- def formatDuration(ms: Long) = Utils.msDurationToString(ms)
-
- 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/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala
index eb7518a020840..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
@@ -21,27 +21,37 @@ 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.scheduler.{Schedulable, StageInfo}
+import org.apache.spark.ui.{WebUIPage, UIUtils}
/** Page showing specific pool details */
-private[spark] class PoolPage(parent: JobProgressUI) {
- def listener = parent.listener
+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 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 =
+
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..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,54 +22,52 @@ 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.{WebUIPage, 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: JobProgressTab) extends WebUIPage("stage") {
+ private val appName = parent.appName
+ private val basePath = parent.basePath
+ private 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), parent.headerTabs, parent)
}
- 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 =
Total task time across all tasks:
- {parent.formatDuration(listener.stageIdToTime.getOrElse(stageId, 0L) + activeTime)}
+ {UIUtils.formatDuration(listener.stageIdToTime.getOrElse(stageId, 0L) + activeTime)}
{if (hasShuffleRead)
@@ -104,42 +102,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))
+ get.getQuantiles().map(ms => UIUtils.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 => UIUtils.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 =>
+ 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,
// 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 +150,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 =>
+ UIUtils.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 +193,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),
+ parent.headerTabs, parent)
}
}
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("")
-
-
+ }
}
}
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..5cc1fcd10a08d 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,26 @@ 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: JobProgressTab) {
+ private val basePath = parent.basePath
+ private 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] = {
@@ -64,7 +63,7 @@ private[spark] class StageTable(val stages: Seq[StageInfo], val parent: JobProgr
val startWidth = "width: %s%%".format((started.toDouble/total)*100)
-
+
{completed}/{total} {failed}
@@ -72,53 +71,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 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 => UIUtils.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 => finishTime - t)
-
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
deleted file mode 100644
index cb2083eb019bf..0000000000000
--- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala
+++ /dev/null
@@ -1,40 +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.storage
-
-import javax.servlet.http.HttpServletRequest
-
-import org.eclipse.jetty.servlet.ServletContextHandler
-
-import org.apache.spark.{Logging, SparkContext}
-import org.apache.spark.ui.JettyUtils._
-
-/** 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)
-
- 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/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala
deleted file mode 100644
index 6a3c41fb1155d..0000000000000
--- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala
+++ /dev/null
@@ -1,65 +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.storage
-
-import javax.servlet.http.HttpServletRequest
-
-import scala.xml.Node
-
-import org.apache.spark.storage.{RDDInfo, StorageUtils}
-import org.apache.spark.ui.Page._
-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
-
- 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)
- }
-
- def rddRow(rdd: RDDInfo): Seq[Node] = {
-
- }
}
diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala
new file mode 100644
index 0000000000000..c5cfee777aab5
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.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.ui.storage
+
+import javax.servlet.http.HttpServletRequest
+
+import scala.xml.Node
+
+import org.apache.spark.storage.RDDInfo
+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: StorageTab) extends WebUIPage("") {
+ private val appName = parent.appName
+ private val basePath = parent.basePath
+ 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 ", parent.headerTabs, parent)
+ }
+
+ /** Header fields for the RDD table */
+ private def rddHeader = Seq(
+ "RDD Name",
+ "Storage Level",
+ "Cached Partitions",
+ "Fraction Cached",
+ "Size in Memory",
+ "Size in Tachyon",
+ "Size on Disk")
+
+ /** Render an HTML row representing an RDD */
+ private def rddRow(rdd: RDDInfo): Seq[Node] = {
+
diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb
index 44d64057f4fb3..bbd56d2fd13bb 100644
--- a/docs/_plugins/copy_api_dirs.rb
+++ b/docs/_plugins/copy_api_dirs.rb
@@ -22,8 +22,11 @@
# 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 }
+ 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
@@ -41,12 +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)
+
+ # 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()) }
+
+ # 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/building-with-maven.md b/docs/building-with-maven.md
index d3bc34e68b240..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 ##
@@ -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/docs/cluster-overview.md b/docs/cluster-overview.md
index e16703292cc22..7f75ea44e4cea 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.
@@ -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/configuration.md b/docs/configuration.md
index a006224d5080c..9c602402f0635 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.
@@ -333,6 +348,32 @@ Apart from these, the following properties are also available, and may be useful
receives no heartbeats.
+
+
spark.worker.cleanup.enabled
+
true
+
+ Enable periodic cleanup of worker / application directories. Note that this only affects standalone
+ mode, as YARN works differently.
+
+
+
+
spark.worker.cleanup.interval
+
1800 (30 minutes)
+
+ Controls the interval, in seconds, at which the worker cleans up old application work dirs
+ on the local machine.
+
+
+
+
spark.worker.cleanup.appDataTtl
+
7 * 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
@@ -375,7 +416,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 +471,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.
@@ -490,7 +531,30 @@ Apart from these, the following properties are also available, and may be useful
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.enabled
+
false
+
+ Whether to log spark events, useful for reconstructing the Web UI after the application has finished.
+
+
+
+
spark.eventLog.compress
+
false
+
+ Whether to compress logged events, if spark.eventLog.enabled is true.
+
+
+
+
spark.eventLog.dir
+
file:///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.
@@ -532,7 +596,16 @@ 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
@@ -540,7 +613,7 @@ Apart from these, the following properties are also available, and may be useful
running on Yarn.
-
+
spark.authenticate.secret
None
@@ -548,12 +621,19 @@ 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.
+
+
+
+
spark.task.cpus
+
1
+
+ Number of cores to allocate for each task.
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/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/index.md b/docs/index.md
index c4f4d79edbc6c..7a13fa9a9a2b6 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
@@ -76,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/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/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/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/mllib-guide.md b/docs/mllib-guide.md
index 76308ec9c0821..a5e0cc50809cf 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
@@ -37,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/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/docs/monitoring.md b/docs/monitoring.md
index e9b1d2b2f4ffb..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 Variable
Meaning
+
+
SPARK_DAEMON_MEMORY
+
Memory to allocate to the history server. (default: 512m).
+
+
+
SPARK_DAEMON_JAVA_OPTS
+
JVM options for the history server (default: none).
+
+
+
+Further, the history server can be configured as follows:
+
+
+
Property Name
Default
Meaning
+
+
spark.history.updateInterval
+
10
+
+ 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.retainedApplications
+
250
+
+ The number of application UIs to retain. If this cap is exceeded, then the oldest
+ applications will be removed.
+
+
+
+
spark.history.ui.port
+
18080
+
+ 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
@@ -48,11 +108,22 @@ Each instance can report to zero or more _sinks_. Sinks are contained in the
* `ConsoleSink`: Logs metrics information to the console.
* `CSVSink`: Exports metrics data to CSV files at regular intervals.
-* `GangliaSink`: Sends metrics to a Ganglia node or multicast group.
* `JmxSink`: Registers metrics for viewing in a JXM console.
* `MetricsServlet`: Adds a servlet within the existing Spark UI to serve metrics data as JSON data.
* `GraphiteSink`: Sends metrics to a Graphite node.
+Spark also supports a Ganglia sink which is not included in the default build due to
+licensing restrictions:
+
+* `GangliaSink`: Sends metrics to a Ganglia node or multicast group.
+
+To install the `GangliaSink` you'll need to perform a custom build of Spark. _**Note that
+by embedding this library you will include [LGPL](http://www.gnu.org/copyleft/lesser.html)-licensed
+code in your Spark package**_. For sbt users, set the
+`SPARK_GANGLIA_LGPL` environment variable before building. For Maven users, enable
+the `-Pspark-ganglia-lgpl` profile. In addition to modifying the cluster's Spark build
+user applications will need to link to the `spark-ganglia-lgpl` artifact.
+
The syntax of the metrics configuration file is defined in an example configuration file,
`$SPARK_HOME/conf/metrics.properties.template`.
diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md
index 57ed54c9cf4c0..888631e7025b0 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.
@@ -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
@@ -152,7 +153,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/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:
diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md
index b17929542c531..982514391ac00 100644
--- a/docs/running-on-yarn.md
+++ b/docs/running-on-yarn.md
@@ -41,17 +41,19 @@ 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
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:
@@ -59,19 +61,18 @@ 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 \
- --master-class
- --master-memory \
- --worker-memory \
- --worker-cores \
+ --arg \
+ --num-executors \
+ --driver-memory \
+ --executor-memory \
+ --executor-cores \
--name \
--queue \
--addJars \
--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
@@ -84,11 +85,12 @@ 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 \
- --num-workers 3 \
- --master-memory 4g \
- --worker-memory 2g \
- --worker-cores 1
+ --arg yarn-cluster \
+ --arg 5 \
+ --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 +102,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.
@@ -122,7 +124,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/scala-programming-guide.md b/docs/scala-programming-guide.md
index 99412733d4268..a07cd2e0a32a2 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:
@@ -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
@@ -73,18 +73,19 @@ The master URL passed to Spark can be in one of the following formats:
Master URL
Meaning
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).
+
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.
+
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.
-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.
@@ -265,11 +266,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:
Storage Level
Meaning
@@ -292,8 +307,16 @@ In addition, each RDD can be stored using a different *storage level*, allowing
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
@@ -307,30 +330,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 +392,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/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/docs/sql-programming-guide.md b/docs/sql-programming-guide.md
new file mode 100644
index 0000000000000..a59393e1424de
--- /dev/null
+++ b/docs/sql-programming-guide.md
@@ -0,0 +1,321 @@
+---
+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.**
+
+
+
+
+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 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.
+
+{% 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 %}
+
+
+
+
+
+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 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
+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 %}
+
+
+
+
+
+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`.
+
+## 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 JavaSchemaRDD.
+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 %}
+
+
+
+
+
+{% 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.
+
+// 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 '`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.
+
+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._
+
+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
+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/pom.xml b/examples/pom.xml
index 9f0e2d0b875b8..0b6212b5d1549 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
@@ -85,6 +70,12 @@
${project.version}provided
+
+ org.apache.spark
+ spark-hive_${scala.binary.version}
+ ${project.version}
+ provided
+ org.apache.sparkspark-graphx_${scala.binary.version}
@@ -119,7 +110,7 @@
org.apache.hbasehbase
- 0.94.6
+ ${hbase.version}asm
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/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/examples/JavaLogQuery.java b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java
index a518fe2f27eb0..2a4278d3c30e5 100644
--- a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java
+++ b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java
@@ -124,8 +124,8 @@ 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);
+ 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 e53925b50c2ce..e31f676f5fd4c 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()));
+ results.add(new Tuple2(n, s._2() / urlCount));
}
return results;
}
@@ -109,9 +114,9 @@ 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);
+ 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 fa1b977ab19f1..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) {
@@ -65,8 +65,8 @@ 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);
+ ctx.stop();
}
}
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/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 76ebdccfd6b67..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
@@ -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,19 +66,19 @@ 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);
- 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 667c72f379e71..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
@@ -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,9 +75,8 @@ 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);
+ sc.stop();
}
}
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/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 c8ecbb8e41a86..37ad4bd0999bd 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))
@@ -121,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 73b0e216cac98..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)
+ // 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 17bafc2218a31..68f151a2c47fe 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))
@@ -113,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
@@ -128,16 +127,16 @@ 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()
}
- 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 cf1fc3e808c76..d8de8745c15d9 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)
@@ -54,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))
))
@@ -75,6 +73,6 @@ object SparkHdfsLR {
}
println("Final w: " + w)
- System.exit(0)
+ sc.stop()
}
}
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/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/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
new file mode 100644
index 0000000000000..5698d4746495d
--- /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)
+ 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
new file mode 100644
index 0000000000000..2b207fd8d3e16
--- /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/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..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]) = {
@@ -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/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 19676fcc1a2b0..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"), Seq(System.getenv("SPARK_EXAMPLES_JAR")))
-
- // 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 = SVD.sparseSVD(SparseMatrix(data, m, n), 1)
- val u = decomposed.U.data
- val s = decomposed.S.data
- val v = decomposed.V.data
-
- println("singular values = " + s.toArray.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..61b9655cd3759
--- /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..9aeebf58eabfb
--- /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/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..62329bde84481
--- /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._
+
+ 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 *': ")
+ hql("SELECT * FROM src").collect.foreach(println)
+
+ // Aggregation queries are also supported.
+ 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 = hql("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 *:")
+ hql("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/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/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 35be7ffa1e872..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.
@@ -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/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/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/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/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/extras/spark-ganglia-lgpl/pom.xml b/extras/spark-ganglia-lgpl/pom.xml
new file mode 100644
index 0000000000000..11ac827ed54a0
--- /dev/null
+++ b/extras/spark-ganglia-lgpl/pom.xml
@@ -0,0 +1,45 @@
+
+
+
+ 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 85%
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
index cd37317da77de..d03d7774e8c80 100644
--- 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
@@ -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/pom.xml b/graphx/pom.xml
index 894a7c2641e39..b4c67ddcd8ca9 100644
--- a/graphx/pom.xml
+++ b/graphx/pom.xml
@@ -29,7 +29,7 @@
spark-graphx_2.10jarSpark Project GraphX
- http://spark-project.org/
+ http://spark.apache.org/
@@ -54,7 +54,7 @@
org.jblasjblas
- 1.2.3
+ ${jblas.version}org.eclipse.jetty
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/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/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/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/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
index 1d029bf009e8c..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,14 +190,14 @@ 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)],
reduceFunc: (A, A) => A,
- activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None) = {
+ activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None): VertexRDD[A] = {
ClosureCleaner.clean(mapFunc)
ClosureCleaner.clean(reduceFunc)
@@ -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..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 {
@@ -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..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
@@ -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) {
@@ -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/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/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/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/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)
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)))
+ }
+}
diff --git a/make-distribution.sh b/make-distribution.sh
index e6b5956d1e7e2..5c780fcbda863 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,34 @@ 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 2>/dev/null || mktemp -d -t 'disttmp'`
+
+ 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"
+
+ 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"
+ 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"
+ fi
+
+ popd > /dev/null
+ rm -rf $TMPD
+fi
+
if [ "$MAKE_TGZ" == "true" ]; then
TARDIR="$FWDIR/spark-$VERSION"
cp -r "$DISTDIR" "$TARDIR"
diff --git a/mllib/pom.xml b/mllib/pom.xml
index 9b65cb4b4ce3f..e7ce00efc4af6 100644
--- a/mllib/pom.xml
+++ b/mllib/pom.xml
@@ -58,7 +58,12 @@
org.jblasjblas
- 1.2.3
+ ${jblas.version}
+
+
+ org.scalanlp
+ breeze_${scala.binary.version}
+ 0.7org.scalatest
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..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
@@ -16,18 +16,23 @@
*/
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.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
/**
+ * :: 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
@@ -108,16 +113,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
}
@@ -125,75 +130,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
@@ -202,13 +259,16 @@ 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] = {
- val data = dataBytesJRDD.rdd.map(xBytes => deserializeDoubleVector(xBytes))
+ 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]()
- ret.add(serializeDoubleMatrix(model.clusterCenters))
+ ret.add(serializeDoubleMatrix(model.clusterCenters.map(_.toArray)))
ret
}
@@ -257,8 +317,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)
}
@@ -269,8 +333,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..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
@@ -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 protected 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
+ }
}
}
@@ -52,28 +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)
- extends GeneralizedLinearAlgorithm[LogisticRegressionModel]
- with Serializable {
-
- val gradient = new LogisticGradient()
- val updater = new SimpleUpdater()
+ private var stepSize: Double,
+ private var numIterations: Int,
+ private var regParam: Double,
+ private var miniBatchFraction: Double)
+ extends GeneralizedLinearAlgorithm[LogisticRegressionModel] with Serializable {
+
+ 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)
+ .setStepSize(stepSize)
+ .setNumIterations(numIterations)
+ .setRegParam(regParam)
+ .setMiniBatchFraction(miniBatchFraction)
+ 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: Array[Double], intercept: Double) = {
+ override protected 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..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
@@ -17,34 +17,51 @@
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.annotation.Experimental
+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
/**
+ * :: 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
*/
-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)
+@Experimental
+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)
+
+ {
+ // 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
+ }
+ }
- 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 +73,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 (private var lambda: Double) extends Serializable with Logging {
+
def this() = this(1.0)
/** Set the smoothing parameter. Default: 1.0. */
@@ -70,45 +86,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 +171,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..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
@@ -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,39 @@ 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 {
- override def predictPoint(dataMatrix: DoubleMatrix, weightMatrix: DoubleMatrix,
+ 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
+ }
+
+ /**
+ * Clears the threshold so that `predict` will output raw prediction scores.
+ */
+ def clearThreshold(): this.type = {
+ threshold = None
+ this
+ }
+
+ override protected 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
+ }
}
}
@@ -50,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: Array[Double], intercept: Double) = {
+ override protected def createModel(weights: Vector, intercept: Double) = {
new SVMModel(weights, intercept)
}
}
@@ -103,11 +124,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 +146,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 +166,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 +180,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 +192,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 e508b76c3f8c5..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
@@ -19,16 +19,16 @@ 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.annotation.Experimental
+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,
@@ -38,16 +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 type ClusterCenters = Array[Array[Double]]
+ 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. */
@@ -76,6 +77,7 @@ 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.
@@ -113,28 +115,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 +168,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 +191,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 +215,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 +252,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,65 +305,99 @@ 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 =
- {
+ initializationMode: String): KMeansModel = {
new KMeans().setK(k)
- .setMaxIterations(maxIterations)
- .setRuns(runs)
- .setInitializationMode(initializationMode)
- .run(data)
+ .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)
+ /**
+ * 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)
}
- def train(data: RDD[Array[Double]], 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)
}
/**
- * 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)
}
+ @Experimental
def main(args: Array[String]) {
if (args.length < 4) {
println("Usage: KMeans []")
@@ -323,14 +406,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/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/MatrixEntry.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixEntry.scala
deleted file mode 100644
index 416996fcbe760..0000000000000
--- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixEntry.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.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)
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/SVD.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala
deleted file mode 100644
index 8803c4c1a07be..0000000000000
--- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala
+++ /dev/null
@@ -1,189 +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: Int = 1
-
- /**
- * Set the number of top-k singular vectors to return
- */
- def setK(k: Int): SVD = {
- this.k = k
- this
- }
-
- /**
- * Compute SVD using the current set parameters
- */
- def compute(matrix: SparseMatrix) : MatrixSVD = {
- SVD.sparseSVD(matrix, k)
- }
-}
-
-
-/**
- * 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 =
- {
- 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.toArray) {
- 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)
- 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")
- }
-
- 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)
-
- // 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)
- }
-
-
- 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 = SVD.sparseSVD(SparseMatrix(data, m, n), k)
- val u = decomposed.U.data
- val s = decomposed.S.data
- val v = decomposed.V.data
-
- println("Computed " + s.toArray.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/SingularValueDecomposition.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala
new file mode 100644
index 0000000000000..46b105457430c
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.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.mllib.linalg
+
+/** 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/SparseMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SparseMatrix.scala
deleted file mode 100644
index cbd1a2a5a4bd8..0000000000000
--- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SparseMatrix.scala
+++ /dev/null
@@ -1,30 +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
-
-
-/**
- * Class that represents a sparse matrix
- *
- * @param data RDD of nonzero entries
- * @param m number of rows
- * @param n numner of columns
- */
-case class SparseMatrix(val data: RDD[MatrixEntry], val m: Int, val n: Int)
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..99a849f1c66b1
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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]
+
+ /**
+ * Gets the value of the ith element.
+ * @param i index
+ */
+ private[mllib] def apply(i: Int): Double = toBreeze(i)
+}
+
+/**
+ * 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 from its values.
+ */
+ @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)
+
+ override def apply(i: Int) = values(i)
+}
+
+/**
+ * A sparse vector represented by an index array and an value array.
+ *
+ * @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(
+ override val size: Int,
+ val indices: Array[Int],
+ val values: Array[Double]) extends Vector {
+
+ override def toString: String = {
+ "(" + size + "," + indices.zip(values).mkString("[", "," ,"]") + ")"
+ }
+
+ override def toArray: Array[Double] = {
+ val data = new Array[Double](size)
+ 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, 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
new file mode 100644
index 0000000000000..56b8fdcda66eb
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.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.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
+
+/**
+ * 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)
+
+/**
+ * :: Experimental ::
+ * 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.
+ */
+@Experimental
+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/distributed/DistributedMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala
new file mode 100644
index 0000000000000..a0e26ce3bc465
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.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.mllib.linalg.distributed
+
+import breeze.linalg.{DenseMatrix => BDM}
+
+/**
+ * Represents a distributively stored matrix backed by one or more RDDs.
+ */
+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..132b3af72d9ce
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.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.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
+
+/**
+ * :: 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.
+ *
+ * @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.
+ */
+@Experimental
+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..f65f43dd3007b
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala
@@ -0,0 +1,347 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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]
+ * @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.
+ */
+@Experimental
+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/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala
index 82124703da6cd..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
@@ -17,39 +17,55 @@
package org.apache.spark.mllib.optimization
-import org.jblas.DoubleMatrix
+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.
*
- * @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
}
/**
+ * :: 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: 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,47 +73,105 @@ 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
+ }
}
}
/**
+ * :: 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: 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
}
}
/**
+ * :: 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: 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..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
@@ -17,19 +17,23 @@
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.{DenseVector => BDV}
-import scala.collection.mutable.ArrayBuffer
+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
@@ -91,24 +95,26 @@ 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
}
}
-// 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.
@@ -133,14 +139,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 +154,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 +182,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 +190,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..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
@@ -19,11 +19,18 @@ package org.apache.spark.mllib.optimization
import org.apache.spark.rdd.RDD
-trait Optimizer {
+import org.apache.spark.annotation.DeveloperApi
+import org.apache.spark.mllib.linalg.Vector
+
+/**
+ * :: DeveloperApi ::
+ * Trait for optimization problem solvers.
+ */
+@DeveloperApi
+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..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
@@ -18,9 +18,14 @@
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.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
@@ -32,6 +37,7 @@ import org.jblas.DoubleMatrix
* 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
@@ -47,24 +53,37 @@ 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)
}
/**
+ * :: 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: 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)
}
}
/**
+ * :: DeveloperApi ::
* Updater for L1 regularized problems.
* R(w) = ||w||_1
* Uses a step-size decreasing with the square root of the number of iterations.
@@ -82,39 +101,56 @@ 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: 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)
}
}
/**
+ * :: 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: 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/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/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala
index 8958040e36640..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
@@ -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
@@ -89,10 +89,20 @@ 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 (
+ 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)
/**
@@ -122,23 +132,37 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l
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
}
+ /** 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.
*/
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
}
@@ -155,7 +179,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
@@ -176,21 +200,41 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l
}
}
- 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)
@@ -198,37 +242,70 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l
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) {
- 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))
- )
- } 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
+ }
+
+ /**
+ * 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
*/
- 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))
@@ -296,8 +373,11 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l
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)
}
/**
@@ -329,7 +409,7 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l
rank: Int,
lambda: Double,
alpha: Double,
- YtY: Broadcast[Option[DoubleMatrix]])
+ YtY: Option[Broadcast[DoubleMatrix]])
: RDD[(Int, Array[Array[Double]])] =
{
val numBlocks = products.partitions.size
@@ -352,12 +432,12 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l
* 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: 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
@@ -376,62 +456,44 @@ 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 {
- 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))
- userXtX(us(i)).addi(tempXtX.mul(confidence - 1))
- // 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)))
}
}
}
}
// 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
+ if (implicitPrefs) {
+ Solve.solvePositive(fullXtX.addi(YtY.get.value), userXy(index)).data
+ } else {
+ Solve.solvePositive(fullXtX, 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 +517,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 +533,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 +582,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 +598,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 +615,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 +623,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 +666,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 +684,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/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala
index 443fc5de5bf04..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
@@ -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,7 @@ 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 f98b0b536deaa..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
@@ -17,11 +17,13 @@
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._
-
-import org.jblas.DoubleMatrix
+import org.apache.spark.mllib.linalg.{Vectors, Vector}
/**
* GeneralizedLinearModel (GLM) represents a model trained using
@@ -31,12 +33,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 +43,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 +51,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 +66,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]
@@ -86,8 +80,10 @@ 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
protected var validateData: Boolean = true
@@ -95,7 +91,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.
@@ -106,8 +102,10 @@ 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
@@ -117,44 +115,56 @@ 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 = Array.fill(nfeatures)(1.0)
+ 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))) {
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, prependOne(labeledPoint.features)))
} else {
input.map(labeledPoint => (labeledPoint.label, labeledPoint.features))
}
val initialWeightsWithIntercept = if (addIntercept) {
- Array(1.0, initialWeights:_*)
+ prependOne(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 = if (addIntercept) weightsWithIntercept(0) else 0.0
+ val weights =
+ if (addIntercept) {
+ Vectors.dense(weightsWithIntercept.toArray.slice(1, weightsWithIntercept.size))
+ } else {
+ weightsWithIntercept
+ }
- 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/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 fb2bc9b92a51c..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
@@ -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,14 +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,
- intercept: Double) = {
- dataMatrix.dot(weightMatrix) + intercept
+ override protected def predictPoint(
+ dataMatrix: Vector,
+ weightMatrix: Vector,
+ intercept: Double): Double = {
+ weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept
}
}
@@ -51,62 +52,37 @@ 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)
- extends GeneralizedLinearAlgorithm[LassoModel]
- with Serializable {
-
- val gradient = new LeastSquaresGradient()
- val updater = new L1Updater()
- @transient val optimizer = new GradientDescent(gradient, updater).setStepSize(stepSize)
+ private var stepSize: Double,
+ private var numIterations: Int,
+ private var regParam: Double,
+ private var miniBatchFraction: Double)
+ extends GeneralizedLinearAlgorithm[LassoModel] with Serializable {
+
+ 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)
// We don't want to penalize the intercept, so set this to false.
- setIntercept(false)
-
- var yMean = 0.0
- var xColMean: DoubleMatrix = _
- var xColSd: DoubleMatrix = _
+ 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)
- def createModel(weights: Array[Double], intercept: Double) = {
- val weightsMat = new DoubleMatrix(weights.length + 1, 1, (Array(intercept) ++ weights):_*)
- val weightsScaled = weightsMat.div(xColSd)
- val interceptScaled = yMean - (weightsMat.transpose().mmul(xColMean.div(xColSd)).get(0))
-
- new LassoModel(weightsScaled.data, interceptScaled)
+ override def setIntercept(addIntercept: Boolean): this.type = {
+ // TODO: Support adding intercept.
+ if (addIntercept) throw new UnsupportedOperationException("Adding intercept is not supported.")
+ this
}
- 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)
}
}
@@ -136,11 +112,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)
}
/**
@@ -160,9 +134,7 @@ object LassoWithSGD {
numIterations: Int,
stepSize: Double,
regParam: Double,
- miniBatchFraction: Double)
- : LassoModel =
- {
+ miniBatchFraction: Double): LassoModel = {
new LassoWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input)
}
@@ -182,9 +154,7 @@ object LassoWithSGD {
input: RDD[LabeledPoint],
numIterations: Int,
stepSize: Double,
- regParam: Double)
- : LassoModel =
- {
+ regParam: Double): LassoModel = {
train(input, numIterations, stepSize, regParam, 1.0)
}
@@ -200,9 +170,7 @@ object LassoWithSGD {
*/
def train(
input: RDD[LabeledPoint],
- numIterations: Int)
- : LassoModel =
- {
+ numIterations: Int): LassoModel = {
train(input, numIterations, 1.0, 1.0, 1.0)
}
@@ -214,7 +182,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 8ee40addb25d9..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
@@ -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,14 +30,15 @@ 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) = {
- dataMatrix.dot(weightMatrix) + intercept
+ override val weights: Vector,
+ override val intercept: Double)
+ extends GeneralizedLinearModel(weights, intercept) with RegressionModel with Serializable {
+
+ override protected def predictPoint(
+ dataMatrix: Vector,
+ weightMatrix: Vector,
+ intercept: Double): Double = {
+ weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept
}
}
@@ -52,24 +52,25 @@ class LinearRegressionModel(
* See also the documentation for the precise formulation.
*/
class LinearRegressionWithSGD private (
- var stepSize: Double,
- var numIterations: Int,
- 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 var stepSize: Double,
+ private var numIterations: Int,
+ private var miniBatchFraction: Double)
+ extends GeneralizedLinearAlgorithm[LinearRegressionModel] with Serializable {
+
+ 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)
- def createModel(weights: Array[Double], intercept: Double) = {
+ override protected def createModel(weights: Vector, intercept: Double) = {
new LinearRegressionModel(weights, intercept)
}
}
@@ -98,11 +99,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 +119,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 +137,7 @@ object LinearRegressionWithSGD {
def train(
input: RDD[LabeledPoint],
numIterations: Int,
- stepSize: Double)
- : LinearRegressionModel =
- {
+ stepSize: Double): LinearRegressionModel = {
train(input, numIterations, stepSize, 1.0)
}
@@ -158,9 +153,7 @@ object LinearRegressionWithSGD {
*/
def train(
input: RDD[LabeledPoint],
- numIterations: Int)
- : LinearRegressionModel =
- {
+ numIterations: Int): LinearRegressionModel = {
train(input, numIterations, 1.0, 1.0)
}
@@ -172,7 +165,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 c504d3d40c773..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
@@ -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,14 +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,
- intercept: Double) = {
- dataMatrix.dot(weightMatrix) + intercept
+ override protected def predictPoint(
+ dataMatrix: Vector,
+ weightMatrix: Vector,
+ intercept: Double): Double = {
+ weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept
}
}
@@ -51,63 +52,38 @@ 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)
// We don't want to penalize the intercept in RidgeRegression, so set this to false.
- setIntercept(false)
-
- var yMean = 0.0
- var xColMean: DoubleMatrix = _
- var xColSd: DoubleMatrix = _
+ 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)
- def createModel(weights: Array[Double], intercept: Double) = {
- val weightsMat = new DoubleMatrix(weights.length + 1, 1, (Array(intercept) ++ weights):_*)
- val weightsScaled = weightsMat.div(xColSd)
- val interceptScaled = yMean - weightsMat.transpose().mmul(xColMean.div(xColSd)).get(0)
-
- new RidgeRegressionModel(weightsScaled.data, interceptScaled)
+ override def setIntercept(addIntercept: Boolean): this.type = {
+ // TODO: Support adding intercept.
+ if (addIntercept) throw new UnsupportedOperationException("Adding intercept is not supported.")
+ this
}
- 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)
}
}
@@ -136,9 +112,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)
}
@@ -159,9 +133,7 @@ object RidgeRegressionWithSGD {
numIterations: Int,
stepSize: Double,
regParam: Double,
- miniBatchFraction: Double)
- : RidgeRegressionModel =
- {
+ miniBatchFraction: Double): RidgeRegressionModel = {
new RidgeRegressionWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input)
}
@@ -180,9 +152,7 @@ object RidgeRegressionWithSGD {
input: RDD[LabeledPoint],
numIterations: Int,
stepSize: Double,
- regParam: Double)
- : RidgeRegressionModel =
- {
+ regParam: Double): RidgeRegressionModel = {
train(input, numIterations, stepSize, regParam, 1.0)
}
@@ -197,23 +167,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
new file mode 100644
index 0000000000000..3019447ce4cd9
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala
@@ -0,0 +1,1154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.annotation.Experimental
+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
+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.
+ */
+@Experimental
+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.size
+ 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.size
+
+ 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.")
+ }
+ }
+
+ private 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).
+ */
+ private 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 = Vectors.dense(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: Vector) = {
+ 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..79a01f58319e8
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.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.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
new file mode 100644
index 0000000000000..f4c877232750f
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.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.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
new file mode 100644
index 0000000000000..7da976e55a722
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.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.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
new file mode 100644
index 0000000000000..8767aca47cd5a
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.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.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
+ * @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.
+ */
+@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
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..60f43e9278d2a
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.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.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 {
+
+ 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
+ */
+ @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))
+ }
+ }
+
+ 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
new file mode 100644
index 0000000000000..c51d76d9b4c5b
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.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.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
+ } else {
+ val total = c0 + c1
+ val f0 = c0 / total
+ val f1 = c1 / total
+ 1 - f0 * f0 - f1 * f1
+ }
+ }
+
+ 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
new file mode 100644
index 0000000000000..8eab247cf0932
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.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.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
new file mode 100644
index 0000000000000..47d07122af30f
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.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.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
new file mode 100644
index 0000000000000..2d71e1e366069
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.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.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
+ */
+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
new file mode 100644
index 0000000000000..bf692ca8c4bd7
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.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.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 {
+
+ /**
+ * 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: Vector): 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[Vector]): 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..2deaf4ae8dcab
--- /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 <,=,>
+ */
+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
new file mode 100644
index 0000000000000..cc8a24cce9614
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.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.model
+
+import org.apache.spark.annotation.DeveloperApi
+
+/**
+ * :: DeveloperApi ::
+ * 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
+ */
+@DeveloperApi
+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..682f213f411a7
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.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.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
+ * @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
+ */
+@DeveloperApi
+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: Vector) : 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..d7ffd386c05ee
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.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.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]) {
+
+ 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
+ */
+private[tree] 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
+ */
+private[tree] 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
+ */
+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..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
@@ -17,23 +17,24 @@
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..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
@@ -19,15 +19,17 @@ 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/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/LinearDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala
index 2e03684e62861..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
@@ -22,15 +22,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 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 {
/**
@@ -74,7 +78,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..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
@@ -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
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 {
/**
@@ -54,7 +57,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/MFDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala
index 348aba1dea5b6..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
@@ -21,35 +21,36 @@ 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
/**
-* 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.
-*/
-
-object MFDataGenerator{
-
+ * :: DeveloperApi ::
+ * 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.
+ */
+@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 64c6136a8b89d..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
@@ -17,19 +17,113 @@
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.SparkContext._
-
-import org.jblas.DoubleMatrix
import org.apache.spark.mllib.regression.LabeledPoint
+import org.apache.spark.mllib.linalg.{Vector, Vectors}
/**
* 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
+ }
+
/**
+ * 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: LabelParser,
+ 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.parse(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 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: LabelParser): 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: LabelParser,
+ numFeatures: Int): RDD[LabeledPoint] =
+ 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.
@@ -39,16 +133,18 @@ 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(',')
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)
}
}
/**
+ * :: 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.
@@ -56,8 +152,9 @@ 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.mkString(" "))
+ val dataStr = data.map(x => x.label + "," + x.features.toArray.mkString(" "))
dataStr.saveAsTextFile(dir)
}
@@ -65,59 +162,107 @@ 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)
+ 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 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))
}
/**
- * 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:
+ *
+ * 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
+ /*
+ * The relative error is
+ *
+ * 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)
+ } 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/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala
index c96c94f70eef7..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
@@ -21,14 +21,18 @@ 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]) {
@@ -58,7 +62,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/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..c6d8425ffc38d
--- /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 scala.Tuple2;
+
+import com.google.common.collect.Lists;
+
+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/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/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/BreezeMatrixConversionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala
new file mode 100644
index 0000000000000..82d49c76ed02b
--- /dev/null
+++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.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.mllib.linalg
+
+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/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/MatricesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala
new file mode 100644
index 0000000000000..9c66b4db9f16b
--- /dev/null
+++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.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.linalg
+
+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")
+ }
+
+ 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/SVDSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala
deleted file mode 100644
index 32f3f141cd652..0000000000000
--- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala
+++ /dev/null
@@ -1,158 +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.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.toArray.map(x => ret.put(x.i, x.j, x.mval))
- 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)
- }
-
- 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 a = SparseMatrix(data, m, n)
-
- val decomposed = SVD.sparseSVD(a, n)
- 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
- assertMatrixEquals(retu, svd(0))
- assertMatrixEquals(rets, DoubleMatrix.diag(svd(1)))
- assertMatrixEquals(retv, svd(2))
-
- // check multiplication guarantee
- assertMatrixEquals(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 = SVD.sparseSVD(a, k)
- val u = decomposed.U
- val s = decomposed.S
- val v = decomposed.V
- val retrank = s.data.toArray.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
- assertMatrixEquals(retu, svd(0).getColumn(0))
- assertMatrixEquals(rets, DoubleMatrix.diag(svd(1).getRow(0)))
- assertMatrixEquals(retv, svd(2).getColumn(0))
-
- // check multiplication guarantee
- assertMatrixEquals(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 = SVD.sparseSVD(a, k)
- val u = decomposed.U
- val s = decomposed.S
- val v = decomposed.V
- val retrank = s.data.toArray.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
- assertMatrixEquals(retu, svd(0).getColumn(0))
- assertMatrixEquals(rets, DoubleMatrix.diag(svd(1).getRow(0)))
- assertMatrixEquals(retv, svd(2).getColumn(0))
- }
-}
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/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)
+ ))
+ }
+ }
+}
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/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/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.
*
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..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
@@ -17,11 +17,9 @@
package org.apache.spark.mllib.regression
-
-import org.scalatest.BeforeAndAfterAll
import org.scalatest.FunSuite
-import org.apache.spark.SparkContext
+import org.apache.spark.mllib.linalg.Vectors
import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext}
class LassoSuite extends FunSuite with LocalSparkContext {
@@ -36,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.
@@ -69,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.
@@ -104,4 +112,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..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
@@ -17,9 +17,9 @@
package org.apache.spark.mllib.regression
-import org.scalatest.BeforeAndAfterAll
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 {
@@ -41,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)
@@ -57,4 +58,67 @@ 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)
+
+ 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)
+ 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)
+ }
+
+ // 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 67dd06cc0f5eb..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,14 +17,12 @@
package org.apache.spark.mllib.regression
+import org.scalatest.FunSuite
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]) = {
@@ -33,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()
@@ -70,8 +68,14 @@ 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 + ")")
}
+
+ test("do not support intercept") {
+ intercept[UnsupportedOperationException] {
+ new RidgeRegressionWithSGD().setIntercept(true)
+ }
+ }
}
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..350130c914f26
--- /dev/null
+++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala
@@ -0,0 +1,426 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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._
+import org.apache.spark.mllib.linalg.Vectors
+
+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, Vectors.dense(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, Vectors.dense(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, Vectors.dense(0.0, 1.0))
+ } else {
+ 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/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/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..e451c350b8d88
--- /dev/null
+++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.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.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 with LocalSparkContext {
+
+ 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")
+ }
+ }
+
+ 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, BinaryLabelParser, 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, 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/pom.xml b/pom.xml
index f0c877dcfe7b2..11511bcb9da52 100644
--- a/pom.xml
+++ b/pom.xml
@@ -21,7 +21,7 @@
org.apacheapache
- 13
+ 14org.apache.sparkspark-parent
@@ -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
@@ -91,6 +91,9 @@
mllibtoolsstreaming
+ sql/catalyst
+ sql/core
+ sql/hivereplassemblyexternal/twitter
@@ -107,9 +110,9 @@
1.6
- 2.10.3
+ 2.10.42.10
- 0.13.0
+ 0.17.0org.spark-project.akka2.2.3-shaded-protobuf1.7.5
@@ -118,6 +121,12 @@
2.4.10.23.70.94.6
+ 0.12.0
+ 1.3.2
+ 1.2.3
+ 8.1.14.v20131031
+ 0.3.1
+ 3.0.064m512m
@@ -127,6 +136,7 @@
maven-repoMaven Repository
+
http://repo.maven.apache.org/maven2true
@@ -135,10 +145,32 @@
false
+
+ apache-repo
+ Apache Repository
+ https://repository.apache.org/content/repositories/releases
+
+ true
+
+
+ false
+
+ jboss-repoJBoss 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-releasestrue
@@ -150,6 +182,12 @@
cloudera-repoCloudera Repositoryhttps://repository.cloudera.com/artifactory/cloudera-repos
+
+ true
+
+
+ false
+
@@ -158,22 +196,22 @@
org.eclipse.jettyjetty-util
- 7.6.8.v20121106
+ ${jetty.version}org.eclipse.jettyjetty-security
- 7.6.8.v20121106
+ ${jetty.version}org.eclipse.jettyjetty-plus
- 7.6.8.v20121106
+ ${jetty.version}org.eclipse.jettyjetty-server
- 7.6.8.v20121106
+ ${jetty.version}com.google.guava
@@ -239,7 +277,7 @@
com.twitterchill_${scala.binary.version}
- 0.3.1
+ ${chill.version}org.ow2.asm
@@ -254,7 +292,7 @@
com.twitterchill-java
- 0.3.1
+ ${chill.version}org.ow2.asm
@@ -339,14 +377,13 @@
org.apache.derbyderby10.4.2.0
- testnet.liftweblift-json_${scala.binary.version}2.5.1
@@ -358,33 +395,38 @@
com.codahale.metricsmetrics-core
- 3.0.0
+ ${codahale.metrics.version}com.codahale.metricsmetrics-jvm
- 3.0.0
+ ${codahale.metrics.version}com.codahale.metricsmetrics-json
- 3.0.0
+ ${codahale.metrics.version}com.codahale.metricsmetrics-ganglia
- 3.0.0
+ ${codahale.metrics.version}com.codahale.metricsmetrics-graphite
- 3.0.0
+ ${codahale.metrics.version}org.scala-langscala-compiler${scala.version}
+
+ org.scala-lang
+ scala-reflect
+ ${scala.version}
+ org.scala-langjline
@@ -401,11 +443,6 @@
1.9.1test
-
- commons-io
- commons-io
- 2.4
- org.easymockeasymock
@@ -542,6 +579,12 @@
+
+
+ org.codehaus.jackson
+ jackson-mapper-asl
+ 1.8.8
+
@@ -551,7 +594,7 @@
org.apache.maven.pluginsmaven-enforcer-plugin
- 1.1.1
+ 1.3.1enforce-versions
@@ -561,7 +604,7 @@
- 3.0.0
+ 3.0.4${java.version}
@@ -574,12 +617,12 @@
org.codehaus.mojobuild-helper-maven-plugin
- 1.7
+ 1.8net.alchim31.mavenscala-maven-plugin
- 3.1.5
+ 3.1.6scala-compile-first
@@ -612,7 +655,6 @@
-deprecation
- -Xms64m-Xms1024m-Xmx1024m-XX:PermSize=${PermGen}
@@ -641,7 +683,7 @@
org.apache.maven.pluginsmaven-surefire-plugin
- 2.12.4
+ 2.17true
@@ -655,7 +697,7 @@
${project.build.directory}/surefire-reports.${project.build.directory}/SparkTestSuite.txt
- -Xms64m -Xmx3g
+ -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m
@@ -680,7 +722,7 @@
org.apache.maven.pluginsmaven-shade-plugin
- 2.0
+ 2.2org.apache.maven.plugins
@@ -756,12 +798,19 @@
0.23.7
-
yarn
+
+
+
+ spark-ganglia-lgpl
+
+ extras/spark-ganglia-lgpl
+
+
java8-tests
@@ -770,7 +819,6 @@
org.apache.maven.pluginsmaven-jar-plugin
- 2.4
@@ -800,5 +848,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/project/MimaBuild.scala b/project/MimaBuild.scala
new file mode 100644
index 0000000000000..4bfa9bf95ad78
--- /dev/null
+++ b/project/MimaBuild.scala
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.streaming.scheduler"),
+ 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") ++
+ excludeSparkClass("streaming.dstream.NetworkReceiver#BlockGenerator") ++
+ excludeSparkClass("streaming.dstream.NetworkReceiver#BlockGenerator#Block") ++
+ excludeSparkClass("streaming.dstream.ReportError") ++
+ excludeSparkClass("org.apache.spark.streaming.dstream.ReportBlock") ++
+ excludeSparkClass("org.apache.spark.streaming.dstream.DStream")
+ 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 8fa220c413291..694f90a83ab67 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -17,18 +17,21 @@
import sbt._
import sbt.Classpaths.publishTask
-import Keys._
+import sbt.Keys._
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._
// 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"
+
// 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.
@@ -40,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"
@@ -52,7 +57,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 +65,21 @@ 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)
+
+ 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, repl, streaming) dependsOn(maybeYarn: _*)
+ .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")
@@ -91,19 +105,31 @@ 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 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
+ 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")
@@ -124,18 +150,18 @@ 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, 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
+ // 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 ++ maybeHiveRef ++ maybeGangliaRef
lazy val allProjects = packageProjects ++ allExternalRefs ++
Seq[ProjectReference](examples, tools, assemblyProj) ++ maybeJava8Tests
- def sharedSettings = Defaults.defaultSettings ++ Seq(
+ def sharedSettings = Defaults.defaultSettings ++ MimaBuild.mimaSettings(file(sparkHome)) ++ Seq(
organization := "org.apache.spark",
- version := "1.0.0-SNAPSHOT",
- scalaVersion := "2.10.3",
+ version := SPARK_VERSION,
+ 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),
@@ -152,12 +178,13 @@ 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.
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",
@@ -167,16 +194,23 @@ 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(
+ // 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 "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/",
+ // also check the local Maven repository ~/.m2
+ Resolver.mavenLocal
+ ),
publishMavenStyle := true,
- //useGpg in Global := true,
+ // useGpg in Global := true,
pomExtra := (
@@ -224,19 +258,18 @@ 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" % 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" % "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",
"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"),
@@ -254,22 +287,35 @@ 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")
+ 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",
+ 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",
- 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",
"com.google.code.findbugs" % "jsr305" % "1.3.9",
@@ -281,25 +327,25 @@ 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",
- "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",
"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" 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"
),
libraryDependencies ++= maybeAvro
@@ -309,7 +355,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 ),
@@ -333,34 +379,91 @@ 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(
name := "spark-graphx",
+ previousArtifact := sparkPreviousArtifact("spark-graphx"),
libraryDependencies ++= Seq(
- "org.jblas" % "jblas" % "1.2.3"
+ "org.jblas" % "jblas" % jblasVersion
)
)
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.jblas" % "jblas" % jblasVersion,
+ "org.scalanlp" %% "breeze" % "0.7"
)
)
- def streamingSettings = sharedSettings ++ Seq(
- name := "spark-streaming",
+ 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(
- "commons-io" % "commons-io" % "2.4"
+ "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" % parquetVersion,
+ "com.twitter" % "parquet-hadoop" % parquetVersion
+ )
+ )
+
+ // Since we don't include hive in the main assembly this project also acts as an alternative
+ // assembly jar.
+ def hiveSettings = sharedSettings ++ Seq(
+ name := "spark-hive",
+ javaOptions += "-XX:MaxPermSize=1g",
+ libraryDependencies ++= Seq(
+ "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,
+ // 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",
+ previousArtifact := sparkPreviousArtifact("spark-streaming")
+ )
+
def yarnCommonSettings = sharedSettings ++ Seq(
unmanagedSourceDirectories in Compile <++= baseDirectory { base =>
Seq(
@@ -384,6 +487,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"),
@@ -426,6 +534,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)
)
@@ -433,6 +542,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"
@@ -445,6 +555,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)
)
@@ -452,14 +563,15 @@ 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)
+ "org.spark-project.akka" %% "akka-zeromq" % akkaVersion excludeAll(excludeNetty)
)
)
def mqttSettings() = streamingSettings ++ Seq(
name := "spark-streaming-mqtt",
- resolvers ++= Seq("Eclipse Repo" at "https://repo.eclipse.org/content/repositories/paho-releases/"),
+ 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..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)
@@ -19,4 +19,7 @@ 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/project/project/SparkPluginBuild.scala b/project/project/SparkPluginBuild.scala
new file mode 100644
index 0000000000000..0142256e90fb7
--- /dev/null
+++ b/project/project/SparkPluginBuild.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.
+ */
+
+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.4",
+ scalacOptions := Seq("-unchecked", "-deprecation"),
+ libraryDependencies ++= Dependencies.scalaStyle
+ )
+
+ object Dependencies {
+ val scalaStyle = Seq("org.scalastyle" %% "scalastyle" % "0.4.0")
+ }
+}
diff --git a/project/spark-style/src/main/scala/org/apache/spark/scalastyle/SparkSpaceAfterCommentStartChecker.scala b/project/spark-style/src/main/scala/org/apache/spark/scalastyle/SparkSpaceAfterCommentStartChecker.scala
new file mode 100644
index 0000000000000..80d3faa3fe749
--- /dev/null
+++ b/project/spark-style/src/main/scala/org/apache/spark/scalastyle/SparkSpaceAfterCommentStartChecker.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.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.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()
+
+ 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()
+
+ 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/python/pyspark/context.py b/python/pyspark/context.py
index c9f42d3aacb58..d8667e84fedff 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
@@ -27,8 +28,10 @@
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
from py4j.java_collections import ListConverter
@@ -83,6 +86,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 +177,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
@@ -243,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)
@@ -369,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):
"""
@@ -411,7 +468,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/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/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/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)
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():
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index e72f57d9d1ab0..91fc7e637e2c6 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
@@ -28,26 +29,31 @@
from tempfile import NamedTemporaryFile
from threading import Thread
import warnings
+import heapq
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
from pyspark.rddsampler import RDDSampler
+from pyspark.storagelevel import StorageLevel
+from pyspark.resultiterable import ResultIterable
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
+ 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 +64,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):
@@ -82,6 +92,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.
@@ -171,6 +248,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)
@@ -259,6 +340,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
@@ -279,6 +361,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:
@@ -558,7 +643,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.
@@ -652,6 +756,60 @@ 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 descending order.
+ >>> sc.parallelize([10, 4, 2, 12, 3]).top(1)
+ [12]
+ >>> 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:
+ heapq.heappush(q, k)
+ else:
+ heapq.heappushpop(q, k)
+ yield q
+
+ def merge(a, b):
+ return next(topIterator(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):
"""
@@ -939,7 +1097,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):
"""
@@ -947,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])]
"""
@@ -962,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):
@@ -1009,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)
@@ -1046,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))
@@ -1081,12 +1253,71 @@ 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)
+
+ 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, 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
+
# 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
# 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/python/pyspark/serializers.py b/python/pyspark/serializers.py
index 8c6ad79059c23..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)
@@ -204,7 +213,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 +221,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 +237,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
@@ -263,7 +299,7 @@ class MarshalSerializer(FramedSerializer):
class UTF8Deserializer(Serializer):
"""
- Deserializes streams written by getBytes.
+ Deserializes streams written by String.getBytes.
"""
def loads(self, stream):
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/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())
diff --git a/python/pyspark/storagelevel.py b/python/pyspark/storagelevel.py
index b31f4762e69bc..7b6660eab231b 100644
--- a/python/pyspark/storagelevel.py
+++ b/python/pyspark/storagelevel.py
@@ -25,19 +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
-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)
+ def __repr__(self):
+ return "StorageLevel(%s, %s, %s, %s, %s)" % (
+ self.useDisk, self.useMemory, self.useOffHeap, self.deserialized, self.replication)
+
+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
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
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-langjline
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..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
}
}
@@ -124,8 +145,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/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
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/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/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)
+}
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()
+ }
+ }
+
+}
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)
+ }
}
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-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-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/start-master.sh b/sbin/start-master.sh
index ec3dfdb4197ec..c5c02491f78e1 100755
--- a/sbin/start-master.sh
+++ b/sbin/start-master.sh
@@ -22,11 +22,24 @@
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
- . "${SPARK_CONF_DIR}/spark-env.sh"
-fi
+. "$SPARK_PREFIX/bin/load-spark-env.sh"
if [ "$SPARK_MASTER_PORT" = "" ]; then
SPARK_MASTER_PORT=7077
@@ -41,3 +54,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..4912d0c0c7dfd 100755
--- a/sbin/start-slaves.sh
+++ b/sbin/start-slaves.sh
@@ -20,11 +20,25 @@
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
- . "${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
@@ -35,6 +49,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-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
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..7c2201100ef97 100755
--- a/sbin/stop-slaves.sh
+++ b/sbin/stop-slaves.sh
@@ -22,8 +22,11 @@ sbin=`cd "$sbin"; pwd`
. "$sbin/spark-config.sh"
-if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then
- . "${SPARK_CONF_DIR}/spark-env.sh"
+. "$SPARK_PREFIX/bin/load-spark-env.sh"
+
+# 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
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/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/README.md b/sql/README.md
new file mode 100644
index 0000000000000..14d5555f0c713
--- /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.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.
+
+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..9d5c6a857bb00
--- /dev/null
+++ b/sql/catalyst/pom.xml
@@ -0,0 +1,82 @@
+
+
+
+
+ 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/
+
+
+
+ yarn-alpha
+
+
+ org.apache.avro
+ avro
+
+
+
+
+
+
+
+ org.scala-lang
+ scala-reflect
+
+
+ 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/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
new file mode 100644
index 0000000000000..446d0e0bd7f54
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.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.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
+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)))
+ // 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 <:< 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.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
+ }
+
+ 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
new file mode 100644
index 0000000000000..5b6aea81cb7d1
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala
@@ -0,0 +1,354 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.parsing.combinator.lexical.StdLexical
+import scala.util.parsing.combinator.syntactical.StandardTokenParsers
+import scala.util.parsing.input.CharArrayReader.EofCh
+
+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 loosely 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 sub-project.
+ */
+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] =
+ allCaseVersions(k.str).map(x => x : Parser[String]).reduce(_ | _)
+
+ 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 INSERT = Keyword("INSERT")
+ protected val INTO = Keyword("INTO")
+ 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 OVERWRITE = Keyword("OVERWRITE")
+ 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")
+ 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])
+
+ /** 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 += (
+ "@", "*", "+", "-", "<", "=", "<>", "!=", "<=", ">=", ">", "/", "(", ")",
+ ",", ";", "%", "{", "}", ":"
+ )
+
+ 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)) }
+ ) | insert
+
+ 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 => Limit(l, withOrder) }.getOrElse(withOrder)
+ 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] =
+ expression ~ (opt(AS) ~> opt(ident)) ^^ {
+ case e ~ None => e
+ case e ~ Some(a) => Alias(e, a)()
+ }
+
+ 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] =
+ 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 tableName ~ alias => UnresolvedRelation(None, tableName, alias)
+ } |
+ "(" ~> 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] =
+ comparisonExpression * (AND ^^^ { (e1: Expression, e2: Expression) => And(e1,e2) })
+
+ 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) } |
+ 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 ~ 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)
+ } |
+ 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..4ebc0e70d946b
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -0,0 +1,202 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.analysis
+
+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
+ * 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 :_*),
+ Batch("AnalysisOperators", fixedPoint,
+ EliminateAnalysisOperators)
+ )
+
+ /**
+ * 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)()
+ case GetField(c, name) => GetField(c, name.toLowerCase)
+ }
+ }
+ }
+
+ /**
+ * Replaces [[UnresolvedAttribute]]s with concrete
+ * [[catalyst.expressions.AttributeReference AttributeReferences]] from a logical plan node's
+ * children.
+ */
+ object ResolveReferences extends Rule[LogicalPlan] {
+ def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
+ 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 [[catalyst.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
+ }
+}
+
+/**
+ * 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/analysis/Catalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
new file mode 100644
index 0000000000000..f30b5d816703a
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
@@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.analysis
+
+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]].
+ */
+trait Catalog {
+ def lookupRelation(
+ databaseName: Option[String],
+ tableName: String,
+ 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]()
+
+ override def registerTable(
+ databaseName: Option[String],
+ tableName: String,
+ plan: LogicalPlan): Unit = {
+ tables += ((tableName, plan))
+ }
+
+ override def unregisterTable(
+ databaseName: Option[String],
+ tableName: String) = {
+ tables -= tableName
+ }
+
+ override def unregisterAllTables() = {
+ tables.clear()
+ }
+
+ override 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"))
+ 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, tableWithQualifiers)).getOrElse(tableWithQualifiers)
+ }
+}
+
+/**
+ * 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)
+ }
+
+ override def unregisterTable(databaseName: Option[String], tableName: String): Unit = {
+ overrides.remove((databaseName, tableName))
+ }
+
+ override def unregisterAllTables(): Unit = {
+ overrides.clear()
+ }
+}
+
+/**
+ * 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
+ }
+
+ def unregisterTable(databaseName: Option[String], tableName: String): Unit = {
+ throw new UnsupportedOperationException
+ }
+
+ override def unregisterAllTables(): Unit = {}
+}
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..c0255701b7ba5
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.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.sql.catalyst.analysis
+
+import org.apache.spark.sql.catalyst.expressions.Expression
+
+/** 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..4557d77160fad
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
@@ -0,0 +1,273 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.analysis
+
+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
+ * 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]] 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 {
+ // 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..a6ce90854dcb4
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.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.catalyst.analysis
+
+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 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.
+ */
+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..9f37ca904ffeb
--- /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 org.apache.spark.sql.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..d629172a7426e
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.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.sql.catalyst.analysis
+
+import org.apache.spark.sql.catalyst.{errors, trees}
+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
+
+/**
+ * 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 {
+ override 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] {
+ 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
+
+ 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 {
+ override def dataType = throw new UnresolvedException(this, "dataType")
+ override def foldable = throw new UnresolvedException(this, "foldable")
+ 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(",")})"
+}
+
+/**
+ * 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] {
+
+ 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
+
+ override def newInstance = this
+ override 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(t) => input.filter(_.qualifiers contains t)
+ }
+ val mappedAttributes = expandedAttributes.map(mapFunction).zip(input).map {
+ case (n: NamedExpression, _) => n
+ case (e, originalAttribute) =>
+ Alias(e, originalAttribute.name)(qualifiers = originalAttribute.qualifiers)
+ }
+ 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/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
new file mode 100644
index 0000000000000..987befe8e22ee
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 java.sql.Timestamp
+
+import scala.language.implicitConversions
+
+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.
+ *
+ * {{{
+ * 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 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)
+
+ def as(s: Symbol) = Alias(expr, s.name)()
+ }
+
+ trait ExpressionConversions {
+ implicit class DslExpression(e: Expression) extends ImplicitOperators {
+ 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 {
+ override def expr: Expression = Literal(s)
+ def attr = analysis.UnresolvedAttribute(s)
+ }
+
+ abstract class ImplicitAttribute extends ImplicitOperators {
+ def s: String
+ def expr = attr
+ def attr = analysis.UnresolvedAttribute(s)
+
+ /** 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 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) {
+ def notNull = a.withNullability(false)
+ def nullable = a.withNullability(true)
+
+ // Protobuf terminology
+ def required = a.withNullability(false)
+
+ def at(ordinal: Int) = BoundReference(ordinal, a)
+ }
+ }
+
+
+ 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..bdeb660b1ecb7
--- /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.catalyst
+
+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) {
+
+ // 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..4ebf6c4584b94
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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
+ * 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 eval(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)
+ }
+ }
+ }
+}
+
+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..17118499d0c87
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
@@ -0,0 +1,203 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.expressions
+
+import java.sql.Timestamp
+
+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
+ 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])
+ }
+
+ // 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"))
+ }
+
+ // 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
+ }
+
+ // 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 => timestampToLong(t))
+ 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 => timestampToLong(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 => timestampToLong(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 => timestampToLong(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 =>
+ // Note that we lose precision here.
+ 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 eval(input: Row): Any = {
+ val evaluated = child.eval(input)
+ if (evaluated == null) {
+ null
+ } else {
+ 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
new file mode 100644
index 0000000000000..8a1db8e796816
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.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.catalyst.expressions
+
+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}
+
+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 eval(input: Row = null): EvaluatedType
+
+ /**
+ * 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[T]` 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.eval(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")
+ }
+ }
+ }
+
+ /**
+ * 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,
+ 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.eval(i)
+ if(evalE1 == null) {
+ null
+ } else {
+ val evalE2 = e2.eval(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")
+ }
+ }
+ }
+ }
+
+ /**
+ * 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,
+ 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.eval(i: Row)
+ if(evalE1 == null) {
+ null
+ } else {
+ val evalE2 = e2.eval(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")
+ }
+ }
+ }
+ }
+
+ /**
+ * 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,
+ 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.eval(i)
+ if(evalE1 == null) {
+ null
+ } else {
+ val evalE2 = e2.eval(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")
+ }
+ }
+ }
+ }
+
+ /**
+ * 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.eval(i)
+ if(evalE1 == null) {
+ null
+ } else {
+ val evalE2 = e2.eval(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] {
+ self: Product =>
+
+ def symbol: String
+
+ override def foldable = left.foldable && right.foldable
+
+ override 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 =>
+
+ 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
new file mode 100644
index 0000000000000..c9b7cea6a3e5f
--- /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.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.length)
+ var i = 0
+ while (i < exprArray.length) {
+ outputArray(i) = exprArray(i).eval(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.length) {
+ mutableRow(i) = exprArray(i).eval(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..38f836f0a1a0e
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.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.catalyst.expressions
+
+import java.util.Random
+import org.apache.spark.sql.catalyst.types.DoubleType
+
+
+case object Rand extends LeafExpression {
+ 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()"
+}
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..77b5429bad432
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala
@@ -0,0 +1,237 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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.
+ *
+ * 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
+
+ /** 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
+ }
+}
+
+/**
+ * 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.eval(a)
+ val right = order.child.eval(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..5e089f7618e0a
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.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.sql.catalyst.expressions
+
+import org.apache.spark.sql.catalyst.types.DataType
+
+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 eval(input: Row): Any = {
+ children.size match {
+ case 1 => function.asInstanceOf[(Any) => Any](children(0).eval(input))
+ case 2 =>
+ function.asInstanceOf[(Any, Any) => Any](
+ children(0).eval(input),
+ children(1).eval(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..08b2f11d20f5e
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.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.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
+
+/**
+ * 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 {
+ 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/WrapDynamic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala
new file mode 100644
index 0000000000000..e787c59e75723
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.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.catalyst.expressions
+
+import scala.language.dynamics
+
+import org.apache.spark.sql.catalyst.types.DataType
+
+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 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)
+ }
+}
+
+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..b152f95f96c70
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala
@@ -0,0 +1,270 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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 =>
+
+ /**
+ * Creates a new instance that can be used to compute this aggregate expression for a group
+ * 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}")
+}
+
+/**
+ * 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 relevant 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 =>
+
+ override type EvaluatedType = Any
+
+ /** Base should return the generic aggregate expression that this function is computing */
+ val base: AggregateExpression
+ override def references = base.references
+ override def nullable = base.nullable
+ override def dataType = base.dataType
+
+ def update(input: Row): Unit
+ override def eval(input: Row): Any
+
+ // Do we really need this?
+ override def newInstance() = makeCopy(productIterator.map { case a: AnyRef => a }.toArray)
+}
+
+case class Count(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] {
+ override def references = child.references
+ override def nullable = false
+ override def dataType = IntegerType
+ override def toString = s"COUNT($child)"
+
+ override 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 {
+ 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] {
+ override def references = child.references
+ override def nullable = false
+ override 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] {
+ override def references = child.references
+ override def nullable = false
+ override 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] {
+
+ 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] {
+ 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 = {
+ 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).eval(EmptyRow))
+ private val sumAsDouble = Cast(sum, DoubleType)
+
+ private val addFunction = Add(sum, expr)
+
+ override def eval(input: Row): Any =
+ sumAsDouble.eval(EmptyRow).asInstanceOf[Double] / count.toDouble
+
+ override 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 = _
+
+ override def update(input: Row): Unit = {
+ val evaluatedExpr = expr.map(_.eval(input))
+ if (evaluatedExpr.map(_ != null).reduceLeft(_ || _)) {
+ count += 1
+ }
+ }
+
+ 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).eval(null))
+
+ private val addFunction = Add(sum, expr)
+
+ override def update(input: Row): Unit = {
+ sum.update(addFunction, input)
+ }
+
+ override def eval(input: Row): Any = sum.eval(null)
+}
+
+case class SumDistinctFunction(expr: Expression, base: AggregateExpression)
+ extends AggregateFunction {
+
+ def this() = this(null, null) // Required for serialization.
+
+ private val seen = new scala.collection.mutable.HashSet[Any]()
+
+ override def update(input: Row): Unit = {
+ val evaluatedExpr = expr.eval(input)
+ if (evaluatedExpr != null) {
+ seen += evaluatedExpr
+ }
+ }
+
+ override def eval(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]()
+
+ override def update(input: Row): Unit = {
+ val evaluatedExpr = expr.map(_.eval(input))
+ if (evaluatedExpr.map(_ != null).reduceLeft(_ && _)) {
+ seen += evaluatedExpr
+ }
+ }
+
+ override def eval(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
+
+ override def update(input: Row): Unit = {
+ if (result == null) {
+ result = expr.eval(input)
+ }
+ }
+
+ 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
new file mode 100644
index 0000000000000..c79c1847cedf5
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.expressions
+
+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
+
+ def dataType = child.dataType
+ override def foldable = child.foldable
+ def nullable = child.nullable
+ override def toString = s"-$child"
+
+ override def eval(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 eval(input: Row): Any = n2(input, left, right, _.plus(_, _))
+}
+
+case class Subtract(left: Expression, right: Expression) extends BinaryArithmetic {
+ def symbol = "-"
+
+ override def eval(input: Row): Any = n2(input, left, right, _.minus(_, _))
+}
+
+case class Multiply(left: Expression, right: Expression) extends BinaryArithmetic {
+ def symbol = "*"
+
+ override def eval(input: Row): Any = n2(input, left, right, _.times(_, _))
+}
+
+case class Divide(left: Expression, right: Expression) extends BinaryArithmetic {
+ def symbol = "/"
+
+ override def eval(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 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
new file mode 100644
index 0000000000000..c947155cb701c
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.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.catalyst.expressions
+
+import org.apache.spark.sql.catalyst.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 eval(input: Row): Any = {
+ if (child.dataType.isInstanceOf[ArrayType]) {
+ 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) {
+ null
+ } else {
+ baseValue(o)
+ }
+ } else {
+ val baseValue = child.eval(input).asInstanceOf[Map[Any, _]]
+ val key = ordinal.eval(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 eval(input: Row): Any = {
+ val baseValue = child.eval(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..dd78614754e12
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.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.catalyst.expressions
+
+import org.apache.spark.sql.catalyst.trees
+import org.apache.spark.sql.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 {
+ self: Product =>
+
+ override type EvaluatedType = TraversableOnce[Row]
+
+ override lazy val dataType =
+ ArrayType(StructType(output.map(a => StructField(a.name, a.dataType, a.nullable))))
+
+ override def nullable = false
+
+ override 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. */
+ 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 = {
+ 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])
+
+ private 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 eval(input: Row): TraversableOnce[Row] = {
+ child.dataType match {
+ case ArrayType(_) =>
+ 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.eval(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..e15e16d633365
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.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.catalyst.expressions
+
+import java.sql.Timestamp
+
+import org.apache.spark.sql.catalyst.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 d: BigDecimal => Literal(d, DecimalType)
+ case t: Timestamp => Literal(t, TimestampType)
+ case a: Array[Byte] => Literal(a, BinaryType)
+ 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 eval(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.eval(input)
+ }
+
+ 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
new file mode 100644
index 0000000000000..a8145c37c20fa
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala
@@ -0,0 +1,160 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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 {
+ 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 subsequent 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 toAttribute = this
+ def newInstance: Attribute
+ override def references = Set(this)
+}
+
+/**
+ * Used to assign a new name to a computation.
+ * For example the SQL expression "1 + 1 AS a" could be represented as follows:
+ * Alias(Add(Literal(1), Literal(1), "a")()
+ *
+ * @param child the computation being performed
+ * @param name 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] {
+
+ override type EvaluatedType = Any
+
+ override def eval(input: Row) = child.eval(input)
+
+ override def dataType = child.dataType
+ override def nullable = child.nullable
+ override def references = child.references
+
+ override 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
+ }
+
+ override 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.
+ */
+ override def withQualifiers(newQualifiers: Seq[String]) = {
+ if (newQualifiers == qualifiers) {
+ this
+ } else {
+ AttributeReference(name, dataType, nullable)(exprId, newQualifiers)
+ }
+ }
+
+ // 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/expressions/nullFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala
new file mode 100644
index 0000000000000..ce6d99c911ab3
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.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.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 {
+ 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 eval(input: Row): Any = {
+ var i = 0
+ var result: Any = null
+ while(i < children.size && result == null) {
+ result = children(i).eval(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 eval(input: Row): Any = {
+ child.eval(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 eval(input: Row): Any = {
+ child.eval(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..573ec052f4266
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.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.sql.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 `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
new file mode 100644
index 0000000000000..da5b2cf5b0362
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
@@ -0,0 +1,171 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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}
+
+object InterpretedPredicate {
+ def apply(expression: Expression): (Row => Boolean) = {
+ (r: Row) => expression.eval(r).asInstanceOf[Boolean]
+ }
+}
+
+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 eval(input: Row): Any = {
+ child.eval(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 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 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 ) {
+ null
+ } else {
+ true
+ }
+ }
+}
+
+case class Or(left: Expression, right: Expression) extends BinaryPredicate {
+ def symbol = "||"
+
+ 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) {
+ null
+ } else {
+ false
+ }
+ }
+}
+
+abstract class BinaryComparison extends BinaryPredicate {
+ self: Product =>
+}
+
+case class Equals(left: Expression, right: Expression) extends BinaryComparison {
+ def symbol = "="
+ 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 eval(input: Row): Any = c2(input, left, right, _.lt(_, _))
+}
+
+case class LessThanOrEqual(left: Expression, right: Expression) extends BinaryComparison {
+ def symbol = "<="
+ override def eval(input: Row): Any = c2(input, left, right, _.lteq(_, _))
+}
+
+case class GreaterThan(left: Expression, right: Expression) extends BinaryComparison {
+ def symbol = ">"
+ override def eval(input: Row): Any = c2(input, left, right, _.gt(_, _))
+}
+
+case class GreaterThanOrEqual(left: Expression, right: Expression) extends BinaryComparison {
+ def symbol = ">="
+ override def eval(input: Row): Any = c2(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 eval(input: Row): Any = {
+ if (predicate.eval(input).asInstanceOf[Boolean]) {
+ trueValue.eval(input)
+ } else {
+ falseValue.eval(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..a27c71db1b999
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.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.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
+
+
+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))
+ }
+
+ 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) {
+ null
+ } else {
+ val r = right.eval(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/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..37b23ba58289c
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
@@ -0,0 +1,168 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.optimizer
+
+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 =
+ Batch("ConstantFolding", Once,
+ ConstantFolding,
+ BooleanSimplification,
+ SimplifyFilters,
+ SimplifyCasts) ::
+ Batch("Filter Pushdown", Once,
+ CombineFilters,
+ PushPredicateThroughProject,
+ PushPredicateThroughInnerJoin) :: Nil
+}
+
+/**
+ * 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.eval(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)
+ }
+}
+
+/**
+ * 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
+ * [[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: Attribute) -> 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..67833664b35ae
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.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.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
+
+/**
+ * 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..0a030b6c21f1f
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/package.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.sql.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..6dd816aa91dd1
--- /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.catalyst.planning
+
+import scala.annotation.tailrec
+
+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.
+ * 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.
+ * [[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)
+
+ 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..8199a80f5d6bd
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.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.catalyst.plans
+
+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 =>
+
+ 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..ae8d7d3e4257f
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.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.catalyst.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..7c616788a3830
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.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.catalyst.plans.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..5eb52d5350f55
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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 =>
+
+ /**
+ * 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..d3f9d0fb93237
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.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.sql.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression}
+
+/**
+ * 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..f8fe558511bfd
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.analysis
+import org.apache.spark.sql.catalyst.expressions.Attribute
+
+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..cfc0b0c3a8d98
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.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.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 {
+ 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: LogicalPlan,
+ 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 Limit(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
+}
+
+/**
+ * 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)
+ case other => other
+ }
+
+ 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..7146fbd540f29
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.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.catalyst.plans.logical
+
+import org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder}
+
+/**
+ * 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..42bdab42b79ff
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/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.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..ffb3a92f8f340
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
@@ -0,0 +1,208 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.plans.physical
+
+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
+
+/**
+ * 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 {
+
+ override def children = expressions
+ override def references = expressions.flatMap(_.references).toSet
+ override def nullable = false
+ override def dataType = IntegerType
+
+ private[this] 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
+ }
+
+ override def eval(input: Row = null): EvaluatedType =
+ throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}")
+}
+
+/**
+ * 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.
+ *
+ * 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 {
+
+ override def children = ordering
+ override def references = ordering.flatMap(_.references).toSet
+ override def nullable = false
+ override def dataType = IntegerType
+
+ private[this] 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
+ }
+
+ 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/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..1076537bc7602
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.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.catalyst.rules
+
+import org.apache.spark.sql.Logging
+import org.apache.spark.sql.catalyst.trees.TreeNode
+
+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..9db96f89dd03c
--- /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 org.apache.spark.sql.catalyst.trees.TreeNode
+import org.apache.spark.sql.catalyst.util.sideBySide
+
+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 (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 (plan, rule) =>
+ val result = rule(plan)
+ if (!result.fastEquals(plan)) {
+ logger.debug(
+ s"""
+ |=== Applying Rule ${rule.ruleName} ===
+ |${sideBySide(plan.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..ca82c5d93d4cb
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/package.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.sql.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..0369129393a08
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
@@ -0,0 +1,377 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.trees
+
+import org.apache.spark.sql.catalyst.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
+ }
+
+ /**
+ * 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(",")})"
+ }
+}
+
+/**
+ * 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..d159ecdd5d781
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.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.sql.catalyst
+
+import org.apache.spark.sql.Logger
+
+/**
+ * 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..cdeb01a9656f4
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala
@@ -0,0 +1,159 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.types
+
+import java.sql.Timestamp
+
+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 */
+ 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
+ @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 {
+ type JvmType = Array[Byte]
+}
+case object BooleanType extends NativeType {
+ type JvmType = Boolean
+ @transient lazy val tag = typeTag[JvmType]
+ 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
+ // 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
+ @transient lazy val tag = typeTag[JvmType]
+ val numeric = implicitly[Numeric[Long]]
+ val integral = implicitly[Integral[Long]]
+ val ordering = implicitly[Ordering[JvmType]]
+}
+
+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]]
+}
+
+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]]
+}
+
+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]]
+}
+
+/** 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
+ @transient lazy val tag = typeTag[JvmType]
+ val numeric = implicitly[Numeric[BigDecimal]]
+ val fractional = implicitly[Fractional[BigDecimal]]
+ val ordering = implicitly[Ordering[JvmType]]
+}
+
+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]]
+}
+
+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]]
+}
+
+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..de24449590f9a
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.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.sql.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..a001d953592db
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.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.sql.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..4589129cd1c90
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/package.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
+
+/**
+ * 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
+
+ val Row = catalyst.expressions.Row
+}
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..46b2250aab231
--- /dev/null
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.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.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)
+ }
+}
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
new file mode 100644
index 0000000000000..4c313585c6386
--- /dev/null
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.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.catalyst.analysis
+
+import org.scalatest.FunSuite
+
+import org.apache.spark.sql.catalyst.plans.logical._
+
+/* Implicit conversions */
+import org.apache.spark.sql.catalyst.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))
+ }
+}
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
new file mode 100644
index 0000000000000..b9e0f8e9dcc5f
--- /dev/null
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.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.catalyst.analysis
+
+import org.scalatest.FunSuite
+
+import org.apache.spark.sql.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/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala
new file mode 100644
index 0000000000000..888a19d79f7e4
--- /dev/null
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala
@@ -0,0 +1,272 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.expressions
+
+import java.sql.Timestamp
+
+import org.scalatest.FunSuite
+
+import org.apache.spark.sql.catalyst.types._
+
+/* Implicit conversions */
+import org.apache.spark.sql.catalyst.dsl.expressions._
+
+class ExpressionEvaluationSuite extends FunSuite {
+
+ test("literals") {
+ assert((Literal(1) + Literal(1)).eval(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.eval(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.eval(null)
+ if (result != answer)
+ fail(s"$expr should not evaluate to $result, expected: $answer")
+ }
+ }
+ }
+
+ def evaluate(expression: Expression, inputRow: Row = EmptyRow): Any = {
+ expression.eval(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]("**")))
+ }
+ }
+
+ 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(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)
+ }
+
+ 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)
+ }
+}
+
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..20dfba847790c
--- /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.catalyst.optimizer
+
+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.{DoubleType, 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("AnalysisNodes", Once,
+ EliminateAnalysisOperators) ::
+ 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)
+ }
+}
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..ef47850455a37
--- /dev/null
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala
@@ -0,0 +1,237 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.optimizer
+
+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._
+
+/* Implicit conversions */
+import org.apache.spark.sql.catalyst.dsl.plans._
+import org.apache.spark.sql.catalyst.dsl.expressions._
+
+class FilterPushdownSuite extends OptimizerTest {
+
+ object Optimize extends RuleExecutor[LogicalPlan] {
+ val batches =
+ Batch("Subqueries", Once,
+ EliminateAnalysisOperators) ::
+ Batch("Filter Pushdown", Once,
+ 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(analysis.EliminateAnalysisOperators(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, analysis.EliminateAnalysisOperators(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, analysis.EliminateAnalysisOperators(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, analysis.EliminateAnalysisOperators(correctAnswer))
+ }
+}
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..89982d5cd8d74
--- /dev/null
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.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.catalyst.optimizer
+
+import org.scalatest.FunSuite
+
+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
+ * 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)
+ }
+}
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
new file mode 100644
index 0000000000000..4b2d45584045f
--- /dev/null
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.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.catalyst.trees
+
+import org.scalatest.FunSuite
+
+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] {
+ 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))
+ }
+}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala
new file mode 100644
index 0000000000000..1ddc41a731ff5
--- /dev/null
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala
@@ -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.
+ */
+
+package org.apache.spark.sql.catalyst.trees
+
+import scala.collection.mutable.ArrayBuffer
+
+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))
+ }
+
+ 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)
+ }
+}
diff --git a/sql/core/pom.xml b/sql/core/pom.xml
new file mode 100644
index 0000000000000..85580ed6b822f
--- /dev/null
+++ b/sql/core/pom.xml
@@ -0,0 +1,87 @@
+
+
+
+
+ 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/
+
+
+ yarn-alpha
+
+
+ org.apache.avro
+ avro
+
+
+
+
+
+
+
+ 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/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
new file mode 100644
index 0000000000000..d3d4c56bafe41
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -0,0 +1,244 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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
+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
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.optimizer.Optimizer
+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._
+
+/**
+ * :: 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
+ 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.
+ */
+ @Experimental
+ 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 Parquet file, returning the result as a [[SchemaRDD]].
+ *
+ * @group userf
+ */
+ def parquetFile(path: String): SchemaRDD =
+ new SchemaRDD(this, parquet.ParquetRelation(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
+ }
+
+ /** 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
+
+ val strategies: Seq[Strategy] =
+ TakeOrdered ::
+ PartialAggregation ::
+ HashJoin ::
+ 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
+ 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 }
+
+ def simpleString: String = stringOrError(executedPlan)
+
+ 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..16da7fd92bffe
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala
@@ -0,0 +1,311 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements. See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF 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.{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._
+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
+
+/**
+ * :: 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.
+ *
+ * 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
+ */
+@AlphaComponent
+class SchemaRDD(
+ @transient val sqlContext: SQLContext,
+ @transient protected[spark] val logicalPlan: LogicalPlan)
+ extends RDD[Row](sqlContext.sparkContext, Nil) with SchemaRDDLike {
+
+ def baseSchemaRDD = this
+
+ // =========================================================================================
+ // 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 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,
+ on: Option[Expression] = None): SchemaRDD =
+ new SchemaRDD(sqlContext, Join(logicalPlan, otherPlan.logicalPlan, joinType, on))
+
+ /**
+ * 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).as('x)
+ * val y = schemaRDD.where('a === 2).as('y)
+ * x.join(y).where("x.a".attr === "y.a".attr),
+ * }}}
+ *
+ * @group Query
+ */
+ def as(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
+ */
+ @Experimental
+ 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
+ */
+ @Experimental
+ 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
+ */
+ @Experimental
+ 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
+ */
+ @Experimental
+ def insertInto(tableName: String, overwrite: Boolean = false) =
+ new SchemaRDD(
+ sqlContext,
+ InsertIntoTable(UnresolvedRelation(None, tableName), Map.empty, logicalPlan, overwrite))
+
+ /**
+ * 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..3dd9897c0d3b8
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.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
+
+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.simpleString}""".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..573345e42c43c
--- /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(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/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..3c39e1d350fa8
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.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.sql.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.columnar.compression.CompressibleColumnAccessor
+
+/**
+ * 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](
+ protected val buffer: ByteBuffer,
+ protected val columnType: ColumnType[T, JvmType])
+ extends ColumnAccessor {
+
+ protected def initialize() {}
+
+ def hasNext = buffer.hasRemaining
+
+ def extractTo(row: MutableRow, ordinal: Int) {
+ columnType.setField(row, ordinal, extractSingle(buffer))
+ }
+
+ def extractSingle(buffer: ByteBuffer): JvmType = columnType.extract(buffer)
+
+ protected def underlyingBuffer = buffer
+}
+
+private[sql] abstract class NativeColumnAccessor[T <: NativeType](
+ 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)
+
+private[sql] class IntColumnAccessor(buffer: ByteBuffer)
+ extends NativeColumnAccessor(buffer, INT)
+
+private[sql] class ShortColumnAccessor(buffer: ByteBuffer)
+ extends NativeColumnAccessor(buffer, SHORT)
+
+private[sql] class LongColumnAccessor(buffer: ByteBuffer)
+ extends NativeColumnAccessor(buffer, LONG)
+
+private[sql] class ByteColumnAccessor(buffer: ByteBuffer)
+ extends NativeColumnAccessor(buffer, BYTE)
+
+private[sql] class DoubleColumnAccessor(buffer: ByteBuffer)
+ extends NativeColumnAccessor(buffer, DOUBLE)
+
+private[sql] class FloatColumnAccessor(buffer: ByteBuffer)
+ extends NativeColumnAccessor(buffer, FLOAT)
+
+private[sql] class StringColumnAccessor(buffer: ByteBuffer)
+ extends NativeColumnAccessor(buffer, STRING)
+
+private[sql] class BinaryColumnAccessor(buffer: ByteBuffer)
+ extends BasicColumnAccessor[BinaryType.type, Array[Byte]](buffer, BINARY)
+ with NullableColumnAccessor
+
+private[sql] class GenericColumnAccessor(buffer: ByteBuffer)
+ extends BasicColumnAccessor[DataType, Array[Byte]](buffer, GENERIC)
+ with NullableColumnAccessor
+
+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 = dup.getInt()
+
+ columnTypeId match {
+ 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/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala
new file mode 100644
index 0000000000000..048ee66bff44b
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.{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.columnar.compression.{AllCompressionSchemes, CompressibleColumnBuilder}
+
+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 = "")
+
+ /**
+ * 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] class BasicColumnBuilder[T <: DataType, JvmType](
+ val columnStats: ColumnStats[T, JvmType],
+ val columnType: ColumnType[T, JvmType])
+ extends ColumnBuilder {
+
+ protected var columnName: String = _
+
+ 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
+
+ // Reserves 4 bytes for column type ID
+ buffer = ByteBuffer.allocate(4 + size * columnType.defaultSize)
+ buffer.order(ByteOrder.nativeOrder()).putInt(columnType.typeId)
+ }
+
+ override def appendFrom(row: Row, ordinal: Int) {
+ val field = columnType.getField(row, ordinal)
+ buffer = ensureFreeSpace(buffer, columnType.actualSize(field))
+ columnType.append(field, buffer)
+ }
+
+ override def build() = {
+ buffer.limit(buffer.position()).rewind()
+ buffer
+ }
+}
+
+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] 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 BooleanColumnBuilder extends NativeColumnBuilder(new BooleanColumnStats, BOOLEAN)
+
+private[sql] class IntColumnBuilder extends NativeColumnBuilder(new IntColumnStats, INT)
+
+private[sql] class ShortColumnBuilder extends NativeColumnBuilder(new ShortColumnStats, SHORT)
+
+private[sql] class LongColumnBuilder extends NativeColumnBuilder(new LongColumnStats, LONG)
+
+private[sql] class ByteColumnBuilder extends NativeColumnBuilder(new ByteColumnStats, BYTE)
+
+private[sql] class DoubleColumnBuilder extends NativeColumnBuilder(new DoubleColumnStats, DOUBLE)
+
+private[sql] class FloatColumnBuilder extends NativeColumnBuilder(new FloatColumnStats, FLOAT)
+
+private[sql] class StringColumnBuilder extends NativeColumnBuilder(new StringColumnStats, STRING)
+
+private[sql] class BinaryColumnBuilder extends ComplexColumnBuilder(BINARY)
+
+// TODO (lian) Add support for array, struct and map
+private[sql] class GenericColumnBuilder extends ComplexColumnBuilder(GENERIC)
+
+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/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala
new file mode 100644
index 0000000000000..95602d321dc6f
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala
@@ -0,0 +1,366 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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._
+
+/**
+ * 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.
+ */
+ 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
new file mode 100644
index 0000000000000..5be76890afe31
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala
@@ -0,0 +1,277 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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
+ * 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
+
+ /**
+ * 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](
+ 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: TypeTag[dataType.JvmType] = 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()
+ }
+
+ 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) {
+ override def append(v: Long, buffer: ByteBuffer) {
+ buffer.putLong(v)
+ }
+
+ 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) {
+ override def append(v: Float, buffer: ByteBuffer) {
+ buffer.putFloat(v)
+ }
+
+ 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) {
+ override def append(v: Double, buffer: ByteBuffer) {
+ buffer.putDouble(v)
+ }
+
+ 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) {
+ 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
+ }
+
+ 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) {
+ override def append(v: Byte, buffer: ByteBuffer) {
+ buffer.put(v)
+ }
+
+ 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) {
+ override def append(v: Short, buffer: ByteBuffer) {
+ buffer.putShort(v)
+ }
+
+ 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) {
+ 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)
+ }
+
+ 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](
+ 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) {
+ 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) {
+ 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 {
+ def apply(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/InMemoryColumnarTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala
new file mode 100644
index 0000000000000..8a24733047423
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.catalyst.expressions.{GenericMutableRow, Attribute}
+import org.apache.spark.sql.execution.{SparkPlan, LeafNode}
+import org.apache.spark.sql.Row
+
+private[sql] case class InMemoryColumnarTableScan(attributes: Seq[Attribute], child: SparkPlan)
+ extends LeafNode {
+
+ override def output: Seq[Attribute] = attributes
+
+ lazy val cachedColumnBuffers = {
+ val output = child.output
+ val cached = child.execute().mapPartitions { iterator =>
+ val columnBuilders = output.map { attribute =>
+ ColumnBuilder(ColumnType(attribute.dataType).typeId, 0, attribute.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/columnar/NullableColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala
new file mode 100644
index 0000000000000..7d49ab07f7a53
--- /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 protected 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..2a3b6fc1e46d3
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.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.columnar
+
+import java.nio.{ByteBuffer, ByteOrder}
+
+import org.apache.spark.sql.Row
+
+/**
+ * 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 = _
+ 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()
+
+ val buffer = ByteBuffer
+ .allocate(4 + 4 + nullDataLen + nonNulls.remaining())
+ .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/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..fd3b1adf9687a
--- /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[T])
+
+ protected def isWorthCompressing(encoder: Encoder[T]) = {
+ 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: Encoder[T] = {
+ 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..c605a8e4434e3
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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[T <: NativeType] {
+ def gatherCompressibilityStats(value: T#JvmType, columnType: NativeColumnType[T]) {}
+
+ def compressedSize: Int
+
+ def uncompressedSize: Int
+
+ def compressionRatio: Double = {
+ if (uncompressedSize > 0) compressedSize.toDouble / uncompressedSize else 1.0
+ }
+
+ def compress(from: ByteBuffer, to: ByteBuffer, columnType: NativeColumnType[T]): 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[T <: NativeType]: Encoder[T]
+
+ 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] = CompressionScheme.all
+}
+
+private[sql] object CompressionScheme {
+ 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) {
+ // 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..df8220b556edd
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala
@@ -0,0 +1,482 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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._
+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[T <: NativeType] = new this.Encoder[T]
+
+ override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = {
+ new this.Decoder(buffer, columnType)
+ }
+
+ class Encoder[T <: NativeType] extends compression.Encoder[T] {
+ override def uncompressedSize = 0
+
+ override def compressedSize = 0
+
+ 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
+ }
+ }
+
+ 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 val typeId = 1
+
+ override def encoder[T <: NativeType] = new this.Encoder[T]
+
+ 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[T <: NativeType] extends compression.Encoder[T] {
+ 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(value: T#JvmType, columnType: NativeColumnType[T]) {
+ 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(from: ByteBuffer, to: ByteBuffer, columnType: NativeColumnType[T]) = {
+ 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 val typeId = 2
+
+ // 32K unique values allowed
+ val MAX_DICT_SIZE = Short.MaxValue
+
+ override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = {
+ new this.Decoder(buffer, columnType)
+ }
+
+ 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[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
+
+ // 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[T#JvmType](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(value: T#JvmType, columnType: NativeColumnType[T]) {
+ 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(from: ByteBuffer, to: ByteBuffer, columnType: NativeColumnType[T]) = {
+ 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), 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
+ }
+}
+
+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/main/scala/org/apache/spark/sql/execution/Aggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala
new file mode 100644
index 0000000000000..3a4f071eebedf
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala
@@ -0,0 +1,202 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution
+
+import java.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._
+
+/**
+ * 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 with NoBind {
+
+ override def requiredChildDistribution =
+ if (partial) {
+ UnspecifiedDistribution :: Nil
+ } else {
+ if (groupingExpressions == Nil) {
+ AllTuples :: Nil
+ } else {
+ ClusteredDistribution(groupingExpressions) :: Nil
+ }
+ }
+
+ 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.
+ private[this] val childOutput = child.output
+
+ override def output = aggregateExpressions.map(_.toAttribute)
+
+ /**
+ * 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
+ private[this] 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
+ private[this] lazy val computedSchema = computedAggregates.map(_.resultAttribute)
+
+ /** Creates a new aggregate buffer for a group. */
+ private[this] 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
+ }
+
+ /** Named attributes used to substitute grouping attributes into the final result. */
+ @transient
+ private[this] lazy val namedGroups = groupingExpressions.map {
+ case ne: NamedExpression => ne -> ne.toAttribute
+ case e => e -> Alias(e, s"groupingExpr:$e")().toAttribute
+ }
+
+ /**
+ * A map of substitutions that are used to insert the aggregate expressions and grouping
+ * expression into the final result expression.
+ */
+ @transient
+ private[this] 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
+ private[this] lazy val resultExpressions = aggregateExpressions.map { agg =>
+ agg.transform {
+ case e: Expression if resultMap.contains(e) => resultMap(e)
+ }
+ }
+
+ override def execute() = attachTree(this, "execute") {
+ 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).eval(EmptyRow)
+ i += 1
+ }
+
+ Iterator(resultProjection(aggregateResults))
+ }
+ } else {
+ 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).eval(EmptyRow)
+ i += 1
+ }
+ resultProjection(joinedRow(aggregateResults, currentGroup))
+ }
+ }
+ }
+ }
+ }
+}
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..450c142c0baa4
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala
@@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution
+
+import org.apache.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
+
+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, shuffle = 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 = 150
+
+ 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..cff4887936ae1
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.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.sql.execution
+
+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
+ * 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 {
+
+ override def output =
+ if (join) child.output ++ generator.output else generator.output
+
+ override 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.eval(row)
+ if (outer && outputRows.isEmpty) {
+ outerProjection(row) :: Nil
+ } else {
+ outputRows.map(or => joinProjection(joinedRow(row, or)))
+ }
+ }
+ }
+ } else {
+ child.execute().mapPartitions(iter => iter.flatMap(row => generator.eval(row)))
+ }
+ }
+}
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..16806c620635f
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecutionException.scala
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution
+
+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..daa423cb8ea1a
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution
+
+import org.apache.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.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 =>
+
+ // 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 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 InMemoryColumnarTableScan(output, child) =>
+ InMemoryColumnarTableScan(output.map(_.newInstance), child)
+ 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/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala
new file mode 100644
index 0000000000000..d8e1b970c1d88
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.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.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]])
+ // 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[_]])
+ 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())
+ }
+}
+
+/**
+ * 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
+ }
+}
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..fe8bd5a508820
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.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.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._
+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] {
+ self: SQLContext#SparkPlanner =>
+
+ 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}")
+ // 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
+ }
+ 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 TakeOrdered extends Strategy {
+ def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
+ case logical.Limit(IntegerLiteral(limit), logical.Sort(order, child)) =>
+ execution.TakeOrdered(limit, order, planLater(child))(sparkContext) :: Nil
+ case _ => Nil
+ }
+ }
+
+ 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)
+ InsertIntoParquetTable(relation, planLater(child), overwrite=true)(sparkContext) :: Nil
+ case logical.InsertIntoTable(table: ParquetRelation, partition, child, overwrite) =>
+ InsertIntoParquetTable(table, planLater(child), overwrite)(sparkContext) :: Nil
+ case PhysicalOperation(projectList, filters, relation: 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
+ 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, 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.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) =>
+ 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 SparkLogicalPlan(existingPlan) => existingPlan :: Nil
+ case _ => Nil
+ }
+ }
+}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
new file mode 100644
index 0000000000000..ab2e62463764a
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/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.execution
+
+import scala.reflect.runtime.universe.TypeTag
+
+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.util.MutablePair
+
+
+case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends UnaryNode {
+ override def output = projectList.map(_.toAttribute)
+
+ 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 {
+ override def output = child.output
+
+ override def execute() = child.execute().mapPartitions { iter =>
+ iter.filter(condition.eval(_).asInstanceOf[Boolean])
+ }
+}
+
+case class Sample(fraction: Double, withReplacement: Boolean, seed: Int, child: SparkPlan)
+ extends UnaryNode {
+
+ override def output = child.output
+
+ // TODO: How to pick 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
+ override def output = children.head.output
+ override def execute() = sc.union(children.map(_.execute()))
+
+ override def otherCopyArgs = sc :: Nil
+}
+
+/**
+ * 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
+
+ override def output = child.output
+
+ override def executeCollect() = child.execute().map(_.copy()).take(limit)
+
+ 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))
+ }
+}
+
+/**
+ * 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
+
+ override 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|.
+ override 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)
+
+ override def execute() = attachTree(this, "sort") {
+ // TODO: Optimize sorting operation?
+ child.execute()
+ .mapPartitions(
+ iterator => iterator.map(_.copy()).toArray.sorted(ordering).iterator,
+ preservesPartitioning = true)
+ }
+
+ override 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 {
+ 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
new file mode 100644
index 0000000000000..40982f1fffbbf
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug.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.sql.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..c89dae9358bf7
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.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.execution
+
+import scala.collection.mutable.{ArrayBuffer, BitSet}
+
+import org.apache.spark.SparkContext
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Partitioning}
+
+sealed abstract class BuildSide
+case object BuildLeft extends BuildSide
+case object BuildRight extends BuildSide
+
+case class HashJoin(
+ leftKeys: Seq[Expression],
+ rightKeys: Seq[Expression],
+ buildSide: BuildSide,
+ left: SparkPlan,
+ right: SparkPlan) extends BinaryNode {
+
+ override def outputPartitioning: Partitioning = left.outputPartitioning
+
+ 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
+
+ @transient lazy val buildSideKeyGenerator = new Projection(buildKeys, buildPlan.output)
+ @transient lazy val streamSideKeyGenerator =
+ () => new MutableProjection(streamedKeys, streamedPlan.output)
+
+ def execute() = {
+
+ 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
+
+ // 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 {
+ 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 =
+ 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 ArrayBuffer[Row]
+ // TODO: Use Spark's BitSet.
+ val includedBroadcastTuples = new 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))) {
+ 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..e4a2dec33231d
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/package.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
+
+/**
+ * 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..4d7c86a3a4fc7
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala
@@ -0,0 +1,315 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.fs.permission.FsAction
+import org.apache.hadoop.mapreduce.Job
+
+import parquet.hadoop.util.ContextUtil
+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 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.{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.
+ *
+ * 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/parquet.file")
+ * }}}
+ *
+ * @param path The path to the Parquet file.
+ */
+private[sql] case class ParquetRelation(val path: String)
+ extends LeafNode with MultiInstanceRelation {
+ self: Product =>
+
+ /** Schema derived from ParquetFile */
+ def parquetSchema: MessageType =
+ ParquetTypesConverter
+ .readMetaData(new Path(path))
+ .getFileMetaData
+ .getSchema
+
+ /** Attributes */
+ override val output =
+ ParquetTypesConverter
+ .convertToAttributes(parquetSchema)
+
+ 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.path == path && p.output == output
+ case _ => false
+ }
+}
+
+private[sql] object ParquetRelation {
+
+ def enableLogForwarding() {
+ // 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.
+ 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
+ * 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 to be used.
+ * @return An empty ParquetRelation with inferred metadata.
+ */
+ def create(pathString: String,
+ child: LogicalPlan,
+ 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)
+ }
+
+ /**
+ * 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)
+ 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 = {
+ 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
+ }
+}
+
+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
+ 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")
+
+ ParquetRelation.enableLogForwarding()
+ 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 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 = {
+ 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)
+ 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)) {
+ ParquetFileReader.readFooter(conf, metadataPath)
+ } else {
+ // 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")
+ }
+ 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
new file mode 100644
index 0000000000000..d5846baa72ada
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala
@@ -0,0 +1,297 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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
+import java.text.SimpleDateFormat
+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.input.{FileInputFormat => NewFileInputFormat}
+import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat, FileOutputCommitter}
+
+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}
+
+/**
+ * Parquet table scan operator. Imports the file that backs the given
+ * [[ParquetRelation]] as a RDD[Row].
+ */
+case class ParquetTableScan(
+ // 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 {
+
+ 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)
+ 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)
+ // 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.newAPIHadoopRDD(conf, classOf[ParquetInputFormat[Row]], classOf[Void], classOf[Row])
+ .map(_._2)
+ }
+
+ override def otherCopyArgs = sc :: Nil
+
+ /**
+ * 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
+ }
+ }
+ }
+}
+
+/**
+ * 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(
+ relation: ParquetRelation,
+ child: SparkPlan,
+ overwrite: Boolean = false)(
+ @transient val sc: SparkContext)
+ extends UnaryNode with SparkHadoopMapReduceUtil {
+
+ /**
+ * Inserts all rows into the Parquet file.
+ */
+ override def execute() = {
+ // TODO: currently we do not check whether the "schema"s are compatible
+ // That means if one first creates a table and then INSERTs data with
+ // and incompatible schema the 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 = ContextUtil.getConfiguration(job)
+ conf.set(RowWriteSupport.PARQUET_ROW_SCHEMA, relation.parquetSchema.toString)
+
+ val fspath = new Path(relation.path)
+ val fs = fspath.getFileSystem(conf)
+
+ 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)
+
+ // We return the child RDD to allow chaining (alternatively, one could return nothing).
+ childRdd
+ }
+
+ 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]]
+ // instead of ``Tuple2``
+ private def saveAsHadoopFile(
+ rdd: RDD[Row],
+ path: String,
+ conf: Configuration) {
+ val job = new Job(conf)
+ val keyType = classOf[Void]
+ job.setOutputKeyClass(keyType)
+ job.setOutputValueClass(classOf[Row])
+ 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.
+ 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 = new AppendingParquetOutputFormat(taskIdOffset)
+ 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 = 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.
+ */
+ 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)
+ }
+}
+
+// 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
new file mode 100644
index 0000000000000..84b1b4609458b
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.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.parquet
+
+import org.apache.hadoop.conf.Configuration
+
+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.Logging
+import org.apache.spark.sql.catalyst.expressions.{Attribute, Row}
+import org.apache.spark.sql.catalyst.types._
+
+/**
+ * A `parquet.io.api.RecordMaterializer` for Rows.
+ *
+ *@param root The root group converter for the record.
+ */
+private[parquet] 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.
+ */
+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 file schema $fileSchema")
+ 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 requested schema $requested_schema")
+ ParquetRelation.enableLogForwarding()
+ new ReadContext(requested_schema, keyValueMetaData)
+ }
+}
+
+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.
+ */
+private[parquet] 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 = {
+ 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)
+ 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]())
+ }
+
+ 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) {
+ // 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()
+ }
+}
+
+private[parquet] 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 `Row` object.
+ *
+ * @param schema The corresponding Catalyst schema in the form of a list of attributes.
+ */
+private[parquet] 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.
+ */
+private[parquet] class CatalystPrimitiveConverter(
+ parent: CatalystGroupConverter,
+ fieldIndex: Int) extends PrimitiveConverter {
+ // TODO: consider refactoring these together with ParquetTypesConverter
+ override def addBinary(value: Binary): Unit =
+ parent.getCurrentRecord.update(fieldIndex, value.getBytes)
+
+ 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.
+ */
+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
new file mode 100644
index 0000000000000..728e3dd1dc02b
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.hadoop.fs.Path
+import org.apache.hadoop.mapreduce.Job
+
+import parquet.hadoop.ParquetWriter
+import parquet.hadoop.util.ContextUtil
+import parquet.schema.{MessageType, MessageTypeParser}
+
+import org.apache.spark.sql.catalyst.expressions.GenericRow
+import org.apache.spark.util.Utils
+
+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 testDir = Utils.createTempDir()
+
+ lazy val testData = new ParquetRelation(testDir.toURI.toString)
+
+ def writeFile() = {
+ 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)
+
+ 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..f2389f8f0591e
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.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.test
+
+import org.apache.spark.{SparkConf, SparkContext}
+import org.apache.spark.sql.SQLContext
+
+/** 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..dffd15a61838b
--- /dev/null
+++ b/sql/core/src/test/resources/log4j.properties
@@ -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.
+#
+
+# 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 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/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala
new file mode 100644
index 0000000000000..7c6a642278226
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.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
+
+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")
+ }
+ }
+
+ 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
new file mode 100644
index 0000000000000..be0f4a4c73b36
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.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
+
+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.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).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) ::
+ (1,1,1,2) ::
+ (1,2,1,1) ::
+ (1,2,1,2) :: Nil
+ )
+ }
+
+ test("inner join, no matches") {
+ 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)
+ }
+
+ test("big inner join, 4 matches per row") {
+ val bigData = testData.unionAll(testData).unionAll(testData).unionAll(testData)
+ val bigDataX = bigData.as('x)
+ val bigDataY = bigData.as('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).as('left)
+ val right = upperCaseData.where('N >= 3).as('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)
+ }
+}
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..d719ceb827691
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.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.sql
+
+import org.scalatest.FunSuite
+
+import org.apache.spark.sql.catalyst.plans._
+import org.apache.spark.sql.catalyst.util._
+
+class QueryTest extends FunSuite {
+ /**
+ * Runs the plan and makes sure the answer matches the expected result.
+ * @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 = {
+ 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..4c4fd6dbbedb4
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.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
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.test._
+
+/* Implicits */
+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"),
+ 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("cartesian 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) 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)
+ }
+
+ 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)
+ }
+
+ 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")))
+ }
+}
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..65eae3357a21e
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 java.sql.Timestamp
+
+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,
+ timestampField: Timestamp,
+ 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), new Timestamp(12345), 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
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..0bb13cf442b59
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala
@@ -0,0 +1,68 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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")
+}
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/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
new file mode 100644
index 0000000000000..1d3608ed2d9ab
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala
@@ -0,0 +1,182 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.ByteBuffer
+
+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 DEFAULT_BUFFER_SIZE = 512
+
+ test("defaultSize") {
+ val checks = Map(
+ INT -> 4, SHORT -> 2, LONG -> 8, BYTE -> 1, DOUBLE -> 8, FLOAT -> 4,
+ BOOLEAN -> 1, STRING -> 8, BINARY -> 16, GENERIC -> 16)
+
+ checks.foreach { case (columnType, expectedSize) =>
+ expectResult(expectedSize, s"Wrong defaultSize for $columnType") {
+ columnType.defaultSize
+ }
+ }
+ }
+
+ test("actualSize") {
+ 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)
+ }
+
+ testNativeColumnType[BooleanType.type](
+ BOOLEAN,
+ (buffer: ByteBuffer, v: Boolean) => {
+ buffer.put((if (v) 1 else 0).toByte)
+ },
+ (buffer: ByteBuffer) => {
+ buffer.get() == 1
+ })
+
+ testNativeColumnType[IntegerType.type](INT, _.putInt(_), _.getInt)
+
+ testNativeColumnType[ShortType.type](SHORT, _.putShort(_), _.getShort)
+
+ testNativeColumnType[LongType.type](LONG, _.putLong(_), _.getLong)
+
+ 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)
+ new String(bytes)
+ })
+
+ testColumnType[BinaryType.type, Array[Byte]](
+ BINARY,
+ (buffer: ByteBuffer, bytes: Array[Byte]) => {
+ buffer.putInt(bytes.length).put(bytes)
+ },
+ (buffer: ByteBuffer) => {
+ val length = buffer.getInt()
+ val bytes = new Array[Byte](length)
+ buffer.get(bytes, 0, length)
+ 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)
+
+ 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)
+
+ 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 testColumnType[T <: DataType, JvmType](
+ columnType: ColumnType[T, JvmType],
+ putter: (ByteBuffer, JvmType) => Unit,
+ getter: (ByteBuffer) => JvmType) {
+
+ val buffer = ByteBuffer.allocate(DEFAULT_BUFFER_SIZE)
+ val seq = (0 until 4).map(_ => makeRandomValue(columnType))
+
+ test(s"$columnType.extract") {
+ buffer.rewind()
+ seq.foreach(putter(buffer, _))
+
+ buffer.rewind()
+ seq.foreach { expected =>
+ assert(
+ expected === columnType.extract(buffer),
+ "Extracted value didn't equal to the original one")
+ }
+ }
+
+ test(s"$columnType.append") {
+ buffer.rewind()
+ seq.foreach(columnType.append(_, buffer))
+
+ buffer.rewind()
+ 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
new file mode 100644
index 0000000000000..2ed4cf2170f9d
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.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.columnar
+
+import org.apache.spark.sql.{QueryTest, TestData}
+import org.apache.spark.sql.execution.SparkLogicalPlan
+import org.apache.spark.sql.test.TestSQLContext
+
+class ColumnarQuerySuite extends QueryTest {
+ 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)
+ }
+
+ 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/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
new file mode 100644
index 0000000000000..4a21eb6201a69
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.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.columnar
+
+import java.nio.ByteBuffer
+
+import org.scalatest.FunSuite
+
+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 ColumnarTestUtils._
+
+ 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("$")
+ val nullRow = makeNullRow(1)
+
+ test(s"Nullable $typeName column accessor: empty column") {
+ val builder = TestNullableColumnBuilder(columnType)
+ val accessor = TestNullableColumnAccessor(builder.build(), columnType)
+ assert(!accessor.hasNext)
+ }
+
+ test(s"Nullable $typeName column accessor: access null values") {
+ val builder = TestNullableColumnBuilder(columnType)
+ val randomRow = makeRandomRow(columnType)
+
+ (0 until 4).foreach { _ =>
+ builder.appendFrom(randomRow, 0)
+ builder.appendFrom(nullRow, 0)
+ }
+
+ val accessor = TestNullableColumnAccessor(builder.build(), columnType)
+ val row = new GenericMutableRow(1)
+
+ (0 until 4).foreach { _ =>
+ accessor.extractTo(row, 0)
+ assert(row(0) === randomRow(0))
+
+ accessor.extractTo(row, 0)
+ 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
new file mode 100644
index 0000000000000..d9d1e1bfddb75
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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._
+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 ColumnarTestUtils._
+
+ Seq(INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, BINARY, GENERIC).foreach {
+ testNullableColumnBuilder(_)
+ }
+
+ def testNullableColumnBuilder[T <: DataType, JvmType](columnType: ColumnType[T, JvmType]) {
+ val typeName = columnType.getClass.getSimpleName.stripSuffix("$")
+
+ test(s"$typeName column builder: empty column") {
+ val columnBuilder = TestNullableColumnBuilder(columnType)
+ val buffer = columnBuilder.build()
+
+ 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") {
+ val columnBuilder = TestNullableColumnBuilder(columnType)
+ val randomRow = makeRandomRow(columnType)
+
+ (0 until 4).foreach { _ =>
+ columnBuilder.appendFrom(randomRow, 0)
+ }
+
+ val buffer = columnBuilder.build()
+
+ expectResult(columnType.typeId, "Wrong column type ID")(buffer.getInt())
+ expectResult(0, "Wrong null count")(buffer.getInt())
+ }
+
+ test(s"$typeName column builder: null values") {
+ val columnBuilder = TestNullableColumnBuilder(columnType)
+ val randomRow = makeRandomRow(columnType)
+ val nullRow = makeNullRow(1)
+
+ (0 until 4).foreach { _ =>
+ columnBuilder.appendFrom(randomRow, 0)
+ columnBuilder.appendFrom(nullRow, 0)
+ }
+
+ val buffer = columnBuilder.build()
+
+ 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(expectResult(_, "Wrong null position")(buffer.getInt()))
+
+ // 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 === 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/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
new file mode 100644
index 0000000000000..eab27987e08ea
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.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.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._
+
+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
+ }
+
+ 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, 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)
+ }
+ }
+
+ test(s"$DictionaryEncoding with $typeName: empty") {
+ skeleton(0, Seq.empty)
+ }
+
+ test(s"$DictionaryEncoding with $typeName: simple case") {
+ skeleton(2, Seq(0, 1, 0, 1))
+ }
+
+ 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
new file mode 100644
index 0000000000000..89f9b60a4397b
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.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.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("$")
+
+ def skeleton(uniqueValueCount: Int, inputRuns: Seq[(Int, Int)]) {
+ // -------------
+ // Tests encoder
+ // -------------
+
+ val builder = TestCompressibleColumnBuilder(columnStats, columnType, RunLengthEncoding)
+ 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)
+
+ // 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 + compressedSize, "Wrong buffer capacity")(buffer.capacity)
+
+ // Skips column header
+ buffer.position(headerSize)
+ expectResult(RunLengthEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt())
+
+ inputRuns.foreach { case (index, run) =>
+ expectResult(values(index), "Wrong column element value")(columnType.extract(buffer))
+ expectResult(run, "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 = RunLengthEncoding.decoder(buffer, columnType)
+
+ inputSeq.foreach { i =>
+ expectResult(values(i), "Wrong decoded value")(decoder.next())
+ }
+
+ assert(!decoder.hasNext)
+ }
+
+ test(s"$RunLengthEncoding with $typeName: empty column") {
+ skeleton(0, Seq.empty)
+ }
+
+ test(s"$RunLengthEncoding with $typeName: simple case") {
+ skeleton(2, Seq(0 -> 2, 1 ->2))
+ }
+
+ test(s"$RunLengthEncoding with $typeName: run length == 1") {
+ skeleton(2, Seq(0 -> 1, 1 ->1))
+ }
+
+ 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
new file mode 100644
index 0000000000000..81bf5e99d19b9
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.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.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[T]) = true
+}
+
+object TestCompressibleColumnBuilder {
+ def apply[T <: NativeType](
+ columnStats: NativeColumnStats[T],
+ columnType: NativeColumnType[T],
+ scheme: CompressionScheme) = {
+
+ val builder = new TestCompressibleColumnBuilder(columnStats, columnType, Seq(scheme))
+ builder.initialize(0)
+ builder
+ }
+}
+
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala
new file mode 100644
index 0000000000000..658ff0927aa85
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution
+
+import org.scalatest.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.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
+ 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/execution/TgfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala
new file mode 100644
index 0000000000000..e55648b8ed15a
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.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.execution
+
+import org.apache.spark.sql.QueryTest
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans._
+
+/* Implicit conversions */
+import org.apache.spark.sql.test.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 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")),
+ 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..fc68d6c5620d3
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala
@@ -0,0 +1,226 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.hadoop.fs.{Path, FileSystem}
+import org.apache.hadoop.mapreduce.Job
+
+import parquet.hadoop.ParquetFileWriter
+import parquet.schema.MessageTypeParser
+import parquet.hadoop.util.ContextUtil
+
+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() {
+ Utils.deleteRecursively(ParquetTestData.testDir)
+ // here we should also unregister the table??
+ }
+
+ test("self-join parquet files") {
+ 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
+ // 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
+ }
+
+ 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 = parquetFile(ParquetTestData.testDir.toString).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)
+ }
+
+ 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()
+ }
+
+ /**
+ * 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(
+ TestSQLContext,
+ parquet.ParquetRelation.createEmpty(path, attributes, sparkContext.hadoopConfiguration))
+ }
+}
+
diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml
new file mode 100644
index 0000000000000..a662da76ce25a
--- /dev/null
+++ b/sql/hive/pom.xml
@@ -0,0 +1,120 @@
+
+
+
+
+ 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/
+
+
+ yarn-alpha
+
+
+ org.apache.avro
+ avro
+
+
+
+
+
+
+
+ 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.codehaus.jackson
+ jackson-mapper-asl
+
+
+ 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
+
+
+
+
+ org.apache.maven.plugins
+ maven-dependency-plugin
+ 2.4
+
+
+ copy-dependencies
+ package
+
+ copy-dependencies
+
+
+
+ ${basedir}/../../lib_managed/jars
+ false
+ false
+ true
+ org.datanucleus
+
+
+
+
+
+
+
diff --git a/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala
new file mode 100644
index 0000000000000..7219c030cb0f0
--- /dev/null
+++ b/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala
@@ -0,0 +1,194 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 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.mapred._
+import org.apache.hadoop.io.Writable
+
+/**
+ * Internal helper class that saves an RDD using a Hive OutputFormat.
+ * It is based on [[SparkHadoopWriter]].
+ */
+protected[spark]
+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..353458432b210
--- /dev/null
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
@@ -0,0 +1,307 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.language.implicitConversions
+
+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.Driver
+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.{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._
+
+/* Implicit conversions */
+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 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 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
+ // generates the RDD lineage for DML queries, but do not perform any execution.
+ result.queryExecution.toRdd
+ 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 {
+ 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 protected[sql] 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 protected[sql] lazy val analyzer =
+ new Analyzer(catalog, HiveFunctionRegistry, caseSensitive = false)
+
+ /**
+ * 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
+ }
+
+ 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)
+
+ 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) {
+ driver.destroy()
+ throw new QueryExecutionException(response.getErrorMessage)
+ }
+ driver.setMaxRows(maxRows)
+ driver.getResults(results)
+ driver.destroy()
+ results
+ case _ =>
+ 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(
+ TakeOrdered,
+ ParquetOperations,
+ HiveTableScans,
+ DataSinks,
+ Scripts,
+ PartialAggregation,
+ HashJoin,
+ BasicOperators,
+ CartesianProduct,
+ BroadcastNestedLoopJoin
+ )
+ }
+
+ @transient
+ 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. */
+ 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)))
+
+ 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
+ }
+
+ override def simpleString: String =
+ logical match {
+ case _: NativeCommand => ""
+ case _ => executedPlan.toString
+ }
+ }
+}
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..fc053c56c052d
--- /dev/null
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
@@ -0,0 +1,257 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 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 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 {
+ 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(
+ EliminateAnalysisOperators(
+ lookupRelation(Some(databaseName), tableName, None)),
+ 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, output) if input.dataType != output.dataType =>
+ Alias(Cast(input, output.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 functionality is desired mix in the in-memory [[OverrideCatalog]].
+ */
+ 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 = ???
+
+ override def unregisterAllTables() = {}
+}
+
+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..4dac25b3f60e4
--- /dev/null
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
@@ -0,0 +1,968 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.hadoop.hive.ql.lib.Node
+import org.apache.hadoop.hive.ql.parse._
+import org.apache.hadoop.hive.ql.plan.PlanUtils
+
+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
+ * 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 statements 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 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`.
+ */
+ 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_DECIMAL", Nil) => DecimalType
+ case Token("TOK_BIGINT", Nil) => LongType
+ case Token("TOK_INT", 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) => 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) =>
+ 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(Limit(_, 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) =>
+ Limit(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)
+ case Token("TOK_FUNCTION", Token("TOK_TIMESTAMP", Nil) :: arg :: Nil) =>
+ Cast(nodeToExpr(arg), TimestampType)
+
+ /* 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) => 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) =>
+ 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..3ca1d93c11fa9
--- /dev/null
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.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.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._
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution._
+
+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 _ => Nil
+ }
+ }
+
+ /**
+ * Retrieves data using a HiveTableScan. Partition pruning predicates are also detected and
+ * applied.
+ */
+ object HiveTableScans extends Strategy {
+ def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
+ 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
+ val (pruningPredicates, otherPredicates) = predicates.partition {
+ _.references.map(_.exprId).subsetOf(partitionKeyIds)
+
+ }
+
+ pruneFilterProject(
+ projectList,
+ otherPredicates,
+ HiveTableScan(_, relation, pruningPredicates.reduceLeftOption(And))(hiveContext)) :: Nil
+ case _ =>
+ Nil
+ }
+ }
+}
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..2610100043659
--- /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.hive
+
+import java.io.{BufferedReader, InputStreamReader}
+
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.execution._
+
+/* Implicit conversions */
+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..0da5eb754cb3f
--- /dev/null
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
@@ -0,0 +1,240 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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.io.Writable
+import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf}
+
+import org.apache.spark.SerializableWritable
+import org.apache.spark.broadcast.Broadcast
+import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD}
+
+/**
+ * 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..2fea9702954d7
--- /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.hive
+
+import java.io.File
+import java.util.{Set => JavaSet}
+
+import scala.collection.mutable
+import scala.language.implicitConversions
+
+import org.apache.hadoop.hive.ql.exec.FunctionRegistry
+import org.apache.hadoop.hive.ql.io.avro.{AvroContainerInputFormat, AvroContainerOutputFormat}
+import org.apache.hadoop.hive.ql.metadata.Table
+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.{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._
+
+/* Implicit conversions */
+import scala.collection.JavaConversions._
+
+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
+ * test cases 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
+
+ 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
+ }
+
+ /**
+ * 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)*)
+
+ protected[hive] implicit class SqlCmd(sql: String) {
+ def cmd = () => new HiveQLQueryExecution(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)
+ }
+
+ catalog.unregisterAllTables()
+
+ 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/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/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..821fb22112f87
--- /dev/null
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala
@@ -0,0 +1,355 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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.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.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.io.Writable
+import org.apache.hadoop.mapred._
+
+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}
+
+/* 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).eval(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.eval(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: 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
+ }
+
+ 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..f9b437d435eba
--- /dev/null
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
@@ -0,0 +1,472 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF 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 scala.collection.mutable.ArrayBuffer
+
+import org.apache.hadoop.hive.common.`type`.HiveDecimal
+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.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 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
+import org.apache.spark.sql.catalyst.types._
+
+/* Implicit conversions */
+import scala.collection.JavaConversions._
+
+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 org.apache.spark.sql.hive.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 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)
+ }
+
+ // 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 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.eval(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
+
+ @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)
+
+ 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 eval(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 eval(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/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 0000000000000..f29def6f8c97e
Binary files /dev/null and b/sql/hive/src/test/resources/data/files/TestSerDe.jar differ
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 0000000000000..ab1f2ef25f1ad
Binary files /dev/null and b/sql/hive/src/test/resources/data/files/archive_corrupt.rc differ
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 0000000000000..c27d5c09b1da8
Binary files /dev/null and b/sql/hive/src/test/resources/data/files/complex.seq differ
diff --git a/sql/hive/src/test/resources/data/files/covar_tab.txt b/sql/hive/src/test/resources/data/files/covar_tab.txt
new file mode 100644
index 0000000000000..204a485fba2f7
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/covar_tab.txt
@@ -0,0 +1,6 @@
+1 15
+2 3
+3 7 12
+4 4 14
+5 8 17
+6 2 11
diff --git a/sql/hive/src/test/resources/data/files/create_nested_type.txt b/sql/hive/src/test/resources/data/files/create_nested_type.txt
new file mode 100644
index 0000000000000..b3b65fb2a4687
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/create_nested_type.txt
@@ -0,0 +1,4 @@
+a0b00b01c001C001c002C002c011\Nc012C012d01d011d012d02d021d022
+a1b10c001C001c002C002d01d011d012d02\N
+a2c001\Nc002C002c011C011c012C012d01\Nd012d02d021d022
+a3\N\N\N
diff --git a/sql/hive/src/test/resources/data/files/csv.txt b/sql/hive/src/test/resources/data/files/csv.txt
new file mode 100644
index 0000000000000..be8c0adb730c8
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/csv.txt
@@ -0,0 +1,18 @@
+why hello there,42,3,100,1412341,true,42.43,85.23423424,alpha:beta:gamma,Earth#42:Control#86:Bob#31,17:true:Abe Linkedin,BLUE,72,,bc
+another record,98,4,101,9999999,false,99.89,0.00000009,beta,Earth#101,1134:false:wazzup,RED,\N,,ef
+third record,45,5,102,999999999,true,89.99,0.00000000000009,alpha:gamma,Earth#237:Bob#723,102:false:BNL,GREEN,\N,,hi
+\N,42,3,100,1412341,true,42.43,85.23423424,alpha:beta:gamma,Earth#42:Control#86:Bob#31,17:true:Abe Linkedin,BLUE,72,,bc
+string,\N,3,100,1412341,true,42.43,85.23423424,alpha:beta:gamma,Earth#42:Control#86:Bob#31,17:true:Abe Linkedin,BLUE,72,,bc
+string,42,\N,100,1412341,true,42.43,85.23423424,alpha:beta:gamma,Earth#42:Control#86:Bob#31,17:true:Abe Linkedin,BLUE,72,,bc
+string,42,3,\N,1412341,true,42.43,85.23423424,alpha:beta:gamma,Earth#42:Control#86:Bob#31,17:true:Abe Linkedin,BLUE,72,,bc
+string,42,3,100,\N,true,42.43,85.23423424,alpha:beta:gamma,Earth#42:Control#86:Bob#31,17:true:Abe Linkedin,BLUE,72,,bc
+string,42,3,100,1412341,\N,42.43,85.23423424,alpha:beta:gamma,Earth#42:Control#86:Bob#31,17:true:Abe Linkedin,BLUE,72,,bc
+string,42,3,100,1412341,true,\N,85.23423424,alpha:beta:gamma,Earth#42:Control#86:Bob#31,17:true:Abe Linkedin,BLUE,72,,bc
+string,42,3,100,1412341,true,42.43,\N,alpha:beta:gamma,Earth#42:Control#86:Bob#31,17:true:Abe Linkedin,BLUE,72,,bc
+string,42,3,100,1412341,true,42.43,85.23423424,\N,Earth#42:Control#86:Bob#31,17:true:Abe Linkedin,BLUE,72,,bc
+string,42,3,100,1412341,true,42.43,85.23423424,alpha:beta:gamma,\N,17:true:Abe Linkedin,BLUE,72,,bc
+string,42,3,100,1412341,true,42.43,85.23423424,alpha:beta:gamma,Earth#42:Control#86:Bob#31,\N,BLUE,72,,bc
+string,42,3,100,1412341,true,42.43,85.23423424,alpha:beta:gamma,Earth#42:Control#86:Bob#31,17:true:Abe Linkedin,\N,72,,bc
+string,42,3,100,1412341,true,42.43,85.23423424,alpha:beta:gamma,Earth#42:Control#86:Bob#31,17:true:Abe Linkedin,BLUE,\N,,bc
+string,42,3,100,1412341,true,42.43,85.23423424,alpha:beta:gamma,Earth#42:Control#86:Bob#31,17:true:Abe Linkedin,BLUE,72,\N,bc
+string,42,3,100,1412341,true,42.43,85.23423424,alpha:beta:gamma,Earth#42:Control#86:Bob#31,17:true:Abe Linkedin,BLUE,72,,\N
diff --git a/sql/hive/src/test/resources/data/files/datatypes.txt b/sql/hive/src/test/resources/data/files/datatypes.txt
new file mode 100644
index 0000000000000..10daa1b11c26b
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/datatypes.txt
@@ -0,0 +1,3 @@
+\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N\N
+-1false-1.1\N\N\N-1-1-1.0-1\N\N\N\N\N\N\N
+1true1.11121x2ykva92.2111.01abcd1111213142212212x1abcd22012-04-22 09:00:00.123456789123456789.0123456YWJjZA==2013-01-01abc123
diff --git a/sql/hive/src/test/resources/data/files/dim-data.txt b/sql/hive/src/test/resources/data/files/dim-data.txt
new file mode 100644
index 0000000000000..c90cc480b19d0
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/dim-data.txt
@@ -0,0 +1,4 @@
+11
+22
+33
+44
diff --git a/sql/hive/src/test/resources/data/files/doctors.avro b/sql/hive/src/test/resources/data/files/doctors.avro
new file mode 100644
index 0000000000000..5840e5f08a39a
Binary files /dev/null and b/sql/hive/src/test/resources/data/files/doctors.avro differ
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 0000000000000..58a028ce19e6a
Binary files /dev/null and b/sql/hive/src/test/resources/data/files/episodes.avro differ
diff --git a/sql/hive/src/test/resources/data/files/escapetest.txt b/sql/hive/src/test/resources/data/files/escapetest.txt
new file mode 100644
index 0000000000000..3ff5fb9b14a03
Binary files /dev/null and b/sql/hive/src/test/resources/data/files/escapetest.txt differ
diff --git a/sql/hive/src/test/resources/data/files/ext_test/test.dat b/sql/hive/src/test/resources/data/files/ext_test/test.dat
new file mode 100644
index 0000000000000..cf0389a5afee1
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/ext_test/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/fact-data.txt b/sql/hive/src/test/resources/data/files/fact-data.txt
new file mode 100644
index 0000000000000..778ce4052f7e0
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/fact-data.txt
@@ -0,0 +1,10 @@
+1212
+111212
+212212
+313212
+414212
+515234
+616234
+717234
+818234
+919234
diff --git a/sql/hive/src/test/resources/data/files/flights_join.txt b/sql/hive/src/test/resources/data/files/flights_join.txt
new file mode 100644
index 0000000000000..664a06e9f978e
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/flights_join.txt
@@ -0,0 +1,20 @@
+BaltimoreNew York2010-10-20-30.01064
+BaltimoreNew York2010-10-2123.01142
+BaltimoreNew York2010-10-226.01599
+ChicagoNew York2010-10-2342.0361
+ChicagoNew York2010-10-2424.0897
+ChicagoNew York2010-10-2515.01531
+ChicagoNew York2010-10-26-6.01610
+ChicagoNew York2010-10-27-2.03198
+BaltimoreNew York2010-10-2817.01064
+BaltimoreNew York2010-10-29105.01142
+BaltimoreNew York2000-11-20-30.01064
+BaltimoreNew York2000-11-2123.01142
+BaltimoreNew York2000-11-226.01599
+ChicagoNew York2000-11-2342.0361
+ChicagoNew York2000-11-2424.0897
+ChicagoNew York2000-11-2515.01531
+ChicagoNew York2000-11-26-6.01610
+ChicagoNew York2000-11-27-2.03198
+BaltimoreNew York2000-11-2817.01064
+BaltimoreNew York2000-11-28105.01142
diff --git a/sql/hive/src/test/resources/data/files/flights_tiny.txt b/sql/hive/src/test/resources/data/files/flights_tiny.txt
new file mode 100644
index 0000000000000..4841c920ec3e8
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/flights_tiny.txt
@@ -0,0 +1,137 @@
+BaltimoreNew York20101020-30.01064
+BaltimoreNew York2010102023.01142
+BaltimoreNew York201010206.01599
+ChicagoNew York2010102042.0361
+ChicagoNew York2010102024.0897
+ChicagoNew York2010102015.01531
+ChicagoNew York20101020-6.01610
+ChicagoNew York20101020-2.03198
+BaltimoreNew York2010102117.01064
+BaltimoreNew York20101021105.01142
+BaltimoreNew York2010102128.01599
+ChicagoNew York20101021142.0361
+ChicagoNew York2010102177.0897
+ChicagoNew York2010102153.01531
+ChicagoNew York20101021-5.01610
+ChicagoNew York2010102151.03198
+BaltimoreNew York20101022-12.01064
+BaltimoreNew York2010102254.01142
+BaltimoreNew York2010102218.01599
+ChicagoNew York201010222.0361
+ChicagoNew York2010102224.0897
+ChicagoNew York2010102216.01531
+ChicagoNew York20101022-6.01610
+ChicagoNew York20101022-11.03198
+BaltimoreNew York2010102318.0272
+BaltimoreNew York20101023-10.01805
+BaltimoreNew York201010236.03171
+ChicagoNew York201010233.0384
+ChicagoNew York2010102332.0426
+ChicagoNew York201010231.0650
+ChicagoNew York2010102311.03085
+BaltimoreNew York2010102412.01599
+BaltimoreNew York2010102420.02571
+ChicagoNew York2010102410.0361
+ChicagoNew York20101024113.0897
+ChicagoNew York20101024-5.01531
+ChicagoNew York20101024-17.01610
+ChicagoNew York20101024-3.03198
+BaltimoreNew York20101025-25.01064
+BaltimoreNew York2010102592.01142
+BaltimoreNew York20101025106.01599
+ChicagoNew York2010102531.0361
+ChicagoNew York20101025-1.0897
+ChicagoNew York2010102543.01531
+ChicagoNew York201010256.01610
+ChicagoNew York20101025-16.03198
+BaltimoreNew York20101026-22.01064
+BaltimoreNew York20101026123.01142
+BaltimoreNew York2010102690.01599
+ChicagoNew York2010102612.0361
+ChicagoNew York201010260.0897
+ChicagoNew York2010102629.01531
+ChicagoNew York20101026-17.01610
+ChicagoNew York201010266.03198
+BaltimoreNew York20101027-18.01064
+BaltimoreNew York2010102749.01142
+BaltimoreNew York2010102792.01599
+ChicagoNew York20101027148.0361
+ChicagoNew York20101027-11.0897
+ChicagoNew York2010102770.01531
+ChicagoNew York201010278.01610
+ChicagoNew York2010102721.03198
+BaltimoreNew York20101028-4.01064
+BaltimoreNew York20101028-14.01142
+BaltimoreNew York20101028-14.01599
+ChicagoNew York201010282.0361
+ChicagoNew York201010282.0897
+ChicagoNew York20101028-11.01531
+ChicagoNew York201010283.01610
+ChicagoNew York20101028-18.03198
+BaltimoreNew York20101029-24.01064
+BaltimoreNew York2010102921.01142
+BaltimoreNew York20101029-2.01599
+ChicagoNew York20101029-12.0361
+ChicagoNew York20101029-11.0897
+ChicagoNew York2010102915.01531
+ChicagoNew York20101029-18.01610
+ChicagoNew York20101029-4.03198
+BaltimoreNew York2010103014.0272
+BaltimoreNew York20101030-1.01805
+BaltimoreNew York201010305.03171
+ChicagoNew York20101030-6.0384
+ChicagoNew York20101030-10.0426
+ChicagoNew York20101030-5.0650
+ChicagoNew York20101030-5.03085
+BaltimoreNew York20101031-1.01599
+BaltimoreNew York20101031-14.02571
+ChicagoNew York20101031-25.0361
+ChicagoNew York20101031-18.0897
+ChicagoNew York20101031-4.01531
+ChicagoNew York20101031-22.01610
+ChicagoNew York20101031-15.03198
+ClevelandNew York20101030-23.02018
+ClevelandNew York20101030-12.02932
+ClevelandNew York20101029-4.02630
+ClevelandNew York20101029-19.02646
+ClevelandNew York20101029-12.03014
+ClevelandNew York201010283.02630
+ClevelandNew York20101028-6.02646
+ClevelandNew York201010281.03014
+ClevelandNew York2010102716.02630
+ClevelandNew York2010102727.03014
+ClevelandNew York201010264.02630
+ClevelandNew York20101026-27.02646
+ClevelandNew York20101026-11.02662
+ClevelandNew York2010102613.03014
+ClevelandNew York20101025-4.02630
+ClevelandNew York2010102581.02646
+ClevelandNew York2010102542.03014
+ClevelandNew York201010245.02254
+ClevelandNew York20101024-11.02630
+ClevelandNew York20101024-20.02646
+ClevelandNew York20101024-9.03014
+ClevelandNew York20101023-21.02932
+ClevelandNew York201010221.02630
+ClevelandNew York20101022-25.02646
+ClevelandNew York20101022-3.03014
+ClevelandNew York201010213.02630
+ClevelandNew York2010102129.02646
+ClevelandNew York2010102172.03014
+ClevelandNew York20101020-8.02630
+ClevelandNew York20101020-15.03014
+WashingtonNew York20101023-25.05832
+WashingtonNew York20101023-21.05904
+WashingtonNew York20101023-18.05917
+WashingtonNew York20101030-27.05904
+WashingtonNew York20101030-16.05917
+WashingtonNew York20101020-2.07291
+WashingtonNew York2010102122.07291
+WashingtonNew York20101023-16.07274
+WashingtonNew York20101024-26.07282
+WashingtonNew York201010259.07291
+WashingtonNew York201010264.07291
+WashingtonNew York2010102726.07291
+WashingtonNew York2010102845.07291
+WashingtonNew York201010291.07291
+WashingtonNew York20101031-18.07282
diff --git a/sql/hive/src/test/resources/data/files/flights_tiny.txt.1 b/sql/hive/src/test/resources/data/files/flights_tiny.txt.1
new file mode 100644
index 0000000000000..532848ad85ab7
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/flights_tiny.txt.1
@@ -0,0 +1,137 @@
+BaltimoreNew York2010-10-20-30.01064
+BaltimoreNew York2010-10-2023.01142
+BaltimoreNew York2010-10-206.01599
+ChicagoNew York2010-10-2042.0361
+ChicagoNew York2010-10-2024.0897
+ChicagoNew York2010-10-2015.01531
+ChicagoNew York2010-10-20-6.01610
+ChicagoNew York2010-10-20-2.03198
+BaltimoreNew York2010-10-2117.01064
+BaltimoreNew York2010-10-21105.01142
+BaltimoreNew York2010-10-2128.01599
+ChicagoNew York2010-10-21142.0361
+ChicagoNew York2010-10-2177.0897
+ChicagoNew York2010-10-2153.01531
+ChicagoNew York2010-10-21-5.01610
+ChicagoNew York2010-10-2151.03198
+BaltimoreNew York2010-10-22-12.01064
+BaltimoreNew York2010-10-2254.01142
+BaltimoreNew York2010-10-2218.01599
+ChicagoNew York2010-10-222.0361
+ChicagoNew York2010-10-2224.0897
+ChicagoNew York2010-10-2216.01531
+ChicagoNew York2010-10-22-6.01610
+ChicagoNew York2010-10-22-11.03198
+BaltimoreNew York2010-10-2318.0272
+BaltimoreNew York2010-10-23-10.01805
+BaltimoreNew York2010-10-236.03171
+ChicagoNew York2010-10-233.0384
+ChicagoNew York2010-10-2332.0426
+ChicagoNew York2010-10-231.0650
+ChicagoNew York2010-10-2311.03085
+BaltimoreNew York2010-10-2412.01599
+BaltimoreNew York2010-10-2420.02571
+ChicagoNew York2010-10-2410.0361
+ChicagoNew York2010-10-24113.0897
+ChicagoNew York2010-10-24-5.01531
+ChicagoNew York2010-10-24-17.01610
+ChicagoNew York2010-10-24-3.03198
+BaltimoreNew York2010-10-25-25.01064
+BaltimoreNew York2010-10-2592.01142
+BaltimoreNew York2010-10-25106.01599
+ChicagoNew York2010-10-2531.0361
+ChicagoNew York2010-10-25-1.0897
+ChicagoNew York2010-10-2543.01531
+ChicagoNew York2010-10-256.01610
+ChicagoNew York2010-10-25-16.03198
+BaltimoreNew York2010-10-26-22.01064
+BaltimoreNew York2010-10-26123.01142
+BaltimoreNew York2010-10-2690.01599
+ChicagoNew York2010-10-2612.0361
+ChicagoNew York2010-10-260.0897
+ChicagoNew York2010-10-2629.01531
+ChicagoNew York2010-10-26-17.01610
+ChicagoNew York2010-10-266.03198
+BaltimoreNew York2010-10-27-18.01064
+BaltimoreNew York2010-10-2749.01142
+BaltimoreNew York2010-10-2792.01599
+ChicagoNew York2010-10-27148.0361
+ChicagoNew York2010-10-27-11.0897
+ChicagoNew York2010-10-2770.01531
+ChicagoNew York2010-10-278.01610
+ChicagoNew York2010-10-2721.03198
+BaltimoreNew York2010-10-28-4.01064
+BaltimoreNew York2010-10-28-14.01142
+BaltimoreNew York2010-10-28-14.01599
+ChicagoNew York2010-10-282.0361
+ChicagoNew York2010-10-282.0897
+ChicagoNew York2010-10-28-11.01531
+ChicagoNew York2010-10-283.01610
+ChicagoNew York2010-10-28-18.03198
+BaltimoreNew York2010-10-29-24.01064
+BaltimoreNew York2010-10-2921.01142
+BaltimoreNew York2010-10-29-2.01599
+ChicagoNew York2010-10-29-12.0361
+ChicagoNew York2010-10-29-11.0897
+ChicagoNew York2010-10-2915.01531
+ChicagoNew York2010-10-29-18.01610
+ChicagoNew York2010-10-29-4.03198
+BaltimoreNew York2010-10-3014.0272
+BaltimoreNew York2010-10-30-1.01805
+BaltimoreNew York2010-10-305.03171
+ChicagoNew York2010-10-30-6.0384
+ChicagoNew York2010-10-30-10.0426
+ChicagoNew York2010-10-30-5.0650
+ChicagoNew York2010-10-30-5.03085
+BaltimoreNew York2010-10-31-1.01599
+BaltimoreNew York2010-10-31-14.02571
+ChicagoNew York2010-10-31-25.0361
+ChicagoNew York2010-10-31-18.0897
+ChicagoNew York2010-10-31-4.01531
+ChicagoNew York2010-10-31-22.01610
+ChicagoNew York2010-10-31-15.03198
+ClevelandNew York2010-10-30-23.02018
+ClevelandNew York2010-10-30-12.02932
+ClevelandNew York2010-10-29-4.02630
+ClevelandNew York2010-10-29-19.02646
+ClevelandNew York2010-10-29-12.03014
+ClevelandNew York2010-10-283.02630
+ClevelandNew York2010-10-28-6.02646
+ClevelandNew York2010-10-281.03014
+ClevelandNew York2010-10-2716.02630
+ClevelandNew York2010-10-2727.03014
+ClevelandNew York2010-10-264.02630
+ClevelandNew York2010-10-26-27.02646
+ClevelandNew York2010-10-26-11.02662
+ClevelandNew York2010-10-2613.03014
+ClevelandNew York2010-10-25-4.02630
+ClevelandNew York2010-10-2581.02646
+ClevelandNew York2010-10-2542.03014
+ClevelandNew York2010-10-245.02254
+ClevelandNew York2010-10-24-11.02630
+ClevelandNew York2010-10-24-20.02646
+ClevelandNew York2010-10-24-9.03014
+ClevelandNew York2010-10-23-21.02932
+ClevelandNew York2010-10-221.02630
+ClevelandNew York2010-10-22-25.02646
+ClevelandNew York2010-10-22-3.03014
+ClevelandNew York2010-10-213.02630
+ClevelandNew York2010-10-2129.02646
+ClevelandNew York2010-10-2172.03014
+ClevelandNew York2010-10-20-8.02630
+ClevelandNew York2010-10-20-15.03014
+WashingtonNew York2010-10-23-25.05832
+WashingtonNew York2010-10-23-21.05904
+WashingtonNew York2010-10-23-18.05917
+WashingtonNew York2010-10-30-27.05904
+WashingtonNew York2010-10-30-16.05917
+WashingtonNew York2010-10-20-2.07291
+WashingtonNew York2010-10-2122.07291
+WashingtonNew York2010-10-23-16.07274
+WashingtonNew York2010-10-24-26.07282
+WashingtonNew York2010-10-259.07291
+WashingtonNew York2010-10-264.07291
+WashingtonNew York2010-10-2726.07291
+WashingtonNew York2010-10-2845.07291
+WashingtonNew York2010-10-291.07291
+WashingtonNew York2010-10-31-18.07282
diff --git a/sql/hive/src/test/resources/data/files/groupby_groupingid.txt b/sql/hive/src/test/resources/data/files/groupby_groupingid.txt
new file mode 100644
index 0000000000000..108d41d9b98fc
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/groupby_groupingid.txt
@@ -0,0 +1,6 @@
+1NULL
+11
+22
+33
+3NULL
+45
diff --git a/sql/hive/src/test/resources/data/files/grouping_sets.txt b/sql/hive/src/test/resources/data/files/grouping_sets.txt
new file mode 100644
index 0000000000000..8ba5e2d61051d
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/grouping_sets.txt
@@ -0,0 +1,6 @@
+8 1 1
+5 2 2
+1 1 3
+2 2 4
+2 3 5
+3 2 8
diff --git a/sql/hive/src/test/resources/data/files/grouping_sets1.txt b/sql/hive/src/test/resources/data/files/grouping_sets1.txt
new file mode 100644
index 0000000000000..3b6cd99163752
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/grouping_sets1.txt
@@ -0,0 +1,6 @@
+8 1 1
+5 1 2
+1 1 3
+2 2 4
+2 3 5
+3 2 8
diff --git a/sql/hive/src/test/resources/data/files/grouping_sets2.txt b/sql/hive/src/test/resources/data/files/grouping_sets2.txt
new file mode 100644
index 0000000000000..5b4e032fc5751
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/grouping_sets2.txt
@@ -0,0 +1,6 @@
+8 1 1
+1 2 2
+1 1 3
+2 2 4
+2 3 5
+2 2 8
diff --git a/sql/hive/src/test/resources/data/files/hive_626_bar.txt b/sql/hive/src/test/resources/data/files/hive_626_bar.txt
new file mode 100644
index 0000000000000..1ac3d25729c89
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/hive_626_bar.txt
@@ -0,0 +1 @@
+10,0,1,1,bar10,a,b,c,d
diff --git a/sql/hive/src/test/resources/data/files/hive_626_count.txt b/sql/hive/src/test/resources/data/files/hive_626_count.txt
new file mode 100644
index 0000000000000..d62e011bba9dc
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/hive_626_count.txt
@@ -0,0 +1 @@
+10,2
diff --git a/sql/hive/src/test/resources/data/files/hive_626_foo.txt b/sql/hive/src/test/resources/data/files/hive_626_foo.txt
new file mode 100644
index 0000000000000..8a63d3a1ed579
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/hive_626_foo.txt
@@ -0,0 +1 @@
+1,foo1,a,b,c,d
diff --git a/sql/hive/src/test/resources/data/files/in1.txt b/sql/hive/src/test/resources/data/files/in1.txt
new file mode 100644
index 0000000000000..1a5949633e65c
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/in1.txt
@@ -0,0 +1,3 @@
+35
+48
+100100
diff --git a/sql/hive/src/test/resources/data/files/in2.txt b/sql/hive/src/test/resources/data/files/in2.txt
new file mode 100644
index 0000000000000..71f7e0963e4f4
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/in2.txt
@@ -0,0 +1,3 @@
+135
+148
+200200
diff --git a/sql/hive/src/test/resources/data/files/in3.txt b/sql/hive/src/test/resources/data/files/in3.txt
new file mode 100644
index 0000000000000..333dd32ab27d5
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/in3.txt
@@ -0,0 +1,4 @@
+1235
+40
+48
+100100
diff --git a/sql/hive/src/test/resources/data/files/in4.txt b/sql/hive/src/test/resources/data/files/in4.txt
new file mode 100644
index 0000000000000..63cf2f6c145f8
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/in4.txt
@@ -0,0 +1,7 @@
+35236
+101000501
+100100103
+12802
+101005
+10100454
+12100757
diff --git a/sql/hive/src/test/resources/data/files/in5.txt b/sql/hive/src/test/resources/data/files/in5.txt
new file mode 100644
index 0000000000000..8da5dcfcd60a6
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/in5.txt
@@ -0,0 +1,19 @@
+51000566
+151001566
+201002066
+251002588
+301003066
+351003588
+401004066
+401004088
+501005088
+501005066
+501005088
+601004066
+601004066
+701004066
+701004066
+801004088
+801004088
+1005066
+66
diff --git a/sql/hive/src/test/resources/data/files/in6.txt b/sql/hive/src/test/resources/data/files/in6.txt
new file mode 100644
index 0000000000000..f96b7a40c5d94
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/in6.txt
@@ -0,0 +1,19 @@
+51000566
+101001066
+201002066
+251002566
+301003088
+351003588
+401004066
+401004088
+501005066
+501005088
+501005066
+601004066
+601004066
+701004088
+701004088
+801004066
+801004066
+1005066
+66
diff --git a/sql/hive/src/test/resources/data/files/in7.txt b/sql/hive/src/test/resources/data/files/in7.txt
new file mode 100644
index 0000000000000..424a2fee06987
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/in7.txt
@@ -0,0 +1,3 @@
+35 40
+48 32
+100100 40
diff --git a/sql/hive/src/test/resources/data/files/in8.txt b/sql/hive/src/test/resources/data/files/in8.txt
new file mode 100644
index 0000000000000..54b8a5bb75395
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/in8.txt
@@ -0,0 +1,6 @@
+
+10
+10
+35
+48
+100100
diff --git a/sql/hive/src/test/resources/data/files/in9.txt b/sql/hive/src/test/resources/data/files/in9.txt
new file mode 100644
index 0000000000000..b38f3c191ee57
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/in9.txt
@@ -0,0 +1,6 @@
+
+110
+110
+135
+148
+200200
diff --git a/sql/hive/src/test/resources/data/files/infer_const_type.txt b/sql/hive/src/test/resources/data/files/infer_const_type.txt
new file mode 100644
index 0000000000000..05c43c1264f79
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/infer_const_type.txt
@@ -0,0 +1,3 @@
+1273276712345-12345906-307.01234
+1263276712345-12345906-307.01234
+1263276712345-12345906-307.01.57
diff --git a/sql/hive/src/test/resources/data/files/int.txt b/sql/hive/src/test/resources/data/files/int.txt
new file mode 100644
index 0000000000000..9553ed297f77e
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/int.txt
@@ -0,0 +1,12 @@
+4
+252
+233
+
+343
+43
+45
+344
+22
+54
+8
+13
diff --git a/sql/hive/src/test/resources/data/files/json.txt b/sql/hive/src/test/resources/data/files/json.txt
new file mode 100644
index 0000000000000..1dcda4315a14d
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/json.txt
@@ -0,0 +1 @@
+{"store":{"fruit":[{"weight":8,"type":"apple"},{"weight":9,"type":"pear"}],"basket":[[1,2,{"b":"y","a":"x"}],[3,4],[5,6]],"book":[{"author":"Nigel Rees","title":"Sayings of the Century","category":"reference","price":8.95},{"author":"Herman Melville","title":"Moby Dick","category":"fiction","price":8.99,"isbn":"0-553-21311-3"},{"author":"J. R. R. Tolkien","title":"The Lord of the Rings","category":"fiction","reader":[{"age":25,"name":"bob"},{"age":26,"name":"jack"}],"price":22.99,"isbn":"0-395-19395-8"}],"bicycle":{"price":19.95,"color":"red"}},"email":"amy@only_for_json_udf_test.net","owner":"amy","zip code":"94025","fb:testid":"1234"}
diff --git a/sql/hive/src/test/resources/data/files/kv1.seq b/sql/hive/src/test/resources/data/files/kv1.seq
new file mode 100644
index 0000000000000..8da0a7751cf12
Binary files /dev/null and b/sql/hive/src/test/resources/data/files/kv1.seq differ
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 0000000000000..49bddeb805061
Binary files /dev/null and b/sql/hive/src/test/resources/data/files/kv1_broken.seq differ
diff --git a/sql/hive/src/test/resources/data/files/kv1_cb.txt b/sql/hive/src/test/resources/data/files/kv1_cb.txt
new file mode 100644
index 0000000000000..12b51587c92eb
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/kv1_cb.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_cc.txt b/sql/hive/src/test/resources/data/files/kv1_cc.txt
new file mode 100644
index 0000000000000..f300770523eab
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/kv1_cc.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/kv1kv2.cogroup.txt b/sql/hive/src/test/resources/data/files/kv1kv2.cogroup.txt
new file mode 100644
index 0000000000000..8a98be384998b
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/kv1kv2.cogroup.txt
@@ -0,0 +1,1000 @@
+val_0
+val_0
+val_0
+0
+0
+1
+val_10
+10
+val_100
+val_100
+100
+100
+101
+102
+val_103
+val_103
+val_104
+val_104
+104
+104
+104
+val_105
+105
+105
+106
+val_11
+11
+11
+11
+110
+val_111
+val_113
+val_113
+val_114
+114
+114
+114
+val_116
+116
+117
+117
+val_118
+val_118
+118
+118
+118
+val_119
+val_119
+val_119
+119
+119
+119
+val_12
+val_12
+12
+val_120
+val_120
+120
+121
+121
+122
+122
+122
+123
+123
+val_125
+val_125
+125
+val_126
+126
+126
+val_128
+val_128
+val_128
+128
+128
+val_129
+val_129
+129
+129
+val_131
+132
+132
+val_133
+133
+val_134
+val_134
+134
+135
+135
+135
+val_136
+136
+val_137
+val_137
+137
+val_138
+val_138
+val_138
+val_138
+138
+138
+140
+val_143
+143
+144
+val_145
+val_146
+val_146
+147
+147
+val_149
+val_149
+149
+val_15
+val_15
+15
+15
+val_150
+151
+151
+val_152
+val_152
+152
+152
+152
+val_153
+153
+153
+val_155
+val_156
+156
+156
+val_157
+157
+157
+val_158
+16
+16
+val_160
+160
+161
+161
+161
+161
+val_162
+162
+val_163
+val_164
+val_164
+164
+164
+val_165
+val_165
+165
+val_166
+val_167
+val_167
+val_167
+167
+val_168
+168
+val_169
+val_169
+val_169
+val_169
+val_17
+val_170
+170
+val_172
+val_172
+172
+val_174
+val_174
+174
+174
+val_175
+val_175
+175
+175
+val_176
+val_176
+val_177
+177
+177
+val_178
+178
+178
+val_179
+val_179
+179
+val_18
+val_18
+val_180
+val_181
+182
+val_183
+183
+184
+185
+val_186
+val_187
+val_187
+val_187
+val_189
+189
+val_19
+19
+val_190
+val_191
+val_191
+191
+val_192
+192
+val_193
+val_193
+val_193
+val_194
+val_195
+val_195
+val_196
+196
+196
+196
+val_197
+val_197
+197
+val_199
+val_199
+val_199
+199
+val_2
+2
+val_20
+20
+20
+val_200
+val_200
+val_201
+val_202
+val_203
+val_203
+204
+val_205
+val_205
+205
+206
+206
+206
+val_207
+val_207
+val_208
+val_208
+val_208
+val_209
+val_209
+209
+209
+21
+21
+21
+21
+212
+val_213
+val_213
+213
+val_214
+215
+val_216
+val_216
+216
+val_217
+val_217
+217
+217
+val_218
+val_219
+val_219
+22
+val_221
+val_221
+val_222
+222
+val_223
+val_223
+val_224
+val_224
+224
+val_226
+226
+226
+226
+226
+227
+val_228
+228
+val_229
+val_229
+23
+val_230
+val_230
+val_230
+val_230
+val_230
+231
+val_233
+val_233
+val_235
+235
+val_237
+val_237
+val_238
+val_238
+238
+val_239
+val_239
+239
+239
+val_24
+val_24
+240
+val_241
+241
+241
+241
+241
+val_242
+val_242
+242
+243
+243
+val_244
+244
+244
+244
+245
+245
+246
+246
+val_247
+val_248
+248
+val_249
+249
+249
+val_252
+252
+254
+val_255
+val_255
+val_256
+val_256
+256
+val_257
+257
+257
+val_258
+258
+259
+259
+val_26
+val_26
+val_260
+260
+260
+261
+val_262
+262
+262
+val_263
+264
+264
+val_265
+val_265
+265
+val_266
+267
+268
+val_27
+271
+val_272
+val_272
+272
+val_273
+val_273
+val_273
+273
+val_274
+274
+val_275
+275
+275
+276
+val_277
+val_277
+val_277
+val_277
+277
+277
+val_278
+val_278
+278
+val_28
+val_280
+val_280
+val_281
+val_281
+281
+281
+281
+val_282
+val_282
+val_283
+val_284
+284
+284
+val_285
+285
+val_286
+286
+val_287
+287
+287
+val_288
+val_288
+val_289
+289
+29
+29
+val_291
+291
+291
+val_292
+292
+292
+293
+293
+295
+295
+val_296
+296
+val_298
+val_298
+val_298
+3
+val_30
+30
+300
+300
+val_302
+302
+303
+303
+304
+val_305
+305
+val_306
+306
+val_307
+val_307
+val_308
+308
+308
+val_309
+val_309
+309
+31
+val_310
+310
+310
+310
+val_311
+val_311
+val_311
+313
+314
+val_315
+val_316
+val_316
+val_316
+val_317
+val_317
+317
+val_318
+val_318
+val_318
+318
+32
+val_321
+val_321
+val_322
+val_322
+322
+val_323
+323
+324
+val_325
+val_325
+326
+val_327
+val_327
+val_327
+328
+328
+val_33
+33
+330
+val_331
+val_331
+331
+331
+val_332
+val_333
+val_333
+333
+334
+val_335
+335
+335
+val_336
+336
+337
+val_338
+338
+val_339
+val_34
+340
+val_341
+341
+341
+341
+val_342
+val_342
+342
+343
+val_344
+val_344
+344
+val_345
+347
+347
+val_348
+val_348
+val_348
+val_348
+val_348
+348
+349
+349
+349
+349
+val_35
+val_35
+val_35
+35
+35
+35
+val_351
+351
+351
+352
+352
+val_353
+val_353
+353
+355
+355
+val_356
+356
+356
+358
+val_360
+360
+val_362
+363
+363
+363
+val_364
+364
+val_365
+val_366
+val_367
+val_367
+367
+367
+val_368
+val_369
+val_369
+val_369
+369
+val_37
+val_37
+371
+371
+371
+371
+val_373
+373
+val_374
+374
+val_375
+375
+375
+375
+375
+375
+376
+val_377
+val_378
+378
+val_379
+379
+381
+val_382
+val_382
+382
+382
+val_384
+val_384
+val_384
+384
+384
+384
+385
+385
+val_386
+386
+386
+388
+val_389
+389
+389
+390
+390
+390
+391
+391
+val_392
+392
+392
+val_393
+393
+393
+val_394
+val_395
+val_395
+395
+395
+val_396
+val_396
+val_396
+val_397
+val_397
+398
+val_399
+val_399
+399
+399
+val_4
+4
+40
+40
+val_400
+val_401
+val_401
+val_401
+val_401
+val_401
+401
+val_402
+402
+402
+402
+val_403
+val_403
+val_403
+val_404
+val_404
+404
+404
+404
+405
+val_406
+val_406
+val_406
+val_406
+406
+val_407
+407
+407
+407
+408
+408
+val_409
+val_409
+val_409
+409
+409
+val_41
+410
+val_411
+411
+412
+412
+val_413
+val_413
+413
+val_414
+val_414
+414
+415
+416
+val_417
+val_417
+val_417
+val_418
+val_419
+val_42
+val_42
+42
+42
+42
+val_421
+421
+421
+423
+val_424
+val_424
+424
+425
+426
+val_427
+427
+427
+428
+val_429
+val_429
+429
+429
+val_43
+val_430
+val_430
+val_430
+430
+val_431
+val_431
+val_431
+431
+val_432
+432
+val_435
+435
+val_436
+436
+val_437
+437
+val_438
+val_438
+val_438
+438
+438
+val_439
+val_439
+439
+439
+val_44
+440
+440
+441
+442
+val_443
+443
+443
+443
+val_444
+val_446
+446
+446
+447
+val_448
+448
+val_449
+450
+450
+451
+val_452
+val_453
+453
+val_454
+val_454
+val_454
+454
+454
+val_455
+455
+455
+val_457
+457
+457
+val_458
+val_458
+val_459
+val_459
+459
+46
+val_460
+461
+val_462
+val_462
+462
+val_463
+val_463
+463
+val_466
+val_466
+val_466
+val_467
+467
+val_468
+val_468
+val_468
+val_468
+468
+468
+468
+val_469
+val_469
+val_469
+val_469
+val_469
+469
+val_47
+47
+val_470
+470
+val_472
+473
+474
+474
+val_475
+475
+476
+476
+val_477
+477
+val_478
+val_478
+478
+478
+val_479
+48
+48
+val_480
+val_480
+val_480
+480
+480
+val_481
+481
+val_482
+482
+val_483
+val_484
+484
+val_485
+485
+485
+486
+val_487
+487
+488
+val_489
+val_489
+val_489
+val_489
+489
+49
+49
+val_490
+490
+val_491
+491
+491
+val_492
+val_492
+492
+492
+val_493
+val_494
+494
+494
+val_495
+495
+val_496
+496
+val_497
+497
+497
+val_498
+val_498
+val_498
+val_5
+val_5
+val_5
+5
+50
+val_51
+val_51
+51
+52
+52
+52
+52
+val_53
+53
+val_54
+56
+val_57
+val_58
+val_58
+58
+58
+59
+6
+6
+60
+61
+62
+62
+63
+val_64
+val_65
+65
+65
+val_66
+val_67
+val_67
+68
+val_69
+69
+val_70
+val_70
+val_70
+70
+71
+val_72
+val_72
+val_74
+75
+val_76
+val_76
+76
+76
+76
+val_77
+77
+77
+val_78
+78
+val_8
+8
+val_80
+80
+val_82
+82
+82
+val_83
+val_83
+val_84
+val_84
+val_85
+85
+val_86
+86
+val_87
+87
+87
+89
+89
+89
+val_9
+val_90
+val_90
+val_90
+91
+val_92
+93
+93
+93
+94
+val_95
+val_95
+val_96
+val_97
+val_97
+97
+97
+val_98
+val_98
+99
diff --git a/sql/hive/src/test/resources/data/files/kv2.txt b/sql/hive/src/test/resources/data/files/kv2.txt
new file mode 100644
index 0000000000000..e58f7e32ddeb4
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/kv2.txt
@@ -0,0 +1,500 @@
+474val_475
+281val_282
+179val_180
+291val_292
+62val_63
+271val_272
+217val_218
+135val_136
+167val_168
+468val_469
+423val_424
+413val_414
+245val_246
+455val_456
+425val_426
+241val_242
+177val_178
+272val_273
+231val_232
+448val_449
+287val_288
+246val_247
+440val_441
+31val_32
+373val_374
+447val_448
+443val_444
+175val_176
+278val_279
+296val_297
+147val_148
+428val_429
+126val_127
+249val_250
+21val_22
+273val_274
+441val_442
+106val_107
+371val_372
+153val_154
+356val_357
+217val_218
+490val_491
+33val_34
+35val_36
+421val_422
+243val_244
+402val_403
+128val_129
+10val_11
+226val_227
+133val_134
+333val_334
+15val_16
+110val_111
+391val_392
+343val_344
+275val_276
+0val_1
+485val_486
+293val_294
+240val_241
+241val_242
+286val_287
+85val_86
+408val_409
+477val_478
+455val_456
+99val_100
+335val_336
+476val_477
+482val_483
+367val_368
+59val_60
+48val_49
+424val_425
+226val_227
+494val_495
+488val_489
+94val_95
+485val_486
+393val_394
+349val_350
+11val_12
+161val_162
+123val_124
+409val_410
+50val_51
+265val_266
+402val_403
+128val_129
+468val_469
+314val_315
+497val_498
+224val_225
+63val_64
+344val_345
+4val_5
+277val_278
+135val_136
+3val_4
+206val_207
+101val_102
+331val_332
+114val_115
+209val_210
+281val_282
+56val_57
+239val_240
+389val_390
+114val_115
+254val_255
+235val_236
+390val_391
+165val_166
+304val_305
+264val_265
+196val_197
+238val_239
+20val_21
+11val_12
+492val_493
+129val_130
+257val_258
+71val_72
+289val_290
+453val_454
+421val_422
+82val_83
+58val_59
+5val_6
+86val_87
+249val_250
+438val_439
+360val_361
+323val_324
+222val_223
+42val_43
+467val_468
+411val_412
+175val_176
+429val_430
+338val_339
+281val_282
+68val_69
+185val_186
+16val_17
+147val_148
+119val_120
+492val_493
+473val_474
+376val_377
+347val_348
+120val_121
+213val_214
+306val_307
+393val_394
+427val_428
+291val_292
+426val_427
+132val_133
+446val_447
+386val_387
+65val_66
+121val_122
+388val_389
+184val_185
+375val_376
+284val_285
+191val_192
+129val_130
+246val_247
+197val_198
+491val_492
+369val_370
+262val_263
+122val_123
+438val_439
+351val_352
+390val_391
+352val_353
+23val_24
+226val_227
+328val_329
+382val_383
+342val_343
+349val_350
+480val_481
+102val_103
+497val_498
+487val_488
+480val_481
+331val_332
+409val_410
+475val_476
+463val_464
+183val_184
+177val_178
+399val_400
+15val_16
+318val_319
+149val_150
+392val_393
+77val_78
+275val_276
+476val_477
+258val_259
+174val_175
+252val_253
+114val_115
+87val_88
+161val_162
+75val_76
+264val_265
+48val_49
+336val_337
+340val_341
+390val_391
+443val_444
+484val_485
+407val_408
+189val_190
+305val_306
+367val_368
+6val_7
+260val_261
+2val_3
+349val_350
+170val_171
+51val_52
+355val_356
+363val_364
+164val_165
+118val_119
+310val_311
+104val_105
+19val_20
+80val_81
+117val_118
+326val_327
+47val_48
+121val_122
+241val_242
+450val_451
+140val_141
+212val_213
+308val_309
+30val_31
+405val_406
+239val_240
+119val_120
+358val_359
+11val_12
+49val_50
+143val_144
+153val_154
+341val_342
+21val_22
+105val_106
+157val_158
+416val_417
+199val_200
+375val_376
+42val_43
+89val_90
+363val_364
+395val_396
+347val_348
+386val_387
+77val_78
+454val_455
+293val_294
+364val_365
+20val_21
+89val_90
+461val_462
+52val_53
+40val_41
+8val_9
+168val_169
+384val_385
+313val_314
+324val_325
+310val_311
+206val_207
+381val_382
+404val_405
+206val_207
+385val_386
+259val_260
+226val_227
+262val_263
+69val_70
+303val_304
+245val_246
+260val_261
+328val_329
+322val_323
+122val_123
+241val_242
+469val_470
+404val_405
+384val_385
+76val_77
+116val_117
+42val_43
+104val_105
+481val_482
+303val_304
+93val_94
+277val_278
+406val_407
+451val_452
+53val_54
+32val_33
+132val_133
+192val_193
+58val_59
+351val_352
+70val_71
+443val_444
+356val_357
+399val_400
+205val_206
+352val_353
+52val_53
+21val_22
+317val_318
+330val_331
+209val_210
+138val_139
+429val_430
+160val_161
+267val_268
+454val_455
+257val_258
+76val_77
+174val_175
+412val_413
+16val_17
+375val_376
+489val_490
+295val_296
+105val_106
+204val_205
+126val_127
+274val_275
+439val_440
+374val_375
+494val_495
+216val_217
+457val_458
+93val_94
+470val_471
+196val_197
+302val_303
+119val_120
+375val_376
+391val_392
+287val_288
+450val_451
+375val_376
+437val_438
+12val_13
+398val_399
+35val_36
+334val_335
+384val_385
+435val_436
+60val_61
+29val_30
+442val_443
+151val_152
+52val_53
+404val_405
+446val_447
+300val_301
+0val_1
+268val_269
+491val_492
+392val_393
+21val_22
+295val_296
+93val_94
+104val_105
+436val_437
+61val_62
+407val_408
+439val_440
+156val_157
+341val_342
+118val_119
+335val_336
+172val_173
+244val_245
+6val_7
+284val_285
+164val_165
+136val_137
+349val_350
+371val_372
+462val_463
+123val_124
+432val_433
+496val_497
+355val_356
+144val_145
+87val_88
+408val_409
+152val_153
+382val_383
+348val_349
+122val_123
+427val_428
+292val_293
+182val_183
+474val_475
+353val_354
+310val_311
+52val_53
+486val_487
+152val_153
+378val_379
+414val_415
+261val_262
+65val_66
+371val_372
+256val_257
+292val_293
+412val_413
+97val_98
+495val_496
+385val_386
+40val_41
+49val_50
+389val_390
+259val_260
+97val_98
+478val_479
+125val_126
+243val_244
+151val_152
+178val_179
+100val_101
+415val_416
+401val_402
+156val_157
+228val_229
+363val_364
+117val_118
+22val_23
+248val_249
+459val_460
+402val_403
+62val_63
+137val_138
+162val_163
+244val_245
+276val_277
+46val_47
+341val_342
+78val_79
+379val_380
+215val_216
+134val_135
+196val_197
+410val_411
+157val_158
+431val_432
+82val_83
+440val_441
+100val_101
+407val_408
+371val_372
+308val_309
+430val_431
+309val_310
+135val_136
+161val_162
+468val_469
+152val_153
+138val_139
+76val_77
+161val_162
+300val_301
+337val_338
+91val_92
+478val_479
+118val_119
+1val_2
+89val_90
+178val_179
+457val_458
+29val_30
+242val_243
+285val_286
+35val_36
+227val_228
+395val_396
+244val_245
diff --git a/sql/hive/src/test/resources/data/files/kv3.txt b/sql/hive/src/test/resources/data/files/kv3.txt
new file mode 100644
index 0000000000000..ca4804703c97d
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/kv3.txt
@@ -0,0 +1,25 @@
+238val_238
+
+311val_311
+val_27
+val_165
+val_409
+255val_255
+278val_278
+98val_98
+val_484
+val_265
+val_193
+401val_401
+150val_150
+273val_273
+224
+369
+66val_66
+128
+213val_213
+146val_146
+406val_406
+
+
+
diff --git a/sql/hive/src/test/resources/data/files/kv4.txt b/sql/hive/src/test/resources/data/files/kv4.txt
new file mode 100644
index 0000000000000..f969221c1460c
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/kv4.txt
@@ -0,0 +1 @@
+邵铮
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/data/files/kv5.txt b/sql/hive/src/test/resources/data/files/kv5.txt
new file mode 100644
index 0000000000000..a6d2b79fa35dd
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/kv5.txt
@@ -0,0 +1,24 @@
+238val_238
+238val_239
+86val_86
+238val_240
+311val_311
+27val_27
+165val_165
+213val_213
+409val_409
+255val_255
+278val_278
+98val_98
+484val_484
+265val_265
+213val_214
+193val_193
+401val_401
+150val_150
+273val_273
+224val_224
+369val_369
+66val_66
+128val_128
+213val_213
diff --git a/sql/hive/src/test/resources/data/files/kv6.txt b/sql/hive/src/test/resources/data/files/kv6.txt
new file mode 100644
index 0000000000000..63de5c5057ed6
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/kv6.txt
@@ -0,0 +1,100 @@
+00
+01
+02
+03
+04
+05
+06
+07
+08
+09
+010
+011
+012
+013
+014
+015
+016
+017
+018
+019
+020
+021
+022
+023
+024
+025
+026
+027
+028
+029
+030
+031
+032
+033
+034
+035
+036
+037
+038
+039
+040
+041
+042
+043
+044
+045
+046
+047
+048
+049
+10
+11
+12
+13
+14
+15
+16
+17
+18
+19
+110
+111
+112
+113
+114
+115
+116
+117
+118
+119
+120
+121
+122
+123
+124
+125
+126
+127
+128
+129
+130
+131
+132
+133
+134
+135
+136
+137
+138
+139
+140
+141
+142
+143
+144
+145
+146
+147
+148
+149
diff --git a/sql/hive/src/test/resources/data/files/kv7.txt b/sql/hive/src/test/resources/data/files/kv7.txt
new file mode 100644
index 0000000000000..b83c46099bbd7
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/kv7.txt
@@ -0,0 +1,38 @@
+-4400 4400
+1E+99 0
+1E-99 0
+0 0
+100 100
+10 10
+1 1
+0.1 0
+0.01 0
+200 200
+20 20
+2 2
+0 0
+0.2 0
+0.02 0
+0.3 0
+0.33 0
+0.333 0
+-0.3 0
+-0.33 0
+-0.333 0
+1.0 1
+2 2
+3.14 3
+-1.12 -1
+-1.12 -1
+-1.122 -11
+1.12 1
+1.122 1
+124.00 124
+125.2 125
+-1255.49 -1255
+3.14 3
+3.14 3
+3.140 4
+0.9999999999999999999999999 1
+-1234567890.1234567890 -1234567890
+1234567890.1234567800 1234567890
diff --git a/sql/hive/src/test/resources/data/files/kv8.txt b/sql/hive/src/test/resources/data/files/kv8.txt
new file mode 100644
index 0000000000000..29a28ac302e90
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/kv8.txt
@@ -0,0 +1,75 @@
+123456789012345678901234567890123456789
+1.23456789012345678901234567890123456789
+12.3456789012345678901234567890123456789
+123.456789012345678901234567890123456789
+1234.56789012345678901234567890123456789
+12345678901234567890123456789012345678
+0.1234567890123456789012345678901234578
+1.2345678901234567890123456789012345678
+12.345678901234567890123456789012345678
+123.45678901234567890123456789012345678
+1234.5678901234567890123456789012345678
+12345.678901234567890123456789012345678
+123456.78901234567890123456789012345678
+1234567.8901234567890123456789012345678
+12345678.901234567890123456789012345678
+123456789.01234567890123456789012345678
+1234567890.1234567890123456789012345678
+12345678901.234567890123456789012345678
+123456789012.34567890123456789012345678
+1234567890123.4567890123456789012345678
+12345678901234.567890123456789012345678
+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
+123456789012345678901234567890123456.78
+12345678901234567890123456789012345.67.8
+12345678901234567890123456789012345678.0
+12345678901234567890123456789012345678.00
+12345678901234567890123456789012345678.000
+00012345678901234567890123456789012345678.000
+99999999999999999999999999999999999999
+999999999999999999999999999999999999
+99999999999999999999999999999999999
+-99999999999999999999999999999999999999
+-999999999999999999999999999999999999
+-99999999999999999999999999999999999
+000000000000000000000000000000000000000000000
+0.0000000000000000000000000000000000001
+-0.0000000000000000000000000000000000001
+0.00000000000000000000000000000000000001
+-0.00000000000000000000000000000000000001
+0.123456789012345
+1.234567890123456
+12.34567890123456
+123.4567890123456
+1234.567890123456
+12345.67890123456
+123456.7890123456
+1234567.890123456
+12345678.90123456
+123456789.0123456
+1234567890.123456
+12345678901.23456
+123456789012.3456
+1234567890123.456
+12345678901234.56
+123456789012345.6
diff --git a/sql/hive/src/test/resources/data/files/leftsemijoin_mr_t1.txt b/sql/hive/src/test/resources/data/files/leftsemijoin_mr_t1.txt
new file mode 100644
index 0000000000000..6ed281c757a96
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/leftsemijoin_mr_t1.txt
@@ -0,0 +1,2 @@
+1
+1
diff --git a/sql/hive/src/test/resources/data/files/leftsemijoin_mr_t2.txt b/sql/hive/src/test/resources/data/files/leftsemijoin_mr_t2.txt
new file mode 100644
index 0000000000000..179ef0e0209e9
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/leftsemijoin_mr_t2.txt
@@ -0,0 +1,20 @@
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
+1
diff --git a/sql/hive/src/test/resources/data/files/lineitem.txt b/sql/hive/src/test/resources/data/files/lineitem.txt
new file mode 100644
index 0000000000000..1e11d95f5e882
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/lineitem.txt
@@ -0,0 +1,100 @@
+1|155190|7706|1|17|21168.23|0.04|0.02|N|O|1996-03-13|1996-02-12|1996-03-22|DELIVER IN PERSON|TRUCK|egular courts above the|
+1|67310|7311|2|36|45983.16|0.09|0.06|N|O|1996-04-12|1996-02-28|1996-04-20|TAKE BACK RETURN|MAIL|ly final dependencies: slyly bold |
+1|63700|3701|3|8|13309.60|0.10|0.02|N|O|1996-01-29|1996-03-05|1996-01-31|TAKE BACK RETURN|REG AIR|riously. regular, express dep|
+1|2132|4633|4|28|28955.64|0.09|0.06|N|O|1996-04-21|1996-03-30|1996-05-16|NONE|AIR|lites. fluffily even de|
+1|24027|1534|5|24|22824.48|0.10|0.04|N|O|1996-03-30|1996-03-14|1996-04-01|NONE|FOB| pending foxes. slyly re|
+1|15635|638|6|32|49620.16|0.07|0.02|N|O|1996-01-30|1996-02-07|1996-02-03|DELIVER IN PERSON|MAIL|arefully slyly ex|
+2|106170|1191|1|38|44694.46|0.00|0.05|N|O|1997-01-28|1997-01-14|1997-02-02|TAKE BACK RETURN|RAIL|ven requests. deposits breach a|
+3|4297|1798|1|45|54058.05|0.06|0.00|R|F|1994-02-02|1994-01-04|1994-02-23|NONE|AIR|ongside of the furiously brave acco|
+3|19036|6540|2|49|46796.47|0.10|0.00|R|F|1993-11-09|1993-12-20|1993-11-24|TAKE BACK RETURN|RAIL| unusual accounts. eve|
+3|128449|3474|3|27|39890.88|0.06|0.07|A|F|1994-01-16|1993-11-22|1994-01-23|DELIVER IN PERSON|SHIP|nal foxes wake. |
+3|29380|1883|4|2|2618.76|0.01|0.06|A|F|1993-12-04|1994-01-07|1994-01-01|NONE|TRUCK|y. fluffily pending d|
+3|183095|650|5|28|32986.52|0.04|0.00|R|F|1993-12-14|1994-01-10|1994-01-01|TAKE BACK RETURN|FOB|ages nag slyly pending|
+3|62143|9662|6|26|28733.64|0.10|0.02|A|F|1993-10-29|1993-12-18|1993-11-04|TAKE BACK RETURN|RAIL|ges sleep after the caref|
+4|88035|5560|1|30|30690.90|0.03|0.08|N|O|1996-01-10|1995-12-14|1996-01-18|DELIVER IN PERSON|REG AIR|- quickly regular packages sleep. idly|
+5|108570|8571|1|15|23678.55|0.02|0.04|R|F|1994-10-31|1994-08-31|1994-11-20|NONE|AIR|ts wake furiously |
+5|123927|3928|2|26|50723.92|0.07|0.08|R|F|1994-10-16|1994-09-25|1994-10-19|NONE|FOB|sts use slyly quickly special instruc|
+5|37531|35|3|50|73426.50|0.08|0.03|A|F|1994-08-08|1994-10-13|1994-08-26|DELIVER IN PERSON|AIR|eodolites. fluffily unusual|
+6|139636|2150|1|37|61998.31|0.08|0.03|A|F|1992-04-27|1992-05-15|1992-05-02|TAKE BACK RETURN|TRUCK|p furiously special foxes|
+7|182052|9607|1|12|13608.60|0.07|0.03|N|O|1996-05-07|1996-03-13|1996-06-03|TAKE BACK RETURN|FOB|ss pinto beans wake against th|
+7|145243|7758|2|9|11594.16|0.08|0.08|N|O|1996-02-01|1996-03-02|1996-02-19|TAKE BACK RETURN|SHIP|es. instructions|
+7|94780|9799|3|46|81639.88|0.10|0.07|N|O|1996-01-15|1996-03-27|1996-02-03|COLLECT COD|MAIL| unusual reques|
+7|163073|3074|4|28|31809.96|0.03|0.04|N|O|1996-03-21|1996-04-08|1996-04-20|NONE|FOB|. slyly special requests haggl|
+7|151894|9440|5|38|73943.82|0.08|0.01|N|O|1996-02-11|1996-02-24|1996-02-18|DELIVER IN PERSON|TRUCK|ns haggle carefully ironic deposits. bl|
+7|79251|1759|6|35|43058.75|0.06|0.03|N|O|1996-01-16|1996-02-23|1996-01-22|TAKE BACK RETURN|FOB|jole. excuses wake carefully alongside of |
+7|157238|2269|7|5|6476.15|0.04|0.02|N|O|1996-02-10|1996-03-26|1996-02-13|NONE|FOB|ithely regula|
+32|82704|7721|1|28|47227.60|0.05|0.08|N|O|1995-10-23|1995-08-27|1995-10-26|TAKE BACK RETURN|TRUCK|sleep quickly. req|
+32|197921|441|2|32|64605.44|0.02|0.00|N|O|1995-08-14|1995-10-07|1995-08-27|COLLECT COD|AIR|lithely regular deposits. fluffily |
+32|44161|6666|3|2|2210.32|0.09|0.02|N|O|1995-08-07|1995-10-07|1995-08-23|DELIVER IN PERSON|AIR| express accounts wake according to the|
+32|2743|7744|4|4|6582.96|0.09|0.03|N|O|1995-08-04|1995-10-01|1995-09-03|NONE|REG AIR|e slyly final pac|
+32|85811|8320|5|44|79059.64|0.05|0.06|N|O|1995-08-28|1995-08-20|1995-09-14|DELIVER IN PERSON|AIR|symptotes nag according to the ironic depo|
+32|11615|4117|6|6|9159.66|0.04|0.03|N|O|1995-07-21|1995-09-23|1995-07-25|COLLECT COD|RAIL| gifts cajole carefully.|
+33|61336|8855|1|31|40217.23|0.09|0.04|A|F|1993-10-29|1993-12-19|1993-11-08|COLLECT COD|TRUCK|ng to the furiously ironic package|
+33|60519|5532|2|32|47344.32|0.02|0.05|A|F|1993-12-09|1994-01-04|1993-12-28|COLLECT COD|MAIL|gular theodolites|
+33|137469|9983|3|5|7532.30|0.05|0.03|A|F|1993-12-09|1993-12-25|1993-12-23|TAKE BACK RETURN|AIR|. stealthily bold exc|
+33|33918|3919|4|41|75928.31|0.09|0.00|R|F|1993-11-09|1994-01-24|1993-11-11|TAKE BACK RETURN|MAIL|unusual packages doubt caref|
+34|88362|871|1|13|17554.68|0.00|0.07|N|O|1998-10-23|1998-09-14|1998-11-06|NONE|REG AIR|nic accounts. deposits are alon|
+34|89414|1923|2|22|30875.02|0.08|0.06|N|O|1998-10-09|1998-10-16|1998-10-12|NONE|FOB|thely slyly p|
+34|169544|4577|3|6|9681.24|0.02|0.06|N|O|1998-10-30|1998-09-20|1998-11-05|NONE|FOB|ar foxes sleep |
+35|450|2951|1|24|32410.80|0.02|0.00|N|O|1996-02-21|1996-01-03|1996-03-18|TAKE BACK RETURN|FOB|, regular tithe|
+35|161940|4457|2|34|68065.96|0.06|0.08|N|O|1996-01-22|1996-01-06|1996-01-27|DELIVER IN PERSON|RAIL|s are carefully against the f|
+35|120896|8433|3|7|13418.23|0.06|0.04|N|O|1996-01-19|1995-12-22|1996-01-29|NONE|MAIL| the carefully regular |
+35|85175|7684|4|25|29004.25|0.06|0.05|N|O|1995-11-26|1995-12-25|1995-12-21|DELIVER IN PERSON|SHIP| quickly unti|
+35|119917|4940|5|34|65854.94|0.08|0.06|N|O|1995-11-08|1996-01-15|1995-11-26|COLLECT COD|MAIL|. silent, unusual deposits boost|
+35|30762|3266|6|28|47397.28|0.03|0.02|N|O|1996-02-01|1995-12-24|1996-02-28|COLLECT COD|RAIL|ly alongside of |
+36|119767|9768|1|42|75043.92|0.09|0.00|N|O|1996-02-03|1996-01-21|1996-02-23|COLLECT COD|SHIP| careful courts. special |
+37|22630|5133|1|40|62105.20|0.09|0.03|A|F|1992-07-21|1992-08-01|1992-08-15|NONE|REG AIR|luffily regular requests. slyly final acco|
+37|126782|1807|2|39|70542.42|0.05|0.02|A|F|1992-07-02|1992-08-18|1992-07-28|TAKE BACK RETURN|RAIL|the final requests. ca|
+37|12903|5405|3|43|78083.70|0.05|0.08|A|F|1992-07-10|1992-07-06|1992-08-02|DELIVER IN PERSON|TRUCK|iously ste|
+38|175839|874|1|44|84252.52|0.04|0.02|N|O|1996-09-29|1996-11-17|1996-09-30|COLLECT COD|MAIL|s. blithely unusual theodolites am|
+39|2320|9821|1|44|53782.08|0.09|0.06|N|O|1996-11-14|1996-12-15|1996-12-12|COLLECT COD|RAIL|eodolites. careful|
+39|186582|4137|2|26|43383.08|0.08|0.04|N|O|1996-11-04|1996-10-20|1996-11-20|NONE|FOB|ckages across the slyly silent|
+39|67831|5350|3|46|82746.18|0.06|0.08|N|O|1996-09-26|1996-12-19|1996-10-26|DELIVER IN PERSON|AIR|he carefully e|
+39|20590|3093|4|32|48338.88|0.07|0.05|N|O|1996-10-02|1996-12-19|1996-10-14|COLLECT COD|MAIL|heodolites sleep silently pending foxes. ac|
+39|54519|9530|5|43|63360.93|0.01|0.01|N|O|1996-10-17|1996-11-14|1996-10-26|COLLECT COD|MAIL|yly regular i|
+39|94368|6878|6|40|54494.40|0.06|0.05|N|O|1996-12-08|1996-10-22|1997-01-01|COLLECT COD|AIR|quickly ironic fox|
+64|85951|5952|1|21|40675.95|0.05|0.02|R|F|1994-09-30|1994-09-18|1994-10-26|DELIVER IN PERSON|REG AIR|ch slyly final, thin platelets.|
+65|59694|4705|1|26|42995.94|0.03|0.03|A|F|1995-04-20|1995-04-25|1995-05-13|NONE|TRUCK|pending deposits nag even packages. ca|
+65|73815|8830|2|22|39353.82|0.00|0.05|N|O|1995-07-17|1995-06-04|1995-07-19|COLLECT COD|FOB| ideas. special, r|
+65|1388|3889|3|21|27076.98|0.09|0.07|N|O|1995-07-06|1995-05-14|1995-07-31|DELIVER IN PERSON|RAIL|bove the even packages. accounts nag carefu|
+66|115118|7630|1|31|35126.41|0.00|0.08|R|F|1994-02-19|1994-03-11|1994-02-20|TAKE BACK RETURN|RAIL|ut the unusual accounts sleep at the bo|
+66|173489|3490|2|41|64061.68|0.04|0.07|A|F|1994-02-21|1994-03-01|1994-03-18|COLLECT COD|AIR| regular de|
+67|21636|9143|1|4|6230.52|0.09|0.04|N|O|1997-04-17|1997-01-31|1997-04-20|NONE|SHIP| cajole thinly expres|
+67|20193|5198|2|12|13358.28|0.09|0.05|N|O|1997-01-27|1997-02-21|1997-02-22|NONE|REG AIR| even packages cajole|
+67|173600|6118|3|5|8368.00|0.03|0.07|N|O|1997-02-20|1997-02-12|1997-02-21|DELIVER IN PERSON|TRUCK|y unusual packages thrash pinto |
+67|87514|7515|4|44|66066.44|0.08|0.06|N|O|1997-03-18|1997-01-29|1997-04-13|DELIVER IN PERSON|RAIL|se quickly above the even, express reques|
+67|40613|8126|5|23|35733.03|0.05|0.07|N|O|1997-04-19|1997-02-14|1997-05-06|DELIVER IN PERSON|REG AIR|ly regular deposit|
+67|178306|824|6|29|40144.70|0.02|0.05|N|O|1997-01-25|1997-01-27|1997-01-27|DELIVER IN PERSON|FOB|ultipliers |
+68|7068|9569|1|3|2925.18|0.05|0.02|N|O|1998-07-04|1998-06-05|1998-07-21|NONE|RAIL|fully special instructions cajole. furious|
+68|175180|2732|2|46|57738.28|0.02|0.05|N|O|1998-06-26|1998-06-07|1998-07-05|NONE|MAIL| requests are unusual, regular pinto |
+68|34980|7484|3|46|88089.08|0.04|0.05|N|O|1998-08-13|1998-07-08|1998-08-29|NONE|RAIL|egular dependencies affix ironically along |
+68|94728|2256|4|20|34454.40|0.07|0.01|N|O|1998-06-27|1998-05-23|1998-07-02|NONE|REG AIR| excuses integrate fluffily |
+68|82758|5267|5|27|47000.25|0.03|0.06|N|O|1998-06-19|1998-06-25|1998-06-29|DELIVER IN PERSON|SHIP|ccounts. deposits use. furiously|
+68|102561|5072|6|30|46906.80|0.05|0.06|N|O|1998-08-11|1998-07-11|1998-08-14|NONE|RAIL|oxes are slyly blithely fin|
+68|139247|1761|7|41|52735.84|0.09|0.08|N|O|1998-06-24|1998-06-27|1998-07-06|NONE|SHIP|eposits nag special ideas. furiousl|
+69|115209|7721|1|48|58761.60|0.01|0.07|A|F|1994-08-17|1994-08-11|1994-09-08|NONE|TRUCK|regular epitaphs. carefully even ideas hag|
+69|104180|9201|2|32|37893.76|0.08|0.06|A|F|1994-08-24|1994-08-17|1994-08-31|NONE|REG AIR|s sleep carefully bold, |
+69|137267|4807|3|17|22172.42|0.09|0.00|A|F|1994-07-02|1994-07-07|1994-07-03|TAKE BACK RETURN|AIR|final, pending instr|
+69|37502|2509|4|3|4318.50|0.09|0.04|R|F|1994-06-06|1994-07-27|1994-06-15|NONE|MAIL| blithely final d|
+69|92070|7089|5|42|44606.94|0.07|0.04|R|F|1994-07-31|1994-07-26|1994-08-28|DELIVER IN PERSON|REG AIR|tect regular, speci|
+69|18504|1006|6|23|32717.50|0.05|0.00|A|F|1994-10-03|1994-08-06|1994-10-24|NONE|SHIP|nding accounts ca|
+70|64128|9141|1|8|8736.96|0.03|0.08|R|F|1994-01-12|1994-02-27|1994-01-14|TAKE BACK RETURN|FOB|ggle. carefully pending dependenc|
+70|196156|1195|2|13|16277.95|0.06|0.06|A|F|1994-03-03|1994-02-13|1994-03-26|COLLECT COD|AIR|lyly special packag|
+70|179809|7361|3|1|1888.80|0.03|0.05|R|F|1994-01-26|1994-03-05|1994-01-28|TAKE BACK RETURN|RAIL|quickly. fluffily unusual theodolites c|
+70|45734|743|4|11|18477.03|0.01|0.05|A|F|1994-03-17|1994-03-17|1994-03-27|NONE|MAIL|alongside of the deposits. fur|
+70|37131|2138|5|37|39520.81|0.09|0.04|R|F|1994-02-13|1994-03-16|1994-02-21|COLLECT COD|MAIL|n accounts are. q|
+70|55655|3171|6|19|30602.35|0.06|0.03|A|F|1994-01-26|1994-02-17|1994-02-06|TAKE BACK RETURN|SHIP| packages wake pending accounts.|
+71|61931|1932|1|25|47323.25|0.09|0.07|N|O|1998-04-10|1998-04-22|1998-04-11|COLLECT COD|FOB|ckly. slyly|
+71|65916|3435|2|3|5645.73|0.09|0.07|N|O|1998-05-23|1998-04-03|1998-06-02|COLLECT COD|SHIP|y. pinto beans haggle after the|
+71|34432|1942|3|45|61489.35|0.00|0.07|N|O|1998-02-23|1998-03-20|1998-03-24|DELIVER IN PERSON|SHIP| ironic packages believe blithely a|
+71|96645|9155|4|33|54174.12|0.00|0.01|N|O|1998-04-12|1998-03-20|1998-04-15|NONE|FOB| serve quickly fluffily bold deposi|
+71|103255|5766|5|39|49071.75|0.08|0.06|N|O|1998-01-29|1998-04-07|1998-02-18|DELIVER IN PERSON|RAIL|l accounts sleep across the pack|
+71|195635|674|6|34|58841.42|0.04|0.01|N|O|1998-03-05|1998-04-22|1998-03-30|DELIVER IN PERSON|TRUCK|s cajole. |
+96|123076|613|1|23|25278.61|0.10|0.06|A|F|1994-07-19|1994-06-29|1994-07-25|DELIVER IN PERSON|TRUCK|ep-- carefully reg|
+96|135390|5391|2|30|42761.70|0.01|0.06|R|F|1994-06-03|1994-05-29|1994-06-22|DELIVER IN PERSON|TRUCK|e quickly even ideas. furiou|
+97|119477|1989|1|13|19454.11|0.00|0.02|R|F|1993-04-01|1993-04-04|1993-04-08|NONE|TRUCK|ayers cajole against the furiously|
+97|49568|2073|2|37|56149.72|0.02|0.06|A|F|1993-04-13|1993-03-30|1993-04-14|DELIVER IN PERSON|SHIP|ic requests boost carefully quic|
+97|77699|5221|3|19|31857.11|0.06|0.08|R|F|1993-05-14|1993-03-05|1993-05-25|TAKE BACK RETURN|RAIL|gifts. furiously ironic packages cajole. |
+98|40216|217|1|28|32373.88|0.06|0.07|A|F|1994-12-24|1994-10-25|1995-01-16|COLLECT COD|REG AIR| pending, regular accounts s|
+98|109743|7274|2|1|1752.74|0.00|0.00|A|F|1994-12-01|1994-12-12|1994-12-15|DELIVER IN PERSON|TRUCK|. unusual instructions against|
+98|44706|4707|3|14|23109.80|0.05|0.02|A|F|1994-12-30|1994-11-22|1995-01-27|COLLECT COD|AIR| cajole furiously. blithely ironic ideas |
diff --git a/sql/hive/src/test/resources/data/files/lt100.sorted.txt b/sql/hive/src/test/resources/data/files/lt100.sorted.txt
new file mode 100644
index 0000000000000..0935135496a6d
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/lt100.sorted.txt
@@ -0,0 +1,84 @@
+0val_0
+0val_0
+0val_0
+10val_10
+11val_11
+12val_12
+12val_12
+15val_15
+15val_15
+17val_17
+18val_18
+18val_18
+19val_19
+2val_2
+20val_20
+24val_24
+24val_24
+26val_26
+26val_26
+27val_27
+28val_28
+30val_30
+33val_33
+34val_34
+35val_35
+35val_35
+35val_35
+37val_37
+37val_37
+4val_4
+41val_41
+42val_42
+42val_42
+43val_43
+44val_44
+47val_47
+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/lt100.txt b/sql/hive/src/test/resources/data/files/lt100.txt
new file mode 100644
index 0000000000000..3f36729612173
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/lt100.txt
@@ -0,0 +1,84 @@
+86val_86
+27val_27
+98val_98
+66val_66
+37val_37
+15val_15
+82val_82
+17val_17
+0val_0
+57val_57
+20val_20
+92val_92
+47val_47
+72val_72
+4val_4
+35val_35
+54val_54
+51val_51
+65val_65
+83val_83
+12val_12
+67val_67
+84val_84
+58val_58
+8val_8
+24val_24
+42val_42
+0val_0
+96val_96
+26val_26
+51val_51
+43val_43
+95val_95
+98val_98
+85val_85
+77val_77
+0val_0
+87val_87
+15val_15
+72val_72
+90val_90
+19val_19
+10val_10
+5val_5
+58val_58
+35val_35
+95val_95
+11val_11
+34val_34
+42val_42
+78val_78
+76val_76
+41val_41
+30val_30
+64val_64
+76val_76
+74val_74
+69val_69
+33val_33
+70val_70
+5val_5
+2val_2
+35val_35
+80val_80
+44val_44
+53val_53
+90val_90
+12val_12
+5val_5
+70val_70
+24val_24
+70val_70
+83val_83
+26val_26
+67val_67
+18val_18
+9val_9
+18val_18
+97val_97
+84val_84
+28val_28
+37val_37
+90val_90
+97val_97
diff --git a/sql/hive/src/test/resources/data/files/lt100.txt.deflate b/sql/hive/src/test/resources/data/files/lt100.txt.deflate
new file mode 100644
index 0000000000000..7ad780adf9c2c
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/lt100.txt.deflate
@@ -0,0 +1,3 @@
+x�MR;nDASZ�f���D�S��=�.�!��*��~~�+��D��z�.�0f-!��d��\�0+����N(!]hַ�D˃���mlB۱�� ��3�8� �!�`�H�u�t�
+ʜ:�L��>�`�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 0000000000000..e39e72ea58bb2
Binary files /dev/null and b/sql/hive/src/test/resources/data/files/part.rc differ
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 0000000000000..b919eb28db697
Binary files /dev/null and b/sql/hive/src/test/resources/data/files/part.seq differ
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 0000000000000..61814607246d1
Binary files /dev/null and b/sql/hive/src/test/resources/data/files/smb_bucket_input.rc differ
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 0000000000000..51dbe23ae8631
Binary files /dev/null and b/sql/hive/src/test/resources/data/files/smbbucket_1.rc differ
diff --git a/sql/hive/src/test/resources/data/files/smbbucket_1.txt b/sql/hive/src/test/resources/data/files/smbbucket_1.txt
new file mode 100644
index 0000000000000..6093b2bcc301e
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/smbbucket_1.txt
@@ -0,0 +1,5 @@
+1val_1
+3val_3
+4val_4
+5val_5
+10val_10
diff --git a/sql/hive/src/test/resources/data/files/smbbucket_2.rc b/sql/hive/src/test/resources/data/files/smbbucket_2.rc
new file mode 100644
index 0000000000000..73a641acaa9a0
Binary files /dev/null and b/sql/hive/src/test/resources/data/files/smbbucket_2.rc differ
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 0000000000000..c0e8a590de753
Binary files /dev/null and b/sql/hive/src/test/resources/data/files/smbbucket_3.rc differ
diff --git a/sql/hive/src/test/resources/data/files/smbbucket_3.txt b/sql/hive/src/test/resources/data/files/smbbucket_3.txt
new file mode 100644
index 0000000000000..8339550931d7e
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/smbbucket_3.txt
@@ -0,0 +1,6 @@
+4val_4
+10val_10
+17val_17
+19val_19
+20val_20
+23val_23
diff --git a/sql/hive/src/test/resources/data/files/source.txt b/sql/hive/src/test/resources/data/files/source.txt
new file mode 100644
index 0000000000000..567d3b07f7ecc
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/source.txt
@@ -0,0 +1,4 @@
+EXPLAIN
+SELECT x.* FROM SRC x;
+
+SELECT x.* FROM SRC x;
diff --git a/sql/hive/src/test/resources/data/files/srcbucket0.txt b/sql/hive/src/test/resources/data/files/srcbucket0.txt
new file mode 100644
index 0000000000000..c11aec09d961a
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/srcbucket0.txt
@@ -0,0 +1,493 @@
+474val_475
+62val_63
+468val_469
+272val_273
+448val_449
+246val_247
+440val_441
+278val_279
+296val_297
+428val_429
+126val_127
+106val_107
+356val_357
+490val_491
+402val_403
+128val_129
+10val_11
+226val_227
+110val_111
+0val_1
+240val_241
+286val_287
+408val_409
+476val_477
+482val_483
+48val_49
+424val_425
+226val_227
+494val_495
+488val_489
+94val_95
+50val_51
+402val_403
+128val_129
+468val_469
+314val_315
+224val_225
+344val_345
+4val_5
+206val_207
+114val_115
+56val_57
+114val_115
+254val_255
+390val_391
+304val_305
+264val_265
+196val_197
+238val_239
+20val_21
+492val_493
+82val_83
+58val_59
+86val_87
+438val_439
+360val_361
+222val_223
+42val_43
+338val_339
+68val_69
+16val_17
+492val_493
+376val_377
+120val_121
+306val_307
+426val_427
+132val_133
+446val_447
+386val_387
+388val_389
+184val_185
+284val_285
+246val_247
+262val_263
+122val_123
+438val_439
+390val_391
+352val_353
+226val_227
+328val_329
+382val_383
+342val_343
+480val_481
+102val_103
+480val_481
+318val_319
+392val_393
+476val_477
+258val_259
+174val_175
+252val_253
+114val_115
+264val_265
+48val_49
+336val_337
+340val_341
+390val_391
+484val_485
+6val_7
+260val_261
+2val_3
+170val_171
+164val_165
+118val_119
+310val_311
+104val_105
+80val_81
+326val_327
+450val_451
+140val_141
+212val_213
+308val_309
+30val_31
+358val_359
+416val_417
+42val_43
+386val_387
+454val_455
+364val_365
+20val_21
+52val_53
+40val_41
+8val_9
+168val_169
+384val_385
+324val_325
+310val_311
+206val_207
+404val_405
+206val_207
+226val_227
+262val_263
+260val_261
+328val_329
+322val_323
+122val_123
+404val_405
+384val_385
+76val_77
+116val_117
+42val_43
+104val_105
+406val_407
+32val_33
+132val_133
+192val_193
+58val_59
+70val_71
+356val_357
+352val_353
+52val_53
+330val_331
+138val_139
+160val_161
+454val_455
+76val_77
+174val_175
+412val_413
+16val_17
+204val_205
+126val_127
+274val_275
+374val_375
+494val_495
+216val_217
+470val_471
+196val_197
+302val_303
+450val_451
+12val_13
+398val_399
+334val_335
+384val_385
+60val_61
+442val_443
+52val_53
+404val_405
+446val_447
+300val_301
+0val_1
+268val_269
+392val_393
+104val_105
+436val_437
+156val_157
+118val_119
+172val_173
+244val_245
+6val_7
+284val_285
+164val_165
+136val_137
+462val_463
+432val_433
+496val_497
+144val_145
+408val_409
+152val_153
+382val_383
+348val_349
+122val_123
+292val_293
+182val_183
+474val_475
+310val_311
+52val_53
+486val_487
+152val_153
+378val_379
+414val_415
+256val_257
+292val_293
+412val_413
+40val_41
+478val_479
+178val_179
+100val_101
+156val_157
+228val_229
+22val_23
+248val_249
+402val_403
+62val_63
+162val_163
+244val_245
+276val_277
+46val_47
+78val_79
+134val_135
+196val_197
+410val_411
+82val_83
+440val_441
+100val_101
+308val_309
+430val_431
+468val_469
+152val_153
+138val_139
+76val_77
+300val_301
+478val_479
+118val_119
+178val_179
+242val_243
+244val_245
+238val_238
+86val_86
+278val_278
+98val_98
+484val_484
+150val_150
+224val_224
+66val_66
+128val_128
+146val_146
+406val_406
+374val_374
+152val_152
+82val_82
+166val_166
+430val_430
+252val_252
+292val_292
+338val_338
+446val_446
+394val_394
+482val_482
+174val_174
+494val_494
+466val_466
+208val_208
+174val_174
+396val_396
+162val_162
+266val_266
+342val_342
+0val_0
+128val_128
+316val_316
+302val_302
+438val_438
+170val_170
+20val_20
+378val_378
+92val_92
+72val_72
+4val_4
+280val_280
+208val_208
+356val_356
+382val_382
+498val_498
+386val_386
+192val_192
+286val_286
+176val_176
+54val_54
+138val_138
+216val_216
+430val_430
+278val_278
+176val_176
+318val_318
+332val_332
+180val_180
+284val_284
+12val_12
+230val_230
+260val_260
+404val_404
+384val_384
+272val_272
+138val_138
+84val_84
+348val_348
+466val_466
+58val_58
+8val_8
+230val_230
+208val_208
+348val_348
+24val_24
+172val_172
+42val_42
+158val_158
+496val_496
+0val_0
+322val_322
+468val_468
+454val_454
+100val_100
+298val_298
+418val_418
+96val_96
+26val_26
+230val_230
+120val_120
+404val_404
+436val_436
+156val_156
+468val_468
+308val_308
+196val_196
+288val_288
+98val_98
+282val_282
+318val_318
+318val_318
+470val_470
+316val_316
+0val_0
+490val_490
+364val_364
+118val_118
+134val_134
+282val_282
+138val_138
+238val_238
+118val_118
+72val_72
+90val_90
+10val_10
+306val_306
+224val_224
+242val_242
+392val_392
+272val_272
+242val_242
+452val_452
+226val_226
+402val_402
+396val_396
+58val_58
+336val_336
+168val_168
+34val_34
+472val_472
+322val_322
+498val_498
+160val_160
+42val_42
+430val_430
+458val_458
+78val_78
+76val_76
+492val_492
+218val_218
+228val_228
+138val_138
+30val_30
+64val_64
+468val_468
+76val_76
+74val_74
+342val_342
+230val_230
+368val_368
+296val_296
+216val_216
+344val_344
+274val_274
+116val_116
+256val_256
+70val_70
+480val_480
+288val_288
+244val_244
+438val_438
+128val_128
+432val_432
+202val_202
+316val_316
+280val_280
+2val_2
+80val_80
+44val_44
+104val_104
+466val_466
+366val_366
+406val_406
+190val_190
+406val_406
+114val_114
+258val_258
+90val_90
+262val_262
+348val_348
+424val_424
+12val_12
+396val_396
+164val_164
+454val_454
+478val_478
+298val_298
+164val_164
+424val_424
+382val_382
+70val_70
+480val_480
+24val_24
+104val_104
+70val_70
+438val_438
+414val_414
+200val_200
+360val_360
+248val_248
+444val_444
+120val_120
+230val_230
+478val_478
+178val_178
+468val_468
+310val_310
+460val_460
+480val_480
+136val_136
+172val_172
+214val_214
+462val_462
+406val_406
+454val_454
+384val_384
+256val_256
+26val_26
+134val_134
+384val_384
+18val_18
+462val_462
+492val_492
+100val_100
+298val_298
+498val_498
+146val_146
+458val_458
+362val_362
+186val_186
+348val_348
+18val_18
+344val_344
+84val_84
+28val_28
+448val_448
+152val_152
+348val_348
+194val_194
+414val_414
+222val_222
+126val_126
+90val_90
+400val_400
+200val_200
diff --git a/sql/hive/src/test/resources/data/files/srcbucket1.txt b/sql/hive/src/test/resources/data/files/srcbucket1.txt
new file mode 100644
index 0000000000000..49bfa85498366
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/srcbucket1.txt
@@ -0,0 +1,507 @@
+281val_282
+179val_180
+291val_292
+271val_272
+217val_218
+135val_136
+167val_168
+423val_424
+413val_414
+245val_246
+455val_456
+425val_426
+241val_242
+177val_178
+231val_232
+287val_288
+31val_32
+373val_374
+447val_448
+443val_444
+175val_176
+147val_148
+249val_250
+21val_22
+273val_274
+441val_442
+371val_372
+153val_154
+217val_218
+33val_34
+35val_36
+421val_422
+243val_244
+133val_134
+333val_334
+15val_16
+391val_392
+343val_344
+275val_276
+485val_486
+293val_294
+241val_242
+85val_86
+477val_478
+455val_456
+99val_100
+335val_336
+367val_368
+59val_60
+485val_486
+393val_394
+349val_350
+11val_12
+161val_162
+123val_124
+409val_410
+265val_266
+497val_498
+63val_64
+277val_278
+135val_136
+3val_4
+101val_102
+331val_332
+209val_210
+281val_282
+239val_240
+389val_390
+235val_236
+165val_166
+11val_12
+129val_130
+257val_258
+71val_72
+289val_290
+453val_454
+421val_422
+5val_6
+249val_250
+323val_324
+467val_468
+411val_412
+175val_176
+429val_430
+281val_282
+185val_186
+147val_148
+119val_120
+473val_474
+347val_348
+213val_214
+393val_394
+427val_428
+291val_292
+65val_66
+121val_122
+375val_376
+191val_192
+129val_130
+197val_198
+491val_492
+369val_370
+351val_352
+23val_24
+349val_350
+497val_498
+487val_488
+331val_332
+409val_410
+475val_476
+463val_464
+183val_184
+177val_178
+399val_400
+15val_16
+149val_150
+77val_78
+275val_276
+87val_88
+161val_162
+75val_76
+443val_444
+407val_408
+189val_190
+305val_306
+367val_368
+349val_350
+51val_52
+355val_356
+363val_364
+19val_20
+117val_118
+47val_48
+121val_122
+241val_242
+405val_406
+239val_240
+119val_120
+11val_12
+49val_50
+143val_144
+153val_154
+341val_342
+21val_22
+105val_106
+157val_158
+199val_200
+375val_376
+89val_90
+363val_364
+395val_396
+347val_348
+77val_78
+293val_294
+89val_90
+461val_462
+313val_314
+381val_382
+385val_386
+259val_260
+69val_70
+303val_304
+245val_246
+241val_242
+469val_470
+481val_482
+303val_304
+93val_94
+277val_278
+451val_452
+53val_54
+351val_352
+443val_444
+399val_400
+205val_206
+21val_22
+317val_318
+209val_210
+429val_430
+267val_268
+257val_258
+375val_376
+489val_490
+295val_296
+105val_106
+439val_440
+457val_458
+93val_94
+119val_120
+375val_376
+391val_392
+287val_288
+375val_376
+437val_438
+35val_36
+435val_436
+29val_30
+151val_152
+491val_492
+21val_22
+295val_296
+93val_94
+61val_62
+407val_408
+439val_440
+341val_342
+335val_336
+349val_350
+371val_372
+123val_124
+355val_356
+87val_88
+427val_428
+353val_354
+261val_262
+65val_66
+371val_372
+97val_98
+495val_496
+385val_386
+49val_50
+389val_390
+259val_260
+97val_98
+125val_126
+243val_244
+151val_152
+415val_416
+401val_402
+363val_364
+117val_118
+459val_460
+137val_138
+341val_342
+379val_380
+215val_216
+157val_158
+431val_432
+407val_408
+371val_372
+309val_310
+135val_136
+161val_162
+161val_162
+337val_338
+91val_92
+1val_2
+89val_90
+457val_458
+29val_30
+285val_286
+35val_36
+227val_228
+395val_396
+311val_311
+27val_27
+165val_165
+409val_409
+255val_255
+265val_265
+193val_193
+401val_401
+273val_273
+369val_369
+213val_213
+429val_429
+469val_469
+145val_145
+495val_495
+37val_37
+327val_327
+281val_281
+277val_277
+209val_209
+15val_15
+403val_403
+417val_417
+219val_219
+287val_287
+153val_153
+193val_193
+459val_459
+237val_237
+413val_413
+207val_207
+199val_199
+399val_399
+247val_247
+417val_417
+489val_489
+377val_377
+397val_397
+309val_309
+365val_365
+439val_439
+367val_367
+325val_325
+167val_167
+195val_195
+475val_475
+17val_17
+113val_113
+155val_155
+203val_203
+339val_339
+455val_455
+311val_311
+57val_57
+205val_205
+149val_149
+345val_345
+129val_129
+489val_489
+157val_157
+221val_221
+111val_111
+47val_47
+35val_35
+427val_427
+277val_277
+399val_399
+169val_169
+125val_125
+437val_437
+469val_469
+187val_187
+459val_459
+51val_51
+103val_103
+239val_239
+213val_213
+289val_289
+221val_221
+65val_65
+311val_311
+275val_275
+137val_137
+241val_241
+83val_83
+333val_333
+181val_181
+67val_67
+489val_489
+353val_353
+373val_373
+217val_217
+411val_411
+463val_463
+431val_431
+179val_179
+129val_129
+119val_119
+197val_197
+393val_393
+199val_199
+191val_191
+165val_165
+327val_327
+205val_205
+131val_131
+51val_51
+43val_43
+469val_469
+95val_95
+481val_481
+457val_457
+197val_197
+187val_187
+409val_409
+137val_137
+369val_369
+169val_169
+413val_413
+85val_85
+77val_77
+87val_87
+179val_179
+395val_395
+419val_419
+15val_15
+307val_307
+19val_19
+435val_435
+277val_277
+273val_273
+309val_309
+389val_389
+327val_327
+369val_369
+331val_331
+401val_401
+177val_177
+5val_5
+497val_497
+317val_317
+395val_395
+35val_35
+95val_95
+11val_11
+229val_229
+233val_233
+143val_143
+195val_195
+321val_321
+119val_119
+489val_489
+41val_41
+223val_223
+149val_149
+449val_449
+453val_453
+209val_209
+69val_69
+33val_33
+103val_103
+113val_113
+367val_367
+167val_167
+219val_219
+239val_239
+485val_485
+223val_223
+263val_263
+487val_487
+401val_401
+191val_191
+5val_5
+467val_467
+229val_229
+469val_469
+463val_463
+35val_35
+283val_283
+331val_331
+235val_235
+193val_193
+321val_321
+335val_335
+175val_175
+403val_403
+483val_483
+53val_53
+105val_105
+257val_257
+409val_409
+401val_401
+203val_203
+201val_201
+217val_217
+431val_431
+125val_125
+431val_431
+187val_187
+5val_5
+397val_397
+291val_291
+351val_351
+255val_255
+163val_163
+119val_119
+491val_491
+237val_237
+439val_439
+479val_479
+305val_305
+417val_417
+199val_199
+429val_429
+169val_169
+443val_443
+323val_323
+325val_325
+277val_277
+317val_317
+333val_333
+493val_493
+207val_207
+249val_249
+265val_265
+83val_83
+353val_353
+233val_233
+133val_133
+175val_175
+189val_189
+375val_375
+401val_401
+421val_421
+407val_407
+67val_67
+379val_379
+9val_9
+341val_341
+285val_285
+167val_167
+273val_273
+183val_183
+281val_281
+97val_97
+469val_469
+315val_315
+37val_37
+307val_307
+477val_477
+169val_169
+403val_403
+97val_97
diff --git a/sql/hive/src/test/resources/data/files/srcbucket20.txt b/sql/hive/src/test/resources/data/files/srcbucket20.txt
new file mode 100644
index 0000000000000..83344151c2a15
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/srcbucket20.txt
@@ -0,0 +1,118 @@
+165val_165
+484val_484
+150val_150
+224val_224
+66val_66
+213val_213
+374val_374
+495val_495
+37val_37
+327val_327
+15val_15
+338val_338
+459val_459
+466val_466
+396val_396
+309val_309
+367val_367
+0val_0
+455val_455
+316val_316
+345val_345
+129val_129
+378val_378
+4val_4
+356val_356
+169val_169
+125val_125
+437val_437
+286val_286
+187val_187
+176val_176
+459val_459
+51val_51
+103val_103
+239val_239
+213val_213
+176val_176
+275val_275
+260val_260
+404val_404
+217val_217
+84val_84
+466val_466
+8val_8
+411val_411
+172val_172
+129val_129
+158val_158
+0val_0
+26val_26
+165val_165
+327val_327
+51val_51
+404val_404
+95val_95
+282val_282
+187val_187
+316val_316
+169val_169
+77val_77
+0val_0
+118val_118
+282val_282
+419val_419
+15val_15
+118val_118
+19val_19
+224val_224
+309val_309
+389val_389
+327val_327
+242val_242
+392val_392
+242val_242
+396val_396
+95val_95
+11val_11
+143val_143
+228val_228
+33val_33
+103val_103
+367val_367
+239val_239
+480val_480
+202val_202
+316val_316
+235val_235
+80val_80
+44val_44
+466val_466
+257val_257
+190val_190
+114val_114
+396val_396
+217val_217
+125val_125
+187val_187
+480val_480
+491val_491
+305val_305
+444val_444
+169val_169
+323val_323
+480val_480
+136val_136
+172val_172
+462val_462
+26val_26
+462val_462
+341val_341
+183val_183
+84val_84
+37val_37
+448val_448
+194val_194
+477val_477
+169val_169
+400val_400
diff --git a/sql/hive/src/test/resources/data/files/srcbucket21.txt b/sql/hive/src/test/resources/data/files/srcbucket21.txt
new file mode 100644
index 0000000000000..c0ccb30d40475
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/srcbucket21.txt
@@ -0,0 +1,120 @@
+27val_27
+409val_409
+265val_265
+401val_401
+166val_166
+430val_430
+287val_287
+207val_207
+199val_199
+247val_247
+489val_489
+162val_162
+397val_397
+342val_342
+195val_195
+155val_155
+203val_203
+339val_339
+302val_302
+438val_438
+489val_489
+221val_221
+92val_92
+111val_111
+427val_427
+382val_382
+386val_386
+430val_430
+221val_221
+137val_137
+180val_180
+12val_12
+67val_67
+489val_489
+353val_353
+272val_272
+463val_463
+119val_119
+496val_496
+393val_393
+100val_100
+298val_298
+199val_199
+191val_191
+96val_96
+481val_481
+409val_409
+470val_470
+137val_137
+85val_85
+364val_364
+306val_306
+272val_272
+331val_331
+401val_401
+452val_452
+177val_177
+5val_5
+317val_317
+34val_34
+229val_229
+195val_195
+430val_430
+119val_119
+489val_489
+78val_78
+41val_41
+492val_492
+449val_449
+218val_218
+30val_30
+74val_74
+342val_342
+368val_368
+485val_485
+70val_70
+401val_401
+191val_191
+5val_5
+438val_438
+467val_467
+229val_229
+463val_463
+283val_283
+331val_331
+335val_335
+104val_104
+409val_409
+401val_401
+258val_258
+203val_203
+12val_12
+478val_478
+298val_298
+382val_382
+5val_5
+70val_70
+397val_397
+104val_104
+70val_70
+438val_438
+119val_119
+360val_360
+199val_199
+478val_478
+317val_317
+207val_207
+265val_265
+353val_353
+214val_214
+133val_133
+375val_375
+401val_401
+67val_67
+379val_379
+492val_492
+100val_100
+298val_298
+9val_9
+126val_126
diff --git a/sql/hive/src/test/resources/data/files/srcbucket22.txt b/sql/hive/src/test/resources/data/files/srcbucket22.txt
new file mode 100644
index 0000000000000..f061ea41cc09b
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/srcbucket22.txt
@@ -0,0 +1,124 @@
+86val_86
+255val_255
+273val_273
+369val_369
+406val_406
+152val_152
+145val_145
+277val_277
+82val_82
+417val_417
+219val_219
+446val_446
+394val_394
+237val_237
+482val_482
+174val_174
+413val_413
+208val_208
+174val_174
+417val_417
+365val_365
+266val_266
+439val_439
+325val_325
+167val_167
+475val_475
+17val_17
+57val_57
+149val_149
+170val_170
+20val_20
+280val_280
+35val_35
+277val_277
+208val_208
+192val_192
+138val_138
+318val_318
+332val_332
+284val_284
+181val_181
+138val_138
+208val_208
+24val_24
+431val_431
+42val_42
+468val_468
+156val_156
+468val_468
+196val_196
+288val_288
+457val_457
+318val_318
+318val_318
+369val_369
+413val_413
+134val_134
+138val_138
+307val_307
+435val_435
+277val_277
+273val_273
+369val_369
+226val_226
+497val_497
+402val_402
+35val_35
+336val_336
+233val_233
+42val_42
+321val_321
+149val_149
+138val_138
+453val_453
+64val_64
+468val_468
+167val_167
+219val_219
+116val_116
+288val_288
+244val_244
+280val_280
+2val_2
+35val_35
+321val_321
+53val_53
+105val_105
+406val_406
+406val_406
+262val_262
+424val_424
+431val_431
+431val_431
+424val_424
+291val_291
+24val_24
+255val_255
+163val_163
+200val_200
+237val_237
+439val_439
+248val_248
+479val_479
+417val_417
+325val_325
+277val_277
+178val_178
+468val_468
+310val_310
+493val_493
+460val_460
+233val_233
+406val_406
+189val_189
+134val_134
+167val_167
+273val_273
+97val_97
+28val_28
+152val_152
+307val_307
+222val_222
+200val_200
+97val_97
diff --git a/sql/hive/src/test/resources/data/files/srcbucket23.txt b/sql/hive/src/test/resources/data/files/srcbucket23.txt
new file mode 100644
index 0000000000000..b3dca9afa45d7
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/srcbucket23.txt
@@ -0,0 +1,138 @@
+238val_238
+311val_311
+278val_278
+98val_98
+193val_193
+128val_128
+146val_146
+429val_429
+469val_469
+281val_281
+209val_209
+403val_403
+252val_252
+292val_292
+153val_153
+193val_193
+494val_494
+399val_399
+377val_377
+113val_113
+128val_128
+311val_311
+205val_205
+157val_157
+47val_47
+72val_72
+399val_399
+498val_498
+469val_469
+54val_54
+216val_216
+278val_278
+289val_289
+65val_65
+311val_311
+241val_241
+83val_83
+333val_333
+230val_230
+384val_384
+373val_373
+348val_348
+58val_58
+230val_230
+348val_348
+179val_179
+322val_322
+197val_197
+454val_454
+418val_418
+230val_230
+205val_205
+120val_120
+131val_131
+43val_43
+436val_436
+469val_469
+308val_308
+98val_98
+197val_197
+490val_490
+87val_87
+179val_179
+395val_395
+238val_238
+72val_72
+90val_90
+10val_10
+395val_395
+58val_58
+168val_168
+472val_472
+322val_322
+498val_498
+160val_160
+458val_458
+76val_76
+223val_223
+209val_209
+76val_76
+69val_69
+230val_230
+296val_296
+113val_113
+216val_216
+344val_344
+274val_274
+223val_223
+256val_256
+263val_263
+487val_487
+128val_128
+432val_432
+469val_469
+193val_193
+366val_366
+175val_175
+403val_403
+483val_483
+90val_90
+348val_348
+201val_201
+164val_164
+454val_454
+164val_164
+351val_351
+414val_414
+120val_120
+429val_429
+443val_443
+230val_230
+333val_333
+249val_249
+83val_83
+175val_175
+454val_454
+421val_421
+407val_407
+384val_384
+256val_256
+384val_384
+18val_18
+498val_498
+146val_146
+458val_458
+362val_362
+186val_186
+285val_285
+348val_348
+18val_18
+281val_281
+344val_344
+469val_469
+315val_315
+348val_348
+414val_414
+90val_90
+403val_403
diff --git a/sql/hive/src/test/resources/data/files/srcsortbucket1outof4.txt b/sql/hive/src/test/resources/data/files/srcsortbucket1outof4.txt
new file mode 100644
index 0000000000000..3d749a441bb0e
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/srcsortbucket1outof4.txt
@@ -0,0 +1,118 @@
+0val_0
+0val_0
+0val_0
+103val_103
+103val_103
+11val_11
+114val_114
+118val_118
+118val_118
+125val_125
+125val_125
+129val_129
+129val_129
+136val_136
+143val_143
+15val_15
+15val_15
+150val_150
+158val_158
+165val_165
+165val_165
+169val_169
+169val_169
+169val_169
+169val_169
+172val_172
+172val_172
+176val_176
+176val_176
+183val_183
+187val_187
+187val_187
+187val_187
+19val_19
+190val_190
+194val_194
+202val_202
+213val_213
+213val_213
+217val_217
+217val_217
+224val_224
+224val_224
+228val_228
+235val_235
+239val_239
+239val_239
+242val_242
+242val_242
+257val_257
+26val_26
+26val_26
+260val_260
+275val_275
+282val_282
+282val_282
+286val_286
+305val_305
+309val_309
+309val_309
+316val_316
+316val_316
+316val_316
+323val_323
+327val_327
+327val_327
+327val_327
+33val_33
+338val_338
+341val_341
+345val_345
+356val_356
+367val_367
+367val_367
+37val_37
+37val_37
+374val_374
+378val_378
+389val_389
+392val_392
+396val_396
+396val_396
+396val_396
+4val_4
+400val_400
+404val_404
+404val_404
+411val_411
+419val_419
+437val_437
+44val_44
+444val_444
+448val_448
+455val_455
+459val_459
+459val_459
+462val_462
+462val_462
+466val_466
+466val_466
+466val_466
+477val_477
+480val_480
+480val_480
+480val_480
+484val_484
+491val_491
+495val_495
+51val_51
+51val_51
+66val_66
+77val_77
+8val_8
+80val_80
+84val_84
+84val_84
+95val_95
+95val_95
diff --git a/sql/hive/src/test/resources/data/files/srcsortbucket2outof4.txt b/sql/hive/src/test/resources/data/files/srcsortbucket2outof4.txt
new file mode 100644
index 0000000000000..0fa67279aacb6
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/srcsortbucket2outof4.txt
@@ -0,0 +1,120 @@
+100val_100
+100val_100
+104val_104
+104val_104
+111val_111
+119val_119
+119val_119
+119val_119
+12val_12
+12val_12
+126val_126
+133val_133
+137val_137
+137val_137
+155val_155
+162val_162
+166val_166
+177val_177
+180val_180
+191val_191
+191val_191
+195val_195
+195val_195
+199val_199
+199val_199
+199val_199
+203val_203
+203val_203
+207val_207
+207val_207
+214val_214
+218val_218
+221val_221
+221val_221
+229val_229
+229val_229
+247val_247
+258val_258
+265val_265
+265val_265
+27val_27
+272val_272
+272val_272
+283val_283
+287val_287
+298val_298
+298val_298
+298val_298
+30val_30
+302val_302
+306val_306
+317val_317
+317val_317
+331val_331
+331val_331
+335val_335
+339val_339
+34val_34
+342val_342
+342val_342
+353val_353
+353val_353
+360val_360
+364val_364
+368val_368
+375val_375
+379val_379
+382val_382
+382val_382
+386val_386
+393val_393
+397val_397
+397val_397
+401val_401
+401val_401
+401val_401
+401val_401
+401val_401
+409val_409
+409val_409
+409val_409
+41val_41
+427val_427
+430val_430
+430val_430
+430val_430
+438val_438
+438val_438
+438val_438
+449val_449
+452val_452
+463val_463
+463val_463
+467val_467
+470val_470
+478val_478
+478val_478
+481val_481
+485val_485
+489val_489
+489val_489
+489val_489
+489val_489
+492val_492
+492val_492
+496val_496
+5val_5
+5val_5
+5val_5
+67val_67
+67val_67
+70val_70
+70val_70
+70val_70
+74val_74
+78val_78
+85val_85
+9val_9
+92val_92
+96val_96
diff --git a/sql/hive/src/test/resources/data/files/srcsortbucket3outof4.txt b/sql/hive/src/test/resources/data/files/srcsortbucket3outof4.txt
new file mode 100644
index 0000000000000..bb8887e794192
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/srcsortbucket3outof4.txt
@@ -0,0 +1,124 @@
+105val_105
+116val_116
+134val_134
+134val_134
+138val_138
+138val_138
+138val_138
+138val_138
+145val_145
+149val_149
+149val_149
+152val_152
+152val_152
+156val_156
+163val_163
+167val_167
+167val_167
+167val_167
+17val_17
+170val_170
+174val_174
+174val_174
+178val_178
+181val_181
+189val_189
+192val_192
+196val_196
+2val_2
+20val_20
+200val_200
+200val_200
+208val_208
+208val_208
+208val_208
+219val_219
+219val_219
+222val_222
+226val_226
+233val_233
+233val_233
+237val_237
+237val_237
+24val_24
+24val_24
+244val_244
+248val_248
+255val_255
+255val_255
+262val_262
+266val_266
+273val_273
+273val_273
+273val_273
+277val_277
+277val_277
+277val_277
+277val_277
+28val_28
+280val_280
+280val_280
+284val_284
+288val_288
+288val_288
+291val_291
+307val_307
+307val_307
+310val_310
+318val_318
+318val_318
+318val_318
+321val_321
+321val_321
+325val_325
+325val_325
+332val_332
+336val_336
+35val_35
+35val_35
+35val_35
+365val_365
+369val_369
+369val_369
+369val_369
+394val_394
+402val_402
+406val_406
+406val_406
+406val_406
+406val_406
+413val_413
+413val_413
+417val_417
+417val_417
+417val_417
+42val_42
+42val_42
+424val_424
+424val_424
+431val_431
+431val_431
+431val_431
+435val_435
+439val_439
+439val_439
+446val_446
+453val_453
+457val_457
+460val_460
+468val_468
+468val_468
+468val_468
+468val_468
+475val_475
+479val_479
+482val_482
+493val_493
+497val_497
+53val_53
+57val_57
+64val_64
+82val_82
+86val_86
+97val_97
+97val_97
diff --git a/sql/hive/src/test/resources/data/files/srcsortbucket4outof4.txt b/sql/hive/src/test/resources/data/files/srcsortbucket4outof4.txt
new file mode 100644
index 0000000000000..400ad1f447b6d
--- /dev/null
+++ b/sql/hive/src/test/resources/data/files/srcsortbucket4outof4.txt
@@ -0,0 +1,138 @@
+10val_10
+113val_113
+113val_113
+120val_120
+120val_120
+128val_128
+128val_128
+128val_128
+131val_131
+146val_146
+146val_146
+153val_153
+157val_157
+160val_160
+164val_164
+164val_164
+168val_168
+175val_175
+175val_175
+179val_179
+179val_179
+18val_18
+18val_18
+186val_186
+193val_193
+193val_193
+193val_193
+197val_197
+197val_197
+201val_201
+205val_205
+205val_205
+209val_209
+209val_209
+216val_216
+216val_216
+223val_223
+223val_223
+230val_230
+230val_230
+230val_230
+230val_230
+230val_230
+238val_238
+238val_238
+241val_241
+249val_249
+252val_252
+256val_256
+256val_256
+263val_263
+274val_274
+278val_278
+278val_278
+281val_281
+281val_281
+285val_285
+289val_289
+292val_292
+296val_296
+308val_308
+311val_311
+311val_311
+311val_311
+315val_315
+322val_322
+322val_322
+333val_333
+333val_333
+344val_344
+344val_344
+348val_348
+348val_348
+348val_348
+348val_348
+348val_348
+351val_351
+362val_362
+366val_366
+373val_373
+377val_377
+384val_384
+384val_384
+384val_384
+395val_395
+395val_395
+399val_399
+399val_399
+403val_403
+403val_403
+403val_403
+407val_407
+414val_414
+414val_414
+418val_418
+421val_421
+429val_429
+429val_429
+43val_43
+432val_432
+436val_436
+443val_443
+454val_454
+454val_454
+454val_454
+458val_458
+458val_458
+469val_469
+469val_469
+469val_469
+469val_469
+469val_469
+47val_47
+472val_472
+483val_483
+487val_487
+490val_490
+494val_494
+498val_498
+498val_498
+498val_498
+54val_54
+58val_58
+58val_58
+65val_65
+69val_69
+72val_72
+72val_72
+76val_76
+76val_76
+83val_83
+83val_83
+87val_87
+90val_90
+90val_90
+90val_90
+98val_98
+98val_98
diff --git a/sql/hive/src/test/resources/data/files/string.txt b/sql/hive/src/test/resources/data/files/string.txt
new file mode 100644
index 0000000000000..d5b7465e04a2a
Binary files /dev/null and b/sql/hive/src/test/resources/data/files/string.txt differ
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.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.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/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