Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion dev/audit-release/audit_release.py
Original file line number Diff line number Diff line change
Expand Up @@ -116,7 +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-kafka",
"spark-graphx", "spark-streaming-flume", "spark-streaming-kafka",
"spark-catalyst", "spark-sql", "spark-hive", "spark-streaming-kinesis-asl"
]
modules = map(lambda m: "%s_%s" % (m, SCALA_BINARY_VERSION), modules)
Expand Down
1 change: 1 addition & 0 deletions dev/run-tests.py
Original file line number Diff line number Diff line change
Expand Up @@ -337,6 +337,7 @@ 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-kinesis-asl-assembly/assembly"]
profiles_and_goals = build_profiles + sbt_goals

Expand Down
34 changes: 34 additions & 0 deletions dev/sparktestsupport/modules.py
Original file line number Diff line number Diff line change
Expand Up @@ -223,6 +223,39 @@ 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_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],
Expand Down Expand Up @@ -294,6 +327,7 @@ def __hash__(self):
pyspark_core,
streaming,
streaming_kafka,
streaming_flume_assembly,
streaming_kinesis_asl
],
source_file_regexes=[
Expand Down
5 changes: 5 additions & 0 deletions examples/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,11 @@
<version>${project.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-streaming-flume_${scala.binary.version}</artifactId>
<version>${project.version}</version>
</dependency>
<dependency>
<groupId>org.apache.spark</groupId>
<artifactId>spark-streaming-kafka_${scala.binary.version}</artifactId>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,75 @@
/*
* 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 <host> <port>
* <host> is the host the Flume receiver will be started on - a receiver
* creates a server and listens for flume events.
* <port> is the port the Flume receiver will listen on.
*
* To run this example:
* `$ bin/run-example org.apache.spark.examples.streaming.JavaFlumeEventCount <host> <port>`
*/
public final class JavaFlumeEventCount {
private JavaFlumeEventCount() {
}

public static void main(String[] args) {
if (args.length != 2) {
System.err.println("Usage: JavaFlumeEventCount <host> <port>");
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<SparkFlumeEvent> flumeStream = FlumeUtils.createStream(ssc, host, port);

flumeStream.count();

flumeStream.count().map(new Function<Long, String>() {
@Override
public String call(Long in) {
return "Received " + in + " flume events.";
}
}).print();

ssc.start();
ssc.awaitTermination();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,70 @@
/*
* 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 <host> <port>
* <host> is the host the Flume receiver will be started on - a receiver
* creates a server and listens for flume events.
* <port> is the port the Flume receiver will listen on.
*
* To run this example:
* `$ bin/run-example org.apache.spark.examples.streaming.FlumeEventCount <host> <port> `
*/
object FlumeEventCount {
def main(args: Array[String]) {
if (args.length < 2) {
System.err.println(
"Usage: FlumeEventCount <host> <port>")
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
Original file line number Diff line number Diff line change
@@ -0,0 +1,67 @@
/*
* 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> <port>
* `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 <host> <port>")
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
Loading