[SPARK-25598][STREAMING][BUILD][TEST-MAVEN] Remove flume connector in Spark 3

## What changes were proposed in this pull request?

Removes all vestiges of Flume in the build, for Spark 3.
I don't think this needs Jenkins config changes.

## How was this patch tested?

Existing tests.

Closes #22692 from srowen/SPARK-25598.

Authored-by: Sean Owen <[email protected]>
Signed-off-by: Sean Owen <[email protected]>


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a0018141
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a0018141
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a0018141

Branch: refs/heads/master
Commit: a00181418911307725524641254439712e95445b
Parents: 69f5e9c
Author: Sean Owen <[email protected]>
Authored: Thu Oct 11 14:28:06 2018 -0700
Committer: Sean Owen <[email protected]>
Committed: Thu Oct 11 14:28:06 2018 -0700

----------------------------------------------------------------------
 dev/create-release/release-build.sh             |   2 +-
 dev/mima                                        |   2 +-
 dev/run-tests.py                                |   1 -
 dev/sbt-checkstyle                              |   1 -
 dev/scalastyle                                  |   1 -
 dev/sparktestsupport/modules.py                 |  52 ----
 dev/test-dependencies.sh                        |   2 +-
 docs/building-spark.md                          |   7 -
 docs/streaming-custom-receivers.md              |   2 +-
 docs/streaming-flume-integration.md             | 169 ----------
 docs/streaming-programming-guide.md             |  29 +-
 .../main/python/streaming/flume_wordcount.py    |  56 ----
 external/flume-assembly/pom.xml                 | 167 ----------
 external/flume-sink/pom.xml                     | 140 ---------
 .../flume-sink/src/main/avro/sparkflume.avdl    |  40 ---
 .../spark/streaming/flume/sink/Logging.scala    | 127 --------
 .../flume/sink/SparkAvroCallbackHandler.scala   | 166 ----------
 .../spark/streaming/flume/sink/SparkSink.scala  | 171 ----------
 .../flume/sink/SparkSinkThreadFactory.scala     |  35 ---
 .../streaming/flume/sink/SparkSinkUtils.scala   |  28 --
 .../flume/sink/TransactionProcessor.scala       | 252 ---------------
 .../src/test/resources/log4j.properties         |  28 --
 .../streaming/flume/sink/SparkSinkSuite.scala   | 218 -------------
 external/flume/pom.xml                          |  89 ------
 .../spark/examples/JavaFlumeEventCount.java     |  67 ----
 .../apache/spark/examples/FlumeEventCount.scala |  68 ----
 .../spark/examples/FlumePollingEventCount.scala |  65 ----
 .../streaming/flume/EventTransformer.scala      |  72 -----
 .../streaming/flume/FlumeBatchFetcher.scala     | 166 ----------
 .../streaming/flume/FlumeInputDStream.scala     | 208 -------------
 .../flume/FlumePollingInputDStream.scala        | 123 --------
 .../spark/streaming/flume/FlumeTestUtils.scala  | 117 -------
 .../spark/streaming/flume/FlumeUtils.scala      | 312 -------------------
 .../streaming/flume/PollingFlumeTestUtils.scala | 209 -------------
 .../spark/streaming/flume/package-info.java     |  21 --
 .../apache/spark/streaming/flume/package.scala  |  23 --
 .../streaming/LocalJavaStreamingContext.java    |  44 ---
 .../flume/JavaFlumePollingStreamSuite.java      |  44 ---
 .../streaming/flume/JavaFlumeStreamSuite.java   |  37 ---
 .../flume/src/test/resources/log4j.properties   |  28 --
 .../spark/streaming/TestOutputStream.scala      |  48 ---
 .../flume/FlumePollingStreamSuite.scala         | 149 ---------
 .../streaming/flume/FlumeStreamSuite.scala      | 103 ------
 .../kafka010/DirectKafkaInputDStream.scala      |   1 -
 .../kafka/DirectKafkaInputDStream.scala         |   1 -
 pom.xml                                         |  54 ----
 project/SparkBuild.scala                        |  18 +-
 python/docs/pyspark.streaming.rst               |   7 -
 python/pyspark/streaming/dstream.py             |   2 +-
 python/pyspark/streaming/flume.py               | 156 ----------
 python/pyspark/streaming/tests.py               | 176 +----------
 .../spark/streaming/StreamingContext.scala      |   2 +-
 .../spark/streaming/api/java/JavaDStream.scala  |   2 +-
 .../spark/streaming/dstream/DStream.scala       |   2 +-
 .../spark/streaming/dstream/InputDStream.scala  |   1 -
 55 files changed, 27 insertions(+), 4084 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/a0018141/dev/create-release/release-build.sh
----------------------------------------------------------------------
diff --git a/dev/create-release/release-build.sh 
b/dev/create-release/release-build.sh
index 89593cf..b80f55d 100755
--- a/dev/create-release/release-build.sh
+++ b/dev/create-release/release-build.sh
@@ -114,7 +114,7 @@ PUBLISH_SCALA_2_10=0
 SCALA_2_10_PROFILES="-Pscala-2.10"
 SCALA_2_11_PROFILES=
 if [[ $SPARK_VERSION > "2.3" ]]; then
-  BASE_PROFILES="$BASE_PROFILES -Pkubernetes -Pflume"
+  BASE_PROFILES="$BASE_PROFILES -Pkubernetes"
   SCALA_2_11_PROFILES="-Pkafka-0-8"
 else
   PUBLISH_SCALA_2_10=1

