Flafka: Apache Flume Meets Apache Kafka for Event Processing

Flafka: Apache Flume Meets Apache Kafka for Event Processing

The new integration between Flume and Kafka offers sub-second-latency event processing without the need for dedicated infrastructure.

In this previous post you learned some Apache Kafka basics and explored a scenario for using Kafka in an online application. This post takes you a step further and highlights the integration of Kafka with Apache Hadoop, demonstrating both a basic ingestion capability as well as how different open-source components can be easily combined to create a near-real time stream processing workflow using Kafka, Apache Flume, and Hadoop.

The Case for Flafka

One key feature of Kafka is its functional simplicity. While there is a lot of sophisticated engineering under the covers, Kafka’s general functionality is relatively straightforward. Part of this simplicity comes from its independence from any other applications (excepting Apache ZooKeeper). As a consequence however, the responsibility is on the developer to write code to either produce or consume messages from Kafka. While there are a number of Kafka clients that support this process, for the most part custom coding is required.

Cloudera engineers and other open source community members have recently committed code for Kafka-Flume integration, informally called “Flafka,” to the Flume project. Flume is a distributed, reliable, and available system for efficiently collecting, aggregating, and moving large amounts of data from many different sources to a centralized data store. Flume provides a tested, production-hardened framework for implementing ingest and real-time processing pipelines. Using the new Flafka source and sink, now available in CDH 5.2, Flume can both read and write messages with Kafka.

Flume can act as a both a consumer (above) and producer for Kafka (below).

Flume-Kafka integration offers the following functionality that Kafka, absent custom coding, does not.

  • Producers – Use Flume sources to write to Kafka
  • Consumers – Write to Flume sinks reading from Kafka
  • A combination of the above
  • In-flight transformations and processing

This functionality expands your ability to utilize all the features of Flume such as bucketing and event modification / routing, Kite SDK Morphline Integration, and NRT indexing with Cloudera Search.

Next, we’ll walk you through an example application using the ingestion of credit-card data as the use case. All example code and configuration info involved are available here. A detailed walkthrough of the setup and example code is in the readme.

Example: Transaction Ingest

Assume that you are ingesting transaction data from a card processing system, and want to pull the transactions directly from Kafka and write them into HDFS.

The record simply contains a UUID for a transaction_id, a dummy credit-card number, timestamp, amount, and store_id for the transaction.

888fc23a-5361-11e4-b76d-22000ada828b|4916177742705110|2014-10-14 01:18:29|67.88|1433
888fdb26-5361-11e4-b76d-22000ada828b|4929011455520|2014-10-14 01:18:29|45.22|886
888ff1e2-5361-11e4-b76d-22000ada828b|4532623020656|2014-10-14 01:18:29|27.14|681
88900c72-5361-11e4-b76d-22000ada828b|4024007162856600|2014-10-14 01:18:29|34.63|577

To import this data directly into HDFS, you could use the following Flume configuration.

# Sources, channels, and sinks are defined per
# agent name, in this case flume1.
flume1.sources  = kafka-source-1
flume1.channels = hdfs-channel-1
flume1.sinks    = hdfs-sink-1

# For each source, channel, and sink, set
# standard properties.
flume1.sources.kafka-source-1.type = org.apache.flume.source.kafka.KafkaSource
flume1.sources.kafka-source-1.zookeeperConnect = flume1.ent.cloudera.com:2181/kafka
flume1.sources.kafka-source-1.topic = flume.txn
flume1.sources.kafka-source-1.batchSize = 100
flume1.sources.kafka-source-1.channels = hdfs-channel-1

flume1.channels.hdfs-channel-1.type   = memory
flume1.sinks.hdfs-sink-1.channel = hdfs-channel-1
flume1.sinks.hdfs-sink-1.type = hdfs
flume1.sinks.hdfs-sink-1.hdfs.writeFormat = Text
flume1.sinks.hdfs-sink-1.hdfs.fileType = DataStream
flume1.sinks.hdfs-sink-1.hdfs.filePrefix = test-events
flume1.sinks.hdfs-sink-1.hdfs.useLocalTimeStamp = true
flume1.sinks.hdfs-sink-1.hdfs.path = /tmp/kafka/%{topic}/%y-%m-%d
flume1.sinks.hdfs-sink-1.hdfs.rollCount=100
flume1.sinks.hdfs-sink-1.hdfs.rollSize=0

# Other properties are specific to each type of
# source, channel, or sink. In this case, we
# specify the capacity of the memory channel.
flume1.channels.hdfs-channel-1.capacity = 10000
flume1.channels.hdfs-channel-1.transactionCapacity = 1000

This configuration defines an agent using the Kafka Source and a standard HDFS sink. Connecting to Kafka from Flume is as simple as setting the topic, ZooKeeper server, and channel. Your generated transactions will be persisted to HDFS with no coding necessary.

