Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[STREAMING] SPARK-1729. Make Flume pull data from source, rather than the current pu... #807

Closed
wants to merge 41 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
41 commits
Select commit Hold shift + click to select a range
6d6776a
SPARK-1729. Make Flume pull data from source, rather than the current…
harishreedharan May 9, 2014
d24d9d4
SPARK-1729. Make Flume pull data from source, rather than the current…
harishreedharan May 18, 2014
08176ad
SPARK-1729. Make Flume pull data from source, rather than the current…
harishreedharan May 18, 2014
03d6c1c
SPARK-1729. Make Flume pull data from source, rather than the current…
harishreedharan May 19, 2014
8df37e4
SPARK-1729. Make Flume pull data from source, rather than the current…
harishreedharan May 20, 2014
87775aa
SPARK-1729. Make Flume pull data from source, rather than the current…
harishreedharan May 21, 2014
0f10788
SPARK-1729. Make Flume pull data from source, rather than the current…
harishreedharan May 24, 2014
c604a3c
SPARK-1729. Optimize imports.
harishreedharan Jun 5, 2014
9741683
SPARK-1729. Fixes based on review.
harishreedharan Jun 6, 2014
e7da512
SPARK-1729. Fixing import order
harishreedharan Jun 6, 2014
d6fa3aa
SPARK-1729. New Flume-Spark integration.
harishreedharan Jun 10, 2014
70bcc2a
SPARK-1729. New Flume-Spark integration.
harishreedharan Jun 10, 2014
3c23c18
SPARK-1729. New Spark-Flume integration.
harishreedharan Jun 10, 2014
0d69604
FLUME-1729. Better Flume-Spark integration.
harishreedharan Jun 16, 2014
bda01fc
FLUME-1729. Flume-Spark integration.
harishreedharan Jun 17, 2014
4b0c7fc
FLUME-1729. New Flume-Spark integration.
harishreedharan Jun 18, 2014
205034d
Merging master in
harishreedharan Jun 18, 2014
86aa274
Merge remote-tracking branch 'asf/master'
harishreedharan Jul 10, 2014
8136aa6
Adding TransactionProcessor to map on returning batch of data
harishreedharan Jul 14, 2014
9fd0da7
SPARK-1729. Use foreach instead of map for all Options.
harishreedharan Jul 14, 2014
120e2a1
SPARK-1729. Some test changes and changes to utils classes.
harishreedharan Jul 15, 2014
393bd94
SPARK-1729. Use LinkedBlockingQueue instead of ArrayBuffer to keep tr…
harishreedharan Jul 15, 2014
8c00289
More debug messages
harishreedharan Jul 15, 2014
1edc806
SPARK-1729. Update logging in Spark Sink.
harishreedharan Jul 15, 2014
10b6214
Changed public API, changed sink package, and added java unit test to…
tdas Jul 17, 2014
d248d22
Merge pull request #1 from tdas/flume-polling
Jul 17, 2014
3c5194c
Merge remote-tracking branch 'asf/master'
harishreedharan Jul 19, 2014
799509f
Fix a compile issue.
harishreedharan Jul 21, 2014
3572180
Adding a license header, making Jenkins happy.
harishreedharan Jul 21, 2014
f3c99d1
Merge remote-tracking branch 'asf/master'
harishreedharan Jul 23, 2014
e59cc20
Use SparkFlumeEvent instead of the new type. Also, Flume Polling Rece…
harishreedharan Jul 23, 2014
65b76b4
Fixing the unit test.
harishreedharan Jul 23, 2014
73d6f6d
Cleaned up tests a bit. Added some docs in multiple places.
harishreedharan Jul 24, 2014
1f47364
Minor fixes.
harishreedharan Jul 25, 2014
a082eb3
Merge remote-tracking branch 'asf/master'
harishreedharan Jul 25, 2014
7a1bc6e
Fix SparkBuild.scala
harishreedharan Jul 25, 2014
981bf62
Merge remote-tracking branch 'asf/master'
harishreedharan Jul 28, 2014
5f212ce
Ignore Spark Sink from mima.
harishreedharan Jul 28, 2014
e48d785
Documenting flume-sink being ignored for Mima checks.
harishreedharan Jul 28, 2014
96cfb6f
Merge remote-tracking branch 'asf/master'
harishreedharan Jul 29, 2014
e7f70a3
Merge remote-tracking branch 'asf-git/master'
harishreedharan Jul 29, 2014
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
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.
*/

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
import java.net.InetSocketAddress

