Spark Streaming + Flume Integration Guide

Apache Flume 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.

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

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 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 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 in the main programming guide for further information).

     groupId = org.apache.spark
     artifactId = spark-streaming-flume_2.11
     version = 2.1.1
    
  2. Programming: In the streaming application code, import FlumeUtils and create input DStream as follows.

     import org.apache.spark.streaming.flume._
    
     val flumeStream = FlumeUtils.createStream(streamingContext, [chosen machine's hostname], [chosen port])
    

    See the API docs and the example.

     import org.apache.spark.streaming.flume.*;
    
     JavaReceiverInputDStream<SparkFlumeEvent> flumeStream =
     	FlumeUtils.createStream(streamingContext, [chosen machine's hostname], [chosen port]);
    

    See the API docs and the example.

     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 and the example.

    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_2.11 and its dependencies into the application JAR. Make sure spark-core_2.11 and spark-streaming_2.11 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 in the main programming guide).

    For Python applications which lack SBT/Maven project management, spark-streaming-flume_2.11 and its dependencies can be directly added to spark-submit using --packages (see Application Submission Guide). That is,

     ./bin/spark-submit --packages org.apache.spark:spark-streaming-flume_2.11:2.1.1 ...
    

    Alternatively, you can also download the JAR of the Maven artifact spark-streaming-flume-assembly from the Maven repository 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.

This ensures stronger reliability and fault-tolerance guarantees 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 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).

     groupId = org.apache.spark
     artifactId = spark-streaming-flume-sink_2.11
     version = 2.1.1
    

    (ii) Scala library JAR: Download the Scala library JAR for Scala 2.11.7. It can be found with the following artifact detail (or, direct link).

     groupId = org.scala-lang
     artifactId = scala-library
     version = 2.11.7
    

    (iii) Commons Lang 3 JAR: Download the Commons Lang 3 JAR. It can be found with the following artifact detail (or, direct link).

     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 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_2.11 (see Linking section in the main programming guide).

  2. Programming: In the streaming application code, import FlumeUtils and create input DStream as follows.

     import org.apache.spark.streaming.flume._
    
     val flumeStream = FlumeUtils.createPollingStream(streamingContext, [sink machine hostname], [sink port])
    
     import org.apache.spark.streaming.flume.*;
    
     JavaReceiverInputDStream<SparkFlumeEvent>flumeStream =
         FlumeUtils.createPollingStream(streamingContext, [sink machine hostname], [sink port]);
    
     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.

    See the Scala example FlumePollingEventCount.

    Note that each input DStream can be configured to receive data from multiple sinks.

  3. Deploying: This is same as the first approach.