http://git-wip-us.apache.org/repos/asf/spark/blob/a0018141/dev/mima
----------------------------------------------------------------------
diff --git a/dev/mima b/dev/mima
index cd2694f..a9ac8af 100755
--- a/dev/mima
+++ b/dev/mima
@@ -24,7 +24,7 @@ set -e
 FWDIR="$(cd "`dirname "$0"`"/..; pwd)"
 cd "$FWDIR"
 
-SPARK_PROFILES="-Pmesos -Pkafka-0-8 -Pkubernetes -Pyarn -Pflume 
-Pspark-ganglia-lgpl -Pkinesis-asl -Phive-thriftserver -Phive"
+SPARK_PROFILES="-Pmesos -Pkafka-0-8 -Pkubernetes -Pyarn -Pspark-ganglia-lgpl 
-Pkinesis-asl -Phive-thriftserver -Phive"
 TOOLS_CLASSPATH="$(build/sbt -DcopyDependencies=false "export 
tools/fullClasspath" | tail -n1)"
 OLD_DEPS_CLASSPATH="$(build/sbt -DcopyDependencies=false $SPARK_PROFILES 
"export oldDeps/fullClasspath" | tail -n1)"
 

http://git-wip-us.apache.org/repos/asf/spark/blob/a0018141/dev/run-tests.py
----------------------------------------------------------------------
diff --git a/dev/run-tests.py b/dev/run-tests.py
index 271360b..a125f5b 100755
--- a/dev/run-tests.py
+++ b/dev/run-tests.py
@@ -333,7 +333,6 @@ def build_spark_sbt(hadoop_version):
     build_profiles = get_hadoop_profiles(hadoop_version) + 
modules.root.build_profile_flags
     sbt_goals = ["test:package",  # Build test jars as some tests depend on 
them
                  "streaming-kafka-0-8-assembly/assembly",
-                 "streaming-flume-assembly/assembly",
                  "streaming-kinesis-asl-assembly/assembly"]
     profiles_and_goals = build_profiles + sbt_goals
 

