+
-This guide shows the programming model and features by walking through a simple sample receiver and corresponding Spark Streaming application.
+{% highlight scala %}
-### Writing a Simple Receiver
+class CustomReceiver(host: String, port: Int)
+ extends Receiver[String](StorageLevel.MEMORY_AND_DISK_2) with Logging {
+
+ def onStart() {
+ // Start the thread that receives data over a connection
+ new Thread("Socket Receiver") {
+ override def run() { receive() }
+ }.start()
+ }
+
+ def onStop() {
+ // There is nothing much to do as the thread calling receive()
+ // is designed to stop by itself isStopped() returns false
+ }
+
+ /** Create a socket connection and receive data until receiver is stopped */
+ private def receive() {
+ var socket: Socket = null
+ var userInput: String = null
+ try {
+ // Connect to host:port
+ socket = new Socket(host, port)
+
+ // Until stopped or connection broken continue reading
+ val reader = new BufferedReader(new InputStreamReader(socket.getInputStream(), "UTF-8"))
+ userInput = reader.readLine()
+ while(!isStopped && userInput != null) {
+ store(userInput)
+ userInput = reader.readLine()
+ }
+ reader.close()
+ socket.close()
+
+ // Restart in an attempt to connect again when server is active again
+ restart("Trying to connect again")
+ } catch {
+ case e: java.net.ConnectException =>
+ // restart if could not connect to server
+ restart("Error connecting to " + host + ":" + port, e)
+ case t: Throwable =>
+ // restart if there is any other error
+ restart("Error receiving data", t)
+ }
+ }
+}
-This starts with implementing [NetworkReceiver](api/scala/index.html#org.apache.spark.streaming.dstream.NetworkReceiver).
+{% endhighlight %}
-The following is a simple socket text-stream receiver.
+
+
+
+{% highlight java %}
+
+public class JavaCustomReceiver extends Receiver {
+
+ String host = null;
+ int port = -1;
+
+ public JavaCustomReceiver(String host_ , int port_) {
+ super(StorageLevel.MEMORY_AND_DISK_2());
+ host = host_;
+ port = port_;
+ }
+
+ public void onStart() {
+ // Start the thread that receives data over a connection
+ new Thread() {
+ @Override public void run() {
+ receive();
+ }
+ }.start();
+ }
+
+ public void onStop() {
+ // There is nothing much to do as the thread calling receive()
+ // is designed to stop by itself isStopped() returns false
+ }
+
+ /** Create a socket connection and receive data until receiver is stopped */
+ private void receive() {
+ Socket socket = null;
+ String userInput = null;
+
+ try {
+ // connect to the server
+ socket = new Socket(host, port);
+
+ BufferedReader reader = new BufferedReader(new InputStreamReader(socket.getInputStream()));
+
+ // Until stopped or connection broken continue reading
+ while (!isStopped() && (userInput = reader.readLine()) != null) {
+ System.out.println("Received data '" + userInput + "'");
+ store(userInput);
+ }
+ reader.close();
+ socket.close();
+
+ // Restart in an attempt to connect again when server is active again
+ restart("Trying to connect again");
+ } catch(ConnectException ce) {
+ // restart if could not connect to server
+ restart("Could not connect", ce);
+ } catch(Throwable t) {
+ // restart if there is any other error
+ restart("Error receiving data", t);
+ }
+ }
+}
-{% highlight scala %}
- class SocketTextStreamReceiver(host: String, port: Int)
- extends NetworkReceiver[String]
- {
- protected lazy val blocksGenerator: BlockGenerator =
- new BlockGenerator(StorageLevel.MEMORY_ONLY_SER_2)
-
- protected def onStart() = {
- blocksGenerator.start()
- val socket = new Socket(host, port)
- val dataInputStream = new BufferedReader(new InputStreamReader(socket.getInputStream(), "UTF-8"))
- var data: String = dataInputStream.readLine()
- while (data != null) {
- blocksGenerator += data
- data = dataInputStream.readLine()
- }
- }
-
- protected def onStop() {
- blocksGenerator.stop()
- }
- }
{% endhighlight %}
+
+
-All we did here is extended NetworkReceiver and called blockGenerator's API method (i.e. +=) to push our blocks of data. Please refer to scala-docs of NetworkReceiver for more details.
+### Using the custom receiver in a Spark Streaming application
-### An Actor as Receiver
+The custom receiver can be used in a Spark Streaming application by using
+`streamingContext.receiverStream()`. This will create
+input DStream using data received by the instance of custom receiver, as shown below
-This starts with implementing [Actor](#References)
-
-Following is a simple socket text-stream receiver, which is appearently overly simplified using Akka's socket.io api.
+
+
{% highlight scala %}
- class SocketTextStreamReceiver (host:String,
- port:Int,
- bytesToString: ByteString => String) extends Actor with Receiver {
-
- override def preStart = IOManager(context.system).connect(host, port)
-
- def receive = {
- case IO.Read(socket, bytes) => pushBlock(bytesToString(bytes))
- }
-
- }
+// Assuming ssc is the StreamingContext
+val customReceiverStream = ssc.receiverStream(new CustomReceiver(host, port))
+val words = lines.flatMap(_.split(" "))
+...
{% endhighlight %}
-All we did here is mixed in trait Receiver and called pushBlock api method to push our blocks of data. Please refer to scala-docs of Receiver for more details.
-
-### A Sample Spark Application
+The full source code is in the example [CustomReceiver.scala](https://github.com/apache/spark/blob/master/examples/src/main/scala/org/apache/spark/streaming/examples/CustomReceiver.scala).
-* First create a Spark streaming context with master url and batchduration.
+
+
-{% highlight scala %}
- val ssc = new StreamingContext(master, "WordCountCustomStreamSource",
- Seconds(batchDuration))
+{% highlight java %}
+// Assuming ssc is the JavaStreamingContext
+JavaDStream customReceiverStream = ssc.receiverStream(new JavaCustomReceiver(host, port));
+JavaDStream words = lines.flatMap(new FlatMapFunction() { ... });
+...
{% endhighlight %}
-* Plug-in the custom receiver into the spark streaming context and create a DStream.
+The full source code is in the example [JavaCustomReceiver.java](https://github.com/apache/spark/blob/master/examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java).
-{% highlight scala %}
- val lines = ssc.networkStream[String](new SocketTextStreamReceiver(
- "localhost", 8445))
-{% endhighlight %}
+
+
-* OR Plug-in the actor as receiver into the spark streaming context and create a DStream.
-{% highlight scala %}
- val lines = ssc.actorStream[String](Props(new SocketTextStreamReceiver(
- "localhost",8445, z => z.utf8String)),"SocketReceiver")
-{% endhighlight %}
-* Process it.
+### Implementing and Using a Custom Actor-based Receiver
-{% highlight scala %}
- val words = lines.flatMap(_.split(" "))
- val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _)
+Custom [Akka Actors](http://doc.akka.io/docs/akka/2.2.4/scala/actors.html) can also be used to
+receive data. The [`ActorHelper`](api/scala/index.html#org.apache.spark.streaming.receiver.ActorHelper)
+trait can be applied on any Akka actor, which allows received data to be stored in Spark using
+ `store(...)` methods. The supervisor strategy of this actor can be configured to handle failures, etc.
- wordCounts.print()
- ssc.start()
+{% highlight scala %}
+class CustomActor extends Actor with ActorHelper {
+ def receive = {
+ case data: String => store(data)
+ }
+}
{% endhighlight %}
-* After processing it, stream can be tested using the netcat utility.
-
- $ nc -l localhost 8445
- hello world
- hello hello
-
-
-## Multiple Homogeneous/Heterogeneous Receivers.
-
-A DStream union operation is provided for taking union on multiple input streams.
+And a new input stream can be created with this custom actor as
{% highlight scala %}
- val lines = ssc.actorStream[String](Props(new SocketTextStreamReceiver(
- "localhost",8445, z => z.utf8String)),"SocketReceiver")
-
- // Another socket stream receiver
- val lines2 = ssc.actorStream[String](Props(new SocketTextStreamReceiver(
- "localhost",8446, z => z.utf8String)),"SocketReceiver")
-
- val union = lines.union(lines2)
+// Assuming ssc is the StreamingContext
+val lines = ssc.actorStream[String](Props(new CustomActor()), "CustomReceiver")
{% endhighlight %}
-Above stream can be easily process as described earlier.
-
-_A more comprehensive example is provided in the spark streaming examples_
+See [ActorWordCount.scala](https://github.com/apache/spark/blob/master/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala)
+for an end-to-end example.
-## References
-1.[Akka Actor documentation](http://doc.akka.io/docs/akka/2.0.5/scala/actors.html)
-2.[NetworkReceiver](api/scala/index.html#org.apache.spark.streaming.dstream.NetworkReceiver)
diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md
index 7ad06427cac5c..939599aa6855b 100644
--- a/docs/streaming-programming-guide.md
+++ b/docs/streaming-programming-guide.md
@@ -76,16 +76,19 @@ Besides Spark's configuration, we specify that any DStream will be processed
in 1 second batches.
{% highlight scala %}
-// Create a StreamingContext with a SparkConf configuration
-val ssc = new StreamingContext(sparkConf, Seconds(1))
+import org.apache.spark.api.java.function._
+import org.apache.spark.streaming._
+import org.apache.spark.streaming.api._
+// Create a StreamingContext with a local master
+val ssc = new StreamingContext("local", "NetworkWordCount", Seconds(1))
{% endhighlight %}
Using this context, we then create a new DStream
by specifying the IP address and port of the data server.
{% highlight scala %}
-// Create a DStream that will connect to serverIP:serverPort
-val lines = ssc.socketTextStream(serverIP, serverPort)
+// Create a DStream that will connect to serverIP:serverPort, like localhost:9999
+val lines = ssc.socketTextStream("localhost", 9999)
{% endhighlight %}
This `lines` DStream represents the stream of data that will be received from the data
@@ -103,6 +106,7 @@ each line will be split into multiple words and the stream of words is represent
`words` DStream. Next, we want to count these words.
{% highlight scala %}
+import org.apache.spark.streaming.StreamingContext._
// Count each word in each batch
val pairs = words.map(word => (word, 1))
val wordCounts = pairs.reduceByKey(_ + _)
@@ -125,29 +129,33 @@ ssc.awaitTermination() // Wait for the computation to terminate
{% endhighlight %}
The complete code can be found in the Spark Streaming example
-[NetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala).
+[NetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala).
First, we create a
-[JavaStreamingContext](api/scala/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext) object,
+[JavaStreamingContext](api/java/org/apache/spark/streaming/api/java/JavaStreamingContext.html) object,
which is the main entry point for all streaming
functionality. Besides Spark's configuration, we specify that any DStream would be processed
in 1 second batches.
{% highlight java %}
-// Create a StreamingContext with a SparkConf configuration
-JavaStreamingContext jssc = StreamingContext(sparkConf, new Duration(1000))
+import org.apache.spark.api.java.function.*;
+import org.apache.spark.streaming.*;
+import org.apache.spark.streaming.api.java.*;
+import scala.Tuple2;
+// Create a StreamingContext with a local master
+JavaStreamingContext jssc = new JavaStreamingContext("local", "JavaNetworkWordCount", new Duration(1000))
{% endhighlight %}
Using this context, we then create a new DStream
by specifying the IP address and port of the data server.
{% highlight java %}
-// Create a DStream that will connect to serverIP:serverPort
-JavaDStream lines = jssc.socketTextStream(serverIP, serverPort);
+// Create a DStream that will connect to serverIP:serverPort, like localhost:9999
+JavaReceiverInputDStream lines = jssc.socketTextStream("localhost", 9999);
{% endhighlight %}
This `lines` DStream represents the stream of data that will be received from the data
@@ -159,7 +167,7 @@ space into words.
JavaDStream words = lines.flatMap(
new FlatMapFunction() {
@Override public Iterable call(String x) {
- return Lists.newArrayList(x.split(" "));
+ return Arrays.asList(x.split(" "));
}
});
{% endhighlight %}
@@ -207,7 +215,7 @@ jssc.awaitTermination(); // Wait for the computation to terminate
{% endhighlight %}
The complete code can be found in the Spark Streaming example
-[JavaNetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java).
+[JavaNetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java).
@@ -226,12 +234,12 @@ Then, in a different terminal, you can start the example by using
{% highlight bash %}
-$ ./bin/run-example org.apache.spark.streaming.examples.NetworkWordCount local[2] localhost 9999
+$ ./bin/run-example org.apache.spark.examples.streaming.NetworkWordCount local[2] localhost 9999
{% endhighlight %}
{% highlight bash %}
-$ ./bin/run-example org.apache.spark.streaming.examples.JavaNetworkWordCount local[2] localhost 9999
+$ ./bin/run-example org.apache.spark.examples.streaming.JavaNetworkWordCount local[2] localhost 9999
{% endhighlight %}
@@ -260,7 +268,7 @@ hello world
{% highlight bash %}
# TERMINAL 2: RUNNING NetworkWordCount or JavaNetworkWordCount
-$ ./bin/run-example org.apache.spark.streaming.examples.NetworkWordCount local[2] localhost 9999
+$ ./bin/run-example org.apache.spark.examples.streaming.NetworkWordCount local[2] localhost 9999
...
-------------------------------------------
Time: 1357008430000 ms
@@ -359,7 +367,7 @@ as explained earlier. Finally, the last two parameters are needed to deploy your
if running in distributed mode, as described in the
[Spark programming guide](scala-programming-guide.html#deploying-code-on-a-cluster).
Additionally, the underlying SparkContext can be accessed as
-`streamingContext.sparkContext`.
+`ssc.sparkContext`.
The batch interval must be set based on the latency requirements of your application
and available cluster resources. See the [Performance Tuning](#setting-the-right-batch-size)
@@ -399,7 +407,7 @@ These operations are discussed in detail in later sections.
## Input Sources
-We have already taken a look at the `streamingContext.socketTextStream(...)` in the [quick
+We have already taken a look at the `ssc.socketTextStream(...)` in the [quick
example](#a-quick-example) which creates a DStream from text
data received over a TCP socket connection. Besides sockets, the core Spark Streaming API provides
methods for creating DStreams from files and Akka actors as input sources.
@@ -409,12 +417,12 @@ Specifically, for files, the DStream can be created as
{% highlight scala %}
-streamingContext.fileStream(dataDirectory)
+ssc.fileStream(dataDirectory)
{% endhighlight %}
{% highlight java %}
-javaStreamingContext.fileStream(dataDirectory);
+jssc.fileStream(dataDirectory);
{% endhighlight %}
@@ -443,13 +451,13 @@ project dependencies, you can create a DStream from Kafka as
{% highlight scala %}
import org.apache.spark.streaming.kafka._
-KafkaUtils.createStream(streamingContext, kafkaParams, ...)
+KafkaUtils.createStream(ssc, kafkaParams, ...)
{% endhighlight %}
{% highlight java %}
-import org.apache.spark.streaming.kafka.*
-KafkaUtils.createStream(javaStreamingContext, kafkaParams, ...);
+import org.apache.spark.streaming.kafka.*;
+KafkaUtils.createStream(jssc, kafkaParams, ...);
{% endhighlight %}
@@ -578,13 +586,14 @@ val runningCounts = pairs.updateStateByKey[Int](updateFunction _)
{% highlight java %}
+import com.google.common.base.Optional;
Function2
, Optional, Optional> updateFunction =
new Function2, Optional, Optional>() {
@Override public Optional call(List values, Optional state) {
Integer newSum = ... // add the new values with the previous running count to get the new count
- return Optional.of(newSum)
+ return Optional.of(newSum);
}
- }
+ };
{% endhighlight %}
This is applied on a DStream containing words (say, the `pairs` DStream containing `(word,
@@ -600,7 +609,7 @@ JavaPairDStream runningCounts = pairs.updateStateByKey(updateFu
The update function will be called for each word, with `newValues` having a sequence of 1's (from
the `(word, 1)` pairs) and the `runningCount` having the previous count. For the complete
Scala code, take a look at the example
-[StatefulNetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala).
+[StatefulNetworkWordCount]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala).
Transform Operation
@@ -617,9 +626,9 @@ spam information (maybe generated with Spark as well) and then filtering based o
{% highlight scala %}
-val spamInfoRDD = sparkContext.hadoopFile(...) // RDD containing spam information
+val spamInfoRDD = ssc.sparkContext.newAPIHadoopRDD(...) // RDD containing spam information
-val cleanedDStream = inputDStream.transform(rdd => {
+val cleanedDStream = wordCounts.transform(rdd => {
rdd.join(spamInfoRDD).filter(...) // join data stream with spam information to do data cleaning
...
})
@@ -629,13 +638,14 @@ val cleanedDStream = inputDStream.transform(rdd => {
{% highlight java %}
+import org.apache.spark.streaming.api.java.*;
// RDD containing spam information
-JavaPairRDD spamInfoRDD = javaSparkContext.hadoopFile(...);
+final JavaPairRDD spamInfoRDD = jssc.sparkContext().newAPIHadoopRDD(...);
-JavaPairDStream cleanedDStream = inputDStream.transform(
+JavaPairDStream cleanedDStream = wordCounts.transform(
new Function, JavaPairRDD>() {
@Override public JavaPairRDD call(JavaPairRDD rdd) throws Exception {
- rdd.join(spamInfoRDD).filter(...) // join data stream with spam information to do data cleaning
+ rdd.join(spamInfoRDD).filter(...); // join data stream with spam information to do data cleaning
...
}
});
@@ -684,7 +694,7 @@ operation `reduceByKeyAndWindow`.
{% highlight scala %}
// Reduce last 30 seconds of data, every 10 seconds
-val windowedWordCounts = pairs.reduceByKeyAndWindow(_ + _, Seconds(30), Seconds(10))
+val windowedWordCounts = pairs.reduceByKeyAndWindow((a:Int,b:Int) => (a + b), Seconds(30), Seconds(10))
{% endhighlight %}
@@ -699,7 +709,7 @@ Function2
reduceFunc = new Function2 windowedWordCounts = pair.reduceByKeyAndWindow(reduceFunc, new Duration(30000), new Duration(10000));
+JavaPairDStream windowedWordCounts = pairs.reduceByKeyAndWindow(reduceFunc, new Duration(30000), new Duration(10000));
{% endhighlight %}
@@ -853,6 +863,51 @@ For DStreams that must be checkpointed (that is, DStreams created by `updateStat
`reduceByKeyAndWindow` with inverse function), the checkpoint interval of the DStream is by
default set to a multiple of the DStream's sliding interval such that its at least 10 seconds.
+## Deployment
+A Spark Streaming application is deployed on a cluster in the same way as any other Spark application.
+Please refer to the [deployment guide](cluster-overview.html) for more details.
+
+If a running Spark Streaming application needs to be upgraded (with new application code), then
+there are two possible mechanism.
+
+- The upgraded Spark Streaming application is started and run in parallel to the existing application.
+Once the new one (receiving the same data as the old one) has been warmed up and ready
+for prime time, the old one be can be brought down. Note that this can be done for data sources that support
+sending the data to two destinations (i.e., the earlier and upgraded applications).
+
+- The existing application is shutdown gracefully (see
+[`StreamingContext.stop(...)`](api/scala/index.html#org.apache.spark.streaming.StreamingContext)
+or [`JavaStreamingContext.stop(...)`](api/java/org/apache/spark/streaming/api/java/JavaStreamingContext.html)
+for graceful shutdown options) which ensure data that have been received is completely
+processed before shutdown. Then the
+upgraded application can be started, which will start processing from the same point where the earlier
+application left off. Note that this can be done only with input sources that support source-side buffering
+(like Kafka, and Flume) as data needs to be buffered while the previous application down and
+the upgraded application is not yet up.
+
+## Monitoring
+Beyond Spark's [monitoring capabilities](monitoring.html), there are additional capabilities
+specific to Spark Streaming. When a StreamingContext is used, the
+[Spark web UI](monitoring.html#web-interfaces) shows
+an additional `Streaming` tab which shows statistics about running receivers (whether
+receivers are active, number of records received, receiver error, etc.)
+and completed batches (batch processing times, queueing delays, etc.). This can be used to
+monitor the progress of the streaming application.
+
+The following two metrics in web UI is particularly important -
+*Processing Time* and *Scheduling Delay* (under *Batch Processing Statistics*). The first is the
+time to process each batch of data, and the second is the time a batch waits in a queue
+for the processing of previous batches to finish. If the batch processing time is consistently more
+than the batch interval and/or the queueing delay keeps increasing, then it indicates the system is
+not able to process the batches as fast they are being generated and falling behind.
+In that case, consider
+[reducing](#reducing-the-processing-time-of-each-batch) the batch processing time.
+
+The progress of a Spark Streaming program can also be monitored using the
+[StreamingListener](api/scala/index.html#org.apache.spark.scheduler.StreamingListener) interface,
+which allows you to get receiver status and processing times. Note that this is a developer API
+and it is likely to be improved upon (i.e., more information reported) in the future.
+
***************************************************************************************************
# Performance Tuning
@@ -865,7 +920,8 @@ improve the performance of you application. At a high level, you need to conside
Reducing the processing time of each batch of data by efficiently using cluster resources.
- Setting the right batch size such that the data processing can keep up with the data ingestion.
+ Setting the right batch size such that the batches of data can be processed as fast as they
+ are received (that is, data processing keeps up with the data ingestion).
@@ -874,7 +930,30 @@ There are a number of optimizations that can be done in Spark to minimize the pr
each batch. These have been discussed in detail in [Tuning Guide](tuning.html). This section
highlights some of the most important ones.
-### Level of Parallelism
+### Level of Parallelism in Data Receiving
+Receiving data over the network (like Kafka, Flume, socket, etc.) requires the data to deserialized
+and stored in Spark. If the data receiving becomes a bottleneck in the system, then consider
+parallelizing the data receiving. Note that each input DStream
+creates a single receiver (running on a worker machine) that receives a single stream of data.
+Receiving multiple data streams can therefore be achieved by creating multiple input DStreams
+and configuring them to receive different partitions of the data stream from the source(s).
+For example, a single Kafka input stream receiving two topics of data can be split into two
+Kafka input streams, each receiving only one topic. This would run two receivers on two workers,
+thus allowing data to received in parallel, and increasing overall throughput.
+
+Another parameter that should be considered is the receiver's blocking interval. For most receivers,
+the received data is coalesced together into large blocks of data before storing inside Spark's memory.
+The number of blocks in each batch determines the number of tasks that will be used to process those
+the received data in a map-like transformation. This blocking interval is determined by the
+[configuration parameter](configuration.html) `spark.streaming.blockInterval` and the default value
+is 200 milliseconds.
+
+An alternative to receiving data with multiple input streams / receivers is to explicitly repartition
+the input data stream (using `inputStream.repartition()`).
+This distributes the received batches of data across all the machines in the cluster
+before further processing.
+
+### Level of Parallelism in Data Processing
Cluster resources maybe under-utilized if the number of parallel tasks used in any stage of the
computation is not high enough. For example, for distributed reduce operations like `reduceByKey`
and `reduceByKeyAndWindow`, the default number of parallel tasks is 8. You can pass the level of
@@ -911,16 +990,22 @@ These changes may reduce batch processing time by 100s of milliseconds,
thus allowing sub-second batch size to be viable.
## Setting the Right Batch Size
-For a Spark Streaming application running on a cluster to be stable, the processing of the data
-streams must keep up with the rate of ingestion of the data streams. Depending on the type of
-computation, the batch size used may have significant impact on the rate of ingestion that can be
-sustained by the Spark Streaming application on a fixed cluster resources. For example, let us
+For a Spark Streaming application running on a cluster to be stable, the system should be able to
+process data as fast as it is being received. In other words, batches of data should be processed
+as fast as they are being generated. Whether this is true for an application can be found by
+[monitoring](#monitoring) the processing times in the streaming web UI, where the batch
+processing time should be less than the batch interval.
+
+Depending on the nature of the streaming
+computation, the batch interval used may have significant impact on the data rates that can be
+sustained by the application on a fixed set of cluster resources. For example, let us
consider the earlier WordCountNetwork example. For a particular data rate, the system may be able
-to keep up with reporting word counts every 2 seconds (i.e., batch size of 2 seconds), but not
-every 500 milliseconds.
+to keep up with reporting word counts every 2 seconds (i.e., batch interval of 2 seconds), but not
+every 500 milliseconds. So the batch interval needs to be set such that the expected data rate in
+production can be sustained.
A good approach to figure out the right batch size for your application is to test it with a
-conservative batch size (say, 5-10 seconds) and a low data rate. To verify whether the system
+conservative batch interval (say, 5-10 seconds) and a low data rate. To verify whether the system
is able to keep up with data rate, you can check the value of the end-to-end delay experienced
by each processed batch (either look for "Total delay" in Spark driver log4j logs, or use the
[StreamingListener](api/scala/index.html#org.apache.spark.streaming.scheduler.StreamingListener)
@@ -932,29 +1017,6 @@ data rate and/or reducing the batch size. Note that momentary increase in the de
temporary data rate increases maybe fine as long as the delay reduces back to a low value
(i.e., less than batch size).
-## 24/7 Operation
-By default, Spark does not forget any of the metadata (RDDs generated, stages processed, etc.).
-But for a Spark Streaming application to operate 24/7, it is necessary for Spark to do periodic
-cleanup of it metadata. This can be enabled by setting the
-[configuration property](configuration.html#spark-properties) `spark.cleaner.ttl` to the number of
-seconds you want any metadata to persist. For example, setting `spark.cleaner.ttl` to 600 would
-cause Spark periodically cleanup all metadata and persisted RDDs that are older than 10 minutes.
-Note, that this property needs to be set before the SparkContext is created.
-
-This value is closely tied with any window operation that is being used. Any window operation
-would require the input data to be persisted in memory for at least the duration of the window.
-Hence it is necessary to set the delay to at least the value of the largest window operation used
-in the Spark Streaming application. If this delay is set too low, the application will throw an
-exception saying so.
-
-## Monitoring
-Besides Spark's in-built [monitoring capabilities](monitoring.html),
-the progress of a Spark Streaming program can also be monitored using the [StreamingListener]
-(api/scala/index.html#org.apache.spark.scheduler.StreamingListener) interface,
-which allows you to get statistics of batch processing times, queueing delays,
-and total end-to-end delays. Note that this is still an experimental API and it is likely to be
-improved upon (i.e., more information reported) in the future.
-
## Memory Tuning
Tuning the memory usage and GC behavior of Spark applications have been discussed in great detail
in the [Tuning Guide](tuning.html). It is recommended that you read that. In this section,
@@ -1073,7 +1135,7 @@ If the `checkpointDirectory` exists, then the context will be recreated from the
If the directory does not exist (i.e., running for the first time),
then the function `functionToCreateContext` will be called to create a new
context and set up the DStreams. See the Scala example
-[RecoverableNetworkWordCount]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala).
+[RecoverableNetworkWordCount]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala).
This example appends the word counts of network data into a file.
You can also explicitly create a `StreamingContext` from the checkpoint data and start the
@@ -1087,7 +1149,7 @@ This behavior is made simple by using `JavaStreamingContext.getOrCreate`. This i
{% highlight java %}
// Create a factory object that can create a and setup a new JavaStreamingContext
JavaStreamingContextFactory contextFactory = new JavaStreamingContextFactory() {
- JavaStreamingContextFactory create() {
+ @Override public JavaStreamingContext create() {
JavaStreamingContext jssc = new JavaStreamingContext(...); // new context
JavaDStream lines = jssc.socketTextStream(...); // create DStreams
...
@@ -1112,7 +1174,7 @@ If the `checkpointDirectory` exists, then the context will be recreated from the
If the directory does not exist (i.e., running for the first time),
then the function `contextFactory` will be called to create a new
context and set up the DStreams. See the Scala example
-[JavaRecoverableWordCount]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/streaming/examples/JavaRecoverableWordCount.scala)
+[JavaRecoverableWordCount]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/streaming/JavaRecoverableWordCount.scala)
(note that this example is missing in the 0.9 release, so you can test it using the master branch).
This example appends the word counts of network data into a file.
@@ -1239,18 +1301,79 @@ in the file. This is what the sequence of outputs would be with and without a dr
If the driver had crashed in the middle of the processing of time 3, then it will process time 3
and output 30 after recovery.
+***************************************************************************************************
+
+# Migration Guide from 0.9.1 or below to 1.x
+Between Spark 0.9.1 and Spark 1.0, there were a few API changes made to ensure future API stability.
+This section elaborates the steps required to migrate your existing code to 1.0.
+
+**Input DStreams**: All operations that create an input stream (e.g., `StreamingContext.socketStream`,
+`FlumeUtils.createStream`, etc.) now returns
+[InputDStream](api/scala/index.html#org.apache.spark.streaming.dstream.InputDStream) /
+[ReceiverInputDStream](api/scala/index.html#org.apache.spark.streaming.dstream.ReceiverInputDStream)
+(instead of DStream) for Scala, and [JavaInputDStream](api/java/org/apache/spark/streaming/api/java/JavaInputDStream.html) /
+[JavaPairInputDStream](api/java/org/apache/spark/streaming/api/java/JavaPairInputDStream.html) /
+[JavaReceiverInputDStream](api/java/org/apache/spark/streaming/api/java/JavaReceiverInputDStream.html) /
+[JavaPairReceiverInputDStream](api/java/org/apache/spark/streaming/api/java/JavaPairReceiverInputDStream.html)
+(instead of JavaDStream) for Java. This ensures that functionality specific to input streams can
+be added to these classes in the future without breaking binary compatibility.
+Note that your existing Spark Streaming applications should not require any change
+(as these new classes are subclasses of DStream/JavaDStream) but may require recompilation with Spark 1.0.
+
+**Custom Network Receivers**: Since the release to Spark Streaming, custom network receivers could be defined
+in Scala using the class NetworkReceiver. However, the API was limited in terms of error handling
+and reporting, and could not be used from Java. Starting Spark 1.0, this class has been
+replaced by [Receiver](api/scala/index.html#org.apache.spark.streaming.receiver.Receiver) which has
+the following advantages.
+
+* Methods like `stop` and `restart` have been added to for better control of the lifecycle of a receiver. See
+the [custom receiver guide](streaming-custom-receiver.html) for more details.
+* Custom receivers can be implemented using both Scala and Java.
+
+To migrate your existing custom receivers from the earlier NetworkReceiver to the new Receiver, you have
+to do the following.
+
+* Make your custom receiver class extend
+[`org.apache.spark.streaming.receiver.Receiver`](api/scala/index.html#org.apache.spark.streaming.receiver.Receiver)
+instead of `org.apache.spark.streaming.dstream.NetworkReceiver`.
+* Earlier, a BlockGenerator object had to be created by the custom receiver, to which received data was
+added for being stored in Spark. It had to be explicitly started and stopped from `onStart()` and `onStop()`
+methods. The new Receiver class makes this unnecessary as it adds a set of methods named `store()`
+that can be called to store the data in Spark. So, to migrate your custom network receiver, remove any
+BlockGenerator object (does not exist any more in Spark 1.0 anyway), and use `store(...)` methods on
+received data.
+
+**Actor-based Receivers**: Data could have been received using any Akka Actors by extending the actor class with
+`org.apache.spark.streaming.receivers.Receiver` trait. This has been renamed to
+[`org.apache.spark.streaming.receiver.ActorHelper`](api/scala/index.html#org.apache.spark.streaming.receiver.ActorHelper)
+and the `pushBlock(...)` methods to store received data has been renamed to `store(...)`. Other helper classes in
+the `org.apache.spark.streaming.receivers` package were also moved
+to [`org.apache.spark.streaming.receiver`](api/scala/index.html#org.apache.spark.streaming.receiver.package)
+package and renamed for better clarity.
+
+***************************************************************************************************
+
# Where to Go from Here
* API documentation
- - Main docs of StreamingContext and DStreams in [Scala](api/scala/index.html#org.apache.spark.streaming.package)
- and [Java](api/scala/index.html#org.apache.spark.streaming.api.java.package)
- - Additional docs for
- [Kafka](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$),
- [Flume](api/scala/index.html#org.apache.spark.streaming.flume.FlumeUtils$),
- [Twitter](api/scala/index.html#org.apache.spark.streaming.twitter.TwitterUtils$),
- [ZeroMQ](api/scala/index.html#org.apache.spark.streaming.zeromq.ZeroMQUtils$), and
- [MQTT](api/scala/index.html#org.apache.spark.streaming.mqtt.MQTTUtils$)
-
-* More examples in [Scala]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/streaming/examples)
- and [Java]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/streaming/examples)
-* [Paper](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf) describing Spark Streaming.
+ - Scala docs
+ * [StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) and
+ [DStream](api/scala/index.html#org.apache.spark.streaming.dstream.DStream)
+ * [KafkaUtils](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$),
+ [FlumeUtils](api/scala/index.html#org.apache.spark.streaming.flume.FlumeUtils$),
+ [TwitterUtils](api/scala/index.html#org.apache.spark.streaming.twitter.TwitterUtils$),
+ [ZeroMQUtils](api/scala/index.html#org.apache.spark.streaming.zeromq.ZeroMQUtils$), and
+ [MQTTUtils](api/scala/index.html#org.apache.spark.streaming.mqtt.MQTTUtils$)
+ - Java docs
+ * [JavaStreamingContext](api/java/org/apache/spark/streaming/api/java/JavaStreamingContext.html),
+ [JavaDStream](api/java/org/apache/spark/streaming/api/java/JavaDStream.html) and
+ [PairJavaDStream](api/java/org/apache/spark/streaming/api/java/PairJavaDStream.html)
+ * [KafkaUtils](api/java/org/apache/spark/streaming/kafka/KafkaUtils.html),
+ [FlumeUtils](api/java/org/apache/spark/streaming/flume/FlumeUtils.html),
+ [TwitterUtils](api/java/org/apache/spark/streaming/twitter/TwitterUtils.html),
+ [ZeroMQUtils](api/java/org/apache/spark/streaming/zeromq/ZeroMQUtils.html), and
+ [MQTTUtils](api/java/org/apache/spark/streaming/mqtt/MQTTUtils.html)
+
+* More examples in [Scala]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/examples/streaming)
+ and [Java]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/examples/streaming)
+* [Paper](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf) and [video](http://youtu.be/g171ndOHgJ0) describing Spark Streaming.
diff --git a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh
index 42e8faa26ed09..3570891be804e 100644
--- a/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh
+++ b/ec2/deploy.generic/root/spark-ec2/ec2-variables.sh
@@ -28,3 +28,5 @@ export SPARK_VERSION="{{spark_version}}"
export SHARK_VERSION="{{shark_version}}"
export HADOOP_MAJOR_VERSION="{{hadoop_major_version}}"
export SWAP_MB="{{swap}}"
+export SPARK_WORKER_INSTANCES="{{spark_worker_instances}}"
+export SPARK_MASTER_OPTS="{{spark_master_opts}}"
diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py
index 31209a662bbe1..0a8d6ca5d7519 100755
--- a/ec2/spark_ec2.py
+++ b/ec2/spark_ec2.py
@@ -103,6 +103,12 @@ def parse_args():
help="When destroying a cluster, delete the security groups that were created")
parser.add_option("--use-existing-master", action="store_true", default=False,
help="Launch fresh slaves, but use an existing stopped master if possible")
+ parser.add_option("--worker-instances", type="int", default=1,
+ help="Number of instances per worker: variable SPARK_WORKER_INSTANCES (default: 1)")
+ parser.add_option("--master-opts", type="string", default="",
+ help="Extra options to give to master through SPARK_MASTER_OPTS variable (e.g -Dspark.worker.timeout=180)")
+
+
(opts, args) = parser.parse_args()
if len(args) != 2:
@@ -223,7 +229,7 @@ def launch_cluster(conn, opts, cluster_name):
sys.exit(1)
if opts.key_pair is None:
print >> stderr, "ERROR: Must provide a key pair name (-k) to use on instances."
- sys.exit(1)
+ sys.exit(1)
print "Setting up security groups..."
master_group = get_or_make_group(conn, cluster_name + "-master")
slave_group = get_or_make_group(conn, cluster_name + "-slaves")
@@ -551,7 +557,9 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules):
"modules": '\n'.join(modules),
"spark_version": spark_v,
"shark_version": shark_v,
- "hadoop_major_version": opts.hadoop_major_version
+ "hadoop_major_version": opts.hadoop_major_version,
+ "spark_worker_instances": "%d" % opts.worker_instances,
+ "spark_master_opts": opts.master_opts
}
# Create a temp directory in which we will place all the files to be
@@ -604,7 +612,7 @@ def ssh_command(opts):
return ['ssh'] + ssh_args(opts)
-# Run a command on a host through ssh, retrying up to two times
+# Run a command on a host through ssh, retrying up to five times
# and then throwing an exception if ssh continues to fail.
def ssh(host, opts, command):
tries = 0
@@ -613,7 +621,7 @@ def ssh(host, opts, command):
return subprocess.check_call(
ssh_command(opts) + ['-t', '-t', '%s@%s' % (opts.user, host), stringify_command(command)])
except subprocess.CalledProcessError as e:
- if (tries > 2):
+ if (tries > 5):
# If this was an ssh failure, provide the user with hints.
if e.returncode == 255:
raise UsageError("Failed to SSH to remote host {0}.\nPlease check that you have provided the correct --identity-file and --key-pair parameters and try again.".format(host))
@@ -640,7 +648,7 @@ def ssh_write(host, opts, command, input):
status = proc.wait()
if status == 0:
break
- elif (tries > 2):
+ elif (tries > 5):
raise RuntimeError("ssh_write failed with error %s" % proc.returncode)
else:
print >> stderr, "Error {0} while executing remote command, retrying after 30 seconds".format(status)
@@ -806,6 +814,7 @@ def main():
real_main()
except UsageError, e:
print >> stderr, "\nError:\n", e
+ sys.exit(1)
if __name__ == "__main__":
diff --git a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java
index 2a4278d3c30e5..3f7a879538016 100644
--- a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java
+++ b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java
@@ -75,7 +75,6 @@ public String toString() {
public static Tuple3 extractKey(String line) {
Matcher m = apacheLogRegex.matcher(line);
- List key = Collections.emptyList();
if (m.find()) {
String ip = m.group(1);
String user = m.group(3);
diff --git a/examples/src/main/java/org/apache/spark/examples/JavaTC.java b/examples/src/main/java/org/apache/spark/examples/JavaTC.java
index 1d776940f06c6..d66b9ba265fe8 100644
--- a/examples/src/main/java/org/apache/spark/examples/JavaTC.java
+++ b/examples/src/main/java/org/apache/spark/examples/JavaTC.java
@@ -85,7 +85,7 @@ public Tuple2 call(Tuple2 e) {
}
});
- long oldCount = 0;
+ long oldCount;
long nextCount = tc.count();
do {
oldCount = nextCount;
diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaALS.java
similarity index 98%
rename from examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java
rename to examples/src/main/java/org/apache/spark/examples/mllib/JavaALS.java
index c516199d61c72..4533c4c5f241a 100644
--- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java
+++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaALS.java
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.mllib.examples;
+package org.apache.spark.examples.mllib;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaKMeans.java
similarity index 98%
rename from examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java
rename to examples/src/main/java/org/apache/spark/examples/mllib/JavaKMeans.java
index 7461609ab9e8f..0cfb8e69ed28f 100644
--- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java
+++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaKMeans.java
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.mllib.examples;
+package org.apache.spark.examples.mllib;
import java.util.regex.Pattern;
diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaLR.java
similarity index 98%
rename from examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java
rename to examples/src/main/java/org/apache/spark/examples/mllib/JavaLR.java
index e3ab87cc722f3..f6e48b498727b 100644
--- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java
+++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaLR.java
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.mllib.examples;
+package org.apache.spark.examples.mllib;
import java.util.regex.Pattern;
diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java
index b5b438e9753a7..d62a72f53443c 100644
--- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java
+++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java
@@ -23,7 +23,6 @@
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;
diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java
similarity index 95%
rename from examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java
rename to examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java
index a94fa621dc328..7f558f3ee713a 100644
--- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaCustomReceiver.java
+++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaCustomReceiver.java
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.streaming.examples;
+package org.apache.spark.examples.streaming;
import com.google.common.collect.Lists;
@@ -26,6 +26,7 @@
import org.apache.spark.streaming.Duration;
import org.apache.spark.streaming.api.java.JavaDStream;
import org.apache.spark.streaming.api.java.JavaPairDStream;
+import org.apache.spark.streaming.api.java.JavaReceiverInputDStream;
import org.apache.spark.streaming.api.java.JavaStreamingContext;
import org.apache.spark.streaming.receiver.Receiver;
import scala.Tuple2;
@@ -47,7 +48,7 @@
* To run this on your local machine, you need to first run a Netcat server
* `$ nc -lk 9999`
* and then run the example
- * `$ ./run org.apache.spark.streaming.examples.JavaCustomReceiver local[2] localhost 9999`
+ * `$ ./run org.apache.spark.examples.streaming.JavaCustomReceiver local[2] localhost 9999`
*/
public class JavaCustomReceiver extends Receiver {
@@ -69,7 +70,7 @@ public static void main(String[] args) {
// Create a input stream with the custom receiver on target ip:port and count the
// words in input stream of \n delimited text (eg. generated by 'nc')
- JavaDStream lines = ssc.receiverStream(
+ JavaReceiverInputDStream lines = ssc.receiverStream(
new JavaCustomReceiver(args[1], Integer.parseInt(args[2])));
JavaDStream words = lines.flatMap(new FlatMapFunction() {
@Override
@@ -148,5 +149,3 @@ private void receive() {
}
}
}
-
-
diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaFlumeEventCount.java
similarity index 92%
rename from examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java
rename to examples/src/main/java/org/apache/spark/examples/streaming/JavaFlumeEventCount.java
index f061001dd264d..a5ece68cef870 100644
--- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaFlumeEventCount.java
+++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaFlumeEventCount.java
@@ -15,9 +15,10 @@
* limitations under the License.
*/
-package org.apache.spark.streaming.examples;
+package org.apache.spark.examples.streaming;
import org.apache.spark.api.java.function.Function;
+import org.apache.spark.examples.streaming.StreamingExamples;
import org.apache.spark.streaming.*;
import org.apache.spark.streaming.api.java.*;
import org.apache.spark.streaming.flume.FlumeUtils;
@@ -58,7 +59,7 @@ public static void main(String[] args) {
JavaStreamingContext ssc = new JavaStreamingContext(master, "FlumeEventCount", batchInterval,
System.getenv("SPARK_HOME"),
JavaStreamingContext.jarOfClass(JavaFlumeEventCount.class));
- JavaDStream flumeStream = FlumeUtils.createStream(ssc, "localhost", port);
+ JavaReceiverInputDStream flumeStream = FlumeUtils.createStream(ssc, "localhost", port);
flumeStream.count();
diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java
similarity index 91%
rename from examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java
rename to examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java
index d704be08d6945..da51eb189a649 100644
--- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java
+++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaKafkaWordCount.java
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.streaming.examples;
+package org.apache.spark.examples.streaming;
import java.util.Map;
import java.util.HashMap;
@@ -26,9 +26,11 @@
import org.apache.spark.api.java.function.Function;
import org.apache.spark.api.java.function.Function2;
import org.apache.spark.api.java.function.PairFunction;
+import org.apache.spark.examples.streaming.StreamingExamples;
import org.apache.spark.streaming.Duration;
import org.apache.spark.streaming.api.java.JavaDStream;
import org.apache.spark.streaming.api.java.JavaPairDStream;
+import org.apache.spark.streaming.api.java.JavaPairReceiverInputDStream;
import org.apache.spark.streaming.api.java.JavaStreamingContext;
import org.apache.spark.streaming.kafka.KafkaUtils;
import scala.Tuple2;
@@ -43,7 +45,7 @@
* is the number of threads the kafka consumer should use
*
* Example:
- * `./bin/run-example org.apache.spark.streaming.examples.JavaKafkaWordCount local[2] zoo01,zoo02,
+ * `./bin/run-example org.apache.spark.examples.streaming.JavaKafkaWordCount local[2] zoo01,zoo02,
* zoo03 my-consumer-group topic1,topic2 1`
*/
@@ -73,7 +75,8 @@ public static void main(String[] args) {
topicMap.put(topic, numThreads);
}
- JavaPairDStream messages = KafkaUtils.createStream(jssc, args[1], args[2], topicMap);
+ JavaPairReceiverInputDStream messages =
+ KafkaUtils.createStream(jssc, args[1], args[2], topicMap);
JavaDStream lines = messages.map(new Function, String>() {
@Override
diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java
similarity index 88%
rename from examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java
rename to examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java
index 0cc9d0ae1a08e..ac84991d87b8b 100644
--- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java
+++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaNetworkWordCount.java
@@ -15,13 +15,15 @@
* limitations under the License.
*/
-package org.apache.spark.streaming.examples;
+package org.apache.spark.examples.streaming;
import com.google.common.collect.Lists;
+import org.apache.spark.streaming.api.java.JavaReceiverInputDStream;
import scala.Tuple2;
import org.apache.spark.api.java.function.FlatMapFunction;
import org.apache.spark.api.java.function.Function2;
import org.apache.spark.api.java.function.PairFunction;
+import org.apache.spark.examples.streaming.StreamingExamples;
import org.apache.spark.streaming.Duration;
import org.apache.spark.streaming.api.java.JavaDStream;
import org.apache.spark.streaming.api.java.JavaPairDStream;
@@ -38,7 +40,7 @@
* To run this on your local machine, you need to first run a Netcat server
* `$ nc -lk 9999`
* and then run the example
- * `$ ./run org.apache.spark.streaming.examples.JavaNetworkWordCount local[2] localhost 9999`
+ * `$ ./run org.apache.spark.examples.streaming.JavaNetworkWordCount local[2] localhost 9999`
*/
public final class JavaNetworkWordCount {
private static final Pattern SPACE = Pattern.compile(" ");
@@ -57,9 +59,9 @@ public static void main(String[] args) {
new Duration(1000), System.getenv("SPARK_HOME"),
JavaStreamingContext.jarOfClass(JavaNetworkWordCount.class));
- // Create a NetworkInputDStream on target ip:port and count the
+ // Create a JavaReceiverInputDStream on target ip:port and count the
// words in input stream of \n delimited text (eg. generated by 'nc')
- JavaDStream lines = ssc.socketTextStream(args[1], Integer.parseInt(args[2]));
+ JavaReceiverInputDStream lines = ssc.socketTextStream(args[1], Integer.parseInt(args[2]));
JavaDStream words = lines.flatMap(new FlatMapFunction() {
@Override
public Iterable call(String x) {
diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaQueueStream.java
similarity index 96%
rename from examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java
rename to examples/src/main/java/org/apache/spark/examples/streaming/JavaQueueStream.java
index 88ad341641e0a..819311968fac5 100644
--- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java
+++ b/examples/src/main/java/org/apache/spark/examples/streaming/JavaQueueStream.java
@@ -15,13 +15,14 @@
* limitations under the License.
*/
-package org.apache.spark.streaming.examples;
+package org.apache.spark.examples.streaming;
import com.google.common.collect.Lists;
import scala.Tuple2;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.function.Function2;
import org.apache.spark.api.java.function.PairFunction;
+import org.apache.spark.examples.streaming.StreamingExamples;
import org.apache.spark.streaming.Duration;
import org.apache.spark.streaming.api.java.JavaDStream;
import org.apache.spark.streaming.api.java.JavaPairDStream;
diff --git a/python/examples/als.py b/examples/src/main/python/als.py
similarity index 100%
rename from python/examples/als.py
rename to examples/src/main/python/als.py
diff --git a/python/examples/kmeans.py b/examples/src/main/python/kmeans.py
similarity index 98%
rename from python/examples/kmeans.py
rename to examples/src/main/python/kmeans.py
index d8387b0b183e6..e3596488faf9e 100755
--- a/python/examples/kmeans.py
+++ b/examples/src/main/python/kmeans.py
@@ -18,7 +18,7 @@
"""
The K-means algorithm written from scratch against PySpark. In practice,
one may prefer to use the KMeans algorithm in MLlib, as shown in
-python/examples/mllib/kmeans.py.
+examples/src/main/python/mllib/kmeans.py.
This example requires NumPy (http://www.numpy.org/).
"""
diff --git a/python/examples/logistic_regression.py b/examples/src/main/python/logistic_regression.py
similarity index 97%
rename from python/examples/logistic_regression.py
rename to examples/src/main/python/logistic_regression.py
index 28d52e6a40b45..fe5373cf799b1 100755
--- a/python/examples/logistic_regression.py
+++ b/examples/src/main/python/logistic_regression.py
@@ -20,7 +20,7 @@
to act on batches of input data using efficient matrix operations.
In practice, one may prefer to use the LogisticRegression algorithm in
-MLlib, as shown in python/examples/mllib/logistic_regression.py.
+MLlib, as shown in examples/src/main/python/mllib/logistic_regression.py.
"""
from collections import namedtuple
diff --git a/python/examples/mllib/kmeans.py b/examples/src/main/python/mllib/kmeans.py
similarity index 100%
rename from python/examples/mllib/kmeans.py
rename to examples/src/main/python/mllib/kmeans.py
diff --git a/python/examples/mllib/logistic_regression.py b/examples/src/main/python/mllib/logistic_regression.py
similarity index 100%
rename from python/examples/mllib/logistic_regression.py
rename to examples/src/main/python/mllib/logistic_regression.py
diff --git a/python/examples/pagerank.py b/examples/src/main/python/pagerank.py
similarity index 100%
rename from python/examples/pagerank.py
rename to examples/src/main/python/pagerank.py
diff --git a/python/examples/pi.py b/examples/src/main/python/pi.py
similarity index 100%
rename from python/examples/pi.py
rename to examples/src/main/python/pi.py
diff --git a/python/examples/sort.py b/examples/src/main/python/sort.py
similarity index 100%
rename from python/examples/sort.py
rename to examples/src/main/python/sort.py
diff --git a/python/examples/transitive_closure.py b/examples/src/main/python/transitive_closure.py
similarity index 100%
rename from python/examples/transitive_closure.py
rename to examples/src/main/python/transitive_closure.py
diff --git a/python/examples/wordcount.py b/examples/src/main/python/wordcount.py
similarity index 100%
rename from python/examples/wordcount.py
rename to examples/src/main/python/wordcount.py
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala
index ec9de022c1d47..4001908c98015 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala
@@ -22,7 +22,7 @@ import scopt.OptionParser
import org.apache.spark.{SparkConf, SparkContext}
import org.apache.spark.mllib.classification.{LogisticRegressionWithSGD, SVMWithSGD}
-import org.apache.spark.mllib.evaluation.binary.BinaryClassificationMetrics
+import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics
import org.apache.spark.mllib.util.MLUtils
import org.apache.spark.mllib.optimization.{SquaredL2Updater, L1Updater}
@@ -96,7 +96,7 @@ object BinaryClassification {
Logger.getRootLogger.setLevel(Level.WARN)
- val examples = MLUtils.loadLibSVMData(sc, params.input).cache()
+ val examples = MLUtils.loadLibSVMFile(sc, params.input).cache()
val splits = examples.randomSplit(Array(0.8, 0.2))
val training = splits(0).cache()
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala
index 1723ca6931021..658d370f8656e 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala
@@ -22,7 +22,7 @@ import scopt.OptionParser
import org.apache.spark.{SparkConf, SparkContext}
import org.apache.spark.mllib.regression.LinearRegressionWithSGD
-import org.apache.spark.mllib.util.{MulticlassLabelParser, MLUtils}
+import org.apache.spark.mllib.util.MLUtils
import org.apache.spark.mllib.optimization.{SimpleUpdater, SquaredL2Updater, L1Updater}
/**
@@ -82,7 +82,7 @@ object LinearRegression extends App {
Logger.getRootLogger.setLevel(Level.WARN)
- val examples = MLUtils.loadLibSVMData(sc, params.input, MulticlassLabelParser).cache()
+ val examples = MLUtils.loadLibSVMFile(sc, params.input, multiclass = true).cache()
val splits = examples.randomSplit(Array(0.8, 0.2))
val training = splits(0).cache()
diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala
index 25b6768b8d72b..537e68a0991aa 100644
--- a/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SparseNaiveBayes.scala
@@ -75,8 +75,8 @@ object SparseNaiveBayes {
val minPartitions =
if (params.minPartitions > 0) params.minPartitions else sc.defaultMinPartitions
- val examples = MLUtils.loadLibSVMData(sc, params.input, MulticlassLabelParser,
- params.numFeatures, minPartitions)
+ val examples =
+ MLUtils.loadLibSVMFile(sc, params.input, multiclass = true, params.numFeatures, minPartitions)
// Cache examples because it will be used in both training and evaluation.
examples.cache()
diff --git a/examples/src/main/scala/org/apache/spark/sql/examples/RDDRelation.scala b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala
similarity index 98%
rename from examples/src/main/scala/org/apache/spark/sql/examples/RDDRelation.scala
rename to examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala
index 8210ad977f066..ff9254b044c24 100644
--- a/examples/src/main/scala/org/apache/spark/sql/examples/RDDRelation.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/sql/RDDRelation.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.examples
+package org.apache.spark.examples.sql
import org.apache.spark.SparkContext
import org.apache.spark.sql.SQLContext
diff --git a/examples/src/main/scala/org/apache/spark/sql/examples/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala
similarity index 98%
rename from examples/src/main/scala/org/apache/spark/sql/examples/HiveFromSpark.scala
rename to examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala
index 62329bde84481..66ce93a26ef42 100644
--- a/examples/src/main/scala/org/apache/spark/sql/examples/HiveFromSpark.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.hive.examples
+package org.apache.spark.examples.sql.hive
import org.apache.spark.SparkContext
import org.apache.spark.sql._
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala
similarity index 95%
rename from examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala
rename to examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala
index c845dd8904c90..84cf43df0f96c 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.streaming.examples
+package org.apache.spark.examples.streaming
import scala.collection.mutable.LinkedList
import scala.reflect.ClassTag
@@ -78,7 +78,7 @@ class FeederActor extends Actor {
* goes and subscribe to a typical publisher/feeder actor and receives
* data.
*
- * @see [[org.apache.spark.streaming.examples.FeederActor]]
+ * @see [[org.apache.spark.examples.streaming.FeederActor]]
*/
class SampleActorReceiver[T: ClassTag](urlOfPublisher: String)
extends Actor with ActorHelper {
@@ -131,9 +131,9 @@ object FeederActor {
* and describe the AkkaSystem that Spark Sample feeder is running on.
*
* To run this example locally, you may run Feeder Actor as
- * `$ ./bin/run-example org.apache.spark.streaming.examples.FeederActor 127.0.1.1 9999`
+ * `$ ./bin/run-example org.apache.spark.examples.streaming.FeederActor 127.0.1.1 9999`
* and then run the example
- * `./bin/run-example org.apache.spark.streaming.examples.ActorWordCount local[2] 127.0.1.1 9999`
+ * `./bin/run-example org.apache.spark.examples.streaming.ActorWordCount local[2] 127.0.1.1 9999`
*/
object ActorWordCount {
def main(args: Array[String]) {
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/CustomReceiver.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala
similarity index 97%
rename from examples/src/main/scala/org/apache/spark/streaming/examples/CustomReceiver.scala
rename to examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala
index eebffd824983f..e317e2d36ae43 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/CustomReceiver.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.streaming.examples
+package org.apache.spark.examples.streaming
import java.io.{InputStreamReader, BufferedReader, InputStream}
import java.net.Socket
@@ -37,7 +37,7 @@ import org.apache.spark.streaming.receiver.Receiver
* To run this on your local machine, you need to first run a Netcat server
* `$ nc -lk 9999`
* and then run the example
- * `$ ./run org.apache.spark.streaming.examples.CustomReceiver local[2] localhost 9999`
+ * `$ ./run org.apache.spark.examples.streaming.CustomReceiver local[2] localhost 9999`
*/
object CustomReceiver {
def main(args: Array[String]) {
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala
similarity index 98%
rename from examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala
rename to examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala
index 26b6024534124..5b2a1035fc779 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.streaming.examples
+package org.apache.spark.examples.streaming
import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming._
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala
similarity index 95%
rename from examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala
rename to examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala
index 7f86fc792aacf..b440956ba3137 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/HdfsWordCount.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.streaming.examples
+package org.apache.spark.examples.streaming
import org.apache.spark.streaming.{Seconds, StreamingContext}
import org.apache.spark.streaming.StreamingContext._
@@ -27,7 +27,7 @@ import org.apache.spark.streaming.StreamingContext._
* is the directory that Spark Streaming will use to find and read new text files.
*
* To run this on your local machine on directory `localdir`, run this example
- * `$ ./bin/run-example org.apache.spark.streaming.examples.HdfsWordCount local[2] localdir`
+ * `$ ./bin/run-example org.apache.spark.examples.streaming.HdfsWordCount local[2] localdir`
* Then create a text file in `localdir` and the words in the file will get counted.
*/
object HdfsWordCount {
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala
similarity index 93%
rename from examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala
rename to examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala
index 2aa4f1474a59e..c3aae5af05b1c 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.streaming.examples
+package org.apache.spark.examples.streaming
import java.util.Properties
@@ -24,7 +24,6 @@ import kafka.producer._
import org.apache.spark.streaming._
import org.apache.spark.streaming.StreamingContext._
import org.apache.spark.streaming.kafka._
-import org.apache.spark.streaming.util.RawTextHelper._
// scalastyle:off
/**
@@ -37,7 +36,7 @@ import org.apache.spark.streaming.util.RawTextHelper._
* is the number of threads the kafka consumer should use
*
* Example:
- * `./bin/run-example org.apache.spark.streaming.examples.KafkaWordCount local[2] zoo01,zoo02,zoo03 my-consumer-group topic1,topic2 1`
+ * `./bin/run-example org.apache.spark.examples.streaming.KafkaWordCount local[2] zoo01,zoo02,zoo03 my-consumer-group topic1,topic2 1`
*/
// scalastyle:on
object KafkaWordCount {
@@ -59,7 +58,7 @@ object KafkaWordCount {
val lines = KafkaUtils.createStream(ssc, zkQuorum, group, topicpMap).map(_._2)
val words = lines.flatMap(_.split(" "))
val wordCounts = words.map(x => (x, 1L))
- .reduceByKeyAndWindow(add _, subtract _, Minutes(10), Seconds(2), 2)
+ .reduceByKeyAndWindow(_ + _, _ - _, Minutes(10), Seconds(2), 2)
wordCounts.print()
ssc.start()
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala
similarity index 95%
rename from examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala
rename to examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala
index 62aef0fb47107..47bf1e5a06439 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.streaming.examples
+package org.apache.spark.examples.streaming
import org.eclipse.paho.client.mqttv3.{MqttClient, MqttClientPersistence, MqttException, MqttMessage, MqttTopic}
import org.eclipse.paho.client.mqttv3.persist.MqttDefaultFilePersistence
@@ -79,9 +79,9 @@ object MQTTPublisher {
* and describe where Mqtt publisher is running.
*
* To run this example locally, you may run publisher as
- * `$ ./bin/run-example org.apache.spark.streaming.examples.MQTTPublisher tcp://localhost:1883 foo`
+ * `$ ./bin/run-example org.apache.spark.examples.streaming.MQTTPublisher tcp://localhost:1883 foo`
* and run the example as
- * `$ ./bin/run-example org.apache.spark.streaming.examples.MQTTWordCount local[2] tcp://localhost:1883 foo`
+ * `$ ./bin/run-example org.apache.spark.examples.streaming.MQTTWordCount local[2] tcp://localhost:1883 foo`
*/
// scalastyle:on
object MQTTWordCount {
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala
similarity index 95%
rename from examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala
rename to examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala
index 272ab11212451..acfe9a4da3596 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/NetworkWordCount.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.streaming.examples
+package org.apache.spark.examples.streaming
import org.apache.spark.streaming.{Seconds, StreamingContext}
import org.apache.spark.streaming.StreamingContext._
@@ -32,7 +32,7 @@ import org.apache.spark.storage.StorageLevel
* To run this on your local machine, you need to first run a Netcat server
* `$ nc -lk 9999`
* and then run the example
- * `$ ./bin/run-example org.apache.spark.streaming.examples.NetworkWordCount local[2] localhost 9999`
+ * `$ ./bin/run-example org.apache.spark.examples.streaming.NetworkWordCount local[2] localhost 9999`
*/
// scalastyle:on
object NetworkWordCount {
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala
similarity index 97%
rename from examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala
rename to examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala
index ff2a205ec1c15..f92f72f2de876 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/QueueStream.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.streaming.examples
+package org.apache.spark.examples.streaming
import scala.collection.mutable.SynchronizedQueue
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/RawNetworkGrep.scala
similarity index 92%
rename from examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala
rename to examples/src/main/scala/org/apache/spark/examples/streaming/RawNetworkGrep.scala
index d915c0c39b334..1b0319a046433 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/RawNetworkGrep.scala
@@ -15,11 +15,10 @@
* limitations under the License.
*/
-package org.apache.spark.streaming.examples
+package org.apache.spark.examples.streaming
import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming._
-import org.apache.spark.streaming.util.RawTextHelper
import org.apache.spark.util.IntParam
/**
@@ -52,9 +51,6 @@ object RawNetworkGrep {
val ssc = new StreamingContext(master, "RawNetworkGrep", Milliseconds(batchMillis),
System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq)
- // Warm up the JVMs on master and slave for JIT compilation to kick in
- RawTextHelper.warmUp(ssc.sparkContext)
-
val rawStreams = (1 to numStreams).map(_ =>
ssc.rawSocketStream[String](host, port, StorageLevel.MEMORY_ONLY_SER_2)).toArray
val union = ssc.union(rawStreams)
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala
similarity index 96%
rename from examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala
rename to examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala
index 4aacbb1991418..b0bc31cc66ab5 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/RecoverableNetworkWordCount.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.streaming.examples
+package org.apache.spark.examples.streaming
import org.apache.spark.streaming.{Time, Seconds, StreamingContext}
import org.apache.spark.streaming.StreamingContext._
@@ -44,7 +44,7 @@ import java.nio.charset.Charset
*
* and run the example as
*
- * `$ ./run-example org.apache.spark.streaming.examples.RecoverableNetworkWordCount \
+ * `$ ./run-example org.apache.spark.examples.streaming.RecoverableNetworkWordCount \
* local[2] localhost 9999 ~/checkpoint/ ~/out`
*
* If the directory ~/checkpoint/ does not exist (e.g. running for the first time), it will create
@@ -56,7 +56,7 @@ import java.nio.charset.Charset
*
* `$ ./spark-class org.apache.spark.deploy.Client -s launch \
* \
- * org.apache.spark.streaming.examples.RecoverableNetworkWordCount \
+ * org.apache.spark.examples.streaming.RecoverableNetworkWordCount \
* localhost 9999 ~/checkpoint ~/out`
*
* would typically be
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala
similarity index 96%
rename from examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala
rename to examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala
index ef94c9298dd93..8001d56c98d86 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.streaming.examples
+package org.apache.spark.examples.streaming
import org.apache.spark.streaming._
import org.apache.spark.streaming.StreamingContext._
@@ -31,7 +31,7 @@ import org.apache.spark.streaming.StreamingContext._
* To run this on your local machine, you need to first run a Netcat server
* `$ nc -lk 9999`
* and then run the example
- * `$ ./bin/run-example org.apache.spark.streaming.examples.StatefulNetworkWordCount local[2] localhost 9999`
+ * `$ ./bin/run-example org.apache.spark.examples.streaming.StatefulNetworkWordCount local[2] localhost 9999`
*/
// scalastyle:on
object StatefulNetworkWordCount {
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/StreamingExamples.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/StreamingExamples.scala
similarity index 97%
rename from examples/src/main/scala/org/apache/spark/streaming/examples/StreamingExamples.scala
rename to examples/src/main/scala/org/apache/spark/examples/streaming/StreamingExamples.scala
index 99f1502046f53..8396e65d0d588 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/StreamingExamples.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/StreamingExamples.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.streaming.examples
+package org.apache.spark.examples.streaming
import org.apache.spark.Logging
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala
similarity index 99%
rename from examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala
rename to examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala
index c38905e8f3663..b12617d881787 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.streaming.examples
+package org.apache.spark.examples.streaming
import com.twitter.algebird._
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala
similarity index 98%
rename from examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala
rename to examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala
index c067046f9020e..22f232c72545c 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.streaming.examples
+package org.apache.spark.examples.streaming
import com.twitter.algebird.HyperLogLogMonoid
import com.twitter.algebird.HyperLogLog._
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala
similarity index 98%
rename from examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala
rename to examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala
index 2597c8178862a..5b58e94600a16 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.streaming.examples
+package org.apache.spark.examples.streaming
import org.apache.spark.streaming.{Seconds, StreamingContext}
import StreamingContext._
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala
similarity index 95%
rename from examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala
rename to examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala
index 109ff855b5228..de46e5f5b10b6 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.streaming.examples
+package org.apache.spark.examples.streaming
import akka.actor.ActorSystem
import akka.actor.actorRef2Scala
@@ -68,9 +68,9 @@ object SimpleZeroMQPublisher {
* and describe where zeroMq publisher is running.
*
* To run this example locally, you may run publisher as
- * `$ ./bin/run-example org.apache.spark.streaming.examples.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar`
+ * `$ ./bin/run-example org.apache.spark.examples.streaming.SimpleZeroMQPublisher tcp://127.0.1.1:1234 foo.bar`
* and run the example as
- * `$ ./bin/run-example org.apache.spark.streaming.examples.ZeroMQWordCount local[2] tcp://127.0.1.1:1234 foo`
+ * `$ ./bin/run-example org.apache.spark.examples.streaming.ZeroMQWordCount local[2] tcp://127.0.1.1:1234 foo`
*/
// scalastyle:on
object ZeroMQWordCount {
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala
similarity index 95%
rename from examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala
rename to examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala
index 251f65fe4df9c..97e0cb92078dc 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewGenerator.scala
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.streaming.examples.clickstream
+package org.apache.spark.examples.streaming.clickstream
import java.net.ServerSocket
import java.io.PrintWriter
@@ -40,8 +40,8 @@ object PageView extends Serializable {
/** Generates streaming events to simulate page views on a website.
*
* This should be used in tandem with PageViewStream.scala. Example:
- * $ ./bin/run-example org.apache.spark.streaming.examples.clickstream.PageViewGenerator 44444 10
- * $ ./bin/run-example org.apache.spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444
+ * $ ./bin/run-example org.apache.spark.examples.streaming.clickstream.PageViewGenerator 44444 10
+ * $ ./bin/run-example org.apache.spark.examples.streaming.clickstream.PageViewStream errorRatePerZipCode localhost 44444
*
* When running this, you may want to set the root logging level to ERROR in
* conf/log4j.properties to reduce the verbosity of the output.
diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala
similarity index 94%
rename from examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala
rename to examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala
index 673013f7cf948..d30ceffbe29cb 100644
--- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala
+++ b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala
@@ -15,19 +15,19 @@
* limitations under the License.
*/
-package org.apache.spark.streaming.examples.clickstream
+package org.apache.spark.examples.streaming.clickstream
import org.apache.spark.SparkContext._
import org.apache.spark.streaming.{Seconds, StreamingContext}
import org.apache.spark.streaming.StreamingContext._
-import org.apache.spark.streaming.examples.StreamingExamples
+import org.apache.spark.examples.streaming.StreamingExamples
// scalastyle:off
/** Analyses a streaming dataset of web page views. This class demonstrates several types of
* operators available in Spark streaming.
*
* This should be used in tandem with PageViewStream.scala. Example:
- * $ ./bin/run-example org.apache.spark.streaming.examples.clickstream.PageViewGenerator 44444 10
- * $ ./bin/run-example org.apache.spark.streaming.examples.clickstream.PageViewStream errorRatePerZipCode localhost 44444
+ * $ ./bin/run-example org.apache.spark.examples.streaming.clickstream.PageViewGenerator 44444 10
+ * $ ./bin/run-example org.apache.spark.examples.streaming.clickstream.PageViewStream errorRatePerZipCode localhost 44444
*/
// scalastyle:on
object PageViewStream {
diff --git a/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java b/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java
index 7eb8b45fc3cf0..feabca6733484 100644
--- a/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java
+++ b/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java
@@ -23,7 +23,7 @@
import scala.Tuple2;
-import com.google.common.collections.Iterables;
+import com.google.common.collect.Iterables;
import com.google.common.base.Optional;
import com.google.common.io.Files;
import org.apache.hadoop.io.IntWritable;
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 6d04bf790e3a5..fa78ca99b8891 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala
@@ -51,18 +51,12 @@ class EdgeRDD[@specialized ED: ClassTag](
override def collect(): Array[Edge[ED]] = this.map(_.copy()).collect()
- override def persist(newLevel: StorageLevel): EdgeRDD[ED] = {
+ override def persist(newLevel: StorageLevel): this.type = {
partitionsRDD.persist(newLevel)
this
}
- /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */
- override def persist(): EdgeRDD[ED] = persist(StorageLevel.MEMORY_ONLY)
-
- /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */
- override def cache(): EdgeRDD[ED] = persist()
-
- override def unpersist(blocking: Boolean = true): EdgeRDD[ED] = {
+ override def unpersist(blocking: Boolean = true): this.type = {
partitionsRDD.unpersist(blocking)
this
}
diff --git a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala
index d6788d4d4b9fd..f0fc605c88575 100644
--- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala
+++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala
@@ -71,18 +71,12 @@ class VertexRDD[@specialized VD: ClassTag](
override protected def getPreferredLocations(s: Partition): Seq[String] =
partitionsRDD.preferredLocations(s)
- override def persist(newLevel: StorageLevel): VertexRDD[VD] = {
+ override def persist(newLevel: StorageLevel): this.type = {
partitionsRDD.persist(newLevel)
this
}
- /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */
- override def persist(): VertexRDD[VD] = persist(StorageLevel.MEMORY_ONLY)
-
- /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */
- override def cache(): VertexRDD[VD] = persist()
-
- override def unpersist(blocking: Boolean = true): VertexRDD[VD] = {
+ override def unpersist(blocking: Boolean = true): this.type = {
partitionsRDD.unpersist(blocking)
this
}
diff --git a/make-distribution.sh b/make-distribution.sh
index c05dcd89d90a7..759e555b4b69a 100755
--- a/make-distribution.sh
+++ b/make-distribution.sh
@@ -39,14 +39,29 @@
# 5) ./bin/spark-shell --master spark://my-master-ip:7077
#
+set -o pipefail
# Figure out where the Spark framework is installed
FWDIR="$(cd `dirname $0`; pwd)"
DISTDIR="$FWDIR/dist"
-VERSION=$(mvn help:evaluate -Dexpression=project.version | grep -v "INFO" | tail -n 1)
-if [ $? == -1 ] ;then
+if [ -z "$JAVA_HOME" ]; then
+ echo "Error: JAVA_HOME is not set, cannot proceed."
+ exit -1
+fi
+
+JAVA_CMD="$JAVA_HOME"/bin/java
+JAVA_VERSION=$("$JAVA_CMD" -version 2>&1)
+if ! [[ "$JAVA_VERSION" =~ "1.6" ]]; then
+ echo "***NOTE***: JAVA_HOME is not set to a JDK 6 installation. The resulting"
+ echo " distribution will not support Java 6. See SPARK-1703."
+ echo "Output from 'java -version' was:"
+ echo "$JAVA_VERSION"
+fi
+
+VERSION=$(mvn help:evaluate -Dexpression=project.version 2>/dev/null | grep -v "INFO" | tail -n 1)
+if [ $? != 0 ]; then
echo -e "You need Maven installed to build Spark."
- echo -e "Download Maven from https://maven.apache.org."
+ echo -e "Download Maven from https://maven.apache.org/"
exit -1;
fi
@@ -116,27 +131,34 @@ cd $FWDIR
export MAVEN_OPTS="-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m"
-if [ "$SPARK_HIVE" == "true" ]; then
- MAYBE_HIVE="-Phive"
-else
- MAYBE_HIVE=""
-fi
-
-if [ "$SPARK_YARN" == "true" ]; then
- if [[ "$SPARK_HADOOP_VERSION" =~ "0.23." ]]; then
- mvn clean package -DskipTests -Pyarn-alpha -Dhadoop.version=$SPARK_HADOOP_VERSION \
- -Dyarn.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE -Phadoop-0.23
- else
- mvn clean package -DskipTests -Pyarn -Dhadoop.version=$SPARK_HADOOP_VERSION \
- -Dyarn.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE
- fi
-else
- if [[ "$SPARK_HADOOP_VERSION" =~ "0.23." ]]; then
- mvn clean package -Phadoop-0.23 -DskipTests -Dhadoop.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE
- else
- mvn clean package -DskipTests -Dhadoop.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE
+BUILD_COMMAND="mvn clean package"
+
+# Use special profiles for hadoop versions 0.23.x, 2.2.x, 2.3.x, 2.4.x
+if [[ "$SPARK_HADOOP_VERSION" =~ ^0\.23\. ]]; then BUILD_COMMAND="$BUILD_COMMAND -Phadoop-0.23"; fi
+if [[ "$SPARK_HADOOP_VERSION" =~ ^2\.2\. ]]; then BUILD_COMMAND="$BUILD_COMMAND -Phadoop-2.2"; fi
+if [[ "$SPARK_HADOOP_VERSION" =~ ^2\.3\. ]]; then BUILD_COMMAND="$BUILD_COMMAND -Phadoop-2.3"; fi
+if [[ "$SPARK_HADOOP_VERSION" =~ ^2\.4\. ]]; then BUILD_COMMAND="$BUILD_COMMAND -Phadoop-2.4"; fi
+if [[ "$SPARK_HIVE" == "true" ]]; then BUILD_COMMAND="$BUILD_COMMAND -Phive"; fi
+if [[ "$SPARK_YARN" == "true" ]]; then
+ # For hadoop versions 0.23.x to 2.1.x, use the yarn-alpha profile
+ if [[ "$SPARK_HADOOP_VERSION" =~ ^0\.2[3-9]\. ]] ||
+ [[ "$SPARK_HADOOP_VERSION" =~ ^0\.[3-9][0-9]\. ]] ||
+ [[ "$SPARK_HADOOP_VERSION" =~ ^1\.[0-9]\. ]] ||
+ [[ "$SPARK_HADOOP_VERSION" =~ ^2\.[0-1]\. ]]; then
+ BUILD_COMMAND="$BUILD_COMMAND -Pyarn-alpha"
+ # For hadoop versions 2.2+, use the yarn profile
+ elif [[ "$SPARK_HADOOP_VERSION" =~ ^2.[2-9]. ]]; then
+ BUILD_COMMAND="$BUILD_COMMAND -Pyarn"
fi
+ BUILD_COMMAND="$BUILD_COMMAND -Dyarn.version=$SPARK_HADOOP_VERSION"
fi
+BUILD_COMMAND="$BUILD_COMMAND -Dhadoop.version=$SPARK_HADOOP_VERSION"
+BUILD_COMMAND="$BUILD_COMMAND -DskipTests"
+
+# Actually build the jar
+echo -e "\nBuilding with..."
+echo -e "\$ $BUILD_COMMAND\n"
+${BUILD_COMMAND}
# Make directories
rm -rf "$DISTDIR"
@@ -147,6 +169,10 @@ echo "Spark $VERSION built for Hadoop $SPARK_HADOOP_VERSION" > "$DISTDIR/RELEASE
cp $FWDIR/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/lib/"
cp $FWDIR/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/"
+if [ "$SPARK_HIVE" == "true" ]; then
+ cp $FWDIR/lib_managed/jars/datanucleus*.jar "$DISTDIR/lib/"
+fi
+
# Copy other things
mkdir "$DISTDIR"/conf
cp "$FWDIR"/conf/*.template "$DISTDIR"/conf
@@ -155,7 +181,6 @@ cp -r "$FWDIR/bin" "$DISTDIR"
cp -r "$FWDIR/python" "$DISTDIR"
cp -r "$FWDIR/sbin" "$DISTDIR"
-
# Download and copy in tachyon, if requested
if [ "$SPARK_TACHYON" == "true" ]; then
TACHYON_VERSION="0.4.1"
@@ -164,7 +189,7 @@ if [ "$SPARK_TACHYON" == "true" ]; then
TMPD=`mktemp -d 2>/dev/null || mktemp -d -t 'disttmp'`
pushd $TMPD > /dev/null
- echo "Fetchting tachyon tgz"
+ echo "Fetching tachyon tgz"
wget "$TACHYON_URL"
tar xf "tachyon-${TACHYON_VERSION}-bin.tar.gz"
diff --git a/mllib/data/sample_naive_bayes_data.txt b/mllib/data/sample_naive_bayes_data.txt
index f874adbaf4665..981da382d6ac8 100644
--- a/mllib/data/sample_naive_bayes_data.txt
+++ b/mllib/data/sample_naive_bayes_data.txt
@@ -1,6 +1,6 @@
-0, 1 0 0
-0, 2 0 0
-1, 0 1 0
-1, 0 2 0
-2, 0 0 1
-2, 0 0 2
+0,1 0 0
+0,2 0 0
+1,0 1 0
+1,0 2 0
+2,0 0 1
+2,0 0 2
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 780e8bae42b84..90aa8ac998ba9 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,6 +17,7 @@
package org.apache.spark.mllib.classification
+import org.apache.spark.annotation.Experimental
import org.apache.spark.mllib.linalg.Vector
import org.apache.spark.mllib.optimization._
import org.apache.spark.mllib.regression._
@@ -29,7 +30,7 @@ import org.apache.spark.rdd.RDD
* @param weights Weights computed for every feature.
* @param intercept Intercept computed for this model.
*/
-class LogisticRegressionModel(
+class LogisticRegressionModel private[mllib] (
override val weights: Vector,
override val intercept: Double)
extends GeneralizedLinearModel(weights, intercept) with ClassificationModel with Serializable {
@@ -37,18 +38,22 @@ class LogisticRegressionModel(
private var threshold: Option[Double] = Some(0.5)
/**
+ * :: Experimental ::
* 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.
*/
+ @Experimental
def setThreshold(threshold: Double): this.type = {
this.threshold = Some(threshold)
this
}
/**
+ * :: Experimental ::
* Clears the threshold so that `predict` will output raw prediction scores.
*/
+ @Experimental
def clearThreshold(): this.type = {
threshold = None
this
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 f6f62ce2de04e..b6e0c4a80e27b 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala
@@ -19,7 +19,6 @@ package org.apache.spark.mllib.classification
import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, argmax => brzArgmax, sum => brzSum}
-import org.apache.spark.annotation.Experimental
import org.apache.spark.Logging
import org.apache.spark.SparkContext._
import org.apache.spark.mllib.linalg.Vector
@@ -27,7 +26,6 @@ import org.apache.spark.mllib.regression.LabeledPoint
import org.apache.spark.rdd.RDD
/**
- * :: Experimental ::
* Model for Naive Bayes Classifiers.
*
* @param labels list of labels
@@ -35,8 +33,7 @@ import org.apache.spark.rdd.RDD
* @param theta log of class conditional probabilities, whose dimension is C-by-D,
* where D is number of features
*/
-@Experimental
-class NaiveBayesModel(
+class NaiveBayesModel private[mllib] (
val labels: Array[Double],
val pi: Array[Double],
val theta: Array[Array[Double]]) extends ClassificationModel with Serializable {
@@ -124,6 +121,9 @@ class NaiveBayes private (private var lambda: Double) extends Serializable with
}
}
+/**
+ * Top-level methods for calling naive Bayes.
+ */
object NaiveBayes {
/**
* Trains a Naive Bayes model given an RDD of `(label, features)` pairs.
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 81b126717e9a4..e05213536e64a 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
@@ -17,6 +17,7 @@
package org.apache.spark.mllib.classification
+import org.apache.spark.annotation.Experimental
import org.apache.spark.mllib.linalg.Vector
import org.apache.spark.mllib.optimization._
import org.apache.spark.mllib.regression._
@@ -29,7 +30,7 @@ import org.apache.spark.rdd.RDD
* @param weights Weights computed for every feature.
* @param intercept Intercept computed for this model.
*/
-class SVMModel(
+class SVMModel private[mllib] (
override val weights: Vector,
override val intercept: Double)
extends GeneralizedLinearModel(weights, intercept) with ClassificationModel with Serializable {
@@ -37,18 +38,22 @@ class SVMModel(
private var threshold: Option[Double] = Some(0.0)
/**
+ * :: Experimental ::
* 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.
*/
+ @Experimental
def setThreshold(threshold: Double): this.type = {
this.threshold = Some(threshold)
this
}
/**
+ * :: Experimental ::
* Clears the threshold so that `predict` will output raw prediction scores.
*/
+ @Experimental
def clearThreshold(): this.type = {
threshold = None
this
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 a64c5d44be406..de22fbb6ffc10 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala
@@ -21,6 +21,7 @@ import scala.collection.mutable.ArrayBuffer
import breeze.linalg.{DenseVector => BDV, Vector => BV, norm => breezeNorm}
+import org.apache.spark.annotation.Experimental
import org.apache.spark.Logging
import org.apache.spark.SparkContext._
import org.apache.spark.mllib.linalg.{Vector, Vectors}
@@ -81,6 +82,7 @@ class KMeans private (
* this many times with random starting conditions (configured by the initialization mode), then
* return the best clustering found over any run. Default: 1.
*/
+ @Experimental
def setRuns(runs: Int): KMeans = {
if (runs <= 0) {
throw new IllegalArgumentException("Number of runs must be positive")
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala
index 18abbf2758b86..ce14b06241932 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala
@@ -24,7 +24,7 @@ import org.apache.spark.mllib.linalg.Vector
/**
* A clustering model for K-means. Each point belongs to the cluster with the closest center.
*/
-class KMeansModel(val clusterCenters: Array[Vector]) extends Serializable {
+class KMeansModel private[mllib] (val clusterCenters: Array[Vector]) extends Serializable {
/** Total number of clusters. */
def k: Int = clusterCenters.length
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala
similarity index 64%
rename from mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetrics.scala
rename to mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala
index ed7b0fc943367..079743742d86d 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetrics.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala
@@ -15,83 +15,22 @@
* limitations under the License.
*/
-package org.apache.spark.mllib.evaluation.binary
+package org.apache.spark.mllib.evaluation
-import org.apache.spark.rdd.{UnionRDD, RDD}
-import org.apache.spark.SparkContext._
-import org.apache.spark.mllib.evaluation.AreaUnderCurve
+import org.apache.spark.annotation.Experimental
import org.apache.spark.Logging
+import org.apache.spark.SparkContext._
+import org.apache.spark.mllib.evaluation.binary._
+import org.apache.spark.rdd.{RDD, UnionRDD}
/**
- * Implementation of [[org.apache.spark.mllib.evaluation.binary.BinaryConfusionMatrix]].
- *
- * @param count label counter for labels with scores greater than or equal to the current score
- * @param totalCount label counter for all labels
- */
-private case class BinaryConfusionMatrixImpl(
- count: LabelCounter,
- totalCount: LabelCounter) extends BinaryConfusionMatrix with Serializable {
-
- /** number of true positives */
- override def numTruePositives: Long = count.numPositives
-
- /** number of false positives */
- override def numFalsePositives: Long = count.numNegatives
-
- /** number of false negatives */
- override def numFalseNegatives: Long = totalCount.numPositives - count.numPositives
-
- /** number of true negatives */
- override def numTrueNegatives: Long = totalCount.numNegatives - count.numNegatives
-
- /** number of positives */
- override def numPositives: Long = totalCount.numPositives
-
- /** number of negatives */
- override def numNegatives: Long = totalCount.numNegatives
-}
-
-/**
+ * :: Experimental ::
* Evaluator for binary classification.
*
* @param scoreAndLabels an RDD of (score, label) pairs.
*/
-class BinaryClassificationMetrics(scoreAndLabels: RDD[(Double, Double)])
- extends Serializable with Logging {
-
- private lazy val (
- cumulativeCounts: RDD[(Double, LabelCounter)],
- confusions: RDD[(Double, BinaryConfusionMatrix)]) = {
- // Create a bin for each distinct score value, count positives and negatives within each bin,
- // and then sort by score values in descending order.
- val counts = scoreAndLabels.combineByKey(
- createCombiner = (label: Double) => new LabelCounter(0L, 0L) += label,
- mergeValue = (c: LabelCounter, label: Double) => c += label,
- mergeCombiners = (c1: LabelCounter, c2: LabelCounter) => c1 += c2
- ).sortByKey(ascending = false)
- val agg = counts.values.mapPartitions({ iter =>
- val agg = new LabelCounter()
- iter.foreach(agg += _)
- Iterator(agg)
- }, preservesPartitioning = true).collect()
- val partitionwiseCumulativeCounts =
- agg.scanLeft(new LabelCounter())((agg: LabelCounter, c: LabelCounter) => agg.clone() += c)
- val totalCount = partitionwiseCumulativeCounts.last
- logInfo(s"Total counts: $totalCount")
- val cumulativeCounts = counts.mapPartitionsWithIndex(
- (index: Int, iter: Iterator[(Double, LabelCounter)]) => {
- val cumCount = partitionwiseCumulativeCounts(index)
- iter.map { case (score, c) =>
- cumCount += c
- (score, cumCount.clone())
- }
- }, preservesPartitioning = true)
- cumulativeCounts.persist()
- val confusions = cumulativeCounts.map { case (score, cumCount) =>
- (score, BinaryConfusionMatrixImpl(cumCount, totalCount).asInstanceOf[BinaryConfusionMatrix])
- }
- (cumulativeCounts, confusions)
- }
+@Experimental
+class BinaryClassificationMetrics(scoreAndLabels: RDD[(Double, Double)]) extends Logging {
/** Unpersist intermediate RDDs used in the computation. */
def unpersist() {
@@ -154,6 +93,41 @@ class BinaryClassificationMetrics(scoreAndLabels: RDD[(Double, Double)])
/** Returns the (threshold, recall) curve. */
def recallByThreshold(): RDD[(Double, Double)] = createCurve(Recall)
+ private lazy val (
+ cumulativeCounts: RDD[(Double, BinaryLabelCounter)],
+ confusions: RDD[(Double, BinaryConfusionMatrix)]) = {
+ // Create a bin for each distinct score value, count positives and negatives within each bin,
+ // and then sort by score values in descending order.
+ val counts = scoreAndLabels.combineByKey(
+ createCombiner = (label: Double) => new BinaryLabelCounter(0L, 0L) += label,
+ mergeValue = (c: BinaryLabelCounter, label: Double) => c += label,
+ mergeCombiners = (c1: BinaryLabelCounter, c2: BinaryLabelCounter) => c1 += c2
+ ).sortByKey(ascending = false)
+ val agg = counts.values.mapPartitions({ iter =>
+ val agg = new BinaryLabelCounter()
+ iter.foreach(agg += _)
+ Iterator(agg)
+ }, preservesPartitioning = true).collect()
+ val partitionwiseCumulativeCounts =
+ agg.scanLeft(new BinaryLabelCounter())(
+ (agg: BinaryLabelCounter, c: BinaryLabelCounter) => agg.clone() += c)
+ val totalCount = partitionwiseCumulativeCounts.last
+ logInfo(s"Total counts: $totalCount")
+ val cumulativeCounts = counts.mapPartitionsWithIndex(
+ (index: Int, iter: Iterator[(Double, BinaryLabelCounter)]) => {
+ val cumCount = partitionwiseCumulativeCounts(index)
+ iter.map { case (score, c) =>
+ cumCount += c
+ (score, cumCount.clone())
+ }
+ }, preservesPartitioning = true)
+ cumulativeCounts.persist()
+ val confusions = cumulativeCounts.map { case (score, cumCount) =>
+ (score, BinaryConfusionMatrixImpl(cumCount, totalCount).asInstanceOf[BinaryConfusionMatrix])
+ }
+ (cumulativeCounts, confusions)
+ }
+
/** Creates a curve of (threshold, metric). */
private def createCurve(y: BinaryClassificationMetricComputer): RDD[(Double, Double)] = {
confusions.map { case (s, c) =>
@@ -170,35 +144,3 @@ class BinaryClassificationMetrics(scoreAndLabels: RDD[(Double, Double)])
}
}
}
-
-/**
- * A counter for positives and negatives.
- *
- * @param numPositives number of positive labels
- * @param numNegatives number of negative labels
- */
-private class LabelCounter(
- var numPositives: Long = 0L,
- var numNegatives: Long = 0L) extends Serializable {
-
- /** Processes a label. */
- def +=(label: Double): LabelCounter = {
- // Though we assume 1.0 for positive and 0.0 for negative, the following check will handle
- // -1.0 for negative as well.
- if (label > 0.5) numPositives += 1L else numNegatives += 1L
- this
- }
-
- /** Merges another counter. */
- def +=(other: LabelCounter): LabelCounter = {
- numPositives += other.numPositives
- numNegatives += other.numNegatives
- this
- }
-
- override def clone: LabelCounter = {
- new LabelCounter(numPositives, numNegatives)
- }
-
- override def toString: String = s"{numPos: $numPositives, numNeg: $numNegatives}"
-}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryConfusionMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryConfusionMatrix.scala
index 75a75b216002a..559c6ef7e7251 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryConfusionMatrix.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryConfusionMatrix.scala
@@ -39,3 +39,32 @@ private[evaluation] trait BinaryConfusionMatrix {
/** number of negatives */
def numNegatives: Long = numFalsePositives + numTrueNegatives
}
+
+/**
+ * Implementation of [[org.apache.spark.mllib.evaluation.binary.BinaryConfusionMatrix]].
+ *
+ * @param count label counter for labels with scores greater than or equal to the current score
+ * @param totalCount label counter for all labels
+ */
+private[evaluation] case class BinaryConfusionMatrixImpl(
+ count: BinaryLabelCounter,
+ totalCount: BinaryLabelCounter) extends BinaryConfusionMatrix {
+
+ /** number of true positives */
+ override def numTruePositives: Long = count.numPositives
+
+ /** number of false positives */
+ override def numFalsePositives: Long = count.numNegatives
+
+ /** number of false negatives */
+ override def numFalseNegatives: Long = totalCount.numPositives - count.numPositives
+
+ /** number of true negatives */
+ override def numTrueNegatives: Long = totalCount.numNegatives - count.numNegatives
+
+ /** number of positives */
+ override def numPositives: Long = totalCount.numPositives
+
+ /** number of negatives */
+ override def numNegatives: Long = totalCount.numNegatives
+}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryLabelCounter.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryLabelCounter.scala
new file mode 100644
index 0000000000000..1e610c20092a7
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryLabelCounter.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.evaluation.binary
+
+/**
+ * A counter for positives and negatives.
+ *
+ * @param numPositives number of positive labels
+ * @param numNegatives number of negative labels
+ */
+private[evaluation] class BinaryLabelCounter(
+ var numPositives: Long = 0L,
+ var numNegatives: Long = 0L) extends Serializable {
+
+ /** Processes a label. */
+ def +=(label: Double): BinaryLabelCounter = {
+ // Though we assume 1.0 for positive and 0.0 for negative, the following check will handle
+ // -1.0 for negative as well.
+ if (label > 0.5) numPositives += 1L else numNegatives += 1L
+ this
+ }
+
+ /** Merges another counter. */
+ def +=(other: BinaryLabelCounter): BinaryLabelCounter = {
+ numPositives += other.numPositives
+ numNegatives += other.numNegatives
+ this
+ }
+
+ override def clone: BinaryLabelCounter = {
+ new BinaryLabelCounter(numPositives, numNegatives)
+ }
+
+ override def toString: String = s"{numPos: $numPositives, numNeg: $numNegatives}"
+}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala
index 46b105457430c..9669c364bad8f 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala
@@ -17,5 +17,11 @@
package org.apache.spark.mllib.linalg
-/** Represents singular value decomposition (SVD) factors. */
+import org.apache.spark.annotation.Experimental
+
+/**
+ * :: Experimental ::
+ * Represents singular value decomposition (SVD) factors.
+ */
+@Experimental
case class SingularValueDecomposition[UType, VType](U: UType, s: Vector, V: VType)
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala
index 56b8fdcda66eb..06d8915f3bfa1 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala
@@ -25,11 +25,13 @@ import org.apache.spark.SparkContext._
import org.apache.spark.mllib.linalg.Vectors
/**
+ * :: Experimental ::
* Represents an entry in an distributed matrix.
* @param i row index
* @param j column index
* @param value value of the entry
*/
+@Experimental
case class MatrixEntry(i: Long, j: Long, value: Double)
/**
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala
index 0c0afcd9ec0d7..b10857fe7c8a1 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala
@@ -427,6 +427,7 @@ class RowMatrix(
}
}
+@Experimental
object RowMatrix {
/**
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 c75909bac9248..7030eeabe400a 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
@@ -21,19 +21,17 @@ import scala.collection.mutable.ArrayBuffer
import breeze.linalg.{DenseVector => BDV}
-import org.apache.spark.annotation.DeveloperApi
+import org.apache.spark.annotation.{Experimental, DeveloperApi}
import org.apache.spark.Logging
import org.apache.spark.rdd.RDD
import org.apache.spark.mllib.linalg.{Vectors, Vector}
/**
- * :: DeveloperApi ::
* Class used to solve an optimization problem using Gradient Descent.
* @param gradient Gradient function to be used.
* @param updater Updater to be used to update weights after every iteration.
*/
-@DeveloperApi
-class GradientDescent(private var gradient: Gradient, private var updater: Updater)
+class GradientDescent private[mllib] (private var gradient: Gradient, private var updater: Updater)
extends Optimizer with Logging {
private var stepSize: Double = 1.0
@@ -51,9 +49,11 @@ class GradientDescent(private var gradient: Gradient, private var updater: Updat
}
/**
+ * :: Experimental ::
* Set fraction of data to be used for each SGD iteration.
* Default 1.0 (corresponding to deterministic/classical gradient descent)
*/
+ @Experimental
def setMiniBatchFraction(fraction: Double): this.type = {
this.miniBatchFraction = fraction
this
@@ -95,6 +95,14 @@ class GradientDescent(private var gradient: Gradient, private var updater: Updat
this
}
+ /**
+ * :: DeveloperApi ::
+ * Runs gradient descent on the given training data.
+ * @param data training data
+ * @param initialWeights initial weights
+ * @return solution vector
+ */
+ @DeveloperApi
def optimize(data: RDD[(Double, Vector)], initialWeights: Vector): Vector = {
val (weights, _) = GradientDescent.runMiniBatchSGD(
data,
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala
index 2a77e1a9efb2e..0cf9a7f909081 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala
@@ -472,13 +472,15 @@ class ALS private (
// Compute the XtX and Xy values for each user by adding products it rated in each product
// block
for (productBlock <- 0 until numBlocks) {
- for (p <- 0 until blockFactors(productBlock).length) {
+ var p = 0
+ while (p < blockFactors(productBlock).length) {
val x = wrapDoubleArray(blockFactors(productBlock)(p))
tempXtX.fill(0.0)
dspr(1.0, x, tempXtX)
val (us, rs) = inLinkBlock.ratingsForBlock(productBlock)(p)
- for (i <- 0 until us.length) {
- if (implicitPrefs) {
+ if (implicitPrefs) {
+ var i = 0
+ while (i < us.length) {
// Extension to the original paper to handle rs(i) < 0. confidence is a function
// of |rs(i)| instead so that it is never negative:
val confidence = 1 + alpha * abs(rs(i))
@@ -489,11 +491,17 @@ class ALS private (
if (rs(i) > 0) {
SimpleBlas.axpy(confidence, x, userXy(us(i)))
}
- } else {
+ i += 1
+ }
+ } else {
+ var i = 0
+ while (i < us.length) {
userXtX(us(i)).addi(tempXtX)
SimpleBlas.axpy(rs(i), x, userXy(us(i)))
+ i += 1
}
}
+ p += 1
}
}
@@ -502,7 +510,11 @@ class ALS private (
// Compute the full XtX matrix from the lower-triangular part we got above
fillFullMatrix(userXtX(index), fullXtX)
// Add regularization
- (0 until rank).foreach(i => fullXtX.data(i*rank + i) += lambda)
+ var i = 0
+ while (i < rank) {
+ fullXtX.data(i * rank + i) += lambda
+ i += 1
+ }
// Solve the resulting matrix, which is symmetric and positive-definite
if (implicitPrefs) {
Solve.solvePositive(fullXtX.addi(YtY.get.value), userXy(index)).data
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala
index 471546cd82c7d..899286d235a9d 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala
@@ -17,7 +17,7 @@
package org.apache.spark.mllib.recommendation
-import org.jblas._
+import org.jblas.DoubleMatrix
import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.api.java.JavaRDD
@@ -25,7 +25,6 @@ import org.apache.spark.rdd.RDD
import org.apache.spark.SparkContext._
import org.apache.spark.mllib.api.python.PythonMLLibAPI
-
/**
* Model representing the result of matrix factorization.
*
@@ -35,12 +34,10 @@ import org.apache.spark.mllib.api.python.PythonMLLibAPI
* @param productFeatures RDD of tuples where each tuple represents the productId
* and the features computed for this product.
*/
-class MatrixFactorizationModel(
+class MatrixFactorizationModel private[mllib] (
val rank: Int,
val userFeatures: RDD[(Int, Array[Double])],
- val productFeatures: RDD[(Int, Array[Double])])
- extends Serializable
-{
+ val productFeatures: RDD[(Int, Array[Double])]) extends Serializable {
/** Predict the rating of one user for one product. */
def predict(user: Int, product: Int): Double = {
val userVector = new DoubleMatrix(userFeatures.lookup(user).head)
@@ -76,6 +73,7 @@ class MatrixFactorizationModel(
* @param usersProductsJRDD A JavaRDD with serialized tuples (user, product)
* @return JavaRDD of serialized Rating objects.
*/
+ @DeveloperApi
def predict(usersProductsJRDD: JavaRDD[Array[Byte]]): JavaRDD[Array[Byte]] = {
val pythonAPI = new PythonMLLibAPI()
val usersProducts = usersProductsJRDD.rdd.map(xBytes => pythonAPI.unpackTuple(xBytes))
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 d969e7aa60061..8cca926f1c92e 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala
@@ -19,13 +19,14 @@ package org.apache.spark.mllib.regression
import breeze.linalg.{DenseVector => BDV, SparseVector => BSV}
-import org.apache.spark.annotation.Experimental
+import org.apache.spark.annotation.DeveloperApi
import org.apache.spark.{Logging, SparkException}
import org.apache.spark.rdd.RDD
import org.apache.spark.mllib.optimization._
import org.apache.spark.mllib.linalg.{Vectors, Vector}
/**
+ * :: DeveloperApi ::
* GeneralizedLinearModel (GLM) represents a model trained using
* GeneralizedLinearAlgorithm. GLMs consist of a weight vector and
* an intercept.
@@ -33,6 +34,7 @@ import org.apache.spark.mllib.linalg.{Vectors, Vector}
* @param weights Weights computed for every feature.
* @param intercept Intercept computed for this model.
*/
+@DeveloperApi
abstract class GeneralizedLinearModel(val weights: Vector, val intercept: Double)
extends Serializable {
@@ -72,9 +74,11 @@ abstract class GeneralizedLinearModel(val weights: Vector, val intercept: Double
}
/**
+ * :: DeveloperApi ::
* GeneralizedLinearAlgorithm implements methods to train a Generalized Linear Model (GLM).
* This class should be extended with an Optimizer to create a new GLM.
*/
+@DeveloperApi
abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel]
extends Logging with Serializable {
@@ -83,8 +87,8 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel]
/** The optimizer to solve the problem. */
def optimizer: Optimizer
- /** Whether to add intercept (default: true). */
- protected var addIntercept: Boolean = true
+ /** Whether to add intercept (default: false). */
+ protected var addIntercept: Boolean = false
protected var validateData: Boolean = true
@@ -94,7 +98,8 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel]
protected def createModel(weights: Vector, intercept: Double): M
/**
- * Set if the algorithm should add an intercept. Default true.
+ * Set if the algorithm should add an intercept. Default false.
+ * We set the default to false because adding the intercept will cause memory allocation.
*/
def setIntercept(addIntercept: Boolean): this.type = {
this.addIntercept = addIntercept
@@ -102,10 +107,8 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel]
}
/**
- * :: Experimental ::
* Set if the algorithm should validate data before training. Default true.
*/
- @Experimental
def setValidateData(validateData: Boolean): this.type = {
this.validateData = validateData
this
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala
index 0e6fb1b1caa41..a05dfc045fb8e 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,6 +17,7 @@
package org.apache.spark.mllib.regression
+import org.apache.spark.annotation.Experimental
import org.apache.spark.mllib.linalg.Vector
import org.apache.spark.mllib.optimization._
import org.apache.spark.rdd.RDD
@@ -27,7 +28,7 @@ import org.apache.spark.rdd.RDD
* @param weights Weights computed for every feature.
* @param intercept Intercept computed for this model.
*/
-class LassoModel(
+class LassoModel private[mllib] (
override val weights: Vector,
override val intercept: Double)
extends GeneralizedLinearModel(weights, intercept)
@@ -64,21 +65,12 @@ class LassoWithSGD private (
.setRegParam(regParam)
.setMiniBatchFraction(miniBatchFraction)
- // We don't want to penalize the intercept, so set this to false.
- super.setIntercept(false)
-
/**
* Construct a Lasso object with default parameters: {stepSize: 1.0, numIterations: 100,
* regParam: 1.0, miniBatchFraction: 1.0}.
*/
def this() = this(1.0, 100, 1.0, 1.0)
- override def setIntercept(addIntercept: Boolean): this.type = {
- // TODO: Support adding intercept.
- if (addIntercept) throw new UnsupportedOperationException("Adding intercept is not supported.")
- this
- }
-
override protected def createModel(weights: Vector, intercept: Double) = {
new LassoModel(weights, intercept)
}
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 1532ff90d846d..0ebad4eb58d88 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
@@ -27,7 +27,7 @@ import org.apache.spark.mllib.optimization._
* @param weights Weights computed for every feature.
* @param intercept Intercept computed for this model.
*/
-class LinearRegressionModel(
+class LinearRegressionModel private[mllib] (
override val weights: Vector,
override val intercept: Double)
extends GeneralizedLinearModel(weights, intercept) with RegressionModel with Serializable {
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 5f7e25a9b8be1..bd983bac001a0 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
@@ -17,6 +17,7 @@
package org.apache.spark.mllib.regression
+import org.apache.spark.annotation.Experimental
import org.apache.spark.rdd.RDD
import org.apache.spark.mllib.optimization._
import org.apache.spark.mllib.linalg.Vector
@@ -27,7 +28,7 @@ import org.apache.spark.mllib.linalg.Vector
* @param weights Weights computed for every feature.
* @param intercept Intercept computed for this model.
*/
-class RidgeRegressionModel(
+class RidgeRegressionModel private[mllib] (
override val weights: Vector,
override val intercept: Double)
extends GeneralizedLinearModel(weights, intercept)
@@ -65,21 +66,12 @@ class RidgeRegressionWithSGD private (
.setRegParam(regParam)
.setMiniBatchFraction(miniBatchFraction)
- // We don't want to penalize the intercept in RidgeRegression, so set this to false.
- super.setIntercept(false)
-
/**
* Construct a RidgeRegression object with default parameters: {stepSize: 1.0, numIterations: 100,
* regParam: 1.0, miniBatchFraction: 1.0}.
*/
def this() = this(1.0, 100, 1.0, 1.0)
- override def setIntercept(addIntercept: Boolean): this.type = {
- // TODO: Support adding intercept.
- if (addIntercept) throw new UnsupportedOperationException("Adding intercept is not supported.")
- this
- }
-
override protected def createModel(weights: Vector, intercept: Double) = {
new RidgeRegressionModel(weights, intercept)
}
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
deleted file mode 100644
index 0fd71aa9735bc..0000000000000
--- a/mllib/src/main/scala/org/apache/spark/mllib/tree/README.md
+++ /dev/null
@@ -1,17 +0,0 @@
-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/rdd/VectorRDDs.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/package.scala
similarity index 69%
rename from mllib/src/main/scala/org/apache/spark/mllib/rdd/VectorRDDs.scala
rename to mllib/src/main/scala/org/apache/spark/mllib/tree/package.scala
index 9096d6a1a16d6..bcaacc1b1f191 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/rdd/VectorRDDs.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/package.scala
@@ -15,18 +15,15 @@
* limitations under the License.
*/
-package org.apache.spark.mllib.rdd
-
-import org.apache.spark.rdd.RDD
-import org.apache.spark.mllib.linalg.{Vectors, Vector}
+package org.apache.spark.mllib
/**
- * Factory methods for `RDD[Vector]`.
+ * This package contains the default implementation of the decision tree algorithm, which supports:
+ * - binary classification,
+ * - regression,
+ * - information loss calculation with entropy and Gini for classification and
+ * variance for regression,
+ * - both continuous and categorical features.
*/
-object VectorRDDs {
-
- /**
- * Converts an `RDD[Array[Double]]` to `RDD[Vector]`.
- */
- def fromArrayRDD(rdd: RDD[Array[Double]]): RDD[Vector] = rdd.map(v => Vectors.dense(v))
+package object tree {
}
diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LabelParsers.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LabelParsers.scala
index f7966d3ebb613..e25bf18b780bf 100644
--- a/mllib/src/main/scala/org/apache/spark/mllib/util/LabelParsers.scala
+++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LabelParsers.scala
@@ -18,16 +18,23 @@
package org.apache.spark.mllib.util
/** Trait for label parsers. */
-trait LabelParser extends Serializable {
+private trait LabelParser extends Serializable {
/** Parses a string label into a double label. */
def parse(labelString: String): Double
}
+/** Factory methods for label parsers. */
+private object LabelParser {
+ def getInstance(multiclass: Boolean): LabelParser = {
+ if (multiclass) MulticlassLabelParser else BinaryLabelParser
+ }
+}
+
/**
* Label parser for binary labels, which outputs 1.0 (positive) if the value is greater than 0.5,
* or 0.0 (negative) otherwise. So it works with +1/-1 labeling and +1/0 labeling.
*/
-object BinaryLabelParser extends LabelParser {
+private object BinaryLabelParser extends LabelParser {
/** Gets the default instance of BinaryLabelParser. */
def getInstance(): LabelParser = this
@@ -41,7 +48,7 @@ object BinaryLabelParser extends LabelParser {
/**
* Label parser for multiclass labels, which converts the input label to double.
*/
-object MulticlassLabelParser extends LabelParser {
+private object MulticlassLabelParser extends LabelParser {
/** Gets the default instance of MulticlassLabelParser. */
def getInstance(): LabelParser = this
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 3d6e7e0d5c953..e598b6cb171a8 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
@@ -19,16 +19,17 @@ package org.apache.spark.mllib.util
import scala.reflect.ClassTag
-import breeze.linalg.{Vector => BV, SparseVector => BSV, squaredDistance => breezeSquaredDistance}
+import breeze.linalg.{Vector => BV, DenseVector => BDV, SparseVector => BSV,
+ squaredDistance => breezeSquaredDistance}
import org.apache.spark.annotation.Experimental
import org.apache.spark.SparkContext
import org.apache.spark.rdd.RDD
import org.apache.spark.rdd.PartitionwiseSampledRDD
-import org.apache.spark.SparkContext._
import org.apache.spark.util.random.BernoulliSampler
import org.apache.spark.mllib.regression.LabeledPoint
-import org.apache.spark.mllib.linalg.Vectors
+import org.apache.spark.mllib.linalg.{Vector, Vectors}
+import org.apache.spark.storage.StorageLevel
/**
* Helper methods to load, save and pre-process data used in ML Lib.
@@ -54,13 +55,16 @@ object MLUtils {
*
* @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 labelParser parser for labels
* @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 minPartitions min number of partitions, default: sc.defaultMinPartitions
+ * nonpositive value is given. This is useful when the dataset is already split
+ * into multiple files and you want to load them separately, because some
+ * features may not present in certain files, which leads to inconsistent
+ * feature dimensions.
+ * @param minPartitions min number of partitions
* @return labeled data stored as an RDD[LabeledPoint]
*/
- def loadLibSVMData(
+ private def loadLibSVMFile(
sc: SparkContext,
path: String,
labelParser: LabelParser,
@@ -68,63 +72,112 @@ object MLUtils {
minPartitions: Int): RDD[LabeledPoint] = {
val parsed = sc.textFile(path, minPartitions)
.map(_.trim)
- .filter(!_.isEmpty)
- .map(_.split(' '))
+ .filter(line => !(line.isEmpty || line.startsWith("#")))
+ .map { line =>
+ val items = line.split(' ')
+ val label = labelParser.parse(items.head)
+ val (indices, values) = items.tail.map { item =>
+ val indexAndValue = item.split(':')
+ val index = indexAndValue(0).toInt - 1 // Convert 1-based indices to 0-based.
+ val value = indexAndValue(1).toDouble
+ (index, value)
+ }.unzip
+ (label, indices.toArray, values.toArray)
+ }
+
// Determine number of features.
- val d = if (numFeatures >= 0) {
+ 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.persist(StorageLevel.MEMORY_ONLY)
+ parsed.map { case (label, indices, values) =>
+ indices.lastOption.getOrElse(0)
+ }.reduce(math.max) + 1
}
- parsed.map { items =>
- val label = labelParser.parse(items.head)
- val (indices, values) = items.tail.map { item =>
- val indexAndValue = item.split(':')
- val index = indexAndValue(0).toInt - 1
- val value = indexAndValue(1).toDouble
- (index, value)
- }.unzip
- LabeledPoint(label, Vectors.sparse(d, indices.toArray, values.toArray))
+
+ parsed.map { case (label, indices, values) =>
+ LabeledPoint(label, Vectors.sparse(d, indices, values))
}
}
- // Convenient methods for calling from Java.
+ // Convenient methods for `loadLibSVMFile`.
/**
- * Loads binary labeled data in the LIBSVM format into an RDD[LabeledPoint],
- * with number of features determined automatically and the default number of partitions.
+ * 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 multiclass whether the input labels contain more than two classes. If false, any label
+ * with value greater than 0.5 will be mapped to 1.0, or 0.0 otherwise. So it
+ * works for both +1/-1 and 1/0 cases. If true, the double value parsed directly
+ * from the label string will be used as the label value.
+ * @param numFeatures number of features, which will be determined from the input data if a
+ * nonpositive value is given. This is useful when the dataset is already split
+ * into multiple files and you want to load them separately, because some
+ * features may not present in certain files, which leads to inconsistent
+ * feature dimensions.
+ * @param minPartitions min number of partitions
+ * @return labeled data stored as an RDD[LabeledPoint]
*/
- def loadLibSVMData(sc: SparkContext, path: String): RDD[LabeledPoint] =
- loadLibSVMData(sc, path, BinaryLabelParser, -1, sc.defaultMinPartitions)
+ def loadLibSVMFile(
+ sc: SparkContext,
+ path: String,
+ multiclass: Boolean,
+ numFeatures: Int,
+ minPartitions: Int): RDD[LabeledPoint] =
+ loadLibSVMFile(sc, path, LabelParser.getInstance(multiclass), numFeatures, minPartitions)
/**
- * 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.
+ * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint], with the default number of
+ * partitions.
*/
- def loadLibSVMData(
+ def loadLibSVMFile(
sc: SparkContext,
path: String,
- labelParser: LabelParser): RDD[LabeledPoint] =
- loadLibSVMData(sc, path, labelParser, -1, sc.defaultMinPartitions)
+ multiclass: Boolean,
+ numFeatures: Int): RDD[LabeledPoint] =
+ loadLibSVMFile(sc, path, multiclass, numFeatures, sc.defaultMinPartitions)
/**
- * 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.
+ * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint], with the number of features
+ * determined automatically and the default number of partitions.
*/
- def loadLibSVMData(
+ def loadLibSVMFile(
sc: SparkContext,
path: String,
- labelParser: LabelParser,
- numFeatures: Int): RDD[LabeledPoint] =
- loadLibSVMData(sc, path, labelParser, numFeatures, sc.defaultMinPartitions)
+ multiclass: Boolean): RDD[LabeledPoint] =
+ loadLibSVMFile(sc, path, multiclass, -1, sc.defaultMinPartitions)
+
+ /**
+ * 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 loadLibSVMFile(sc: SparkContext, path: String): RDD[LabeledPoint] =
+ loadLibSVMFile(sc, path, multiclass = false, -1, sc.defaultMinPartitions)
+
+ /**
+ * Save labeled data in LIBSVM format.
+ * @param data an RDD of LabeledPoint to be saved
+ * @param dir directory to save the data
+ *
+ * @see [[org.apache.spark.mllib.util.MLUtils#loadLibSVMFile]]
+ */
+ def saveAsLibSVMFile(data: RDD[LabeledPoint], dir: String) {
+ // TODO: allow to specify label precision and feature precision.
+ val dataStr = data.map { case LabeledPoint(label, features) =>
+ val featureStrings = features.toBreeze.activeIterator.map { case (i, v) =>
+ s"${i + 1}:$v"
+ }
+ (Iterator(label) ++ featureStrings).mkString(" ")
+ }
+ dataStr.saveAsTextFile(dir)
+ }
/**
* :: Experimental ::
@@ -163,10 +216,12 @@ object MLUtils {
}
/**
+ * :: Experimental ::
* Return a k element array of pairs of RDDs with the first element of each pair
* containing the training data, a complement of the validation data and the second
* element, the validation data, containing a unique 1/kth of the data. Where k=numFolds.
*/
+ @Experimental
def kFold[T: ClassTag](rdd: RDD[T], numFolds: Int, seed: Int): Array[(RDD[T], RDD[T])] = {
val numFoldsF = numFolds.toFloat
(1 to numFolds).map { fold =>
@@ -178,6 +233,18 @@ object MLUtils {
}.toArray
}
+ /**
+ * Returns a new vector with `1.0` (bias) appended to the input vector.
+ */
+ def appendBias(vector: Vector): Vector = {
+ val vector1 = vector.toBreeze match {
+ case dv: BDV[Double] => BDV.vertcat(dv, new BDV[Double](Array(1.0)))
+ case sv: BSV[Double] => BSV.vertcat(sv, new BSV[Double](Array(0), Array(1.0), 1))
+ case v: Any => throw new IllegalArgumentException("Do not support vector type " + v.getClass)
+ }
+ Vectors.fromBreeze(vector1)
+ }
+
/**
* Returns the squared Euclidean distance between two vectors. The following formula will be used
* if it does not introduce too much numerical error:
diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java
index e18e3bc6a86be..d75d3a6b26730 100644
--- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java
+++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaLogisticRegressionSuite.java
@@ -68,6 +68,7 @@ public void runLRUsingConstructor() {
LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 17);
LogisticRegressionWithSGD lrImpl = new LogisticRegressionWithSGD();
+ lrImpl.setIntercept(true);
lrImpl.optimizer().setStepSize(1.0)
.setRegParam(1.0)
.setNumIterations(100);
@@ -80,8 +81,8 @@ public void runLRUsingConstructor() {
@Test
public void runLRUsingStaticMethods() {
int nPoints = 10000;
- double A = 2.0;
- double B = -1.5;
+ double A = 0.0;
+ double B = -2.5;
JavaRDD testRDD = sc.parallelize(
LogisticRegressionSuite.generateLogisticInputAsList(A, B, nPoints, 42), 2).cache();
@@ -92,6 +93,7 @@ public void runLRUsingStaticMethods() {
testRDD.rdd(), 100, 1.0, 1.0);
int numAccurate = validatePrediction(validationData, model);
+ System.out.println(numAccurate);
Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0);
}
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 4701a5e545020..667f76a1bd55f 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
@@ -67,6 +67,7 @@ public void runSVMUsingConstructor() {
SVMSuite.generateSVMInputAsList(A, weights, nPoints, 17);
SVMWithSGD svmSGDImpl = new SVMWithSGD();
+ svmSGDImpl.setIntercept(true);
svmSGDImpl.optimizer().setStepSize(1.0)
.setRegParam(1.0)
.setNumIterations(100);
@@ -79,7 +80,7 @@ public void runSVMUsingConstructor() {
@Test
public void runSVMUsingStaticMethods() {
int nPoints = 10000;
- double A = 2.0;
+ double A = 0.0;
double[] weights = {-1.5, 1.0};
JavaRDD testRDD = sc.parallelize(SVMSuite.generateSVMInputAsList(A,
diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java
index 5a4410a632649..7151e553512b3 100644
--- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java
+++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLinearRegressionSuite.java
@@ -68,6 +68,7 @@ public void runLinearRegressionUsingConstructor() {
LinearDataGenerator.generateLinearInputAsList(A, weights, nPoints, 17, 0.1);
LinearRegressionWithSGD linSGDImpl = new LinearRegressionWithSGD();
+ linSGDImpl.setIntercept(true);
LinearRegressionModel model = linSGDImpl.run(testRDD.rdd());
int numAccurate = validatePrediction(validationData, model);
@@ -77,7 +78,7 @@ public void runLinearRegressionUsingConstructor() {
@Test
public void runLinearRegressionUsingStaticMethods() {
int nPoints = 100;
- double A = 3.0;
+ double A = 0.0;
double[] weights = {10, 10};
JavaRDD testRDD = sc.parallelize(
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 1e03c9df820b0..4d7b984e3ec29 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
@@ -46,24 +46,14 @@ object LogisticRegressionSuite {
val rnd = new Random(seed)
val x1 = Array.fill[Double](nPoints)(rnd.nextGaussian())
- // NOTE: if U is uniform[0, 1] then ln(u) - ln(1-u) is Logistic(0,1)
- val unifRand = new scala.util.Random(45)
- val rLogis = (0 until nPoints).map { i =>
- val u = unifRand.nextDouble()
- math.log(u) - math.log(1.0-u)
- }
-
- // y <- A + B*x + rLogis()
- // y <- as.numeric(y > 0)
- val y: Seq[Int] = (0 until nPoints).map { i =>
- val yVal = offset + scale * x1(i) + rLogis(i)
- if (yVal > 0) 1 else 0
+ val y = (0 until nPoints).map { i =>
+ val p = 1.0 / (1.0 + math.exp(-(offset + scale * x1(i))))
+ if (rnd.nextDouble() < p) 1.0 else 0.0
}
val testData = (0 until nPoints).map(i => LabeledPoint(y(i), Vectors.dense(Array(x1(i)))))
testData
}
-
}
class LogisticRegressionSuite extends FunSuite with LocalSparkContext with ShouldMatchers {
@@ -85,7 +75,7 @@ class LogisticRegressionSuite extends FunSuite with LocalSparkContext with Shoul
val testRDD = sc.parallelize(testData, 2)
testRDD.cache()
- val lr = new LogisticRegressionWithSGD()
+ val lr = new LogisticRegressionWithSGD().setIntercept(true)
lr.optimizer.setStepSize(10.0).setNumIterations(20)
val model = lr.run(testRDD)
@@ -118,7 +108,7 @@ class LogisticRegressionSuite extends FunSuite with LocalSparkContext with Shoul
testRDD.cache()
// Use half as many iterations as the previous test.
- val lr = new LogisticRegressionWithSGD()
+ val lr = new LogisticRegressionWithSGD().setIntercept(true)
lr.optimizer.setStepSize(10.0).setNumIterations(10)
val model = lr.run(testRDD, initialWeights)
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 dfacbfeee6fb4..77d6f04b32320 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
@@ -69,7 +69,6 @@ class SVMSuite extends FunSuite with LocalSparkContext {
assert(numOffPredictions < input.length / 5)
}
-
test("SVM using local random SGD") {
val nPoints = 10000
@@ -83,7 +82,7 @@ class SVMSuite extends FunSuite with LocalSparkContext {
val testRDD = sc.parallelize(testData, 2)
testRDD.cache()
- val svm = new SVMWithSGD()
+ val svm = new SVMWithSGD().setIntercept(true)
svm.optimizer.setStepSize(1.0).setRegParam(1.0).setNumIterations(100)
val model = svm.run(testRDD)
@@ -115,7 +114,7 @@ class SVMSuite extends FunSuite with LocalSparkContext {
val testRDD = sc.parallelize(testData, 2)
testRDD.cache()
- val svm = new SVMWithSGD()
+ val svm = new SVMWithSGD().setIntercept(true)
svm.optimizer.setStepSize(1.0).setRegParam(1.0).setNumIterations(100)
val model = svm.run(testRDD, initialWeights)
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala
similarity index 96%
rename from mllib/src/test/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricsSuite.scala
rename to mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala
index 173fdaefab3da..9d16182f9d8c4 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricsSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala
@@ -15,12 +15,11 @@
* limitations under the License.
*/
-package org.apache.spark.mllib.evaluation.binary
+package org.apache.spark.mllib.evaluation
import org.scalatest.FunSuite
import org.apache.spark.mllib.util.LocalSparkContext
-import org.apache.spark.mllib.evaluation.AreaUnderCurve
class BinaryClassificationMetricsSuite extends FunSuite with LocalSparkContext {
test("binary evaluation metrics") {
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala
index 4dfcd4b52ec66..2d944f3eb7ff9 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala
@@ -27,7 +27,6 @@ import org.jblas.DoubleMatrix
import org.apache.spark.mllib.util.LocalSparkContext
import org.apache.spark.SparkContext._
-import org.apache.spark.Partitioner
object ALSSuite {
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 6aad9eb84e13c..bfa42959c8ead 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
@@ -112,10 +112,4 @@ class LassoSuite extends FunSuite with LocalSparkContext {
// Test prediction on Array.
validatePrediction(validationData.map(row => model.predict(row.features)), validationData)
}
-
- test("do not support intercept") {
- intercept[UnsupportedOperationException] {
- new LassoWithSGD().setIntercept(true)
- }
- }
}
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala
index 2f7d30708ce17..7aaad7d7a3e39 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
@@ -37,7 +37,7 @@ class LinearRegressionSuite extends FunSuite with LocalSparkContext {
test("linear regression") {
val testRDD = sc.parallelize(LinearDataGenerator.generateLinearInput(
3.0, Array(10.0, 10.0), 100, 42), 2).cache()
- val linReg = new LinearRegressionWithSGD()
+ val linReg = new LinearRegressionWithSGD().setIntercept(true)
linReg.optimizer.setNumIterations(1000).setStepSize(1.0)
val model = linReg.run(testRDD)
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 f66fc6ea6c1ec..67768e17fbe6d 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
@@ -72,10 +72,4 @@ class RidgeRegressionSuite extends FunSuite with LocalSparkContext {
assert(ridgeErr < linearErr,
"ridgeError (" + ridgeErr + ") was not less than linearError(" + linearErr + ")")
}
-
- test("do not support intercept") {
- intercept[UnsupportedOperationException] {
- new RidgeRegressionWithSGD().setIntercept(true)
- }
- }
}
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala
index bc3b1a3fbe95c..35e92d71dc63f 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala
+++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala
@@ -17,10 +17,8 @@
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
@@ -29,19 +27,9 @@ import org.apache.spark.mllib.tree.configuration.{FeatureType, Strategy}
import org.apache.spark.mllib.tree.configuration.Algo._
import org.apache.spark.mllib.tree.configuration.FeatureType._
import org.apache.spark.mllib.linalg.Vectors
+import org.apache.spark.mllib.util.LocalSparkContext
-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")
- }
+class DecisionTreeSuite extends FunSuite with LocalSparkContext {
test("split and bin calculation") {
val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1()
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 674378a34ce34..3f64baf6fe41f 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
@@ -19,8 +19,8 @@ package org.apache.spark.mllib.util
import java.io.File
+import scala.io.Source
import scala.math
-import scala.util.Random
import org.scalatest.FunSuite
@@ -29,7 +29,8 @@ import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, norm => breezeNor
import com.google.common.base.Charsets
import com.google.common.io.Files
-import org.apache.spark.mllib.linalg.Vectors
+import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors}
+import org.apache.spark.mllib.regression.LabeledPoint
import org.apache.spark.mllib.util.MLUtils._
class MLUtilsSuite extends FunSuite with LocalSparkContext {
@@ -58,7 +59,7 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext {
}
}
- test("loadLibSVMData") {
+ test("loadLibSVMFile") {
val lines =
"""
|+1 1:1.0 3:2.0 5:3.0
@@ -70,8 +71,8 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext {
Files.write(lines, file, Charsets.US_ASCII)
val path = tempDir.toURI.toString
- val pointsWithNumFeatures = MLUtils.loadLibSVMData(sc, path, BinaryLabelParser, 6).collect()
- val pointsWithoutNumFeatures = MLUtils.loadLibSVMData(sc, path).collect()
+ val pointsWithNumFeatures = loadLibSVMFile(sc, path, multiclass = false, 6).collect()
+ val pointsWithoutNumFeatures = loadLibSVMFile(sc, path).collect()
for (points <- Seq(pointsWithNumFeatures, pointsWithoutNumFeatures)) {
assert(points.length === 3)
@@ -83,29 +84,54 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext {
assert(points(2).features === Vectors.sparse(6, Seq((1, 4.0), (3, 5.0), (5, 6.0))))
}
- val multiclassPoints = MLUtils.loadLibSVMData(sc, path, MulticlassLabelParser).collect()
+ val multiclassPoints = loadLibSVMFile(sc, path, multiclass = true).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 =>
- }
+ deleteQuietly(tempDir)
+ }
+
+ test("saveAsLibSVMFile") {
+ val examples = sc.parallelize(Seq(
+ LabeledPoint(1.1, Vectors.sparse(3, Seq((0, 1.23), (2, 4.56)))),
+ LabeledPoint(0.0, Vectors.dense(1.01, 2.02, 3.03))
+ ), 2)
+ val tempDir = Files.createTempDir()
+ val outputDir = new File(tempDir, "output")
+ MLUtils.saveAsLibSVMFile(examples, outputDir.toURI.toString)
+ val lines = outputDir.listFiles()
+ .filter(_.getName.startsWith("part-"))
+ .flatMap(Source.fromFile(_).getLines())
+ .toSet
+ val expected = Set("1.1 1:1.23 3:4.56", "0.0 1:1.01 2:2.02 3:3.03")
+ assert(lines === expected)
+ deleteQuietly(tempDir)
+ }
+
+ test("appendBias") {
+ val sv = Vectors.sparse(3, Seq((0, 1.0), (2, 3.0)))
+ val sv1 = appendBias(sv).asInstanceOf[SparseVector]
+ assert(sv1.size === 4)
+ assert(sv1.indices === Array(0, 2, 3))
+ assert(sv1.values === Array(1.0, 3.0, 1.0))
+
+ val dv = Vectors.dense(1.0, 0.0, 3.0)
+ val dv1 = appendBias(dv).asInstanceOf[DenseVector]
+ assert(dv1.size === 4)
+ assert(dv1.values === Array(1.0, 0.0, 3.0, 1.0))
}
test("kFold") {
val data = sc.parallelize(1 to 100, 2)
val collectedData = data.collect().sorted
- val twoFoldedRdd = MLUtils.kFold(data, 2, 1)
+ val twoFoldedRdd = kFold(data, 2, 1)
assert(twoFoldedRdd(0)._1.collect().sorted === twoFoldedRdd(1)._2.collect().sorted)
assert(twoFoldedRdd(0)._2.collect().sorted === twoFoldedRdd(1)._1.collect().sorted)
for (folds <- 2 to 10) {
for (seed <- 1 to 5) {
- val foldedRdds = MLUtils.kFold(data, folds, seed)
+ val foldedRdds = kFold(data, folds, seed)
assert(foldedRdds.size === folds)
foldedRdds.map { case (training, validation) =>
val result = validation.union(training).collect().sorted
@@ -132,4 +158,16 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext {
}
}
+ /** Delete a file/directory quietly. */
+ def deleteQuietly(f: File) {
+ if (f.isDirectory) {
+ f.listFiles().foreach(deleteQuietly)
+ }
+ try {
+ f.delete()
+ } catch {
+ case _: Throwable =>
+ }
+ }
}
+
diff --git a/pom.xml b/pom.xml
index 646753fe30301..fb904e868cfaf 100644
--- a/pom.xml
+++ b/pom.xml
@@ -16,7 +16,8 @@
~ limitations under the License.
-->
-
+
4.0.0
org.apache
@@ -119,7 +120,7 @@
1.2.17
1.0.4
2.4.1
- 0.23.7
+ ${hadoop.version}
0.94.6
0.12.0
1.3.2
@@ -128,6 +129,7 @@
0.3.6
3.0.0
1.7.4
+ 0.7.1
64m
512m
@@ -135,7 +137,8 @@
- maven-repo
+ maven-repo
+
Maven Repository
http://repo.maven.apache.org/maven2
@@ -219,6 +222,11 @@
guava
14.0.1
+
+ org.apache.commons
+ commons-lang3
+ 3.3.2
+
com.google.code.findbugs
jsr305
@@ -553,10 +561,18 @@
+
net.java.dev.jets3t
jets3t
- 0.7.1
+ ${jets3t.version}
+ runtime
+
+
+ commons-logging
+ commons-logging
+
+
org.apache.hadoop
@@ -581,6 +597,29 @@
org.apache.hadoop
hadoop-yarn-common
${yarn.version}
+
+
+ asm
+ asm
+
+
+ org.ow2.asm
+ asm
+
+
+ org.jboss.netty
+ netty
+
+
+ javax.servlet
+ servlet-api
+
+
+
+
+ org.apache.hadoop
+ hadoop-yarn-server-web-proxy
+ ${yarn.version}
asm
@@ -596,7 +635,6 @@
-
org.apache.hadoop
hadoop-yarn-client
@@ -614,6 +652,10 @@
org.jboss.netty
netty
+
+ javax.servlet
+ servlet-api
+
@@ -828,35 +870,6 @@
-
-
- hadoop-0.23
-
-
- org.apache.avro
- avro
-
-
-
-
-
- yarn-alpha
-
- 2
-
- 0.23.7
-
-
-
- yarn
-
-
-
- org.apache.avro
- avro
-
-
-
@@ -891,17 +904,54 @@
+
+
- yarn
+ hadoop-0.23
+
+
+
+ org.apache.avro
+ avro
+
+
+
+
+
+ hadoop-2.2
- 2
- 2.2.0
2.5.0
+
+
+
+ hadoop-2.3
+
+ 2.5.0
+ 0.9.0
+
+
+
+
+ hadoop-2.4
+
+ 2.5.0
+ 0.9.0
+
+
+
+
+ yarn-alpha
yarn
+
+
+ yarn
+
+ yarn
+
@@ -926,6 +976,11 @@
hadoop-yarn-common
provided
+
+ org.apache.hadoop
+ hadoop-yarn-server-web-proxy
+ provided
+
org.apache.hadoop
hadoop-yarn-client
diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala
index d540dc0a986e9..efdb38e907d14 100644
--- a/project/MimaBuild.scala
+++ b/project/MimaBuild.scala
@@ -74,6 +74,8 @@ object MimaBuild {
) ++
excludeSparkClass("rdd.ClassTags") ++
excludeSparkClass("util.XORShiftRandom") ++
+ excludeSparkClass("graphx.EdgeRDD") ++
+ excludeSparkClass("graphx.VertexRDD") ++
excludeSparkClass("mllib.recommendation.MFDataGenerator") ++
excludeSparkClass("mllib.optimization.SquaredGradient") ++
excludeSparkClass("mllib.regression.RidgeRegressionWithSGD") ++
diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala
index 51f7335111166..7f9746ec4acc0 100644
--- a/project/SparkBuild.scala
+++ b/project/SparkBuild.scala
@@ -95,7 +95,7 @@ object SparkBuild extends Build {
lazy val hadoopVersion = Properties.envOrElse("SPARK_HADOOP_VERSION", DEFAULT_HADOOP_VERSION)
lazy val isNewHadoop = Properties.envOrNone("SPARK_IS_NEW_HADOOP") match {
case None => {
- val isNewHadoopVersion = "2.[2-9]+".r.findFirstIn(hadoopVersion).isDefined
+ val isNewHadoopVersion = "^2\\.[2-9]+".r.findFirstIn(hadoopVersion).isDefined
(isNewHadoopVersion|| DEFAULT_IS_NEW_HADOOP)
}
case Some(v) => v.toBoolean
@@ -297,6 +297,7 @@ object SparkBuild extends Build {
val chillVersion = "0.3.6"
val codahaleMetricsVersion = "3.0.0"
val jblasVersion = "1.2.3"
+ val jets3tVersion = if ("^2\\.[3-9]+".r.findFirstIn(hadoopVersion).isDefined) "0.9.0" else "0.7.1"
val jettyVersion = "8.1.14.v20131031"
val hiveVersion = "0.12.0"
val parquetVersion = "1.3.2"
@@ -326,6 +327,7 @@ object SparkBuild extends Build {
name := "spark-core",
libraryDependencies ++= Seq(
"com.google.guava" % "guava" % "14.0.1",
+ "org.apache.commons" % "commons-lang3" % "3.3.2",
"com.google.code.findbugs" % "jsr305" % "1.3.9",
"log4j" % "log4j" % "1.2.17",
"org.slf4j" % "slf4j-api" % slf4jVersion,
@@ -342,7 +344,7 @@ object SparkBuild extends Build {
"colt" % "colt" % "1.2.0",
"org.apache.mesos" % "mesos" % "0.13.0",
"commons-net" % "commons-net" % "2.2",
- "net.java.dev.jets3t" % "jets3t" % "0.7.1" excludeAll(excludeCommonsLogging),
+ "net.java.dev.jets3t" % "jets3t" % jets3tVersion excludeAll(excludeCommonsLogging),
"org.apache.derby" % "derby" % "10.4.2.0" % "test",
"org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeCommonsLogging, excludeSLF4J, excludeOldAsm),
"org.apache.curator" % "curator-recipes" % "2.4.0" excludeAll(excludeNetty),
@@ -354,7 +356,8 @@ object SparkBuild extends Build {
"com.twitter" % "chill-java" % chillVersion excludeAll(excludeAsm),
"org.tachyonproject" % "tachyon" % "0.4.1-thrift" excludeAll(excludeHadoop, excludeCurator, excludeEclipseJetty, excludePowermock),
"com.clearspring.analytics" % "stream" % "2.5.1" excludeAll(excludeFastutil),
- "org.spark-project" % "pyrolite" % "2.0.1"
+ "org.spark-project" % "pyrolite" % "2.0.1",
+ "net.sf.py4j" % "py4j" % "0.8.1"
),
libraryDependencies ++= maybeAvro
)
@@ -561,12 +564,12 @@ object SparkBuild extends Build {
"org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm),
"org.apache.hadoop" % "hadoop-yarn-api" % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm),
"org.apache.hadoop" % "hadoop-yarn-common" % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm),
- "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm)
+ "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm),
+ "org.apache.hadoop" % "hadoop-yarn-server-web-proxy" % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm)
)
)
def assemblyProjSettings = sharedSettings ++ Seq(
- libraryDependencies += "net.sf.py4j" % "py4j" % "0.8.1",
name := "spark-assembly",
assembleDeps in Compile <<= (packageProjects.map(packageBin in Compile in _) ++ Seq(packageDependency in Compile)).dependOn,
jarName in assembly <<= version map { v => "spark-assembly-" + v + "-hadoop" + hadoopVersion + ".jar" },
diff --git a/python/.gitignore b/python/.gitignore
index 5c56e638f923a..80b361ffbd51c 100644
--- a/python/.gitignore
+++ b/python/.gitignore
@@ -1,2 +1,5 @@
*.pyc
docs/
+pyspark.egg-info
+build/
+dist/
diff --git a/python/lib/PY4J_VERSION.txt b/python/lib/PY4J_VERSION.txt
deleted file mode 100644
index 04a0cd52a8d9c..0000000000000
--- a/python/lib/PY4J_VERSION.txt
+++ /dev/null
@@ -1 +0,0 @@
-b7924aabe9c5e63f0a4d8bbd17019534c7ec014e
diff --git a/python/pyspark/__init__.py b/python/pyspark/__init__.py
index 73fe7378ffa63..07df8697bd1a8 100644
--- a/python/pyspark/__init__.py
+++ b/python/pyspark/__init__.py
@@ -49,13 +49,6 @@
Main entry point for accessing data stored in Apache Hive..
"""
-
-
-import sys
-import os
-sys.path.insert(0, os.path.join(os.environ["SPARK_HOME"], "python/lib/py4j-0.8.1-src.zip"))
-
-
from pyspark.conf import SparkConf
from pyspark.context import SparkContext
from pyspark.sql import SQLContext
diff --git a/python/pyspark/context.py b/python/pyspark/context.py
index c74dc5fd4f854..cac133d0fcf6c 100644
--- a/python/pyspark/context.py
+++ b/python/pyspark/context.py
@@ -158,6 +158,12 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None,
for path in (pyFiles or []):
self.addPyFile(path)
+ # Deploy code dependencies set by spark-submit; these will already have been added
+ # with SparkContext.addFile, so we just need to add them
+ for path in self._conf.get("spark.submit.pyFiles", "").split(","):
+ if path != "":
+ self._python_includes.append(os.path.basename(path))
+
# Create a temporary directory inside spark.local.dir:
local_dir = self._jvm.org.apache.spark.util.Utils.getLocalDir(self._jsc.sc().conf())
self._temp_dir = \
@@ -447,7 +453,7 @@ def setJobGroup(self, groupId, description, interruptOnCancel=False):
>>> lock = threading.Lock()
>>> def map_func(x):
... sleep(100)
- ... return x * x
+ ... raise Exception("Task should have been cancelled")
>>> def start_job(x):
... global result
... try:
diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py
index eb18ec08c9139..b2f226a55ec13 100644
--- a/python/pyspark/daemon.py
+++ b/python/pyspark/daemon.py
@@ -74,6 +74,17 @@ def handle_sigchld(*args):
raise
signal.signal(SIGCHLD, handle_sigchld)
+ # Blocks until the socket is closed by draining the input stream
+ # until it raises an exception or returns EOF.
+ def waitSocketClose(sock):
+ try:
+ while True:
+ # Empty string is returned upon EOF (and only then).
+ if sock.recv(4096) == '':
+ return
+ except:
+ pass
+
# Handle clients
while not should_exit():
# Wait until a client arrives or we have to exit
@@ -105,7 +116,8 @@ def handle_sigchld(*args):
exit_code = exc.code
finally:
outfile.flush()
- sock.close()
+ # The Scala side will close the socket upon task completion.
+ waitSocketClose(sock)
os._exit(compute_real_exit_code(exit_code))
else:
sock.close()
diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py
index 6bb6c877c942d..3d0936fdca911 100644
--- a/python/pyspark/java_gateway.py
+++ b/python/pyspark/java_gateway.py
@@ -24,41 +24,46 @@
from py4j.java_gateway import java_import, JavaGateway, GatewayClient
-SPARK_HOME = os.environ["SPARK_HOME"]
-
-
def launch_gateway():
- # Launch the Py4j gateway using Spark's run command so that we pick up the
- # proper classpath and settings from spark-env.sh
- on_windows = platform.system() == "Windows"
- script = "./bin/spark-class.cmd" if on_windows else "./bin/spark-class"
- command = [os.path.join(SPARK_HOME, script), "py4j.GatewayServer",
- "--die-on-broken-pipe", "0"]
- if not on_windows:
- # Don't send ctrl-c / SIGINT to the Java gateway:
- def preexec_func():
- signal.signal(signal.SIGINT, signal.SIG_IGN)
- proc = Popen(command, stdout=PIPE, stdin=PIPE, preexec_fn=preexec_func)
+ SPARK_HOME = os.environ["SPARK_HOME"]
+
+ gateway_port = -1
+ if "PYSPARK_GATEWAY_PORT" in os.environ:
+ gateway_port = int(os.environ["PYSPARK_GATEWAY_PORT"])
else:
- # preexec_fn not supported on Windows
- proc = Popen(command, stdout=PIPE, stdin=PIPE)
- # Determine which ephemeral port the server started on:
- port = int(proc.stdout.readline())
- # Create a thread to echo output from the GatewayServer, which is required
- # for Java log output to show up:
- class EchoOutputThread(Thread):
- def __init__(self, stream):
- Thread.__init__(self)
- self.daemon = True
- self.stream = stream
+ # Launch the Py4j gateway using Spark's run command so that we pick up the
+ # proper classpath and settings from spark-env.sh
+ on_windows = platform.system() == "Windows"
+ script = "./bin/spark-class.cmd" if on_windows else "./bin/spark-class"
+ command = [os.path.join(SPARK_HOME, script), "py4j.GatewayServer",
+ "--die-on-broken-pipe", "0"]
+ if not on_windows:
+ # Don't send ctrl-c / SIGINT to the Java gateway:
+ def preexec_func():
+ signal.signal(signal.SIGINT, signal.SIG_IGN)
+ proc = Popen(command, stdout=PIPE, stdin=PIPE, preexec_fn=preexec_func)
+ else:
+ # preexec_fn not supported on Windows
+ proc = Popen(command, stdout=PIPE, stdin=PIPE)
+ # Determine which ephemeral port the server started on:
+ gateway_port = int(proc.stdout.readline())
+ # Create a thread to echo output from the GatewayServer, which is required
+ # for Java log output to show up:
+ class EchoOutputThread(Thread):
+ def __init__(self, stream):
+ Thread.__init__(self)
+ self.daemon = True
+ self.stream = stream
+
+ def run(self):
+ while True:
+ line = self.stream.readline()
+ sys.stderr.write(line)
+ EchoOutputThread(proc.stdout).start()
- def run(self):
- while True:
- line = self.stream.readline()
- sys.stderr.write(line)
- EchoOutputThread(proc.stdout).start()
# Connect to the gateway
- gateway = JavaGateway(GatewayClient(port=port), auto_convert=False)
+ gateway = JavaGateway(GatewayClient(port=gateway_port), auto_convert=False)
+
# Import the classes used by PySpark
java_import(gateway.jvm, "org.apache.spark.SparkConf")
java_import(gateway.jvm, "org.apache.spark.api.java.*")
@@ -69,4 +74,5 @@ def run(self):
java_import(gateway.jvm, "org.apache.spark.sql.hive.LocalHiveContext")
java_import(gateway.jvm, "org.apache.spark.sql.hive.TestHiveContext")
java_import(gateway.jvm, "scala.Tuple2")
+
return gateway
diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py
index c5844597c95f2..6772e4337ef39 100644
--- a/python/pyspark/mllib/classification.py
+++ b/python/pyspark/mllib/classification.py
@@ -90,7 +90,7 @@ class SVMModel(LinearModel):
>>> svm.predict(array([1.0])) > 0
True
>>> sparse_data = [
- ... LabeledPoint(0.0, SparseVector(2, {0: 0.0})),
+ ... LabeledPoint(0.0, SparseVector(2, {0: -1.0})),
... LabeledPoint(1.0, SparseVector(2, {1: 1.0})),
... LabeledPoint(0.0, SparseVector(2, {0: 0.0})),
... LabeledPoint(1.0, SparseVector(2, {1: 2.0}))
@@ -98,7 +98,7 @@ class SVMModel(LinearModel):
>>> svm = SVMWithSGD.train(sc.parallelize(sparse_data))
>>> svm.predict(SparseVector(2, {1: 1.0})) > 0
True
- >>> svm.predict(SparseVector(2, {1: 0.0})) <= 0
+ >>> svm.predict(SparseVector(2, {0: -1.0})) <= 0
True
"""
def predict(self, x):
diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py
index a59778c72130e..3a1c56af5b221 100644
--- a/python/pyspark/rdd.py
+++ b/python/pyspark/rdd.py
@@ -537,8 +537,8 @@ def pipe(self, command, env={}):
"""
Return an RDD created by piping elements to a forked external process.
- >>> sc.parallelize([1, 2, 3]).pipe('cat').collect()
- ['1', '2', '3']
+ >>> sc.parallelize(['1', '2', '', '3']).pipe('cat').collect()
+ ['1', '2', '', '3']
"""
def func(iterator):
pipe = Popen(shlex.split(command), env=env, stdin=PIPE, stdout=PIPE)
@@ -547,7 +547,7 @@ def pipe_objs(out):
out.write(str(obj).rstrip('\n') + '\n')
out.close()
Thread(target=pipe_objs, args=[pipe.stdin]).start()
- return (x.rstrip('\n') for x in pipe.stdout)
+ return (x.rstrip('\n') for x in iter(pipe.stdout.readline, ''))
return self.mapPartitions(func)
def foreach(self, f):
diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py
index 1a62031db5c41..6789d7002b3b7 100644
--- a/python/pyspark/sql.py
+++ b/python/pyspark/sql.py
@@ -360,6 +360,35 @@ def getCheckpointFile(self):
else:
return None
+ def coalesce(self, numPartitions, shuffle=False):
+ rdd = self._jschema_rdd.coalesce(numPartitions, shuffle)
+ return SchemaRDD(rdd, self.sql_ctx)
+
+ def distinct(self):
+ rdd = self._jschema_rdd.distinct()
+ return SchemaRDD(rdd, self.sql_ctx)
+
+ def intersection(self, other):
+ if (other.__class__ is SchemaRDD):
+ rdd = self._jschema_rdd.intersection(other._jschema_rdd)
+ return SchemaRDD(rdd, self.sql_ctx)
+ else:
+ raise ValueError("Can only intersect with another SchemaRDD")
+
+ def repartition(self, numPartitions):
+ rdd = self._jschema_rdd.repartition(numPartitions)
+ return SchemaRDD(rdd, self.sql_ctx)
+
+ def subtract(self, other, numPartitions=None):
+ if (other.__class__ is SchemaRDD):
+ if numPartitions is None:
+ rdd = self._jschema_rdd.subtract(other._jschema_rdd)
+ else:
+ rdd = self._jschema_rdd.subtract(other._jschema_rdd, numPartitions)
+ return SchemaRDD(rdd, self.sql_ctx)
+ else:
+ raise ValueError("Can only subtract another SchemaRDD")
+
def _test():
import doctest
from pyspark.context import SparkContext
diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py
index 527104587fd31..64f2eeb12b4fc 100644
--- a/python/pyspark/tests.py
+++ b/python/pyspark/tests.py
@@ -22,18 +22,23 @@
from fileinput import input
from glob import glob
import os
+import re
import shutil
+import subprocess
import sys
-from tempfile import NamedTemporaryFile
+import tempfile
import time
import unittest
+import zipfile
from pyspark.context import SparkContext
from pyspark.files import SparkFiles
-from pyspark.java_gateway import SPARK_HOME
from pyspark.serializers import read_int
+SPARK_HOME = os.environ["SPARK_HOME"]
+
+
class PySparkTestCase(unittest.TestCase):
def setUp(self):
@@ -53,7 +58,7 @@ class TestCheckpoint(PySparkTestCase):
def setUp(self):
PySparkTestCase.setUp(self)
- self.checkpointDir = NamedTemporaryFile(delete=False)
+ self.checkpointDir = tempfile.NamedTemporaryFile(delete=False)
os.unlink(self.checkpointDir.name)
self.sc.setCheckpointDir(self.checkpointDir.name)
@@ -146,7 +151,7 @@ def test_save_as_textfile_with_unicode(self):
# Regression test for SPARK-970
x = u"\u00A1Hola, mundo!"
data = self.sc.parallelize([x])
- tempFile = NamedTemporaryFile(delete=True)
+ tempFile = tempfile.NamedTemporaryFile(delete=True)
tempFile.close()
data.saveAsTextFile(tempFile.name)
raw_contents = ''.join(input(glob(tempFile.name + "/part-0000*")))
@@ -170,7 +175,7 @@ def test_cartesian_on_textfile(self):
def test_deleting_input_files(self):
# Regression test for SPARK-1025
- tempFile = NamedTemporaryFile(delete=False)
+ tempFile = tempfile.NamedTemporaryFile(delete=False)
tempFile.write("Hello World!")
tempFile.close()
data = self.sc.textFile(tempFile.name)
@@ -234,5 +239,125 @@ def test_termination_sigterm(self):
from signal import SIGTERM
self.do_termination_test(lambda daemon: os.kill(daemon.pid, SIGTERM))
+
+class TestSparkSubmit(unittest.TestCase):
+ def setUp(self):
+ self.programDir = tempfile.mkdtemp()
+ self.sparkSubmit = os.path.join(os.environ.get("SPARK_HOME"), "bin", "spark-submit")
+
+ def tearDown(self):
+ shutil.rmtree(self.programDir)
+
+ def createTempFile(self, name, content):
+ """
+ Create a temp file with the given name and content and return its path.
+ Strips leading spaces from content up to the first '|' in each line.
+ """
+ pattern = re.compile(r'^ *\|', re.MULTILINE)
+ content = re.sub(pattern, '', content.strip())
+ path = os.path.join(self.programDir, name)
+ with open(path, "w") as f:
+ f.write(content)
+ return path
+
+ def createFileInZip(self, name, content):
+ """
+ Create a zip archive containing a file with the given content and return its path.
+ Strips leading spaces from content up to the first '|' in each line.
+ """
+ pattern = re.compile(r'^ *\|', re.MULTILINE)
+ content = re.sub(pattern, '', content.strip())
+ path = os.path.join(self.programDir, name + ".zip")
+ with zipfile.ZipFile(path, 'w') as zip:
+ zip.writestr(name, content)
+ return path
+
+ def test_single_script(self):
+ """Submit and test a single script file"""
+ script = self.createTempFile("test.py", """
+ |from pyspark import SparkContext
+ |
+ |sc = SparkContext()
+ |print sc.parallelize([1, 2, 3]).map(lambda x: x * 2).collect()
+ """)
+ proc = subprocess.Popen([self.sparkSubmit, script], stdout=subprocess.PIPE)
+ out, err = proc.communicate()
+ self.assertEqual(0, proc.returncode)
+ self.assertIn("[2, 4, 6]", out)
+
+ def test_script_with_local_functions(self):
+ """Submit and test a single script file calling a global function"""
+ script = self.createTempFile("test.py", """
+ |from pyspark import SparkContext
+ |
+ |def foo(x):
+ | return x * 3
+ |
+ |sc = SparkContext()
+ |print sc.parallelize([1, 2, 3]).map(foo).collect()
+ """)
+ proc = subprocess.Popen([self.sparkSubmit, script], stdout=subprocess.PIPE)
+ out, err = proc.communicate()
+ self.assertEqual(0, proc.returncode)
+ self.assertIn("[3, 6, 9]", out)
+
+ def test_module_dependency(self):
+ """Submit and test a script with a dependency on another module"""
+ script = self.createTempFile("test.py", """
+ |from pyspark import SparkContext
+ |from mylib import myfunc
+ |
+ |sc = SparkContext()
+ |print sc.parallelize([1, 2, 3]).map(myfunc).collect()
+ """)
+ zip = self.createFileInZip("mylib.py", """
+ |def myfunc(x):
+ | return x + 1
+ """)
+ proc = subprocess.Popen([self.sparkSubmit, "--py-files", zip, script],
+ stdout=subprocess.PIPE)
+ out, err = proc.communicate()
+ self.assertEqual(0, proc.returncode)
+ self.assertIn("[2, 3, 4]", out)
+
+ def test_module_dependency_on_cluster(self):
+ """Submit and test a script with a dependency on another module on a cluster"""
+ script = self.createTempFile("test.py", """
+ |from pyspark import SparkContext
+ |from mylib import myfunc
+ |
+ |sc = SparkContext()
+ |print sc.parallelize([1, 2, 3]).map(myfunc).collect()
+ """)
+ zip = self.createFileInZip("mylib.py", """
+ |def myfunc(x):
+ | return x + 1
+ """)
+ proc = subprocess.Popen(
+ [self.sparkSubmit, "--py-files", zip, "--master", "local-cluster[1,1,512]", script],
+ stdout=subprocess.PIPE)
+ out, err = proc.communicate()
+ self.assertEqual(0, proc.returncode)
+ self.assertIn("[2, 3, 4]", out)
+
+ def test_single_script_on_cluster(self):
+ """Submit and test a single script on a cluster"""
+ script = self.createTempFile("test.py", """
+ |from pyspark import SparkContext
+ |
+ |def foo(x):
+ | return x * 2
+ |
+ |sc = SparkContext()
+ |print sc.parallelize([1, 2, 3]).map(foo).collect()
+ """)
+ proc = subprocess.Popen(
+ [self.sparkSubmit, "--master", "local-cluster[1,1,512]", script],
+ stdout=subprocess.PIPE)
+ out, err = proc.communicate()
+ self.assertEqual(0, proc.returncode)
+ self.assertIn("[2, 4, 6]", out)
+
+
if __name__ == "__main__":
unittest.main()
diff --git a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala
index e33f4f9803054..566d96e16ed83 100644
--- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala
+++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala
@@ -45,8 +45,7 @@ class ReplSuite extends FunSuite {
}
val interp = new SparkILoop(in, new PrintWriter(out), master)
org.apache.spark.repl.Main.interp = interp
- val separator = System.getProperty("path.separator")
- interp.process(Array("-classpath", paths.mkString(separator)))
+ interp.process(Array("-classpath", paths.mkString(File.pathSeparator)))
org.apache.spark.repl.Main.interp = null
if (interp.sparkContext != null) {
interp.sparkContext.stop()
diff --git a/sbin/spark-config.sh b/sbin/spark-config.sh
index cd2c7b7b0d496..147b506dd5ca3 100755
--- a/sbin/spark-config.sh
+++ b/sbin/spark-config.sh
@@ -34,3 +34,6 @@ this="$config_bin/$script"
export SPARK_PREFIX=`dirname "$this"`/..
export SPARK_HOME=${SPARK_PREFIX}
export SPARK_CONF_DIR="$SPARK_HOME/conf"
+# Add the PySpark classes to the PYTHONPATH:
+export PYTHONPATH=$SPARK_HOME/python:$PYTHONPATH
+export PYTHONPATH=$SPARK_HOME/python/lib/py4j-0.8.1-src.zip:$PYTHONPATH
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 446d0e0bd7f54..792ef6cee6f5d 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
@@ -44,7 +44,8 @@ object ScalaReflection {
case t if t <:< typeOf[Product] =>
val params = t.member("": TermName).asMethod.paramss
StructType(
- params.head.map(p => StructField(p.name.toString, schemaFor(p.typeSignature), true)))
+ params.head.map(p =>
+ StructField(p.name.toString, schemaFor(p.typeSignature), nullable = 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[_]] =>
@@ -58,6 +59,17 @@ object ScalaReflection {
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 <:< typeOf[Option[_]] =>
+ val TypeRef(_, _, Seq(optType)) = t
+ schemaFor(optType)
+ case t if t <:< typeOf[java.lang.Integer] => IntegerType
+ case t if t <:< typeOf[java.lang.Long] => LongType
+ case t if t <:< typeOf[java.lang.Double] => DoubleType
+ case t if t <:< typeOf[java.lang.Float] => FloatType
+ case t if t <:< typeOf[java.lang.Short] => ShortType
+ case t if t <:< typeOf[java.lang.Byte] => ByteType
+ case t if t <:< typeOf[java.lang.Boolean] => BooleanType
+ // TODO: The following datatypes could be marked as non-nullable.
case t if t <:< definitions.IntTpe => IntegerType
case t if t <:< definitions.LongTpe => LongType
case t if t <:< definitions.DoubleTpe => DoubleType
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 987befe8e22ee..dc83485df195c 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
@@ -114,37 +114,37 @@ package object dsl {
def attr = analysis.UnresolvedAttribute(s)
/** Creates a new AttributeReference of type boolean */
- def boolean = AttributeReference(s, BooleanType, nullable = false)()
+ def boolean = AttributeReference(s, BooleanType, nullable = true)()
/** Creates a new AttributeReference of type byte */
- def byte = AttributeReference(s, ByteType, nullable = false)()
+ def byte = AttributeReference(s, ByteType, nullable = true)()
/** Creates a new AttributeReference of type short */
- def short = AttributeReference(s, ShortType, nullable = false)()
+ def short = AttributeReference(s, ShortType, nullable = true)()
/** Creates a new AttributeReference of type int */
- def int = AttributeReference(s, IntegerType, nullable = false)()
+ def int = AttributeReference(s, IntegerType, nullable = true)()
/** Creates a new AttributeReference of type long */
- def long = AttributeReference(s, LongType, nullable = false)()
+ def long = AttributeReference(s, LongType, nullable = true)()
/** Creates a new AttributeReference of type float */
- def float = AttributeReference(s, FloatType, nullable = false)()
+ def float = AttributeReference(s, FloatType, nullable = true)()
/** Creates a new AttributeReference of type double */
- def double = AttributeReference(s, DoubleType, nullable = false)()
+ def double = AttributeReference(s, DoubleType, nullable = true)()
/** Creates a new AttributeReference of type string */
- def string = AttributeReference(s, StringType, nullable = false)()
+ def string = AttributeReference(s, StringType, nullable = true)()
/** Creates a new AttributeReference of type decimal */
- def decimal = AttributeReference(s, DecimalType, nullable = false)()
+ def decimal = AttributeReference(s, DecimalType, nullable = true)()
/** Creates a new AttributeReference of type timestamp */
- def timestamp = AttributeReference(s, TimestampType, nullable = false)()
+ def timestamp = AttributeReference(s, TimestampType, nullable = true)()
/** Creates a new AttributeReference of type binary */
- def binary = AttributeReference(s, BinaryType, nullable = false)()
+ def binary = AttributeReference(s, BinaryType, nullable = true)()
}
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 40d2b42a0cda3..0b3a4e728ec54 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
@@ -182,7 +182,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression {
case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toFloat(b)
}
- def cast: Any => Any = dataType match {
+ private lazy val cast: Any => Any = dataType match {
case StringType => castToString
case BinaryType => castToBinary
case DecimalType => castToDecimal
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 dd9332ada80dd..41398ff956edd 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
@@ -44,7 +44,6 @@ abstract class Expression extends TreeNode[Expression] {
* - A [[expressions.Cast Cast]] or [[expressions.UnaryMinus UnaryMinus]] is foldable if its
* child is foldable.
*/
- // TODO: Supporting more foldable expressions. For example, deterministic Hive UDFs.
def foldable: Boolean = false
def nullable: Boolean
def references: Set[Attribute]
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala
index 08b2f11d20f5e..d2b7685e73065 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala
@@ -18,6 +18,7 @@
package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.errors.TreeNodeException
+import org.apache.spark.sql.catalyst.trees
abstract sealed class SortDirection
case object Ascending extends SortDirection
@@ -27,7 +28,10 @@ case object Descending extends SortDirection
* An expression that can be used to sort a tuple. This class extends expression primarily so that
* transformations over expression will descend into its child.
*/
-case class SortOrder(child: Expression, direction: SortDirection) extends UnaryExpression {
+case class SortOrder(child: Expression, direction: SortDirection) extends Expression
+ with trees.UnaryNode[Expression] {
+
+ override def references = child.references
override def dataType = child.dataType
override def nullable = child.nullable
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala
index c947155cb701c..195ca2eb3d589 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala
@@ -28,6 +28,7 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression {
val children = child :: ordinal :: Nil
/** `Null` is returned for invalid ordinals. */
override def nullable = true
+ override def foldable = child.foldable && ordinal.foldable
override def references = children.flatMap(_.references).toSet
def dataType = child.dataType match {
case ArrayType(dt) => dt
@@ -40,23 +41,27 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression {
override def toString = s"$child[$ordinal]"
override def eval(input: Row): Any = {
- if (child.dataType.isInstanceOf[ArrayType]) {
- val baseValue = child.eval(input).asInstanceOf[Seq[_]]
- val o = ordinal.eval(input).asInstanceOf[Int]
- if (baseValue == null) {
- null
- } else if (o >= baseValue.size || o < 0) {
- null
- } else {
- baseValue(o)
- }
+ val value = child.eval(input)
+ if (value == null) {
+ null
} else {
- val baseValue = child.eval(input).asInstanceOf[Map[Any, _]]
val key = ordinal.eval(input)
- if (baseValue == null) {
+ if (key == null) {
null
} else {
- baseValue.get(key).orNull
+ if (child.dataType.isInstanceOf[ArrayType]) {
+ val baseValue = value.asInstanceOf[Seq[_]]
+ val o = key.asInstanceOf[Int]
+ if (o >= baseValue.size || o < 0) {
+ null
+ } else {
+ baseValue(o)
+ }
+ } else {
+ val baseValue = value.asInstanceOf[Map[Any, _]]
+ val key = ordinal.eval(input)
+ baseValue.get(key).orNull
+ }
}
}
}
@@ -69,7 +74,8 @@ case class GetField(child: Expression, fieldName: String) extends UnaryExpressio
type EvaluatedType = Any
def dataType = field.dataType
- def nullable = field.nullable
+ override def nullable = field.nullable
+ override def foldable = child.foldable
protected def structType = child.dataType match {
case s: StructType => s
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 82c7af684459f..6ee479939d25c 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
@@ -65,8 +65,7 @@ abstract class BinaryPredicate extends BinaryExpression with Predicate {
def nullable = left.nullable || right.nullable
}
-case class Not(child: Expression) extends Predicate with trees.UnaryNode[Expression] {
- def references = child.references
+case class Not(child: Expression) extends UnaryExpression with Predicate {
override def foldable = child.foldable
def nullable = child.nullable
override def toString = s"NOT $child"
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
index c0a09a16ac98d..3037d45cc6e35 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
@@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.types._
object Optimizer extends RuleExecutor[LogicalPlan] {
val batches =
Batch("ConstantFolding", Once,
+ NullPropagation,
ConstantFolding,
BooleanSimplification,
SimplifyFilters,
@@ -85,6 +86,72 @@ object ColumnPruning extends Rule[LogicalPlan] {
}
}
+/**
+ * Replaces [[catalyst.expressions.Expression Expressions]] that can be statically evaluated with
+ * equivalent [[catalyst.expressions.Literal Literal]] values. This rule is more specific with
+ * Null value propagation from bottom to top of the expression tree.
+ */
+object NullPropagation extends Rule[LogicalPlan] {
+ def apply(plan: LogicalPlan): LogicalPlan = plan transform {
+ case q: LogicalPlan => q transformExpressionsUp {
+ case e @ Count(Literal(null, _)) => Literal(0, e.dataType)
+ case e @ Sum(Literal(c, _)) if c == 0 => Literal(0, e.dataType)
+ case e @ Average(Literal(c, _)) if c == 0 => Literal(0.0, e.dataType)
+ case e @ IsNull(c) if c.nullable == false => Literal(false, BooleanType)
+ case e @ IsNotNull(c) if c.nullable == false => Literal(true, BooleanType)
+ case e @ GetItem(Literal(null, _), _) => Literal(null, e.dataType)
+ case e @ GetItem(_, Literal(null, _)) => Literal(null, e.dataType)
+ case e @ GetField(Literal(null, _), _) => Literal(null, e.dataType)
+ case e @ Coalesce(children) => {
+ val newChildren = children.filter(c => c match {
+ case Literal(null, _) => false
+ case _ => true
+ })
+ if (newChildren.length == 0) {
+ Literal(null, e.dataType)
+ } else if (newChildren.length == 1) {
+ newChildren(0)
+ } else {
+ Coalesce(newChildren)
+ }
+ }
+ case e @ If(Literal(v, _), trueValue, falseValue) => if (v == true) trueValue else falseValue
+ case e @ In(Literal(v, _), list) if (list.exists(c => c match {
+ case Literal(candidate, _) if candidate == v => true
+ case _ => false
+ })) => Literal(true, BooleanType)
+ case e: UnaryMinus => e.child match {
+ case Literal(null, _) => Literal(null, e.dataType)
+ case _ => e
+ }
+ case e: Cast => e.child match {
+ case Literal(null, _) => Literal(null, e.dataType)
+ case _ => e
+ }
+ case e: Not => e.child match {
+ case Literal(null, _) => Literal(null, e.dataType)
+ case _ => e
+ }
+ // Put exceptional cases above if any
+ case e: BinaryArithmetic => e.children match {
+ case Literal(null, _) :: right :: Nil => Literal(null, e.dataType)
+ case left :: Literal(null, _) :: Nil => Literal(null, e.dataType)
+ case _ => e
+ }
+ case e: BinaryComparison => e.children match {
+ case Literal(null, _) :: right :: Nil => Literal(null, e.dataType)
+ case left :: Literal(null, _) :: Nil => Literal(null, e.dataType)
+ case _ => e
+ }
+ case e: StringRegexExpression => e.children match {
+ case Literal(null, _) :: right :: Nil => Literal(null, e.dataType)
+ case left :: Literal(null, _) :: Nil => Literal(null, e.dataType)
+ case _ => e
+ }
+ }
+ }
+}
+
/**
* Replaces [[catalyst.expressions.Expression Expressions]] that can be statically evaluated with
* equivalent [[catalyst.expressions.Literal Literal]] values.
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala
index 9db96f89dd03c..e32adb76fe146 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala
@@ -50,18 +50,18 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging {
var curPlan = plan
batches.foreach { batch =>
+ val batchStartPlan = curPlan
var iteration = 1
var lastPlan = curPlan
- curPlan = batch.rules.foldLeft(curPlan) { case (plan, rule) => rule(plan) }
+ var continue = true
// Run until fix point (or the max number of iterations as specified in the strategy.
- while (iteration < batch.strategy.maxIterations && !curPlan.fastEquals(lastPlan)) {
- lastPlan = curPlan
+ while (continue) {
curPlan = batch.rules.foldLeft(curPlan) {
case (plan, rule) =>
val result = rule(plan)
if (!result.fastEquals(plan)) {
- logger.debug(
+ logger.trace(
s"""
|=== Applying Rule ${rule.ruleName} ===
|${sideBySide(plan.treeString, result.treeString).mkString("\n")}
@@ -71,6 +71,26 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging {
result
}
iteration += 1
+ if (iteration > batch.strategy.maxIterations) {
+ logger.info(s"Max iterations ($iteration) reached for batch ${batch.name}")
+ continue = false
+ }
+
+ if (curPlan.fastEquals(lastPlan)) {
+ logger.trace(s"Fixed point reached for batch ${batch.name} after $iteration iterations.")
+ continue = false
+ }
+ lastPlan = curPlan
+ }
+
+ if (!batchStartPlan.fastEquals(curPlan)) {
+ logger.debug(
+ s"""
+ |=== Result of Batch ${batch.name} ===
+ |${sideBySide(plan.treeString, curPlan.treeString).mkString("\n")}
+ """.stripMargin)
+ } else {
+ logger.trace(s"Batch ${batch.name} has no effect.")
}
}
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 d287ad73b9e9f..91605d0a260e5 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
@@ -108,9 +108,7 @@ class ExpressionEvaluationSuite extends FunSuite {
truthTable.foreach {
case (l,r,answer) =>
val expr = op(Literal(l, BooleanType), Literal(r, BooleanType))
- val result = expr.eval(null)
- if (result != answer)
- fail(s"$expr should not evaluate to $result, expected: $answer")
+ checkEvaluation(expr, answer)
}
}
}
@@ -131,6 +129,7 @@ class ExpressionEvaluationSuite extends FunSuite {
test("LIKE literal Regular Expression") {
checkEvaluation(Literal(null, StringType).like("a"), null)
+ checkEvaluation(Literal("a", StringType).like(Literal(null, StringType)), null)
checkEvaluation(Literal(null, StringType).like(Literal(null, StringType)), null)
checkEvaluation("abdef" like "abdef", true)
checkEvaluation("a_%b" like "a\\__b", true)
@@ -159,9 +158,14 @@ class ExpressionEvaluationSuite extends FunSuite {
checkEvaluation("abc" like regEx, true, new GenericRow(Array[Any]("a%")))
checkEvaluation("abc" like regEx, false, new GenericRow(Array[Any]("b%")))
checkEvaluation("abc" like regEx, false, new GenericRow(Array[Any]("bc%")))
+
+ checkEvaluation(Literal(null, StringType) like regEx, null, new GenericRow(Array[Any]("bc%")))
}
test("RLIKE literal Regular Expression") {
+ checkEvaluation(Literal(null, StringType) rlike "abdef", null)
+ checkEvaluation("abdef" rlike Literal(null, StringType), null)
+ checkEvaluation(Literal(null, StringType) rlike Literal(null, StringType), null)
checkEvaluation("abdef" rlike "abdef", true)
checkEvaluation("abbbbc" rlike "a.*c", true)
@@ -257,6 +261,8 @@ class ExpressionEvaluationSuite extends FunSuite {
assert(("abcdef" cast DecimalType).nullable === true)
assert(("abcdef" cast DoubleType).nullable === true)
assert(("abcdef" cast FloatType).nullable === true)
+
+ checkEvaluation(Cast(Literal(null, IntegerType), ShortType), null)
}
test("timestamp") {
@@ -287,5 +293,108 @@ class ExpressionEvaluationSuite extends FunSuite {
// A test for higher precision than millis
checkEvaluation(Cast(Cast(0.00000001, TimestampType), DoubleType), 0.00000001)
}
+
+ test("null checking") {
+ val row = new GenericRow(Array[Any]("^Ba*n", null, true, null))
+ val c1 = 'a.string.at(0)
+ val c2 = 'a.string.at(1)
+ val c3 = 'a.boolean.at(2)
+ val c4 = 'a.boolean.at(3)
+
+ checkEvaluation(IsNull(c1), false, row)
+ checkEvaluation(IsNotNull(c1), true, row)
+
+ checkEvaluation(IsNull(c2), true, row)
+ checkEvaluation(IsNotNull(c2), false, row)
+
+ checkEvaluation(IsNull(Literal(1, ShortType)), false)
+ checkEvaluation(IsNotNull(Literal(1, ShortType)), true)
+
+ checkEvaluation(IsNull(Literal(null, ShortType)), true)
+ checkEvaluation(IsNotNull(Literal(null, ShortType)), false)
+
+ checkEvaluation(Coalesce(c1 :: c2 :: Nil), "^Ba*n", row)
+ checkEvaluation(Coalesce(Literal(null, StringType) :: Nil), null, row)
+ checkEvaluation(Coalesce(Literal(null, StringType) :: c1 :: c2 :: Nil), "^Ba*n", row)
+
+ checkEvaluation(If(c3, Literal("a", StringType), Literal("b", StringType)), "a", row)
+ checkEvaluation(If(c3, c1, c2), "^Ba*n", row)
+ checkEvaluation(If(c4, c2, c1), "^Ba*n", row)
+ checkEvaluation(If(Literal(null, BooleanType), c2, c1), "^Ba*n", row)
+ checkEvaluation(If(Literal(true, BooleanType), c1, c2), "^Ba*n", row)
+ checkEvaluation(If(Literal(false, BooleanType), c2, c1), "^Ba*n", row)
+ checkEvaluation(If(Literal(false, BooleanType),
+ Literal("a", StringType), Literal("b", StringType)), "b", row)
+
+ checkEvaluation(In(c1, c1 :: c2 :: Nil), true, row)
+ checkEvaluation(In(Literal("^Ba*n", StringType),
+ Literal("^Ba*n", StringType) :: Nil), true, row)
+ checkEvaluation(In(Literal("^Ba*n", StringType),
+ Literal("^Ba*n", StringType) :: c2 :: Nil), true, row)
+ }
+
+ test("complex type") {
+ val row = new GenericRow(Array[Any](
+ "^Ba*n", // 0
+ null.asInstanceOf[String], // 1
+ new GenericRow(Array[Any]("aa", "bb")), // 2
+ Map("aa"->"bb"), // 3
+ Seq("aa", "bb") // 4
+ ))
+
+ val typeS = StructType(
+ StructField("a", StringType, true) :: StructField("b", StringType, true) :: Nil
+ )
+ val typeMap = MapType(StringType, StringType)
+ val typeArray = ArrayType(StringType)
+
+ checkEvaluation(GetItem(BoundReference(3, AttributeReference("c", typeMap)()),
+ Literal("aa")), "bb", row)
+ checkEvaluation(GetItem(Literal(null, typeMap), Literal("aa")), null, row)
+ checkEvaluation(GetItem(Literal(null, typeMap), Literal(null, StringType)), null, row)
+ checkEvaluation(GetItem(BoundReference(3, AttributeReference("c", typeMap)()),
+ Literal(null, StringType)), null, row)
+
+ checkEvaluation(GetItem(BoundReference(4, AttributeReference("c", typeArray)()),
+ Literal(1)), "bb", row)
+ checkEvaluation(GetItem(Literal(null, typeArray), Literal(1)), null, row)
+ checkEvaluation(GetItem(Literal(null, typeArray), Literal(null, IntegerType)), null, row)
+ checkEvaluation(GetItem(BoundReference(4, AttributeReference("c", typeArray)()),
+ Literal(null, IntegerType)), null, row)
+
+ checkEvaluation(GetField(BoundReference(2, AttributeReference("c", typeS)()), "a"), "aa", row)
+ checkEvaluation(GetField(Literal(null, typeS), "a"), null, row)
+ }
+
+ test("arithmetic") {
+ val row = new GenericRow(Array[Any](1, 2, 3, null))
+ val c1 = 'a.int.at(0)
+ val c2 = 'a.int.at(1)
+ val c3 = 'a.int.at(2)
+ val c4 = 'a.int.at(3)
+
+ checkEvaluation(UnaryMinus(c1), -1, row)
+ checkEvaluation(UnaryMinus(Literal(100, IntegerType)), -100)
+
+ checkEvaluation(Add(c1, c4), null, row)
+ checkEvaluation(Add(c1, c2), 3, row)
+ checkEvaluation(Add(c1, Literal(null, IntegerType)), null, row)
+ checkEvaluation(Add(Literal(null, IntegerType), c2), null, row)
+ checkEvaluation(Add(Literal(null, IntegerType), Literal(null, IntegerType)), null, row)
+ }
+
+ test("BinaryComparison") {
+ val row = new GenericRow(Array[Any](1, 2, 3, null))
+ val c1 = 'a.int.at(0)
+ val c2 = 'a.int.at(1)
+ val c3 = 'a.int.at(2)
+ val c4 = 'a.int.at(3)
+
+ checkEvaluation(LessThan(c1, c4), null, row)
+ checkEvaluation(LessThan(c1, c2), true, row)
+ checkEvaluation(LessThan(c1, Literal(null, IntegerType)), null, row)
+ checkEvaluation(LessThan(Literal(null, IntegerType), c2), null, row)
+ checkEvaluation(LessThan(Literal(null, IntegerType), Literal(null, IntegerType)), null, row)
+ }
}
diff --git a/mllib/src/test/scala/org/apache/spark/mllib/rdd/VectorRDDsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala
similarity index 51%
rename from mllib/src/test/scala/org/apache/spark/mllib/rdd/VectorRDDsSuite.scala
rename to sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala
index 692f025e959ae..890d6289b9dfb 100644
--- a/mllib/src/test/scala/org/apache/spark/mllib/rdd/VectorRDDsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala
@@ -15,19 +15,22 @@
* limitations under the License.
*/
-package org.apache.spark.mllib.rdd
+package org.apache.spark.sql.catalyst.optimizer
-import org.scalatest.FunSuite
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.logical._
-import org.apache.spark.mllib.linalg.Vectors
-import org.apache.spark.mllib.util.LocalSparkContext
-
-class VectorRDDsSuite extends FunSuite with LocalSparkContext {
-
- test("from array rdd") {
- val data = Seq(Array(1.0, 2.0), Array(3.0, 4.0))
- val arrayRdd = sc.parallelize(data, 2)
- val vectorRdd = VectorRDDs.fromArrayRDD(arrayRdd)
- assert(arrayRdd.collect().map(v => Vectors.dense(v)) === vectorRdd.collect())
+/**
+ * Overrides our expression evaluation tests and reruns them after optimization has occured. This
+ * is to ensure that constant folding and other optimizations do not break anything.
+ */
+class ExpressionOptimizationSuite extends ExpressionEvaluationSuite {
+ override def checkEvaluation(
+ expression: Expression,
+ expected: Any,
+ inputRow: Row = EmptyRow): Unit = {
+ val plan = Project(Alias(expression, s"Optimized($expression)")() :: Nil, NoRelation)
+ val optimizedPlan = Optimizer(plan)
+ super.checkEvaluation(optimizedPlan.expressions.head, expected, inputRow)
}
-}
+}
\ No newline at end of file
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 e25201a6c1775..bfebfa0c28c52 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
@@ -162,8 +162,11 @@ class SQLContext(@transient val sparkContext: SparkContext)
/** Caches the specified table in-memory. */
def cacheTable(tableName: String): Unit = {
val currentTable = catalog.lookupRelation(None, tableName)
+ val useCompression =
+ sparkContext.conf.getBoolean("spark.sql.inMemoryColumnarStorage.compressed", false)
val asInMemoryRelation =
- InMemoryColumnarTableScan(currentTable.output, executePlan(currentTable).executedPlan)
+ InMemoryColumnarTableScan(
+ currentTable.output, executePlan(currentTable).executedPlan, useCompression)
catalog.registerTable(None, tableName, SparkLogicalPlan(asInMemoryRelation))
}
@@ -173,7 +176,7 @@ class SQLContext(@transient val sparkContext: SparkContext)
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)) =>
+ case SparkLogicalPlan(inMem @ InMemoryColumnarTableScan(_, e: ExistingRdd, _)) =>
inMem.cachedColumnBuffers.unpersist()
catalog.unregisterTable(None, tableName)
catalog.registerTable(None, tableName, SparkLogicalPlan(e))
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 ca6e0a696405a..34200be3ac955 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
@@ -19,14 +19,16 @@ package org.apache.spark.sql
import net.razorvine.pickle.Pickler
-import org.apache.spark.{Dependency, OneToOneDependency, Partition, TaskContext}
+import org.apache.spark.{Dependency, OneToOneDependency, Partition, Partitioner, TaskContext}
import org.apache.spark.annotation.{AlphaComponent, Experimental}
import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.api.java.JavaSchemaRDD
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.sql.execution.{ExistingRdd, SparkLogicalPlan}
import org.apache.spark.api.java.JavaRDD
import java.util.{Map => JMap}
@@ -133,7 +135,7 @@ class SchemaRDD(
new SchemaRDD(sqlContext, Project(exprs, logicalPlan))
/**
- * Filters the ouput, only returning those rows where `condition` evaluates to true.
+ * Filters the output, only returning those rows where `condition` evaluates to true.
*
* {{{
* schemaRDD.where('a === 'b)
@@ -151,9 +153,9 @@ class SchemaRDD(
*
* @param otherPlan the [[SchemaRDD]] that should be joined with this one.
* @param joinType One of `Inner`, `LeftOuter`, `RightOuter`, or `FullOuter`. Defaults to `Inner.`
- * @param on An optional condition for the join operation. This is equivilent to the `ON`
+ * @param on An optional condition for the join operation. This is equivalent to the `ON`
* clause in standard SQL. In the case of `Inner` joins, specifying a
- * `condition` is equivilent to adding `where` clauses after the `join`.
+ * `condition` is equivalent to adding `where` clauses after the `join`.
*
* @group Query
*/
@@ -195,7 +197,7 @@ class SchemaRDD(
/**
* Applies a qualifier to the attributes of this relation. Can be used to disambiguate attributes
- * with the same name, for example, when peforming self-joins.
+ * with the same name, for example, when performing self-joins.
*
* {{{
* val x = schemaRDD.where('a === 1).as('x)
@@ -296,6 +298,13 @@ class SchemaRDD(
*/
def toSchemaRDD = this
+ /**
+ * Returns this RDD as a JavaSchemaRDD.
+ *
+ * @group schema
+ */
+ def toJavaSchemaRDD: JavaSchemaRDD = new JavaSchemaRDD(sqlContext, logicalPlan)
+
private[sql] def javaToPython: JavaRDD[Array[Byte]] = {
val fieldNames: Seq[String] = this.queryExecution.analyzed.output.map(_.name)
this.mapPartitions { iter =>
@@ -314,4 +323,60 @@ class SchemaRDD(
}
}
}
+
+ /**
+ * Creates SchemaRDD by applying own schema to derived RDD. Typically used to wrap return value
+ * of base RDD functions that do not change schema.
+ *
+ * @param rdd RDD derived from this one and has same schema
+ *
+ * @group schema
+ */
+ private def applySchema(rdd: RDD[Row]): SchemaRDD = {
+ new SchemaRDD(sqlContext, SparkLogicalPlan(ExistingRdd(logicalPlan.output, rdd)))
+ }
+
+ // =======================================================================
+ // Base RDD functions that do NOT change schema
+ // =======================================================================
+
+ // Transformations (return a new RDD)
+
+ override def coalesce(numPartitions: Int, shuffle: Boolean = false)
+ (implicit ord: Ordering[Row] = null): SchemaRDD =
+ applySchema(super.coalesce(numPartitions, shuffle)(ord))
+
+ override def distinct(): SchemaRDD =
+ applySchema(super.distinct())
+
+ override def distinct(numPartitions: Int)
+ (implicit ord: Ordering[Row] = null): SchemaRDD =
+ applySchema(super.distinct(numPartitions)(ord))
+
+ override def filter(f: Row => Boolean): SchemaRDD =
+ applySchema(super.filter(f))
+
+ override def intersection(other: RDD[Row]): SchemaRDD =
+ applySchema(super.intersection(other))
+
+ override def intersection(other: RDD[Row], partitioner: Partitioner)
+ (implicit ord: Ordering[Row] = null): SchemaRDD =
+ applySchema(super.intersection(other, partitioner)(ord))
+
+ override def intersection(other: RDD[Row], numPartitions: Int): SchemaRDD =
+ applySchema(super.intersection(other, numPartitions))
+
+ override def repartition(numPartitions: Int)
+ (implicit ord: Ordering[Row] = null): SchemaRDD =
+ applySchema(super.repartition(numPartitions)(ord))
+
+ override def subtract(other: RDD[Row]): SchemaRDD =
+ applySchema(super.subtract(other))
+
+ override def subtract(other: RDD[Row], numPartitions: Int): SchemaRDD =
+ applySchema(super.subtract(other, numPartitions))
+
+ override def subtract(other: RDD[Row], p: Partitioner)
+ (implicit ord: Ordering[Row] = null): SchemaRDD =
+ applySchema(super.subtract(other, p)(ord))
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala
index a7347088794a8..57facbe10fc96 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala
@@ -132,6 +132,14 @@ class JavaSQLContext(sparkContext: JavaSparkContext) {
case c: Class[_] if c == java.lang.Byte.TYPE => ByteType
case c: Class[_] if c == java.lang.Float.TYPE => FloatType
case c: Class[_] if c == java.lang.Boolean.TYPE => BooleanType
+
+ case c: Class[_] if c == classOf[java.lang.Short] => ShortType
+ case c: Class[_] if c == classOf[java.lang.Integer] => IntegerType
+ case c: Class[_] if c == classOf[java.lang.Long] => LongType
+ case c: Class[_] if c == classOf[java.lang.Double] => DoubleType
+ case c: Class[_] if c == classOf[java.lang.Byte] => ByteType
+ case c: Class[_] if c == classOf[java.lang.Float] => FloatType
+ case c: Class[_] if c == classOf[java.lang.Boolean] => BooleanType
}
// TODO: Nullability could be stricter.
AttributeReference(property.getName, dataType, nullable = true)()
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
index d43d672938f51..22f57b758dd02 100644
--- 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
@@ -17,10 +17,13 @@
package org.apache.spark.sql.api.java
+import org.apache.spark.Partitioner
import org.apache.spark.api.java.{JavaRDDLike, JavaRDD}
+import org.apache.spark.api.java.function.{Function => JFunction}
import org.apache.spark.sql.{SQLContext, SchemaRDD, SchemaRDDLike}
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.rdd.RDD
+import org.apache.spark.storage.StorageLevel
/**
* An RDD of [[Row]] objects that is returned as the result of a Spark SQL query. In addition to
@@ -45,4 +48,141 @@ class JavaSchemaRDD(
override def wrapRDD(rdd: RDD[Row]): JavaRDD[Row] = JavaRDD.fromRDD(rdd)
val rdd = baseSchemaRDD.map(new Row(_))
+
+ override def toString: String = baseSchemaRDD.toString
+
+ // =======================================================================
+ // Base RDD functions that do NOT change schema
+ // =======================================================================
+
+ // Common RDD functions
+
+ /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */
+ def cache(): JavaSchemaRDD = {
+ baseSchemaRDD.cache()
+ this
+ }
+
+ /** Persist this RDD with the default storage level (`MEMORY_ONLY`). */
+ def persist(): JavaSchemaRDD = {
+ baseSchemaRDD.persist()
+ this
+ }
+
+ /**
+ * Set this RDD's storage level to persist its values across operations after the first time
+ * it is computed. This can only be used to assign a new storage level if the RDD does not
+ * have a storage level set yet..
+ */
+ def persist(newLevel: StorageLevel): JavaSchemaRDD = {
+ baseSchemaRDD.persist(newLevel)
+ this
+ }
+
+ /**
+ * Mark the RDD as non-persistent, and remove all blocks for it from memory and disk.
+ *
+ * @param blocking Whether to block until all blocks are deleted.
+ * @return This RDD.
+ */
+ def unpersist(blocking: Boolean = true): JavaSchemaRDD = {
+ baseSchemaRDD.unpersist(blocking)
+ this
+ }
+
+ /** Assign a name to this RDD */
+ def setName(name: String): JavaSchemaRDD = {
+ baseSchemaRDD.setName(name)
+ this
+ }
+
+ // Transformations (return a new RDD)
+
+ /**
+ * Return a new RDD that is reduced into `numPartitions` partitions.
+ */
+ def coalesce(numPartitions: Int, shuffle: Boolean = false): JavaSchemaRDD =
+ baseSchemaRDD.coalesce(numPartitions, shuffle).toJavaSchemaRDD
+
+ /**
+ * Return a new RDD containing the distinct elements in this RDD.
+ */
+ def distinct(): JavaSchemaRDD =
+ baseSchemaRDD.distinct().toJavaSchemaRDD
+
+ /**
+ * Return a new RDD containing the distinct elements in this RDD.
+ */
+ def distinct(numPartitions: Int): JavaSchemaRDD =
+ baseSchemaRDD.distinct(numPartitions).toJavaSchemaRDD
+
+ /**
+ * Return a new RDD containing only the elements that satisfy a predicate.
+ */
+ def filter(f: JFunction[Row, java.lang.Boolean]): JavaSchemaRDD =
+ baseSchemaRDD.filter(x => f.call(new Row(x)).booleanValue()).toJavaSchemaRDD
+
+ /**
+ * Return the intersection of this RDD and another one. The output will not contain any
+ * duplicate elements, even if the input RDDs did.
+ *
+ * Note that this method performs a shuffle internally.
+ */
+ def intersection(other: JavaSchemaRDD): JavaSchemaRDD =
+ this.baseSchemaRDD.intersection(other.baseSchemaRDD).toJavaSchemaRDD
+
+ /**
+ * Return the intersection of this RDD and another one. The output will not contain any
+ * duplicate elements, even if the input RDDs did.
+ *
+ * Note that this method performs a shuffle internally.
+ *
+ * @param partitioner Partitioner to use for the resulting RDD
+ */
+ def intersection(other: JavaSchemaRDD, partitioner: Partitioner): JavaSchemaRDD =
+ this.baseSchemaRDD.intersection(other.baseSchemaRDD, partitioner).toJavaSchemaRDD
+
+ /**
+ * Return the intersection of this RDD and another one. The output will not contain any
+ * duplicate elements, even if the input RDDs did. Performs a hash partition across the cluster
+ *
+ * Note that this method performs a shuffle internally.
+ *
+ * @param numPartitions How many partitions to use in the resulting RDD
+ */
+ def intersection(other: JavaSchemaRDD, numPartitions: Int): JavaSchemaRDD =
+ this.baseSchemaRDD.intersection(other.baseSchemaRDD, numPartitions).toJavaSchemaRDD
+
+ /**
+ * Return a new RDD that has exactly `numPartitions` partitions.
+ *
+ * Can increase or decrease the level of parallelism in this RDD. Internally, this uses
+ * a shuffle to redistribute data.
+ *
+ * If you are decreasing the number of partitions in this RDD, consider using `coalesce`,
+ * which can avoid performing a shuffle.
+ */
+ def repartition(numPartitions: Int): JavaSchemaRDD =
+ baseSchemaRDD.repartition(numPartitions).toJavaSchemaRDD
+
+ /**
+ * Return an RDD with the elements from `this` that are not in `other`.
+ *
+ * Uses `this` partitioner/partition size, because even if `other` is huge, the resulting
+ * RDD will be <= us.
+ */
+ def subtract(other: JavaSchemaRDD): JavaSchemaRDD =
+ this.baseSchemaRDD.subtract(other.baseSchemaRDD).toJavaSchemaRDD
+
+ /**
+ * Return an RDD with the elements from `this` that are not in `other`.
+ */
+ def subtract(other: JavaSchemaRDD, numPartitions: Int): JavaSchemaRDD =
+ this.baseSchemaRDD.subtract(other.baseSchemaRDD, numPartitions).toJavaSchemaRDD
+
+ /**
+ * Return an RDD with the elements from `this` that are not in `other`.
+ */
+ def subtract(other: JavaSchemaRDD, p: Partitioner): JavaSchemaRDD =
+ this.baseSchemaRDD.subtract(other.baseSchemaRDD, p).toJavaSchemaRDD
}
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
index 362fe769581d7..9b0dd2176149b 100644
--- 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
@@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.expressions.{Row => ScalaRow}
/**
* A result row from a SparkSQL query.
*/
-class Row(row: ScalaRow) extends Serializable {
+class Row(private[spark] val row: ScalaRow) extends Serializable {
/** Returns the number of columns present in this Row. */
def length: Int = row.length
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 048ee66bff44b..4be048cd742d6 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
@@ -28,7 +28,7 @@ private[sql] trait ColumnBuilder {
/**
* Initializes with an approximate lower bound on the expected number of elements in this column.
*/
- def initialize(initialSize: Int, columnName: String = "")
+ def initialize(initialSize: Int, columnName: String = "", useCompression: Boolean = false)
/**
* Appends `row(ordinal)` to the column builder.
@@ -55,7 +55,11 @@ private[sql] class BasicColumnBuilder[T <: DataType, JvmType](
protected var buffer: ByteBuffer = _
- override def initialize(initialSize: Int, columnName: String = "") = {
+ override def initialize(
+ initialSize: Int,
+ columnName: String = "",
+ useCompression: Boolean = false) = {
+
val size = if (initialSize == 0) DEFAULT_INITIAL_BUFFER_SIZE else initialSize
this.columnName = columnName
@@ -130,7 +134,12 @@ private[sql] object ColumnBuilder {
}
}
- def apply(typeId: Int, initialSize: Int = 0, columnName: String = ""): ColumnBuilder = {
+ def apply(
+ typeId: Int,
+ initialSize: Int = 0,
+ columnName: String = "",
+ useCompression: Boolean = false): ColumnBuilder = {
+
val builder = (typeId match {
case INT.typeId => new IntColumnBuilder
case LONG.typeId => new LongColumnBuilder
@@ -144,7 +153,7 @@ private[sql] object ColumnBuilder {
case GENERIC.typeId => new GenericColumnBuilder
}).asInstanceOf[ColumnBuilder]
- builder.initialize(initialSize, columnName)
+ builder.initialize(initialSize, columnName, useCompression)
builder
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala
index 8a24733047423..fdf28e1bb1261 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala
@@ -20,8 +20,12 @@ package org.apache.spark.sql.columnar
import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, Attribute}
import org.apache.spark.sql.execution.{SparkPlan, LeafNode}
import org.apache.spark.sql.Row
+import org.apache.spark.SparkConf
-private[sql] case class InMemoryColumnarTableScan(attributes: Seq[Attribute], child: SparkPlan)
+private[sql] case class InMemoryColumnarTableScan(
+ attributes: Seq[Attribute],
+ child: SparkPlan,
+ useCompression: Boolean)
extends LeafNode {
override def output: Seq[Attribute] = attributes
@@ -30,7 +34,7 @@ private[sql] case class InMemoryColumnarTableScan(attributes: Seq[Attribute], ch
val output = child.output
val cached = child.execute().mapPartitions { iterator =>
val columnBuilders = output.map { attribute =>
- ColumnBuilder(ColumnType(attribute.dataType).typeId, 0, attribute.name)
+ ColumnBuilder(ColumnType(attribute.dataType).typeId, 0, attribute.name, useCompression)
}.toArray
var row: Row = null
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 2a3b6fc1e46d3..d008806eedbe1 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
@@ -40,12 +40,12 @@ private[sql] trait NullableColumnBuilder extends ColumnBuilder {
private var pos: Int = _
private var nullCount: Int = _
- abstract override def initialize(initialSize: Int, columnName: String) {
+ abstract override def initialize(initialSize: Int, columnName: String, useCompression: Boolean) {
nulls = ByteBuffer.allocate(1024)
nulls.order(ByteOrder.nativeOrder())
pos = 0
nullCount = 0
- super.initialize(initialSize, columnName)
+ super.initialize(initialSize, columnName, useCompression)
}
abstract override def appendFrom(row: Row, ordinal: Int) {
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
index 878cb84de106f..b4120a3d4368b 100644
--- 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
@@ -32,5 +32,7 @@ private[sql] trait CompressibleColumnAccessor[T <: NativeType] extends ColumnAcc
decoder = CompressionScheme(underlyingBuffer.getInt()).decoder(buffer, columnType)
}
- abstract override def extractSingle(buffer: ByteBuffer): T#JvmType = decoder.next()
+ abstract override def hasNext = super.hasNext || decoder.hasNext
+
+ 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
index 0f808f68f2eec..4c6675c3c87bf 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala
@@ -47,7 +47,17 @@ private[sql] trait CompressibleColumnBuilder[T <: NativeType]
import CompressionScheme._
- val compressionEncoders = schemes.filter(_.supports(columnType)).map(_.encoder[T])
+ var compressionEncoders: Seq[Encoder[T]] = _
+
+ abstract override def initialize(initialSize: Int, columnName: String, useCompression: Boolean) {
+ compressionEncoders =
+ if (useCompression) {
+ schemes.filter(_.supports(columnType)).map(_.encoder[T])
+ } else {
+ Seq(PassThrough.encoder)
+ }
+ super.initialize(initialSize, columnName, useCompression)
+ }
protected def isWorthCompressing(encoder: Encoder[T]) = {
encoder.compressionRatio < 0.8
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala
index 800009d3195e1..8cf9ec74ca2de 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala
@@ -157,7 +157,7 @@ private[sql] case object RunLengthEncoding extends CompressionScheme {
currentValue
}
- override def hasNext = buffer.hasRemaining
+ override def hasNext = valueCount < run || buffer.hasRemaining
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala
index 50124dd407447..235a9b1692460 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala
@@ -77,7 +77,7 @@ case class SparkLogicalPlan(alreadyPlanned: SparkPlan)
SparkLogicalPlan(
alreadyPlanned match {
case ExistingRdd(output, rdd) => ExistingRdd(output.map(_.newInstance), rdd)
- case scan @ InMemoryColumnarTableScan(output, child) =>
+ case scan @ InMemoryColumnarTableScan(output, _, _) =>
scan.copy(attributes = output.map(_.newInstance))
case _ => sys.error("Multiple instance of the same relation detected.")
}).asInstanceOf[this.type]
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
index d807187a5ffb8..8969794c69933 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
@@ -164,6 +164,7 @@ case class Sort(
@DeveloperApi
object ExistingRdd {
def convertToCatalyst(a: Any): Any = a match {
+ case o: Option[_] => o.orNull
case s: Seq[Any] => s.map(convertToCatalyst)
case p: Product => new GenericRow(p.productIterator.map(convertToCatalyst).toArray)
case other => other
@@ -180,7 +181,7 @@ object ExistingRdd {
bufferedIterator.map { r =>
var i = 0
while (i < mutableRow.length) {
- mutableRow(i) = r.productElement(i)
+ mutableRow(i) = convertToCatalyst(r.productElement(i))
i += 1
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala
index 1cbf973c34917..f2934da9a031d 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala
@@ -36,6 +36,24 @@ case class ReflectData(
timestampField: Timestamp,
seqInt: Seq[Int])
+case class NullReflectData(
+ intField: java.lang.Integer,
+ longField: java.lang.Long,
+ floatField: java.lang.Float,
+ doubleField: java.lang.Double,
+ shortField: java.lang.Short,
+ byteField: java.lang.Byte,
+ booleanField: java.lang.Boolean)
+
+case class OptionalReflectData(
+ intField: Option[Int],
+ longField: Option[Long],
+ floatField: Option[Float],
+ doubleField: Option[Double],
+ shortField: Option[Short],
+ byteField: Option[Byte],
+ booleanField: Option[Boolean])
+
case class ReflectBinary(data: Array[Byte])
class ScalaReflectionRelationSuite extends FunSuite {
@@ -48,6 +66,22 @@ class ScalaReflectionRelationSuite extends FunSuite {
assert(sql("SELECT * FROM reflectData").collect().head === data.productIterator.toSeq)
}
+ test("query case class RDD with nulls") {
+ val data = NullReflectData(null, null, null, null, null, null, null)
+ val rdd = sparkContext.parallelize(data :: Nil)
+ rdd.registerAsTable("reflectNullData")
+
+ assert(sql("SELECT * FROM reflectNullData").collect().head === Seq.fill(7)(null))
+ }
+
+ test("query case class RDD with Nones") {
+ val data = OptionalReflectData(None, None, None, None, None, None, None)
+ val rdd = sparkContext.parallelize(data :: Nil)
+ rdd.registerAsTable("reflectOptionalData")
+
+ assert(sql("SELECT * FROM reflectOptionalData").collect().head === Seq.fill(7)(null))
+ }
+
// Equality is broken for Arrays, so we test that separately.
test("query binary data") {
val rdd = sparkContext.parallelize(ReflectBinary(Array[Byte](1)) :: Nil)
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala
index 002b7f0adafab..b5973c0f51be8 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala
@@ -73,4 +73,15 @@ object TestData {
ArrayData(Seq(1,2,3), Seq(Seq(1,2,3))) ::
ArrayData(Seq(2,3,4), Seq(Seq(2,3,4))) :: Nil)
arrayData.registerAsTable("arrayData")
+
+ case class StringData(s: String)
+ val repeatedData =
+ TestSQLContext.sparkContext.parallelize(List.fill(2)(StringData("test")))
+ repeatedData.registerAsTable("repeatedData")
+
+ val nullableRepeatedData =
+ TestSQLContext.sparkContext.parallelize(
+ List.fill(2)(StringData(null)) ++
+ List.fill(2)(StringData("test")))
+ nullableRepeatedData.registerAsTable("nullableRepeatedData")
}
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
index def0e046a3831..9fff7222fe840 100644
--- 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
@@ -35,6 +35,17 @@ class PersonBean extends Serializable {
var age: Int = _
}
+class AllTypesBean extends Serializable {
+ @BeanProperty var stringField: String = _
+ @BeanProperty var intField: java.lang.Integer = _
+ @BeanProperty var longField: java.lang.Long = _
+ @BeanProperty var floatField: java.lang.Float = _
+ @BeanProperty var doubleField: java.lang.Double = _
+ @BeanProperty var shortField: java.lang.Short = _
+ @BeanProperty var byteField: java.lang.Byte = _
+ @BeanProperty var booleanField: java.lang.Boolean = _
+}
+
class JavaSQLSuite extends FunSuite {
val javaCtx = new JavaSparkContext(TestSQLContext.sparkContext)
val javaSqlCtx = new JavaSQLContext(javaCtx)
@@ -50,4 +61,54 @@ class JavaSQLSuite extends FunSuite {
schemaRDD.registerAsTable("people")
javaSqlCtx.sql("SELECT * FROM people").collect()
}
+
+ test("all types in JavaBeans") {
+ val bean = new AllTypesBean
+ bean.setStringField("")
+ bean.setIntField(0)
+ bean.setLongField(0)
+ bean.setFloatField(0.0F)
+ bean.setDoubleField(0.0)
+ bean.setShortField(0.toShort)
+ bean.setByteField(0.toByte)
+ bean.setBooleanField(false)
+
+ val rdd = javaCtx.parallelize(bean :: Nil)
+ val schemaRDD = javaSqlCtx.applySchema(rdd, classOf[AllTypesBean])
+ schemaRDD.registerAsTable("allTypes")
+
+ assert(
+ javaSqlCtx.sql(
+ """
+ |SELECT stringField, intField, longField, floatField, doubleField, shortField, byteField,
+ | booleanField
+ |FROM allTypes
+ """.stripMargin).collect.head.row ===
+ Seq("", 0, 0L, 0F, 0.0, 0.toShort, 0.toByte, false))
+ }
+
+ test("all types null in JavaBeans") {
+ val bean = new AllTypesBean
+ bean.setStringField(null)
+ bean.setIntField(null)
+ bean.setLongField(null)
+ bean.setFloatField(null)
+ bean.setDoubleField(null)
+ bean.setShortField(null)
+ bean.setByteField(null)
+ bean.setBooleanField(null)
+
+ val rdd = javaCtx.parallelize(bean :: Nil)
+ val schemaRDD = javaSqlCtx.applySchema(rdd, classOf[AllTypesBean])
+ schemaRDD.registerAsTable("allTypes")
+
+ assert(
+ javaSqlCtx.sql(
+ """
+ |SELECT stringField, intField, longField, floatField, doubleField, shortField, byteField,
+ | booleanField
+ |FROM allTypes
+ """.stripMargin).collect.head.row ===
+ Seq.fill(8)(null))
+ }
}
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 325173cf95fdf..71be41056768f 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
@@ -21,11 +21,12 @@ import java.nio.ByteBuffer
import org.scalatest.FunSuite
+import org.apache.spark.sql.Logging
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 {
+class ColumnTypeSuite extends FunSuite with Logging {
val DEFAULT_BUFFER_SIZE = 512
test("defaultSize") {
@@ -163,7 +164,7 @@ class ColumnTypeSuite extends FunSuite {
buffer.rewind()
seq.foreach { expected =>
- println("buffer = " + buffer + ", expected = " + expected)
+ logger.info("buffer = " + buffer + ", expected = " + expected)
val extracted = columnType.extract(buffer)
assert(
expected === extracted,
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala
index 16a13b8a74960..31c5dfba92954 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala
@@ -28,14 +28,14 @@ class InMemoryColumnarQuerySuite extends QueryTest {
test("simple columnar query") {
val plan = TestSQLContext.executePlan(testData.logicalPlan).executedPlan
- val scan = SparkLogicalPlan(InMemoryColumnarTableScan(plan.output, plan))
+ val scan = SparkLogicalPlan(InMemoryColumnarTableScan(plan.output, plan, true))
checkAnswer(scan, testData.collect().toSeq)
}
test("projection") {
val plan = TestSQLContext.executePlan(testData.select('value, 'key).logicalPlan).executedPlan
- val scan = SparkLogicalPlan(InMemoryColumnarTableScan(plan.output, plan))
+ val scan = SparkLogicalPlan(InMemoryColumnarTableScan(plan.output, plan, true))
checkAnswer(scan, testData.collect().map {
case Row(key: Int, value: String) => value -> key
@@ -44,9 +44,33 @@ class InMemoryColumnarQuerySuite extends QueryTest {
test("SPARK-1436 regression: in-memory columns must be able to be accessed multiple times") {
val plan = TestSQLContext.executePlan(testData.logicalPlan).executedPlan
- val scan = SparkLogicalPlan(InMemoryColumnarTableScan(plan.output, plan))
+ val scan = SparkLogicalPlan(InMemoryColumnarTableScan(plan.output, plan, true))
checkAnswer(scan, testData.collect().toSeq)
checkAnswer(scan, testData.collect().toSeq)
}
+
+ test("SPARK-1678 regression: compression must not lose repeated values") {
+ checkAnswer(
+ sql("SELECT * FROM repeatedData"),
+ repeatedData.collect().toSeq)
+
+ TestSQLContext.cacheTable("repeatedData")
+
+ checkAnswer(
+ sql("SELECT * FROM repeatedData"),
+ repeatedData.collect().toSeq)
+ }
+
+ test("with null values") {
+ checkAnswer(
+ sql("SELECT * FROM nullableRepeatedData"),
+ nullableRepeatedData.collect().toSeq)
+
+ TestSQLContext.cacheTable("nullableRepeatedData")
+
+ checkAnswer(
+ sql("SELECT * FROM nullableRepeatedData"),
+ nullableRepeatedData.collect().toSeq)
+ }
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala
index a754f98f7fbf1..93259a19b9fe7 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala
@@ -72,7 +72,12 @@ class BooleanBitSetSuite extends FunSuite {
buffer.rewind().position(headerSize + 4)
val decoder = BooleanBitSet.decoder(buffer, BOOLEAN)
- values.foreach(expectResult(_, "Wrong decoded value")(decoder.next()))
+ if (values.nonEmpty) {
+ values.foreach {
+ assert(decoder.hasNext)
+ expectResult(_, "Wrong decoded value")(decoder.next())
+ }
+ }
assert(!decoder.hasNext)
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala
index eab27987e08ea..198dcd8819341 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala
@@ -98,8 +98,11 @@ class DictionaryEncodingSuite extends FunSuite {
val decoder = DictionaryEncoding.decoder(buffer, columnType)
- inputSeq.foreach { i =>
- expectResult(values(i), "Wrong decoded value")(decoder.next())
+ if (inputSeq.nonEmpty) {
+ inputSeq.foreach { i =>
+ assert(decoder.hasNext)
+ 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/IntegralDeltaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala
index ce419ca7269ba..46af6e001c633 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala
@@ -96,7 +96,12 @@ class IntegralDeltaSuite extends FunSuite {
buffer.rewind().position(headerSize + 4)
val decoder = scheme.decoder(buffer, columnType)
- input.foreach(expectResult(_, "Wrong decoded value")(decoder.next()))
+ if (input.nonEmpty) {
+ input.foreach{
+ assert(decoder.hasNext)
+ expectResult(_, "Wrong decoded value")(decoder.next())
+ }
+ }
assert(!decoder.hasNext)
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala
index 89f9b60a4397b..d3b73ba19d476 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala
@@ -81,8 +81,11 @@ class RunLengthEncodingSuite extends FunSuite {
val decoder = RunLengthEncoding.decoder(buffer, columnType)
- inputSeq.foreach { i =>
- expectResult(values(i), "Wrong decoded value")(decoder.next())
+ if (inputSeq.nonEmpty) {
+ inputSeq.foreach { i =>
+ assert(decoder.hasNext)
+ 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
index 81bf5e99d19b9..6d688ea95cfc0 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala
@@ -38,7 +38,7 @@ object TestCompressibleColumnBuilder {
scheme: CompressionScheme) = {
val builder = new TestCompressibleColumnBuilder(columnStats, columnType, Seq(scheme))
- builder.initialize(0)
+ builder.initialize(0, "", useCompression = true)
builder
}
}
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 6c907887db79e..ba837a274c51c 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
@@ -130,7 +130,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with
castChildOutput(p, table, child)
case p @ logical.InsertIntoTable(SparkLogicalPlan(InMemoryColumnarTableScan(
- _, HiveTableScan(_, table, _))), _, child, _) =>
+ _, HiveTableScan(_, table, _), _)), _, child, _) =>
castChildOutput(p, table, child)
}
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 1777e96b6713d..1f688fe1117fe 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
@@ -347,7 +347,11 @@ private[hive] object HiveQl {
protected def nodeToPlan(node: Node): LogicalPlan = node match {
// Just fake explain for any of the native commands.
case Token("TOK_EXPLAIN", explainArgs) if nativeCommands contains explainArgs.head.getText =>
- NoRelation
+ ExplainCommand(NoRelation)
+ // Create tables aren't native commands due to CTAS queries, but we still don't need to
+ // explain them.
+ case Token("TOK_EXPLAIN", explainArgs) if explainArgs.head.getText == "TOK_CREATETABLE" =>
+ ExplainCommand(NoRelation)
case Token("TOK_EXPLAIN", explainArgs) =>
// Ignore FORMATTED if present.
val Some(query) :: _ :: _ :: Nil =
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
index d9a6e0e88932e..b2157074a41bf 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
@@ -45,7 +45,7 @@ private[hive] trait HiveStrategies {
case logical.InsertIntoTable(table: MetastoreRelation, partition, child, overwrite) =>
InsertIntoHiveTable(table, partition, planLater(child), overwrite)(hiveContext) :: Nil
case logical.InsertIntoTable(SparkLogicalPlan(InMemoryColumnarTableScan(
- _, HiveTableScan(_, table, _))), partition, child, overwrite) =>
+ _, HiveTableScan(_, table, _), _)), partition, child, overwrite) =>
InsertIntoHiveTable(table, partition, planLater(child), overwrite)(hiveContext) :: Nil
case _ => Nil
}
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
index c7de4ab6d3955..d50e2c65b7b36 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
@@ -22,6 +22,7 @@ import scala.collection.mutable.ArrayBuffer
import org.apache.hadoop.hive.common.`type`.HiveDecimal
import org.apache.hadoop.hive.ql.exec.UDF
import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry}
+import org.apache.hadoop.hive.ql.udf.{UDFType => HiveUDFType}
import org.apache.hadoop.hive.ql.udf.generic._
import org.apache.hadoop.hive.serde2.objectinspector._
import org.apache.hadoop.hive.serde2.objectinspector.primitive._
@@ -237,6 +238,16 @@ private[hive] case class HiveGenericUdf(name: String, children: Seq[Expression])
@transient
protected lazy val returnInspector = function.initialize(argumentInspectors.toArray)
+ @transient
+ protected lazy val isUDFDeterministic = {
+ val udfType = function.getClass().getAnnotation(classOf[HiveUDFType])
+ (udfType != null && udfType.deterministic())
+ }
+
+ override def foldable = {
+ isUDFDeterministic && children.foldLeft(true)((prev, n) => prev && n.foldable)
+ }
+
val dataType: DataType = inspectorToDataType(returnInspector)
override def eval(input: Row): Any = {
diff --git a/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-45f5619d9e4510195fe67f7c8d14a5c0 b/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-45f5619d9e4510195fe67f7c8d14a5c0
new file mode 100644
index 0000000000000..7643569a2c234
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-45f5619d9e4510195fe67f7c8d14a5c0
@@ -0,0 +1,309 @@
+0.0 0 0 0 3
+2.0 0 2 0 1
+4.0 0 4 0 1
+5.0 0 15 0 3
+8.0 0 8 0 1
+9.0 0 9 0 1
+10.0 0 10 0 1
+11.0 0 11 0 1
+12.0 0 24 0 2
+15.0 0 30 0 2
+17.0 0 17 0 1
+18.0 0 36 0 2
+19.0 0 19 0 1
+20.0 0 20 0 1
+24.0 0 48 0 2
+26.0 0 52 0 2
+27.0 0 27 0 1
+28.0 0 28 0 1
+30.0 0 30 0 1
+33.0 0 33 0 1
+34.0 0 34 0 1
+35.0 0 105 0 3
+37.0 0 74 0 2
+41.0 0 41 0 1
+42.0 0 84 0 2
+43.0 0 43 0 1
+44.0 0 44 0 1
+47.0 0 47 0 1
+51.0 0 102 0 2
+53.0 0 53 0 1
+54.0 0 54 0 1
+57.0 0 57 0 1
+58.0 0 116 0 2
+64.0 0 64 0 1
+65.0 0 65 0 1
+66.0 0 66 0 1
+67.0 0 134 0 2
+69.0 0 69 0 1
+70.0 0 210 0 3
+72.0 0 144 0 2
+74.0 0 74 0 1
+76.0 0 152 0 2
+77.0 0 77 0 1
+78.0 0 78 0 1
+80.0 0 80 0 1
+82.0 0 82 0 1
+83.0 0 166 0 2
+84.0 0 168 0 2
+85.0 0 85 0 1
+86.0 0 86 0 1
+87.0 0 87 0 1
+90.0 0 270 0 3
+92.0 0 92 0 1
+95.0 0 190 0 2
+96.0 0 96 0 1
+97.0 0 194 0 2
+98.0 0 196 0 2
+100.0 0 200 0 2
+103.0 0 206 0 2
+104.0 0 208 0 2
+105.0 0 105 0 1
+111.0 0 111 0 1
+113.0 0 226 0 2
+114.0 0 114 0 1
+116.0 0 116 0 1
+118.0 0 236 0 2
+119.0 0 357 0 3
+120.0 0 240 0 2
+125.0 0 250 0 2
+126.0 0 126 0 1
+128.0 0 384 0 3
+129.0 0 258 0 2
+131.0 0 131 0 1
+133.0 0 133 0 1
+134.0 0 268 0 2
+136.0 0 136 0 1
+137.0 0 274 0 2
+138.0 0 552 0 4
+143.0 0 143 0 1
+145.0 0 145 0 1
+146.0 0 292 0 2
+149.0 0 298 0 2
+150.0 0 150 0 1
+152.0 0 304 0 2
+153.0 0 153 0 1
+155.0 0 155 0 1
+156.0 0 156 0 1
+157.0 0 157 0 1
+158.0 0 158 0 1
+160.0 0 160 0 1
+162.0 0 162 0 1
+163.0 0 163 0 1
+164.0 0 328 0 2
+165.0 0 330 0 2
+166.0 0 166 0 1
+167.0 0 501 0 3
+168.0 0 168 0 1
+169.0 0 676 0 4
+170.0 0 170 0 1
+172.0 0 344 0 2
+174.0 0 348 0 2
+175.0 0 350 0 2
+176.0 0 352 0 2
+177.0 0 177 0 1
+178.0 0 178 0 1
+179.0 0 358 0 2
+180.0 0 180 0 1
+181.0 0 181 0 1
+183.0 0 183 0 1
+186.0 0 186 0 1
+187.0 0 561 0 3
+189.0 0 189 0 1
+190.0 0 190 0 1
+191.0 0 382 0 2
+192.0 0 192 0 1
+193.0 0 579 0 3
+194.0 0 194 0 1
+195.0 0 390 0 2
+196.0 0 196 0 1
+197.0 0 394 0 2
+199.0 0 597 0 3
+200.0 0 400 0 2
+201.0 0 201 0 1
+202.0 0 202 0 1
+203.0 0 406 0 2
+205.0 0 410 0 2
+207.0 0 414 0 2
+208.0 0 624 0 3
+209.0 0 418 0 2
+213.0 0 426 0 2
+214.0 0 214 0 1
+216.0 0 432 0 2
+217.0 0 434 0 2
+218.0 0 218 0 1
+219.0 0 438 0 2
+221.0 0 442 0 2
+222.0 0 222 0 1
+223.0 0 446 0 2
+224.0 0 448 0 2
+226.0 0 226 0 1
+228.0 0 228 0 1
+229.0 0 458 0 2
+230.0 0 1150 0 5
+233.0 0 466 0 2
+235.0 0 235 0 1
+237.0 0 474 0 2
+238.0 0 476 0 2
+239.0 0 478 0 2
+241.0 0 241 0 1
+242.0 0 484 0 2
+244.0 0 244 0 1
+247.0 0 247 0 1
+248.0 0 248 0 1
+249.0 0 249 0 1
+252.0 0 252 0 1
+255.0 0 510 0 2
+256.0 0 512 0 2
+257.0 0 257 0 1
+258.0 0 258 0 1
+260.0 0 260 0 1
+262.0 0 262 0 1
+263.0 0 263 0 1
+265.0 0 530 0 2
+266.0 0 266 0 1
+272.0 0 544 0 2
+273.0 0 819 0 3
+274.0 0 274 0 1
+275.0 0 275 0 1
+277.0 0 1108 0 4
+278.0 0 556 0 2
+280.0 0 560 0 2
+281.0 0 562 0 2
+282.0 0 564 0 2
+283.0 0 283 0 1
+284.0 0 284 0 1
+285.0 0 285 0 1
+286.0 0 286 0 1
+287.0 0 287 0 1
+288.0 0 576 0 2
+289.0 0 289 0 1
+291.0 0 291 0 1
+292.0 0 292 0 1
+296.0 0 296 0 1
+298.0 0 894 0 3
+302.0 0 302 0 1
+305.0 0 305 0 1
+306.0 0 306 0 1
+307.0 0 614 0 2
+308.0 0 308 0 1
+309.0 0 618 0 2
+310.0 0 310 0 1
+311.0 0 933 0 3
+315.0 0 315 0 1
+316.0 0 948 0 3
+317.0 0 634 0 2
+318.0 0 954 0 3
+321.0 0 642 0 2
+322.0 0 644 0 2
+323.0 0 323 0 1
+325.0 0 650 0 2
+327.0 0 981 0 3
+331.0 0 662 0 2
+332.0 0 332 0 1
+333.0 0 666 0 2
+335.0 0 335 0 1
+336.0 0 336 0 1
+338.0 0 338 0 1
+339.0 0 339 0 1
+341.0 0 341 0 1
+342.0 0 684 0 2
+344.0 0 688 0 2
+345.0 0 345 0 1
+348.0 0 1740 0 5
+351.0 0 351 0 1
+353.0 0 706 0 2
+356.0 0 356 0 1
+360.0 0 360 0 1
+362.0 0 362 0 1
+364.0 0 364 0 1
+365.0 0 365 0 1
+366.0 0 366 0 1
+367.0 0 734 0 2
+368.0 0 368 0 1
+369.0 0 1107 0 3
+373.0 0 373 0 1
+374.0 0 374 0 1
+375.0 0 375 0 1
+377.0 0 377 0 1
+378.0 0 378 0 1
+379.0 0 379 0 1
+382.0 0 764 0 2
+384.0 0 1152 0 3
+386.0 0 386 0 1
+389.0 0 389 0 1
+392.0 0 392 0 1
+393.0 0 393 0 1
+394.0 0 394 0 1
+395.0 0 790 0 2
+396.0 0 1188 0 3
+397.0 0 794 0 2
+399.0 0 798 0 2
+400.0 0 400 0 1
+401.0 0 2005 0 5
+402.0 0 402 0 1
+403.0 0 1209 0 3
+404.0 0 808 0 2
+406.0 0 1624 0 4
+407.0 0 407 0 1
+409.0 0 1227 0 3
+411.0 0 411 0 1
+413.0 0 826 0 2
+414.0 0 828 0 2
+417.0 0 1251 0 3
+418.0 0 418 0 1
+419.0 0 419 0 1
+421.0 0 421 0 1
+424.0 0 848 0 2
+427.0 0 427 0 1
+429.0 0 858 0 2
+430.0 0 1290 0 3
+431.0 0 1293 0 3
+432.0 0 432 0 1
+435.0 0 435 0 1
+436.0 0 436 0 1
+437.0 0 437 0 1
+438.0 0 1314 0 3
+439.0 0 878 0 2
+443.0 0 443 0 1
+444.0 0 444 0 1
+446.0 0 446 0 1
+448.0 0 448 0 1
+449.0 0 449 0 1
+452.0 0 452 0 1
+453.0 0 453 0 1
+454.0 0 1362 0 3
+455.0 0 455 0 1
+457.0 0 457 0 1
+458.0 0 916 0 2
+459.0 0 918 0 2
+460.0 0 460 0 1
+462.0 0 924 0 2
+463.0 0 926 0 2
+466.0 0 1398 0 3
+467.0 0 467 0 1
+468.0 0 1872 0 4
+469.0 0 2345 0 5
+470.0 0 470 0 1
+472.0 0 472 0 1
+475.0 0 475 0 1
+477.0 0 477 0 1
+478.0 0 956 0 2
+479.0 0 479 0 1
+480.0 0 1440 0 3
+481.0 0 481 0 1
+482.0 0 482 0 1
+483.0 0 483 0 1
+484.0 0 484 0 1
+485.0 0 485 0 1
+487.0 0 487 0 1
+489.0 0 1956 0 4
+490.0 0 490 0 1
+491.0 0 491 0 1
+492.0 0 984 0 2
+493.0 0 493 0 1
+494.0 0 494 0 1
+495.0 0 495 0 1
+496.0 0 496 0 1
+497.0 0 497 0 1
+498.0 0 1494 0 3
diff --git a/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-a393cfc24ad74f930f3284743254c10c b/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-a393cfc24ad74f930f3284743254c10c
new file mode 100644
index 0000000000000..f23b45c32ecca
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-a393cfc24ad74f930f3284743254c10c
@@ -0,0 +1,309 @@
+0.0 0.0 0 0 0 3
+0.0 2.0 0 2 0 1
+0.0 4.0 0 4 0 1
+0.0 5.0 0 15 0 3
+0.0 8.0 0 8 0 1
+0.0 9.0 0 9 0 1
+0.0 10.0 0 10 0 1
+0.0 11.0 0 11 0 1
+0.0 12.0 0 24 0 2
+0.0 15.0 0 30 0 2
+0.0 17.0 0 17 0 1
+0.0 18.0 0 36 0 2
+0.0 19.0 0 19 0 1
+0.0 20.0 0 20 0 1
+0.0 24.0 0 48 0 2
+0.0 26.0 0 52 0 2
+0.0 27.0 0 27 0 1
+0.0 28.0 0 28 0 1
+0.0 30.0 0 30 0 1
+0.0 33.0 0 33 0 1
+0.0 34.0 0 34 0 1
+0.0 35.0 0 105 0 3
+0.0 37.0 0 74 0 2
+0.0 41.0 0 41 0 1
+0.0 42.0 0 84 0 2
+0.0 43.0 0 43 0 1
+0.0 44.0 0 44 0 1
+0.0 47.0 0 47 0 1
+0.0 51.0 0 102 0 2
+0.0 53.0 0 53 0 1
+0.0 54.0 0 54 0 1
+0.0 57.0 0 57 0 1
+0.0 58.0 0 116 0 2
+0.0 64.0 0 64 0 1
+0.0 65.0 0 65 0 1
+0.0 66.0 0 66 0 1
+0.0 67.0 0 134 0 2
+0.0 69.0 0 69 0 1
+0.0 70.0 0 210 0 3
+0.0 72.0 0 144 0 2
+0.0 74.0 0 74 0 1
+0.0 76.0 0 152 0 2
+0.0 77.0 0 77 0 1
+0.0 78.0 0 78 0 1
+0.0 80.0 0 80 0 1
+0.0 82.0 0 82 0 1
+0.0 83.0 0 166 0 2
+0.0 84.0 0 168 0 2
+0.0 85.0 0 85 0 1
+0.0 86.0 0 86 0 1
+0.0 87.0 0 87 0 1
+0.0 90.0 0 270 0 3
+0.0 92.0 0 92 0 1
+0.0 95.0 0 190 0 2
+0.0 96.0 0 96 0 1
+0.0 97.0 0 194 0 2
+0.0 98.0 0 196 0 2
+0.0 100.0 0 200 0 2
+0.0 103.0 0 206 0 2
+0.0 104.0 0 208 0 2
+0.0 105.0 0 105 0 1
+0.0 111.0 0 111 0 1
+0.0 113.0 0 226 0 2
+0.0 114.0 0 114 0 1
+0.0 116.0 0 116 0 1
+0.0 118.0 0 236 0 2
+0.0 119.0 0 357 0 3
+0.0 120.0 0 240 0 2
+0.0 125.0 0 250 0 2
+0.0 126.0 0 126 0 1
+0.0 128.0 0 384 0 3
+0.0 129.0 0 258 0 2
+0.0 131.0 0 131 0 1
+0.0 133.0 0 133 0 1
+0.0 134.0 0 268 0 2
+0.0 136.0 0 136 0 1
+0.0 137.0 0 274 0 2
+0.0 138.0 0 552 0 4
+0.0 143.0 0 143 0 1
+0.0 145.0 0 145 0 1
+0.0 146.0 0 292 0 2
+0.0 149.0 0 298 0 2
+0.0 150.0 0 150 0 1
+0.0 152.0 0 304 0 2
+0.0 153.0 0 153 0 1
+0.0 155.0 0 155 0 1
+0.0 156.0 0 156 0 1
+0.0 157.0 0 157 0 1
+0.0 158.0 0 158 0 1
+0.0 160.0 0 160 0 1
+0.0 162.0 0 162 0 1
+0.0 163.0 0 163 0 1
+0.0 164.0 0 328 0 2
+0.0 165.0 0 330 0 2
+0.0 166.0 0 166 0 1
+0.0 167.0 0 501 0 3
+0.0 168.0 0 168 0 1
+0.0 169.0 0 676 0 4
+0.0 170.0 0 170 0 1
+0.0 172.0 0 344 0 2
+0.0 174.0 0 348 0 2
+0.0 175.0 0 350 0 2
+0.0 176.0 0 352 0 2
+0.0 177.0 0 177 0 1
+0.0 178.0 0 178 0 1
+0.0 179.0 0 358 0 2
+0.0 180.0 0 180 0 1
+0.0 181.0 0 181 0 1
+0.0 183.0 0 183 0 1
+0.0 186.0 0 186 0 1
+0.0 187.0 0 561 0 3
+0.0 189.0 0 189 0 1
+0.0 190.0 0 190 0 1
+0.0 191.0 0 382 0 2
+0.0 192.0 0 192 0 1
+0.0 193.0 0 579 0 3
+0.0 194.0 0 194 0 1
+0.0 195.0 0 390 0 2
+0.0 196.0 0 196 0 1
+0.0 197.0 0 394 0 2
+0.0 199.0 0 597 0 3
+0.0 200.0 0 400 0 2
+0.0 201.0 0 201 0 1
+0.0 202.0 0 202 0 1
+0.0 203.0 0 406 0 2
+0.0 205.0 0 410 0 2
+0.0 207.0 0 414 0 2
+0.0 208.0 0 624 0 3
+0.0 209.0 0 418 0 2
+0.0 213.0 0 426 0 2
+0.0 214.0 0 214 0 1
+0.0 216.0 0 432 0 2
+0.0 217.0 0 434 0 2
+0.0 218.0 0 218 0 1
+0.0 219.0 0 438 0 2
+0.0 221.0 0 442 0 2
+0.0 222.0 0 222 0 1
+0.0 223.0 0 446 0 2
+0.0 224.0 0 448 0 2
+0.0 226.0 0 226 0 1
+0.0 228.0 0 228 0 1
+0.0 229.0 0 458 0 2
+0.0 230.0 0 1150 0 5
+0.0 233.0 0 466 0 2
+0.0 235.0 0 235 0 1
+0.0 237.0 0 474 0 2
+0.0 238.0 0 476 0 2
+0.0 239.0 0 478 0 2
+0.0 241.0 0 241 0 1
+0.0 242.0 0 484 0 2
+0.0 244.0 0 244 0 1
+0.0 247.0 0 247 0 1
+0.0 248.0 0 248 0 1
+0.0 249.0 0 249 0 1
+0.0 252.0 0 252 0 1
+0.0 255.0 0 510 0 2
+0.0 256.0 0 512 0 2
+0.0 257.0 0 257 0 1
+0.0 258.0 0 258 0 1
+0.0 260.0 0 260 0 1
+0.0 262.0 0 262 0 1
+0.0 263.0 0 263 0 1
+0.0 265.0 0 530 0 2
+0.0 266.0 0 266 0 1
+0.0 272.0 0 544 0 2
+0.0 273.0 0 819 0 3
+0.0 274.0 0 274 0 1
+0.0 275.0 0 275 0 1
+0.0 277.0 0 1108 0 4
+0.0 278.0 0 556 0 2
+0.0 280.0 0 560 0 2
+0.0 281.0 0 562 0 2
+0.0 282.0 0 564 0 2
+0.0 283.0 0 283 0 1
+0.0 284.0 0 284 0 1
+0.0 285.0 0 285 0 1
+0.0 286.0 0 286 0 1
+0.0 287.0 0 287 0 1
+0.0 288.0 0 576 0 2
+0.0 289.0 0 289 0 1
+0.0 291.0 0 291 0 1
+0.0 292.0 0 292 0 1
+0.0 296.0 0 296 0 1
+0.0 298.0 0 894 0 3
+0.0 302.0 0 302 0 1
+0.0 305.0 0 305 0 1
+0.0 306.0 0 306 0 1
+0.0 307.0 0 614 0 2
+0.0 308.0 0 308 0 1
+0.0 309.0 0 618 0 2
+0.0 310.0 0 310 0 1
+0.0 311.0 0 933 0 3
+0.0 315.0 0 315 0 1
+0.0 316.0 0 948 0 3
+0.0 317.0 0 634 0 2
+0.0 318.0 0 954 0 3
+0.0 321.0 0 642 0 2
+0.0 322.0 0 644 0 2
+0.0 323.0 0 323 0 1
+0.0 325.0 0 650 0 2
+0.0 327.0 0 981 0 3
+0.0 331.0 0 662 0 2
+0.0 332.0 0 332 0 1
+0.0 333.0 0 666 0 2
+0.0 335.0 0 335 0 1
+0.0 336.0 0 336 0 1
+0.0 338.0 0 338 0 1
+0.0 339.0 0 339 0 1
+0.0 341.0 0 341 0 1
+0.0 342.0 0 684 0 2
+0.0 344.0 0 688 0 2
+0.0 345.0 0 345 0 1
+0.0 348.0 0 1740 0 5
+0.0 351.0 0 351 0 1
+0.0 353.0 0 706 0 2
+0.0 356.0 0 356 0 1
+0.0 360.0 0 360 0 1
+0.0 362.0 0 362 0 1
+0.0 364.0 0 364 0 1
+0.0 365.0 0 365 0 1
+0.0 366.0 0 366 0 1
+0.0 367.0 0 734 0 2
+0.0 368.0 0 368 0 1
+0.0 369.0 0 1107 0 3
+0.0 373.0 0 373 0 1
+0.0 374.0 0 374 0 1
+0.0 375.0 0 375 0 1
+0.0 377.0 0 377 0 1
+0.0 378.0 0 378 0 1
+0.0 379.0 0 379 0 1
+0.0 382.0 0 764 0 2
+0.0 384.0 0 1152 0 3
+0.0 386.0 0 386 0 1
+0.0 389.0 0 389 0 1
+0.0 392.0 0 392 0 1
+0.0 393.0 0 393 0 1
+0.0 394.0 0 394 0 1
+0.0 395.0 0 790 0 2
+0.0 396.0 0 1188 0 3
+0.0 397.0 0 794 0 2
+0.0 399.0 0 798 0 2
+0.0 400.0 0 400 0 1
+0.0 401.0 0 2005 0 5
+0.0 402.0 0 402 0 1
+0.0 403.0 0 1209 0 3
+0.0 404.0 0 808 0 2
+0.0 406.0 0 1624 0 4
+0.0 407.0 0 407 0 1
+0.0 409.0 0 1227 0 3
+0.0 411.0 0 411 0 1
+0.0 413.0 0 826 0 2
+0.0 414.0 0 828 0 2
+0.0 417.0 0 1251 0 3
+0.0 418.0 0 418 0 1
+0.0 419.0 0 419 0 1
+0.0 421.0 0 421 0 1
+0.0 424.0 0 848 0 2
+0.0 427.0 0 427 0 1
+0.0 429.0 0 858 0 2
+0.0 430.0 0 1290 0 3
+0.0 431.0 0 1293 0 3
+0.0 432.0 0 432 0 1
+0.0 435.0 0 435 0 1
+0.0 436.0 0 436 0 1
+0.0 437.0 0 437 0 1
+0.0 438.0 0 1314 0 3
+0.0 439.0 0 878 0 2
+0.0 443.0 0 443 0 1
+0.0 444.0 0 444 0 1
+0.0 446.0 0 446 0 1
+0.0 448.0 0 448 0 1
+0.0 449.0 0 449 0 1
+0.0 452.0 0 452 0 1
+0.0 453.0 0 453 0 1
+0.0 454.0 0 1362 0 3
+0.0 455.0 0 455 0 1
+0.0 457.0 0 457 0 1
+0.0 458.0 0 916 0 2
+0.0 459.0 0 918 0 2
+0.0 460.0 0 460 0 1
+0.0 462.0 0 924 0 2
+0.0 463.0 0 926 0 2
+0.0 466.0 0 1398 0 3
+0.0 467.0 0 467 0 1
+0.0 468.0 0 1872 0 4
+0.0 469.0 0 2345 0 5
+0.0 470.0 0 470 0 1
+0.0 472.0 0 472 0 1
+0.0 475.0 0 475 0 1
+0.0 477.0 0 477 0 1
+0.0 478.0 0 956 0 2
+0.0 479.0 0 479 0 1
+0.0 480.0 0 1440 0 3
+0.0 481.0 0 481 0 1
+0.0 482.0 0 482 0 1
+0.0 483.0 0 483 0 1
+0.0 484.0 0 484 0 1
+0.0 485.0 0 485 0 1
+0.0 487.0 0 487 0 1
+0.0 489.0 0 1956 0 4
+0.0 490.0 0 490 0 1
+0.0 491.0 0 491 0 1
+0.0 492.0 0 984 0 2
+0.0 493.0 0 493 0 1
+0.0 494.0 0 494 0 1
+0.0 495.0 0 495 0 1
+0.0 496.0 0 496 0 1
+0.0 497.0 0 497 0 1
+0.0 498.0 0 1494 0 3
diff --git a/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-ae497f1556f548c1e2da9244397a985d b/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-ae497f1556f548c1e2da9244397a985d
new file mode 100644
index 0000000000000..7839d714c25d2
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-ae497f1556f548c1e2da9244397a985d
@@ -0,0 +1,309 @@
+0 3
+0 1
+0 1
+0 3
+0 1
+0 1
+0 1
+0 1
+0 2
+0 2
+0 1
+0 2
+0 1
+0 1
+0 2
+0 2
+0 1
+0 1
+0 1
+0 1
+0 1
+0 3
+0 2
+0 1
+0 2
+0 1
+0 1
+0 1
+0 2
+0 1
+0 1
+0 1
+0 2
+0 1
+0 1
+0 1
+0 2
+0 1
+0 3
+0 2
+0 1
+0 2
+0 1
+0 1
+0 1
+0 1
+0 2
+0 2
+0 1
+0 1
+0 1
+0 3
+0 1
+0 2
+0 1
+0 2
+0 2
+0 2
+0 2
+0 2
+0 1
+0 1
+0 2
+0 1
+0 1
+0 2
+0 3
+0 2
+0 2
+0 1
+0 3
+0 2
+0 1
+0 1
+0 2
+0 1
+0 2
+0 4
+0 1
+0 1
+0 2
+0 2
+0 1
+0 2
+0 1
+0 1
+0 1
+0 1
+0 1
+0 1
+0 1
+0 1
+0 2
+0 2
+0 1
+0 3
+0 1
+0 4
+0 1
+0 2
+0 2
+0 2
+0 2
+0 1
+0 1
+0 2
+0 1
+0 1
+0 1
+0 1
+0 3
+0 1
+0 1
+0 2
+0 1
+0 3
+0 1
+0 2
+0 1
+0 2
+0 3
+0 2
+0 1
+0 1
+0 2
+0 2
+0 2
+0 3
+0 2
+0 2
+0 1
+0 2
+0 2
+0 1
+0 2
+0 2
+0 1
+0 2
+0 2
+0 1
+0 1
+0 2
+0 5
+0 2
+0 1
+0 2
+0 2
+0 2
+0 1
+0 2
+0 1
+0 1
+0 1
+0 1
+0 1
+0 2
+0 2
+0 1
+0 1
+0 1
+0 1
+0 1
+0 2
+0 1
+0 2
+0 3
+0 1
+0 1
+0 4
+0 2
+0 2
+0 2
+0 2
+0 1
+0 1
+0 1
+0 1
+0 1
+0 2
+0 1
+0 1
+0 1
+0 1
+0 3
+0 1
+0 1
+0 1
+0 2
+0 1
+0 2
+0 1
+0 3
+0 1
+0 3
+0 2
+0 3
+0 2
+0 2
+0 1
+0 2
+0 3
+0 2
+0 1
+0 2
+0 1
+0 1
+0 1
+0 1
+0 1
+0 2
+0 2
+0 1
+0 5
+0 1
+0 2
+0 1
+0 1
+0 1
+0 1
+0 1
+0 1
+0 2
+0 1
+0 3
+0 1
+0 1
+0 1
+0 1
+0 1
+0 1
+0 2
+0 3
+0 1
+0 1
+0 1
+0 1
+0 1
+0 2
+0 3
+0 2
+0 2
+0 1
+0 5
+0 1
+0 3
+0 2
+0 4
+0 1
+0 3
+0 1
+0 2
+0 2
+0 3
+0 1
+0 1
+0 1
+0 2
+0 1
+0 2
+0 3
+0 3
+0 1
+0 1
+0 1
+0 1
+0 3
+0 2
+0 1
+0 1
+0 1
+0 1
+0 1
+0 1
+0 1
+0 3
+0 1
+0 1
+0 2
+0 2
+0 1
+0 2
+0 2
+0 3
+0 1
+0 4
+0 5
+0 1
+0 1
+0 1
+0 1
+0 2
+0 1
+0 3
+0 1
+0 1
+0 1
+0 1
+0 1
+0 1
+0 4
+0 1
+0 1
+0 2
+0 1
+0 1
+0 1
+0 1
+0 1
+0 3
diff --git a/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-c60b6075da793b826db5eb4b08d7bab9 b/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-c60b6075da793b826db5eb4b08d7bab9
new file mode 100644
index 0000000000000..cd6b14ac2a425
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/Constant Folding Optimization for AVG_SUM_COUNT-0-c60b6075da793b826db5eb4b08d7bab9
@@ -0,0 +1,309 @@
+0.0 0 0 3
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 3
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 3
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 3
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 3
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 2
+0.0 0 0 2
+0.0 0 0 2
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 3
+0.0 0 0 2
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 3
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 4
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 3
+0.0 0 0 1
+0.0 0 0 4
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 2
+0.0 0 0 2
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 3
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 3
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 3
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 2
+0.0 0 0 2
+0.0 0 0 3
+0.0 0 0 2
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 5
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 2
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 3
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 4
+0.0 0 0 2
+0.0 0 0 2
+0.0 0 0 2
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 3
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 3
+0.0 0 0 1
+0.0 0 0 3
+0.0 0 0 2
+0.0 0 0 3
+0.0 0 0 2
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 3
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 5
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 3
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 3
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 3
+0.0 0 0 2
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 5
+0.0 0 0 1
+0.0 0 0 3
+0.0 0 0 2
+0.0 0 0 4
+0.0 0 0 1
+0.0 0 0 3
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 2
+0.0 0 0 3
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 3
+0.0 0 0 3
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 3
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 3
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 2
+0.0 0 0 3
+0.0 0 0 1
+0.0 0 0 4
+0.0 0 0 5
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 3
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 4
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 2
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 1
+0.0 0 0 3
diff --git a/sql/hive/src/test/resources/golden/create_union_table-0-8e765b54f15b948fc88392da69da283 b/sql/hive/src/test/resources/golden/create_union_table-0-8e765b54f15b948fc88392da69da283
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/create_union_table-1-aee4ce62fc2631423af0f569f4448353 b/sql/hive/src/test/resources/golden/create_union_table-1-aee4ce62fc2631423af0f569f4448353
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/create_union_table-2-b1feb4a197caf28d5223e72e10a91e78 b/sql/hive/src/test/resources/golden/create_union_table-2-b1feb4a197caf28d5223e72e10a91e78
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/ctas_uses_database_location-0-1c3ace37d0bbb5c8033c48cde7680d21 b/sql/hive/src/test/resources/golden/ctas_uses_database_location-0-1c3ace37d0bbb5c8033c48cde7680d21
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/ctas_uses_database_location-0-1c3ace37d0bbb5c8033c48cde7680d21
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/fileformat_sequencefile-0-7df8fcbfff1c967de72295b90c530776 b/sql/hive/src/test/resources/golden/fileformat_sequencefile-0-7df8fcbfff1c967de72295b90c530776
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/fileformat_sequencefile-1-3114fdebb1e9b0830de9e9c3fff8a67e b/sql/hive/src/test/resources/golden/fileformat_sequencefile-1-3114fdebb1e9b0830de9e9c3fff8a67e
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/fileformat_sequencefile-2-80ec34a069bc561aa6dc87314391b131 b/sql/hive/src/test/resources/golden/fileformat_sequencefile-2-80ec34a069bc561aa6dc87314391b131
new file mode 100644
index 0000000000000..6280b32facd66
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/fileformat_sequencefile-2-80ec34a069bc561aa6dc87314391b131
@@ -0,0 +1,4 @@
+key int None
+value string None
+
+Detailed Table Information Table(tableName:dest1, dbName:default, owner:marmbrus, createTime:1398823397, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse6323689881248298063/dest1, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1398823397}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
diff --git a/sql/hive/src/test/resources/golden/fileformat_sequencefile-3-699522b1ff3f38cf6a75b06d83820b87 b/sql/hive/src/test/resources/golden/fileformat_sequencefile-3-699522b1ff3f38cf6a75b06d83820b87
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/fileformat_sequencefile-4-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/fileformat_sequencefile-4-adc1ec67836b26b60d8547c4996bfd8f
new file mode 100644
index 0000000000000..e614f37ecc8bf
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/fileformat_sequencefile-4-adc1ec67836b26b60d8547c4996bfd8f
@@ -0,0 +1,10 @@
+0 val_0
+4 val_4
+8 val_8
+0 val_0
+0 val_0
+5 val_5
+5 val_5
+2 val_2
+5 val_5
+9 val_9
diff --git a/sql/hive/src/test/resources/golden/fileformat_text-0-c84258297070d93009fabc59c1f40a1e b/sql/hive/src/test/resources/golden/fileformat_text-0-c84258297070d93009fabc59c1f40a1e
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/fileformat_text-1-6375f4ba7af19f94c6afc366dc75429e b/sql/hive/src/test/resources/golden/fileformat_text-1-6375f4ba7af19f94c6afc366dc75429e
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/fileformat_text-2-80ec34a069bc561aa6dc87314391b131 b/sql/hive/src/test/resources/golden/fileformat_text-2-80ec34a069bc561aa6dc87314391b131
new file mode 100644
index 0000000000000..e793ec2f946e5
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/fileformat_text-2-80ec34a069bc561aa6dc87314391b131
@@ -0,0 +1,4 @@
+key int None
+value string None
+
+Detailed Table Information Table(tableName:dest1, dbName:default, owner:marmbrus, createTime:1398823407, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:key, type:int, comment:null), FieldSchema(name:value, type:string, comment:null)], location:file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse6323689881248298063/dest1, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1398823407}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE)
diff --git a/sql/hive/src/test/resources/golden/fileformat_text-3-699522b1ff3f38cf6a75b06d83820b87 b/sql/hive/src/test/resources/golden/fileformat_text-3-699522b1ff3f38cf6a75b06d83820b87
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/fileformat_text-4-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/fileformat_text-4-adc1ec67836b26b60d8547c4996bfd8f
new file mode 100644
index 0000000000000..e614f37ecc8bf
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/fileformat_text-4-adc1ec67836b26b60d8547c4996bfd8f
@@ -0,0 +1,10 @@
+0 val_0
+4 val_4
+8 val_8
+0 val_0
+0 val_0
+5 val_5
+5 val_5
+2 val_2
+5 val_5
+9 val_9
diff --git a/sql/hive/src/test/resources/golden/input15-0-1570712216ce86428ee39974242ae961 b/sql/hive/src/test/resources/golden/input15-0-1570712216ce86428ee39974242ae961
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/input15-1-86edc12357bf278d5e601a654358c32f b/sql/hive/src/test/resources/golden/input15-1-86edc12357bf278d5e601a654358c32f
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/input15-2-ae5e0fbdf88ecca2c7d67df1ad141919 b/sql/hive/src/test/resources/golden/input15-2-ae5e0fbdf88ecca2c7d67df1ad141919
new file mode 100644
index 0000000000000..ded361eb294f0
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/input15-2-ae5e0fbdf88ecca2c7d67df1ad141919
@@ -0,0 +1,2 @@
+key int None
+value string None
diff --git a/sql/hive/src/test/resources/golden/inputddl1-0-52f161c97a2a02494d26ee0737b28364 b/sql/hive/src/test/resources/golden/inputddl1-0-52f161c97a2a02494d26ee0737b28364
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/inputddl1-1-dfe05b7247284b326f39481b3d66a0c3 b/sql/hive/src/test/resources/golden/inputddl1-1-dfe05b7247284b326f39481b3d66a0c3
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/inputddl1-2-b516db5218781d89aebd2e4813ea3660 b/sql/hive/src/test/resources/golden/inputddl1-2-b516db5218781d89aebd2e4813ea3660
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/inputddl2-0-4519aea06db13ec6780f60b4d101d262 b/sql/hive/src/test/resources/golden/inputddl2-0-4519aea06db13ec6780f60b4d101d262
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/inputddl2-1-f526507cae3cad09c69d20c7f0e36abe b/sql/hive/src/test/resources/golden/inputddl2-1-f526507cae3cad09c69d20c7f0e36abe
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/inputddl2-2-7c1c0606b5ea8a1c90d46fe221993b57 b/sql/hive/src/test/resources/golden/inputddl2-2-7c1c0606b5ea8a1c90d46fe221993b57
new file mode 100644
index 0000000000000..679d54cb5cb5e
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/inputddl2-2-7c1c0606b5ea8a1c90d46fe221993b57
@@ -0,0 +1,10 @@
+key int None
+value string None
+ds string None
+country string None
+
+# Partition Information
+# col_name data_type comment
+
+ds string None
+country string None
diff --git a/sql/hive/src/test/resources/golden/inputddl3-0-a14253f6c752c9f5e9f56875152f8353 b/sql/hive/src/test/resources/golden/inputddl3-0-a14253f6c752c9f5e9f56875152f8353
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/inputddl3-1-f066827dd440213f306ab7d680bc8526 b/sql/hive/src/test/resources/golden/inputddl3-1-f066827dd440213f306ab7d680bc8526
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/inputddl3-2-73f945a673d2e388847c317f683f160c b/sql/hive/src/test/resources/golden/inputddl3-2-73f945a673d2e388847c317f683f160c
new file mode 100644
index 0000000000000..ded361eb294f0
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/inputddl3-2-73f945a673d2e388847c317f683f160c
@@ -0,0 +1,2 @@
+key int None
+value string None
diff --git a/sql/hive/src/test/resources/golden/load_binary_data-4-3ebc340f7f63740f8534706d42dd37ca b/sql/hive/src/test/resources/golden/load_binary_data-4-3ebc340f7f63740f8534706d42dd37ca
new file mode 100644
index 0000000000000..de5212a3c320f
Binary files /dev/null and b/sql/hive/src/test/resources/golden/load_binary_data-4-3ebc340f7f63740f8534706d42dd37ca differ
diff --git a/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-1-8c68ce10095a8924c68c8ee4b3c0071a b/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-1-8c68ce10095a8924c68c8ee4b3c0071a
new file mode 100644
index 0000000000000..f81251e1360f2
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-1-8c68ce10095a8924c68c8ee4b3c0071a
@@ -0,0 +1,22 @@
+NULL NULL 66 val_66 66 val_66
+NULL NULL 98 val_98 98 val_98
+NULL NULL 98 val_98 98 val_98
+NULL NULL 128 128 val_128
+NULL NULL 128 128 val_128
+NULL NULL 128 128 val_128
+NULL NULL 146 val_146 146 val_146
+NULL NULL 146 val_146 146 val_146
+NULL NULL 150 val_150 150 val_150
+NULL NULL 213 val_213 213 val_213
+NULL NULL 213 val_213 213 val_213
+NULL NULL 224 224 val_224
+NULL NULL 224 224 val_224
+NULL NULL 238 val_238 238 val_238
+NULL NULL 238 val_238 238 val_238
+NULL NULL 255 val_255 255 val_255
+NULL NULL 255 val_255 255 val_255
+NULL NULL 273 val_273 273 val_273
+NULL NULL 273 val_273 273 val_273
+NULL NULL 273 val_273 273 val_273
+NULL NULL 278 val_278 278 val_278
+NULL NULL 278 val_278 278 val_278
diff --git a/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-2-8e5d5472f2e214c091c879f6830a0c b/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-2-8e5d5472f2e214c091c879f6830a0c
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-3-381cc29f131d8caba744dd9fe6c121e7 b/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-3-381cc29f131d8caba744dd9fe6c121e7
new file mode 100644
index 0000000000000..f81251e1360f2
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-3-381cc29f131d8caba744dd9fe6c121e7
@@ -0,0 +1,22 @@
+NULL NULL 66 val_66 66 val_66
+NULL NULL 98 val_98 98 val_98
+NULL NULL 98 val_98 98 val_98
+NULL NULL 128 128 val_128
+NULL NULL 128 128 val_128
+NULL NULL 128 128 val_128
+NULL NULL 146 val_146 146 val_146
+NULL NULL 146 val_146 146 val_146
+NULL NULL 150 val_150 150 val_150
+NULL NULL 213 val_213 213 val_213
+NULL NULL 213 val_213 213 val_213
+NULL NULL 224 224 val_224
+NULL NULL 224 224 val_224
+NULL NULL 238 val_238 238 val_238
+NULL NULL 238 val_238 238 val_238
+NULL NULL 255 val_255 255 val_255
+NULL NULL 255 val_255 255 val_255
+NULL NULL 273 val_273 273 val_273
+NULL NULL 273 val_273 273 val_273
+NULL NULL 273 val_273 273 val_273
+NULL NULL 278 val_278 278 val_278
+NULL NULL 278 val_278 278 val_278
diff --git a/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-4-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-4-ce1ef910fff98f174931cc641f7cef3a
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-4-ce1ef910fff98f174931cc641f7cef3a
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-5-82cbc3186de23f3a2411e9ab87c0008c b/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-5-82cbc3186de23f3a2411e9ab87c0008c
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-6-c6844de37be0926316f5fbf36a905306 b/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-6-c6844de37be0926316f5fbf36a905306
new file mode 100644
index 0000000000000..f81251e1360f2
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-6-c6844de37be0926316f5fbf36a905306
@@ -0,0 +1,22 @@
+NULL NULL 66 val_66 66 val_66
+NULL NULL 98 val_98 98 val_98
+NULL NULL 98 val_98 98 val_98
+NULL NULL 128 128 val_128
+NULL NULL 128 128 val_128
+NULL NULL 128 128 val_128
+NULL NULL 146 val_146 146 val_146
+NULL NULL 146 val_146 146 val_146
+NULL NULL 150 val_150 150 val_150
+NULL NULL 213 val_213 213 val_213
+NULL NULL 213 val_213 213 val_213
+NULL NULL 224 224 val_224
+NULL NULL 224 224 val_224
+NULL NULL 238 val_238 238 val_238
+NULL NULL 238 val_238 238 val_238
+NULL NULL 255 val_255 255 val_255
+NULL NULL 255 val_255 255 val_255
+NULL NULL 273 val_273 273 val_273
+NULL NULL 273 val_273 273 val_273
+NULL NULL 273 val_273 273 val_273
+NULL NULL 278 val_278 278 val_278
+NULL NULL 278 val_278 278 val_278
diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-0-3ca1c197c5209d2fc9423fe84ad66e7d b/sql/hive/src/test/resources/golden/parallel_orderby-0-3ca1c197c5209d2fc9423fe84ad66e7d
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-1-be7423a5e77b7289153f74bc3fd97f44 b/sql/hive/src/test/resources/golden/parallel_orderby-1-be7423a5e77b7289153f74bc3fd97f44
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-10-767848077fc2929ccedfd28e50564b19 b/sql/hive/src/test/resources/golden/parallel_orderby-10-767848077fc2929ccedfd28e50564b19
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/parallel_orderby-10-767848077fc2929ccedfd28e50564b19
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-11-39767502cbda75590c0c4b8fd089b793 b/sql/hive/src/test/resources/golden/parallel_orderby-11-39767502cbda75590c0c4b8fd089b793
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-12-7943485bcc72b2040c45c62e45ac8853 b/sql/hive/src/test/resources/golden/parallel_orderby-12-7943485bcc72b2040c45c62e45ac8853
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-13-988591bf9ab008fdc4d71560aa57a736 b/sql/hive/src/test/resources/golden/parallel_orderby-13-988591bf9ab008fdc4d71560aa57a736
new file mode 100644
index 0000000000000..b431d3fc6dcf6
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/parallel_orderby-13-988591bf9ab008fdc4d71560aa57a736
@@ -0,0 +1,32 @@
+# col_name data_type comment
+
+key string None
+value string None
+
+# Detailed Table Information
+Database: default
+Owner: marmbrus
+CreateTime: Tue Apr 29 20:55:07 PDT 2014
+LastAccessTime: UNKNOWN
+Protect Mode: None
+Retention: 0
+Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse6323689881248298063/total_ordered
+Table Type: MANAGED_TABLE
+Table Parameters:
+ numFiles 1
+ numPartitions 0
+ numRows 48
+ rawDataSize 512
+ totalSize 560
+ transient_lastDdlTime 1398830107
+
+# Storage Information
+SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+InputFormat: org.apache.hadoop.mapred.TextInputFormat
+OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+Compressed: No
+Num Buckets: -1
+Bucket Columns: []
+Sort Columns: []
+Storage Desc Params:
+ serialization.format 1
diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-14-d93951df2ffc18dc09ab0ba2d46f1823 b/sql/hive/src/test/resources/golden/parallel_orderby-14-d93951df2ffc18dc09ab0ba2d46f1823
new file mode 100644
index 0000000000000..3b733e2d6c451
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/parallel_orderby-14-d93951df2ffc18dc09ab0ba2d46f1823
@@ -0,0 +1,48 @@
+128 val_128
+128 val_128
+150 val_150
+150 val_150
+165 val_165
+165 val_165
+193 val_193
+193 val_193
+213 val_213
+213 val_213
+213 val_213
+213 val_213
+213 val_214
+213 val_214
+224 val_224
+224 val_224
+238 val_238
+238 val_238
+238 val_239
+238 val_239
+238 val_240
+238 val_240
+255 val_255
+255 val_255
+265 val_265
+265 val_265
+27 val_27
+27 val_27
+273 val_273
+273 val_273
+278 val_278
+278 val_278
+311 val_311
+311 val_311
+369 val_369
+369 val_369
+401 val_401
+401 val_401
+409 val_409
+409 val_409
+484 val_484
+484 val_484
+66 val_66
+66 val_66
+86 val_86
+86 val_86
+98 val_98
+98 val_98
diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-2-be7423a5e77b7289153f74bc3fd97f44 b/sql/hive/src/test/resources/golden/parallel_orderby-2-be7423a5e77b7289153f74bc3fd97f44
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-3-8ae9591fe39cd390619181e9664a92c1 b/sql/hive/src/test/resources/golden/parallel_orderby-3-8ae9591fe39cd390619181e9664a92c1
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/parallel_orderby-3-8ae9591fe39cd390619181e9664a92c1
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-4-a928b93138e9c5547c40ff2024b2b4b6 b/sql/hive/src/test/resources/golden/parallel_orderby-4-a928b93138e9c5547c40ff2024b2b4b6
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/parallel_orderby-4-a928b93138e9c5547c40ff2024b2b4b6
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-5-a58344acc57aaa38e2cb5f11a0576681 b/sql/hive/src/test/resources/golden/parallel_orderby-5-a58344acc57aaa38e2cb5f11a0576681
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/parallel_orderby-5-a58344acc57aaa38e2cb5f11a0576681
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-6-670ec1b1f28d92d72a924c29d622aa8f b/sql/hive/src/test/resources/golden/parallel_orderby-6-670ec1b1f28d92d72a924c29d622aa8f
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-7-7943485bcc72b2040c45c62e45ac8853 b/sql/hive/src/test/resources/golden/parallel_orderby-7-7943485bcc72b2040c45c62e45ac8853
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-8-988591bf9ab008fdc4d71560aa57a736 b/sql/hive/src/test/resources/golden/parallel_orderby-8-988591bf9ab008fdc4d71560aa57a736
new file mode 100644
index 0000000000000..7f67251e61787
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/parallel_orderby-8-988591bf9ab008fdc4d71560aa57a736
@@ -0,0 +1,32 @@
+# col_name data_type comment
+
+key string None
+value string None
+
+# Detailed Table Information
+Database: default
+Owner: marmbrus
+CreateTime: Tue Apr 29 20:54:55 PDT 2014
+LastAccessTime: UNKNOWN
+Protect Mode: None
+Retention: 0
+Location: file:/private/var/folders/36/cjkbrr953xg2p_krwrmn8h_r0000gn/T/sparkHiveWarehouse6323689881248298063/total_ordered
+Table Type: MANAGED_TABLE
+Table Parameters:
+ numFiles 1
+ numPartitions 0
+ numRows 48
+ rawDataSize 512
+ totalSize 560
+ transient_lastDdlTime 1398830095
+
+# Storage Information
+SerDe Library: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+InputFormat: org.apache.hadoop.mapred.TextInputFormat
+OutputFormat: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+Compressed: No
+Num Buckets: -1
+Bucket Columns: []
+Sort Columns: []
+Storage Desc Params:
+ serialization.format 1
diff --git a/sql/hive/src/test/resources/golden/parallel_orderby-9-d93951df2ffc18dc09ab0ba2d46f1823 b/sql/hive/src/test/resources/golden/parallel_orderby-9-d93951df2ffc18dc09ab0ba2d46f1823
new file mode 100644
index 0000000000000..3b733e2d6c451
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/parallel_orderby-9-d93951df2ffc18dc09ab0ba2d46f1823
@@ -0,0 +1,48 @@
+128 val_128
+128 val_128
+150 val_150
+150 val_150
+165 val_165
+165 val_165
+193 val_193
+193 val_193
+213 val_213
+213 val_213
+213 val_213
+213 val_213
+213 val_214
+213 val_214
+224 val_224
+224 val_224
+238 val_238
+238 val_238
+238 val_239
+238 val_239
+238 val_240
+238 val_240
+255 val_255
+255 val_255
+265 val_265
+265 val_265
+27 val_27
+27 val_27
+273 val_273
+273 val_273
+278 val_278
+278 val_278
+311 val_311
+311 val_311
+369 val_369
+369 val_369
+401 val_401
+401 val_401
+409 val_409
+409 val_409
+484 val_484
+484 val_484
+66 val_66
+66 val_66
+86 val_86
+86 val_86
+98 val_98
+98 val_98
diff --git a/sql/hive/src/test/resources/golden/serde_regex-0-60462d14b99bb445b02800e9fb22760e b/sql/hive/src/test/resources/golden/serde_regex-0-60462d14b99bb445b02800e9fb22760e
new file mode 100644
index 0000000000000..d00ee7786a57c
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/serde_regex-0-60462d14b99bb445b02800e9fb22760e
@@ -0,0 +1,22 @@
+ABSTRACT SYNTAX TREE:
+ (TOK_CREATETABLE (TOK_TABNAME serde_regex) TOK_LIKETABLE (TOK_TABCOLLIST (TOK_TABCOL host TOK_STRING) (TOK_TABCOL identity TOK_STRING) (TOK_TABCOL user TOK_STRING) (TOK_TABCOL time TOK_STRING) (TOK_TABCOL request TOK_STRING) (TOK_TABCOL status TOK_STRING) (TOK_TABCOL size TOK_INT) (TOK_TABCOL referer TOK_STRING) (TOK_TABCOL agent TOK_STRING)) (TOK_TABLESERIALIZER (TOK_SERDENAME 'org.apache.hadoop.hive.serde2.RegexSerDe' (TOK_TABLEPROPERTIES (TOK_TABLEPROPLIST (TOK_TABLEPROPERTY "input.regex" "([^ ]*) ([^ ]*) ([^ ]*) (-|\\[[^\\]]*\\]) ([^ \"]*|\"[^\"]*\") (-|[0-9]*) (-|[0-9]*)(?: ([^ \"]*|\"[^\"]*\") ([^ \"]*|\"[^\"]*\"))?"))))) TOK_TBLTEXTFILE)
+
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Create Table Operator:
+ Create Table
+ columns: host string, identity string, user string, time string, request string, status string, size int, referer string, agent string
+ if not exists: false
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ # buckets: -1
+ output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
+ serde name: org.apache.hadoop.hive.serde2.RegexSerDe
+ serde properties:
+ input.regex ([^ ]*) ([^ ]*) ([^ ]*) (-|\[[^\]]*\]) ([^ "]*|"[^"]*") (-|[0-9]*) (-|[0-9]*)(?: ([^ "]*|"[^"]*") ([^ "]*|"[^"]*"))?
+ name: serde_regex
+ isExternal: false
+
+
diff --git a/sql/hive/src/test/resources/golden/serde_regex-1-dea03bd88cbaabcf438b398e23c139f6 b/sql/hive/src/test/resources/golden/serde_regex-1-dea03bd88cbaabcf438b398e23c139f6
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429 b/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429
new file mode 100644
index 0000000000000..3e290231c27e2
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/serde_regex-10-c5b3ec90419a40660e5f83736241c429
@@ -0,0 +1,38 @@
+NULL 0
+NULL 0
+-1234567890.123456789 -1234567890
+-4400 4400
+-1255.49 -1255
+-1.122 -11
+-1.12 -1
+-1.12 -1
+-0.333 0
+-0.33 0
+-0.3 0
+0 0
+0 0
+0.01 0
+0.02 0
+0.1 0
+0.2 0
+0.3 0
+0.33 0
+0.333 0
+0.9999999999999999999999999 1
+1 1
+1 1
+1.12 1
+1.122 1
+2 2
+2 2
+3.14 3
+3.14 3
+3.14 3
+3.14 4
+10 10
+20 20
+100 100
+124 124
+125.2 125
+200 200
+1234567890.12345678 1234567890
diff --git a/sql/hive/src/test/resources/golden/serde_regex-11-b00710a6f7a3e5b92c2e17da54cd9ad1 b/sql/hive/src/test/resources/golden/serde_regex-11-b00710a6f7a3e5b92c2e17da54cd9ad1
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/serde_regex-2-9d00484beaee46cf72b154a1351aeee9 b/sql/hive/src/test/resources/golden/serde_regex-2-9d00484beaee46cf72b154a1351aeee9
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/serde_regex-3-817190d8871b70611483cd2abe2e55dc b/sql/hive/src/test/resources/golden/serde_regex-3-817190d8871b70611483cd2abe2e55dc
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/serde_regex-4-c3e345183543f40a14d2dd742ebd5346 b/sql/hive/src/test/resources/golden/serde_regex-4-c3e345183543f40a14d2dd742ebd5346
new file mode 100644
index 0000000000000..c55f3dd475574
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/serde_regex-4-c3e345183543f40a14d2dd742ebd5346
@@ -0,0 +1,2 @@
+127.0.0.1 - frank [10/Oct/2000:13:55:36 -0700] "GET /apache_pb.gif HTTP/1.0" 200 2326 NULL NULL
+127.0.0.1 - - [26/May/2009:00:00:00 +0000] "GET /someurl/?track=Blabla(Main) HTTP/1.1" 200 5864 - "Mozilla/5.0 (Windows; U; Windows NT 6.0; en-US) AppleWebKit/525.19 (KHTML, like Gecko) Chrome/1.0.154.65 Safari/525.19"
diff --git a/sql/hive/src/test/resources/golden/serde_regex-5-e2d14a76a87a6014c6d2d3501a0c9129 b/sql/hive/src/test/resources/golden/serde_regex-5-e2d14a76a87a6014c6d2d3501a0c9129
new file mode 100644
index 0000000000000..8bd185bc66ebb
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/serde_regex-5-e2d14a76a87a6014c6d2d3501a0c9129
@@ -0,0 +1,2 @@
+127.0.0.1 2326 200 [10/Oct/2000:13:55:36 -0700]
+127.0.0.1 5864 200 [26/May/2009:00:00:00 +0000]
diff --git a/sql/hive/src/test/resources/golden/serde_regex-6-f818821654f219d1f4e2482951fae4f1 b/sql/hive/src/test/resources/golden/serde_regex-6-f818821654f219d1f4e2482951fae4f1
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/serde_regex-7-4db287576a17c0963219ca63ee0b20e0 b/sql/hive/src/test/resources/golden/serde_regex-7-4db287576a17c0963219ca63ee0b20e0
new file mode 100644
index 0000000000000..da61769c6599d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/serde_regex-7-4db287576a17c0963219ca63ee0b20e0
@@ -0,0 +1,22 @@
+ABSTRACT SYNTAX TREE:
+ (TOK_CREATETABLE (TOK_TABNAME serde_regex1) TOK_LIKETABLE (TOK_TABCOLLIST (TOK_TABCOL key TOK_DECIMAL) (TOK_TABCOL value TOK_INT)) (TOK_TABLESERIALIZER (TOK_SERDENAME 'org.apache.hadoop.hive.serde2.RegexSerDe' (TOK_TABLEPROPERTIES (TOK_TABLEPROPLIST (TOK_TABLEPROPERTY "input.regex" "([^ ]*) ([^ ]*)"))))) TOK_TBLTEXTFILE)
+
+STAGE DEPENDENCIES:
+ Stage-0 is a root stage
+
+STAGE PLANS:
+ Stage: Stage-0
+ Create Table Operator:
+ Create Table
+ columns: key decimal, value int
+ if not exists: false
+ input format: org.apache.hadoop.mapred.TextInputFormat
+ # buckets: -1
+ output format: org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat
+ serde name: org.apache.hadoop.hive.serde2.RegexSerDe
+ serde properties:
+ input.regex ([^ ]*) ([^ ]*)
+ name: serde_regex1
+ isExternal: false
+
+
diff --git a/sql/hive/src/test/resources/golden/serde_regex-8-c429ee76b751e674992f61a29c95af77 b/sql/hive/src/test/resources/golden/serde_regex-8-c429ee76b751e674992f61a29c95af77
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/serde_regex-9-f0e8d394ad18dcbd381792fe9bd8894b b/sql/hive/src/test/resources/golden/serde_regex-9-f0e8d394ad18dcbd381792fe9bd8894b
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-0-31cd5ae86d250c1c873260c4b73d2336 b/sql/hive/src/test/resources/golden/smb_mapjoin9-0-31cd5ae86d250c1c873260c4b73d2336
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-1-c5c86975c400b3a1ec0f522b75635338 b/sql/hive/src/test/resources/golden/smb_mapjoin9-1-c5c86975c400b3a1ec0f522b75635338
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-10-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/smb_mapjoin9-10-ffe97dc8c1df3195982e38263fbe8717
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin9-10-ffe97dc8c1df3195982e38263fbe8717
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-11-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/smb_mapjoin9-11-43d53504df013e6b35f81811138a167a
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin9-11-43d53504df013e6b35f81811138a167a
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-12-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin9-12-7cccbdffc32975f8935eeba14a28147
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin9-12-7cccbdffc32975f8935eeba14a28147
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-13-d0eac0de35b5a7595e4567edec8e555d b/sql/hive/src/test/resources/golden/smb_mapjoin9-13-d0eac0de35b5a7595e4567edec8e555d
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-14-f0de4fd75ca13f379fbea16010dfd3f9 b/sql/hive/src/test/resources/golden/smb_mapjoin9-14-f0de4fd75ca13f379fbea16010dfd3f9
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-15-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin9-15-b89ea2173180c8ae423d856f943e061f
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin9-15-b89ea2173180c8ae423d856f943e061f
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-16-5645429104736d72293601247e874df7 b/sql/hive/src/test/resources/golden/smb_mapjoin9-16-5645429104736d72293601247e874df7
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-17-1018a99eaede08dc2b427d3fb4b91d6b b/sql/hive/src/test/resources/golden/smb_mapjoin9-17-1018a99eaede08dc2b427d3fb4b91d6b
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-18-bda1fd3b5d0326dc3a41a8fb480b6c1c b/sql/hive/src/test/resources/golden/smb_mapjoin9-18-bda1fd3b5d0326dc3a41a8fb480b6c1c
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-19-823e8f68baaa45d6b761b9b9890bb902 b/sql/hive/src/test/resources/golden/smb_mapjoin9-19-823e8f68baaa45d6b761b9b9890bb902
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-2-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin9-2-3b0f76816be2c1b18a2058027a19bc9f
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin9-2-3b0f76816be2c1b18a2058027a19bc9f
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-20-78ea4515eba2d8a79bb6895133a82051 b/sql/hive/src/test/resources/golden/smb_mapjoin9-20-78ea4515eba2d8a79bb6895133a82051
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-3-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin9-3-86473a0498e4361e4db0b4a22f2e8571
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin9-3-86473a0498e4361e4db0b4a22f2e8571
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-4-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin9-4-b89ea2173180c8ae423d856f943e061f
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin9-4-b89ea2173180c8ae423d856f943e061f
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-5-94bc4acde9b9c9944e2eb9a9c57d3450 b/sql/hive/src/test/resources/golden/smb_mapjoin9-5-94bc4acde9b9c9944e2eb9a9c57d3450
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-6-6c0963a1bdf2def39b79bea21a9cdda9 b/sql/hive/src/test/resources/golden/smb_mapjoin9-6-6c0963a1bdf2def39b79bea21a9cdda9
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-7-ba9bc547b5d2a4912f4f8567fea4009c b/sql/hive/src/test/resources/golden/smb_mapjoin9-7-ba9bc547b5d2a4912f4f8567fea4009c
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-8-d528ff47621b8c86b370f72512a7cd3c b/sql/hive/src/test/resources/golden/smb_mapjoin9-8-d528ff47621b8c86b370f72512a7cd3c
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin9-9-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/smb_mapjoin9-9-e39f59c35ebbe686a18d45d9d8bf3ab0
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/smb_mapjoin9-9-e39f59c35ebbe686a18d45d9d8bf3ab0
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-0-3b2fa9592648fc07c0d43e0d7d7f9411 b/sql/hive/src/test/resources/golden/symlink_text_input_format-0-3b2fa9592648fc07c0d43e0d7d7f9411
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-1-d498fb503b8f47db4741fdad3d266b4a b/sql/hive/src/test/resources/golden/symlink_text_input_format-1-d498fb503b8f47db4741fdad3d266b4a
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-2-42119039bf8023f90b7f474f235c5dd5 b/sql/hive/src/test/resources/golden/symlink_text_input_format-2-42119039bf8023f90b7f474f235c5dd5
new file mode 100644
index 0000000000000..3a2e3f4984a0e
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/symlink_text_input_format-2-42119039bf8023f90b7f474f235c5dd5
@@ -0,0 +1 @@
+-1
diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-3-77b57147024eb6b28cc9f525fdaab615 b/sql/hive/src/test/resources/golden/symlink_text_input_format-3-77b57147024eb6b28cc9f525fdaab615
new file mode 100644
index 0000000000000..3a2e3f4984a0e
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/symlink_text_input_format-3-77b57147024eb6b28cc9f525fdaab615
@@ -0,0 +1 @@
+-1
diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-4-676cb274a770a6b9ca86df5dc7f912d4 b/sql/hive/src/test/resources/golden/symlink_text_input_format-4-676cb274a770a6b9ca86df5dc7f912d4
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-5-ef3052815ec41b5957627698ba06707b b/sql/hive/src/test/resources/golden/symlink_text_input_format-5-ef3052815ec41b5957627698ba06707b
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-6-891be0baec05e358a647dcca77724446 b/sql/hive/src/test/resources/golden/symlink_text_input_format-6-891be0baec05e358a647dcca77724446
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-7-208bcc9c918cbeb52907c8871be19cd5 b/sql/hive/src/test/resources/golden/symlink_text_input_format-7-208bcc9c918cbeb52907c8871be19cd5
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-8-fb2e7127e07ad9f7e97ad3df3eba3a35 b/sql/hive/src/test/resources/golden/symlink_text_input_format-8-fb2e7127e07ad9f7e97ad3df3eba3a35
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/symlink_text_input_format-9-433d5dbbcf75ff68d6897f31baa46841 b/sql/hive/src/test/resources/golden/symlink_text_input_format-9-433d5dbbcf75ff68d6897f31baa46841
new file mode 100644
index 0000000000000..573541ac9702d
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/symlink_text_input_format-9-433d5dbbcf75ff68d6897f31baa46841
@@ -0,0 +1 @@
+0
diff --git a/sql/hive/src/test/resources/golden/udf_hour-4-73bfac513b993dedbe143306865a44a b/sql/hive/src/test/resources/golden/udf_hour-4-73bfac513b993dedbe143306865a44a
new file mode 100644
index 0000000000000..b1bd38b62a080
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_hour-4-73bfac513b993dedbe143306865a44a
@@ -0,0 +1 @@
+13
diff --git a/sql/hive/src/test/resources/golden/udf_substr-8-ba6ca6bac87ca99aabd60b8e76537ade b/sql/hive/src/test/resources/golden/udf_substr-8-ba6ca6bac87ca99aabd60b8e76537ade
new file mode 100644
index 0000000000000..c14d8d4279c5f
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_substr-8-ba6ca6bac87ca99aabd60b8e76537ade
@@ -0,0 +1 @@
+NULL NULL NULL NULL NULL A AB ABC ABC A AB ABC ABC B BC BC BC C C C C C C C C B BC BC BC A AB ABC ABC
diff --git a/sql/hive/src/test/resources/golden/udf_substr-9-a9aa82080133620d017160f6a644455d b/sql/hive/src/test/resources/golden/udf_substr-9-a9aa82080133620d017160f6a644455d
new file mode 100644
index 0000000000000..94a57d86c88fd
--- /dev/null
+++ b/sql/hive/src/test/resources/golden/udf_substr-9-a9aa82080133620d017160f6a644455d
@@ -0,0 +1 @@
+玩 玩 玩玩玩 abc 玩玩玩
diff --git a/sql/hive/src/test/resources/golden/union25-0-f4e883ee4edf8fbb95efed8ae41cff1c b/sql/hive/src/test/resources/golden/union25-0-f4e883ee4edf8fbb95efed8ae41cff1c
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/union25-1-8f4a32f3c1551b4e4f22c3b776a92043 b/sql/hive/src/test/resources/golden/union25-1-8f4a32f3c1551b4e4f22c3b776a92043
new file mode 100644
index 0000000000000..e69de29bb2d1d
diff --git a/sql/hive/src/test/resources/golden/union25-2-f61103bb2045761ba95828898e63b92b b/sql/hive/src/test/resources/golden/union25-2-f61103bb2045761ba95828898e63b92b
new file mode 100644
index 0000000000000..e69de29bb2d1d
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 0bb76f31c373d..d83732b51e9c2 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
@@ -112,6 +112,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"stats1.*",
"stats20",
"alter_merge_stats",
+ "columnstats.*",
+
// Hive seems to think 1.0 > NaN = true && 1.0 < NaN = false... which is wrong.
// http://stackoverflow.com/a/1573715
@@ -176,6 +178,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
*/
override def whiteList = Seq(
"add_part_exist",
+ "add_part_multiple",
"add_partition_no_whitelist",
"add_partition_with_whitelist",
"alias_casted_column",
@@ -290,6 +293,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"disable_file_format_check",
"drop_function",
"drop_index",
+ "drop_multi_partitions",
"drop_partitions_filter",
"drop_partitions_filter2",
"drop_partitions_filter3",
@@ -302,6 +306,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"escape_orderby1",
"escape_sortby1",
"fetch_aggregation",
+ "fileformat_sequencefile",
+ "fileformat_text",
"filter_join_breaktask",
"filter_join_breaktask2",
"groupby1",
@@ -310,6 +316,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"groupby1_map_nomap",
"groupby1_map_skew",
"groupby1_noskew",
+ "groupby2",
+ "groupby2_map",
+ "groupby2_map_skew",
+ "groupby2_noskew",
"groupby4",
"groupby4_map",
"groupby4_map_skew",
@@ -333,10 +343,12 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"groupby8_noskew",
"groupby9",
"groupby_distinct_samekey",
+ "groupby_map_ppr",
"groupby_multi_insert_common_distinct",
"groupby_multi_single_reducer2",
"groupby_mutli_insert_common_distinct",
"groupby_neg_float",
+ "groupby_ppr",
"groupby_sort_10",
"groupby_sort_2",
"groupby_sort_3",
@@ -352,13 +364,17 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"inoutdriver",
"input",
"input0",
+ "input1",
+ "input10",
"input11",
"input11_limit",
"input12",
"input12_hadoop20",
"input14",
+ "input15",
"input19",
"input1_limit",
+ "input2",
"input21",
"input22",
"input23",
@@ -367,6 +383,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"input26",
"input28",
"input2_limit",
+ "input3",
+ "input4",
"input40",
"input41",
"input4_cb_delim",
@@ -374,9 +392,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"input7",
"input8",
"input9",
- "inputddl4",
- "inputddl7",
- "inputddl8",
"input_limit",
"input_part0",
"input_part1",
@@ -391,6 +406,13 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"input_part8",
"input_part9",
"input_testsequencefile",
+ "inputddl1",
+ "inputddl2",
+ "inputddl3",
+ "inputddl4",
+ "inputddl6",
+ "inputddl7",
+ "inputddl8",
"insert1",
"insert2_overwrite_partitions",
"insert_compressed",
@@ -452,7 +474,9 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"join_reorder4",
"join_star",
"join_view",
+ "lateral_view",
"lateral_view_cp",
+ "lateral_view_outer",
"lateral_view_ppd",
"lineage1",
"literal_double",
@@ -463,6 +487,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"loadpart1",
"louter_join_ppr",
"mapjoin_distinct",
+ "mapjoin_filter_on_outerjoin",
"mapjoin_mapjoin",
"mapjoin_subquery",
"mapjoin_subquery2",
@@ -568,6 +593,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"select_unquote_and",
"select_unquote_not",
"select_unquote_or",
+ "serde_regex",
"serde_reported_schema",
"set_variable_sub",
"show_describe_func_quotes",
@@ -576,6 +602,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"skewjoinopt13",
"skewjoinopt18",
"skewjoinopt9",
+ "smb_mapjoin9",
"smb_mapjoin_1",
"smb_mapjoin_10",
"smb_mapjoin_13",
@@ -620,8 +647,11 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"udf_10_trims",
"udf2",
"udf6",
+ "udf7",
"udf8",
"udf9",
+ "udf_E",
+ "udf_PI",
"udf_abs",
"udf_acos",
"udf_add",
@@ -645,6 +675,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"udf_ceil",
"udf_ceiling",
"udf_concat",
+ "udf_concat_insert1",
"udf_concat_insert2",
"udf_concat_ws",
"udf_conv",
@@ -659,6 +690,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"udf_div",
"udf_double",
"udf_E",
+ "udf_elt",
"udf_exp",
"udf_field",
"udf_find_in_set",
@@ -668,9 +700,11 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"udf_from_unixtime",
"udf_greaterthan",
"udf_greaterthanorequal",
+ "udf_hash",
"udf_hex",
"udf_if",
"udf_index",
+ "udf_instr",
"udf_int",
"udf_isnotnull",
"udf_isnull",
@@ -681,6 +715,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"udf_lessthanorequal",
"udf_like",
"udf_ln",
+ "udf_locate",
"udf_log",
"udf_log10",
"udf_log2",
@@ -739,9 +774,9 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"udf_trim",
"udf_ucase",
"udf_upper",
- "udf_variance",
"udf_var_pop",
"udf_var_samp",
+ "udf_variance",
"udf_weekofyear",
"udf_when",
"udf_xpath",
@@ -767,6 +802,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter {
"union22",
"union23",
"union24",
+ "union25",
"union26",
"union27",
"union28",
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 d224d2ee600ec..87a92d83383ab 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
@@ -33,6 +33,9 @@ class HiveQuerySuite extends HiveComparisonTest {
hiveql("FROM src SELECT key").collect()
}
+ createQueryTest("Constant Folding Optimization for AVG_SUM_COUNT",
+ "SELECT AVG(0), SUM(0), COUNT(null), COUNT(value) FROM src GROUP BY key")
+
createQueryTest("Simple Average",
"SELECT AVG(key) FROM src")
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala
index cb8e8f00a7b82..c4bdf01fa3744 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala
@@ -262,7 +262,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])(
* @param windowDuration width of the window; must be a multiple of this DStream's
* batching interval
*/
- def reduceByKeyAndWindow(reduceFunc: Function2[V, V, V], windowDuration: Duration)
+ def reduceByKeyAndWindow(reduceFunc: JFunction2[V, V, V], windowDuration: Duration)
:JavaPairDStream[K, V] = {
dstream.reduceByKeyAndWindow(reduceFunc, windowDuration)
}
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala
index b310c22b3ab78..5acf8a9a811ee 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala
@@ -28,9 +28,11 @@ import org.apache.spark.annotation.DeveloperApi
/**
* :: DeveloperApi ::
* Abstract class of a receiver that can be run on worker nodes to receive external data. A
- * custom receiver can be defined by defining the functions onStart() and onStop(). onStart()
+ * custom receiver can be defined by defining the functions `onStart()` and `onStop()`. `onStart()`
* should define the setup steps necessary to start receiving data,
- * and onStop() should define the cleanup steps necessary to stop receiving data.
+ * and `onStop()` should define the cleanup steps necessary to stop receiving data.
+ * Exceptions while receiving can be handled either by restarting the receiver with `restart(...)`
+ * or stopped completely by `stop(...)` or
*
* A custom receiver in Scala would look like this.
*
diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala
index bbf57ef9275c0..a73d6f3bf0661 100644
--- a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala
+++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala
@@ -25,7 +25,7 @@ import scala.collection.JavaConversions.mapAsScalaMap
private[streaming]
object RawTextHelper {
- /**
+ /**
* Splits lines and counts the words.
*/
def splitAndCountPartitions(iter: Iterator[String]): Iterator[(String, Long)] = {
@@ -114,4 +114,3 @@ object RawTextHelper {
def max(v1: Long, v2: Long) = math.max(v1, v2)
}
-
diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala
index ee0bc8b7d6a71..cd86019f63e7e 100644
--- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala
+++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala
@@ -183,6 +183,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w
"Received records = " + TestReceiver.counter.get() + ", " +
"processed records = " + runningCount
)
+ Thread.sleep(100)
}
}
diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
index fc13dbecb4555..8f0ecb855718e 100644
--- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
+++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
@@ -70,9 +70,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration,
private var registered = false
- private val sparkUser = Option(System.getenv("SPARK_USER")).getOrElse(
- SparkContext.SPARK_UNKNOWN_USER)
-
def run() {
// Setup the directories so things go to yarn approved directories rather
// then user specified and /tmp.
@@ -192,7 +189,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration,
false /* initialize */ ,
Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]])
val t = new Thread {
- override def run(): Unit = SparkHadoopUtil.get.runAsUser(sparkUser) { () =>
+ override def run() {
+
var successed = false
try {
// Copy
@@ -480,6 +478,8 @@ object ApplicationMaster {
def main(argStrings: Array[String]) {
val args = new ApplicationMasterArguments(argStrings)
- new ApplicationMaster(args).run()
+ SparkHadoopUtil.get.runAsSparkUser { () =>
+ new ApplicationMaster(args).run()
+ }
}
}
diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala
index 65b7215afbd4c..a3bd91590fc25 100644
--- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala
+++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala
@@ -29,10 +29,11 @@ import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
import akka.actor._
import akka.remote._
import akka.actor.Terminated
-import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext}
+import org.apache.spark.{Logging, SecurityManager, SparkConf}
import org.apache.spark.util.{Utils, AkkaUtils}
import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
import org.apache.spark.scheduler.SplitInfo
+import org.apache.spark.deploy.SparkHadoopUtil
/**
* An application master that allocates executors on behalf of a driver that is running outside
@@ -279,6 +280,8 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp
object ExecutorLauncher {
def main(argStrings: Array[String]) {
val args = new ApplicationMasterArguments(argStrings)
- new ExecutorLauncher(args).run()
+ SparkHadoopUtil.get.runAsSparkUser { () =>
+ new ExecutorLauncher(args).run()
+ }
}
}
diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala
index f2be8217a2f8a..27a518ccda459 100644
--- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala
+++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala
@@ -263,9 +263,13 @@ trait ClientBase extends Logging {
distCacheMgr.setDistFilesEnv(env)
distCacheMgr.setDistArchivesEnv(env)
- // Allow users to specify some environment variables.
- YarnSparkHadoopUtil.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"),
- File.pathSeparator)
+ sys.env.get("SPARK_YARN_USER_ENV").foreach { userEnvs =>
+ // Allow users to specify some environment variables.
+ YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs, File.pathSeparator)
+
+ // Pass SPARK_YARN_USER_ENV itself to the AM so it can use it to set up executor environments.
+ env("SPARK_YARN_USER_ENV") = userEnvs
+ }
env
}
@@ -322,6 +326,12 @@ trait ClientBase extends Logging {
JAVA_OPTS += "-XX:CMSIncrementalDutyCycle=10"
}
+ // SPARK_JAVA_OPTS is deprecated, but for backwards compatibility:
+ sys.env.get("SPARK_JAVA_OPTS").foreach { opts =>
+ sparkConf.set("spark.executor.extraJavaOptions", opts)
+ sparkConf.set("spark.driver.extraJavaOptions", opts)
+ }
+
// TODO: it might be nicer to pass these as an internal environment variable rather than
// as Java options, due to complications with string parsing of nested quotes.
if (args.amClass == classOf[ExecutorLauncher].getName) {
diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala
index 7d07f6f68046a..96f8aa93394f5 100644
--- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala
+++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala
@@ -71,8 +71,8 @@ trait ExecutorRunnableUtil extends Logging {
/*
else {
// If no java_opts specified, default to using -XX:+CMSIncrementalMode
- // It might be possible that other modes/config is being done in SPARK_JAVA_OPTS, so we dont
- // want to mess with it.
+ // It might be possible that other modes/config is being done in spark.executor.extraJavaOptions,
+ // so we dont want to mess with it.
// In our expts, using (default) throughput collector has severe perf ramnifications in
// multi-tennent machines
// The options are based on
diff --git a/yarn/pom.xml b/yarn/pom.xml
index 02f36627431b9..6993c89525d8c 100644
--- a/yarn/pom.xml
+++ b/yarn/pom.xml
@@ -28,7 +28,7 @@
yarn-parent_2.10
pom
Spark Project YARN Parent POM
-
+
org.apache.spark
@@ -43,6 +43,10 @@
org.apache.hadoop
hadoop-yarn-common
+
+ org.apache.hadoop
+ hadoop-yarn-server-web-proxy
+
org.apache.hadoop
hadoop-yarn-client
@@ -50,7 +54,6 @@
org.apache.hadoop
hadoop-client
- ${yarn.version}
org.scalatest
@@ -128,7 +131,7 @@
-
+
diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
index 90e807160d4b6..c1dfe3f53b40b 100644
--- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
+++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala
@@ -71,9 +71,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration,
sparkConf.getInt("spark.yarn.max.worker.failures", math.max(args.numExecutors * 2, 3)))
private var registered = false
-
- private val sparkUser = Option(System.getenv("SPARK_USER")).getOrElse(
- SparkContext.SPARK_UNKNOWN_USER)
def run() {
// Setup the directories so things go to YARN approved directories rather
@@ -179,8 +176,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration,
false /* initialize */ ,
Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]])
val t = new Thread {
- override def run(): Unit = SparkHadoopUtil.get.runAsUser(sparkUser) { () =>
- var successed = false
+ override def run() {
+
+ var successed = false
try {
// Copy
var mainArgs: Array[String] = new Array[String](args.userArgs.size)
@@ -462,6 +460,8 @@ object ApplicationMaster {
def main(argStrings: Array[String]) {
val args = new ApplicationMasterArguments(argStrings)
- new ApplicationMaster(args).run()
+ SparkHadoopUtil.get.runAsSparkUser { () =>
+ new ApplicationMaster(args).run()
+ }
}
}
diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala
index a14bb377aa133..a4ce8766d347c 100644
--- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala
+++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala
@@ -28,12 +28,13 @@ import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
import akka.actor._
import akka.remote._
import akka.actor.Terminated
-import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext}
+import org.apache.spark.{Logging, SecurityManager, SparkConf}
import org.apache.spark.util.{Utils, AkkaUtils}
import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
import org.apache.spark.scheduler.SplitInfo
import org.apache.hadoop.yarn.client.api.AMRMClient
import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest
+import org.apache.spark.deploy.SparkHadoopUtil
/**
* An application master that allocates executors on behalf of a driver that is running outside
@@ -255,6 +256,8 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp
object ExecutorLauncher {
def main(argStrings: Array[String]) {
val args = new ApplicationMasterArguments(argStrings)
- new ExecutorLauncher(args).run()
+ SparkHadoopUtil.get.runAsSparkUser { () =>
+ new ExecutorLauncher(args).run()
+ }
}
}