From 2573add94cf920a88f74d80d8ea94218d812704d Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 24 Dec 2013 18:30:31 +0530 Subject: spark-544, introducing SparkConf and related configuration overhaul. --- .../org/apache/spark/examples/bagel/WikipediaPageRank.scala | 10 ++++++---- .../spark/examples/bagel/WikipediaPageRankStandalone.scala | 8 ++++++-- .../org/apache/spark/streaming/examples/ActorWordCount.scala | 3 ++- 3 files changed, 14 insertions(+), 7 deletions(-) (limited to 'examples') diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala index 72b5c7b88e..12c430be27 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRank.scala @@ -36,16 +36,18 @@ object WikipediaPageRank { System.err.println("Usage: WikipediaPageRank ") System.exit(-1) } - - System.setProperty("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - System.setProperty("spark.kryo.registrator", classOf[PRKryoRegistrator].getName) + val sparkConf = new SparkConf() + sparkConf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + sparkConf.set("spark.kryo.registrator", classOf[PRKryoRegistrator].getName) val inputFile = args(0) val threshold = args(1).toDouble val numPartitions = args(2).toInt val host = args(3) val usePartitioner = args(4).toBoolean - val sc = new SparkContext(host, "WikipediaPageRank") + + sparkConf.setMasterUrl(host).setAppName("WikipediaPageRank") + val sc = new SparkContext(sparkConf) // Parse the Wikipedia page data into a graph val input = sc.textFile(inputFile) diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala index ddf6855325..5bf0b7a24a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala @@ -34,15 +34,19 @@ object WikipediaPageRankStandalone { System.err.println("Usage: WikipediaPageRankStandalone ") System.exit(-1) } + val sparkConf = new SparkConf() + sparkConf.set("spark.serializer", "spark.bagel.examples.WPRSerializer") - System.setProperty("spark.serializer", "spark.bagel.examples.WPRSerializer") val inputFile = args(0) val threshold = args(1).toDouble val numIterations = args(2).toInt val host = args(3) val usePartitioner = args(4).toBoolean - val sc = new SparkContext(host, "WikipediaPageRankStandalone") + + sparkConf.setMasterUrl(host).setAppName("WikipediaPageRankStandalone") + + val sc = new SparkContext(sparkConf) val input = sc.textFile(inputFile) val partitioner = new HashPartitioner(sc.defaultParallelism) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala index 50e3f9639c..2402409e6e 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala @@ -26,6 +26,7 @@ import akka.actor.ActorRef import akka.actor.Props import akka.actor.actorRef2Scala +import org.apache.spark.SparkConf import org.apache.spark.streaming.Seconds import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions @@ -116,7 +117,7 @@ object FeederActor { val Seq(host, port) = args.toSeq - val actorSystem = AkkaUtils.createActorSystem("test", host, port.toInt)._1 + val actorSystem = AkkaUtils.createActorSystem("test", host, port.toInt, conf = new SparkConf)._1 val feeder = actorSystem.actorOf(Props[FeederActor], "FeederActor") println("Feeder started as:" + feeder) -- cgit v1.2.3