The Kafka Source allows for a number of different configuration options.

Property Default Description
type* Must be set to org.apache.flume.source.kafka.KafkaSource
topic* The Kafka topic from which this source reads messages. Flume supports only one topic per source.
zookeeperConnect* The URI of the ZooKeeper server or quorum used by Kafka. This URI can be a single node (for example, zk01.example.com:2181) or a comma-separated list of nodes in a ZooKeeper quorum (for example, zk01.example.com:2181, zk02.example.com:2181, zk03.example.com:2181). If you have created a path in ZooKeeper for storing Kafka data, specify the path in the last entry in the list (for example, zk01.example.com:2181, zk02.example.com:2181, zk03.example.com:2181/kafka). Use the /kafka ZooKeeper path for Cloudera Labs Kafka, because it is created automatically at installation.
batchSize 1000 The maximum number of messages that can be written to a channel in a single batch.
batchDurationMillis 1000 The maximum time (in ms) before a batch is written to the channel. The batch is written when the batchSize limit or batchDurationMillis limit is reached, whichever comes first.
consumer.timeout.ms 10 kafka.consumer.timeout.ms (polling interval for new data for batch)
auto.commit.enabled false If true, periodically commit to ZooKeeper the offset of messages already fetched by the consumer. This committed offset will be used when the process fails as the position from which the new consumer will begin.
groupId flume The unique identifier of the Kafka consumer group. Set the same groupID in all sources to indicate that they belong to the same consumer group.

*Required

Any other properties to pass when creating a Kafka consumer can be accomplished by using the kafka. prefix.

You can declare the batch size can be declared in one of two ways: by specifying the size of the batch in terms of number of events (batchSize), or as a number of milliseconds (batchDurationMillis) to wait while receiving events from Kafka. In this manner, latency-based SLAs can be maintained for lower volume flows.

Note: With any real-time ingestion or processing system there is a tradeoff involved between throughput and single-event processing latency. There is some overhead in processing a batch of events; and so by decreasing the batch size, this overhead is incurred more frequently. Furthermore, events wait until the batch size is attained so per-event latency can suffer. You should experiment with different batch sizes to attain the proper latency and throughput SLAs.

By default, Flume uses the groupId “flume” when reading from Kafka. Adding multiple Flume sources with the same groupId will mean that each Flume agent will get a subset of the messages and can increase throughput. It is best to have any other consumers outside of Flume use a separate groupId so as to avoid message loss.

Example: Event Processing During Ingest

Let’s take our example further and assume that you not only want to use Hadoop for a long-term persistence layer, but also like to build a pipeline for performing arbitrary event processing. Flume provides a key component called the interceptor, part of the Flume extensibility model. Interceptors have the following characteristics; they can

  • Inspect events as they pass between source and channel
  • Modify or drop events as required
  • Be chained together to form a processing pipeline
  • Execute any custom code within the event processing

You can use Flume interceptors to do a variety of processing against incoming events as they pass through the system. In this example, you’ll be calculating a simple “Travel Score” to attempt to identify whether a banking customer is traveling while using their debit card. The exact use case is fabricated, but the architecture can be used to apply virtually any online model or scoring while returning results in sub-second times. Other uses of the interceptor could include:

  • Inspecting the content of the message for proper routing to a particular location such as by geo region
  • Calculating a streaming TopN list
  • Callout to a machine learning serving layer
  • Event enrichment / augmentation
  • In-flight data masking

Thus you can essentially deploy a Hadoop-enabled Kafka consumer group with built-in metrics and manageability via Cloudera Manager—as any Java code, such as a Spring Integration or Apache Camel flow, can be dropped into the interceptor.

(Note: For complex stream processing use cases, Spark Streaming provides the most flexible and feature rich execution engine. Flume Interceptors provide a great way to process events with very low latency and minimal complexity. For per-event response latencies under 50 ms, building a custom application is the right choice.)

To do any meaningful processing of the event as it arrives, you need to enrich the incoming transaction with information from your other systems. For that, call Apache HBase to get additional values related to the transaction and modify the record to reflect the results of the processing performed by Interceptor.

Now you can write your event directly to HDFS as before or back to Kafka, where the event could be picked up by other systems or for more comprehensive stream processing. In this case, you’ll return it directly back to Kafka so that the authorization result can be immediately returned to the client.

The updated Flume configuration looks like this:

# Sources, channels, and sinks are defined per
# agent name, in this case flume1.
flume1.sources  = kafka-source-1
flume1.channels = hdfs-channel-1
flume1.sinks    = kafka-sink-1