http://git-wip-us.apache.org/repos/asf/spark/blob/a0018141/dev/sbt-checkstyle
----------------------------------------------------------------------
diff --git a/dev/sbt-checkstyle b/dev/sbt-checkstyle
index 8821a7c..1e825db 100755
--- a/dev/sbt-checkstyle
+++ b/dev/sbt-checkstyle
@@ -26,7 +26,6 @@ ERRORS=$(echo -e "q\n" \
         -Pkafka-0-8 \
         -Pkubernetes \
         -Pyarn \
-        -Pflume \
         -Phive \
         -Phive-thriftserver \
         checkstyle test:checkstyle \

http://git-wip-us.apache.org/repos/asf/spark/blob/a0018141/dev/scalastyle
----------------------------------------------------------------------
diff --git a/dev/scalastyle b/dev/scalastyle
index b0ad025..0448e1d 100755
--- a/dev/scalastyle
+++ b/dev/scalastyle
@@ -26,7 +26,6 @@ ERRORS=$(echo -e "q\n" \
         -Pkafka-0-8 \
         -Pkubernetes \
         -Pyarn \
-        -Pflume \
         -Phive \
         -Phive-thriftserver \
         -Pspark-ganglia-lgpl \

http://git-wip-us.apache.org/repos/asf/spark/blob/a0018141/dev/sparktestsupport/modules.py
----------------------------------------------------------------------
diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py
index e7ac063..bd5f009 100644
--- a/dev/sparktestsupport/modules.py
+++ b/dev/sparktestsupport/modules.py
@@ -283,56 +283,6 @@ streaming_kafka_0_10 = Module(
     ]
 )
 
-streaming_flume_sink = Module(
-    name="streaming-flume-sink",
-    dependencies=[streaming],
-    source_file_regexes=[
-        "external/flume-sink",
-    ],
-    build_profile_flags=[
-        "-Pflume",
-    ],
-    environ={
-        "ENABLE_FLUME_TESTS": "1"
-    },
-    sbt_test_goals=[
-        "streaming-flume-sink/test",
-    ]
-)
-
-
-streaming_flume = Module(
-    name="streaming-flume",
-    dependencies=[streaming],
-    source_file_regexes=[
-        "external/flume",
-    ],
-    build_profile_flags=[
-        "-Pflume",
-    ],
-    environ={
-        "ENABLE_FLUME_TESTS": "1"
-    },
-    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",
-    ],
-    build_profile_flags=[
-        "-Pflume",
-    ],
-    environ={
-        "ENABLE_FLUME_TESTS": "1"
-    }
-)
-
 
 mllib_local = Module(
     name="mllib-local",
@@ -425,14 +375,12 @@ pyspark_streaming = Module(
         pyspark_core,
         streaming,
         streaming_kafka,
-        streaming_flume_assembly,
         streaming_kinesis_asl
     ],
     source_file_regexes=[
         "python/pyspark/streaming"
     ],
     environ={
-        "ENABLE_FLUME_TESTS": "1",
         "ENABLE_KAFKA_0_8_TESTS": "1"
     },
     python_test_goals=[

http://git-wip-us.apache.org/repos/asf/spark/blob/a0018141/dev/test-dependencies.sh
----------------------------------------------------------------------
diff --git a/dev/test-dependencies.sh b/dev/test-dependencies.sh
index a3627c9..cc8f5d3 100755
--- a/dev/test-dependencies.sh
+++ b/dev/test-dependencies.sh
@@ -29,7 +29,7 @@ export LC_ALL=C
 # TODO: This would be much nicer to do in SBT, once SBT supports Maven-style 
resolution.
 
 # NOTE: These should match those in the release publishing script
-HADOOP2_MODULE_PROFILES="-Phive-thriftserver -Pmesos -Pkafka-0-8 -Pkubernetes 
-Pyarn -Pflume -Phive"
+HADOOP2_MODULE_PROFILES="-Phive-thriftserver -Pmesos -Pkafka-0-8 -Pkubernetes 
-Pyarn -Phive"
 MVN="build/mvn"
 HADOOP_PROFILES=(
     hadoop-2.7

http://git-wip-us.apache.org/repos/asf/spark/blob/a0018141/docs/building-spark.md
----------------------------------------------------------------------
diff --git a/docs/building-spark.md b/docs/building-spark.md
index b9e1715..55830d3 100644
--- a/docs/building-spark.md
+++ b/docs/building-spark.md
@@ -99,13 +99,6 @@ Note: Kafka 0.8 support is deprecated as of Spark 2.3.0.
 
 Kafka 0.10 support is still automatically built.
 
-## Building with Flume support
-
-Apache Flume support must be explicitly enabled with the `flume` profile.
-Note: Flume support is deprecated as of Spark 2.3.0.
-
-    ./build/mvn -Pflume -DskipTests clean package
-
 ## Building submodules individually
 
 It's possible to build Spark submodules using the `mvn -pl` option.

http://git-wip-us.apache.org/repos/asf/spark/blob/a0018141/docs/streaming-custom-receivers.md
----------------------------------------------------------------------
diff --git a/docs/streaming-custom-receivers.md 
b/docs/streaming-custom-receivers.md
index 44ae52e..a83ebd9 100644
--- a/docs/streaming-custom-receivers.md
+++ b/docs/streaming-custom-receivers.md
@@ -4,7 +4,7 @@ title: Spark Streaming Custom Receivers
 ---
 
 Spark Streaming can receive streaming data from any arbitrary data source 
beyond
-the ones for which it has built-in support (that is, beyond Flume, Kafka, 
Kinesis, files, sockets, etc.).
+the ones for which it has built-in support (that is, beyond Kafka, Kinesis, 
files, sockets, etc.).
 This requires the developer to implement a *receiver* that is customized for 
receiving data from
 the concerned data source. This guide walks through the process of 
implementing a custom receiver
 and using it in a Spark Streaming application. Note that custom receivers can 
be implemented

http://git-wip-us.apache.org/repos/asf/spark/blob/a0018141/docs/streaming-flume-integration.md
----------------------------------------------------------------------
diff --git a/docs/streaming-flume-integration.md 
b/docs/streaming-flume-integration.md
deleted file mode 100644
index a1b6942..0000000
--- a/docs/streaming-flume-integration.md
+++ /dev/null
@@ -1,169 +0,0 @@
----
-layout: global
-title: Spark Streaming + Flume Integration Guide
----
-
-[Apache Flume](https://flume.apache.org/) is a distributed, reliable, and 
available service for efficiently collecting, aggregating, and moving large 
amounts of log data. Here we explain how to configure Flume and Spark Streaming 
to receive data from Flume. There are two approaches to this.
-
-**Note: Flume support is deprecated as of Spark 2.3.0.**
-
-## Approach 1: Flume-style Push-based Approach
-Flume is designed to push data between Flume agents. In this approach, Spark 
Streaming essentially sets up a receiver that acts an Avro agent for Flume, to 
which Flume can push the data. Here are the configuration steps.
-
-#### General Requirements
-Choose a machine in your cluster such that
-
-- When your Flume + Spark Streaming application is launched, one of the Spark 
workers must run on that machine.
-
-- Flume can be configured to push data to a port on that machine.
-
-Due to the push model, the streaming application needs to be up, with the 
receiver scheduled and listening on the chosen port, for Flume to be able to 
push data.
-
-#### Configuring Flume
-Configure Flume agent to send data to an Avro sink by having the following in 
the configuration file.
-
-       agent.sinks = avroSink
-       agent.sinks.avroSink.type = avro
-    agent.sinks.avroSink.channel = memoryChannel
-    agent.sinks.avroSink.hostname = <chosen machine's hostname>
-       agent.sinks.avroSink.port = <chosen port on the machine>
-
-See the [Flume's documentation](https://flume.apache.org/documentation.html) 
for more information about
-configuring Flume agents.
-
-#### Configuring Spark Streaming Application
-1. **Linking:** In your SBT/Maven project definition, link your streaming 
application against the following artifact (see [Linking 
section](streaming-programming-guide.html#linking) in the main programming 
guide for further information).
-
-               groupId = org.apache.spark
-               artifactId = spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}
-               version = {{site.SPARK_VERSION_SHORT}}
-
-2. **Programming:** In the streaming application code, import `FlumeUtils` and 
create input DStream as follows.
-
-       <div class="codetabs">
-       <div data-lang="scala" markdown="1">
-               import org.apache.spark.streaming.flume._
-
-               val flumeStream = FlumeUtils.createStream(streamingContext, 
[chosen machine's hostname], [chosen port])
-
-       See the [API 
docs](api/scala/index.html#org.apache.spark.streaming.flume.FlumeUtils$).
-       </div>
-       <div data-lang="java" markdown="1">
-               import org.apache.spark.streaming.flume.*;
-
-               JavaReceiverInputDStream<SparkFlumeEvent> flumeStream =
-               FlumeUtils.createStream(streamingContext, [chosen machine's 
hostname], [chosen port]);
-
-       See the [API 
docs](api/java/index.html?org/apache/spark/streaming/flume/FlumeUtils.html).
-       </div>
-       <div data-lang="python" markdown="1">
-               from pyspark.streaming.flume import FlumeUtils
-
-               flumeStream = FlumeUtils.createStream(streamingContext, [chosen 
machine's hostname], [chosen port])
-
-       By default, the Python API will decode Flume event body as UTF8 encoded 
strings. You can specify your custom decoding function to decode the body byte 
arrays in Flume events to any arbitrary data type. 
-       See the [API 
docs](api/python/pyspark.streaming.html#pyspark.streaming.flume.FlumeUtils).
-       </div>
-       </div>
-
-       Note that the hostname should be the same as the one used by the 
resource manager in the
-    cluster (Mesos, YARN or Spark Standalone), so that resource allocation can 
match the names and launch
-    the receiver in the right machine.
-
-3. **Deploying:** As with any Spark applications, `spark-submit` is used to 
launch your application. However, the details are slightly different for 
Scala/Java applications and Python applications.
-
-       For Scala and Java applications, if you are using SBT or Maven for 
project management, then package 
`spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}` and its dependencies into 
the application JAR. Make sure `spark-core_{{site.SCALA_BINARY_VERSION}}` and 
`spark-streaming_{{site.SCALA_BINARY_VERSION}}` are marked as `provided` 
dependencies as those are already present in a Spark installation. Then use 
`spark-submit` to launch your application (see [Deploying 
section](streaming-programming-guide.html#deploying-applications) in the main 
programming guide).
-
-       For Python applications which lack SBT/Maven project management, 
`spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}` and its dependencies can 
be directly added to `spark-submit` using `--packages` (see [Application 
Submission Guide](submitting-applications.html)). That is,
-
-           ./bin/spark-submit --packages 
org.apache.spark:spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}}
 ...
-
-       Alternatively, you can also download the JAR of the Maven artifact 
`spark-streaming-flume-assembly` from the
-       [Maven 
repository](http://search.maven.org/#search|ga|1|a%3A%22spark-streaming-flume-assembly_{{site.SCALA_BINARY_VERSION}}%22%20AND%20v%3A%22{{site.SPARK_VERSION_SHORT}}%22)
 and add it to `spark-submit` with `--jars`.
-
-## Approach 2: Pull-based Approach using a Custom Sink
-Instead of Flume pushing data directly to Spark Streaming, this approach runs 
a custom Flume sink that allows the following.
-
-- Flume pushes data into the sink, and the data stays buffered.
-- Spark Streaming uses a [reliable Flume 
receiver](streaming-programming-guide.html#receiver-reliability)
-  and transactions to pull data from the sink. Transactions succeed only after 
data is received and
-  replicated by Spark Streaming.
-
-This ensures stronger reliability and
-[fault-tolerance 
guarantees](streaming-programming-guide.html#fault-tolerance-semantics)
-than the previous approach. However, this requires configuring Flume to run a 
custom sink.
-Here are the configuration steps.
-
-#### General Requirements
-Choose a machine that will run the custom sink in a Flume agent. The rest of 
the Flume pipeline is configured to send data to that agent. Machines in the 
Spark cluster should have access to the chosen machine running the custom sink.
-
-#### Configuring Flume
-Configuring Flume on the chosen machine requires the following two steps.
-
-1. **Sink JARs**: Add the following JARs to Flume's classpath (see [Flume's 
documentation](https://flume.apache.org/documentation.html) to see how) in the 
machine designated to run the custom sink.
-
-       (i) *Custom sink JAR*: Download the JAR corresponding to the following 
artifact (or [direct 
link](http://search.maven.org/remotecontent?filepath=org/apache/spark/spark-streaming-flume-sink_{{site.SCALA_BINARY_VERSION}}/{{site.SPARK_VERSION_SHORT}}/spark-streaming-flume-sink_{{site.SCALA_BINARY_VERSION}}-{{site.SPARK_VERSION_SHORT}}.jar)).
-
-               groupId = org.apache.spark
-               artifactId = 
spark-streaming-flume-sink_{{site.SCALA_BINARY_VERSION}}
-               version = {{site.SPARK_VERSION_SHORT}}
-
-       (ii) *Scala library JAR*: Download the Scala library JAR for Scala 
{{site.SCALA_VERSION}}. It can be found with the following artifact detail (or, 
[direct 
link](http://search.maven.org/remotecontent?filepath=org/scala-lang/scala-library/{{site.SCALA_VERSION}}/scala-library-{{site.SCALA_VERSION}}.jar)).
-
-               groupId = org.scala-lang
-               artifactId = scala-library
-               version = {{site.SCALA_VERSION}}
-
-       (iii) *Commons Lang 3 JAR*: Download the Commons Lang 3 JAR. It can be 
found with the following artifact detail (or, [direct 
link](http://search.maven.org/remotecontent?filepath=org/apache/commons/commons-lang3/3.5/commons-lang3-3.5.jar)).
-
-               groupId = org.apache.commons
-               artifactId = commons-lang3
-               version = 3.5
-
-2. **Configuration file**: On that machine, configure Flume agent to send data 
to an Avro sink by having the following in the configuration file.
-
-               agent.sinks = spark
-               agent.sinks.spark.type = 
org.apache.spark.streaming.flume.sink.SparkSink
-               agent.sinks.spark.hostname = <hostname of the local machine>
-               agent.sinks.spark.port = <port to listen on for connection from 
Spark>
-               agent.sinks.spark.channel = memoryChannel
-
-       Also, make sure that the upstream Flume pipeline is configured to send 
the data to the Flume agent running this sink.
-
-See the [Flume's documentation](https://flume.apache.org/documentation.html) 
for more information about
-configuring Flume agents.
-
-#### Configuring Spark Streaming Application
-1. **Linking:** In your SBT/Maven project definition, link your streaming 
application against the `spark-streaming-flume_{{site.SCALA_BINARY_VERSION}}` 
(see [Linking section](streaming-programming-guide.html#linking) in the main 
programming guide).
-
-2. **Programming:** In the streaming application code, import `FlumeUtils` and 
create input DStream as follows.
-
-       <div class="codetabs">
-       <div data-lang="scala" markdown="1">
-               import org.apache.spark.streaming.flume._
-
-               val flumeStream = 
FlumeUtils.createPollingStream(streamingContext, [sink machine hostname], [sink 
port])
-       </div>
-       <div data-lang="java" markdown="1">
-               import org.apache.spark.streaming.flume.*;
-
-               JavaReceiverInputDStream<SparkFlumeEvent>flumeStream =
-                       FlumeUtils.createPollingStream(streamingContext, [sink 
machine hostname], [sink port]);
-       </div>
-       <div data-lang="python" markdown="1">
-               from pyspark.streaming.flume import FlumeUtils
-
-               addresses = [([sink machine hostname 1], [sink port 1]), ([sink 
machine hostname 2], [sink port 2])]
-               flumeStream = FlumeUtils.createPollingStream(streamingContext, 
addresses)
-
-       By default, the Python API will decode Flume event body as UTF8 encoded 
strings. You can specify your custom decoding function to decode the body byte 
arrays in Flume events to any arbitrary data type.
-       See the [API 
docs](api/python/pyspark.streaming.html#pyspark.streaming.flume.FlumeUtils).
-       </div>
-       </div>
-
-       Note that each input DStream can be configured to receive data from 
multiple sinks.
-
-3. **Deploying:** This is same as the first approach.
-
-
-

http://git-wip-us.apache.org/repos/asf/spark/blob/a0018141/docs/streaming-programming-guide.md
----------------------------------------------------------------------
diff --git a/docs/streaming-programming-guide.md 
b/docs/streaming-programming-guide.md
index 0ca0f2a..1103d5c 100644
--- a/docs/streaming-programming-guide.md
+++ b/docs/streaming-programming-guide.md
@@ -11,7 +11,7 @@ description: Spark Streaming programming guide and tutorial 
for Spark SPARK_VERS
 # Overview
 Spark Streaming is an extension of the core Spark API that enables scalable, 
high-throughput,
 fault-tolerant stream processing of live data streams. Data can be ingested 
from many sources
-like Kafka, Flume, Kinesis, or TCP sockets, and can be processed using complex
+like Kafka, Kinesis, or TCP sockets, and can be processed using complex
 algorithms expressed with high-level functions like `map`, `reduce`, `join` 
and `window`.
 Finally, processed data can be pushed out to filesystems, databases,
 and live dashboards. In fact, you can apply Spark's
@@ -40,7 +40,7 @@ stream of results in batches.
 
 Spark Streaming provides a high-level abstraction called *discretized stream* 
or *DStream*,
 which represents a continuous stream of data. DStreams can be created either 
from input data
-streams from sources such as Kafka, Flume, and Kinesis, or by applying 
high-level
+streams from sources such as Kafka, and Kinesis, or by applying high-level
 operations on other DStreams. Internally, a DStream is represented as a 
sequence of
 [RDDs](api/scala/index.html#org.apache.spark.rdd.RDD).
 
@@ -393,7 +393,7 @@ Similar to Spark, Spark Streaming is available through 
Maven Central. To write y
 </div>
 </div>
 
-For ingesting data from sources like Kafka, Flume, and Kinesis that are not 
present in the Spark
+For ingesting data from sources like Kafka and Kinesis that are not present in 
the Spark
 Streaming core
  API, you will have to add the corresponding
 artifact `spark-streaming-xyz_{{site.SCALA_BINARY_VERSION}}` to the 
dependencies. For example,
@@ -402,7 +402,6 @@ some of the common ones are as follows.
 <table class="table">
 <tr><th>Source</th><th>Artifact</th></tr>
 <tr><td> Kafka </td><td> 
spark-streaming-kafka-0-10_{{site.SCALA_BINARY_VERSION}} </td></tr>
-<tr><td> Flume </td><td> spark-streaming-flume_{{site.SCALA_BINARY_VERSION}} 
</td></tr>
 <tr><td> 
Kinesis<br/></td><td>spark-streaming-kinesis-asl_{{site.SCALA_BINARY_VERSION}} 
[Amazon Software License] </td></tr>
 <tr><td></td><td></td></tr>
 </table>
@@ -577,7 +576,7 @@ Spark Streaming provides two categories of built-in 
streaming sources.
 
 - *Basic sources*: Sources directly available in the StreamingContext API.
   Examples: file systems, and socket connections.
-- *Advanced sources*: Sources like Kafka, Flume, Kinesis, etc. are available 
through
+- *Advanced sources*: Sources like Kafka, Kinesis, etc. are available through
   extra utility classes. These require linking against extra dependencies as 
discussed in the
   [linking](#linking) section.
 
@@ -597,7 +596,7 @@ as well as to run the receiver(s).
 
 - When running a Spark Streaming program locally, do not use "local" or 
"local[1]" as the master URL.
   Either of these means that only one thread will be used for running tasks 
locally. If you are using
-  an input DStream based on a receiver (e.g. sockets, Kafka, Flume, etc.), 
then the single thread will
+  an input DStream based on a receiver (e.g. sockets, Kafka, etc.), then the 
single thread will
   be used to run the receiver, leaving no thread for processing the received 
data. Hence, when
   running locally, always use "local[*n*]" as the master URL, where *n* > 
number of receivers to run
   (see [Spark Properties](configuration.html#spark-properties) for information 
on how to set
@@ -732,10 +731,10 @@ for Java, and 
[StreamingContext](api/python/pyspark.streaming.html#pyspark.strea
 {:.no_toc}
 
 <span class="badge" style="background-color: grey">Python API</span> As of 
Spark {{site.SPARK_VERSION_SHORT}},
-out of these sources, Kafka, Kinesis and Flume are available in the Python API.
+out of these sources, Kafka and Kinesis are available in the Python API.
 
 This category of sources require interfacing with external non-Spark 
libraries, some of them with
-complex dependencies (e.g., Kafka and Flume). Hence, to minimize issues 
related to version conflicts
+complex dependencies (e.g., Kafka). Hence, to minimize issues related to 
version conflicts
 of dependencies, the functionality to create DStreams from these sources has 
been moved to separate
 libraries that can be [linked](#linking) to explicitly when necessary.
 
@@ -748,8 +747,6 @@ Some of these advanced sources are as follows.
 
 - **Kafka:** Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with 
Kafka broker versions 0.8.2.1 or higher. See the [Kafka Integration 
Guide](streaming-kafka-integration.html) for more details.
 
-- **Flume:** Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with 
Flume 1.6.0. See the [Flume Integration 
Guide](streaming-flume-integration.html) for more details.
-
 - **Kinesis:** Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with 
Kinesis Client Library 1.2.1. See the [Kinesis Integration 
Guide](streaming-kinesis-integration.html) for more details.
 
 ### Custom Sources
@@ -766,7 +763,7 @@ Guide](streaming-custom-receivers.html) for details.
 {:.no_toc}
 
 There can be two kinds of data sources based on their *reliability*. Sources
-(like Kafka and Flume) allow the transferred data to be acknowledged. If the 
system receiving
+(like Kafka) allow the transferred data to be acknowledged. If the system 
receiving
 data from these *reliable* sources acknowledges the received data correctly, 
it can be ensured
 that no data will be lost due to any kind of failure. This leads to two kinds 
of receivers:
 
@@ -1603,7 +1600,7 @@ operations on the same data). For window-based operations 
like `reduceByWindow`
 Hence, DStreams generated by window-based operations are automatically 
persisted in memory, without
 the developer calling `persist()`.
 
-For input streams that receive data over the network (such as, Kafka, Flume, 
sockets, etc.), the
+For input streams that receive data over the network (such as, Kafka, sockets, 
etc.), the
 default persistence level is set to replicate the data to two nodes for 
fault-tolerance.
 
 Note that, unlike RDDs, the default persistence level of DStreams keeps the 
data serialized in
@@ -1973,7 +1970,7 @@ To run a Spark Streaming applications, you need to have 
the following.
 - *Package the application JAR* - You have to compile your streaming 
application into a JAR.
   If you are using [`spark-submit`](submitting-applications.html) to start the
   application, then you will not need to provide Spark and Spark Streaming in 
the JAR. However,
-  if your application uses [advanced sources](#advanced-sources) (e.g. Kafka, 
Flume),
+  if your application uses [advanced sources](#advanced-sources) (e.g. Kafka),
   then you will have to package the extra artifact they link to, along with 
their dependencies,
   in the JAR that is used to deploy the application. For example, an 
application using `KafkaUtils`
   will have to include 
`spark-streaming-kafka-0-10_{{site.SCALA_BINARY_VERSION}}` and all its
@@ -2060,7 +2057,7 @@ for graceful shutdown options) which ensure data that has 
been received is compl
 processed before shutdown. Then the
 upgraded application can be started, which will start processing from the same 
point where the earlier
 application left off. Note that this can be done only with input sources that 
support source-side buffering
-(like Kafka, and Flume) as data needs to be buffered while the previous 
application was down and
+(like Kafka) as data needs to be buffered while the previous application was 
down and
 the upgraded application is not yet up. And restarting from earlier checkpoint
 information of pre-upgrade code cannot be done. The checkpoint information 
essentially
 contains serialized Scala/Java/Python objects and trying to deserialize 
objects with new,
@@ -2115,7 +2112,7 @@ highlights some of the most important ones.
 
 ### Level of Parallelism in Data Receiving
 {:.no_toc}
-Receiving data over the network (like Kafka, Flume, socket, etc.) requires the 
data to be deserialized
+Receiving data over the network (like Kafka, socket, etc.) requires the data 
to be deserialized
 and stored in Spark. If the data receiving becomes a bottleneck in the system, 
then consider
 parallelizing the data receiving. Note that each input DStream
 creates a single receiver (running on a worker machine) that receives a single 
stream of data.
@@ -2475,14 +2472,12 @@ additional effort may be necessary to achieve 
exactly-once semantics. There are
     * 
[StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext)
 and
   [DStream](api/scala/index.html#org.apache.spark.streaming.dstream.DStream)
     * 
[KafkaUtils](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$),
-    
[FlumeUtils](api/scala/index.html#org.apache.spark.streaming.flume.FlumeUtils$),
     
[KinesisUtils](api/scala/index.html#org.apache.spark.streaming.kinesis.KinesisUtils$),
   - Java docs
     * 
[JavaStreamingContext](api/java/index.html?org/apache/spark/streaming/api/java/JavaStreamingContext.html),
     
[JavaDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaDStream.html)
 and
     
[JavaPairDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaPairDStream.html)
     * 
[KafkaUtils](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html),
-    
[FlumeUtils](api/java/index.html?org/apache/spark/streaming/flume/FlumeUtils.html),
     
[KinesisUtils](api/java/index.html?org/apache/spark/streaming/kinesis/KinesisUtils.html)
   - Python docs
     * 
[StreamingContext](api/python/pyspark.streaming.html#pyspark.streaming.StreamingContext)
 and [DStream](api/python/pyspark.streaming.html#pyspark.streaming.DStream)

http://git-wip-us.apache.org/repos/asf/spark/blob/a0018141/examples/src/main/python/streaming/flume_wordcount.py
----------------------------------------------------------------------
diff --git a/examples/src/main/python/streaming/flume_wordcount.py 
b/examples/src/main/python/streaming/flume_wordcount.py
deleted file mode 100644
index c8ea92b..0000000
--- a/examples/src/main/python/streaming/flume_wordcount.py
+++ /dev/null
@@ -1,56 +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.
-#
-
-r"""
- Counts words in UTF8 encoded, '\n' delimited text received from the network 
every second.
- Usage: flume_wordcount.py <hostname> <port>
-
- To run this on your local machine, you need to setup Flume first, see
- https://flume.apache.org/documentation.html
-
- and then run the example
-    `$ bin/spark-submit --jars \
-      
external/flume-assembly/target/scala-*/spark-streaming-flume-assembly-*.jar \
-      examples/src/main/python/streaming/flume_wordcount.py \
-      localhost 12345
-"""
-from __future__ import print_function
-
-import sys
-
-from pyspark import SparkContext
-from pyspark.streaming import StreamingContext
-from pyspark.streaming.flume import FlumeUtils
-
-if __name__ == "__main__":
-    if len(sys.argv) != 3:
-        print("Usage: flume_wordcount.py <hostname> <port>", file=sys.stderr)
-        sys.exit(-1)
-
-    sc = SparkContext(appName="PythonStreamingFlumeWordCount")
-    ssc = StreamingContext(sc, 1)
-
-    hostname, port = sys.argv[1:]
-    kvs = FlumeUtils.createStream(ssc, hostname, int(port))
-    lines = kvs.map(lambda x: x[1])
-    counts = lines.flatMap(lambda line: line.split(" ")) \
-        .map(lambda word: (word, 1)) \
-        .reduceByKey(lambda a, b: a+b)
-    counts.pprint()
-
-    ssc.start()
-    ssc.awaitTermination()

http://git-wip-us.apache.org/repos/asf/spark/blob/a0018141/external/flume-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml
deleted file mode 100644
index 002bd6f..0000000
--- a/external/flume-assembly/pom.xml
+++ /dev/null
@@ -1,167 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  ~ 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.
-  -->
-
-<project xmlns="http://maven.apache.org/POM/4.0.0"; 
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"; 
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd";>
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <groupId>org.apache.spark</groupId>
-    <artifactId>spark-parent_2.11</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
-    <relativePath>../../pom.xml</relativePath>
-  </parent>
-
-  <artifactId>spark-streaming-flume-assembly_2.11</artifactId>
-  <packaging>jar</packaging>
-  <name>Spark Project External Flume Assembly</name>
-  <url>http://spark.apache.org/</url>
-
-  <properties>
-    <hadoop.deps.scope>provided</hadoop.deps.scope>
-    <sbt.project.name>streaming-flume-assembly</sbt.project.name>
-  </properties>
-
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-streaming-flume_${scala.binary.version}</artifactId>
-      <version>${project.version}</version>
-      <exclusions>
-        <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
-          <artifactId>jetty</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.mortbay.jetty</groupId>
-          <artifactId>jetty-util</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-streaming_${scala.binary.version}</artifactId>
-      <version>${project.version}</version>
-      <scope>provided</scope>
-    </dependency>
-    <!--
-      Demote already included in the Spark assembly. These are transitive 
dependencies of flume
-      or spark-streaming-flume, and this need to be explicitly included even 
through the parent
-      pom may declare them with ${hadoop.deps.scope}.
-    -->
-    <dependency>
-      <groupId>commons-codec</groupId>
-      <artifactId>commons-codec</artifactId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>commons-lang</groupId>
-      <artifactId>commons-lang</artifactId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>commons-net</groupId>
-      <artifactId>commons-net</artifactId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>com.google.protobuf</groupId>
-      <artifactId>protobuf-java</artifactId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.avro</groupId>
-      <artifactId>avro</artifactId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.avro</groupId>
-      <artifactId>avro-ipc</artifactId>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.avro</groupId>
-      <artifactId>avro-mapred</artifactId>
-      <classifier>${avro.mapred.classifier}</classifier>
-      <scope>provided</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.scala-lang</groupId>
-      <artifactId>scala-library</artifactId>
-      <scope>provided</scope>
-    </dependency>
-  </dependencies>
-
-  <build>
-    
<outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
-    
<testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-shade-plugin</artifactId>
-        <configuration>
-          <shadedArtifactAttached>false</shadedArtifactAttached>
-          <artifactSet>
-            <includes>
-              <include>*:*</include>
-            </includes>
-          </artifactSet>
-          <filters>
-            <filter>
-              <artifact>*:*</artifact>
-              <excludes>
-                <exclude>META-INF/*.SF</exclude>
-                <exclude>META-INF/*.DSA</exclude>
-                <exclude>META-INF/*.RSA</exclude>
-              </excludes>
-            </filter>
-          </filters>
-        </configuration>
-        <executions>
-          <execution>
-            <phase>package</phase>
-            <goals>
-              <goal>shade</goal>
-            </goals>
-            <configuration>
-              <transformers>
-                <transformer 
implementation="org.apache.maven.plugins.shade.resource.ServicesResourceTransformer"/>
-                <transformer 
implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
-                  <resource>reference.conf</resource>
-                </transformer>
-                <transformer 
implementation="org.apache.maven.plugins.shade.resource.DontIncludeResourceTransformer">
-                  <resource>log4j.properties</resource>
-                </transformer>
-                <transformer 
implementation="org.apache.maven.plugins.shade.resource.ApacheLicenseResourceTransformer"/>
-                <transformer 
implementation="org.apache.maven.plugins.shade.resource.ApacheNoticeResourceTransformer"/>
-              </transformers>
-            </configuration>
-          </execution>
-        </executions>
-      </plugin>
-    </plugins>
-  </build>
-
-  <profiles>
-    <profile>
-      <id>flume-provided</id>
-      <properties>
-        <flume.deps.scope>provided</flume.deps.scope>
-      </properties>
-    </profile>
-  </profiles>
-</project>
-

http://git-wip-us.apache.org/repos/asf/spark/blob/a0018141/external/flume-sink/pom.xml
----------------------------------------------------------------------
diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml
deleted file mode 100644
index 168d9d3..0000000
--- a/external/flume-sink/pom.xml
+++ /dev/null
@@ -1,140 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  ~ 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.
-  -->
-
-<project xmlns="http://maven.apache.org/POM/4.0.0"; 
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"; 
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 
http://maven.apache.org/xsd/maven-4.0.0.xsd";>
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <groupId>org.apache.spark</groupId>
-    <artifactId>spark-parent_2.11</artifactId>
-    <version>3.0.0-SNAPSHOT</version>
-    <relativePath>../../pom.xml</relativePath>
-  </parent>
-
-  <artifactId>spark-streaming-flume-sink_2.11</artifactId>
-  <properties>
-    <sbt.project.name>streaming-flume-sink</sbt.project.name>
-  </properties>
-  <packaging>jar</packaging>
-  <name>Spark Project External Flume Sink</name>
-  <url>http://spark.apache.org/</url>
-
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.flume</groupId>
-      <artifactId>flume-ng-sdk</artifactId>
-      <exclusions>
-        <!-- Guava is excluded to avoid its use in this module. -->
-        <exclusion>
-          <groupId>com.google.guava</groupId>
-          <artifactId>guava</artifactId>
-        </exclusion>
-        <!--
-          Exclude libthrift since the flume poms seem to confuse sbt, which 
fails to find the
-          dependency.
-        -->
-        <exclusion>
-          <groupId>org.apache.thrift</groupId>
-          <artifactId>libthrift</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.flume</groupId>
-      <artifactId>flume-ng-core</artifactId>
-      <exclusions>
-        <exclusion>
-          <groupId>com.google.guava</groupId>
-          <artifactId>guava</artifactId>
-        </exclusion>
-        <exclusion>
-          <groupId>org.apache.thrift</groupId>
-          <artifactId>libthrift</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>org.scala-lang</groupId>
-      <artifactId>scala-library</artifactId>
-    </dependency>
-    <dependency>
-      <!-- Add Guava in test scope since flume actually needs it. -->
-      <groupId>com.google.guava</groupId>
-      <artifactId>guava</artifactId>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <!--
-        Netty explicitly added in test as it has been excluded from
-        Flume dependency (to avoid runtime problems when running with
-        Spark) but unit tests need it. Version of Netty on which
-        Flume 1.4.0 depends on is "3.4.0.Final" .
-      -->
-      <groupId>io.netty</groupId>
-      <artifactId>netty</artifactId>
-      <version>3.4.0.Final</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-tags_${scala.binary.version}</artifactId>
-    </dependency>
-
-    <!--
-      This spark-tags test-dep is needed even though it isn't used in this 
module, otherwise testing-cmds that exclude
-      them will yield errors.
-    -->
-    <dependency>
-      <groupId>org.apache.spark</groupId>
-      <artifactId>spark-tags_${scala.binary.version}</artifactId>
-      <type>test-jar</type>
-      <scope>test</scope>
-    </dependency>
-
-  </dependencies>
-  <build>
-    
<outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
-    
<testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.avro</groupId>
-        <artifactId>avro-maven-plugin</artifactId>
-        <version>${avro.version}</version>
-        <configuration>
-          <!-- Generate the output in the same directory as the 
sbt-avro-plugin -->
-          
<outputDirectory>${project.basedir}/target/scala-${scala.binary.version}/src_managed/main/compiled_avro</outputDirectory>
-        </configuration>
-        <executions>
-          <execution>
-            <phase>generate-sources</phase>
-            <goals>
-              <goal>idl-protocol</goal>
-            </goals>
-          </execution>
-        </executions>
-      </plugin>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-shade-plugin</artifactId>
-        <configuration>
-          <!-- Disable all relocations defined in the parent pom. -->
-          <relocations combine.self="override" />
-        </configuration>
-      </plugin>
-    </plugins>
-  </build>
-</project>

http://git-wip-us.apache.org/repos/asf/spark/blob/a0018141/external/flume-sink/src/main/avro/sparkflume.avdl
----------------------------------------------------------------------
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 8806e86..0000000
--- 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<string> headers;
-    bytes body;
-  }
-
-  record EventBatch {
-    string errorMsg = ""; // If this is empty it is a valid message, else it 
represents an error
-    string sequenceNumber;
-    array<SparkSinkEvent> events;
-  }
-
-  EventBatch getEventBatch (int n);
-
-  void ack (string sequenceNumber);
-
-  void nack (string sequenceNumber);
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/a0018141/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/Logging.scala
----------------------------------------------------------------------
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 09d3fe9..0000000
--- 
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 
multiple 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
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/a0018141/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala
----------------------------------------------------------------------
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 8050ec3..0000000
--- 
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())
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/a0018141/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSink.scala
----------------------------------------------------------------------
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 e5b63aa..0000000
--- 
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 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
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/a0018141/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkThreadFactory.scala
----------------------------------------------------------------------
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 845fc8d..0000000
--- 
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
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/a0018141/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkSinkUtils.scala
----------------------------------------------------------------------
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 47c0e29..0000000
--- 
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.
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/a0018141/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/TransactionProcessor.scala
----------------------------------------------------------------------
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 19e736f..0000000
--- 
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
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/a0018141/external/flume-sink/src/test/resources/log4j.properties
----------------------------------------------------------------------
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 1e3f163..0000000
--- 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
-


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to