diff --git a/dev/audit-release/audit_release.py b/dev/audit-release/audit_release.py index 4dabb51254af7..426b3117f14d4 100755 --- a/dev/audit-release/audit_release.py +++ b/dev/audit-release/audit_release.py @@ -116,8 +116,7 @@ def ensure_path_not_present(path): # dependencies within those projects. modules = [ "spark-core", "spark-mllib", "spark-streaming", "spark-repl", - "spark-graphx", "spark-streaming-flume", "spark-streaming-kafka", - "spark-streaming-mqtt", "spark-streaming-twitter", "spark-streaming-zeromq", + "spark-graphx", "spark-streaming-kafka", "spark-catalyst", "spark-sql", "spark-hive", "spark-streaming-kinesis-asl" ] modules = map(lambda m: "%s_%s" % (m, SCALA_BINARY_VERSION), modules) diff --git a/dev/run-tests.py b/dev/run-tests.py index a1e6f1bdb560e..d940cdad3e278 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -337,9 +337,6 @@ def build_spark_sbt(hadoop_version): build_profiles = get_hadoop_profiles(hadoop_version) + modules.root.build_profile_flags sbt_goals = ["package", "streaming-kafka-assembly/assembly", - "streaming-flume-assembly/assembly", - "streaming-mqtt-assembly/assembly", - "streaming-mqtt/test:assembly", "streaming-kinesis-asl-assembly/assembly"] profiles_and_goals = build_profiles + sbt_goals diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 1781de4c657ce..d1184886e2c19 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -210,43 +210,6 @@ def __hash__(self): ) -streaming_zeromq = Module( - name="streaming-zeromq", - dependencies=[streaming], - source_file_regexes=[ - "external/zeromq", - ], - sbt_test_goals=[ - "streaming-zeromq/test", - ] -) - - -streaming_twitter = Module( - name="streaming-twitter", - dependencies=[streaming], - source_file_regexes=[ - "external/twitter", - ], - sbt_test_goals=[ - "streaming-twitter/test", - ] -) - - -streaming_mqtt = Module( - name="streaming-mqtt", - dependencies=[streaming], - source_file_regexes=[ - "external/mqtt", - "external/mqtt-assembly", - ], - sbt_test_goals=[ - "streaming-mqtt/test", - ] -) - - streaming_kafka = Module( name="streaming-kafka", dependencies=[streaming], @@ -260,51 +223,6 @@ def __hash__(self): ) -streaming_flume_sink = Module( - name="streaming-flume-sink", - dependencies=[streaming], - source_file_regexes=[ - "external/flume-sink", - ], - sbt_test_goals=[ - "streaming-flume-sink/test", - ] -) - - -streaming_akka = Module( - name="streaming-akka", - dependencies=[streaming], - source_file_regexes=[ - "external/akka", - ], - sbt_test_goals=[ - "streaming-akka/test", - ] -) - - -streaming_flume = Module( - name="streaming-flume", - dependencies=[streaming], - source_file_regexes=[ - "external/flume", - ], - sbt_test_goals=[ - "streaming-flume/test", - ] -) - - -streaming_flume_assembly = Module( - name="streaming-flume-assembly", - dependencies=[streaming_flume, streaming_flume_sink], - source_file_regexes=[ - "external/flume-assembly", - ] -) - - mllib = Module( name="mllib", dependencies=[streaming, sql], @@ -376,8 +294,6 @@ def __hash__(self): pyspark_core, streaming, streaming_kafka, - streaming_flume_assembly, - streaming_mqtt, streaming_kinesis_asl ], source_file_regexes=[ diff --git a/examples/pom.xml b/examples/pom.xml index 3a3f547915015..92bb373c7382d 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -65,37 +65,6 @@ ${project.version} provided - - org.apache.spark - spark-streaming-twitter_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-streaming-flume_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-streaming-akka_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-streaming-mqtt_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-streaming-zeromq_${scala.binary.version} - ${project.version} - - - org.spark-project.protobuf - protobuf-java - - - org.apache.spark spark-streaming-kafka_${scala.binary.version} diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaActorWordCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaActorWordCount.java deleted file mode 100644 index 7884b8cdfff84..0000000000000 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaActorWordCount.java +++ /dev/null @@ -1,144 +0,0 @@ -/* - * 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.util.Arrays; -import java.util.Iterator; - -import scala.Tuple2; - -import akka.actor.ActorSelection; -import akka.actor.Props; - -import org.apache.spark.SparkConf; -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.api.java.JavaDStream; -import org.apache.spark.streaming.api.java.JavaStreamingContext; -import org.apache.spark.streaming.akka.AkkaUtils; -import org.apache.spark.streaming.akka.JavaActorReceiver; - -/** - * A sample actor as receiver, is also simplest. This receiver actor - * goes and subscribe to a typical publisher/feeder actor and receives - * data. - * - * @see [[org.apache.spark.examples.streaming.FeederActor]] - */ -class JavaSampleActorReceiver extends JavaActorReceiver { - - private final String urlOfPublisher; - - public JavaSampleActorReceiver(String urlOfPublisher) { - this.urlOfPublisher = urlOfPublisher; - } - - private ActorSelection remotePublisher; - - @Override - public void preStart() { - remotePublisher = getContext().actorSelection(urlOfPublisher); - remotePublisher.tell(new SubscribeReceiver(getSelf()), getSelf()); - } - - @Override - public void onReceive(Object msg) throws Exception { - @SuppressWarnings("unchecked") - T msgT = (T) msg; - store(msgT); - } - - @Override - public void postStop() { - remotePublisher.tell(new UnsubscribeReceiver(getSelf()), getSelf()); - } -} - -/** - * A sample word count program demonstrating the use of plugging in - * Actor as Receiver - * Usage: JavaActorWordCount - * 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.examples.streaming.FeederActor localhost 9999
- * 
- * and then run the example - *
- *     $ bin/run-example org.apache.spark.examples.streaming.JavaActorWordCount localhost 9999
- * 
- */ -public class JavaActorWordCount { - - public static void main(String[] args) { - if (args.length < 2) { - System.err.println("Usage: JavaActorWordCount "); - System.exit(1); - } - - StreamingExamples.setStreamingLogLevels(); - - final String host = args[0]; - final String port = args[1]; - SparkConf sparkConf = new SparkConf().setAppName("JavaActorWordCount"); - // Create the context and set the batch size - JavaStreamingContext jssc = new JavaStreamingContext(sparkConf, new Duration(2000)); - - String feederActorURI = "akka.tcp://test@" + host + ":" + port + "/user/FeederActor"; - - /* - * Following is the use of AkkaUtils.createStream to plug in custom actor as receiver - * - * An important point to note: - * Since Actor may exist outside the spark framework, It is thus user's responsibility - * to ensure the type safety, i.e type of data received and InputDstream - * should be same. - * - * For example: Both AkkaUtils.createStream and JavaSampleActorReceiver are parameterized - * to same type to ensure type safety. - */ - JavaDStream lines = AkkaUtils.createStream( - jssc, - Props.create(JavaSampleActorReceiver.class, feederActorURI), - "SampleReceiver"); - - // compute wordcount - lines.flatMap(new FlatMapFunction() { - @Override - public Iterator call(String s) { - return Arrays.asList(s.split("\\s+")).iterator(); - } - }).mapToPair(new PairFunction() { - @Override - public Tuple2 call(String s) { - return new Tuple2<>(s, 1); - } - }).reduceByKey(new Function2() { - @Override - public Integer call(Integer i1, Integer i2) { - return i1 + i2; - } - }).print(); - - jssc.start(); - jssc.awaitTermination(); - } -} diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaFlumeEventCount.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaFlumeEventCount.java deleted file mode 100644 index da56637fe891a..0000000000000 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaFlumeEventCount.java +++ /dev/null @@ -1,75 +0,0 @@ -/* - * 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 org.apache.spark.SparkConf; -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; -import org.apache.spark.streaming.flume.SparkFlumeEvent; - -/** - * Produces a count of events received from Flume. - * - * This should be used in conjunction with an AvroSink in Flume. It will start - * an Avro server on at the request host:port address and listen for requests. - * Your Flume AvroSink should be pointed to this address. - * - * Usage: JavaFlumeEventCount - * is the host the Flume receiver will be started on - a receiver - * creates a server and listens for flume events. - * is the port the Flume receiver will listen on. - * - * To run this example: - * `$ bin/run-example org.apache.spark.examples.streaming.JavaFlumeEventCount ` - */ -public final class JavaFlumeEventCount { - private JavaFlumeEventCount() { - } - - public static void main(String[] args) { - if (args.length != 2) { - System.err.println("Usage: JavaFlumeEventCount "); - System.exit(1); - } - - StreamingExamples.setStreamingLogLevels(); - - String host = args[0]; - int port = Integer.parseInt(args[1]); - - Duration batchInterval = new Duration(2000); - SparkConf sparkConf = new SparkConf().setAppName("JavaFlumeEventCount"); - JavaStreamingContext ssc = new JavaStreamingContext(sparkConf, batchInterval); - JavaReceiverInputDStream flumeStream = FlumeUtils.createStream(ssc, host, port); - - flumeStream.count(); - - flumeStream.count().map(new Function() { - @Override - public String call(Long in) { - return "Received " + in + " flume events."; - } - }).print(); - - ssc.start(); - ssc.awaitTermination(); - } -} diff --git a/examples/src/main/java/org/apache/spark/examples/streaming/JavaTwitterHashTagJoinSentiments.java b/examples/src/main/java/org/apache/spark/examples/streaming/JavaTwitterHashTagJoinSentiments.java deleted file mode 100644 index f0ae9a99bae47..0000000000000 --- a/examples/src/main/java/org/apache/spark/examples/streaming/JavaTwitterHashTagJoinSentiments.java +++ /dev/null @@ -1,175 +0,0 @@ -/* - * 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 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.Function; -import org.apache.spark.api.java.function.Function2; -import org.apache.spark.api.java.function.PairFunction; -import org.apache.spark.api.java.function.VoidFunction; -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.twitter.TwitterUtils; -import scala.Tuple2; -import twitter4j.Status; - -import java.util.Arrays; -import java.util.Iterator; -import java.util.List; - -/** - * Displays the most positive hash tags by joining the streaming Twitter data with a static RDD of - * the AFINN word list (http://neuro.imm.dtu.dk/wiki/AFINN) - */ -public class JavaTwitterHashTagJoinSentiments { - - public static void main(String[] args) { - if (args.length < 4) { - System.err.println("Usage: JavaTwitterHashTagJoinSentiments " + - " []"); - System.exit(1); - } - - StreamingExamples.setStreamingLogLevels(); - - String consumerKey = args[0]; - String consumerSecret = args[1]; - String accessToken = args[2]; - String accessTokenSecret = args[3]; - String[] filters = Arrays.copyOfRange(args, 4, args.length); - - // Set the system properties so that Twitter4j library used by Twitter stream - // can use them to generate OAuth credentials - System.setProperty("twitter4j.oauth.consumerKey", consumerKey); - System.setProperty("twitter4j.oauth.consumerSecret", consumerSecret); - System.setProperty("twitter4j.oauth.accessToken", accessToken); - System.setProperty("twitter4j.oauth.accessTokenSecret", accessTokenSecret); - - SparkConf sparkConf = new SparkConf().setAppName("JavaTwitterHashTagJoinSentiments"); - JavaStreamingContext jssc = new JavaStreamingContext(sparkConf, new Duration(2000)); - JavaReceiverInputDStream stream = TwitterUtils.createStream(jssc, filters); - - JavaDStream words = stream.flatMap(new FlatMapFunction() { - @Override - public Iterator call(Status s) { - return Arrays.asList(s.getText().split(" ")).iterator(); - } - }); - - JavaDStream hashTags = words.filter(new Function() { - @Override - public Boolean call(String word) { - return word.startsWith("#"); - } - }); - - // Read in the word-sentiment list and create a static RDD from it - String wordSentimentFilePath = "data/streaming/AFINN-111.txt"; - final JavaPairRDD wordSentiments = jssc.sparkContext().textFile(wordSentimentFilePath) - .mapToPair(new PairFunction(){ - @Override - public Tuple2 call(String line) { - String[] columns = line.split("\t"); - return new Tuple2<>(columns[0], Double.parseDouble(columns[1])); - } - }); - - JavaPairDStream hashTagCount = hashTags.mapToPair( - new PairFunction() { - @Override - public Tuple2 call(String s) { - // leave out the # character - return new Tuple2<>(s.substring(1), 1); - } - }); - - JavaPairDStream hashTagTotals = hashTagCount.reduceByKeyAndWindow( - new Function2() { - @Override - public Integer call(Integer a, Integer b) { - return a + b; - } - }, new Duration(10000)); - - // Determine the hash tags with the highest sentiment values by joining the streaming RDD - // with the static RDD inside the transform() method and then multiplying - // the frequency of the hash tag by its sentiment value - JavaPairDStream> joinedTuples = - hashTagTotals.transformToPair(new Function, - JavaPairRDD>>() { - @Override - public JavaPairRDD> call( - JavaPairRDD topicCount) { - return wordSentiments.join(topicCount); - } - }); - - JavaPairDStream topicHappiness = joinedTuples.mapToPair( - new PairFunction>, String, Double>() { - @Override - public Tuple2 call(Tuple2> topicAndTuplePair) { - Tuple2 happinessAndCount = topicAndTuplePair._2(); - return new Tuple2<>(topicAndTuplePair._1(), - happinessAndCount._1() * happinessAndCount._2()); - } - }); - - JavaPairDStream happinessTopicPairs = topicHappiness.mapToPair( - new PairFunction, Double, String>() { - @Override - public Tuple2 call(Tuple2 topicHappiness) { - return new Tuple2<>(topicHappiness._2(), - topicHappiness._1()); - } - }); - - JavaPairDStream happiest10 = happinessTopicPairs.transformToPair( - new Function, JavaPairRDD>() { - @Override - public JavaPairRDD call( - JavaPairRDD happinessAndTopics) { - return happinessAndTopics.sortByKey(false); - } - } - ); - - // Print hash tags with the most positive sentiment values - happiest10.foreachRDD(new VoidFunction>() { - @Override - public void call(JavaPairRDD happinessTopicPairs) { - List> topList = happinessTopicPairs.take(10); - System.out.println( - String.format("\nHappiest topics in last 10 seconds (%s total):", - happinessTopicPairs.count())); - for (Tuple2 pair : topList) { - System.out.println( - String.format("%s (%s happiness)", pair._2(), pair._1())); - } - } - }); - - jssc.start(); - jssc.awaitTermination(); - } -} diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala deleted file mode 100644 index 844772a289284..0000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/ActorWordCount.scala +++ /dev/null @@ -1,175 +0,0 @@ -/* - * 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. - */ - -// scalastyle:off println -package org.apache.spark.examples.streaming - -import scala.collection.mutable.LinkedHashSet -import scala.util.Random - -import akka.actor._ -import com.typesafe.config.ConfigFactory - -import org.apache.spark.SparkConf -import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.akka.{ActorReceiver, AkkaUtils} - -case class SubscribeReceiver(receiverActor: ActorRef) -case class UnsubscribeReceiver(receiverActor: ActorRef) - -/** - * Sends the random content to every receiver subscribed with 1/2 - * second delay. - */ -class FeederActor extends Actor { - - val rand = new Random() - val receivers = new LinkedHashSet[ActorRef]() - - val strings: Array[String] = Array("words ", "may ", "count ") - - def makeMessage(): String = { - val x = rand.nextInt(3) - strings(x) + strings(2 - x) - } - - /* - * A thread to generate random messages - */ - new Thread() { - override def run() { - while (true) { - Thread.sleep(500) - receivers.foreach(_ ! makeMessage) - } - } - }.start() - - def receive: Receive = { - case SubscribeReceiver(receiverActor: ActorRef) => - println("received subscribe from %s".format(receiverActor.toString)) - receivers += receiverActor - - case UnsubscribeReceiver(receiverActor: ActorRef) => - println("received unsubscribe from %s".format(receiverActor.toString)) - receivers -= receiverActor - } -} - -/** - * A sample actor as receiver, is also simplest. This receiver actor - * goes and subscribe to a typical publisher/feeder actor and receives - * data. - * - * @see [[org.apache.spark.examples.streaming.FeederActor]] - */ -class SampleActorReceiver[T](urlOfPublisher: String) extends ActorReceiver { - - lazy private val remotePublisher = context.actorSelection(urlOfPublisher) - - override def preStart(): Unit = remotePublisher ! SubscribeReceiver(context.self) - - def receive: PartialFunction[Any, Unit] = { - case msg => store(msg.asInstanceOf[T]) - } - - override def postStop(): Unit = remotePublisher ! UnsubscribeReceiver(context.self) - -} - -/** - * A sample feeder actor - * - * Usage: FeederActor - * and describe the AkkaSystem that Spark Sample feeder would start on. - */ -object FeederActor { - - def main(args: Array[String]) { - if (args.length < 2) { - System.err.println("Usage: FeederActor \n") - System.exit(1) - } - val Seq(host, port) = args.toSeq - - val akkaConf = ConfigFactory.parseString( - s"""akka.actor.provider = "akka.remote.RemoteActorRefProvider" - |akka.remote.enabled-transports = ["akka.remote.netty.tcp"] - |akka.remote.netty.tcp.hostname = "$host" - |akka.remote.netty.tcp.port = $port - |""".stripMargin) - val actorSystem = ActorSystem("test", akkaConf) - val feeder = actorSystem.actorOf(Props[FeederActor], "FeederActor") - - println("Feeder started as:" + feeder) - - actorSystem.awaitTermination() - } -} - -/** - * A sample word count program demonstrating the use of plugging in - * - * Actor as Receiver - * Usage: ActorWordCount - * 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.examples.streaming.FeederActor localhost 9999` - * and then run the example - * `$ bin/run-example org.apache.spark.examples.streaming.ActorWordCount localhost 9999` - */ -object ActorWordCount { - def main(args: Array[String]) { - if (args.length < 2) { - System.err.println( - "Usage: ActorWordCount ") - System.exit(1) - } - - StreamingExamples.setStreamingLogLevels() - - val Seq(host, port) = args.toSeq - val sparkConf = new SparkConf().setAppName("ActorWordCount") - // Create the context and set the batch size - val ssc = new StreamingContext(sparkConf, Seconds(2)) - - /* - * Following is the use of AkkaUtils.createStream to plug in custom actor as receiver - * - * An important point to note: - * Since Actor may exist outside the spark framework, It is thus user's responsibility - * to ensure the type safety, i.e type of data received and InputDStream - * should be same. - * - * For example: Both AkkaUtils.createStream and SampleActorReceiver are parameterized - * to same type to ensure type safety. - */ - val lines = AkkaUtils.createStream[String]( - ssc, - Props(classOf[SampleActorReceiver[String]], - "akka.tcp://test@%s:%s/user/FeederActor".format(host, port.toInt)), - "SampleReceiver") - - // compute wordcount - lines.flatMap(_.split("\\s+")).map(x => (x, 1)).reduceByKey(_ + _).print() - - ssc.start() - ssc.awaitTermination() - } -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala deleted file mode 100644 index 91e52e4eff5a7..0000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/FlumeEventCount.scala +++ /dev/null @@ -1,70 +0,0 @@ -/* - * 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. - */ - -// scalastyle:off println -package org.apache.spark.examples.streaming - -import org.apache.spark.SparkConf -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming._ -import org.apache.spark.streaming.flume._ -import org.apache.spark.util.IntParam - -/** - * Produces a count of events received from Flume. - * - * This should be used in conjunction with an AvroSink in Flume. It will start - * an Avro server on at the request host:port address and listen for requests. - * Your Flume AvroSink should be pointed to this address. - * - * Usage: FlumeEventCount - * is the host the Flume receiver will be started on - a receiver - * creates a server and listens for flume events. - * is the port the Flume receiver will listen on. - * - * To run this example: - * `$ bin/run-example org.apache.spark.examples.streaming.FlumeEventCount ` - */ -object FlumeEventCount { - def main(args: Array[String]) { - if (args.length < 2) { - System.err.println( - "Usage: FlumeEventCount ") - System.exit(1) - } - - StreamingExamples.setStreamingLogLevels() - - val Array(host, IntParam(port)) = args - - val batchInterval = Milliseconds(2000) - - // Create the context and set the batch size - val sparkConf = new SparkConf().setAppName("FlumeEventCount") - val ssc = new StreamingContext(sparkConf, batchInterval) - - // Create a flume stream - val stream = FlumeUtils.createStream(ssc, host, port, StorageLevel.MEMORY_ONLY_SER_2) - - // Print out the count of events received from this server in each batch - stream.count().map(cnt => "Received " + cnt + " flume events." ).print() - - ssc.start() - ssc.awaitTermination() - } -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/FlumePollingEventCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/FlumePollingEventCount.scala deleted file mode 100644 index dd725d72c23ef..0000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/FlumePollingEventCount.scala +++ /dev/null @@ -1,67 +0,0 @@ -/* - * 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. - */ - -// scalastyle:off println -package org.apache.spark.examples.streaming - -import org.apache.spark.SparkConf -import org.apache.spark.streaming._ -import org.apache.spark.streaming.flume._ -import org.apache.spark.util.IntParam - -/** - * Produces a count of events received from Flume. - * - * This should be used in conjunction with the Spark Sink running in a Flume agent. See - * the Spark Streaming programming guide for more details. - * - * Usage: FlumePollingEventCount - * `host` is the host on which the Spark Sink is running. - * `port` is the port at which the Spark Sink is listening. - * - * To run this example: - * `$ bin/run-example org.apache.spark.examples.streaming.FlumePollingEventCount [host] [port] ` - */ -object FlumePollingEventCount { - def main(args: Array[String]) { - if (args.length < 2) { - System.err.println( - "Usage: FlumePollingEventCount ") - System.exit(1) - } - - StreamingExamples.setStreamingLogLevels() - - val Array(host, IntParam(port)) = args - - val batchInterval = Milliseconds(2000) - - // Create the context and set the batch size - val sparkConf = new SparkConf().setAppName("FlumePollingEventCount") - val ssc = new StreamingContext(sparkConf, batchInterval) - - // Create a flume stream that polls the Spark Sink running in a Flume agent - val stream = FlumeUtils.createPollingStream(ssc, host, port) - - // Print out the count of events received from this server in each batch - stream.count().map(cnt => "Received " + cnt + " flume events." ).print() - - ssc.start() - ssc.awaitTermination() - } -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala deleted file mode 100644 index d772ae309f40d..0000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/MQTTWordCount.scala +++ /dev/null @@ -1,119 +0,0 @@ -/* - * 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. - */ - -// scalastyle:off println -package org.apache.spark.examples.streaming - -import org.eclipse.paho.client.mqttv3._ -import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence - -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.mqtt._ -import org.apache.spark.SparkConf - -/** - * A simple Mqtt publisher for demonstration purposes, repeatedly publishes - * Space separated String Message "hello mqtt demo for spark streaming" - */ -object MQTTPublisher { - - def main(args: Array[String]) { - if (args.length < 2) { - System.err.println("Usage: MQTTPublisher ") - System.exit(1) - } - - StreamingExamples.setStreamingLogLevels() - - val Seq(brokerUrl, topic) = args.toSeq - - var client: MqttClient = null - - try { - val persistence = new MemoryPersistence() - client = new MqttClient(brokerUrl, MqttClient.generateClientId(), persistence) - - client.connect() - - val msgtopic = client.getTopic(topic) - val msgContent = "hello mqtt demo for spark streaming" - val message = new MqttMessage(msgContent.getBytes("utf-8")) - - while (true) { - try { - msgtopic.publish(message) - println(s"Published data. topic: ${msgtopic.getName()}; Message: $message") - } catch { - case e: MqttException if e.getReasonCode == MqttException.REASON_CODE_MAX_INFLIGHT => - Thread.sleep(10) - println("Queue is full, wait for to consume data from the message queue") - } - } - } catch { - case e: MqttException => println("Exception Caught: " + e) - } finally { - if (client != null) { - client.disconnect() - } - } - } -} - -/** - * A sample wordcount with MqttStream stream - * - * To work with Mqtt, Mqtt Message broker/server required. - * Mosquitto (http://mosquitto.org/) is an open source Mqtt Broker - * In ubuntu mosquitto can be installed using the command `$ sudo apt-get install mosquitto` - * Eclipse paho project provides Java library for Mqtt Client http://www.eclipse.org/paho/ - * Example Java code for Mqtt Publisher and Subscriber can be found here - * https://bitbucket.org/mkjinesh/mqttclient - * Usage: MQTTWordCount - * and describe where Mqtt publisher is running. - * - * To run this example locally, you may run publisher as - * `$ bin/run-example \ - * org.apache.spark.examples.streaming.MQTTPublisher tcp://localhost:1883 foo` - * and run the example as - * `$ bin/run-example \ - * org.apache.spark.examples.streaming.MQTTWordCount tcp://localhost:1883 foo` - */ -object MQTTWordCount { - - def main(args: Array[String]) { - if (args.length < 2) { - // scalastyle:off println - System.err.println( - "Usage: MQTTWordCount ") - // scalastyle:on println - System.exit(1) - } - - val Seq(brokerUrl, topic) = args.toSeq - val sparkConf = new SparkConf().setAppName("MQTTWordCount") - val ssc = new StreamingContext(sparkConf, Seconds(2)) - val lines = MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_ONLY_SER_2) - val words = lines.flatMap(x => x.split(" ")) - val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) - - wordCounts.print() - ssc.start() - ssc.awaitTermination() - } -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala deleted file mode 100644 index 5af82e161a2f7..0000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdCMS.scala +++ /dev/null @@ -1,116 +0,0 @@ -/* - * 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. - */ - -// scalastyle:off println -package org.apache.spark.examples.streaming - -import com.twitter.algebird._ -import com.twitter.algebird.CMSHasherImplicits._ - -import org.apache.spark.SparkConf -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.twitter._ - -// scalastyle:off -/** - * Illustrates the use of the Count-Min Sketch, from Twitter's Algebird library, to compute - * windowed and global Top-K estimates of user IDs occurring in a Twitter stream. - *
- * Note that since Algebird's implementation currently only supports Long inputs, - * the example operates on Long IDs. Once the implementation supports other inputs (such as String), - * the same approach could be used for computing popular topics for example. - *

- *

- * - * This blog post has a good overview of the Count-Min Sketch (CMS). The CMS is a data - * structure for approximate frequency estimation in data streams (e.g. Top-K elements, frequency - * of any given element, etc), that uses space sub-linear in the number of elements in the - * stream. Once elements are added to the CMS, the estimated count of an element can be computed, - * as well as "heavy-hitters" that occur more than a threshold percentage of the overall total - * count. - *

- * Algebird's implementation is a monoid, so we can succinctly merge two CMS instances in the - * reduce operation. - */ -// scalastyle:on -object TwitterAlgebirdCMS { - def main(args: Array[String]) { - StreamingExamples.setStreamingLogLevels() - - // CMS parameters - val DELTA = 1E-3 - val EPS = 0.01 - val SEED = 1 - val PERC = 0.001 - // K highest frequency elements to take - val TOPK = 10 - - val filters = args - val sparkConf = new SparkConf().setAppName("TwitterAlgebirdCMS") - val ssc = new StreamingContext(sparkConf, Seconds(10)) - val stream = TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_ONLY_SER_2) - - val users = stream.map(status => status.getUser.getId) - - // val cms = new CountMinSketchMonoid(EPS, DELTA, SEED, PERC) - val cms = TopPctCMS.monoid[Long](EPS, DELTA, SEED, PERC) - var globalCMS = cms.zero - val mm = new MapMonoid[Long, Int]() - var globalExact = Map[Long, Int]() - - val approxTopUsers = users.mapPartitions(ids => { - ids.map(id => cms.create(id)) - }).reduce(_ ++ _) - - val exactTopUsers = users.map(id => (id, 1)) - .reduceByKey((a, b) => a + b) - - approxTopUsers.foreachRDD(rdd => { - if (rdd.count() != 0) { - val partial = rdd.first() - val partialTopK = partial.heavyHitters.map(id => - (id, partial.frequency(id).estimate)).toSeq.sortBy(_._2).reverse.slice(0, TOPK) - globalCMS ++= partial - val globalTopK = globalCMS.heavyHitters.map(id => - (id, globalCMS.frequency(id).estimate)).toSeq.sortBy(_._2).reverse.slice(0, TOPK) - println("Approx heavy hitters at %2.2f%% threshold this batch: %s".format(PERC, - partialTopK.mkString("[", ",", "]"))) - println("Approx heavy hitters at %2.2f%% threshold overall: %s".format(PERC, - globalTopK.mkString("[", ",", "]"))) - } - }) - - exactTopUsers.foreachRDD(rdd => { - if (rdd.count() != 0) { - val partialMap = rdd.collect().toMap - val partialTopK = rdd.map( - {case (id, count) => (count, id)}) - .sortByKey(ascending = false).take(TOPK) - globalExact = mm.plus(globalExact.toMap, partialMap) - val globalTopK = globalExact.toSeq.sortBy(_._2).reverse.slice(0, TOPK) - println("Exact heavy hitters this batch: %s".format(partialTopK.mkString("[", ",", "]"))) - println("Exact heavy hitters overall: %s".format(globalTopK.mkString("[", ",", "]"))) - } - }) - - ssc.start() - ssc.awaitTermination() - } -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala deleted file mode 100644 index 6442b2a4e294b..0000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterAlgebirdHLL.scala +++ /dev/null @@ -1,94 +0,0 @@ -/* - * 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. - */ - -// scalastyle:off println -package org.apache.spark.examples.streaming - -import com.twitter.algebird.HyperLogLog._ -import com.twitter.algebird.HyperLogLogMonoid - -import org.apache.spark.SparkConf -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.twitter._ - -// scalastyle:off -/** - * Illustrates the use of the HyperLogLog algorithm, from Twitter's Algebird library, to compute - * a windowed and global estimate of the unique user IDs occurring in a Twitter stream. - *

- *

- * This - * blog post and this - * - * blog post - * have good overviews of HyperLogLog (HLL). HLL is a memory-efficient datastructure for - * estimating the cardinality of a data stream, i.e. the number of unique elements. - *

- * Algebird's implementation is a monoid, so we can succinctly merge two HLL instances in the - * reduce operation. - */ -// scalastyle:on -object TwitterAlgebirdHLL { - def main(args: Array[String]) { - - StreamingExamples.setStreamingLogLevels() - - /** Bit size parameter for HyperLogLog, trades off accuracy vs size */ - val BIT_SIZE = 12 - val filters = args - val sparkConf = new SparkConf().setAppName("TwitterAlgebirdHLL") - val ssc = new StreamingContext(sparkConf, Seconds(5)) - val stream = TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_ONLY_SER) - - val users = stream.map(status => status.getUser.getId) - - val hll = new HyperLogLogMonoid(BIT_SIZE) - var globalHll = hll.zero - var userSet: Set[Long] = Set() - - val approxUsers = users.mapPartitions(ids => { - ids.map(id => hll.create(id)) - }).reduce(_ + _) - - val exactUsers = users.map(id => Set(id)).reduce(_ ++ _) - - approxUsers.foreachRDD(rdd => { - if (rdd.count() != 0) { - val partial = rdd.first() - globalHll += partial - println("Approx distinct users this batch: %d".format(partial.estimatedSize.toInt)) - println("Approx distinct users overall: %d".format(globalHll.estimatedSize.toInt)) - } - }) - - exactUsers.foreachRDD(rdd => { - if (rdd.count() != 0) { - val partial = rdd.first() - userSet ++= partial - println("Exact distinct users this batch: %d".format(partial.size)) - println("Exact distinct users overall: %d".format(userSet.size)) - println("Error rate: %2.5f%%".format(((globalHll.estimatedSize / userSet.size.toDouble) - 1 - ) * 100)) - } - }) - - ssc.start() - ssc.awaitTermination() - } -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterHashTagJoinSentiments.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterHashTagJoinSentiments.scala deleted file mode 100644 index a8d392ca35b40..0000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterHashTagJoinSentiments.scala +++ /dev/null @@ -1,96 +0,0 @@ -/* - * 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. - */ - -// scalastyle:off println -package org.apache.spark.examples.streaming - -import org.apache.spark.SparkConf -import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.twitter.TwitterUtils - -/** - * Displays the most positive hash tags by joining the streaming Twitter data with a static RDD of - * the AFINN word list (http://neuro.imm.dtu.dk/wiki/AFINN) - */ -object TwitterHashTagJoinSentiments { - def main(args: Array[String]) { - if (args.length < 4) { - System.err.println("Usage: TwitterHashTagJoinSentiments " + - " []") - System.exit(1) - } - - StreamingExamples.setStreamingLogLevels() - - val Array(consumerKey, consumerSecret, accessToken, accessTokenSecret) = args.take(4) - val filters = args.takeRight(args.length - 4) - - // Set the system properties so that Twitter4j library used by Twitter stream - // can use them to generate OAuth credentials - System.setProperty("twitter4j.oauth.consumerKey", consumerKey) - System.setProperty("twitter4j.oauth.consumerSecret", consumerSecret) - System.setProperty("twitter4j.oauth.accessToken", accessToken) - System.setProperty("twitter4j.oauth.accessTokenSecret", accessTokenSecret) - - val sparkConf = new SparkConf().setAppName("TwitterHashTagJoinSentiments") - val ssc = new StreamingContext(sparkConf, Seconds(2)) - val stream = TwitterUtils.createStream(ssc, None, filters) - - val hashTags = stream.flatMap(status => status.getText.split(" ").filter(_.startsWith("#"))) - - // Read in the word-sentiment list and create a static RDD from it - val wordSentimentFilePath = "data/streaming/AFINN-111.txt" - val wordSentiments = ssc.sparkContext.textFile(wordSentimentFilePath).map { line => - val Array(word, happinessValue) = line.split("\t") - (word, happinessValue.toInt) - }.cache() - - // Determine the hash tags with the highest sentiment values by joining the streaming RDD - // with the static RDD inside the transform() method and then multiplying - // the frequency of the hash tag by its sentiment value - val happiest60 = hashTags.map(hashTag => (hashTag.tail, 1)) - .reduceByKeyAndWindow(_ + _, Seconds(60)) - .transform{topicCount => wordSentiments.join(topicCount)} - .map{case (topic, tuple) => (topic, tuple._1 * tuple._2)} - .map{case (topic, happinessValue) => (happinessValue, topic)} - .transform(_.sortByKey(false)) - - val happiest10 = hashTags.map(hashTag => (hashTag.tail, 1)) - .reduceByKeyAndWindow(_ + _, Seconds(10)) - .transform{topicCount => wordSentiments.join(topicCount)} - .map{case (topic, tuple) => (topic, tuple._1 * tuple._2)} - .map{case (topic, happinessValue) => (happinessValue, topic)} - .transform(_.sortByKey(false)) - - // Print hash tags with the most positive sentiment values - happiest60.foreachRDD(rdd => { - val topList = rdd.take(10) - println("\nHappiest topics in last 60 seconds (%s total):".format(rdd.count())) - topList.foreach{case (happiness, tag) => println("%s (%s happiness)".format(tag, happiness))} - }) - - happiest10.foreachRDD(rdd => { - val topList = rdd.take(10) - println("\nHappiest topics in last 10 seconds (%s total):".format(rdd.count())) - topList.foreach{case (happiness, tag) => println("%s (%s happiness)".format(tag, happiness))} - }) - - ssc.start() - ssc.awaitTermination() - } -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala deleted file mode 100644 index 5b69963cc8880..0000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/TwitterPopularTags.scala +++ /dev/null @@ -1,85 +0,0 @@ -/* - * 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. - */ - -// scalastyle:off println -package org.apache.spark.examples.streaming - -import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.twitter._ -import org.apache.spark.SparkConf - -/** - * Calculates popular hashtags (topics) over sliding 10 and 60 second windows from a Twitter - * stream. The stream is instantiated with credentials and optionally filters supplied by the - * command line arguments. - * - * Run this on your local machine as - * - */ -object TwitterPopularTags { - def main(args: Array[String]) { - if (args.length < 4) { - System.err.println("Usage: TwitterPopularTags " + - " []") - System.exit(1) - } - - StreamingExamples.setStreamingLogLevels() - - val Array(consumerKey, consumerSecret, accessToken, accessTokenSecret) = args.take(4) - val filters = args.takeRight(args.length - 4) - - // Set the system properties so that Twitter4j library used by twitter stream - // can use them to generate OAuth credentials - System.setProperty("twitter4j.oauth.consumerKey", consumerKey) - System.setProperty("twitter4j.oauth.consumerSecret", consumerSecret) - System.setProperty("twitter4j.oauth.accessToken", accessToken) - System.setProperty("twitter4j.oauth.accessTokenSecret", accessTokenSecret) - - val sparkConf = new SparkConf().setAppName("TwitterPopularTags") - val ssc = new StreamingContext(sparkConf, Seconds(2)) - val stream = TwitterUtils.createStream(ssc, None, filters) - - val hashTags = stream.flatMap(status => status.getText.split(" ").filter(_.startsWith("#"))) - - val topCounts60 = hashTags.map((_, 1)).reduceByKeyAndWindow(_ + _, Seconds(60)) - .map{case (topic, count) => (count, topic)} - .transform(_.sortByKey(false)) - - val topCounts10 = hashTags.map((_, 1)).reduceByKeyAndWindow(_ + _, Seconds(10)) - .map{case (topic, count) => (count, topic)} - .transform(_.sortByKey(false)) - - - // Print popular hashtags - topCounts60.foreachRDD(rdd => { - val topList = rdd.take(10) - println("\nPopular topics in last 60 seconds (%s total):".format(rdd.count())) - topList.foreach{case (count, tag) => println("%s (%s tweets)".format(tag, count))} - }) - - topCounts10.foreachRDD(rdd => { - val topList = rdd.take(10) - println("\nPopular topics in last 10 seconds (%s total):".format(rdd.count())) - topList.foreach{case (count, tag) => println("%s (%s tweets)".format(tag, count))} - }) - - ssc.start() - ssc.awaitTermination() - } -} -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala deleted file mode 100644 index 99b561750bf9f..0000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/ZeroMQWordCount.scala +++ /dev/null @@ -1,105 +0,0 @@ -/* - * 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. - */ - -// scalastyle:off println -package org.apache.spark.examples.streaming - -import scala.language.implicitConversions - -import akka.actor.ActorSystem -import akka.actor.actorRef2Scala -import akka.util.ByteString -import akka.zeromq._ -import akka.zeromq.Subscribe - -import org.apache.spark.SparkConf -import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.zeromq._ - -/** - * A simple publisher for demonstration purposes, repeatedly publishes random Messages - * every one second. - */ -object SimpleZeroMQPublisher { - - def main(args: Array[String]): Unit = { - if (args.length < 2) { - System.err.println("Usage: SimpleZeroMQPublisher ") - System.exit(1) - } - - val Seq(url, topic) = args.toSeq - val acs: ActorSystem = ActorSystem() - - val pubSocket = ZeroMQExtension(acs).newSocket(SocketType.Pub, Bind(url)) - implicit def stringToByteString(x: String): ByteString = ByteString(x) - val messages: List[ByteString] = List("words ", "may ", "count ") - while (true) { - Thread.sleep(1000) - pubSocket ! ZMQMessage(ByteString(topic) :: messages) - } - acs.awaitTermination() - } -} - -// scalastyle:off -/** - * A sample wordcount with ZeroMQStream stream - * - * To work with zeroMQ, some native libraries have to be installed. - * Install zeroMQ (release 2.1) core libraries. [ZeroMQ Install guide] - * (http://www.zeromq.org/intro:get-the-software) - * - * Usage: ZeroMQWordCount - * and describe where zeroMq publisher is running. - * - * To run this example locally, you may run publisher as - * `$ bin/run-example \ - * org.apache.spark.examples.streaming.SimpleZeroMQPublisher tcp://127.0.0.1:1234 foo` - * and run the example as - * `$ bin/run-example \ - * org.apache.spark.examples.streaming.ZeroMQWordCount tcp://127.0.0.1:1234 foo` - */ -// scalastyle:on -object ZeroMQWordCount { - def main(args: Array[String]) { - if (args.length < 2) { - System.err.println("Usage: ZeroMQWordCount ") - System.exit(1) - } - StreamingExamples.setStreamingLogLevels() - val Seq(url, topic) = args.toSeq - val sparkConf = new SparkConf().setAppName("ZeroMQWordCount") - // Create the context and set the batch size - val ssc = new StreamingContext(sparkConf, Seconds(2)) - - def bytesToStringIterator(x: Seq[ByteString]): Iterator[String] = x.map(_.utf8String).iterator - - // For this stream, a zeroMQ publisher should be running. - val lines = ZeroMQUtils.createStream( - ssc, - url, - Subscribe(topic), - bytesToStringIterator _) - val words = lines.flatMap(_.split(" ")) - val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) - wordCounts.print() - ssc.start() - ssc.awaitTermination() - } -} -// scalastyle:on println diff --git a/external/akka/pom.xml b/external/akka/pom.xml deleted file mode 100644 index bbe644e3b32b3..0000000000000 --- a/external/akka/pom.xml +++ /dev/null @@ -1,70 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent_2.11 - 2.0.0-SNAPSHOT - ../../pom.xml - - - org.apache.spark - spark-streaming-akka_2.11 - - streaming-akka - - jar - Spark Project External Akka - http://spark.apache.org/ - - - - org.apache.spark - spark-streaming_${scala.binary.version} - ${project.version} - provided - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - test-jar - test - - - org.apache.spark - spark-test-tags_${scala.binary.version} - - - ${akka.group} - akka-actor_${scala.binary.version} - ${akka.version} - - - ${akka.group} - akka-remote_${scala.binary.version} - ${akka.version} - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - diff --git a/external/akka/src/main/scala/org/apache/spark/streaming/akka/ActorReceiver.scala b/external/akka/src/main/scala/org/apache/spark/streaming/akka/ActorReceiver.scala deleted file mode 100644 index 33415c15be2ef..0000000000000 --- a/external/akka/src/main/scala/org/apache/spark/streaming/akka/ActorReceiver.scala +++ /dev/null @@ -1,306 +0,0 @@ -/* - * 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.streaming.akka - -import java.nio.ByteBuffer -import java.util.concurrent.atomic.AtomicInteger - -import scala.concurrent.Future -import scala.concurrent.duration._ -import scala.language.postfixOps -import scala.reflect.ClassTag - -import akka.actor._ -import akka.actor.SupervisorStrategy.{Escalate, Restart} -import akka.pattern.ask -import akka.util.Timeout -import com.typesafe.config.ConfigFactory - -import org.apache.spark.{Logging, TaskContext} -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.receiver.Receiver - -/** - * :: DeveloperApi :: - * A helper with set of defaults for supervisor strategy - */ -@DeveloperApi -object ActorReceiver { - - /** - * A OneForOneStrategy supervisor strategy with `maxNrOfRetries = 10` and - * `withinTimeRange = 15 millis`. For RuntimeException, it will restart the ActorReceiver; for - * others, it just escalates the failure to the supervisor of the supervisor. - */ - val defaultSupervisorStrategy = OneForOneStrategy(maxNrOfRetries = 10, withinTimeRange = - 15 millis) { - case _: RuntimeException => Restart - case _: Exception => Escalate - } - - /** - * A default ActorSystem creator. It will use a unique system name - * (streaming-actor-system-) to start an ActorSystem that supports remote - * communication. - */ - val defaultActorSystemCreator: () => ActorSystem = () => { - val uniqueSystemName = s"streaming-actor-system-${TaskContext.get().taskAttemptId()}" - val akkaConf = ConfigFactory.parseString( - s"""akka.actor.provider = "akka.remote.RemoteActorRefProvider" - |akka.remote.enabled-transports = ["akka.remote.netty.tcp"] - |""".stripMargin) - ActorSystem(uniqueSystemName, akkaConf) - } -} - -/** - * :: DeveloperApi :: - * A base Actor that provides APIs for pushing received data into Spark Streaming for processing. - * - * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html - * - * @example {{{ - * class MyActor extends ActorReceiver { - * def receive { - * case anything: String => store(anything) - * } - * } - * - * AkkaUtils.createStream[String](ssc, Props[MyActor](),"MyActorReceiver") - * - * }}} - * - * @note Since Actor may exist outside the spark framework, It is thus user's responsibility - * to ensure the type safety, i.e. parametrized type of push block and InputDStream - * should be same. - */ -@DeveloperApi -abstract class ActorReceiver extends Actor { - - /** Store an iterator of received data as a data block into Spark's memory. */ - def store[T](iter: Iterator[T]) { - context.parent ! IteratorData(iter) - } - - /** - * Store the bytes of received data as a data block into Spark's memory. Note - * that the data in the ByteBuffer must be serialized using the same serializer - * that Spark is configured to use. - */ - def store(bytes: ByteBuffer) { - context.parent ! ByteBufferData(bytes) - } - - /** - * Store a single item of received data to Spark's memory asynchronously. - * These single items will be aggregated together into data blocks before - * being pushed into Spark's memory. - */ - def store[T](item: T) { - context.parent ! SingleItemData(item) - } - - /** - * Store a single item of received data to Spark's memory and returns a `Future`. - * The `Future` will be completed when the operator finishes, or with an - * `akka.pattern.AskTimeoutException` after the given timeout has expired. - * These single items will be aggregated together into data blocks before - * being pushed into Spark's memory. - * - * This method allows the user to control the flow speed using `Future` - */ - def store[T](item: T, timeout: Timeout): Future[Unit] = { - context.parent.ask(AskStoreSingleItemData(item))(timeout).map(_ => ())(context.dispatcher) - } -} - -/** - * :: DeveloperApi :: - * A Java UntypedActor that provides APIs for pushing received data into Spark Streaming for - * processing. - * - * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html - * - * @example {{{ - * class MyActor extends JavaActorReceiver { - * @Override - * public void onReceive(Object msg) throws Exception { - * store((String) msg); - * } - * } - * - * AkkaUtils.createStream(jssc, Props.create(MyActor.class), "MyActorReceiver"); - * - * }}} - * - * @note Since Actor may exist outside the spark framework, It is thus user's responsibility - * to ensure the type safety, i.e. parametrized type of push block and InputDStream - * should be same. - */ -@DeveloperApi -abstract class JavaActorReceiver extends UntypedActor { - - /** Store an iterator of received data as a data block into Spark's memory. */ - def store[T](iter: Iterator[T]) { - context.parent ! IteratorData(iter) - } - - /** - * Store the bytes of received data as a data block into Spark's memory. Note - * that the data in the ByteBuffer must be serialized using the same serializer - * that Spark is configured to use. - */ - def store(bytes: ByteBuffer) { - context.parent ! ByteBufferData(bytes) - } - - /** - * Store a single item of received data to Spark's memory. - * These single items will be aggregated together into data blocks before - * being pushed into Spark's memory. - */ - def store[T](item: T) { - context.parent ! SingleItemData(item) - } - - /** - * Store a single item of received data to Spark's memory and returns a `Future`. - * The `Future` will be completed when the operator finishes, or with an - * `akka.pattern.AskTimeoutException` after the given timeout has expired. - * These single items will be aggregated together into data blocks before - * being pushed into Spark's memory. - * - * This method allows the user to control the flow speed using `Future` - */ - def store[T](item: T, timeout: Timeout): Future[Unit] = { - context.parent.ask(AskStoreSingleItemData(item))(timeout).map(_ => ())(context.dispatcher) - } -} - -/** - * :: DeveloperApi :: - * Statistics for querying the supervisor about state of workers. Used in - * conjunction with `AkkaUtils.createStream` and - * [[org.apache.spark.streaming.akka.ActorReceiverSupervisor]]. - */ -@DeveloperApi -case class Statistics(numberOfMsgs: Int, - numberOfWorkers: Int, - numberOfHiccups: Int, - otherInfo: String) - -/** Case class to receive data sent by child actors */ -private[akka] sealed trait ActorReceiverData -private[akka] case class SingleItemData[T](item: T) extends ActorReceiverData -private[akka] case class AskStoreSingleItemData[T](item: T) extends ActorReceiverData -private[akka] case class IteratorData[T](iterator: Iterator[T]) extends ActorReceiverData -private[akka] case class ByteBufferData(bytes: ByteBuffer) extends ActorReceiverData -private[akka] object Ack extends ActorReceiverData - -/** - * Provides Actors as receivers for receiving stream. - * - * As Actors can also be used to receive data from almost any stream source. - * A nice set of abstraction(s) for actors as receivers is already provided for - * a few general cases. It is thus exposed as an API where user may come with - * their own Actor to run as receiver for Spark Streaming input source. - * - * This starts a supervisor actor which starts workers and also provides - * [http://doc.akka.io/docs/akka/snapshot/scala/fault-tolerance.html fault-tolerance]. - * - * Here's a way to start more supervisor/workers as its children. - * - * @example {{{ - * context.parent ! Props(new Supervisor) - * }}} OR {{{ - * context.parent ! Props(new Worker, "Worker") - * }}} - */ -private[akka] class ActorReceiverSupervisor[T: ClassTag]( - actorSystemCreator: () => ActorSystem, - props: Props, - name: String, - storageLevel: StorageLevel, - receiverSupervisorStrategy: SupervisorStrategy - ) extends Receiver[T](storageLevel) with Logging { - - private lazy val actorSystem = actorSystemCreator() - protected lazy val actorSupervisor = actorSystem.actorOf(Props(new Supervisor), - "Supervisor" + streamId) - - class Supervisor extends Actor { - - override val supervisorStrategy = receiverSupervisorStrategy - private val worker = context.actorOf(props, name) - logInfo("Started receiver worker at:" + worker.path) - - private val n: AtomicInteger = new AtomicInteger(0) - private val hiccups: AtomicInteger = new AtomicInteger(0) - - override def receive: PartialFunction[Any, Unit] = { - - case IteratorData(iterator) => - logDebug("received iterator") - store(iterator.asInstanceOf[Iterator[T]]) - - case SingleItemData(msg) => - logDebug("received single") - store(msg.asInstanceOf[T]) - n.incrementAndGet - - case AskStoreSingleItemData(msg) => - logDebug("received single sync") - store(msg.asInstanceOf[T]) - n.incrementAndGet - sender() ! Ack - - case ByteBufferData(bytes) => - logDebug("received bytes") - store(bytes) - - case props: Props => - val worker = context.actorOf(props) - logInfo("Started receiver worker at:" + worker.path) - sender ! worker - - case (props: Props, name: String) => - val worker = context.actorOf(props, name) - logInfo("Started receiver worker at:" + worker.path) - sender ! worker - - case _: PossiblyHarmful => hiccups.incrementAndGet() - - case _: Statistics => - val workers = context.children - sender ! Statistics(n.get, workers.size, hiccups.get, workers.mkString("\n")) - - } - } - - def onStart(): Unit = { - actorSupervisor - logInfo("Supervision tree for receivers initialized at:" + actorSupervisor.path) - } - - def onStop(): Unit = { - actorSupervisor ! PoisonPill - actorSystem.shutdown() - actorSystem.awaitTermination() - } -} diff --git a/external/akka/src/main/scala/org/apache/spark/streaming/akka/AkkaUtils.scala b/external/akka/src/main/scala/org/apache/spark/streaming/akka/AkkaUtils.scala deleted file mode 100644 index 38c35c5ae7a18..0000000000000 --- a/external/akka/src/main/scala/org/apache/spark/streaming/akka/AkkaUtils.scala +++ /dev/null @@ -1,147 +0,0 @@ -/* - * 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.streaming.akka - -import scala.reflect.ClassTag - -import akka.actor.{ActorSystem, Props, SupervisorStrategy} - -import org.apache.spark.api.java.function.{Function0 => JFunction0} -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaStreamingContext} -import org.apache.spark.streaming.dstream.ReceiverInputDStream - -object AkkaUtils { - - /** - * Create an input stream with a user-defined actor. See [[ActorReceiver]] for more details. - * - * @param ssc The StreamingContext instance - * @param propsForActor Props object defining creation of the actor - * @param actorName Name of the actor - * @param storageLevel RDD storage level (default: StorageLevel.MEMORY_AND_DISK_SER_2) - * @param actorSystemCreator A function to create ActorSystem in executors. `ActorSystem` will - * be shut down when the receiver is stopping (default: - * ActorReceiver.defaultActorSystemCreator) - * @param supervisorStrategy the supervisor strategy (default: ActorReceiver.defaultStrategy) - * - * @note An important point to note: - * Since Actor may exist outside the spark framework, It is thus user's responsibility - * to ensure the type safety, i.e. parametrized type of data received and createStream - * should be same. - */ - def createStream[T: ClassTag]( - ssc: StreamingContext, - propsForActor: Props, - actorName: String, - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2, - actorSystemCreator: () => ActorSystem = ActorReceiver.defaultActorSystemCreator, - supervisorStrategy: SupervisorStrategy = ActorReceiver.defaultSupervisorStrategy - ): ReceiverInputDStream[T] = ssc.withNamedScope("actor stream") { - val cleanF = ssc.sc.clean(actorSystemCreator) - ssc.receiverStream(new ActorReceiverSupervisor[T]( - cleanF, - propsForActor, - actorName, - storageLevel, - supervisorStrategy)) - } - - /** - * Create an input stream with a user-defined actor. See [[JavaActorReceiver]] for more details. - * - * @param jssc The StreamingContext instance - * @param propsForActor Props object defining creation of the actor - * @param actorName Name of the actor - * @param storageLevel Storage level to use for storing the received objects - * @param actorSystemCreator A function to create ActorSystem in executors. `ActorSystem` will - * be shut down when the receiver is stopping. - * @param supervisorStrategy the supervisor strategy - * - * @note An important point to note: - * Since Actor may exist outside the spark framework, It is thus user's responsibility - * to ensure the type safety, i.e. parametrized type of data received and createStream - * should be same. - */ - def createStream[T]( - jssc: JavaStreamingContext, - propsForActor: Props, - actorName: String, - storageLevel: StorageLevel, - actorSystemCreator: JFunction0[ActorSystem], - supervisorStrategy: SupervisorStrategy - ): JavaReceiverInputDStream[T] = { - implicit val cm: ClassTag[T] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - createStream[T]( - jssc.ssc, - propsForActor, - actorName, - storageLevel, - () => actorSystemCreator.call(), - supervisorStrategy) - } - - /** - * Create an input stream with a user-defined actor. See [[JavaActorReceiver]] for more details. - * - * @param jssc The StreamingContext instance - * @param propsForActor Props object defining creation of the actor - * @param actorName Name of the actor - * @param storageLevel Storage level to use for storing the received objects - * - * @note An important point to note: - * Since Actor may exist outside the spark framework, It is thus user's responsibility - * to ensure the type safety, i.e. parametrized type of data received and createStream - * should be same. - */ - def createStream[T]( - jssc: JavaStreamingContext, - propsForActor: Props, - actorName: String, - storageLevel: StorageLevel - ): JavaReceiverInputDStream[T] = { - implicit val cm: ClassTag[T] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - createStream[T](jssc.ssc, propsForActor, actorName, storageLevel) - } - - /** - * Create an input stream with a user-defined actor. Storage level of the data will be the default - * StorageLevel.MEMORY_AND_DISK_SER_2. See [[JavaActorReceiver]] for more details. - * - * @param jssc The StreamingContext instance - * @param propsForActor Props object defining creation of the actor - * @param actorName Name of the actor - * - * @note An important point to note: - * Since Actor may exist outside the spark framework, It is thus user's responsibility - * to ensure the type safety, i.e. parametrized type of data received and createStream - * should be same. - */ - def createStream[T]( - jssc: JavaStreamingContext, - propsForActor: Props, - actorName: String - ): JavaReceiverInputDStream[T] = { - implicit val cm: ClassTag[T] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - createStream[T](jssc.ssc, propsForActor, actorName) - } -} diff --git a/external/akka/src/test/java/org/apache/spark/streaming/akka/JavaAkkaUtilsSuite.java b/external/akka/src/test/java/org/apache/spark/streaming/akka/JavaAkkaUtilsSuite.java deleted file mode 100644 index ac5ef31c8b355..0000000000000 --- a/external/akka/src/test/java/org/apache/spark/streaming/akka/JavaAkkaUtilsSuite.java +++ /dev/null @@ -1,68 +0,0 @@ -/* - * 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.streaming.akka; - -import akka.actor.ActorSystem; -import akka.actor.Props; -import akka.actor.SupervisorStrategy; -import akka.util.Timeout; -import org.apache.spark.streaming.Duration; -import org.apache.spark.streaming.api.java.JavaStreamingContext; -import org.junit.Test; - -import org.apache.spark.api.java.function.Function0; -import org.apache.spark.storage.StorageLevel; -import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; - -public class JavaAkkaUtilsSuite { - - @Test // tests the API, does not actually test data receiving - public void testAkkaUtils() { - JavaStreamingContext jsc = new JavaStreamingContext("local[2]", "test", new Duration(1000)); - try { - JavaReceiverInputDStream test1 = AkkaUtils.createStream( - jsc, Props.create(JavaTestActor.class), "test"); - JavaReceiverInputDStream test2 = AkkaUtils.createStream( - jsc, Props.create(JavaTestActor.class), "test", StorageLevel.MEMORY_AND_DISK_SER_2()); - JavaReceiverInputDStream test3 = AkkaUtils.createStream( - jsc, - Props.create(JavaTestActor.class), - "test", StorageLevel.MEMORY_AND_DISK_SER_2(), - new ActorSystemCreatorForTest(), - SupervisorStrategy.defaultStrategy()); - } finally { - jsc.stop(); - } - } -} - -class ActorSystemCreatorForTest implements Function0 { - @Override - public ActorSystem call() { - return null; - } -} - - -class JavaTestActor extends JavaActorReceiver { - @Override - public void onReceive(Object message) throws Exception { - store((String) message); - store((String) message, new Timeout(1000)); - } -} diff --git a/external/akka/src/test/scala/org/apache/spark/streaming/akka/AkkaUtilsSuite.scala b/external/akka/src/test/scala/org/apache/spark/streaming/akka/AkkaUtilsSuite.scala deleted file mode 100644 index ce95d9dd72f90..0000000000000 --- a/external/akka/src/test/scala/org/apache/spark/streaming/akka/AkkaUtilsSuite.scala +++ /dev/null @@ -1,67 +0,0 @@ -/* - * 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.streaming.akka - -import scala.concurrent.duration._ - -import akka.actor.{Props, SupervisorStrategy} - -import org.apache.spark.SparkFunSuite -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.dstream.ReceiverInputDStream - -class AkkaUtilsSuite extends SparkFunSuite { - - test("createStream") { - val ssc: StreamingContext = new StreamingContext("local[2]", "test", Seconds(1000)) - try { - // tests the API, does not actually test data receiving - val test1: ReceiverInputDStream[String] = AkkaUtils.createStream( - ssc, Props[TestActor](), "test") - val test2: ReceiverInputDStream[String] = AkkaUtils.createStream( - ssc, Props[TestActor](), "test", StorageLevel.MEMORY_AND_DISK_SER_2) - val test3: ReceiverInputDStream[String] = AkkaUtils.createStream( - ssc, - Props[TestActor](), - "test", - StorageLevel.MEMORY_AND_DISK_SER_2, - supervisorStrategy = SupervisorStrategy.defaultStrategy) - val test4: ReceiverInputDStream[String] = AkkaUtils.createStream( - ssc, Props[TestActor](), "test", StorageLevel.MEMORY_AND_DISK_SER_2, () => null) - val test5: ReceiverInputDStream[String] = AkkaUtils.createStream( - ssc, Props[TestActor](), "test", StorageLevel.MEMORY_AND_DISK_SER_2, () => null) - val test6: ReceiverInputDStream[String] = AkkaUtils.createStream( - ssc, - Props[TestActor](), - "test", - StorageLevel.MEMORY_AND_DISK_SER_2, - () => null, - SupervisorStrategy.defaultStrategy) - } finally { - ssc.stop() - } - } -} - -class TestActor extends ActorReceiver { - override def receive: Receive = { - case m: String => store(m) - case m => store(m, 10.seconds) - } -} diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml deleted file mode 100644 index ac15b93c048da..0000000000000 --- a/external/flume-assembly/pom.xml +++ /dev/null @@ -1,168 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent_2.11 - 2.0.0-SNAPSHOT - ../../pom.xml - - - org.apache.spark - spark-streaming-flume-assembly_2.11 - jar - Spark Project External Flume Assembly - http://spark.apache.org/ - - - provided - streaming-flume-assembly - - - - - org.apache.spark - spark-streaming-flume_${scala.binary.version} - ${project.version} - - - org.mortbay.jetty - jetty - - - org.mortbay.jetty - jetty-util - - - - - org.apache.spark - spark-streaming_${scala.binary.version} - ${project.version} - provided - - - - commons-codec - commons-codec - provided - - - commons-lang - commons-lang - provided - - - commons-net - commons-net - provided - - - com.google.protobuf - protobuf-java - provided - - - org.apache.avro - avro - provided - - - org.apache.avro - avro-ipc - provided - - - org.apache.avro - avro-mapred - ${avro.mapred.classifier} - provided - - - org.scala-lang - scala-library - provided - - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - org.apache.maven.plugins - maven-shade-plugin - - false - - - *:* - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - - package - - shade - - - - - - reference.conf - - - log4j.properties - - - - - - - - - - - - - - flume-provided - - provided - - - - - diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml deleted file mode 100644 index e4effe158c826..0000000000000 --- a/external/flume-sink/pom.xml +++ /dev/null @@ -1,129 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent_2.11 - 2.0.0-SNAPSHOT - ../../pom.xml - - - org.apache.spark - spark-streaming-flume-sink_2.11 - - streaming-flume-sink - - jar - Spark Project External Flume Sink - http://spark.apache.org/ - - - - org.apache.flume - flume-ng-sdk - - - - com.google.guava - guava - - - - org.apache.thrift - libthrift - - - - - org.apache.flume - flume-ng-core - - - com.google.guava - guava - - - org.apache.thrift - libthrift - - - - - org.scala-lang - scala-library - - - - com.google.guava - guava - test - - - - io.netty - netty - 3.4.0.Final - test - - - org.apache.spark - spark-test-tags_${scala.binary.version} - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - org.apache.avro - avro-maven-plugin - ${avro.version} - - - ${project.basedir}/target/scala-${scala.binary.version}/src_managed/main/compiled_avro - - - - generate-sources - - idl-protocol - - - - - - org.apache.maven.plugins - maven-shade-plugin - - - - - - - - diff --git a/external/flume-sink/src/main/avro/sparkflume.avdl b/external/flume-sink/src/main/avro/sparkflume.avdl deleted file mode 100644 index 8806e863ac7c6..0000000000000 --- a/external/flume-sink/src/main/avro/sparkflume.avdl +++ /dev/null @@ -1,40 +0,0 @@ -/** - * 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. - */ - -@namespace("org.apache.spark.streaming.flume.sink") - -protocol SparkFlumeProtocol { - - record SparkSinkEvent { - map headers; - bytes body; - } - - record EventBatch { - string errorMsg = ""; // If this is empty it is a valid message, else it represents an error - string sequenceNumber; - array events; - } - - EventBatch getEventBatch (int n); - - void ack (string sequenceNumber); - - void nack (string sequenceNumber); -} diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala deleted file mode 100644 index aa530a7121bd0..0000000000000 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala +++ /dev/null @@ -1,127 +0,0 @@ -/* - * 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.streaming.flume.sink - -import org.slf4j.{Logger, LoggerFactory} - -/** - * Copy of the org.apache.spark.Logging for being used in the Spark Sink. - * The org.apache.spark.Logging is not used so that all of Spark is not brought - * in as a dependency. - */ -private[sink] trait Logging { - // Make the log field transient so that objects with Logging can - // be serialized and used on another machine - @transient private var _log: Logger = null - - // Method to get or create the logger for this object - protected def log: Logger = { - if (_log == null) { - initializeIfNecessary() - var className = this.getClass.getName - // Ignore trailing $'s in the class names for Scala objects - if (className.endsWith("$")) { - className = className.substring(0, className.length - 1) - } - _log = LoggerFactory.getLogger(className) - } - _log - } - - // Log methods that take only a String - protected def logInfo(msg: => String) { - if (log.isInfoEnabled) log.info(msg) - } - - protected def logDebug(msg: => String) { - if (log.isDebugEnabled) log.debug(msg) - } - - protected def logTrace(msg: => String) { - if (log.isTraceEnabled) log.trace(msg) - } - - protected def logWarning(msg: => String) { - if (log.isWarnEnabled) log.warn(msg) - } - - protected def logError(msg: => String) { - if (log.isErrorEnabled) log.error(msg) - } - - // Log methods that take Throwables (Exceptions/Errors) too - protected def logInfo(msg: => String, throwable: Throwable) { - if (log.isInfoEnabled) log.info(msg, throwable) - } - - protected def logDebug(msg: => String, throwable: Throwable) { - if (log.isDebugEnabled) log.debug(msg, throwable) - } - - protected def logTrace(msg: => String, throwable: Throwable) { - if (log.isTraceEnabled) log.trace(msg, throwable) - } - - protected def logWarning(msg: => String, throwable: Throwable) { - if (log.isWarnEnabled) log.warn(msg, throwable) - } - - protected def logError(msg: => String, throwable: Throwable) { - if (log.isErrorEnabled) log.error(msg, throwable) - } - - protected def isTraceEnabled(): Boolean = { - log.isTraceEnabled - } - - private def initializeIfNecessary() { - if (!Logging.initialized) { - Logging.initLock.synchronized { - if (!Logging.initialized) { - initializeLogging() - } - } - } - } - - private def initializeLogging() { - Logging.initialized = true - - // Force a call into slf4j to initialize it. Avoids this happening from mutliple threads - // and triggering this: http://mailman.qos.ch/pipermail/slf4j-dev/2010-April/002956.html - log - } -} - -private[sink] object Logging { - @volatile private var initialized = false - val initLock = new Object() - try { - // We use reflection here to handle the case where users remove the - // slf4j-to-jul bridge order to route their logs to JUL. - // scalastyle:off classforname - val bridgeClass = Class.forName("org.slf4j.bridge.SLF4JBridgeHandler") - // scalastyle:on classforname - bridgeClass.getMethod("removeHandlersForRootLogger").invoke(null) - val installed = bridgeClass.getMethod("isInstalled").invoke(null).asInstanceOf[Boolean] - if (!installed) { - bridgeClass.getMethod("install").invoke(null) - } - } catch { - case e: ClassNotFoundException => // can't log anything yet so just fail silently - } -} diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala deleted file mode 100644 index 719fca0938b3a..0000000000000 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala +++ /dev/null @@ -1,166 +0,0 @@ -/* - * 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.streaming.flume.sink - -import java.util.UUID -import java.util.concurrent.{CountDownLatch, Executors} -import java.util.concurrent.atomic.AtomicLong - -import scala.collection.mutable - -import org.apache.flume.Channel - -/** - * Class that implements the SparkFlumeProtocol, that is used by the Avro Netty Server to process - * requests. Each getEvents, ack and nack call is forwarded to an instance of this class. - * @param threads Number of threads to use to process requests. - * @param channel The channel that the sink pulls events from - * @param transactionTimeout Timeout in millis after which the transaction if not acked by Spark - * is rolled back. - */ -// Flume forces transactions to be thread-local. So each transaction *must* be committed, or -// rolled back from the thread it was originally created in. So each getEvents call from Spark -// creates a TransactionProcessor which runs in a new thread, in which the transaction is created -// and events are pulled off the channel. Once the events are sent to spark, -// that thread is blocked and the TransactionProcessor is saved in a map, -// until an ACK or NACK comes back or the transaction times out (after the specified timeout). -// When the response comes or a timeout is hit, the TransactionProcessor is retrieved and then -// unblocked, at which point the transaction is committed or rolled back. - -private[flume] class SparkAvroCallbackHandler(val threads: Int, val channel: Channel, - val transactionTimeout: Int, val backOffInterval: Int) extends SparkFlumeProtocol with Logging { - val transactionExecutorOpt = Option(Executors.newFixedThreadPool(threads, - new SparkSinkThreadFactory("Spark Sink Processor Thread - %d"))) - // Protected by `sequenceNumberToProcessor` - private val sequenceNumberToProcessor = mutable.HashMap[CharSequence, TransactionProcessor]() - // This sink will not persist sequence numbers and reuses them if it gets restarted. - // So it is possible to commit a transaction which may have been meant for the sink before the - // restart. - // Since the new txn may not have the same sequence number we must guard against accidentally - // committing a new transaction. To reduce the probability of that happening a random string is - // prepended to the sequence number. Does not change for life of sink - private val seqBase = UUID.randomUUID().toString.substring(0, 8) - private val seqCounter = new AtomicLong(0) - - // Protected by `sequenceNumberToProcessor` - private var stopped = false - - @volatile private var isTest = false - private var testLatch: CountDownLatch = null - - /** - * Returns a bunch of events to Spark over Avro RPC. - * @param n Maximum number of events to return in a batch - * @return [[EventBatch]] instance that has a sequence number and an array of at most n events - */ - override def getEventBatch(n: Int): EventBatch = { - logDebug("Got getEventBatch call from Spark.") - val sequenceNumber = seqBase + seqCounter.incrementAndGet() - createProcessor(sequenceNumber, n) match { - case Some(processor) => - transactionExecutorOpt.foreach(_.submit(processor)) - // Wait until a batch is available - will be an error if error message is non-empty - val batch = processor.getEventBatch - if (SparkSinkUtils.isErrorBatch(batch)) { - // Remove the processor if it is an error batch since no ACK is sent. - removeAndGetProcessor(sequenceNumber) - logWarning("Received an error batch - no events were received from channel! ") - } - batch - case None => - new EventBatch("Spark sink has been stopped!", "", java.util.Collections.emptyList()) - } - } - - private def createProcessor(seq: String, n: Int): Option[TransactionProcessor] = { - sequenceNumberToProcessor.synchronized { - if (!stopped) { - val processor = new TransactionProcessor( - channel, seq, n, transactionTimeout, backOffInterval, this) - sequenceNumberToProcessor.put(seq, processor) - if (isTest) { - processor.countDownWhenBatchAcked(testLatch) - } - Some(processor) - } else { - None - } - } - } - - /** - * Called by Spark to indicate successful commit of a batch - * @param sequenceNumber The sequence number of the event batch that was successful - */ - override def ack(sequenceNumber: CharSequence): Void = { - logDebug("Received Ack for batch with sequence number: " + sequenceNumber) - completeTransaction(sequenceNumber, success = true) - null - } - - /** - * Called by Spark to indicate failed commit of a batch - * @param sequenceNumber The sequence number of the event batch that failed - * @return - */ - override def nack(sequenceNumber: CharSequence): Void = { - completeTransaction(sequenceNumber, success = false) - logInfo("Spark failed to commit transaction. Will reattempt events.") - null - } - - /** - * Helper method to commit or rollback a transaction. - * @param sequenceNumber The sequence number of the batch that was completed - * @param success Whether the batch was successful or not. - */ - private def completeTransaction(sequenceNumber: CharSequence, success: Boolean) { - removeAndGetProcessor(sequenceNumber).foreach(processor => { - processor.batchProcessed(success) - }) - } - - /** - * Helper method to remove the TxnProcessor for a Sequence Number. Can be used to avoid a leak. - * @param sequenceNumber - * @return An `Option` of the transaction processor for the corresponding batch. Note that this - * instance is no longer tracked and the caller is responsible for that txn processor. - */ - private[sink] def removeAndGetProcessor(sequenceNumber: CharSequence): - Option[TransactionProcessor] = { - sequenceNumberToProcessor.synchronized { - sequenceNumberToProcessor.remove(sequenceNumber.toString) - } - } - - private[sink] def countDownWhenBatchAcked(latch: CountDownLatch) { - testLatch = latch - isTest = true - } - - /** - * Shuts down the executor used to process transactions. - */ - def shutdown() { - logInfo("Shutting down Spark Avro Callback Handler") - sequenceNumberToProcessor.synchronized { - stopped = true - sequenceNumberToProcessor.values.foreach(_.shutdown()) - } - transactionExecutorOpt.foreach(_.shutdownNow()) - } -} diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala deleted file mode 100644 index 14dffb15fef98..0000000000000 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala +++ /dev/null @@ -1,171 +0,0 @@ -/* - * 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.streaming.flume.sink - -import java.net.InetSocketAddress -import java.util.concurrent._ - -import org.apache.avro.ipc.NettyServer -import org.apache.avro.ipc.specific.SpecificResponder -import org.apache.flume.Context -import org.apache.flume.Sink.Status -import org.apache.flume.conf.{Configurable, ConfigurationException} -import org.apache.flume.sink.AbstractSink - -/** - * A sink that uses Avro RPC to run a server that can be polled by Spark's - * FlumePollingInputDStream. This sink has the following configuration parameters: - * - * hostname - The hostname to bind to. Default: 0.0.0.0 - * port - The port to bind to. (No default - mandatory) - * timeout - Time in seconds after which a transaction is rolled back, - * if an ACK is not received from Spark within that time - * threads - Number of threads to use to receive requests from Spark (Default: 10) - * - * This sink is unlike other Flume sinks in the sense that it does not push data, - * instead the process method in this sink simply blocks the SinkRunner the first time it is - * called. This sink starts up an Avro IPC server that uses the SparkFlumeProtocol. - * - * Each time a getEventBatch call comes, creates a transaction and reads events - * from the channel. When enough events are read, the events are sent to the Spark receiver and - * the thread itself is blocked and a reference to it saved off. - * - * When the ack for that batch is received, - * the thread which created the transaction is is retrieved and it commits the transaction with the - * channel from the same thread it was originally created in (since Flume transactions are - * thread local). If a nack is received instead, the sink rolls back the transaction. If no ack - * is received within the specified timeout, the transaction is rolled back too. If an ack comes - * after that, it is simply ignored and the events get re-sent. - * - */ - -class SparkSink extends AbstractSink with Logging with Configurable { - - // Size of the pool to use for holding transaction processors. - private var poolSize: Integer = SparkSinkConfig.DEFAULT_THREADS - - // Timeout for each transaction. If spark does not respond in this much time, - // rollback the transaction - private var transactionTimeout = SparkSinkConfig.DEFAULT_TRANSACTION_TIMEOUT - - // Address info to bind on - private var hostname: String = SparkSinkConfig.DEFAULT_HOSTNAME - private var port: Int = 0 - - private var backOffInterval: Int = 200 - - // Handle to the server - private var serverOpt: Option[NettyServer] = None - - // The handler that handles the callback from Avro - private var handler: Option[SparkAvroCallbackHandler] = None - - // Latch that blocks off the Flume framework from wasting 1 thread. - private val blockingLatch = new CountDownLatch(1) - - override def start() { - logInfo("Starting Spark Sink: " + getName + " on port: " + port + " and interface: " + - hostname + " with " + "pool size: " + poolSize + " and transaction timeout: " + - transactionTimeout + ".") - handler = Option(new SparkAvroCallbackHandler(poolSize, getChannel, transactionTimeout, - backOffInterval)) - val responder = new SpecificResponder(classOf[SparkFlumeProtocol], handler.get) - // Using the constructor that takes specific thread-pools requires bringing in netty - // dependencies which are being excluded in the build. In practice, - // Netty dependencies are already available on the JVM as Flume would have pulled them in. - serverOpt = Option(new NettyServer(responder, new InetSocketAddress(hostname, port))) - serverOpt.foreach(server => { - logInfo("Starting Avro server for sink: " + getName) - server.start() - }) - super.start() - } - - override def stop() { - logInfo("Stopping Spark Sink: " + getName) - handler.foreach(callbackHandler => { - callbackHandler.shutdown() - }) - serverOpt.foreach(server => { - logInfo("Stopping Avro Server for sink: " + getName) - server.close() - server.join() - }) - blockingLatch.countDown() - super.stop() - } - - override def configure(ctx: Context) { - import SparkSinkConfig._ - hostname = ctx.getString(CONF_HOSTNAME, DEFAULT_HOSTNAME) - port = Option(ctx.getInteger(CONF_PORT)). - getOrElse(throw new ConfigurationException("The port to bind to must be specified")) - poolSize = ctx.getInteger(THREADS, DEFAULT_THREADS) - transactionTimeout = ctx.getInteger(CONF_TRANSACTION_TIMEOUT, DEFAULT_TRANSACTION_TIMEOUT) - backOffInterval = ctx.getInteger(CONF_BACKOFF_INTERVAL, DEFAULT_BACKOFF_INTERVAL) - logInfo("Configured Spark Sink with hostname: " + hostname + ", port: " + port + ", " + - "poolSize: " + poolSize + ", transactionTimeout: " + transactionTimeout + ", " + - "backoffInterval: " + backOffInterval) - } - - override def process(): Status = { - // This method is called in a loop by the Flume framework - block it until the sink is - // stopped to save CPU resources. The sink runner will interrupt this thread when the sink is - // being shut down. - logInfo("Blocking Sink Runner, sink will continue to run..") - blockingLatch.await() - Status.BACKOFF - } - - private[flume] def getPort(): Int = { - serverOpt - .map(_.getPort) - .getOrElse( - throw new RuntimeException("Server was not started!") - ) - } - - /** - * Pass in a [[CountDownLatch]] for testing purposes. This batch is counted down when each - * batch is received. The test can simply call await on this latch till the expected number of - * batches are received. - * @param latch - */ - private[flume] def countdownWhenBatchReceived(latch: CountDownLatch) { - handler.foreach(_.countDownWhenBatchAcked(latch)) - } -} - -/** - * Configuration parameters and their defaults. - */ -private[flume] -object SparkSinkConfig { - val THREADS = "threads" - val DEFAULT_THREADS = 10 - - val CONF_TRANSACTION_TIMEOUT = "timeout" - val DEFAULT_TRANSACTION_TIMEOUT = 60 - - val CONF_HOSTNAME = "hostname" - val DEFAULT_HOSTNAME = "0.0.0.0" - - val CONF_PORT = "port" - - val CONF_BACKOFF_INTERVAL = "backoffInterval" - val DEFAULT_BACKOFF_INTERVAL = 200 -} diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkThreadFactory.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkThreadFactory.scala deleted file mode 100644 index 845fc8debda75..0000000000000 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkThreadFactory.scala +++ /dev/null @@ -1,35 +0,0 @@ -/* - * 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.streaming.flume.sink - -import java.util.concurrent.ThreadFactory -import java.util.concurrent.atomic.AtomicLong - -/** - * Thread factory that generates daemon threads with a specified name format. - */ -private[sink] class SparkSinkThreadFactory(nameFormat: String) extends ThreadFactory { - - private val threadId = new AtomicLong() - - override def newThread(r: Runnable): Thread = { - val t = new Thread(r, nameFormat.format(threadId.incrementAndGet())) - t.setDaemon(true) - t - } - -} diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkUtils.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkUtils.scala deleted file mode 100644 index 47c0e294d6b52..0000000000000 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkUtils.scala +++ /dev/null @@ -1,28 +0,0 @@ -/* - * 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.streaming.flume.sink - -private[flume] object SparkSinkUtils { - /** - * This method determines if this batch represents an error or not. - * @param batch - The batch to check - * @return - true if the batch represents an error - */ - def isErrorBatch(batch: EventBatch): Boolean = { - !batch.getErrorMsg.toString.equals("") // If there is an error message, it is an error batch. - } -} diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala deleted file mode 100644 index b15c2097e550c..0000000000000 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala +++ /dev/null @@ -1,252 +0,0 @@ -/* - * 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.streaming.flume.sink - -import java.nio.ByteBuffer -import java.util -import java.util.concurrent.{Callable, CountDownLatch, TimeUnit} - -import scala.util.control.Breaks - -import org.apache.flume.{Channel, Transaction} - -// Flume forces transactions to be thread-local (horrible, I know!) -// So the sink basically spawns a new thread to pull the events out within a transaction. -// The thread fills in the event batch object that is set before the thread is scheduled. -// After filling it in, the thread waits on a condition - which is released only -// when the success message comes back for the specific sequence number for that event batch. -/** - * This class represents a transaction on the Flume channel. This class runs a separate thread - * which owns the transaction. The thread is blocked until the success call for that transaction - * comes back with an ACK or NACK. - * @param channel The channel from which to pull events - * @param seqNum The sequence number to use for the transaction. Must be unique - * @param maxBatchSize The maximum number of events to process per batch - * @param transactionTimeout Time in seconds after which a transaction must be rolled back - * without waiting for an ACK from Spark - * @param parent The parent [[SparkAvroCallbackHandler]] instance, for reporting timeouts - */ -private class TransactionProcessor(val channel: Channel, val seqNum: String, - var maxBatchSize: Int, val transactionTimeout: Int, val backOffInterval: Int, - val parent: SparkAvroCallbackHandler) extends Callable[Void] with Logging { - - // If a real batch is not returned, we always have to return an error batch. - @volatile private var eventBatch: EventBatch = new EventBatch("Unknown Error", "", - util.Collections.emptyList()) - - // Synchronization primitives - val batchGeneratedLatch = new CountDownLatch(1) - val batchAckLatch = new CountDownLatch(1) - - // Sanity check to ensure we don't loop like crazy - val totalAttemptsToRemoveFromChannel = Int.MaxValue / 2 - - // OK to use volatile, since the change would only make this true (otherwise it will be - // changed to false - we never apply a negation operation to this) - which means the transaction - // succeeded. - @volatile private var batchSuccess = false - - @volatile private var stopped = false - - @volatile private var isTest = false - - private var testLatch: CountDownLatch = null - - // The transaction that this processor would handle - var txOpt: Option[Transaction] = None - - /** - * Get an event batch from the channel. This method will block until a batch of events is - * available from the channel. If no events are available after a large number of attempts of - * polling the channel, this method will return an [[EventBatch]] with a non-empty error message - * - * @return An [[EventBatch]] instance with sequence number set to seqNum, filled with a - * maximum of maxBatchSize events - */ - def getEventBatch: EventBatch = { - batchGeneratedLatch.await() - eventBatch - } - - /** - * This method is to be called by the sink when it receives an ACK or NACK from Spark. This - * method is a no-op if it is called after transactionTimeout has expired since - * getEventBatch returned a batch of events. - * @param success True if an ACK was received and the transaction should be committed, else false. - */ - def batchProcessed(success: Boolean) { - logDebug("Batch processed for sequence number: " + seqNum) - batchSuccess = success - batchAckLatch.countDown() - } - - private[flume] def shutdown(): Unit = { - logDebug("Shutting down transaction processor") - stopped = true - } - - /** - * Populates events into the event batch. If the batch cannot be populated, - * this method will not set the events into the event batch, but it sets an error message. - */ - private def populateEvents() { - try { - txOpt = Option(channel.getTransaction) - if(txOpt.isEmpty) { - eventBatch.setErrorMsg("Something went wrong. Channel was " + - "unable to create a transaction!") - } - txOpt.foreach(tx => { - tx.begin() - val events = new util.ArrayList[SparkSinkEvent](maxBatchSize) - val loop = new Breaks - var gotEventsInThisTxn = false - var loopCounter: Int = 0 - loop.breakable { - while (!stopped && events.size() < maxBatchSize - && loopCounter < totalAttemptsToRemoveFromChannel) { - loopCounter += 1 - Option(channel.take()) match { - case Some(event) => - events.add(new SparkSinkEvent(toCharSequenceMap(event.getHeaders), - ByteBuffer.wrap(event.getBody))) - gotEventsInThisTxn = true - case None => - if (!gotEventsInThisTxn && !stopped) { - logDebug("Sleeping for " + backOffInterval + " millis as no events were read in" + - " the current transaction") - TimeUnit.MILLISECONDS.sleep(backOffInterval) - } else { - loop.break() - } - } - } - } - if (!gotEventsInThisTxn && !stopped) { - val msg = "Tried several times, " + - "but did not get any events from the channel!" - logWarning(msg) - eventBatch.setErrorMsg(msg) - } else { - // At this point, the events are available, so fill them into the event batch - eventBatch = new EventBatch("", seqNum, events) - } - }) - } catch { - case interrupted: InterruptedException => - // Don't pollute logs if the InterruptedException came from this being stopped - if (!stopped) { - logWarning("Error while processing transaction.", interrupted) - } - case e: Exception => - logWarning("Error while processing transaction.", e) - eventBatch.setErrorMsg(e.getMessage) - try { - txOpt.foreach(tx => { - rollbackAndClose(tx, close = true) - }) - } finally { - txOpt = None - } - } finally { - batchGeneratedLatch.countDown() - } - } - - /** - * Waits for upto transactionTimeout seconds for an ACK. If an ACK comes in - * this method commits the transaction with the channel. If the ACK does not come in within - * that time or a NACK comes in, this method rolls back the transaction. - */ - private def processAckOrNack() { - batchAckLatch.await(transactionTimeout, TimeUnit.SECONDS) - txOpt.foreach(tx => { - if (batchSuccess) { - try { - logDebug("Committing transaction") - tx.commit() - } catch { - case e: Exception => - logWarning("Error while attempting to commit transaction. Transaction will be rolled " + - "back", e) - rollbackAndClose(tx, close = false) // tx will be closed later anyway - } finally { - tx.close() - if (isTest) { - testLatch.countDown() - } - } - } else { - logWarning("Spark could not commit transaction, NACK received. Rolling back transaction.") - rollbackAndClose(tx, close = true) - // This might have been due to timeout or a NACK. Either way the following call does not - // cause issues. This is required to ensure the TransactionProcessor instance is not leaked - parent.removeAndGetProcessor(seqNum) - } - }) - } - - /** - * Helper method to rollback and optionally close a transaction - * @param tx The transaction to rollback - * @param close Whether the transaction should be closed or not after rolling back - */ - private def rollbackAndClose(tx: Transaction, close: Boolean) { - try { - logWarning("Spark was unable to successfully process the events. Transaction is being " + - "rolled back.") - tx.rollback() - } catch { - case e: Exception => - logError("Error rolling back transaction. Rollback may have failed!", e) - } finally { - if (close) { - tx.close() - } - } - } - - /** - * Helper method to convert a Map[String, String] to Map[CharSequence, CharSequence] - * @param inMap The map to be converted - * @return The converted map - */ - private def toCharSequenceMap(inMap: java.util.Map[String, String]): java.util.Map[CharSequence, - CharSequence] = { - val charSeqMap = new util.HashMap[CharSequence, CharSequence](inMap.size()) - charSeqMap.putAll(inMap) - charSeqMap - } - - /** - * When the thread is started it sets as many events as the batch size or less (if enough - * events aren't available) into the eventBatch and object and lets any threads waiting on the - * [[getEventBatch]] method to proceed. Then this thread waits for acks or nacks to come in, - * or for a specified timeout and commits or rolls back the transaction. - * @return - */ - override def call(): Void = { - populateEvents() - processAckOrNack() - null - } - - private[sink] def countDownWhenBatchAcked(latch: CountDownLatch) { - testLatch = latch - isTest = true - } -} diff --git a/external/flume-sink/src/test/resources/log4j.properties b/external/flume-sink/src/test/resources/log4j.properties deleted file mode 100644 index 42df8792f147f..0000000000000 --- a/external/flume-sink/src/test/resources/log4j.properties +++ /dev/null @@ -1,28 +0,0 @@ -# -# 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. -# - -# Set everything to be logged to the file streaming/target/unit-tests.log -log4j.rootCategory=INFO, file -log4j.appender.file=org.apache.log4j.FileAppender -log4j.appender.file.append=true -log4j.appender.file.file=target/unit-tests.log -log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n - -# Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.spark-project.jetty=WARN - diff --git a/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala b/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala deleted file mode 100644 index 7f6cecf9cd18d..0000000000000 --- a/external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala +++ /dev/null @@ -1,216 +0,0 @@ -/* - * 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.streaming.flume.sink - -import java.net.InetSocketAddress -import java.util.concurrent.{CountDownLatch, Executors, TimeUnit} -import java.util.concurrent.atomic.AtomicInteger - -import scala.collection.JavaConverters._ -import scala.concurrent.{ExecutionContext, Future} -import scala.util.{Failure, Success} - -import org.apache.avro.ipc.NettyTransceiver -import org.apache.avro.ipc.specific.SpecificRequestor -import org.apache.flume.Context -import org.apache.flume.channel.MemoryChannel -import org.apache.flume.event.EventBuilder -import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory - -// Due to MNG-1378, there is not a way to include test dependencies transitively. -// We cannot include Spark core tests as a dependency here because it depends on -// Spark core main, which has too many dependencies to require here manually. -// For this reason, we continue to use FunSuite and ignore the scalastyle checks -// that fail if this is detected. -// scalastyle:off -import org.scalatest.FunSuite - -class SparkSinkSuite extends FunSuite { -// scalastyle:on - - val eventsPerBatch = 1000 - val channelCapacity = 5000 - - test("Success with ack") { - val (channel, sink, latch) = initializeChannelAndSink() - channel.start() - sink.start() - - putEvents(channel, eventsPerBatch) - - val port = sink.getPort - val address = new InetSocketAddress("0.0.0.0", port) - - val (transceiver, client) = getTransceiverAndClient(address, 1)(0) - val events = client.getEventBatch(1000) - client.ack(events.getSequenceNumber) - assert(events.getEvents.size() === 1000) - latch.await(1, TimeUnit.SECONDS) - assertChannelIsEmpty(channel) - sink.stop() - channel.stop() - transceiver.close() - } - - test("Failure with nack") { - val (channel, sink, latch) = initializeChannelAndSink() - channel.start() - sink.start() - putEvents(channel, eventsPerBatch) - - val port = sink.getPort - val address = new InetSocketAddress("0.0.0.0", port) - - val (transceiver, client) = getTransceiverAndClient(address, 1)(0) - val events = client.getEventBatch(1000) - assert(events.getEvents.size() === 1000) - client.nack(events.getSequenceNumber) - latch.await(1, TimeUnit.SECONDS) - assert(availableChannelSlots(channel) === 4000) - sink.stop() - channel.stop() - transceiver.close() - } - - test("Failure with timeout") { - val (channel, sink, latch) = initializeChannelAndSink(Map(SparkSinkConfig - .CONF_TRANSACTION_TIMEOUT -> 1.toString)) - channel.start() - sink.start() - putEvents(channel, eventsPerBatch) - val port = sink.getPort - val address = new InetSocketAddress("0.0.0.0", port) - - val (transceiver, client) = getTransceiverAndClient(address, 1)(0) - val events = client.getEventBatch(1000) - assert(events.getEvents.size() === 1000) - latch.await(1, TimeUnit.SECONDS) - assert(availableChannelSlots(channel) === 4000) - sink.stop() - channel.stop() - transceiver.close() - } - - test("Multiple consumers") { - testMultipleConsumers(failSome = false) - } - - test("Multiple consumers with some failures") { - testMultipleConsumers(failSome = true) - } - - def testMultipleConsumers(failSome: Boolean): Unit = { - implicit val executorContext = ExecutionContext - .fromExecutorService(Executors.newFixedThreadPool(5)) - val (channel, sink, latch) = initializeChannelAndSink(Map.empty, 5) - channel.start() - sink.start() - (1 to 5).foreach(_ => putEvents(channel, eventsPerBatch)) - val port = sink.getPort - val address = new InetSocketAddress("0.0.0.0", port) - val transceiversAndClients = getTransceiverAndClient(address, 5) - val batchCounter = new CountDownLatch(5) - val counter = new AtomicInteger(0) - transceiversAndClients.foreach(x => { - Future { - val client = x._2 - val events = client.getEventBatch(1000) - if (!failSome || counter.getAndIncrement() % 2 == 0) { - client.ack(events.getSequenceNumber) - } else { - client.nack(events.getSequenceNumber) - throw new RuntimeException("Sending NACK for failure!") - } - events - }.onComplete { - case Success(events) => - assert(events.getEvents.size() === 1000) - batchCounter.countDown() - case Failure(t) => - // Don't re-throw the exception, causes a nasty unnecessary stack trace on stdout - batchCounter.countDown() - } - }) - batchCounter.await() - latch.await(1, TimeUnit.SECONDS) - executorContext.shutdown() - if(failSome) { - assert(availableChannelSlots(channel) === 3000) - } else { - assertChannelIsEmpty(channel) - } - sink.stop() - channel.stop() - transceiversAndClients.foreach(x => x._1.close()) - } - - private def initializeChannelAndSink(overrides: Map[String, String] = Map.empty, - batchCounter: Int = 1): (MemoryChannel, SparkSink, CountDownLatch) = { - val channel = new MemoryChannel() - val channelContext = new Context() - - channelContext.put("capacity", channelCapacity.toString) - channelContext.put("transactionCapacity", 1000.toString) - channelContext.put("keep-alive", 0.toString) - channelContext.putAll(overrides.asJava) - channel.setName(scala.util.Random.nextString(10)) - channel.configure(channelContext) - - val sink = new SparkSink() - val sinkContext = new Context() - sinkContext.put(SparkSinkConfig.CONF_HOSTNAME, "0.0.0.0") - sinkContext.put(SparkSinkConfig.CONF_PORT, 0.toString) - sink.configure(sinkContext) - sink.setChannel(channel) - val latch = new CountDownLatch(batchCounter) - sink.countdownWhenBatchReceived(latch) - (channel, sink, latch) - } - - private def putEvents(ch: MemoryChannel, count: Int): Unit = { - val tx = ch.getTransaction - tx.begin() - (1 to count).foreach(x => ch.put(EventBuilder.withBody(x.toString.getBytes))) - tx.commit() - tx.close() - } - - private def getTransceiverAndClient(address: InetSocketAddress, - count: Int): Seq[(NettyTransceiver, SparkFlumeProtocol.Callback)] = { - - (1 to count).map(_ => { - lazy val channelFactoryExecutor = Executors.newCachedThreadPool( - new SparkSinkThreadFactory("Flume Receiver Channel Thread - %d")) - lazy val channelFactory = - new NioClientSocketChannelFactory(channelFactoryExecutor, channelFactoryExecutor) - val transceiver = new NettyTransceiver(address, channelFactory) - val client = SpecificRequestor.getClient(classOf[SparkFlumeProtocol.Callback], transceiver) - (transceiver, client) - }) - } - - private def assertChannelIsEmpty(channel: MemoryChannel): Unit = { - assert(availableChannelSlots(channel) === channelCapacity) - } - - private def availableChannelSlots(channel: MemoryChannel): Int = { - val queueRemaining = channel.getClass.getDeclaredField("queueRemaining") - queueRemaining.setAccessible(true) - val m = queueRemaining.get(channel).getClass.getDeclaredMethod("availablePermits") - m.invoke(queueRemaining.get(channel)).asInstanceOf[Int] - } -} diff --git a/external/flume/pom.xml b/external/flume/pom.xml deleted file mode 100644 index d650dd034d636..0000000000000 --- a/external/flume/pom.xml +++ /dev/null @@ -1,78 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent_2.11 - 2.0.0-SNAPSHOT - ../../pom.xml - - - org.apache.spark - spark-streaming-flume_2.11 - - streaming-flume - - jar - Spark Project External Flume - http://spark.apache.org/ - - - - org.apache.spark - spark-streaming_${scala.binary.version} - ${project.version} - provided - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - test-jar - test - - - org.apache.spark - spark-streaming-flume-sink_${scala.binary.version} - ${project.version} - - - org.apache.flume - flume-ng-core - - - org.apache.flume - flume-ng-sdk - - - org.scalacheck - scalacheck_${scala.binary.version} - test - - - org.apache.spark - spark-test-tags_${scala.binary.version} - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala deleted file mode 100644 index 5c773d4b07cf6..0000000000000 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/EventTransformer.scala +++ /dev/null @@ -1,72 +0,0 @@ -/* - * 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.streaming.flume - -import java.io.{ObjectInput, ObjectOutput} - -import scala.collection.JavaConverters._ - -import org.apache.spark.Logging -import org.apache.spark.util.Utils - -/** - * A simple object that provides the implementation of readExternal and writeExternal for both - * the wrapper classes for Flume-style Events. - */ -private[streaming] object EventTransformer extends Logging { - def readExternal(in: ObjectInput): (java.util.HashMap[CharSequence, CharSequence], - Array[Byte]) = { - val bodyLength = in.readInt() - val bodyBuff = new Array[Byte](bodyLength) - in.readFully(bodyBuff) - - val numHeaders = in.readInt() - val headers = new java.util.HashMap[CharSequence, CharSequence] - - for (i <- 0 until numHeaders) { - val keyLength = in.readInt() - val keyBuff = new Array[Byte](keyLength) - in.readFully(keyBuff) - val key: String = Utils.deserialize(keyBuff) - - val valLength = in.readInt() - val valBuff = new Array[Byte](valLength) - in.readFully(valBuff) - val value: String = Utils.deserialize(valBuff) - - headers.put(key, value) - } - (headers, bodyBuff) - } - - def writeExternal(out: ObjectOutput, headers: java.util.Map[CharSequence, CharSequence], - body: Array[Byte]) { - out.writeInt(body.length) - out.write(body) - val numHeaders = headers.size() - out.writeInt(numHeaders) - for ((k, v) <- headers.asScala) { - val keyBuff = Utils.serialize(k.toString) - out.writeInt(keyBuff.length) - out.write(keyBuff) - val valBuff = Utils.serialize(v.toString) - out.writeInt(valBuff.length) - out.write(valBuff) - } - } -} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeBatchFetcher.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeBatchFetcher.scala deleted file mode 100644 index b9d4e762ca05d..0000000000000 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeBatchFetcher.scala +++ /dev/null @@ -1,166 +0,0 @@ -/* - * 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.streaming.flume - -import scala.collection.mutable.ArrayBuffer - -import com.google.common.base.Throwables - -import org.apache.spark.Logging -import org.apache.spark.streaming.flume.sink._ - -/** - * This class implements the core functionality of [[FlumePollingReceiver]]. When started it - * pulls data from Flume, stores it to Spark and then sends an Ack or Nack. This class should be - * run via an [[java.util.concurrent.Executor]] as this implements [[Runnable]] - * - * @param receiver The receiver that owns this instance. - */ - -private[flume] class FlumeBatchFetcher(receiver: FlumePollingReceiver) extends Runnable with - Logging { - - def run(): Unit = { - while (!receiver.isStopped()) { - val connection = receiver.getConnections.poll() - val client = connection.client - var batchReceived = false - var seq: CharSequence = null - try { - getBatch(client) match { - case Some(eventBatch) => - batchReceived = true - seq = eventBatch.getSequenceNumber - val events = toSparkFlumeEvents(eventBatch.getEvents) - if (store(events)) { - sendAck(client, seq) - } else { - sendNack(batchReceived, client, seq) - } - case None => - } - } catch { - case e: Exception => - Throwables.getRootCause(e) match { - // If the cause was an InterruptedException, then check if the receiver is stopped - - // if yes, just break out of the loop. Else send a Nack and log a warning. - // In the unlikely case, the cause was not an Exception, - // then just throw it out and exit. - case interrupted: InterruptedException => - if (!receiver.isStopped()) { - logWarning("Interrupted while receiving data from Flume", interrupted) - sendNack(batchReceived, client, seq) - } - case exception: Exception => - logWarning("Error while receiving data from Flume", exception) - sendNack(batchReceived, client, seq) - } - } finally { - receiver.getConnections.add(connection) - } - } - } - - /** - * Gets a batch of events from the specified client. This method does not handle any exceptions - * which will be propogated to the caller. - * @param client Client to get events from - * @return [[Some]] which contains the event batch if Flume sent any events back, else [[None]] - */ - private def getBatch(client: SparkFlumeProtocol.Callback): Option[EventBatch] = { - val eventBatch = client.getEventBatch(receiver.getMaxBatchSize) - if (!SparkSinkUtils.isErrorBatch(eventBatch)) { - // No error, proceed with processing data - logDebug(s"Received batch of ${eventBatch.getEvents.size} events with sequence " + - s"number: ${eventBatch.getSequenceNumber}") - Some(eventBatch) - } else { - logWarning("Did not receive events from Flume agent due to error on the Flume agent: " + - eventBatch.getErrorMsg) - None - } - } - - /** - * Store the events in the buffer to Spark. This method will not propogate any exceptions, - * but will propogate any other errors. - * @param buffer The buffer to store - * @return true if the data was stored without any exception being thrown, else false - */ - private def store(buffer: ArrayBuffer[SparkFlumeEvent]): Boolean = { - try { - receiver.store(buffer) - true - } catch { - case e: Exception => - logWarning("Error while attempting to store data received from Flume", e) - false - } - } - - /** - * Send an ack to the client for the sequence number. This method does not handle any exceptions - * which will be propagated to the caller. - * @param client client to send the ack to - * @param seq sequence number of the batch to be ack-ed. - * @return - */ - private def sendAck(client: SparkFlumeProtocol.Callback, seq: CharSequence): Unit = { - logDebug("Sending ack for sequence number: " + seq) - client.ack(seq) - logDebug("Ack sent for sequence number: " + seq) - } - - /** - * This method sends a Nack if a batch was received to the client with the given sequence - * number. Any exceptions thrown by the RPC call is simply thrown out as is - no effort is made - * to handle it. - * @param batchReceived true if a batch was received. If this is false, no nack is sent - * @param client The client to which the nack should be sent - * @param seq The sequence number of the batch that is being nack-ed. - */ - private def sendNack(batchReceived: Boolean, client: SparkFlumeProtocol.Callback, - seq: CharSequence): Unit = { - if (batchReceived) { - // Let Flume know that the events need to be pushed back into the channel. - logDebug("Sending nack for sequence number: " + seq) - client.nack(seq) // If the agent is down, even this could fail and throw - logDebug("Nack sent for sequence number: " + seq) - } - } - - /** - * Utility method to convert [[SparkSinkEvent]]s to [[SparkFlumeEvent]]s - * @param events - Events to convert to SparkFlumeEvents - * @return - The SparkFlumeEvent generated from SparkSinkEvent - */ - private def toSparkFlumeEvents(events: java.util.List[SparkSinkEvent]): - ArrayBuffer[SparkFlumeEvent] = { - // Convert each Flume event to a serializable SparkFlumeEvent - val buffer = new ArrayBuffer[SparkFlumeEvent](events.size()) - var j = 0 - while (j < events.size()) { - val event = events.get(j) - val sparkFlumeEvent = new SparkFlumeEvent() - sparkFlumeEvent.event.setBody(event.getBody) - sparkFlumeEvent.event.setHeaders(event.getHeaders) - buffer += sparkFlumeEvent - j += 1 - } - buffer - } -} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala deleted file mode 100644 index 74bd0165c6209..0000000000000 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala +++ /dev/null @@ -1,205 +0,0 @@ -/* - * 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.streaming.flume - -import java.io.{Externalizable, ObjectInput, ObjectOutput} -import java.net.InetSocketAddress -import java.nio.ByteBuffer -import java.util.concurrent.Executors - -import scala.collection.JavaConverters._ -import scala.reflect.ClassTag - -import org.apache.avro.ipc.NettyServer -import org.apache.avro.ipc.specific.SpecificResponder -import org.apache.flume.source.avro.{AvroFlumeEvent, AvroSourceProtocol, Status} -import org.jboss.netty.channel.{ChannelPipeline, ChannelPipelineFactory, Channels} -import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory -import org.jboss.netty.handler.codec.compression._ - -import org.apache.spark.Logging -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.dstream._ -import org.apache.spark.streaming.receiver.Receiver -import org.apache.spark.util.Utils - -private[streaming] -class FlumeInputDStream[T: ClassTag]( - _ssc: StreamingContext, - host: String, - port: Int, - storageLevel: StorageLevel, - enableDecompression: Boolean -) extends ReceiverInputDStream[SparkFlumeEvent](_ssc) { - - override def getReceiver(): Receiver[SparkFlumeEvent] = { - new FlumeReceiver(host, port, storageLevel, enableDecompression) - } -} - -/** - * A wrapper class for AvroFlumeEvent's with a custom serialization format. - * - * This is necessary because AvroFlumeEvent uses inner data structures - * which are not serializable. - */ -class SparkFlumeEvent() extends Externalizable { - var event: AvroFlumeEvent = new AvroFlumeEvent() - - /* De-serialize from bytes. */ - def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { - val bodyLength = in.readInt() - val bodyBuff = new Array[Byte](bodyLength) - in.readFully(bodyBuff) - - val numHeaders = in.readInt() - val headers = new java.util.HashMap[CharSequence, CharSequence] - - for (i <- 0 until numHeaders) { - val keyLength = in.readInt() - val keyBuff = new Array[Byte](keyLength) - in.readFully(keyBuff) - val key: String = Utils.deserialize(keyBuff) - - val valLength = in.readInt() - val valBuff = new Array[Byte](valLength) - in.readFully(valBuff) - val value: String = Utils.deserialize(valBuff) - - headers.put(key, value) - } - - event.setBody(ByteBuffer.wrap(bodyBuff)) - event.setHeaders(headers) - } - - /* Serialize to bytes. */ - def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { - val body = event.getBody - out.writeInt(body.remaining()) - Utils.writeByteBuffer(body, out) - - val numHeaders = event.getHeaders.size() - out.writeInt(numHeaders) - for ((k, v) <- event.getHeaders.asScala) { - val keyBuff = Utils.serialize(k.toString) - out.writeInt(keyBuff.length) - out.write(keyBuff) - val valBuff = Utils.serialize(v.toString) - out.writeInt(valBuff.length) - out.write(valBuff) - } - } -} - -private[streaming] object SparkFlumeEvent { - def fromAvroFlumeEvent(in: AvroFlumeEvent): SparkFlumeEvent = { - val event = new SparkFlumeEvent - event.event = in - event - } -} - -/** A simple server that implements Flume's Avro protocol. */ -private[streaming] -class FlumeEventServer(receiver: FlumeReceiver) extends AvroSourceProtocol { - override def append(event: AvroFlumeEvent): Status = { - receiver.store(SparkFlumeEvent.fromAvroFlumeEvent(event)) - Status.OK - } - - override def appendBatch(events: java.util.List[AvroFlumeEvent]): Status = { - events.asScala.foreach(event => receiver.store(SparkFlumeEvent.fromAvroFlumeEvent(event))) - Status.OK - } -} - -/** A NetworkReceiver which listens for events using the - * Flume Avro interface. */ -private[streaming] -class FlumeReceiver( - host: String, - port: Int, - storageLevel: StorageLevel, - enableDecompression: Boolean - ) extends Receiver[SparkFlumeEvent](storageLevel) with Logging { - - lazy val responder = new SpecificResponder( - classOf[AvroSourceProtocol], new FlumeEventServer(this)) - var server: NettyServer = null - - private def initServer() = { - if (enableDecompression) { - val channelFactory = new NioServerSocketChannelFactory(Executors.newCachedThreadPool(), - Executors.newCachedThreadPool()) - val channelPipelineFactory = new CompressionChannelPipelineFactory() - - new NettyServer( - responder, - new InetSocketAddress(host, port), - channelFactory, - channelPipelineFactory, - null) - } else { - new NettyServer(responder, new InetSocketAddress(host, port)) - } - } - - def onStart() { - synchronized { - if (server == null) { - server = initServer() - server.start() - } else { - logWarning("Flume receiver being asked to start more then once with out close") - } - } - logInfo("Flume receiver started") - } - - def onStop() { - synchronized { - if (server != null) { - server.close() - server = null - } - } - logInfo("Flume receiver stopped") - } - - override def preferredLocation: Option[String] = Option(host) - - /** A Netty Pipeline factory that will decompress incoming data from - * and the Netty client and compress data going back to the client. - * - * The compression on the return is required because Flume requires - * a successful response to indicate it can remove the event/batch - * from the configured channel - */ - private[streaming] - class CompressionChannelPipelineFactory extends ChannelPipelineFactory { - def getPipeline(): ChannelPipeline = { - val pipeline = Channels.pipeline() - val encoder = new ZlibEncoder(6) - pipeline.addFirst("deflater", encoder) - pipeline.addFirst("inflater", new ZlibDecoder()) - pipeline - } - } -} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala deleted file mode 100644 index d9c25e86540db..0000000000000 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala +++ /dev/null @@ -1,123 +0,0 @@ -/* - * 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.streaming.flume - - -import java.net.InetSocketAddress -import java.util.concurrent.{Executors, LinkedBlockingQueue, TimeUnit} - -import scala.collection.JavaConverters._ -import scala.reflect.ClassTag - -import com.google.common.util.concurrent.ThreadFactoryBuilder -import org.apache.avro.ipc.NettyTransceiver -import org.apache.avro.ipc.specific.SpecificRequestor -import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory - -import org.apache.spark.Logging -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.dstream.ReceiverInputDStream -import org.apache.spark.streaming.flume.sink._ -import org.apache.spark.streaming.receiver.Receiver - -/** - * A [[ReceiverInputDStream]] that can be used to read data from several Flume agents running - * [[org.apache.spark.streaming.flume.sink.SparkSink]]s. - * @param _ssc Streaming context that will execute this input stream - * @param addresses List of addresses at which SparkSinks are listening - * @param maxBatchSize Maximum size of a batch - * @param parallelism Number of parallel connections to open - * @param storageLevel The storage level to use. - * @tparam T Class type of the object of this stream - */ -private[streaming] class FlumePollingInputDStream[T: ClassTag]( - _ssc: StreamingContext, - val addresses: Seq[InetSocketAddress], - val maxBatchSize: Int, - val parallelism: Int, - storageLevel: StorageLevel - ) extends ReceiverInputDStream[SparkFlumeEvent](_ssc) { - - override def getReceiver(): Receiver[SparkFlumeEvent] = { - new FlumePollingReceiver(addresses, maxBatchSize, parallelism, storageLevel) - } -} - -private[streaming] class FlumePollingReceiver( - addresses: Seq[InetSocketAddress], - maxBatchSize: Int, - parallelism: Int, - storageLevel: StorageLevel - ) extends Receiver[SparkFlumeEvent](storageLevel) with Logging { - - lazy val channelFactoryExecutor = - Executors.newCachedThreadPool(new ThreadFactoryBuilder().setDaemon(true). - setNameFormat("Flume Receiver Channel Thread - %d").build()) - - lazy val channelFactory = - new NioClientSocketChannelFactory(channelFactoryExecutor, channelFactoryExecutor) - - lazy val receiverExecutor = Executors.newFixedThreadPool(parallelism, - new ThreadFactoryBuilder().setDaemon(true).setNameFormat("Flume Receiver Thread - %d").build()) - - private lazy val connections = new LinkedBlockingQueue[FlumeConnection]() - - override def onStart(): Unit = { - // Create the connections to each Flume agent. - addresses.foreach(host => { - val transceiver = new NettyTransceiver(host, channelFactory) - val client = SpecificRequestor.getClient(classOf[SparkFlumeProtocol.Callback], transceiver) - connections.add(new FlumeConnection(transceiver, client)) - }) - for (i <- 0 until parallelism) { - logInfo("Starting Flume Polling Receiver worker threads..") - // Threads that pull data from Flume. - receiverExecutor.submit(new FlumeBatchFetcher(this)) - } - } - - override def onStop(): Unit = { - logInfo("Shutting down Flume Polling Receiver") - receiverExecutor.shutdown() - // Wait upto a minute for the threads to die - if (!receiverExecutor.awaitTermination(60, TimeUnit.SECONDS)) { - receiverExecutor.shutdownNow() - } - connections.asScala.foreach(_.transceiver.close()) - channelFactory.releaseExternalResources() - } - - private[flume] def getConnections: LinkedBlockingQueue[FlumeConnection] = { - this.connections - } - - private[flume] def getMaxBatchSize: Int = { - this.maxBatchSize - } -} - -/** - * A wrapper around the transceiver and the Avro IPC API. - * @param transceiver The transceiver to use for communication with Flume - * @param client The client that the callbacks are received on. - */ -private[flume] class FlumeConnection(val transceiver: NettyTransceiver, - val client: SparkFlumeProtocol.Callback) - - - diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeTestUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeTestUtils.scala deleted file mode 100644 index 3f87ce46e5952..0000000000000 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeTestUtils.scala +++ /dev/null @@ -1,117 +0,0 @@ -/* - * 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.streaming.flume - -import java.net.{InetSocketAddress, ServerSocket} -import java.nio.ByteBuffer -import java.util.{List => JList} -import java.util.Collections - -import scala.collection.JavaConverters._ - -import com.google.common.base.Charsets.UTF_8 -import org.apache.avro.ipc.NettyTransceiver -import org.apache.avro.ipc.specific.SpecificRequestor -import org.apache.commons.lang3.RandomUtils -import org.apache.flume.source.avro -import org.apache.flume.source.avro.{AvroFlumeEvent, AvroSourceProtocol} -import org.jboss.netty.channel.ChannelPipeline -import org.jboss.netty.channel.socket.SocketChannel -import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory -import org.jboss.netty.handler.codec.compression.{ZlibDecoder, ZlibEncoder} - -import org.apache.spark.util.Utils -import org.apache.spark.SparkConf - -/** - * Share codes for Scala and Python unit tests - */ -private[flume] class FlumeTestUtils { - - private var transceiver: NettyTransceiver = null - - private val testPort: Int = findFreePort() - - def getTestPort(): Int = testPort - - /** Find a free port */ - private def findFreePort(): Int = { - val candidatePort = RandomUtils.nextInt(1024, 65536) - Utils.startServiceOnPort(candidatePort, (trialPort: Int) => { - val socket = new ServerSocket(trialPort) - socket.close() - (null, trialPort) - }, new SparkConf())._2 - } - - /** Send data to the flume receiver */ - def writeInput(input: JList[String], enableCompression: Boolean): Unit = { - val testAddress = new InetSocketAddress("localhost", testPort) - - val inputEvents = input.asScala.map { item => - val event = new AvroFlumeEvent - event.setBody(ByteBuffer.wrap(item.getBytes(UTF_8))) - event.setHeaders(Collections.singletonMap("test", "header")) - event - } - - // if last attempted transceiver had succeeded, close it - close() - - // Create transceiver - transceiver = { - if (enableCompression) { - new NettyTransceiver(testAddress, new CompressionChannelFactory(6)) - } else { - new NettyTransceiver(testAddress) - } - } - - // Create Avro client with the transceiver - val client = SpecificRequestor.getClient(classOf[AvroSourceProtocol], transceiver) - if (client == null) { - throw new AssertionError("Cannot create client") - } - - // Send data - val status = client.appendBatch(inputEvents.asJava) - if (status != avro.Status.OK) { - throw new AssertionError("Sent events unsuccessfully") - } - } - - def close(): Unit = { - if (transceiver != null) { - transceiver.close() - transceiver = null - } - } - - /** Class to create socket channel with compression */ - private class CompressionChannelFactory(compressionLevel: Int) - extends NioClientSocketChannelFactory { - - override def newChannel(pipeline: ChannelPipeline): SocketChannel = { - val encoder = new ZlibEncoder(compressionLevel) - pipeline.addFirst("deflater", encoder) - pipeline.addFirst("inflater", new ZlibDecoder()) - super.newChannel(pipeline) - } - } - -} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala deleted file mode 100644 index 3e3ed712f0dbf..0000000000000 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala +++ /dev/null @@ -1,311 +0,0 @@ -/* - * 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.streaming.flume - -import java.io.{ByteArrayOutputStream, DataOutputStream} -import java.net.InetSocketAddress -import java.util.{List => JList, Map => JMap} - -import scala.collection.JavaConverters._ - -import org.apache.spark.api.java.function.PairFunction -import org.apache.spark.api.python.PythonRDD -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaPairDStream, JavaReceiverInputDStream, JavaStreamingContext} -import org.apache.spark.streaming.dstream.ReceiverInputDStream - -object FlumeUtils { - private val DEFAULT_POLLING_PARALLELISM = 5 - private val DEFAULT_POLLING_BATCH_SIZE = 1000 - - /** - * Create a input stream from a Flume source. - * @param ssc StreamingContext object - * @param hostname Hostname of the slave machine to which the flume data will be sent - * @param port Port of the slave machine to which the flume data will be sent - * @param storageLevel Storage level to use for storing the received objects - */ - def createStream ( - ssc: StreamingContext, - hostname: String, - port: Int, - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): ReceiverInputDStream[SparkFlumeEvent] = { - createStream(ssc, hostname, port, storageLevel, false) - } - - /** - * Create a input stream from a Flume source. - * @param ssc StreamingContext object - * @param hostname Hostname of the slave machine to which the flume data will be sent - * @param port Port of the slave machine to which the flume data will be sent - * @param storageLevel Storage level to use for storing the received objects - * @param enableDecompression should netty server decompress input stream - */ - def createStream ( - ssc: StreamingContext, - hostname: String, - port: Int, - storageLevel: StorageLevel, - enableDecompression: Boolean - ): ReceiverInputDStream[SparkFlumeEvent] = { - val inputStream = new FlumeInputDStream[SparkFlumeEvent]( - ssc, hostname, port, storageLevel, enableDecompression) - - inputStream - } - - /** - * Creates a input stream from a Flume source. - * Storage level of the data will be the default StorageLevel.MEMORY_AND_DISK_SER_2. - * @param hostname Hostname of the slave machine to which the flume data will be sent - * @param port Port of the slave machine to which the flume data will be sent - */ - def createStream( - jssc: JavaStreamingContext, - hostname: String, - port: Int - ): JavaReceiverInputDStream[SparkFlumeEvent] = { - createStream(jssc.ssc, hostname, port) - } - - /** - * Creates a input stream from a Flume source. - * @param hostname Hostname of the slave machine to which the flume data will be sent - * @param port Port of the slave machine to which the flume data will be sent - * @param storageLevel Storage level to use for storing the received objects - */ - def createStream( - jssc: JavaStreamingContext, - hostname: String, - port: Int, - storageLevel: StorageLevel - ): JavaReceiverInputDStream[SparkFlumeEvent] = { - createStream(jssc.ssc, hostname, port, storageLevel, false) - } - - /** - * Creates a input stream from a Flume source. - * @param hostname Hostname of the slave machine to which the flume data will be sent - * @param port Port of the slave machine to which the flume data will be sent - * @param storageLevel Storage level to use for storing the received objects - * @param enableDecompression should netty server decompress input stream - */ - def createStream( - jssc: JavaStreamingContext, - hostname: String, - port: Int, - storageLevel: StorageLevel, - enableDecompression: Boolean - ): JavaReceiverInputDStream[SparkFlumeEvent] = { - createStream(jssc.ssc, hostname, port, storageLevel, enableDecompression) - } - - /** - * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. - * This stream will poll the sink for data and will pull events as they are available. - * This stream will use a batch size of 1000 events and run 5 threads to pull data. - * @param hostname Address of the host on which the Spark Sink is running - * @param port Port of the host at which the Spark Sink is listening - * @param storageLevel Storage level to use for storing the received objects - */ - def createPollingStream( - ssc: StreamingContext, - hostname: String, - port: Int, - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): ReceiverInputDStream[SparkFlumeEvent] = { - createPollingStream(ssc, Seq(new InetSocketAddress(hostname, port)), storageLevel) - } - - /** - * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. - * This stream will poll the sink for data and will pull events as they are available. - * This stream will use a batch size of 1000 events and run 5 threads to pull data. - * @param addresses List of InetSocketAddresses representing the hosts to connect to. - * @param storageLevel Storage level to use for storing the received objects - */ - def createPollingStream( - ssc: StreamingContext, - addresses: Seq[InetSocketAddress], - storageLevel: StorageLevel - ): ReceiverInputDStream[SparkFlumeEvent] = { - createPollingStream(ssc, addresses, storageLevel, - DEFAULT_POLLING_BATCH_SIZE, DEFAULT_POLLING_PARALLELISM) - } - - /** - * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. - * This stream will poll the sink for data and will pull events as they are available. - * @param addresses List of InetSocketAddresses representing the hosts to connect to. - * @param maxBatchSize Maximum number of events to be pulled from the Spark sink in a - * single RPC call - * @param parallelism Number of concurrent requests this stream should send to the sink. Note - * that having a higher number of requests concurrently being pulled will - * result in this stream using more threads - * @param storageLevel Storage level to use for storing the received objects - */ - def createPollingStream( - ssc: StreamingContext, - addresses: Seq[InetSocketAddress], - storageLevel: StorageLevel, - maxBatchSize: Int, - parallelism: Int - ): ReceiverInputDStream[SparkFlumeEvent] = { - new FlumePollingInputDStream[SparkFlumeEvent](ssc, addresses, maxBatchSize, - parallelism, storageLevel) - } - - /** - * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. - * This stream will poll the sink for data and will pull events as they are available. - * This stream will use a batch size of 1000 events and run 5 threads to pull data. - * @param hostname Hostname of the host on which the Spark Sink is running - * @param port Port of the host at which the Spark Sink is listening - */ - def createPollingStream( - jssc: JavaStreamingContext, - hostname: String, - port: Int - ): JavaReceiverInputDStream[SparkFlumeEvent] = { - createPollingStream(jssc, hostname, port, StorageLevel.MEMORY_AND_DISK_SER_2) - } - - /** - * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. - * This stream will poll the sink for data and will pull events as they are available. - * This stream will use a batch size of 1000 events and run 5 threads to pull data. - * @param hostname Hostname of the host on which the Spark Sink is running - * @param port Port of the host at which the Spark Sink is listening - * @param storageLevel Storage level to use for storing the received objects - */ - def createPollingStream( - jssc: JavaStreamingContext, - hostname: String, - port: Int, - storageLevel: StorageLevel - ): JavaReceiverInputDStream[SparkFlumeEvent] = { - createPollingStream(jssc, Array(new InetSocketAddress(hostname, port)), storageLevel) - } - - /** - * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. - * This stream will poll the sink for data and will pull events as they are available. - * This stream will use a batch size of 1000 events and run 5 threads to pull data. - * @param addresses List of InetSocketAddresses on which the Spark Sink is running. - * @param storageLevel Storage level to use for storing the received objects - */ - def createPollingStream( - jssc: JavaStreamingContext, - addresses: Array[InetSocketAddress], - storageLevel: StorageLevel - ): JavaReceiverInputDStream[SparkFlumeEvent] = { - createPollingStream(jssc, addresses, storageLevel, - DEFAULT_POLLING_BATCH_SIZE, DEFAULT_POLLING_PARALLELISM) - } - - /** - * Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. - * This stream will poll the sink for data and will pull events as they are available. - * @param addresses List of InetSocketAddresses on which the Spark Sink is running - * @param maxBatchSize The maximum number of events to be pulled from the Spark sink in a - * single RPC call - * @param parallelism Number of concurrent requests this stream should send to the sink. Note - * that having a higher number of requests concurrently being pulled will - * result in this stream using more threads - * @param storageLevel Storage level to use for storing the received objects - */ - def createPollingStream( - jssc: JavaStreamingContext, - addresses: Array[InetSocketAddress], - storageLevel: StorageLevel, - maxBatchSize: Int, - parallelism: Int - ): JavaReceiverInputDStream[SparkFlumeEvent] = { - createPollingStream(jssc.ssc, addresses, storageLevel, maxBatchSize, parallelism) - } -} - -/** - * This is a helper class that wraps the methods in FlumeUtils into more Python-friendly class and - * function so that it can be easily instantiated and called from Python's FlumeUtils. - */ -private[flume] class FlumeUtilsPythonHelper { - - def createStream( - jssc: JavaStreamingContext, - hostname: String, - port: Int, - storageLevel: StorageLevel, - enableDecompression: Boolean - ): JavaPairDStream[Array[Byte], Array[Byte]] = { - val dstream = FlumeUtils.createStream(jssc, hostname, port, storageLevel, enableDecompression) - FlumeUtilsPythonHelper.toByteArrayPairDStream(dstream) - } - - def createPollingStream( - jssc: JavaStreamingContext, - hosts: JList[String], - ports: JList[Int], - storageLevel: StorageLevel, - maxBatchSize: Int, - parallelism: Int - ): JavaPairDStream[Array[Byte], Array[Byte]] = { - assert(hosts.size() == ports.size()) - val addresses = hosts.asScala.zip(ports.asScala).map { - case (host, port) => new InetSocketAddress(host, port) - } - val dstream = FlumeUtils.createPollingStream( - jssc.ssc, addresses, storageLevel, maxBatchSize, parallelism) - FlumeUtilsPythonHelper.toByteArrayPairDStream(dstream) - } - -} - -private object FlumeUtilsPythonHelper { - - private def stringMapToByteArray(map: JMap[CharSequence, CharSequence]): Array[Byte] = { - val byteStream = new ByteArrayOutputStream() - val output = new DataOutputStream(byteStream) - try { - output.writeInt(map.size) - map.asScala.foreach { kv => - PythonRDD.writeUTF(kv._1.toString, output) - PythonRDD.writeUTF(kv._2.toString, output) - } - byteStream.toByteArray - } - finally { - output.close() - } - } - - private def toByteArrayPairDStream(dstream: JavaReceiverInputDStream[SparkFlumeEvent]): - JavaPairDStream[Array[Byte], Array[Byte]] = { - dstream.mapToPair(new PairFunction[SparkFlumeEvent, Array[Byte], Array[Byte]] { - override def call(sparkEvent: SparkFlumeEvent): (Array[Byte], Array[Byte]) = { - val event = sparkEvent.event - val byteBuffer = event.getBody - val body = new Array[Byte](byteBuffer.remaining()) - byteBuffer.get(body) - (stringMapToByteArray(event.getHeaders), body) - } - }) - } -} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/PollingFlumeTestUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/PollingFlumeTestUtils.scala deleted file mode 100644 index 9515d07c5ee5b..0000000000000 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/PollingFlumeTestUtils.scala +++ /dev/null @@ -1,208 +0,0 @@ -/* - * 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.streaming.flume - -import java.util.{Collections, List => JList, Map => JMap} -import java.util.concurrent._ - -import scala.collection.mutable.ArrayBuffer - -import com.google.common.base.Charsets.UTF_8 -import org.apache.flume.event.EventBuilder -import org.apache.flume.Context -import org.apache.flume.channel.MemoryChannel -import org.apache.flume.conf.Configurables - -import org.apache.spark.streaming.flume.sink.{SparkSink, SparkSinkConfig} - -/** - * Share codes for Scala and Python unit tests - */ -private[flume] class PollingFlumeTestUtils { - - private val batchCount = 5 - val eventsPerBatch = 100 - private val totalEventsPerChannel = batchCount * eventsPerBatch - private val channelCapacity = 5000 - - def getTotalEvents: Int = totalEventsPerChannel * channels.size - - private val channels = new ArrayBuffer[MemoryChannel] - private val sinks = new ArrayBuffer[SparkSink] - - /** - * Start a sink and return the port of this sink - */ - def startSingleSink(): Int = { - channels.clear() - sinks.clear() - - // Start the channel and sink. - val context = new Context() - context.put("capacity", channelCapacity.toString) - context.put("transactionCapacity", "1000") - context.put("keep-alive", "0") - val channel = new MemoryChannel() - Configurables.configure(channel, context) - - val sink = new SparkSink() - context.put(SparkSinkConfig.CONF_HOSTNAME, "localhost") - context.put(SparkSinkConfig.CONF_PORT, String.valueOf(0)) - Configurables.configure(sink, context) - sink.setChannel(channel) - sink.start() - - channels += (channel) - sinks += sink - - sink.getPort() - } - - /** - * Start 2 sinks and return the ports - */ - def startMultipleSinks(): Seq[Int] = { - channels.clear() - sinks.clear() - - // Start the channel and sink. - val context = new Context() - context.put("capacity", channelCapacity.toString) - context.put("transactionCapacity", "1000") - context.put("keep-alive", "0") - val channel = new MemoryChannel() - Configurables.configure(channel, context) - - val channel2 = new MemoryChannel() - Configurables.configure(channel2, context) - - val sink = new SparkSink() - context.put(SparkSinkConfig.CONF_HOSTNAME, "localhost") - context.put(SparkSinkConfig.CONF_PORT, String.valueOf(0)) - Configurables.configure(sink, context) - sink.setChannel(channel) - sink.start() - - val sink2 = new SparkSink() - context.put(SparkSinkConfig.CONF_HOSTNAME, "localhost") - context.put(SparkSinkConfig.CONF_PORT, String.valueOf(0)) - Configurables.configure(sink2, context) - sink2.setChannel(channel2) - sink2.start() - - sinks += sink - sinks += sink2 - channels += channel - channels += channel2 - - sinks.map(_.getPort()) - } - - /** - * Send data and wait until all data has been received - */ - def sendDatAndEnsureAllDataHasBeenReceived(): Unit = { - val executor = Executors.newCachedThreadPool() - val executorCompletion = new ExecutorCompletionService[Void](executor) - - val latch = new CountDownLatch(batchCount * channels.size) - sinks.foreach(_.countdownWhenBatchReceived(latch)) - - channels.foreach(channel => { - executorCompletion.submit(new TxnSubmitter(channel)) - }) - - for (i <- 0 until channels.size) { - executorCompletion.take() - } - - latch.await(15, TimeUnit.SECONDS) // Ensure all data has been received. - } - - /** - * A Python-friendly method to assert the output - */ - def assertOutput( - outputHeaders: JList[JMap[String, String]], outputBodies: JList[String]): Unit = { - require(outputHeaders.size == outputBodies.size) - val eventSize = outputHeaders.size - if (eventSize != totalEventsPerChannel * channels.size) { - throw new AssertionError( - s"Expected ${totalEventsPerChannel * channels.size} events, but was $eventSize") - } - var counter = 0 - for (k <- 0 until channels.size; i <- 0 until totalEventsPerChannel) { - val eventBodyToVerify = s"${channels(k).getName}-$i" - val eventHeaderToVerify: JMap[String, String] = Collections.singletonMap(s"test-$i", "header") - var found = false - var j = 0 - while (j < eventSize && !found) { - if (eventBodyToVerify == outputBodies.get(j) && - eventHeaderToVerify == outputHeaders.get(j)) { - found = true - counter += 1 - } - j += 1 - } - } - if (counter != totalEventsPerChannel * channels.size) { - throw new AssertionError( - s"111 Expected ${totalEventsPerChannel * channels.size} events, but was $counter") - } - } - - def assertChannelsAreEmpty(): Unit = { - channels.foreach(assertChannelIsEmpty) - } - - private def assertChannelIsEmpty(channel: MemoryChannel): Unit = { - val queueRemaining = channel.getClass.getDeclaredField("queueRemaining") - queueRemaining.setAccessible(true) - val m = queueRemaining.get(channel).getClass.getDeclaredMethod("availablePermits") - if (m.invoke(queueRemaining.get(channel)).asInstanceOf[Int] != 5000) { - throw new AssertionError(s"Channel ${channel.getName} is not empty") - } - } - - def close(): Unit = { - sinks.foreach(_.stop()) - sinks.clear() - channels.foreach(_.stop()) - channels.clear() - } - - private class TxnSubmitter(channel: MemoryChannel) extends Callable[Void] { - override def call(): Void = { - var t = 0 - for (i <- 0 until batchCount) { - val tx = channel.getTransaction - tx.begin() - for (j <- 0 until eventsPerBatch) { - channel.put(EventBuilder.withBody(s"${channel.getName}-$t".getBytes(UTF_8), - Collections.singletonMap(s"test-$t", "header"))) - t += 1 - } - tx.commit() - tx.close() - Thread.sleep(500) // Allow some time for the events to reach - } - null - } - } - -} diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/package-info.java b/external/flume/src/main/scala/org/apache/spark/streaming/flume/package-info.java deleted file mode 100644 index d31aa5f5c096c..0000000000000 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/package-info.java +++ /dev/null @@ -1,21 +0,0 @@ -/* - * 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. - */ - -/** - * Spark streaming receiver for Flume. - */ -package org.apache.spark.streaming.flume; \ No newline at end of file diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala deleted file mode 100644 index 9bfab68c4b8b7..0000000000000 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/package.scala +++ /dev/null @@ -1,23 +0,0 @@ -/* - * 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.streaming - -/** - * Spark streaming receiver for Flume. - */ -package object flume diff --git a/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java deleted file mode 100644 index cfedb5a042a35..0000000000000 --- a/external/flume/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * 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.streaming; - -import org.apache.spark.SparkConf; -import org.apache.spark.streaming.api.java.JavaStreamingContext; -import org.junit.After; -import org.junit.Before; - -public abstract class LocalJavaStreamingContext { - - protected transient JavaStreamingContext ssc; - - @Before - public void setUp() { - SparkConf conf = new SparkConf() - .setMaster("local[2]") - .setAppName("test") - .set("spark.streaming.clock", "org.apache.spark.util.ManualClock"); - ssc = new JavaStreamingContext(conf, new Duration(1000)); - ssc.checkpoint("checkpoint"); - } - - @After - public void tearDown() { - ssc.stop(); - ssc = null; - } -} diff --git a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumePollingStreamSuite.java b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumePollingStreamSuite.java deleted file mode 100644 index 79c5b91654b42..0000000000000 --- a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumePollingStreamSuite.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * 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.streaming.flume; - -import java.net.InetSocketAddress; - -import org.apache.spark.storage.StorageLevel; -import org.apache.spark.streaming.LocalJavaStreamingContext; - -import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; -import org.junit.Test; - -public class JavaFlumePollingStreamSuite extends LocalJavaStreamingContext { - @Test - public void testFlumeStream() { - // tests the API, does not actually test data receiving - InetSocketAddress[] addresses = new InetSocketAddress[] { - new InetSocketAddress("localhost", 12345) - }; - JavaReceiverInputDStream test1 = - FlumeUtils.createPollingStream(ssc, "localhost", 12345); - JavaReceiverInputDStream test2 = FlumeUtils.createPollingStream( - ssc, "localhost", 12345, StorageLevel.MEMORY_AND_DISK_SER_2()); - JavaReceiverInputDStream test3 = FlumeUtils.createPollingStream( - ssc, addresses, StorageLevel.MEMORY_AND_DISK_SER_2()); - JavaReceiverInputDStream test4 = FlumeUtils.createPollingStream( - ssc, addresses, StorageLevel.MEMORY_AND_DISK_SER_2(), 100, 5); - } -} diff --git a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java deleted file mode 100644 index 3b5e0c7746b2c..0000000000000 --- a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java +++ /dev/null @@ -1,36 +0,0 @@ -/* - * 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.streaming.flume; - -import org.apache.spark.storage.StorageLevel; -import org.apache.spark.streaming.LocalJavaStreamingContext; - -import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; -import org.junit.Test; - -public class JavaFlumeStreamSuite extends LocalJavaStreamingContext { - @Test - public void testFlumeStream() { - // tests the API, does not actually test data receiving - JavaReceiverInputDStream test1 = FlumeUtils.createStream(ssc, "localhost", 12345); - JavaReceiverInputDStream test2 = FlumeUtils.createStream(ssc, "localhost", 12345, - StorageLevel.MEMORY_AND_DISK_SER_2()); - JavaReceiverInputDStream test3 = FlumeUtils.createStream(ssc, "localhost", 12345, - StorageLevel.MEMORY_AND_DISK_SER_2(), false); - } -} diff --git a/external/flume/src/test/resources/log4j.properties b/external/flume/src/test/resources/log4j.properties deleted file mode 100644 index 75e3b53a093f6..0000000000000 --- a/external/flume/src/test/resources/log4j.properties +++ /dev/null @@ -1,28 +0,0 @@ -# -# 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. -# - -# Set everything to be logged to the file target/unit-tests.log -log4j.rootCategory=INFO, file -log4j.appender.file=org.apache.log4j.FileAppender -log4j.appender.file.append=true -log4j.appender.file.file=target/unit-tests.log -log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n - -# Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.spark-project.jetty=WARN - diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/TestOutputStream.scala b/external/flume/src/test/scala/org/apache/spark/streaming/TestOutputStream.scala deleted file mode 100644 index c97a27ca7c7aa..0000000000000 --- a/external/flume/src/test/scala/org/apache/spark/streaming/TestOutputStream.scala +++ /dev/null @@ -1,48 +0,0 @@ -/* - * 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.streaming - -import java.io.{IOException, ObjectInputStream} -import java.util.concurrent.ConcurrentLinkedQueue - -import scala.reflect.ClassTag - -import org.apache.spark.rdd.RDD -import org.apache.spark.streaming.dstream.{DStream, ForEachDStream} -import org.apache.spark.util.Utils - -/** - * This is a output stream just for the testsuites. All the output is collected into a - * ArrayBuffer. This buffer is wiped clean on being restored from checkpoint. - * - * The buffer contains a sequence of RDD's, each containing a sequence of items - */ -class TestOutputStream[T: ClassTag](parent: DStream[T], - val output: ConcurrentLinkedQueue[Seq[T]] = new ConcurrentLinkedQueue[Seq[T]]()) - extends ForEachDStream[T](parent, (rdd: RDD[T], t: Time) => { - val collected = rdd.collect() - output.add(collected) - }, false) { - - // This is to clear the output buffer every it is read from a checkpoint - @throws(classOf[IOException]) - private def readObject(ois: ObjectInputStream): Unit = Utils.tryOrIOException { - ois.defaultReadObject() - output.clear() - } -} diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala deleted file mode 100644 index 10dcbf98bc3b6..0000000000000 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumePollingStreamSuite.scala +++ /dev/null @@ -1,129 +0,0 @@ -/* - * 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.streaming.flume - -import java.net.InetSocketAddress -import java.util.concurrent.ConcurrentLinkedQueue - -import scala.collection.JavaConverters._ -import scala.concurrent.duration._ -import scala.language.postfixOps - -import org.scalatest.BeforeAndAfter -import org.scalatest.concurrent.Eventually._ - -import org.apache.spark.{Logging, SparkConf, SparkFunSuite} -import org.apache.spark.network.util.JavaUtils -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.{Seconds, StreamingContext, TestOutputStream} -import org.apache.spark.streaming.dstream.ReceiverInputDStream -import org.apache.spark.util.{ManualClock, Utils} - -class FlumePollingStreamSuite extends SparkFunSuite with BeforeAndAfter with Logging { - - val maxAttempts = 5 - val batchDuration = Seconds(1) - - val conf = new SparkConf() - .setMaster("local[2]") - .setAppName(this.getClass.getSimpleName) - .set("spark.streaming.clock", "org.apache.spark.util.ManualClock") - - val utils = new PollingFlumeTestUtils - - test("flume polling test") { - testMultipleTimes(testFlumePolling) - } - - test("flume polling test multiple hosts") { - testMultipleTimes(testFlumePollingMultipleHost) - } - - /** - * Run the given test until no more java.net.BindException's are thrown. - * Do this only up to a certain attempt limit. - */ - private def testMultipleTimes(test: () => Unit): Unit = { - var testPassed = false - var attempt = 0 - while (!testPassed && attempt < maxAttempts) { - try { - test() - testPassed = true - } catch { - case e: Exception if Utils.isBindCollision(e) => - logWarning("Exception when running flume polling test: " + e) - attempt += 1 - } - } - assert(testPassed, s"Test failed after $attempt attempts!") - } - - private def testFlumePolling(): Unit = { - try { - val port = utils.startSingleSink() - - writeAndVerify(Seq(port)) - utils.assertChannelsAreEmpty() - } finally { - utils.close() - } - } - - private def testFlumePollingMultipleHost(): Unit = { - try { - val ports = utils.startMultipleSinks() - writeAndVerify(ports) - utils.assertChannelsAreEmpty() - } finally { - utils.close() - } - } - - def writeAndVerify(sinkPorts: Seq[Int]): Unit = { - // Set up the streaming context and input streams - val ssc = new StreamingContext(conf, batchDuration) - val addresses = sinkPorts.map(port => new InetSocketAddress("localhost", port)) - val flumeStream: ReceiverInputDStream[SparkFlumeEvent] = - FlumeUtils.createPollingStream(ssc, addresses, StorageLevel.MEMORY_AND_DISK, - utils.eventsPerBatch, 5) - val outputQueue = new ConcurrentLinkedQueue[Seq[SparkFlumeEvent]] - val outputStream = new TestOutputStream(flumeStream, outputQueue) - outputStream.register() - - ssc.start() - try { - utils.sendDatAndEnsureAllDataHasBeenReceived() - val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] - clock.advance(batchDuration.milliseconds) - - // The eventually is required to ensure that all data in the batch has been processed. - eventually(timeout(10 seconds), interval(100 milliseconds)) { - val flattenOutput = outputQueue.asScala.toSeq.flatten - val headers = flattenOutput.map(_.event.getHeaders.asScala.map { - case (key, value) => (key.toString, value.toString) - }).map(_.asJava) - val bodies = flattenOutput.map(e => JavaUtils.bytesToString(e.event.getBody)) - utils.assertOutput(headers.asJava, bodies.asJava) - } - } finally { - ssc.stop() - } - } - -} diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala deleted file mode 100644 index 38208c651805f..0000000000000 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala +++ /dev/null @@ -1,102 +0,0 @@ -/* - * 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.streaming.flume - -import java.util.concurrent.ConcurrentLinkedQueue - -import scala.collection.JavaConverters._ -import scala.concurrent.duration._ -import scala.language.postfixOps - -import org.jboss.netty.channel.ChannelPipeline -import org.jboss.netty.channel.socket.SocketChannel -import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory -import org.jboss.netty.handler.codec.compression._ -import org.scalatest.{BeforeAndAfter, Matchers} -import org.scalatest.concurrent.Eventually._ - -import org.apache.spark.{Logging, SparkConf, SparkFunSuite} -import org.apache.spark.network.util.JavaUtils -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.{Milliseconds, StreamingContext, TestOutputStream} - -class FlumeStreamSuite extends SparkFunSuite with BeforeAndAfter with Matchers with Logging { - val conf = new SparkConf().setMaster("local[4]").setAppName("FlumeStreamSuite") - var ssc: StreamingContext = null - - test("flume input stream") { - testFlumeStream(testCompression = false) - } - - test("flume input compressed stream") { - testFlumeStream(testCompression = true) - } - - /** Run test on flume stream */ - private def testFlumeStream(testCompression: Boolean): Unit = { - val input = (1 to 100).map { _.toString } - val utils = new FlumeTestUtils - try { - val outputQueue = startContext(utils.getTestPort(), testCompression) - - eventually(timeout(10 seconds), interval(100 milliseconds)) { - utils.writeInput(input.asJava, testCompression) - } - - eventually(timeout(10 seconds), interval(100 milliseconds)) { - val outputEvents = outputQueue.asScala.toSeq.flatten.map { _.event } - outputEvents.foreach { - event => - event.getHeaders.get("test") should be("header") - } - val output = outputEvents.map(event => JavaUtils.bytesToString(event.getBody)) - output should be (input) - } - } finally { - if (ssc != null) { - ssc.stop() - } - utils.close() - } - } - - /** Setup and start the streaming context */ - private def startContext( - testPort: Int, testCompression: Boolean): (ConcurrentLinkedQueue[Seq[SparkFlumeEvent]]) = { - ssc = new StreamingContext(conf, Milliseconds(200)) - val flumeStream = FlumeUtils.createStream( - ssc, "localhost", testPort, StorageLevel.MEMORY_AND_DISK, testCompression) - val outputQueue = new ConcurrentLinkedQueue[Seq[SparkFlumeEvent]] - val outputStream = new TestOutputStream(flumeStream, outputQueue) - outputStream.register() - ssc.start() - outputQueue - } - - /** Class to create socket channel with compression */ - private class CompressionChannelFactory(compressionLevel: Int) - extends NioClientSocketChannelFactory { - - override def newChannel(pipeline: ChannelPipeline): SocketChannel = { - val encoder = new ZlibEncoder(compressionLevel) - pipeline.addFirst("deflater", encoder) - pipeline.addFirst("inflater", new ZlibDecoder()) - super.newChannel(pipeline) - } - } -} diff --git a/external/mqtt-assembly/pom.xml b/external/mqtt-assembly/pom.xml deleted file mode 100644 index ac2a3f65ed2f5..0000000000000 --- a/external/mqtt-assembly/pom.xml +++ /dev/null @@ -1,175 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent_2.11 - 2.0.0-SNAPSHOT - ../../pom.xml - - - org.apache.spark - spark-streaming-mqtt-assembly_2.11 - jar - Spark Project External MQTT Assembly - http://spark.apache.org/ - - - streaming-mqtt-assembly - - - - - org.apache.spark - spark-streaming-mqtt_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-streaming_${scala.binary.version} - ${project.version} - provided - - - - commons-lang - commons-lang - provided - - - com.google.protobuf - protobuf-java - provided - - - com.sun.jersey - jersey-server - provided - - - com.sun.jersey - jersey-core - provided - - - org.apache.hadoop - hadoop-client - provided - - - org.apache.avro - avro-mapred - ${avro.mapred.classifier} - provided - - - org.apache.curator - curator-recipes - provided - - - org.apache.zookeeper - zookeeper - provided - - - log4j - log4j - provided - - - net.java.dev.jets3t - jets3t - provided - - - org.scala-lang - scala-library - provided - - - org.slf4j - slf4j-api - provided - - - org.slf4j - slf4j-log4j12 - provided - - - org.xerial.snappy - snappy-java - provided - - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - org.apache.maven.plugins - maven-shade-plugin - - false - - - *:* - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - - package - - shade - - - - - - reference.conf - - - log4j.properties - - - - - - - - - - - diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml deleted file mode 100644 index d0d968782c7f1..0000000000000 --- a/external/mqtt/pom.xml +++ /dev/null @@ -1,104 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent_2.11 - 2.0.0-SNAPSHOT - ../../pom.xml - - - org.apache.spark - spark-streaming-mqtt_2.11 - - streaming-mqtt - - jar - Spark Project External MQTT - http://spark.apache.org/ - - - - org.apache.spark - spark-streaming_${scala.binary.version} - ${project.version} - provided - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - test-jar - test - - - org.eclipse.paho - org.eclipse.paho.client.mqttv3 - 1.0.2 - - - org.scalacheck - scalacheck_${scala.binary.version} - test - - - org.apache.activemq - activemq-core - 5.7.0 - test - - - org.apache.spark - spark-test-tags_${scala.binary.version} - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - - - org.apache.maven.plugins - maven-assembly-plugin - - - test-jar-with-dependencies - package - - single - - - - spark-streaming-mqtt-test-${project.version} - ${project.build.directory}/scala-${scala.binary.version}/ - false - - false - - src/main/assembly/assembly.xml - - - - - - - - diff --git a/external/mqtt/src/main/assembly/assembly.xml b/external/mqtt/src/main/assembly/assembly.xml deleted file mode 100644 index c110b01b34e10..0000000000000 --- a/external/mqtt/src/main/assembly/assembly.xml +++ /dev/null @@ -1,44 +0,0 @@ - - - test-jar-with-dependencies - - jar - - false - - - - ${project.build.directory}/scala-${scala.binary.version}/test-classes - - - - - - - true - test - true - - org.apache.hadoop:*:jar - org.apache.zookeeper:*:jar - org.apache.avro:*:jar - - - - - diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala deleted file mode 100644 index 079bd8a9a87ea..0000000000000 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala +++ /dev/null @@ -1,100 +0,0 @@ -/* - * 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.streaming.mqtt - -import org.eclipse.paho.client.mqttv3.IMqttDeliveryToken -import org.eclipse.paho.client.mqttv3.MqttCallback -import org.eclipse.paho.client.mqttv3.MqttClient -import org.eclipse.paho.client.mqttv3.MqttMessage -import org.eclipse.paho.client.mqttv3.persist.MemoryPersistence - -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.dstream._ -import org.apache.spark.streaming.receiver.Receiver - -/** - * Input stream that subscribe messages from a Mqtt Broker. - * Uses eclipse paho as MqttClient http://www.eclipse.org/paho/ - * @param brokerUrl Url of remote mqtt publisher - * @param topic topic name to subscribe to - * @param storageLevel RDD storage level. - */ - -private[streaming] -class MQTTInputDStream( - _ssc: StreamingContext, - brokerUrl: String, - topic: String, - storageLevel: StorageLevel - ) extends ReceiverInputDStream[String](_ssc) { - - private[streaming] override def name: String = s"MQTT stream [$id]" - - def getReceiver(): Receiver[String] = { - new MQTTReceiver(brokerUrl, topic, storageLevel) - } -} - -private[streaming] -class MQTTReceiver( - brokerUrl: String, - topic: String, - storageLevel: StorageLevel - ) extends Receiver[String](storageLevel) { - - def onStop() { - - } - - def onStart() { - - // Set up persistence for messages - val persistence = new MemoryPersistence() - - // Initializing Mqtt Client specifying brokerUrl, clientID and MqttClientPersistance - val client = new MqttClient(brokerUrl, MqttClient.generateClientId(), persistence) - - // Callback automatically triggers as and when new message arrives on specified topic - val callback = new MqttCallback() { - - // Handles Mqtt message - override def messageArrived(topic: String, message: MqttMessage) { - store(new String(message.getPayload(), "utf-8")) - } - - override def deliveryComplete(token: IMqttDeliveryToken) { - } - - override def connectionLost(cause: Throwable) { - restart("Connection lost ", cause) - } - } - - // Set up callback for MqttClient. This needs to happen before - // connecting or subscribing, otherwise messages may be lost - client.setCallback(callback) - - // Connect to MqttBroker - client.connect() - - // Subscribe to Mqtt topic - client.subscribe(topic) - - } -} diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala deleted file mode 100644 index 7b8d56d6faf2d..0000000000000 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala +++ /dev/null @@ -1,92 +0,0 @@ -/* - * 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.streaming.mqtt - -import scala.reflect.ClassTag - -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaDStream, JavaReceiverInputDStream, JavaStreamingContext} -import org.apache.spark.streaming.dstream.ReceiverInputDStream - -object MQTTUtils { - /** - * Create an input stream that receives messages pushed by a MQTT publisher. - * @param ssc StreamingContext object - * @param brokerUrl Url of remote MQTT publisher - * @param topic Topic name to subscribe to - * @param storageLevel RDD storage level. Defaults to StorageLevel.MEMORY_AND_DISK_SER_2. - */ - def createStream( - ssc: StreamingContext, - brokerUrl: String, - topic: String, - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): ReceiverInputDStream[String] = { - new MQTTInputDStream(ssc, brokerUrl, topic, storageLevel) - } - - /** - * Create an input stream that receives messages pushed by a MQTT publisher. - * Storage level of the data will be the default StorageLevel.MEMORY_AND_DISK_SER_2. - * @param jssc JavaStreamingContext object - * @param brokerUrl Url of remote MQTT publisher - * @param topic Topic name to subscribe to - */ - def createStream( - jssc: JavaStreamingContext, - brokerUrl: String, - topic: String - ): JavaReceiverInputDStream[String] = { - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] - createStream(jssc.ssc, brokerUrl, topic) - } - - /** - * Create an input stream that receives messages pushed by a MQTT publisher. - * @param jssc JavaStreamingContext object - * @param brokerUrl Url of remote MQTT publisher - * @param topic Topic name to subscribe to - * @param storageLevel RDD storage level. - */ - def createStream( - jssc: JavaStreamingContext, - brokerUrl: String, - topic: String, - storageLevel: StorageLevel - ): JavaReceiverInputDStream[String] = { - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] - createStream(jssc.ssc, brokerUrl, topic, storageLevel) - } -} - -/** - * This is a helper class that wraps the methods in MQTTUtils into more Python-friendly class and - * function so that it can be easily instantiated and called from Python's MQTTUtils. - */ -private[mqtt] class MQTTUtilsPythonHelper { - - def createStream( - jssc: JavaStreamingContext, - brokerUrl: String, - topic: String, - storageLevel: StorageLevel - ): JavaDStream[String] = { - MQTTUtils.createStream(jssc, brokerUrl, topic, storageLevel) - } -} diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package-info.java b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package-info.java deleted file mode 100644 index 728e0d8663d01..0000000000000 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package-info.java +++ /dev/null @@ -1,21 +0,0 @@ -/* - * 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. - */ - -/** - * MQTT receiver for Spark Streaming. - */ -package org.apache.spark.streaming.mqtt; \ No newline at end of file diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala deleted file mode 100644 index 63d0d138183a9..0000000000000 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/package.scala +++ /dev/null @@ -1,23 +0,0 @@ -/* - * 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.streaming - -/** - * MQTT receiver for Spark Streaming. - */ -package object mqtt diff --git a/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java deleted file mode 100644 index cfedb5a042a35..0000000000000 --- a/external/mqtt/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * 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.streaming; - -import org.apache.spark.SparkConf; -import org.apache.spark.streaming.api.java.JavaStreamingContext; -import org.junit.After; -import org.junit.Before; - -public abstract class LocalJavaStreamingContext { - - protected transient JavaStreamingContext ssc; - - @Before - public void setUp() { - SparkConf conf = new SparkConf() - .setMaster("local[2]") - .setAppName("test") - .set("spark.streaming.clock", "org.apache.spark.util.ManualClock"); - ssc = new JavaStreamingContext(conf, new Duration(1000)); - ssc.checkpoint("checkpoint"); - } - - @After - public void tearDown() { - ssc.stop(); - ssc = null; - } -} diff --git a/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java b/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java deleted file mode 100644 index ce5aa1e0cdda4..0000000000000 --- a/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java +++ /dev/null @@ -1,37 +0,0 @@ -/* - * 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.streaming.mqtt; - -import org.apache.spark.storage.StorageLevel; -import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; -import org.junit.Test; - -import org.apache.spark.streaming.LocalJavaStreamingContext; - -public class JavaMQTTStreamSuite extends LocalJavaStreamingContext { - @Test - public void testMQTTStream() { - String brokerUrl = "abc"; - String topic = "def"; - - // tests the API, does not actually test data receiving - JavaReceiverInputDStream test1 = MQTTUtils.createStream(ssc, brokerUrl, topic); - JavaReceiverInputDStream test2 = MQTTUtils.createStream(ssc, brokerUrl, topic, - StorageLevel.MEMORY_AND_DISK_SER_2()); - } -} diff --git a/external/mqtt/src/test/resources/log4j.properties b/external/mqtt/src/test/resources/log4j.properties deleted file mode 100644 index 75e3b53a093f6..0000000000000 --- a/external/mqtt/src/test/resources/log4j.properties +++ /dev/null @@ -1,28 +0,0 @@ -# -# 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. -# - -# Set everything to be logged to the file target/unit-tests.log -log4j.rootCategory=INFO, file -log4j.appender.file=org.apache.log4j.FileAppender -log4j.appender.file.append=true -log4j.appender.file.file=target/unit-tests.log -log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n - -# Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.spark-project.jetty=WARN - diff --git a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala deleted file mode 100644 index fdcd18c6fb048..0000000000000 --- a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala +++ /dev/null @@ -1,79 +0,0 @@ -/* - * 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.streaming.mqtt - -import scala.concurrent.duration._ -import scala.language.postfixOps - -import org.scalatest.BeforeAndAfter -import org.scalatest.concurrent.Eventually - -import org.apache.spark.SparkFunSuite -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.{Milliseconds, StreamingContext} - -class MQTTStreamSuite extends SparkFunSuite with Eventually with BeforeAndAfter { - - private val batchDuration = Milliseconds(500) - private val master = "local[2]" - private val framework = this.getClass.getSimpleName - private val topic = "def" - - private var ssc: StreamingContext = _ - private var mqttTestUtils: MQTTTestUtils = _ - - before { - ssc = new StreamingContext(master, framework, batchDuration) - mqttTestUtils = new MQTTTestUtils - mqttTestUtils.setup() - } - - after { - if (ssc != null) { - ssc.stop() - ssc = null - } - if (mqttTestUtils != null) { - mqttTestUtils.teardown() - mqttTestUtils = null - } - } - - test("mqtt input stream") { - val sendMessage = "MQTT demo for spark streaming" - val receiveStream = MQTTUtils.createStream(ssc, "tcp://" + mqttTestUtils.brokerUri, topic, - StorageLevel.MEMORY_ONLY) - - @volatile var receiveMessage: List[String] = List() - receiveStream.foreachRDD { rdd => - if (rdd.collect.length > 0) { - receiveMessage = receiveMessage ::: List(rdd.first) - receiveMessage - } - } - - ssc.start() - - // Retry it because we don't know when the receiver will start. - eventually(timeout(10000 milliseconds), interval(100 milliseconds)) { - mqttTestUtils.publishData(topic, sendMessage) - assert(sendMessage.equals(receiveMessage(0))) - } - ssc.stop() - } -} diff --git a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTTestUtils.scala b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTTestUtils.scala deleted file mode 100644 index 26c6dc45d5115..0000000000000 --- a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTTestUtils.scala +++ /dev/null @@ -1,111 +0,0 @@ -/* - * 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.streaming.mqtt - -import java.net.{ServerSocket, URI} - -import scala.language.postfixOps - -import com.google.common.base.Charsets.UTF_8 -import org.apache.activemq.broker.{BrokerService, TransportConnector} -import org.apache.commons.lang3.RandomUtils -import org.eclipse.paho.client.mqttv3._ -import org.eclipse.paho.client.mqttv3.persist.MqttDefaultFilePersistence - -import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.util.Utils - -/** - * Share codes for Scala and Python unit tests - */ -private[mqtt] class MQTTTestUtils extends Logging { - - private val persistenceDir = Utils.createTempDir() - private val brokerHost = "localhost" - private val brokerPort = findFreePort() - - private var broker: BrokerService = _ - private var connector: TransportConnector = _ - - def brokerUri: String = { - s"$brokerHost:$brokerPort" - } - - def setup(): Unit = { - broker = new BrokerService() - broker.setDataDirectoryFile(Utils.createTempDir()) - connector = new TransportConnector() - connector.setName("mqtt") - connector.setUri(new URI("mqtt://" + brokerUri)) - broker.addConnector(connector) - broker.start() - } - - def teardown(): Unit = { - if (broker != null) { - broker.stop() - broker = null - } - if (connector != null) { - connector.stop() - connector = null - } - Utils.deleteRecursively(persistenceDir) - } - - private def findFreePort(): Int = { - val candidatePort = RandomUtils.nextInt(1024, 65536) - Utils.startServiceOnPort(candidatePort, (trialPort: Int) => { - val socket = new ServerSocket(trialPort) - socket.close() - (null, trialPort) - }, new SparkConf())._2 - } - - def publishData(topic: String, data: String): Unit = { - var client: MqttClient = null - try { - val persistence = new MqttDefaultFilePersistence(persistenceDir.getAbsolutePath) - client = new MqttClient("tcp://" + brokerUri, MqttClient.generateClientId(), persistence) - client.connect() - if (client.isConnected) { - val msgTopic = client.getTopic(topic) - val message = new MqttMessage(data.getBytes(UTF_8)) - message.setQos(1) - message.setRetained(true) - - for (i <- 0 to 10) { - try { - msgTopic.publish(message) - } catch { - case e: MqttException if e.getReasonCode == MqttException.REASON_CODE_MAX_INFLIGHT => - // wait for Spark streaming to consume something from the message queue - Thread.sleep(50) - } - } - } - } finally { - if (client != null) { - client.disconnect() - client.close() - client = null - } - } - } - -} diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml deleted file mode 100644 index 5d4053afcbba7..0000000000000 --- a/external/twitter/pom.xml +++ /dev/null @@ -1,70 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent_2.11 - 2.0.0-SNAPSHOT - ../../pom.xml - - - org.apache.spark - spark-streaming-twitter_2.11 - - streaming-twitter - - jar - Spark Project External Twitter - http://spark.apache.org/ - - - - org.apache.spark - spark-streaming_${scala.binary.version} - ${project.version} - provided - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - test-jar - test - - - org.twitter4j - twitter4j-stream - 4.0.4 - - - org.scalacheck - scalacheck_${scala.binary.version} - test - - - org.apache.spark - spark-test-tags_${scala.binary.version} - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala deleted file mode 100644 index bdd57fdde3b89..0000000000000 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala +++ /dev/null @@ -1,115 +0,0 @@ -/* - * 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.streaming.twitter - -import twitter4j._ -import twitter4j.auth.Authorization -import twitter4j.auth.OAuthAuthorization -import twitter4j.conf.ConfigurationBuilder - -import org.apache.spark.Logging -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming._ -import org.apache.spark.streaming.dstream._ -import org.apache.spark.streaming.receiver.Receiver - -/* A stream of Twitter statuses, potentially filtered by one or more keywords. -* -* @constructor create a new Twitter stream using the supplied Twitter4J authentication credentials. -* An optional set of string filters can be used to restrict the set of tweets. The Twitter API is -* such that this may return a sampled subset of all tweets during each interval. -* -* If no Authorization object is provided, initializes OAuth authorization using the system -* properties twitter4j.oauth.consumerKey, .consumerSecret, .accessToken and .accessTokenSecret. -*/ -private[streaming] -class TwitterInputDStream( - _ssc: StreamingContext, - twitterAuth: Option[Authorization], - filters: Seq[String], - storageLevel: StorageLevel - ) extends ReceiverInputDStream[Status](_ssc) { - - private def createOAuthAuthorization(): Authorization = { - new OAuthAuthorization(new ConfigurationBuilder().build()) - } - - private val authorization = twitterAuth.getOrElse(createOAuthAuthorization()) - - override def getReceiver(): Receiver[Status] = { - new TwitterReceiver(authorization, filters, storageLevel) - } -} - -private[streaming] -class TwitterReceiver( - twitterAuth: Authorization, - filters: Seq[String], - storageLevel: StorageLevel - ) extends Receiver[Status](storageLevel) with Logging { - - @volatile private var twitterStream: TwitterStream = _ - @volatile private var stopped = false - - def onStart() { - try { - val newTwitterStream = new TwitterStreamFactory().getInstance(twitterAuth) - newTwitterStream.addListener(new StatusListener { - def onStatus(status: Status): Unit = { - store(status) - } - // Unimplemented - def onDeletionNotice(statusDeletionNotice: StatusDeletionNotice) {} - def onTrackLimitationNotice(i: Int) {} - def onScrubGeo(l: Long, l1: Long) {} - def onStallWarning(stallWarning: StallWarning) {} - def onException(e: Exception) { - if (!stopped) { - restart("Error receiving tweets", e) - } - } - }) - - val query = new FilterQuery - if (filters.size > 0) { - query.track(filters.mkString(",")) - newTwitterStream.filter(query) - } else { - newTwitterStream.sample() - } - setTwitterStream(newTwitterStream) - logInfo("Twitter receiver started") - stopped = false - } catch { - case e: Exception => restart("Error starting Twitter stream", e) - } - } - - def onStop() { - stopped = true - setTwitterStream(null) - logInfo("Twitter receiver stopped") - } - - private def setTwitterStream(newTwitterStream: TwitterStream) = synchronized { - if (twitterStream != null) { - twitterStream.shutdown() - } - twitterStream = newTwitterStream - } -} diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala deleted file mode 100644 index 9cb0106ab1e7b..0000000000000 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala +++ /dev/null @@ -1,132 +0,0 @@ -/* - * 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.streaming.twitter - -import twitter4j.Status -import twitter4j.auth.Authorization - -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaStreamingContext} -import org.apache.spark.streaming.dstream.ReceiverInputDStream - -object TwitterUtils { - /** - * Create a input stream that returns tweets received from Twitter. - * @param ssc StreamingContext object - * @param twitterAuth Twitter4J authentication, or None to use Twitter4J's default OAuth - * authorization; this uses the system properties twitter4j.oauth.consumerKey, - * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and - * twitter4j.oauth.accessTokenSecret - * @param filters Set of filter strings to get only those tweets that match them - * @param storageLevel Storage level to use for storing the received objects - */ - def createStream( - ssc: StreamingContext, - twitterAuth: Option[Authorization], - filters: Seq[String] = Nil, - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): ReceiverInputDStream[Status] = { - new TwitterInputDStream(ssc, twitterAuth, filters, storageLevel) - } - - /** - * Create a input stream that returns tweets received from Twitter using Twitter4J's default - * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, - * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and - * twitter4j.oauth.accessTokenSecret. - * Storage level of the data will be the default StorageLevel.MEMORY_AND_DISK_SER_2. - * @param jssc JavaStreamingContext object - */ - def createStream(jssc: JavaStreamingContext): JavaReceiverInputDStream[Status] = { - createStream(jssc.ssc, None) - } - - /** - * Create a input stream that returns tweets received from Twitter using Twitter4J's default - * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, - * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and - * twitter4j.oauth.accessTokenSecret. - * Storage level of the data will be the default StorageLevel.MEMORY_AND_DISK_SER_2. - * @param jssc JavaStreamingContext object - * @param filters Set of filter strings to get only those tweets that match them - */ - def createStream(jssc: JavaStreamingContext, filters: Array[String] - ): JavaReceiverInputDStream[Status] = { - createStream(jssc.ssc, None, filters) - } - - /** - * Create a input stream that returns tweets received from Twitter using Twitter4J's default - * OAuth authentication; this requires the system properties twitter4j.oauth.consumerKey, - * twitter4j.oauth.consumerSecret, twitter4j.oauth.accessToken and - * twitter4j.oauth.accessTokenSecret. - * @param jssc JavaStreamingContext object - * @param filters Set of filter strings to get only those tweets that match them - * @param storageLevel Storage level to use for storing the received objects - */ - def createStream( - jssc: JavaStreamingContext, - filters: Array[String], - storageLevel: StorageLevel - ): JavaReceiverInputDStream[Status] = { - createStream(jssc.ssc, None, filters, storageLevel) - } - - /** - * Create a input stream that returns tweets received from Twitter. - * Storage level of the data will be the default StorageLevel.MEMORY_AND_DISK_SER_2. - * @param jssc JavaStreamingContext object - * @param twitterAuth Twitter4J Authorization - */ - def createStream(jssc: JavaStreamingContext, twitterAuth: Authorization - ): JavaReceiverInputDStream[Status] = { - createStream(jssc.ssc, Some(twitterAuth)) - } - - /** - * Create a input stream that returns tweets received from Twitter. - * Storage level of the data will be the default StorageLevel.MEMORY_AND_DISK_SER_2. - * @param jssc JavaStreamingContext object - * @param twitterAuth Twitter4J Authorization - * @param filters Set of filter strings to get only those tweets that match them - */ - def createStream( - jssc: JavaStreamingContext, - twitterAuth: Authorization, - filters: Array[String] - ): JavaReceiverInputDStream[Status] = { - createStream(jssc.ssc, Some(twitterAuth), filters) - } - - /** - * Create a input stream that returns tweets received from Twitter. - * @param jssc JavaStreamingContext object - * @param twitterAuth Twitter4J Authorization object - * @param filters Set of filter strings to get only those tweets that match them - * @param storageLevel Storage level to use for storing the received objects - */ - def createStream( - jssc: JavaStreamingContext, - twitterAuth: Authorization, - filters: Array[String], - storageLevel: StorageLevel - ): JavaReceiverInputDStream[Status] = { - createStream(jssc.ssc, Some(twitterAuth), filters, storageLevel) - } -} diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package-info.java b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package-info.java deleted file mode 100644 index 258c0950a0aa7..0000000000000 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package-info.java +++ /dev/null @@ -1,21 +0,0 @@ -/* - * 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. - */ - -/** - * Twitter feed receiver for spark streaming. - */ -package org.apache.spark.streaming.twitter; \ No newline at end of file diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala deleted file mode 100644 index 580e37fa8f814..0000000000000 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/package.scala +++ /dev/null @@ -1,23 +0,0 @@ -/* - * 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.streaming - -/** - * Twitter feed receiver for spark streaming. - */ -package object twitter diff --git a/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java deleted file mode 100644 index cfedb5a042a35..0000000000000 --- a/external/twitter/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * 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.streaming; - -import org.apache.spark.SparkConf; -import org.apache.spark.streaming.api.java.JavaStreamingContext; -import org.junit.After; -import org.junit.Before; - -public abstract class LocalJavaStreamingContext { - - protected transient JavaStreamingContext ssc; - - @Before - public void setUp() { - SparkConf conf = new SparkConf() - .setMaster("local[2]") - .setAppName("test") - .set("spark.streaming.clock", "org.apache.spark.util.ManualClock"); - ssc = new JavaStreamingContext(conf, new Duration(1000)); - ssc.checkpoint("checkpoint"); - } - - @After - public void tearDown() { - ssc.stop(); - ssc = null; - } -} diff --git a/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java b/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java deleted file mode 100644 index 26ec8af455bcf..0000000000000 --- a/external/twitter/src/test/java/org/apache/spark/streaming/twitter/JavaTwitterStreamSuite.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * 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.streaming.twitter; - -import org.junit.Test; -import twitter4j.Status; -import twitter4j.auth.Authorization; -import twitter4j.auth.NullAuthorization; -import org.apache.spark.storage.StorageLevel; -import org.apache.spark.streaming.LocalJavaStreamingContext; -import org.apache.spark.streaming.api.java.JavaDStream; - -public class JavaTwitterStreamSuite extends LocalJavaStreamingContext { - @Test - public void testTwitterStream() { - String[] filters = { "filter1", "filter2" }; - Authorization auth = NullAuthorization.getInstance(); - - // tests the API, does not actually test data receiving - JavaDStream test1 = TwitterUtils.createStream(ssc); - JavaDStream test2 = TwitterUtils.createStream(ssc, filters); - JavaDStream test3 = TwitterUtils.createStream( - ssc, filters, StorageLevel.MEMORY_AND_DISK_SER_2()); - JavaDStream test4 = TwitterUtils.createStream(ssc, auth); - JavaDStream test5 = TwitterUtils.createStream(ssc, auth, filters); - JavaDStream test6 = TwitterUtils.createStream(ssc, - auth, filters, StorageLevel.MEMORY_AND_DISK_SER_2()); - } -} diff --git a/external/twitter/src/test/resources/log4j.properties b/external/twitter/src/test/resources/log4j.properties deleted file mode 100644 index 9a3569789d2e0..0000000000000 --- a/external/twitter/src/test/resources/log4j.properties +++ /dev/null @@ -1,28 +0,0 @@ -# -# 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. -# - -# Set everything to be logged to the filetarget/unit-tests.log -log4j.rootCategory=INFO, file -log4j.appender.file=org.apache.log4j.FileAppender -log4j.appender.file.append=true -log4j.appender.file.file=target/unit-tests.log -log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n - -# Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.spark-project.jetty=WARN - diff --git a/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala b/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala deleted file mode 100644 index 7e5fc0cbb9b30..0000000000000 --- a/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala +++ /dev/null @@ -1,59 +0,0 @@ -/* - * 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.streaming.twitter - -import org.scalatest.BeforeAndAfter -import twitter4j.Status -import twitter4j.auth.{Authorization, NullAuthorization} - -import org.apache.spark.{Logging, SparkFunSuite} -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.dstream.ReceiverInputDStream - -class TwitterStreamSuite extends SparkFunSuite with BeforeAndAfter with Logging { - - val batchDuration = Seconds(1) - - private val master: String = "local[2]" - - private val framework: String = this.getClass.getSimpleName - - test("twitter input stream") { - val ssc = new StreamingContext(master, framework, batchDuration) - val filters = Seq("filter1", "filter2") - val authorization: Authorization = NullAuthorization.getInstance() - - // tests the API, does not actually test data receiving - val test1: ReceiverInputDStream[Status] = TwitterUtils.createStream(ssc, None) - val test2: ReceiverInputDStream[Status] = - TwitterUtils.createStream(ssc, None, filters) - val test3: ReceiverInputDStream[Status] = - TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_AND_DISK_SER_2) - val test4: ReceiverInputDStream[Status] = - TwitterUtils.createStream(ssc, Some(authorization)) - val test5: ReceiverInputDStream[Status] = - TwitterUtils.createStream(ssc, Some(authorization), filters) - val test6: ReceiverInputDStream[Status] = TwitterUtils.createStream( - ssc, Some(authorization), filters, StorageLevel.MEMORY_AND_DISK_SER_2) - - // Note that actually testing the data receiving is hard as authentication keys are - // necessary for accessing Twitter live stream - ssc.stop() - } -} diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml deleted file mode 100644 index f16bc0f319744..0000000000000 --- a/external/zeromq/pom.xml +++ /dev/null @@ -1,74 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent_2.11 - 2.0.0-SNAPSHOT - ../../pom.xml - - - org.apache.spark - spark-streaming-zeromq_2.11 - - streaming-zeromq - - jar - Spark Project External ZeroMQ - http://spark.apache.org/ - - - - org.apache.spark - spark-streaming_${scala.binary.version} - ${project.version} - provided - - - org.apache.spark - spark-streaming-akka_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-core_${scala.binary.version} - ${project.version} - test-jar - test - - - ${akka.group} - akka-zeromq_${scala.binary.version} - - - org.scalacheck - scalacheck_${scala.binary.version} - test - - - org.apache.spark - spark-test-tags_${scala.binary.version} - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala deleted file mode 100644 index dd367cd43b807..0000000000000 --- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala +++ /dev/null @@ -1,55 +0,0 @@ -/* - * 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.streaming.zeromq - -import scala.reflect.ClassTag - -import akka.util.ByteString -import akka.zeromq._ - -import org.apache.spark.Logging -import org.apache.spark.streaming.akka.ActorReceiver - -/** - * A receiver to subscribe to ZeroMQ stream. - */ -private[streaming] class ZeroMQReceiver[T: ClassTag]( - publisherUrl: String, - subscribe: Subscribe, - bytesToObjects: Seq[ByteString] => Iterator[T]) - extends ActorReceiver with Logging { - - override def preStart(): Unit = { - ZeroMQExtension(context.system) - .newSocket(SocketType.Sub, Listener(self), Connect(publisherUrl), subscribe) - } - - def receive: Receive = { - - case Connecting => logInfo("connecting ...") - - case m: ZMQMessage => - logDebug("Received message for:" + m.frame(0)) - - // We ignore first frame for processing as it is the topic - val bytes = m.frames.tail - store(bytesToObjects(bytes)) - - case Closed => logInfo("received closed ") - } -} diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala deleted file mode 100644 index 1784d6e8623ad..0000000000000 --- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala +++ /dev/null @@ -1,163 +0,0 @@ -/* - * 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.streaming.zeromq - -import scala.collection.JavaConverters._ -import scala.reflect.ClassTag - -import akka.actor.{ActorSystem, Props, SupervisorStrategy} -import akka.util.ByteString -import akka.zeromq.Subscribe - -import org.apache.spark.api.java.function.{Function => JFunction, Function0 => JFunction0} -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.akka.{ActorReceiver, AkkaUtils} -import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaStreamingContext} -import org.apache.spark.streaming.dstream.ReceiverInputDStream - -object ZeroMQUtils { - /** - * Create an input stream that receives messages pushed by a zeromq publisher. - * @param ssc StreamingContext object - * @param publisherUrl Url of remote zeromq publisher - * @param subscribe Topic to subscribe to - * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic - * and each frame has sequence of byte thus it needs the converter - * (which might be deserializer of bytes) to translate from sequence - * of sequence of bytes, where sequence refer to a frame - * and sub sequence refer to its payload. - * @param storageLevel RDD storage level. Defaults to StorageLevel.MEMORY_AND_DISK_SER_2. - * @param actorSystemCreator A function to create ActorSystem in executors. `ActorSystem` will - * be shut down when the receiver is stopping (default: - * ActorReceiver.defaultActorSystemCreator) - * @param supervisorStrategy the supervisor strategy (default: ActorReceiver.defaultStrategy) - */ - def createStream[T: ClassTag]( - ssc: StreamingContext, - publisherUrl: String, - subscribe: Subscribe, - bytesToObjects: Seq[ByteString] => Iterator[T], - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2, - actorSystemCreator: () => ActorSystem = ActorReceiver.defaultActorSystemCreator, - supervisorStrategy: SupervisorStrategy = ActorReceiver.defaultSupervisorStrategy - ): ReceiverInputDStream[T] = { - AkkaUtils.createStream( - ssc, - Props(new ZeroMQReceiver(publisherUrl, subscribe, bytesToObjects)), - "ZeroMQReceiver", - storageLevel, - actorSystemCreator, - supervisorStrategy) - } - - /** - * Create an input stream that receives messages pushed by a zeromq publisher. - * @param jssc JavaStreamingContext object - * @param publisherUrl Url of remote ZeroMQ publisher - * @param subscribe Topic to subscribe to - * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each - * frame has sequence of byte thus it needs the converter(which might be - * deserializer of bytes) to translate from sequence of sequence of bytes, - * where sequence refer to a frame and sub sequence refer to its payload. - * @param storageLevel Storage level to use for storing the received objects - * @param actorSystemCreator A function to create ActorSystem in executors. `ActorSystem` will - * be shut down when the receiver is stopping. - * @param supervisorStrategy the supervisor strategy (default: ActorReceiver.defaultStrategy) - */ - def createStream[T]( - jssc: JavaStreamingContext, - publisherUrl: String, - subscribe: Subscribe, - bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]], - storageLevel: StorageLevel, - actorSystemCreator: JFunction0[ActorSystem], - supervisorStrategy: SupervisorStrategy - ): JavaReceiverInputDStream[T] = { - implicit val cm: ClassTag[T] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - val fn = - (x: Seq[ByteString]) => bytesToObjects.call(x.map(_.toArray).toArray).iterator().asScala - createStream[T]( - jssc.ssc, - publisherUrl, - subscribe, - fn, - storageLevel, - () => actorSystemCreator.call(), - supervisorStrategy) - } - - /** - * Create an input stream that receives messages pushed by a zeromq publisher. - * @param jssc JavaStreamingContext object - * @param publisherUrl Url of remote zeromq publisher - * @param subscribe Topic to subscribe to - * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each - * frame has sequence of byte thus it needs the converter(which might be - * deserializer of bytes) to translate from sequence of sequence of bytes, - * where sequence refer to a frame and sub sequence refer to its payload. - * @param storageLevel RDD storage level. - */ - def createStream[T]( - jssc: JavaStreamingContext, - publisherUrl: String, - subscribe: Subscribe, - bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]], - storageLevel: StorageLevel - ): JavaReceiverInputDStream[T] = { - implicit val cm: ClassTag[T] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - val fn = - (x: Seq[ByteString]) => bytesToObjects.call(x.map(_.toArray).toArray).iterator().asScala - createStream[T]( - jssc.ssc, - publisherUrl, - subscribe, - fn, - storageLevel) - } - - /** - * Create an input stream that receives messages pushed by a zeromq publisher. - * @param jssc JavaStreamingContext object - * @param publisherUrl Url of remote zeromq publisher - * @param subscribe Topic to subscribe to - * @param bytesToObjects A zeroMQ stream publishes sequence of frames for each topic and each - * frame has sequence of byte thus it needs the converter(which might - * be deserializer of bytes) to translate from sequence of sequence of - * bytes, where sequence refer to a frame and sub sequence refer to its - * payload. - */ - def createStream[T]( - jssc: JavaStreamingContext, - publisherUrl: String, - subscribe: Subscribe, - bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]] - ): JavaReceiverInputDStream[T] = { - implicit val cm: ClassTag[T] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - val fn = - (x: Seq[ByteString]) => bytesToObjects.call(x.map(_.toArray).toArray).iterator().asScala - createStream[T]( - jssc.ssc, - publisherUrl, - subscribe, - fn) - } -} diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package-info.java b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package-info.java deleted file mode 100644 index 587c524e2120f..0000000000000 --- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package-info.java +++ /dev/null @@ -1,21 +0,0 @@ -/* - * 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. - */ - -/** - * Zeromq receiver for spark streaming. - */ -package org.apache.spark.streaming.zeromq; \ No newline at end of file diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala deleted file mode 100644 index 65e6e57f2c05d..0000000000000 --- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/package.scala +++ /dev/null @@ -1,23 +0,0 @@ -/* - * 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.streaming - -/** - * Zeromq receiver for spark streaming. - */ -package object zeromq diff --git a/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java b/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java deleted file mode 100644 index cfedb5a042a35..0000000000000 --- a/external/zeromq/src/test/java/org/apache/spark/streaming/LocalJavaStreamingContext.java +++ /dev/null @@ -1,44 +0,0 @@ -/* - * 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.streaming; - -import org.apache.spark.SparkConf; -import org.apache.spark.streaming.api.java.JavaStreamingContext; -import org.junit.After; -import org.junit.Before; - -public abstract class LocalJavaStreamingContext { - - protected transient JavaStreamingContext ssc; - - @Before - public void setUp() { - SparkConf conf = new SparkConf() - .setMaster("local[2]") - .setAppName("test") - .set("spark.streaming.clock", "org.apache.spark.util.ManualClock"); - ssc = new JavaStreamingContext(conf, new Duration(1000)); - ssc.checkpoint("checkpoint"); - } - - @After - public void tearDown() { - ssc.stop(); - ssc = null; - } -} diff --git a/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java b/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java deleted file mode 100644 index 9ff4b41f97d50..0000000000000 --- a/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * 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.streaming.zeromq; - -import akka.actor.ActorSystem; -import akka.actor.SupervisorStrategy; -import akka.util.ByteString; -import akka.zeromq.Subscribe; -import org.junit.Test; - -import org.apache.spark.api.java.function.Function; -import org.apache.spark.api.java.function.Function0; -import org.apache.spark.storage.StorageLevel; -import org.apache.spark.streaming.LocalJavaStreamingContext; -import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; - -public class JavaZeroMQStreamSuite extends LocalJavaStreamingContext { - - @Test // tests the API, does not actually test data receiving - public void testZeroMQStream() { - String publishUrl = "abc"; - Subscribe subscribe = new Subscribe((ByteString)null); - Function> bytesToObjects = new BytesToObjects(); - Function0 actorSystemCreator = new ActorSystemCreatorForTest(); - - JavaReceiverInputDStream test1 = ZeroMQUtils.createStream( - ssc, publishUrl, subscribe, bytesToObjects); - JavaReceiverInputDStream test2 = ZeroMQUtils.createStream( - ssc, publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2()); - JavaReceiverInputDStream test3 = ZeroMQUtils.createStream( - ssc, publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2(), actorSystemCreator, - SupervisorStrategy.defaultStrategy()); - } -} - -class BytesToObjects implements Function> { - @Override - public Iterable call(byte[][] bytes) throws Exception { - return null; - } -} - -class ActorSystemCreatorForTest implements Function0 { - @Override - public ActorSystem call() { - return null; - } -} diff --git a/external/zeromq/src/test/resources/log4j.properties b/external/zeromq/src/test/resources/log4j.properties deleted file mode 100644 index 75e3b53a093f6..0000000000000 --- a/external/zeromq/src/test/resources/log4j.properties +++ /dev/null @@ -1,28 +0,0 @@ -# -# 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. -# - -# Set everything to be logged to the file target/unit-tests.log -log4j.rootCategory=INFO, file -log4j.appender.file=org.apache.log4j.FileAppender -log4j.appender.file.append=true -log4j.appender.file.file=target/unit-tests.log -log4j.appender.file.layout=org.apache.log4j.PatternLayout -log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n - -# Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.spark-project.jetty=WARN - diff --git a/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala b/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala deleted file mode 100644 index bac2679cabae5..0000000000000 --- a/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala +++ /dev/null @@ -1,63 +0,0 @@ -/* - * 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.streaming.zeromq - -import akka.actor.SupervisorStrategy -import akka.util.ByteString -import akka.zeromq.Subscribe - -import org.apache.spark.SparkFunSuite -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.{Seconds, StreamingContext} -import org.apache.spark.streaming.dstream.ReceiverInputDStream - -class ZeroMQStreamSuite extends SparkFunSuite { - - val batchDuration = Seconds(1) - - private val master: String = "local[2]" - - private val framework: String = this.getClass.getSimpleName - - test("zeromq input stream") { - val ssc = new StreamingContext(master, framework, batchDuration) - val publishUrl = "abc" - val subscribe = new Subscribe(null.asInstanceOf[ByteString]) - val bytesToObjects = (bytes: Seq[ByteString]) => null.asInstanceOf[Iterator[String]] - - // tests the API, does not actually test data receiving - val test1: ReceiverInputDStream[String] = - ZeroMQUtils.createStream( - ssc, publishUrl, subscribe, bytesToObjects, actorSystemCreator = () => null) - val test2: ReceiverInputDStream[String] = ZeroMQUtils.createStream( - ssc, publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2, () => null) - val test3: ReceiverInputDStream[String] = ZeroMQUtils.createStream( - ssc, publishUrl, subscribe, bytesToObjects, - StorageLevel.MEMORY_AND_DISK_SER_2, () => null, SupervisorStrategy.defaultStrategy) - val test4: ReceiverInputDStream[String] = - ZeroMQUtils.createStream(ssc, publishUrl, subscribe, bytesToObjects) - val test5: ReceiverInputDStream[String] = ZeroMQUtils.createStream( - ssc, publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2) - val test6: ReceiverInputDStream[String] = ZeroMQUtils.createStream( - ssc, publishUrl, subscribe, bytesToObjects, - StorageLevel.MEMORY_AND_DISK_SER_2, supervisorStrategy = SupervisorStrategy.defaultStrategy) - - // TODO: Actually test data receiving. A real test needs the native ZeroMQ library - ssc.stop() - } -} diff --git a/pom.xml b/pom.xml index ea5da3dc4c6a2..0faa691c5e78b 100644 --- a/pom.xml +++ b/pom.xml @@ -101,14 +101,6 @@ sql/hive external/docker-integration-tests assembly - external/twitter - external/flume - external/flume-sink - external/flume-assembly - external/akka - external/mqtt - external/mqtt-assembly - external/zeromq examples repl launcher @@ -119,8 +111,6 @@ UTF-8 UTF-8 - com.typesafe.akka - 2.3.11 1.7 3.3.9 spark @@ -133,7 +123,6 @@ ${hadoop.version} 0.98.17-hadoop2 hbase - 1.6.0 3.4.5 2.4.0 org.spark-project.hive @@ -197,7 +186,6 @@ during compilation if the dependency is transivite (e.g. "graphx/" depending on "core/" and needing Hadoop classes in the classpath to compile). --> - compile compile compile compile @@ -511,37 +499,6 @@ ${protobuf.version} ${hadoop.deps.scope} - - ${akka.group} - akka-actor_${scala.binary.version} - ${akka.version} - - - ${akka.group} - akka-remote_${scala.binary.version} - ${akka.version} - - - ${akka.group} - akka-slf4j_${scala.binary.version} - ${akka.version} - - - ${akka.group} - akka-testkit_${scala.binary.version} - ${akka.version} - - - ${akka.group} - akka-zeromq_${scala.binary.version} - ${akka.version} - - - ${akka.group} - akka-actor_${scala.binary.version} - - - org.apache.mesos mesos @@ -1630,46 +1587,6 @@ ${hive.parquet.version} compile - - org.apache.flume - flume-ng-core - ${flume.version} - ${flume.deps.scope} - - - io.netty - netty - - - org.apache.flume - flume-ng-auth - - - org.apache.thrift - libthrift - - - org.mortbay.jetty - servlet-api - - - - - org.apache.flume - flume-ng-sdk - ${flume.version} - ${flume.deps.scope} - - - io.netty - netty - - - org.apache.thrift - libthrift - - - org.apache.calcite calcite-core @@ -2521,9 +2438,6 @@ maven does not complain when they're provided on the command line for a sub-module that does not have them. --> - - flume-provided - hadoop-provided diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index e74fb174725d3..36ed3e2f2990b 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -39,11 +39,9 @@ object BuildCommons { ).map(ProjectRef(buildLocation, _)) val streamingProjects@Seq( - streaming, streamingFlumeSink, streamingFlume, streamingAkka, streamingKafka, streamingMqtt, - streamingTwitter, streamingZeromq + streaming, streamingKafka ) = Seq( - "streaming", "streaming-flume-sink", "streaming-flume", "streaming-akka", "streaming-kafka", - "streaming-mqtt", "streaming-twitter", "streaming-zeromq" + "streaming", "streaming-kafka" ).map(ProjectRef(buildLocation, _)) val allProjects@Seq( @@ -58,8 +56,8 @@ object BuildCommons { Seq("yarn", "java8-tests", "ganglia-lgpl", "streaming-kinesis-asl", "docker-integration-tests").map(ProjectRef(buildLocation, _)) - val assemblyProjects@Seq(assembly, examples, networkYarn, streamingFlumeAssembly, streamingKafkaAssembly, streamingMqttAssembly, streamingKinesisAslAssembly) = - Seq("assembly", "examples", "network-yarn", "streaming-flume-assembly", "streaming-kafka-assembly", "streaming-mqtt-assembly", "streaming-kinesis-asl-assembly") + val assemblyProjects@Seq(assembly, examples, networkYarn, streamingKafkaAssembly, streamingKinesisAslAssembly) = + Seq("assembly", "examples", "network-yarn", "streaming-kafka-assembly", "streaming-kinesis-asl-assembly") .map(ProjectRef(buildLocation, _)) val tools = ProjectRef(buildLocation, "tools") @@ -246,11 +244,10 @@ object SparkBuild extends PomBuild { /* Enable tests settings for all projects except examples, assembly and tools */ (allProjects ++ optionallyEnabledProjects).foreach(enable(TestSettings.settings)) - // TODO: remove streamingAkka and sketch from this list after 2.0.0 allProjects.filterNot { x => Seq( spark, hive, hiveThriftServer, catalyst, repl, networkCommon, networkShuffle, networkYarn, - unsafe, streamingAkka, testTags, sketch + unsafe, testTags, sketch ).contains(x) }.foreach { x => enable(MimaBuild.mimaSettings(sparkHome, x))(x) @@ -262,9 +259,6 @@ object SparkBuild extends PomBuild { /* Enable Assembly for all assembly projects */ assemblyProjects.foreach(enable(Assembly.settings)) - /* Enable Assembly for streamingMqtt test */ - enable(inConfig(Test)(Assembly.settings))(streamingMqtt) - /* Package pyspark artifacts in a separate zip file for YARN. */ enable(PySparkAssembly.settings)(assembly) @@ -280,8 +274,6 @@ object SparkBuild extends PomBuild { /* Hive console settings */ enable(Hive.settings)(hive) - enable(Flume.settings)(streamingFlumeSink) - enable(Java8TestSettings.settings)(java8Tests) enable(DockerIntegrationTests.settings)(dockerIntegrationTests) @@ -347,10 +339,6 @@ object Unsafe { ) } -object Flume { - lazy val settings = sbtavro.SbtAvro.avroSettings -} - object DockerIntegrationTests { // This serves to override the override specified in DependencyOverrides: lazy val settings = Seq( @@ -388,10 +376,6 @@ object OldDeps { name := "old-deps", scalaVersion := "2.10.5", libraryDependencies := Seq( - "spark-streaming-mqtt", - "spark-streaming-zeromq", - "spark-streaming-flume", - "spark-streaming-twitter", "spark-streaming", "spark-mllib", "spark-graphx", @@ -532,7 +516,7 @@ object Assembly { .getOrElse(SbtPomKeys.effectivePom.value.getProperties.get("hadoop.version").asInstanceOf[String]) }, jarName in assembly <<= (version, moduleName, hadoopVersion) map { (v, mName, hv) => - if (mName.contains("streaming-flume-assembly") || mName.contains("streaming-kafka-assembly") || mName.contains("streaming-mqtt-assembly") || mName.contains("streaming-kinesis-asl-assembly")) { + if (mName.contains("streaming-kafka-assembly") || mName.contains("streaming-kinesis-asl-assembly")) { // This must match the same name used in maven (see external/kafka-assembly/pom.xml) s"${mName}-${v}.jar" } else { @@ -630,7 +614,6 @@ object Unidoc { private def ignoreUndocumentedPackages(packages: Seq[Seq[File]]): Seq[Seq[File]] = { packages .map(_.filterNot(_.getName.contains("$"))) - .map(_.filterNot(_.getCanonicalPath.contains("akka"))) .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/deploy"))) .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/examples"))) .map(_.filterNot(_.getCanonicalPath.contains("org/apache/spark/memory"))) @@ -651,9 +634,9 @@ object Unidoc { publish := {}, unidocProjectFilter in(ScalaUnidoc, unidoc) := - inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, streamingFlumeSink, yarn, testTags), + inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, yarn, testTags), unidocProjectFilter in(JavaUnidoc, unidoc) := - inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, streamingFlumeSink, yarn, testTags), + inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, yarn, testTags), // Skip actual catalyst, but include the subproject. // Catalyst is not public API and contains quasiquotes which break scaladoc. @@ -672,8 +655,7 @@ object Unidoc { "-public", "-group", "Core Java API", packageList("api.java", "api.java.function"), "-group", "Spark Streaming", packageList( - "streaming.api.java", "streaming.flume", "streaming.akka", "streaming.kafka", - "streaming.mqtt", "streaming.twitter", "streaming.zeromq", "streaming.kinesis" + "streaming.api.java", "streaming.kafka", "streaming.kinesis" ), "-group", "MLlib", packageList( "mllib.classification", "mllib.clustering", "mllib.evaluation.binary", "mllib.linalg", @@ -773,7 +755,6 @@ object TestSettings { scalacOptions in (Compile, doc) := Seq( "-groups", "-skip-packages", Seq( - "akka", "org.apache.spark.api.python", "org.apache.spark.network", "org.apache.spark.deploy", diff --git a/python/pyspark/streaming/flume.py b/python/pyspark/streaming/flume.py deleted file mode 100644 index edd5886a85079..0000000000000 --- a/python/pyspark/streaming/flume.py +++ /dev/null @@ -1,143 +0,0 @@ -# -# 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. -# - -import sys -if sys.version >= "3": - from io import BytesIO -else: - from StringIO import StringIO -from py4j.protocol import Py4JJavaError - -from pyspark.storagelevel import StorageLevel -from pyspark.serializers import PairDeserializer, NoOpSerializer, UTF8Deserializer, read_int -from pyspark.streaming import DStream - -__all__ = ['FlumeUtils', 'utf8_decoder'] - - -def utf8_decoder(s): - """ Decode the unicode as UTF-8 """ - if s is None: - return None - return s.decode('utf-8') - - -class FlumeUtils(object): - - @staticmethod - def createStream(ssc, hostname, port, - storageLevel=StorageLevel.MEMORY_AND_DISK_2, - enableDecompression=False, - bodyDecoder=utf8_decoder): - """ - Create an input stream that pulls events from Flume. - - :param ssc: StreamingContext object - :param hostname: Hostname of the slave machine to which the flume data will be sent - :param port: Port of the slave machine to which the flume data will be sent - :param storageLevel: Storage level to use for storing the received objects - :param enableDecompression: Should netty server decompress input stream - :param bodyDecoder: A function used to decode body (default is utf8_decoder) - :return: A DStream object - """ - jlevel = ssc._sc._getJavaStorageLevel(storageLevel) - helper = FlumeUtils._get_helper(ssc._sc) - jstream = helper.createStream(ssc._jssc, hostname, port, jlevel, enableDecompression) - return FlumeUtils._toPythonDStream(ssc, jstream, bodyDecoder) - - @staticmethod - def createPollingStream(ssc, addresses, - storageLevel=StorageLevel.MEMORY_AND_DISK_2, - maxBatchSize=1000, - parallelism=5, - bodyDecoder=utf8_decoder): - """ - Creates an input stream that is to be used with the Spark Sink deployed on a Flume agent. - This stream will poll the sink for data and will pull events as they are available. - - :param ssc: StreamingContext object - :param addresses: List of (host, port)s on which the Spark Sink is running. - :param storageLevel: Storage level to use for storing the received objects - :param maxBatchSize: The maximum number of events to be pulled from the Spark sink - in a single RPC call - :param parallelism: Number of concurrent requests this stream should send to the sink. - Note that having a higher number of requests concurrently being pulled - will result in this stream using more threads - :param bodyDecoder: A function used to decode body (default is utf8_decoder) - :return: A DStream object - """ - jlevel = ssc._sc._getJavaStorageLevel(storageLevel) - hosts = [] - ports = [] - for (host, port) in addresses: - hosts.append(host) - ports.append(port) - helper = FlumeUtils._get_helper(ssc._sc) - jstream = helper.createPollingStream( - ssc._jssc, hosts, ports, jlevel, maxBatchSize, parallelism) - return FlumeUtils._toPythonDStream(ssc, jstream, bodyDecoder) - - @staticmethod - def _toPythonDStream(ssc, jstream, bodyDecoder): - ser = PairDeserializer(NoOpSerializer(), NoOpSerializer()) - stream = DStream(jstream, ssc, ser) - - def func(event): - headersBytes = BytesIO(event[0]) if sys.version >= "3" else StringIO(event[0]) - headers = {} - strSer = UTF8Deserializer() - for i in range(0, read_int(headersBytes)): - key = strSer.loads(headersBytes) - value = strSer.loads(headersBytes) - headers[key] = value - body = bodyDecoder(event[1]) - return (headers, body) - return stream.map(func) - - @staticmethod - def _get_helper(sc): - try: - helperClass = sc._jvm.java.lang.Thread.currentThread().getContextClassLoader() \ - .loadClass("org.apache.spark.streaming.flume.FlumeUtilsPythonHelper") - return helperClass.newInstance() - except Py4JJavaError as e: - # TODO: use --jar once it also work on driver - if 'ClassNotFoundException' in str(e.java_exception): - FlumeUtils._printErrorMsg(sc) - raise - - @staticmethod - def _printErrorMsg(sc): - print(""" -________________________________________________________________________________________________ - - Spark Streaming's Flume libraries not found in class path. Try one of the following. - - 1. Include the Flume library and its dependencies with in the - spark-submit command as - - $ bin/spark-submit --packages org.apache.spark:spark-streaming-flume:%s ... - - 2. Download the JAR of the artifact from Maven Central http://search.maven.org/, - Group Id = org.apache.spark, Artifact Id = spark-streaming-flume-assembly, Version = %s. - Then, include the jar in the spark-submit command as - - $ bin/spark-submit --jars ... - -________________________________________________________________________________________________ - -""" % (sc.version, sc.version)) diff --git a/python/pyspark/streaming/mqtt.py b/python/pyspark/streaming/mqtt.py deleted file mode 100644 index 388e9526ba73a..0000000000000 --- a/python/pyspark/streaming/mqtt.py +++ /dev/null @@ -1,73 +0,0 @@ -# -# 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. -# - -from py4j.protocol import Py4JJavaError - -from pyspark.storagelevel import StorageLevel -from pyspark.serializers import UTF8Deserializer -from pyspark.streaming import DStream - -__all__ = ['MQTTUtils'] - - -class MQTTUtils(object): - - @staticmethod - def createStream(ssc, brokerUrl, topic, - storageLevel=StorageLevel.MEMORY_AND_DISK_2): - """ - Create an input stream that pulls messages from a Mqtt Broker. - - :param ssc: StreamingContext object - :param brokerUrl: Url of remote mqtt publisher - :param topic: topic name to subscribe to - :param storageLevel: RDD storage level. - :return: A DStream object - """ - jlevel = ssc._sc._getJavaStorageLevel(storageLevel) - - try: - helperClass = ssc._jvm.java.lang.Thread.currentThread().getContextClassLoader() \ - .loadClass("org.apache.spark.streaming.mqtt.MQTTUtilsPythonHelper") - helper = helperClass.newInstance() - jstream = helper.createStream(ssc._jssc, brokerUrl, topic, jlevel) - except Py4JJavaError as e: - if 'ClassNotFoundException' in str(e.java_exception): - MQTTUtils._printErrorMsg(ssc.sparkContext) - raise - - return DStream(jstream, ssc, UTF8Deserializer()) - - @staticmethod - def _printErrorMsg(sc): - print(""" -________________________________________________________________________________________________ - - Spark Streaming's MQTT libraries not found in class path. Try one of the following. - - 1. Include the MQTT library and its dependencies with in the - spark-submit command as - - $ bin/spark-submit --packages org.apache.spark:spark-streaming-mqtt:%s ... - - 2. Download the JAR of the artifact from Maven Central http://search.maven.org/, - Group Id = org.apache.spark, Artifact Id = spark-streaming-mqtt-assembly, Version = %s. - Then, include the jar in the spark-submit command as - - $ bin/spark-submit --jars ... -________________________________________________________________________________________________ -""" % (sc.version, sc.version)) diff --git a/python/pyspark/streaming/tests.py b/python/pyspark/streaming/tests.py index 469c068134a46..b455fecf647fc 100644 --- a/python/pyspark/streaming/tests.py +++ b/python/pyspark/streaming/tests.py @@ -45,8 +45,6 @@ from pyspark.storagelevel import StorageLevel from pyspark.streaming.context import StreamingContext from pyspark.streaming.kafka import Broker, KafkaUtils, OffsetRange, TopicAndPartition -from pyspark.streaming.flume import FlumeUtils -from pyspark.streaming.mqtt import MQTTUtils from pyspark.streaming.kinesis import KinesisUtils, InitialPositionInStream from pyspark.streaming.listener import StreamingListener @@ -1265,216 +1263,6 @@ def getKeyAndDoubleMessage(m): self._validateStreamResult({"aa": 1, "bb": 2, "cc": 3}, stream) -class FlumeStreamTests(PySparkStreamingTestCase): - timeout = 20 # seconds - duration = 1 - - def setUp(self): - super(FlumeStreamTests, self).setUp() - - utilsClz = self.ssc._jvm.java.lang.Thread.currentThread().getContextClassLoader() \ - .loadClass("org.apache.spark.streaming.flume.FlumeTestUtils") - self._utils = utilsClz.newInstance() - - def tearDown(self): - if self._utils is not None: - self._utils.close() - self._utils = None - - super(FlumeStreamTests, self).tearDown() - - def _startContext(self, n, compressed): - # Start the StreamingContext and also collect the result - dstream = FlumeUtils.createStream(self.ssc, "localhost", self._utils.getTestPort(), - enableDecompression=compressed) - result = [] - - def get_output(_, rdd): - for event in rdd.collect(): - if len(result) < n: - result.append(event) - dstream.foreachRDD(get_output) - self.ssc.start() - return result - - def _validateResult(self, input, result): - # Validate both the header and the body - header = {"test": "header"} - self.assertEqual(len(input), len(result)) - for i in range(0, len(input)): - self.assertEqual(header, result[i][0]) - self.assertEqual(input[i], result[i][1]) - - def _writeInput(self, input, compressed): - # Try to write input to the receiver until success or timeout - start_time = time.time() - while True: - try: - self._utils.writeInput(input, compressed) - break - except: - if time.time() - start_time < self.timeout: - time.sleep(0.01) - else: - raise - - def test_flume_stream(self): - input = [str(i) for i in range(1, 101)] - result = self._startContext(len(input), False) - self._writeInput(input, False) - self.wait_for(result, len(input)) - self._validateResult(input, result) - - def test_compressed_flume_stream(self): - input = [str(i) for i in range(1, 101)] - result = self._startContext(len(input), True) - self._writeInput(input, True) - self.wait_for(result, len(input)) - self._validateResult(input, result) - - -class FlumePollingStreamTests(PySparkStreamingTestCase): - timeout = 20 # seconds - duration = 1 - maxAttempts = 5 - - def setUp(self): - utilsClz = \ - self.sc._jvm.java.lang.Thread.currentThread().getContextClassLoader() \ - .loadClass("org.apache.spark.streaming.flume.PollingFlumeTestUtils") - self._utils = utilsClz.newInstance() - - def tearDown(self): - if self._utils is not None: - self._utils.close() - self._utils = None - - def _writeAndVerify(self, ports): - # Set up the streaming context and input streams - ssc = StreamingContext(self.sc, self.duration) - try: - addresses = [("localhost", port) for port in ports] - dstream = FlumeUtils.createPollingStream( - ssc, - addresses, - maxBatchSize=self._utils.eventsPerBatch(), - parallelism=5) - outputBuffer = [] - - def get_output(_, rdd): - for e in rdd.collect(): - outputBuffer.append(e) - - dstream.foreachRDD(get_output) - ssc.start() - self._utils.sendDatAndEnsureAllDataHasBeenReceived() - - self.wait_for(outputBuffer, self._utils.getTotalEvents()) - outputHeaders = [event[0] for event in outputBuffer] - outputBodies = [event[1] for event in outputBuffer] - self._utils.assertOutput(outputHeaders, outputBodies) - finally: - ssc.stop(False) - - def _testMultipleTimes(self, f): - attempt = 0 - while True: - try: - f() - break - except: - attempt += 1 - if attempt >= self.maxAttempts: - raise - else: - import traceback - traceback.print_exc() - - def _testFlumePolling(self): - try: - port = self._utils.startSingleSink() - self._writeAndVerify([port]) - self._utils.assertChannelsAreEmpty() - finally: - self._utils.close() - - def _testFlumePollingMultipleHosts(self): - try: - port = self._utils.startSingleSink() - self._writeAndVerify([port]) - self._utils.assertChannelsAreEmpty() - finally: - self._utils.close() - - def test_flume_polling(self): - self._testMultipleTimes(self._testFlumePolling) - - def test_flume_polling_multiple_hosts(self): - self._testMultipleTimes(self._testFlumePollingMultipleHosts) - - -class MQTTStreamTests(PySparkStreamingTestCase): - timeout = 20 # seconds - duration = 1 - - def setUp(self): - super(MQTTStreamTests, self).setUp() - - MQTTTestUtilsClz = self.ssc._jvm.java.lang.Thread.currentThread().getContextClassLoader() \ - .loadClass("org.apache.spark.streaming.mqtt.MQTTTestUtils") - self._MQTTTestUtils = MQTTTestUtilsClz.newInstance() - self._MQTTTestUtils.setup() - - def tearDown(self): - if self._MQTTTestUtils is not None: - self._MQTTTestUtils.teardown() - self._MQTTTestUtils = None - - super(MQTTStreamTests, self).tearDown() - - def _randomTopic(self): - return "topic-%d" % random.randint(0, 10000) - - def _startContext(self, topic): - # Start the StreamingContext and also collect the result - stream = MQTTUtils.createStream(self.ssc, "tcp://" + self._MQTTTestUtils.brokerUri(), topic) - result = [] - - def getOutput(_, rdd): - for data in rdd.collect(): - result.append(data) - - stream.foreachRDD(getOutput) - self.ssc.start() - return result - - def test_mqtt_stream(self): - """Test the Python MQTT stream API.""" - sendData = "MQTT demo for spark streaming" - topic = self._randomTopic() - result = self._startContext(topic) - - def retry(): - self._MQTTTestUtils.publishData(topic, sendData) - # Because "publishData" sends duplicate messages, here we should use > 0 - self.assertTrue(len(result) > 0) - self.assertEqual(sendData, result[0]) - - # Retry it because we don't know when the receiver will start. - self._retry_or_timeout(retry) - - def _retry_or_timeout(self, test_func): - start_time = time.time() - while True: - try: - test_func() - break - except: - if time.time() - start_time > self.timeout: - raise - time.sleep(0.01) - - class KinesisStreamTests(PySparkStreamingTestCase): def test_kinesis_stream_api(self): @@ -1566,57 +1354,6 @@ def search_kafka_assembly_jar(): return jars[0] -def search_flume_assembly_jar(): - SPARK_HOME = os.environ["SPARK_HOME"] - flume_assembly_dir = os.path.join(SPARK_HOME, "external/flume-assembly") - jars = search_jar(flume_assembly_dir, "spark-streaming-flume-assembly") - if not jars: - raise Exception( - ("Failed to find Spark Streaming Flume assembly jar in %s. " % flume_assembly_dir) + - "You need to build Spark with " - "'build/sbt assembly/assembly streaming-flume-assembly/assembly' or " - "'build/mvn package' before running this test.") - elif len(jars) > 1: - raise Exception(("Found multiple Spark Streaming Flume assembly JARs: %s; please " - "remove all but one") % (", ".join(jars))) - else: - return jars[0] - - -def search_mqtt_assembly_jar(): - SPARK_HOME = os.environ["SPARK_HOME"] - mqtt_assembly_dir = os.path.join(SPARK_HOME, "external/mqtt-assembly") - jars = search_jar(mqtt_assembly_dir, "spark-streaming-mqtt-assembly") - if not jars: - raise Exception( - ("Failed to find Spark Streaming MQTT assembly jar in %s. " % mqtt_assembly_dir) + - "You need to build Spark with " - "'build/sbt assembly/assembly streaming-mqtt-assembly/assembly' or " - "'build/mvn package' before running this test") - elif len(jars) > 1: - raise Exception(("Found multiple Spark Streaming MQTT assembly JARs: %s; please " - "remove all but one") % (", ".join(jars))) - else: - return jars[0] - - -def search_mqtt_test_jar(): - SPARK_HOME = os.environ["SPARK_HOME"] - mqtt_test_dir = os.path.join(SPARK_HOME, "external/mqtt") - jars = glob.glob( - os.path.join(mqtt_test_dir, "target/scala-*/spark-streaming-mqtt-test-*.jar")) - if not jars: - raise Exception( - ("Failed to find Spark Streaming MQTT test jar in %s. " % mqtt_test_dir) + - "You need to build Spark with " - "'build/sbt assembly/assembly streaming-mqtt/test:assembly'") - elif len(jars) > 1: - raise Exception(("Found multiple Spark Streaming MQTT test JARs: %s; please " - "remove all but one") % (", ".join(jars))) - else: - return jars[0] - - def search_kinesis_asl_assembly_jar(): SPARK_HOME = os.environ["SPARK_HOME"] kinesis_asl_assembly_dir = os.path.join(SPARK_HOME, "external/kinesis-asl-assembly") @@ -1637,24 +1374,18 @@ def search_kinesis_asl_assembly_jar(): if __name__ == "__main__": from pyspark.streaming.tests import * kafka_assembly_jar = search_kafka_assembly_jar() - flume_assembly_jar = search_flume_assembly_jar() - mqtt_assembly_jar = search_mqtt_assembly_jar() - mqtt_test_jar = search_mqtt_test_jar() kinesis_asl_assembly_jar = search_kinesis_asl_assembly_jar() if kinesis_asl_assembly_jar is None: kinesis_jar_present = False - jars = "%s,%s,%s,%s" % (kafka_assembly_jar, flume_assembly_jar, mqtt_assembly_jar, - mqtt_test_jar) + jars = kafka_assembly_jar else: kinesis_jar_present = True - jars = "%s,%s,%s,%s,%s" % (kafka_assembly_jar, flume_assembly_jar, mqtt_assembly_jar, - mqtt_test_jar, kinesis_asl_assembly_jar) + jars = "%s,%s" % (kafka_assembly_jar, kinesis_asl_assembly_jar) os.environ["PYSPARK_SUBMIT_ARGS"] = "--jars %s pyspark-shell" % jars testcases = [BasicOperationTests, WindowFunctionTests, StreamingContextTests, CheckpointTests, - KafkaStreamTests, FlumeStreamTests, FlumePollingStreamTests, MQTTStreamTests, - StreamingListenerTests] + KafkaStreamTests, StreamingListenerTests] if kinesis_jar_present is True: testcases.append(KinesisStreamTests)