# For each source, channel, and sink, set
# standard properties.
flume1.sources.kafka-source-1.type = org.apache.flume.source.kafka.KafkaSource
flume1.sources.kafka-source-1.zookeeperConnect = kafka1.ent.cloudera.com:2181/kafka
flume1.sources.kafka-source-1.topic = flume.txn
flume1.sources.kafka-source-1.batchSize = 5
flume1.sources.kafka-source-1.batchDurationMillis = 200
flume1.sources.kafka-source-1.channels = hdfs-channel-1

flume1.sources.kafka-source-1.interceptors = int-1
flume1.sources.kafka-source-1.interceptors.int-1.type=cloudera.se.fraud.demo.flume.interceptor.FraudEventInterceptor$Builder
flume1.sources.kafka-source-1.interceptors.int-1.threadNum = 200
flume1.channels.hdfs-channel-1.type   = memory

flume1.sinks.kafka-sink-1.channel = hdfs-channel-1
flume1.sinks.kafka-sink-1.type = org.apache.flume.sink.kafka.KafkaSink
flume1.sinks.kafka-sink-1.batchSize = 5
flume1.sinks.kafka-sink-1.brokerList = kafka1.ent.cloudera.com:9092
flume1.sinks.kafka-sink-1.topic = flume.auths

# Other properties are specific to each type of
# source, channel, or sink. In this case, we
# specify the capacity of the memory channel.
flume1.channels.hdfs-channel-1.capacity = 10000
flume1.channels.hdfs-channel-1.transactionCapacity = 10000

Configuring the Flafka sink is as easy as configuring the source, with just a few declarations needed. The interceptor also just needs a few lines for configuration. After configuration is done, place the project jar in the Flume classpath, restart, and the pipeline is ready to go.

Like the source, the sink also supports passing configs to use in the Kafka producer by using the kafka. prefix. The sink supports the following:

Property Default Description
type* Must be set to org.apache.flume.sink.kafka.KafkaSink
brokerList* The brokers the Kafka sink uses to discover topic partitions formatted as a comma-separated list of hostname:port entries. You do not need to specify the entire list of brokers but Cloudera recommends that you specify at least two for HA.
topic default-flume-topic The Kafka topic to which messages are published by default. If the event header contains a topic field the event is published to the designated topic, overriding the configured topic.
batchSize 100 The number of messages to process in a single batch. Specifying a larger batchSize can improve throughput and increase latency.
requiredAcks 1 The number of replicas that must acknowledge a message before it is written successfully. Possible values are 0 (do not wait for an acknowledgement), 1 (wait for the leader to acknowledge only) and -1 (wait for all replicas to acknowledge). To avoid potential loss of data in case of a leader failure, set this to -1.

 

*Required

Furthermore, the sink supports the addition of per-event topic and key headers as set in the interceptor. As mentioned previously, if the source of the message is the Kafka source, the topic header will be set to the topic of the Flume source.

In testing this simple scenario, we were able to achieve sub-150ms latency using one Flume agent, one Kafka partition, and one broker using a small 3-node m2.2xlarge cluster in AWS.

Flume’s Kafka Channel

The recent commit of FLUME-2500 introduces Kafka as a channel in Flume in addition to the traditional file and memory channels. This functionality will be available in CDH 5.3/Flume 1.6, and provides the ability to:

  • Write to Hadoop directly from Kafka without using a source
  • Be used as a reliable and highly available channel for any source/sink combination

The Flume memory channel does not protect against data loss in the event of agent failure, and the when using the file channel, any data in a channel not yet written to a sink will be unavailable until the agent is recovered. The Kafka channel addresses both of these limitations.

Utilizing a Flume source allows you to use interceptors and selectors before writing to Kafka. But the channel can also be utilized in the following way:

Building on our example to instead use the Kafka channel, the configuration might look like this:

# Sources, channels, and sinks are defined per
# agent name, in this case flume1.
flume1.sources  = kafka-source-1
flume1.channels = kafka-channel-1
flume1.sinks    = hdfs-sink-1

# For each source, channel, and sink, set
# standard properties.
flume1.sources.kafka-source-1.type = org.apache.flume.source.kafka.KafkaSource
flume1.sources.kafka-source-1.zookeeperConnect = kafka1.ent.cloudera.com:2181/kafka
flume1.sources.kafka-source-1.topic = flume.txn
flume1.sources.kafka-source-1.batchSize = 5
flume1.sources.kafka-source-1.batchDurationMillis = 200
flume1.sources.kafka-source-1.channels = hdfs-channel-1

flume1.sources.kafka-source-1.interceptors = int-1
flume1.sources.kafka-source-1.interceptors.int-1.type=cloudera.se.fraud.demo.flume.interceptor.FraudEventInterceptor$Builder
flume1.sources.kafka-source-1.interceptors.int-1.threadNum = 200