/**
* 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()
}
}
100 changes: 100 additions & 0 deletions external/flume-sink/pom.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,100 @@
<?xml version="1.0" encoding="UTF-8"?>
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Pardon for jumping in with comments, but why a new module instead of external/flume?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The sink will be deployed to a Flume agent and not within the spark application. Adding it in external/flume would require that all of the spark dependencies be bundled with the jar, while keeping this module separate (which does not depend on the rest of Spark) allows the user to simply deploy this jar to the Flume plugins directory. In fact, this module does not have any dependencies that Flume already does not pull in by default.

<!--
~ 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</artifactId>
<version>1.1.0-SNAPSHOT</version>
<relativePath>../../pom.xml</relativePath>
</parent>

<artifactId>spark-streaming-flume-sink_2.10</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>
<version>1.4.0</version>
<exclusions>
<exclusion>
<groupId>io.netty</groupId>
<artifactId>netty</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.thrift</groupId>
<artifactId>libthrift</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.apache.flume</groupId>
<artifactId>flume-ng-core</artifactId>
<version>1.4.0</version>
<exclusions>
<exclusion>
<groupId>io.netty</groupId>
<artifactId>netty</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.thrift</groupId>
<artifactId>libthrift</artifactId>
</exclusion>
</exclusions>
</dependency>
<dependency>
<groupId>org.scala-lang</groupId>
<artifactId>scala-library</artifactId>
<version>2.10.4</version>
</dependency>
</dependencies>
<build>
<outputDirectory>target/scala-${scala.binary.version}/classes</outputDirectory>
<testOutputDirectory>target/scala-${scala.binary.version}/test-classes</testOutputDirectory>
<plugins>
<plugin>
<groupId>org.scalatest</groupId>
<artifactId>scalatest-maven-plugin</artifactId>
</plugin>
<plugin>
<groupId>org.apache.avro</groupId>
<artifactId>avro-maven-plugin</artifactId>
<version>1.7.3</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>
</plugins>
</build>
</project>
40 changes: 40 additions & 0 deletions external/flume-sink/src/main/avro/sparkflume.avdl
Original file line number Diff line number Diff line change
@@ -0,0 +1,40 @@
/**
* 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);
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,125 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.streaming.flume.sink

import org.slf4j.{Logger, LoggerFactory}

/**
* Copy of the org.apache.spark.Logging for being used in the Spark Sink.
* The org.apache.spark.Logging is not used so that all of Spark is not brought
* in as a dependency.
*/
private[sink] trait Logging {
// Make the log field transient so that objects with Logging can
// be serialized and used on another machine
@transient private var log_ : Logger = null

// Method to get or create the logger for this object
protected def log: Logger = {
if (log_ == null) {
initializeIfNecessary()
var className = this.getClass.getName
// Ignore trailing $'s in the class names for Scala objects
if (className.endsWith("$")) {
className = className.substring(0, className.length - 1)
}
log_ = LoggerFactory.getLogger(className)
}
log_
}

// Log methods that take only a String
protected def logInfo(msg: => String) {
if (log.isInfoEnabled) log.info(msg)
}

protected def logDebug(msg: => String) {
if (log.isDebugEnabled) log.debug(msg)
}

protected def logTrace(msg: => String) {
if (log.isTraceEnabled) log.trace(msg)
}

protected def logWarning(msg: => String) {
if (log.isWarnEnabled) log.warn(msg)
}

protected def logError(msg: => String) {
if (log.isErrorEnabled) log.error(msg)
}

// Log methods that take Throwables (Exceptions/Errors) too
protected def logInfo(msg: => String, throwable: Throwable) {
if (log.isInfoEnabled) log.info(msg, throwable)
}

protected def logDebug(msg: => String, throwable: Throwable) {
if (log.isDebugEnabled) log.debug(msg, throwable)
}

protected def logTrace(msg: => String, throwable: Throwable) {
if (log.isTraceEnabled) log.trace(msg, throwable)
}

protected def logWarning(msg: => String, throwable: Throwable) {
if (log.isWarnEnabled) log.warn(msg, throwable)
}

protected def logError(msg: => String, throwable: Throwable) {
if (log.isErrorEnabled) log.error(msg, throwable)
}

protected def isTraceEnabled(): Boolean = {
log.isTraceEnabled
}

private def initializeIfNecessary() {
if (!Logging.initialized) {
Logging.initLock.synchronized {
if (!Logging.initialized) {
initializeLogging()
}
}
}
}

private def initializeLogging() {
Logging.initialized = true

// Force a call into slf4j to initialize it. Avoids this happening from mutliple threads
// and triggering this: http://mailman.qos.ch/pipermail/slf4j-dev/2010-April/002956.html
log
}
}

private[sink] object Logging {
@volatile private var initialized = false
val initLock = new Object()
try {
// We use reflection here to handle the case where users remove the
// slf4j-to-jul bridge order to route their logs to JUL.
val bridgeClass = Class.forName("org.slf4j.bridge.SLF4JBridgeHandler")
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
}
}
Loading