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/bin/compute-classpath.sh b/bin/compute-classpath.sh
index d6f1ff9084c6f..bef42df71ce01 100755
--- a/bin/compute-classpath.sh
+++ b/bin/compute-classpath.sh
@@ -36,7 +36,6 @@ CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf"
# Hopefully we will find a way to avoid uber-jars entirely and deploy only the needed packages in
# the future.
if [ -f "$FWDIR"/sql/hive/target/scala-$SCALA_VERSION/spark-hive-assembly-*.jar ]; then
- echo "Hive assembly found, including hive support. If this isn't desired run sbt hive/clean."
# Datanucleus jars do not work if only included in the uberjar as plugin.xml metadata is lost.
DATANUCLEUSJARS=$(JARS=("$FWDIR/lib_managed/jars"/datanucleus-*.jar); IFS=:; echo "${JARS[*]}")
diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala
index d35f2c6b38373..905328ba6b5a4 100644
--- a/core/src/main/scala/org/apache/spark/Dependency.scala
+++ b/core/src/main/scala/org/apache/spark/Dependency.scala
@@ -44,8 +44,9 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) {
* Represents a dependency on the output of a shuffle stage.
* @param rdd the parent RDD
* @param partitioner partitioner used to partition the shuffle output
- * @param serializer [[Serializer]] to use. If set to null, the default serializer, as specified
- * by `spark.serializer` config option, will be used.
+ * @param serializer [[org.apache.spark.serializer.Serializer Serializer]] to use. If set to null,
+ * the default serializer, as specified by `spark.serializer` config option, will
+ * be used.
*/
class ShuffleDependency[K, V](
@transient rdd: RDD[_ <: Product2[K, V]],
diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala
index 689180fcd719b..fe84b812ba8d0 100644
--- a/core/src/main/scala/org/apache/spark/SparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/SparkContext.scala
@@ -35,6 +35,7 @@ import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHad
import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat}
import org.apache.mesos.MesosNativeLibrary
+import org.apache.spark.broadcast.Broadcast
import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil}
import org.apache.spark.partial.{ApproximateEvaluator, PartialResult}
import org.apache.spark.rdd._
@@ -232,7 +233,7 @@ class SparkContext(
postEnvironmentUpdate()
/** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */
- val hadoopConfiguration = {
+ val hadoopConfiguration: Configuration = {
val env = SparkEnv.get
val hadoopConf = SparkHadoopUtil.get.newConfiguration()
// Explicitly check for S3 environment variables
@@ -632,7 +633,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)
}
@@ -772,7 +773,7 @@ class SparkContext(
* filesystems), an HTTP, HTTPS or FTP URI, or local:/path for a file on every worker node.
*/
def addJar(path: String) {
- if (path == null) {
+ if (path == null) {
logWarning("null specified as parameter to addJar")
} else {
var key = ""
@@ -883,7 +884,8 @@ class SparkContext(
* has overridden the call site, this will return the user's version.
*/
private[spark] def getCallSite(): String = {
- Option(getLocalProperty("externalCallSite")).getOrElse(Utils.formatCallSiteInfo())
+ val defaultCallSite = Utils.getCallSiteInfo
+ Option(getLocalProperty("externalCallSite")).getOrElse(defaultCallSite.toString)
}
/**
@@ -1131,7 +1133,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](
@@ -1168,27 +1170,33 @@ object SparkContext extends Logging {
}
// Helper objects for converting common types to Writable
- private def simpleWritableConverter[T, W <: Writable: ClassTag](convert: W => T) = {
+ private def simpleWritableConverter[T, W <: Writable: ClassTag](convert: W => T)
+ : WritableConverter[T] = {
val wClass = classTag[W].runtimeClass.asInstanceOf[Class[W]]
new WritableConverter[T](_ => wClass, x => convert(x.asInstanceOf[W]))
}
- implicit def intWritableConverter() = simpleWritableConverter[Int, IntWritable](_.get)
+ implicit def intWritableConverter(): WritableConverter[Int] =
+ simpleWritableConverter[Int, IntWritable](_.get)
- implicit def longWritableConverter() = simpleWritableConverter[Long, LongWritable](_.get)
+ implicit def longWritableConverter(): WritableConverter[Long] =
+ simpleWritableConverter[Long, LongWritable](_.get)
- implicit def doubleWritableConverter() = simpleWritableConverter[Double, DoubleWritable](_.get)
+ implicit def doubleWritableConverter(): WritableConverter[Double] =
+ simpleWritableConverter[Double, DoubleWritable](_.get)
- implicit def floatWritableConverter() = simpleWritableConverter[Float, FloatWritable](_.get)
+ implicit def floatWritableConverter(): WritableConverter[Float] =
+ simpleWritableConverter[Float, FloatWritable](_.get)
- implicit def booleanWritableConverter() =
+ implicit def booleanWritableConverter(): WritableConverter[Boolean] =
simpleWritableConverter[Boolean, BooleanWritable](_.get)
- implicit def bytesWritableConverter() = {
+ implicit def bytesWritableConverter(): WritableConverter[Array[Byte]] = {
simpleWritableConverter[Array[Byte], BytesWritable](_.getBytes)
}
- implicit def stringWritableConverter() = simpleWritableConverter[String, Text](_.toString)
+ implicit def stringWritableConverter(): WritableConverter[String] =
+ simpleWritableConverter[String, Text](_.toString)
implicit def writableWritableConverter[T <: Writable]() =
new WritableConverter[T](_.runtimeClass.asInstanceOf[Class[T]], _.asInstanceOf[T])
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
index 05b89b985736d..e03b8e78d5f52 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
@@ -391,19 +391,24 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
/**
* Save this RDD as a text file, using string representations of elements.
*/
- def saveAsTextFile(path: String) = rdd.saveAsTextFile(path)
+ def saveAsTextFile(path: String): Unit = {
+ rdd.saveAsTextFile(path)
+ }
/**
* Save this RDD as a compressed text file, using string representations of elements.
*/
- def saveAsTextFile(path: String, codec: Class[_ <: CompressionCodec]) =
+ def saveAsTextFile(path: String, codec: Class[_ <: CompressionCodec]): Unit = {
rdd.saveAsTextFile(path, codec)
+ }
/**
* Save this RDD as a SequenceFile of serialized objects.
*/
- def saveAsObjectFile(path: String) = rdd.saveAsObjectFile(path)
+ def saveAsObjectFile(path: String): Unit = {
+ rdd.saveAsObjectFile(path)
+ }
/**
* Creates tuples of the elements in this RDD by applying `f`.
@@ -420,7 +425,9 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
* executed on this RDD. It is strongly recommended that this RDD is persisted in
* memory, otherwise saving it on a file will require recomputation.
*/
- def checkpoint() = rdd.checkpoint()
+ def checkpoint(): Unit = {
+ rdd.checkpoint()
+ }
/**
* Return whether this RDD has been checkpointed or not
@@ -481,7 +488,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
/**
* Returns the maximum element from this RDD as defined by the specified
* Comparator[T].
- * @params comp the comparator that defines ordering
+ * @param comp the comparator that defines ordering
* @return the maximum of the RDD
* */
def max(comp: Comparator[T]): T = {
@@ -491,7 +498,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
/**
* Returns the minimum element from this RDD as defined by the specified
* Comparator[T].
- * @params comp the comparator that defines ordering
+ * @param comp the comparator that defines ordering
* @return the minimum of the RDD
* */
def min(comp: Comparator[T]): T = {
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
index 35508b6e5acba..e531a57aced31 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala
@@ -463,7 +463,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
sc.setCheckpointDir(dir)
}
- def getCheckpointDir = JavaUtils.optionToOptional(sc.getCheckpointDir)
+ def getCheckpointDir: Optional[String] = JavaUtils.optionToOptional(sc.getCheckpointDir)
protected def checkpointFile[T](path: String): JavaRDD[T] = {
implicit val ctag: ClassTag[T] = fakeClassTag
diff --git a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala
index 00f5cd54ad650..c07838f798799 100644
--- a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala
@@ -112,5 +112,5 @@ private[spark] class ClientArguments(args: Array[String]) {
}
object ClientArguments {
- def isValidJarUrl(s: String) = s.matches("(.+):(.+)jar")
+ def isValidJarUrl(s: String): Boolean = s.matches("(.+):(.+)jar")
}
diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
index d2d8d6d662d55..9bdbfb33bf54f 100644
--- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala
@@ -32,7 +32,7 @@ import scala.collection.JavaConversions._
* Contains util methods to interact with Hadoop from Spark.
*/
class SparkHadoopUtil {
- val conf = newConfiguration()
+ val conf: Configuration = newConfiguration()
UserGroupInformation.setConfiguration(conf)
def runAsUser(user: String)(func: () => Unit) {
diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
index 5413ff671ad8d..834dfedee52ce 100644
--- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala
@@ -20,6 +20,7 @@ package org.apache.spark.deploy.master
import scala.collection.JavaConversions._
import akka.serialization.Serialization
+import org.apache.curator.framework.CuratorFramework
import org.apache.zookeeper.CreateMode
import org.apache.spark.{Logging, SparkConf}
@@ -29,7 +30,7 @@ class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf)
with Logging
{
val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/master_status"
- val zk = SparkCuratorUtil.newClient(conf)
+ val zk: CuratorFramework = SparkCuratorUtil.newClient(conf)
SparkCuratorUtil.mkdir(zk, WORKING_DIR)
diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala
index 4d2ffc54d8983..64eac73605388 100644
--- a/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala
@@ -38,7 +38,7 @@ class ConsoleSink(val property: Properties, val registry: MetricRegistry,
case None => CONSOLE_DEFAULT_PERIOD
}
- val pollUnit = Option(property.getProperty(CONSOLE_KEY_UNIT)) match {
+ val pollUnit: TimeUnit = Option(property.getProperty(CONSOLE_KEY_UNIT)) match {
case Some(s) => TimeUnit.valueOf(s.toUpperCase())
case None => TimeUnit.valueOf(CONSOLE_DEFAULT_UNIT)
}
diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala
index 319f40815d65f..544848d4150b6 100644
--- a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala
@@ -41,7 +41,7 @@ class CsvSink(val property: Properties, val registry: MetricRegistry,
case None => CSV_DEFAULT_PERIOD
}
- val pollUnit = Option(property.getProperty(CSV_KEY_UNIT)) match {
+ val pollUnit: TimeUnit = Option(property.getProperty(CSV_KEY_UNIT)) match {
case Some(s) => TimeUnit.valueOf(s.toUpperCase())
case None => TimeUnit.valueOf(CSV_DEFAULT_UNIT)
}
diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala
index 0ffdf3846dc4a..7f0a2fd16fa99 100644
--- a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala
+++ b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala
@@ -39,7 +39,7 @@ class GraphiteSink(val property: Properties, val registry: MetricRegistry,
val GRAPHITE_KEY_UNIT = "unit"
val GRAPHITE_KEY_PREFIX = "prefix"
- def propertyToOption(prop: String) = Option(property.getProperty(prop))
+ def propertyToOption(prop: String): Option[String] = Option(property.getProperty(prop))
if (!propertyToOption(GRAPHITE_KEY_HOST).isDefined) {
throw new Exception("Graphite sink requires 'host' property.")
@@ -57,7 +57,7 @@ class GraphiteSink(val property: Properties, val registry: MetricRegistry,
case None => GRAPHITE_DEFAULT_PERIOD
}
- val pollUnit = propertyToOption(GRAPHITE_KEY_UNIT) match {
+ val pollUnit: TimeUnit = propertyToOption(GRAPHITE_KEY_UNIT) match {
case Some(s) => TimeUnit.valueOf(s.toUpperCase())
case None => TimeUnit.valueOf(GRAPHITE_DEFAULT_UNIT)
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
index 8561711931047..9aa454a5c8b88 100644
--- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala
@@ -103,7 +103,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part:
array
}
- override val partitioner = Some(part)
+ override val partitioner: Some[Partitioner] = Some(part)
override def compute(s: Partition, context: TaskContext): Iterator[(K, CoGroupCombiner)] = {
val sparkConf = SparkEnv.get.conf
diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
index 932ff5bf369c7..3af008bd72378 100644
--- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala
@@ -171,7 +171,7 @@ class HadoopRDD[K, V](
array
}
- override def compute(theSplit: Partition, context: TaskContext) = {
+ override def compute(theSplit: Partition, context: TaskContext): InterruptibleIterator[(K, V)] = {
val iter = new NextIterator[(K, V)] {
val split = theSplit.asInstanceOf[HadoopPartition]
diff --git a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala
index 8df8718f3b65b..1b503743ac117 100644
--- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala
@@ -116,7 +116,7 @@ class JdbcRDD[T: ClassTag](
}
object JdbcRDD {
- def resultSetToObjectArray(rs: ResultSet) = {
+ def resultSetToObjectArray(rs: ResultSet): Array[Object] = {
Array.tabulate[Object](rs.getMetaData.getColumnCount)(i => rs.getObject(i + 1))
}
}
diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
index d1fff296878c3..461a749eac48b 100644
--- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala
@@ -80,7 +80,7 @@ class NewHadoopRDD[K, V](
result
}
- override def compute(theSplit: Partition, context: TaskContext) = {
+ override def compute(theSplit: Partition, context: TaskContext): InterruptibleIterator[(K, V)] = {
val iter = new Iterator[(K, V)] {
val split = theSplit.asInstanceOf[NewHadoopPartition]
logInfo("Input split: " + split.serializableHadoopSplit)
diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
index 2b7e3d99e68cb..e5638d0132e88 100644
--- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala
+++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala
@@ -121,7 +121,7 @@ abstract class RDD[T: ClassTag](
@transient var name: String = null
/** Assign a name to this RDD */
- def setName(_name: String) = {
+ def setName(_name: String): RDD[T] = {
name = _name
this
}
@@ -929,32 +929,49 @@ abstract class RDD[T: ClassTag](
}
/**
- * Returns the top K elements from this RDD as defined by
- * the specified implicit Ordering[T].
+ * Returns the top K (largest) elements from this RDD as defined by the specified
+ * implicit Ordering[T]. This does the opposite of [[takeOrdered]]. For example:
+ * {{{
+ * sc.parallelize([10, 4, 2, 12, 3]).top(1)
+ * // returns [12]
+ *
+ * sc.parallelize([2, 3, 4, 5, 6]).top(2)
+ * // returns [6, 5]
+ * }}}
+ *
* @param num the number of top elements to return
* @param ord the implicit ordering for T
* @return an array of top elements
*/
- def top(num: Int)(implicit ord: Ordering[T]): Array[T] = {
- mapPartitions { items =>
- val queue = new BoundedPriorityQueue[T](num)
- queue ++= items
- Iterator.single(queue)
- }.reduce { (queue1, queue2) =>
- queue1 ++= queue2
- queue1
- }.toArray.sorted(ord.reverse)
- }
+ def top(num: Int)(implicit ord: Ordering[T]): Array[T] = takeOrdered(num)(ord.reverse)
/**
- * Returns the first K elements from this RDD as defined by
- * the specified implicit Ordering[T] and maintains the
- * ordering.
+ * Returns the first K (smallest) elements from this RDD as defined by the specified
+ * implicit Ordering[T] and maintains the ordering. This does the opposite of [[top]].
+ * For example:
+ * {{{
+ * sc.parallelize([10, 4, 2, 12, 3]).takeOrdered(1)
+ * // returns [12]
+ *
+ * sc.parallelize([2, 3, 4, 5, 6]).takeOrdered(2)
+ * // returns [2, 3]
+ * }}}
+ *
* @param num the number of top elements to return
* @param ord the implicit ordering for T
* @return an array of top elements
*/
- def takeOrdered(num: Int)(implicit ord: Ordering[T]): Array[T] = top(num)(ord.reverse)
+ def takeOrdered(num: Int)(implicit ord: Ordering[T]): Array[T] = {
+ mapPartitions { items =>
+ // Priority keeps the largest elements, so let's reverse the ordering.
+ val queue = new BoundedPriorityQueue[T](num)(ord.reverse)
+ queue ++= util.collection.Utils.takeOrdered(items, num)(ord)
+ Iterator.single(queue)
+ }.reduce { (queue1, queue2) =>
+ queue1 ++= queue2
+ queue1
+ }.toArray.sorted(ord)
+ }
/**
* Returns the max of this RDD as defined by the implicit Ordering[T].
@@ -1043,7 +1060,7 @@ abstract class RDD[T: ClassTag](
/** User code that created this RDD (e.g. `textFile`, `parallelize`). */
@transient private[spark] val creationSiteInfo = Utils.getCallSiteInfo
- private[spark] def getCreationSite = Utils.formatCallSiteInfo(creationSiteInfo)
+ private[spark] def getCreationSite: String = creationSiteInfo.toString
private[spark] def elementClassTag: ClassTag[T] = classTag[T]
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
index 30bceb47b9e7d..a92922166f595 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala
@@ -62,6 +62,9 @@ private[spark] class TaskSchedulerImpl(
// Threshold above which we warn user initial TaskSet may be starved
val STARVATION_TIMEOUT = conf.getLong("spark.starvation.timeout", 15000)
+ // CPUs to request per task
+ val CPUS_PER_TASK = conf.getInt("spark.task.cpus", 1)
+
// TaskSetManagers are not thread safe, so any access to one should be synchronized
// on this class.
val activeTaskSets = new HashMap[String, TaskSetManager]
@@ -228,16 +231,18 @@ private[spark] class TaskSchedulerImpl(
for (i <- 0 until shuffledOffers.size) {
val execId = shuffledOffers(i).executorId
val host = shuffledOffers(i).host
- for (task <- taskSet.resourceOffer(execId, host, availableCpus(i), maxLocality)) {
- tasks(i) += task
- val tid = task.taskId
- taskIdToTaskSetId(tid) = taskSet.taskSet.id
- taskIdToExecutorId(tid) = execId
- activeExecutorIds += execId
- executorsByHost(host) += execId
- availableCpus(i) -= taskSet.CPUS_PER_TASK
- assert (availableCpus(i) >= 0)
- launchedTask = true
+ if (availableCpus(i) >= CPUS_PER_TASK) {
+ for (task <- taskSet.resourceOffer(execId, host, maxLocality)) {
+ tasks(i) += task
+ val tid = task.taskId
+ taskIdToTaskSetId(tid) = taskSet.taskSet.id
+ taskIdToExecutorId(tid) = execId
+ activeExecutorIds += execId
+ executorsByHost(host) += execId
+ availableCpus(i) -= CPUS_PER_TASK
+ assert (availableCpus(i) >= 0)
+ launchedTask = true
+ }
}
}
} while (launchedTask)
diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
index a73343c1c0826..86d2050a03f18 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala
@@ -56,9 +56,6 @@ private[spark] class TaskSetManager(
{
val conf = sched.sc.conf
- // CPUs to request per task
- val CPUS_PER_TASK = conf.getInt("spark.task.cpus", 1)
-
/*
* Sometimes if an executor is dead or in an otherwise invalid state, the driver
* does not realize right away leading to repeated task failures. If enabled,
@@ -384,11 +381,10 @@ private[spark] class TaskSetManager(
def resourceOffer(
execId: String,
host: String,
- availableCpus: Int,
maxLocality: TaskLocality.TaskLocality)
: Option[TaskDescription] =
{
- if (!isZombie && availableCpus >= CPUS_PER_TASK) {
+ if (!isZombie) {
val curTime = clock.getTime()
var allowedLocality = getAllowedLocalityLevel(curTime)
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
index fad03731572e7..990e01a3e7959 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala
@@ -89,7 +89,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A
scheduler.statusUpdate(taskId, state, data.value)
if (TaskState.isFinished(state)) {
if (executorActor.contains(executorId)) {
- freeCores(executorId) += 1
+ freeCores(executorId) += scheduler.CPUS_PER_TASK
makeOffers(executorId)
} else {
// Ignoring the update since we don't know about the executor.
@@ -140,7 +140,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A
// Launch tasks returned by a set of resource offers
def launchTasks(tasks: Seq[Seq[TaskDescription]]) {
for (task <- tasks.flatten) {
- freeCores(task.executorId) -= 1
+ freeCores(task.executorId) -= scheduler.CPUS_PER_TASK
executorActor(task.executorId) ! LaunchTask(task)
}
}
diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
index 4092dd04b112b..dfdcafe19fb93 100644
--- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
+++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala
@@ -246,7 +246,7 @@ private[spark] class MesosSchedulerBackend(
val cpuResource = Resource.newBuilder()
.setName("cpus")
.setType(Value.Type.SCALAR)
- .setScalar(Value.Scalar.newBuilder().setValue(1).build())
+ .setScalar(Value.Scalar.newBuilder().setValue(scheduler.CPUS_PER_TASK).build())
.build()
MesosTaskInfo.newBuilder()
.setTaskId(taskId)
diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala
index 1b7934d59fa1d..4212a539dab4b 100644
--- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala
+++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala
@@ -126,15 +126,16 @@ object StorageLevel {
val MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, 2)
/** Create a new StorageLevel object */
- def apply(useDisk: Boolean, useMemory: Boolean, deserialized: Boolean, replication: Int = 1) =
+ def apply(useDisk: Boolean, useMemory: Boolean, deserialized: Boolean,
+ replication: Int = 1): StorageLevel =
getCachedStorageLevel(new StorageLevel(useDisk, useMemory, deserialized, replication))
/** Create a new StorageLevel object from its integer representation */
- def apply(flags: Int, replication: Int) =
+ def apply(flags: Int, replication: Int): StorageLevel =
getCachedStorageLevel(new StorageLevel(flags, replication))
/** Read StorageLevel object from ObjectInput stream */
- def apply(in: ObjectInput) = {
+ def apply(in: ObjectInput): StorageLevel = {
val obj = new StorageLevel()
obj.readExternal(in)
getCachedStorageLevel(obj)
diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
index fd638c83aac6e..ef1ad872c8ef7 100644
--- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
+++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala
@@ -47,7 +47,8 @@ private[spark] class SparkUI(
val securityManager = if (live) sc.env.securityManager else new SecurityManager(conf)
- private val host = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(Utils.localHostName())
+ private val bindHost = Utils.localHostName()
+ private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(bindHost)
private val port = conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt
private var serverInfo: Option[ServerInfo] = None
@@ -79,8 +80,8 @@ private[spark] class SparkUI(
/** Bind the HTTP server which backs this web interface */
def bind() {
try {
- serverInfo = Some(startJettyServer(host, port, handlers, sc.conf))
- logInfo("Started Spark Web UI at http://%s:%d".format(host, boundPort))
+ serverInfo = Some(startJettyServer(bindHost, port, handlers, sc.conf))
+ logInfo("Started Spark Web UI at http://%s:%d".format(publicHost, boundPort))
} catch {
case e: Exception =>
logError("Failed to create Spark JettyUtils", e)
@@ -111,7 +112,7 @@ private[spark] class SparkUI(
logInfo("Stopped Spark Web UI at %s".format(appUIAddress))
}
- private[spark] def appUIAddress = "http://" + host + ":" + boundPort
+ private[spark] def appUIAddress = "http://" + publicHost + ":" + boundPort
}
diff --git a/core/src/main/scala/org/apache/spark/util/Distribution.scala b/core/src/main/scala/org/apache/spark/util/Distribution.scala
index ab738c4b868fa..5b347555fe708 100644
--- a/core/src/main/scala/org/apache/spark/util/Distribution.scala
+++ b/core/src/main/scala/org/apache/spark/util/Distribution.scala
@@ -19,6 +19,8 @@ package org.apache.spark.util
import java.io.PrintStream
+import scala.collection.immutable.IndexedSeq
+
/**
* Util for getting some stats from a small sample of numeric values, with some handy
* summary functions.
@@ -40,7 +42,8 @@ class Distribution(val data: Array[Double], val startIdx: Int, val endIdx: Int)
* given from 0 to 1
* @param probabilities
*/
- def getQuantiles(probabilities: Traversable[Double] = defaultProbabilities) = {
+ def getQuantiles(probabilities: Traversable[Double] = defaultProbabilities)
+ : IndexedSeq[Double] = {
probabilities.toIndexedSeq.map{p:Double => data(closestIndex(p))}
}
@@ -48,7 +51,7 @@ class Distribution(val data: Array[Double], val startIdx: Int, val endIdx: Int)
math.min((p * length).toInt + startIdx, endIdx - 1)
}
- def showQuantiles(out: PrintStream = System.out) = {
+ def showQuantiles(out: PrintStream = System.out): Unit = {
out.println("min\t25%\t50%\t75%\tmax")
getQuantiles(defaultProbabilities).foreach{q => out.print(q + "\t")}
out.println
diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala
index e541591ee7582..0b07bdcf63b97 100644
--- a/core/src/main/scala/org/apache/spark/util/Utils.scala
+++ b/core/src/main/scala/org/apache/spark/util/Utils.scala
@@ -679,7 +679,13 @@ private[spark] object Utils extends Logging {
private val SPARK_CLASS_REGEX = """^org\.apache\.spark(\.api\.java)?(\.util)?(\.rdd)?\.[A-Z]""".r
private[spark] class CallSiteInfo(val lastSparkMethod: String, val firstUserFile: String,
- val firstUserLine: Int, val firstUserClass: String)
+ val firstUserLine: Int, val firstUserClass: String) {
+
+ /** Returns a printable version of the call site info suitable for logs. */
+ override def toString = {
+ "%s at %s:%s".format(lastSparkMethod, firstUserFile, firstUserLine)
+ }
+ }
/**
* When called inside a class in the spark package, returns the name of the user code class
@@ -687,8 +693,8 @@ private[spark] object Utils extends Logging {
* This is used, for example, to tell users where in their code each RDD got created.
*/
def getCallSiteInfo: CallSiteInfo = {
- val trace = Thread.currentThread.getStackTrace().filter( el =>
- (!el.getMethodName.contains("getStackTrace")))
+ val trace = Thread.currentThread.getStackTrace()
+ .filterNot(_.getMethodName.contains("getStackTrace"))
// Keep crawling up the stack trace until we find the first function not inside of the spark
// package. We track the last (shallowest) contiguous Spark method. This might be an RDD
@@ -721,12 +727,6 @@ private[spark] object Utils extends Logging {
new CallSiteInfo(lastSparkMethod, firstUserFile, firstUserLine, firstUserClass)
}
- /** Returns a printable version of the call site info suitable for logs. */
- def formatCallSiteInfo(callSiteInfo: CallSiteInfo = Utils.getCallSiteInfo) = {
- "%s at %s:%s".format(callSiteInfo.lastSparkMethod, callSiteInfo.firstUserFile,
- callSiteInfo.firstUserLine)
- }
-
/** Return a string containing part of a file from byte 'start' to 'end'. */
def offsetBytes(path: String, start: Long, end: Long): String = {
val file = new File(path)
diff --git a/core/src/main/scala/org/apache/spark/util/collection/Utils.scala b/core/src/main/scala/org/apache/spark/util/collection/Utils.scala
new file mode 100644
index 0000000000000..c5268c0fae0ef
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/util/collection/Utils.scala
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.util.collection
+
+import scala.collection.JavaConversions.{collectionAsScalaIterable, asJavaIterator}
+
+import com.google.common.collect.{Ordering => GuavaOrdering}
+
+/**
+ * Utility functions for collections.
+ */
+private[spark] object Utils {
+
+ /**
+ * Returns the first K elements from the input as defined by the specified implicit Ordering[T]
+ * and maintains the ordering.
+ */
+ def takeOrdered[T](input: Iterator[T], num: Int)(implicit ord: Ordering[T]): Iterator[T] = {
+ val ordering = new GuavaOrdering[T] {
+ override def compare(l: T, r: T) = ord.compare(l, r)
+ }
+ collectionAsScalaIterable(ordering.leastOf(asJavaIterator(input), num)).iterator
+ }
+}
diff --git a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala
index 5cb49d9a7fce1..cd3887dcc7371 100644
--- a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala
@@ -17,7 +17,7 @@
package org.apache.spark
-import org.scalatest.FunSuite
+import org.scalatest.{Assertions, FunSuite}
class SparkContextInfoSuite extends FunSuite with LocalSparkContext {
test("getPersistentRDDs only returns RDDs that are marked as cached") {
@@ -56,4 +56,38 @@ class SparkContextInfoSuite extends FunSuite with LocalSparkContext {
rdd.collect()
assert(sc.getRDDStorageInfo.size === 1)
}
+
+ test("call sites report correct locations") {
+ sc = new SparkContext("local", "test")
+ testPackage.runCallSiteTest(sc)
+ }
+}
+
+/** Call site must be outside of usual org.apache.spark packages (see Utils#SPARK_CLASS_REGEX). */
+package object testPackage extends Assertions {
+ private val CALL_SITE_REGEX = "(.+) at (.+):([0-9]+)".r
+
+ def runCallSiteTest(sc: SparkContext) {
+ val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2)
+ val rddCreationSite = rdd.getCreationSite
+ val curCallSite = sc.getCallSite() // note: 2 lines after definition of "rdd"
+
+ val rddCreationLine = rddCreationSite match {
+ case CALL_SITE_REGEX(func, file, line) => {
+ assert(func === "makeRDD")
+ assert(file === "SparkContextInfoSuite.scala")
+ line.toInt
+ }
+ case _ => fail("Did not match expected call site format")
+ }
+
+ curCallSite match {
+ case CALL_SITE_REGEX(func, file, line) => {
+ assert(func === "getCallSite") // this is correct because we called it from outside of Spark
+ assert(file === "SparkContextInfoSuite.scala")
+ assert(line.toInt === rddCreationLine.toInt + 2)
+ }
+ case _ => fail("Did not match expected call site format")
+ }
+ }
}
diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala
index 9274e01632d58..356e28dd19bc5 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala
@@ -80,7 +80,6 @@ class FakeTaskSetManager(
override def resourceOffer(
execId: String,
host: String,
- availableCpus: Int,
maxLocality: TaskLocality.TaskLocality)
: Option[TaskDescription] =
{
@@ -125,7 +124,7 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin
manager.parent.name, manager.parent.runningTasks, manager.name, manager.runningTasks))
}
for (taskSet <- taskSetQueue) {
- taskSet.resourceOffer("execId_1", "hostname_1", 1, TaskLocality.ANY) match {
+ taskSet.resourceOffer("execId_1", "hostname_1", TaskLocality.ANY) match {
case Some(task) =>
return taskSet.stageId
case None => {}
@@ -293,4 +292,43 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin
assert(count > 0)
assert(count < numTrials)
}
+
+ test("Scheduler correctly accounts for multiple CPUs per task") {
+ sc = new SparkContext("local", "TaskSchedulerImplSuite")
+ val taskCpus = 2
+
+ sc.conf.set("spark.task.cpus", taskCpus.toString)
+ val taskScheduler = new TaskSchedulerImpl(sc)
+ taskScheduler.initialize(new FakeSchedulerBackend)
+ // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks.
+ val dagScheduler = new DAGScheduler(sc, taskScheduler) {
+ override def taskStarted(task: Task[_], taskInfo: TaskInfo) {}
+ override def executorAdded(execId: String, host: String) {}
+ }
+
+ // Give zero core offers. Should not generate any tasks
+ val zeroCoreWorkerOffers = Seq(new WorkerOffer("executor0", "host0", 0),
+ new WorkerOffer("executor1", "host1", 0))
+ val taskSet = FakeTask.createTaskSet(1)
+ taskScheduler.submitTasks(taskSet)
+ var taskDescriptions = taskScheduler.resourceOffers(zeroCoreWorkerOffers).flatten
+ assert(0 === taskDescriptions.length)
+
+ // No tasks should run as we only have 1 core free.
+ val numFreeCores = 1
+ val singleCoreWorkerOffers = Seq(new WorkerOffer("executor0", "host0", numFreeCores),
+ new WorkerOffer("executor1", "host1", numFreeCores))
+ taskScheduler.submitTasks(taskSet)
+ taskDescriptions = taskScheduler.resourceOffers(singleCoreWorkerOffers).flatten
+ assert(0 === taskDescriptions.length)
+
+ // Now change the offers to have 2 cores in one executor and verify if it
+ // is chosen.
+ val multiCoreWorkerOffers = Seq(new WorkerOffer("executor0", "host0", taskCpus),
+ new WorkerOffer("executor1", "host1", numFreeCores))
+ taskScheduler.submitTasks(taskSet)
+ taskDescriptions = taskScheduler.resourceOffers(multiCoreWorkerOffers).flatten
+ assert(1 === taskDescriptions.length)
+ assert("executor0" === taskDescriptions(0).executorId)
+ }
}
diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
index 9af5d3a303ae2..c92b6dc96c8eb 100644
--- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala
@@ -93,19 +93,16 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
val taskSet = FakeTask.createTaskSet(1)
val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES)
- // Offer a host with no CPUs
- assert(manager.resourceOffer("exec1", "host1", 0, ANY) === None)
-
// Offer a host with process-local as the constraint; this should work because the TaskSet
// above won't have any locality preferences
- val taskOption = manager.resourceOffer("exec1", "host1", 2, TaskLocality.PROCESS_LOCAL)
+ val taskOption = manager.resourceOffer("exec1", "host1", TaskLocality.PROCESS_LOCAL)
assert(taskOption.isDefined)
val task = taskOption.get
assert(task.executorId === "exec1")
assert(sched.startedTasks.contains(0))
// Re-offer the host -- now we should get no more tasks
- assert(manager.resourceOffer("exec1", "host1", 2, PROCESS_LOCAL) === None)
+ assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) === None)
// Tell it the task has finished
manager.handleSuccessfulTask(0, createTaskResult(0))
@@ -121,7 +118,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
// First three offers should all find tasks
for (i <- 0 until 3) {
- val taskOption = manager.resourceOffer("exec1", "host1", 1, PROCESS_LOCAL)
+ val taskOption = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL)
assert(taskOption.isDefined)
val task = taskOption.get
assert(task.executorId === "exec1")
@@ -129,7 +126,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
assert(sched.startedTasks.toSet === Set(0, 1, 2))
// Re-offer the host -- now we should get no more tasks
- assert(manager.resourceOffer("exec1", "host1", 1, PROCESS_LOCAL) === None)
+ assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) === None)
// Finish the first two tasks
manager.handleSuccessfulTask(0, createTaskResult(0))
@@ -157,35 +154,35 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
// First offer host1, exec1: first task should be chosen
- assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0)
+ assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0)
// Offer host1, exec1 again: the last task, which has no prefs, should be chosen
- assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 3)
+ assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 3)
// Offer host1, exec1 again, at PROCESS_LOCAL level: nothing should get chosen
- assert(manager.resourceOffer("exec1", "host1", 1, PROCESS_LOCAL) === None)
+ assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) === None)
clock.advance(LOCALITY_WAIT)
// Offer host1, exec1 again, at PROCESS_LOCAL level: nothing should get chosen
- assert(manager.resourceOffer("exec1", "host1", 1, PROCESS_LOCAL) === None)
+ assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) === None)
// Offer host1, exec1 again, at NODE_LOCAL level: we should choose task 2
- assert(manager.resourceOffer("exec1", "host1", 1, NODE_LOCAL).get.index == 2)
+ assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL).get.index == 2)
// Offer host1, exec1 again, at NODE_LOCAL level: nothing should get chosen
- assert(manager.resourceOffer("exec1", "host1", 1, NODE_LOCAL) === None)
+ assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL) === None)
// Offer host1, exec1 again, at ANY level: nothing should get chosen
- assert(manager.resourceOffer("exec1", "host1", 1, ANY) === None)
+ assert(manager.resourceOffer("exec1", "host1", ANY) === None)
clock.advance(LOCALITY_WAIT)
// Offer host1, exec1 again, at ANY level: task 1 should get chosen
- assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 1)
+ assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 1)
// Offer host1, exec1 again, at ANY level: nothing should be chosen as we've launched all tasks
- assert(manager.resourceOffer("exec1", "host1", 1, ANY) === None)
+ assert(manager.resourceOffer("exec1", "host1", ANY) === None)
}
test("delay scheduling with fallback") {
@@ -203,29 +200,29 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
// First offer host1: first task should be chosen
- assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0)
+ assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0)
// Offer host1 again: nothing should get chosen
- assert(manager.resourceOffer("exec1", "host1", 1, ANY) === None)
+ assert(manager.resourceOffer("exec1", "host1", ANY) === None)
clock.advance(LOCALITY_WAIT)
// Offer host1 again: second task (on host2) should get chosen
- assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 1)
+ assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 1)
// Offer host1 again: third task (on host2) should get chosen
- assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 2)
+ assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 2)
// Offer host2: fifth task (also on host2) should get chosen
- assert(manager.resourceOffer("exec2", "host2", 1, ANY).get.index === 4)
+ assert(manager.resourceOffer("exec2", "host2", ANY).get.index === 4)
// Now that we've launched a local task, we should no longer launch the task for host3
- assert(manager.resourceOffer("exec2", "host2", 1, ANY) === None)
+ assert(manager.resourceOffer("exec2", "host2", ANY) === None)
clock.advance(LOCALITY_WAIT)
// After another delay, we can go ahead and launch that task non-locally
- assert(manager.resourceOffer("exec2", "host2", 1, ANY).get.index === 3)
+ assert(manager.resourceOffer("exec2", "host2", ANY).get.index === 3)
}
test("delay scheduling with failed hosts") {
@@ -240,24 +237,24 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
// First offer host1: first task should be chosen
- assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0)
+ assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0)
// Offer host1 again: third task should be chosen immediately because host3 is not up
- assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 2)
+ assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 2)
// After this, nothing should get chosen
- assert(manager.resourceOffer("exec1", "host1", 1, ANY) === None)
+ assert(manager.resourceOffer("exec1", "host1", ANY) === None)
// Now mark host2 as dead
sched.removeExecutor("exec2")
manager.executorLost("exec2", "host2")
// Task 1 should immediately be launched on host1 because its original host is gone
- assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 1)
+ assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 1)
// Now that all tasks have launched, nothing new should be launched anywhere else
- assert(manager.resourceOffer("exec1", "host1", 1, ANY) === None)
- assert(manager.resourceOffer("exec2", "host2", 1, ANY) === None)
+ assert(manager.resourceOffer("exec1", "host1", ANY) === None)
+ assert(manager.resourceOffer("exec2", "host2", ANY) === None)
}
test("task result lost") {
@@ -267,14 +264,14 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
val clock = new FakeClock
val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock)
- assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0)
+ assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0)
// Tell it the task has finished but the result was lost.
manager.handleFailedTask(0, TaskState.FINISHED, TaskResultLost)
assert(sched.endedTasks(0) === TaskResultLost)
// Re-offer the host -- now we should get task 0 again.
- assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0)
+ assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0)
}
test("repeated failures lead to task set abortion") {
@@ -287,7 +284,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
// Fail the task MAX_TASK_FAILURES times, and check that the task set is aborted
// after the last failure.
(1 to manager.maxTaskFailures).foreach { index =>
- val offerResult = manager.resourceOffer("exec1", "host1", 1, ANY)
+ val offerResult = manager.resourceOffer("exec1", "host1", ANY)
assert(offerResult.isDefined,
"Expect resource offer on iteration %s to return a task".format(index))
assert(offerResult.get.index === 0)
@@ -317,7 +314,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
val manager = new TaskSetManager(sched, taskSet, 4, clock)
{
- val offerResult = manager.resourceOffer("exec1", "host1", 1, TaskLocality.PROCESS_LOCAL)
+ val offerResult = manager.resourceOffer("exec1", "host1", TaskLocality.PROCESS_LOCAL)
assert(offerResult.isDefined, "Expect resource offer to return a task")
assert(offerResult.get.index === 0)
@@ -328,15 +325,15 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
assert(!sched.taskSetsFailed.contains(taskSet.id))
// Ensure scheduling on exec1 fails after failure 1 due to blacklist
- assert(manager.resourceOffer("exec1", "host1", 1, TaskLocality.PROCESS_LOCAL).isEmpty)
- assert(manager.resourceOffer("exec1", "host1", 1, TaskLocality.NODE_LOCAL).isEmpty)
- assert(manager.resourceOffer("exec1", "host1", 1, TaskLocality.RACK_LOCAL).isEmpty)
- assert(manager.resourceOffer("exec1", "host1", 1, TaskLocality.ANY).isEmpty)
+ assert(manager.resourceOffer("exec1", "host1", TaskLocality.PROCESS_LOCAL).isEmpty)
+ assert(manager.resourceOffer("exec1", "host1", TaskLocality.NODE_LOCAL).isEmpty)
+ assert(manager.resourceOffer("exec1", "host1", TaskLocality.RACK_LOCAL).isEmpty)
+ assert(manager.resourceOffer("exec1", "host1", TaskLocality.ANY).isEmpty)
}
// Run the task on exec1.1 - should work, and then fail it on exec1.1
{
- val offerResult = manager.resourceOffer("exec1.1", "host1", 1, TaskLocality.NODE_LOCAL)
+ val offerResult = manager.resourceOffer("exec1.1", "host1", TaskLocality.NODE_LOCAL)
assert(offerResult.isDefined,
"Expect resource offer to return a task for exec1.1, offerResult = " + offerResult)
@@ -348,12 +345,12 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
assert(!sched.taskSetsFailed.contains(taskSet.id))
// Ensure scheduling on exec1.1 fails after failure 2 due to blacklist
- assert(manager.resourceOffer("exec1.1", "host1", 1, TaskLocality.NODE_LOCAL).isEmpty)
+ assert(manager.resourceOffer("exec1.1", "host1", TaskLocality.NODE_LOCAL).isEmpty)
}
// Run the task on exec2 - should work, and then fail it on exec2
{
- val offerResult = manager.resourceOffer("exec2", "host2", 1, TaskLocality.ANY)
+ val offerResult = manager.resourceOffer("exec2", "host2", TaskLocality.ANY)
assert(offerResult.isDefined, "Expect resource offer to return a task")
assert(offerResult.get.index === 0)
@@ -364,20 +361,20 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging {
assert(!sched.taskSetsFailed.contains(taskSet.id))
// Ensure scheduling on exec2 fails after failure 3 due to blacklist
- assert(manager.resourceOffer("exec2", "host2", 1, TaskLocality.ANY).isEmpty)
+ assert(manager.resourceOffer("exec2", "host2", TaskLocality.ANY).isEmpty)
}
// After reschedule delay, scheduling on exec1 should be possible.
clock.advance(rescheduleDelay)
{
- val offerResult = manager.resourceOffer("exec1", "host1", 1, TaskLocality.PROCESS_LOCAL)
+ val offerResult = manager.resourceOffer("exec1", "host1", TaskLocality.PROCESS_LOCAL)
assert(offerResult.isDefined, "Expect resource offer to return a task")
assert(offerResult.get.index === 0)
assert(offerResult.get.executorId === "exec1")
- assert(manager.resourceOffer("exec1", "host1", 1, TaskLocality.PROCESS_LOCAL).isEmpty)
+ assert(manager.resourceOffer("exec1", "host1", TaskLocality.PROCESS_LOCAL).isEmpty)
// Cause exec1 to fail : failure 4
manager.handleFailedTask(offerResult.get.taskId, TaskState.FINISHED, TaskResultLost)
diff --git a/docs/configuration.md b/docs/configuration.md
index 16ee5ec0f230f..1ff0150567255 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -579,6 +579,13 @@ Apart from these, the following properties are also available, and may be useful
out and giving up.
+
+ spark.task.cpus |
+ 1 |
+
+ Number of cores to allocate for each task.
+ |
+
## Viewing Spark Properties
diff --git a/extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala b/extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala
index cd37317da77de..d03d7774e8c80 100644
--- a/extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala
+++ b/extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala
@@ -23,6 +23,7 @@ import java.util.concurrent.TimeUnit
import com.codahale.metrics.MetricRegistry
import com.codahale.metrics.ganglia.GangliaReporter
import info.ganglia.gmetric4j.gmetric.GMetric
+import info.ganglia.gmetric4j.gmetric.GMetric.UDPAddressingMode
import org.apache.spark.SecurityManager
import org.apache.spark.metrics.MetricsSystem
@@ -33,10 +34,10 @@ class GangliaSink(val property: Properties, val registry: MetricRegistry,
val GANGLIA_DEFAULT_PERIOD = 10
val GANGLIA_KEY_UNIT = "unit"
- val GANGLIA_DEFAULT_UNIT = TimeUnit.SECONDS
+ val GANGLIA_DEFAULT_UNIT: TimeUnit = TimeUnit.SECONDS
val GANGLIA_KEY_MODE = "mode"
- val GANGLIA_DEFAULT_MODE = GMetric.UDPAddressingMode.MULTICAST
+ val GANGLIA_DEFAULT_MODE: UDPAddressingMode = GMetric.UDPAddressingMode.MULTICAST
// TTL for multicast messages. If listeners are X hops away in network, must be at least X.
val GANGLIA_KEY_TTL = "ttl"
@@ -45,7 +46,7 @@ class GangliaSink(val property: Properties, val registry: MetricRegistry,
val GANGLIA_KEY_HOST = "host"
val GANGLIA_KEY_PORT = "port"
- def propertyToOption(prop: String) = Option(property.getProperty(prop))
+ def propertyToOption(prop: String): Option[String] = Option(property.getProperty(prop))
if (!propertyToOption(GANGLIA_KEY_HOST).isDefined) {
throw new Exception("Ganglia sink requires 'host' property.")
@@ -58,11 +59,12 @@ class GangliaSink(val property: Properties, val registry: MetricRegistry,
val host = propertyToOption(GANGLIA_KEY_HOST).get
val port = propertyToOption(GANGLIA_KEY_PORT).get.toInt
val ttl = propertyToOption(GANGLIA_KEY_TTL).map(_.toInt).getOrElse(GANGLIA_DEFAULT_TTL)
- val mode = propertyToOption(GANGLIA_KEY_MODE)
+ val mode: UDPAddressingMode = propertyToOption(GANGLIA_KEY_MODE)
.map(u => GMetric.UDPAddressingMode.valueOf(u.toUpperCase)).getOrElse(GANGLIA_DEFAULT_MODE)
val pollPeriod = propertyToOption(GANGLIA_KEY_PERIOD).map(_.toInt)
.getOrElse(GANGLIA_DEFAULT_PERIOD)
- val pollUnit = propertyToOption(GANGLIA_KEY_UNIT).map(u => TimeUnit.valueOf(u.toUpperCase))
+ val pollUnit: TimeUnit = propertyToOption(GANGLIA_KEY_UNIT)
+ .map(u => TimeUnit.valueOf(u.toUpperCase))
.getOrElse(GANGLIA_DEFAULT_UNIT)
MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod)
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
index 65a1a8c68f6d2..ef05623d7a0a1 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala
@@ -419,5 +419,6 @@ object Graph {
* All the convenience operations are defined in the [[GraphOps]] class which may be
* shared across multiple graph implementations.
*/
- implicit def graphToGraphOps[VD: ClassTag, ED: ClassTag](g: Graph[VD, ED]) = g.ops
+ implicit def graphToGraphOps[VD: ClassTag, ED: ClassTag]
+ (g: Graph[VD, ED]): GraphOps[VD, ED] = g.ops
} // end of Graph object
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
index 5e9be18990ba3..43ac11d8957f6 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala
@@ -197,7 +197,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected (
override def mapReduceTriplets[A: ClassTag](
mapFunc: EdgeTriplet[VD, ED] => Iterator[(VertexId, A)],
reduceFunc: (A, A) => A,
- activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None) = {
+ activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None): VertexRDD[A] = {
ClosureCleaner.clean(mapFunc)
ClosureCleaner.clean(reduceFunc)
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala
index 24699dfdd38b0..fa533a512d53b 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala
@@ -26,7 +26,7 @@ import org.apache.spark.graphx.PartitionStrategy._
*/
object Analytics extends Logging {
- def main(args: Array[String]) = {
+ def main(args: Array[String]): Unit = {
val host = args(0)
val taskType = args(1)
val fname = args(2)
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala
index b9621530efa22..3e1ed91bf6729 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala
@@ -136,25 +136,28 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel]
// Prepend an extra variable consisting of all 1.0's for the intercept.
val data = if (addIntercept) {
- input.map(labeledPoint => (labeledPoint.label, labeledPoint.features.+:(1.0)))
+ input.map(labeledPoint => (labeledPoint.label, 1.0 +: labeledPoint.features))
} else {
input.map(labeledPoint => (labeledPoint.label, labeledPoint.features))
}
val initialWeightsWithIntercept = if (addIntercept) {
- initialWeights.+:(1.0)
+ 0.0 +: initialWeights
} else {
initialWeights
}
- val weights = optimizer.optimize(data, initialWeightsWithIntercept)
- val intercept = weights(0)
- val weightsScaled = weights.tail
+ val weightsWithIntercept = optimizer.optimize(data, initialWeightsWithIntercept)
- val model = createModel(weightsScaled, intercept)
+ val (intercept, weights) = if (addIntercept) {
+ (weightsWithIntercept(0), weightsWithIntercept.tail)
+ } else {
+ (0.0, weightsWithIntercept)
+ }
+
+ logInfo("Final weights " + weights.mkString(","))
+ logInfo("Final intercept " + intercept)
- logInfo("Final model weights " + model.weights.mkString(","))
- logInfo("Final model intercept " + model.intercept)
- model
+ createModel(weights, intercept)
}
}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala
index fb2bc9b92a51c..be63ce8538fef 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala
@@ -36,8 +36,10 @@ class LassoModel(
extends GeneralizedLinearModel(weights, intercept)
with RegressionModel with Serializable {
- override def predictPoint(dataMatrix: DoubleMatrix, weightMatrix: DoubleMatrix,
- intercept: Double) = {
+ override def predictPoint(
+ dataMatrix: DoubleMatrix,
+ weightMatrix: DoubleMatrix,
+ intercept: Double): Double = {
dataMatrix.dot(weightMatrix) + intercept
}
}
@@ -66,7 +68,7 @@ class LassoWithSGD private (
.setMiniBatchFraction(miniBatchFraction)
// We don't want to penalize the intercept, so set this to false.
- setIntercept(false)
+ super.setIntercept(false)
var yMean = 0.0
var xColMean: DoubleMatrix = _
@@ -77,10 +79,16 @@ class LassoWithSGD private (
*/
def this() = this(1.0, 100, 1.0, 1.0)
- def createModel(weights: Array[Double], intercept: Double) = {
- val weightsMat = new DoubleMatrix(weights.length + 1, 1, (Array(intercept) ++ weights):_*)
+ override def setIntercept(addIntercept: Boolean): this.type = {
+ // TODO: Support adding intercept.
+ if (addIntercept) throw new UnsupportedOperationException("Adding intercept is not supported.")
+ this
+ }
+
+ override def createModel(weights: Array[Double], intercept: Double) = {
+ val weightsMat = new DoubleMatrix(weights.length, 1, weights: _*)
val weightsScaled = weightsMat.div(xColSd)
- val interceptScaled = yMean - (weightsMat.transpose().mmul(xColMean.div(xColSd)).get(0))
+ val interceptScaled = yMean - weightsMat.transpose().mmul(xColMean.div(xColSd)).get(0)
new LassoModel(weightsScaled.data, interceptScaled)
}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala
index 8ee40addb25d9..f5f15d1a33f4d 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala
@@ -31,13 +31,14 @@ import org.jblas.DoubleMatrix
* @param intercept Intercept computed for this model.
*/
class LinearRegressionModel(
- override val weights: Array[Double],
- override val intercept: Double)
- extends GeneralizedLinearModel(weights, intercept)
- with RegressionModel with Serializable {
-
- override def predictPoint(dataMatrix: DoubleMatrix, weightMatrix: DoubleMatrix,
- intercept: Double) = {
+ override val weights: Array[Double],
+ override val intercept: Double)
+ extends GeneralizedLinearModel(weights, intercept) with RegressionModel with Serializable {
+
+ override def predictPoint(
+ dataMatrix: DoubleMatrix,
+ weightMatrix: DoubleMatrix,
+ intercept: Double): Double = {
dataMatrix.dot(weightMatrix) + intercept
}
}
@@ -55,8 +56,7 @@ class LinearRegressionWithSGD private (
var stepSize: Double,
var numIterations: Int,
var miniBatchFraction: Double)
- extends GeneralizedLinearAlgorithm[LinearRegressionModel]
- with Serializable {
+ extends GeneralizedLinearAlgorithm[LinearRegressionModel] with Serializable {
val gradient = new LeastSquaresGradient()
val updater = new SimpleUpdater()
@@ -69,7 +69,7 @@ class LinearRegressionWithSGD private (
*/
def this() = this(1.0, 100, 1.0)
- def createModel(weights: Array[Double], intercept: Double) = {
+ override def createModel(weights: Array[Double], intercept: Double) = {
new LinearRegressionModel(weights, intercept)
}
}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala
index c504d3d40c773..feb100f21888f 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala
@@ -36,8 +36,10 @@ class RidgeRegressionModel(
extends GeneralizedLinearModel(weights, intercept)
with RegressionModel with Serializable {
- override def predictPoint(dataMatrix: DoubleMatrix, weightMatrix: DoubleMatrix,
- intercept: Double) = {
+ override def predictPoint(
+ dataMatrix: DoubleMatrix,
+ weightMatrix: DoubleMatrix,
+ intercept: Double): Double = {
dataMatrix.dot(weightMatrix) + intercept
}
}
@@ -67,7 +69,7 @@ class RidgeRegressionWithSGD private (
.setMiniBatchFraction(miniBatchFraction)
// We don't want to penalize the intercept in RidgeRegression, so set this to false.
- setIntercept(false)
+ super.setIntercept(false)
var yMean = 0.0
var xColMean: DoubleMatrix = _
@@ -78,8 +80,14 @@ class RidgeRegressionWithSGD private (
*/
def this() = this(1.0, 100, 1.0, 1.0)
- def createModel(weights: Array[Double], intercept: Double) = {
- val weightsMat = new DoubleMatrix(weights.length + 1, 1, (Array(intercept) ++ weights):_*)
+ override def setIntercept(addIntercept: Boolean): this.type = {
+ // TODO: Support adding intercept.
+ if (addIntercept) throw new UnsupportedOperationException("Adding intercept is not supported.")
+ this
+ }
+
+ override def createModel(weights: Array[Double], intercept: Double) = {
+ val weightsMat = new DoubleMatrix(weights.length, 1, weights: _*)
val weightsScaled = weightsMat.div(xColSd)
val interceptScaled = yMean - weightsMat.transpose().mmul(xColMean.div(xColSd)).get(0)
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala
index 64e4cbb860f61..2cebac943e15f 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala
@@ -17,11 +17,8 @@
package org.apache.spark.mllib.regression
-
-import org.scalatest.BeforeAndAfterAll
import org.scalatest.FunSuite
-import org.apache.spark.SparkContext
import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext}
class LassoSuite extends FunSuite with LocalSparkContext {
@@ -104,4 +101,10 @@ class LassoSuite extends FunSuite with LocalSparkContext {
// Test prediction on Array.
validatePrediction(validationData.map(row => model.predict(row.features)), validationData)
}
+
+ test("do not support intercept") {
+ intercept[UnsupportedOperationException] {
+ new LassoWithSGD().setIntercept(true)
+ }
+ }
}
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala
index 281f9df36ddb3..5d251bcbf35db 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala
@@ -17,7 +17,6 @@
package org.apache.spark.mllib.regression
-import org.scalatest.BeforeAndAfterAll
import org.scalatest.FunSuite
import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext}
@@ -57,4 +56,29 @@ class LinearRegressionSuite extends FunSuite with LocalSparkContext {
// Test prediction on Array.
validatePrediction(validationData.map(row => model.predict(row.features)), validationData)
}
+
+ // Test if we can correctly learn Y = 10*X1 + 10*X2
+ test("linear regression without intercept") {
+ val testRDD = sc.parallelize(LinearDataGenerator.generateLinearInput(
+ 0.0, Array(10.0, 10.0), 100, 42), 2).cache()
+ val linReg = new LinearRegressionWithSGD().setIntercept(false)
+ linReg.optimizer.setNumIterations(1000).setStepSize(1.0)
+
+ val model = linReg.run(testRDD)
+
+ assert(model.intercept === 0.0)
+ assert(model.weights.length === 2)
+ assert(model.weights(0) >= 9.0 && model.weights(0) <= 11.0)
+ assert(model.weights(1) >= 9.0 && model.weights(1) <= 11.0)
+
+ val validationData = LinearDataGenerator.generateLinearInput(
+ 0.0, Array(10.0, 10.0), 100, 17)
+ val validationRDD = sc.parallelize(validationData, 2).cache()
+
+ // Test prediction on RDD.
+ validatePrediction(model.predict(validationRDD.map(_.features)).collect(), validationData)
+
+ // Test prediction on Array.
+ validatePrediction(validationData.map(row => model.predict(row.features)), validationData)
+ }
}
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala
index 67dd06cc0f5eb..b2044ed0d8066 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala
@@ -17,14 +17,11 @@
package org.apache.spark.mllib.regression
-
import org.jblas.DoubleMatrix
-import org.scalatest.BeforeAndAfterAll
import org.scalatest.FunSuite
import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext}
-
class RidgeRegressionSuite extends FunSuite with LocalSparkContext {
def predictionError(predictions: Seq[Double], input: Seq[LabeledPoint]) = {
@@ -74,4 +71,10 @@ class RidgeRegressionSuite extends FunSuite with LocalSparkContext {
assert(ridgeErr < linearErr,
"ridgeError (" + ridgeErr + ") was not less than linearError(" + linearErr + ")")
}
+
+ test("do not support intercept") {
+ intercept[UnsupportedOperationException] {
+ new RidgeRegressionWithSGD().setIntercept(true)
+ }
+ }
}
diff --git a/pom.xml b/pom.xml
index deb89b18ada73..72acf2b402703 100644
--- a/pom.xml
+++ b/pom.xml
@@ -419,6 +419,11 @@
scala-compiler
${scala.version}
+
+ org.scala-lang
+ scala-reflect
+ ${scala.version}
+
org.scala-lang
jline
@@ -641,7 +646,6 @@
-deprecation
- -Xms64m
-Xms1024m
-Xmx1024m
-XX:PermSize=${PermGen}
@@ -684,7 +688,7 @@
${project.build.directory}/surefire-reports
.
${project.build.directory}/SparkTestSuite.txt
- -Xms64m -Xmx3g
+ -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 60f14ba37e35c..9e269e6551341 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -356,7 +356,9 @@ object SparkBuild extends Build {
) ++ assemblySettings ++ extraAssemblySettings
def toolsSettings = sharedSettings ++ Seq(
- name := "spark-tools"
+ name := "spark-tools",
+ libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-compiler" % v ),
+ libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-reflect" % v )
) ++ assemblySettings ++ extraAssemblySettings
def graphxSettings = sharedSettings ++ Seq(
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index ca2dc119a461a..019c249699c2d 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -693,11 +693,11 @@ def top(self, num):
"""
Get the top N elements from a RDD.
- Note: It returns the list sorted in ascending order.
+ Note: It returns the list sorted in descending order.
>>> sc.parallelize([10, 4, 2, 12, 3]).top(1)
[12]
>>> sc.parallelize([2, 3, 4, 5, 6]).cache().top(2)
- [5, 6]
+ [6, 5]
"""
def topIterator(iterator):
q = []
@@ -711,7 +711,7 @@ def topIterator(iterator):
def merge(a, b):
return next(topIterator(a + b))
- return sorted(self.mapPartitions(topIterator).reduce(merge))
+ return sorted(self.mapPartitions(topIterator).reduce(merge), reverse=True)
def take(self, num):
"""
diff --git a/repl/pom.xml b/repl/pom.xml
index fc49c8b811316..78d2fe13c27eb 100644
--- a/repl/pom.xml
+++ b/repl/pom.xml
@@ -77,6 +77,11 @@
scala-compiler
${scala.version}
+
+ org.scala-lang
+ scala-reflect
+ ${scala.version}
+
org.scala-lang
jline
diff --git a/sbt/sbt b/sbt/sbt
index 3ffa4ed9ab5a7..9de265bd07dcb 100755
--- a/sbt/sbt
+++ b/sbt/sbt
@@ -1,5 +1,13 @@
#!/usr/bin/env bash
+# When creating new tests for Spark SQL Hive, the HADOOP_CLASSPATH must contain the hive jars so
+# that we can run Hive to generate the golden answer. This is not required for normal development
+# or testing.
+for i in $HIVE_HOME/lib/*
+do HADOOP_CLASSPATH=$HADOOP_CLASSPATH:$i
+done
+export HADOOP_CLASSPATH
+
realpath () {
(
TARGET_FILE=$1
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
index bf7318d2e078b..976dda8d7e59a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
+package org.apache.spark.sql.catalyst
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.expressions.AttributeReference
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 161d28eba070e..4ebc0e70d946b 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package analysis
+package org.apache.spark.sql.catalyst.analysis
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical._
@@ -89,7 +87,8 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool
/**
* Replaces [[UnresolvedAttribute]]s with concrete
- * [[expressions.AttributeReference AttributeReferences]] from a logical plan node's children.
+ * [[catalyst.expressions.AttributeReference AttributeReferences]] from a logical plan node's
+ * children.
*/
object ResolveReferences extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
@@ -106,7 +105,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool
}
/**
- * Replaces [[UnresolvedFunction]]s with concrete [[expressions.Expression Expressions]].
+ * Replaces [[UnresolvedFunction]]s with concrete [[catalyst.expressions.Expression Expressions]].
*/
object ResolveFunctions extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
index b77f0bbb2f21b..ff66177a03b8c 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package analysis
+package org.apache.spark.sql.catalyst.analysis
import scala.collection.mutable
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
index eed058d3c7482..c0255701b7ba5 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package analysis
+package org.apache.spark.sql.catalyst.analysis
import org.apache.spark.sql.catalyst.expressions.Expression
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
index a6ecf6e2ebec5..4557d77160fad 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package analysis
+package org.apache.spark.sql.catalyst.analysis
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, Union}
@@ -37,8 +35,8 @@ trait HiveTypeCoercion {
StringToIntegralCasts, FunctionArgumentConversion)
/**
- * Applies any changes to [[catalyst.expressions.AttributeReference AttributeReference]] dataTypes
- * that are made by other rules to instances higher in the query tree.
+ * Applies any changes to [[catalyst.expressions.AttributeReference AttributeReference]] data
+ * types that are made by other rules to instances higher in the query tree.
*/
object PropagateTypes extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala
index 3cad3a5d4db4e..a6ce90854dcb4 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.catalyst
-package analysis
+package org.apache.spark.sql.catalyst.analysis
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala
index 30c55bacc7427..9f37ca904ffeb 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
+package org.apache.spark.sql.catalyst
/**
* Provides a logical query plan [[Analyzer]] and supporting classes for performing analysis.
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
index 04ae481102c74..41e9bcef3cd7f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
@@ -15,10 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package analysis
+package org.apache.spark.sql.catalyst.analysis
+import org.apache.spark.sql.catalyst.{errors, trees}
import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression}
import org.apache.spark.sql.catalyst.plans.logical.BaseRelation
import org.apache.spark.sql.catalyst.trees.TreeNode
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
index e6255bcafa8aa..67cddb351c185 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
+package org.apache.spark.sql.catalyst
import scala.language.implicitConversions
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala
index d8b69946fab10..bdeb660b1ecb7 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
+package org.apache.spark.sql.catalyst
import org.apache.spark.sql.catalyst.trees.TreeNode
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala
index 3fa4148f0dff2..f70e80b7f27f2 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala
@@ -15,13 +15,13 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
+import org.apache.spark.sql.catalyst.trees
import org.apache.spark.sql.catalyst.errors.attachTree
import org.apache.spark.sql.catalyst.plans.QueryPlan
import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.Logging
/**
* A bound reference points to a specific slot in the input tuple, allowing the actual value
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
index 71f64ef95064e..c26fc3d0f305f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.types._
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
index 2454a3355b11f..81fd160e00ca1 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
@@ -15,13 +15,12 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
+import org.apache.spark.sql.catalyst.trees
+import org.apache.spark.sql.catalyst.errors.TreeNodeException
import org.apache.spark.sql.catalyst.trees.TreeNode
import org.apache.spark.sql.catalyst.types.{DataType, FractionalType, IntegralType, NumericType}
-import org.apache.spark.sql.catalyst.errors.TreeNodeException
abstract class Expression extends TreeNode[Expression] {
self: Product =>
@@ -69,7 +68,7 @@ abstract class Expression extends TreeNode[Expression] {
def childrenResolved = !children.exists(!_.resolved)
/**
- * A set of helper functions that return the correct descendant of [[scala.math.Numeric]] type
+ * A set of helper functions that return the correct descendant of `scala.math.Numeric[T]` type
* and do any casting necessary of child evaluation.
*/
@inline
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala
index 8c407d2fddb42..38542d3fc7290 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
/**
* Converts a [[Row]] to another Row given a sequence of expression that define each column of the
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala
index 0d173afec8901..0bde621602944 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.types.DoubleType
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala
index 79c91ebaa4746..31d42b9ee71a0 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.types.NativeType
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala
index cc339480550a7..f53d8504b083f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.types.DataType
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala
index 171997b90ecf4..d5d93778f4b8d 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
abstract sealed class SortDirection
case object Ascending extends SortDirection
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala
index 01b7a14d4aca6..9828d0b9bd8b2 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
import scala.language.dynamics
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala
index a16bb80df37af..7303b155cae3d 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala
@@ -15,11 +15,10 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.catalyst.trees
abstract class AggregateExpression extends Expression {
self: Product =>
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
index 81e4a487bdbf9..fba056e7c07e3 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.analysis.UnresolvedException
import org.apache.spark.sql.catalyst.types._
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala
index 9ec0f6ade7467..ab96618d73df7 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.types._
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
index 9097c635ee96f..e9b491b10a5f2 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
@@ -15,10 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
+import org.apache.spark.sql.catalyst.trees
import org.apache.spark.sql.catalyst.types._
/**
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
index 0d01312c71c70..b82a12e0f754e 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.types._
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala
index 47b1241e714af..69c8bed309c18 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala
@@ -15,10 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
+import org.apache.spark.sql.catalyst.trees
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
import org.apache.spark.sql.catalyst.types._
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala
index 38e38371b1dec..5a47768dcb4a1 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala
@@ -15,10 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
+import org.apache.spark.sql.catalyst.trees
import org.apache.spark.sql.catalyst.analysis.UnresolvedException
case class Coalesce(children: Seq[Expression]) extends Expression {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala
index 76554e160b1d1..573ec052f4266 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
+package org.apache.spark.sql.catalyst
/**
* A set of classes that can be used to represent trees of relational expressions. A key goal of
@@ -46,6 +45,6 @@ package catalyst
* ability to reason about which subquery produced a given attribute.
*
* ==Evaluation==
- * The result of expressions can be evaluated using the [[Evaluate]] object.
+ * The result of expressions can be evaluated using the `Expression.apply(Row)` method.
*/
package object expressions
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
index e7f3e8ca60a25..722ff517d250e 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
@@ -15,12 +15,11 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
-import org.apache.spark.sql.catalyst.types.{BooleanType, StringType}
+import org.apache.spark.sql.catalyst.trees
import org.apache.spark.sql.catalyst.analysis.UnresolvedException
+import org.apache.spark.sql.catalyst.types.{BooleanType, StringType}
trait Predicate extends Expression {
self: Product =>
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
index 7584fe03cf745..e195f2ac7efd1 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.types.BooleanType
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
index f28076999ddbf..3dd6818029bcf 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package optimizer
+package org.apache.spark.sql.catalyst.optimizer
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.Inner
@@ -125,7 +123,6 @@ object PushPredicateThroughProject extends Rule[LogicalPlan] {
grandChild))
}
- //
def replaceAlias(condition: Expression, sourceAliases: Map[Attribute, Expression]): Expression = {
condition transform {
case a: AttributeReference => sourceAliases.getOrElse(a, a)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala
index d50b963dfc25b..67833664b35ae 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala
@@ -15,10 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package planning
+package org.apache.spark.sql.catalyst.planning
+import org.apache.spark.sql.Logging
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.trees.TreeNode
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/package.scala
index 64370ec7c0cc4..0a030b6c21f1f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/package.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
+package org.apache.spark.sql.catalyst
/**
* Contains classes for enumerating possible physical plans for a given logical query plan.
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala
index ff0ea90e54842..6dd816aa91dd1 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package planning
+package org.apache.spark.sql.catalyst.planning
import scala.annotation.tailrec
@@ -48,7 +46,9 @@ object FilteredOperation extends PredicateHelper {
/**
* A pattern that matches any number of project or filter operations on top of another relational
* operator. All filter operators are collected and their conditions are broken up and returned
- * together with the top project operator. [[Alias Aliases]] are in-lined/substituted if necessary.
+ * together with the top project operator.
+ * [[org.apache.spark.sql.catalyst.expressions.Alias Aliases]] are in-lined/substituted if
+ * necessary.
*/
object PhysicalOperation extends PredicateHelper {
type ReturnType = (Seq[NamedExpression], Seq[Expression], LogicalPlan)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
index 848db2452ae9c..8199a80f5d6bd 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package plans
+package org.apache.spark.sql.catalyst.plans
import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression}
import org.apache.spark.sql.catalyst.trees.TreeNode
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala
index 9f2283ad4346e..ae8d7d3e4257f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package plans
+package org.apache.spark.sql.catalyst.plans
sealed abstract class JoinType
case object Inner extends JoinType
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala
index 48ff45c3d3ebb..7c616788a3830 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala
@@ -15,10 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package plans
-package logical
+package org.apache.spark.sql.catalyst.plans.logical
abstract class BaseRelation extends LeafNode {
self: Product =>
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala
index 225dd260fb9de..5eb52d5350f55 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala
@@ -15,14 +15,13 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package plans
-package logical
+package org.apache.spark.sql.catalyst.plans.logical
import org.apache.spark.sql.catalyst.errors.TreeNodeException
import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.QueryPlan
import org.apache.spark.sql.catalyst.types.StructType
+import org.apache.spark.sql.catalyst.trees
abstract class LogicalPlan extends QueryPlan[LogicalPlan] {
self: Product =>
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala
index 5a3ea9f0a0464..d3f9d0fb93237 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala
@@ -15,10 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package plans
-package logical
+package org.apache.spark.sql.catalyst.plans.logical
import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala
index ac7d2d6001779..f8fe558511bfd 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala
@@ -15,11 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package plans
-package logical
+package org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.analysis
import org.apache.spark.sql.catalyst.expressions.Attribute
object LocalRelation {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
index 61481de65e76e..9d16189deedfe 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
@@ -15,12 +15,10 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package plans
-package logical
+package org.apache.spark.sql.catalyst.plans.logical
import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.JoinType
import org.apache.spark.sql.catalyst.types._
case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends UnaryNode {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala
index 775e50bbd5128..7146fbd540f29 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala
@@ -15,10 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package plans
-package logical
+package org.apache.spark.sql.catalyst.plans.logical
import org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala
index a40ab4bbb154b..42bdab42b79ff 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
+package org.apache.spark.sql.catalyst
/**
* A a collection of common abstractions for query plans as well as
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
index 20e2a45678c89..8893744eb2e7a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
@@ -15,10 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package plans
-package physical
+package org.apache.spark.sql.catalyst.plans.physical
import org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder}
import org.apache.spark.sql.catalyst.types.IntegerType
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala
index c7632a62a00d2..1076537bc7602 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala
@@ -15,10 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package rules
+package org.apache.spark.sql.catalyst.rules
+import org.apache.spark.sql.Logging
import org.apache.spark.sql.catalyst.trees.TreeNode
abstract class Rule[TreeType <: TreeNode[_]] extends Logging {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/package.scala
index 26ab5430821f4..ca82c5d93d4cb 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/package.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
+package org.apache.spark.sql.catalyst
/**
* A framework for applying batches rewrite rules to trees, possibly to fixed point.
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
index 89e27d81dad0b..0369129393a08 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package trees
+package org.apache.spark.sql.catalyst.trees
import org.apache.spark.sql.catalyst.errors._
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala
index e2da1d2439bbf..d159ecdd5d781 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala
@@ -15,8 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
+package org.apache.spark.sql.catalyst
+
+import org.apache.spark.sql.Logger
/**
* A library for easily manipulating trees of operators. Operators that extend TreeNode are
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala
index 90a9f9f7e548d..7a45d1a1b8195 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package types
+package org.apache.spark.sql.catalyst.types
import scala.reflect.runtime.universe.{typeTag, TypeTag}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala
index b65a5617d9a7f..de24449590f9a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
+package org.apache.spark.sql.catalyst
/**
* Contains a type system for attributes produced by relations, including complex types like
* structs, arrays and maps.
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala
index 52adea2661363..a001d953592db 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
+package org.apache.spark.sql.catalyst
import java.io.{PrintWriter, ByteArrayOutputStream, FileInputStream, File}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala
index fb25e1c246117..46b2250aab231 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
+package org.apache.spark.sql.catalyst
import org.scalatest.FunSuite
@@ -172,4 +171,4 @@ class DistributionSuite extends FunSuite {
AllTuples,
false)
}
-}
\ No newline at end of file
+}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
index 78ec48ba77628..4c313585c6386 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package analysis
+package org.apache.spark.sql.catalyst.analysis
import org.scalatest.FunSuite
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala
index b85b72a2840f9..b9e0f8e9dcc5f 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package analysis
+package org.apache.spark.sql.catalyst.analysis
import org.scalatest.FunSuite
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala
index c8fd581aa7b47..94894adf81202 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
import org.scalatest.FunSuite
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala
index 53f760fb4ceb2..2ab14f48ccc8a 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala
@@ -15,11 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package optimizer
+package org.apache.spark.sql.catalyst.optimizer
-import org.apache.spark.sql.catalyst.analysis._
+import org.apache.spark.sql.catalyst.analysis.EliminateAnalysisOperators
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan}
import org.apache.spark.sql.catalyst.rules.RuleExecutor
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala
index ae1b2b13dd8f1..ef47850455a37 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala
@@ -15,12 +15,10 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package optimizer
+package org.apache.spark.sql.catalyst.optimizer
-
-import org.apache.spark.sql.catalyst.analysis._
+import org.apache.spark.sql.catalyst.analysis
+import org.apache.spark.sql.catalyst.analysis.EliminateAnalysisOperators
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules._
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.scala
index 0c6e4d5acee83..89982d5cd8d74 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package optimizer
+package org.apache.spark.sql.catalyst.optimizer
import org.scalatest.FunSuite
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala
index 738cfa85fbc6a..4b2d45584045f 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package trees
+package org.apache.spark.sql.catalyst.trees
import org.scalatest.FunSuite
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala
index ddbeba6203aa4..e0c98ecdf8f22 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package columnar
+package org.apache.spark.sql.columnar
import java.nio.{ByteOrder, ByteBuffer}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala
index 6bd1841821875..3e622adfd3d6a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala
@@ -15,12 +15,13 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package columnar
+package org.apache.spark.sql.columnar
-import java.nio.{ByteOrder, ByteBuffer}
+import java.nio.{ByteBuffer, ByteOrder}
+import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.columnar.ColumnBuilder._
import org.apache.spark.sql.execution.SparkSqlSerializer
private[sql] trait ColumnBuilder {
@@ -35,7 +36,6 @@ private[sql] trait ColumnBuilder {
}
private[sql] abstract class BasicColumnBuilder[T <: DataType, JvmType] extends ColumnBuilder {
- import ColumnBuilder._
private var columnName: String = _
protected var buffer: ByteBuffer = _
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala
index 3b759a51cc695..a452b86f0cda3 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala
@@ -14,8 +14,8 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.spark.sql
-package columnar
+
+package org.apache.spark.sql.columnar
import java.nio.ByteBuffer
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala
index 1661c3f3ff4a9..048d1f05c7df2 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala
@@ -15,10 +15,11 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package columnar
+package org.apache.spark.sql.columnar
-import java.nio.{ByteOrder, ByteBuffer}
+import java.nio.{ByteBuffer, ByteOrder}
+
+import org.apache.spark.sql.Row
/**
* Builds a nullable column. The byte buffer of a nullable column contains:
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala
index c7efd30e87da4..f853759e5a306 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala
@@ -15,18 +15,18 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package columnar
+package org.apache.spark.sql.columnar
import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, Attribute}
import org.apache.spark.sql.execution.{SparkPlan, LeafNode}
+import org.apache.spark.sql.Row
+
+/* Implicit conversions */
+import org.apache.spark.sql.columnar.ColumnType._
private[sql] case class InMemoryColumnarTableScan(attributes: Seq[Attribute], child: SparkPlan)
extends LeafNode {
- // For implicit conversion from `DataType` to `ColumnType`
- import ColumnType._
-
override def output: Seq[Attribute] = attributes
lazy val cachedColumnBuffers = {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala
index 65d77e3a40cf4..869673b1fe978 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala
@@ -15,16 +15,16 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package execution
+package org.apache.spark.sql.execution
+import org.apache.spark.{HashPartitioner, RangePartitioner, SparkConf}
import org.apache.spark.rdd.ShuffledRDD
+import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.errors.attachTree
import org.apache.spark.sql.catalyst.expressions.{MutableProjection, RowOrdering}
import org.apache.spark.sql.catalyst.plans.physical._
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.util.MutablePair
-import org.apache.spark.{HashPartitioner, RangePartitioner, SparkConf}
case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends UnaryNode {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala
index 7e50fda4ceac5..e902e6ced521d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala
@@ -15,10 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package execution
+package org.apache.spark.sql.execution
-import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.{Generator, JoinedRow, Literal, Projection}
/**
* Applies a [[catalyst.expressions.Generator Generator]] to a stream of input rows, combining the
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecutionException.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecutionException.scala
index 7ce8608d2061d..16806c620635f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecutionException.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecutionException.scala
@@ -15,7 +15,6 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package execution
+package org.apache.spark.sql.execution
class QueryExecutionException(message: String) extends Exception(message)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala
index 5626181d184e5..acb1ee83a72f6 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala
@@ -15,16 +15,15 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package execution
+package org.apache.spark.sql.execution
import org.apache.spark.rdd.RDD
-
+import org.apache.spark.sql.{Logging, Row}
+import org.apache.spark.sql.catalyst.trees
import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.expressions.GenericRow
+import org.apache.spark.sql.catalyst.plans.{QueryPlan, logical}
import org.apache.spark.sql.catalyst.plans.physical._
-import org.apache.spark.sql.catalyst.trees
abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging {
self: Product =>
@@ -47,7 +46,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging {
def executeCollect(): Array[Row] = execute().collect()
protected def buildRow(values: Seq[Any]): Row =
- new catalyst.expressions.GenericRow(values.toArray)
+ new GenericRow(values.toArray)
}
/**
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala
index ad7cd58b6aaaf..915f551fb2f01 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package execution
+package org.apache.spark.sql.execution
import java.nio.ByteBuffer
@@ -33,6 +32,7 @@ class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) {
kryo.setRegistrationRequired(false)
kryo.register(classOf[MutablePair[_, _]])
kryo.register(classOf[Array[Any]])
+ kryo.register(classOf[scala.collection.immutable.Map$Map1], new MapSerializer)
kryo.register(classOf[org.apache.spark.sql.catalyst.expressions.GenericRow])
kryo.register(classOf[org.apache.spark.sql.catalyst.expressions.GenericMutableRow])
kryo.register(classOf[scala.collection.mutable.ArrayBuffer[_]])
@@ -71,3 +71,20 @@ class BigDecimalSerializer extends Serializer[BigDecimal] {
BigDecimal(input.readString())
}
}
+
+/**
+ * Maps do not have a no arg constructor and so cannot be serialized by default. So, we serialize
+ * them as `Array[(k,v)]`.
+ */
+class MapSerializer extends Serializer[Map[_,_]] {
+ def write(kryo: Kryo, output: Output, map: Map[_,_]) {
+ kryo.writeObject(output, map.flatMap(e => Seq(e._1, e._2)).toArray)
+ }
+
+ def read(kryo: Kryo, input: Input, tpe: Class[Map[_,_]]): Map[_,_] = {
+ kryo.readObject(input, classOf[Array[Any]])
+ .sliding(2,2)
+ .map { case Array(k,v) => (k,v) }
+ .toMap
+ }
+}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
index 8a39ded0a9ec4..86f9d3e0fa954 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
@@ -15,9 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package execution
+package org.apache.spark.sql.execution
+import org.apache.spark.sql.{SQLContext, execution}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.planning._
import org.apache.spark.sql.catalyst.plans._
@@ -175,7 +175,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
InsertIntoParquetTable(relation, planLater(child))(sparkContext) :: Nil
case logical.InsertIntoTable(table: ParquetRelation, partition, child, overwrite) =>
InsertIntoParquetTable(table, planLater(child))(sparkContext) :: Nil
- case PhysicalOperation(projectList, filters, relation: parquet.ParquetRelation) =>
+ case PhysicalOperation(projectList, filters, relation: ParquetRelation) =>
// TODO: Should be pushing down filters as well.
pruneFilterProject(
projectList,
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala
index 14e5ab628f23d..8515a18f18c55 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package execution
+package org.apache.spark.sql.execution
import org.apache.spark.SparkContext
import org.apache.spark.sql.catalyst.errors._
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
index e4f918b6787fe..65cb8f8becefa 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package execution
+package org.apache.spark.sql.execution
import scala.reflect.runtime.universe.TypeTag
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug.scala
index db259b4c4b95b..40982f1fffbbf 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package execution
+package org.apache.spark.sql.execution
object DebugQuery {
def apply(plan: SparkPlan): SparkPlan = {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala
index a6e3892e88703..f0d21143ba5d1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package execution
+package org.apache.spark.sql.execution
import scala.collection.mutable
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala
index 011aaf7440cb1..2b825f84ee910 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala
@@ -82,11 +82,10 @@ object ParquetRelation {
type RowType = org.apache.spark.sql.catalyst.expressions.GenericMutableRow
/**
- * Creates a new ParquetRelation and underlying Parquetfile for the given
- * LogicalPlan. Note that this is used inside [[SparkStrategies]] to
- * create a resolved relation as a data sink for writing to a Parquetfile.
- * The relation is empty but is initialized with ParquetMetadata and
- * can be inserted into.
+ * Creates a new ParquetRelation and underlying Parquetfile for the given LogicalPlan. Note that
+ * this is used inside [[org.apache.spark.sql.execution.SparkStrategies SparkStrategies]] to
+ * create a resolved relation as a data sink for writing to a Parquetfile. The relation is empty
+ * but is initialized with ParquetMetadata and can be inserted into.
*
* @param pathString The directory the Parquetfile will be stored in.
* @param child The child node that will be used for extracting the schema.
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala
index 91b4848fe403f..c21e400282004 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala
@@ -135,8 +135,7 @@ object RowWriteSupport {
}
/**
- * A `parquet.io.api.GroupConverter` that is able to convert a Parquet record
- * to a [[org.apache.spark.sql.catalyst.expressions.Row]] object.
+ * A `parquet.io.api.GroupConverter` that is able to convert a Parquet record to a `Row` object.
*
* @param schema The corresponding Catalyst schema in the form of a list of attributes.
*/
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala
index ca56c4476bcc2..f2389f8f0591e 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala
@@ -15,9 +15,10 @@
* limitations under the License.
*/
-package org.apache.spark
-package sql
-package test
+package org.apache.spark.sql.test
+
+import org.apache.spark.{SparkConf, SparkContext}
+import org.apache.spark.sql.SQLContext
/** A SQLContext that can be used for local testing. */
object TestSQLContext
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala
index 5c8cb086ee3b8..d719ceb827691 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala
@@ -22,8 +22,6 @@ import org.scalatest.FunSuite
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.util._
-/* Implicits */
-
class QueryTest extends FunSuite {
/**
* Runs the plan and makes sure the answer matches the expected result.
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala
index 1997d8638cb61..2d431affbcfcc 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package columnar
+package org.apache.spark.sql.columnar
import java.nio.ByteBuffer
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala
index 279607ccfaa5b..d413d483f4e7e 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package columnar
+package org.apache.spark.sql.columnar
import org.scalatest.FunSuite
import org.apache.spark.sql.catalyst.types.DataType
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala
index 3354da3fa3e0f..5222a47e1ab87 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package columnar
+package org.apache.spark.sql.columnar
import org.scalatest.FunSuite
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala
index 93b2a308a410a..ca5c8b8eb63dc 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package execution
+package org.apache.spark.sql.execution
import org.apache.spark.sql.QueryTest
import org.apache.spark.sql.catalyst.expressions._
diff --git a/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala
index d96c2f70e0c74..7219c030cb0f0 100644
--- a/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala
@@ -113,11 +113,10 @@ class SparkHiveHadoopWriter(
cmtr.commitTask(taCtxt)
logInfo (taID + ": Committed")
} catch {
- case e: IOException => {
+ case e: IOException =>
logError("Error committing the output of task: " + taID.value, e)
cmtr.abortTask(taCtxt)
throw e
- }
}
} else {
logWarning ("No need to commit output of task: " + taID.value)
@@ -161,12 +160,12 @@ class SparkHiveHadoopWriter(
taskContext
}
- private def setIDs(jobid: Int, splitid: Int, attemptid: Int) {
- jobID = jobid
- splitID = splitid
- attemptID = attemptid
+ private def setIDs(jobId: Int, splitId: Int, attemptId: Int) {
+ jobID = jobId
+ splitID = splitId
+ attemptID = attemptId
- jID = new SerializableWritable[JobID](SparkHadoopWriter.createJobID(now, jobid))
+ jID = new SerializableWritable[JobID](SparkHadoopWriter.createJobID(now, jobId))
taID = new SerializableWritable[TaskAttemptID](
new TaskAttemptID(new TaskID(jID.value, true, splitID), attemptID))
}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
index 3bcf586662f2c..fc5057b73fe24 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
@@ -32,7 +32,7 @@ import org.apache.spark.SparkContext
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.analysis.{Analyzer, OverrideCatalog}
import org.apache.spark.sql.catalyst.expressions.GenericRow
-import org.apache.spark.sql.catalyst.plans.logical.{BaseRelation, LogicalPlan, LowerCaseSchema}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LowerCaseSchema}
import org.apache.spark.sql.catalyst.plans.logical.{NativeCommand, ExplainCommand}
import org.apache.spark.sql.catalyst.types._
import org.apache.spark.sql.execution._
@@ -86,7 +86,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
val input = new java.io.InputStream {
val iterator = (start ++ end).iterator
- def read(): Int = if (iterator.hasNext) iterator.next else -1
+ def read(): Int = if (iterator.hasNext) iterator.next() else -1
}
val reader = new BufferedReader(new InputStreamReader(input))
val stringBuilder = new StringBuilder
@@ -148,24 +148,24 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
SessionState.start(sessionState)
- if (proc.isInstanceOf[Driver]) {
- val driver: Driver = proc.asInstanceOf[Driver]
- driver.init()
+ proc match {
+ case driver: Driver =>
+ driver.init()
- val results = new JArrayList[String]
- val response: CommandProcessorResponse = driver.run(cmd)
- // Throw an exception if there is an error in query processing.
- if (response.getResponseCode != 0) {
+ val results = new JArrayList[String]
+ val response: CommandProcessorResponse = driver.run(cmd)
+ // Throw an exception if there is an error in query processing.
+ if (response.getResponseCode != 0) {
+ driver.destroy()
+ throw new QueryExecutionException(response.getErrorMessage)
+ }
+ driver.setMaxRows(maxRows)
+ driver.getResults(results)
driver.destroy()
- throw new QueryExecutionException(response.getErrorMessage)
- }
- driver.setMaxRows(maxRows)
- driver.getResults(results)
- driver.destroy()
- results
- } else {
- sessionState.out.println(tokens(0) + " " + cmd_1)
- Seq(proc.run(cmd_1).getResponseCode.toString)
+ results
+ case _ =>
+ sessionState.out.println(tokens(0) + " " + cmd_1)
+ Seq(proc.run(cmd_1).getResponseCode.toString)
}
} catch {
case e: Exception =>
@@ -226,7 +226,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
sparkContext.parallelize(asRows, 1)
}
case _ =>
- executedPlan.execute.map(_.copy())
+ executedPlan.execute().map(_.copy())
}
protected val primitiveTypes =
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
index 1667a217297b1..4f8353666a12b 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
+package org.apache.spark.sql.hive
import scala.util.parsing.combinator.RegexParsers
@@ -27,14 +26,15 @@ import org.apache.hadoop.hive.ql.plan.TableDesc
import org.apache.hadoop.hive.ql.session.SessionState
import org.apache.hadoop.hive.serde2.Deserializer
-
-import org.apache.spark.sql.catalyst.analysis.{Catalog, EliminateAnalysisOperators}
+import org.apache.spark.sql.Logging
+import org.apache.spark.sql.catalyst.analysis.{EliminateAnalysisOperators, Catalog}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules._
import org.apache.spark.sql.catalyst.types._
+/* Implicit conversions */
import scala.collection.JavaConversions._
class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging {
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
index 8e76a7348e957..f4b61381f9a27 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
+package org.apache.spark.sql.hive
import org.apache.hadoop.hive.ql.lib.Node
import org.apache.hadoop.hive.ql.parse._
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
index 141067247d736..3ca1d93c11fa9 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
@@ -15,9 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
+package org.apache.spark.sql.hive
+import org.apache.spark.sql.SQLContext
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.planning._
import org.apache.spark.sql.catalyst.plans._
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala
index dc4181ec9905c..2610100043659 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
+package org.apache.spark.sql.hive
import java.io.{BufferedReader, InputStreamReader}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
index 99dc85ec19868..ca5311344615f 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
+package org.apache.spark.sql.hive
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{Path, PathFilter}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
index a26b0ff2319d4..bc3447b9d802d 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
+package org.apache.spark.sql.hive
import java.io.File
import java.util.{Set => JavaSet}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala
index 78f69e7ff5731..e2d9d8de2572a 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
+package org.apache.spark.sql.hive
import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar}
import org.apache.hadoop.hive.metastore.MetaStoreUtils
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
index 72ccd4f4a4302..44901db3f963b 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
+package org.apache.spark.sql.hive
import scala.collection.mutable.ArrayBuffer
@@ -29,6 +28,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive._
import org.apache.hadoop.hive.serde2.{io => hiveIo}
import org.apache.hadoop.{io => hadoopIo}
+import org.apache.spark.sql.Logging
import org.apache.spark.sql.catalyst.analysis
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.types
diff --git a/sql/hive/src/test/resources/golden/partitioned table scan-0-3e8898a13ccef627603f340d1f8bdd80 b/sql/hive/src/test/resources/golden/partitioned table scan-0-3e8898a13ccef627603f340d1f8bdd80
new file mode 100644
index 0000000000000..a3cb00feaca62
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/partitioned table scan-0-3e8898a13ccef627603f340d1f8bdd80
@@ -0,0 +1,2000 @@
+2008-04-08 11 238 val_238
+2008-04-08 11 86 val_86
+2008-04-08 11 311 val_311
+2008-04-08 11 27 val_27
+2008-04-08 11 165 val_165
+2008-04-08 11 409 val_409
+2008-04-08 11 255 val_255
+2008-04-08 11 278 val_278
+2008-04-08 11 98 val_98
+2008-04-08 11 484 val_484
+2008-04-08 11 265 val_265
+2008-04-08 11 193 val_193
+2008-04-08 11 401 val_401
+2008-04-08 11 150 val_150
+2008-04-08 11 273 val_273
+2008-04-08 11 224 val_224
+2008-04-08 11 369 val_369
+2008-04-08 11 66 val_66
+2008-04-08 11 128 val_128
+2008-04-08 11 213 val_213
+2008-04-08 11 146 val_146
+2008-04-08 11 406 val_406
+2008-04-08 11 429 val_429
+2008-04-08 11 374 val_374
+2008-04-08 11 152 val_152
+2008-04-08 11 469 val_469
+2008-04-08 11 145 val_145
+2008-04-08 11 495 val_495
+2008-04-08 11 37 val_37
+2008-04-08 11 327 val_327
+2008-04-08 11 281 val_281
+2008-04-08 11 277 val_277
+2008-04-08 11 209 val_209
+2008-04-08 11 15 val_15
+2008-04-08 11 82 val_82
+2008-04-08 11 403 val_403
+2008-04-08 11 166 val_166
+2008-04-08 11 417 val_417
+2008-04-08 11 430 val_430
+2008-04-08 11 252 val_252
+2008-04-08 11 292 val_292
+2008-04-08 11 219 val_219
+2008-04-08 11 287 val_287
+2008-04-08 11 153 val_153
+2008-04-08 11 193 val_193
+2008-04-08 11 338 val_338
+2008-04-08 11 446 val_446
+2008-04-08 11 459 val_459
+2008-04-08 11 394 val_394
+2008-04-08 11 237 val_237
+2008-04-08 11 482 val_482
+2008-04-08 11 174 val_174
+2008-04-08 11 413 val_413
+2008-04-08 11 494 val_494
+2008-04-08 11 207 val_207
+2008-04-08 11 199 val_199
+2008-04-08 11 466 val_466
+2008-04-08 11 208 val_208
+2008-04-08 11 174 val_174
+2008-04-08 11 399 val_399
+2008-04-08 11 396 val_396
+2008-04-08 11 247 val_247
+2008-04-08 11 417 val_417
+2008-04-08 11 489 val_489
+2008-04-08 11 162 val_162
+2008-04-08 11 377 val_377
+2008-04-08 11 397 val_397
+2008-04-08 11 309 val_309
+2008-04-08 11 365 val_365
+2008-04-08 11 266 val_266
+2008-04-08 11 439 val_439
+2008-04-08 11 342 val_342
+2008-04-08 11 367 val_367
+2008-04-08 11 325 val_325
+2008-04-08 11 167 val_167
+2008-04-08 11 195 val_195
+2008-04-08 11 475 val_475
+2008-04-08 11 17 val_17
+2008-04-08 11 113 val_113
+2008-04-08 11 155 val_155
+2008-04-08 11 203 val_203
+2008-04-08 11 339 val_339
+2008-04-08 11 0 val_0
+2008-04-08 11 455 val_455
+2008-04-08 11 128 val_128
+2008-04-08 11 311 val_311
+2008-04-08 11 316 val_316
+2008-04-08 11 57 val_57
+2008-04-08 11 302 val_302
+2008-04-08 11 205 val_205
+2008-04-08 11 149 val_149
+2008-04-08 11 438 val_438
+2008-04-08 11 345 val_345
+2008-04-08 11 129 val_129
+2008-04-08 11 170 val_170
+2008-04-08 11 20 val_20
+2008-04-08 11 489 val_489
+2008-04-08 11 157 val_157
+2008-04-08 11 378 val_378
+2008-04-08 11 221 val_221
+2008-04-08 11 92 val_92
+2008-04-08 11 111 val_111
+2008-04-08 11 47 val_47
+2008-04-08 11 72 val_72
+2008-04-08 11 4 val_4
+2008-04-08 11 280 val_280
+2008-04-08 11 35 val_35
+2008-04-08 11 427 val_427
+2008-04-08 11 277 val_277
+2008-04-08 11 208 val_208
+2008-04-08 11 356 val_356
+2008-04-08 11 399 val_399
+2008-04-08 11 169 val_169
+2008-04-08 11 382 val_382
+2008-04-08 11 498 val_498
+2008-04-08 11 125 val_125
+2008-04-08 11 386 val_386
+2008-04-08 11 437 val_437
+2008-04-08 11 469 val_469
+2008-04-08 11 192 val_192
+2008-04-08 11 286 val_286
+2008-04-08 11 187 val_187
+2008-04-08 11 176 val_176
+2008-04-08 11 54 val_54
+2008-04-08 11 459 val_459
+2008-04-08 11 51 val_51
+2008-04-08 11 138 val_138
+2008-04-08 11 103 val_103
+2008-04-08 11 239 val_239
+2008-04-08 11 213 val_213
+2008-04-08 11 216 val_216
+2008-04-08 11 430 val_430
+2008-04-08 11 278 val_278
+2008-04-08 11 176 val_176
+2008-04-08 11 289 val_289
+2008-04-08 11 221 val_221
+2008-04-08 11 65 val_65
+2008-04-08 11 318 val_318
+2008-04-08 11 332 val_332
+2008-04-08 11 311 val_311
+2008-04-08 11 275 val_275
+2008-04-08 11 137 val_137
+2008-04-08 11 241 val_241
+2008-04-08 11 83 val_83
+2008-04-08 11 333 val_333
+2008-04-08 11 180 val_180
+2008-04-08 11 284 val_284
+2008-04-08 11 12 val_12
+2008-04-08 11 230 val_230
+2008-04-08 11 181 val_181
+2008-04-08 11 67 val_67
+2008-04-08 11 260 val_260
+2008-04-08 11 404 val_404
+2008-04-08 11 384 val_384
+2008-04-08 11 489 val_489
+2008-04-08 11 353 val_353
+2008-04-08 11 373 val_373
+2008-04-08 11 272 val_272
+2008-04-08 11 138 val_138
+2008-04-08 11 217 val_217
+2008-04-08 11 84 val_84
+2008-04-08 11 348 val_348
+2008-04-08 11 466 val_466
+2008-04-08 11 58 val_58
+2008-04-08 11 8 val_8
+2008-04-08 11 411 val_411
+2008-04-08 11 230 val_230
+2008-04-08 11 208 val_208
+2008-04-08 11 348 val_348
+2008-04-08 11 24 val_24
+2008-04-08 11 463 val_463
+2008-04-08 11 431 val_431
+2008-04-08 11 179 val_179
+2008-04-08 11 172 val_172
+2008-04-08 11 42 val_42
+2008-04-08 11 129 val_129
+2008-04-08 11 158 val_158
+2008-04-08 11 119 val_119
+2008-04-08 11 496 val_496
+2008-04-08 11 0 val_0
+2008-04-08 11 322 val_322
+2008-04-08 11 197 val_197
+2008-04-08 11 468 val_468
+2008-04-08 11 393 val_393
+2008-04-08 11 454 val_454
+2008-04-08 11 100 val_100
+2008-04-08 11 298 val_298
+2008-04-08 11 199 val_199
+2008-04-08 11 191 val_191
+2008-04-08 11 418 val_418
+2008-04-08 11 96 val_96
+2008-04-08 11 26 val_26
+2008-04-08 11 165 val_165
+2008-04-08 11 327 val_327
+2008-04-08 11 230 val_230
+2008-04-08 11 205 val_205
+2008-04-08 11 120 val_120
+2008-04-08 11 131 val_131
+2008-04-08 11 51 val_51
+2008-04-08 11 404 val_404
+2008-04-08 11 43 val_43
+2008-04-08 11 436 val_436
+2008-04-08 11 156 val_156
+2008-04-08 11 469 val_469
+2008-04-08 11 468 val_468
+2008-04-08 11 308 val_308
+2008-04-08 11 95 val_95
+2008-04-08 11 196 val_196
+2008-04-08 11 288 val_288
+2008-04-08 11 481 val_481
+2008-04-08 11 457 val_457
+2008-04-08 11 98 val_98
+2008-04-08 11 282 val_282
+2008-04-08 11 197 val_197
+2008-04-08 11 187 val_187
+2008-04-08 11 318 val_318
+2008-04-08 11 318 val_318
+2008-04-08 11 409 val_409
+2008-04-08 11 470 val_470
+2008-04-08 11 137 val_137
+2008-04-08 11 369 val_369
+2008-04-08 11 316 val_316
+2008-04-08 11 169 val_169
+2008-04-08 11 413 val_413
+2008-04-08 11 85 val_85
+2008-04-08 11 77 val_77
+2008-04-08 11 0 val_0
+2008-04-08 11 490 val_490
+2008-04-08 11 87 val_87
+2008-04-08 11 364 val_364
+2008-04-08 11 179 val_179
+2008-04-08 11 118 val_118
+2008-04-08 11 134 val_134
+2008-04-08 11 395 val_395
+2008-04-08 11 282 val_282
+2008-04-08 11 138 val_138
+2008-04-08 11 238 val_238
+2008-04-08 11 419 val_419
+2008-04-08 11 15 val_15
+2008-04-08 11 118 val_118
+2008-04-08 11 72 val_72
+2008-04-08 11 90 val_90
+2008-04-08 11 307 val_307
+2008-04-08 11 19 val_19
+2008-04-08 11 435 val_435
+2008-04-08 11 10 val_10
+2008-04-08 11 277 val_277
+2008-04-08 11 273 val_273
+2008-04-08 11 306 val_306
+2008-04-08 11 224 val_224
+2008-04-08 11 309 val_309
+2008-04-08 11 389 val_389
+2008-04-08 11 327 val_327
+2008-04-08 11 242 val_242
+2008-04-08 11 369 val_369
+2008-04-08 11 392 val_392
+2008-04-08 11 272 val_272
+2008-04-08 11 331 val_331
+2008-04-08 11 401 val_401
+2008-04-08 11 242 val_242
+2008-04-08 11 452 val_452
+2008-04-08 11 177 val_177
+2008-04-08 11 226 val_226
+2008-04-08 11 5 val_5
+2008-04-08 11 497 val_497
+2008-04-08 11 402 val_402
+2008-04-08 11 396 val_396
+2008-04-08 11 317 val_317
+2008-04-08 11 395 val_395
+2008-04-08 11 58 val_58
+2008-04-08 11 35 val_35
+2008-04-08 11 336 val_336
+2008-04-08 11 95 val_95
+2008-04-08 11 11 val_11
+2008-04-08 11 168 val_168
+2008-04-08 11 34 val_34
+2008-04-08 11 229 val_229
+2008-04-08 11 233 val_233
+2008-04-08 11 143 val_143
+2008-04-08 11 472 val_472
+2008-04-08 11 322 val_322
+2008-04-08 11 498 val_498
+2008-04-08 11 160 val_160
+2008-04-08 11 195 val_195
+2008-04-08 11 42 val_42
+2008-04-08 11 321 val_321
+2008-04-08 11 430 val_430
+2008-04-08 11 119 val_119
+2008-04-08 11 489 val_489
+2008-04-08 11 458 val_458
+2008-04-08 11 78 val_78
+2008-04-08 11 76 val_76
+2008-04-08 11 41 val_41
+2008-04-08 11 223 val_223
+2008-04-08 11 492 val_492
+2008-04-08 11 149 val_149
+2008-04-08 11 449 val_449
+2008-04-08 11 218 val_218
+2008-04-08 11 228 val_228
+2008-04-08 11 138 val_138
+2008-04-08 11 453 val_453
+2008-04-08 11 30 val_30
+2008-04-08 11 209 val_209
+2008-04-08 11 64 val_64
+2008-04-08 11 468 val_468
+2008-04-08 11 76 val_76
+2008-04-08 11 74 val_74
+2008-04-08 11 342 val_342
+2008-04-08 11 69 val_69
+2008-04-08 11 230 val_230
+2008-04-08 11 33 val_33
+2008-04-08 11 368 val_368
+2008-04-08 11 103 val_103
+2008-04-08 11 296 val_296
+2008-04-08 11 113 val_113
+2008-04-08 11 216 val_216
+2008-04-08 11 367 val_367
+2008-04-08 11 344 val_344
+2008-04-08 11 167 val_167
+2008-04-08 11 274 val_274
+2008-04-08 11 219 val_219
+2008-04-08 11 239 val_239
+2008-04-08 11 485 val_485
+2008-04-08 11 116 val_116
+2008-04-08 11 223 val_223
+2008-04-08 11 256 val_256
+2008-04-08 11 263 val_263
+2008-04-08 11 70 val_70
+2008-04-08 11 487 val_487
+2008-04-08 11 480 val_480
+2008-04-08 11 401 val_401
+2008-04-08 11 288 val_288
+2008-04-08 11 191 val_191
+2008-04-08 11 5 val_5
+2008-04-08 11 244 val_244
+2008-04-08 11 438 val_438
+2008-04-08 11 128 val_128
+2008-04-08 11 467 val_467
+2008-04-08 11 432 val_432
+2008-04-08 11 202 val_202
+2008-04-08 11 316 val_316
+2008-04-08 11 229 val_229
+2008-04-08 11 469 val_469
+2008-04-08 11 463 val_463
+2008-04-08 11 280 val_280
+2008-04-08 11 2 val_2
+2008-04-08 11 35 val_35
+2008-04-08 11 283 val_283
+2008-04-08 11 331 val_331
+2008-04-08 11 235 val_235
+2008-04-08 11 80 val_80
+2008-04-08 11 44 val_44
+2008-04-08 11 193 val_193
+2008-04-08 11 321 val_321
+2008-04-08 11 335 val_335
+2008-04-08 11 104 val_104
+2008-04-08 11 466 val_466
+2008-04-08 11 366 val_366
+2008-04-08 11 175 val_175
+2008-04-08 11 403 val_403
+2008-04-08 11 483 val_483
+2008-04-08 11 53 val_53
+2008-04-08 11 105 val_105
+2008-04-08 11 257 val_257
+2008-04-08 11 406 val_406
+2008-04-08 11 409 val_409
+2008-04-08 11 190 val_190
+2008-04-08 11 406 val_406
+2008-04-08 11 401 val_401
+2008-04-08 11 114 val_114
+2008-04-08 11 258 val_258
+2008-04-08 11 90 val_90
+2008-04-08 11 203 val_203
+2008-04-08 11 262 val_262
+2008-04-08 11 348 val_348
+2008-04-08 11 424 val_424
+2008-04-08 11 12 val_12
+2008-04-08 11 396 val_396
+2008-04-08 11 201 val_201
+2008-04-08 11 217 val_217
+2008-04-08 11 164 val_164
+2008-04-08 11 431 val_431
+2008-04-08 11 454 val_454
+2008-04-08 11 478 val_478
+2008-04-08 11 298 val_298
+2008-04-08 11 125 val_125
+2008-04-08 11 431 val_431
+2008-04-08 11 164 val_164
+2008-04-08 11 424 val_424
+2008-04-08 11 187 val_187
+2008-04-08 11 382 val_382
+2008-04-08 11 5 val_5
+2008-04-08 11 70 val_70
+2008-04-08 11 397 val_397
+2008-04-08 11 480 val_480
+2008-04-08 11 291 val_291
+2008-04-08 11 24 val_24
+2008-04-08 11 351 val_351
+2008-04-08 11 255 val_255
+2008-04-08 11 104 val_104
+2008-04-08 11 70 val_70
+2008-04-08 11 163 val_163
+2008-04-08 11 438 val_438
+2008-04-08 11 119 val_119
+2008-04-08 11 414 val_414
+2008-04-08 11 200 val_200
+2008-04-08 11 491 val_491
+2008-04-08 11 237 val_237
+2008-04-08 11 439 val_439
+2008-04-08 11 360 val_360
+2008-04-08 11 248 val_248
+2008-04-08 11 479 val_479
+2008-04-08 11 305 val_305
+2008-04-08 11 417 val_417
+2008-04-08 11 199 val_199
+2008-04-08 11 444 val_444
+2008-04-08 11 120 val_120
+2008-04-08 11 429 val_429
+2008-04-08 11 169 val_169
+2008-04-08 11 443 val_443
+2008-04-08 11 323 val_323
+2008-04-08 11 325 val_325
+2008-04-08 11 277 val_277
+2008-04-08 11 230 val_230
+2008-04-08 11 478 val_478
+2008-04-08 11 178 val_178
+2008-04-08 11 468 val_468
+2008-04-08 11 310 val_310
+2008-04-08 11 317 val_317
+2008-04-08 11 333 val_333
+2008-04-08 11 493 val_493
+2008-04-08 11 460 val_460
+2008-04-08 11 207 val_207
+2008-04-08 11 249 val_249
+2008-04-08 11 265 val_265
+2008-04-08 11 480 val_480
+2008-04-08 11 83 val_83
+2008-04-08 11 136 val_136
+2008-04-08 11 353 val_353
+2008-04-08 11 172 val_172
+2008-04-08 11 214 val_214
+2008-04-08 11 462 val_462
+2008-04-08 11 233 val_233
+2008-04-08 11 406 val_406
+2008-04-08 11 133 val_133
+2008-04-08 11 175 val_175
+2008-04-08 11 189 val_189
+2008-04-08 11 454 val_454
+2008-04-08 11 375 val_375
+2008-04-08 11 401 val_401
+2008-04-08 11 421 val_421
+2008-04-08 11 407 val_407
+2008-04-08 11 384 val_384
+2008-04-08 11 256 val_256
+2008-04-08 11 26 val_26
+2008-04-08 11 134 val_134
+2008-04-08 11 67 val_67
+2008-04-08 11 384 val_384
+2008-04-08 11 379 val_379
+2008-04-08 11 18 val_18
+2008-04-08 11 462 val_462
+2008-04-08 11 492 val_492
+2008-04-08 11 100 val_100
+2008-04-08 11 298 val_298
+2008-04-08 11 9 val_9
+2008-04-08 11 341 val_341
+2008-04-08 11 498 val_498
+2008-04-08 11 146 val_146
+2008-04-08 11 458 val_458
+2008-04-08 11 362 val_362
+2008-04-08 11 186 val_186
+2008-04-08 11 285 val_285
+2008-04-08 11 348 val_348
+2008-04-08 11 167 val_167
+2008-04-08 11 18 val_18
+2008-04-08 11 273 val_273
+2008-04-08 11 183 val_183
+2008-04-08 11 281 val_281
+2008-04-08 11 344 val_344
+2008-04-08 11 97 val_97
+2008-04-08 11 469 val_469
+2008-04-08 11 315 val_315
+2008-04-08 11 84 val_84
+2008-04-08 11 28 val_28
+2008-04-08 11 37 val_37
+2008-04-08 11 448 val_448
+2008-04-08 11 152 val_152
+2008-04-08 11 348 val_348
+2008-04-08 11 307 val_307
+2008-04-08 11 194 val_194
+2008-04-08 11 414 val_414
+2008-04-08 11 477 val_477
+2008-04-08 11 222 val_222
+2008-04-08 11 126 val_126
+2008-04-08 11 90 val_90
+2008-04-08 11 169 val_169
+2008-04-08 11 403 val_403
+2008-04-08 11 400 val_400
+2008-04-08 11 200 val_200
+2008-04-08 11 97 val_97
+2008-04-08 12 238 val_238
+2008-04-08 12 86 val_86
+2008-04-08 12 311 val_311
+2008-04-08 12 27 val_27
+2008-04-08 12 165 val_165
+2008-04-08 12 409 val_409
+2008-04-08 12 255 val_255
+2008-04-08 12 278 val_278
+2008-04-08 12 98 val_98
+2008-04-08 12 484 val_484
+2008-04-08 12 265 val_265
+2008-04-08 12 193 val_193
+2008-04-08 12 401 val_401
+2008-04-08 12 150 val_150
+2008-04-08 12 273 val_273
+2008-04-08 12 224 val_224
+2008-04-08 12 369 val_369
+2008-04-08 12 66 val_66
+2008-04-08 12 128 val_128
+2008-04-08 12 213 val_213
+2008-04-08 12 146 val_146
+2008-04-08 12 406 val_406
+2008-04-08 12 429 val_429
+2008-04-08 12 374 val_374
+2008-04-08 12 152 val_152
+2008-04-08 12 469 val_469
+2008-04-08 12 145 val_145
+2008-04-08 12 495 val_495
+2008-04-08 12 37 val_37
+2008-04-08 12 327 val_327
+2008-04-08 12 281 val_281
+2008-04-08 12 277 val_277
+2008-04-08 12 209 val_209
+2008-04-08 12 15 val_15
+2008-04-08 12 82 val_82
+2008-04-08 12 403 val_403
+2008-04-08 12 166 val_166
+2008-04-08 12 417 val_417
+2008-04-08 12 430 val_430
+2008-04-08 12 252 val_252
+2008-04-08 12 292 val_292
+2008-04-08 12 219 val_219
+2008-04-08 12 287 val_287
+2008-04-08 12 153 val_153
+2008-04-08 12 193 val_193
+2008-04-08 12 338 val_338
+2008-04-08 12 446 val_446
+2008-04-08 12 459 val_459
+2008-04-08 12 394 val_394
+2008-04-08 12 237 val_237
+2008-04-08 12 482 val_482
+2008-04-08 12 174 val_174
+2008-04-08 12 413 val_413
+2008-04-08 12 494 val_494
+2008-04-08 12 207 val_207
+2008-04-08 12 199 val_199
+2008-04-08 12 466 val_466
+2008-04-08 12 208 val_208
+2008-04-08 12 174 val_174
+2008-04-08 12 399 val_399
+2008-04-08 12 396 val_396
+2008-04-08 12 247 val_247
+2008-04-08 12 417 val_417
+2008-04-08 12 489 val_489
+2008-04-08 12 162 val_162
+2008-04-08 12 377 val_377
+2008-04-08 12 397 val_397
+2008-04-08 12 309 val_309
+2008-04-08 12 365 val_365
+2008-04-08 12 266 val_266
+2008-04-08 12 439 val_439
+2008-04-08 12 342 val_342
+2008-04-08 12 367 val_367
+2008-04-08 12 325 val_325
+2008-04-08 12 167 val_167
+2008-04-08 12 195 val_195
+2008-04-08 12 475 val_475
+2008-04-08 12 17 val_17
+2008-04-08 12 113 val_113
+2008-04-08 12 155 val_155
+2008-04-08 12 203 val_203
+2008-04-08 12 339 val_339
+2008-04-08 12 0 val_0
+2008-04-08 12 455 val_455
+2008-04-08 12 128 val_128
+2008-04-08 12 311 val_311
+2008-04-08 12 316 val_316
+2008-04-08 12 57 val_57
+2008-04-08 12 302 val_302
+2008-04-08 12 205 val_205
+2008-04-08 12 149 val_149
+2008-04-08 12 438 val_438
+2008-04-08 12 345 val_345
+2008-04-08 12 129 val_129
+2008-04-08 12 170 val_170
+2008-04-08 12 20 val_20
+2008-04-08 12 489 val_489
+2008-04-08 12 157 val_157
+2008-04-08 12 378 val_378
+2008-04-08 12 221 val_221
+2008-04-08 12 92 val_92
+2008-04-08 12 111 val_111
+2008-04-08 12 47 val_47
+2008-04-08 12 72 val_72
+2008-04-08 12 4 val_4
+2008-04-08 12 280 val_280
+2008-04-08 12 35 val_35
+2008-04-08 12 427 val_427
+2008-04-08 12 277 val_277
+2008-04-08 12 208 val_208
+2008-04-08 12 356 val_356
+2008-04-08 12 399 val_399
+2008-04-08 12 169 val_169
+2008-04-08 12 382 val_382
+2008-04-08 12 498 val_498
+2008-04-08 12 125 val_125
+2008-04-08 12 386 val_386
+2008-04-08 12 437 val_437
+2008-04-08 12 469 val_469
+2008-04-08 12 192 val_192
+2008-04-08 12 286 val_286
+2008-04-08 12 187 val_187
+2008-04-08 12 176 val_176
+2008-04-08 12 54 val_54
+2008-04-08 12 459 val_459
+2008-04-08 12 51 val_51
+2008-04-08 12 138 val_138
+2008-04-08 12 103 val_103
+2008-04-08 12 239 val_239
+2008-04-08 12 213 val_213
+2008-04-08 12 216 val_216
+2008-04-08 12 430 val_430
+2008-04-08 12 278 val_278
+2008-04-08 12 176 val_176
+2008-04-08 12 289 val_289
+2008-04-08 12 221 val_221
+2008-04-08 12 65 val_65
+2008-04-08 12 318 val_318
+2008-04-08 12 332 val_332
+2008-04-08 12 311 val_311
+2008-04-08 12 275 val_275
+2008-04-08 12 137 val_137
+2008-04-08 12 241 val_241
+2008-04-08 12 83 val_83
+2008-04-08 12 333 val_333
+2008-04-08 12 180 val_180
+2008-04-08 12 284 val_284
+2008-04-08 12 12 val_12
+2008-04-08 12 230 val_230
+2008-04-08 12 181 val_181
+2008-04-08 12 67 val_67
+2008-04-08 12 260 val_260
+2008-04-08 12 404 val_404
+2008-04-08 12 384 val_384
+2008-04-08 12 489 val_489
+2008-04-08 12 353 val_353
+2008-04-08 12 373 val_373
+2008-04-08 12 272 val_272
+2008-04-08 12 138 val_138
+2008-04-08 12 217 val_217
+2008-04-08 12 84 val_84
+2008-04-08 12 348 val_348
+2008-04-08 12 466 val_466
+2008-04-08 12 58 val_58
+2008-04-08 12 8 val_8
+2008-04-08 12 411 val_411
+2008-04-08 12 230 val_230
+2008-04-08 12 208 val_208
+2008-04-08 12 348 val_348
+2008-04-08 12 24 val_24
+2008-04-08 12 463 val_463
+2008-04-08 12 431 val_431
+2008-04-08 12 179 val_179
+2008-04-08 12 172 val_172
+2008-04-08 12 42 val_42
+2008-04-08 12 129 val_129
+2008-04-08 12 158 val_158
+2008-04-08 12 119 val_119
+2008-04-08 12 496 val_496
+2008-04-08 12 0 val_0
+2008-04-08 12 322 val_322
+2008-04-08 12 197 val_197
+2008-04-08 12 468 val_468
+2008-04-08 12 393 val_393
+2008-04-08 12 454 val_454
+2008-04-08 12 100 val_100
+2008-04-08 12 298 val_298
+2008-04-08 12 199 val_199
+2008-04-08 12 191 val_191
+2008-04-08 12 418 val_418
+2008-04-08 12 96 val_96
+2008-04-08 12 26 val_26
+2008-04-08 12 165 val_165
+2008-04-08 12 327 val_327
+2008-04-08 12 230 val_230
+2008-04-08 12 205 val_205
+2008-04-08 12 120 val_120
+2008-04-08 12 131 val_131
+2008-04-08 12 51 val_51
+2008-04-08 12 404 val_404
+2008-04-08 12 43 val_43
+2008-04-08 12 436 val_436
+2008-04-08 12 156 val_156
+2008-04-08 12 469 val_469
+2008-04-08 12 468 val_468
+2008-04-08 12 308 val_308
+2008-04-08 12 95 val_95
+2008-04-08 12 196 val_196
+2008-04-08 12 288 val_288
+2008-04-08 12 481 val_481
+2008-04-08 12 457 val_457
+2008-04-08 12 98 val_98
+2008-04-08 12 282 val_282
+2008-04-08 12 197 val_197
+2008-04-08 12 187 val_187
+2008-04-08 12 318 val_318
+2008-04-08 12 318 val_318
+2008-04-08 12 409 val_409
+2008-04-08 12 470 val_470
+2008-04-08 12 137 val_137
+2008-04-08 12 369 val_369
+2008-04-08 12 316 val_316
+2008-04-08 12 169 val_169
+2008-04-08 12 413 val_413
+2008-04-08 12 85 val_85
+2008-04-08 12 77 val_77
+2008-04-08 12 0 val_0
+2008-04-08 12 490 val_490
+2008-04-08 12 87 val_87
+2008-04-08 12 364 val_364
+2008-04-08 12 179 val_179
+2008-04-08 12 118 val_118
+2008-04-08 12 134 val_134
+2008-04-08 12 395 val_395
+2008-04-08 12 282 val_282
+2008-04-08 12 138 val_138
+2008-04-08 12 238 val_238
+2008-04-08 12 419 val_419
+2008-04-08 12 15 val_15
+2008-04-08 12 118 val_118
+2008-04-08 12 72 val_72
+2008-04-08 12 90 val_90
+2008-04-08 12 307 val_307
+2008-04-08 12 19 val_19
+2008-04-08 12 435 val_435
+2008-04-08 12 10 val_10
+2008-04-08 12 277 val_277
+2008-04-08 12 273 val_273
+2008-04-08 12 306 val_306
+2008-04-08 12 224 val_224
+2008-04-08 12 309 val_309
+2008-04-08 12 389 val_389
+2008-04-08 12 327 val_327
+2008-04-08 12 242 val_242
+2008-04-08 12 369 val_369
+2008-04-08 12 392 val_392
+2008-04-08 12 272 val_272
+2008-04-08 12 331 val_331
+2008-04-08 12 401 val_401
+2008-04-08 12 242 val_242
+2008-04-08 12 452 val_452
+2008-04-08 12 177 val_177
+2008-04-08 12 226 val_226
+2008-04-08 12 5 val_5
+2008-04-08 12 497 val_497
+2008-04-08 12 402 val_402
+2008-04-08 12 396 val_396
+2008-04-08 12 317 val_317
+2008-04-08 12 395 val_395
+2008-04-08 12 58 val_58
+2008-04-08 12 35 val_35
+2008-04-08 12 336 val_336
+2008-04-08 12 95 val_95
+2008-04-08 12 11 val_11
+2008-04-08 12 168 val_168
+2008-04-08 12 34 val_34
+2008-04-08 12 229 val_229
+2008-04-08 12 233 val_233
+2008-04-08 12 143 val_143
+2008-04-08 12 472 val_472
+2008-04-08 12 322 val_322
+2008-04-08 12 498 val_498
+2008-04-08 12 160 val_160
+2008-04-08 12 195 val_195
+2008-04-08 12 42 val_42
+2008-04-08 12 321 val_321
+2008-04-08 12 430 val_430
+2008-04-08 12 119 val_119
+2008-04-08 12 489 val_489
+2008-04-08 12 458 val_458
+2008-04-08 12 78 val_78
+2008-04-08 12 76 val_76
+2008-04-08 12 41 val_41
+2008-04-08 12 223 val_223
+2008-04-08 12 492 val_492
+2008-04-08 12 149 val_149
+2008-04-08 12 449 val_449
+2008-04-08 12 218 val_218
+2008-04-08 12 228 val_228
+2008-04-08 12 138 val_138
+2008-04-08 12 453 val_453
+2008-04-08 12 30 val_30
+2008-04-08 12 209 val_209
+2008-04-08 12 64 val_64
+2008-04-08 12 468 val_468
+2008-04-08 12 76 val_76
+2008-04-08 12 74 val_74
+2008-04-08 12 342 val_342
+2008-04-08 12 69 val_69
+2008-04-08 12 230 val_230
+2008-04-08 12 33 val_33
+2008-04-08 12 368 val_368
+2008-04-08 12 103 val_103
+2008-04-08 12 296 val_296
+2008-04-08 12 113 val_113
+2008-04-08 12 216 val_216
+2008-04-08 12 367 val_367
+2008-04-08 12 344 val_344
+2008-04-08 12 167 val_167
+2008-04-08 12 274 val_274
+2008-04-08 12 219 val_219
+2008-04-08 12 239 val_239
+2008-04-08 12 485 val_485
+2008-04-08 12 116 val_116
+2008-04-08 12 223 val_223
+2008-04-08 12 256 val_256
+2008-04-08 12 263 val_263
+2008-04-08 12 70 val_70
+2008-04-08 12 487 val_487
+2008-04-08 12 480 val_480
+2008-04-08 12 401 val_401
+2008-04-08 12 288 val_288
+2008-04-08 12 191 val_191
+2008-04-08 12 5 val_5
+2008-04-08 12 244 val_244
+2008-04-08 12 438 val_438
+2008-04-08 12 128 val_128
+2008-04-08 12 467 val_467
+2008-04-08 12 432 val_432
+2008-04-08 12 202 val_202
+2008-04-08 12 316 val_316
+2008-04-08 12 229 val_229
+2008-04-08 12 469 val_469
+2008-04-08 12 463 val_463
+2008-04-08 12 280 val_280
+2008-04-08 12 2 val_2
+2008-04-08 12 35 val_35
+2008-04-08 12 283 val_283
+2008-04-08 12 331 val_331
+2008-04-08 12 235 val_235
+2008-04-08 12 80 val_80
+2008-04-08 12 44 val_44
+2008-04-08 12 193 val_193
+2008-04-08 12 321 val_321
+2008-04-08 12 335 val_335
+2008-04-08 12 104 val_104
+2008-04-08 12 466 val_466
+2008-04-08 12 366 val_366
+2008-04-08 12 175 val_175
+2008-04-08 12 403 val_403
+2008-04-08 12 483 val_483
+2008-04-08 12 53 val_53
+2008-04-08 12 105 val_105
+2008-04-08 12 257 val_257
+2008-04-08 12 406 val_406
+2008-04-08 12 409 val_409
+2008-04-08 12 190 val_190
+2008-04-08 12 406 val_406
+2008-04-08 12 401 val_401
+2008-04-08 12 114 val_114
+2008-04-08 12 258 val_258
+2008-04-08 12 90 val_90
+2008-04-08 12 203 val_203
+2008-04-08 12 262 val_262
+2008-04-08 12 348 val_348
+2008-04-08 12 424 val_424
+2008-04-08 12 12 val_12
+2008-04-08 12 396 val_396
+2008-04-08 12 201 val_201
+2008-04-08 12 217 val_217
+2008-04-08 12 164 val_164
+2008-04-08 12 431 val_431
+2008-04-08 12 454 val_454
+2008-04-08 12 478 val_478
+2008-04-08 12 298 val_298
+2008-04-08 12 125 val_125
+2008-04-08 12 431 val_431
+2008-04-08 12 164 val_164
+2008-04-08 12 424 val_424
+2008-04-08 12 187 val_187
+2008-04-08 12 382 val_382
+2008-04-08 12 5 val_5
+2008-04-08 12 70 val_70
+2008-04-08 12 397 val_397
+2008-04-08 12 480 val_480
+2008-04-08 12 291 val_291
+2008-04-08 12 24 val_24
+2008-04-08 12 351 val_351
+2008-04-08 12 255 val_255
+2008-04-08 12 104 val_104
+2008-04-08 12 70 val_70
+2008-04-08 12 163 val_163
+2008-04-08 12 438 val_438
+2008-04-08 12 119 val_119
+2008-04-08 12 414 val_414
+2008-04-08 12 200 val_200
+2008-04-08 12 491 val_491
+2008-04-08 12 237 val_237
+2008-04-08 12 439 val_439
+2008-04-08 12 360 val_360
+2008-04-08 12 248 val_248
+2008-04-08 12 479 val_479
+2008-04-08 12 305 val_305
+2008-04-08 12 417 val_417
+2008-04-08 12 199 val_199
+2008-04-08 12 444 val_444
+2008-04-08 12 120 val_120
+2008-04-08 12 429 val_429
+2008-04-08 12 169 val_169
+2008-04-08 12 443 val_443
+2008-04-08 12 323 val_323
+2008-04-08 12 325 val_325
+2008-04-08 12 277 val_277
+2008-04-08 12 230 val_230
+2008-04-08 12 478 val_478
+2008-04-08 12 178 val_178
+2008-04-08 12 468 val_468
+2008-04-08 12 310 val_310
+2008-04-08 12 317 val_317
+2008-04-08 12 333 val_333
+2008-04-08 12 493 val_493
+2008-04-08 12 460 val_460
+2008-04-08 12 207 val_207
+2008-04-08 12 249 val_249
+2008-04-08 12 265 val_265
+2008-04-08 12 480 val_480
+2008-04-08 12 83 val_83
+2008-04-08 12 136 val_136
+2008-04-08 12 353 val_353
+2008-04-08 12 172 val_172
+2008-04-08 12 214 val_214
+2008-04-08 12 462 val_462
+2008-04-08 12 233 val_233
+2008-04-08 12 406 val_406
+2008-04-08 12 133 val_133
+2008-04-08 12 175 val_175
+2008-04-08 12 189 val_189
+2008-04-08 12 454 val_454
+2008-04-08 12 375 val_375
+2008-04-08 12 401 val_401
+2008-04-08 12 421 val_421
+2008-04-08 12 407 val_407
+2008-04-08 12 384 val_384
+2008-04-08 12 256 val_256
+2008-04-08 12 26 val_26
+2008-04-08 12 134 val_134
+2008-04-08 12 67 val_67
+2008-04-08 12 384 val_384
+2008-04-08 12 379 val_379
+2008-04-08 12 18 val_18
+2008-04-08 12 462 val_462
+2008-04-08 12 492 val_492
+2008-04-08 12 100 val_100
+2008-04-08 12 298 val_298
+2008-04-08 12 9 val_9
+2008-04-08 12 341 val_341
+2008-04-08 12 498 val_498
+2008-04-08 12 146 val_146
+2008-04-08 12 458 val_458
+2008-04-08 12 362 val_362
+2008-04-08 12 186 val_186
+2008-04-08 12 285 val_285
+2008-04-08 12 348 val_348
+2008-04-08 12 167 val_167
+2008-04-08 12 18 val_18
+2008-04-08 12 273 val_273
+2008-04-08 12 183 val_183
+2008-04-08 12 281 val_281
+2008-04-08 12 344 val_344
+2008-04-08 12 97 val_97
+2008-04-08 12 469 val_469
+2008-04-08 12 315 val_315
+2008-04-08 12 84 val_84
+2008-04-08 12 28 val_28
+2008-04-08 12 37 val_37
+2008-04-08 12 448 val_448
+2008-04-08 12 152 val_152
+2008-04-08 12 348 val_348
+2008-04-08 12 307 val_307
+2008-04-08 12 194 val_194
+2008-04-08 12 414 val_414
+2008-04-08 12 477 val_477
+2008-04-08 12 222 val_222
+2008-04-08 12 126 val_126
+2008-04-08 12 90 val_90
+2008-04-08 12 169 val_169
+2008-04-08 12 403 val_403
+2008-04-08 12 400 val_400
+2008-04-08 12 200 val_200
+2008-04-08 12 97 val_97
+2008-04-09 11 238 val_238
+2008-04-09 11 86 val_86
+2008-04-09 11 311 val_311
+2008-04-09 11 27 val_27
+2008-04-09 11 165 val_165
+2008-04-09 11 409 val_409
+2008-04-09 11 255 val_255
+2008-04-09 11 278 val_278
+2008-04-09 11 98 val_98
+2008-04-09 11 484 val_484
+2008-04-09 11 265 val_265
+2008-04-09 11 193 val_193
+2008-04-09 11 401 val_401
+2008-04-09 11 150 val_150
+2008-04-09 11 273 val_273
+2008-04-09 11 224 val_224
+2008-04-09 11 369 val_369
+2008-04-09 11 66 val_66
+2008-04-09 11 128 val_128
+2008-04-09 11 213 val_213
+2008-04-09 11 146 val_146
+2008-04-09 11 406 val_406
+2008-04-09 11 429 val_429
+2008-04-09 11 374 val_374
+2008-04-09 11 152 val_152
+2008-04-09 11 469 val_469
+2008-04-09 11 145 val_145
+2008-04-09 11 495 val_495
+2008-04-09 11 37 val_37
+2008-04-09 11 327 val_327
+2008-04-09 11 281 val_281
+2008-04-09 11 277 val_277
+2008-04-09 11 209 val_209
+2008-04-09 11 15 val_15
+2008-04-09 11 82 val_82
+2008-04-09 11 403 val_403
+2008-04-09 11 166 val_166
+2008-04-09 11 417 val_417
+2008-04-09 11 430 val_430
+2008-04-09 11 252 val_252
+2008-04-09 11 292 val_292
+2008-04-09 11 219 val_219
+2008-04-09 11 287 val_287
+2008-04-09 11 153 val_153
+2008-04-09 11 193 val_193
+2008-04-09 11 338 val_338
+2008-04-09 11 446 val_446
+2008-04-09 11 459 val_459
+2008-04-09 11 394 val_394
+2008-04-09 11 237 val_237
+2008-04-09 11 482 val_482
+2008-04-09 11 174 val_174
+2008-04-09 11 413 val_413
+2008-04-09 11 494 val_494
+2008-04-09 11 207 val_207
+2008-04-09 11 199 val_199
+2008-04-09 11 466 val_466
+2008-04-09 11 208 val_208
+2008-04-09 11 174 val_174
+2008-04-09 11 399 val_399
+2008-04-09 11 396 val_396
+2008-04-09 11 247 val_247
+2008-04-09 11 417 val_417
+2008-04-09 11 489 val_489
+2008-04-09 11 162 val_162
+2008-04-09 11 377 val_377
+2008-04-09 11 397 val_397
+2008-04-09 11 309 val_309
+2008-04-09 11 365 val_365
+2008-04-09 11 266 val_266
+2008-04-09 11 439 val_439
+2008-04-09 11 342 val_342
+2008-04-09 11 367 val_367
+2008-04-09 11 325 val_325
+2008-04-09 11 167 val_167
+2008-04-09 11 195 val_195
+2008-04-09 11 475 val_475
+2008-04-09 11 17 val_17
+2008-04-09 11 113 val_113
+2008-04-09 11 155 val_155
+2008-04-09 11 203 val_203
+2008-04-09 11 339 val_339
+2008-04-09 11 0 val_0
+2008-04-09 11 455 val_455
+2008-04-09 11 128 val_128
+2008-04-09 11 311 val_311
+2008-04-09 11 316 val_316
+2008-04-09 11 57 val_57
+2008-04-09 11 302 val_302
+2008-04-09 11 205 val_205
+2008-04-09 11 149 val_149
+2008-04-09 11 438 val_438
+2008-04-09 11 345 val_345
+2008-04-09 11 129 val_129
+2008-04-09 11 170 val_170
+2008-04-09 11 20 val_20
+2008-04-09 11 489 val_489
+2008-04-09 11 157 val_157
+2008-04-09 11 378 val_378
+2008-04-09 11 221 val_221
+2008-04-09 11 92 val_92
+2008-04-09 11 111 val_111
+2008-04-09 11 47 val_47
+2008-04-09 11 72 val_72
+2008-04-09 11 4 val_4
+2008-04-09 11 280 val_280
+2008-04-09 11 35 val_35
+2008-04-09 11 427 val_427
+2008-04-09 11 277 val_277
+2008-04-09 11 208 val_208
+2008-04-09 11 356 val_356
+2008-04-09 11 399 val_399
+2008-04-09 11 169 val_169
+2008-04-09 11 382 val_382
+2008-04-09 11 498 val_498
+2008-04-09 11 125 val_125
+2008-04-09 11 386 val_386
+2008-04-09 11 437 val_437
+2008-04-09 11 469 val_469
+2008-04-09 11 192 val_192
+2008-04-09 11 286 val_286
+2008-04-09 11 187 val_187
+2008-04-09 11 176 val_176
+2008-04-09 11 54 val_54
+2008-04-09 11 459 val_459
+2008-04-09 11 51 val_51
+2008-04-09 11 138 val_138
+2008-04-09 11 103 val_103
+2008-04-09 11 239 val_239
+2008-04-09 11 213 val_213
+2008-04-09 11 216 val_216
+2008-04-09 11 430 val_430
+2008-04-09 11 278 val_278
+2008-04-09 11 176 val_176
+2008-04-09 11 289 val_289
+2008-04-09 11 221 val_221
+2008-04-09 11 65 val_65
+2008-04-09 11 318 val_318
+2008-04-09 11 332 val_332
+2008-04-09 11 311 val_311
+2008-04-09 11 275 val_275
+2008-04-09 11 137 val_137
+2008-04-09 11 241 val_241
+2008-04-09 11 83 val_83
+2008-04-09 11 333 val_333
+2008-04-09 11 180 val_180
+2008-04-09 11 284 val_284
+2008-04-09 11 12 val_12
+2008-04-09 11 230 val_230
+2008-04-09 11 181 val_181
+2008-04-09 11 67 val_67
+2008-04-09 11 260 val_260
+2008-04-09 11 404 val_404
+2008-04-09 11 384 val_384
+2008-04-09 11 489 val_489
+2008-04-09 11 353 val_353
+2008-04-09 11 373 val_373
+2008-04-09 11 272 val_272
+2008-04-09 11 138 val_138
+2008-04-09 11 217 val_217
+2008-04-09 11 84 val_84
+2008-04-09 11 348 val_348
+2008-04-09 11 466 val_466
+2008-04-09 11 58 val_58
+2008-04-09 11 8 val_8
+2008-04-09 11 411 val_411
+2008-04-09 11 230 val_230
+2008-04-09 11 208 val_208
+2008-04-09 11 348 val_348
+2008-04-09 11 24 val_24
+2008-04-09 11 463 val_463
+2008-04-09 11 431 val_431
+2008-04-09 11 179 val_179
+2008-04-09 11 172 val_172
+2008-04-09 11 42 val_42
+2008-04-09 11 129 val_129
+2008-04-09 11 158 val_158
+2008-04-09 11 119 val_119
+2008-04-09 11 496 val_496
+2008-04-09 11 0 val_0
+2008-04-09 11 322 val_322
+2008-04-09 11 197 val_197
+2008-04-09 11 468 val_468
+2008-04-09 11 393 val_393
+2008-04-09 11 454 val_454
+2008-04-09 11 100 val_100
+2008-04-09 11 298 val_298
+2008-04-09 11 199 val_199
+2008-04-09 11 191 val_191
+2008-04-09 11 418 val_418
+2008-04-09 11 96 val_96
+2008-04-09 11 26 val_26
+2008-04-09 11 165 val_165
+2008-04-09 11 327 val_327
+2008-04-09 11 230 val_230
+2008-04-09 11 205 val_205
+2008-04-09 11 120 val_120
+2008-04-09 11 131 val_131
+2008-04-09 11 51 val_51
+2008-04-09 11 404 val_404
+2008-04-09 11 43 val_43
+2008-04-09 11 436 val_436
+2008-04-09 11 156 val_156
+2008-04-09 11 469 val_469
+2008-04-09 11 468 val_468
+2008-04-09 11 308 val_308
+2008-04-09 11 95 val_95
+2008-04-09 11 196 val_196
+2008-04-09 11 288 val_288
+2008-04-09 11 481 val_481
+2008-04-09 11 457 val_457
+2008-04-09 11 98 val_98
+2008-04-09 11 282 val_282
+2008-04-09 11 197 val_197
+2008-04-09 11 187 val_187
+2008-04-09 11 318 val_318
+2008-04-09 11 318 val_318
+2008-04-09 11 409 val_409
+2008-04-09 11 470 val_470
+2008-04-09 11 137 val_137
+2008-04-09 11 369 val_369
+2008-04-09 11 316 val_316
+2008-04-09 11 169 val_169
+2008-04-09 11 413 val_413
+2008-04-09 11 85 val_85
+2008-04-09 11 77 val_77
+2008-04-09 11 0 val_0
+2008-04-09 11 490 val_490
+2008-04-09 11 87 val_87
+2008-04-09 11 364 val_364
+2008-04-09 11 179 val_179
+2008-04-09 11 118 val_118
+2008-04-09 11 134 val_134
+2008-04-09 11 395 val_395
+2008-04-09 11 282 val_282
+2008-04-09 11 138 val_138
+2008-04-09 11 238 val_238
+2008-04-09 11 419 val_419
+2008-04-09 11 15 val_15
+2008-04-09 11 118 val_118
+2008-04-09 11 72 val_72
+2008-04-09 11 90 val_90
+2008-04-09 11 307 val_307
+2008-04-09 11 19 val_19
+2008-04-09 11 435 val_435
+2008-04-09 11 10 val_10
+2008-04-09 11 277 val_277
+2008-04-09 11 273 val_273
+2008-04-09 11 306 val_306
+2008-04-09 11 224 val_224
+2008-04-09 11 309 val_309
+2008-04-09 11 389 val_389
+2008-04-09 11 327 val_327
+2008-04-09 11 242 val_242
+2008-04-09 11 369 val_369
+2008-04-09 11 392 val_392
+2008-04-09 11 272 val_272
+2008-04-09 11 331 val_331
+2008-04-09 11 401 val_401
+2008-04-09 11 242 val_242
+2008-04-09 11 452 val_452
+2008-04-09 11 177 val_177
+2008-04-09 11 226 val_226
+2008-04-09 11 5 val_5
+2008-04-09 11 497 val_497
+2008-04-09 11 402 val_402
+2008-04-09 11 396 val_396
+2008-04-09 11 317 val_317
+2008-04-09 11 395 val_395
+2008-04-09 11 58 val_58
+2008-04-09 11 35 val_35
+2008-04-09 11 336 val_336
+2008-04-09 11 95 val_95
+2008-04-09 11 11 val_11
+2008-04-09 11 168 val_168
+2008-04-09 11 34 val_34
+2008-04-09 11 229 val_229
+2008-04-09 11 233 val_233
+2008-04-09 11 143 val_143
+2008-04-09 11 472 val_472
+2008-04-09 11 322 val_322
+2008-04-09 11 498 val_498
+2008-04-09 11 160 val_160
+2008-04-09 11 195 val_195
+2008-04-09 11 42 val_42
+2008-04-09 11 321 val_321
+2008-04-09 11 430 val_430
+2008-04-09 11 119 val_119
+2008-04-09 11 489 val_489
+2008-04-09 11 458 val_458
+2008-04-09 11 78 val_78
+2008-04-09 11 76 val_76
+2008-04-09 11 41 val_41
+2008-04-09 11 223 val_223
+2008-04-09 11 492 val_492
+2008-04-09 11 149 val_149
+2008-04-09 11 449 val_449
+2008-04-09 11 218 val_218
+2008-04-09 11 228 val_228
+2008-04-09 11 138 val_138
+2008-04-09 11 453 val_453
+2008-04-09 11 30 val_30
+2008-04-09 11 209 val_209
+2008-04-09 11 64 val_64
+2008-04-09 11 468 val_468
+2008-04-09 11 76 val_76
+2008-04-09 11 74 val_74
+2008-04-09 11 342 val_342
+2008-04-09 11 69 val_69
+2008-04-09 11 230 val_230
+2008-04-09 11 33 val_33
+2008-04-09 11 368 val_368
+2008-04-09 11 103 val_103
+2008-04-09 11 296 val_296
+2008-04-09 11 113 val_113
+2008-04-09 11 216 val_216
+2008-04-09 11 367 val_367
+2008-04-09 11 344 val_344
+2008-04-09 11 167 val_167
+2008-04-09 11 274 val_274
+2008-04-09 11 219 val_219
+2008-04-09 11 239 val_239
+2008-04-09 11 485 val_485
+2008-04-09 11 116 val_116
+2008-04-09 11 223 val_223
+2008-04-09 11 256 val_256
+2008-04-09 11 263 val_263
+2008-04-09 11 70 val_70
+2008-04-09 11 487 val_487
+2008-04-09 11 480 val_480
+2008-04-09 11 401 val_401
+2008-04-09 11 288 val_288
+2008-04-09 11 191 val_191
+2008-04-09 11 5 val_5
+2008-04-09 11 244 val_244
+2008-04-09 11 438 val_438
+2008-04-09 11 128 val_128
+2008-04-09 11 467 val_467
+2008-04-09 11 432 val_432
+2008-04-09 11 202 val_202
+2008-04-09 11 316 val_316
+2008-04-09 11 229 val_229
+2008-04-09 11 469 val_469
+2008-04-09 11 463 val_463
+2008-04-09 11 280 val_280
+2008-04-09 11 2 val_2
+2008-04-09 11 35 val_35
+2008-04-09 11 283 val_283
+2008-04-09 11 331 val_331
+2008-04-09 11 235 val_235
+2008-04-09 11 80 val_80
+2008-04-09 11 44 val_44
+2008-04-09 11 193 val_193
+2008-04-09 11 321 val_321
+2008-04-09 11 335 val_335
+2008-04-09 11 104 val_104
+2008-04-09 11 466 val_466
+2008-04-09 11 366 val_366
+2008-04-09 11 175 val_175
+2008-04-09 11 403 val_403
+2008-04-09 11 483 val_483
+2008-04-09 11 53 val_53
+2008-04-09 11 105 val_105
+2008-04-09 11 257 val_257
+2008-04-09 11 406 val_406
+2008-04-09 11 409 val_409
+2008-04-09 11 190 val_190
+2008-04-09 11 406 val_406
+2008-04-09 11 401 val_401
+2008-04-09 11 114 val_114
+2008-04-09 11 258 val_258
+2008-04-09 11 90 val_90
+2008-04-09 11 203 val_203
+2008-04-09 11 262 val_262
+2008-04-09 11 348 val_348
+2008-04-09 11 424 val_424
+2008-04-09 11 12 val_12
+2008-04-09 11 396 val_396
+2008-04-09 11 201 val_201
+2008-04-09 11 217 val_217
+2008-04-09 11 164 val_164
+2008-04-09 11 431 val_431
+2008-04-09 11 454 val_454
+2008-04-09 11 478 val_478
+2008-04-09 11 298 val_298
+2008-04-09 11 125 val_125
+2008-04-09 11 431 val_431
+2008-04-09 11 164 val_164
+2008-04-09 11 424 val_424
+2008-04-09 11 187 val_187
+2008-04-09 11 382 val_382
+2008-04-09 11 5 val_5
+2008-04-09 11 70 val_70
+2008-04-09 11 397 val_397
+2008-04-09 11 480 val_480
+2008-04-09 11 291 val_291
+2008-04-09 11 24 val_24
+2008-04-09 11 351 val_351
+2008-04-09 11 255 val_255
+2008-04-09 11 104 val_104
+2008-04-09 11 70 val_70
+2008-04-09 11 163 val_163
+2008-04-09 11 438 val_438
+2008-04-09 11 119 val_119
+2008-04-09 11 414 val_414
+2008-04-09 11 200 val_200
+2008-04-09 11 491 val_491
+2008-04-09 11 237 val_237
+2008-04-09 11 439 val_439
+2008-04-09 11 360 val_360
+2008-04-09 11 248 val_248
+2008-04-09 11 479 val_479
+2008-04-09 11 305 val_305
+2008-04-09 11 417 val_417
+2008-04-09 11 199 val_199
+2008-04-09 11 444 val_444
+2008-04-09 11 120 val_120
+2008-04-09 11 429 val_429
+2008-04-09 11 169 val_169
+2008-04-09 11 443 val_443
+2008-04-09 11 323 val_323
+2008-04-09 11 325 val_325
+2008-04-09 11 277 val_277
+2008-04-09 11 230 val_230
+2008-04-09 11 478 val_478
+2008-04-09 11 178 val_178
+2008-04-09 11 468 val_468
+2008-04-09 11 310 val_310
+2008-04-09 11 317 val_317
+2008-04-09 11 333 val_333
+2008-04-09 11 493 val_493
+2008-04-09 11 460 val_460
+2008-04-09 11 207 val_207
+2008-04-09 11 249 val_249
+2008-04-09 11 265 val_265
+2008-04-09 11 480 val_480
+2008-04-09 11 83 val_83
+2008-04-09 11 136 val_136
+2008-04-09 11 353 val_353
+2008-04-09 11 172 val_172
+2008-04-09 11 214 val_214
+2008-04-09 11 462 val_462
+2008-04-09 11 233 val_233
+2008-04-09 11 406 val_406
+2008-04-09 11 133 val_133
+2008-04-09 11 175 val_175
+2008-04-09 11 189 val_189
+2008-04-09 11 454 val_454
+2008-04-09 11 375 val_375
+2008-04-09 11 401 val_401
+2008-04-09 11 421 val_421
+2008-04-09 11 407 val_407
+2008-04-09 11 384 val_384
+2008-04-09 11 256 val_256
+2008-04-09 11 26 val_26
+2008-04-09 11 134 val_134
+2008-04-09 11 67 val_67
+2008-04-09 11 384 val_384
+2008-04-09 11 379 val_379
+2008-04-09 11 18 val_18
+2008-04-09 11 462 val_462
+2008-04-09 11 492 val_492
+2008-04-09 11 100 val_100
+2008-04-09 11 298 val_298
+2008-04-09 11 9 val_9
+2008-04-09 11 341 val_341
+2008-04-09 11 498 val_498
+2008-04-09 11 146 val_146
+2008-04-09 11 458 val_458
+2008-04-09 11 362 val_362
+2008-04-09 11 186 val_186
+2008-04-09 11 285 val_285
+2008-04-09 11 348 val_348
+2008-04-09 11 167 val_167
+2008-04-09 11 18 val_18
+2008-04-09 11 273 val_273
+2008-04-09 11 183 val_183
+2008-04-09 11 281 val_281
+2008-04-09 11 344 val_344
+2008-04-09 11 97 val_97
+2008-04-09 11 469 val_469
+2008-04-09 11 315 val_315
+2008-04-09 11 84 val_84
+2008-04-09 11 28 val_28
+2008-04-09 11 37 val_37
+2008-04-09 11 448 val_448
+2008-04-09 11 152 val_152
+2008-04-09 11 348 val_348
+2008-04-09 11 307 val_307
+2008-04-09 11 194 val_194
+2008-04-09 11 414 val_414
+2008-04-09 11 477 val_477
+2008-04-09 11 222 val_222
+2008-04-09 11 126 val_126
+2008-04-09 11 90 val_90
+2008-04-09 11 169 val_169
+2008-04-09 11 403 val_403
+2008-04-09 11 400 val_400
+2008-04-09 11 200 val_200
+2008-04-09 11 97 val_97
+2008-04-09 12 238 val_238
+2008-04-09 12 86 val_86
+2008-04-09 12 311 val_311
+2008-04-09 12 27 val_27
+2008-04-09 12 165 val_165
+2008-04-09 12 409 val_409
+2008-04-09 12 255 val_255
+2008-04-09 12 278 val_278
+2008-04-09 12 98 val_98
+2008-04-09 12 484 val_484
+2008-04-09 12 265 val_265
+2008-04-09 12 193 val_193
+2008-04-09 12 401 val_401
+2008-04-09 12 150 val_150
+2008-04-09 12 273 val_273
+2008-04-09 12 224 val_224
+2008-04-09 12 369 val_369
+2008-04-09 12 66 val_66
+2008-04-09 12 128 val_128
+2008-04-09 12 213 val_213
+2008-04-09 12 146 val_146
+2008-04-09 12 406 val_406
+2008-04-09 12 429 val_429
+2008-04-09 12 374 val_374
+2008-04-09 12 152 val_152
+2008-04-09 12 469 val_469
+2008-04-09 12 145 val_145
+2008-04-09 12 495 val_495
+2008-04-09 12 37 val_37
+2008-04-09 12 327 val_327
+2008-04-09 12 281 val_281
+2008-04-09 12 277 val_277
+2008-04-09 12 209 val_209
+2008-04-09 12 15 val_15
+2008-04-09 12 82 val_82
+2008-04-09 12 403 val_403
+2008-04-09 12 166 val_166
+2008-04-09 12 417 val_417
+2008-04-09 12 430 val_430
+2008-04-09 12 252 val_252
+2008-04-09 12 292 val_292
+2008-04-09 12 219 val_219
+2008-04-09 12 287 val_287
+2008-04-09 12 153 val_153
+2008-04-09 12 193 val_193
+2008-04-09 12 338 val_338
+2008-04-09 12 446 val_446
+2008-04-09 12 459 val_459
+2008-04-09 12 394 val_394
+2008-04-09 12 237 val_237
+2008-04-09 12 482 val_482
+2008-04-09 12 174 val_174
+2008-04-09 12 413 val_413
+2008-04-09 12 494 val_494
+2008-04-09 12 207 val_207
+2008-04-09 12 199 val_199
+2008-04-09 12 466 val_466
+2008-04-09 12 208 val_208
+2008-04-09 12 174 val_174
+2008-04-09 12 399 val_399
+2008-04-09 12 396 val_396
+2008-04-09 12 247 val_247
+2008-04-09 12 417 val_417
+2008-04-09 12 489 val_489
+2008-04-09 12 162 val_162
+2008-04-09 12 377 val_377
+2008-04-09 12 397 val_397
+2008-04-09 12 309 val_309
+2008-04-09 12 365 val_365
+2008-04-09 12 266 val_266
+2008-04-09 12 439 val_439
+2008-04-09 12 342 val_342
+2008-04-09 12 367 val_367
+2008-04-09 12 325 val_325
+2008-04-09 12 167 val_167
+2008-04-09 12 195 val_195
+2008-04-09 12 475 val_475
+2008-04-09 12 17 val_17
+2008-04-09 12 113 val_113
+2008-04-09 12 155 val_155
+2008-04-09 12 203 val_203
+2008-04-09 12 339 val_339
+2008-04-09 12 0 val_0
+2008-04-09 12 455 val_455
+2008-04-09 12 128 val_128
+2008-04-09 12 311 val_311
+2008-04-09 12 316 val_316
+2008-04-09 12 57 val_57
+2008-04-09 12 302 val_302
+2008-04-09 12 205 val_205
+2008-04-09 12 149 val_149
+2008-04-09 12 438 val_438
+2008-04-09 12 345 val_345
+2008-04-09 12 129 val_129
+2008-04-09 12 170 val_170
+2008-04-09 12 20 val_20
+2008-04-09 12 489 val_489
+2008-04-09 12 157 val_157
+2008-04-09 12 378 val_378
+2008-04-09 12 221 val_221
+2008-04-09 12 92 val_92
+2008-04-09 12 111 val_111
+2008-04-09 12 47 val_47
+2008-04-09 12 72 val_72
+2008-04-09 12 4 val_4
+2008-04-09 12 280 val_280
+2008-04-09 12 35 val_35
+2008-04-09 12 427 val_427
+2008-04-09 12 277 val_277
+2008-04-09 12 208 val_208
+2008-04-09 12 356 val_356
+2008-04-09 12 399 val_399
+2008-04-09 12 169 val_169
+2008-04-09 12 382 val_382
+2008-04-09 12 498 val_498
+2008-04-09 12 125 val_125
+2008-04-09 12 386 val_386
+2008-04-09 12 437 val_437
+2008-04-09 12 469 val_469
+2008-04-09 12 192 val_192
+2008-04-09 12 286 val_286
+2008-04-09 12 187 val_187
+2008-04-09 12 176 val_176
+2008-04-09 12 54 val_54
+2008-04-09 12 459 val_459
+2008-04-09 12 51 val_51
+2008-04-09 12 138 val_138
+2008-04-09 12 103 val_103
+2008-04-09 12 239 val_239
+2008-04-09 12 213 val_213
+2008-04-09 12 216 val_216
+2008-04-09 12 430 val_430
+2008-04-09 12 278 val_278
+2008-04-09 12 176 val_176
+2008-04-09 12 289 val_289
+2008-04-09 12 221 val_221
+2008-04-09 12 65 val_65
+2008-04-09 12 318 val_318
+2008-04-09 12 332 val_332
+2008-04-09 12 311 val_311
+2008-04-09 12 275 val_275
+2008-04-09 12 137 val_137
+2008-04-09 12 241 val_241
+2008-04-09 12 83 val_83
+2008-04-09 12 333 val_333
+2008-04-09 12 180 val_180
+2008-04-09 12 284 val_284
+2008-04-09 12 12 val_12
+2008-04-09 12 230 val_230
+2008-04-09 12 181 val_181
+2008-04-09 12 67 val_67
+2008-04-09 12 260 val_260
+2008-04-09 12 404 val_404
+2008-04-09 12 384 val_384
+2008-04-09 12 489 val_489
+2008-04-09 12 353 val_353
+2008-04-09 12 373 val_373
+2008-04-09 12 272 val_272
+2008-04-09 12 138 val_138
+2008-04-09 12 217 val_217
+2008-04-09 12 84 val_84
+2008-04-09 12 348 val_348
+2008-04-09 12 466 val_466
+2008-04-09 12 58 val_58
+2008-04-09 12 8 val_8
+2008-04-09 12 411 val_411
+2008-04-09 12 230 val_230
+2008-04-09 12 208 val_208
+2008-04-09 12 348 val_348
+2008-04-09 12 24 val_24
+2008-04-09 12 463 val_463
+2008-04-09 12 431 val_431
+2008-04-09 12 179 val_179
+2008-04-09 12 172 val_172
+2008-04-09 12 42 val_42
+2008-04-09 12 129 val_129
+2008-04-09 12 158 val_158
+2008-04-09 12 119 val_119
+2008-04-09 12 496 val_496
+2008-04-09 12 0 val_0
+2008-04-09 12 322 val_322
+2008-04-09 12 197 val_197
+2008-04-09 12 468 val_468
+2008-04-09 12 393 val_393
+2008-04-09 12 454 val_454
+2008-04-09 12 100 val_100
+2008-04-09 12 298 val_298
+2008-04-09 12 199 val_199
+2008-04-09 12 191 val_191
+2008-04-09 12 418 val_418
+2008-04-09 12 96 val_96
+2008-04-09 12 26 val_26
+2008-04-09 12 165 val_165
+2008-04-09 12 327 val_327
+2008-04-09 12 230 val_230
+2008-04-09 12 205 val_205
+2008-04-09 12 120 val_120
+2008-04-09 12 131 val_131
+2008-04-09 12 51 val_51
+2008-04-09 12 404 val_404
+2008-04-09 12 43 val_43
+2008-04-09 12 436 val_436
+2008-04-09 12 156 val_156
+2008-04-09 12 469 val_469
+2008-04-09 12 468 val_468
+2008-04-09 12 308 val_308
+2008-04-09 12 95 val_95
+2008-04-09 12 196 val_196
+2008-04-09 12 288 val_288
+2008-04-09 12 481 val_481
+2008-04-09 12 457 val_457
+2008-04-09 12 98 val_98
+2008-04-09 12 282 val_282
+2008-04-09 12 197 val_197
+2008-04-09 12 187 val_187
+2008-04-09 12 318 val_318
+2008-04-09 12 318 val_318
+2008-04-09 12 409 val_409
+2008-04-09 12 470 val_470
+2008-04-09 12 137 val_137
+2008-04-09 12 369 val_369
+2008-04-09 12 316 val_316
+2008-04-09 12 169 val_169
+2008-04-09 12 413 val_413
+2008-04-09 12 85 val_85
+2008-04-09 12 77 val_77
+2008-04-09 12 0 val_0
+2008-04-09 12 490 val_490
+2008-04-09 12 87 val_87
+2008-04-09 12 364 val_364
+2008-04-09 12 179 val_179
+2008-04-09 12 118 val_118
+2008-04-09 12 134 val_134
+2008-04-09 12 395 val_395
+2008-04-09 12 282 val_282
+2008-04-09 12 138 val_138
+2008-04-09 12 238 val_238
+2008-04-09 12 419 val_419
+2008-04-09 12 15 val_15
+2008-04-09 12 118 val_118
+2008-04-09 12 72 val_72
+2008-04-09 12 90 val_90
+2008-04-09 12 307 val_307
+2008-04-09 12 19 val_19
+2008-04-09 12 435 val_435
+2008-04-09 12 10 val_10
+2008-04-09 12 277 val_277
+2008-04-09 12 273 val_273
+2008-04-09 12 306 val_306
+2008-04-09 12 224 val_224
+2008-04-09 12 309 val_309
+2008-04-09 12 389 val_389
+2008-04-09 12 327 val_327
+2008-04-09 12 242 val_242
+2008-04-09 12 369 val_369
+2008-04-09 12 392 val_392
+2008-04-09 12 272 val_272
+2008-04-09 12 331 val_331
+2008-04-09 12 401 val_401
+2008-04-09 12 242 val_242
+2008-04-09 12 452 val_452
+2008-04-09 12 177 val_177
+2008-04-09 12 226 val_226
+2008-04-09 12 5 val_5
+2008-04-09 12 497 val_497
+2008-04-09 12 402 val_402
+2008-04-09 12 396 val_396
+2008-04-09 12 317 val_317
+2008-04-09 12 395 val_395
+2008-04-09 12 58 val_58
+2008-04-09 12 35 val_35
+2008-04-09 12 336 val_336
+2008-04-09 12 95 val_95
+2008-04-09 12 11 val_11
+2008-04-09 12 168 val_168
+2008-04-09 12 34 val_34
+2008-04-09 12 229 val_229
+2008-04-09 12 233 val_233
+2008-04-09 12 143 val_143
+2008-04-09 12 472 val_472
+2008-04-09 12 322 val_322
+2008-04-09 12 498 val_498
+2008-04-09 12 160 val_160
+2008-04-09 12 195 val_195
+2008-04-09 12 42 val_42
+2008-04-09 12 321 val_321
+2008-04-09 12 430 val_430
+2008-04-09 12 119 val_119
+2008-04-09 12 489 val_489
+2008-04-09 12 458 val_458
+2008-04-09 12 78 val_78
+2008-04-09 12 76 val_76
+2008-04-09 12 41 val_41
+2008-04-09 12 223 val_223
+2008-04-09 12 492 val_492
+2008-04-09 12 149 val_149
+2008-04-09 12 449 val_449
+2008-04-09 12 218 val_218
+2008-04-09 12 228 val_228
+2008-04-09 12 138 val_138
+2008-04-09 12 453 val_453
+2008-04-09 12 30 val_30
+2008-04-09 12 209 val_209
+2008-04-09 12 64 val_64
+2008-04-09 12 468 val_468
+2008-04-09 12 76 val_76
+2008-04-09 12 74 val_74
+2008-04-09 12 342 val_342
+2008-04-09 12 69 val_69
+2008-04-09 12 230 val_230
+2008-04-09 12 33 val_33
+2008-04-09 12 368 val_368
+2008-04-09 12 103 val_103
+2008-04-09 12 296 val_296
+2008-04-09 12 113 val_113
+2008-04-09 12 216 val_216
+2008-04-09 12 367 val_367
+2008-04-09 12 344 val_344
+2008-04-09 12 167 val_167
+2008-04-09 12 274 val_274
+2008-04-09 12 219 val_219
+2008-04-09 12 239 val_239
+2008-04-09 12 485 val_485
+2008-04-09 12 116 val_116
+2008-04-09 12 223 val_223
+2008-04-09 12 256 val_256
+2008-04-09 12 263 val_263
+2008-04-09 12 70 val_70
+2008-04-09 12 487 val_487
+2008-04-09 12 480 val_480
+2008-04-09 12 401 val_401
+2008-04-09 12 288 val_288
+2008-04-09 12 191 val_191
+2008-04-09 12 5 val_5
+2008-04-09 12 244 val_244
+2008-04-09 12 438 val_438
+2008-04-09 12 128 val_128
+2008-04-09 12 467 val_467
+2008-04-09 12 432 val_432
+2008-04-09 12 202 val_202
+2008-04-09 12 316 val_316
+2008-04-09 12 229 val_229
+2008-04-09 12 469 val_469
+2008-04-09 12 463 val_463
+2008-04-09 12 280 val_280
+2008-04-09 12 2 val_2
+2008-04-09 12 35 val_35
+2008-04-09 12 283 val_283
+2008-04-09 12 331 val_331
+2008-04-09 12 235 val_235
+2008-04-09 12 80 val_80
+2008-04-09 12 44 val_44
+2008-04-09 12 193 val_193
+2008-04-09 12 321 val_321
+2008-04-09 12 335 val_335
+2008-04-09 12 104 val_104
+2008-04-09 12 466 val_466
+2008-04-09 12 366 val_366
+2008-04-09 12 175 val_175
+2008-04-09 12 403 val_403
+2008-04-09 12 483 val_483
+2008-04-09 12 53 val_53
+2008-04-09 12 105 val_105
+2008-04-09 12 257 val_257
+2008-04-09 12 406 val_406
+2008-04-09 12 409 val_409
+2008-04-09 12 190 val_190
+2008-04-09 12 406 val_406
+2008-04-09 12 401 val_401
+2008-04-09 12 114 val_114
+2008-04-09 12 258 val_258
+2008-04-09 12 90 val_90
+2008-04-09 12 203 val_203
+2008-04-09 12 262 val_262
+2008-04-09 12 348 val_348
+2008-04-09 12 424 val_424
+2008-04-09 12 12 val_12
+2008-04-09 12 396 val_396
+2008-04-09 12 201 val_201
+2008-04-09 12 217 val_217
+2008-04-09 12 164 val_164
+2008-04-09 12 431 val_431
+2008-04-09 12 454 val_454
+2008-04-09 12 478 val_478
+2008-04-09 12 298 val_298
+2008-04-09 12 125 val_125
+2008-04-09 12 431 val_431
+2008-04-09 12 164 val_164
+2008-04-09 12 424 val_424
+2008-04-09 12 187 val_187
+2008-04-09 12 382 val_382
+2008-04-09 12 5 val_5
+2008-04-09 12 70 val_70
+2008-04-09 12 397 val_397
+2008-04-09 12 480 val_480
+2008-04-09 12 291 val_291
+2008-04-09 12 24 val_24
+2008-04-09 12 351 val_351
+2008-04-09 12 255 val_255
+2008-04-09 12 104 val_104
+2008-04-09 12 70 val_70
+2008-04-09 12 163 val_163
+2008-04-09 12 438 val_438
+2008-04-09 12 119 val_119
+2008-04-09 12 414 val_414
+2008-04-09 12 200 val_200
+2008-04-09 12 491 val_491
+2008-04-09 12 237 val_237
+2008-04-09 12 439 val_439
+2008-04-09 12 360 val_360
+2008-04-09 12 248 val_248
+2008-04-09 12 479 val_479
+2008-04-09 12 305 val_305
+2008-04-09 12 417 val_417
+2008-04-09 12 199 val_199
+2008-04-09 12 444 val_444
+2008-04-09 12 120 val_120
+2008-04-09 12 429 val_429
+2008-04-09 12 169 val_169
+2008-04-09 12 443 val_443
+2008-04-09 12 323 val_323
+2008-04-09 12 325 val_325
+2008-04-09 12 277 val_277
+2008-04-09 12 230 val_230
+2008-04-09 12 478 val_478
+2008-04-09 12 178 val_178
+2008-04-09 12 468 val_468
+2008-04-09 12 310 val_310
+2008-04-09 12 317 val_317
+2008-04-09 12 333 val_333
+2008-04-09 12 493 val_493
+2008-04-09 12 460 val_460
+2008-04-09 12 207 val_207
+2008-04-09 12 249 val_249
+2008-04-09 12 265 val_265
+2008-04-09 12 480 val_480
+2008-04-09 12 83 val_83
+2008-04-09 12 136 val_136
+2008-04-09 12 353 val_353
+2008-04-09 12 172 val_172
+2008-04-09 12 214 val_214
+2008-04-09 12 462 val_462
+2008-04-09 12 233 val_233
+2008-04-09 12 406 val_406
+2008-04-09 12 133 val_133
+2008-04-09 12 175 val_175
+2008-04-09 12 189 val_189
+2008-04-09 12 454 val_454
+2008-04-09 12 375 val_375
+2008-04-09 12 401 val_401
+2008-04-09 12 421 val_421
+2008-04-09 12 407 val_407
+2008-04-09 12 384 val_384
+2008-04-09 12 256 val_256
+2008-04-09 12 26 val_26
+2008-04-09 12 134 val_134
+2008-04-09 12 67 val_67
+2008-04-09 12 384 val_384
+2008-04-09 12 379 val_379
+2008-04-09 12 18 val_18
+2008-04-09 12 462 val_462
+2008-04-09 12 492 val_492
+2008-04-09 12 100 val_100
+2008-04-09 12 298 val_298
+2008-04-09 12 9 val_9
+2008-04-09 12 341 val_341
+2008-04-09 12 498 val_498
+2008-04-09 12 146 val_146
+2008-04-09 12 458 val_458
+2008-04-09 12 362 val_362
+2008-04-09 12 186 val_186
+2008-04-09 12 285 val_285
+2008-04-09 12 348 val_348
+2008-04-09 12 167 val_167
+2008-04-09 12 18 val_18
+2008-04-09 12 273 val_273
+2008-04-09 12 183 val_183
+2008-04-09 12 281 val_281
+2008-04-09 12 344 val_344
+2008-04-09 12 97 val_97
+2008-04-09 12 469 val_469
+2008-04-09 12 315 val_315
+2008-04-09 12 84 val_84
+2008-04-09 12 28 val_28
+2008-04-09 12 37 val_37
+2008-04-09 12 448 val_448
+2008-04-09 12 152 val_152
+2008-04-09 12 348 val_348
+2008-04-09 12 307 val_307
+2008-04-09 12 194 val_194
+2008-04-09 12 414 val_414
+2008-04-09 12 477 val_477
+2008-04-09 12 222 val_222
+2008-04-09 12 126 val_126
+2008-04-09 12 90 val_90
+2008-04-09 12 169 val_169
+2008-04-09 12 403 val_403
+2008-04-09 12 400 val_400
+2008-04-09 12 200 val_200
+2008-04-09 12 97 val_97
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala
index 4b45e698601eb..8488f23abd0fd 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala
@@ -15,19 +15,17 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
-package execution
+package org.apache.spark.sql.hive.execution
import java.io.File
+import org.apache.spark.sql.hive.TestHive._
+
/**
* A set of test cases based on the big-data-benchmark.
* https://amplab.cs.berkeley.edu/benchmark/
*/
class BigDataBenchmarkSuite extends HiveComparisonTest {
- import TestHive._
-
val testDataDirectory = new File("target/big-data-benchmark-testdata")
val testTables = Seq(
@@ -123,4 +121,4 @@ class BigDataBenchmarkSuite extends HiveComparisonTest {
|-- SELECT * FROM url_counts_total
""".stripMargin)
}
-}
\ No newline at end of file
+}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala
index 02ee2a0ebc354..ac87f2cb10d12 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala
@@ -15,12 +15,11 @@
* limitations under the License.
*/
-package org.apache.spark
-package sql
-package hive
-package execution
+package org.apache.spark.sql.hive.execution
-import org.scalatest.{FunSuite, BeforeAndAfterAll}
+import org.apache.spark.{SparkConf, SparkContext}
+import org.apache.spark.sql.hive.TestHiveContext
+import org.scalatest.{BeforeAndAfterAll, FunSuite}
class ConcurrentHiveSuite extends FunSuite with BeforeAndAfterAll {
ignore("multiple instances not supported") {
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
index e8fcc272354fa..c7a350ef94edd 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
@@ -15,17 +15,16 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
-package execution
+package org.apache.spark.sql.hive.execution
import java.io._
-import org.scalatest.{BeforeAndAfterAll, FunSuite, GivenWhenThen}
-
+import org.apache.spark.sql.Logging
import org.apache.spark.sql.catalyst.plans.logical.{ExplainCommand, NativeCommand}
import org.apache.spark.sql.catalyst.util._
import org.apache.spark.sql.execution.Sort
+import org.scalatest.{BeforeAndAfterAll, FunSuite, GivenWhenThen}
+import org.apache.spark.sql.hive.TestHive
/**
* Allows the creations of tests that execute the same query against both hive
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
index d5b54fa90fed3..f74b0fbb97c83 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
@@ -15,9 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
-package execution
+package org.apache.spark.sql.hive.execution
+
+import org.apache.spark.sql.hive.TestHive
/**
* Runs the test cases that are included in the hive distribution.
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala
index 2d2f13333a65c..50ab71a9003d3 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
-package execution
+package org.apache.spark.sql.hive.execution
import java.io.File
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
index b804634db1f5f..c184ebe288af4 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
@@ -15,16 +15,14 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
-package execution
+package org.apache.spark.sql.hive.execution
+
+import org.apache.spark.sql.hive.TestHive._
/**
* A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution.
*/
class HiveQuerySuite extends HiveComparisonTest {
- import TestHive._
-
createQueryTest("Simple Average",
"SELECT AVG(key) FROM src")
@@ -55,10 +53,8 @@ class HiveQuerySuite extends HiveComparisonTest {
createQueryTest("length.udf",
"SELECT length(\"test\") FROM src LIMIT 1")
- ignore("partitioned table scan") {
- createQueryTest("partitioned table scan",
- "SELECT ds, hr, key, value FROM srcpart")
- }
+ createQueryTest("partitioned table scan",
+ "SELECT ds, hr, key, value FROM srcpart")
createQueryTest("hash",
"SELECT hash('test') FROM src LIMIT 1")
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
index 4bdea214677ad..d77900ddc950c 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
@@ -15,11 +15,10 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
-package execution
+package org.apache.spark.sql.hive.execution
-import TestHive._
+import org.apache.spark.sql.hive.TestHive
+import org.apache.spark.sql.hive.TestHive._
case class Data(a: Int, B: Int, n: Nested)
case class Nested(a: Int, B: Int)
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala
index c2264926f4356..df9bae96494d5 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
-package execution
+package org.apache.spark.sql.hive.execution
/**
* A set of tests that validates support for Hive SerDe.
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala
index bb33583e5f582..e030c8ee3dfc8 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala
@@ -21,7 +21,6 @@ package org.apache.spark.sql.hive.execution
* A set of tests that validate type promotion rules.
*/
class HiveTypeCoercionSuite extends HiveComparisonTest {
-
val baseTypes = Seq("1", "1.0", "1L", "1S", "1Y", "'1'")
baseTypes.foreach { i =>
@@ -29,4 +28,4 @@ class HiveTypeCoercionSuite extends HiveComparisonTest {
createQueryTest(s"$i + $j", s"SELECT $i + $j FROM src LIMIT 1")
}
}
-}
\ No newline at end of file
+}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala
index d2f8e5df5b29e..1318ac1968dad 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala
@@ -15,11 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
-package execution
+package org.apache.spark.sql.hive.execution
-import org.apache.spark.sql.hive.TestHive
+import org.apache.spark.sql.hive.{HiveTableScan, TestHive}
/* Implicit conversions */
import scala.collection.JavaConversions._
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
index 062b888e80b48..e198c69470c1f 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala
@@ -431,7 +431,7 @@ class StreamingContext private[streaming] (
* Stop the execution of the streams.
* @param stopSparkContext Stop the associated SparkContext or not
*/
- def stop(stopSparkContext: Boolean = true) = synchronized {
+ def stop(stopSparkContext: Boolean = true): Unit = synchronized {
scheduler.stop()
logInfo("StreamingContext stopped successfully")
waiter.notifyStop()
@@ -489,7 +489,7 @@ object StreamingContext extends Logging {
* Find the JAR from which a given class was loaded, to make it easy for users to pass
* their JARs to StreamingContext.
*/
- def jarOfClass(cls: Class[_]) = SparkContext.jarOfClass(cls)
+ def jarOfClass(cls: Class[_]): Seq[String] = SparkContext.jarOfClass(cls)
private[streaming] def createNewSparkContext(conf: SparkConf): SparkContext = {
// Set the default cleaner delay to an hour if not already set.
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala
index a85cd04c9319c..bb2f492d06a00 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala
@@ -49,7 +49,9 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
* Print the first ten elements of each RDD generated in this DStream. This is an output
* operator, so this DStream will be registered as an output stream and there materialized.
*/
- def print() = dstream.print()
+ def print(): Unit = {
+ dstream.print()
+ }
/**
* Return a new DStream in which each RDD has a single element generated by counting each RDD
@@ -401,7 +403,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T
* Enable periodic checkpointing of RDDs of this DStream.
* @param interval Time interval after which generated RDD will be checkpointed
*/
- def checkpoint(interval: Duration) = {
+ def checkpoint(interval: Duration): DStream[T] = {
dstream.checkpoint(interval)
}
}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
index c48d754e439e9..b705d2ec9a58e 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala
@@ -477,31 +477,41 @@ class JavaStreamingContext(val ssc: StreamingContext) {
/**
* Start the execution of the streams.
*/
- def start() = ssc.start()
+ def start(): Unit = {
+ ssc.start()
+ }
/**
* Wait for the execution to stop. Any exceptions that occurs during the execution
* will be thrown in this thread.
*/
- def awaitTermination() = ssc.awaitTermination()
+ def awaitTermination(): Unit = {
+ ssc.awaitTermination()
+ }
/**
* Wait for the execution to stop. Any exceptions that occurs during the execution
* will be thrown in this thread.
* @param timeout time to wait in milliseconds
*/
- def awaitTermination(timeout: Long) = ssc.awaitTermination(timeout)
+ def awaitTermination(timeout: Long): Unit = {
+ ssc.awaitTermination(timeout)
+ }
/**
* Stop the execution of the streams. Will stop the associated JavaSparkContext as well.
*/
- def stop() = ssc.stop()
+ def stop(): Unit = {
+ ssc.stop()
+ }
/**
* Stop the execution of the streams.
* @param stopSparkContext Stop the associated SparkContext or not
*/
- def stop(stopSparkContext: Boolean) = ssc.stop(stopSparkContext)
+ def stop(stopSparkContext: Boolean): Unit = {
+ ssc.stop(stopSparkContext)
+ }
}
/**
@@ -579,7 +589,7 @@ object JavaStreamingContext {
* Find the JAR from which a given class was loaded, to make it easy for users to pass
* their JARs to StreamingContext.
*/
- def jarOfClass(cls: Class[_]) = SparkContext.jarOfClass(cls).toArray
+ def jarOfClass(cls: Class[_]): Array[String] = SparkContext.jarOfClass(cls).toArray
}
/**
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala
index 3208359306bee..d043200f71a0b 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala
@@ -505,14 +505,18 @@ abstract class DStream[T: ClassTag] (
* 'this' DStream will be registered as an output stream and therefore materialized.
*/
@deprecated("use foreachRDD", "0.9.0")
- def foreach(foreachFunc: RDD[T] => Unit) = this.foreachRDD(foreachFunc)
+ def foreach(foreachFunc: RDD[T] => Unit): Unit = {
+ this.foreachRDD(foreachFunc)
+ }
/**
* Apply a function to each RDD in this DStream. This is an output operator, so
* 'this' DStream will be registered as an output stream and therefore materialized.
*/
@deprecated("use foreachRDD", "0.9.0")
- def foreach(foreachFunc: (RDD[T], Time) => Unit) = this.foreachRDD(foreachFunc)
+ def foreach(foreachFunc: (RDD[T], Time) => Unit): Unit = {
+ this.foreachRDD(foreachFunc)
+ }
/**
* Apply a function to each RDD in this DStream. This is an output operator, so
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala
index 4e8d07fe921fb..7f3cd2f8eb1fd 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala
@@ -39,17 +39,19 @@ case class BatchInfo(
* was submitted to the streaming scheduler. Essentially, it is
* `processingStartTime` - `submissionTime`.
*/
- def schedulingDelay = processingStartTime.map(_ - submissionTime)
+ def schedulingDelay: Option[Long] = processingStartTime.map(_ - submissionTime)
/**
* Time taken for the all jobs of this batch to finish processing from the time they started
* processing. Essentially, it is `processingEndTime` - `processingStartTime`.
*/
- def processingDelay = processingEndTime.zip(processingStartTime).map(x => x._1 - x._2).headOption
+ def processingDelay: Option[Long] = processingEndTime.zip(processingStartTime)
+ .map(x => x._1 - x._2).headOption
/**
* Time taken for all the jobs of this batch to finish processing from the time they
* were submitted. Essentially, it is `processingDelay` + `schedulingDelay`.
*/
- def totalDelay = schedulingDelay.zip(processingDelay).map(x => x._1 + x._2).headOption
+ def totalDelay: Option[Long] = schedulingDelay.zip(processingDelay)
+ .map(x => x._1 + x._2).headOption
}
diff --git a/tools/pom.xml b/tools/pom.xml
index 11433e596f5b0..ae2ba64e07c21 100644
--- a/tools/pom.xml
+++ b/tools/pom.xml
@@ -55,6 +55,14 @@
spark-streaming_${scala.binary.version}
${project.version}
+
+ org.scala-lang
+ scala-reflect
+
+
+ org.scala-lang
+ scala-compiler
+
org.scalatest
scalatest_${scala.binary.version}