-For example:
+To pass multiple arguments the "arg" option can be specified multiple times. For example:
# Build the Spark assembly JAR and the Spark examples JAR
$ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly
@@ -85,7 +85,8 @@ For example:
./bin/spark-class org.apache.spark.deploy.yarn.Client \
--jar examples/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \
--class org.apache.spark.examples.SparkPi \
- --args yarn-cluster \
+ --arg yarn-cluster \
+ --arg 5 \
--num-executors 3 \
--driver-memory 4g \
--executor-memory 2g \
diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md
index b6f21a5dc62c3..f849716f7a48f 100644
--- a/docs/sql-programming-guide.md
+++ b/docs/sql-programming-guide.md
@@ -8,6 +8,10 @@ title: Spark SQL Programming Guide
{:toc}
# Overview
+
+
+
+
Spark SQL allows relational queries expressed in SQL, HiveQL, or Scala to be executed using
Spark. At the core of this component is a new type of RDD,
[SchemaRDD](api/sql/core/index.html#org.apache.spark.sql.SchemaRDD). SchemaRDDs are composed
@@ -18,11 +22,27 @@ file, or by running HiveQL against data stored in [Apache Hive](http://hive.apac
**All of the examples on this page use sample data included in the Spark distribution and can be run in the spark-shell.**
+
+
+
+Spark SQL allows relational queries expressed in SQL, HiveQL, or Scala to be executed using
+Spark. At the core of this component is a new type of RDD,
+[JavaSchemaRDD](api/sql/core/index.html#org.apache.spark.sql.api.java.JavaSchemaRDD). JavaSchemaRDDs are composed
+[Row](api/sql/catalyst/index.html#org.apache.spark.sql.api.java.Row) objects along with
+a schema that describes the data types of each column in the row. A JavaSchemaRDD is similar to a table
+in a traditional relational database. A JavaSchemaRDD can be created from an existing RDD, parquet
+file, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/).
+
+
+
***************************************************************************************************
# Getting Started
-The entry point into all relational functionallity in Spark is the
+
+
+
+The entry point into all relational functionality in Spark is the
[SQLContext](api/sql/core/index.html#org.apache.spark.sql.SQLContext) class, or one of its
decendents. To create a basic SQLContext, all you need is a SparkContext.
@@ -34,8 +54,30 @@ val sqlContext = new org.apache.spark.sql.SQLContext(sc)
import sqlContext._
{% endhighlight %}
+
+
+
+
+The entry point into all relational functionality in Spark is the
+[JavaSQLContext](api/sql/core/index.html#org.apache.spark.sql.api.java.JavaSQLContext) class, or one
+of its decendents. To create a basic JavaSQLContext, all you need is a JavaSparkContext.
+
+{% highlight java %}
+JavaSparkContext ctx = ...; // An existing JavaSparkContext.
+JavaSQLContext sqlCtx = new org.apache.spark.sql.api.java.JavaSQLContext(ctx);
+{% endhighlight %}
+
+
+
+
+
## Running SQL on RDDs
-One type of table that is supported by Spark SQL is an RDD of Scala case classetees. The case class
+
+
+
+
+
+One type of table that is supported by Spark SQL is an RDD of Scala case classes. The case class
defines the schema of the table. The names of the arguments to the case class are read using
reflection and become the names of the columns. Case classes can also be nested or contain complex
types such as Sequences or Arrays. This RDD can be implicitly converted to a SchemaRDD and then be
@@ -60,7 +102,83 @@ val teenagers = sql("SELECT name FROM people WHERE age >= 13 AND age <= 19")
teenagers.map(t => "Name: " + t(0)).collect().foreach(println)
{% endhighlight %}
-**Note that Spark SQL currently uses a very basic SQL parser, and the keywords are case sensitive.**
+
+
+
+
+One type of table that is supported by Spark SQL is an RDD of [JavaBeans](http://stackoverflow.com/questions/3295496/what-is-a-javabean-exactly). The BeanInfo
+defines the schema of the table. Currently, Spark SQL does not support JavaBeans that contain
+nested or contain complex types such as Lists or Arrays. You can create a JavaBean by creating a
+class that implements Serializable and has getters and setters for all of its fields.
+
+{% highlight java %}
+
+public static class Person implements Serializable {
+ private String name;
+ private int age;
+
+ String getName() {
+ return name;
+ }
+
+ void setName(String name) {
+ this.name = name;
+ }
+
+ int getAge() {
+ return age;
+ }
+
+ void setAge(int age) {
+ this.age = age;
+ }
+}
+
+{% endhighlight %}
+
+
+A schema can be applied to an existing RDD by calling `applySchema` and providing the Class object
+for the JavaBean.
+
+{% highlight java %}
+JavaSQLContext ctx = new org.apache.spark.sql.api.java.JavaSQLContext(sc)
+
+// Load a text file and convert each line to a JavaBean.
+JavaRDD
people = ctx.textFile("examples/src/main/resources/people.txt").map(
+ new Function() {
+ public Person call(String line) throws Exception {
+ String[] parts = line.split(",");
+
+ Person person = new Person();
+ person.setName(parts[0]);
+ person.setAge(Integer.parseInt(parts[1].trim()));
+
+ return person;
+ }
+ });
+
+// Apply a schema to an RDD of JavaBeans and register it as a table.
+JavaSchemaRDD schemaPeople = sqlCtx.applySchema(people, Person.class);
+schemaPeople.registerAsTable("people");
+
+// SQL can be run over RDDs that have been registered as tables.
+JavaSchemaRDD teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19")
+
+// The results of SQL queries are SchemaRDDs and support all the normal RDD operations.
+// The columns of a row in the result can be accessed by ordinal.
+List teenagerNames = teenagers.map(new Function() {
+ public String call(Row row) {
+ return "Name: " + row.getString(0);
+ }
+}).collect();
+
+{% endhighlight %}
+
+
+
+
+
+**Note that Spark SQL currently uses a very basic SQL parser.**
Users that want a more complete dialect of SQL should look at the HiveQL support provided by
`HiveContext`.
@@ -70,17 +188,21 @@ Parquet is a columnar format that is supported by many other data processing sys
provides support for both reading and writing parquet files that automatically preserves the schema
of the original data. Using the data from the above example:
+
+
+
+
{% highlight scala %}
val sqlContext = new org.apache.spark.sql.SQLContext(sc)
import sqlContext._
-val people: RDD[Person] // An RDD of case class objects, from the previous example.
+val people: RDD[Person] = ... // An RDD of case class objects, from the previous example.
// The RDD is implicitly converted to a SchemaRDD, allowing it to be stored using parquet.
people.saveAsParquetFile("people.parquet")
// Read in the parquet file created above. Parquet files are self-describing so the schema is preserved.
-// The result of loading a parquet file is also a SchemaRDD.
+// The result of loading a parquet file is also a JavaSchemaRDD.
val parquetFile = sqlContext.parquetFile("people.parquet")
//Parquet files can also be registered as tables and then used in SQL statements.
@@ -89,15 +211,43 @@ val teenagers = sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"
teenagers.collect().foreach(println)
{% endhighlight %}
+
+
+
+
+{% highlight java %}
+
+JavaSchemaRDD schemaPeople = ... // The JavaSchemaRDD from the previous example.
+
+// JavaSchemaRDDs can be saved as parquet files, maintaining the schema information.
+schemaPeople.saveAsParquetFile("people.parquet");
+
+// Read in the parquet file created above. Parquet files are self-describing so the schema is preserved.
+// The result of loading a parquet file is also a JavaSchemaRDD.
+JavaSchemaRDD parquetFile = sqlCtx.parquetFile("people.parquet");
+
+//Parquet files can also be registered as tables and then used in SQL statements.
+parquetFile.registerAsTable("parquetFile");
+JavaSchemaRDD teenagers = sqlCtx.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19");
+
+
+{% endhighlight %}
+
+
+
+
+
## Writing Language-Integrated Relational Queries
+**Language-Integrated queries are currently only supported in Scala.**
+
Spark SQL also supports a domain specific language for writing queries. Once again,
using the data from the above examples:
{% highlight scala %}
val sqlContext = new org.apache.spark.sql.SQLContext(sc)
import sqlContext._
-val people: RDD[Person] // An RDD of case class objects, from the first example.
+val people: RDD[Person] = ... // An RDD of case class objects, from the first example.
// The following is the same as 'SELECT name FROM people WHERE age >= 10 AND age <= 19'
val teenagers = people.where('age >= 10).where('age <= 19).select('name)
@@ -114,14 +264,17 @@ evaluated by the SQL execution engine. A full list of the functions supported c
Spark SQL also supports reading and writing data stored in [Apache Hive](http://hive.apache.org/).
However, since Hive has a large number of dependencies, it is not included in the default Spark assembly.
-In order to use Hive you must first run '`sbt/sbt hive/assembly`'. This command builds a new assembly
-jar that includes Hive. When this jar is present, Spark will use the Hive
-assembly instead of the normal Spark assembly. Note that this Hive assembly jar must also be present
+In order to use Hive you must first run '`SPARK_HIVE=true sbt/sbt assembly/assembly`'. This command builds a new assembly
+jar that includes Hive. Note that this Hive assembly jar must also be present
on all of the worker nodes, as they will need access to the Hive serialization and deserialization libraries
(SerDes) in order to acccess data stored in Hive.
Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`.
+
+
+
+
When working with Hive one must construct a `HiveContext`, which inherits from `SQLContext`, and
adds support for finding tables in in the MetaStore and writing queries using HiveQL. Users who do
not have an existing Hive deployment can also experiment with the `LocalHiveContext`,
@@ -135,9 +288,34 @@ val hiveContext = new org.apache.spark.sql.hive.HiveContext(sc)
// Importing the SQL context gives access to all the public SQL functions and implicit conversions.
import hiveContext._
-sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)")
-sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src")
+hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)")
+hql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src")
// Queries are expressed in HiveQL
-sql("SELECT key, value FROM src").collect().foreach(println)
-{% endhighlight %}
\ No newline at end of file
+hql("FROM src SELECT key, value").collect().foreach(println)
+{% endhighlight %}
+
+
+
+
+
+When working with Hive one must construct a `JavaHiveContext`, which inherits from `JavaSQLContext`, and
+adds support for finding tables in in the MetaStore and writing queries using HiveQL. In addition to
+the `sql` method a `JavaHiveContext` also provides an `hql` methods, which allows queries to be
+expressed in HiveQL.
+
+{% highlight java %}
+JavaSparkContext ctx = ...; // An existing JavaSparkContext.
+JavaHiveContext hiveCtx = new org.apache.spark.sql.hive.api.java.HiveContext(ctx);
+
+hiveCtx.hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)");
+hiveCtx.hql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src");
+
+// Queries are expressed in HiveQL.
+Row[] results = hiveCtx.hql("FROM src SELECT key, value").collect();
+
+{% endhighlight %}
+
+
+
+
diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java
new file mode 100644
index 0000000000000..e8e63d2745692
--- /dev/null
+++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.examples.sql;
+
+import java.io.Serializable;
+import java.util.List;
+
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.api.java.function.VoidFunction;
+
+import org.apache.spark.sql.api.java.JavaSQLContext;
+import org.apache.spark.sql.api.java.JavaSchemaRDD;
+import org.apache.spark.sql.api.java.Row;
+
+public class JavaSparkSQL {
+ public static class Person implements Serializable {
+ private String name;
+ private int age;
+
+ String getName() {
+ return name;
+ }
+
+ void setName(String name) {
+ this.name = name;
+ }
+
+ int getAge() {
+ return age;
+ }
+
+ void setAge(int age) {
+ this.age = age;
+ }
+ }
+
+ public static void main(String[] args) throws Exception {
+ JavaSparkContext ctx = new JavaSparkContext("local", "JavaSparkSQL",
+ System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaSparkSQL.class));
+ JavaSQLContext sqlCtx = new JavaSQLContext(ctx);
+
+ // Load a text file and convert each line to a Java Bean.
+ JavaRDD people = ctx.textFile("examples/src/main/resources/people.txt").map(
+ new Function() {
+ public Person call(String line) throws Exception {
+ String[] parts = line.split(",");
+
+ Person person = new Person();
+ person.setName(parts[0]);
+ person.setAge(Integer.parseInt(parts[1].trim()));
+
+ return person;
+ }
+ });
+
+ // Apply a schema to an RDD of Java Beans and register it as a table.
+ JavaSchemaRDD schemaPeople = sqlCtx.applySchema(people, Person.class);
+ schemaPeople.registerAsTable("people");
+
+ // SQL can be run over RDDs that have been registered as tables.
+ JavaSchemaRDD teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19");
+
+ // The results of SQL queries are SchemaRDDs and support all the normal RDD operations.
+ // The columns of a row in the result can be accessed by ordinal.
+ List teenagerNames = teenagers.map(new Function() {
+ public String call(Row row) {
+ return "Name: " + row.getString(0);
+ }
+ }).collect();
+
+ // JavaSchemaRDDs can be saved as parquet files, maintaining the schema information.
+ schemaPeople.saveAsParquetFile("people.parquet");
+
+ // Read in the parquet file created above. Parquet files are self-describing so the schema is preserved.
+ // The result of loading a parquet file is also a JavaSchemaRDD.
+ JavaSchemaRDD parquetFile = sqlCtx.parquetFile("people.parquet");
+
+ //Parquet files can also be registered as tables and then used in SQL statements.
+ parquetFile.registerAsTable("parquetFile");
+ JavaSchemaRDD teenagers2 = sqlCtx.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19");
+ }
+}
diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java
index 667c72f379e71..cd8879ff886e2 100644
--- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java
+++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java
@@ -17,6 +17,7 @@
package org.apache.spark.mllib.examples;
+import java.util.regex.Pattern;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
@@ -24,11 +25,9 @@
import org.apache.spark.mllib.classification.LogisticRegressionWithSGD;
import org.apache.spark.mllib.classification.LogisticRegressionModel;
+import org.apache.spark.mllib.linalg.Vectors;
import org.apache.spark.mllib.regression.LabeledPoint;
-import java.util.Arrays;
-import java.util.regex.Pattern;
-
/**
* Logistic regression based classification using ML Lib.
*/
@@ -47,14 +46,10 @@ public LabeledPoint call(String line) {
for (int i = 0; i < tok.length; ++i) {
x[i] = Double.parseDouble(tok[i]);
}
- return new LabeledPoint(y, x);
+ return new LabeledPoint(y, Vectors.dense(x));
}
}
- public static void printWeights(double[] a) {
- System.out.println(Arrays.toString(a));
- }
-
public static void main(String[] args) {
if (args.length != 4) {
System.err.println("Usage: JavaLR ");
@@ -80,8 +75,7 @@ public static void main(String[] args) {
LogisticRegressionModel model = LogisticRegressionWithSGD.train(points.rdd(),
iterations, stepSize);
- System.out.print("Final w: ");
- printWeights(model.weights());
+ System.out.print("Final w: " + model.weights());
System.exit(0);
}
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala
index f2296a865e1b3..6d04bf790e3a5 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala
@@ -45,7 +45,8 @@ class EdgeRDD[@specialized ED: ClassTag](
partitionsRDD.partitioner.orElse(Some(Partitioner.defaultPartitioner(partitionsRDD)))
override def compute(part: Partition, context: TaskContext): Iterator[Edge[ED]] = {
- firstParent[(PartitionID, EdgePartition[ED])].iterator(part, context).next._2.iterator
+ val p = firstParent[(PartitionID, EdgePartition[ED])].iterator(part, context)
+ p.next._2.iterator.map(_.copy())
}
override def collect(): Array[Edge[ED]] = this.map(_.copy()).collect()
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala
index 57fa5eefd5e09..2e05f5d4e4969 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala
@@ -56,6 +56,9 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double)
* Construct a new edge partition by applying the function f to all
* edges in this partition.
*
+ * Be careful not to keep references to the objects passed to `f`.
+ * To improve GC performance the same object is re-used for each call.
+ *
* @param f a function from an edge to a new attribute
* @tparam ED2 the type of the new attribute
* @return a new edge partition with the result of the function `f`
@@ -84,12 +87,12 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double)
* order of the edges returned by `EdgePartition.iterator` and
* should return attributes equal to the number of edges.
*
- * @param f a function from an edge to a new attribute
+ * @param iter an iterator for the new attribute values
* @tparam ED2 the type of the new attribute
- * @return a new edge partition with the result of the function `f`
- * applied to each edge
+ * @return a new edge partition with the attribute values replaced
*/
def map[ED2: ClassTag](iter: Iterator[ED2]): EdgePartition[ED2] = {
+ // Faster than iter.toArray, because the expected size is known.
val newData = new Array[ED2](data.size)
var i = 0
while (iter.hasNext) {
@@ -188,6 +191,9 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double)
/**
* Get an iterator over the edges in this partition.
*
+ * Be careful not to keep references to the objects from this iterator.
+ * To improve GC performance the same object is re-used in `next()`.
+ *
* @return an iterator over edges in the partition
*/
def iterator = new Iterator[Edge[ED]] {
@@ -216,6 +222,9 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double)
/**
* Get an iterator over the cluster of edges in this partition with source vertex id `srcId`. The
* cluster must start at position `index`.
+ *
+ * Be careful not to keep references to the objects from this iterator. To improve GC performance
+ * the same object is re-used in `next()`.
*/
private def clusterIterator(srcId: VertexId, index: Int) = new Iterator[Edge[ED]] {
private[this] val edge = new Edge[ED]
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala
index 886c250d7cffd..220a89d73d711 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala
@@ -37,20 +37,15 @@ class EdgeTripletIterator[VD: ClassTag, ED: ClassTag](
// Current position in the array.
private var pos = 0
- // A triplet object that this iterator.next() call returns. We reuse this object to avoid
- // allocating too many temporary Java objects.
- private val triplet = new EdgeTriplet[VD, ED]
-
private val vmap = new PrimitiveKeyOpenHashMap[VertexId, VD](vidToIndex, vertexArray)
override def hasNext: Boolean = pos < edgePartition.size
override def next() = {
+ val triplet = new EdgeTriplet[VD, ED]
triplet.srcId = edgePartition.srcIds(pos)
- // assert(vmap.containsKey(e.src.id))
triplet.srcAttr = vmap(triplet.srcId)
triplet.dstId = edgePartition.dstIds(pos)
- // assert(vmap.containsKey(e.dst.id))
triplet.dstAttr = vmap(triplet.dstId)
triplet.attr = edgePartition.data(pos)
pos += 1
diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgeTripletIteratorSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgeTripletIteratorSuite.scala
new file mode 100644
index 0000000000000..9cbb2d2acdc2d
--- /dev/null
+++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgeTripletIteratorSuite.scala
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.graphx.impl
+
+import scala.reflect.ClassTag
+import scala.util.Random
+
+import org.scalatest.FunSuite
+
+import org.apache.spark.graphx._
+
+class EdgeTripletIteratorSuite extends FunSuite {
+ test("iterator.toList") {
+ val builder = new EdgePartitionBuilder[Int]
+ builder.add(1, 2, 0)
+ builder.add(1, 3, 0)
+ builder.add(1, 4, 0)
+ val vidmap = new VertexIdToIndexMap
+ vidmap.add(1)
+ vidmap.add(2)
+ vidmap.add(3)
+ vidmap.add(4)
+ val vs = Array.fill(vidmap.capacity)(0)
+ val iter = new EdgeTripletIterator[Int, Int](vidmap, vs, builder.toEdgePartition)
+ val result = iter.toList.map(et => (et.srcId, et.dstId))
+ assert(result === Seq((1, 2), (1, 3), (1, 4)))
+ }
+}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala
index 3449c698da60b..2df5b0d02b699 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala
@@ -110,16 +110,16 @@ class PythonMLLibAPI extends Serializable {
private def trainRegressionModel(
trainFunc: (RDD[LabeledPoint], Array[Double]) => GeneralizedLinearModel,
- dataBytesJRDD: JavaRDD[Array[Byte]], initialWeightsBA: Array[Byte]):
- java.util.LinkedList[java.lang.Object] = {
+ dataBytesJRDD: JavaRDD[Array[Byte]],
+ initialWeightsBA: Array[Byte]): java.util.LinkedList[java.lang.Object] = {
val data = dataBytesJRDD.rdd.map(xBytes => {
val x = deserializeDoubleVector(xBytes)
- LabeledPoint(x(0), x.slice(1, x.length))
+ LabeledPoint(x(0), Vectors.dense(x.slice(1, x.length)))
})
val initialWeights = deserializeDoubleVector(initialWeightsBA)
val model = trainFunc(data, initialWeights)
val ret = new java.util.LinkedList[java.lang.Object]()
- ret.add(serializeDoubleVector(model.weights))
+ ret.add(serializeDoubleVector(model.weights.toArray))
ret.add(model.intercept: java.lang.Double)
ret
}
@@ -127,75 +127,127 @@ class PythonMLLibAPI extends Serializable {
/**
* Java stub for Python mllib LinearRegressionWithSGD.train()
*/
- def trainLinearRegressionModelWithSGD(dataBytesJRDD: JavaRDD[Array[Byte]],
- numIterations: Int, stepSize: Double, miniBatchFraction: Double,
+ def trainLinearRegressionModelWithSGD(
+ dataBytesJRDD: JavaRDD[Array[Byte]],
+ numIterations: Int,
+ stepSize: Double,
+ miniBatchFraction: Double,
initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = {
- trainRegressionModel((data, initialWeights) =>
- LinearRegressionWithSGD.train(data, numIterations, stepSize,
- miniBatchFraction, initialWeights),
- dataBytesJRDD, initialWeightsBA)
+ trainRegressionModel(
+ (data, initialWeights) =>
+ LinearRegressionWithSGD.train(
+ data,
+ numIterations,
+ stepSize,
+ miniBatchFraction,
+ Vectors.dense(initialWeights)),
+ dataBytesJRDD,
+ initialWeightsBA)
}
/**
* Java stub for Python mllib LassoWithSGD.train()
*/
- def trainLassoModelWithSGD(dataBytesJRDD: JavaRDD[Array[Byte]], numIterations: Int,
- stepSize: Double, regParam: Double, miniBatchFraction: Double,
+ def trainLassoModelWithSGD(
+ dataBytesJRDD: JavaRDD[Array[Byte]],
+ numIterations: Int,
+ stepSize: Double,
+ regParam: Double,
+ miniBatchFraction: Double,
initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = {
- trainRegressionModel((data, initialWeights) =>
- LassoWithSGD.train(data, numIterations, stepSize, regParam,
- miniBatchFraction, initialWeights),
- dataBytesJRDD, initialWeightsBA)
+ trainRegressionModel(
+ (data, initialWeights) =>
+ LassoWithSGD.train(
+ data,
+ numIterations,
+ stepSize,
+ regParam,
+ miniBatchFraction,
+ Vectors.dense(initialWeights)),
+ dataBytesJRDD,
+ initialWeightsBA)
}
/**
* Java stub for Python mllib RidgeRegressionWithSGD.train()
*/
- def trainRidgeModelWithSGD(dataBytesJRDD: JavaRDD[Array[Byte]], numIterations: Int,
- stepSize: Double, regParam: Double, miniBatchFraction: Double,
+ def trainRidgeModelWithSGD(
+ dataBytesJRDD: JavaRDD[Array[Byte]],
+ numIterations: Int,
+ stepSize: Double,
+ regParam: Double,
+ miniBatchFraction: Double,
initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = {
- trainRegressionModel((data, initialWeights) =>
- RidgeRegressionWithSGD.train(data, numIterations, stepSize, regParam,
- miniBatchFraction, initialWeights),
- dataBytesJRDD, initialWeightsBA)
+ trainRegressionModel(
+ (data, initialWeights) =>
+ RidgeRegressionWithSGD.train(
+ data,
+ numIterations,
+ stepSize,
+ regParam,
+ miniBatchFraction,
+ Vectors.dense(initialWeights)),
+ dataBytesJRDD,
+ initialWeightsBA)
}
/**
* Java stub for Python mllib SVMWithSGD.train()
*/
- def trainSVMModelWithSGD(dataBytesJRDD: JavaRDD[Array[Byte]], numIterations: Int,
- stepSize: Double, regParam: Double, miniBatchFraction: Double,
+ def trainSVMModelWithSGD(
+ dataBytesJRDD: JavaRDD[Array[Byte]],
+ numIterations: Int,
+ stepSize: Double,
+ regParam: Double,
+ miniBatchFraction: Double,
initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = {
- trainRegressionModel((data, initialWeights) =>
- SVMWithSGD.train(data, numIterations, stepSize, regParam,
- miniBatchFraction, initialWeights),
- dataBytesJRDD, initialWeightsBA)
+ trainRegressionModel(
+ (data, initialWeights) =>
+ SVMWithSGD.train(
+ data,
+ numIterations,
+ stepSize,
+ regParam,
+ miniBatchFraction,
+ Vectors.dense(initialWeights)),
+ dataBytesJRDD,
+ initialWeightsBA)
}
/**
* Java stub for Python mllib LogisticRegressionWithSGD.train()
*/
- def trainLogisticRegressionModelWithSGD(dataBytesJRDD: JavaRDD[Array[Byte]],
- numIterations: Int, stepSize: Double, miniBatchFraction: Double,
+ def trainLogisticRegressionModelWithSGD(
+ dataBytesJRDD: JavaRDD[Array[Byte]],
+ numIterations: Int,
+ stepSize: Double,
+ miniBatchFraction: Double,
initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = {
- trainRegressionModel((data, initialWeights) =>
- LogisticRegressionWithSGD.train(data, numIterations, stepSize,
- miniBatchFraction, initialWeights),
- dataBytesJRDD, initialWeightsBA)
+ trainRegressionModel(
+ (data, initialWeights) =>
+ LogisticRegressionWithSGD.train(
+ data,
+ numIterations,
+ stepSize,
+ miniBatchFraction,
+ Vectors.dense(initialWeights)),
+ dataBytesJRDD,
+ initialWeightsBA)
}
/**
* Java stub for NaiveBayes.train()
*/
- def trainNaiveBayes(dataBytesJRDD: JavaRDD[Array[Byte]], lambda: Double)
- : java.util.List[java.lang.Object] =
- {
+ def trainNaiveBayes(
+ dataBytesJRDD: JavaRDD[Array[Byte]],
+ lambda: Double): java.util.List[java.lang.Object] = {
val data = dataBytesJRDD.rdd.map(xBytes => {
val x = deserializeDoubleVector(xBytes)
- LabeledPoint(x(0), x.slice(1, x.length))
+ LabeledPoint(x(0), Vectors.dense(x.slice(1, x.length)))
})
val model = NaiveBayes.train(data, lambda)
val ret = new java.util.LinkedList[java.lang.Object]()
+ ret.add(serializeDoubleVector(model.labels))
ret.add(serializeDoubleVector(model.pi))
ret.add(serializeDoubleMatrix(model.theta))
ret
@@ -204,9 +256,12 @@ class PythonMLLibAPI extends Serializable {
/**
* Java stub for Python mllib KMeans.train()
*/
- def trainKMeansModel(dataBytesJRDD: JavaRDD[Array[Byte]], k: Int,
- maxIterations: Int, runs: Int, initializationMode: String):
- java.util.List[java.lang.Object] = {
+ def trainKMeansModel(
+ dataBytesJRDD: JavaRDD[Array[Byte]],
+ k: Int,
+ maxIterations: Int,
+ runs: Int,
+ initializationMode: String): java.util.List[java.lang.Object] = {
val data = dataBytesJRDD.rdd.map(xBytes => Vectors.dense(deserializeDoubleVector(xBytes)))
val model = KMeans.train(data, k, maxIterations, runs, initializationMode)
val ret = new java.util.LinkedList[java.lang.Object]()
@@ -259,8 +314,12 @@ class PythonMLLibAPI extends Serializable {
* needs to be taken in the Python code to ensure it gets freed on exit; see
* the Py4J documentation.
*/
- def trainALSModel(ratingsBytesJRDD: JavaRDD[Array[Byte]], rank: Int,
- iterations: Int, lambda: Double, blocks: Int): MatrixFactorizationModel = {
+ def trainALSModel(
+ ratingsBytesJRDD: JavaRDD[Array[Byte]],
+ rank: Int,
+ iterations: Int,
+ lambda: Double,
+ blocks: Int): MatrixFactorizationModel = {
val ratings = ratingsBytesJRDD.rdd.map(unpackRating)
ALS.train(ratings, rank, iterations, lambda, blocks)
}
@@ -271,8 +330,13 @@ class PythonMLLibAPI extends Serializable {
* Extra care needs to be taken in the Python code to ensure it gets freed on
* exit; see the Py4J documentation.
*/
- def trainImplicitALSModel(ratingsBytesJRDD: JavaRDD[Array[Byte]], rank: Int,
- iterations: Int, lambda: Double, blocks: Int, alpha: Double): MatrixFactorizationModel = {
+ def trainImplicitALSModel(
+ ratingsBytesJRDD: JavaRDD[Array[Byte]],
+ rank: Int,
+ iterations: Int,
+ lambda: Double,
+ blocks: Int,
+ alpha: Double): MatrixFactorizationModel = {
val ratings = ratingsBytesJRDD.rdd.map(unpackRating)
ALS.trainImplicit(ratings, rank, iterations, lambda, blocks, alpha)
}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala
index 391f5b9b7a7de..bd10e2e9e10e2 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala
@@ -17,22 +17,27 @@
package org.apache.spark.mllib.classification
+import org.apache.spark.mllib.linalg.Vector
import org.apache.spark.rdd.RDD
+/**
+ * Represents a classification model that predicts to which of a set of categories an example
+ * belongs. The categories are represented by double values: 0.0, 1.0, 2.0, etc.
+ */
trait ClassificationModel extends Serializable {
/**
* Predict values for the given data set using the model trained.
*
* @param testData RDD representing data points to be predicted
- * @return RDD[Int] where each entry contains the corresponding prediction
+ * @return an RDD[Double] where each entry contains the corresponding prediction
*/
- def predict(testData: RDD[Array[Double]]): RDD[Double]
+ def predict(testData: RDD[Vector]): RDD[Double]
/**
* Predict values for a single data point using the model trained.
*
* @param testData array representing a single data point
- * @return Int prediction from the trained model
+ * @return predicted category from the trained model
*/
- def predict(testData: Array[Double]): Double
+ def predict(testData: Vector): Double
}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala
index a481f522761e2..798f3a5c94740 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala
@@ -17,16 +17,12 @@
package org.apache.spark.mllib.classification
-import scala.math.round
-
import org.apache.spark.SparkContext
-import org.apache.spark.rdd.RDD
+import org.apache.spark.mllib.linalg.Vector
import org.apache.spark.mllib.optimization._
import org.apache.spark.mllib.regression._
-import org.apache.spark.mllib.util.MLUtils
-import org.apache.spark.mllib.util.DataValidators
-
-import org.jblas.DoubleMatrix
+import org.apache.spark.mllib.util.{DataValidators, MLUtils}
+import org.apache.spark.rdd.RDD
/**
* Classification model trained using Logistic Regression.
@@ -35,15 +31,38 @@ import org.jblas.DoubleMatrix
* @param intercept Intercept computed for this model.
*/
class LogisticRegressionModel(
- override val weights: Array[Double],
+ override val weights: Vector,
override val intercept: Double)
- extends GeneralizedLinearModel(weights, intercept)
- with ClassificationModel with Serializable {
+ extends GeneralizedLinearModel(weights, intercept) with ClassificationModel with Serializable {
+
+ private var threshold: Option[Double] = Some(0.5)
+
+ /**
+ * Sets the threshold that separates positive predictions from negative predictions. An example
+ * with prediction score greater than or equal to this threshold is identified as an positive,
+ * and negative otherwise. The default value is 0.5.
+ */
+ def setThreshold(threshold: Double): this.type = {
+ this.threshold = Some(threshold)
+ this
+ }
- override def predictPoint(dataMatrix: DoubleMatrix, weightMatrix: DoubleMatrix,
+ /**
+ * Clears the threshold so that `predict` will output raw prediction scores.
+ */
+ def clearThreshold(): this.type = {
+ threshold = None
+ this
+ }
+
+ override def predictPoint(dataMatrix: Vector, weightMatrix: Vector,
intercept: Double) = {
- val margin = dataMatrix.mmul(weightMatrix).get(0) + intercept
- round(1.0/ (1.0 + math.exp(margin * -1)))
+ val margin = weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept
+ val score = 1.0/ (1.0 + math.exp(-margin))
+ threshold match {
+ case Some(t) => if (score < t) 0.0 else 1.0
+ case None => score
+ }
}
}
@@ -56,16 +75,15 @@ class LogisticRegressionWithSGD private (
var numIterations: Int,
var regParam: Double,
var miniBatchFraction: Double)
- extends GeneralizedLinearAlgorithm[LogisticRegressionModel]
- with Serializable {
+ extends GeneralizedLinearAlgorithm[LogisticRegressionModel] with Serializable {
val gradient = new LogisticGradient()
val updater = new SimpleUpdater()
override val optimizer = new GradientDescent(gradient, updater)
- .setStepSize(stepSize)
- .setNumIterations(numIterations)
- .setRegParam(regParam)
- .setMiniBatchFraction(miniBatchFraction)
+ .setStepSize(stepSize)
+ .setNumIterations(numIterations)
+ .setRegParam(regParam)
+ .setMiniBatchFraction(miniBatchFraction)
override val validators = List(DataValidators.classificationLabels)
/**
@@ -73,7 +91,7 @@ class LogisticRegressionWithSGD private (
*/
def this() = this(1.0, 100, 0.0, 1.0)
- def createModel(weights: Array[Double], intercept: Double) = {
+ def createModel(weights: Vector, intercept: Double) = {
new LogisticRegressionModel(weights, intercept)
}
}
@@ -105,11 +123,9 @@ object LogisticRegressionWithSGD {
numIterations: Int,
stepSize: Double,
miniBatchFraction: Double,
- initialWeights: Array[Double])
- : LogisticRegressionModel =
- {
- new LogisticRegressionWithSGD(stepSize, numIterations, 0.0, miniBatchFraction).run(
- input, initialWeights)
+ initialWeights: Vector): LogisticRegressionModel = {
+ new LogisticRegressionWithSGD(stepSize, numIterations, 0.0, miniBatchFraction)
+ .run(input, initialWeights)
}
/**
@@ -128,11 +144,9 @@ object LogisticRegressionWithSGD {
input: RDD[LabeledPoint],
numIterations: Int,
stepSize: Double,
- miniBatchFraction: Double)
- : LogisticRegressionModel =
- {
- new LogisticRegressionWithSGD(stepSize, numIterations, 0.0, miniBatchFraction).run(
- input)
+ miniBatchFraction: Double): LogisticRegressionModel = {
+ new LogisticRegressionWithSGD(stepSize, numIterations, 0.0, miniBatchFraction)
+ .run(input)
}
/**
@@ -150,9 +164,7 @@ object LogisticRegressionWithSGD {
def train(
input: RDD[LabeledPoint],
numIterations: Int,
- stepSize: Double)
- : LogisticRegressionModel =
- {
+ stepSize: Double): LogisticRegressionModel = {
train(input, numIterations, stepSize, 1.0)
}
@@ -168,9 +180,7 @@ object LogisticRegressionWithSGD {
*/
def train(
input: RDD[LabeledPoint],
- numIterations: Int)
- : LogisticRegressionModel =
- {
+ numIterations: Int): LogisticRegressionModel = {
train(input, numIterations, 1.0, 1.0)
}
@@ -183,7 +193,7 @@ object LogisticRegressionWithSGD {
val sc = new SparkContext(args(0), "LogisticRegression")
val data = MLUtils.loadLabeledData(sc, args(1))
val model = LogisticRegressionWithSGD.train(data, args(3).toInt, args(2).toDouble)
- println("Weights: " + model.weights.mkString("[", ", ", "]"))
+ println("Weights: " + model.weights)
println("Intercept: " + model.intercept)
sc.stop()
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala
index 6539b2f339465..e956185319a69 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala
@@ -17,14 +17,14 @@
package org.apache.spark.mllib.classification
-import scala.collection.mutable
+import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, argmax => brzArgmax, sum => brzSum}
-import org.jblas.DoubleMatrix
-
-import org.apache.spark.{SparkContext, Logging}
+import org.apache.spark.{Logging, SparkContext}
+import org.apache.spark.SparkContext._
+import org.apache.spark.mllib.linalg.Vector
import org.apache.spark.mllib.regression.LabeledPoint
-import org.apache.spark.rdd.RDD
import org.apache.spark.mllib.util.MLUtils
+import org.apache.spark.rdd.RDD
/**
* Model for Naive Bayes Classifiers.
@@ -32,19 +32,28 @@ import org.apache.spark.mllib.util.MLUtils
* @param pi Log of class priors, whose dimension is C.
* @param theta Log of class conditional probabilities, whose dimension is CxD.
*/
-class NaiveBayesModel(val pi: Array[Double], val theta: Array[Array[Double]])
- extends ClassificationModel with Serializable {
-
- // Create a column vector that can be used for predictions
- private val _pi = new DoubleMatrix(pi.length, 1, pi: _*)
- private val _theta = new DoubleMatrix(theta)
+class NaiveBayesModel(
+ val labels: Array[Double],
+ val pi: Array[Double],
+ val theta: Array[Array[Double]]) extends ClassificationModel with Serializable {
+
+ private val brzPi = new BDV[Double](pi)
+ private val brzTheta = new BDM[Double](theta.length, theta(0).length)
+
+ var i = 0
+ while (i < theta.length) {
+ var j = 0
+ while (j < theta(i).length) {
+ brzTheta(i, j) = theta(i)(j)
+ j += 1
+ }
+ i += 1
+ }
- def predict(testData: RDD[Array[Double]]): RDD[Double] = testData.map(predict)
+ override def predict(testData: RDD[Vector]): RDD[Double] = testData.map(predict)
- def predict(testData: Array[Double]): Double = {
- val dataMatrix = new DoubleMatrix(testData.length, 1, testData: _*)
- val result = _pi.add(_theta.mmul(dataMatrix))
- result.argmax()
+ override def predict(testData: Vector): Double = {
+ labels(brzArgmax(brzPi + brzTheta * testData.toBreeze))
}
}
@@ -56,9 +65,8 @@ class NaiveBayesModel(val pi: Array[Double], val theta: Array[Array[Double]])
* document classification. By making every vector a 0-1 vector, it can also be used as
* Bernoulli NB ([[http://tinyurl.com/p7c96j6]]).
*/
-class NaiveBayes private (var lambda: Double)
- extends Serializable with Logging
-{
+class NaiveBayes private (var lambda: Double) extends Serializable with Logging {
+
def this() = this(1.0)
/** Set the smoothing parameter. Default: 1.0. */
@@ -70,45 +78,42 @@ class NaiveBayes private (var lambda: Double)
/**
* Run the algorithm with the configured parameters on an input RDD of LabeledPoint entries.
*
- * @param data RDD of (label, array of features) pairs.
+ * @param data RDD of [[org.apache.spark.mllib.regression.LabeledPoint]].
*/
def run(data: RDD[LabeledPoint]) = {
- // Aggregates all sample points to driver side to get sample count and summed feature vector
- // for each label. The shape of `zeroCombiner` & `aggregated` is:
- //
- // label: Int -> (count: Int, featuresSum: DoubleMatrix)
- val zeroCombiner = mutable.Map.empty[Int, (Int, DoubleMatrix)]
- val aggregated = data.aggregate(zeroCombiner)({ (combiner, point) =>
- point match {
- case LabeledPoint(label, features) =>
- val (count, featuresSum) = combiner.getOrElse(label.toInt, (0, DoubleMatrix.zeros(1)))
- val fs = new DoubleMatrix(features.length, 1, features: _*)
- combiner += label.toInt -> (count + 1, featuresSum.addi(fs))
- }
- }, { (lhs, rhs) =>
- for ((label, (c, fs)) <- rhs) {
- val (count, featuresSum) = lhs.getOrElse(label, (0, DoubleMatrix.zeros(1)))
- lhs(label) = (count + c, featuresSum.addi(fs))
+ // Aggregates term frequencies per label.
+ // TODO: Calling combineByKey and collect creates two stages, we can implement something
+ // TODO: similar to reduceByKeyLocally to save one stage.
+ val aggregated = data.map(p => (p.label, p.features)).combineByKey[(Long, BDV[Double])](
+ createCombiner = (v: Vector) => (1L, v.toBreeze.toDenseVector),
+ mergeValue = (c: (Long, BDV[Double]), v: Vector) => (c._1 + 1L, c._2 += v.toBreeze),
+ mergeCombiners = (c1: (Long, BDV[Double]), c2: (Long, BDV[Double])) =>
+ (c1._1 + c2._1, c1._2 += c2._2)
+ ).collect()
+ val numLabels = aggregated.length
+ var numDocuments = 0L
+ aggregated.foreach { case (_, (n, _)) =>
+ numDocuments += n
+ }
+ val numFeatures = aggregated.head match { case (_, (_, v)) => v.size }
+ val labels = new Array[Double](numLabels)
+ val pi = new Array[Double](numLabels)
+ val theta = Array.fill(numLabels)(new Array[Double](numFeatures))
+ val piLogDenom = math.log(numDocuments + numLabels * lambda)
+ var i = 0
+ aggregated.foreach { case (label, (n, sumTermFreqs)) =>
+ labels(i) = label
+ val thetaLogDenom = math.log(brzSum(sumTermFreqs) + numFeatures * lambda)
+ pi(i) = math.log(n + lambda) - piLogDenom
+ var j = 0
+ while (j < numFeatures) {
+ theta(i)(j) = math.log(sumTermFreqs(j) + lambda) - thetaLogDenom
+ j += 1
}
- lhs
- })
-
- // Kinds of label
- val C = aggregated.size
- // Total sample count
- val N = aggregated.values.map(_._1).sum
-
- val pi = new Array[Double](C)
- val theta = new Array[Array[Double]](C)
- val piLogDenom = math.log(N + C * lambda)
-
- for ((label, (count, fs)) <- aggregated) {
- val thetaLogDenom = math.log(fs.sum() + fs.length * lambda)
- pi(label) = math.log(count + lambda) - piLogDenom
- theta(label) = fs.toArray.map(f => math.log(f + lambda) - thetaLogDenom)
+ i += 1
}
- new NaiveBayesModel(pi, theta)
+ new NaiveBayesModel(labels, pi, theta)
}
}
@@ -158,8 +163,9 @@ object NaiveBayes {
} else {
NaiveBayes.train(data, args(2).toDouble)
}
- println("Pi: " + model.pi.mkString("[", ", ", "]"))
- println("Theta:\n" + model.theta.map(_.mkString("[", ", ", "]")).mkString("[", "\n ", "]"))
+
+ println("Pi\n: " + model.pi)
+ println("Theta:\n" + model.theta)
sc.stop()
}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala
index 6dff29dfb45cc..e31a08899f8bc 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala
@@ -18,13 +18,11 @@
package org.apache.spark.mllib.classification
import org.apache.spark.SparkContext
-import org.apache.spark.rdd.RDD
+import org.apache.spark.mllib.linalg.Vector
import org.apache.spark.mllib.optimization._
import org.apache.spark.mllib.regression._
-import org.apache.spark.mllib.util.MLUtils
-import org.apache.spark.mllib.util.DataValidators
-
-import org.jblas.DoubleMatrix
+import org.apache.spark.mllib.util.{DataValidators, MLUtils}
+import org.apache.spark.rdd.RDD
/**
* Model for Support Vector Machines (SVMs).
@@ -33,15 +31,37 @@ import org.jblas.DoubleMatrix
* @param intercept Intercept computed for this model.
*/
class SVMModel(
- override val weights: Array[Double],
+ override val weights: Vector,
override val intercept: Double)
- extends GeneralizedLinearModel(weights, intercept)
- with ClassificationModel with Serializable {
+ extends GeneralizedLinearModel(weights, intercept) with ClassificationModel with Serializable {
+
+ private var threshold: Option[Double] = Some(0.0)
+
+ /**
+ * Sets the threshold that separates positive predictions from negative predictions. An example
+ * with prediction score greater than or equal to this threshold is identified as an positive,
+ * and negative otherwise. The default value is 0.0.
+ */
+ def setThreshold(threshold: Double): this.type = {
+ this.threshold = Some(threshold)
+ this
+ }
- override def predictPoint(dataMatrix: DoubleMatrix, weightMatrix: DoubleMatrix,
+ /**
+ * Clears the threshold so that `predict` will output raw prediction scores.
+ */
+ def clearThreshold(): this.type = {
+ threshold = None
+ this
+ }
+
+ override def predictPoint(dataMatrix: Vector, weightMatrix: Vector,
intercept: Double) = {
- val margin = dataMatrix.dot(weightMatrix) + intercept
- if (margin < 0) 0.0 else 1.0
+ val margin = weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept
+ threshold match {
+ case Some(t) => if (margin < 0) 0.0 else 1.0
+ case None => margin
+ }
}
}
@@ -71,7 +91,7 @@ class SVMWithSGD private (
*/
def this() = this(1.0, 100, 1.0, 1.0)
- def createModel(weights: Array[Double], intercept: Double) = {
+ def createModel(weights: Vector, intercept: Double) = {
new SVMModel(weights, intercept)
}
}
@@ -103,11 +123,9 @@ object SVMWithSGD {
stepSize: Double,
regParam: Double,
miniBatchFraction: Double,
- initialWeights: Array[Double])
- : SVMModel =
- {
- new SVMWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input,
- initialWeights)
+ initialWeights: Vector): SVMModel = {
+ new SVMWithSGD(stepSize, numIterations, regParam, miniBatchFraction)
+ .run(input, initialWeights)
}
/**
@@ -127,9 +145,7 @@ object SVMWithSGD {
numIterations: Int,
stepSize: Double,
regParam: Double,
- miniBatchFraction: Double)
- : SVMModel =
- {
+ miniBatchFraction: Double): SVMModel = {
new SVMWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input)
}
@@ -149,9 +165,7 @@ object SVMWithSGD {
input: RDD[LabeledPoint],
numIterations: Int,
stepSize: Double,
- regParam: Double)
- : SVMModel =
- {
+ regParam: Double): SVMModel = {
train(input, numIterations, stepSize, regParam, 1.0)
}
@@ -165,11 +179,7 @@ object SVMWithSGD {
* @param numIterations Number of iterations of gradient descent to run.
* @return a SVMModel which has the weights and offset from training.
*/
- def train(
- input: RDD[LabeledPoint],
- numIterations: Int)
- : SVMModel =
- {
+ def train(input: RDD[LabeledPoint], numIterations: Int): SVMModel = {
train(input, numIterations, 1.0, 1.0, 1.0)
}
@@ -181,7 +191,8 @@ object SVMWithSGD {
val sc = new SparkContext(args(0), "SVM")
val data = MLUtils.loadLabeledData(sc, args(1))
val model = SVMWithSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble)
- println("Weights: " + model.weights.mkString("[", ", ", "]"))
+
+ println("Weights: " + model.weights)
println("Intercept: " + model.intercept)
sc.stop()
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala
index b412738e3f00a..a78503df3134d 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala
@@ -42,8 +42,7 @@ class KMeans private (
var runs: Int,
var initializationMode: String,
var initializationSteps: Int,
- var epsilon: Double)
- extends Serializable with Logging {
+ var epsilon: Double) extends Serializable with Logging {
def this() = this(2, 20, 1, KMeans.K_MEANS_PARALLEL, 5, 1e-4)
/** Set the number of clusters to create (k). Default: 2. */
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala
index 01c1501548f87..2cea58cd3fd22 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala
@@ -54,6 +54,12 @@ trait Vector extends Serializable {
* Converts the instance to a breeze vector.
*/
private[mllib] def toBreeze: BV[Double]
+
+ /**
+ * Gets the value of the ith element.
+ * @param i index
+ */
+ private[mllib] def apply(i: Int): Double = toBreeze(i)
}
/**
@@ -145,6 +151,8 @@ class DenseVector(val values: Array[Double]) extends Vector {
override def toArray: Array[Double] = values
private[mllib] override def toBreeze: BV[Double] = new BDV[Double](values)
+
+ override def apply(i: Int) = values(i)
}
/**
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala
index 82124703da6cd..20654284965ed 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala
@@ -17,7 +17,9 @@
package org.apache.spark.mllib.optimization
-import org.jblas.DoubleMatrix
+import breeze.linalg.{axpy => brzAxpy}
+
+import org.apache.spark.mllib.linalg.{Vectors, Vector}
/**
* Class used to compute the gradient for a loss function, given a single data point.
@@ -26,17 +28,26 @@ abstract class Gradient extends Serializable {
/**
* Compute the gradient and loss given the features of a single data point.
*
- * @param data - Feature values for one data point. Column matrix of size dx1
- * where d is the number of features.
- * @param label - Label for this data item.
- * @param weights - Column matrix containing weights for every feature.
+ * @param data features for one data point
+ * @param label label for this data point
+ * @param weights weights/coefficients corresponding to features
*
- * @return A tuple of 2 elements. The first element is a column matrix containing the computed
- * gradient and the second element is the loss computed at this data point.
+ * @return (gradient: Vector, loss: Double)
+ */
+ def compute(data: Vector, label: Double, weights: Vector): (Vector, Double)
+
+ /**
+ * Compute the gradient and loss given the features of a single data point,
+ * add the gradient to a provided vector to avoid creating new objects, and return loss.
*
+ * @param data features for one data point
+ * @param label label for this data point
+ * @param weights weights/coefficients corresponding to features
+ * @param cumGradient the computed gradient will be added to this vector
+ *
+ * @return loss
*/
- def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix):
- (DoubleMatrix, Double)
+ def compute(data: Vector, label: Double, weights: Vector, cumGradient: Vector): Double
}
/**
@@ -44,12 +55,12 @@ abstract class Gradient extends Serializable {
* See also the documentation for the precise formulation.
*/
class LogisticGradient extends Gradient {
- override def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix):
- (DoubleMatrix, Double) = {
- val margin: Double = -1.0 * data.dot(weights)
+ override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = {
+ val brzData = data.toBreeze
+ val brzWeights = weights.toBreeze
+ val margin: Double = -1.0 * brzWeights.dot(brzData)
val gradientMultiplier = (1.0 / (1.0 + math.exp(margin))) - label
-
- val gradient = data.mul(gradientMultiplier)
+ val gradient = brzData * gradientMultiplier
val loss =
if (label > 0) {
math.log(1 + math.exp(margin))
@@ -57,7 +68,26 @@ class LogisticGradient extends Gradient {
math.log(1 + math.exp(margin)) - margin
}
- (gradient, loss)
+ (Vectors.fromBreeze(gradient), loss)
+ }
+
+ override def compute(
+ data: Vector,
+ label: Double,
+ weights: Vector,
+ cumGradient: Vector): Double = {
+ val brzData = data.toBreeze
+ val brzWeights = weights.toBreeze
+ val margin: Double = -1.0 * brzWeights.dot(brzData)
+ val gradientMultiplier = (1.0 / (1.0 + math.exp(margin))) - label
+
+ brzAxpy(gradientMultiplier, brzData, cumGradient.toBreeze)
+
+ if (label > 0) {
+ math.log(1 + math.exp(margin))
+ } else {
+ math.log(1 + math.exp(margin)) - margin
+ }
}
}
@@ -68,14 +98,28 @@ class LogisticGradient extends Gradient {
* See also the documentation for the precise formulation.
*/
class LeastSquaresGradient extends Gradient {
- override def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix):
- (DoubleMatrix, Double) = {
- val diff: Double = data.dot(weights) - label
-
+ override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = {
+ val brzData = data.toBreeze
+ val brzWeights = weights.toBreeze
+ val diff = brzWeights.dot(brzData) - label
val loss = diff * diff
- val gradient = data.mul(2.0 * diff)
+ val gradient = brzData * (2.0 * diff)
- (gradient, loss)
+ (Vectors.fromBreeze(gradient), loss)
+ }
+
+ override def compute(
+ data: Vector,
+ label: Double,
+ weights: Vector,
+ cumGradient: Vector): Double = {
+ val brzData = data.toBreeze
+ val brzWeights = weights.toBreeze
+ val diff = brzWeights.dot(brzData) - label
+
+ brzAxpy(2.0 * diff, brzData, cumGradient.toBreeze)
+
+ diff * diff
}
}
@@ -85,19 +129,40 @@ class LeastSquaresGradient extends Gradient {
* NOTE: This assumes that the labels are {0,1}
*/
class HingeGradient extends Gradient {
- override def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix):
- (DoubleMatrix, Double) = {
+ override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = {
+ val brzData = data.toBreeze
+ val brzWeights = weights.toBreeze
+ val dotProduct = brzWeights.dot(brzData)
+
+ // Our loss function with {0, 1} labels is max(0, 1 - (2y – 1) (f_w(x)))
+ // Therefore the gradient is -(2y - 1)*x
+ val labelScaled = 2 * label - 1.0
+
+ if (1.0 > labelScaled * dotProduct) {
+ (Vectors.fromBreeze(brzData * (-labelScaled)), 1.0 - labelScaled * dotProduct)
+ } else {
+ (Vectors.dense(new Array[Double](weights.size)), 0.0)
+ }
+ }
- val dotProduct = data.dot(weights)
+ override def compute(
+ data: Vector,
+ label: Double,
+ weights: Vector,
+ cumGradient: Vector): Double = {
+ val brzData = data.toBreeze
+ val brzWeights = weights.toBreeze
+ val dotProduct = brzWeights.dot(brzData)
// Our loss function with {0, 1} labels is max(0, 1 - (2y – 1) (f_w(x)))
// Therefore the gradient is -(2y - 1)*x
val labelScaled = 2 * label - 1.0
if (1.0 > labelScaled * dotProduct) {
- (data.mul(-labelScaled), 1.0 - labelScaled * dotProduct)
+ brzAxpy(-labelScaled, brzData, cumGradient.toBreeze)
+ 1.0 - labelScaled * dotProduct
} else {
- (DoubleMatrix.zeros(1, weights.length), 0.0)
+ 0.0
}
}
}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala
index b967b22e818d3..d0777ffd63ff8 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala
@@ -17,12 +17,13 @@
package org.apache.spark.mllib.optimization
-import org.apache.spark.Logging
-import org.apache.spark.rdd.RDD
+import scala.collection.mutable.ArrayBuffer
-import org.jblas.DoubleMatrix
+import breeze.linalg.{Vector => BV, DenseVector => BDV}
-import scala.collection.mutable.ArrayBuffer
+import org.apache.spark.Logging
+import org.apache.spark.rdd.RDD
+import org.apache.spark.mllib.linalg.{Vectors, Vector}
/**
* Class used to solve an optimization problem using Gradient Descent.
@@ -91,18 +92,16 @@ class GradientDescent(var gradient: Gradient, var updater: Updater)
this
}
- def optimize(data: RDD[(Double, Array[Double])], initialWeights: Array[Double])
- : Array[Double] = {
-
- val (weights, stochasticLossHistory) = GradientDescent.runMiniBatchSGD(
- data,
- gradient,
- updater,
- stepSize,
- numIterations,
- regParam,
- miniBatchFraction,
- initialWeights)
+ def optimize(data: RDD[(Double, Vector)], initialWeights: Vector): Vector = {
+ val (weights, _) = GradientDescent.runMiniBatchSGD(
+ data,
+ gradient,
+ updater,
+ stepSize,
+ numIterations,
+ regParam,
+ miniBatchFraction,
+ initialWeights)
weights
}
@@ -133,14 +132,14 @@ object GradientDescent extends Logging {
* stochastic loss computed for every iteration.
*/
def runMiniBatchSGD(
- data: RDD[(Double, Array[Double])],
+ data: RDD[(Double, Vector)],
gradient: Gradient,
updater: Updater,
stepSize: Double,
numIterations: Int,
regParam: Double,
miniBatchFraction: Double,
- initialWeights: Array[Double]) : (Array[Double], Array[Double]) = {
+ initialWeights: Vector): (Vector, Array[Double]) = {
val stochasticLossHistory = new ArrayBuffer[Double](numIterations)
@@ -148,24 +147,27 @@ object GradientDescent extends Logging {
val miniBatchSize = nexamples * miniBatchFraction
// Initialize weights as a column vector
- var weights = new DoubleMatrix(initialWeights.length, 1, initialWeights:_*)
+ var weights = Vectors.dense(initialWeights.toArray)
/**
* For the first iteration, the regVal will be initialized as sum of sqrt of
* weights if it's L2 update; for L1 update; the same logic is followed.
*/
var regVal = updater.compute(
- weights, new DoubleMatrix(initialWeights.length, 1), 0, 1, regParam)._2
+ weights, Vectors.dense(new Array[Double](weights.size)), 0, 1, regParam)._2
for (i <- 1 to numIterations) {
// Sample a subset (fraction miniBatchFraction) of the total data
// compute and sum up the subgradients on this subset (this is one map-reduce)
- val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42 + i).map {
- case (y, features) =>
- val featuresCol = new DoubleMatrix(features.length, 1, features:_*)
- val (grad, loss) = gradient.compute(featuresCol, y, weights)
- (grad, loss)
- }.reduce((a, b) => (a._1.addi(b._1), a._2 + b._2))
+ val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42 + i)
+ .aggregate((BDV.zeros[Double](weights.size), 0.0))(
+ seqOp = (c, v) => (c, v) match { case ((grad, loss), (label, features)) =>
+ val l = gradient.compute(features, label, weights, Vectors.fromBreeze(grad))
+ (grad, loss + l)
+ },
+ combOp = (c1, c2) => (c1, c2) match { case ((grad1, loss1), (grad2, loss2)) =>
+ (grad1 += grad2, loss1 + loss2)
+ })
/**
* NOTE(Xinghao): lossSum is computed using the weights from the previous iteration
@@ -173,7 +175,7 @@ object GradientDescent extends Logging {
*/
stochasticLossHistory.append(lossSum / miniBatchSize + regVal)
val update = updater.compute(
- weights, gradientSum.div(miniBatchSize), stepSize, i, regParam)
+ weights, Vectors.fromBreeze(gradientSum / miniBatchSize), stepSize, i, regParam)
weights = update._1
regVal = update._2
}
@@ -181,6 +183,6 @@ object GradientDescent extends Logging {
logInfo("GradientDescent.runMiniBatchSGD finished. Last 10 stochastic losses %s".format(
stochasticLossHistory.takeRight(10).mkString(", ")))
- (weights.toArray, stochasticLossHistory.toArray)
+ (weights, stochasticLossHistory.toArray)
}
}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala
index 94d30b56f212b..f9ce908a5f3b0 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala
@@ -19,11 +19,12 @@ package org.apache.spark.mllib.optimization
import org.apache.spark.rdd.RDD
-trait Optimizer {
+import org.apache.spark.mllib.linalg.Vector
+
+trait Optimizer extends Serializable {
/**
* Solve the provided convex optimization problem.
*/
- def optimize(data: RDD[(Double, Array[Double])], initialWeights: Array[Double]): Array[Double]
-
+ def optimize(data: RDD[(Double, Vector)], initialWeights: Vector): Vector
}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala
index bf8f731459e99..3b7754cd7ac28 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala
@@ -18,7 +18,10 @@
package org.apache.spark.mllib.optimization
import scala.math._
-import org.jblas.DoubleMatrix
+
+import breeze.linalg.{norm => brzNorm, axpy => brzAxpy, Vector => BV}
+
+import org.apache.spark.mllib.linalg.{Vectors, Vector}
/**
* Class used to perform steps (weight update) using Gradient Descent methods.
@@ -47,8 +50,12 @@ abstract class Updater extends Serializable {
* @return A tuple of 2 elements. The first element is a column matrix containing updated weights,
* and the second element is the regularization value computed using updated weights.
*/
- def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int,
- regParam: Double): (DoubleMatrix, Double)
+ def compute(
+ weightsOld: Vector,
+ gradient: Vector,
+ stepSize: Double,
+ iter: Int,
+ regParam: Double): (Vector, Double)
}
/**
@@ -56,11 +63,17 @@ abstract class Updater extends Serializable {
* Uses a step-size decreasing with the square root of the number of iterations.
*/
class SimpleUpdater extends Updater {
- override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix,
- stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) = {
+ override def compute(
+ weightsOld: Vector,
+ gradient: Vector,
+ stepSize: Double,
+ iter: Int,
+ regParam: Double): (Vector, Double) = {
val thisIterStepSize = stepSize / math.sqrt(iter)
- val step = gradient.mul(thisIterStepSize)
- (weightsOld.sub(step), 0)
+ val brzWeights: BV[Double] = weightsOld.toBreeze.toDenseVector
+ brzAxpy(-thisIterStepSize, gradient.toBreeze, brzWeights)
+
+ (Vectors.fromBreeze(brzWeights), 0)
}
}
@@ -83,19 +96,26 @@ class SimpleUpdater extends Updater {
* Equivalently, set weight component to signum(w) * max(0.0, abs(w) - shrinkageVal)
*/
class L1Updater extends Updater {
- override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix,
- stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) = {
+ override def compute(
+ weightsOld: Vector,
+ gradient: Vector,
+ stepSize: Double,
+ iter: Int,
+ regParam: Double): (Vector, Double) = {
val thisIterStepSize = stepSize / math.sqrt(iter)
- val step = gradient.mul(thisIterStepSize)
// Take gradient step
- val newWeights = weightsOld.sub(step)
+ val brzWeights: BV[Double] = weightsOld.toBreeze.toDenseVector
+ brzAxpy(-thisIterStepSize, gradient.toBreeze, brzWeights)
// Apply proximal operator (soft thresholding)
val shrinkageVal = regParam * thisIterStepSize
- (0 until newWeights.length).foreach { i =>
- val wi = newWeights.get(i)
- newWeights.put(i, signum(wi) * max(0.0, abs(wi) - shrinkageVal))
+ var i = 0
+ while (i < brzWeights.length) {
+ val wi = brzWeights(i)
+ brzWeights(i) = signum(wi) * max(0.0, abs(wi) - shrinkageVal)
+ i += 1
}
- (newWeights, newWeights.norm1 * regParam)
+
+ (Vectors.fromBreeze(brzWeights), brzNorm(brzWeights, 1.0) * regParam)
}
}
@@ -105,16 +125,23 @@ class L1Updater extends Updater {
* Uses a step-size decreasing with the square root of the number of iterations.
*/
class SquaredL2Updater extends Updater {
- override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix,
- stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) = {
- val thisIterStepSize = stepSize / math.sqrt(iter)
- val step = gradient.mul(thisIterStepSize)
+ override def compute(
+ weightsOld: Vector,
+ gradient: Vector,
+ stepSize: Double,
+ iter: Int,
+ regParam: Double): (Vector, Double) = {
// add up both updates from the gradient of the loss (= step) as well as
// the gradient of the regularizer (= regParam * weightsOld)
// w' = w - thisIterStepSize * (gradient + regParam * w)
// w' = (1 - thisIterStepSize * regParam) * w - thisIterStepSize * gradient
- val newWeights = weightsOld.mul(1.0 - thisIterStepSize * regParam).sub(step)
- (newWeights, 0.5 * pow(newWeights.norm2, 2.0) * regParam)
+ val thisIterStepSize = stepSize / math.sqrt(iter)
+ val brzWeights: BV[Double] = weightsOld.toBreeze.toDenseVector
+ brzWeights :*= (1.0 - thisIterStepSize * regParam)
+ brzAxpy(-thisIterStepSize, gradient.toBreeze, brzWeights)
+ val norm = brzNorm(brzWeights, 2.0)
+
+ (Vectors.fromBreeze(brzWeights), 0.5 * regParam * norm * norm)
}
}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala
index 3e1ed91bf6729..80dc0f12ff84f 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala
@@ -17,11 +17,12 @@
package org.apache.spark.mllib.regression
+import breeze.linalg.{DenseVector => BDV, SparseVector => BSV}
+
import org.apache.spark.{Logging, SparkException}
import org.apache.spark.rdd.RDD
import org.apache.spark.mllib.optimization._
-
-import org.jblas.DoubleMatrix
+import org.apache.spark.mllib.linalg.{Vectors, Vector}
/**
* GeneralizedLinearModel (GLM) represents a model trained using
@@ -31,12 +32,9 @@ import org.jblas.DoubleMatrix
* @param weights Weights computed for every feature.
* @param intercept Intercept computed for this model.
*/
-abstract class GeneralizedLinearModel(val weights: Array[Double], val intercept: Double)
+abstract class GeneralizedLinearModel(val weights: Vector, val intercept: Double)
extends Serializable {
- // Create a column vector that can be used for predictions
- private val weightsMatrix = new DoubleMatrix(weights.length, 1, weights:_*)
-
/**
* Predict the result given a data point and the weights learned.
*
@@ -44,8 +42,7 @@ abstract class GeneralizedLinearModel(val weights: Array[Double], val intercept:
* @param weightMatrix Column vector containing the weights of the model
* @param intercept Intercept of the model.
*/
- def predictPoint(dataMatrix: DoubleMatrix, weightMatrix: DoubleMatrix,
- intercept: Double): Double
+ protected def predictPoint(dataMatrix: Vector, weightMatrix: Vector, intercept: Double): Double
/**
* Predict values for the given data set using the model trained.
@@ -53,16 +50,13 @@ abstract class GeneralizedLinearModel(val weights: Array[Double], val intercept:
* @param testData RDD representing data points to be predicted
* @return RDD[Double] where each entry contains the corresponding prediction
*/
- def predict(testData: RDD[Array[Double]]): RDD[Double] = {
+ def predict(testData: RDD[Vector]): RDD[Double] = {
// A small optimization to avoid serializing the entire model. Only the weightsMatrix
// and intercept is needed.
- val localWeights = weightsMatrix
+ val localWeights = weights
val localIntercept = intercept
- testData.map { x =>
- val dataMatrix = new DoubleMatrix(1, x.length, x:_*)
- predictPoint(dataMatrix, localWeights, localIntercept)
- }
+ testData.map(v => predictPoint(v, localWeights, localIntercept))
}
/**
@@ -71,14 +65,13 @@ abstract class GeneralizedLinearModel(val weights: Array[Double], val intercept:
* @param testData array representing a single data point
* @return Double prediction from the trained model
*/
- def predict(testData: Array[Double]): Double = {
- val dataMat = new DoubleMatrix(1, testData.length, testData:_*)
- predictPoint(dataMat, weightsMatrix, intercept)
+ def predict(testData: Vector): Double = {
+ predictPoint(testData, weights, intercept)
}
}
/**
- * GeneralizedLinearAlgorithm implements methods to train a Genearalized Linear Model (GLM).
+ * GeneralizedLinearAlgorithm implements methods to train a Generalized Linear Model (GLM).
* This class should be extended with an Optimizer to create a new GLM.
*/
abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel]
@@ -88,6 +81,7 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel]
val optimizer: Optimizer
+ /** Whether to add intercept (default: true). */
protected var addIntercept: Boolean = true
protected var validateData: Boolean = true
@@ -95,7 +89,7 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel]
/**
* Create a model given the weights and intercept
*/
- protected def createModel(weights: Array[Double], intercept: Double): M
+ protected def createModel(weights: Vector, intercept: Double): M
/**
* Set if the algorithm should add an intercept. Default true.
@@ -117,17 +111,27 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel]
* Run the algorithm with the configured parameters on an input
* RDD of LabeledPoint entries.
*/
- def run(input: RDD[LabeledPoint]) : M = {
- val nfeatures: Int = input.first().features.length
- val initialWeights = new Array[Double](nfeatures)
+ def run(input: RDD[LabeledPoint]): M = {
+ val numFeatures: Int = input.first().features.size
+ val initialWeights = Vectors.dense(new Array[Double](numFeatures))
run(input, initialWeights)
}
+ /** Prepends one to the input vector. */
+ private def prependOne(vector: Vector): Vector = {
+ val vector1 = vector.toBreeze match {
+ case dv: BDV[Double] => BDV.vertcat(BDV.ones[Double](1), dv)
+ case sv: BSV[Double] => BSV.vertcat(new BSV[Double](Array(0), Array(1.0), 1), sv)
+ case v: Any => throw new IllegalArgumentException("Do not support vector type " + v.getClass)
+ }
+ Vectors.fromBreeze(vector1)
+ }
+
/**
* Run the algorithm with the configured parameters on an input RDD
* of LabeledPoint entries starting from the initial weights provided.
*/
- def run(input: RDD[LabeledPoint], initialWeights: Array[Double]) : M = {
+ def run(input: RDD[LabeledPoint], initialWeights: Vector): M = {
// Check the data properties before running the optimizer
if (validateData && !validators.forall(func => func(input))) {
@@ -136,27 +140,26 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel]
// Prepend an extra variable consisting of all 1.0's for the intercept.
val data = if (addIntercept) {
- input.map(labeledPoint => (labeledPoint.label, 1.0 +: labeledPoint.features))
+ input.map(labeledPoint => (labeledPoint.label, prependOne(labeledPoint.features)))
} else {
input.map(labeledPoint => (labeledPoint.label, labeledPoint.features))
}
val initialWeightsWithIntercept = if (addIntercept) {
- 0.0 +: initialWeights
+ prependOne(initialWeights)
} else {
initialWeights
}
val weightsWithIntercept = optimizer.optimize(data, initialWeightsWithIntercept)
- val (intercept, weights) = if (addIntercept) {
- (weightsWithIntercept(0), weightsWithIntercept.tail)
- } else {
- (0.0, weightsWithIntercept)
- }
-
- logInfo("Final weights " + weights.mkString(","))
- logInfo("Final intercept " + intercept)
+ val intercept = if (addIntercept) weightsWithIntercept(0) else 0.0
+ val weights =
+ if (addIntercept) {
+ Vectors.dense(weightsWithIntercept.toArray.slice(1, weightsWithIntercept.size))
+ } else {
+ weightsWithIntercept
+ }
createModel(weights, intercept)
}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala
index 1a18292fe3f3b..3deab1ab785b9 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala
@@ -17,14 +17,16 @@
package org.apache.spark.mllib.regression
+import org.apache.spark.mllib.linalg.Vector
+
/**
* Class that represents the features and labels of a data point.
*
* @param label Label for this data point.
* @param features List of features for this data point.
*/
-case class LabeledPoint(label: Double, features: Array[Double]) {
+case class LabeledPoint(label: Double, features: Vector) {
override def toString: String = {
- "LabeledPoint(%s, %s)".format(label, features.mkString("[", ", ", "]"))
+ "LabeledPoint(%s, %s)".format(label, features)
}
}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala
index be63ce8538fef..25920d0dc976e 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala
@@ -17,12 +17,11 @@
package org.apache.spark.mllib.regression
-import org.apache.spark.{Logging, SparkContext}
-import org.apache.spark.rdd.RDD
+import org.apache.spark.SparkContext
+import org.apache.spark.mllib.linalg.Vector
import org.apache.spark.mllib.optimization._
import org.apache.spark.mllib.util.MLUtils
-
-import org.jblas.DoubleMatrix
+import org.apache.spark.rdd.RDD
/**
* Regression model trained using Lasso.
@@ -31,16 +30,16 @@ import org.jblas.DoubleMatrix
* @param intercept Intercept computed for this model.
*/
class LassoModel(
- override val weights: Array[Double],
+ override val weights: Vector,
override val intercept: Double)
extends GeneralizedLinearModel(weights, intercept)
with RegressionModel with Serializable {
- override def predictPoint(
- dataMatrix: DoubleMatrix,
- weightMatrix: DoubleMatrix,
+ override protected def predictPoint(
+ dataMatrix: Vector,
+ weightMatrix: Vector,
intercept: Double): Double = {
- dataMatrix.dot(weightMatrix) + intercept
+ weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept
}
}
@@ -57,8 +56,7 @@ class LassoWithSGD private (
var numIterations: Int,
var regParam: Double,
var miniBatchFraction: Double)
- extends GeneralizedLinearAlgorithm[LassoModel]
- with Serializable {
+ extends GeneralizedLinearAlgorithm[LassoModel] with Serializable {
val gradient = new LeastSquaresGradient()
val updater = new L1Updater()
@@ -70,10 +68,6 @@ class LassoWithSGD private (
// We don't want to penalize the intercept, so set this to false.
super.setIntercept(false)
- var yMean = 0.0
- var xColMean: DoubleMatrix = _
- var xColSd: DoubleMatrix = _
-
/**
* Construct a Lasso object with default parameters
*/
@@ -85,36 +79,8 @@ class LassoWithSGD private (
this
}
- override def createModel(weights: Array[Double], intercept: Double) = {
- val weightsMat = new DoubleMatrix(weights.length, 1, weights: _*)
- val weightsScaled = weightsMat.div(xColSd)
- val interceptScaled = yMean - weightsMat.transpose().mmul(xColMean.div(xColSd)).get(0)
-
- new LassoModel(weightsScaled.data, interceptScaled)
- }
-
- override def run(
- input: RDD[LabeledPoint],
- initialWeights: Array[Double])
- : LassoModel =
- {
- val nfeatures: Int = input.first.features.length
- val nexamples: Long = input.count()
-
- // To avoid penalizing the intercept, we center and scale the data.
- val stats = MLUtils.computeStats(input, nfeatures, nexamples)
- yMean = stats._1
- xColMean = stats._2
- xColSd = stats._3
-
- val normalizedData = input.map { point =>
- val yNormalized = point.label - yMean
- val featuresMat = new DoubleMatrix(nfeatures, 1, point.features:_*)
- val featuresNormalized = featuresMat.sub(xColMean).divi(xColSd)
- LabeledPoint(yNormalized, featuresNormalized.toArray)
- }
-
- super.run(normalizedData, initialWeights)
+ override protected def createModel(weights: Vector, intercept: Double) = {
+ new LassoModel(weights, intercept)
}
}
@@ -144,11 +110,9 @@ object LassoWithSGD {
stepSize: Double,
regParam: Double,
miniBatchFraction: Double,
- initialWeights: Array[Double])
- : LassoModel =
- {
- new LassoWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input,
- initialWeights)
+ initialWeights: Vector): LassoModel = {
+ new LassoWithSGD(stepSize, numIterations, regParam, miniBatchFraction)
+ .run(input, initialWeights)
}
/**
@@ -168,9 +132,7 @@ object LassoWithSGD {
numIterations: Int,
stepSize: Double,
regParam: Double,
- miniBatchFraction: Double)
- : LassoModel =
- {
+ miniBatchFraction: Double): LassoModel = {
new LassoWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input)
}
@@ -190,9 +152,7 @@ object LassoWithSGD {
input: RDD[LabeledPoint],
numIterations: Int,
stepSize: Double,
- regParam: Double)
- : LassoModel =
- {
+ regParam: Double): LassoModel = {
train(input, numIterations, stepSize, regParam, 1.0)
}
@@ -208,9 +168,7 @@ object LassoWithSGD {
*/
def train(
input: RDD[LabeledPoint],
- numIterations: Int)
- : LassoModel =
- {
+ numIterations: Int): LassoModel = {
train(input, numIterations, 1.0, 1.0, 1.0)
}
@@ -222,7 +180,8 @@ object LassoWithSGD {
val sc = new SparkContext(args(0), "Lasso")
val data = MLUtils.loadLabeledData(sc, args(1))
val model = LassoWithSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble)
- println("Weights: " + model.weights.mkString("[", ", ", "]"))
+
+ println("Weights: " + model.weights)
println("Intercept: " + model.intercept)
sc.stop()
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala
index f5f15d1a33f4d..9ed927994e795 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala
@@ -19,11 +19,10 @@ package org.apache.spark.mllib.regression
import org.apache.spark.SparkContext
import org.apache.spark.rdd.RDD
+import org.apache.spark.mllib.linalg.Vector
import org.apache.spark.mllib.optimization._
import org.apache.spark.mllib.util.MLUtils
-import org.jblas.DoubleMatrix
-
/**
* Regression model trained using LinearRegression.
*
@@ -31,15 +30,15 @@ import org.jblas.DoubleMatrix
* @param intercept Intercept computed for this model.
*/
class LinearRegressionModel(
- override val weights: Array[Double],
+ override val weights: Vector,
override val intercept: Double)
extends GeneralizedLinearModel(weights, intercept) with RegressionModel with Serializable {
- override def predictPoint(
- dataMatrix: DoubleMatrix,
- weightMatrix: DoubleMatrix,
+ override protected def predictPoint(
+ dataMatrix: Vector,
+ weightMatrix: Vector,
intercept: Double): Double = {
- dataMatrix.dot(weightMatrix) + intercept
+ weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept
}
}
@@ -69,7 +68,7 @@ class LinearRegressionWithSGD private (
*/
def this() = this(1.0, 100, 1.0)
- override def createModel(weights: Array[Double], intercept: Double) = {
+ override protected def createModel(weights: Vector, intercept: Double) = {
new LinearRegressionModel(weights, intercept)
}
}
@@ -98,11 +97,9 @@ object LinearRegressionWithSGD {
numIterations: Int,
stepSize: Double,
miniBatchFraction: Double,
- initialWeights: Array[Double])
- : LinearRegressionModel =
- {
- new LinearRegressionWithSGD(stepSize, numIterations, miniBatchFraction).run(input,
- initialWeights)
+ initialWeights: Vector): LinearRegressionModel = {
+ new LinearRegressionWithSGD(stepSize, numIterations, miniBatchFraction)
+ .run(input, initialWeights)
}
/**
@@ -120,9 +117,7 @@ object LinearRegressionWithSGD {
input: RDD[LabeledPoint],
numIterations: Int,
stepSize: Double,
- miniBatchFraction: Double)
- : LinearRegressionModel =
- {
+ miniBatchFraction: Double): LinearRegressionModel = {
new LinearRegressionWithSGD(stepSize, numIterations, miniBatchFraction).run(input)
}
@@ -140,9 +135,7 @@ object LinearRegressionWithSGD {
def train(
input: RDD[LabeledPoint],
numIterations: Int,
- stepSize: Double)
- : LinearRegressionModel =
- {
+ stepSize: Double): LinearRegressionModel = {
train(input, numIterations, stepSize, 1.0)
}
@@ -158,9 +151,7 @@ object LinearRegressionWithSGD {
*/
def train(
input: RDD[LabeledPoint],
- numIterations: Int)
- : LinearRegressionModel =
- {
+ numIterations: Int): LinearRegressionModel = {
train(input, numIterations, 1.0, 1.0)
}
@@ -172,7 +163,7 @@ object LinearRegressionWithSGD {
val sc = new SparkContext(args(0), "LinearRegression")
val data = MLUtils.loadLabeledData(sc, args(1))
val model = LinearRegressionWithSGD.train(data, args(3).toInt, args(2).toDouble)
- println("Weights: " + model.weights.mkString("[", ", ", "]"))
+ println("Weights: " + model.weights)
println("Intercept: " + model.intercept)
sc.stop()
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala
index 423afc32d665c..5e4b8a345b1c5 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala
@@ -18,6 +18,7 @@
package org.apache.spark.mllib.regression
import org.apache.spark.rdd.RDD
+import org.apache.spark.mllib.linalg.Vector
trait RegressionModel extends Serializable {
/**
@@ -26,7 +27,7 @@ trait RegressionModel extends Serializable {
* @param testData RDD representing data points to be predicted
* @return RDD[Double] where each entry contains the corresponding prediction
*/
- def predict(testData: RDD[Array[Double]]): RDD[Double]
+ def predict(testData: RDD[Vector]): RDD[Double]
/**
* Predict values for a single data point using the model trained.
@@ -34,5 +35,5 @@ trait RegressionModel extends Serializable {
* @param testData array representing a single data point
* @return Double prediction from the trained model
*/
- def predict(testData: Array[Double]): Double
+ def predict(testData: Vector): Double
}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala
index feb100f21888f..1f17d2107f940 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala
@@ -21,8 +21,7 @@ import org.apache.spark.SparkContext
import org.apache.spark.rdd.RDD
import org.apache.spark.mllib.optimization._
import org.apache.spark.mllib.util.MLUtils
-
-import org.jblas.DoubleMatrix
+import org.apache.spark.mllib.linalg.Vector
/**
* Regression model trained using RidgeRegression.
@@ -31,16 +30,16 @@ import org.jblas.DoubleMatrix
* @param intercept Intercept computed for this model.
*/
class RidgeRegressionModel(
- override val weights: Array[Double],
+ override val weights: Vector,
override val intercept: Double)
extends GeneralizedLinearModel(weights, intercept)
with RegressionModel with Serializable {
- override def predictPoint(
- dataMatrix: DoubleMatrix,
- weightMatrix: DoubleMatrix,
+ override protected def predictPoint(
+ dataMatrix: Vector,
+ weightMatrix: Vector,
intercept: Double): Double = {
- dataMatrix.dot(weightMatrix) + intercept
+ weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept
}
}
@@ -57,8 +56,7 @@ class RidgeRegressionWithSGD private (
var numIterations: Int,
var regParam: Double,
var miniBatchFraction: Double)
- extends GeneralizedLinearAlgorithm[RidgeRegressionModel]
- with Serializable {
+ extends GeneralizedLinearAlgorithm[RidgeRegressionModel] with Serializable {
val gradient = new LeastSquaresGradient()
val updater = new SquaredL2Updater()
@@ -71,10 +69,6 @@ class RidgeRegressionWithSGD private (
// We don't want to penalize the intercept in RidgeRegression, so set this to false.
super.setIntercept(false)
- var yMean = 0.0
- var xColMean: DoubleMatrix = _
- var xColSd: DoubleMatrix = _
-
/**
* Construct a RidgeRegression object with default parameters
*/
@@ -86,36 +80,8 @@ class RidgeRegressionWithSGD private (
this
}
- override def createModel(weights: Array[Double], intercept: Double) = {
- val weightsMat = new DoubleMatrix(weights.length, 1, weights: _*)
- val weightsScaled = weightsMat.div(xColSd)
- val interceptScaled = yMean - weightsMat.transpose().mmul(xColMean.div(xColSd)).get(0)
-
- new RidgeRegressionModel(weightsScaled.data, interceptScaled)
- }
-
- override def run(
- input: RDD[LabeledPoint],
- initialWeights: Array[Double])
- : RidgeRegressionModel =
- {
- val nfeatures: Int = input.first().features.length
- val nexamples: Long = input.count()
-
- // To avoid penalizing the intercept, we center and scale the data.
- val stats = MLUtils.computeStats(input, nfeatures, nexamples)
- yMean = stats._1
- xColMean = stats._2
- xColSd = stats._3
-
- val normalizedData = input.map { point =>
- val yNormalized = point.label - yMean
- val featuresMat = new DoubleMatrix(nfeatures, 1, point.features:_*)
- val featuresNormalized = featuresMat.sub(xColMean).divi(xColSd)
- LabeledPoint(yNormalized, featuresNormalized.toArray)
- }
-
- super.run(normalizedData, initialWeights)
+ override protected def createModel(weights: Vector, intercept: Double) = {
+ new RidgeRegressionModel(weights, intercept)
}
}
@@ -144,9 +110,7 @@ object RidgeRegressionWithSGD {
stepSize: Double,
regParam: Double,
miniBatchFraction: Double,
- initialWeights: Array[Double])
- : RidgeRegressionModel =
- {
+ initialWeights: Vector): RidgeRegressionModel = {
new RidgeRegressionWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(
input, initialWeights)
}
@@ -167,9 +131,7 @@ object RidgeRegressionWithSGD {
numIterations: Int,
stepSize: Double,
regParam: Double,
- miniBatchFraction: Double)
- : RidgeRegressionModel =
- {
+ miniBatchFraction: Double): RidgeRegressionModel = {
new RidgeRegressionWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input)
}
@@ -188,9 +150,7 @@ object RidgeRegressionWithSGD {
input: RDD[LabeledPoint],
numIterations: Int,
stepSize: Double,
- regParam: Double)
- : RidgeRegressionModel =
- {
+ regParam: Double): RidgeRegressionModel = {
train(input, numIterations, stepSize, regParam, 1.0)
}
@@ -205,23 +165,22 @@ object RidgeRegressionWithSGD {
*/
def train(
input: RDD[LabeledPoint],
- numIterations: Int)
- : RidgeRegressionModel =
- {
+ numIterations: Int): RidgeRegressionModel = {
train(input, numIterations, 1.0, 1.0, 1.0)
}
def main(args: Array[String]) {
if (args.length != 5) {
- println("Usage: RidgeRegression " +
- " ")
+ println("Usage: RidgeRegression " +
+ " ")
System.exit(1)
}
val sc = new SparkContext(args(0), "RidgeRegression")
val data = MLUtils.loadLabeledData(sc, args(1))
val model = RidgeRegressionWithSGD.train(data, args(4).toInt, args(2).toDouble,
args(3).toDouble)
- println("Weights: " + model.weights.mkString("[", ", ", "]"))
+
+ println("Weights: " + model.weights)
println("Intercept: " + model.intercept)
sc.stop()
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala
new file mode 100644
index 0000000000000..dee9594a9dd79
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala
@@ -0,0 +1,1151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.tree
+
+import scala.util.control.Breaks._
+
+import org.apache.spark.{Logging, SparkContext}
+import org.apache.spark.SparkContext._
+import org.apache.spark.mllib.regression.LabeledPoint
+import org.apache.spark.mllib.tree.configuration.Strategy
+import org.apache.spark.mllib.tree.configuration.Algo._
+import org.apache.spark.mllib.tree.configuration.FeatureType._
+import org.apache.spark.mllib.tree.configuration.QuantileStrategy._
+import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Impurity, Variance}
+import org.apache.spark.mllib.tree.model._
+import org.apache.spark.rdd.RDD
+import org.apache.spark.util.random.XORShiftRandom
+import org.apache.spark.mllib.linalg.{Vector, Vectors}
+
+/**
+ * A class that implements a decision tree algorithm for classification and regression. It
+ * supports both continuous and categorical features.
+ * @param strategy The configuration parameters for the tree algorithm which specify the type
+ * of algorithm (classification, regression, etc.), feature type (continuous,
+ * categorical), depth of the tree, quantile calculation strategy, etc.
+ */
+class DecisionTree private(val strategy: Strategy) extends Serializable with Logging {
+
+ /**
+ * Method to train a decision tree model over an RDD
+ * @param input RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] used as training data
+ * @return a DecisionTreeModel that can be used for prediction
+ */
+ def train(input: RDD[LabeledPoint]): DecisionTreeModel = {
+
+ // Cache input RDD for speedup during multiple passes.
+ input.cache()
+ logDebug("algo = " + strategy.algo)
+
+ // Find the splits and the corresponding bins (interval between the splits) using a sample
+ // of the input data.
+ val (splits, bins) = DecisionTree.findSplitsBins(input, strategy)
+ logDebug("numSplits = " + bins(0).length)
+
+ // depth of the decision tree
+ val maxDepth = strategy.maxDepth
+ // the max number of nodes possible given the depth of the tree
+ val maxNumNodes = scala.math.pow(2, maxDepth).toInt - 1
+ // Initialize an array to hold filters applied to points for each node.
+ val filters = new Array[List[Filter]](maxNumNodes)
+ // The filter at the top node is an empty list.
+ filters(0) = List()
+ // Initialize an array to hold parent impurity calculations for each node.
+ val parentImpurities = new Array[Double](maxNumNodes)
+ // dummy value for top node (updated during first split calculation)
+ val nodes = new Array[Node](maxNumNodes)
+
+
+ /*
+ * The main idea here is to perform level-wise training of the decision tree nodes thus
+ * reducing the passes over the data from l to log2(l) where l is the total number of nodes.
+ * Each data sample is checked for validity w.r.t to each node at a given level -- i.e.,
+ * the sample is only used for the split calculation at the node if the sampled would have
+ * still survived the filters of the parent nodes.
+ */
+
+ // TODO: Convert for loop to while loop
+ breakable {
+ for (level <- 0 until maxDepth) {
+
+ logDebug("#####################################")
+ logDebug("level = " + level)
+ logDebug("#####################################")
+
+ // Find best split for all nodes at a level.
+ val splitsStatsForLevel = DecisionTree.findBestSplits(input, parentImpurities, strategy,
+ level, filters, splits, bins)
+
+ for ((nodeSplitStats, index) <- splitsStatsForLevel.view.zipWithIndex) {
+ // Extract info for nodes at the current level.
+ extractNodeInfo(nodeSplitStats, level, index, nodes)
+ // Extract info for nodes at the next lower level.
+ extractInfoForLowerLevels(level, index, maxDepth, nodeSplitStats, parentImpurities,
+ filters)
+ logDebug("final best split = " + nodeSplitStats._1)
+ }
+ require(scala.math.pow(2, level) == splitsStatsForLevel.length)
+ // Check whether all the nodes at the current level at leaves.
+ val allLeaf = splitsStatsForLevel.forall(_._2.gain <= 0)
+ logDebug("all leaf = " + allLeaf)
+ if (allLeaf) break // no more tree construction
+ }
+ }
+
+ // Initialize the top or root node of the tree.
+ val topNode = nodes(0)
+ // Build the full tree using the node info calculated in the level-wise best split calculations.
+ topNode.build(nodes)
+
+ new DecisionTreeModel(topNode, strategy.algo)
+ }
+
+ /**
+ * Extract the decision tree node information for the given tree level and node index
+ */
+ private def extractNodeInfo(
+ nodeSplitStats: (Split, InformationGainStats),
+ level: Int,
+ index: Int,
+ nodes: Array[Node]): Unit = {
+ val split = nodeSplitStats._1
+ val stats = nodeSplitStats._2
+ val nodeIndex = scala.math.pow(2, level).toInt - 1 + index
+ val isLeaf = (stats.gain <= 0) || (level == strategy.maxDepth - 1)
+ val node = new Node(nodeIndex, stats.predict, isLeaf, Some(split), None, None, Some(stats))
+ logDebug("Node = " + node)
+ nodes(nodeIndex) = node
+ }
+
+ /**
+ * Extract the decision tree node information for the children of the node
+ */
+ private def extractInfoForLowerLevels(
+ level: Int,
+ index: Int,
+ maxDepth: Int,
+ nodeSplitStats: (Split, InformationGainStats),
+ parentImpurities: Array[Double],
+ filters: Array[List[Filter]]): Unit = {
+ // 0 corresponds to the left child node and 1 corresponds to the right child node.
+ // TODO: Convert to while loop
+ for (i <- 0 to 1) {
+ // Calculate the index of the node from the node level and the index at the current level.
+ val nodeIndex = scala.math.pow(2, level + 1).toInt - 1 + 2 * index + i
+ if (level < maxDepth - 1) {
+ val impurity = if (i == 0) {
+ nodeSplitStats._2.leftImpurity
+ } else {
+ nodeSplitStats._2.rightImpurity
+ }
+ logDebug("nodeIndex = " + nodeIndex + ", impurity = " + impurity)
+ // noting the parent impurities
+ parentImpurities(nodeIndex) = impurity
+ // noting the parents filters for the child nodes
+ val childFilter = new Filter(nodeSplitStats._1, if (i == 0) -1 else 1)
+ filters(nodeIndex) = childFilter :: filters((nodeIndex - 1) / 2)
+ for (filter <- filters(nodeIndex)) {
+ logDebug("Filter = " + filter)
+ }
+ }
+ }
+ }
+}
+
+object DecisionTree extends Serializable with Logging {
+
+ /**
+ * Method to train a decision tree model where the instances are represented as an RDD of
+ * (label, features) pairs. The method supports binary classification and regression. For the
+ * binary classification, the label for each instance should either be 0 or 1 to denote the two
+ * classes. The parameters for the algorithm are specified using the strategy parameter.
+ *
+ * @param input RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] used as training data
+ * for DecisionTree
+ * @param strategy The configuration parameters for the tree algorithm which specify the type
+ * of algorithm (classification, regression, etc.), feature type (continuous,
+ * categorical), depth of the tree, quantile calculation strategy, etc.
+ * @return a DecisionTreeModel that can be used for prediction
+ */
+ def train(input: RDD[LabeledPoint], strategy: Strategy): DecisionTreeModel = {
+ new DecisionTree(strategy).train(input: RDD[LabeledPoint])
+ }
+
+ /**
+ * Method to train a decision tree model where the instances are represented as an RDD of
+ * (label, features) pairs. The method supports binary classification and regression. For the
+ * binary classification, the label for each instance should either be 0 or 1 to denote the two
+ * classes.
+ *
+ * @param input input RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] used as
+ * training data
+ * @param algo algorithm, classification or regression
+ * @param impurity impurity criterion used for information gain calculation
+ * @param maxDepth maxDepth maximum depth of the tree
+ * @return a DecisionTreeModel that can be used for prediction
+ */
+ def train(
+ input: RDD[LabeledPoint],
+ algo: Algo,
+ impurity: Impurity,
+ maxDepth: Int): DecisionTreeModel = {
+ val strategy = new Strategy(algo,impurity,maxDepth)
+ new DecisionTree(strategy).train(input: RDD[LabeledPoint])
+ }
+
+
+ /**
+ * Method to train a decision tree model where the instances are represented as an RDD of
+ * (label, features) pairs. The decision tree method supports binary classification and
+ * regression. For the binary classification, the label for each instance should either be 0 or
+ * 1 to denote the two classes. The method also supports categorical features inputs where the
+ * number of categories can specified using the categoricalFeaturesInfo option.
+ *
+ * @param input input RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] used as
+ * training data for DecisionTree
+ * @param algo classification or regression
+ * @param impurity criterion used for information gain calculation
+ * @param maxDepth maximum depth of the tree
+ * @param maxBins maximum number of bins used for splitting features
+ * @param quantileCalculationStrategy algorithm for calculating quantiles
+ * @param categoricalFeaturesInfo A map storing information about the categorical variables and
+ * the number of discrete values they take. For example,
+ * an entry (n -> k) implies the feature n is categorical with k
+ * categories 0, 1, 2, ... , k-1. It's important to note that
+ * features are zero-indexed.
+ * @return a DecisionTreeModel that can be used for prediction
+ */
+ def train(
+ input: RDD[LabeledPoint],
+ algo: Algo,
+ impurity: Impurity,
+ maxDepth: Int,
+ maxBins: Int,
+ quantileCalculationStrategy: QuantileStrategy,
+ categoricalFeaturesInfo: Map[Int,Int]): DecisionTreeModel = {
+ val strategy = new Strategy(algo, impurity, maxDepth, maxBins, quantileCalculationStrategy,
+ categoricalFeaturesInfo)
+ new DecisionTree(strategy).train(input: RDD[LabeledPoint])
+ }
+
+ private val InvalidBinIndex = -1
+
+ /**
+ * Returns an array of optimal splits for all nodes at a given level
+ *
+ * @param input RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] used as training data
+ * for DecisionTree
+ * @param parentImpurities Impurities for all parent nodes for the current level
+ * @param strategy [[org.apache.spark.mllib.tree.configuration.Strategy]] instance containing
+ * parameters for construction the DecisionTree
+ * @param level Level of the tree
+ * @param filters Filters for all nodes at a given level
+ * @param splits possible splits for all features
+ * @param bins possible bins for all features
+ * @return array of splits with best splits for all nodes at a given level.
+ */
+ protected[tree] def findBestSplits(
+ input: RDD[LabeledPoint],
+ parentImpurities: Array[Double],
+ strategy: Strategy,
+ level: Int,
+ filters: Array[List[Filter]],
+ splits: Array[Array[Split]],
+ bins: Array[Array[Bin]]): Array[(Split, InformationGainStats)] = {
+
+ /*
+ * The high-level description for the best split optimizations are noted here.
+ *
+ * *Level-wise training*
+ * We perform bin calculations for all nodes at the given level to avoid making multiple
+ * passes over the data. Thus, for a slightly increased computation and storage cost we save
+ * several iterations over the data especially at higher levels of the decision tree.
+ *
+ * *Bin-wise computation*
+ * We use a bin-wise best split computation strategy instead of a straightforward best split
+ * computation strategy. Instead of analyzing each sample for contribution to the left/right
+ * child node impurity of every split, we first categorize each feature of a sample into a
+ * bin. Each bin is an interval between a low and high split. Since each splits, and thus bin,
+ * is ordered (read ordering for categorical variables in the findSplitsBins method),
+ * we exploit this structure to calculate aggregates for bins and then use these aggregates
+ * to calculate information gain for each split.
+ *
+ * *Aggregation over partitions*
+ * Instead of performing a flatMap/reduceByKey operation, we exploit the fact that we know
+ * the number of splits in advance. Thus, we store the aggregates (at the appropriate
+ * indices) in a single array for all bins and rely upon the RDD aggregate method to
+ * drastically reduce the communication overhead.
+ */
+
+ // common calculations for multiple nested methods
+ val numNodes = scala.math.pow(2, level).toInt
+ logDebug("numNodes = " + numNodes)
+ // Find the number of features by looking at the first sample.
+ val numFeatures = input.first().features.size
+ logDebug("numFeatures = " + numFeatures)
+ val numBins = bins(0).length
+ logDebug("numBins = " + numBins)
+
+ /** Find the filters used before reaching the current code. */
+ def findParentFilters(nodeIndex: Int): List[Filter] = {
+ if (level == 0) {
+ List[Filter]()
+ } else {
+ val nodeFilterIndex = scala.math.pow(2, level).toInt - 1 + nodeIndex
+ filters(nodeFilterIndex)
+ }
+ }
+
+ /**
+ * Find whether the sample is valid input for the current node, i.e., whether it passes through
+ * all the filters for the current node.
+ */
+ def isSampleValid(parentFilters: List[Filter], labeledPoint: LabeledPoint): Boolean = {
+ // leaf
+ if ((level > 0) & (parentFilters.length == 0)) {
+ return false
+ }
+
+ // Apply each filter and check sample validity. Return false when invalid condition found.
+ for (filter <- parentFilters) {
+ val features = labeledPoint.features
+ val featureIndex = filter.split.feature
+ val threshold = filter.split.threshold
+ val comparison = filter.comparison
+ val categories = filter.split.categories
+ val isFeatureContinuous = filter.split.featureType == Continuous
+ val feature = features(featureIndex)
+ if (isFeatureContinuous) {
+ comparison match {
+ case -1 => if (feature > threshold) return false
+ case 1 => if (feature <= threshold) return false
+ }
+ } else {
+ val containsFeature = categories.contains(feature)
+ comparison match {
+ case -1 => if (!containsFeature) return false
+ case 1 => if (containsFeature) return false
+ }
+
+ }
+ }
+
+ // Return true when the sample is valid for all filters.
+ true
+ }
+
+ /**
+ * Find bin for one feature.
+ */
+ def findBin(
+ featureIndex: Int,
+ labeledPoint: LabeledPoint,
+ isFeatureContinuous: Boolean): Int = {
+ val binForFeatures = bins(featureIndex)
+ val feature = labeledPoint.features(featureIndex)
+
+ /**
+ * Binary search helper method for continuous feature.
+ */
+ def binarySearchForBins(): Int = {
+ var left = 0
+ var right = binForFeatures.length - 1
+ while (left <= right) {
+ val mid = left + (right - left) / 2
+ val bin = binForFeatures(mid)
+ val lowThreshold = bin.lowSplit.threshold
+ val highThreshold = bin.highSplit.threshold
+ if ((lowThreshold < feature) & (highThreshold >= feature)){
+ return mid
+ }
+ else if (lowThreshold >= feature) {
+ right = mid - 1
+ }
+ else {
+ left = mid + 1
+ }
+ }
+ -1
+ }
+
+ /**
+ * Sequential search helper method to find bin for categorical feature.
+ */
+ def sequentialBinSearchForCategoricalFeature(): Int = {
+ val numCategoricalBins = strategy.categoricalFeaturesInfo(featureIndex)
+ var binIndex = 0
+ while (binIndex < numCategoricalBins) {
+ val bin = bins(featureIndex)(binIndex)
+ val category = bin.category
+ val features = labeledPoint.features
+ if (category == features(featureIndex)) {
+ return binIndex
+ }
+ binIndex += 1
+ }
+ -1
+ }
+
+ if (isFeatureContinuous) {
+ // Perform binary search for finding bin for continuous features.
+ val binIndex = binarySearchForBins()
+ if (binIndex == -1){
+ throw new UnknownError("no bin was found for continuous variable.")
+ }
+ binIndex
+ } else {
+ // Perform sequential search to find bin for categorical features.
+ val binIndex = sequentialBinSearchForCategoricalFeature()
+ if (binIndex == -1){
+ throw new UnknownError("no bin was found for categorical variable.")
+ }
+ binIndex
+ }
+ }
+
+ /**
+ * Finds bins for all nodes (and all features) at a given level.
+ * For l nodes, k features the storage is as follows:
+ * label, b_11, b_12, .. , b_1k, b_21, b_22, .. , b_2k, b_l1, b_l2, .. , b_lk,
+ * where b_ij is an integer between 0 and numBins - 1.
+ * Invalid sample is denoted by noting bin for feature 1 as -1.
+ */
+ def findBinsForLevel(labeledPoint: LabeledPoint): Array[Double] = {
+ // Calculate bin index and label per feature per node.
+ val arr = new Array[Double](1 + (numFeatures * numNodes))
+ arr(0) = labeledPoint.label
+ var nodeIndex = 0
+ while (nodeIndex < numNodes) {
+ val parentFilters = findParentFilters(nodeIndex)
+ // Find out whether the sample qualifies for the particular node.
+ val sampleValid = isSampleValid(parentFilters, labeledPoint)
+ val shift = 1 + numFeatures * nodeIndex
+ if (!sampleValid) {
+ // Mark one bin as -1 is sufficient.
+ arr(shift) = InvalidBinIndex
+ } else {
+ var featureIndex = 0
+ while (featureIndex < numFeatures) {
+ val isFeatureContinuous = strategy.categoricalFeaturesInfo.get(featureIndex).isEmpty
+ arr(shift + featureIndex) = findBin(featureIndex, labeledPoint,isFeatureContinuous)
+ featureIndex += 1
+ }
+ }
+ nodeIndex += 1
+ }
+ arr
+ }
+
+ /**
+ * Performs a sequential aggregation over a partition for classification. For l nodes,
+ * k features, either the left count or the right count of one of the p bins is
+ * incremented based upon whether the feature is classified as 0 or 1.
+ *
+ * @param agg Array[Double] storing aggregate calculation of size
+ * 2 * numSplits * numFeatures*numNodes for classification
+ * @param arr Array[Double] of size 1 + (numFeatures * numNodes)
+ * @return Array[Double] storing aggregate calculation of size
+ * 2 * numSplits * numFeatures * numNodes for classification
+ */
+ def classificationBinSeqOp(arr: Array[Double], agg: Array[Double]) {
+ // Iterate over all nodes.
+ var nodeIndex = 0
+ while (nodeIndex < numNodes) {
+ // Check whether the instance was valid for this nodeIndex.
+ val validSignalIndex = 1 + numFeatures * nodeIndex
+ val isSampleValidForNode = arr(validSignalIndex) != InvalidBinIndex
+ if (isSampleValidForNode) {
+ // actual class label
+ val label = arr(0)
+ // Iterate over all features.
+ var featureIndex = 0
+ while (featureIndex < numFeatures) {
+ // Find the bin index for this feature.
+ val arrShift = 1 + numFeatures * nodeIndex
+ val arrIndex = arrShift + featureIndex
+ // Update the left or right count for one bin.
+ val aggShift = 2 * numBins * numFeatures * nodeIndex
+ val aggIndex = aggShift + 2 * featureIndex * numBins + arr(arrIndex).toInt * 2
+ label match {
+ case 0.0 => agg(aggIndex) = agg(aggIndex) + 1
+ case 1.0 => agg(aggIndex + 1) = agg(aggIndex + 1) + 1
+ }
+ featureIndex += 1
+ }
+ }
+ nodeIndex += 1
+ }
+ }
+
+ /**
+ * Performs a sequential aggregation over a partition for regression. For l nodes, k features,
+ * the count, sum, sum of squares of one of the p bins is incremented.
+ *
+ * @param agg Array[Double] storing aggregate calculation of size
+ * 3 * numSplits * numFeatures * numNodes for classification
+ * @param arr Array[Double] of size 1 + (numFeatures * numNodes)
+ * @return Array[Double] storing aggregate calculation of size
+ * 3 * numSplits * numFeatures * numNodes for regression
+ */
+ def regressionBinSeqOp(arr: Array[Double], agg: Array[Double]) {
+ // Iterate over all nodes.
+ var nodeIndex = 0
+ while (nodeIndex < numNodes) {
+ // Check whether the instance was valid for this nodeIndex.
+ val validSignalIndex = 1 + numFeatures * nodeIndex
+ val isSampleValidForNode = arr(validSignalIndex) != InvalidBinIndex
+ if (isSampleValidForNode) {
+ // actual class label
+ val label = arr(0)
+ // Iterate over all features.
+ var featureIndex = 0
+ while (featureIndex < numFeatures) {
+ // Find the bin index for this feature.
+ val arrShift = 1 + numFeatures * nodeIndex
+ val arrIndex = arrShift + featureIndex
+ // Update count, sum, and sum^2 for one bin.
+ val aggShift = 3 * numBins * numFeatures * nodeIndex
+ val aggIndex = aggShift + 3 * featureIndex * numBins + arr(arrIndex).toInt * 3
+ agg(aggIndex) = agg(aggIndex) + 1
+ agg(aggIndex + 1) = agg(aggIndex + 1) + label
+ agg(aggIndex + 2) = agg(aggIndex + 2) + label*label
+ featureIndex += 1
+ }
+ }
+ nodeIndex += 1
+ }
+ }
+
+ /**
+ * Performs a sequential aggregation over a partition.
+ */
+ def binSeqOp(agg: Array[Double], arr: Array[Double]): Array[Double] = {
+ strategy.algo match {
+ case Classification => classificationBinSeqOp(arr, agg)
+ case Regression => regressionBinSeqOp(arr, agg)
+ }
+ agg
+ }
+
+ // Calculate bin aggregate length for classification or regression.
+ val binAggregateLength = strategy.algo match {
+ case Classification => 2 * numBins * numFeatures * numNodes
+ case Regression => 3 * numBins * numFeatures * numNodes
+ }
+ logDebug("binAggregateLength = " + binAggregateLength)
+
+ /**
+ * Combines the aggregates from partitions.
+ * @param agg1 Array containing aggregates from one or more partitions
+ * @param agg2 Array containing aggregates from one or more partitions
+ * @return Combined aggregate from agg1 and agg2
+ */
+ def binCombOp(agg1: Array[Double], agg2: Array[Double]): Array[Double] = {
+ var index = 0
+ val combinedAggregate = new Array[Double](binAggregateLength)
+ while (index < binAggregateLength) {
+ combinedAggregate(index) = agg1(index) + agg2(index)
+ index += 1
+ }
+ combinedAggregate
+ }
+
+ // Find feature bins for all nodes at a level.
+ val binMappedRDD = input.map(x => findBinsForLevel(x))
+
+ // Calculate bin aggregates.
+ val binAggregates = {
+ binMappedRDD.aggregate(Array.fill[Double](binAggregateLength)(0))(binSeqOp,binCombOp)
+ }
+ logDebug("binAggregates.length = " + binAggregates.length)
+
+ /**
+ * Calculates the information gain for all splits based upon left/right split aggregates.
+ * @param leftNodeAgg left node aggregates
+ * @param featureIndex feature index
+ * @param splitIndex split index
+ * @param rightNodeAgg right node aggregate
+ * @param topImpurity impurity of the parent node
+ * @return information gain and statistics for all splits
+ */
+ def calculateGainForSplit(
+ leftNodeAgg: Array[Array[Double]],
+ featureIndex: Int,
+ splitIndex: Int,
+ rightNodeAgg: Array[Array[Double]],
+ topImpurity: Double): InformationGainStats = {
+ strategy.algo match {
+ case Classification =>
+ val left0Count = leftNodeAgg(featureIndex)(2 * splitIndex)
+ val left1Count = leftNodeAgg(featureIndex)(2 * splitIndex + 1)
+ val leftCount = left0Count + left1Count
+
+ val right0Count = rightNodeAgg(featureIndex)(2 * splitIndex)
+ val right1Count = rightNodeAgg(featureIndex)(2 * splitIndex + 1)
+ val rightCount = right0Count + right1Count
+
+ val impurity = {
+ if (level > 0) {
+ topImpurity
+ } else {
+ // Calculate impurity for root node.
+ strategy.impurity.calculate(left0Count + right0Count, left1Count + right1Count)
+ }
+ }
+
+ if (leftCount == 0) {
+ return new InformationGainStats(0, topImpurity, Double.MinValue, topImpurity,1)
+ }
+ if (rightCount == 0) {
+ return new InformationGainStats(0, topImpurity, topImpurity, Double.MinValue,0)
+ }
+
+ val leftImpurity = strategy.impurity.calculate(left0Count, left1Count)
+ val rightImpurity = strategy.impurity.calculate(right0Count, right1Count)
+
+ val leftWeight = leftCount.toDouble / (leftCount + rightCount)
+ val rightWeight = rightCount.toDouble / (leftCount + rightCount)
+
+ val gain = {
+ if (level > 0) {
+ impurity - leftWeight * leftImpurity - rightWeight * rightImpurity
+ } else {
+ impurity - leftWeight * leftImpurity - rightWeight * rightImpurity
+ }
+ }
+
+ val predict = (left1Count + right1Count) / (leftCount + rightCount)
+
+ new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, predict)
+ case Regression =>
+ val leftCount = leftNodeAgg(featureIndex)(3 * splitIndex)
+ val leftSum = leftNodeAgg(featureIndex)(3 * splitIndex + 1)
+ val leftSumSquares = leftNodeAgg(featureIndex)(3 * splitIndex + 2)
+
+ val rightCount = rightNodeAgg(featureIndex)(3 * splitIndex)
+ val rightSum = rightNodeAgg(featureIndex)(3 * splitIndex + 1)
+ val rightSumSquares = rightNodeAgg(featureIndex)(3 * splitIndex + 2)
+
+ val impurity = {
+ if (level > 0) {
+ topImpurity
+ } else {
+ // Calculate impurity for root node.
+ val count = leftCount + rightCount
+ val sum = leftSum + rightSum
+ val sumSquares = leftSumSquares + rightSumSquares
+ strategy.impurity.calculate(count, sum, sumSquares)
+ }
+ }
+
+ if (leftCount == 0) {
+ return new InformationGainStats(0, topImpurity, Double.MinValue, topImpurity,
+ rightSum / rightCount)
+ }
+ if (rightCount == 0) {
+ return new InformationGainStats(0, topImpurity ,topImpurity,
+ Double.MinValue, leftSum / leftCount)
+ }
+
+ val leftImpurity = strategy.impurity.calculate(leftCount, leftSum, leftSumSquares)
+ val rightImpurity = strategy.impurity.calculate(rightCount, rightSum, rightSumSquares)
+
+ val leftWeight = leftCount.toDouble / (leftCount + rightCount)
+ val rightWeight = rightCount.toDouble / (leftCount + rightCount)
+
+ val gain = {
+ if (level > 0) {
+ impurity - leftWeight * leftImpurity - rightWeight * rightImpurity
+ } else {
+ impurity - leftWeight * leftImpurity - rightWeight * rightImpurity
+ }
+ }
+
+ val predict = (leftSum + rightSum) / (leftCount + rightCount)
+ new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, predict)
+ }
+ }
+
+ /**
+ * Extracts left and right split aggregates.
+ * @param binData Array[Double] of size 2*numFeatures*numSplits
+ * @return (leftNodeAgg, rightNodeAgg) tuple of type (Array[Double],
+ * Array[Double]) where each array is of size(numFeature,2*(numSplits-1))
+ */
+ def extractLeftRightNodeAggregates(
+ binData: Array[Double]): (Array[Array[Double]], Array[Array[Double]]) = {
+ strategy.algo match {
+ case Classification =>
+ // Initialize left and right split aggregates.
+ val leftNodeAgg = Array.ofDim[Double](numFeatures, 2 * (numBins - 1))
+ val rightNodeAgg = Array.ofDim[Double](numFeatures, 2 * (numBins - 1))
+ // Iterate over all features.
+ var featureIndex = 0
+ while (featureIndex < numFeatures) {
+ // shift for this featureIndex
+ val shift = 2 * featureIndex * numBins
+
+ // left node aggregate for the lowest split
+ leftNodeAgg(featureIndex)(0) = binData(shift + 0)
+ leftNodeAgg(featureIndex)(1) = binData(shift + 1)
+
+ // right node aggregate for the highest split
+ rightNodeAgg(featureIndex)(2 * (numBins - 2))
+ = binData(shift + (2 * (numBins - 1)))
+ rightNodeAgg(featureIndex)(2 * (numBins - 2) + 1)
+ = binData(shift + (2 * (numBins - 1)) + 1)
+
+ // Iterate over all splits.
+ var splitIndex = 1
+ while (splitIndex < numBins - 1) {
+ // calculating left node aggregate for a split as a sum of left node aggregate of a
+ // lower split and the left bin aggregate of a bin where the split is a high split
+ leftNodeAgg(featureIndex)(2 * splitIndex) = binData(shift + 2 * splitIndex) +
+ leftNodeAgg(featureIndex)(2 * splitIndex - 2)
+ leftNodeAgg(featureIndex)(2 * splitIndex + 1) = binData(shift + 2 * splitIndex + 1) +
+ leftNodeAgg(featureIndex)(2 * splitIndex - 2 + 1)
+
+ // calculating right node aggregate for a split as a sum of right node aggregate of a
+ // higher split and the right bin aggregate of a bin where the split is a low split
+ rightNodeAgg(featureIndex)(2 * (numBins - 2 - splitIndex)) =
+ binData(shift + (2 *(numBins - 2 - splitIndex))) +
+ rightNodeAgg(featureIndex)(2 * (numBins - 1 - splitIndex))
+ rightNodeAgg(featureIndex)(2 * (numBins - 2 - splitIndex) + 1) =
+ binData(shift + (2* (numBins - 2 - splitIndex) + 1)) +
+ rightNodeAgg(featureIndex)(2 * (numBins - 1 - splitIndex) + 1)
+
+ splitIndex += 1
+ }
+ featureIndex += 1
+ }
+ (leftNodeAgg, rightNodeAgg)
+ case Regression =>
+ // Initialize left and right split aggregates.
+ val leftNodeAgg = Array.ofDim[Double](numFeatures, 3 * (numBins - 1))
+ val rightNodeAgg = Array.ofDim[Double](numFeatures, 3 * (numBins - 1))
+ // Iterate over all features.
+ var featureIndex = 0
+ while (featureIndex < numFeatures) {
+ // shift for this featureIndex
+ val shift = 3 * featureIndex * numBins
+ // left node aggregate for the lowest split
+ leftNodeAgg(featureIndex)(0) = binData(shift + 0)
+ leftNodeAgg(featureIndex)(1) = binData(shift + 1)
+ leftNodeAgg(featureIndex)(2) = binData(shift + 2)
+
+ // right node aggregate for the highest split
+ rightNodeAgg(featureIndex)(3 * (numBins - 2)) =
+ binData(shift + (3 * (numBins - 1)))
+ rightNodeAgg(featureIndex)(3 * (numBins - 2) + 1) =
+ binData(shift + (3 * (numBins - 1)) + 1)
+ rightNodeAgg(featureIndex)(3 * (numBins - 2) + 2) =
+ binData(shift + (3 * (numBins - 1)) + 2)
+
+ // Iterate over all splits.
+ var splitIndex = 1
+ while (splitIndex < numBins - 1) {
+ // calculating left node aggregate for a split as a sum of left node aggregate of a
+ // lower split and the left bin aggregate of a bin where the split is a high split
+ leftNodeAgg(featureIndex)(3 * splitIndex) = binData(shift + 3 * splitIndex) +
+ leftNodeAgg(featureIndex)(3 * splitIndex - 3)
+ leftNodeAgg(featureIndex)(3 * splitIndex + 1) = binData(shift + 3 * splitIndex + 1) +
+ leftNodeAgg(featureIndex)(3 * splitIndex - 3 + 1)
+ leftNodeAgg(featureIndex)(3 * splitIndex + 2) = binData(shift + 3 * splitIndex + 2) +
+ leftNodeAgg(featureIndex)(3 * splitIndex - 3 + 2)
+
+ // calculating right node aggregate for a split as a sum of right node aggregate of a
+ // higher split and the right bin aggregate of a bin where the split is a low split
+ rightNodeAgg(featureIndex)(3 * (numBins - 2 - splitIndex)) =
+ binData(shift + (3 * (numBins - 2 - splitIndex))) +
+ rightNodeAgg(featureIndex)(3 * (numBins - 1 - splitIndex))
+ rightNodeAgg(featureIndex)(3 * (numBins - 2 - splitIndex) + 1) =
+ binData(shift + (3 * (numBins - 2 - splitIndex) + 1)) +
+ rightNodeAgg(featureIndex)(3 * (numBins - 1 - splitIndex) + 1)
+ rightNodeAgg(featureIndex)(3 * (numBins - 2 - splitIndex) + 2) =
+ binData(shift + (3 * (numBins - 2 - splitIndex) + 2)) +
+ rightNodeAgg(featureIndex)(3 * (numBins - 1 - splitIndex) + 2)
+
+ splitIndex += 1
+ }
+ featureIndex += 1
+ }
+ (leftNodeAgg, rightNodeAgg)
+ }
+ }
+
+ /**
+ * Calculates information gain for all nodes splits.
+ */
+ def calculateGainsForAllNodeSplits(
+ leftNodeAgg: Array[Array[Double]],
+ rightNodeAgg: Array[Array[Double]],
+ nodeImpurity: Double): Array[Array[InformationGainStats]] = {
+ val gains = Array.ofDim[InformationGainStats](numFeatures, numBins - 1)
+
+ for (featureIndex <- 0 until numFeatures) {
+ for (splitIndex <- 0 until numBins - 1) {
+ gains(featureIndex)(splitIndex) = calculateGainForSplit(leftNodeAgg, featureIndex,
+ splitIndex, rightNodeAgg, nodeImpurity)
+ }
+ }
+ gains
+ }
+
+ /**
+ * Find the best split for a node.
+ * @param binData Array[Double] of size 2 * numSplits * numFeatures
+ * @param nodeImpurity impurity of the top node
+ * @return tuple of split and information gain
+ */
+ def binsToBestSplit(
+ binData: Array[Double],
+ nodeImpurity: Double): (Split, InformationGainStats) = {
+
+ logDebug("node impurity = " + nodeImpurity)
+
+ // Extract left right node aggregates.
+ val (leftNodeAgg, rightNodeAgg) = extractLeftRightNodeAggregates(binData)
+
+ // Calculate gains for all splits.
+ val gains = calculateGainsForAllNodeSplits(leftNodeAgg, rightNodeAgg, nodeImpurity)
+
+ val (bestFeatureIndex,bestSplitIndex, gainStats) = {
+ // Initialize with infeasible values.
+ var bestFeatureIndex = Int.MinValue
+ var bestSplitIndex = Int.MinValue
+ var bestGainStats = new InformationGainStats(Double.MinValue, -1.0, -1.0, -1.0, -1.0)
+ // Iterate over features.
+ var featureIndex = 0
+ while (featureIndex < numFeatures) {
+ // Iterate over all splits.
+ var splitIndex = 0
+ while (splitIndex < numBins - 1) {
+ val gainStats = gains(featureIndex)(splitIndex)
+ if (gainStats.gain > bestGainStats.gain) {
+ bestGainStats = gainStats
+ bestFeatureIndex = featureIndex
+ bestSplitIndex = splitIndex
+ }
+ splitIndex += 1
+ }
+ featureIndex += 1
+ }
+ (bestFeatureIndex, bestSplitIndex, bestGainStats)
+ }
+
+ logDebug("best split bin = " + bins(bestFeatureIndex)(bestSplitIndex))
+ logDebug("best split bin = " + splits(bestFeatureIndex)(bestSplitIndex))
+
+ (splits(bestFeatureIndex)(bestSplitIndex), gainStats)
+ }
+
+ /**
+ * Get bin data for one node.
+ */
+ def getBinDataForNode(node: Int): Array[Double] = {
+ strategy.algo match {
+ case Classification =>
+ val shift = 2 * node * numBins * numFeatures
+ val binsForNode = binAggregates.slice(shift, shift + 2 * numBins * numFeatures)
+ binsForNode
+ case Regression =>
+ val shift = 3 * node * numBins * numFeatures
+ val binsForNode = binAggregates.slice(shift, shift + 3 * numBins * numFeatures)
+ binsForNode
+ }
+ }
+
+ // Calculate best splits for all nodes at a given level
+ val bestSplits = new Array[(Split, InformationGainStats)](numNodes)
+ // Iterating over all nodes at this level
+ var node = 0
+ while (node < numNodes) {
+ val nodeImpurityIndex = scala.math.pow(2, level).toInt - 1 + node
+ val binsForNode: Array[Double] = getBinDataForNode(node)
+ logDebug("nodeImpurityIndex = " + nodeImpurityIndex)
+ val parentNodeImpurity = parentImpurities(nodeImpurityIndex)
+ logDebug("node impurity = " + parentNodeImpurity)
+ bestSplits(node) = binsToBestSplit(binsForNode, parentNodeImpurity)
+ node += 1
+ }
+
+ bestSplits
+ }
+
+ /**
+ * Returns split and bins for decision tree calculation.
+ * @param input RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] used as training data
+ * for DecisionTree
+ * @param strategy [[org.apache.spark.mllib.tree.configuration.Strategy]] instance containing
+ * parameters for construction the DecisionTree
+ * @return a tuple of (splits,bins) where splits is an Array of [org.apache.spark.mllib.tree
+ * .model.Split] of size (numFeatures, numSplits-1) and bins is an Array of [org.apache
+ * .spark.mllib.tree.model.Bin] of size (numFeatures, numSplits1)
+ */
+ protected[tree] def findSplitsBins(
+ input: RDD[LabeledPoint],
+ strategy: Strategy): (Array[Array[Split]], Array[Array[Bin]]) = {
+ val count = input.count()
+
+ // Find the number of features by looking at the first sample
+ val numFeatures = input.take(1)(0).features.size
+
+ val maxBins = strategy.maxBins
+ val numBins = if (maxBins <= count) maxBins else count.toInt
+ logDebug("numBins = " + numBins)
+
+ /*
+ * TODO: Add a require statement ensuring #bins is always greater than the categories.
+ * It's a limitation of the current implementation but a reasonable trade-off since features
+ * with large number of categories get favored over continuous features.
+ */
+ if (strategy.categoricalFeaturesInfo.size > 0) {
+ val maxCategoriesForFeatures = strategy.categoricalFeaturesInfo.maxBy(_._2)._2
+ require(numBins >= maxCategoriesForFeatures)
+ }
+
+ // Calculate the number of sample for approximate quantile calculation.
+ val requiredSamples = numBins*numBins
+ val fraction = if (requiredSamples < count) requiredSamples.toDouble / count else 1.0
+ logDebug("fraction of data used for calculating quantiles = " + fraction)
+
+ // sampled input for RDD calculation
+ val sampledInput = input.sample(false, fraction, new XORShiftRandom().nextInt()).collect()
+ val numSamples = sampledInput.length
+
+ val stride: Double = numSamples.toDouble / numBins
+ logDebug("stride = " + stride)
+
+ strategy.quantileCalculationStrategy match {
+ case Sort =>
+ val splits = Array.ofDim[Split](numFeatures, numBins - 1)
+ val bins = Array.ofDim[Bin](numFeatures, numBins)
+
+ // Find all splits.
+
+ // Iterate over all features.
+ var featureIndex = 0
+ while (featureIndex < numFeatures){
+ // Check whether the feature is continuous.
+ val isFeatureContinuous = strategy.categoricalFeaturesInfo.get(featureIndex).isEmpty
+ if (isFeatureContinuous) {
+ val featureSamples = sampledInput.map(lp => lp.features(featureIndex)).sorted
+ val stride: Double = numSamples.toDouble / numBins
+ logDebug("stride = " + stride)
+ for (index <- 0 until numBins - 1) {
+ val sampleIndex = (index + 1) * stride.toInt
+ val split = new Split(featureIndex, featureSamples(sampleIndex), Continuous, List())
+ splits(featureIndex)(index) = split
+ }
+ } else {
+ val maxFeatureValue = strategy.categoricalFeaturesInfo(featureIndex)
+ require(maxFeatureValue < numBins, "number of categories should be less than number " +
+ "of bins")
+
+ // For categorical variables, each bin is a category. The bins are sorted and they
+ // are ordered by calculating the centroid of their corresponding labels.
+ val centroidForCategories =
+ sampledInput.map(lp => (lp.features(featureIndex),lp.label))
+ .groupBy(_._1)
+ .mapValues(x => x.map(_._2).sum / x.map(_._1).length)
+
+ // Check for missing categorical variables and putting them last in the sorted list.
+ val fullCentroidForCategories = scala.collection.mutable.Map[Double,Double]()
+ for (i <- 0 until maxFeatureValue) {
+ if (centroidForCategories.contains(i)) {
+ fullCentroidForCategories(i) = centroidForCategories(i)
+ } else {
+ fullCentroidForCategories(i) = Double.MaxValue
+ }
+ }
+
+ // bins sorted by centroids
+ val categoriesSortedByCentroid = fullCentroidForCategories.toList.sortBy(_._2)
+
+ logDebug("centriod for categorical variable = " + categoriesSortedByCentroid)
+
+ var categoriesForSplit = List[Double]()
+ categoriesSortedByCentroid.iterator.zipWithIndex.foreach {
+ case ((key, value), index) =>
+ categoriesForSplit = key :: categoriesForSplit
+ splits(featureIndex)(index) = new Split(featureIndex, Double.MinValue, Categorical,
+ categoriesForSplit)
+ bins(featureIndex)(index) = {
+ if (index == 0) {
+ new Bin(new DummyCategoricalSplit(featureIndex, Categorical),
+ splits(featureIndex)(0), Categorical, key)
+ } else {
+ new Bin(splits(featureIndex)(index-1), splits(featureIndex)(index),
+ Categorical, key)
+ }
+ }
+ }
+ }
+ featureIndex += 1
+ }
+
+ // Find all bins.
+ featureIndex = 0
+ while (featureIndex < numFeatures) {
+ val isFeatureContinuous = strategy.categoricalFeaturesInfo.get(featureIndex).isEmpty
+ if (isFeatureContinuous) { // Bins for categorical variables are already assigned.
+ bins(featureIndex)(0) = new Bin(new DummyLowSplit(featureIndex, Continuous),
+ splits(featureIndex)(0), Continuous, Double.MinValue)
+ for (index <- 1 until numBins - 1){
+ val bin = new Bin(splits(featureIndex)(index-1), splits(featureIndex)(index),
+ Continuous, Double.MinValue)
+ bins(featureIndex)(index) = bin
+ }
+ bins(featureIndex)(numBins-1) = new Bin(splits(featureIndex)(numBins-2),
+ new DummyHighSplit(featureIndex, Continuous), Continuous, Double.MinValue)
+ }
+ featureIndex += 1
+ }
+ (splits,bins)
+ case MinMax =>
+ throw new UnsupportedOperationException("minmax not supported yet.")
+ case ApproxHist =>
+ throw new UnsupportedOperationException("approximate histogram not supported yet.")
+ }
+ }
+
+ val usage = """
+ Usage: DecisionTreeRunner [slices] --algo --trainDataDir path --testDataDir path --maxDepth num [--impurity ] [--maxBins num]
+ """
+
+ def main(args: Array[String]) {
+
+ if (args.length < 2) {
+ System.err.println(usage)
+ System.exit(1)
+ }
+
+ val sc = new SparkContext(args(0), "DecisionTree")
+
+ val argList = args.toList.drop(1)
+ type OptionMap = Map[Symbol, Any]
+
+ def nextOption(map : OptionMap, list: List[String]): OptionMap = {
+ list match {
+ case Nil => map
+ case "--algo" :: string :: tail => nextOption(map ++ Map('algo -> string), tail)
+ case "--impurity" :: string :: tail => nextOption(map ++ Map('impurity -> string), tail)
+ case "--maxDepth" :: string :: tail => nextOption(map ++ Map('maxDepth -> string), tail)
+ case "--maxBins" :: string :: tail => nextOption(map ++ Map('maxBins -> string), tail)
+ case "--trainDataDir" :: string :: tail => nextOption(map ++ Map('trainDataDir -> string)
+ , tail)
+ case "--testDataDir" :: string :: tail => nextOption(map ++ Map('testDataDir -> string),
+ tail)
+ case string :: Nil => nextOption(map ++ Map('infile -> string), list.tail)
+ case option :: tail => logError("Unknown option " + option)
+ sys.exit(1)
+ }
+ }
+ val options = nextOption(Map(), argList)
+ logDebug(options.toString())
+
+ // Load training data.
+ val trainData = loadLabeledData(sc, options.get('trainDataDir).get.toString)
+
+ // Identify the type of algorithm.
+ val algoStr = options.get('algo).get.toString
+ val algo = algoStr match {
+ case "Classification" => Classification
+ case "Regression" => Regression
+ }
+
+ // Identify the type of impurity.
+ val impurityStr = options.getOrElse('impurity,
+ if (algo == Classification) "Gini" else "Variance").toString
+ val impurity = impurityStr match {
+ case "Gini" => Gini
+ case "Entropy" => Entropy
+ case "Variance" => Variance
+ }
+
+ val maxDepth = options.getOrElse('maxDepth, "1").toString.toInt
+ val maxBins = options.getOrElse('maxBins, "100").toString.toInt
+
+ val strategy = new Strategy(algo, impurity, maxDepth, maxBins)
+ val model = DecisionTree.train(trainData, strategy)
+
+ // Load test data.
+ val testData = loadLabeledData(sc, options.get('testDataDir).get.toString)
+
+ // Measure algorithm accuracy
+ if (algo == Classification) {
+ val accuracy = accuracyScore(model, testData)
+ logDebug("accuracy = " + accuracy)
+ }
+
+ if (algo == Regression) {
+ val mse = meanSquaredError(model, testData)
+ logDebug("mean square error = " + mse)
+ }
+
+ sc.stop()
+ }
+
+ /**
+ * Load labeled data from a file. The data format used here is
+ * , ...,
+ * where , are feature values in Double and is the corresponding label as Double.
+ *
+ * @param sc SparkContext
+ * @param dir Directory to the input data files.
+ * @return An RDD of LabeledPoint. Each labeled point has two elements: the first element is
+ * the label, and the second element represents the feature values (an array of Double).
+ */
+ def loadLabeledData(sc: SparkContext, dir: String): RDD[LabeledPoint] = {
+ sc.textFile(dir).map { line =>
+ val parts = line.trim().split(",")
+ val label = parts(0).toDouble
+ val features = Vectors.dense(parts.slice(1,parts.length).map(_.toDouble))
+ LabeledPoint(label, features)
+ }
+ }
+
+ // TODO: Port this method to a generic metrics package.
+ /**
+ * Calculates the classifier accuracy.
+ */
+ private def accuracyScore(model: DecisionTreeModel, data: RDD[LabeledPoint],
+ threshold: Double = 0.5): Double = {
+ def predictedValue(features: Vector) = {
+ if (model.predict(features) < threshold) 0.0 else 1.0
+ }
+ val correctCount = data.filter(y => predictedValue(y.features) == y.label).count()
+ val count = data.count()
+ logDebug("correct prediction count = " + correctCount)
+ logDebug("data count = " + count)
+ correctCount.toDouble / count
+ }
+
+ // TODO: Port this method to a generic metrics package
+ /**
+ * Calculates the mean squared error for regression.
+ */
+ private def meanSquaredError(tree: DecisionTreeModel, data: RDD[LabeledPoint]): Double = {
+ data.map { y =>
+ val err = tree.predict(y.features) - y.label
+ err * err
+ }.mean()
+ }
+}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/README.md b/mllib/src/main/scala/org/apache/spark/mllib/tree/README.md
new file mode 100644
index 0000000000000..0fd71aa9735bc
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/README.md
@@ -0,0 +1,17 @@
+This package contains the default implementation of the decision tree algorithm.
+
+The decision tree algorithm supports:
++ Binary classification
++ Regression
++ Information loss calculation with entropy and gini for classification and variance for regression
++ Both continuous and categorical features
+
+# Tree improvements
++ Node model pruning
++ Printing to dot files
+
+# Future Ensemble Extensions
+
++ Random forests
++ Boosting
++ Extremely randomized trees
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala
new file mode 100644
index 0000000000000..2dd1f0f27b8f5
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.tree.configuration
+
+/**
+ * Enum to select the algorithm for the decision tree
+ */
+object Algo extends Enumeration {
+ type Algo = Value
+ val Classification, Regression = Value
+}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala
new file mode 100644
index 0000000000000..09ee0586c58fa
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.tree.configuration
+
+/**
+ * Enum to describe whether a feature is "continuous" or "categorical"
+ */
+object FeatureType extends Enumeration {
+ type FeatureType = Value
+ val Continuous, Categorical = Value
+}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala
new file mode 100644
index 0000000000000..2457a480c2a14
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.tree.configuration
+
+/**
+ * Enum for selecting the quantile calculation strategy
+ */
+object QuantileStrategy extends Enumeration {
+ type QuantileStrategy = Value
+ val Sort, MinMax, ApproxHist = Value
+}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala
new file mode 100644
index 0000000000000..df565f3eb8859
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.tree.configuration
+
+import org.apache.spark.mllib.tree.impurity.Impurity
+import org.apache.spark.mllib.tree.configuration.Algo._
+import org.apache.spark.mllib.tree.configuration.QuantileStrategy._
+
+/**
+ * Stores all the configuration options for tree construction
+ * @param algo classification or regression
+ * @param impurity criterion used for information gain calculation
+ * @param maxDepth maximum depth of the tree
+ * @param maxBins maximum number of bins used for splitting features
+ * @param quantileCalculationStrategy algorithm for calculating quantiles
+ * @param categoricalFeaturesInfo A map storing information about the categorical variables and the
+ * number of discrete values they take. For example, an entry (n ->
+ * k) implies the feature n is categorical with k categories 0,
+ * 1, 2, ... , k-1. It's important to note that features are
+ * zero-indexed.
+ */
+class Strategy (
+ val algo: Algo,
+ val impurity: Impurity,
+ val maxDepth: Int,
+ val maxBins: Int = 100,
+ val quantileCalculationStrategy: QuantileStrategy = Sort,
+ val categoricalFeaturesInfo: Map[Int,Int] = Map[Int,Int]()) extends Serializable
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala
new file mode 100644
index 0000000000000..b93995fcf9441
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.tree.impurity
+
+/**
+ * Class for calculating [[http://en.wikipedia.org/wiki/Binary_entropy_function entropy]] during
+ * binary classification.
+ */
+object Entropy extends Impurity {
+
+ def log2(x: Double) = scala.math.log(x) / scala.math.log(2)
+
+ /**
+ * entropy calculation
+ * @param c0 count of instances with label 0
+ * @param c1 count of instances with label 1
+ * @return entropy value
+ */
+ def calculate(c0: Double, c1: Double): Double = {
+ if (c0 == 0 || c1 == 0) {
+ 0
+ } else {
+ val total = c0 + c1
+ val f0 = c0 / total
+ val f1 = c1 / total
+ -(f0 * log2(f0)) - (f1 * log2(f1))
+ }
+ }
+
+ def calculate(count: Double, sum: Double, sumSquares: Double): Double =
+ throw new UnsupportedOperationException("Entropy.calculate")
+}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala
new file mode 100644
index 0000000000000..c0407554a91b3
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.tree.impurity
+
+/**
+ * Class for calculating the
+ * [[http://en.wikipedia.org/wiki/Decision_tree_learning#Gini_impurity Gini impurity]]
+ * during binary classification.
+ */
+object Gini extends Impurity {
+
+ /**
+ * Gini coefficient calculation
+ * @param c0 count of instances with label 0
+ * @param c1 count of instances with label 1
+ * @return Gini coefficient value
+ */
+ override def calculate(c0: Double, c1: Double): Double = {
+ if (c0 == 0 || c1 == 0) {
+ 0
+ } else {
+ val total = c0 + c1
+ val f0 = c0 / total
+ val f1 = c1 / total
+ 1 - f0 * f0 - f1 * f1
+ }
+ }
+
+ def calculate(count: Double, sum: Double, sumSquares: Double): Double =
+ throw new UnsupportedOperationException("Gini.calculate")
+}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala
new file mode 100644
index 0000000000000..a4069063af2ad
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.tree.impurity
+
+/**
+ * Trait for calculating information gain.
+ */
+trait Impurity extends Serializable {
+
+ /**
+ * information calculation for binary classification
+ * @param c0 count of instances with label 0
+ * @param c1 count of instances with label 1
+ * @return information value
+ */
+ def calculate(c0 : Double, c1 : Double): Double
+
+ /**
+ * information calculation for regression
+ * @param count number of instances
+ * @param sum sum of labels
+ * @param sumSquares summation of squares of the labels
+ * @return information value
+ */
+ def calculate(count: Double, sum: Double, sumSquares: Double): Double
+
+}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala
new file mode 100644
index 0000000000000..b74577dcec167
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.tree.impurity
+
+/**
+ * Class for calculating variance during regression
+ */
+object Variance extends Impurity {
+ override def calculate(c0: Double, c1: Double): Double =
+ throw new UnsupportedOperationException("Variance.calculate")
+
+ /**
+ * variance calculation
+ * @param count number of instances
+ * @param sum sum of labels
+ * @param sumSquares summation of squares of the labels
+ */
+ override def calculate(count: Double, sum: Double, sumSquares: Double): Double = {
+ val squaredLoss = sumSquares - (sum * sum) / count
+ squaredLoss / count
+ }
+}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala
new file mode 100644
index 0000000000000..a57faa13745f7
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.tree.model
+
+import org.apache.spark.mllib.tree.configuration.FeatureType._
+
+/**
+ * Used for "binning" the features bins for faster best split calculation. For a continuous
+ * feature, a bin is determined by a low and a high "split". For a categorical feature,
+ * the a bin is determined using a single label value (category).
+ * @param lowSplit signifying the lower threshold for the continuous feature to be
+ * accepted in the bin
+ * @param highSplit signifying the upper threshold for the continuous feature to be
+ * accepted in the bin
+ * @param featureType type of feature -- categorical or continuous
+ * @param category categorical label value accepted in the bin
+ */
+case class Bin(lowSplit: Split, highSplit: Split, featureType: FeatureType, category: Double)
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala
new file mode 100644
index 0000000000000..a6dca84a2ce09
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala
@@ -0,0 +1,50 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.tree.model
+
+import org.apache.spark.mllib.tree.configuration.Algo._
+import org.apache.spark.rdd.RDD
+import org.apache.spark.mllib.linalg.Vector
+
+/**
+ * Model to store the decision tree parameters
+ * @param topNode root node
+ * @param algo algorithm type -- classification or regression
+ */
+class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable {
+
+ /**
+ * Predict values for a single data point using the model trained.
+ *
+ * @param features array representing a single data point
+ * @return Double prediction from the trained model
+ */
+ def predict(features: Vector): Double = {
+ topNode.predictIfLeaf(features)
+ }
+
+ /**
+ * Predict values for the given data set using the model trained.
+ *
+ * @param features RDD representing data points to be predicted
+ * @return RDD[Int] where each entry contains the corresponding prediction
+ */
+ def predict(features: RDD[Vector]): RDD[Double] = {
+ features.map(x => predict(x))
+ }
+}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Filter.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Filter.scala
new file mode 100644
index 0000000000000..ebc9595eafef3
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Filter.scala
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.tree.model
+
+/**
+ * Filter specifying a split and type of comparison to be applied on features
+ * @param split split specifying the feature index, type and threshold
+ * @param comparison integer specifying <,=,>
+ */
+case class Filter(split: Split, comparison: Int) {
+ // Comparison -1,0,1 signifies <.=,>
+ override def toString = " split = " + split + "comparison = " + comparison
+}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala
new file mode 100644
index 0000000000000..99bf79cf12e45
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.tree.model
+
+/**
+ * Information gain statistics for each split
+ * @param gain information gain value
+ * @param impurity current node impurity
+ * @param leftImpurity left node impurity
+ * @param rightImpurity right node impurity
+ * @param predict predicted value
+ */
+class InformationGainStats(
+ val gain: Double,
+ val impurity: Double,
+ val leftImpurity: Double,
+ val rightImpurity: Double,
+ val predict: Double) extends Serializable {
+
+ override def toString = {
+ "gain = %f, impurity = %f, left impurity = %f, right impurity = %f, predict = %f"
+ .format(gain, impurity, leftImpurity, rightImpurity, predict)
+ }
+}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala
new file mode 100644
index 0000000000000..aac3f9ce308f7
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.tree.model
+
+import org.apache.spark.Logging
+import org.apache.spark.mllib.tree.configuration.FeatureType._
+import org.apache.spark.mllib.linalg.Vector
+
+/**
+ * Node in a decision tree
+ * @param id integer node id
+ * @param predict predicted value at the node
+ * @param isLeaf whether the leaf is a node
+ * @param split split to calculate left and right nodes
+ * @param leftNode left child
+ * @param rightNode right child
+ * @param stats information gain stats
+ */
+class Node (
+ val id: Int,
+ val predict: Double,
+ val isLeaf: Boolean,
+ val split: Option[Split],
+ var leftNode: Option[Node],
+ var rightNode: Option[Node],
+ val stats: Option[InformationGainStats]) extends Serializable with Logging {
+
+ override def toString = "id = " + id + ", isLeaf = " + isLeaf + ", predict = " + predict + ", " +
+ "split = " + split + ", stats = " + stats
+
+ /**
+ * build the left node and right nodes if not leaf
+ * @param nodes array of nodes
+ */
+ def build(nodes: Array[Node]): Unit = {
+
+ logDebug("building node " + id + " at level " +
+ (scala.math.log(id + 1)/scala.math.log(2)).toInt )
+ logDebug("id = " + id + ", split = " + split)
+ logDebug("stats = " + stats)
+ logDebug("predict = " + predict)
+ if (!isLeaf) {
+ val leftNodeIndex = id * 2 + 1
+ val rightNodeIndex = id * 2 + 2
+ leftNode = Some(nodes(leftNodeIndex))
+ rightNode = Some(nodes(rightNodeIndex))
+ leftNode.get.build(nodes)
+ rightNode.get.build(nodes)
+ }
+ }
+
+ /**
+ * predict value if node is not leaf
+ * @param feature feature value
+ * @return predicted value
+ */
+ def predictIfLeaf(feature: Vector) : Double = {
+ if (isLeaf) {
+ predict
+ } else{
+ if (split.get.featureType == Continuous) {
+ if (feature(split.get.feature) <= split.get.threshold) {
+ leftNode.get.predictIfLeaf(feature)
+ } else {
+ rightNode.get.predictIfLeaf(feature)
+ }
+ } else {
+ if (split.get.categories.contains(feature(split.get.feature))) {
+ leftNode.get.predictIfLeaf(feature)
+ } else {
+ rightNode.get.predictIfLeaf(feature)
+ }
+ }
+ }
+ }
+}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala
new file mode 100644
index 0000000000000..4e64a81dda74e
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.tree.model
+
+import org.apache.spark.mllib.tree.configuration.FeatureType.FeatureType
+
+/**
+ * Split applied to a feature
+ * @param feature feature index
+ * @param threshold threshold for continuous feature
+ * @param featureType type of feature -- categorical or continuous
+ * @param categories accepted values for categorical variables
+ */
+case class Split(
+ feature: Int,
+ threshold: Double,
+ featureType: FeatureType,
+ categories: List[Double]){
+
+ override def toString =
+ "Feature = " + feature + ", threshold = " + threshold + ", featureType = " + featureType +
+ ", categories = " + categories
+}
+
+/**
+ * Split with minimum threshold for continuous features. Helps with the smallest bin creation.
+ * @param feature feature index
+ * @param featureType type of feature -- categorical or continuous
+ */
+class DummyLowSplit(feature: Int, featureType: FeatureType)
+ extends Split(feature, Double.MinValue, featureType, List())
+
+/**
+ * Split with maximum threshold for continuous features. Helps with the highest bin creation.
+ * @param feature feature index
+ * @param featureType type of feature -- categorical or continuous
+ */
+class DummyHighSplit(feature: Int, featureType: FeatureType)
+ extends Split(feature, Double.MaxValue, featureType, List())
+
+/**
+ * Split with no acceptable feature values for categorical features. Helps with the first bin
+ * creation.
+ * @param feature feature index
+ * @param featureType type of feature -- categorical or continuous
+ */
+class DummyCategoricalSplit(feature: Int, featureType: FeatureType)
+ extends Split(feature, Double.MaxValue, featureType, List())
+
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala
index 2e03684e62861..81e4eda2a68c4 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala
@@ -24,6 +24,7 @@ import org.jblas.DoubleMatrix
import org.apache.spark.SparkContext
import org.apache.spark.rdd.RDD
+import org.apache.spark.mllib.linalg.Vectors
import org.apache.spark.mllib.regression.LabeledPoint
/**
@@ -74,7 +75,7 @@ object LinearDataGenerator {
val y = x.map { xi =>
new DoubleMatrix(1, xi.length, xi: _*).dot(weightsMat) + intercept + eps * rnd.nextGaussian()
}
- y.zip(x).map(p => LabeledPoint(p._1, p._2))
+ y.zip(x).map(p => LabeledPoint(p._1, Vectors.dense(p._2)))
}
/**
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala
index 52c4a71d621a1..61498dcc2be00 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala
@@ -22,6 +22,7 @@ import scala.util.Random
import org.apache.spark.SparkContext
import org.apache.spark.rdd.RDD
import org.apache.spark.mllib.regression.LabeledPoint
+import org.apache.spark.mllib.linalg.Vectors
/**
* Generate test data for LogisticRegression. This class chooses positive labels
@@ -54,7 +55,7 @@ object LogisticRegressionDataGenerator {
val x = Array.fill[Double](nfeatures) {
rnd.nextGaussian() + (y * eps)
}
- LabeledPoint(y, x)
+ LabeledPoint(y, Vectors.dense(x))
}
data
}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala
index 08cd9ab05547b..cb85e433bfc73 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala
@@ -17,15 +17,13 @@
package org.apache.spark.mllib.util
+import breeze.linalg.{Vector => BV, DenseVector => BDV, SparseVector => BSV,
+ squaredDistance => breezeSquaredDistance}
+
import org.apache.spark.SparkContext
import org.apache.spark.rdd.RDD
-import org.apache.spark.SparkContext._
-
-import org.jblas.DoubleMatrix
-
import org.apache.spark.mllib.regression.LabeledPoint
-
-import breeze.linalg.{Vector => BV, SparseVector => BSV, squaredDistance => breezeSquaredDistance}
+import org.apache.spark.mllib.linalg.{Vector, Vectors}
/**
* Helper methods to load, save and pre-process data used in ML Lib.
@@ -40,6 +38,107 @@ object MLUtils {
eps
}
+ /**
+ * Multiclass label parser, which parses a string into double.
+ */
+ val multiclassLabelParser: String => Double = _.toDouble
+
+ /**
+ * Binary label parser, which outputs 1.0 (positive) if the value is greater than 0.5,
+ * or 0.0 (negative) otherwise.
+ */
+ val binaryLabelParser: String => Double = label => if (label.toDouble > 0.5) 1.0 else 0.0
+
+ /**
+ * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint].
+ * The LIBSVM format is a text-based format used by LIBSVM and LIBLINEAR.
+ * Each line represents a labeled sparse feature vector using the following format:
+ * {{{label index1:value1 index2:value2 ...}}}
+ * where the indices are one-based and in ascending order.
+ * This method parses each line into a [[org.apache.spark.mllib.regression.LabeledPoint]],
+ * where the feature indices are converted to zero-based.
+ *
+ * @param sc Spark context
+ * @param path file or directory path in any Hadoop-supported file system URI
+ * @param labelParser parser for labels, default: 1.0 if label > 0.5 or 0.0 otherwise
+ * @param numFeatures number of features, which will be determined from the input data if a
+ * negative value is given. The default value is -1.
+ * @param minSplits min number of partitions, default: sc.defaultMinSplits
+ * @return labeled data stored as an RDD[LabeledPoint]
+ */
+ def loadLibSVMData(
+ sc: SparkContext,
+ path: String,
+ labelParser: String => Double,
+ numFeatures: Int,
+ minSplits: Int): RDD[LabeledPoint] = {
+ val parsed = sc.textFile(path, minSplits)
+ .map(_.trim)
+ .filter(!_.isEmpty)
+ .map(_.split(' '))
+ // Determine number of features.
+ val d = if (numFeatures >= 0) {
+ numFeatures
+ } else {
+ parsed.map { items =>
+ if (items.length > 1) {
+ items.last.split(':')(0).toInt
+ } else {
+ 0
+ }
+ }.reduce(math.max)
+ }
+ parsed.map { items =>
+ val label = labelParser(items.head)
+ val (indices, values) = items.tail.map { item =>
+ val indexAndValue = item.split(':')
+ val index = indexAndValue(0).toInt - 1
+ val value = indexAndValue(1).toDouble
+ (index, value)
+ }.unzip
+ LabeledPoint(label, Vectors.sparse(d, indices.toArray, values.toArray))
+ }
+ }
+
+ // Convenient methods for calling from Java.
+
+ /**
+ * Loads binary labeled data in the LIBSVM format into an RDD[LabeledPoint],
+ * with number of features determined automatically and the default number of partitions.
+ */
+ def loadLibSVMData(sc: SparkContext, path: String): RDD[LabeledPoint] =
+ loadLibSVMData(sc, path, binaryLabelParser, -1, sc.defaultMinSplits)
+
+ /**
+ * Loads binary labeled data in the LIBSVM format into an RDD[LabeledPoint],
+ * with number of features specified explicitly and the default number of partitions.
+ */
+ def loadLibSVMData(sc: SparkContext, path: String, numFeatures: Int): RDD[LabeledPoint] =
+ loadLibSVMData(sc, path, binaryLabelParser, numFeatures, sc.defaultMinSplits)
+
+ /**
+ * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint],
+ * with the given label parser, number of features determined automatically,
+ * and the default number of partitions.
+ */
+ def loadLibSVMData(
+ sc: SparkContext,
+ path: String,
+ labelParser: String => Double): RDD[LabeledPoint] =
+ loadLibSVMData(sc, path, labelParser, -1, sc.defaultMinSplits)
+
+ /**
+ * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint],
+ * with the given label parser, number of features specified explicitly,
+ * and the default number of partitions.
+ */
+ def loadLibSVMData(
+ sc: SparkContext,
+ path: String,
+ labelParser: String => Double,
+ numFeatures: Int): RDD[LabeledPoint] =
+ loadLibSVMData(sc, path, labelParser, numFeatures, sc.defaultMinSplits)
+
/**
* Load labeled data from a file. The data format used here is
* , ...
@@ -54,7 +153,7 @@ object MLUtils {
sc.textFile(dir).map { line =>
val parts = line.split(',')
val label = parts(0).toDouble
- val features = parts(1).trim().split(' ').map(_.toDouble)
+ val features = Vectors.dense(parts(1).trim().split(' ').map(_.toDouble))
LabeledPoint(label, features)
}
}
@@ -68,7 +167,7 @@ object MLUtils {
* @param dir Directory to save the data.
*/
def saveLabeledData(data: RDD[LabeledPoint], dir: String) {
- val dataStr = data.map(x => x.label + "," + x.features.mkString(" "))
+ val dataStr = data.map(x => x.label + "," + x.features.toArray.mkString(" "))
dataStr.saveAsTextFile(dir)
}
@@ -76,44 +175,52 @@ object MLUtils {
* Utility function to compute mean and standard deviation on a given dataset.
*
* @param data - input data set whose statistics are computed
- * @param nfeatures - number of features
- * @param nexamples - number of examples in input dataset
+ * @param numFeatures - number of features
+ * @param numExamples - number of examples in input dataset
*
* @return (yMean, xColMean, xColSd) - Tuple consisting of
* yMean - mean of the labels
* xColMean - Row vector with mean for every column (or feature) of the input data
* xColSd - Row vector standard deviation for every column (or feature) of the input data.
*/
- def computeStats(data: RDD[LabeledPoint], nfeatures: Int, nexamples: Long):
- (Double, DoubleMatrix, DoubleMatrix) = {
- val yMean: Double = data.map { labeledPoint => labeledPoint.label }.reduce(_ + _) / nexamples
-
- // NOTE: We shuffle X by column here to compute column sum and sum of squares.
- val xColSumSq: RDD[(Int, (Double, Double))] = data.flatMap { labeledPoint =>
- val nCols = labeledPoint.features.length
- // Traverse over every column and emit (col, value, value^2)
- Iterator.tabulate(nCols) { i =>
- (i, (labeledPoint.features(i), labeledPoint.features(i)*labeledPoint.features(i)))
- }
- }.reduceByKey { case(x1, x2) =>
- (x1._1 + x2._1, x1._2 + x2._2)
+ def computeStats(
+ data: RDD[LabeledPoint],
+ numFeatures: Int,
+ numExamples: Long): (Double, Vector, Vector) = {
+ val brzData = data.map { case LabeledPoint(label, features) =>
+ (label, features.toBreeze)
}
- val xColSumsMap = xColSumSq.collectAsMap()
-
- val xColMean = DoubleMatrix.zeros(nfeatures, 1)
- val xColSd = DoubleMatrix.zeros(nfeatures, 1)
-
- // Compute mean and unbiased variance using column sums
- var col = 0
- while (col < nfeatures) {
- xColMean.put(col, xColSumsMap(col)._1 / nexamples)
- val variance =
- (xColSumsMap(col)._2 - (math.pow(xColSumsMap(col)._1, 2) / nexamples)) / nexamples
- xColSd.put(col, math.sqrt(variance))
- col += 1
+ val aggStats = brzData.aggregate(
+ (0L, 0.0, BDV.zeros[Double](numFeatures), BDV.zeros[Double](numFeatures))
+ )(
+ seqOp = (c, v) => (c, v) match {
+ case ((n, sumLabel, sum, sumSq), (label, features)) =>
+ features.activeIterator.foreach { case (i, x) =>
+ sumSq(i) += x * x
+ }
+ (n + 1L, sumLabel + label, sum += features, sumSq)
+ },
+ combOp = (c1, c2) => (c1, c2) match {
+ case ((n1, sumLabel1, sum1, sumSq1), (n2, sumLabel2, sum2, sumSq2)) =>
+ (n1 + n2, sumLabel1 + sumLabel2, sum1 += sum2, sumSq1 += sumSq2)
+ }
+ )
+ val (nl, sumLabel, sum, sumSq) = aggStats
+
+ require(nl > 0, "Input data is empty.")
+ require(nl == numExamples)
+
+ val n = nl.toDouble
+ val yMean = sumLabel / n
+ val mean = sum / n
+ val std = new Array[Double](sum.length)
+ var i = 0
+ while (i < numFeatures) {
+ std(i) = sumSq(i) / n - mean(i) * mean(i)
+ i += 1
}
- (yMean, xColMean, xColSd)
+ (yMean, Vectors.fromBreeze(mean), Vectors.dense(std))
}
/**
@@ -144,6 +251,18 @@ object MLUtils {
val sumSquaredNorm = norm1 * norm1 + norm2 * norm2
val normDiff = norm1 - norm2
var sqDist = 0.0
+ /*
+ * The relative error is
+ *
+ * EPSILON * ( \|a\|_2^2 + \|b\\_2^2 + 2 |a^T b|) / ( \|a - b\|_2^2 ),
+ *
+ * which is bounded by
+ *
+ * 2.0 * EPSILON * ( \|a\|_2^2 + \|b\|_2^2 ) / ( (\|a\|_2 - \|b\|_2)^2 ).
+ *
+ * The bound doesn't need the inner product, so we can use it as a sufficient condition to
+ * check quickly whether the inner product approach is accurate.
+ */
val precisionBound1 = 2.0 * EPSILON * sumSquaredNorm / (normDiff * normDiff + EPSILON)
if (precisionBound1 < precision) {
sqDist = sumSquaredNorm - 2.0 * v1.dot(v2)
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala
index c96c94f70eef7..e300c3dbe1fe0 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala
@@ -23,6 +23,7 @@ import org.jblas.DoubleMatrix
import org.apache.spark.SparkContext
import org.apache.spark.rdd.RDD
+import org.apache.spark.mllib.linalg.Vectors
import org.apache.spark.mllib.regression.LabeledPoint
/**
@@ -58,7 +59,7 @@ object SVMDataGenerator {
}
val yD = new DoubleMatrix(1, x.length, x: _*).dot(trueWeights) + rnd.nextGaussian() * 0.1
val y = if (yD < 0) 0.0 else 1.0
- LabeledPoint(y, x)
+ LabeledPoint(y, Vectors.dense(x))
}
MLUtils.saveLabeledData(data, outputPath)
diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java
index 073ded6f36933..c80b1134ed1b2 100644
--- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java
+++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java
@@ -19,6 +19,7 @@
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.mllib.linalg.Vectors;
import org.apache.spark.mllib.regression.LabeledPoint;
import org.junit.After;
import org.junit.Assert;
@@ -45,12 +46,12 @@ public void tearDown() {
}
private static final List POINTS = Arrays.asList(
- new LabeledPoint(0, new double[] {1.0, 0.0, 0.0}),
- new LabeledPoint(0, new double[] {2.0, 0.0, 0.0}),
- new LabeledPoint(1, new double[] {0.0, 1.0, 0.0}),
- new LabeledPoint(1, new double[] {0.0, 2.0, 0.0}),
- new LabeledPoint(2, new double[] {0.0, 0.0, 1.0}),
- new LabeledPoint(2, new double[] {0.0, 0.0, 2.0})
+ new LabeledPoint(0, Vectors.dense(1.0, 0.0, 0.0)),
+ new LabeledPoint(0, Vectors.dense(2.0, 0.0, 0.0)),
+ new LabeledPoint(1, Vectors.dense(0.0, 1.0, 0.0)),
+ new LabeledPoint(1, Vectors.dense(0.0, 2.0, 0.0)),
+ new LabeledPoint(2, Vectors.dense(0.0, 0.0, 1.0)),
+ new LabeledPoint(2, Vectors.dense(0.0, 0.0, 2.0))
);
private int validatePrediction(List points, NaiveBayesModel model) {
diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java
index 117e5eaa8b78e..4701a5e545020 100644
--- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java
+++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java
@@ -17,7 +17,6 @@
package org.apache.spark.mllib.classification;
-
import java.io.Serializable;
import java.util.List;
@@ -28,7 +27,6 @@
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
-
import org.apache.spark.mllib.regression.LabeledPoint;
public class JavaSVMSuite implements Serializable {
@@ -94,5 +92,4 @@ public void runSVMUsingStaticMethods() {
int numAccurate = validatePrediction(validationData, model);
Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0);
}
-
}
diff --git a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java
index 2c4d795f96e4e..c6d8425ffc38d 100644
--- a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java
+++ b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java
@@ -19,10 +19,10 @@
import java.io.Serializable;
-import com.google.common.collect.Lists;
-
import scala.Tuple2;
+import com.google.common.collect.Lists;
+
import org.junit.Test;
import static org.junit.Assert.*;
@@ -36,7 +36,7 @@ public void denseArrayConstruction() {
@Test
public void sparseArrayConstruction() {
- Vector v = Vectors.sparse(3, Lists.newArrayList(
+ Vector v = Vectors.sparse(3, Lists.>newArrayList(
new Tuple2(0, 2.0),
new Tuple2(2, 3.0)));
assertArrayEquals(new double[]{2.0, 0.0, 3.0}, v.toArray(), 0.0);
diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java
index f44b25cd44d19..f725924a2d971 100644
--- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java
+++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java
@@ -59,7 +59,7 @@ int validatePrediction(List validationData, LassoModel model) {
@Test
public void runLassoUsingConstructor() {
int nPoints = 10000;
- double A = 2.0;
+ double A = 0.0;
double[] weights = {-1.5, 1.0e-2};
JavaRDD testRDD = sc.parallelize(LinearDataGenerator.generateLinearInputAsList(A,
@@ -80,7 +80,7 @@ public void runLassoUsingConstructor() {
@Test
public void runLassoUsingStaticMethods() {
int nPoints = 10000;
- double A = 2.0;
+ double A = 0.0;
double[] weights = {-1.5, 1.0e-2};
JavaRDD testRDD = sc.parallelize(LinearDataGenerator.generateLinearInputAsList(A,
diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java
index 2fdd5fc8fdca6..03714ae7e4d00 100644
--- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java
+++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java
@@ -55,30 +55,27 @@ public void tearDown() {
return errorSum / validationData.size();
}
- List generateRidgeData(int numPoints, int nfeatures, double eps) {
+ List generateRidgeData(int numPoints, int numFeatures, double std) {
org.jblas.util.Random.seed(42);
// Pick weights as random values distributed uniformly in [-0.5, 0.5]
- DoubleMatrix w = DoubleMatrix.rand(nfeatures, 1).subi(0.5);
- // Set first two weights to eps
- w.put(0, 0, eps);
- w.put(1, 0, eps);
- return LinearDataGenerator.generateLinearInputAsList(0.0, w.data, numPoints, 42, eps);
+ DoubleMatrix w = DoubleMatrix.rand(numFeatures, 1).subi(0.5);
+ return LinearDataGenerator.generateLinearInputAsList(0.0, w.data, numPoints, 42, std);
}
@Test
public void runRidgeRegressionUsingConstructor() {
- int nexamples = 200;
- int nfeatures = 20;
- double eps = 10.0;
- List data = generateRidgeData(2*nexamples, nfeatures, eps);
+ int numExamples = 50;
+ int numFeatures = 20;
+ List data = generateRidgeData(2*numExamples, numFeatures, 10.0);
- JavaRDD testRDD = sc.parallelize(data.subList(0, nexamples));
- List validationData = data.subList(nexamples, 2*nexamples);
+ JavaRDD testRDD = sc.parallelize(data.subList(0, numExamples));
+ List validationData = data.subList(numExamples, 2 * numExamples);
RidgeRegressionWithSGD ridgeSGDImpl = new RidgeRegressionWithSGD();
- ridgeSGDImpl.optimizer().setStepSize(1.0)
- .setRegParam(0.0)
- .setNumIterations(200);
+ ridgeSGDImpl.optimizer()
+ .setStepSize(1.0)
+ .setRegParam(0.0)
+ .setNumIterations(200);
RidgeRegressionModel model = ridgeSGDImpl.run(testRDD.rdd());
double unRegularizedErr = predictionError(validationData, model);
@@ -91,13 +88,12 @@ public void runRidgeRegressionUsingConstructor() {
@Test
public void runRidgeRegressionUsingStaticMethods() {
- int nexamples = 200;
- int nfeatures = 20;
- double eps = 10.0;
- List data = generateRidgeData(2*nexamples, nfeatures, eps);
+ int numExamples = 50;
+ int numFeatures = 20;
+ List data = generateRidgeData(2 * numExamples, numFeatures, 10.0);
- JavaRDD testRDD = sc.parallelize(data.subList(0, nexamples));
- List validationData = data.subList(nexamples, 2*nexamples);
+ JavaRDD testRDD = sc.parallelize(data.subList(0, numExamples));
+ List validationData = data.subList(numExamples, 2 * numExamples);
RidgeRegressionModel model = RidgeRegressionWithSGD.train(testRDD.rdd(), 200, 1.0, 0.0);
double unRegularizedErr = predictionError(validationData, model);
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala
index 05322b024d5f6..1e03c9df820b0 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala
@@ -20,11 +20,10 @@ package org.apache.spark.mllib.classification
import scala.util.Random
import scala.collection.JavaConversions._
-import org.scalatest.BeforeAndAfterAll
import org.scalatest.FunSuite
import org.scalatest.matchers.ShouldMatchers
-import org.apache.spark.SparkContext
+import org.apache.spark.mllib.linalg.Vectors
import org.apache.spark.mllib.regression._
import org.apache.spark.mllib.util.LocalSparkContext
@@ -61,7 +60,7 @@ object LogisticRegressionSuite {
if (yVal > 0) 1 else 0
}
- val testData = (0 until nPoints).map(i => LabeledPoint(y(i), Array(x1(i))))
+ val testData = (0 until nPoints).map(i => LabeledPoint(y(i), Vectors.dense(Array(x1(i)))))
testData
}
@@ -113,7 +112,7 @@ class LogisticRegressionSuite extends FunSuite with LocalSparkContext with Shoul
val testData = LogisticRegressionSuite.generateLogisticInput(A, B, nPoints, 42)
val initialB = -1.0
- val initialWeights = Array(initialB)
+ val initialWeights = Vectors.dense(initialB)
val testRDD = sc.parallelize(testData, 2)
testRDD.cache()
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala
index 9dd6c79ee6ad8..516895d04222d 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala
@@ -19,9 +19,9 @@ package org.apache.spark.mllib.classification
import scala.util.Random
-import org.scalatest.BeforeAndAfterAll
import org.scalatest.FunSuite
+import org.apache.spark.mllib.linalg.Vectors
import org.apache.spark.mllib.regression.LabeledPoint
import org.apache.spark.mllib.util.LocalSparkContext
@@ -54,7 +54,7 @@ object NaiveBayesSuite {
if (rnd.nextDouble() < _theta(y)(j)) 1 else 0
}
- LabeledPoint(y, xi)
+ LabeledPoint(y, Vectors.dense(xi))
}
}
}
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala
index bc7abb568a172..dfacbfeee6fb4 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala
@@ -20,7 +20,6 @@ package org.apache.spark.mllib.classification
import scala.util.Random
import scala.collection.JavaConversions._
-import org.scalatest.BeforeAndAfterAll
import org.scalatest.FunSuite
import org.jblas.DoubleMatrix
@@ -28,6 +27,7 @@ import org.jblas.DoubleMatrix
import org.apache.spark.SparkException
import org.apache.spark.mllib.regression._
import org.apache.spark.mllib.util.LocalSparkContext
+import org.apache.spark.mllib.linalg.Vectors
object SVMSuite {
@@ -54,7 +54,7 @@ object SVMSuite {
intercept + 0.01 * rnd.nextGaussian()
if (yD < 0) 0.0 else 1.0
}
- y.zip(x).map(p => LabeledPoint(p._1, p._2))
+ y.zip(x).map(p => LabeledPoint(p._1, Vectors.dense(p._2)))
}
}
@@ -110,7 +110,7 @@ class SVMSuite extends FunSuite with LocalSparkContext {
val initialB = -1.0
val initialC = -1.0
- val initialWeights = Array(initialB,initialC)
+ val initialWeights = Vectors.dense(initialB, initialC)
val testRDD = sc.parallelize(testData, 2)
testRDD.cache()
@@ -150,10 +150,10 @@ class SVMSuite extends FunSuite with LocalSparkContext {
}
intercept[SparkException] {
- val model = SVMWithSGD.train(testRDDInvalid, 100)
+ SVMWithSGD.train(testRDDInvalid, 100)
}
// Turning off data validation should not throw an exception
- val noValidationModel = new SVMWithSGD().setValidateData(false).run(testRDDInvalid)
+ new SVMWithSGD().setValidateData(false).run(testRDDInvalid)
}
}
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala
index 631d0e2ad9cdb..c4b433499a091 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala
@@ -20,13 +20,12 @@ package org.apache.spark.mllib.optimization
import scala.util.Random
import scala.collection.JavaConversions._
-import org.scalatest.BeforeAndAfterAll
import org.scalatest.FunSuite
import org.scalatest.matchers.ShouldMatchers
-import org.apache.spark.SparkContext
import org.apache.spark.mllib.regression._
import org.apache.spark.mllib.util.LocalSparkContext
+import org.apache.spark.mllib.linalg.Vectors
object GradientDescentSuite {
@@ -58,8 +57,7 @@ object GradientDescentSuite {
if (yVal > 0) 1 else 0
}
- val testData = (0 until nPoints).map(i => LabeledPoint(y(i), Array(x1(i))))
- testData
+ (0 until nPoints).map(i => LabeledPoint(y(i), Vectors.dense(x1(i))))
}
}
@@ -83,11 +81,11 @@ class GradientDescentSuite extends FunSuite with LocalSparkContext with ShouldMa
// Add a extra variable consisting of all 1.0's for the intercept.
val testData = GradientDescentSuite.generateGDInput(A, B, nPoints, 42)
val data = testData.map { case LabeledPoint(label, features) =>
- label -> Array(1.0, features: _*)
+ label -> Vectors.dense(1.0, features.toArray: _*)
}
val dataRDD = sc.parallelize(data, 2).cache()
- val initialWeightsWithIntercept = Array(1.0, initialWeights: _*)
+ val initialWeightsWithIntercept = Vectors.dense(1.0, initialWeights: _*)
val (_, loss) = GradientDescent.runMiniBatchSGD(
dataRDD,
@@ -113,13 +111,13 @@ class GradientDescentSuite extends FunSuite with LocalSparkContext with ShouldMa
// Add a extra variable consisting of all 1.0's for the intercept.
val testData = GradientDescentSuite.generateGDInput(2.0, -1.5, 10000, 42)
val data = testData.map { case LabeledPoint(label, features) =>
- label -> Array(1.0, features: _*)
+ label -> Vectors.dense(1.0, features.toArray: _*)
}
val dataRDD = sc.parallelize(data, 2).cache()
// Prepare non-zero weights
- val initialWeightsWithIntercept = Array(1.0, 0.5)
+ val initialWeightsWithIntercept = Vectors.dense(1.0, 0.5)
val regParam0 = 0
val (newWeights0, loss0) = GradientDescent.runMiniBatchSGD(
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala
index 2cebac943e15f..6aad9eb84e13c 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala
@@ -19,6 +19,7 @@ package org.apache.spark.mllib.regression
import org.scalatest.FunSuite
+import org.apache.spark.mllib.linalg.Vectors
import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext}
class LassoSuite extends FunSuite with LocalSparkContext {
@@ -33,29 +34,33 @@ class LassoSuite extends FunSuite with LocalSparkContext {
}
test("Lasso local random SGD") {
- val nPoints = 10000
+ val nPoints = 1000
val A = 2.0
val B = -1.5
val C = 1.0e-2
- val testData = LinearDataGenerator.generateLinearInput(A, Array[Double](B,C), nPoints, 42)
-
- val testRDD = sc.parallelize(testData, 2)
- testRDD.cache()
+ val testData = LinearDataGenerator.generateLinearInput(A, Array[Double](B, C), nPoints, 42)
+ .map { case LabeledPoint(label, features) =>
+ LabeledPoint(label, Vectors.dense(1.0 +: features.toArray))
+ }
+ val testRDD = sc.parallelize(testData, 2).cache()
val ls = new LassoWithSGD()
- ls.optimizer.setStepSize(1.0).setRegParam(0.01).setNumIterations(20)
+ ls.optimizer.setStepSize(1.0).setRegParam(0.01).setNumIterations(40)
val model = ls.run(testRDD)
-
val weight0 = model.weights(0)
val weight1 = model.weights(1)
- assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]")
- assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]")
- assert(weight1 >= -1.0e-3 && weight1 <= 1.0e-3, weight1 + " not in [-0.001, 0.001]")
+ val weight2 = model.weights(2)
+ assert(weight0 >= 1.9 && weight0 <= 2.1, weight0 + " not in [1.9, 2.1]")
+ assert(weight1 >= -1.60 && weight1 <= -1.40, weight1 + " not in [-1.6, -1.4]")
+ assert(weight2 >= -1.0e-3 && weight2 <= 1.0e-3, weight2 + " not in [-0.001, 0.001]")
val validationData = LinearDataGenerator.generateLinearInput(A, Array[Double](B,C), nPoints, 17)
+ .map { case LabeledPoint(label, features) =>
+ LabeledPoint(label, Vectors.dense(1.0 +: features.toArray))
+ }
val validationRDD = sc.parallelize(validationData, 2)
// Test prediction on RDD.
@@ -66,33 +71,39 @@ class LassoSuite extends FunSuite with LocalSparkContext {
}
test("Lasso local random SGD with initial weights") {
- val nPoints = 10000
+ val nPoints = 1000
val A = 2.0
val B = -1.5
val C = 1.0e-2
- val testData = LinearDataGenerator.generateLinearInput(A, Array[Double](B,C), nPoints, 42)
+ val testData = LinearDataGenerator.generateLinearInput(A, Array[Double](B, C), nPoints, 42)
+ .map { case LabeledPoint(label, features) =>
+ LabeledPoint(label, Vectors.dense(1.0 +: features.toArray))
+ }
+ val initialA = -1.0
val initialB = -1.0
val initialC = -1.0
- val initialWeights = Array(initialB,initialC)
+ val initialWeights = Vectors.dense(initialA, initialB, initialC)
- val testRDD = sc.parallelize(testData, 2)
- testRDD.cache()
+ val testRDD = sc.parallelize(testData, 2).cache()
val ls = new LassoWithSGD()
- ls.optimizer.setStepSize(1.0).setRegParam(0.01).setNumIterations(20)
+ ls.optimizer.setStepSize(1.0).setRegParam(0.01).setNumIterations(40)
val model = ls.run(testRDD, initialWeights)
-
val weight0 = model.weights(0)
val weight1 = model.weights(1)
- assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]")
- assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]")
- assert(weight1 >= -1.0e-3 && weight1 <= 1.0e-3, weight1 + " not in [-0.001, 0.001]")
+ val weight2 = model.weights(2)
+ assert(weight0 >= 1.9 && weight0 <= 2.1, weight0 + " not in [1.9, 2.1]")
+ assert(weight1 >= -1.60 && weight1 <= -1.40, weight1 + " not in [-1.6, -1.4]")
+ assert(weight2 >= -1.0e-3 && weight2 <= 1.0e-3, weight2 + " not in [-0.001, 0.001]")
val validationData = LinearDataGenerator.generateLinearInput(A, Array[Double](B,C), nPoints, 17)
+ .map { case LabeledPoint(label, features) =>
+ LabeledPoint(label, Vectors.dense(1.0 +: features.toArray))
+ }
val validationRDD = sc.parallelize(validationData,2)
// Test prediction on RDD.
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala
index 5d251bcbf35db..2f7d30708ce17 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala
@@ -19,6 +19,7 @@ package org.apache.spark.mllib.regression
import org.scalatest.FunSuite
+import org.apache.spark.mllib.linalg.Vectors
import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext}
class LinearRegressionSuite extends FunSuite with LocalSparkContext {
@@ -40,11 +41,12 @@ class LinearRegressionSuite extends FunSuite with LocalSparkContext {
linReg.optimizer.setNumIterations(1000).setStepSize(1.0)
val model = linReg.run(testRDD)
-
assert(model.intercept >= 2.5 && model.intercept <= 3.5)
- assert(model.weights.length === 2)
- assert(model.weights(0) >= 9.0 && model.weights(0) <= 11.0)
- assert(model.weights(1) >= 9.0 && model.weights(1) <= 11.0)
+
+ val weights = model.weights
+ assert(weights.size === 2)
+ assert(weights(0) >= 9.0 && weights(0) <= 11.0)
+ assert(weights(1) >= 9.0 && weights(1) <= 11.0)
val validationData = LinearDataGenerator.generateLinearInput(
3.0, Array(10.0, 10.0), 100, 17)
@@ -67,9 +69,11 @@ class LinearRegressionSuite extends FunSuite with LocalSparkContext {
val model = linReg.run(testRDD)
assert(model.intercept === 0.0)
- assert(model.weights.length === 2)
- assert(model.weights(0) >= 9.0 && model.weights(0) <= 11.0)
- assert(model.weights(1) >= 9.0 && model.weights(1) <= 11.0)
+
+ val weights = model.weights
+ assert(weights.size === 2)
+ assert(weights(0) >= 9.0 && weights(0) <= 11.0)
+ assert(weights(1) >= 9.0 && weights(1) <= 11.0)
val validationData = LinearDataGenerator.generateLinearInput(
0.0, Array(10.0, 10.0), 100, 17)
@@ -81,4 +85,40 @@ class LinearRegressionSuite extends FunSuite with LocalSparkContext {
// Test prediction on Array.
validatePrediction(validationData.map(row => model.predict(row.features)), validationData)
}
+
+ // Test if we can correctly learn Y = 10*X1 + 10*X10000
+ test("sparse linear regression without intercept") {
+ val denseRDD = sc.parallelize(
+ LinearDataGenerator.generateLinearInput(0.0, Array(10.0, 10.0), 100, 42), 2)
+ val sparseRDD = denseRDD.map { case LabeledPoint(label, v) =>
+ val sv = Vectors.sparse(10000, Seq((0, v(0)), (9999, v(1))))
+ LabeledPoint(label, sv)
+ }.cache()
+ val linReg = new LinearRegressionWithSGD().setIntercept(false)
+ linReg.optimizer.setNumIterations(1000).setStepSize(1.0)
+
+ val model = linReg.run(sparseRDD)
+
+ assert(model.intercept === 0.0)
+
+ val weights = model.weights
+ assert(weights.size === 10000)
+ assert(weights(0) >= 9.0 && weights(0) <= 11.0)
+ assert(weights(9999) >= 9.0 && weights(9999) <= 11.0)
+
+ val validationData = LinearDataGenerator.generateLinearInput(0.0, Array(10.0, 10.0), 100, 17)
+ val sparseValidationData = validationData.map { case LabeledPoint(label, v) =>
+ val sv = Vectors.sparse(10000, Seq((0, v(0)), (9999, v(1))))
+ LabeledPoint(label, sv)
+ }
+ val sparseValidationRDD = sc.parallelize(sparseValidationData, 2)
+
+ // Test prediction on RDD.
+ validatePrediction(
+ model.predict(sparseValidationRDD.map(_.features)).collect(), sparseValidationData)
+
+ // Test prediction on Array.
+ validatePrediction(
+ sparseValidationData.map(row => model.predict(row.features)), sparseValidationData)
+ }
}
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala
index b2044ed0d8066..f66fc6ea6c1ec 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala
@@ -17,9 +17,10 @@
package org.apache.spark.mllib.regression
-import org.jblas.DoubleMatrix
import org.scalatest.FunSuite
+import org.jblas.DoubleMatrix
+
import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext}
class RidgeRegressionSuite extends FunSuite with LocalSparkContext {
@@ -30,22 +31,22 @@ class RidgeRegressionSuite extends FunSuite with LocalSparkContext {
}.reduceLeft(_ + _) / predictions.size
}
- test("regularization with skewed weights") {
- val nexamples = 200
- val nfeatures = 20
- val eps = 10
+ test("ridge regression can help avoid overfitting") {
+
+ // For small number of examples and large variance of error distribution,
+ // ridge regression should give smaller generalization error that linear regression.
+
+ val numExamples = 50
+ val numFeatures = 20
org.jblas.util.Random.seed(42)
// Pick weights as random values distributed uniformly in [-0.5, 0.5]
- val w = DoubleMatrix.rand(nfeatures, 1).subi(0.5)
- // Set first two weights to eps
- w.put(0, 0, eps)
- w.put(1, 0, eps)
+ val w = DoubleMatrix.rand(numFeatures, 1).subi(0.5)
// Use half of data for training and other half for validation
- val data = LinearDataGenerator.generateLinearInput(3.0, w.toArray, 2*nexamples, 42, eps)
- val testData = data.take(nexamples)
- val validationData = data.takeRight(nexamples)
+ val data = LinearDataGenerator.generateLinearInput(3.0, w.toArray, 2 * numExamples, 42, 10.0)
+ val testData = data.take(numExamples)
+ val validationData = data.takeRight(numExamples)
val testRDD = sc.parallelize(testData, 2).cache()
val validationRDD = sc.parallelize(validationData, 2).cache()
@@ -67,7 +68,7 @@ class RidgeRegressionSuite extends FunSuite with LocalSparkContext {
val ridgeErr = predictionError(
ridgeModel.predict(validationRDD.map(_.features)).collect(), validationData)
- // Ridge CV-error should be lower than linear regression
+ // Ridge validation error should be lower than linear regression.
assert(ridgeErr < linearErr,
"ridgeError (" + ridgeErr + ") was not less than linearError(" + linearErr + ")")
}
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala
new file mode 100644
index 0000000000000..350130c914f26
--- /dev/null
+++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala
@@ -0,0 +1,426 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.mllib.tree
+
+import org.scalatest.BeforeAndAfterAll
+import org.scalatest.FunSuite
+
+import org.apache.spark.SparkContext
+import org.apache.spark.mllib.regression.LabeledPoint
+import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Variance}
+import org.apache.spark.mllib.tree.model.Filter
+import org.apache.spark.mllib.tree.configuration.Strategy
+import org.apache.spark.mllib.tree.configuration.Algo._
+import org.apache.spark.mllib.tree.configuration.FeatureType._
+import org.apache.spark.mllib.linalg.Vectors
+
+class DecisionTreeSuite extends FunSuite with BeforeAndAfterAll {
+
+ @transient private var sc: SparkContext = _
+
+ override def beforeAll() {
+ sc = new SparkContext("local", "test")
+ }
+
+ override def afterAll() {
+ sc.stop()
+ System.clearProperty("spark.driver.port")
+ }
+
+ test("split and bin calculation") {
+ val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1()
+ assert(arr.length === 1000)
+ val rdd = sc.parallelize(arr)
+ val strategy = new Strategy(Classification, Gini, 3, 100)
+ val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy)
+ assert(splits.length === 2)
+ assert(bins.length === 2)
+ assert(splits(0).length === 99)
+ assert(bins(0).length === 100)
+ }
+
+ test("split and bin calculation for categorical variables") {
+ val arr = DecisionTreeSuite.generateCategoricalDataPoints()
+ assert(arr.length === 1000)
+ val rdd = sc.parallelize(arr)
+ val strategy = new Strategy(
+ Classification,
+ Gini,
+ maxDepth = 3,
+ maxBins = 100,
+ categoricalFeaturesInfo = Map(0 -> 2, 1-> 2))
+ val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy)
+ assert(splits.length === 2)
+ assert(bins.length === 2)
+ assert(splits(0).length === 99)
+ assert(bins(0).length === 100)
+
+ // Check splits.
+
+ assert(splits(0)(0).feature === 0)
+ assert(splits(0)(0).threshold === Double.MinValue)
+ assert(splits(0)(0).featureType === Categorical)
+ assert(splits(0)(0).categories.length === 1)
+ assert(splits(0)(0).categories.contains(1.0))
+
+ assert(splits(0)(1).feature === 0)
+ assert(splits(0)(1).threshold === Double.MinValue)
+ assert(splits(0)(1).featureType === Categorical)
+ assert(splits(0)(1).categories.length === 2)
+ assert(splits(0)(1).categories.contains(1.0))
+ assert(splits(0)(1).categories.contains(0.0))
+
+ assert(splits(0)(2) === null)
+
+ assert(splits(1)(0).feature === 1)
+ assert(splits(1)(0).threshold === Double.MinValue)
+ assert(splits(1)(0).featureType === Categorical)
+ assert(splits(1)(0).categories.length === 1)
+ assert(splits(1)(0).categories.contains(0.0))
+
+ assert(splits(1)(1).feature === 1)
+ assert(splits(1)(1).threshold === Double.MinValue)
+ assert(splits(1)(1).featureType === Categorical)
+ assert(splits(1)(1).categories.length === 2)
+ assert(splits(1)(1).categories.contains(1.0))
+ assert(splits(1)(1).categories.contains(0.0))
+
+ assert(splits(1)(2) === null)
+
+ // Check bins.
+
+ assert(bins(0)(0).category === 1.0)
+ assert(bins(0)(0).lowSplit.categories.length === 0)
+ assert(bins(0)(0).highSplit.categories.length === 1)
+ assert(bins(0)(0).highSplit.categories.contains(1.0))
+
+ assert(bins(0)(1).category === 0.0)
+ assert(bins(0)(1).lowSplit.categories.length === 1)
+ assert(bins(0)(1).lowSplit.categories.contains(1.0))
+ assert(bins(0)(1).highSplit.categories.length === 2)
+ assert(bins(0)(1).highSplit.categories.contains(1.0))
+ assert(bins(0)(1).highSplit.categories.contains(0.0))
+
+ assert(bins(0)(2) === null)
+
+ assert(bins(1)(0).category === 0.0)
+ assert(bins(1)(0).lowSplit.categories.length === 0)
+ assert(bins(1)(0).highSplit.categories.length === 1)
+ assert(bins(1)(0).highSplit.categories.contains(0.0))
+
+ assert(bins(1)(1).category === 1.0)
+ assert(bins(1)(1).lowSplit.categories.length === 1)
+ assert(bins(1)(1).lowSplit.categories.contains(0.0))
+ assert(bins(1)(1).highSplit.categories.length === 2)
+ assert(bins(1)(1).highSplit.categories.contains(0.0))
+ assert(bins(1)(1).highSplit.categories.contains(1.0))
+
+ assert(bins(1)(2) === null)
+ }
+
+ test("split and bin calculations for categorical variables with no sample for one category") {
+ val arr = DecisionTreeSuite.generateCategoricalDataPoints()
+ assert(arr.length === 1000)
+ val rdd = sc.parallelize(arr)
+ val strategy = new Strategy(
+ Classification,
+ Gini,
+ maxDepth = 3,
+ maxBins = 100,
+ categoricalFeaturesInfo = Map(0 -> 3, 1-> 3))
+ val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy)
+
+ // Check splits.
+
+ assert(splits(0)(0).feature === 0)
+ assert(splits(0)(0).threshold === Double.MinValue)
+ assert(splits(0)(0).featureType === Categorical)
+ assert(splits(0)(0).categories.length === 1)
+ assert(splits(0)(0).categories.contains(1.0))
+
+ assert(splits(0)(1).feature === 0)
+ assert(splits(0)(1).threshold === Double.MinValue)
+ assert(splits(0)(1).featureType === Categorical)
+ assert(splits(0)(1).categories.length === 2)
+ assert(splits(0)(1).categories.contains(1.0))
+ assert(splits(0)(1).categories.contains(0.0))
+
+ assert(splits(0)(2).feature === 0)
+ assert(splits(0)(2).threshold === Double.MinValue)
+ assert(splits(0)(2).featureType === Categorical)
+ assert(splits(0)(2).categories.length === 3)
+ assert(splits(0)(2).categories.contains(1.0))
+ assert(splits(0)(2).categories.contains(0.0))
+ assert(splits(0)(2).categories.contains(2.0))
+
+ assert(splits(0)(3) === null)
+
+ assert(splits(1)(0).feature === 1)
+ assert(splits(1)(0).threshold === Double.MinValue)
+ assert(splits(1)(0).featureType === Categorical)
+ assert(splits(1)(0).categories.length === 1)
+ assert(splits(1)(0).categories.contains(0.0))
+
+ assert(splits(1)(1).feature === 1)
+ assert(splits(1)(1).threshold === Double.MinValue)
+ assert(splits(1)(1).featureType === Categorical)
+ assert(splits(1)(1).categories.length === 2)
+ assert(splits(1)(1).categories.contains(1.0))
+ assert(splits(1)(1).categories.contains(0.0))
+
+ assert(splits(1)(2).feature === 1)
+ assert(splits(1)(2).threshold === Double.MinValue)
+ assert(splits(1)(2).featureType === Categorical)
+ assert(splits(1)(2).categories.length === 3)
+ assert(splits(1)(2).categories.contains(1.0))
+ assert(splits(1)(2).categories.contains(0.0))
+ assert(splits(1)(2).categories.contains(2.0))
+
+ assert(splits(1)(3) === null)
+
+ // Check bins.
+
+ assert(bins(0)(0).category === 1.0)
+ assert(bins(0)(0).lowSplit.categories.length === 0)
+ assert(bins(0)(0).highSplit.categories.length === 1)
+ assert(bins(0)(0).highSplit.categories.contains(1.0))
+
+ assert(bins(0)(1).category === 0.0)
+ assert(bins(0)(1).lowSplit.categories.length === 1)
+ assert(bins(0)(1).lowSplit.categories.contains(1.0))
+ assert(bins(0)(1).highSplit.categories.length === 2)
+ assert(bins(0)(1).highSplit.categories.contains(1.0))
+ assert(bins(0)(1).highSplit.categories.contains(0.0))
+
+ assert(bins(0)(2).category === 2.0)
+ assert(bins(0)(2).lowSplit.categories.length === 2)
+ assert(bins(0)(2).lowSplit.categories.contains(1.0))
+ assert(bins(0)(2).lowSplit.categories.contains(0.0))
+ assert(bins(0)(2).highSplit.categories.length === 3)
+ assert(bins(0)(2).highSplit.categories.contains(1.0))
+ assert(bins(0)(2).highSplit.categories.contains(0.0))
+ assert(bins(0)(2).highSplit.categories.contains(2.0))
+
+ assert(bins(0)(3) === null)
+
+ assert(bins(1)(0).category === 0.0)
+ assert(bins(1)(0).lowSplit.categories.length === 0)
+ assert(bins(1)(0).highSplit.categories.length === 1)
+ assert(bins(1)(0).highSplit.categories.contains(0.0))
+
+ assert(bins(1)(1).category === 1.0)
+ assert(bins(1)(1).lowSplit.categories.length === 1)
+ assert(bins(1)(1).lowSplit.categories.contains(0.0))
+ assert(bins(1)(1).highSplit.categories.length === 2)
+ assert(bins(1)(1).highSplit.categories.contains(0.0))
+ assert(bins(1)(1).highSplit.categories.contains(1.0))
+
+ assert(bins(1)(2).category === 2.0)
+ assert(bins(1)(2).lowSplit.categories.length === 2)
+ assert(bins(1)(2).lowSplit.categories.contains(0.0))
+ assert(bins(1)(2).lowSplit.categories.contains(1.0))
+ assert(bins(1)(2).highSplit.categories.length === 3)
+ assert(bins(1)(2).highSplit.categories.contains(0.0))
+ assert(bins(1)(2).highSplit.categories.contains(1.0))
+ assert(bins(1)(2).highSplit.categories.contains(2.0))
+
+ assert(bins(1)(3) === null)
+ }
+
+ test("classification stump with all categorical variables") {
+ val arr = DecisionTreeSuite.generateCategoricalDataPoints()
+ assert(arr.length === 1000)
+ val rdd = sc.parallelize(arr)
+ val strategy = new Strategy(
+ Classification,
+ Gini,
+ maxDepth = 3,
+ maxBins = 100,
+ categoricalFeaturesInfo = Map(0 -> 3, 1-> 3))
+ val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy)
+ val bestSplits = DecisionTree.findBestSplits(rdd, new Array(7), strategy, 0,
+ Array[List[Filter]](), splits, bins)
+
+ val split = bestSplits(0)._1
+ assert(split.categories.length === 1)
+ assert(split.categories.contains(1.0))
+ assert(split.featureType === Categorical)
+ assert(split.threshold === Double.MinValue)
+
+ val stats = bestSplits(0)._2
+ assert(stats.gain > 0)
+ assert(stats.predict > 0.4)
+ assert(stats.predict < 0.5)
+ assert(stats.impurity > 0.2)
+ }
+
+ test("regression stump with all categorical variables") {
+ val arr = DecisionTreeSuite.generateCategoricalDataPoints()
+ assert(arr.length === 1000)
+ val rdd = sc.parallelize(arr)
+ val strategy = new Strategy(
+ Regression,
+ Variance,
+ maxDepth = 3,
+ maxBins = 100,
+ categoricalFeaturesInfo = Map(0 -> 3, 1-> 3))
+ val (splits, bins) = DecisionTree.findSplitsBins(rdd,strategy)
+ val bestSplits = DecisionTree.findBestSplits(rdd, new Array(7), strategy, 0,
+ Array[List[Filter]](), splits, bins)
+
+ val split = bestSplits(0)._1
+ assert(split.categories.length === 1)
+ assert(split.categories.contains(1.0))
+ assert(split.featureType === Categorical)
+ assert(split.threshold === Double.MinValue)
+
+ val stats = bestSplits(0)._2
+ assert(stats.gain > 0)
+ assert(stats.predict > 0.4)
+ assert(stats.predict < 0.5)
+ assert(stats.impurity > 0.2)
+ }
+
+ test("stump with fixed label 0 for Gini") {
+ val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel0()
+ assert(arr.length === 1000)
+ val rdd = sc.parallelize(arr)
+ val strategy = new Strategy(Classification, Gini, 3, 100)
+ val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy)
+ assert(splits.length === 2)
+ assert(splits(0).length === 99)
+ assert(bins.length === 2)
+ assert(bins(0).length === 100)
+ assert(splits(0).length === 99)
+ assert(bins(0).length === 100)
+
+ val bestSplits = DecisionTree.findBestSplits(rdd, new Array(7), strategy, 0,
+ Array[List[Filter]](), splits, bins)
+ assert(bestSplits.length === 1)
+ assert(bestSplits(0)._1.feature === 0)
+ assert(bestSplits(0)._1.threshold === 10)
+ assert(bestSplits(0)._2.gain === 0)
+ assert(bestSplits(0)._2.leftImpurity === 0)
+ assert(bestSplits(0)._2.rightImpurity === 0)
+ }
+
+ test("stump with fixed label 1 for Gini") {
+ val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1()
+ assert(arr.length === 1000)
+ val rdd = sc.parallelize(arr)
+ val strategy = new Strategy(Classification, Gini, 3, 100)
+ val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy)
+ assert(splits.length === 2)
+ assert(splits(0).length === 99)
+ assert(bins.length === 2)
+ assert(bins(0).length === 100)
+ assert(splits(0).length === 99)
+ assert(bins(0).length === 100)
+
+ val bestSplits = DecisionTree.findBestSplits(rdd, Array(0.0), strategy, 0,
+ Array[List[Filter]](), splits, bins)
+ assert(bestSplits.length === 1)
+ assert(bestSplits(0)._1.feature === 0)
+ assert(bestSplits(0)._1.threshold === 10)
+ assert(bestSplits(0)._2.gain === 0)
+ assert(bestSplits(0)._2.leftImpurity === 0)
+ assert(bestSplits(0)._2.rightImpurity === 0)
+ assert(bestSplits(0)._2.predict === 1)
+ }
+
+ test("stump with fixed label 0 for Entropy") {
+ val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel0()
+ assert(arr.length === 1000)
+ val rdd = sc.parallelize(arr)
+ val strategy = new Strategy(Classification, Entropy, 3, 100)
+ val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy)
+ assert(splits.length === 2)
+ assert(splits(0).length === 99)
+ assert(bins.length === 2)
+ assert(bins(0).length === 100)
+ assert(splits(0).length === 99)
+ assert(bins(0).length === 100)
+
+ val bestSplits = DecisionTree.findBestSplits(rdd, Array(0.0), strategy, 0,
+ Array[List[Filter]](), splits, bins)
+ assert(bestSplits.length === 1)
+ assert(bestSplits(0)._1.feature === 0)
+ assert(bestSplits(0)._1.threshold === 10)
+ assert(bestSplits(0)._2.gain === 0)
+ assert(bestSplits(0)._2.leftImpurity === 0)
+ assert(bestSplits(0)._2.rightImpurity === 0)
+ assert(bestSplits(0)._2.predict === 0)
+ }
+
+ test("stump with fixed label 1 for Entropy") {
+ val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1()
+ assert(arr.length === 1000)
+ val rdd = sc.parallelize(arr)
+ val strategy = new Strategy(Classification, Entropy, 3, 100)
+ val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy)
+ assert(splits.length === 2)
+ assert(splits(0).length === 99)
+ assert(bins.length === 2)
+ assert(bins(0).length === 100)
+ assert(splits(0).length === 99)
+ assert(bins(0).length === 100)
+
+ val bestSplits = DecisionTree.findBestSplits(rdd, Array(0.0), strategy, 0,
+ Array[List[Filter]](), splits, bins)
+ assert(bestSplits.length === 1)
+ assert(bestSplits(0)._1.feature === 0)
+ assert(bestSplits(0)._1.threshold === 10)
+ assert(bestSplits(0)._2.gain === 0)
+ assert(bestSplits(0)._2.leftImpurity === 0)
+ assert(bestSplits(0)._2.rightImpurity === 0)
+ assert(bestSplits(0)._2.predict === 1)
+ }
+}
+
+object DecisionTreeSuite {
+
+ def generateOrderedLabeledPointsWithLabel0(): Array[LabeledPoint] = {
+ val arr = new Array[LabeledPoint](1000)
+ for (i <- 0 until 1000){
+ val lp = new LabeledPoint(0.0, Vectors.dense(i.toDouble, 1000.0 - i))
+ arr(i) = lp
+ }
+ arr
+ }
+
+ def generateOrderedLabeledPointsWithLabel1(): Array[LabeledPoint] = {
+ val arr = new Array[LabeledPoint](1000)
+ for (i <- 0 until 1000){
+ val lp = new LabeledPoint(1.0, Vectors.dense(i.toDouble, 999.0 - i))
+ arr(i) = lp
+ }
+ arr
+ }
+
+ def generateCategoricalDataPoints(): Array[LabeledPoint] = {
+ val arr = new Array[LabeledPoint](1000)
+ for (i <- 0 until 1000){
+ if (i < 600){
+ arr(i) = new LabeledPoint(1.0, Vectors.dense(0.0, 1.0))
+ } else {
+ arr(i) = new LabeledPoint(0.0, Vectors.dense(1.0, 0.0))
+ }
+ }
+ arr
+ }
+}
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala
index 60f053b381305..27d41c7869aa0 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala
@@ -17,14 +17,20 @@
package org.apache.spark.mllib.util
+import java.io.File
+
import org.scalatest.FunSuite
import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, norm => breezeNorm,
squaredDistance => breezeSquaredDistance}
+import com.google.common.base.Charsets
+import com.google.common.io.Files
+import org.apache.spark.mllib.linalg.Vectors
+import org.apache.spark.mllib.regression.LabeledPoint
import org.apache.spark.mllib.util.MLUtils._
-class MLUtilsSuite extends FunSuite {
+class MLUtilsSuite extends FunSuite with LocalSparkContext {
test("epsilon computation") {
assert(1.0 + EPSILON > 1.0, s"EPSILON is too small: $EPSILON.")
@@ -49,4 +55,55 @@ class MLUtilsSuite extends FunSuite {
assert((fastSquaredDist2 - squaredDist) <= precision * squaredDist, s"failed with m = $m")
}
}
+
+ test("compute stats") {
+ val data = Seq.fill(3)(Seq(
+ LabeledPoint(1.0, Vectors.dense(1.0, 2.0, 3.0)),
+ LabeledPoint(0.0, Vectors.dense(3.0, 4.0, 5.0))
+ )).flatten
+ val rdd = sc.parallelize(data, 2)
+ val (meanLabel, mean, std) = MLUtils.computeStats(rdd, 3, 6)
+ assert(meanLabel === 0.5)
+ assert(mean === Vectors.dense(2.0, 3.0, 4.0))
+ assert(std === Vectors.dense(1.0, 1.0, 1.0))
+ }
+
+ test("loadLibSVMData") {
+ val lines =
+ """
+ |+1 1:1.0 3:2.0 5:3.0
+ |-1
+ |-1 2:4.0 4:5.0 6:6.0
+ """.stripMargin
+ val tempDir = Files.createTempDir()
+ val file = new File(tempDir.getPath, "part-00000")
+ Files.write(lines, file, Charsets.US_ASCII)
+ val path = tempDir.toURI.toString
+
+ val pointsWithNumFeatures = MLUtils.loadLibSVMData(sc, path, 6).collect()
+ val pointsWithoutNumFeatures = MLUtils.loadLibSVMData(sc, path).collect()
+
+ for (points <- Seq(pointsWithNumFeatures, pointsWithoutNumFeatures)) {
+ assert(points.length === 3)
+ assert(points(0).label === 1.0)
+ assert(points(0).features === Vectors.sparse(6, Seq((0, 1.0), (2, 2.0), (4, 3.0))))
+ assert(points(1).label == 0.0)
+ assert(points(1).features == Vectors.sparse(6, Seq()))
+ assert(points(2).label === 0.0)
+ assert(points(2).features === Vectors.sparse(6, Seq((1, 4.0), (3, 5.0), (5, 6.0))))
+ }
+
+ val multiclassPoints = MLUtils.loadLibSVMData(sc, path, MLUtils.multiclassLabelParser).collect()
+ assert(multiclassPoints.length === 3)
+ assert(multiclassPoints(0).label === 1.0)
+ assert(multiclassPoints(1).label === -1.0)
+ assert(multiclassPoints(2).label === -1.0)
+
+ try {
+ file.delete()
+ tempDir.delete()
+ } catch {
+ case t: Throwable =>
+ }
+ }
}
diff --git a/pom.xml b/pom.xml
index 72acf2b402703..7d58060cba606 100644
--- a/pom.xml
+++ b/pom.xml
@@ -110,7 +110,7 @@
1.6
- 2.10.3
+ 2.10.4
2.10
0.13.0
org.spark-project.akka
@@ -192,22 +192,22 @@
org.eclipse.jetty
jetty-util
- 7.6.8.v20121106
+ 8.1.14.v20131031
org.eclipse.jetty
jetty-security
- 7.6.8.v20121106
+ 8.1.14.v20131031
org.eclipse.jetty
jetty-plus
- 7.6.8.v20121106
+ 8.1.14.v20131031
org.eclipse.jetty
jetty-server
- 7.6.8.v20121106
+ 8.1.14.v20131031
com.google.guava
@@ -380,7 +380,7 @@
lift-json_${scala.binary.version}
2.5.1
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 2549bc9710f1f..c5c697e8e2427 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -152,7 +152,7 @@ object SparkBuild extends Build {
def sharedSettings = Defaults.defaultSettings ++ MimaBuild.mimaSettings(file(sparkHome)) ++ Seq(
organization := "org.apache.spark",
version := SPARK_VERSION,
- scalaVersion := "2.10.3",
+ scalaVersion := "2.10.4",
scalacOptions := Seq("-Xmax-classfile-name", "120", "-unchecked", "-deprecation",
"-target:" + SCALAC_JVM_VERSION),
javacOptions := Seq("-target", JAVAC_JVM_VERSION, "-source", JAVAC_JVM_VERSION),
@@ -248,13 +248,13 @@ object SparkBuild extends Build {
*/
libraryDependencies ++= Seq(
- "io.netty" % "netty-all" % "4.0.17.Final",
- "org.eclipse.jetty" % "jetty-server" % "7.6.8.v20121106",
- "org.eclipse.jetty" % "jetty-util" % "7.6.8.v20121106",
- "org.eclipse.jetty" % "jetty-plus" % "7.6.8.v20121106",
- "org.eclipse.jetty" % "jetty-security" % "7.6.8.v20121106",
+ "io.netty" % "netty-all" % "4.0.17.Final",
+ "org.eclipse.jetty" % "jetty-server" % "8.1.14.v20131031",
+ "org.eclipse.jetty" % "jetty-util" % "8.1.14.v20131031",
+ "org.eclipse.jetty" % "jetty-plus" % "8.1.14.v20131031",
+ "org.eclipse.jetty" % "jetty-security" % "8.1.14.v20131031",
/** Workaround for SPARK-959. Dependency used by org.eclipse.jetty. Fixed in ivy 2.3.0. */
- "org.eclipse.jetty.orbit" % "javax.servlet" % "2.5.0.v201103041518" artifacts Artifact("javax.servlet", "jar", "jar"),
+ "org.eclipse.jetty.orbit" % "javax.servlet" % "3.0.0.v201112011016" artifacts Artifact("javax.servlet", "jar", "jar"),
"org.scalatest" %% "scalatest" % "1.9.1" % "test",
"org.scalacheck" %% "scalacheck" % "1.10.0" % "test",
"com.novocode" % "junit-interface" % "0.10" % "test",
diff --git a/project/plugins.sbt b/project/plugins.sbt
index 5aa8a1ec2409b..d787237ddc540 100644
--- a/project/plugins.sbt
+++ b/project/plugins.sbt
@@ -1,4 +1,4 @@
-scalaVersion := "2.10.3"
+scalaVersion := "2.10.4"
resolvers += Resolver.url("artifactory", url("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases"))(Resolver.ivyStylePatterns)
diff --git a/project/project/SparkPluginBuild.scala b/project/project/SparkPluginBuild.scala
index 5a307044ba123..0142256e90fb7 100644
--- a/project/project/SparkPluginBuild.scala
+++ b/project/project/SparkPluginBuild.scala
@@ -32,7 +32,7 @@ object SparkPluginDef extends Build {
name := "spark-style",
organization := "org.apache.spark",
version := sparkVersion,
- scalaVersion := "2.10.3",
+ scalaVersion := "2.10.4",
scalacOptions := Seq("-unchecked", "-deprecation"),
libraryDependencies ++= Dependencies.scalaStyle
)
diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py
index 19b90dfd6e167..d2f9cdb3f4298 100644
--- a/python/pyspark/mllib/classification.py
+++ b/python/pyspark/mllib/classification.py
@@ -87,18 +87,19 @@ class NaiveBayesModel(object):
>>> data = array([0.0, 0.0, 1.0, 0.0, 0.0, 2.0, 1.0, 1.0, 0.0]).reshape(3,3)
>>> model = NaiveBayes.train(sc.parallelize(data))
>>> model.predict(array([0.0, 1.0]))
- 0
+ 0.0
>>> model.predict(array([1.0, 0.0]))
- 1
+ 1.0
"""
- def __init__(self, pi, theta):
+ def __init__(self, labels, pi, theta):
+ self.labels = labels
self.pi = pi
self.theta = theta
def predict(self, x):
"""Return the most likely class for a data vector x"""
- return numpy.argmax(self.pi + dot(x, self.theta))
+ return self.labels[numpy.argmax(self.pi + dot(x, self.theta))]
class NaiveBayes(object):
@classmethod
@@ -122,7 +123,8 @@ def train(cls, data, lambda_=1.0):
ans = sc._jvm.PythonMLLibAPI().trainNaiveBayes(dataBytes._jrdd, lambda_)
return NaiveBayesModel(
_deserialize_double_vector(ans[0]),
- _deserialize_double_matrix(ans[1]))
+ _deserialize_double_vector(ans[1]),
+ _deserialize_double_matrix(ans[2]))
def _test():
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index 019c249699c2d..9943296b927dc 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -29,7 +29,7 @@
from tempfile import NamedTemporaryFile
from threading import Thread
import warnings
-from heapq import heappush, heappop, heappushpop
+import heapq
from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \
BatchedSerializer, CloudPickleSerializer, PairDeserializer, pack_long
@@ -41,9 +41,9 @@
from py4j.java_collections import ListConverter, MapConverter
-
__all__ = ["RDD"]
+
def _extract_concise_traceback():
"""
This function returns the traceback info for a callsite, returns a dict
@@ -91,6 +91,73 @@ def __exit__(self, type, value, tb):
if _spark_stack_depth == 0:
self._context._jsc.setCallSite(None)
+class MaxHeapQ(object):
+ """
+ An implementation of MaxHeap.
+ >>> import pyspark.rdd
+ >>> heap = pyspark.rdd.MaxHeapQ(5)
+ >>> [heap.insert(i) for i in range(10)]
+ [None, None, None, None, None, None, None, None, None, None]
+ >>> sorted(heap.getElements())
+ [0, 1, 2, 3, 4]
+ >>> heap = pyspark.rdd.MaxHeapQ(5)
+ >>> [heap.insert(i) for i in range(9, -1, -1)]
+ [None, None, None, None, None, None, None, None, None, None]
+ >>> sorted(heap.getElements())
+ [0, 1, 2, 3, 4]
+ >>> heap = pyspark.rdd.MaxHeapQ(1)
+ >>> [heap.insert(i) for i in range(9, -1, -1)]
+ [None, None, None, None, None, None, None, None, None, None]
+ >>> heap.getElements()
+ [0]
+ """
+
+ def __init__(self, maxsize):
+ # we start from q[1], this makes calculating children as trivial as 2 * k
+ self.q = [0]
+ self.maxsize = maxsize
+
+ def _swim(self, k):
+ while (k > 1) and (self.q[k/2] < self.q[k]):
+ self._swap(k, k/2)
+ k = k/2
+
+ def _swap(self, i, j):
+ t = self.q[i]
+ self.q[i] = self.q[j]
+ self.q[j] = t
+
+ def _sink(self, k):
+ N = self.size()
+ while 2 * k <= N:
+ j = 2 * k
+ # Here we test if both children are greater than parent
+ # if not swap with larger one.
+ if j < N and self.q[j] < self.q[j + 1]:
+ j = j + 1
+ if(self.q[k] > self.q[j]):
+ break
+ self._swap(k, j)
+ k = j
+
+ def size(self):
+ return len(self.q) - 1
+
+ def insert(self, value):
+ if (self.size()) < self.maxsize:
+ self.q.append(value)
+ self._swim(self.size())
+ else:
+ self._replaceRoot(value)
+
+ def getElements(self):
+ return self.q[1:]
+
+ def _replaceRoot(self, value):
+ if(self.q[1] > value):
+ self.q[1] = value
+ self._sink(1)
+
class RDD(object):
"""
A Resilient Distributed Dataset (RDD), the basic abstraction in Spark.
@@ -696,16 +763,16 @@ def top(self, num):
Note: It returns the list sorted in descending order.
>>> sc.parallelize([10, 4, 2, 12, 3]).top(1)
[12]
- >>> sc.parallelize([2, 3, 4, 5, 6]).cache().top(2)
+ >>> sc.parallelize([2, 3, 4, 5, 6], 2).cache().top(2)
[6, 5]
"""
def topIterator(iterator):
q = []
for k in iterator:
if len(q) < num:
- heappush(q, k)
+ heapq.heappush(q, k)
else:
- heappushpop(q, k)
+ heapq.heappushpop(q, k)
yield q
def merge(a, b):
@@ -713,6 +780,36 @@ def merge(a, b):
return sorted(self.mapPartitions(topIterator).reduce(merge), reverse=True)
+ def takeOrdered(self, num, key=None):
+ """
+ Get the N elements from a RDD ordered in ascending order or as specified
+ by the optional key function.
+
+ >>> sc.parallelize([10, 1, 2, 9, 3, 4, 5, 6, 7]).takeOrdered(6)
+ [1, 2, 3, 4, 5, 6]
+ >>> sc.parallelize([10, 1, 2, 9, 3, 4, 5, 6, 7], 2).takeOrdered(6, key=lambda x: -x)
+ [10, 9, 7, 6, 5, 4]
+ """
+
+ def topNKeyedElems(iterator, key_=None):
+ q = MaxHeapQ(num)
+ for k in iterator:
+ if key_ != None:
+ k = (key_(k), k)
+ q.insert(k)
+ yield q.getElements()
+
+ def unKey(x, key_=None):
+ if key_ != None:
+ x = [i[1] for i in x]
+ return x
+
+ def merge(a, b):
+ return next(topNKeyedElems(a + b))
+ result = self.mapPartitions(lambda i: topNKeyedElems(i, key)).reduce(merge)
+ return sorted(unKey(result, key), key=key)
+
+
def take(self, num):
"""
Take the first num elements of the RDD.
diff --git a/sql/README.md b/sql/README.md
index 4192fecb92fb0..14d5555f0c713 100644
--- a/sql/README.md
+++ b/sql/README.md
@@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.util._
import org.apache.spark.sql.execution
import org.apache.spark.sql.hive._
import org.apache.spark.sql.hive.TestHive._
-Welcome to Scala version 2.10.3 (Java HotSpot(TM) 64-Bit Server VM, Java 1.7.0_45).
+Welcome to Scala version 2.10.4 (Java HotSpot(TM) 64-Bit Server VM, Java 1.7.0_45).
Type in expressions to have them evaluated.
Type :help for more information.
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
index 976dda8d7e59a..446d0e0bd7f54 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
@@ -17,6 +17,8 @@
package org.apache.spark.sql.catalyst
+import java.sql.Timestamp
+
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.expressions.AttributeReference
import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
@@ -43,15 +45,26 @@ object ScalaReflection {
val params = t.member("": TermName).asMethod.paramss
StructType(
params.head.map(p => StructField(p.name.toString, schemaFor(p.typeSignature), true)))
+ // Need to decide if we actually need a special type here.
+ case t if t <:< typeOf[Array[Byte]] => BinaryType
+ case t if t <:< typeOf[Array[_]] =>
+ sys.error(s"Only Array[Byte] supported now, use Seq instead of $t")
case t if t <:< typeOf[Seq[_]] =>
val TypeRef(_, _, Seq(elementType)) = t
ArrayType(schemaFor(elementType))
+ case t if t <:< typeOf[Map[_,_]] =>
+ val TypeRef(_, _, Seq(keyType, valueType)) = t
+ MapType(schemaFor(keyType), schemaFor(valueType))
case t if t <:< typeOf[String] => StringType
+ case t if t <:< typeOf[Timestamp] => TimestampType
+ case t if t <:< typeOf[BigDecimal] => DecimalType
case t if t <:< definitions.IntTpe => IntegerType
case t if t <:< definitions.LongTpe => LongType
case t if t <:< definitions.DoubleTpe => DoubleType
+ case t if t <:< definitions.FloatTpe => FloatType
case t if t <:< definitions.ShortTpe => ShortType
case t if t <:< definitions.ByteTpe => ByteType
+ case t if t <:< definitions.BooleanTpe => BooleanType
}
implicit class CaseClassRelation[A <: Product : TypeTag](data: Seq[A]) {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala
index 0c851c2ee2183..4ea80fee23e1e 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala
@@ -106,6 +106,8 @@ class SqlParser extends StandardTokenParsers {
protected val IF = Keyword("IF")
protected val IN = Keyword("IN")
protected val INNER = Keyword("INNER")
+ protected val INSERT = Keyword("INSERT")
+ protected val INTO = Keyword("INTO")
protected val IS = Keyword("IS")
protected val JOIN = Keyword("JOIN")
protected val LEFT = Keyword("LEFT")
@@ -114,6 +116,7 @@ class SqlParser extends StandardTokenParsers {
protected val NULL = Keyword("NULL")
protected val ON = Keyword("ON")
protected val OR = Keyword("OR")
+ protected val OVERWRITE = Keyword("OVERWRITE")
protected val LIKE = Keyword("LIKE")
protected val RLIKE = Keyword("RLIKE")
protected val REGEXP = Keyword("REGEXP")
@@ -162,7 +165,7 @@ class SqlParser extends StandardTokenParsers {
select * (
UNION ~ ALL ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Union(q1, q2) } |
UNION ~ opt(DISTINCT) ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Distinct(Union(q1, q2)) }
- )
+ ) | insert
protected lazy val select: Parser[LogicalPlan] =
SELECT ~> opt(DISTINCT) ~ projections ~
@@ -181,10 +184,17 @@ class SqlParser extends StandardTokenParsers {
val withDistinct = d.map(_ => Distinct(withProjection)).getOrElse(withProjection)
val withHaving = h.map(h => Filter(h, withDistinct)).getOrElse(withDistinct)
val withOrder = o.map(o => Sort(o, withHaving)).getOrElse(withHaving)
- val withLimit = l.map { l => StopAfter(l, withOrder) }.getOrElse(withOrder)
+ val withLimit = l.map { l => Limit(l, withOrder) }.getOrElse(withOrder)
withLimit
}
+ protected lazy val insert: Parser[LogicalPlan] =
+ INSERT ~> opt(OVERWRITE) ~ inTo ~ select <~ opt(";") ^^ {
+ case o ~ r ~ s =>
+ val overwrite: Boolean = o.getOrElse("") == "OVERWRITE"
+ InsertIntoTable(r, Map[String, Option[String]](), s, overwrite)
+ }
+
protected lazy val projections: Parser[Seq[Expression]] = repsep(projection, ",")
protected lazy val projection: Parser[Expression] =
@@ -195,6 +205,8 @@ class SqlParser extends StandardTokenParsers {
protected lazy val from: Parser[LogicalPlan] = FROM ~> relations
+ protected lazy val inTo: Parser[LogicalPlan] = INTO ~> relation
+
// Based very loosely on the MySQL Grammar.
// http://dev.mysql.com/doc/refman/5.0/en/join.html
protected lazy val relations: Parser[LogicalPlan] =
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
index e09182dd8d5df..f30b5d816703a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
@@ -31,18 +31,33 @@ trait Catalog {
alias: Option[String] = None): LogicalPlan
def registerTable(databaseName: Option[String], tableName: String, plan: LogicalPlan): Unit
+
+ def unregisterTable(databaseName: Option[String], tableName: String): Unit
+
+ def unregisterAllTables(): Unit
}
class SimpleCatalog extends Catalog {
val tables = new mutable.HashMap[String, LogicalPlan]()
- def registerTable(databaseName: Option[String],tableName: String, plan: LogicalPlan): Unit = {
+ override def registerTable(
+ databaseName: Option[String],
+ tableName: String,
+ plan: LogicalPlan): Unit = {
tables += ((tableName, plan))
}
- def dropTable(tableName: String) = tables -= tableName
+ override def unregisterTable(
+ databaseName: Option[String],
+ tableName: String) = {
+ tables -= tableName
+ }
- def lookupRelation(
+ override def unregisterAllTables() = {
+ tables.clear()
+ }
+
+ override def lookupRelation(
databaseName: Option[String],
tableName: String,
alias: Option[String] = None): LogicalPlan = {
@@ -87,6 +102,14 @@ trait OverrideCatalog extends Catalog {
plan: LogicalPlan): Unit = {
overrides.put((databaseName, tableName), plan)
}
+
+ override def unregisterTable(databaseName: Option[String], tableName: String): Unit = {
+ overrides.remove((databaseName, tableName))
+ }
+
+ override def unregisterAllTables(): Unit = {
+ overrides.clear()
+ }
}
/**
@@ -104,4 +127,10 @@ object EmptyCatalog extends Catalog {
def registerTable(databaseName: Option[String], tableName: String, plan: LogicalPlan): Unit = {
throw new UnsupportedOperationException
}
+
+ def unregisterTable(databaseName: Option[String], tableName: String): Unit = {
+ throw new UnsupportedOperationException
+ }
+
+ override def unregisterAllTables(): Unit = {}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
index 44abe671c07a4..2c4bf1715b646 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
@@ -17,6 +17,8 @@
package org.apache.spark.sql.catalyst
+import java.sql.Timestamp
+
import scala.language.implicitConversions
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
@@ -72,6 +74,7 @@ package object dsl {
def like(other: Expression) = Like(expr, other)
def rlike(other: Expression) = RLike(expr, other)
+ def cast(to: DataType) = Cast(expr, to)
def asc = SortOrder(expr, Ascending)
def desc = SortOrder(expr, Descending)
@@ -84,15 +87,22 @@ package object dsl {
def expr = e
}
+ implicit def booleanToLiteral(b: Boolean) = Literal(b)
+ implicit def byteToLiteral(b: Byte) = Literal(b)
+ implicit def shortToLiteral(s: Short) = Literal(s)
implicit def intToLiteral(i: Int) = Literal(i)
implicit def longToLiteral(l: Long) = Literal(l)
implicit def floatToLiteral(f: Float) = Literal(f)
implicit def doubleToLiteral(d: Double) = Literal(d)
implicit def stringToLiteral(s: String) = Literal(s)
+ implicit def decimalToLiteral(d: BigDecimal) = Literal(d)
+ implicit def timestampToLiteral(t: Timestamp) = Literal(t)
+ implicit def binaryToLiteral(a: Array[Byte]) = Literal(a)
implicit def symbolToUnresolvedAttribute(s: Symbol) = analysis.UnresolvedAttribute(s.name)
implicit class DslSymbol(sym: Symbol) extends ImplicitAttribute { def s = sym.name }
+ // TODO more implicit class for literal?
implicit class DslString(val s: String) extends ImplicitOperators {
def expr: Expression = Literal(s)
def attr = analysis.UnresolvedAttribute(s)
@@ -103,11 +113,38 @@ package object dsl {
def expr = attr
def attr = analysis.UnresolvedAttribute(s)
- /** Creates a new typed attributes of type int */
+ /** Creates a new AttributeReference of type boolean */
+ def boolean = AttributeReference(s, BooleanType, nullable = false)()
+
+ /** Creates a new AttributeReference of type byte */
+ def byte = AttributeReference(s, ByteType, nullable = false)()
+
+ /** Creates a new AttributeReference of type short */
+ def short = AttributeReference(s, ShortType, nullable = false)()
+
+ /** Creates a new AttributeReference of type int */
def int = AttributeReference(s, IntegerType, nullable = false)()
- /** Creates a new typed attributes of type string */
+ /** Creates a new AttributeReference of type long */
+ def long = AttributeReference(s, LongType, nullable = false)()
+
+ /** Creates a new AttributeReference of type float */
+ def float = AttributeReference(s, FloatType, nullable = false)()
+
+ /** Creates a new AttributeReference of type double */
+ def double = AttributeReference(s, DoubleType, nullable = false)()
+
+ /** Creates a new AttributeReference of type string */
def string = AttributeReference(s, StringType, nullable = false)()
+
+ /** Creates a new AttributeReference of type decimal */
+ def decimal = AttributeReference(s, DecimalType, nullable = false)()
+
+ /** Creates a new AttributeReference of type timestamp */
+ def timestamp = AttributeReference(s, TimestampType, nullable = false)()
+
+ /** Creates a new AttributeReference of type binary */
+ def binary = AttributeReference(s, BinaryType, nullable = false)()
}
implicit class DslAttribute(a: AttributeReference) {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
index c26fc3d0f305f..941b53fe70d23 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
@@ -17,6 +17,8 @@
package org.apache.spark.sql.catalyst.expressions
+import java.sql.Timestamp
+
import org.apache.spark.sql.catalyst.types._
/** Cast the child expression to the target data type. */
@@ -26,52 +28,169 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression {
override def toString = s"CAST($child, $dataType)"
type EvaluatedType = Any
+
+ def nullOrCast[T](a: Any, func: T => Any): Any = if(a == null) {
+ null
+ } else {
+ func(a.asInstanceOf[T])
+ }
- lazy val castingFunction: Any => Any = (child.dataType, dataType) match {
- case (BinaryType, StringType) => a: Any => new String(a.asInstanceOf[Array[Byte]])
- case (StringType, BinaryType) => a: Any => a.asInstanceOf[String].getBytes
- case (_, StringType) => a: Any => a.toString
- case (StringType, IntegerType) => a: Any => castOrNull(a, _.toInt)
- case (StringType, DoubleType) => a: Any => castOrNull(a, _.toDouble)
- case (StringType, FloatType) => a: Any => castOrNull(a, _.toFloat)
- case (StringType, LongType) => a: Any => castOrNull(a, _.toLong)
- case (StringType, ShortType) => a: Any => castOrNull(a, _.toShort)
- case (StringType, ByteType) => a: Any => castOrNull(a, _.toByte)
- case (StringType, DecimalType) => a: Any => castOrNull(a, BigDecimal(_))
- case (BooleanType, ByteType) => {
- case null => null
- case true => 1.toByte
- case false => 0.toByte
- }
- case (dt, IntegerType) =>
- a: Any => dt.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].toInt(a)
- case (dt, DoubleType) =>
- a: Any => dt.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].toDouble(a)
- case (dt, FloatType) =>
- a: Any => dt.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].toFloat(a)
- case (dt, LongType) =>
- a: Any => dt.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].toLong(a)
- case (dt, ShortType) =>
- a: Any => dt.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].toInt(a).toShort
- case (dt, ByteType) =>
- a: Any => dt.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].toInt(a).toByte
- case (dt, DecimalType) =>
- a: Any =>
- BigDecimal(dt.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].toDouble(a))
+ // UDFToString
+ def castToString: Any => Any = child.dataType match {
+ case BinaryType => nullOrCast[Array[Byte]](_, new String(_, "UTF-8"))
+ case _ => nullOrCast[Any](_, _.toString)
+ }
+
+ // BinaryConverter
+ def castToBinary: Any => Any = child.dataType match {
+ case StringType => nullOrCast[String](_, _.getBytes("UTF-8"))
}
- @inline
- protected def castOrNull[A](a: Any, f: String => A) =
- try f(a.asInstanceOf[String]) catch {
- case _: java.lang.NumberFormatException => null
- }
+ // UDFToBoolean
+ def castToBoolean: Any => Any = child.dataType match {
+ case StringType => nullOrCast[String](_, _.length() != 0)
+ case TimestampType => nullOrCast[Timestamp](_, b => {(b.getTime() != 0 || b.getNanos() != 0)})
+ case LongType => nullOrCast[Long](_, _ != 0)
+ case IntegerType => nullOrCast[Int](_, _ != 0)
+ case ShortType => nullOrCast[Short](_, _ != 0)
+ case ByteType => nullOrCast[Byte](_, _ != 0)
+ case DecimalType => nullOrCast[BigDecimal](_, _ != 0)
+ case DoubleType => nullOrCast[Double](_, _ != 0)
+ case FloatType => nullOrCast[Float](_, _ != 0)
+ }
+
+ // TimestampConverter
+ def castToTimestamp: Any => Any = child.dataType match {
+ case StringType => nullOrCast[String](_, s => {
+ // Throw away extra if more than 9 decimal places
+ val periodIdx = s.indexOf(".");
+ var n = s
+ if (periodIdx != -1) {
+ if (n.length() - periodIdx > 9) {
+ n = n.substring(0, periodIdx + 10)
+ }
+ }
+ try Timestamp.valueOf(n) catch { case _: java.lang.IllegalArgumentException => null}
+ })
+ case BooleanType => nullOrCast[Boolean](_, b => new Timestamp((if(b) 1 else 0) * 1000))
+ case LongType => nullOrCast[Long](_, l => new Timestamp(l * 1000))
+ case IntegerType => nullOrCast[Int](_, i => new Timestamp(i * 1000))
+ case ShortType => nullOrCast[Short](_, s => new Timestamp(s * 1000))
+ case ByteType => nullOrCast[Byte](_, b => new Timestamp(b * 1000))
+ // TimestampWritable.decimalToTimestamp
+ case DecimalType => nullOrCast[BigDecimal](_, d => decimalToTimestamp(d))
+ // TimestampWritable.doubleToTimestamp
+ case DoubleType => nullOrCast[Double](_, d => decimalToTimestamp(d))
+ // TimestampWritable.floatToTimestamp
+ case FloatType => nullOrCast[Float](_, f => decimalToTimestamp(f))
+ }
+
+ private def decimalToTimestamp(d: BigDecimal) = {
+ val seconds = d.longValue()
+ val bd = (d - seconds) * (1000000000)
+ val nanos = bd.intValue()
+
+ // Convert to millis
+ val millis = seconds * 1000
+ val t = new Timestamp(millis)
+
+ // remaining fractional portion as nanos
+ t.setNanos(nanos)
+
+ t
+ }
+
+ private def timestampToDouble(t: Timestamp) = (t.getSeconds() + t.getNanos().toDouble / 1000)
+
+ def castToLong: Any => Any = child.dataType match {
+ case StringType => nullOrCast[String](_, s => try s.toLong catch {
+ case _: NumberFormatException => null
+ })
+ case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0)
+ case TimestampType => nullOrCast[Timestamp](_, t => timestampToDouble(t).toLong)
+ case DecimalType => nullOrCast[BigDecimal](_, _.toLong)
+ case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toLong(b)
+ }
+
+ def castToInt: Any => Any = child.dataType match {
+ case StringType => nullOrCast[String](_, s => try s.toInt catch {
+ case _: NumberFormatException => null
+ })
+ case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0)
+ case TimestampType => nullOrCast[Timestamp](_, t => timestampToDouble(t).toInt)
+ case DecimalType => nullOrCast[BigDecimal](_, _.toInt)
+ case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b)
+ }
+
+ def castToShort: Any => Any = child.dataType match {
+ case StringType => nullOrCast[String](_, s => try s.toShort catch {
+ case _: NumberFormatException => null
+ })
+ case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0)
+ case TimestampType => nullOrCast[Timestamp](_, t => timestampToDouble(t).toShort)
+ case DecimalType => nullOrCast[BigDecimal](_, _.toShort)
+ case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b).toShort
+ }
+
+ def castToByte: Any => Any = child.dataType match {
+ case StringType => nullOrCast[String](_, s => try s.toByte catch {
+ case _: NumberFormatException => null
+ })
+ case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0)
+ case TimestampType => nullOrCast[Timestamp](_, t => timestampToDouble(t).toByte)
+ case DecimalType => nullOrCast[BigDecimal](_, _.toByte)
+ case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b).toByte
+ }
+
+ def castToDecimal: Any => Any = child.dataType match {
+ case StringType => nullOrCast[String](_, s => try BigDecimal(s.toDouble) catch {
+ case _: NumberFormatException => null
+ })
+ case BooleanType => nullOrCast[Boolean](_, b => if(b) BigDecimal(1) else BigDecimal(0))
+ case TimestampType => nullOrCast[Timestamp](_, t => BigDecimal(timestampToDouble(t)))
+ case x: NumericType => b => BigDecimal(x.numeric.asInstanceOf[Numeric[Any]].toDouble(b))
+ }
+
+ def castToDouble: Any => Any = child.dataType match {
+ case StringType => nullOrCast[String](_, s => try s.toDouble catch {
+ case _: NumberFormatException => null
+ })
+ case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0)
+ case TimestampType => nullOrCast[Timestamp](_, t => timestampToDouble(t))
+ case DecimalType => nullOrCast[BigDecimal](_, _.toDouble)
+ case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toDouble(b)
+ }
+
+ def castToFloat: Any => Any = child.dataType match {
+ case StringType => nullOrCast[String](_, s => try s.toFloat catch {
+ case _: NumberFormatException => null
+ })
+ case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0)
+ case TimestampType => nullOrCast[Timestamp](_, t => timestampToDouble(t).toFloat)
+ case DecimalType => nullOrCast[BigDecimal](_, _.toFloat)
+ case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toFloat(b)
+ }
+
+ def cast: Any => Any = dataType match {
+ case StringType => castToString
+ case BinaryType => castToBinary
+ case DecimalType => castToDecimal
+ case TimestampType => castToTimestamp
+ case BooleanType => castToBoolean
+ case ByteType => castToByte
+ case ShortType => castToShort
+ case IntegerType => castToInt
+ case FloatType => castToFloat
+ case LongType => castToLong
+ case DoubleType => castToDouble
+ }
override def apply(input: Row): Any = {
val evaluated = child.apply(input)
if (evaluated == null) {
null
} else {
- castingFunction(evaluated)
+ cast(evaluated)
}
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
index 81fd160e00ca1..a3d19525503ba 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
@@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.trees
import org.apache.spark.sql.catalyst.errors.TreeNodeException
import org.apache.spark.sql.catalyst.trees.TreeNode
-import org.apache.spark.sql.catalyst.types.{DataType, FractionalType, IntegralType, NumericType}
+import org.apache.spark.sql.catalyst.types.{DataType, FractionalType, IntegralType, NumericType, NativeType}
abstract class Expression extends TreeNode[Expression] {
self: Product =>
@@ -86,6 +86,11 @@ abstract class Expression extends TreeNode[Expression] {
}
}
+ /**
+ * Evaluation helper function for 2 Numeric children expressions. Those expressions are supposed
+ * to be in the same data type, and also the return type.
+ * Either one of the expressions result is null, the evaluation result should be null.
+ */
@inline
protected final def n2(
i: Row,
@@ -115,6 +120,11 @@ abstract class Expression extends TreeNode[Expression] {
}
}
+ /**
+ * Evaluation helper function for 2 Fractional children expressions. Those expressions are
+ * supposed to be in the same data type, and also the return type.
+ * Either one of the expressions result is null, the evaluation result should be null.
+ */
@inline
protected final def f2(
i: Row,
@@ -143,6 +153,11 @@ abstract class Expression extends TreeNode[Expression] {
}
}
+ /**
+ * Evaluation helper function for 2 Integral children expressions. Those expressions are
+ * supposed to be in the same data type, and also the return type.
+ * Either one of the expressions result is null, the evaluation result should be null.
+ */
@inline
protected final def i2(
i: Row,
@@ -170,6 +185,43 @@ abstract class Expression extends TreeNode[Expression] {
}
}
}
+
+ /**
+ * Evaluation helper function for 2 Comparable children expressions. Those expressions are
+ * supposed to be in the same data type, and the return type should be Integer:
+ * Negative value: 1st argument less than 2nd argument
+ * Zero: 1st argument equals 2nd argument
+ * Positive value: 1st argument greater than 2nd argument
+ *
+ * Either one of the expressions result is null, the evaluation result should be null.
+ */
+ @inline
+ protected final def c2(
+ i: Row,
+ e1: Expression,
+ e2: Expression,
+ f: ((Ordering[Any], Any, Any) => Any)): Any = {
+ if (e1.dataType != e2.dataType) {
+ throw new TreeNodeException(this, s"Types do not match ${e1.dataType} != ${e2.dataType}")
+ }
+
+ val evalE1 = e1.apply(i)
+ if(evalE1 == null) {
+ null
+ } else {
+ val evalE2 = e2.apply(i)
+ if (evalE2 == null) {
+ null
+ } else {
+ e1.dataType match {
+ case i: NativeType =>
+ f.asInstanceOf[(Ordering[i.JvmType], i.JvmType, i.JvmType) => Boolean](
+ i.ordering, evalE1.asInstanceOf[i.JvmType], evalE2.asInstanceOf[i.JvmType])
+ case other => sys.error(s"Type $other does not support ordered operations")
+ }
+ }
+ }
+ }
}
abstract class BinaryExpression extends Expression with trees.BinaryNode[Expression] {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala
index 31d42b9ee71a0..6f939e6c41f6b 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala
@@ -44,6 +44,16 @@ trait Row extends Seq[Any] with Serializable {
s"[${this.mkString(",")}]"
def copy(): Row
+
+ /** Returns true if there are any NULL values in this row. */
+ def anyNull: Boolean = {
+ var i = 0
+ while (i < length) {
+ if (isNullAt(i)) { return true }
+ i += 1
+ }
+ false
+ }
}
/**
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
index b82a12e0f754e..d879b2b5e8ba1 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
@@ -17,6 +17,8 @@
package org.apache.spark.sql.catalyst.expressions
+import java.sql.Timestamp
+
import org.apache.spark.sql.catalyst.types._
object Literal {
@@ -29,6 +31,9 @@ object Literal {
case s: Short => Literal(s, ShortType)
case s: String => Literal(s, StringType)
case b: Boolean => Literal(b, BooleanType)
+ case d: BigDecimal => Literal(d, DecimalType)
+ case t: Timestamp => Literal(t, TimestampType)
+ case a: Array[Byte] => Literal(a, BinaryType)
case null => Literal(null, NullType)
}
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
index 722ff517d250e..b74809e5ca67d 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
@@ -18,8 +18,15 @@
package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.trees
+import org.apache.spark.sql.catalyst.errors.TreeNodeException
import org.apache.spark.sql.catalyst.analysis.UnresolvedException
-import org.apache.spark.sql.catalyst.types.{BooleanType, StringType}
+import org.apache.spark.sql.catalyst.types.{BooleanType, StringType, TimestampType}
+
+object InterpretedPredicate {
+ def apply(expression: Expression): (Row => Boolean) = {
+ (r: Row) => expression.apply(r).asInstanceOf[Boolean]
+ }
+}
trait Predicate extends Expression {
self: Product =>
@@ -117,70 +124,22 @@ case class Equals(left: Expression, right: Expression) extends BinaryComparison
case class LessThan(left: Expression, right: Expression) extends BinaryComparison {
def symbol = "<"
- override def apply(input: Row): Any = {
- if (left.dataType == StringType && right.dataType == StringType) {
- val l = left.apply(input)
- val r = right.apply(input)
- if(l == null || r == null) {
- null
- } else {
- l.asInstanceOf[String] < r.asInstanceOf[String]
- }
- } else {
- n2(input, left, right, _.lt(_, _))
- }
- }
+ override def apply(input: Row): Any = c2(input, left, right, _.lt(_, _))
}
case class LessThanOrEqual(left: Expression, right: Expression) extends BinaryComparison {
def symbol = "<="
- override def apply(input: Row): Any = {
- if (left.dataType == StringType && right.dataType == StringType) {
- val l = left.apply(input)
- val r = right.apply(input)
- if(l == null || r == null) {
- null
- } else {
- l.asInstanceOf[String] <= r.asInstanceOf[String]
- }
- } else {
- n2(input, left, right, _.lteq(_, _))
- }
- }
+ override def apply(input: Row): Any = c2(input, left, right, _.lteq(_, _))
}
case class GreaterThan(left: Expression, right: Expression) extends BinaryComparison {
def symbol = ">"
- override def apply(input: Row): Any = {
- if (left.dataType == StringType && right.dataType == StringType) {
- val l = left.apply(input)
- val r = right.apply(input)
- if(l == null || r == null) {
- null
- } else {
- l.asInstanceOf[String] > r.asInstanceOf[String]
- }
- } else {
- n2(input, left, right, _.gt(_, _))
- }
- }
+ override def apply(input: Row): Any = c2(input, left, right, _.gt(_, _))
}
case class GreaterThanOrEqual(left: Expression, right: Expression) extends BinaryComparison {
def symbol = ">="
- override def apply(input: Row): Any = {
- if (left.dataType == StringType && right.dataType == StringType) {
- val l = left.apply(input)
- val r = right.apply(input)
- if(l == null || r == null) {
- null
- } else {
- l.asInstanceOf[String] >= r.asInstanceOf[String]
- }
- } else {
- n2(input, left, right, _.gteq(_, _))
- }
- }
+ override def apply(input: Row): Any = c2(input, left, right, _.gteq(_, _))
}
case class If(predicate: Expression, trueValue: Expression, falseValue: Expression)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
index 9d16189deedfe..b39c2b32cc42c 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
@@ -130,7 +130,7 @@ case class Aggregate(
def references = child.references
}
-case class StopAfter(limit: Expression, child: LogicalPlan) extends UnaryNode {
+case class Limit(limit: Expression, child: LogicalPlan) extends UnaryNode {
def output = child.output
def references = limit.references
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala
index 7a45d1a1b8195..cdeb01a9656f4 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala
@@ -17,6 +17,8 @@
package org.apache.spark.sql.catalyst.types
+import java.sql.Timestamp
+
import scala.reflect.runtime.universe.{typeTag, TypeTag}
import org.apache.spark.sql.catalyst.expressions.Expression
@@ -51,6 +53,16 @@ case object BooleanType extends NativeType {
val ordering = implicitly[Ordering[JvmType]]
}
+case object TimestampType extends NativeType {
+ type JvmType = Timestamp
+
+ @transient lazy val tag = typeTag[JvmType]
+
+ val ordering = new Ordering[JvmType] {
+ def compare(x: Timestamp, y: Timestamp) = x.compareTo(y)
+ }
+}
+
abstract class NumericType extends NativeType {
// Unfortunately we can't get this implicitly as that breaks Spark Serialization. In order for
// implicitly[Numeric[JvmType]] to be valid, we have to change JvmType from a type variable to a
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala
index 52a205be3e9f4..43876033d327b 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala
@@ -17,6 +17,8 @@
package org.apache.spark.sql.catalyst.expressions
+import java.sql.Timestamp
+
import org.scalatest.FunSuite
import org.apache.spark.sql.catalyst.types._
@@ -191,5 +193,56 @@ class ExpressionEvaluationSuite extends FunSuite {
evaluate("abbbbc" rlike regEx, new GenericRow(Array[Any]("**")))
}
}
+
+ test("data type casting") {
+
+ val sts = "1970-01-01 00:00:01.0"
+ val ts = Timestamp.valueOf(sts)
+
+ checkEvaluation("abdef" cast StringType, "abdef")
+ checkEvaluation("abdef" cast DecimalType, null)
+ checkEvaluation("abdef" cast TimestampType, null)
+ checkEvaluation("12.65" cast DecimalType, BigDecimal(12.65))
+
+ checkEvaluation(Literal(1) cast LongType, 1)
+ checkEvaluation(Cast(Literal(1) cast TimestampType, LongType), 1)
+ checkEvaluation(Cast(Literal(BigDecimal(1)) cast TimestampType, DecimalType), 1)
+ checkEvaluation(Cast(Literal(1.toDouble) cast TimestampType, DoubleType), 1.toDouble)
+
+ checkEvaluation(Cast(Literal(sts) cast TimestampType, StringType), sts)
+ checkEvaluation(Cast(Literal(ts) cast StringType, TimestampType), ts)
+
+ checkEvaluation(Cast("abdef" cast BinaryType, StringType), "abdef")
+
+ checkEvaluation(Cast(Cast(Cast(Cast(
+ Cast("5" cast ByteType, ShortType), IntegerType), FloatType), DoubleType), LongType), 5)
+ checkEvaluation(Cast(Cast(Cast(Cast(
+ Cast("5" cast ByteType, TimestampType), DecimalType), LongType), StringType), ShortType), 5)
+ checkEvaluation(Cast(Cast(Cast(Cast(
+ Cast("5" cast TimestampType, ByteType), DecimalType), LongType), StringType), ShortType), null)
+ checkEvaluation(Cast(Cast(Cast(Cast(
+ Cast("5" cast DecimalType, ByteType), TimestampType), LongType), StringType), ShortType), 5)
+ checkEvaluation(Literal(true) cast IntegerType, 1)
+ checkEvaluation(Literal(false) cast IntegerType, 0)
+ checkEvaluation(Cast(Literal(1) cast BooleanType, IntegerType), 1)
+ checkEvaluation(Cast(Literal(0) cast BooleanType, IntegerType), 0)
+ checkEvaluation("23" cast DoubleType, 23)
+ checkEvaluation("23" cast IntegerType, 23)
+ checkEvaluation("23" cast FloatType, 23)
+ checkEvaluation("23" cast DecimalType, 23)
+ checkEvaluation("23" cast ByteType, 23)
+ checkEvaluation("23" cast ShortType, 23)
+ checkEvaluation("2012-12-11" cast DoubleType, null)
+ checkEvaluation(Literal(123) cast IntegerType, 123)
+
+ intercept[Exception] {evaluate(Literal(1) cast BinaryType, null)}
+ }
+
+ test("timestamp") {
+ val ts1 = new Timestamp(12)
+ val ts2 = new Timestamp(123)
+ checkEvaluation(Literal("ab") < Literal("abc"), true)
+ checkEvaluation(Literal(ts1) < Literal(ts2), true)
+ }
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
index cf3c06acce5b0..36059c6630aa4 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala
@@ -26,8 +26,9 @@ import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.dsl
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.optimizer.Optimizer
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.{Subquery, LogicalPlan}
import org.apache.spark.sql.catalyst.rules.RuleExecutor
+import org.apache.spark.sql.columnar.InMemoryColumnarTableScan
import org.apache.spark.sql.execution._
/**
@@ -79,12 +80,12 @@ class SQLContext(@transient val sparkContext: SparkContext)
new SchemaRDD(this, SparkLogicalPlan(ExistingRdd.fromProductRdd(rdd)))
/**
- * Loads a parequet file, returning the result as a [[SchemaRDD]].
+ * Loads a Parquet file, returning the result as a [[SchemaRDD]].
*
* @group userf
*/
def parquetFile(path: String): SchemaRDD =
- new SchemaRDD(this, parquet.ParquetRelation("ParquetFile", path))
+ new SchemaRDD(this, parquet.ParquetRelation(path))
/**
@@ -111,13 +112,42 @@ class SQLContext(@transient val sparkContext: SparkContext)
result
}
+ /** Returns the specified table as a SchemaRDD */
+ def table(tableName: String): SchemaRDD =
+ new SchemaRDD(this, catalog.lookupRelation(None, tableName))
+
+ /** Caches the specified table in-memory. */
+ def cacheTable(tableName: String): Unit = {
+ val currentTable = catalog.lookupRelation(None, tableName)
+ val asInMemoryRelation =
+ InMemoryColumnarTableScan(currentTable.output, executePlan(currentTable).executedPlan)
+
+ catalog.registerTable(None, tableName, SparkLogicalPlan(asInMemoryRelation))
+ }
+
+ /** Removes the specified table from the in-memory cache. */
+ def uncacheTable(tableName: String): Unit = {
+ EliminateAnalysisOperators(catalog.lookupRelation(None, tableName)) match {
+ // This is kind of a hack to make sure that if this was just an RDD registered as a table,
+ // we reregister the RDD as a table.
+ case SparkLogicalPlan(inMem @ InMemoryColumnarTableScan(_, e: ExistingRdd)) =>
+ inMem.cachedColumnBuffers.unpersist()
+ catalog.unregisterTable(None, tableName)
+ catalog.registerTable(None, tableName, SparkLogicalPlan(e))
+ case SparkLogicalPlan(inMem: InMemoryColumnarTableScan) =>
+ inMem.cachedColumnBuffers.unpersist()
+ catalog.unregisterTable(None, tableName)
+ case plan => throw new IllegalArgumentException(s"Table $tableName is not cached: $plan")
+ }
+ }
+
protected[sql] class SparkPlanner extends SparkStrategies {
val sparkContext = self.sparkContext
val strategies: Seq[Strategy] =
- TopK ::
+ TakeOrdered ::
PartialAggregation ::
- SparkEquiInnerJoin ::
+ HashJoin ::
ParquetOperations ::
BasicOperators ::
CartesianProduct ::
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala
index 770cabcb31d13..a62cb8aa1321f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala
@@ -17,13 +17,13 @@
package org.apache.spark.sql
+import org.apache.spark.{Dependency, OneToOneDependency, Partition, TaskContext}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.plans.{Inner, JoinType}
import org.apache.spark.sql.catalyst.types.BooleanType
-import org.apache.spark.{Dependency, OneToOneDependency, Partition, TaskContext}
/**
* ALPHA COMPONENT
@@ -92,23 +92,10 @@ import org.apache.spark.{Dependency, OneToOneDependency, Partition, TaskContext}
*/
class SchemaRDD(
@transient val sqlContext: SQLContext,
- @transient val logicalPlan: LogicalPlan)
- extends RDD[Row](sqlContext.sparkContext, Nil) {
+ @transient protected[spark] val logicalPlan: LogicalPlan)
+ extends RDD[Row](sqlContext.sparkContext, Nil) with SchemaRDDLike {
- /**
- * A lazily computed query execution workflow. All other RDD operations are passed
- * through to the RDD that is produced by this workflow.
- *
- * We want this to be lazy because invoking the whole query optimization pipeline can be
- * expensive.
- */
- @transient
- protected[spark] lazy val queryExecution = sqlContext.executePlan(logicalPlan)
-
- override def toString =
- s"""${super.toString}
- |== Query Plan ==
- |${queryExecution.executedPlan}""".stripMargin.trim
+ def baseSchemaRDD = this
// =========================================================================================
// RDD functions: Copy the interal row representation so we present immutable data to users.
@@ -312,31 +299,12 @@ class SchemaRDD(
sqlContext,
InsertIntoTable(UnresolvedRelation(None, tableName), Map.empty, logicalPlan, overwrite))
- /**
- * Saves the contents of this `SchemaRDD` as a parquet file, preserving the schema. Files that
- * are written out using this method can be read back in as a SchemaRDD using the ``function
- *
- * @group schema
- */
- def saveAsParquetFile(path: String): Unit = {
- sqlContext.executePlan(WriteToFile(path, logicalPlan)).toRdd
- }
-
- /**
- * Registers this RDD as a temporary table using the given name. The lifetime of this temporary
- * table is tied to the [[SQLContext]] that was used to create this SchemaRDD.
- *
- * @group schema
- */
- def registerAsTable(tableName: String): Unit = {
- sqlContext.registerRDDAsTable(this, tableName)
- }
-
/**
* Returns this RDD as a SchemaRDD.
* @group schema
*/
def toSchemaRDD = this
+ /** FOR INTERNAL USE ONLY */
def analyze = sqlContext.analyzer(logicalPlan)
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala
new file mode 100644
index 0000000000000..840803a52c1cf
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala
@@ -0,0 +1,66 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements. See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.catalyst.plans.logical._
+
+/**
+ * Contains functions that are shared between all SchemaRDD types (i.e., Scala, Java)
+ */
+trait SchemaRDDLike {
+ @transient val sqlContext: SQLContext
+ @transient protected[spark] val logicalPlan: LogicalPlan
+
+ private[sql] def baseSchemaRDD: SchemaRDD
+
+ /**
+ * A lazily computed query execution workflow. All other RDD operations are passed
+ * through to the RDD that is produced by this workflow.
+ *
+ * We want this to be lazy because invoking the whole query optimization pipeline can be
+ * expensive.
+ */
+ @transient
+ protected[spark] lazy val queryExecution = sqlContext.executePlan(logicalPlan)
+
+ override def toString =
+ s"""${super.toString}
+ |== Query Plan ==
+ |${queryExecution.executedPlan}""".stripMargin.trim
+
+
+ /**
+ * Saves the contents of this `SchemaRDD` as a parquet file, preserving the schema. Files that
+ * are written out using this method can be read back in as a SchemaRDD using the ``function
+ *
+ * @group schema
+ */
+ def saveAsParquetFile(path: String): Unit = {
+ sqlContext.executePlan(WriteToFile(path, logicalPlan)).toRdd
+ }
+
+ /**
+ * Registers this RDD as a temporary table using the given name. The lifetime of this temporary
+ * table is tied to the [[SQLContext]] that was used to create this SchemaRDD.
+ *
+ * @group schema
+ */
+ def registerAsTable(tableName: String): Unit = {
+ sqlContext.registerRDDAsTable(baseSchemaRDD, tableName)
+ }
+}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala
new file mode 100644
index 0000000000000..573345e42c43c
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala
@@ -0,0 +1,100 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements. See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+
+package org.apache.spark.sql.api.java
+
+import java.beans.{Introspector, PropertyDescriptor}
+
+import org.apache.spark.api.java.{JavaRDD, JavaSparkContext}
+import org.apache.spark.sql.SQLContext
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRow, Row => ScalaRow}
+import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.parquet.ParquetRelation
+import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan}
+
+/**
+ * The entry point for executing Spark SQL queries from a Java program.
+ */
+class JavaSQLContext(sparkContext: JavaSparkContext) {
+
+ val sqlContext = new SQLContext(sparkContext.sc)
+
+ /**
+ * Executes a query expressed in SQL, returning the result as a JavaSchemaRDD
+ */
+ def sql(sqlQuery: String): JavaSchemaRDD = {
+ val result = new JavaSchemaRDD(sqlContext, sqlContext.parseSql(sqlQuery))
+ // We force query optimization to happen right away instead of letting it happen lazily like
+ // when using the query DSL. This is so DDL commands behave as expected. This is only
+ // generates the RDD lineage for DML queries, but do not perform any execution.
+ result.queryExecution.toRdd
+ result
+ }
+
+ /**
+ * Applies a schema to an RDD of Java Beans.
+ */
+ def applySchema(rdd: JavaRDD[_], beanClass: Class[_]): JavaSchemaRDD = {
+ // TODO: All of this could probably be moved to Catalyst as it is mostly not Spark specific.
+ val beanInfo = Introspector.getBeanInfo(beanClass)
+
+ val fields = beanInfo.getPropertyDescriptors.filterNot(_.getName == "class")
+ val schema = fields.map { property =>
+ val dataType = property.getPropertyType match {
+ case c: Class[_] if c == classOf[java.lang.String] => StringType
+ case c: Class[_] if c == java.lang.Short.TYPE => ShortType
+ case c: Class[_] if c == java.lang.Integer.TYPE => IntegerType
+ case c: Class[_] if c == java.lang.Long.TYPE => LongType
+ case c: Class[_] if c == java.lang.Double.TYPE => DoubleType
+ case c: Class[_] if c == java.lang.Byte.TYPE => ByteType
+ case c: Class[_] if c == java.lang.Float.TYPE => FloatType
+ case c: Class[_] if c == java.lang.Boolean.TYPE => BooleanType
+ }
+
+ AttributeReference(property.getName, dataType, true)()
+ }
+
+ val className = beanClass.getCanonicalName
+ val rowRdd = rdd.rdd.mapPartitions { iter =>
+ // BeanInfo is not serializable so we must rediscover it remotely for each partition.
+ val localBeanInfo = Introspector.getBeanInfo(Class.forName(className))
+ val extractors =
+ localBeanInfo.getPropertyDescriptors.filterNot(_.getName == "class").map(_.getReadMethod)
+
+ iter.map { row =>
+ new GenericRow(extractors.map(e => e.invoke(row)).toArray[Any]): ScalaRow
+ }
+ }
+ new JavaSchemaRDD(sqlContext, SparkLogicalPlan(ExistingRdd(schema, rowRdd)))
+ }
+
+
+ /**
+ * Loads a parquet file, returning the result as a [[JavaSchemaRDD]].
+ */
+ def parquetFile(path: String): JavaSchemaRDD =
+ new JavaSchemaRDD(sqlContext, ParquetRelation(path))
+
+
+ /**
+ * Registers the given RDD as a temporary table in the catalog. Temporary tables exist only
+ * during the lifetime of this instance of SQLContext.
+ */
+ def registerRDDAsTable(rdd: JavaSchemaRDD, tableName: String): Unit = {
+ sqlContext.registerRDDAsTable(rdd.baseSchemaRDD, tableName)
+ }
+}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala
new file mode 100644
index 0000000000000..d43d672938f51
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.api.java
+
+import org.apache.spark.api.java.{JavaRDDLike, JavaRDD}
+import org.apache.spark.sql.{SQLContext, SchemaRDD, SchemaRDDLike}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.rdd.RDD
+
+/**
+ * An RDD of [[Row]] objects that is returned as the result of a Spark SQL query. In addition to
+ * standard RDD operations, a JavaSchemaRDD can also be registered as a table in the JavaSQLContext
+ * that was used to create. Registering a JavaSchemaRDD allows its contents to be queried in
+ * future SQL statement.
+ *
+ * @groupname schema SchemaRDD Functions
+ * @groupprio schema -1
+ * @groupname Ungrouped Base RDD Functions
+ */
+class JavaSchemaRDD(
+ @transient val sqlContext: SQLContext,
+ @transient protected[spark] val logicalPlan: LogicalPlan)
+ extends JavaRDDLike[Row, JavaRDD[Row]]
+ with SchemaRDDLike {
+
+ private[sql] val baseSchemaRDD = new SchemaRDD(sqlContext, logicalPlan)
+
+ override val classTag = scala.reflect.classTag[Row]
+
+ override def wrapRDD(rdd: RDD[Row]): JavaRDD[Row] = JavaRDD.fromRDD(rdd)
+
+ val rdd = baseSchemaRDD.map(new Row(_))
+}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala
new file mode 100644
index 0000000000000..362fe769581d7
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.api.java
+
+import org.apache.spark.sql.catalyst.expressions.{Row => ScalaRow}
+
+/**
+ * A result row from a SparkSQL query.
+ */
+class Row(row: ScalaRow) extends Serializable {
+
+ /** Returns the number of columns present in this Row. */
+ def length: Int = row.length
+
+ /** Returns the value of column `i`. */
+ def get(i: Int): Any =
+ row(i)
+
+ /** Returns true if value at column `i` is NULL. */
+ def isNullAt(i: Int) = get(i) == null
+
+ /**
+ * Returns the value of column `i` as an int. This function will throw an exception if the value
+ * is at `i` is not an integer, or if it is null.
+ */
+ def getInt(i: Int): Int =
+ row.getInt(i)
+
+ /**
+ * Returns the value of column `i` as a long. This function will throw an exception if the value
+ * is at `i` is not a long, or if it is null.
+ */
+ def getLong(i: Int): Long =
+ row.getLong(i)
+
+ /**
+ * Returns the value of column `i` as a double. This function will throw an exception if the
+ * value is at `i` is not a double, or if it is null.
+ */
+ def getDouble(i: Int): Double =
+ row.getDouble(i)
+
+ /**
+ * Returns the value of column `i` as a bool. This function will throw an exception if the value
+ * is at `i` is not a boolean, or if it is null.
+ */
+ def getBoolean(i: Int): Boolean =
+ row.getBoolean(i)
+
+ /**
+ * Returns the value of column `i` as a short. This function will throw an exception if the value
+ * is at `i` is not a short, or if it is null.
+ */
+ def getShort(i: Int): Short =
+ row.getShort(i)
+
+ /**
+ * Returns the value of column `i` as a byte. This function will throw an exception if the value
+ * is at `i` is not a byte, or if it is null.
+ */
+ def getByte(i: Int): Byte =
+ row.getByte(i)
+
+ /**
+ * Returns the value of column `i` as a float. This function will throw an exception if the value
+ * is at `i` is not a float, or if it is null.
+ */
+ def getFloat(i: Int): Float =
+ row.getFloat(i)
+
+ /**
+ * Returns the value of column `i` as a String. This function will throw an exception if the
+ * value is at `i` is not a String.
+ */
+ def getString(i: Int): String =
+ row.getString(i)
+}
+
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala
index e0c98ecdf8f22..ffd4894b5213d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala
@@ -21,7 +21,7 @@ import java.nio.{ByteOrder, ByteBuffer}
import org.apache.spark.sql.catalyst.types.{BinaryType, NativeType, DataType}
import org.apache.spark.sql.catalyst.expressions.MutableRow
-import org.apache.spark.sql.execution.SparkSqlSerializer
+import org.apache.spark.sql.columnar.compression.CompressibleColumnAccessor
/**
* An `Iterator` like trait used to extract values from columnar byte buffer. When a value is
@@ -41,121 +41,66 @@ private[sql] trait ColumnAccessor {
protected def underlyingBuffer: ByteBuffer
}
-private[sql] abstract class BasicColumnAccessor[T <: DataType, JvmType](buffer: ByteBuffer)
+private[sql] abstract class BasicColumnAccessor[T <: DataType, JvmType](
+ protected val buffer: ByteBuffer,
+ protected val columnType: ColumnType[T, JvmType])
extends ColumnAccessor {
protected def initialize() {}
- def columnType: ColumnType[T, JvmType]
-
def hasNext = buffer.hasRemaining
def extractTo(row: MutableRow, ordinal: Int) {
- doExtractTo(row, ordinal)
+ columnType.setField(row, ordinal, extractSingle(buffer))
}
- protected def doExtractTo(row: MutableRow, ordinal: Int)
+ def extractSingle(buffer: ByteBuffer): JvmType = columnType.extract(buffer)
protected def underlyingBuffer = buffer
}
private[sql] abstract class NativeColumnAccessor[T <: NativeType](
- buffer: ByteBuffer,
- val columnType: NativeColumnType[T])
- extends BasicColumnAccessor[T, T#JvmType](buffer)
+ override protected val buffer: ByteBuffer,
+ override protected val columnType: NativeColumnType[T])
+ extends BasicColumnAccessor(buffer, columnType)
with NullableColumnAccessor
+ with CompressibleColumnAccessor[T]
private[sql] class BooleanColumnAccessor(buffer: ByteBuffer)
- extends NativeColumnAccessor(buffer, BOOLEAN) {
-
- override protected def doExtractTo(row: MutableRow, ordinal: Int) {
- row.setBoolean(ordinal, columnType.extract(buffer))
- }
-}
+ extends NativeColumnAccessor(buffer, BOOLEAN)
private[sql] class IntColumnAccessor(buffer: ByteBuffer)
- extends NativeColumnAccessor(buffer, INT) {
-
- override protected def doExtractTo(row: MutableRow, ordinal: Int) {
- row.setInt(ordinal, columnType.extract(buffer))
- }
-}
+ extends NativeColumnAccessor(buffer, INT)
private[sql] class ShortColumnAccessor(buffer: ByteBuffer)
- extends NativeColumnAccessor(buffer, SHORT) {
-
- override protected def doExtractTo(row: MutableRow, ordinal: Int) {
- row.setShort(ordinal, columnType.extract(buffer))
- }
-}
+ extends NativeColumnAccessor(buffer, SHORT)
private[sql] class LongColumnAccessor(buffer: ByteBuffer)
- extends NativeColumnAccessor(buffer, LONG) {
-
- override protected def doExtractTo(row: MutableRow, ordinal: Int) {
- row.setLong(ordinal, columnType.extract(buffer))
- }
-}
+ extends NativeColumnAccessor(buffer, LONG)
private[sql] class ByteColumnAccessor(buffer: ByteBuffer)
- extends NativeColumnAccessor(buffer, BYTE) {
-
- override protected def doExtractTo(row: MutableRow, ordinal: Int) {
- row.setByte(ordinal, columnType.extract(buffer))
- }
-}
+ extends NativeColumnAccessor(buffer, BYTE)
private[sql] class DoubleColumnAccessor(buffer: ByteBuffer)
- extends NativeColumnAccessor(buffer, DOUBLE) {
-
- override protected def doExtractTo(row: MutableRow, ordinal: Int) {
- row.setDouble(ordinal, columnType.extract(buffer))
- }
-}
+ extends NativeColumnAccessor(buffer, DOUBLE)
private[sql] class FloatColumnAccessor(buffer: ByteBuffer)
- extends NativeColumnAccessor(buffer, FLOAT) {
-
- override protected def doExtractTo(row: MutableRow, ordinal: Int) {
- row.setFloat(ordinal, columnType.extract(buffer))
- }
-}
+ extends NativeColumnAccessor(buffer, FLOAT)
private[sql] class StringColumnAccessor(buffer: ByteBuffer)
- extends NativeColumnAccessor(buffer, STRING) {
-
- override protected def doExtractTo(row: MutableRow, ordinal: Int) {
- row.setString(ordinal, columnType.extract(buffer))
- }
-}
+ extends NativeColumnAccessor(buffer, STRING)
private[sql] class BinaryColumnAccessor(buffer: ByteBuffer)
- extends BasicColumnAccessor[BinaryType.type, Array[Byte]](buffer)
- with NullableColumnAccessor {
-
- def columnType = BINARY
-
- override protected def doExtractTo(row: MutableRow, ordinal: Int) {
- row(ordinal) = columnType.extract(buffer)
- }
-}
+ extends BasicColumnAccessor[BinaryType.type, Array[Byte]](buffer, BINARY)
+ with NullableColumnAccessor
private[sql] class GenericColumnAccessor(buffer: ByteBuffer)
- extends BasicColumnAccessor[DataType, Array[Byte]](buffer)
- with NullableColumnAccessor {
-
- def columnType = GENERIC
-
- override protected def doExtractTo(row: MutableRow, ordinal: Int) {
- val serialized = columnType.extract(buffer)
- row(ordinal) = SparkSqlSerializer.deserialize[Any](serialized)
- }
-}
+ extends BasicColumnAccessor[DataType, Array[Byte]](buffer, GENERIC)
+ with NullableColumnAccessor
private[sql] object ColumnAccessor {
- def apply(b: ByteBuffer): ColumnAccessor = {
- // The first 4 bytes in the buffer indicates the column type.
- val buffer = b.duplicate().order(ByteOrder.nativeOrder())
+ def apply(buffer: ByteBuffer): ColumnAccessor = {
+ // The first 4 bytes in the buffer indicate the column type.
val columnTypeId = buffer.getInt()
columnTypeId match {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala
index 3e622adfd3d6a..048ee66bff44b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala
@@ -22,7 +22,7 @@ import java.nio.{ByteBuffer, ByteOrder}
import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.types._
import org.apache.spark.sql.columnar.ColumnBuilder._
-import org.apache.spark.sql.execution.SparkSqlSerializer
+import org.apache.spark.sql.columnar.compression.{AllCompressionSchemes, CompressibleColumnBuilder}
private[sql] trait ColumnBuilder {
/**
@@ -30,37 +30,44 @@ private[sql] trait ColumnBuilder {
*/
def initialize(initialSize: Int, columnName: String = "")
+ /**
+ * Appends `row(ordinal)` to the column builder.
+ */
def appendFrom(row: Row, ordinal: Int)
+ /**
+ * Column statistics information
+ */
+ def columnStats: ColumnStats[_, _]
+
+ /**
+ * Returns the final columnar byte buffer.
+ */
def build(): ByteBuffer
}
-private[sql] abstract class BasicColumnBuilder[T <: DataType, JvmType] extends ColumnBuilder {
+private[sql] class BasicColumnBuilder[T <: DataType, JvmType](
+ val columnStats: ColumnStats[T, JvmType],
+ val columnType: ColumnType[T, JvmType])
+ extends ColumnBuilder {
- private var columnName: String = _
- protected var buffer: ByteBuffer = _
+ protected var columnName: String = _
- def columnType: ColumnType[T, JvmType]
+ protected var buffer: ByteBuffer = _
override def initialize(initialSize: Int, columnName: String = "") = {
val size = if (initialSize == 0) DEFAULT_INITIAL_BUFFER_SIZE else initialSize
this.columnName = columnName
- buffer = ByteBuffer.allocate(4 + 4 + size * columnType.defaultSize)
+
+ // Reserves 4 bytes for column type ID
+ buffer = ByteBuffer.allocate(4 + size * columnType.defaultSize)
buffer.order(ByteOrder.nativeOrder()).putInt(columnType.typeId)
}
- // Have to give a concrete implementation to make mixin possible
override def appendFrom(row: Row, ordinal: Int) {
- doAppendFrom(row, ordinal)
- }
-
- // Concrete `ColumnBuilder`s can override this method to append values
- protected def doAppendFrom(row: Row, ordinal: Int)
-
- // Helper method to append primitive values (to avoid boxing cost)
- protected def appendValue(v: JvmType) {
- buffer = ensureFreeSpace(buffer, columnType.actualSize(v))
- columnType.append(v, buffer)
+ val field = columnType.getField(row, ordinal)
+ buffer = ensureFreeSpace(buffer, columnType.actualSize(field))
+ columnType.append(field, buffer)
}
override def build() = {
@@ -69,83 +76,39 @@ private[sql] abstract class BasicColumnBuilder[T <: DataType, JvmType] extends C
}
}
-private[sql] abstract class NativeColumnBuilder[T <: NativeType](
- val columnType: NativeColumnType[T])
- extends BasicColumnBuilder[T, T#JvmType]
+private[sql] abstract class ComplexColumnBuilder[T <: DataType, JvmType](
+ columnType: ColumnType[T, JvmType])
+ extends BasicColumnBuilder[T, JvmType](new NoopColumnStats[T, JvmType], columnType)
with NullableColumnBuilder
-private[sql] class BooleanColumnBuilder extends NativeColumnBuilder(BOOLEAN) {
- override def doAppendFrom(row: Row, ordinal: Int) {
- appendValue(row.getBoolean(ordinal))
- }
-}
-
-private[sql] class IntColumnBuilder extends NativeColumnBuilder(INT) {
- override def doAppendFrom(row: Row, ordinal: Int) {
- appendValue(row.getInt(ordinal))
- }
-}
+private[sql] abstract class NativeColumnBuilder[T <: NativeType](
+ override val columnStats: NativeColumnStats[T],
+ override val columnType: NativeColumnType[T])
+ extends BasicColumnBuilder[T, T#JvmType](columnStats, columnType)
+ with NullableColumnBuilder
+ with AllCompressionSchemes
+ with CompressibleColumnBuilder[T]
-private[sql] class ShortColumnBuilder extends NativeColumnBuilder(SHORT) {
- override def doAppendFrom(row: Row, ordinal: Int) {
- appendValue(row.getShort(ordinal))
- }
-}
+private[sql] class BooleanColumnBuilder extends NativeColumnBuilder(new BooleanColumnStats, BOOLEAN)
-private[sql] class LongColumnBuilder extends NativeColumnBuilder(LONG) {
- override def doAppendFrom(row: Row, ordinal: Int) {
- appendValue(row.getLong(ordinal))
- }
-}
+private[sql] class IntColumnBuilder extends NativeColumnBuilder(new IntColumnStats, INT)
-private[sql] class ByteColumnBuilder extends NativeColumnBuilder(BYTE) {
- override def doAppendFrom(row: Row, ordinal: Int) {
- appendValue(row.getByte(ordinal))
- }
-}
+private[sql] class ShortColumnBuilder extends NativeColumnBuilder(new ShortColumnStats, SHORT)
-private[sql] class DoubleColumnBuilder extends NativeColumnBuilder(DOUBLE) {
- override def doAppendFrom(row: Row, ordinal: Int) {
- appendValue(row.getDouble(ordinal))
- }
-}
+private[sql] class LongColumnBuilder extends NativeColumnBuilder(new LongColumnStats, LONG)
-private[sql] class FloatColumnBuilder extends NativeColumnBuilder(FLOAT) {
- override def doAppendFrom(row: Row, ordinal: Int) {
- appendValue(row.getFloat(ordinal))
- }
-}
+private[sql] class ByteColumnBuilder extends NativeColumnBuilder(new ByteColumnStats, BYTE)
-private[sql] class StringColumnBuilder extends NativeColumnBuilder(STRING) {
- override def doAppendFrom(row: Row, ordinal: Int) {
- appendValue(row.getString(ordinal))
- }
-}
+private[sql] class DoubleColumnBuilder extends NativeColumnBuilder(new DoubleColumnStats, DOUBLE)
-private[sql] class BinaryColumnBuilder
- extends BasicColumnBuilder[BinaryType.type, Array[Byte]]
- with NullableColumnBuilder {
+private[sql] class FloatColumnBuilder extends NativeColumnBuilder(new FloatColumnStats, FLOAT)
- def columnType = BINARY
+private[sql] class StringColumnBuilder extends NativeColumnBuilder(new StringColumnStats, STRING)
- override def doAppendFrom(row: Row, ordinal: Int) {
- appendValue(row(ordinal).asInstanceOf[Array[Byte]])
- }
-}
+private[sql] class BinaryColumnBuilder extends ComplexColumnBuilder(BINARY)
// TODO (lian) Add support for array, struct and map
-private[sql] class GenericColumnBuilder
- extends BasicColumnBuilder[DataType, Array[Byte]]
- with NullableColumnBuilder {
-
- def columnType = GENERIC
-
- override def doAppendFrom(row: Row, ordinal: Int) {
- val serialized = SparkSqlSerializer.serialize(row(ordinal))
- buffer = ColumnBuilder.ensureFreeSpace(buffer, columnType.actualSize(serialized))
- columnType.append(serialized, buffer)
- }
-}
+private[sql] class GenericColumnBuilder extends ComplexColumnBuilder(GENERIC)
private[sql] object ColumnBuilder {
val DEFAULT_INITIAL_BUFFER_SIZE = 10 * 1024 * 104
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala
new file mode 100644
index 0000000000000..30c6bdc7912fc
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala
@@ -0,0 +1,360 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.columnar
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.types._
+
+private[sql] sealed abstract class ColumnStats[T <: DataType, JvmType] extends Serializable {
+ /**
+ * Closed lower bound of this column.
+ */
+ def lowerBound: JvmType
+
+ /**
+ * Closed upper bound of this column.
+ */
+ def upperBound: JvmType
+
+ /**
+ * Gathers statistics information from `row(ordinal)`.
+ */
+ def gatherStats(row: Row, ordinal: Int)
+
+ /**
+ * Returns `true` if `lower <= row(ordinal) <= upper`.
+ */
+ def contains(row: Row, ordinal: Int): Boolean
+
+ /**
+ * Returns `true` if `row(ordinal) < upper` holds.
+ */
+ def isAbove(row: Row, ordinal: Int): Boolean
+
+ /**
+ * Returns `true` if `lower < row(ordinal)` holds.
+ */
+ def isBelow(row: Row, ordinal: Int): Boolean
+
+ /**
+ * Returns `true` if `row(ordinal) <= upper` holds.
+ */
+ def isAtOrAbove(row: Row, ordinal: Int): Boolean
+
+ /**
+ * Returns `true` if `lower <= row(ordinal)` holds.
+ */
+ def isAtOrBelow(row: Row, ordinal: Int): Boolean
+}
+
+private[sql] sealed abstract class NativeColumnStats[T <: NativeType]
+ extends ColumnStats[T, T#JvmType] {
+
+ type JvmType = T#JvmType
+
+ protected var (_lower, _upper) = initialBounds
+
+ def initialBounds: (JvmType, JvmType)
+
+ protected def columnType: NativeColumnType[T]
+
+ override def lowerBound: T#JvmType = _lower
+
+ override def upperBound: T#JvmType = _upper
+
+ override def isAtOrAbove(row: Row, ordinal: Int) = {
+ contains(row, ordinal) || isAbove(row, ordinal)
+ }
+
+ override def isAtOrBelow(row: Row, ordinal: Int) = {
+ contains(row, ordinal) || isBelow(row, ordinal)
+ }
+}
+
+private[sql] class NoopColumnStats[T <: DataType, JvmType] extends ColumnStats[T, JvmType] {
+ override def isAtOrBelow(row: Row, ordinal: Int) = true
+
+ override def isAtOrAbove(row: Row, ordinal: Int) = true
+
+ override def isBelow(row: Row, ordinal: Int) = true
+
+ override def isAbove(row: Row, ordinal: Int) = true
+
+ override def contains(row: Row, ordinal: Int) = true
+
+ override def gatherStats(row: Row, ordinal: Int) {}
+
+ override def upperBound = null.asInstanceOf[JvmType]
+
+ override def lowerBound = null.asInstanceOf[JvmType]
+}
+
+private[sql] abstract class BasicColumnStats[T <: NativeType](
+ protected val columnType: NativeColumnType[T])
+ extends NativeColumnStats[T]
+
+private[sql] class BooleanColumnStats extends BasicColumnStats(BOOLEAN) {
+ override def initialBounds = (true, false)
+
+ override def isBelow(row: Row, ordinal: Int) = {
+ lowerBound < columnType.getField(row, ordinal)
+ }
+
+ override def isAbove(row: Row, ordinal: Int) = {
+ columnType.getField(row, ordinal) < upperBound
+ }
+
+ override def contains(row: Row, ordinal: Int) = {
+ val field = columnType.getField(row, ordinal)
+ lowerBound <= field && field <= upperBound
+ }
+
+ override def gatherStats(row: Row, ordinal: Int) {
+ val field = columnType.getField(row, ordinal)
+ if (field > upperBound) _upper = field
+ if (field < lowerBound) _lower = field
+ }
+}
+
+private[sql] class ByteColumnStats extends BasicColumnStats(BYTE) {
+ override def initialBounds = (Byte.MaxValue, Byte.MinValue)
+
+ override def isBelow(row: Row, ordinal: Int) = {
+ lowerBound < columnType.getField(row, ordinal)
+ }
+
+ override def isAbove(row: Row, ordinal: Int) = {
+ columnType.getField(row, ordinal) < upperBound
+ }
+
+ override def contains(row: Row, ordinal: Int) = {
+ val field = columnType.getField(row, ordinal)
+ lowerBound <= field && field <= upperBound
+ }
+
+ override def gatherStats(row: Row, ordinal: Int) {
+ val field = columnType.getField(row, ordinal)
+ if (field > upperBound) _upper = field
+ if (field < lowerBound) _lower = field
+ }
+}
+
+private[sql] class ShortColumnStats extends BasicColumnStats(SHORT) {
+ override def initialBounds = (Short.MaxValue, Short.MinValue)
+
+ override def isBelow(row: Row, ordinal: Int) = {
+ lowerBound < columnType.getField(row, ordinal)
+ }
+
+ override def isAbove(row: Row, ordinal: Int) = {
+ columnType.getField(row, ordinal) < upperBound
+ }
+
+ override def contains(row: Row, ordinal: Int) = {
+ val field = columnType.getField(row, ordinal)
+ lowerBound <= field && field <= upperBound
+ }
+
+ override def gatherStats(row: Row, ordinal: Int) {
+ val field = columnType.getField(row, ordinal)
+ if (field > upperBound) _upper = field
+ if (field < lowerBound) _lower = field
+ }
+}
+
+private[sql] class LongColumnStats extends BasicColumnStats(LONG) {
+ override def initialBounds = (Long.MaxValue, Long.MinValue)
+
+ override def isBelow(row: Row, ordinal: Int) = {
+ lowerBound < columnType.getField(row, ordinal)
+ }
+
+ override def isAbove(row: Row, ordinal: Int) = {
+ columnType.getField(row, ordinal) < upperBound
+ }
+
+ override def contains(row: Row, ordinal: Int) = {
+ val field = columnType.getField(row, ordinal)
+ lowerBound <= field && field <= upperBound
+ }
+
+ override def gatherStats(row: Row, ordinal: Int) {
+ val field = columnType.getField(row, ordinal)
+ if (field > upperBound) _upper = field
+ if (field < lowerBound) _lower = field
+ }
+}
+
+private[sql] class DoubleColumnStats extends BasicColumnStats(DOUBLE) {
+ override def initialBounds = (Double.MaxValue, Double.MinValue)
+
+ override def isBelow(row: Row, ordinal: Int) = {
+ lowerBound < columnType.getField(row, ordinal)
+ }
+
+ override def isAbove(row: Row, ordinal: Int) = {
+ columnType.getField(row, ordinal) < upperBound
+ }
+
+ override def contains(row: Row, ordinal: Int) = {
+ val field = columnType.getField(row, ordinal)
+ lowerBound <= field && field <= upperBound
+ }
+
+ override def gatherStats(row: Row, ordinal: Int) {
+ val field = columnType.getField(row, ordinal)
+ if (field > upperBound) _upper = field
+ if (field < lowerBound) _lower = field
+ }
+}
+
+private[sql] class FloatColumnStats extends BasicColumnStats(FLOAT) {
+ override def initialBounds = (Float.MaxValue, Float.MinValue)
+
+ override def isBelow(row: Row, ordinal: Int) = {
+ lowerBound < columnType.getField(row, ordinal)
+ }
+
+ override def isAbove(row: Row, ordinal: Int) = {
+ columnType.getField(row, ordinal) < upperBound
+ }
+
+ override def contains(row: Row, ordinal: Int) = {
+ val field = columnType.getField(row, ordinal)
+ lowerBound <= field && field <= upperBound
+ }
+
+ override def gatherStats(row: Row, ordinal: Int) {
+ val field = columnType.getField(row, ordinal)
+ if (field > upperBound) _upper = field
+ if (field < lowerBound) _lower = field
+ }
+}
+
+private[sql] object IntColumnStats {
+ val UNINITIALIZED = 0
+ val INITIALIZED = 1
+ val ASCENDING = 2
+ val DESCENDING = 3
+ val UNORDERED = 4
+}
+
+/**
+ * Statistical information for `Int` columns. More information is collected since `Int` is
+ * frequently used. Extra information include:
+ *
+ * - Ordering state (ascending/descending/unordered), may be used to decide whether binary search
+ * is applicable when searching elements.
+ * - Maximum delta between adjacent elements, may be used to guide the `IntDelta` compression
+ * scheme.
+ *
+ * (This two kinds of information are not used anywhere yet and might be removed later.)
+ */
+private[sql] class IntColumnStats extends BasicColumnStats(INT) {
+ import IntColumnStats._
+
+ private var orderedState = UNINITIALIZED
+ private var lastValue: Int = _
+ private var _maxDelta: Int = _
+
+ def isAscending = orderedState != DESCENDING && orderedState != UNORDERED
+ def isDescending = orderedState != ASCENDING && orderedState != UNORDERED
+ def isOrdered = isAscending || isDescending
+ def maxDelta = _maxDelta
+
+ override def initialBounds = (Int.MaxValue, Int.MinValue)
+
+ override def isBelow(row: Row, ordinal: Int) = {
+ lowerBound < columnType.getField(row, ordinal)
+ }
+
+ override def isAbove(row: Row, ordinal: Int) = {
+ columnType.getField(row, ordinal) < upperBound
+ }
+
+ override def contains(row: Row, ordinal: Int) = {
+ val field = columnType.getField(row, ordinal)
+ lowerBound <= field && field <= upperBound
+ }
+
+ override def gatherStats(row: Row, ordinal: Int) {
+ val field = columnType.getField(row, ordinal)
+
+ if (field > upperBound) _upper = field
+ if (field < lowerBound) _lower = field
+
+ orderedState = orderedState match {
+ case UNINITIALIZED =>
+ lastValue = field
+ INITIALIZED
+
+ case INITIALIZED =>
+ // If all the integers in the column are the same, ordered state is set to Ascending.
+ // TODO (lian) Confirm whether this is the standard behaviour.
+ val nextState = if (field >= lastValue) ASCENDING else DESCENDING
+ _maxDelta = math.abs(field - lastValue)
+ lastValue = field
+ nextState
+
+ case ASCENDING if field < lastValue =>
+ UNORDERED
+
+ case DESCENDING if field > lastValue =>
+ UNORDERED
+
+ case state @ (ASCENDING | DESCENDING) =>
+ _maxDelta = _maxDelta.max(field - lastValue)
+ lastValue = field
+ state
+
+ case _ =>
+ orderedState
+ }
+ }
+}
+
+private[sql] class StringColumnStats extends BasicColumnStats(STRING) {
+ override def initialBounds = (null, null)
+
+ override def gatherStats(row: Row, ordinal: Int) {
+ val field = columnType.getField(row, ordinal)
+ if ((upperBound eq null) || field.compareTo(upperBound) > 0) _upper = field
+ if ((lowerBound eq null) || field.compareTo(lowerBound) < 0) _lower = field
+ }
+
+ override def contains(row: Row, ordinal: Int) = {
+ !(upperBound eq null) && {
+ val field = columnType.getField(row, ordinal)
+ lowerBound.compareTo(field) <= 0 && field.compareTo(upperBound) <= 0
+ }
+ }
+
+ override def isAbove(row: Row, ordinal: Int) = {
+ !(upperBound eq null) && {
+ val field = columnType.getField(row, ordinal)
+ field.compareTo(upperBound) < 0
+ }
+ }
+
+ override def isBelow(row: Row, ordinal: Int) = {
+ !(lowerBound eq null) && {
+ val field = columnType.getField(row, ordinal)
+ lowerBound.compareTo(field) < 0
+ }
+ }
+}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala
index a452b86f0cda3..5be76890afe31 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala
@@ -19,7 +19,12 @@ package org.apache.spark.sql.columnar
import java.nio.ByteBuffer
+import scala.reflect.runtime.universe.TypeTag
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.expressions.MutableRow
import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.execution.SparkSqlSerializer
/**
* An abstract class that represents type of a column. Used to append/extract Java objects into/from
@@ -50,10 +55,24 @@ private[sql] sealed abstract class ColumnType[T <: DataType, JvmType](
*/
def actualSize(v: JvmType): Int = defaultSize
+ /**
+ * Returns `row(ordinal)`. Subclasses should override this method to avoid boxing/unboxing costs
+ * whenever possible.
+ */
+ def getField(row: Row, ordinal: Int): JvmType
+
+ /**
+ * Sets `row(ordinal)` to `field`. Subclasses should override this method to avoid boxing/unboxing
+ * costs whenever possible.
+ */
+ def setField(row: MutableRow, ordinal: Int, value: JvmType)
+
/**
* Creates a duplicated copy of the value.
*/
def clone(v: JvmType): JvmType = v
+
+ override def toString = getClass.getSimpleName.stripSuffix("$")
}
private[sql] abstract class NativeColumnType[T <: NativeType](
@@ -65,7 +84,7 @@ private[sql] abstract class NativeColumnType[T <: NativeType](
/**
* Scala TypeTag. Can be used to create primitive arrays and hash tables.
*/
- def scalaTag = dataType.tag
+ def scalaTag: TypeTag[dataType.JvmType] = dataType.tag
}
private[sql] object INT extends NativeColumnType(IntegerType, 0, 4) {
@@ -76,6 +95,12 @@ private[sql] object INT extends NativeColumnType(IntegerType, 0, 4) {
def extract(buffer: ByteBuffer) = {
buffer.getInt()
}
+
+ override def setField(row: MutableRow, ordinal: Int, value: Int) {
+ row.setInt(ordinal, value)
+ }
+
+ override def getField(row: Row, ordinal: Int) = row.getInt(ordinal)
}
private[sql] object LONG extends NativeColumnType(LongType, 1, 8) {
@@ -86,6 +111,12 @@ private[sql] object LONG extends NativeColumnType(LongType, 1, 8) {
override def extract(buffer: ByteBuffer) = {
buffer.getLong()
}
+
+ override def setField(row: MutableRow, ordinal: Int, value: Long) {
+ row.setLong(ordinal, value)
+ }
+
+ override def getField(row: Row, ordinal: Int) = row.getLong(ordinal)
}
private[sql] object FLOAT extends NativeColumnType(FloatType, 2, 4) {
@@ -96,6 +127,12 @@ private[sql] object FLOAT extends NativeColumnType(FloatType, 2, 4) {
override def extract(buffer: ByteBuffer) = {
buffer.getFloat()
}
+
+ override def setField(row: MutableRow, ordinal: Int, value: Float) {
+ row.setFloat(ordinal, value)
+ }
+
+ override def getField(row: Row, ordinal: Int) = row.getFloat(ordinal)
}
private[sql] object DOUBLE extends NativeColumnType(DoubleType, 3, 8) {
@@ -106,6 +143,12 @@ private[sql] object DOUBLE extends NativeColumnType(DoubleType, 3, 8) {
override def extract(buffer: ByteBuffer) = {
buffer.getDouble()
}
+
+ override def setField(row: MutableRow, ordinal: Int, value: Double) {
+ row.setDouble(ordinal, value)
+ }
+
+ override def getField(row: Row, ordinal: Int) = row.getDouble(ordinal)
}
private[sql] object BOOLEAN extends NativeColumnType(BooleanType, 4, 1) {
@@ -116,6 +159,12 @@ private[sql] object BOOLEAN extends NativeColumnType(BooleanType, 4, 1) {
override def extract(buffer: ByteBuffer) = {
if (buffer.get() == 1) true else false
}
+
+ override def setField(row: MutableRow, ordinal: Int, value: Boolean) {
+ row.setBoolean(ordinal, value)
+ }
+
+ override def getField(row: Row, ordinal: Int) = row.getBoolean(ordinal)
}
private[sql] object BYTE extends NativeColumnType(ByteType, 5, 1) {
@@ -126,6 +175,12 @@ private[sql] object BYTE extends NativeColumnType(ByteType, 5, 1) {
override def extract(buffer: ByteBuffer) = {
buffer.get()
}
+
+ override def setField(row: MutableRow, ordinal: Int, value: Byte) {
+ row.setByte(ordinal, value)
+ }
+
+ override def getField(row: Row, ordinal: Int) = row.getByte(ordinal)
}
private[sql] object SHORT extends NativeColumnType(ShortType, 6, 2) {
@@ -136,6 +191,12 @@ private[sql] object SHORT extends NativeColumnType(ShortType, 6, 2) {
override def extract(buffer: ByteBuffer) = {
buffer.getShort()
}
+
+ override def setField(row: MutableRow, ordinal: Int, value: Short) {
+ row.setShort(ordinal, value)
+ }
+
+ override def getField(row: Row, ordinal: Int) = row.getShort(ordinal)
}
private[sql] object STRING extends NativeColumnType(StringType, 7, 8) {
@@ -152,6 +213,12 @@ private[sql] object STRING extends NativeColumnType(StringType, 7, 8) {
buffer.get(stringBytes, 0, length)
new String(stringBytes)
}
+
+ override def setField(row: MutableRow, ordinal: Int, value: String) {
+ row.setString(ordinal, value)
+ }
+
+ override def getField(row: Row, ordinal: Int) = row.getString(ordinal)
}
private[sql] sealed abstract class ByteArrayColumnType[T <: DataType](
@@ -173,15 +240,27 @@ private[sql] sealed abstract class ByteArrayColumnType[T <: DataType](
}
}
-private[sql] object BINARY extends ByteArrayColumnType[BinaryType.type](8, 16)
+private[sql] object BINARY extends ByteArrayColumnType[BinaryType.type](8, 16) {
+ override def setField(row: MutableRow, ordinal: Int, value: Array[Byte]) {
+ row(ordinal) = value
+ }
+
+ override def getField(row: Row, ordinal: Int) = row(ordinal).asInstanceOf[Array[Byte]]
+}
// Used to process generic objects (all types other than those listed above). Objects should be
// serialized first before appending to the column `ByteBuffer`, and is also extracted as serialized
// byte array.
-private[sql] object GENERIC extends ByteArrayColumnType[DataType](9, 16)
+private[sql] object GENERIC extends ByteArrayColumnType[DataType](9, 16) {
+ override def setField(row: MutableRow, ordinal: Int, value: Array[Byte]) {
+ row(ordinal) = SparkSqlSerializer.deserialize[Any](value)
+ }
+
+ override def getField(row: Row, ordinal: Int) = SparkSqlSerializer.serialize(row(ordinal))
+}
private[sql] object ColumnType {
- implicit def dataTypeToColumnType(dataType: DataType): ColumnType[_, _] = {
+ def apply(dataType: DataType): ColumnType[_, _] = {
dataType match {
case IntegerType => INT
case LongType => LONG
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala
similarity index 93%
rename from sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala
rename to sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala
index f853759e5a306..8a24733047423 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala
@@ -21,9 +21,6 @@ import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, Attribute}
import org.apache.spark.sql.execution.{SparkPlan, LeafNode}
import org.apache.spark.sql.Row
-/* Implicit conversions */
-import org.apache.spark.sql.columnar.ColumnType._
-
private[sql] case class InMemoryColumnarTableScan(attributes: Seq[Attribute], child: SparkPlan)
extends LeafNode {
@@ -32,8 +29,8 @@ private[sql] case class InMemoryColumnarTableScan(attributes: Seq[Attribute], ch
lazy val cachedColumnBuffers = {
val output = child.output
val cached = child.execute().mapPartitions { iterator =>
- val columnBuilders = output.map { a =>
- ColumnBuilder(a.dataType.typeId, 0, a.name)
+ val columnBuilders = output.map { attribute =>
+ ColumnBuilder(ColumnType(attribute.dataType).typeId, 0, attribute.name)
}.toArray
var row: Row = null
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala
index 2970c609b928d..7d49ab07f7a53 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala
@@ -29,7 +29,7 @@ private[sql] trait NullableColumnAccessor extends ColumnAccessor {
private var nextNullIndex: Int = _
private var pos: Int = 0
- abstract override def initialize() {
+ abstract override protected def initialize() {
nullsBuffer = underlyingBuffer.duplicate().order(ByteOrder.nativeOrder())
nullCount = nullsBuffer.getInt()
nextNullIndex = if (nullCount > 0) nullsBuffer.getInt() else -1
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala
index 048d1f05c7df2..2a3b6fc1e46d3 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala
@@ -22,10 +22,18 @@ import java.nio.{ByteBuffer, ByteOrder}
import org.apache.spark.sql.Row
/**
- * Builds a nullable column. The byte buffer of a nullable column contains:
- * - 4 bytes for the null count (number of nulls)
- * - positions for each null, in ascending order
- * - the non-null data (column data type, compression type, data...)
+ * A stackable trait used for building byte buffer for a column containing null values. Memory
+ * layout of the final byte buffer is:
+ * {{{
+ * .----------------------- Column type ID (4 bytes)
+ * | .------------------- Null count N (4 bytes)
+ * | | .--------------- Null positions (4 x N bytes, empty if null count is zero)
+ * | | | .--------- Non-null elements
+ * V V V V
+ * +---+---+-----+---------+
+ * | | | ... | ... ... |
+ * +---+---+-----+---------+
+ * }}}
*/
private[sql] trait NullableColumnBuilder extends ColumnBuilder {
private var nulls: ByteBuffer = _
@@ -59,19 +67,8 @@ private[sql] trait NullableColumnBuilder extends ColumnBuilder {
nulls.limit(nullDataLen)
nulls.rewind()
- // Column type ID is moved to the front, follows the null count, then non-null data
- //
- // +---------+
- // | 4 bytes | Column type ID
- // +---------+
- // | 4 bytes | Null count
- // +---------+
- // | ... | Null positions (if null count is not zero)
- // +---------+
- // | ... | Non-null part (without column type ID)
- // +---------+
val buffer = ByteBuffer
- .allocate(4 + nullDataLen + nonNulls.limit)
+ .allocate(4 + 4 + nullDataLen + nonNulls.remaining())
.order(ByteOrder.nativeOrder())
.putInt(typeId)
.putInt(nullCount)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala
new file mode 100644
index 0000000000000..878cb84de106f
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.columnar.compression
+
+import java.nio.ByteBuffer
+
+import org.apache.spark.sql.catalyst.types.NativeType
+import org.apache.spark.sql.columnar.{ColumnAccessor, NativeColumnAccessor}
+
+private[sql] trait CompressibleColumnAccessor[T <: NativeType] extends ColumnAccessor {
+ this: NativeColumnAccessor[T] =>
+
+ private var decoder: Decoder[T] = _
+
+ abstract override protected def initialize() = {
+ super.initialize()
+ decoder = CompressionScheme(underlyingBuffer.getInt()).decoder(buffer, columnType)
+ }
+
+ abstract override def extractSingle(buffer: ByteBuffer): T#JvmType = decoder.next()
+}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala
new file mode 100644
index 0000000000000..3ac4b358ddf83
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.columnar.compression
+
+import java.nio.{ByteBuffer, ByteOrder}
+
+import org.apache.spark.sql.{Logging, Row}
+import org.apache.spark.sql.catalyst.types.NativeType
+import org.apache.spark.sql.columnar.{ColumnBuilder, NativeColumnBuilder}
+
+/**
+ * A stackable trait that builds optionally compressed byte buffer for a column. Memory layout of
+ * the final byte buffer is:
+ * {{{
+ * .--------------------------- Column type ID (4 bytes)
+ * | .----------------------- Null count N (4 bytes)
+ * | | .------------------- Null positions (4 x N bytes, empty if null count is zero)
+ * | | | .------------- Compression scheme ID (4 bytes)
+ * | | | | .--------- Compressed non-null elements
+ * V V V V V
+ * +---+---+-----+---+---------+
+ * | | | ... | | ... ... |
+ * +---+---+-----+---+---------+
+ * \-----------/ \-----------/
+ * header body
+ * }}}
+ */
+private[sql] trait CompressibleColumnBuilder[T <: NativeType]
+ extends ColumnBuilder with Logging {
+
+ this: NativeColumnBuilder[T] with WithCompressionSchemes =>
+
+ import CompressionScheme._
+
+ val compressionEncoders = schemes.filter(_.supports(columnType)).map(_.encoder)
+
+ protected def isWorthCompressing(encoder: Encoder) = {
+ encoder.compressionRatio < 0.8
+ }
+
+ private def gatherCompressibilityStats(row: Row, ordinal: Int) {
+ val field = columnType.getField(row, ordinal)
+
+ var i = 0
+ while (i < compressionEncoders.length) {
+ compressionEncoders(i).gatherCompressibilityStats(field, columnType)
+ i += 1
+ }
+ }
+
+ abstract override def appendFrom(row: Row, ordinal: Int) {
+ super.appendFrom(row, ordinal)
+ gatherCompressibilityStats(row, ordinal)
+ }
+
+ abstract override def build() = {
+ val rawBuffer = super.build()
+ val encoder = {
+ val candidate = compressionEncoders.minBy(_.compressionRatio)
+ if (isWorthCompressing(candidate)) candidate else PassThrough.encoder
+ }
+
+ val headerSize = columnHeaderSize(rawBuffer)
+ val compressedSize = if (encoder.compressedSize == 0) {
+ rawBuffer.limit - headerSize
+ } else {
+ encoder.compressedSize
+ }
+
+ // Reserves 4 bytes for compression scheme ID
+ val compressedBuffer = ByteBuffer
+ .allocate(headerSize + 4 + compressedSize)
+ .order(ByteOrder.nativeOrder)
+
+ copyColumnHeader(rawBuffer, compressedBuffer)
+
+ logger.info(s"Compressor for [$columnName]: $encoder, ratio: ${encoder.compressionRatio}")
+ encoder.compress(rawBuffer, compressedBuffer, columnType)
+ }
+}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala
new file mode 100644
index 0000000000000..d3a4ac8df926b
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.columnar.compression
+
+import java.nio.ByteBuffer
+
+import org.apache.spark.sql.catalyst.types.NativeType
+import org.apache.spark.sql.columnar.{ColumnType, NativeColumnType}
+
+private[sql] trait Encoder {
+ def gatherCompressibilityStats[T <: NativeType](
+ value: T#JvmType,
+ columnType: ColumnType[T, T#JvmType]) {}
+
+ def compressedSize: Int
+
+ def uncompressedSize: Int
+
+ def compressionRatio: Double = {
+ if (uncompressedSize > 0) compressedSize.toDouble / uncompressedSize else 1.0
+ }
+
+ def compress[T <: NativeType](
+ from: ByteBuffer,
+ to: ByteBuffer,
+ columnType: ColumnType[T, T#JvmType]): ByteBuffer
+}
+
+private[sql] trait Decoder[T <: NativeType] extends Iterator[T#JvmType]
+
+private[sql] trait CompressionScheme {
+ def typeId: Int
+
+ def supports(columnType: ColumnType[_, _]): Boolean
+
+ def encoder: Encoder
+
+ def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]): Decoder[T]
+}
+
+private[sql] trait WithCompressionSchemes {
+ def schemes: Seq[CompressionScheme]
+}
+
+private[sql] trait AllCompressionSchemes extends WithCompressionSchemes {
+ override val schemes: Seq[CompressionScheme] = {
+ Seq(PassThrough, RunLengthEncoding, DictionaryEncoding)
+ }
+}
+
+private[sql] object CompressionScheme {
+ def apply(typeId: Int): CompressionScheme = typeId match {
+ case PassThrough.typeId => PassThrough
+ case _ => throw new UnsupportedOperationException()
+ }
+
+ def copyColumnHeader(from: ByteBuffer, to: ByteBuffer) {
+ // Writes column type ID
+ to.putInt(from.getInt())
+
+ // Writes null count
+ val nullCount = from.getInt()
+ to.putInt(nullCount)
+
+ // Writes null positions
+ var i = 0
+ while (i < nullCount) {
+ to.putInt(from.getInt())
+ i += 1
+ }
+ }
+
+ def columnHeaderSize(columnBuffer: ByteBuffer): Int = {
+ val header = columnBuffer.duplicate()
+ val nullCount = header.getInt(4)
+ // Column type ID + null count + null positions
+ 4 + 4 + 4 * nullCount
+ }
+}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala
new file mode 100644
index 0000000000000..dc2c153faf8ad
--- /dev/null
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala
@@ -0,0 +1,288 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.columnar.compression
+
+import java.nio.ByteBuffer
+
+import scala.collection.mutable
+import scala.reflect.ClassTag
+import scala.reflect.runtime.universe.runtimeMirror
+
+import org.apache.spark.sql.catalyst.expressions.GenericMutableRow
+import org.apache.spark.sql.catalyst.types.NativeType
+import org.apache.spark.sql.columnar._
+
+private[sql] case object PassThrough extends CompressionScheme {
+ override val typeId = 0
+
+ override def supports(columnType: ColumnType[_, _]) = true
+
+ override def encoder = new this.Encoder
+
+ override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = {
+ new this.Decoder(buffer, columnType)
+ }
+
+ class Encoder extends compression.Encoder {
+ override def uncompressedSize = 0
+
+ override def compressedSize = 0
+
+ override def compress[T <: NativeType](
+ from: ByteBuffer,
+ to: ByteBuffer,
+ columnType: ColumnType[T, T#JvmType]) = {
+
+ // Writes compression type ID and copies raw contents
+ to.putInt(PassThrough.typeId).put(from).rewind()
+ to
+ }
+ }
+
+ class Decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T])
+ extends compression.Decoder[T] {
+
+ override def next() = columnType.extract(buffer)
+
+ override def hasNext = buffer.hasRemaining
+ }
+}
+
+private[sql] case object RunLengthEncoding extends CompressionScheme {
+ override def typeId = 1
+
+ override def encoder = new this.Encoder
+
+ override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = {
+ new this.Decoder(buffer, columnType)
+ }
+
+ override def supports(columnType: ColumnType[_, _]) = columnType match {
+ case INT | LONG | SHORT | BYTE | STRING | BOOLEAN => true
+ case _ => false
+ }
+
+ class Encoder extends compression.Encoder {
+ private var _uncompressedSize = 0
+ private var _compressedSize = 0
+
+ // Using `MutableRow` to store the last value to avoid boxing/unboxing cost.
+ private val lastValue = new GenericMutableRow(1)
+ private var lastRun = 0
+
+ override def uncompressedSize = _uncompressedSize
+
+ override def compressedSize = _compressedSize
+
+ override def gatherCompressibilityStats[T <: NativeType](
+ value: T#JvmType,
+ columnType: ColumnType[T, T#JvmType]) {
+
+ val actualSize = columnType.actualSize(value)
+ _uncompressedSize += actualSize
+
+ if (lastValue.isNullAt(0)) {
+ columnType.setField(lastValue, 0, value)
+ lastRun = 1
+ _compressedSize += actualSize + 4
+ } else {
+ if (columnType.getField(lastValue, 0) == value) {
+ lastRun += 1
+ } else {
+ _compressedSize += actualSize + 4
+ columnType.setField(lastValue, 0, value)
+ lastRun = 1
+ }
+ }
+ }
+
+ override def compress[T <: NativeType](
+ from: ByteBuffer,
+ to: ByteBuffer,
+ columnType: ColumnType[T, T#JvmType]) = {
+
+ to.putInt(RunLengthEncoding.typeId)
+
+ if (from.hasRemaining) {
+ var currentValue = columnType.extract(from)
+ var currentRun = 1
+
+ while (from.hasRemaining) {
+ val value = columnType.extract(from)
+
+ if (value == currentValue) {
+ currentRun += 1
+ } else {
+ // Writes current run
+ columnType.append(currentValue, to)
+ to.putInt(currentRun)
+
+ // Resets current run
+ currentValue = value
+ currentRun = 1
+ }
+ }
+
+ // Writes the last run
+ columnType.append(currentValue, to)
+ to.putInt(currentRun)
+ }
+
+ to.rewind()
+ to
+ }
+ }
+
+ class Decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T])
+ extends compression.Decoder[T] {
+
+ private var run = 0
+ private var valueCount = 0
+ private var currentValue: T#JvmType = _
+
+ override def next() = {
+ if (valueCount == run) {
+ currentValue = columnType.extract(buffer)
+ run = buffer.getInt()
+ valueCount = 1
+ } else {
+ valueCount += 1
+ }
+
+ currentValue
+ }
+
+ override def hasNext = buffer.hasRemaining
+ }
+}
+
+private[sql] case object DictionaryEncoding extends CompressionScheme {
+ override def typeId: Int = 2
+
+ // 32K unique values allowed
+ private val MAX_DICT_SIZE = Short.MaxValue - 1
+
+ override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = {
+ new this.Decoder[T](buffer, columnType)
+ }
+
+ override def encoder = new this.Encoder
+
+ override def supports(columnType: ColumnType[_, _]) = columnType match {
+ case INT | LONG | STRING => true
+ case _ => false
+ }
+
+ class Encoder extends compression.Encoder{
+ // Size of the input, uncompressed, in bytes. Note that we only count until the dictionary
+ // overflows.
+ private var _uncompressedSize = 0
+
+ // If the number of distinct elements is too large, we discard the use of dictionary encoding
+ // and set the overflow flag to true.
+ private var overflow = false
+
+ // Total number of elements.
+ private var count = 0
+
+ // The reverse mapping of _dictionary, i.e. mapping encoded integer to the value itself.
+ private var values = new mutable.ArrayBuffer[Any](1024)
+
+ // The dictionary that maps a value to the encoded short integer.
+ private val dictionary = mutable.HashMap.empty[Any, Short]
+
+ // Size of the serialized dictionary in bytes. Initialized to 4 since we need at least an `Int`
+ // to store dictionary element count.
+ private var dictionarySize = 4
+
+ override def gatherCompressibilityStats[T <: NativeType](
+ value: T#JvmType,
+ columnType: ColumnType[T, T#JvmType]) {
+
+ if (!overflow) {
+ val actualSize = columnType.actualSize(value)
+ count += 1
+ _uncompressedSize += actualSize
+
+ if (!dictionary.contains(value)) {
+ if (dictionary.size < MAX_DICT_SIZE) {
+ val clone = columnType.clone(value)
+ values += clone
+ dictionarySize += actualSize
+ dictionary(clone) = dictionary.size.toShort
+ } else {
+ overflow = true
+ values.clear()
+ dictionary.clear()
+ }
+ }
+ }
+ }
+
+ override def compress[T <: NativeType](
+ from: ByteBuffer,
+ to: ByteBuffer,
+ columnType: ColumnType[T, T#JvmType]) = {
+
+ if (overflow) {
+ throw new IllegalStateException(
+ "Dictionary encoding should not be used because of dictionary overflow.")
+ }
+
+ to.putInt(DictionaryEncoding.typeId)
+ .putInt(dictionary.size)
+
+ var i = 0
+ while (i < values.length) {
+ columnType.append(values(i).asInstanceOf[T#JvmType], to)
+ i += 1
+ }
+
+ while (from.hasRemaining) {
+ to.putShort(dictionary(columnType.extract(from)))
+ }
+
+ to.rewind()
+ to
+ }
+
+ override def uncompressedSize = _uncompressedSize
+
+ override def compressedSize = if (overflow) Int.MaxValue else dictionarySize + count * 2
+ }
+
+ class Decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T])
+ extends compression.Decoder[T] {
+
+ private val dictionary = {
+ // TODO Can we clean up this mess? Maybe move this to `DataType`?
+ implicit val classTag = {
+ val mirror = runtimeMirror(getClass.getClassLoader)
+ ClassTag[T#JvmType](mirror.runtimeClass(columnType.scalaTag.tpe))
+ }
+
+ Array.fill(buffer.getInt()) {
+ columnType.extract(buffer)
+ }
+ }
+
+ override def next() = dictionary(buffer.getShort())
+
+ override def hasNext = buffer.hasRemaining
+ }
+}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala
index 915f551fb2f01..d8e1b970c1d88 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala
@@ -32,7 +32,13 @@ class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) {
kryo.setRegistrationRequired(false)
kryo.register(classOf[MutablePair[_, _]])
kryo.register(classOf[Array[Any]])
+ // This is kinda hacky...
kryo.register(classOf[scala.collection.immutable.Map$Map1], new MapSerializer)
+ kryo.register(classOf[scala.collection.immutable.Map$Map2], new MapSerializer)
+ kryo.register(classOf[scala.collection.immutable.Map$Map3], new MapSerializer)
+ kryo.register(classOf[scala.collection.immutable.Map$Map4], new MapSerializer)
+ kryo.register(classOf[scala.collection.immutable.Map[_,_]], new MapSerializer)
+ kryo.register(classOf[scala.collection.Map[_,_]], new MapSerializer)
kryo.register(classOf[org.apache.spark.sql.catalyst.expressions.GenericRow])
kryo.register(classOf[org.apache.spark.sql.catalyst.expressions.GenericMutableRow])
kryo.register(classOf[scala.collection.mutable.ArrayBuffer[_]])
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
index 86f9d3e0fa954..fe8bd5a508820 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
@@ -28,7 +28,7 @@ import org.apache.spark.sql.parquet._
abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
self: SQLContext#SparkPlanner =>
- object SparkEquiInnerJoin extends Strategy {
+ object HashJoin extends Strategy {
def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
case FilteredOperation(predicates, logical.Join(left, right, Inner, condition)) =>
logger.debug(s"Considering join: ${predicates ++ condition}")
@@ -51,8 +51,8 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
val leftKeys = joinKeys.map(_._1)
val rightKeys = joinKeys.map(_._2)
- val joinOp = execution.SparkEquiInnerJoin(
- leftKeys, rightKeys, planLater(left), planLater(right))
+ val joinOp = execution.HashJoin(
+ leftKeys, rightKeys, BuildRight, planLater(left), planLater(right))
// Make sure other conditions are met if present.
if (otherPredicates.nonEmpty) {
@@ -158,10 +158,10 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
case other => other
}
- object TopK extends Strategy {
+ object TakeOrdered extends Strategy {
def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
- case logical.StopAfter(IntegerLiteral(limit), logical.Sort(order, child)) =>
- execution.TopK(limit, order, planLater(child))(sparkContext) :: Nil
+ case logical.Limit(IntegerLiteral(limit), logical.Sort(order, child)) =>
+ execution.TakeOrdered(limit, order, planLater(child))(sparkContext) :: Nil
case _ => Nil
}
}
@@ -171,10 +171,10 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
// TODO: need to support writing to other types of files. Unify the below code paths.
case logical.WriteToFile(path, child) =>
val relation =
- ParquetRelation.create(path, child, sparkContext.hadoopConfiguration, None)
- InsertIntoParquetTable(relation, planLater(child))(sparkContext) :: Nil
+ ParquetRelation.create(path, child, sparkContext.hadoopConfiguration)
+ InsertIntoParquetTable(relation, planLater(child), overwrite=true)(sparkContext) :: Nil
case logical.InsertIntoTable(table: ParquetRelation, partition, child, overwrite) =>
- InsertIntoParquetTable(table, planLater(child))(sparkContext) :: Nil
+ InsertIntoParquetTable(table, planLater(child), overwrite)(sparkContext) :: Nil
case PhysicalOperation(projectList, filters, relation: ParquetRelation) =>
// TODO: Should be pushing down filters as well.
pruneFilterProject(
@@ -213,8 +213,8 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
sparkContext.parallelize(data.map(r =>
new GenericRow(r.productIterator.map(convertToCatalyst).toArray): Row))
execution.ExistingRdd(output, dataAsRdd) :: Nil
- case logical.StopAfter(IntegerLiteral(limit), child) =>
- execution.StopAfter(limit, planLater(child))(sparkContext) :: Nil
+ case logical.Limit(IntegerLiteral(limit), child) =>
+ execution.Limit(limit, planLater(child))(sparkContext) :: Nil
case Unions(unionChildren) =>
execution.Union(unionChildren.map(planLater))(sparkContext) :: Nil
case logical.Generate(generator, join, outer, _, child) =>
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
index 65cb8f8becefa..524e5022ee14b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
@@ -19,27 +19,28 @@ package org.apache.spark.sql.execution
import scala.reflect.runtime.universe.TypeTag
-import org.apache.spark.rdd.RDD
-import org.apache.spark.SparkContext
-
+import org.apache.spark.{HashPartitioner, SparkConf, SparkContext}
+import org.apache.spark.rdd.{RDD, ShuffledRDD}
+import org.apache.spark.sql.catalyst.ScalaReflection
import org.apache.spark.sql.catalyst.errors._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.physical.{OrderedDistribution, UnspecifiedDistribution}
-import org.apache.spark.sql.catalyst.ScalaReflection
+import org.apache.spark.util.MutablePair
+
case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends UnaryNode {
- def output = projectList.map(_.toAttribute)
+ override def output = projectList.map(_.toAttribute)
- def execute() = child.execute().mapPartitions { iter =>
+ override def execute() = child.execute().mapPartitions { iter =>
@transient val reusableProjection = new MutableProjection(projectList)
iter.map(reusableProjection)
}
}
case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode {
- def output = child.output
+ override def output = child.output
- def execute() = child.execute().mapPartitions { iter =>
+ override def execute() = child.execute().mapPartitions { iter =>
iter.filter(condition.apply(_).asInstanceOf[Boolean])
}
}
@@ -47,37 +48,59 @@ case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode {
case class Sample(fraction: Double, withReplacement: Boolean, seed: Int, child: SparkPlan)
extends UnaryNode {
- def output = child.output
+ override def output = child.output
// TODO: How to pick seed?
- def execute() = child.execute().sample(withReplacement, fraction, seed)
+ override def execute() = child.execute().sample(withReplacement, fraction, seed)
}
case class Union(children: Seq[SparkPlan])(@transient sc: SparkContext) extends SparkPlan {
// TODO: attributes output by union should be distinct for nullability purposes
- def output = children.head.output
- def execute() = sc.union(children.map(_.execute()))
+ override def output = children.head.output
+ override def execute() = sc.union(children.map(_.execute()))
override def otherCopyArgs = sc :: Nil
}
-case class StopAfter(limit: Int, child: SparkPlan)(@transient sc: SparkContext) extends UnaryNode {
+/**
+ * Take the first limit elements. Note that the implementation is different depending on whether
+ * this is a terminal operator or not. If it is terminal and is invoked using executeCollect,
+ * this operator uses Spark's take method on the Spark driver. If it is not terminal or is
+ * invoked using execute, we first take the limit on each partition, and then repartition all the
+ * data to a single partition to compute the global limit.
+ */
+case class Limit(limit: Int, child: SparkPlan)(@transient sc: SparkContext) extends UnaryNode {
+ // TODO: Implement a partition local limit, and use a strategy to generate the proper limit plan:
+ // partition local limit -> exchange into one partition -> partition local limit again
+
override def otherCopyArgs = sc :: Nil
- def output = child.output
+ override def output = child.output
override def executeCollect() = child.execute().map(_.copy()).take(limit)
- // TODO: Terminal split should be implemented differently from non-terminal split.
- // TODO: Pick num splits based on |limit|.
- def execute() = sc.makeRDD(executeCollect(), 1)
+ override def execute() = {
+ val rdd = child.execute().mapPartitions { iter =>
+ val mutablePair = new MutablePair[Boolean, Row]()
+ iter.take(limit).map(row => mutablePair.update(false, row))
+ }
+ val part = new HashPartitioner(1)
+ val shuffled = new ShuffledRDD[Boolean, Row, MutablePair[Boolean, Row]](rdd, part)
+ shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false)))
+ shuffled.mapPartitions(_.take(limit).map(_._2))
+ }
}
-case class TopK(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan)
- (@transient sc: SparkContext) extends UnaryNode {
+/**
+ * Take the first limit elements as defined by the sortOrder. This is logically equivalent to
+ * having a [[Limit]] operator after a [[Sort]] operator. This could have been named TopK, but
+ * Spark's top operator does the opposite in ordering so we name it TakeOrdered to avoid confusion.
+ */
+case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan)
+ (@transient sc: SparkContext) extends UnaryNode {
override def otherCopyArgs = sc :: Nil
- def output = child.output
+ override def output = child.output
@transient
lazy val ordering = new RowOrdering(sortOrder)
@@ -86,7 +109,7 @@ case class TopK(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan)
// TODO: Terminal split should be implemented differently from non-terminal split.
// TODO: Pick num splits based on |limit|.
- def execute() = sc.makeRDD(executeCollect(), 1)
+ override def execute() = sc.makeRDD(executeCollect(), 1)
}
@@ -101,7 +124,7 @@ case class Sort(
@transient
lazy val ordering = new RowOrdering(sortOrder)
- def execute() = attachTree(this, "sort") {
+ override def execute() = attachTree(this, "sort") {
// TODO: Optimize sorting operation?
child.execute()
.mapPartitions(
@@ -109,7 +132,7 @@ case class Sort(
preservesPartitioning = true)
}
- def output = child.output
+ override def output = child.output
}
object ExistingRdd {
@@ -130,6 +153,6 @@ object ExistingRdd {
}
case class ExistingRdd(output: Seq[Attribute], rdd: RDD[Row]) extends LeafNode {
- def execute() = rdd
+ override def execute() = rdd
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala
index f0d21143ba5d1..c89dae9358bf7 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala
@@ -17,21 +17,22 @@
package org.apache.spark.sql.execution
-import scala.collection.mutable
+import scala.collection.mutable.{ArrayBuffer, BitSet}
-import org.apache.spark.rdd.RDD
import org.apache.spark.SparkContext
-import org.apache.spark.sql.catalyst.errors._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Partitioning}
-import org.apache.spark.rdd.PartitionLocalRDDFunctions._
+sealed abstract class BuildSide
+case object BuildLeft extends BuildSide
+case object BuildRight extends BuildSide
-case class SparkEquiInnerJoin(
+case class HashJoin(
leftKeys: Seq[Expression],
rightKeys: Seq[Expression],
+ buildSide: BuildSide,
left: SparkPlan,
right: SparkPlan) extends BinaryNode {
@@ -40,33 +41,93 @@ case class SparkEquiInnerJoin(
override def requiredChildDistribution =
ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil
+ val (buildPlan, streamedPlan) = buildSide match {
+ case BuildLeft => (left, right)
+ case BuildRight => (right, left)
+ }
+
+ val (buildKeys, streamedKeys) = buildSide match {
+ case BuildLeft => (leftKeys, rightKeys)
+ case BuildRight => (rightKeys, leftKeys)
+ }
+
def output = left.output ++ right.output
- def execute() = attachTree(this, "execute") {
- val leftWithKeys = left.execute().mapPartitions { iter =>
- val generateLeftKeys = new Projection(leftKeys, left.output)
- iter.map(row => (generateLeftKeys(row), row.copy()))
- }
+ @transient lazy val buildSideKeyGenerator = new Projection(buildKeys, buildPlan.output)
+ @transient lazy val streamSideKeyGenerator =
+ () => new MutableProjection(streamedKeys, streamedPlan.output)
- val rightWithKeys = right.execute().mapPartitions { iter =>
- val generateRightKeys = new Projection(rightKeys, right.output)
- iter.map(row => (generateRightKeys(row), row.copy()))
- }
+ def execute() = {
- // Do the join.
- val joined = filterNulls(leftWithKeys).joinLocally(filterNulls(rightWithKeys))
- // Drop join keys and merge input tuples.
- joined.map { case (_, (leftTuple, rightTuple)) => buildRow(leftTuple ++ rightTuple) }
- }
+ buildPlan.execute().zipPartitions(streamedPlan.execute()) { (buildIter, streamIter) =>
+ // TODO: Use Spark's HashMap implementation.
+ val hashTable = new java.util.HashMap[Row, ArrayBuffer[Row]]()
+ var currentRow: Row = null
+
+ // Create a mapping of buildKeys -> rows
+ while (buildIter.hasNext) {
+ currentRow = buildIter.next()
+ val rowKey = buildSideKeyGenerator(currentRow)
+ if(!rowKey.anyNull) {
+ val existingMatchList = hashTable.get(rowKey)
+ val matchList = if (existingMatchList == null) {
+ val newMatchList = new ArrayBuffer[Row]()
+ hashTable.put(rowKey, newMatchList)
+ newMatchList
+ } else {
+ existingMatchList
+ }
+ matchList += currentRow.copy()
+ }
+ }
+
+ new Iterator[Row] {
+ private[this] var currentStreamedRow: Row = _
+ private[this] var currentHashMatches: ArrayBuffer[Row] = _
+ private[this] var currentMatchPosition: Int = -1
- /**
- * Filters any rows where the any of the join keys is null, ensuring three-valued
- * logic for the equi-join conditions.
- */
- protected def filterNulls(rdd: RDD[(Row, Row)]) =
- rdd.filter {
- case (key: Seq[_], _) => !key.exists(_ == null)
+ // Mutable per row objects.
+ private[this] val joinRow = new JoinedRow
+
+ private[this] val joinKeys = streamSideKeyGenerator()
+
+ override final def hasNext: Boolean =
+ (currentMatchPosition != -1 && currentMatchPosition < currentHashMatches.size) ||
+ (streamIter.hasNext && fetchNext())
+
+ override final def next() = {
+ val ret = joinRow(currentStreamedRow, currentHashMatches(currentMatchPosition))
+ currentMatchPosition += 1
+ ret
+ }
+
+ /**
+ * Searches the streamed iterator for the next row that has at least one match in hashtable.
+ *
+ * @return true if the search is successful, and false the streamed iterator runs out of
+ * tuples.
+ */
+ private final def fetchNext(): Boolean = {
+ currentHashMatches = null
+ currentMatchPosition = -1
+
+ while (currentHashMatches == null && streamIter.hasNext) {
+ currentStreamedRow = streamIter.next()
+ if (!joinKeys(currentStreamedRow).anyNull) {
+ currentHashMatches = hashTable.get(joinKeys.currentValue)
+ }
+ }
+
+ if (currentHashMatches == null) {
+ false
+ } else {
+ currentMatchPosition = 0
+ true
+ }
+ }
+ }
}
+ }
}
case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNode {
@@ -95,17 +156,19 @@ case class BroadcastNestedLoopJoin(
def right = broadcast
@transient lazy val boundCondition =
- condition
- .map(c => BindReferences.bindReference(c, left.output ++ right.output))
- .getOrElse(Literal(true))
+ InterpretedPredicate(
+ condition
+ .map(c => BindReferences.bindReference(c, left.output ++ right.output))
+ .getOrElse(Literal(true)))
def execute() = {
val broadcastedRelation = sc.broadcast(broadcast.execute().map(_.copy()).collect().toIndexedSeq)
val streamedPlusMatches = streamed.execute().mapPartitions { streamedIter =>
- val matchedRows = new mutable.ArrayBuffer[Row]
- val includedBroadcastTuples = new mutable.BitSet(broadcastedRelation.value.size)
+ val matchedRows = new ArrayBuffer[Row]
+ // TODO: Use Spark's BitSet.
+ val includedBroadcastTuples = new BitSet(broadcastedRelation.value.size)
val joinedRow = new JoinedRow
streamedIter.foreach { streamedRow =>
@@ -115,7 +178,7 @@ case class BroadcastNestedLoopJoin(
while (i < broadcastedRelation.value.size) {
// TODO: One bitset per partition instead of per row.
val broadcastedRow = broadcastedRelation.value(i)
- if (boundCondition(joinedRow(streamedRow, broadcastedRow)).asInstanceOf[Boolean]) {
+ if (boundCondition(joinedRow(streamedRow, broadcastedRow))) {
matchedRows += buildRow(streamedRow ++ broadcastedRow)
matched = true
includedBroadcastTuples += i
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala
index 4ab755c096bd8..114bfbb719ee9 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala
@@ -17,30 +17,29 @@
package org.apache.spark.sql.parquet
-import java.io.{IOException, FileNotFoundException}
-
-import scala.collection.JavaConversions._
+import java.io.IOException
import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.fs.permission.FsAction
import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.fs.permission.FsAction
import org.apache.hadoop.mapreduce.Job
-import parquet.hadoop.metadata.{FileMetaData, ParquetMetadata}
import parquet.hadoop.util.ContextUtil
-import parquet.hadoop.{Footer, ParquetFileReader, ParquetFileWriter}
+import parquet.hadoop.{ParquetOutputFormat, Footer, ParquetFileWriter, ParquetFileReader}
+import parquet.hadoop.metadata.{CompressionCodecName, FileMetaData, ParquetMetadata}
import parquet.io.api.{Binary, RecordConsumer}
+import parquet.schema.{Type => ParquetType, PrimitiveType => ParquetPrimitiveType, MessageType, MessageTypeParser}
import parquet.schema.PrimitiveType.{PrimitiveTypeName => ParquetPrimitiveTypeName}
import parquet.schema.Type.Repetition
-import parquet.schema.{MessageType, MessageTypeParser}
-import parquet.schema.{PrimitiveType => ParquetPrimitiveType}
-import parquet.schema.{Type => ParquetType}
import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, UnresolvedException}
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Row}
-import org.apache.spark.sql.catalyst.plans.logical.{BaseRelation, LogicalPlan}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LeafNode}
import org.apache.spark.sql.catalyst.types._
+// Implicits
+import scala.collection.JavaConversions._
+
/**
* Relation that consists of data stored in a Parquet columnar format.
*
@@ -48,14 +47,14 @@ import org.apache.spark.sql.catalyst.types._
* of using this class directly.
*
* {{{
- * val parquetRDD = sqlContext.parquetFile("path/to/parequet.file")
+ * val parquetRDD = sqlContext.parquetFile("path/to/parquet.file")
* }}}
*
- * @param tableName The name of the relation that can be used in queries.
* @param path The path to the Parquet file.
*/
-case class ParquetRelation(tableName: String, path: String)
- extends BaseRelation with MultiInstanceRelation {
+private[sql] case class ParquetRelation(val path: String)
+ extends LeafNode with MultiInstanceRelation {
+ self: Product =>
/** Schema derived from ParquetFile */
def parquetSchema: MessageType =
@@ -65,33 +64,59 @@ case class ParquetRelation(tableName: String, path: String)
.getSchema
/** Attributes */
- val attributes =
+ override val output =
ParquetTypesConverter
- .convertToAttributes(parquetSchema)
+ .convertToAttributes(parquetSchema)
- /** Output */
- override val output = attributes
-
- // Parquet files have no concepts of keys, therefore no Partitioner
- // Note: we could allow Block level access; needs to be thought through
- override def isPartitioned = false
-
- override def newInstance = ParquetRelation(tableName, path).asInstanceOf[this.type]
+ override def newInstance = ParquetRelation(path).asInstanceOf[this.type]
// Equals must also take into account the output attributes so that we can distinguish between
// different instances of the same relation,
override def equals(other: Any) = other match {
case p: ParquetRelation =>
- p.tableName == tableName && p.path == path && p.output == output
+ p.path == path && p.output == output
case _ => false
}
}
-object ParquetRelation {
+private[sql] object ParquetRelation {
+
+ def enableLogForwarding() {
+ // Note: Parquet does not use forwarding to parent loggers which
+ // is required for the JUL-SLF4J bridge to work. Also there is
+ // a default logger that appends to Console which needs to be
+ // reset.
+ import org.slf4j.bridge.SLF4JBridgeHandler
+ import java.util.logging.Logger
+ import java.util.logging.LogManager
+
+ val loggerNames = Seq(
+ "parquet.hadoop.ColumnChunkPageWriteStore",
+ "parquet.hadoop.InternalParquetRecordWriter",
+ "parquet.hadoop.ParquetRecordReader",
+ "parquet.hadoop.ParquetInputFormat",
+ "parquet.hadoop.ParquetOutputFormat",
+ "parquet.hadoop.ParquetFileReader",
+ "parquet.hadoop.InternalParquetRecordReader",
+ "parquet.hadoop.codec.CodecConfig")
+ LogManager.getLogManager.reset()
+ SLF4JBridgeHandler.install()
+ for(name <- loggerNames) {
+ val logger = Logger.getLogger(name)
+ logger.setParent(Logger.getGlobal)
+ logger.setUseParentHandlers(true)
+ }
+ }
// The element type for the RDDs that this relation maps to.
type RowType = org.apache.spark.sql.catalyst.expressions.GenericMutableRow
+ // The compression type
+ type CompressionType = parquet.hadoop.metadata.CompressionCodecName
+
+ // The default compression
+ val defaultCompression = CompressionCodecName.GZIP
+
/**
* Creates a new ParquetRelation and underlying Parquetfile for the given LogicalPlan. Note that
* this is used inside [[org.apache.spark.sql.execution.SparkStrategies SparkStrategies]] to
@@ -100,24 +125,39 @@ object ParquetRelation {
*
* @param pathString The directory the Parquetfile will be stored in.
* @param child The child node that will be used for extracting the schema.
- * @param conf A configuration configuration to be used.
- * @param tableName The name of the resulting relation.
- * @return An empty ParquetRelation inferred metadata.
+ * @param conf A configuration to be used.
+ * @return An empty ParquetRelation with inferred metadata.
*/
def create(pathString: String,
child: LogicalPlan,
- conf: Configuration,
- tableName: Option[String]): ParquetRelation = {
+ conf: Configuration): ParquetRelation = {
if (!child.resolved) {
throw new UnresolvedException[LogicalPlan](
child,
"Attempt to create Parquet table from unresolved child (when schema is not available)")
}
+ createEmpty(pathString, child.output, conf)
+ }
- val name = s"${tableName.getOrElse(child.nodeName)}_parquet"
+ /**
+ * Creates an empty ParquetRelation and underlying Parquetfile that only
+ * consists of the Metadata for the given schema.
+ *
+ * @param pathString The directory the Parquetfile will be stored in.
+ * @param attributes The schema of the relation.
+ * @param conf A configuration to be used.
+ * @return An empty ParquetRelation.
+ */
+ def createEmpty(pathString: String,
+ attributes: Seq[Attribute],
+ conf: Configuration): ParquetRelation = {
val path = checkPath(pathString, conf)
- ParquetTypesConverter.writeMetaData(child.output, path, conf)
- new ParquetRelation(name, path.toString)
+ if (conf.get(ParquetOutputFormat.COMPRESSION) == null) {
+ conf.set(ParquetOutputFormat.COMPRESSION, ParquetRelation.defaultCompression.name())
+ }
+ ParquetRelation.enableLogForwarding()
+ ParquetTypesConverter.writeMetaData(attributes, path, conf)
+ new ParquetRelation(path.toString)
}
private def checkPath(pathStr: String, conf: Configuration): Path = {
@@ -143,7 +183,7 @@ object ParquetRelation {
}
}
-object ParquetTypesConverter {
+private[parquet] object ParquetTypesConverter {
def toDataType(parquetType : ParquetPrimitiveTypeName): DataType = parquetType match {
// for now map binary to string type
// TODO: figure out how Parquet uses strings or why we can't use them in a MessageType schema
@@ -242,6 +282,7 @@ object ParquetTypesConverter {
extraMetadata,
"Spark")
+ ParquetRelation.enableLogForwarding()
ParquetFileWriter.writeMetadataFile(
conf,
path,
@@ -268,16 +309,24 @@ object ParquetTypesConverter {
throw new IllegalArgumentException(s"Incorrectly formatted Parquet metadata path $origPath")
}
val path = origPath.makeQualified(fs)
+ if (!fs.getFileStatus(path).isDir) {
+ throw new IllegalArgumentException(
+ s"Expected $path for be a directory with Parquet files/metadata")
+ }
+ ParquetRelation.enableLogForwarding()
val metadataPath = new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE)
+ // if this is a new table that was just created we will find only the metadata file
if (fs.exists(metadataPath) && fs.isFile(metadataPath)) {
- // TODO: improve exception handling, etc.
ParquetFileReader.readFooter(conf, metadataPath)
} else {
- if (!fs.exists(path) || !fs.isFile(path)) {
- throw new FileNotFoundException(
- s"Could not find file ${path.toString} when trying to read metadata")
+ // there may be one or more Parquet files in the given directory
+ val footers = ParquetFileReader.readFooters(conf, fs.getFileStatus(path))
+ // TODO: for now we assume that all footers (if there is more than one) have identical
+ // metadata; we may want to add a check here at some point
+ if (footers.size() == 0) {
+ throw new IllegalArgumentException(s"Could not find Parquet metadata at path $path")
}
- ParquetFileReader.readFooter(conf, path)
+ footers(0).getParquetMetadata
}
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala
index 7285f5b88b9bf..d5846baa72ada 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala
@@ -24,26 +24,29 @@ import java.util.Date
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hadoop.mapreduce._
-import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat}
+import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat}
+import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat, FileOutputCommitter}
-import parquet.hadoop.util.ContextUtil
import parquet.hadoop.{ParquetInputFormat, ParquetOutputFormat}
+import parquet.hadoop.util.ContextUtil
import parquet.io.InvalidRecordException
import parquet.schema.MessageType
+import org.apache.spark.{SerializableWritable, SparkContext, TaskContext}
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Row}
import org.apache.spark.sql.execution.{LeafNode, SparkPlan, UnaryNode}
-import org.apache.spark.{SerializableWritable, SparkContext, TaskContext}
/**
* Parquet table scan operator. Imports the file that backs the given
* [[ParquetRelation]] as a RDD[Row].
*/
case class ParquetTableScan(
- @transient output: Seq[Attribute],
- @transient relation: ParquetRelation,
- @transient columnPruningPred: Option[Expression])(
+ // note: output cannot be transient, see
+ // https://issues.apache.org/jira/browse/SPARK-1367
+ output: Seq[Attribute],
+ relation: ParquetRelation,
+ columnPruningPred: Option[Expression])(
@transient val sc: SparkContext)
extends LeafNode {
@@ -53,6 +56,12 @@ case class ParquetTableScan(
job,
classOf[org.apache.spark.sql.parquet.RowReadSupport])
val conf: Configuration = ContextUtil.getConfiguration(job)
+ val fileList = FileSystemHelper.listFiles(relation.path, conf)
+ // add all paths in the directory but skip "hidden" ones such
+ // as "_SUCCESS" and "_metadata"
+ for (path <- fileList if !path.getName.startsWith("_")) {
+ NewFileInputFormat.addInputPath(job, path)
+ }
conf.set(
RowReadSupport.PARQUET_ROW_REQUESTED_SCHEMA,
ParquetTypesConverter.convertFromAttributes(output).toString)
@@ -63,14 +72,12 @@ case class ParquetTableScan(
``FilteredRecordReader`` (via Configuration, for example). Simple
filter-rows-by-column-values however should be supported.
*/
- sc.newAPIHadoopFile(
- relation.path,
- classOf[ParquetInputFormat[Row]],
- classOf[Void], classOf[Row],
- conf)
+ sc.newAPIHadoopRDD(conf, classOf[ParquetInputFormat[Row]], classOf[Void], classOf[Row])
.map(_._2)
}
+ override def otherCopyArgs = sc :: Nil
+
/**
* Applies a (candidate) projection.
*
@@ -108,15 +115,31 @@ case class ParquetTableScan(
}
}
+/**
+ * Operator that acts as a sink for queries on RDDs and can be used to
+ * store the output inside a directory of Parquet files. This operator
+ * is similar to Hive's INSERT INTO TABLE operation in the sense that
+ * one can choose to either overwrite or append to a directory. Note
+ * that consecutive insertions to the same table must have compatible
+ * (source) schemas.
+ *
+ * WARNING: EXPERIMENTAL! InsertIntoParquetTable with overwrite=false may
+ * cause data corruption in the case that multiple users try to append to
+ * the same table simultaneously. Inserting into a table that was
+ * previously generated by other means (e.g., by creating an HDFS
+ * directory and importing Parquet files generated by other tools) may
+ * cause unpredicted behaviour and therefore results in a RuntimeException
+ * (only detected via filename pattern so will not catch all cases).
+ */
case class InsertIntoParquetTable(
- @transient relation: ParquetRelation,
- @transient child: SparkPlan)(
+ relation: ParquetRelation,
+ child: SparkPlan,
+ overwrite: Boolean = false)(
@transient val sc: SparkContext)
extends UnaryNode with SparkHadoopMapReduceUtil {
/**
- * Inserts all the rows in the Parquet file. Note that OVERWRITE is implicit, since
- * Parquet files are write-once.
+ * Inserts all rows into the Parquet file.
*/
override def execute() = {
// TODO: currently we do not check whether the "schema"s are compatible
@@ -135,19 +158,21 @@ case class InsertIntoParquetTable(
classOf[org.apache.spark.sql.parquet.RowWriteSupport])
// TODO: move that to function in object
- val conf = job.getConfiguration
+ val conf = ContextUtil.getConfiguration(job)
conf.set(RowWriteSupport.PARQUET_ROW_SCHEMA, relation.parquetSchema.toString)
val fspath = new Path(relation.path)
val fs = fspath.getFileSystem(conf)
- try {
- fs.delete(fspath, true)
- } catch {
- case e: IOException =>
- throw new IOException(
- s"Unable to clear output directory ${fspath.toString} prior"
- + s" to InsertIntoParquetTable:\n${e.toString}")
+ if (overwrite) {
+ try {
+ fs.delete(fspath, true)
+ } catch {
+ case e: IOException =>
+ throw new IOException(
+ s"Unable to clear output directory ${fspath.toString} prior"
+ + s" to InsertIntoParquetTable:\n${e.toString}")
+ }
}
saveAsHadoopFile(childRdd, relation.path.toString, conf)
@@ -157,6 +182,8 @@ case class InsertIntoParquetTable(
override def output = child.output
+ override def otherCopyArgs = sc :: Nil
+
// based on ``saveAsNewAPIHadoopFile`` in [[PairRDDFunctions]]
// TODO: Maybe PairRDDFunctions should use Product2 instead of Tuple2?
// .. then we could use the default one and could use [[MutablePair]]
@@ -167,15 +194,21 @@ case class InsertIntoParquetTable(
conf: Configuration) {
val job = new Job(conf)
val keyType = classOf[Void]
- val outputFormatType = classOf[parquet.hadoop.ParquetOutputFormat[Row]]
job.setOutputKeyClass(keyType)
job.setOutputValueClass(classOf[Row])
- val wrappedConf = new SerializableWritable(job.getConfiguration)
NewFileOutputFormat.setOutputPath(job, new Path(path))
+ val wrappedConf = new SerializableWritable(job.getConfiguration)
val formatter = new SimpleDateFormat("yyyyMMddHHmm")
val jobtrackerID = formatter.format(new Date())
val stageId = sc.newRddId()
+ val taskIdOffset =
+ if (overwrite) 1
+ else {
+ FileSystemHelper
+ .findMaxTaskId(NewFileOutputFormat.getOutputPath(job).toString, job.getConfiguration) + 1
+ }
+
def writeShard(context: TaskContext, iter: Iterator[Row]): Int = {
// Hadoop wants a 32-bit task attempt ID, so if ours is bigger than Int.MaxValue, roll it
// around by taking a mod. We expect that no task will be attempted 2 billion times.
@@ -184,7 +217,7 @@ case class InsertIntoParquetTable(
val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.partitionId,
attemptNumber)
val hadoopContext = newTaskAttemptContext(wrappedConf.value, attemptId)
- val format = outputFormatType.newInstance
+ val format = new AppendingParquetOutputFormat(taskIdOffset)
val committer = format.getOutputCommitter(hadoopContext)
committer.setupTask(hadoopContext)
val writer = format.getRecordWriter(hadoopContext)
@@ -196,7 +229,7 @@ case class InsertIntoParquetTable(
committer.commitTask(hadoopContext)
return 1
}
- val jobFormat = outputFormatType.newInstance
+ val jobFormat = new AppendingParquetOutputFormat(taskIdOffset)
/* apparently we need a TaskAttemptID to construct an OutputCommitter;
* however we're only going to use this local OutputCommitter for
* setupJob/commitJob, so we just use a dummy "map" task.
@@ -210,3 +243,55 @@ case class InsertIntoParquetTable(
}
}
+// TODO: this will be able to append to directories it created itself, not necessarily
+// to imported ones
+private[parquet] class AppendingParquetOutputFormat(offset: Int)
+ extends parquet.hadoop.ParquetOutputFormat[Row] {
+ // override to accept existing directories as valid output directory
+ override def checkOutputSpecs(job: JobContext): Unit = {}
+
+ // override to choose output filename so not overwrite existing ones
+ override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = {
+ val taskId: TaskID = context.getTaskAttemptID.getTaskID
+ val partition: Int = taskId.getId
+ val filename = s"part-r-${partition + offset}.parquet"
+ val committer: FileOutputCommitter =
+ getOutputCommitter(context).asInstanceOf[FileOutputCommitter]
+ new Path(committer.getWorkPath, filename)
+ }
+}
+
+private[parquet] object FileSystemHelper {
+ def listFiles(pathStr: String, conf: Configuration): Seq[Path] = {
+ val origPath = new Path(pathStr)
+ val fs = origPath.getFileSystem(conf)
+ if (fs == null) {
+ throw new IllegalArgumentException(
+ s"ParquetTableOperations: Path $origPath is incorrectly formatted")
+ }
+ val path = origPath.makeQualified(fs)
+ if (!fs.exists(path) || !fs.getFileStatus(path).isDir) {
+ throw new IllegalArgumentException(
+ s"ParquetTableOperations: path $path does not exist or is not a directory")
+ }
+ fs.listStatus(path).map(_.getPath)
+ }
+
+ // finds the maximum taskid in the output file names at the given path
+ def findMaxTaskId(pathStr: String, conf: Configuration): Int = {
+ val files = FileSystemHelper.listFiles(pathStr, conf)
+ // filename pattern is part-r-.parquet
+ val nameP = new scala.util.matching.Regex("""part-r-(\d{1,}).parquet""", "taskid")
+ val hiddenFileP = new scala.util.matching.Regex("_.*")
+ files.map(_.getName).map {
+ case nameP(taskid) => taskid.toInt
+ case hiddenFileP() => 0
+ case other: String => {
+ sys.error("ERROR: attempting to append to set of Parquet files and found file" +
+ s"that does not match name pattern: $other")
+ 0
+ }
+ case _ => 0
+ }.reduceLeft((a, b) => if (a < b) b else a)
+ }
+}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala
index c21e400282004..84b1b4609458b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala
@@ -35,7 +35,8 @@ import org.apache.spark.sql.catalyst.types._
*
*@param root The root group converter for the record.
*/
-class RowRecordMaterializer(root: CatalystGroupConverter) extends RecordMaterializer[Row] {
+private[parquet] class RowRecordMaterializer(root: CatalystGroupConverter)
+ extends RecordMaterializer[Row] {
def this(parquetSchema: MessageType) =
this(new CatalystGroupConverter(ParquetTypesConverter.convertToAttributes(parquetSchema)))
@@ -48,14 +49,14 @@ class RowRecordMaterializer(root: CatalystGroupConverter) extends RecordMaterial
/**
* A `parquet.hadoop.api.ReadSupport` for Row objects.
*/
-class RowReadSupport extends ReadSupport[Row] with Logging {
+private[parquet] class RowReadSupport extends ReadSupport[Row] with Logging {
override def prepareForRead(
conf: Configuration,
stringMap: java.util.Map[String, String],
fileSchema: MessageType,
readContext: ReadContext): RecordMaterializer[Row] = {
- log.debug(s"preparing for read with schema ${fileSchema.toString}")
+ log.debug(s"preparing for read with file schema $fileSchema")
new RowRecordMaterializer(readContext.getRequestedSchema)
}
@@ -67,20 +68,20 @@ class RowReadSupport extends ReadSupport[Row] with Logging {
configuration.get(RowReadSupport.PARQUET_ROW_REQUESTED_SCHEMA, fileSchema.toString)
val requested_schema =
MessageTypeParser.parseMessageType(requested_schema_string)
-
- log.debug(s"read support initialized for original schema ${requested_schema.toString}")
+ log.debug(s"read support initialized for requested schema $requested_schema")
+ ParquetRelation.enableLogForwarding()
new ReadContext(requested_schema, keyValueMetaData)
}
}
-object RowReadSupport {
+private[parquet] object RowReadSupport {
val PARQUET_ROW_REQUESTED_SCHEMA = "org.apache.spark.sql.parquet.row.requested_schema"
}
/**
* A `parquet.hadoop.api.WriteSupport` for Row ojects.
*/
-class RowWriteSupport extends WriteSupport[Row] with Logging {
+private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging {
def setSchema(schema: MessageType, configuration: Configuration) {
// for testing
this.schema = schema
@@ -104,6 +105,8 @@ class RowWriteSupport extends WriteSupport[Row] with Logging {
override def init(configuration: Configuration): WriteSupport.WriteContext = {
schema = if (schema == null) getSchema(configuration) else schema
attributes = ParquetTypesConverter.convertToAttributes(schema)
+ log.debug(s"write support initialized for requested schema $schema")
+ ParquetRelation.enableLogForwarding()
new WriteSupport.WriteContext(
schema,
new java.util.HashMap[java.lang.String, java.lang.String]())
@@ -111,10 +114,16 @@ class RowWriteSupport extends WriteSupport[Row] with Logging {
override def prepareForWrite(recordConsumer: RecordConsumer): Unit = {
writer = recordConsumer
+ log.debug(s"preparing for write with schema $schema")
}
// TODO: add groups (nested fields)
override def write(record: Row): Unit = {
+ if (attributes.size > record.size) {
+ throw new IndexOutOfBoundsException(
+ s"Trying to write more fields than contained in row (${attributes.size}>${record.size})")
+ }
+
var index = 0
writer.startMessage()
while(index < attributes.size) {
@@ -130,7 +139,7 @@ class RowWriteSupport extends WriteSupport[Row] with Logging {
}
}
-object RowWriteSupport {
+private[parquet] object RowWriteSupport {
val PARQUET_ROW_SCHEMA: String = "org.apache.spark.sql.parquet.row.schema"
}
@@ -139,7 +148,7 @@ object RowWriteSupport {
*
* @param schema The corresponding Catalyst schema in the form of a list of attributes.
*/
-class CatalystGroupConverter(
+private[parquet] class CatalystGroupConverter(
schema: Seq[Attribute],
protected[parquet] val current: ParquetRelation.RowType) extends GroupConverter {
@@ -177,13 +186,12 @@ class CatalystGroupConverter(
* @param parent The parent group converter.
* @param fieldIndex The index inside the record.
*/
-class CatalystPrimitiveConverter(
+private[parquet] class CatalystPrimitiveConverter(
parent: CatalystGroupConverter,
fieldIndex: Int) extends PrimitiveConverter {
// TODO: consider refactoring these together with ParquetTypesConverter
override def addBinary(value: Binary): Unit =
- // TODO: fix this once a setBinary will become available in MutableRow
- parent.getCurrentRecord.setByte(fieldIndex, value.getBytes.apply(0))
+ parent.getCurrentRecord.update(fieldIndex, value.getBytes)
override def addBoolean(value: Boolean): Unit =
parent.getCurrentRecord.setBoolean(fieldIndex, value)
@@ -208,10 +216,9 @@ class CatalystPrimitiveConverter(
* @param parent The parent group converter.
* @param fieldIndex The index inside the record.
*/
-class CatalystPrimitiveStringConverter(
+private[parquet] class CatalystPrimitiveStringConverter(
parent: CatalystGroupConverter,
fieldIndex: Int) extends CatalystPrimitiveConverter(parent, fieldIndex) {
override def addBinary(value: Binary): Unit =
parent.getCurrentRecord.setString(fieldIndex, value.toStringUsingUTF8)
}
-
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala
index 3340c3ff81f0a..728e3dd1dc02b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala
@@ -26,7 +26,7 @@ import parquet.hadoop.util.ContextUtil
import parquet.schema.{MessageType, MessageTypeParser}
import org.apache.spark.sql.catalyst.expressions.GenericRow
-import org.apache.spark.sql.catalyst.util.getTempFilePath
+import org.apache.spark.util.Utils
object ParquetTestData {
@@ -64,13 +64,13 @@ object ParquetTestData {
"mylong:Long"
)
- val testFile = getTempFilePath("testParquetFile").getCanonicalFile
+ val testDir = Utils.createTempDir()
- lazy val testData = new ParquetRelation("testData", testFile.toURI.toString)
+ lazy val testData = new ParquetRelation(testDir.toURI.toString)
def writeFile() = {
- testFile.delete
- val path: Path = new Path(testFile.toURI)
+ testDir.delete
+ val path: Path = new Path(new Path(testDir.toURI), new Path("part-r-0.parquet"))
val job = new Job()
val configuration: Configuration = ContextUtil.getConfiguration(job)
val schema: MessageType = MessageTypeParser.parseMessageType(testSchema)
diff --git a/sql/core/src/test/resources/log4j.properties b/sql/core/src/test/resources/log4j.properties
index 7bb6789bd33a5..dffd15a61838b 100644
--- a/sql/core/src/test/resources/log4j.properties
+++ b/sql/core/src/test/resources/log4j.properties
@@ -45,8 +45,6 @@ log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=OFF
log4j.additivity.hive.ql.metadata.Hive=false
log4j.logger.hive.ql.metadata.Hive=OFF
-# Parquet logging
-parquet.hadoop.InternalParquetRecordReader=WARN
-log4j.logger.parquet.hadoop.InternalParquetRecordReader=WARN
-parquet.hadoop.ParquetInputFormat=WARN
-log4j.logger.parquet.hadoop.ParquetInputFormat=WARN
+# Parquet related logging
+log4j.logger.parquet.hadoop=WARN
+log4j.logger.org.apache.spark.sql.parquet=INFO
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala
new file mode 100644
index 0000000000000..e5902c3cae381
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.scalatest.FunSuite
+import org.apache.spark.sql.TestData._
+import org.apache.spark.sql.test.TestSQLContext
+import org.apache.spark.sql.execution.SparkLogicalPlan
+import org.apache.spark.sql.columnar.InMemoryColumnarTableScan
+
+class CachedTableSuite extends QueryTest {
+ TestData // Load test tables.
+
+ test("read from cached table and uncache") {
+ TestSQLContext.cacheTable("testData")
+
+ checkAnswer(
+ TestSQLContext.table("testData"),
+ testData.collect().toSeq
+ )
+
+ TestSQLContext.table("testData").queryExecution.analyzed match {
+ case SparkLogicalPlan(_ : InMemoryColumnarTableScan) => // Found evidence of caching
+ case noCache => fail(s"No cache node found in plan $noCache")
+ }
+
+ TestSQLContext.uncacheTable("testData")
+
+ checkAnswer(
+ TestSQLContext.table("testData"),
+ testData.collect().toSeq
+ )
+
+ TestSQLContext.table("testData").queryExecution.analyzed match {
+ case cachePlan @ SparkLogicalPlan(_ : InMemoryColumnarTableScan) =>
+ fail(s"Table still cached after uncache: $cachePlan")
+ case noCache => // Table uncached successfully
+ }
+ }
+
+ test("correct error on uncache of non-cached table") {
+ intercept[IllegalArgumentException] {
+ TestSQLContext.uncacheTable("testData")
+ }
+ }
+}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala
new file mode 100644
index 0000000000000..65eae3357a21e
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import java.sql.Timestamp
+
+import org.scalatest.FunSuite
+
+import org.apache.spark.sql.test.TestSQLContext._
+
+case class ReflectData(
+ stringField: String,
+ intField: Int,
+ longField: Long,
+ floatField: Float,
+ doubleField: Double,
+ shortField: Short,
+ byteField: Byte,
+ booleanField: Boolean,
+ decimalField: BigDecimal,
+ timestampField: Timestamp,
+ seqInt: Seq[Int])
+
+case class ReflectBinary(data: Array[Byte])
+
+class ScalaReflectionRelationSuite extends FunSuite {
+ test("query case class RDD") {
+ val data = ReflectData("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true,
+ BigDecimal(1), new Timestamp(12345), Seq(1,2,3))
+ val rdd = sparkContext.parallelize(data :: Nil)
+ rdd.registerAsTable("reflectData")
+
+ assert(sql("SELECT * FROM reflectData").collect().head === data.productIterator.toSeq)
+ }
+
+ // Equality is broken for Arrays, so we test that separately.
+ test("query binary data") {
+ val rdd = sparkContext.parallelize(ReflectBinary(Array[Byte](1)) :: Nil)
+ rdd.registerAsTable("reflectBinary")
+
+ val result = sql("SELECT data FROM reflectBinary").collect().head(0).asInstanceOf[Array[Byte]]
+ assert(result.toSeq === Seq[Byte](1))
+ }
+}
\ No newline at end of file
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala
new file mode 100644
index 0000000000000..def0e046a3831
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.api.java
+
+import scala.beans.BeanProperty
+
+import org.scalatest.FunSuite
+
+import org.apache.spark.api.java.JavaSparkContext
+import org.apache.spark.sql.test.TestSQLContext
+
+// Implicits
+import scala.collection.JavaConversions._
+
+class PersonBean extends Serializable {
+ @BeanProperty
+ var name: String = _
+
+ @BeanProperty
+ var age: Int = _
+}
+
+class JavaSQLSuite extends FunSuite {
+ val javaCtx = new JavaSparkContext(TestSQLContext.sparkContext)
+ val javaSqlCtx = new JavaSQLContext(javaCtx)
+
+ test("schema from JavaBeans") {
+ val person = new PersonBean
+ person.setName("Michael")
+ person.setAge(29)
+
+ val rdd = javaCtx.parallelize(person :: Nil)
+ val schemaRDD = javaSqlCtx.applySchema(rdd, classOf[PersonBean])
+
+ schemaRDD.registerAsTable("people")
+ javaSqlCtx.sql("SELECT * FROM people").collect()
+ }
+}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala
new file mode 100644
index 0000000000000..78640b876d4aa
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.columnar
+
+import org.scalatest.FunSuite
+
+import org.apache.spark.sql.catalyst.types._
+
+class ColumnStatsSuite extends FunSuite {
+ testColumnStats(classOf[BooleanColumnStats], BOOLEAN)
+ testColumnStats(classOf[ByteColumnStats], BYTE)
+ testColumnStats(classOf[ShortColumnStats], SHORT)
+ testColumnStats(classOf[IntColumnStats], INT)
+ testColumnStats(classOf[LongColumnStats], LONG)
+ testColumnStats(classOf[FloatColumnStats], FLOAT)
+ testColumnStats(classOf[DoubleColumnStats], DOUBLE)
+ testColumnStats(classOf[StringColumnStats], STRING)
+
+ def testColumnStats[T <: NativeType, U <: NativeColumnStats[T]](
+ columnStatsClass: Class[U],
+ columnType: NativeColumnType[T]) {
+
+ val columnStatsName = columnStatsClass.getSimpleName
+
+ test(s"$columnStatsName: empty") {
+ val columnStats = columnStatsClass.newInstance()
+ expectResult(columnStats.initialBounds, "Wrong initial bounds") {
+ (columnStats.lowerBound, columnStats.upperBound)
+ }
+ }
+
+ test(s"$columnStatsName: non-empty") {
+ import ColumnarTestUtils._
+
+ val columnStats = columnStatsClass.newInstance()
+ val rows = Seq.fill(10)(makeRandomRow(columnType))
+ rows.foreach(columnStats.gatherStats(_, 0))
+
+ val values = rows.map(_.head.asInstanceOf[T#JvmType])
+ val ordering = columnType.dataType.ordering.asInstanceOf[Ordering[T#JvmType]]
+
+ expectResult(values.min(ordering), "Wrong lower bound")(columnStats.lowerBound)
+ expectResult(values.max(ordering), "Wrong upper bound")(columnStats.upperBound)
+ }
+ }
+}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala
index 2d431affbcfcc..1d3608ed2d9ab 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala
@@ -19,46 +19,56 @@ package org.apache.spark.sql.columnar
import java.nio.ByteBuffer
-import scala.util.Random
-
import org.scalatest.FunSuite
import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.columnar.ColumnarTestUtils._
import org.apache.spark.sql.execution.SparkSqlSerializer
class ColumnTypeSuite extends FunSuite {
- val columnTypes = Seq(INT, SHORT, LONG, BYTE, DOUBLE, FLOAT, STRING, BINARY, GENERIC)
+ val DEFAULT_BUFFER_SIZE = 512
test("defaultSize") {
- val defaultSize = Seq(4, 2, 8, 1, 8, 4, 8, 16, 16)
+ val checks = Map(
+ INT -> 4, SHORT -> 2, LONG -> 8, BYTE -> 1, DOUBLE -> 8, FLOAT -> 4,
+ BOOLEAN -> 1, STRING -> 8, BINARY -> 16, GENERIC -> 16)
- columnTypes.zip(defaultSize).foreach { case (columnType, size) =>
- assert(columnType.defaultSize === size)
+ checks.foreach { case (columnType, expectedSize) =>
+ expectResult(expectedSize, s"Wrong defaultSize for $columnType") {
+ columnType.defaultSize
+ }
}
}
test("actualSize") {
- val expectedSizes = Seq(4, 2, 8, 1, 8, 4, 4 + 5, 4 + 4, 4 + 11)
- val actualSizes = Seq(
- INT.actualSize(Int.MaxValue),
- SHORT.actualSize(Short.MaxValue),
- LONG.actualSize(Long.MaxValue),
- BYTE.actualSize(Byte.MaxValue),
- DOUBLE.actualSize(Double.MaxValue),
- FLOAT.actualSize(Float.MaxValue),
- STRING.actualSize("hello"),
- BINARY.actualSize(new Array[Byte](4)),
- GENERIC.actualSize(SparkSqlSerializer.serialize(Map(1 -> "a"))))
-
- expectedSizes.zip(actualSizes).foreach { case (expected, actual) =>
- assert(expected === actual)
+ def checkActualSize[T <: DataType, JvmType](
+ columnType: ColumnType[T, JvmType],
+ value: JvmType,
+ expected: Int) {
+
+ expectResult(expected, s"Wrong actualSize for $columnType") {
+ columnType.actualSize(value)
+ }
}
+
+ checkActualSize(INT, Int.MaxValue, 4)
+ checkActualSize(SHORT, Short.MaxValue, 2)
+ checkActualSize(LONG, Long.MaxValue, 8)
+ checkActualSize(BYTE, Byte.MaxValue, 1)
+ checkActualSize(DOUBLE, Double.MaxValue, 8)
+ checkActualSize(FLOAT, Float.MaxValue, 4)
+ checkActualSize(BOOLEAN, true, 1)
+ checkActualSize(STRING, "hello", 4 + 5)
+
+ val binary = Array.fill[Byte](4)(0: Byte)
+ checkActualSize(BINARY, binary, 4 + 4)
+
+ val generic = Map(1 -> "a")
+ checkActualSize(GENERIC, SparkSqlSerializer.serialize(generic), 4 + 11)
}
- testNumericColumnType[BooleanType.type, Boolean](
+ testNativeColumnType[BooleanType.type](
BOOLEAN,
- Array.fill(4)(Random.nextBoolean()),
- ByteBuffer.allocate(32),
(buffer: ByteBuffer, v: Boolean) => {
buffer.put((if (v) 1 else 0).toByte)
},
@@ -66,105 +76,42 @@ class ColumnTypeSuite extends FunSuite {
buffer.get() == 1
})
- testNumericColumnType[IntegerType.type, Int](
- INT,
- Array.fill(4)(Random.nextInt()),
- ByteBuffer.allocate(32),
- (_: ByteBuffer).putInt(_),
- (_: ByteBuffer).getInt)
-
- testNumericColumnType[ShortType.type, Short](
- SHORT,
- Array.fill(4)(Random.nextInt(Short.MaxValue).asInstanceOf[Short]),
- ByteBuffer.allocate(32),
- (_: ByteBuffer).putShort(_),
- (_: ByteBuffer).getShort)
-
- testNumericColumnType[LongType.type, Long](
- LONG,
- Array.fill(4)(Random.nextLong()),
- ByteBuffer.allocate(64),
- (_: ByteBuffer).putLong(_),
- (_: ByteBuffer).getLong)
-
- testNumericColumnType[ByteType.type, Byte](
- BYTE,
- Array.fill(4)(Random.nextInt(Byte.MaxValue).asInstanceOf[Byte]),
- ByteBuffer.allocate(64),
- (_: ByteBuffer).put(_),
- (_: ByteBuffer).get)
-
- testNumericColumnType[DoubleType.type, Double](
- DOUBLE,
- Array.fill(4)(Random.nextDouble()),
- ByteBuffer.allocate(64),
- (_: ByteBuffer).putDouble(_),
- (_: ByteBuffer).getDouble)
-
- testNumericColumnType[FloatType.type, Float](
- FLOAT,
- Array.fill(4)(Random.nextFloat()),
- ByteBuffer.allocate(64),
- (_: ByteBuffer).putFloat(_),
- (_: ByteBuffer).getFloat)
-
- test("STRING") {
- val buffer = ByteBuffer.allocate(128)
- val seq = Array("hello", "world", "spark", "sql")
-
- seq.map(_.getBytes).foreach { bytes: Array[Byte] =>
- buffer.putInt(bytes.length).put(bytes)
- }
+ testNativeColumnType[IntegerType.type](INT, _.putInt(_), _.getInt)
- buffer.rewind()
- seq.foreach { s =>
- assert(s === STRING.extract(buffer))
- }
+ testNativeColumnType[ShortType.type](SHORT, _.putShort(_), _.getShort)
- buffer.rewind()
- seq.foreach(STRING.append(_, buffer))
+ testNativeColumnType[LongType.type](LONG, _.putLong(_), _.getLong)
- buffer.rewind()
- seq.foreach { s =>
- val length = buffer.getInt
- assert(length === s.getBytes.length)
+ testNativeColumnType[ByteType.type](BYTE, _.put(_), _.get)
+
+ testNativeColumnType[DoubleType.type](DOUBLE, _.putDouble(_), _.getDouble)
+
+ testNativeColumnType[FloatType.type](FLOAT, _.putFloat(_), _.getFloat)
+ testNativeColumnType[StringType.type](
+ STRING,
+ (buffer: ByteBuffer, string: String) => {
+ val bytes = string.getBytes()
+ buffer.putInt(bytes.length).put(string.getBytes)
+ },
+ (buffer: ByteBuffer) => {
+ val length = buffer.getInt()
val bytes = new Array[Byte](length)
buffer.get(bytes, 0, length)
- assert(s === new String(bytes))
- }
- }
-
- test("BINARY") {
- val buffer = ByteBuffer.allocate(128)
- val seq = Array.fill(4) {
- val bytes = new Array[Byte](4)
- Random.nextBytes(bytes)
- bytes
- }
+ new String(bytes)
+ })
- seq.foreach { bytes =>
+ testColumnType[BinaryType.type, Array[Byte]](
+ BINARY,
+ (buffer: ByteBuffer, bytes: Array[Byte]) => {
buffer.putInt(bytes.length).put(bytes)
- }
-
- buffer.rewind()
- seq.foreach { b =>
- assert(b === BINARY.extract(buffer))
- }
-
- buffer.rewind()
- seq.foreach(BINARY.append(_, buffer))
-
- buffer.rewind()
- seq.foreach { b =>
- val length = buffer.getInt
- assert(length === b.length)
-
+ },
+ (buffer: ByteBuffer) => {
+ val length = buffer.getInt()
val bytes = new Array[Byte](length)
buffer.get(bytes, 0, length)
- assert(b === bytes)
- }
- }
+ bytes
+ })
test("GENERIC") {
val buffer = ByteBuffer.allocate(512)
@@ -177,43 +124,58 @@ class ColumnTypeSuite extends FunSuite {
val length = buffer.getInt()
assert(length === serializedObj.length)
- val bytes = new Array[Byte](length)
- buffer.get(bytes, 0, length)
- assert(obj === SparkSqlSerializer.deserialize(bytes))
+ expectResult(obj, "Deserialized object didn't equal to the original object") {
+ val bytes = new Array[Byte](length)
+ buffer.get(bytes, 0, length)
+ SparkSqlSerializer.deserialize(bytes)
+ }
buffer.rewind()
buffer.putInt(serializedObj.length).put(serializedObj)
- buffer.rewind()
- assert(obj === SparkSqlSerializer.deserialize(GENERIC.extract(buffer)))
+ expectResult(obj, "Deserialized object didn't equal to the original object") {
+ buffer.rewind()
+ SparkSqlSerializer.deserialize(GENERIC.extract(buffer))
+ }
+ }
+
+ def testNativeColumnType[T <: NativeType](
+ columnType: NativeColumnType[T],
+ putter: (ByteBuffer, T#JvmType) => Unit,
+ getter: (ByteBuffer) => T#JvmType) {
+
+ testColumnType[T, T#JvmType](columnType, putter, getter)
}
- def testNumericColumnType[T <: DataType, JvmType](
+ def testColumnType[T <: DataType, JvmType](
columnType: ColumnType[T, JvmType],
- seq: Seq[JvmType],
- buffer: ByteBuffer,
putter: (ByteBuffer, JvmType) => Unit,
getter: (ByteBuffer) => JvmType) {
- val columnTypeName = columnType.getClass.getSimpleName.stripSuffix("$")
+ val buffer = ByteBuffer.allocate(DEFAULT_BUFFER_SIZE)
+ val seq = (0 until 4).map(_ => makeRandomValue(columnType))
- test(s"$columnTypeName.extract") {
+ test(s"$columnType.extract") {
buffer.rewind()
seq.foreach(putter(buffer, _))
buffer.rewind()
- seq.foreach { i =>
- assert(i === columnType.extract(buffer))
+ seq.foreach { expected =>
+ assert(
+ expected === columnType.extract(buffer),
+ "Extracted value didn't equal to the original one")
}
}
- test(s"$columnTypeName.append") {
+ test(s"$columnType.append") {
buffer.rewind()
seq.foreach(columnType.append(_, buffer))
buffer.rewind()
- seq.foreach { i =>
- assert(i === getter(buffer))
+ seq.foreach { expected =>
+ assert(
+ expected === getter(buffer),
+ "Extracted value didn't equal to the original one")
}
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala
index 928851a385d41..70b2e851737f8 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala
@@ -17,11 +17,11 @@
package org.apache.spark.sql.columnar
+import org.apache.spark.sql.{QueryTest, TestData}
import org.apache.spark.sql.execution.SparkLogicalPlan
import org.apache.spark.sql.test.TestSQLContext
-import org.apache.spark.sql.{TestData, DslQuerySuite}
-class ColumnarQuerySuite extends DslQuerySuite {
+class ColumnarQuerySuite extends QueryTest {
import TestData._
import TestSQLContext._
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestData.scala
deleted file mode 100644
index ddcdede8d1a4a..0000000000000
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestData.scala
+++ /dev/null
@@ -1,55 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.columnar
-
-import scala.util.Random
-
-import org.apache.spark.sql.catalyst.expressions.GenericMutableRow
-
-// TODO Enrich test data
-object ColumnarTestData {
- object GenericMutableRow {
- def apply(values: Any*) = {
- val row = new GenericMutableRow(values.length)
- row.indices.foreach { i =>
- row(i) = values(i)
- }
- row
- }
- }
-
- def randomBytes(length: Int) = {
- val bytes = new Array[Byte](length)
- Random.nextBytes(bytes)
- bytes
- }
-
- val nonNullRandomRow = GenericMutableRow(
- Random.nextInt(),
- Random.nextLong(),
- Random.nextFloat(),
- Random.nextDouble(),
- Random.nextBoolean(),
- Random.nextInt(Byte.MaxValue).asInstanceOf[Byte],
- Random.nextInt(Short.MaxValue).asInstanceOf[Short],
- Random.nextString(Random.nextInt(64)),
- randomBytes(Random.nextInt(64)),
- Map(Random.nextInt() -> Random.nextString(4)))
-
- val nullRow = GenericMutableRow(Seq.fill(10)(null): _*)
-}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala
new file mode 100644
index 0000000000000..04bdc43d95328
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.columnar
+
+import scala.collection.immutable.HashSet
+import scala.util.Random
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.expressions.GenericMutableRow
+import org.apache.spark.sql.catalyst.types.{DataType, NativeType}
+
+object ColumnarTestUtils {
+ def makeNullRow(length: Int) = {
+ val row = new GenericMutableRow(length)
+ (0 until length).foreach(row.setNullAt)
+ row
+ }
+
+ def makeRandomValue[T <: DataType, JvmType](columnType: ColumnType[T, JvmType]): JvmType = {
+ def randomBytes(length: Int) = {
+ val bytes = new Array[Byte](length)
+ Random.nextBytes(bytes)
+ bytes
+ }
+
+ (columnType match {
+ case BYTE => (Random.nextInt(Byte.MaxValue * 2) - Byte.MaxValue).toByte
+ case SHORT => (Random.nextInt(Short.MaxValue * 2) - Short.MaxValue).toShort
+ case INT => Random.nextInt()
+ case LONG => Random.nextLong()
+ case FLOAT => Random.nextFloat()
+ case DOUBLE => Random.nextDouble()
+ case STRING => Random.nextString(Random.nextInt(32))
+ case BOOLEAN => Random.nextBoolean()
+ case BINARY => randomBytes(Random.nextInt(32))
+ case _ =>
+ // Using a random one-element map instead of an arbitrary object
+ Map(Random.nextInt() -> Random.nextString(Random.nextInt(32)))
+ }).asInstanceOf[JvmType]
+ }
+
+ def makeRandomValues(
+ head: ColumnType[_ <: DataType, _],
+ tail: ColumnType[_ <: DataType, _]*): Seq[Any] = makeRandomValues(Seq(head) ++ tail)
+
+ def makeRandomValues(columnTypes: Seq[ColumnType[_ <: DataType, _]]): Seq[Any] = {
+ columnTypes.map(makeRandomValue(_))
+ }
+
+ def makeUniqueRandomValues[T <: DataType, JvmType](
+ columnType: ColumnType[T, JvmType],
+ count: Int): Seq[JvmType] = {
+
+ Iterator.iterate(HashSet.empty[JvmType]) { set =>
+ set + Iterator.continually(makeRandomValue(columnType)).filterNot(set.contains).next()
+ }.drop(count).next().toSeq
+ }
+
+ def makeRandomRow(
+ head: ColumnType[_ <: DataType, _],
+ tail: ColumnType[_ <: DataType, _]*): Row = makeRandomRow(Seq(head) ++ tail)
+
+ def makeRandomRow(columnTypes: Seq[ColumnType[_ <: DataType, _]]): Row = {
+ val row = new GenericMutableRow(columnTypes.length)
+ makeRandomValues(columnTypes).zipWithIndex.foreach { case (value, index) =>
+ row(index) = value
+ }
+ row
+ }
+
+ def makeUniqueValuesAndSingleValueRows[T <: NativeType](
+ columnType: NativeColumnType[T],
+ count: Int) = {
+
+ val values = makeUniqueRandomValues(columnType, count)
+ val rows = values.map { value =>
+ val row = new GenericMutableRow(1)
+ row(0) = value
+ row
+ }
+
+ (values, rows)
+ }
+
+}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala
index d413d483f4e7e..4a21eb6201a69 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala
@@ -17,12 +17,29 @@
package org.apache.spark.sql.columnar
+import java.nio.ByteBuffer
+
import org.scalatest.FunSuite
-import org.apache.spark.sql.catalyst.types.DataType
+
import org.apache.spark.sql.catalyst.expressions.GenericMutableRow
+import org.apache.spark.sql.catalyst.types.DataType
+
+class TestNullableColumnAccessor[T <: DataType, JvmType](
+ buffer: ByteBuffer,
+ columnType: ColumnType[T, JvmType])
+ extends BasicColumnAccessor(buffer, columnType)
+ with NullableColumnAccessor
+
+object TestNullableColumnAccessor {
+ def apply[T <: DataType, JvmType](buffer: ByteBuffer, columnType: ColumnType[T, JvmType]) = {
+ // Skips the column type ID
+ buffer.getInt()
+ new TestNullableColumnAccessor(buffer, columnType)
+ }
+}
class NullableColumnAccessorSuite extends FunSuite {
- import ColumnarTestData._
+ import ColumnarTestUtils._
Seq(INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, BINARY, GENERIC).foreach {
testNullableColumnAccessor(_)
@@ -30,30 +47,32 @@ class NullableColumnAccessorSuite extends FunSuite {
def testNullableColumnAccessor[T <: DataType, JvmType](columnType: ColumnType[T, JvmType]) {
val typeName = columnType.getClass.getSimpleName.stripSuffix("$")
+ val nullRow = makeNullRow(1)
- test(s"$typeName accessor: empty column") {
- val builder = ColumnBuilder(columnType.typeId, 4)
- val accessor = ColumnAccessor(builder.build())
+ test(s"Nullable $typeName column accessor: empty column") {
+ val builder = TestNullableColumnBuilder(columnType)
+ val accessor = TestNullableColumnAccessor(builder.build(), columnType)
assert(!accessor.hasNext)
}
- test(s"$typeName accessor: access null values") {
- val builder = ColumnBuilder(columnType.typeId, 4)
+ test(s"Nullable $typeName column accessor: access null values") {
+ val builder = TestNullableColumnBuilder(columnType)
+ val randomRow = makeRandomRow(columnType)
(0 until 4).foreach { _ =>
- builder.appendFrom(nonNullRandomRow, columnType.typeId)
- builder.appendFrom(nullRow, columnType.typeId)
+ builder.appendFrom(randomRow, 0)
+ builder.appendFrom(nullRow, 0)
}
- val accessor = ColumnAccessor(builder.build())
+ val accessor = TestNullableColumnAccessor(builder.build(), columnType)
val row = new GenericMutableRow(1)
(0 until 4).foreach { _ =>
accessor.extractTo(row, 0)
- assert(row(0) === nonNullRandomRow(columnType.typeId))
+ assert(row(0) === randomRow(0))
accessor.extractTo(row, 0)
- assert(row(0) === null)
+ assert(row.isNullAt(0))
}
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala
index 5222a47e1ab87..d9d1e1bfddb75 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala
@@ -19,63 +19,71 @@ package org.apache.spark.sql.columnar
import org.scalatest.FunSuite
-import org.apache.spark.sql.catalyst.types.DataType
+import org.apache.spark.sql.catalyst.types._
import org.apache.spark.sql.execution.SparkSqlSerializer
+class TestNullableColumnBuilder[T <: DataType, JvmType](columnType: ColumnType[T, JvmType])
+ extends BasicColumnBuilder[T, JvmType](new NoopColumnStats[T, JvmType], columnType)
+ with NullableColumnBuilder
+
+object TestNullableColumnBuilder {
+ def apply[T <: DataType, JvmType](columnType: ColumnType[T, JvmType], initialSize: Int = 0) = {
+ val builder = new TestNullableColumnBuilder(columnType)
+ builder.initialize(initialSize)
+ builder
+ }
+}
+
class NullableColumnBuilderSuite extends FunSuite {
- import ColumnarTestData._
+ import ColumnarTestUtils._
Seq(INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, BINARY, GENERIC).foreach {
testNullableColumnBuilder(_)
}
def testNullableColumnBuilder[T <: DataType, JvmType](columnType: ColumnType[T, JvmType]) {
- val columnBuilder = ColumnBuilder(columnType.typeId)
val typeName = columnType.getClass.getSimpleName.stripSuffix("$")
test(s"$typeName column builder: empty column") {
- columnBuilder.initialize(4)
-
+ val columnBuilder = TestNullableColumnBuilder(columnType)
val buffer = columnBuilder.build()
- // For column type ID
- assert(buffer.getInt() === columnType.typeId)
- // For null count
- assert(buffer.getInt === 0)
+ expectResult(columnType.typeId, "Wrong column type ID")(buffer.getInt())
+ expectResult(0, "Wrong null count")(buffer.getInt())
assert(!buffer.hasRemaining)
}
test(s"$typeName column builder: buffer size auto growth") {
- columnBuilder.initialize(4)
+ val columnBuilder = TestNullableColumnBuilder(columnType)
+ val randomRow = makeRandomRow(columnType)
- (0 until 4) foreach { _ =>
- columnBuilder.appendFrom(nonNullRandomRow, columnType.typeId)
+ (0 until 4).foreach { _ =>
+ columnBuilder.appendFrom(randomRow, 0)
}
val buffer = columnBuilder.build()
- // For column type ID
- assert(buffer.getInt() === columnType.typeId)
- // For null count
- assert(buffer.getInt() === 0)
+ expectResult(columnType.typeId, "Wrong column type ID")(buffer.getInt())
+ expectResult(0, "Wrong null count")(buffer.getInt())
}
test(s"$typeName column builder: null values") {
- columnBuilder.initialize(4)
+ val columnBuilder = TestNullableColumnBuilder(columnType)
+ val randomRow = makeRandomRow(columnType)
+ val nullRow = makeNullRow(1)
- (0 until 4) foreach { _ =>
- columnBuilder.appendFrom(nonNullRandomRow, columnType.typeId)
- columnBuilder.appendFrom(nullRow, columnType.typeId)
+ (0 until 4).foreach { _ =>
+ columnBuilder.appendFrom(randomRow, 0)
+ columnBuilder.appendFrom(nullRow, 0)
}
val buffer = columnBuilder.build()
- // For column type ID
- assert(buffer.getInt() === columnType.typeId)
- // For null count
- assert(buffer.getInt() === 4)
+ expectResult(columnType.typeId, "Wrong column type ID")(buffer.getInt())
+ expectResult(4, "Wrong null count")(buffer.getInt())
+
// For null positions
- (1 to 7 by 2).foreach(i => assert(buffer.getInt() === i))
+ (1 to 7 by 2).foreach(expectResult(_, "Wrong null position")(buffer.getInt()))
// For non-null values
(0 until 4).foreach { _ =>
@@ -84,7 +92,8 @@ class NullableColumnBuilderSuite extends FunSuite {
} else {
columnType.extract(buffer)
}
- assert(actual === nonNullRandomRow(columnType.typeId))
+
+ assert(actual === randomRow(0), "Extracted value didn't equal to the original one")
}
assert(!buffer.hasRemaining)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala
new file mode 100644
index 0000000000000..184691ab5b46a
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.columnar.compression
+
+import java.nio.ByteBuffer
+
+import org.scalatest.FunSuite
+
+import org.apache.spark.sql.catalyst.types.NativeType
+import org.apache.spark.sql.columnar._
+import org.apache.spark.sql.columnar.ColumnarTestUtils._
+import org.apache.spark.sql.catalyst.expressions.GenericMutableRow
+
+class DictionaryEncodingSuite extends FunSuite {
+ testDictionaryEncoding(new IntColumnStats, INT)
+ testDictionaryEncoding(new LongColumnStats, LONG)
+ testDictionaryEncoding(new StringColumnStats, STRING)
+
+ def testDictionaryEncoding[T <: NativeType](
+ columnStats: NativeColumnStats[T],
+ columnType: NativeColumnType[T]) {
+
+ val typeName = columnType.getClass.getSimpleName.stripSuffix("$")
+
+ def buildDictionary(buffer: ByteBuffer) = {
+ (0 until buffer.getInt()).map(columnType.extract(buffer) -> _.toShort).toMap
+ }
+
+ test(s"$DictionaryEncoding with $typeName: simple case") {
+ // -------------
+ // Tests encoder
+ // -------------
+
+ val builder = TestCompressibleColumnBuilder(columnStats, columnType, DictionaryEncoding)
+ val (values, rows) = makeUniqueValuesAndSingleValueRows(columnType, 2)
+
+ builder.initialize(0)
+ builder.appendFrom(rows(0), 0)
+ builder.appendFrom(rows(1), 0)
+ builder.appendFrom(rows(0), 0)
+ builder.appendFrom(rows(1), 0)
+
+ val buffer = builder.build()
+ val headerSize = CompressionScheme.columnHeaderSize(buffer)
+ // 4 extra bytes for dictionary size
+ val dictionarySize = 4 + values.map(columnType.actualSize).sum
+ // 4 `Short`s, 2 bytes each
+ val compressedSize = dictionarySize + 2 * 4
+ // 4 extra bytes for compression scheme type ID
+ expectResult(headerSize + 4 + compressedSize, "Wrong buffer capacity")(buffer.capacity)
+
+ // Skips column header
+ buffer.position(headerSize)
+ expectResult(DictionaryEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt())
+
+ val dictionary = buildDictionary(buffer)
+ Array[Short](0, 1).foreach { i =>
+ expectResult(i, "Wrong dictionary entry")(dictionary(values(i)))
+ }
+
+ Array[Short](0, 1, 0, 1).foreach {
+ expectResult(_, "Wrong column element value")(buffer.getShort())
+ }
+
+ // -------------
+ // Tests decoder
+ // -------------
+
+ // Rewinds, skips column header and 4 more bytes for compression scheme ID
+ buffer.rewind().position(headerSize + 4)
+
+ val decoder = new DictionaryEncoding.Decoder[T](buffer, columnType)
+
+ Array[Short](0, 1, 0, 1).foreach { i =>
+ expectResult(values(i), "Wrong decoded value")(decoder.next())
+ }
+
+ assert(!decoder.hasNext)
+ }
+ }
+
+ test(s"$DictionaryEncoding: overflow") {
+ val builder = TestCompressibleColumnBuilder(new IntColumnStats, INT, DictionaryEncoding)
+ builder.initialize(0)
+
+ (0 to Short.MaxValue).foreach { n =>
+ val row = new GenericMutableRow(1)
+ row.setInt(0, n)
+ builder.appendFrom(row, 0)
+ }
+
+ withClue("Dictionary overflowed, encoding should fail") {
+ intercept[Throwable] {
+ builder.build()
+ }
+ }
+ }
+}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala
new file mode 100644
index 0000000000000..2089ad120d4f2
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.columnar.compression
+
+import org.scalatest.FunSuite
+
+import org.apache.spark.sql.catalyst.types.NativeType
+import org.apache.spark.sql.columnar._
+import org.apache.spark.sql.columnar.ColumnarTestUtils._
+
+class RunLengthEncodingSuite extends FunSuite {
+ testRunLengthEncoding(new BooleanColumnStats, BOOLEAN)
+ testRunLengthEncoding(new ByteColumnStats, BYTE)
+ testRunLengthEncoding(new ShortColumnStats, SHORT)
+ testRunLengthEncoding(new IntColumnStats, INT)
+ testRunLengthEncoding(new LongColumnStats, LONG)
+ testRunLengthEncoding(new StringColumnStats, STRING)
+
+ def testRunLengthEncoding[T <: NativeType](
+ columnStats: NativeColumnStats[T],
+ columnType: NativeColumnType[T]) {
+
+ val typeName = columnType.getClass.getSimpleName.stripSuffix("$")
+
+ test(s"$RunLengthEncoding with $typeName: simple case") {
+ // -------------
+ // Tests encoder
+ // -------------
+
+ val builder = TestCompressibleColumnBuilder(columnStats, columnType, RunLengthEncoding)
+ val (values, rows) = makeUniqueValuesAndSingleValueRows(columnType, 2)
+
+ builder.initialize(0)
+ builder.appendFrom(rows(0), 0)
+ builder.appendFrom(rows(0), 0)
+ builder.appendFrom(rows(1), 0)
+ builder.appendFrom(rows(1), 0)
+
+ val buffer = builder.build()
+ val headerSize = CompressionScheme.columnHeaderSize(buffer)
+ // 4 extra bytes each run for run length
+ val compressedSize = values.map(columnType.actualSize(_) + 4).sum
+ // 4 extra bytes for compression scheme type ID
+ expectResult(headerSize + 4 + compressedSize, "Wrong buffer capacity")(buffer.capacity)
+
+ // Skips column header
+ buffer.position(headerSize)
+ expectResult(RunLengthEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt())
+
+ Array(0, 1).foreach { i =>
+ expectResult(values(i), "Wrong column element value")(columnType.extract(buffer))
+ expectResult(2, "Wrong run length")(buffer.getInt())
+ }
+
+ // -------------
+ // Tests decoder
+ // -------------
+
+ // Rewinds, skips column header and 4 more bytes for compression scheme ID
+ buffer.rewind().position(headerSize + 4)
+
+ val decoder = new RunLengthEncoding.Decoder[T](buffer, columnType)
+
+ Array(0, 0, 1, 1).foreach { i =>
+ expectResult(values(i), "Wrong decoded value")(decoder.next())
+ }
+
+ assert(!decoder.hasNext)
+ }
+
+ test(s"$RunLengthEncoding with $typeName: run length == 1") {
+ // -------------
+ // Tests encoder
+ // -------------
+
+ val builder = TestCompressibleColumnBuilder(columnStats, columnType, RunLengthEncoding)
+ val (values, rows) = makeUniqueValuesAndSingleValueRows(columnType, 2)
+
+ builder.initialize(0)
+ builder.appendFrom(rows(0), 0)
+ builder.appendFrom(rows(1), 0)
+
+ val buffer = builder.build()
+ val headerSize = CompressionScheme.columnHeaderSize(buffer)
+ // 4 bytes each run for run length
+ val compressedSize = values.map(columnType.actualSize(_) + 4).sum
+ // 4 bytes for compression scheme type ID
+ expectResult(headerSize + 4 + compressedSize, "Wrong buffer capacity")(buffer.capacity)
+
+ // Skips column header
+ buffer.position(headerSize)
+ expectResult(RunLengthEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt())
+
+ Array(0, 1).foreach { i =>
+ expectResult(values(i), "Wrong column element value")(columnType.extract(buffer))
+ expectResult(1, "Wrong run length")(buffer.getInt())
+ }
+
+ // -------------
+ // Tests decoder
+ // -------------
+
+ // Rewinds, skips column header and 4 more bytes for compression scheme ID
+ buffer.rewind().position(headerSize + 4)
+
+ val decoder = new RunLengthEncoding.Decoder[T](buffer, columnType)
+
+ Array(0, 1).foreach { i =>
+ expectResult(values(i), "Wrong decoded value")(decoder.next())
+ }
+
+ assert(!decoder.hasNext)
+ }
+ }
+}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala
new file mode 100644
index 0000000000000..e0ec812863dcf
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.columnar.compression
+
+import org.apache.spark.sql.catalyst.types.NativeType
+import org.apache.spark.sql.columnar._
+
+class TestCompressibleColumnBuilder[T <: NativeType](
+ override val columnStats: NativeColumnStats[T],
+ override val columnType: NativeColumnType[T],
+ override val schemes: Seq[CompressionScheme])
+ extends NativeColumnBuilder(columnStats, columnType)
+ with NullableColumnBuilder
+ with CompressibleColumnBuilder[T] {
+
+ override protected def isWorthCompressing(encoder: Encoder) = true
+}
+
+object TestCompressibleColumnBuilder {
+ def apply[T <: NativeType](
+ columnStats: NativeColumnStats[T],
+ columnType: NativeColumnType[T],
+ scheme: CompressionScheme) = {
+
+ new TestCompressibleColumnBuilder(columnStats, columnType, Seq(scheme))
+ }
+}
+
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala
index ea1733b3614e5..a62a3c4d02354 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala
@@ -19,27 +19,40 @@ package org.apache.spark.sql.parquet
import org.scalatest.{BeforeAndAfterAll, FunSuite}
-import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.fs.{Path, FileSystem}
import org.apache.hadoop.mapreduce.Job
+
import parquet.hadoop.ParquetFileWriter
-import parquet.hadoop.util.ContextUtil
import parquet.schema.MessageTypeParser
+import parquet.hadoop.util.ContextUtil
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.expressions.Row
import org.apache.spark.sql.catalyst.util.getTempFilePath
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Row}
import org.apache.spark.sql.test.TestSQLContext
+import org.apache.spark.util.Utils
+import org.apache.spark.sql.catalyst.types.{StringType, IntegerType, DataType}
+import org.apache.spark.sql.{parquet, SchemaRDD}
+import org.apache.spark.sql.catalyst.expressions.AttributeReference
+import scala.Tuple2
// Implicits
import org.apache.spark.sql.test.TestSQLContext._
+case class TestRDDEntry(key: Int, value: String)
+
class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll {
+
+ var testRDD: SchemaRDD = null
+
override def beforeAll() {
ParquetTestData.writeFile()
+ testRDD = parquetFile(ParquetTestData.testDir.toString)
+ testRDD.registerAsTable("testsource")
}
override def afterAll() {
- ParquetTestData.testFile.delete()
+ Utils.deleteRecursively(ParquetTestData.testDir)
+ // here we should also unregister the table??
}
test("self-join parquet files") {
@@ -55,11 +68,18 @@ class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll {
case Seq(_, _) => // All good
}
- // TODO: We can't run this query as it NPEs
+ val result = query.collect()
+ assert(result.size === 9, "self-join result has incorrect size")
+ assert(result(0).size === 12, "result row has incorrect size")
+ result.zipWithIndex.foreach {
+ case (row, index) => row.zipWithIndex.foreach {
+ case (field, column) => assert(field != null, s"self-join contains null value in row $index field $column")
+ }
+ }
}
test("Import of simple Parquet file") {
- val result = getRDD(ParquetTestData.testData).collect()
+ val result = parquetFile(ParquetTestData.testDir.toString).collect()
assert(result.size === 15)
result.zipWithIndex.foreach {
case (row, index) => {
@@ -125,20 +145,82 @@ class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll {
fs.delete(path, true)
}
+ test("Creating case class RDD table") {
+ TestSQLContext.sparkContext.parallelize((1 to 100))
+ .map(i => TestRDDEntry(i, s"val_$i"))
+ .registerAsTable("tmp")
+ val rdd = sql("SELECT * FROM tmp").collect().sortBy(_.getInt(0))
+ var counter = 1
+ rdd.foreach {
+ // '===' does not like string comparison?
+ row: Row => {
+ assert(row.getString(1).equals(s"val_$counter"), s"row $counter value ${row.getString(1)} does not match val_$counter")
+ counter = counter + 1
+ }
+ }
+ }
+
+ test("Saving case class RDD table to file and reading it back in") {
+ val file = getTempFilePath("parquet")
+ val path = file.toString
+ val rdd = TestSQLContext.sparkContext.parallelize((1 to 100))
+ .map(i => TestRDDEntry(i, s"val_$i"))
+ rdd.saveAsParquetFile(path)
+ val readFile = parquetFile(path)
+ readFile.registerAsTable("tmpx")
+ val rdd_copy = sql("SELECT * FROM tmpx").collect()
+ val rdd_orig = rdd.collect()
+ for(i <- 0 to 99) {
+ assert(rdd_copy(i).apply(0) === rdd_orig(i).key, s"key error in line $i")
+ assert(rdd_copy(i).apply(1) === rdd_orig(i).value, s"value in line $i")
+ }
+ Utils.deleteRecursively(file)
+ assert(true)
+ }
+
+ test("insert (overwrite) via Scala API (new SchemaRDD)") {
+ val dirname = Utils.createTempDir()
+ val source_rdd = TestSQLContext.sparkContext.parallelize((1 to 100))
+ .map(i => TestRDDEntry(i, s"val_$i"))
+ source_rdd.registerAsTable("source")
+ val dest_rdd = createParquetFile(dirname.toString, ("key", IntegerType), ("value", StringType))
+ dest_rdd.registerAsTable("dest")
+ sql("INSERT OVERWRITE INTO dest SELECT * FROM source").collect()
+ val rdd_copy1 = sql("SELECT * FROM dest").collect()
+ assert(rdd_copy1.size === 100)
+ assert(rdd_copy1(0).apply(0) === 1)
+ assert(rdd_copy1(0).apply(1) === "val_1")
+ sql("INSERT INTO dest SELECT * FROM source").collect()
+ val rdd_copy2 = sql("SELECT * FROM dest").collect()
+ assert(rdd_copy2.size === 200)
+ Utils.deleteRecursively(dirname)
+ }
+
+ test("insert (appending) to same table via Scala API") {
+ sql("INSERT INTO testsource SELECT * FROM testsource").collect()
+ val double_rdd = sql("SELECT * FROM testsource").collect()
+ assert(double_rdd != null)
+ assert(double_rdd.size === 30)
+ for(i <- (0 to 14)) {
+ assert(double_rdd(i) === double_rdd(i+15), s"error: lines $i and ${i+15} to not match")
+ }
+ // let's restore the original test data
+ Utils.deleteRecursively(ParquetTestData.testDir)
+ ParquetTestData.writeFile()
+ }
+
/**
- * Computes the given [[ParquetRelation]] and returns its RDD.
+ * Creates an empty SchemaRDD backed by a ParquetRelation.
*
- * @param parquetRelation The Parquet relation.
- * @return An RDD of Rows.
+ * TODO: since this is so experimental it is better to have it here and not
+ * in SQLContext. Also note that when creating new AttributeReferences
+ * one needs to take care not to create duplicate Attribute ID's.
*/
- private def getRDD(parquetRelation: ParquetRelation): RDD[Row] = {
- val scanner = new ParquetTableScan(
- parquetRelation.output,
- parquetRelation,
- None)(TestSQLContext.sparkContext)
- scanner
- .execute
- .map(_.copy())
+ private def createParquetFile(path: String, schema: (Tuple2[String, DataType])*): SchemaRDD = {
+ val attributes = schema.map(t => new AttributeReference(t._1, t._2)())
+ new SchemaRDD(
+ TestSQLContext,
+ parquet.ParquetRelation.createEmpty(path, attributes, sparkContext.hadoopConfiguration))
}
}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
index fc5057b73fe24..ff8eaacded4c8 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
@@ -71,6 +71,18 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
override def executePlan(plan: LogicalPlan): this.QueryExecution =
new this.QueryExecution { val logical = plan }
+ /**
+ * Executes a query expressed in HiveQL using Spark, returning the result as a SchemaRDD.
+ */
+ def hql(hqlQuery: String): SchemaRDD = {
+ val result = new SchemaRDD(this, HiveQl.parseSql(hqlQuery))
+ // We force query optimization to happen right away instead of letting it happen lazily like
+ // when using the query DSL. This is so DDL commands behave as expected. This is only
+ // generates the RDD lineage for DML queries, but do not perform any execution.
+ result.queryExecution.toRdd
+ result
+ }
+
// Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur.
@transient
protected val outputBuffer = new java.io.OutputStream {
@@ -188,13 +200,13 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
val hiveContext = self
override val strategies: Seq[Strategy] = Seq(
- TopK,
+ TakeOrdered,
ParquetOperations,
HiveTableScans,
DataSinks,
Scripts,
PartialAggregation,
- SparkEquiInnerJoin,
+ HashJoin,
BasicOperators,
CartesianProduct,
BroadcastNestedLoopJoin
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
index 4f8353666a12b..fc053c56c052d 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
@@ -141,6 +141,15 @@ class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging {
*/
override def registerTable(
databaseName: Option[String], tableName: String, plan: LogicalPlan): Unit = ???
+
+ /**
+ * UNIMPLEMENTED: It needs to be decided how we will persist in-memory tables to the metastore.
+ * For now, if this functionality is desired mix in the in-memory [[OverrideCatalog]].
+ */
+ override def unregisterTable(
+ databaseName: Option[String], tableName: String): Unit = ???
+
+ override def unregisterAllTables() = {}
}
object HiveMetastoreTypes extends RegexParsers {
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
index 490a592a588d0..4dac25b3f60e4 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
@@ -300,14 +300,17 @@ object HiveQl {
}
protected def nodeToDataType(node: Node): DataType = node match {
- case Token("TOK_BIGINT", Nil) => IntegerType
+ case Token("TOK_DECIMAL", Nil) => DecimalType
+ case Token("TOK_BIGINT", Nil) => LongType
case Token("TOK_INT", Nil) => IntegerType
- case Token("TOK_TINYINT", Nil) => IntegerType
- case Token("TOK_SMALLINT", Nil) => IntegerType
+ case Token("TOK_TINYINT", Nil) => ByteType
+ case Token("TOK_SMALLINT", Nil) => ShortType
case Token("TOK_BOOLEAN", Nil) => BooleanType
case Token("TOK_STRING", Nil) => StringType
case Token("TOK_FLOAT", Nil) => FloatType
- case Token("TOK_DOUBLE", Nil) => FloatType
+ case Token("TOK_DOUBLE", Nil) => DoubleType
+ case Token("TOK_TIMESTAMP", Nil) => TimestampType
+ case Token("TOK_BINARY", Nil) => BinaryType
case Token("TOK_LIST", elementType :: Nil) => ArrayType(nodeToDataType(elementType))
case Token("TOK_STRUCT",
Token("TOK_TABCOLLIST", fields) :: Nil) =>
@@ -529,7 +532,7 @@ object HiveQl {
val withLimit =
limitClause.map(l => nodeToExpr(l.getChildren.head))
- .map(StopAfter(_, withSort))
+ .map(Limit(_, withSort))
.getOrElse(withSort)
// TOK_INSERT_INTO means to add files to the table.
@@ -602,7 +605,7 @@ object HiveQl {
case Token("TOK_TABLESPLITSAMPLE",
Token("TOK_ROWCOUNT", Nil) ::
Token(count, Nil) :: Nil) =>
- StopAfter(Literal(count.toInt), relation)
+ Limit(Literal(count.toInt), relation)
case Token("TOK_TABLESPLITSAMPLE",
Token("TOK_PERCENT", Nil) ::
Token(fraction, Nil) :: Nil) =>
@@ -829,6 +832,8 @@ object HiveQl {
Cast(nodeToExpr(arg), BooleanType)
case Token("TOK_FUNCTION", Token("TOK_DECIMAL", Nil) :: arg :: Nil) =>
Cast(nodeToExpr(arg), DecimalType)
+ case Token("TOK_FUNCTION", Token("TOK_TIMESTAMP", Nil) :: arg :: Nil) =>
+ Cast(nodeToExpr(arg), TimestampType)
/* Arithmetic */
case Token("-", child :: Nil) => UnaryMinus(nodeToExpr(child))
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
index bc3447b9d802d..0a6bea0162430 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
@@ -313,6 +313,8 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) {
catalog.client.dropDatabase(db, true, false, true)
}
+ catalog.unregisterAllTables()
+
FunctionRegistry.getFunctionNames.filterNot(originalUdfs.contains(_)).foreach { udfName =>
FunctionRegistry.unregisterTemporaryUDF(udfName)
}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala
new file mode 100644
index 0000000000000..6df76fa825101
--- /dev/null
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hive.api.java
+
+import org.apache.spark.api.java.JavaSparkContext
+import org.apache.spark.sql.api.java.{JavaSQLContext, JavaSchemaRDD}
+import org.apache.spark.sql.hive.{HiveContext, HiveQl}
+
+/**
+ * The entry point for executing Spark SQL queries from a Java program.
+ */
+class JavaHiveContext(sparkContext: JavaSparkContext) extends JavaSQLContext(sparkContext) {
+
+ override val sqlContext = new HiveContext(sparkContext)
+
+ /**
+ * Executes a query expressed in HiveQL, returning the result as a JavaSchemaRDD.
+ */
+ def hql(hqlQuery: String): JavaSchemaRDD = {
+ val result = new JavaSchemaRDD(sqlContext, HiveQl.parseSql(hqlQuery))
+ // We force query optimization to happen right away instead of letting it happen lazily like
+ // when using the query DSL. This is so DDL commands behave as expected. This is only
+ // generates the RDD lineage for DML queries, but do not perform any execution.
+ result.queryExecution.toRdd
+ result
+ }
+}
diff --git a/sql/hive/src/test/resources/golden/alias.*-0-7bdb861d11e895aaea545810cdac316d b/sql/hive/src/test/resources/golden/alias.*-0-7bdb861d11e895aaea545810cdac316d
deleted file mode 100644
index 5f4de85940513..0000000000000
--- a/sql/hive/src/test/resources/golden/alias.*-0-7bdb861d11e895aaea545810cdac316d
+++ /dev/null
@@ -1 +0,0 @@
-0 val_0
\ No newline at end of file
diff --git a/sql/hive/src/test/resources/golden/alias.star-0-7bdb861d11e895aaea545810cdac316d b/sql/hive/src/test/resources/golden/alias.star-0-7bdb861d11e895aaea545810cdac316d
new file mode 100644
index 0000000000000..016f64cc26f2a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/alias.star-0-7bdb861d11e895aaea545810cdac316d
@@ -0,0 +1 @@
+0 val_0
diff --git a/sql/hive/src/test/resources/golden/insert1-0-7faa9807151781e4207103aa568e321c b/sql/hive/src/test/resources/golden/insert1-0-7faa9807151781e4207103aa568e321c
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert1-1-91d7b05c9024bff60b55f415cbeacc8b b/sql/hive/src/test/resources/golden/insert1-1-91d7b05c9024bff60b55f415cbeacc8b
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert1-10-64f83491a8fe675ef3a4a9a474ac0439 b/sql/hive/src/test/resources/golden/insert1-10-64f83491a8fe675ef3a4a9a474ac0439
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert1-11-6f2797b6f81943d3b53b8d247ae8512b b/sql/hive/src/test/resources/golden/insert1-11-6f2797b6f81943d3b53b8d247ae8512b
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert1-12-7a3c0a3f06484c912b9e951d8a2d8ac6 b/sql/hive/src/test/resources/golden/insert1-12-7a3c0a3f06484c912b9e951d8a2d8ac6
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert1-13-42b03f938894fdafc7fff640711a9b2f b/sql/hive/src/test/resources/golden/insert1-13-42b03f938894fdafc7fff640711a9b2f
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert1-14-e021dfb28597811870c03b3242972927 b/sql/hive/src/test/resources/golden/insert1-14-e021dfb28597811870c03b3242972927
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert1-15-c7fca497a4580b54a0a13b3b72da5d7c b/sql/hive/src/test/resources/golden/insert1-15-c7fca497a4580b54a0a13b3b72da5d7c
new file mode 100644
index 0000000000000..5be49cad9a8ba
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/insert1-15-c7fca497a4580b54a0a13b3b72da5d7c
@@ -0,0 +1,2 @@
+db2_insert1
+db2_insert2
diff --git a/sql/hive/src/test/resources/golden/insert1-16-7a9e67189d3d4151f23b12c22bde06b5 b/sql/hive/src/test/resources/golden/insert1-16-7a9e67189d3d4151f23b12c22bde06b5
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert1-17-5528e36b3b0f5b14313898cc45f9c23a b/sql/hive/src/test/resources/golden/insert1-17-5528e36b3b0f5b14313898cc45f9c23a
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert1-18-16d78fba2d86277bc2f804037cc0a8b4 b/sql/hive/src/test/resources/golden/insert1-18-16d78fba2d86277bc2f804037cc0a8b4
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert1-19-62518ff6810db9cdd8926702192a206b b/sql/hive/src/test/resources/golden/insert1-19-62518ff6810db9cdd8926702192a206b
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert1-2-3f1de4475930285c3fdbe3a5ccd4e868 b/sql/hive/src/test/resources/golden/insert1-2-3f1de4475930285c3fdbe3a5ccd4e868
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert1-20-f4dc51ad64bb8662d066a8b9003da3d4 b/sql/hive/src/test/resources/golden/insert1-20-f4dc51ad64bb8662d066a8b9003da3d4
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert1-21-bb7624250ab556f2d40bfb8d419be487 b/sql/hive/src/test/resources/golden/insert1-21-bb7624250ab556f2d40bfb8d419be487
new file mode 100644
index 0000000000000..1e3637ebc6af2
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/insert1-21-bb7624250ab556f2d40bfb8d419be487
@@ -0,0 +1,2 @@
+db1_insert1
+db1_insert2
diff --git a/sql/hive/src/test/resources/golden/insert1-3-89f8a028e32fae213b575b4df4e26e9c b/sql/hive/src/test/resources/golden/insert1-3-89f8a028e32fae213b575b4df4e26e9c
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert1-4-c7a68c0884785d0f5e62b287eb305d64 b/sql/hive/src/test/resources/golden/insert1-4-c7a68c0884785d0f5e62b287eb305d64
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert1-5-cb87ee12092fdf05daed82485c32a285 b/sql/hive/src/test/resources/golden/insert1-5-cb87ee12092fdf05daed82485c32a285
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert1-6-b97ba93a2c9ae671ecfc4fa95c024dda b/sql/hive/src/test/resources/golden/insert1-6-b97ba93a2c9ae671ecfc4fa95c024dda
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert1-7-a2cd0615b9e79befd9c1842516150a61 b/sql/hive/src/test/resources/golden/insert1-7-a2cd0615b9e79befd9c1842516150a61
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert1-8-5942e331621fe522fc297844046d2370 b/sql/hive/src/test/resources/golden/insert1-8-5942e331621fe522fc297844046d2370
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert1-9-5c5132707d7a4fb6e6a3de1a6719721a b/sql/hive/src/test/resources/golden/insert1-9-5c5132707d7a4fb6e6a3de1a6719721a
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-0-5528e36b3b0f5b14313898cc45f9c23a b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-0-5528e36b3b0f5b14313898cc45f9c23a
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-1-deb504f4f70fd7db975950c3c47959ee b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-1-deb504f4f70fd7db975950c3c47959ee
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-10-fda2e4be738186c0938f92d5072df55a b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-10-fda2e4be738186c0938f92d5072df55a
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-11-9fb177236623d1b62acff28507033436 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-11-9fb177236623d1b62acff28507033436
new file mode 100644
index 0000000000000..01f2b7063f91b
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-11-9fb177236623d1b62acff28507033436
@@ -0,0 +1,5 @@
+98 val_98
+98 val_98
+98 val_98
+97 val_97
+97 val_97
diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-12-99d5ad32bb81640cb284312841b60000 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-12-99d5ad32bb81640cb284312841b60000
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-13-9dda06e1aae1860bd19eee97703a8217 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-13-9dda06e1aae1860bd19eee97703a8217
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-14-19daabdd4c0d403c8781967248d09c53 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-14-19daabdd4c0d403c8781967248d09c53
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-15-812006e1f11e005e5029866d1cf004f6 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-15-812006e1f11e005e5029866d1cf004f6
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-2-bd042746328158822a25d711ffed18dd b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-2-bd042746328158822a25d711ffed18dd
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-3-b7aaedd7d624af4e48637ff1acabe485 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-3-b7aaedd7d624af4e48637ff1acabe485
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-4-dece2650bf0615e566cd6c84181ce026 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-4-dece2650bf0615e566cd6c84181ce026
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-5-1eb5c694e5a02aa292e24a0849350108 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-5-1eb5c694e5a02aa292e24a0849350108
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-6-ab49e0665a80a6b34dadc96f1d18ce26 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-6-ab49e0665a80a6b34dadc96f1d18ce26
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-7-fda2e4be738186c0938f92d5072df55a b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-7-fda2e4be738186c0938f92d5072df55a
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-8-9fb177236623d1b62acff28507033436 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-8-9fb177236623d1b62acff28507033436
new file mode 100644
index 0000000000000..01f2b7063f91b
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-8-9fb177236623d1b62acff28507033436
@@ -0,0 +1,5 @@
+98 val_98
+98 val_98
+98 val_98
+97 val_97
+97 val_97
diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-9-ab49e0665a80a6b34dadc96f1d18ce26 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-9-ab49e0665a80a6b34dadc96f1d18ce26
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/load_binary_data-0-491edd0c42ceb79e799ba50555bc8c15 b/sql/hive/src/test/resources/golden/load_binary_data-0-491edd0c42ceb79e799ba50555bc8c15
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/load_binary_data-1-5d72f8449b69df3c08e3f444f09428bc b/sql/hive/src/test/resources/golden/load_binary_data-1-5d72f8449b69df3c08e3f444f09428bc
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/load_binary_data-2-242b1655c7e7325ee9f26552ea8fc25 b/sql/hive/src/test/resources/golden/load_binary_data-2-242b1655c7e7325ee9f26552ea8fc25
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/load_binary_data-3-2a72df8d3e398d0963ef91162ce7d268 b/sql/hive/src/test/resources/golden/load_binary_data-3-2a72df8d3e398d0963ef91162ce7d268
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/read from cached table-0-ce3797dc14a603cba2a5e58c8612de5b b/sql/hive/src/test/resources/golden/read from cached table-0-ce3797dc14a603cba2a5e58c8612de5b
new file mode 100644
index 0000000000000..60878ffb77064
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/read from cached table-0-ce3797dc14a603cba2a5e58c8612de5b
@@ -0,0 +1 @@
+238 val_238
diff --git a/sql/hive/src/test/resources/golden/read from uncached table-0-ce3797dc14a603cba2a5e58c8612de5b b/sql/hive/src/test/resources/golden/read from uncached table-0-ce3797dc14a603cba2a5e58c8612de5b
new file mode 100644
index 0000000000000..60878ffb77064
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/read from uncached table-0-ce3797dc14a603cba2a5e58c8612de5b
@@ -0,0 +1 @@
+238 val_238
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-0-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-0-86a409d8b868dc5f1a3bd1e04c2bc28c
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-0-86a409d8b868dc5f1a3bd1e04c2bc28c
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-1-2b1df88619e34f221d39598b5cd73283 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-1-2b1df88619e34f221d39598b5cd73283
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-1-2b1df88619e34f221d39598b5cd73283
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-10-60eadbb52f8857830a3034952c631ace b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-10-60eadbb52f8857830a3034952c631ace
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-11-dbe79f90862dc5c6cc4a4fa4b4b6c655 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-11-dbe79f90862dc5c6cc4a4fa4b4b6c655
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-12-60018cae9a0476dc6a0ab4264310edb5 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-12-60018cae9a0476dc6a0ab4264310edb5
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-2-7562d4fee13f3ba935a2e824f86a4224 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-2-7562d4fee13f3ba935a2e824f86a4224
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-2-7562d4fee13f3ba935a2e824f86a4224
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-3-bdb30a5d6887ee4fb089f8676313eafd b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-3-bdb30a5d6887ee4fb089f8676313eafd
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-3-bdb30a5d6887ee4fb089f8676313eafd
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-4-10713b30ecb3c88acdd775bf9628c38c b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-4-10713b30ecb3c88acdd775bf9628c38c
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-4-10713b30ecb3c88acdd775bf9628c38c
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-5-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-5-bab89dfffa77258e34a595e0e79986e3
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-5-bab89dfffa77258e34a595e0e79986e3
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-6-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-6-6f53d5613262d393d82d159ec5dc16dc
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-6-6f53d5613262d393d82d159ec5dc16dc
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-7-ad4ddb5c5d6b994f4dba35f6162b6a9f b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-7-ad4ddb5c5d6b994f4dba35f6162b6a9f
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-8-f9dd797f1c90e2108cfee585f443c132 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-8-f9dd797f1c90e2108cfee585f443c132
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-9-22fdd8380f2652de2492b34a425d46d7 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-9-22fdd8380f2652de2492b34a425d46d7
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-10-22fdd8380f2652de2492b34a425d46d7 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-10-22fdd8380f2652de2492b34a425d46d7
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-11-60eadbb52f8857830a3034952c631ace b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-11-60eadbb52f8857830a3034952c631ace
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-12-dbe79f90862dc5c6cc4a4fa4b4b6c655 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-12-dbe79f90862dc5c6cc4a4fa4b4b6c655
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-13-60018cae9a0476dc6a0ab4264310edb5 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-13-60018cae9a0476dc6a0ab4264310edb5
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-2-2b1df88619e34f221d39598b5cd73283 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-2-2b1df88619e34f221d39598b5cd73283
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-2-2b1df88619e34f221d39598b5cd73283
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-3-7562d4fee13f3ba935a2e824f86a4224 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-3-7562d4fee13f3ba935a2e824f86a4224
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-3-7562d4fee13f3ba935a2e824f86a4224
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-4-bdb30a5d6887ee4fb089f8676313eafd b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-4-bdb30a5d6887ee4fb089f8676313eafd
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-4-bdb30a5d6887ee4fb089f8676313eafd
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-5-10713b30ecb3c88acdd775bf9628c38c b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-5-10713b30ecb3c88acdd775bf9628c38c
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-5-10713b30ecb3c88acdd775bf9628c38c
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-6-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-6-bab89dfffa77258e34a595e0e79986e3
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-6-bab89dfffa77258e34a595e0e79986e3
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-7-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-7-6f53d5613262d393d82d159ec5dc16dc
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-7-6f53d5613262d393d82d159ec5dc16dc
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-8-7a45282169e5a15d70ae0afb9e67ec9a b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-8-7a45282169e5a15d70ae0afb9e67ec9a
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-9-f9dd797f1c90e2108cfee585f443c132 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-9-f9dd797f1c90e2108cfee585f443c132
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-0-48751533b44ea9e8ac3131767c2fed05 b/sql/hive/src/test/resources/golden/timestamp_comparison-0-48751533b44ea9e8ac3131767c2fed05
new file mode 100644
index 0000000000000..c508d5366f70b
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/timestamp_comparison-0-48751533b44ea9e8ac3131767c2fed05
@@ -0,0 +1 @@
+false
diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-1-60557e7bd2822c89fa8b076a9d0520fc b/sql/hive/src/test/resources/golden/timestamp_comparison-1-60557e7bd2822c89fa8b076a9d0520fc
new file mode 100644
index 0000000000000..c508d5366f70b
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/timestamp_comparison-1-60557e7bd2822c89fa8b076a9d0520fc
@@ -0,0 +1 @@
+false
diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-2-f96a9d88327951bd93f672dc2463ecd4 b/sql/hive/src/test/resources/golden/timestamp_comparison-2-f96a9d88327951bd93f672dc2463ecd4
new file mode 100644
index 0000000000000..27ba77ddaf615
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/timestamp_comparison-2-f96a9d88327951bd93f672dc2463ecd4
@@ -0,0 +1 @@
+true
diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-3-13e17ed811165196416f777cbc162592 b/sql/hive/src/test/resources/golden/timestamp_comparison-3-13e17ed811165196416f777cbc162592
new file mode 100644
index 0000000000000..c508d5366f70b
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/timestamp_comparison-3-13e17ed811165196416f777cbc162592
@@ -0,0 +1 @@
+false
diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-4-4fa8a36edbefde4427c2ab2cf30e6399 b/sql/hive/src/test/resources/golden/timestamp_comparison-4-4fa8a36edbefde4427c2ab2cf30e6399
new file mode 100644
index 0000000000000..27ba77ddaf615
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/timestamp_comparison-4-4fa8a36edbefde4427c2ab2cf30e6399
@@ -0,0 +1 @@
+true
diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-5-7e4fb6e8ba01df422e4c67e06a0c8453 b/sql/hive/src/test/resources/golden/timestamp_comparison-5-7e4fb6e8ba01df422e4c67e06a0c8453
new file mode 100644
index 0000000000000..27ba77ddaf615
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/timestamp_comparison-5-7e4fb6e8ba01df422e4c67e06a0c8453
@@ -0,0 +1 @@
+true
diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-6-8c8e73673a950f6b3d960b08fcea076f b/sql/hive/src/test/resources/golden/timestamp_comparison-6-8c8e73673a950f6b3d960b08fcea076f
new file mode 100644
index 0000000000000..c508d5366f70b
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/timestamp_comparison-6-8c8e73673a950f6b3d960b08fcea076f
@@ -0,0 +1 @@
+false
diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-7-510c0a2a57dc5df8588bd13c4152f8bc b/sql/hive/src/test/resources/golden/timestamp_comparison-7-510c0a2a57dc5df8588bd13c4152f8bc
new file mode 100644
index 0000000000000..27ba77ddaf615
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/timestamp_comparison-7-510c0a2a57dc5df8588bd13c4152f8bc
@@ -0,0 +1 @@
+true
diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-8-659d5b1ae8200f13f265270e52a3dd65 b/sql/hive/src/test/resources/golden/timestamp_comparison-8-659d5b1ae8200f13f265270e52a3dd65
new file mode 100644
index 0000000000000..27ba77ddaf615
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/timestamp_comparison-8-659d5b1ae8200f13f265270e52a3dd65
@@ -0,0 +1 @@
+true
diff --git a/sql/hive/src/test/resources/golden/type_cast_1-0-60ea21e6e7d054a65f959fc89acf1b3d b/sql/hive/src/test/resources/golden/type_cast_1-0-60ea21e6e7d054a65f959fc89acf1b3d
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/type_cast_1-1-53a667981ad567b2ab977f67d65c5825 b/sql/hive/src/test/resources/golden/type_cast_1-1-53a667981ad567b2ab977f67d65c5825
new file mode 100644
index 0000000000000..7ed6ff82de6bc
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/type_cast_1-1-53a667981ad567b2ab977f67d65c5825
@@ -0,0 +1 @@
+5
diff --git a/sql/hive/src/test/resources/golden/udf_printf-0-e86d559aeb84a4cc017a103182c22bfb b/sql/hive/src/test/resources/golden/udf_printf-0-e86d559aeb84a4cc017a103182c22bfb
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/udf_printf-1-19c61fce27310ab2590062d643f7b26e b/sql/hive/src/test/resources/golden/udf_printf-1-19c61fce27310ab2590062d643f7b26e
new file mode 100644
index 0000000000000..1635ff88dd768
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_printf-1-19c61fce27310ab2590062d643f7b26e
@@ -0,0 +1 @@
+printf(String format, Obj... args) - function that can format strings according to printf-style format strings
diff --git a/sql/hive/src/test/resources/golden/udf_printf-2-25aa6950cae2bb781c336378f63ceaee b/sql/hive/src/test/resources/golden/udf_printf-2-25aa6950cae2bb781c336378f63ceaee
new file mode 100644
index 0000000000000..62440ee68e145
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_printf-2-25aa6950cae2bb781c336378f63ceaee
@@ -0,0 +1,4 @@
+printf(String format, Obj... args) - function that can format strings according to printf-style format strings
+Example:
+ > SELECT printf("Hello World %d %s", 100, "days")FROM src LIMIT 1;
+ "Hello World 100 days"
diff --git a/sql/hive/src/test/resources/golden/udf_printf-3-9c568a0473888396bd46507e8b330c36 b/sql/hive/src/test/resources/golden/udf_printf-3-9c568a0473888396bd46507e8b330c36
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/udf_printf-4-91728e546b450bdcbb05ef30f13be475 b/sql/hive/src/test/resources/golden/udf_printf-4-91728e546b450bdcbb05ef30f13be475
new file mode 100644
index 0000000000000..39cb945991403
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_printf-4-91728e546b450bdcbb05ef30f13be475
@@ -0,0 +1 @@
+Hello World 100 days
diff --git a/sql/hive/src/test/resources/golden/udf_printf-5-3141a0421605b091ee5a9e99d7d605fb b/sql/hive/src/test/resources/golden/udf_printf-5-3141a0421605b091ee5a9e99d7d605fb
new file mode 100644
index 0000000000000..04bf5e552a576
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_printf-5-3141a0421605b091ee5a9e99d7d605fb
@@ -0,0 +1 @@
+All Type Test: false, A, 15000, 1.234000e+01, +27183.2401, 2300.41, 32, corret, 0x1.002p8
diff --git a/sql/hive/src/test/resources/golden/udf_printf-6-ec37b73012f3cbbbc0422744b0db8294 b/sql/hive/src/test/resources/golden/udf_printf-6-ec37b73012f3cbbbc0422744b0db8294
new file mode 100644
index 0000000000000..2e9f7509968a3
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_printf-6-ec37b73012f3cbbbc0422744b0db8294
@@ -0,0 +1 @@
+Color red, String Null: null, number1 123456, number2 00089, Integer Null: null, hex 0xff, float 3.14 Double Null: null
diff --git a/sql/hive/src/test/resources/golden/udf_printf-7-5769f3a5b3300ca1d8b861229e976126 b/sql/hive/src/test/resources/golden/udf_printf-7-5769f3a5b3300ca1d8b861229e976126
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-10-51822ac740629bebd81d2abda6e1144 b/sql/hive/src/test/resources/golden/udf_to_boolean-10-51822ac740629bebd81d2abda6e1144
new file mode 100644
index 0000000000000..c508d5366f70b
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_boolean-10-51822ac740629bebd81d2abda6e1144
@@ -0,0 +1 @@
+false
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-11-441306cae24618c49ec63445a31bf16b b/sql/hive/src/test/resources/golden/udf_to_boolean-11-441306cae24618c49ec63445a31bf16b
new file mode 100644
index 0000000000000..c508d5366f70b
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_boolean-11-441306cae24618c49ec63445a31bf16b
@@ -0,0 +1 @@
+false
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-12-bfcc534e73e320a1cfad9c584678d870 b/sql/hive/src/test/resources/golden/udf_to_boolean-12-bfcc534e73e320a1cfad9c584678d870
new file mode 100644
index 0000000000000..c508d5366f70b
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_boolean-12-bfcc534e73e320a1cfad9c584678d870
@@ -0,0 +1 @@
+false
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-13-a2bddaa5db1841bb4617239b9f17a06d b/sql/hive/src/test/resources/golden/udf_to_boolean-13-a2bddaa5db1841bb4617239b9f17a06d
new file mode 100644
index 0000000000000..c508d5366f70b
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_boolean-13-a2bddaa5db1841bb4617239b9f17a06d
@@ -0,0 +1 @@
+false
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-14-773801b833cf72d35016916b786275b5 b/sql/hive/src/test/resources/golden/udf_to_boolean-14-773801b833cf72d35016916b786275b5
new file mode 100644
index 0000000000000..c508d5366f70b
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_boolean-14-773801b833cf72d35016916b786275b5
@@ -0,0 +1 @@
+false
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-15-4071ed0ff57b53963d5ee662fa9db0b0 b/sql/hive/src/test/resources/golden/udf_to_boolean-15-4071ed0ff57b53963d5ee662fa9db0b0
new file mode 100644
index 0000000000000..c508d5366f70b
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_boolean-15-4071ed0ff57b53963d5ee662fa9db0b0
@@ -0,0 +1 @@
+false
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-16-6b441df08afdc0c6c4a82670997dabb5 b/sql/hive/src/test/resources/golden/udf_to_boolean-16-6b441df08afdc0c6c4a82670997dabb5
new file mode 100644
index 0000000000000..c508d5366f70b
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_boolean-16-6b441df08afdc0c6c4a82670997dabb5
@@ -0,0 +1 @@
+false
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-17-85342c694d7f35e7eedb24e850d0c7df b/sql/hive/src/test/resources/golden/udf_to_boolean-17-85342c694d7f35e7eedb24e850d0c7df
new file mode 100644
index 0000000000000..c508d5366f70b
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_boolean-17-85342c694d7f35e7eedb24e850d0c7df
@@ -0,0 +1 @@
+false
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-18-fcd7af0e71d3e2d934239ba606e3ed87 b/sql/hive/src/test/resources/golden/udf_to_boolean-18-fcd7af0e71d3e2d934239ba606e3ed87
new file mode 100644
index 0000000000000..7951defec192a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_boolean-18-fcd7af0e71d3e2d934239ba606e3ed87
@@ -0,0 +1 @@
+NULL
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-19-dcdb12fe551aa68a56921822f5d1a343 b/sql/hive/src/test/resources/golden/udf_to_boolean-19-dcdb12fe551aa68a56921822f5d1a343
new file mode 100644
index 0000000000000..7951defec192a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_boolean-19-dcdb12fe551aa68a56921822f5d1a343
@@ -0,0 +1 @@
+NULL
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-20-131900d39d9a20b431731a32fb9715f8 b/sql/hive/src/test/resources/golden/udf_to_boolean-20-131900d39d9a20b431731a32fb9715f8
new file mode 100644
index 0000000000000..7951defec192a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_boolean-20-131900d39d9a20b431731a32fb9715f8
@@ -0,0 +1 @@
+NULL
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-21-a5e28f4eb819e5a5e292e279f2990a7a b/sql/hive/src/test/resources/golden/udf_to_boolean-21-a5e28f4eb819e5a5e292e279f2990a7a
new file mode 100644
index 0000000000000..7951defec192a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_boolean-21-a5e28f4eb819e5a5e292e279f2990a7a
@@ -0,0 +1 @@
+NULL
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-22-93278c10d642fa242f303d89b3b1961d b/sql/hive/src/test/resources/golden/udf_to_boolean-22-93278c10d642fa242f303d89b3b1961d
new file mode 100644
index 0000000000000..7951defec192a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_boolean-22-93278c10d642fa242f303d89b3b1961d
@@ -0,0 +1 @@
+NULL
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-23-828558020ce907ffa7e847762a5e2358 b/sql/hive/src/test/resources/golden/udf_to_boolean-23-828558020ce907ffa7e847762a5e2358
new file mode 100644
index 0000000000000..7951defec192a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_boolean-23-828558020ce907ffa7e847762a5e2358
@@ -0,0 +1 @@
+NULL
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-24-e8ca597d87932af16c0cf29d662e92da b/sql/hive/src/test/resources/golden/udf_to_boolean-24-e8ca597d87932af16c0cf29d662e92da
new file mode 100644
index 0000000000000..7951defec192a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_boolean-24-e8ca597d87932af16c0cf29d662e92da
@@ -0,0 +1 @@
+NULL
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-25-86245727f90de9ce65a12c97a03a5635 b/sql/hive/src/test/resources/golden/udf_to_boolean-25-86245727f90de9ce65a12c97a03a5635
new file mode 100644
index 0000000000000..7951defec192a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_boolean-25-86245727f90de9ce65a12c97a03a5635
@@ -0,0 +1 @@
+NULL
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-26-552d7ec5a4e0c93dc59a61973e2d63a2 b/sql/hive/src/test/resources/golden/udf_to_boolean-26-552d7ec5a4e0c93dc59a61973e2d63a2
new file mode 100644
index 0000000000000..7951defec192a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_boolean-26-552d7ec5a4e0c93dc59a61973e2d63a2
@@ -0,0 +1 @@
+NULL
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-27-b61509b01b2fe3e7e4b72fedc74ff4f9 b/sql/hive/src/test/resources/golden/udf_to_boolean-27-b61509b01b2fe3e7e4b72fedc74ff4f9
new file mode 100644
index 0000000000000..7951defec192a
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_boolean-27-b61509b01b2fe3e7e4b72fedc74ff4f9
@@ -0,0 +1 @@
+NULL
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-8-37229f303635a030f6cab20e0381f51f b/sql/hive/src/test/resources/golden/udf_to_boolean-8-37229f303635a030f6cab20e0381f51f
new file mode 100644
index 0000000000000..27ba77ddaf615
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_boolean-8-37229f303635a030f6cab20e0381f51f
@@ -0,0 +1 @@
+true
diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-9-be623247e4dbf119b43458b72d1be017 b/sql/hive/src/test/resources/golden/udf_to_boolean-9-be623247e4dbf119b43458b72d1be017
new file mode 100644
index 0000000000000..c508d5366f70b
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_to_boolean-9-be623247e4dbf119b43458b72d1be017
@@ -0,0 +1 @@
+false
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala
new file mode 100644
index 0000000000000..79ec1f1cde019
--- /dev/null
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hive
+
+import org.apache.spark.sql.execution.SparkLogicalPlan
+import org.apache.spark.sql.columnar.InMemoryColumnarTableScan
+import org.apache.spark.sql.hive.execution.HiveComparisonTest
+
+class CachedTableSuite extends HiveComparisonTest {
+ TestHive.loadTestTable("src")
+
+ test("cache table") {
+ TestHive.cacheTable("src")
+ }
+
+ createQueryTest("read from cached table",
+ "SELECT * FROM src LIMIT 1", reset = false)
+
+ test("check that table is cached and uncache") {
+ TestHive.table("src").queryExecution.analyzed match {
+ case SparkLogicalPlan(_ : InMemoryColumnarTableScan) => // Found evidence of caching
+ case noCache => fail(s"No cache node found in plan $noCache")
+ }
+ TestHive.uncacheTable("src")
+ }
+
+ createQueryTest("read from uncached table",
+ "SELECT * FROM src LIMIT 1", reset = false)
+
+ test("make sure table is uncached") {
+ TestHive.table("src").queryExecution.analyzed match {
+ case cachePlan @ SparkLogicalPlan(_ : InMemoryColumnarTableScan) =>
+ fail(s"Table still cached after uncache: $cachePlan")
+ case noCache => // Table uncached successfully
+ }
+ }
+
+ test("correct error on uncache of non-cached table") {
+ intercept[IllegalArgumentException] {
+ TestHive.uncacheTable("src")
+ }
+ }
+}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveSuite.scala
new file mode 100644
index 0000000000000..8137f99b227f4
--- /dev/null
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveSuite.scala
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hive.api.java
+
+import org.scalatest.FunSuite
+
+import org.apache.spark.api.java.JavaSparkContext
+import org.apache.spark.sql.test.TestSQLContext
+import org.apache.spark.sql.hive.TestHive
+
+// Implicits
+import scala.collection.JavaConversions._
+
+class JavaHiveSQLSuite extends FunSuite {
+ ignore("SELECT * FROM src") {
+ val javaCtx = new JavaSparkContext(TestSQLContext.sparkContext)
+ // There is a little trickery here to avoid instantiating two HiveContexts in the same JVM
+ val javaSqlCtx = new JavaHiveContext(javaCtx) {
+ override val sqlContext = TestHive
+ }
+
+ assert(
+ javaSqlCtx.hql("SELECT * FROM src").collect().map(_.getInt(0)) ===
+ TestHive.sql("SELECT * FROM src").collect().map(_.getInt(0)).toSeq)
+ }
+}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
index c7a350ef94edd..18654b308d234 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
@@ -170,7 +170,7 @@ abstract class HiveComparisonTest
}
val installHooksCommand = "(?i)SET.*hooks".r
- def createQueryTest(testCaseName: String, sql: String) {
+ def createQueryTest(testCaseName: String, sql: String, reset: Boolean = true) {
// If test sharding is enable, skip tests that are not in the correct shard.
shardInfo.foreach {
case (shardId, numShards) if testCaseName.hashCode % numShards != shardId => return
@@ -228,7 +228,7 @@ abstract class HiveComparisonTest
try {
// MINOR HACK: You must run a query before calling reset the first time.
TestHive.sql("SHOW TABLES")
- TestHive.reset()
+ if (reset) { TestHive.reset() }
val hiveCacheFiles = queryList.zipWithIndex.map {
case (queryString, i) =>
@@ -295,7 +295,7 @@ abstract class HiveComparisonTest
fail(errorMessage)
}
}.toSeq
- TestHive.reset()
+ if (reset) { TestHive.reset() }
computedResults
}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
index f74b0fbb97c83..f76e16bc1afc5 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
@@ -42,6 +42,9 @@ class HiveCompatibilitySuite extends HiveQueryFileTest {
"bucket_num_reducers",
"column_access_stats",
"concatenate_inherit_table_location",
+ "describe_pretty",
+ "describe_syntax",
+ "orc_ends_with_nulls",
// Setting a default property does not seem to get reset and thus changes the answer for many
// subsequent tests.
@@ -80,7 +83,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest {
"index_auto_update",
"index_auto_self_join",
"index_stale.*",
- "type_cast_1",
"index_compression",
"index_bitmap_compression",
"index_auto_multiple",
@@ -237,9 +239,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest {
"compute_stats_binary",
"compute_stats_boolean",
"compute_stats_double",
- "compute_stats_table",
+ "compute_stats_empty_table",
"compute_stats_long",
"compute_stats_string",
+ "compute_stats_table",
"convert_enum_to_string",
"correlationoptimizer11",
"correlationoptimizer15",
@@ -266,8 +269,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest {
"desc_non_existent_tbl",
"describe_comment_indent",
"describe_database_json",
- "describe_pretty",
- "describe_syntax",
+ "describe_formatted_view_partitioned",
+ "describe_formatted_view_partitioned_json",
"describe_table_json",
"diff_part_input_formats",
"disable_file_format_check",
@@ -339,8 +342,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest {
"input11_limit",
"input12",
"input12_hadoop20",
+ "input14",
"input19",
"input1_limit",
+ "input21",
"input22",
"input23",
"input24",
@@ -355,6 +360,9 @@ class HiveCompatibilitySuite extends HiveQueryFileTest {
"input7",
"input8",
"input9",
+ "inputddl4",
+ "inputddl7",
+ "inputddl8",
"input_limit",
"input_part0",
"input_part1",
@@ -368,9 +376,9 @@ class HiveCompatibilitySuite extends HiveQueryFileTest {
"input_part7",
"input_part8",
"input_part9",
- "inputddl4",
- "inputddl7",
- "inputddl8",
+ "input_testsequencefile",
+ "insert1",
+ "insert2_overwrite_partitions",
"insert_compressed",
"join0",
"join1",
@@ -385,6 +393,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest {
"join17",
"join18",
"join19",
+ "join_1to1",
"join2",
"join20",
"join21",
@@ -400,6 +409,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest {
"join30",
"join31",
"join32",
+ "join32_lessSize",
"join33",
"join34",
"join35",
@@ -415,13 +425,14 @@ class HiveCompatibilitySuite extends HiveQueryFileTest {
"join7",
"join8",
"join9",
- "join_1to1",
"join_array",
"join_casesensitive",
"join_empty",
"join_filters",
"join_hive_626",
+ "join_map_ppr",
"join_nulls",
+ "join_rc",
"join_reorder2",
"join_reorder3",
"join_reorder4",
@@ -435,22 +446,32 @@ class HiveCompatibilitySuite extends HiveQueryFileTest {
"literal_string",
"load_dyn_part7",
"load_file_with_space_in_the_name",
+ "loadpart1",
"louter_join_ppr",
"mapjoin_distinct",
"mapjoin_mapjoin",
"mapjoin_subquery",
"mapjoin_subquery2",
"mapjoin_test_outer",
+ "mapreduce1",
+ "mapreduce2",
"mapreduce3",
+ "mapreduce4",
+ "mapreduce5",
+ "mapreduce6",
"mapreduce7",
+ "mapreduce8",
"merge1",
"merge2",
"mergejoins",
"mergejoins_mixed",
+ "multigroupby_singlemr",
+ "multi_insert_gby",
+ "multi_insert_gby3",
+ "multi_insert_lateral_view",
+ "multi_join_union",
"multiMapJoin1",
"multiMapJoin2",
- "multi_join_union",
- "multigroupby_singlemr",
"noalias_subq1",
"nomore_ambiguous_table_col",
"nonblock_op_deduplicate",
@@ -466,16 +487,30 @@ class HiveCompatibilitySuite extends HiveQueryFileTest {
"nullinput2",
"nullscript",
"optional_outer",
+ "orc_dictionary_threshold",
+ "orc_empty_files",
"order",
"order2",
"outer_join_ppr",
+ "parallel",
+ "parenthesis_star_by",
+ "partcols1",
"part_inherit_tbl_props",
"part_inherit_tbl_props_empty",
"part_inherit_tbl_props_with_star",
"partition_schema1",
+ "partition_serde_format",
"partition_varchar1",
+ "partition_wise_fileformat4",
+ "partition_wise_fileformat5",
+ "partition_wise_fileformat6",
+ "partition_wise_fileformat7",
+ "partition_wise_fileformat9",
"plan_json",
"ppd1",
+ "ppd2",
+ "ppd_clusterby",
+ "ppd_constant_expr",
"ppd_constant_where",
"ppd_gby",
"ppd_gby2",
@@ -491,6 +526,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest {
"ppd_outer_join5",
"ppd_random",
"ppd_repeated_alias",
+ "ppd_transform",
"ppd_udf_col",
"ppd_union",
"ppr_allchildsarenull",
@@ -503,7 +539,15 @@ class HiveCompatibilitySuite extends HiveQueryFileTest {
"query_with_semi",
"quote1",
"quote2",
+ "rcfile_columnar",
+ "rcfile_lazydecompress",
+ "rcfile_null_value",
+ "rcfile_toleratecorruptions",
+ "rcfile_union",
+ "reduce_deduplicate",
+ "reduce_deduplicate_exclude_gby",
"reduce_deduplicate_exclude_join",
+ "reducesink_dedup",
"rename_column",
"router_join_ppr",
"select_as_omitted",
@@ -531,6 +575,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest {
"smb_mapjoin_3",
"smb_mapjoin_4",
"smb_mapjoin_5",
+ "smb_mapjoin_6",
+ "smb_mapjoin_7",
"smb_mapjoin_8",
"sort",
"sort_merge_join_desc_1",
@@ -541,21 +587,27 @@ class HiveCompatibilitySuite extends HiveQueryFileTest {
"sort_merge_join_desc_6",
"sort_merge_join_desc_7",
"stats0",
+ "stats_aggregator_error_1",
"stats_empty_partition",
+ "stats_publisher_error_1",
"subq2",
"tablename_with_select",
+ "timestamp_comparison",
"touch",
+ "transform_ppr1",
+ "transform_ppr2",
+ "type_cast_1",
"type_widening",
"udaf_collect_set",
"udaf_corr",
"udaf_covar_pop",
"udaf_covar_samp",
+ "udaf_histogram_numeric",
+ "udf_10_trims",
"udf2",
"udf6",
+ "udf8",
"udf9",
- "udf_10_trims",
- "udf_E",
- "udf_PI",
"udf_abs",
"udf_acos",
"udf_add",
@@ -585,13 +637,14 @@ class HiveCompatibilitySuite extends HiveQueryFileTest {
"udf_cos",
"udf_count",
"udf_date_add",
- "udf_date_sub",
"udf_datediff",
+ "udf_date_sub",
"udf_day",
"udf_dayofmonth",
"udf_degrees",
"udf_div",
"udf_double",
+ "udf_E",
"udf_exp",
"udf_field",
"udf_find_in_set",
@@ -631,6 +684,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest {
"udf_nvl",
"udf_or",
"udf_parse_url",
+ "udf_PI",
"udf_positive",
"udf_pow",
"udf_power",
@@ -671,9 +725,9 @@ class HiveCompatibilitySuite extends HiveQueryFileTest {
"udf_trim",
"udf_ucase",
"udf_upper",
+ "udf_variance",
"udf_var_pop",
"udf_var_samp",
- "udf_variance",
"udf_weekofyear",
"udf_when",
"udf_xpath",
@@ -703,8 +757,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest {
"union27",
"union28",
"union29",
+ "union3",
"union30",
"union31",
+ "union33",
"union34",
"union4",
"union5",
@@ -714,6 +770,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest {
"union9",
"union_lateralview",
"union_ppr",
+ "union_remove_11",
"union_remove_3",
"union_remove_6",
"union_script",
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
index d77900ddc950c..40c4e23f90fb8 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
@@ -48,7 +48,7 @@ class HiveResolutionSuite extends HiveComparisonTest {
createQueryTest("attr",
"SELECT key FROM src a ORDER BY key LIMIT 1")
- createQueryTest("alias.*",
+ createQueryTest("alias.star",
"SELECT a.* FROM src a ORDER BY key LIMIT 1")
test("case insensitivity with scala reflection") {
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala
index 05ad85b622ac8..314ca48ad8f6a 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala
@@ -17,147 +17,138 @@
package org.apache.spark.sql.parquet
-import java.io.File
-
import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
-import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
-import org.apache.spark.sql.catalyst.expressions.Row
-import org.apache.spark.sql.catalyst.plans.logical._
-import org.apache.spark.sql.catalyst.util.getTempFilePath
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Row}
+import org.apache.spark.sql.catalyst.types.{DataType, StringType, IntegerType}
+import org.apache.spark.sql.{parquet, SchemaRDD}
import org.apache.spark.sql.hive.TestHive
+import org.apache.spark.util.Utils
+
+// Implicits
+import org.apache.spark.sql.hive.TestHive._
class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfterEach {
- val filename = getTempFilePath("parquettest").getCanonicalFile.toURI.toString
-
- // runs a SQL and optionally resolves one Parquet table
- def runQuery(
- querystr: String,
- tableName: Option[String] = None,
- filename: Option[String] = None): Array[Row] = {
-
- // call to resolve references in order to get CREATE TABLE AS to work
- val query = TestHive
- .parseSql(querystr)
- val finalQuery =
- if (tableName.nonEmpty && filename.nonEmpty)
- resolveParquetTable(tableName.get, filename.get, query)
- else
- query
- TestHive.executePlan(finalQuery)
- .toRdd
- .collect()
- }
- // stores a query output to a Parquet file
- def storeQuery(querystr: String, filename: String): Unit = {
- val query = WriteToFile(
- filename,
- TestHive.parseSql(querystr))
- TestHive
- .executePlan(query)
- .stringResult()
- }
+ val dirname = Utils.createTempDir()
- /**
- * TODO: This function is necessary as long as there is no notion of a Catalog for
- * Parquet tables. Once such a thing exists this functionality should be moved there.
- */
- def resolveParquetTable(tableName: String, filename: String, plan: LogicalPlan): LogicalPlan = {
- TestHive.loadTestTable("src") // may not be loaded now
- plan.transform {
- case relation @ UnresolvedRelation(databaseName, name, alias) =>
- if (name == tableName)
- ParquetRelation(tableName, filename)
- else
- relation
- case op @ InsertIntoCreatedTable(databaseName, name, child) =>
- if (name == tableName) {
- // note: at this stage the plan is not yet analyzed but Parquet needs to know the schema
- // and for that we need the child to be resolved
- val relation = ParquetRelation.create(
- filename,
- TestHive.analyzer(child),
- TestHive.sparkContext.hadoopConfiguration,
- Some(tableName))
- InsertIntoTable(
- relation.asInstanceOf[BaseRelation],
- Map.empty,
- child,
- overwrite = false)
- } else
- op
- }
- }
+ var testRDD: SchemaRDD = null
override def beforeAll() {
// write test data
- ParquetTestData.writeFile()
- // Override initial Parquet test table
- TestHive.catalog.registerTable(Some[String]("parquet"), "testsource", ParquetTestData.testData)
+ ParquetTestData.writeFile
+ testRDD = parquetFile(ParquetTestData.testDir.toString)
+ testRDD.registerAsTable("testsource")
}
override def afterAll() {
- ParquetTestData.testFile.delete()
+ Utils.deleteRecursively(ParquetTestData.testDir)
+ Utils.deleteRecursively(dirname)
+ reset() // drop all tables that were registered as part of the tests
}
+ // in case tests are failing we delete before and after each test
override def beforeEach() {
- new File(filename).getAbsoluteFile.delete()
+ Utils.deleteRecursively(dirname)
}
override def afterEach() {
- new File(filename).getAbsoluteFile.delete()
+ Utils.deleteRecursively(dirname)
}
test("SELECT on Parquet table") {
- val rdd = runQuery("SELECT * FROM parquet.testsource")
+ val rdd = sql("SELECT * FROM testsource").collect()
assert(rdd != null)
assert(rdd.forall(_.size == 6))
}
test("Simple column projection + filter on Parquet table") {
- val rdd = runQuery("SELECT myboolean, mylong FROM parquet.testsource WHERE myboolean=true")
+ val rdd = sql("SELECT myboolean, mylong FROM testsource WHERE myboolean=true").collect()
assert(rdd.size === 5, "Filter returned incorrect number of rows")
assert(rdd.forall(_.getBoolean(0)), "Filter returned incorrect Boolean field value")
}
- test("Converting Hive to Parquet Table via WriteToFile") {
- storeQuery("SELECT * FROM src", filename)
- val rddOne = runQuery("SELECT * FROM src").sortBy(_.getInt(0))
- val rddTwo = runQuery("SELECT * from ptable", Some("ptable"), Some(filename)).sortBy(_.getInt(0))
+ test("Converting Hive to Parquet Table via saveAsParquetFile") {
+ sql("SELECT * FROM src").saveAsParquetFile(dirname.getAbsolutePath)
+ parquetFile(dirname.getAbsolutePath).registerAsTable("ptable")
+ val rddOne = sql("SELECT * FROM src").collect().sortBy(_.getInt(0))
+ val rddTwo = sql("SELECT * from ptable").collect().sortBy(_.getInt(0))
compareRDDs(rddOne, rddTwo, "src (Hive)", Seq("key:Int", "value:String"))
}
test("INSERT OVERWRITE TABLE Parquet table") {
- storeQuery("SELECT * FROM parquet.testsource", filename)
- runQuery("INSERT OVERWRITE TABLE ptable SELECT * FROM parquet.testsource", Some("ptable"), Some(filename))
- runQuery("INSERT OVERWRITE TABLE ptable SELECT * FROM parquet.testsource", Some("ptable"), Some(filename))
- val rddCopy = runQuery("SELECT * FROM ptable", Some("ptable"), Some(filename))
- val rddOrig = runQuery("SELECT * FROM parquet.testsource")
- compareRDDs(rddOrig, rddCopy, "parquet.testsource", ParquetTestData.testSchemaFieldNames)
+ sql("SELECT * FROM testsource").saveAsParquetFile(dirname.getAbsolutePath)
+ parquetFile(dirname.getAbsolutePath).registerAsTable("ptable")
+ // let's do three overwrites for good measure
+ sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect()
+ sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect()
+ sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect()
+ val rddCopy = sql("SELECT * FROM ptable").collect()
+ val rddOrig = sql("SELECT * FROM testsource").collect()
+ assert(rddCopy.size === rddOrig.size, "INSERT OVERWRITE changed size of table??")
+ compareRDDs(rddOrig, rddCopy, "testsource", ParquetTestData.testSchemaFieldNames)
}
- test("CREATE TABLE AS Parquet table") {
- runQuery("CREATE TABLE ptable AS SELECT * FROM src", Some("ptable"), Some(filename))
- val rddCopy = runQuery("SELECT * FROM ptable", Some("ptable"), Some(filename))
+ test("CREATE TABLE of Parquet table") {
+ createParquetFile(dirname.getAbsolutePath, ("key", IntegerType), ("value", StringType))
+ .registerAsTable("tmp")
+ val rddCopy =
+ sql("INSERT INTO TABLE tmp SELECT * FROM src")
+ .collect()
.sortBy[Int](_.apply(0) match {
case x: Int => x
case _ => 0
})
- val rddOrig = runQuery("SELECT * FROM src").sortBy(_.getInt(0))
+ val rddOrig = sql("SELECT * FROM src")
+ .collect()
+ .sortBy(_.getInt(0))
compareRDDs(rddOrig, rddCopy, "src (Hive)", Seq("key:Int", "value:String"))
}
+ test("Appending to Parquet table") {
+ createParquetFile(dirname.getAbsolutePath, ("key", IntegerType), ("value", StringType))
+ .registerAsTable("tmpnew")
+ sql("INSERT INTO TABLE tmpnew SELECT * FROM src").collect()
+ sql("INSERT INTO TABLE tmpnew SELECT * FROM src").collect()
+ sql("INSERT INTO TABLE tmpnew SELECT * FROM src").collect()
+ val rddCopies = sql("SELECT * FROM tmpnew").collect()
+ val rddOrig = sql("SELECT * FROM src").collect()
+ assert(rddCopies.size === 3 * rddOrig.size, "number of copied rows via INSERT INTO did not match correct number")
+ }
+
+ test("Appending to and then overwriting Parquet table") {
+ createParquetFile(dirname.getAbsolutePath, ("key", IntegerType), ("value", StringType))
+ .registerAsTable("tmp")
+ sql("INSERT INTO TABLE tmp SELECT * FROM src").collect()
+ sql("INSERT INTO TABLE tmp SELECT * FROM src").collect()
+ sql("INSERT OVERWRITE TABLE tmp SELECT * FROM src").collect()
+ val rddCopies = sql("SELECT * FROM tmp").collect()
+ val rddOrig = sql("SELECT * FROM src").collect()
+ assert(rddCopies.size === rddOrig.size, "INSERT OVERWRITE did not actually overwrite")
+ }
+
private def compareRDDs(rddOne: Array[Row], rddTwo: Array[Row], tableName: String, fieldNames: Seq[String]) {
var counter = 0
(rddOne, rddTwo).zipped.foreach {
(a,b) => (a,b).zipped.toArray.zipWithIndex.foreach {
- case ((value_1:Array[Byte], value_2:Array[Byte]), index) =>
- assert(new String(value_1) === new String(value_2), s"table $tableName row $counter field ${fieldNames(index)} don't match")
case ((value_1, value_2), index) =>
assert(value_1 === value_2, s"table $tableName row $counter field ${fieldNames(index)} don't match")
}
counter = counter + 1
}
}
+
+ /**
+ * Creates an empty SchemaRDD backed by a ParquetRelation.
+ *
+ * TODO: since this is so experimental it is better to have it here and not
+ * in SQLContext. Also note that when creating new AttributeReferences
+ * one needs to take care not to create duplicate Attribute ID's.
+ */
+ private def createParquetFile(path: String, schema: (Tuple2[String, DataType])*): SchemaRDD = {
+ val attributes = schema.map(t => new AttributeReference(t._1, t._2)())
+ new SchemaRDD(
+ TestHive,
+ parquet.ParquetRelation.createEmpty(path, attributes, sparkContext.hadoopConfiguration))
+ }
}
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/RateLimitedOutputStreamSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/RateLimitedOutputStreamSuite.scala
index 7d18a0fcf7ba8..9ebf7b484f421 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/util/RateLimitedOutputStreamSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/util/RateLimitedOutputStreamSuite.scala
@@ -36,8 +36,9 @@ class RateLimitedOutputStreamSuite extends FunSuite {
val stream = new RateLimitedOutputStream(underlying, desiredBytesPerSec = 10000)
val elapsedNs = benchmark { stream.write(data.getBytes("UTF-8")) }
- // We accept anywhere from 4.0 to 4.99999 seconds since the value is rounded down.
- assert(SECONDS.convert(elapsedNs, NANOSECONDS) === 4)
+ val seconds = SECONDS.convert(elapsedNs, NANOSECONDS)
+ assert(seconds >= 4, s"Seconds value ($seconds) is less than 4.")
+ assert(seconds <= 30, s"Took more than 30 seconds ($seconds) to write data.")
assert(underlying.toString("UTF-8") === data)
}
}
diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index 71a64ecf5879a..0179b0600c61f 100644
--- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -167,6 +167,9 @@ class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: Spa
object Client {
def main(argStrings: Array[String]) {
+ println("WARNING: This client is deprecated and will be removed in a future version of Spark.")
+ println("Use ./bin/spark-submit with \"--master yarn\"")
+
// Set an env variable indicating we are running in YARN mode.
// Note that anything with SPARK prefix gets propagated to all (remote) processes
System.setProperty("SPARK_YARN_MODE", "true")
diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
index c565f2dde24fc..3e4c739e34fe9 100644
--- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
+++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala
@@ -63,7 +63,10 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) {
userClass = value
args = tail
- case ("--args") :: value :: tail =>
+ case ("--args" | "--arg") :: value :: tail =>
+ if (args(0) == "--args") {
+ println("--args is deprecated. Use --arg instead.")
+ }
userArgsBuffer += value
args = tail
@@ -146,8 +149,8 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) {
"Options:\n" +
" --jar JAR_PATH Path to your application's JAR file (required in yarn-cluster mode)\n" +
" --class CLASS_NAME Name of your application's main class (required)\n" +
- " --args ARGS Arguments to be passed to your application's main class.\n" +
- " Mutliple invocations are possible, each will be passed in order.\n" +
+ " --arg ARGS Argument to be passed to your application's main class.\n" +
+ " Multiple invocations are possible, each will be passed in order.\n" +
" --num-executors NUM Number of executors to start (Default: 2)\n" +
" --executor-cores NUM Number of cores for the executors (Default: 1).\n" +
" --driver-memory MEM Memory for driver (e.g. 1000M, 2G) (Default: 512 Mb)\n" +
diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
index 837b7e12cb0de..77eb1276a0c4e 100644
--- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
+++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala
@@ -173,6 +173,9 @@ class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: Spa
object Client {
def main(argStrings: Array[String]) {
+ println("WARNING: This client is deprecated and will be removed in a future version of Spark.")
+ println("Use ./bin/spark-submit with \"--master yarn\"")
+
// Set an env variable indicating we are running in YARN mode.
// Note: anything env variable with SPARK_ prefix gets propagated to all (remote) processes -
// see Client#setupLaunchEnv().