flume1.channels.kafka-channel-1.type = org.apache.flume.channel.kafka.KafkaChannel
flume1.channels.kafka-channel-1.brokerList = kafka1.ent.cloudera.com:9092
flume1.channels.kafka-channel-1.topic = flume.auths
flume1.channels.kafka-channel-1.zookeeperConnect = kafka1.ent.cloudera.com:2181/kafka

flume1.sinks.hdfs-sink-1.channel = kafka-channel-1
flume1.sinks.hdfs-sink-1.type = hdfs
flume1.sinks.hdfs-sink-1.hdfs.writeFormat = Text
flume1.sinks.hdfs-sink-1.hdfs.fileType = DataStream
flume1.sinks.hdfs-sink-1.hdfs.filePrefix = test-events
flume1.sinks.hdfs-sink-1.hdfs.useLocalTimeStamp = true
flume1.sinks.hdfs-sink-1.hdfs.path = /tmp/kafka/%{topic}/%y-%m-%d
flume1.sinks.hdfs-sink-1.hdfs.rollCount=100
flume1.sinks.hdfs-sink-1.hdfs.rollSize=0

# specify the capacity of the memory channel.
flume1.channels.kafka-channel-1.capacity = 10000
flume1.channels.kafka-channel-1.transactionCapacity = 10000

Using this configuration, your enriched transaction would go directly to Kafka and then on to HDFS using the HDFS sink.

The Kafka channel implements both a Kafka consumer and producer and is configured as follows.

Property Default Description
type* Must be set to org.apache.flume.channel.kafka.KafkaChannel
brokerList* The brokers the Kafka channel uses to discover topic partitions formatted as a comma-separated list of hostname:port entries. You do not need to specify the entire list of brokers but Cloudera recommends that you specify at least two for HA.
zookeeperConnect* The URI of the ZooKeeper server or quorum used by Kafka. This can be a single node (for example, zk01.example.com:2181) or a comma-separated list of nodes in a ZooKeeper quorum (for example, zk01.example.com:2181,zk02.example.com:2181, zk03.example.com:2181). If you have created a path in ZooKeeper for storing Kafka data, specify the path in the last entry in the list (for example, zk01.example.com:2181, zk02.example.com:2181, zk03.example.com:2181/kafka). Use the /kafka ZooKeeper path for Cloudera Labs Kafka, because it is created automatically at installation.
topic flume-channel The Kafka topic the channel will use.
groupId flume Consumer group ID the channel uses to register with Kafka.
parseAsFlumeEvent true This should be true if a Flume source is writing to the channel and will expect AvroDataums with the FlumeEvent schema (org.apache.flume.source.avro.AvroFlumeEvent) in the channel. Should be set to false if other producers are writing into the topic that the channel is using.
readSmallestOffset false If true will read all data in the topic, if false will only read data written after the channel has started. Only relevant when parseAsFlumeEvent is false.
consumer.timeout.ms 100 kafka.consumer.timeout.ms (polling interval when writing to the sink)

*Required

Other properties can be overridden as with the Source and Sink by supplying the kafka. prefix.

When parseAsFlumeEvent is set to true, if other consumers are reading from the channel they will need the FlumeEvent class as mentioned in the table above. The channel in this case serializes the event as an <AvroFlumeEvent>. To provide reliability you should configure multiple agents with the same topic and groupId for the channel so that when an agent fails, other agents can remove data from the channel. The producer mode is always set to sync (required acks -1) and auto.commit.enabled is always overridden to false.

As Kafka sink and Kafka channel provide overlapping functionality, our recommendations are as follows:

  • If you are ingesting from Kafka to Hadoop and need the capabilities of an interceptor or selector, use the Kafka source and file or Kafka channel and standard Flume sink that you require.
  • If you want to ingest directly from Kafka to HDFS, then the Kafka channel by itself is recommended.
  • For writing events to Kafka from either Kafka or other source, the Kafka channel is recommended.
  • If you can’t wait until CDH 5.3/Flume 1.6, the Kafka sink provides this functionality today.

Conclusion

Flafka provides a lot of flexibility in pipeline architecture. The right combination of options will depend on your requirements.

We hope that this post demonstrates the ease of use of Flafka as well as that implementing fairly sophisticated event processing doesn’t necessarily dictate the need for a dedicated stream-processing system when sub-second latencies are required.

Gwen Shapira is a Software Engineer at Cloudera, and a Kafka contributor.

Jeff Holoman is a Systems Engineer at Cloudera.

Gwen Shapira
More by this author
Jeff Holoman
More by this author

2 Comments

by Gautam on

Nice explanation.

by Alka Sharma on

Hi…Came across your article.The blog is quite informative.Found it quite interesting & helpful for anyone who wants to learn Apache Flume in details. This article shows you have immense knowledge on this field which inspires many people to learn this. If you come across anyone willing to take training along with certification guidance ,you can ask him to reach us on Apache Flume Training

Leave a comment

Your email address will not be published. Links are not permitted in comments.