")
if test ! -z "$ERRORS"; then
echo -e "Scalastyle checks failed at following occurrences:\n$ERRORS"
exit 1
diff --git a/docs/configuration.md b/docs/configuration.md
index 1ff0150567255..b6005acac8b93 100644
--- a/docs/configuration.md
+++ b/docs/configuration.md
@@ -122,6 +122,21 @@ Apart from these, the following properties are also available, and may be useful
spark.storage.memoryFraction
.
+
+ spark.tachyonStore.baseDir |
+ System.getProperty("java.io.tmpdir") |
+
+ Directories of the Tachyon File System that store RDDs. The Tachyon file system's URL is set by spark.tachyonStore.url .
+ It can also be a comma-separated list of multiple directories on Tachyon file system.
+ |
+
+
+ spark.tachyonStore.url |
+ tachyon://localhost:19998 |
+
+ The URL of the underlying Tachyon file system in the TachyonStore.
+ |
+
spark.mesos.coarse |
false |
@@ -161,13 +176,13 @@ Apart from these, the following properties are also available, and may be useful
spark.ui.acls.enable |
false |
- Whether spark web ui acls should are enabled. If enabled, this checks to see if the user has
+ Whether spark web ui acls should are enabled. If enabled, this checks to see if the user has
access permissions to view the web ui. See spark.ui.view.acls for more details.
Also note this requires the user to be known, if the user comes across as null no checks
are done. Filters can be used to authenticate and set the user.
|
-
+
spark.ui.view.acls |
Empty |
@@ -276,10 +291,10 @@ Apart from these, the following properties are also available, and may be useful
| spark.serializer.objectStreamReset |
10000 |
- When serializing using org.apache.spark.serializer.JavaSerializer, the serializer caches
- objects to prevent writing redundant data, however that stops garbage collection of those
- objects. By calling 'reset' you flush that info from the serializer, and allow old
- objects to be collected. To turn off this periodic reset set it to a value of <= 0.
+ When serializing using org.apache.spark.serializer.JavaSerializer, the serializer caches
+ objects to prevent writing redundant data, however that stops garbage collection of those
+ objects. By calling 'reset' you flush that info from the serializer, and allow old
+ objects to be collected. To turn off this periodic reset set it to a value of <= 0.
By default it will reset the serializer every 10,000 objects.
|
@@ -375,7 +390,7 @@ Apart from these, the following properties are also available, and may be useful
spark.akka.heartbeat.interval |
1000 |
- This is set to a larger value to disable failure detector that comes inbuilt akka. It can be enabled again, if you plan to use this feature (Not recommended). A larger interval value in seconds reduces network overhead and a smaller value ( ~ 1 s) might be more informative for akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` and `spark.akka.failure-detector.threshold` if you need to. Only positive use case for using failure detector can be, a sensistive failure detector can help evict rogue executors really quick. However this is usually not the case as gc pauses and network lags are expected in a real spark cluster. Apart from that enabling this leads to a lot of exchanges of heart beats between nodes leading to flooding the network with those.
+ This is set to a larger value to disable failure detector that comes inbuilt akka. It can be enabled again, if you plan to use this feature (Not recommended). A larger interval value in seconds reduces network overhead and a smaller value ( ~ 1 s) might be more informative for akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` and `spark.akka.failure-detector.threshold` if you need to. Only positive use case for using failure detector can be, a sensistive failure detector can help evict rogue executors really quick. However this is usually not the case as gc pauses and network lags are expected in a real spark cluster. Apart from that enabling this leads to a lot of exchanges of heart beats between nodes leading to flooding the network with those.
|
@@ -430,7 +445,7 @@ Apart from these, the following properties are also available, and may be useful
spark.broadcast.blockSize |
4096 |
- Size of each piece of a block in kilobytes for TorrentBroadcastFactory .
+ Size of each piece of a block in kilobytes for TorrentBroadcastFactory .
Too large a value decreases parallelism during broadcast (makes it slower); however, if it is too small, BlockManager might take a performance hit.
|
@@ -555,7 +570,7 @@ Apart from these, the following properties are also available, and may be useful
the driver.
-
+
spark.authenticate |
false |
@@ -563,7 +578,7 @@ Apart from these, the following properties are also available, and may be useful
running on Yarn.
|
-
+
spark.authenticate.secret |
None |
@@ -571,12 +586,12 @@ Apart from these, the following properties are also available, and may be useful
not running on Yarn and authentication is enabled.
|
-
+
spark.core.connection.auth.wait.timeout |
30 |
Number of seconds for the connection to wait for authentication to occur before timing
- out and giving up.
+ out and giving up.
|
diff --git a/docs/quick-start.md b/docs/quick-start.md
index 13df6beea16e8..60e8b1ba0eb46 100644
--- a/docs/quick-start.md
+++ b/docs/quick-start.md
@@ -124,7 +124,7 @@ object SimpleApp {
}
{% endhighlight %}
-This program just counts the number of lines containing 'a' and the number containing 'b' in the Spark README. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, we initialize a SparkContext as part of the proogram. We pass the SparkContext constructor four arguments, the type of scheduler we want to use (in this case, a local scheduler), a name for the application, the directory where Spark is installed, and a name for the jar file containing the application's code. The final two arguments are needed in a distributed setting, where Spark is running across several nodes, so we include them for completeness. Spark will automatically ship the jar files you list to slave nodes.
+This program just counts the number of lines containing 'a' and the number containing 'b' in the Spark README. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, we initialize a SparkContext as part of the program. We pass the SparkContext constructor four arguments, the type of scheduler we want to use (in this case, a local scheduler), a name for the application, the directory where Spark is installed, and a name for the jar file containing the application's code. The final two arguments are needed in a distributed setting, where Spark is running across several nodes, so we include them for completeness. Spark will automatically ship the jar files you list to slave nodes.
This file depends on the Spark API, so we'll also include an sbt configuration file, `simple.sbt` which explains that Spark is a dependency. This file also adds a repository that Spark depends on:
diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md
index 99412733d4268..77373890eead7 100644
--- a/docs/scala-programming-guide.md
+++ b/docs/scala-programming-guide.md
@@ -23,7 +23,7 @@ To write a Spark application, you need to add a dependency on Spark. If you use
groupId = org.apache.spark
artifactId = spark-core_{{site.SCALA_BINARY_VERSION}}
- version = {{site.SPARK_VERSION}}
+ version = {{site.SPARK_VERSION}}
In addition, if you wish to access an HDFS cluster, you need to add a dependency on `hadoop-client` for your version of HDFS:
@@ -73,14 +73,14 @@ The master URL passed to Spark can be in one of the following formats:
Master URL | Meaning |
local | Run Spark locally with one worker thread (i.e. no parallelism at all). |
- local[K] | Run Spark locally with K worker threads (ideally, set this to the number of cores on your machine).
+ |
local[K] | Run Spark locally with K worker threads (ideally, set this to the number of cores on your machine).
|
- spark://HOST:PORT | Connect to the given Spark standalone
- cluster master. The port must be whichever one your master is configured to use, which is 7077 by default.
+ |
spark://HOST:PORT | Connect to the given Spark standalone
+ cluster master. The port must be whichever one your master is configured to use, which is 7077 by default.
|
- mesos://HOST:PORT | Connect to the given Mesos cluster.
- The host parameter is the hostname of the Mesos master. The port must be whichever one the master is configured to use,
- which is 5050 by default.
+ |
mesos://HOST:PORT | Connect to the given Mesos cluster.
+ The host parameter is the hostname of the Mesos master. The port must be whichever one the master is configured to use,
+ which is 5050 by default.
|
@@ -265,11 +265,25 @@ A complete list of actions is available in the [RDD API doc](api/core/index.html
## RDD Persistence
-One of the most important capabilities in Spark is *persisting* (or *caching*) a dataset in memory across operations. When you persist an RDD, each node stores any slices of it that it computes in memory and reuses them in other actions on that dataset (or datasets derived from it). This allows future actions to be much faster (often by more than 10x). Caching is a key tool for building iterative algorithms with Spark and for interactive use from the interpreter.
-
-You can mark an RDD to be persisted using the `persist()` or `cache()` methods on it. The first time it is computed in an action, it will be kept in memory on the nodes. The cache is fault-tolerant -- if any partition of an RDD is lost, it will automatically be recomputed using the transformations that originally created it.
-
-In addition, each RDD can be stored using a different *storage level*, allowing you, for example, to persist the dataset on disk, or persist it in memory but as serialized Java objects (to save space), or even replicate it across nodes. These levels are chosen by passing a [`org.apache.spark.storage.StorageLevel`](api/core/index.html#org.apache.spark.storage.StorageLevel) object to `persist()`. The `cache()` method is a shorthand for using the default storage level, which is `StorageLevel.MEMORY_ONLY` (store deserialized objects in memory). The complete set of available storage levels is:
+One of the most important capabilities in Spark is *persisting* (or *caching*) a dataset in memory
+across operations. When you persist an RDD, each node stores any slices of it that it computes in
+memory and reuses them in other actions on that dataset (or datasets derived from it). This allows
+future actions to be much faster (often by more than 10x). Caching is a key tool for building
+iterative algorithms with Spark and for interactive use from the interpreter.
+
+You can mark an RDD to be persisted using the `persist()` or `cache()` methods on it. The first time
+it is computed in an action, it will be kept in memory on the nodes. The cache is fault-tolerant --
+if any partition of an RDD is lost, it will automatically be recomputed using the transformations
+that originally created it.
+
+In addition, each RDD can be stored using a different *storage level*, allowing you, for example, to
+persist the dataset on disk, or persist it in memory but as serialized Java objects (to save space),
+or replicate it across nodes, or store the data in off-heap memory in [Tachyon](http://tachyon-project.org/).
+These levels are chosen by passing a
+[`org.apache.spark.storage.StorageLevel`](api/core/index.html#org.apache.spark.storage.StorageLevel)
+object to `persist()`. The `cache()` method is a shorthand for using the default storage level,
+which is `StorageLevel.MEMORY_ONLY` (store deserialized objects in memory). The complete set of
+available storage levels is:
Storage Level | Meaning |
@@ -292,8 +306,16 @@ In addition, each RDD can be stored using a different *storage level*, allowing
MEMORY_AND_DISK_SER |
- Similar to MEMORY_ONLY_SER, but spill partitions that don't fit in memory to disk instead of recomputing them
- on the fly each time they're needed. |
+ Similar to MEMORY_ONLY_SER, but spill partitions that don't fit in memory to disk instead of
+ recomputing them on the fly each time they're needed. |
+
+
+ OFF_HEAP |
+ Store RDD in a serialized format in Tachyon.
+ This is generally more space-efficient than deserialized objects, especially when using a
+ fast serializer, but more CPU-intensive to read.
+ This also significantly reduces the overheads of GC.
+ |
DISK_ONLY |
@@ -307,30 +329,59 @@ In addition, each RDD can be stored using a different *storage level*, allowing
### Which Storage Level to Choose?
-Spark's storage levels are meant to provide different tradeoffs between memory usage and CPU efficiency.
-We recommend going through the following process to select one:
-
-* If your RDDs fit comfortably with the default storage level (`MEMORY_ONLY`), leave them that way. This is the most
- CPU-efficient option, allowing operations on the RDDs to run as fast as possible.
-* If not, try using `MEMORY_ONLY_SER` and [selecting a fast serialization library](tuning.html) to make the objects
- much more space-efficient, but still reasonably fast to access.
-* Don't spill to disk unless the functions that computed your datasets are expensive, or they filter a large
- amount of the data. Otherwise, recomputing a partition is about as fast as reading it from disk.
-* Use the replicated storage levels if you want fast fault recovery (e.g. if using Spark to serve requests from a web
- application). *All* the storage levels provide full fault tolerance by recomputing lost data, but the replicated ones
- let you continue running tasks on the RDD without waiting to recompute a lost partition.
-
-If you want to define your own storage level (say, with replication factor of 3 instead of 2), then use the function factor method `apply()` of the [`StorageLevel`](api/core/index.html#org.apache.spark.storage.StorageLevel$) singleton object.
+Spark's storage levels are meant to provide different trade-offs between memory usage and CPU
+efficiency. It allows uses to choose memory, disk, or Tachyon for storing data. We recommend going
+through the following process to select one:
+
+* If your RDDs fit comfortably with the default storage level (`MEMORY_ONLY`), leave them that way.
+ This is the most CPU-efficient option, allowing operations on the RDDs to run as fast as possible.
+
+* If not, try using `MEMORY_ONLY_SER` and [selecting a fast serialization library](tuning.html) to
+make the objects much more space-efficient, but still reasonably fast to access. You can also use
+`OFF_HEAP` mode to store the data off the heap in [Tachyon](http://tachyon-project.org/). This will
+significantly reduce JVM GC overhead.
+
+* Don't spill to disk unless the functions that computed your datasets are expensive, or they filter
+a large amount of the data. Otherwise, recomputing a partition is about as fast as reading it from
+disk.
+
+* Use the replicated storage levels if you want fast fault recovery (e.g. if using Spark to serve
+requests from a web application). *All* the storage levels provide full fault tolerance by
+recomputing lost data, but the replicated ones let you continue running tasks on the RDD without
+waiting to recompute a lost partition.
+
+If you want to define your own storage level (say, with replication factor of 3 instead of 2), then
+use the function factor method `apply()` of the
+[`StorageLevel`](api/core/index.html#org.apache.spark.storage.StorageLevel$) singleton object.
+
+Spark has a block manager inside the Executors that let you chose memory, disk, or off-heap. The
+latter is for storing RDDs off-heap outside the Executor JVM on top of the memory management system
+[Tachyon](http://tachyon-project.org/). This mode has the following advantages:
+
+* Cached data will not be lost if individual executors crash.
+* Executors can have a smaller memory footprint, allowing you to run more executors on the same
+machine as the bulk of the memory will be inside Tachyon.
+* Reduced GC overhead since data is stored in Tachyon.
# Shared Variables
-Normally, when a function passed to a Spark operation (such as `map` or `reduce`) is executed on a remote cluster node, it works on separate copies of all the variables used in the function. These variables are copied to each machine, and no updates to the variables on the remote machine are propagated back to the driver program. Supporting general, read-write shared variables across tasks would be inefficient. However, Spark does provide two limited types of *shared variables* for two common usage patterns: broadcast variables and accumulators.
+Normally, when a function passed to a Spark operation (such as `map` or `reduce`) is executed on a
+remote cluster node, it works on separate copies of all the variables used in the function. These
+variables are copied to each machine, and no updates to the variables on the remote machine are
+propagated back to the driver program. Supporting general, read-write shared variables across tasks
+would be inefficient. However, Spark does provide two limited types of *shared variables* for two
+common usage patterns: broadcast variables and accumulators.
## Broadcast Variables
-Broadcast variables allow the programmer to keep a read-only variable cached on each machine rather than shipping a copy of it with tasks. They can be used, for example, to give every node a copy of a large input dataset in an efficient manner. Spark also attempts to distribute broadcast variables using efficient broadcast algorithms to reduce communication cost.
+Broadcast variables allow the programmer to keep a read-only variable cached on each machine rather
+than shipping a copy of it with tasks. They can be used, for example, to give every node a copy of a
+large input dataset in an efficient manner. Spark also attempts to distribute broadcast variables
+using efficient broadcast algorithms to reduce communication cost.
-Broadcast variables are created from a variable `v` by calling `SparkContext.broadcast(v)`. The broadcast variable is a wrapper around `v`, and its value can be accessed by calling the `value` method. The interpreter session below shows this:
+Broadcast variables are created from a variable `v` by calling `SparkContext.broadcast(v)`. The
+broadcast variable is a wrapper around `v`, and its value can be accessed by calling the `value`
+method. The interpreter session below shows this:
{% highlight scala %}
scala> val broadcastVar = sc.broadcast(Array(1, 2, 3))
@@ -340,13 +391,21 @@ scala> broadcastVar.value
res0: Array[Int] = Array(1, 2, 3)
{% endhighlight %}
-After the broadcast variable is created, it should be used instead of the value `v` in any functions run on the cluster so that `v` is not shipped to the nodes more than once. In addition, the object `v` should not be modified after it is broadcast in order to ensure that all nodes get the same value of the broadcast variable (e.g. if the variable is shipped to a new node later).
+After the broadcast variable is created, it should be used instead of the value `v` in any functions
+run on the cluster so that `v` is not shipped to the nodes more than once. In addition, the object
+`v` should not be modified after it is broadcast in order to ensure that all nodes get the same
+value of the broadcast variable (e.g. if the variable is shipped to a new node later).
## Accumulators
-Accumulators are variables that are only "added" to through an associative operation and can therefore be efficiently supported in parallel. They can be used to implement counters (as in MapReduce) or sums. Spark natively supports accumulators of numeric value types and standard mutable collections, and programmers can add support for new types.
+Accumulators are variables that are only "added" to through an associative operation and can
+therefore be efficiently supported in parallel. They can be used to implement counters (as in
+MapReduce) or sums. Spark natively supports accumulators of numeric value types and standard mutable
+collections, and programmers can add support for new types.
-An accumulator is created from an initial value `v` by calling `SparkContext.accumulator(v)`. Tasks running on the cluster can then add to it using the `+=` operator. However, they cannot read its value. Only the driver program can read the accumulator's value, using its `value` method.
+An accumulator is created from an initial value `v` by calling `SparkContext.accumulator(v)`. Tasks
+running on the cluster can then add to it using the `+=` operator. However, they cannot read its
+value. Only the driver program can read the accumulator's value, using its `value` method.
The interpreter session below shows an accumulator being used to add up the elements of an array:
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala
index e5a09ecec006f..d3babc3ed12c8 100644
--- a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala
@@ -18,8 +18,8 @@
package org.apache.spark.examples
import scala.math.random
+
import org.apache.spark._
-import SparkContext._
/** Computes an approximation to pi */
object SparkPi {
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala
new file mode 100644
index 0000000000000..53b303d658386
--- /dev/null
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.examples
+
+import java.util.Random
+import scala.math.exp
+import org.apache.spark.util.Vector
+import org.apache.spark._
+import org.apache.spark.deploy.SparkHadoopUtil
+import org.apache.spark.scheduler.InputFormatInfo
+import org.apache.spark.storage.StorageLevel
+
+/**
+ * Logistic regression based classification.
+ * This example uses Tachyon to persist rdds during computation.
+ */
+object SparkTachyonHdfsLR {
+ val D = 10 // Numer of dimensions
+ val rand = new Random(42)
+
+ case class DataPoint(x: Vector, y: Double)
+
+ def parsePoint(line: String): DataPoint = {
+ val tok = new java.util.StringTokenizer(line, " ")
+ var y = tok.nextToken.toDouble
+ var x = new Array[Double](D)
+ var i = 0
+ while (i < D) {
+ x(i) = tok.nextToken.toDouble; i += 1
+ }
+ DataPoint(new Vector(x), y)
+ }
+
+ def main(args: Array[String]) {
+ if (args.length < 3) {
+ System.err.println("Usage: SparkTachyonHdfsLR ")
+ System.exit(1)
+ }
+ val inputPath = args(1)
+ val conf = SparkHadoopUtil.get.newConfiguration()
+ val sc = new SparkContext(args(0), "SparkTachyonHdfsLR",
+ System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass), Map(),
+ InputFormatInfo.computePreferredLocations(
+ Seq(new InputFormatInfo(conf, classOf[org.apache.hadoop.mapred.TextInputFormat], inputPath))
+ ))
+ val lines = sc.textFile(inputPath)
+ val points = lines.map(parsePoint _).persist(StorageLevel.OFF_HEAP)
+ val ITERATIONS = args(2).toInt
+
+ // Initialize w to a random value
+ var w = Vector(D, _ => 2 * rand.nextDouble - 1)
+ println("Initial w: " + w)
+
+ for (i <- 1 to ITERATIONS) {
+ println("On iteration " + i)
+ val gradient = points.map { p =>
+ (1 / (1 + exp(-p.y * (w dot p.x))) - 1) * p.y * p.x
+ }.reduce(_ + _)
+ w -= gradient
+ }
+
+ println("Final w: " + w)
+ System.exit(0)
+ }
+}
diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala
new file mode 100644
index 0000000000000..ce78f0876ed7c
--- /dev/null
+++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.examples
+
+import scala.math.random
+
+import org.apache.spark._
+import org.apache.spark.storage.StorageLevel
+
+/**
+ * Computes an approximation to pi
+ * This example uses Tachyon to persist rdds during computation.
+ */
+object SparkTachyonPi {
+ def main(args: Array[String]) {
+ if (args.length == 0) {
+ System.err.println("Usage: SparkTachyonPi []")
+ System.exit(1)
+ }
+ val spark = new SparkContext(args(0), "SparkTachyonPi",
+ System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass))
+
+ val slices = if (args.length > 1) args(1).toInt else 2
+ val n = 100000 * slices
+
+ val rdd = spark.parallelize(1 to n, slices)
+ rdd.persist(StorageLevel.OFF_HEAP)
+ val count = rdd.map { i =>
+ val x = random * 2 - 1
+ val y = random * 2 - 1
+ if (x * x + y * y < 1) 1 else 0
+ }.reduce(_ + _)
+ println("Pi is roughly " + 4.0 * count / n)
+
+ spark.stop()
+ }
+}
diff --git a/examples/src/main/scala/org/apache/spark/sql/examples/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/sql/examples/HiveFromSpark.scala
index abcc1f04d4279..62329bde84481 100644
--- a/examples/src/main/scala/org/apache/spark/sql/examples/HiveFromSpark.scala
+++ b/examples/src/main/scala/org/apache/spark/sql/examples/HiveFromSpark.scala
@@ -33,20 +33,20 @@ object HiveFromSpark {
val hiveContext = new LocalHiveContext(sc)
import hiveContext._
- sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)")
- sql("LOAD DATA LOCAL INPATH 'src/main/resources/kv1.txt' INTO TABLE src")
+ hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)")
+ hql("LOAD DATA LOCAL INPATH 'src/main/resources/kv1.txt' INTO TABLE src")
// Queries are expressed in HiveQL
println("Result of 'SELECT *': ")
- sql("SELECT * FROM src").collect.foreach(println)
+ hql("SELECT * FROM src").collect.foreach(println)
// Aggregation queries are also supported.
- val count = sql("SELECT COUNT(*) FROM src").collect().head.getInt(0)
+ val count = hql("SELECT COUNT(*) FROM src").collect().head.getInt(0)
println(s"COUNT(*): $count")
// The results of SQL queries are themselves RDDs and support all normal RDD functions. The
// items in the RDD are of type Row, which allows you to access each column by ordinal.
- val rddFromSql = sql("SELECT key, value FROM src WHERE key < 10 ORDER BY key")
+ val rddFromSql = hql("SELECT key, value FROM src WHERE key < 10 ORDER BY key")
println("Result of RDD.map:")
val rddAsStrings = rddFromSql.map {
@@ -59,6 +59,6 @@ object HiveFromSpark {
// Queries can then join RDD data with data stored in Hive.
println("Result of SELECT *:")
- sql("SELECT * FROM records r JOIN src s ON r.key = s.key").collect().foreach(println)
+ hql("SELECT * FROM records r JOIN src s ON r.key = s.key").collect().foreach(println)
}
}
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index c5c697e8e2427..843a874fbfdb0 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -30,7 +30,7 @@ import scala.collection.JavaConversions._
// import com.jsuereth.pgp.sbtplugin.PgpKeys._
object SparkBuild extends Build {
- val SPARK_VERSION = "1.0.0-SNAPSHOT"
+ val SPARK_VERSION = "1.0.0-SNAPSHOT"
// Hadoop version to build against. For example, "1.0.4" for Apache releases, or
// "2.0.0-mr1-cdh4.2.0" for Cloudera Hadoop. Note that these variables can be set
@@ -185,15 +185,14 @@ object SparkBuild extends Build {
concurrentRestrictions in Global += Tags.limit(Tags.Test, 1),
resolvers ++= Seq(
- // HTTPS is unavailable for Maven Central
"Maven Repository" at "http://repo.maven.apache.org/maven2",
"Apache Repository" at "https://repository.apache.org/content/repositories/releases",
"JBoss Repository" at "https://repository.jboss.org/nexus/content/repositories/releases/",
"MQTT Repository" at "https://repo.eclipse.org/content/repositories/paho-releases/",
- "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/",
+ "Cloudera Repository" at "http://repository.cloudera.com/artifactory/cloudera-repos/",
// For Sonatype publishing
- //"sonatype-snapshots" at "https://oss.sonatype.org/content/repositories/snapshots",
- //"sonatype-staging" at "https://oss.sonatype.org/service/local/staging/deploy/maven2/",
+ // "sonatype-snapshots" at "https://oss.sonatype.org/content/repositories/snapshots",
+ // "sonatype-staging" at "https://oss.sonatype.org/service/local/staging/deploy/maven2/",
// also check the local Maven repository ~/.m2
Resolver.mavenLocal
),
@@ -280,13 +279,18 @@ object SparkBuild extends Build {
val slf4jVersion = "1.7.5"
val excludeNetty = ExclusionRule(organization = "org.jboss.netty")
+ val excludeEclipseJetty = ExclusionRule(organization = "org.eclipse.jetty")
val excludeAsm = ExclusionRule(organization = "org.ow2.asm")
val excludeOldAsm = ExclusionRule(organization = "asm")
val excludeCommonsLogging = ExclusionRule(organization = "commons-logging")
val excludeSLF4J = ExclusionRule(organization = "org.slf4j")
val excludeScalap = ExclusionRule(organization = "org.scala-lang", artifact = "scalap")
+ val excludeHadoop = ExclusionRule(organization = "org.apache.hadoop")
+ val excludeCurator = ExclusionRule(organization = "org.apache.curator")
+ val excludePowermock = ExclusionRule(organization = "org.powermock")
- def sparkPreviousArtifact(id: String, organization: String = "org.apache.spark",
+
+ def sparkPreviousArtifact(id: String, organization: String = "org.apache.spark",
version: String = "0.9.0-incubating", crossVersion: String = "2.10"): Option[sbt.ModuleID] = {
val fullId = if (crossVersion.isEmpty) id else id + "_" + crossVersion
Some(organization % fullId % version) // the artifact to compare binary compatibility with
@@ -323,6 +327,7 @@ object SparkBuild extends Build {
"com.codahale.metrics" % "metrics-graphite" % "3.0.0",
"com.twitter" %% "chill" % "0.3.1" excludeAll(excludeAsm),
"com.twitter" % "chill-java" % "0.3.1" excludeAll(excludeAsm),
+ "org.tachyonproject" % "tachyon" % "0.4.1-thrift" excludeAll(excludeHadoop, excludeCurator, excludeEclipseJetty, excludePowermock),
"com.clearspring.analytics" % "stream" % "2.5.1"
),
libraryDependencies ++= maybeAvro
diff --git a/python/pyspark/context.py b/python/pyspark/context.py
index ff1023bbfa539..d8667e84fedff 100644
--- a/python/pyspark/context.py
+++ b/python/pyspark/context.py
@@ -423,8 +423,11 @@ def _getJavaStorageLevel(self, storageLevel):
raise Exception("storageLevel must be of type pyspark.StorageLevel")
newStorageLevel = self._jvm.org.apache.spark.storage.StorageLevel
- return newStorageLevel(storageLevel.useDisk, storageLevel.useMemory,
- storageLevel.deserialized, storageLevel.replication)
+ return newStorageLevel(storageLevel.useDisk,
+ storageLevel.useMemory,
+ storageLevel.useOffHeap,
+ storageLevel.deserialized,
+ storageLevel.replication)
def setJobGroup(self, groupId, description):
"""
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index 9943296b927dc..fb27863e07f55 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -1302,11 +1302,12 @@ def getStorageLevel(self):
Get the RDD's current storage level.
>>> rdd1 = sc.parallelize([1,2])
>>> rdd1.getStorageLevel()
- StorageLevel(False, False, False, 1)
+ StorageLevel(False, False, False, False, 1)
"""
java_storage_level = self._jrdd.getStorageLevel()
storage_level = StorageLevel(java_storage_level.useDisk(),
java_storage_level.useMemory(),
+ java_storage_level.useOffHeap(),
java_storage_level.deserialized(),
java_storage_level.replication())
return storage_level
diff --git a/python/pyspark/storagelevel.py b/python/pyspark/storagelevel.py
index c3e3a44e8e7ab..7b6660eab231b 100644
--- a/python/pyspark/storagelevel.py
+++ b/python/pyspark/storagelevel.py
@@ -25,23 +25,25 @@ class StorageLevel:
Also contains static constants for some commonly used storage levels, such as MEMORY_ONLY.
"""
- def __init__(self, useDisk, useMemory, deserialized, replication = 1):
+ def __init__(self, useDisk, useMemory, useOffHeap, deserialized, replication = 1):
self.useDisk = useDisk
self.useMemory = useMemory
+ self.useOffHeap = useOffHeap
self.deserialized = deserialized
self.replication = replication
def __repr__(self):
- return "StorageLevel(%s, %s, %s, %s)" % (
- self.useDisk, self.useMemory, self.deserialized, self.replication)
+ return "StorageLevel(%s, %s, %s, %s, %s)" % (
+ self.useDisk, self.useMemory, self.useOffHeap, self.deserialized, self.replication)
-StorageLevel.DISK_ONLY = StorageLevel(True, False, False)
-StorageLevel.DISK_ONLY_2 = StorageLevel(True, False, False, 2)
-StorageLevel.MEMORY_ONLY = StorageLevel(False, True, True)
-StorageLevel.MEMORY_ONLY_2 = StorageLevel(False, True, True, 2)
-StorageLevel.MEMORY_ONLY_SER = StorageLevel(False, True, False)
-StorageLevel.MEMORY_ONLY_SER_2 = StorageLevel(False, True, False, 2)
-StorageLevel.MEMORY_AND_DISK = StorageLevel(True, True, True)
-StorageLevel.MEMORY_AND_DISK_2 = StorageLevel(True, True, True, 2)
-StorageLevel.MEMORY_AND_DISK_SER = StorageLevel(True, True, False)
-StorageLevel.MEMORY_AND_DISK_SER_2 = StorageLevel(True, True, False, 2)
+StorageLevel.DISK_ONLY = StorageLevel(True, False, False, False)
+StorageLevel.DISK_ONLY_2 = StorageLevel(True, False, False, False, 2)
+StorageLevel.MEMORY_ONLY = StorageLevel(False, True, False, True)
+StorageLevel.MEMORY_ONLY_2 = StorageLevel(False, True, False, True, 2)
+StorageLevel.MEMORY_ONLY_SER = StorageLevel(False, True, False, False)
+StorageLevel.MEMORY_ONLY_SER_2 = StorageLevel(False, True, False, False, 2)
+StorageLevel.MEMORY_AND_DISK = StorageLevel(True, True, False, True)
+StorageLevel.MEMORY_AND_DISK_2 = StorageLevel(True, True, False, True, 2)
+StorageLevel.MEMORY_AND_DISK_SER = StorageLevel(True, True, False, False)
+StorageLevel.MEMORY_AND_DISK_SER_2 = StorageLevel(True, True, False, False, 2)
+StorageLevel.OFF_HEAP = StorageLevel(False, False, True, False, 1)
\ No newline at end of file
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
index ff8eaacded4c8..f66a667c0a942 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
@@ -67,14 +67,13 @@ class LocalHiveContext(sc: SparkContext) extends HiveContext(sc) {
class HiveContext(sc: SparkContext) extends SQLContext(sc) {
self =>
- override def parseSql(sql: String): LogicalPlan = HiveQl.parseSql(sql)
- override def executePlan(plan: LogicalPlan): this.QueryExecution =
+ override protected[sql] def executePlan(plan: LogicalPlan): this.QueryExecution =
new this.QueryExecution { val logical = plan }
/**
* Executes a query expressed in HiveQL using Spark, returning the result as a SchemaRDD.
*/
- def hql(hqlQuery: String): SchemaRDD = {
+ def hiveql(hqlQuery: String): SchemaRDD = {
val result = new SchemaRDD(this, HiveQl.parseSql(hqlQuery))
// We force query optimization to happen right away instead of letting it happen lazily like
// when using the query DSL. This is so DDL commands behave as expected. This is only
@@ -83,6 +82,9 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
result
}
+ /** An alias for `hiveql`. */
+ def hql(hqlQuery: String): SchemaRDD = hiveql(hqlQuery)
+
// Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur.
@transient
protected val outputBuffer = new java.io.OutputStream {
@@ -120,7 +122,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
/* A catalyst metadata catalog that points to the Hive Metastore. */
@transient
- override lazy val catalog = new HiveMetastoreCatalog(this) with OverrideCatalog {
+ override protected[sql] lazy val catalog = new HiveMetastoreCatalog(this) with OverrideCatalog {
override def lookupRelation(
databaseName: Option[String],
tableName: String,
@@ -132,7 +134,8 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
/* An analyzer that uses the Hive metastore. */
@transient
- override lazy val analyzer = new Analyzer(catalog, HiveFunctionRegistry, caseSensitive = false)
+ override protected[sql] lazy val analyzer =
+ new Analyzer(catalog, HiveFunctionRegistry, caseSensitive = false)
/**
* Runs the specified SQL query using Hive.
@@ -214,14 +217,14 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
}
@transient
- override val planner = hivePlanner
+ override protected[sql] val planner = hivePlanner
@transient
protected lazy val emptyResult =
sparkContext.parallelize(Seq(new GenericRow(Array[Any]()): Row), 1)
/** Extends QueryExecution with hive specific features. */
- abstract class QueryExecution extends super.QueryExecution {
+ protected[sql] abstract class QueryExecution extends super.QueryExecution {
// TODO: Create mixin for the analyzer instead of overriding things here.
override lazy val optimizedPlan =
optimizer(catalog.PreInsertionCasts(catalog.CreateTables(analyzed)))
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
index 0a6bea0162430..2fea9702954d7 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
@@ -110,10 +110,10 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) {
val describedTable = "DESCRIBE (\\w+)".r
- class SqlQueryExecution(sql: String) extends this.QueryExecution {
- lazy val logical = HiveQl.parseSql(sql)
- def hiveExec() = runSqlHive(sql)
- override def toString = sql + "\n" + super.toString
+ protected[hive] class HiveQLQueryExecution(hql: String) extends this.QueryExecution {
+ lazy val logical = HiveQl.parseSql(hql)
+ def hiveExec() = runSqlHive(hql)
+ override def toString = hql + "\n" + super.toString
}
/**
@@ -140,8 +140,8 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) {
case class TestTable(name: String, commands: (()=>Unit)*)
- implicit class SqlCmd(sql: String) {
- def cmd = () => new SqlQueryExecution(sql).stringResult(): Unit
+ protected[hive] implicit class SqlCmd(sql: String) {
+ def cmd = () => new HiveQLQueryExecution(sql).stringResult(): Unit
}
/**
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
index 18654b308d234..3cc4562a88d66 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
@@ -125,7 +125,7 @@ abstract class HiveComparisonTest
}
protected def prepareAnswer(
- hiveQuery: TestHive.type#SqlQueryExecution,
+ hiveQuery: TestHive.type#HiveQLQueryExecution,
answer: Seq[String]): Seq[String] = {
val orderedAnswer = hiveQuery.logical match {
// Clean out non-deterministic time schema info.
@@ -227,7 +227,7 @@ abstract class HiveComparisonTest
try {
// MINOR HACK: You must run a query before calling reset the first time.
- TestHive.sql("SHOW TABLES")
+ TestHive.hql("SHOW TABLES")
if (reset) { TestHive.reset() }
val hiveCacheFiles = queryList.zipWithIndex.map {
@@ -256,7 +256,7 @@ abstract class HiveComparisonTest
hiveCachedResults
} else {
- val hiveQueries = queryList.map(new TestHive.SqlQueryExecution(_))
+ val hiveQueries = queryList.map(new TestHive.HiveQLQueryExecution(_))
// Make sure we can at least parse everything before attempting hive execution.
hiveQueries.foreach(_.logical)
val computedResults = (queryList.zipWithIndex, hiveQueries, hiveCacheFiles).zipped.map {
@@ -302,7 +302,7 @@ abstract class HiveComparisonTest
// Run w/ catalyst
val catalystResults = queryList.zip(hiveResults).map { case (queryString, hive) =>
- val query = new TestHive.SqlQueryExecution(queryString)
+ val query = new TestHive.HiveQLQueryExecution(queryString)
try { (query, prepareAnswer(query, query.stringResult())) } catch {
case e: Exception =>
val errorMessage =
@@ -359,7 +359,7 @@ abstract class HiveComparisonTest
// When we encounter an error we check to see if the environment is still okay by running a simple query.
// If this fails then we halt testing since something must have gone seriously wrong.
try {
- new TestHive.SqlQueryExecution("SELECT key FROM src").stringResult()
+ new TestHive.HiveQLQueryExecution("SELECT key FROM src").stringResult()
TestHive.runSqlHive("SELECT key FROM src")
} catch {
case e: Exception =>
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
index c184ebe288af4..0c27498a93507 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
@@ -23,6 +23,16 @@ import org.apache.spark.sql.hive.TestHive._
* A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution.
*/
class HiveQuerySuite extends HiveComparisonTest {
+
+ test("Query expressed in SQL") {
+ assert(sql("SELECT 1").collect() === Array(Seq(1)))
+ }
+
+ test("Query expressed in HiveQL") {
+ hql("FROM src SELECT key").collect()
+ hiveql("FROM src SELECT key").collect()
+ }
+
createQueryTest("Simple Average",
"SELECT AVG(key) FROM src")
@@ -133,7 +143,7 @@ class HiveQuerySuite extends HiveComparisonTest {
"SELECT * FROM src LATERAL VIEW explode(map(key+3,key+4)) D as k, v")
test("sampling") {
- sql("SELECT * FROM src TABLESAMPLE(0.1 PERCENT) s")
+ hql("SELECT * FROM src TABLESAMPLE(0.1 PERCENT) s")
}
}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
index 40c4e23f90fb8..8883e5b16d4da 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
@@ -56,7 +56,7 @@ class HiveResolutionSuite extends HiveComparisonTest {
TestHive.sparkContext.parallelize(Data(1, 2, Nested(1,2)) :: Nil)
.registerAsTable("caseSensitivityTest")
- sql("SELECT a, b, A, B, n.a, n.b, n.A, n.B FROM caseSensitivityTest")
+ hql("SELECT a, b, A, B, n.a, n.b, n.A, n.B FROM caseSensitivityTest")
}
/**
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala
index 1318ac1968dad..d9ccb93e23923 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala
@@ -136,7 +136,7 @@ class PruningSuite extends HiveComparisonTest {
expectedScannedColumns: Seq[String],
expectedPartValues: Seq[Seq[String]]) = {
test(s"$testCaseName - pruning test") {
- val plan = new TestHive.SqlQueryExecution(sql).executedPlan
+ val plan = new TestHive.HiveQLQueryExecution(sql).executedPlan
val actualOutputColumns = plan.output.map(_.name)
val (actualScannedColumns, actualPartValues) = plan.collect {
case p @ HiveTableScan(columns, relation, _) =>
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala
index 314ca48ad8f6a..aade62eb8f84e 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala
@@ -57,34 +57,34 @@ class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAft
}
test("SELECT on Parquet table") {
- val rdd = sql("SELECT * FROM testsource").collect()
+ val rdd = hql("SELECT * FROM testsource").collect()
assert(rdd != null)
assert(rdd.forall(_.size == 6))
}
test("Simple column projection + filter on Parquet table") {
- val rdd = sql("SELECT myboolean, mylong FROM testsource WHERE myboolean=true").collect()
+ val rdd = hql("SELECT myboolean, mylong FROM testsource WHERE myboolean=true").collect()
assert(rdd.size === 5, "Filter returned incorrect number of rows")
assert(rdd.forall(_.getBoolean(0)), "Filter returned incorrect Boolean field value")
}
test("Converting Hive to Parquet Table via saveAsParquetFile") {
- sql("SELECT * FROM src").saveAsParquetFile(dirname.getAbsolutePath)
+ hql("SELECT * FROM src").saveAsParquetFile(dirname.getAbsolutePath)
parquetFile(dirname.getAbsolutePath).registerAsTable("ptable")
- val rddOne = sql("SELECT * FROM src").collect().sortBy(_.getInt(0))
- val rddTwo = sql("SELECT * from ptable").collect().sortBy(_.getInt(0))
+ val rddOne = hql("SELECT * FROM src").collect().sortBy(_.getInt(0))
+ val rddTwo = hql("SELECT * from ptable").collect().sortBy(_.getInt(0))
compareRDDs(rddOne, rddTwo, "src (Hive)", Seq("key:Int", "value:String"))
}
test("INSERT OVERWRITE TABLE Parquet table") {
- sql("SELECT * FROM testsource").saveAsParquetFile(dirname.getAbsolutePath)
+ hql("SELECT * FROM testsource").saveAsParquetFile(dirname.getAbsolutePath)
parquetFile(dirname.getAbsolutePath).registerAsTable("ptable")
// let's do three overwrites for good measure
- sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect()
- sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect()
- sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect()
- val rddCopy = sql("SELECT * FROM ptable").collect()
- val rddOrig = sql("SELECT * FROM testsource").collect()
+ hql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect()
+ hql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect()
+ hql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect()
+ val rddCopy = hql("SELECT * FROM ptable").collect()
+ val rddOrig = hql("SELECT * FROM testsource").collect()
assert(rddCopy.size === rddOrig.size, "INSERT OVERWRITE changed size of table??")
compareRDDs(rddOrig, rddCopy, "testsource", ParquetTestData.testSchemaFieldNames)
}
@@ -93,13 +93,13 @@ class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAft
createParquetFile(dirname.getAbsolutePath, ("key", IntegerType), ("value", StringType))
.registerAsTable("tmp")
val rddCopy =
- sql("INSERT INTO TABLE tmp SELECT * FROM src")
+ hql("INSERT INTO TABLE tmp SELECT * FROM src")
.collect()
.sortBy[Int](_.apply(0) match {
case x: Int => x
case _ => 0
})
- val rddOrig = sql("SELECT * FROM src")
+ val rddOrig = hql("SELECT * FROM src")
.collect()
.sortBy(_.getInt(0))
compareRDDs(rddOrig, rddCopy, "src (Hive)", Seq("key:Int", "value:String"))
@@ -108,22 +108,22 @@ class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAft
test("Appending to Parquet table") {
createParquetFile(dirname.getAbsolutePath, ("key", IntegerType), ("value", StringType))
.registerAsTable("tmpnew")
- sql("INSERT INTO TABLE tmpnew SELECT * FROM src").collect()
- sql("INSERT INTO TABLE tmpnew SELECT * FROM src").collect()
- sql("INSERT INTO TABLE tmpnew SELECT * FROM src").collect()
- val rddCopies = sql("SELECT * FROM tmpnew").collect()
- val rddOrig = sql("SELECT * FROM src").collect()
+ hql("INSERT INTO TABLE tmpnew SELECT * FROM src").collect()
+ hql("INSERT INTO TABLE tmpnew SELECT * FROM src").collect()
+ hql("INSERT INTO TABLE tmpnew SELECT * FROM src").collect()
+ val rddCopies = hql("SELECT * FROM tmpnew").collect()
+ val rddOrig = hql("SELECT * FROM src").collect()
assert(rddCopies.size === 3 * rddOrig.size, "number of copied rows via INSERT INTO did not match correct number")
}
test("Appending to and then overwriting Parquet table") {
createParquetFile(dirname.getAbsolutePath, ("key", IntegerType), ("value", StringType))
.registerAsTable("tmp")
- sql("INSERT INTO TABLE tmp SELECT * FROM src").collect()
- sql("INSERT INTO TABLE tmp SELECT * FROM src").collect()
- sql("INSERT OVERWRITE TABLE tmp SELECT * FROM src").collect()
- val rddCopies = sql("SELECT * FROM tmp").collect()
- val rddOrig = sql("SELECT * FROM src").collect()
+ hql("INSERT INTO TABLE tmp SELECT * FROM src").collect()
+ hql("INSERT INTO TABLE tmp SELECT * FROM src").collect()
+ hql("INSERT OVERWRITE TABLE tmp SELECT * FROM src").collect()
+ val rddCopies = hql("SELECT * FROM tmp").collect()
+ val rddOrig = hql("SELECT * FROM src").collect()
assert(rddCopies.size === rddOrig.size, "INSERT OVERWRITE did not actually overwrite")
}