Skip to content

Commit

Permalink
Merge remote-tracking branch 'apache-github/branch-1.1' into branch-1.1
Browse files Browse the repository at this point in the history
  • Loading branch information
tdas committed Nov 11, 2014
2 parents 2ed927f + 86b1bd0 commit 593214a
Show file tree
Hide file tree
Showing 10 changed files with 222 additions and 50 deletions.
2 changes: 1 addition & 1 deletion core/src/main/scala/org/apache/spark/util/Utils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -1528,7 +1528,7 @@ private[spark] object Utils extends Logging {
def isBindCollision(exception: Throwable): Boolean = {
exception match {
case e: BindException =>
if (e.getMessage != null && e.getMessage.contains("Address already in use")) {
if (e.getMessage != null) {
return true
}
isBindCollision(e.getCause)
Expand Down
25 changes: 14 additions & 11 deletions core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,8 @@ import scala.util.Random
import java.io.{File, ByteArrayOutputStream, ByteArrayInputStream, FileOutputStream}
import java.net.{BindException, ServerSocket, URI}
import java.nio.{ByteBuffer, ByteOrder}
import java.text.DecimalFormatSymbols
import java.util.Locale

import com.google.common.base.Charsets
import com.google.common.io.Files
Expand Down Expand Up @@ -101,14 +103,16 @@ class UtilsSuite extends FunSuite {
val hour = minute * 60
def str = Utils.msDurationToString(_)

val sep = new DecimalFormatSymbols(Locale.getDefault()).getDecimalSeparator()

assert(str(123) === "123 ms")
assert(str(second) === "1.0 s")
assert(str(second + 462) === "1.5 s")
assert(str(hour) === "1.00 h")
assert(str(minute) === "1.0 m")
assert(str(minute + 4 * second + 34) === "1.1 m")
assert(str(10 * hour + minute + 4 * second) === "10.02 h")
assert(str(10 * hour + 59 * minute + 59 * second + 999) === "11.00 h")
assert(str(second) === "1" + sep + "0 s")
assert(str(second + 462) === "1" + sep + "5 s")
assert(str(hour) === "1" + sep + "00 h")
assert(str(minute) === "1" + sep + "0 m")
assert(str(minute + 4 * second + 34) === "1" + sep + "1 m")
assert(str(10 * hour + minute + 4 * second) === "10" + sep + "02 h")
assert(str(10 * hour + 59 * minute + 59 * second + 999) === "11" + sep + "00 h")
}

test("reading offset bytes of a file") {
Expand Down Expand Up @@ -271,12 +275,11 @@ class UtilsSuite extends FunSuite {
assert(!Utils.isBindCollision(new Exception))
assert(!Utils.isBindCollision(new Exception(new Exception)))
assert(!Utils.isBindCollision(new Exception(new BindException)))
assert(!Utils.isBindCollision(new Exception(new BindException("Random message"))))

// Positives
val be = new BindException("Address already in use")
val be1 = new Exception(new BindException("Address already in use"))
val be2 = new Exception(new Exception(new BindException("Address already in use")))
val be = new BindException("Random Message")
val be1 = new Exception(new BindException("Random Message"))
val be2 = new Exception(new Exception(new BindException("Random Message")))
assert(Utils.isBindCollision(be))
assert(Utils.isBindCollision(be1))
assert(Utils.isBindCollision(be2))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,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')
JavaReceiverInputDStream<String> lines = ssc.receiverStream(
new JavaCustomReceiver(args[1], Integer.parseInt(args[2])));
new JavaCustomReceiver(args[0], Integer.parseInt(args[1])));
JavaDStream<String> words = lines.flatMap(new FlatMapFunction<String, String>() {
@Override
public Iterable<String> call(String x) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,8 +35,9 @@

/**
* Counts words in UTF8 encoded, '\n' delimited text received from the network every second.
*
* Usage: JavaNetworkWordCount <hostname> <port>
* <hostname> and <port> describe the TCP server that Spark Streaming would connect to receive data.
* <hostname> and <port> describe the TCP server that Spark Streaming would connect to receive data.
*
* To run this on your local machine, you need to first run a Netcat server
* `$ nc -lk 9999`
Expand All @@ -56,7 +57,7 @@ public static void main(String[] args) {

// Create the context with a 1 second batch size
SparkConf sparkConf = new SparkConf().setAppName("JavaNetworkWordCount");
JavaStreamingContext ssc = new JavaStreamingContext(sparkConf, new Duration(1000));
JavaStreamingContext ssc = new JavaStreamingContext(sparkConf, new Duration(1000));

// Create a JavaReceiverInputDStream on target ip:port and count the
// words in input stream of \n delimited text (eg. generated by 'nc')
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,154 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.examples.streaming;

import java.io.File;
import java.io.IOException;
import java.nio.charset.Charset;
import java.util.Arrays;
import java.util.regex.Pattern;

import scala.Tuple2;
import com.google.common.collect.Lists;
import com.google.common.io.Files;

import org.apache.spark.SparkConf;
import org.apache.spark.api.java.JavaPairRDD;
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.streaming.Duration;
import org.apache.spark.streaming.Time;
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.api.java.JavaStreamingContextFactory;

/**
* Counts words in text encoded with UTF8 received from the network every second.
*
* Usage: JavaRecoverableNetworkWordCount <hostname> <port> <checkpoint-directory> <output-file>
* <hostname> and <port> describe the TCP server that Spark Streaming would connect to receive
* data. <checkpoint-directory> directory to HDFS-compatible file system which checkpoint data
* <output-file> file to which the word counts will be appended
*
* <checkpoint-directory> and <output-file> must be absolute paths
*
* To run this on your local machine, you need to first run a Netcat server
*
* `$ nc -lk 9999`
*
* and run the example as
*
* `$ ./bin/run-example org.apache.spark.examples.streaming.JavaRecoverableNetworkWordCount \
* localhost 9999 ~/checkpoint/ ~/out`
*
* If the directory ~/checkpoint/ does not exist (e.g. running for the first time), it will create
* a new StreamingContext (will print "Creating new context" to the console). Otherwise, if
* checkpoint data exists in ~/checkpoint/, then it will create StreamingContext from
* the checkpoint data.
*
* Refer to the online documentation for more details.
*/
public final class JavaRecoverableNetworkWordCount {
private static final Pattern SPACE = Pattern.compile(" ");

private static JavaStreamingContext createContext(String ip,
int port,
String checkpointDirectory,
String outputPath) {

// If you do not see this printed, that means the StreamingContext has been loaded
// from the new checkpoint
System.out.println("Creating new context");
final File outputFile = new File(outputPath);
if (outputFile.exists()) {
outputFile.delete();
}
SparkConf sparkConf = new SparkConf().setAppName("JavaRecoverableNetworkWordCount");
// Create the context with a 1 second batch size
JavaStreamingContext ssc = new JavaStreamingContext(sparkConf, new Duration(1000));
ssc.checkpoint(checkpointDirectory);

// Create a socket stream on target ip:port and count the
// words in input stream of \n delimited text (eg. generated by 'nc')
JavaReceiverInputDStream<String> lines = ssc.socketTextStream(ip, port);
JavaDStream<String> words = lines.flatMap(new FlatMapFunction<String, String>() {
@Override
public Iterable<String> call(String x) {
return Lists.newArrayList(SPACE.split(x));
}
});
JavaPairDStream<String, Integer> wordCounts = words.mapToPair(
new PairFunction<String, String, Integer>() {
@Override
public Tuple2<String, Integer> call(String s) {
return new Tuple2<String, Integer>(s, 1);
}
}).reduceByKey(new Function2<Integer, Integer, Integer>() {
@Override
public Integer call(Integer i1, Integer i2) {
return i1 + i2;
}
});

wordCounts.foreachRDD(new Function2<JavaPairRDD<String, Integer>, Time, Void>() {
@Override
public Void call(JavaPairRDD<String, Integer> rdd, Time time) throws IOException {
String counts = "Counts at time " + time + " " + rdd.collect();
System.out.println(counts);
System.out.println("Appending to " + outputFile.getAbsolutePath());
Files.append(counts + "\n", outputFile, Charset.defaultCharset());
return null;
}
});

return ssc;
}

public static void main(String[] args) {
if (args.length != 4) {
System.err.println("You arguments were " + Arrays.asList(args));
System.err.println(
"Usage: JavaRecoverableNetworkWordCount <hostname> <port> <checkpoint-directory>\n" +
" <output-file>. <hostname> and <port> describe the TCP server that Spark\n" +
" Streaming would connect to receive data. <checkpoint-directory> directory to\n" +
" HDFS-compatible file system which checkpoint data <output-file> file to which\n" +
" the word counts will be appended\n" +
"\n" +
"In local mode, <master> should be 'local[n]' with n > 1\n" +
"Both <checkpoint-directory> and <output-file> must be absolute paths");
System.exit(1);
}

final String ip = args[0];
final int port = Integer.parseInt(args[1]);
final String checkpointDirectory = args[2];
final String outputPath = args[3];
JavaStreamingContextFactory factory = new JavaStreamingContextFactory() {
@Override
public JavaStreamingContext create() {
return createContext(ip, port, checkpointDirectory, outputPath);
}
};
JavaStreamingContext ssc = JavaStreamingContext.getOrCreate(checkpointDirectory, factory);
ssc.start();
ssc.awaitTermination();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -31,15 +31,13 @@ import org.apache.spark.util.IntParam
/**
* Counts words in text encoded with UTF8 received from the network every second.
*
* Usage: NetworkWordCount <hostname> <port> <checkpoint-directory> <output-file>
* Usage: RecoverableNetworkWordCount <hostname> <port> <checkpoint-directory> <output-file>
* <hostname> and <port> describe the TCP server that Spark Streaming would connect to receive
* data. <checkpoint-directory> directory to HDFS-compatible file system which checkpoint data
* <output-file> file to which the word counts will be appended
*
* In local mode, <master> should be 'local[n]' with n > 1
* <checkpoint-directory> and <output-file> must be absolute paths
*
*
* To run this on your local machine, you need to first run a Netcat server
*
* `$ nc -lk 9999`
Expand All @@ -54,22 +52,11 @@ import org.apache.spark.util.IntParam
* checkpoint data exists in ~/checkpoint/, then it will create StreamingContext from
* the checkpoint data.
*
* To run this example in a local standalone cluster with automatic driver recovery,
*
* `$ bin/spark-class org.apache.spark.deploy.Client -s launch <cluster-url> \
* <path-to-examples-jar> \
* org.apache.spark.examples.streaming.RecoverableNetworkWordCount <cluster-url> \
* localhost 9999 ~/checkpoint ~/out`
*
* <path-to-examples-jar> would typically be
* <spark-dir>/examples/target/scala-XX/spark-examples....jar
*
* Refer to the online documentation for more details.
*/

object RecoverableNetworkWordCount {

def createContext(ip: String, port: Int, outputPath: String) = {
def createContext(ip: String, port: Int, outputPath: String, checkpointDirectory: String) = {

// If you do not see this printed, that means the StreamingContext has been loaded
// from the new checkpoint
Expand All @@ -79,6 +66,7 @@ object RecoverableNetworkWordCount {
val sparkConf = new SparkConf().setAppName("RecoverableNetworkWordCount")
// Create the context with a 1 second batch size
val ssc = new StreamingContext(sparkConf, Seconds(1))
ssc.checkpoint(checkpointDirectory)

// Create a socket stream on target ip:port and count the
// words in input stream of \n delimited text (eg. generated by 'nc')
Expand Down Expand Up @@ -114,7 +102,7 @@ object RecoverableNetworkWordCount {
val Array(ip, IntParam(port), checkpointDirectory, outputPath) = args
val ssc = StreamingContext.getOrCreate(checkpointDirectory,
() => {
createContext(ip, port, outputPath)
createContext(ip, port, outputPath, checkpointDirectory)
})
ssc.start()
ssc.awaitTermination()
Expand Down
2 changes: 2 additions & 0 deletions python/pyspark/rdd.py
Original file line number Diff line number Diff line change
Expand Up @@ -598,6 +598,8 @@ def sortPartition(iterator):
# the key-space into bins such that the bins have roughly the same
# number of (key, value) pairs falling into them
rddSize = self.count()
if not rddSize:
return self # empty RDD
maxSampleSize = numPartitions * 20.0 # constant from Spark's RangePartitioner
fraction = min(maxSampleSize / max(rddSize, 1), 1.0)
samples = self.sample(False, fraction, 1).map(lambda (k, v): k).collect()
Expand Down
3 changes: 3 additions & 0 deletions python/pyspark/tests.py
Original file line number Diff line number Diff line change
Expand Up @@ -470,6 +470,9 @@ def test_histogram(self):
self.assertEquals(([1, "b"], [5]), rdd.histogram(1))
self.assertRaises(TypeError, lambda: rdd.histogram(2))

def test_sort_on_empty_rdd(self):
self.assertEqual([], self.sc.parallelize(zip([], [])).sortByKey().collect())

def test_sample(self):
rdd = self.sc.parallelize(range(0, 100), 4)
wo = rdd.sample(False, 0.1, 2).collect()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -436,10 +436,10 @@ class StreamingContext private[streaming] (

/**
* Start the execution of the streams.
*
* @throws SparkException if the context has already been started or stopped.
*/
def start(): Unit = synchronized {
// Throw exception if the context has already been started once
// or if a stopped context is being started again
if (state == Started) {
throw new SparkException("StreamingContext has already been started")
}
Expand Down Expand Up @@ -472,8 +472,10 @@ class StreamingContext private[streaming] (
/**
* Stop the execution of the streams immediately (does not wait for all received data
* to be processed).
* @param stopSparkContext Stop the associated SparkContext or not
*
* @param stopSparkContext if true, stops the associated SparkContext. The underlying SparkContext
* will be stopped regardless of whether this StreamingContext has been
* started.
*/
def stop(stopSparkContext: Boolean = true): Unit = synchronized {
stop(stopSparkContext, false)
Expand All @@ -482,25 +484,27 @@ class StreamingContext private[streaming] (
/**
* Stop the execution of the streams, with option of ensuring all received data
* has been processed.
* @param stopSparkContext Stop the associated SparkContext or not
* @param stopGracefully Stop gracefully by waiting for the processing of all
*
* @param stopSparkContext if true, stops the associated SparkContext. The underlying SparkContext
* will be stopped regardless of whether this StreamingContext has been
* started.
* @param stopGracefully if true, stops gracefully by waiting for the processing of all
* received data to be completed
*/
def stop(stopSparkContext: Boolean, stopGracefully: Boolean): Unit = synchronized {
// Warn (but not fail) if context is stopped twice,
// or context is stopped before starting
if (state == Initialized) {
logWarning("StreamingContext has not been started yet")
return
state match {
case Initialized => logWarning("StreamingContext has not been started yet")
case Stopped => logWarning("StreamingContext has already been stopped")
case Started =>
scheduler.stop(stopGracefully)
logInfo("StreamingContext stopped successfully")
waiter.notifyStop()
}
if (state == Stopped) {
logWarning("StreamingContext has already been stopped")
return
} // no need to throw an exception as its okay to stop twice
scheduler.stop(stopGracefully)
logInfo("StreamingContext stopped successfully")
waiter.notifyStop()
// Even if the streaming context has not been started, we still need to stop the SparkContext.
// Even if we have already stopped, we still need to attempt to stop the SparkContext because
// a user might stop(stopSparkContext = false) and then call stop(stopSparkContext = true).
if (stopSparkContext) sc.stop()
// The state should always be Stopped after calling `stop()`, even if we haven't started yet:
state = Stopped
}
}
Expand Down
Loading

0 comments on commit 593214a

Please sign in to comment.