diff options
author | Matei Zaharia <matei@eecs.berkeley.edu> | 2013-06-29 14:36:09 -0700 |
---|---|---|
committer | Matei Zaharia <matei@eecs.berkeley.edu> | 2013-06-29 14:36:09 -0700 |
commit | 16671585445f59225a0aab65ff61ed099f3f765e (patch) | |
tree | b1b6fa3a407c80f28cf850f5eebea0c355c2a209 /examples | |
parent | 50ca17635a904f9496ccf996cd2f90325168bb9b (diff) | |
parent | 176193b1e8acdbe2f1cfaed16b8f42f89e226f79 (diff) | |
download | spark-16671585445f59225a0aab65ff61ed099f3f765e.tar.gz spark-16671585445f59225a0aab65ff61ed099f3f765e.tar.bz2 spark-16671585445f59225a0aab65ff61ed099f3f765e.zip |
Merge remote-tracking branch 'mrpotes/master'
Diffstat (limited to 'examples')
3 files changed, 12 insertions, 15 deletions
diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala index a9642100e3..528778ed72 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdCMS.scala @@ -26,8 +26,8 @@ import spark.SparkContext._ */ object TwitterAlgebirdCMS { def main(args: Array[String]) { - if (args.length < 3) { - System.err.println("Usage: TwitterAlgebirdCMS <master> <twitter_username> <twitter_password>" + + if (args.length < 1) { + System.err.println("Usage: TwitterAlgebirdCMS <master>" + " [filter1] [filter2] ... [filter n]") System.exit(1) } @@ -40,12 +40,11 @@ object TwitterAlgebirdCMS { // K highest frequency elements to take val TOPK = 10 - val Array(master, username, password) = args.slice(0, 3) - val filters = args.slice(3, args.length) + val (master, filters) = (args.head, args.tail) val ssc = new StreamingContext(master, "TwitterAlgebirdCMS", Seconds(10), System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) - val stream = ssc.twitterStream(username, password, filters, StorageLevel.MEMORY_ONLY_SER) + val stream = ssc.twitterStream(None, filters, StorageLevel.MEMORY_ONLY_SER) val users = stream.map(status => status.getUser.getId) diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala index f3288bfb85..896e9fd8af 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterAlgebirdHLL.scala @@ -21,20 +21,19 @@ import spark.streaming.dstream.TwitterInputDStream */ object TwitterAlgebirdHLL { def main(args: Array[String]) { - if (args.length < 3) { - System.err.println("Usage: TwitterAlgebirdHLL <master> <twitter_username> <twitter_password>" + + if (args.length < 1) { + System.err.println("Usage: TwitterAlgebirdHLL <master>" + " [filter1] [filter2] ... [filter n]") System.exit(1) } /** Bit size parameter for HyperLogLog, trades off accuracy vs size */ val BIT_SIZE = 12 - val Array(master, username, password) = args.slice(0, 3) - val filters = args.slice(3, args.length) + val (master, filters) = (args.head, args.tail) val ssc = new StreamingContext(master, "TwitterAlgebirdHLL", Seconds(5), System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) - val stream = ssc.twitterStream(username, password, filters, StorageLevel.MEMORY_ONLY_SER) + val stream = ssc.twitterStream(None, filters, StorageLevel.MEMORY_ONLY_SER) val users = stream.map(status => status.getUser.getId) diff --git a/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala index 9d4494c6f2..65f0b6d352 100644 --- a/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala +++ b/examples/src/main/scala/spark/streaming/examples/TwitterPopularTags.scala @@ -12,18 +12,17 @@ import spark.SparkContext._ */ object TwitterPopularTags { def main(args: Array[String]) { - if (args.length < 3) { - System.err.println("Usage: TwitterPopularTags <master> <twitter_username> <twitter_password>" + + if (args.length < 1) { + System.err.println("Usage: TwitterPopularTags <master>" + " [filter1] [filter2] ... [filter n]") System.exit(1) } - val Array(master, username, password) = args.slice(0, 3) - val filters = args.slice(3, args.length) + val (master, filters) = (args.head, args.tail) val ssc = new StreamingContext(master, "TwitterPopularTags", Seconds(2), System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) - val stream = ssc.twitterStream(username, password, filters) + val stream = ssc.twitterStream(None, filters) val hashTags = stream.flatMap(status => status.getText.split(" ").filter(_.startsWith("#"))) |