Repository: flume Updated Branches: refs/heads/flume-1.6 832594a29 -> ae2ded9ce
FLUME-2523. Document Kafka channel (Gwen Shapira via Hari) Project: http://git-wip-us.apache.org/repos/asf/flume/repo Commit: http://git-wip-us.apache.org/repos/asf/flume/commit/ae2ded9c Tree: http://git-wip-us.apache.org/repos/asf/flume/tree/ae2ded9c Diff: http://git-wip-us.apache.org/repos/asf/flume/diff/ae2ded9c Branch: refs/heads/flume-1.6 Commit: ae2ded9ce6e112f8a423853e6781b1e02862ab3c Parents: 832594a Author: Hari Shreedharan <[email protected]> Authored: Fri Oct 31 14:27:21 2014 -0700 Committer: Hari Shreedharan <[email protected]> Committed: Fri Oct 31 14:28:02 2014 -0700 ---------------------------------------------------------------------- flume-ng-doc/sphinx/FlumeUserGuide.rst | 54 +++++++++++++++++++++++++++++ 1 file changed, 54 insertions(+) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/flume/blob/ae2ded9c/flume-ng-doc/sphinx/FlumeUserGuide.rst ---------------------------------------------------------------------- diff --git a/flume-ng-doc/sphinx/FlumeUserGuide.rst b/flume-ng-doc/sphinx/FlumeUserGuide.rst index e3aedeb..0ab23fd 100644 --- a/flume-ng-doc/sphinx/FlumeUserGuide.rst +++ b/flume-ng-doc/sphinx/FlumeUserGuide.rst @@ -2305,6 +2305,60 @@ Example for agent named a1: a1.channels = c1 a1.channels.c1.type = jdbc +Kafka Channel +~~~~~~~~~~~~~ + +The events are stored in a Kafka cluster (must be installed separately). Kafka provides high availability and +replication, so in case an agent or a kafka broker crashes, the events are immediately available to other sinks + +The Kafka channel can be used for multiple scenarios: +* With Flume source and sink - it provides a reliable and highly available channel for events +* With Flume source and interceptor but no sink - it allows writing Flume events into a Kafka topic, for use by other apps +* With Flume sink, but no source - it is a low-latency, fault tolerant way to send events from Kafka to Flume sources such as HDFS, HBase or Solr + +Required properties are in **bold**. + +====================== ========================== =============================================================================================================== +Property Name Default Description +====================== ========================== =============================================================================================================== +**type** -- The component type name, needs to be ``org.apache.flume.channel.kafka.KafkaChannel`` +**brokerList** -- List of brokers in the Kafka cluster used by the channel + This can be a partial list of brokers, but we recommend at least two for HA. + The format is comma separated list of hostname:port +**zookeeperConnect** -- URI of ZooKeeper used by Kafka cluster + The format is comma separated list of hostname:port. If chroot is used, it is added once at the end. + For example: zookeeper-1:2181,zookeeper-2:2182,zookeeper-3:2181/kafka +topic flume-channel Kafka topic which the channel will use +groupId flume Consumer group ID the channel uses to register with Kafka. + Multiple channels must use the same topic and group to ensure that when one agent fails another can get the data + Note that having non-channel consumers with the same ID can lead to data loss. +parseAsFlumeEvent true Expecting Avro datums with FlumeEvent schema in the channel. + This should be true if Flume source is writing to the channel + And false if other producers are writing into the topic that the channel is using + Flume source messages to Kafka can be parsed outside of Flume by using + org.apache.flume.source.avro.AvroFlumeEvent provided by the flume-ng-sdk artifact +readSmallestOffset false When set to true, the channel will read all data in the topic, starting from the oldest event + when false, it will read only events written after the channel started + When "parseAsFlumeEvent" is true, this will be false. Flume source will start prior to the sinks and this + guarantees that events sent by source before sinks start will not be lost. +Other Kafka Properties -- These properties are used to configure the Kafka Producer and Consumer used by the channel. + Any property supported by Kafka can be used. + The only requirement is to prepend the property name with the prefix ``kafka.``. + For example: kafka.producer.type +====================== ========================== =============================================================================================================== + +.. note:: Due to the way the channel is load balanced, there may be duplicate events when the agent first starts up + +Example for agent named a1: + +.. code-block:: properties + + a1.channels.channel1.type = org.apache.flume.channel.kafka.KafkaChannel + a1.channels.channel1.capacity = 10000 + a1.channels.channel1.transactionCapacity = 1000 + a1.channels.channel1.brokerList=kafka-2:9092,kafka-3:9092 + a1.channels.channel1.topic=channel1 + a1.channels.channel1.zookeeperConnect=kafka-1:2181 File Channel ~~~~~~~~~~~~
