From ed28eb4f017e78665847fc892907c77cf51e7c50 Mon Sep 17 00:00:00 2001 From: Derrick Or Date: Fri, 9 Dec 2016 17:12:04 -0800 Subject: [PATCH] get docs setup with handlebars and seperate streams out to its own page move template variables to central location change file type add apache license info use version variable for javadoc links --- docs/api.html | 158 +-- docs/configuration.html | 474 +++---- docs/connect.html | 594 ++++----- docs/design.html | 1130 ++++++++-------- docs/documentation.html | 2 + docs/implementation.html | 778 +++++------ docs/introduction.html | 386 +++--- docs/js/templateData.js | 21 + docs/ops.html | 2686 +++++++++++++++++++------------------- docs/security.html | 1350 +++++++++---------- docs/streams.html | 737 ++++++----- 11 files changed, 4215 insertions(+), 4101 deletions(-) create mode 100644 docs/js/templateData.js diff --git a/docs/api.html b/docs/api.html index 366814a830d83..20c3642d00e9e 100644 --- a/docs/api.html +++ b/docs/api.html @@ -15,80 +15,84 @@ limitations under the License. --> -Kafka includes four core apis: -
    -
  1. The Producer API allows applications to send streams of data to topics in the Kafka cluster. -
  2. The Consumer API allows applications to read streams of data from topics in the Kafka cluster. -
  3. The Streams API allows transforming streams of data from input topics to output topics. -
  4. The Connect API allows implementing connectors that continually pull from some source system or application into Kafka or push from Kafka into some sink system or application. -
- -Kafka exposes all its functionality over a language independent protocol which has clients available in many programming languages. However only the Java clients are maintained as part of the main Kafka project, the others are available as independent open source projects. A list of non-Java clients is available here. - -

2.1 Producer API

- -The Producer API allows applications to send streams of data to topics in the Kafka cluster. -

-Examples showing how to use the producer are given in the -javadocs. -

-To use the producer, you can use the following maven dependency: - -

-	<dependency>
-	    <groupId>org.apache.kafka</groupId>
-	    <artifactId>kafka-clients</artifactId>
-	    <version>0.10.0.0</version>
-	</dependency>
-
- -

2.2 Consumer API

- -The Consumer API allows applications to read streams of data from topics in the Kafka cluster. -

-Examples showing how to use the consumer are given in the -javadocs. -

-To use the consumer, you can use the following maven dependency: -

-	<dependency>
-	    <groupId>org.apache.kafka</groupId>
-	    <artifactId>kafka-clients</artifactId>
-	    <version>0.10.0.0</version>
-	</dependency>
-
- -

2.3 Streams API

- -The Streams API allows transforming streams of data from input topics to output topics. -

-Examples showing how to use this library are given in the -javadocs -

-Additional documentation on using the Streams API is available here. -

-To use Kafka Streams you can use the following maven dependency: - -

-	<dependency>
-	    <groupId>org.apache.kafka</groupId>
-	    <artifactId>kafka-streams</artifactId>
-	    <version>0.10.0.0</version>
-	</dependency>
-
- -

2.4 Connect API

- -The Connect API allows implementing connectors that continually pull from some source data system into Kafka or push from Kafka into some sink data system. -

-Many users of Connect won't need to use this API directly, though, they can use pre-built connectors without needing to write any code. Additional information on using Connect is available here. -

-Those who want to implement custom connectors can see the javadoc. -

- -

2.5 Legacy APIs

- -

-A more limited legacy producer and consumer api is also included in Kafka. These old Scala APIs are deprecated and only still available for compatibility purposes. Information on them can be found here -here. -

+ + +
diff --git a/docs/configuration.html b/docs/configuration.html index 53343fa02ee0a..2cad283c4281f 100644 --- a/docs/configuration.html +++ b/docs/configuration.html @@ -15,239 +15,243 @@ limitations under the License. --> -Kafka uses key-value pairs in the property file format for configuration. These values can be supplied either from a file or programmatically. - -

3.1 Broker Configs

- -The essential configurations are the following: - - -Topic-level configurations and defaults are discussed in more detail below. - - - -

More details about broker configuration can be found in the scala class kafka.server.KafkaConfig.

- -Topic-level configuration - -Configurations pertinent to topics have both a server default as well an optional per-topic override. If no per-topic configuration is given the server default is used. The override can be set at topic creation time by giving one or more --config options. This example creates a topic named my-topic with a custom max message size and flush rate: -
- > bin/kafka-topics.sh --zookeeper localhost:2181 --create --topic my-topic --partitions 1
-        --replication-factor 1 --config max.message.bytes=64000 --config flush.messages=1
-
-Overrides can also be changed or set later using the alter configs command. This example updates the max message size for my-topic: -
- > bin/kafka-configs.sh --zookeeper localhost:2181 --entity-type topics --entity-name my-topic --alter --add-config max.message.bytes=128000
-
- -To check overrides set on the topic you can do -
- > bin/kafka-configs.sh --zookeeper localhost:2181 --entity-type topics --entity-name my-topic --describe
-
- -To remove an override you can do -
- > bin/kafka-configs.sh --zookeeper localhost:2181  --entity-type topics --entity-name my-topic --alter --delete-config max.message.bytes
-
- -The following are the topic-level configurations. The server's default configuration for this property is given under the Server Default Property heading. A given server default config value only applies to a topic if it does not have an explicit topic config override. - - - -

3.2 Producer Configs

- -Below is the configuration of the Java producer: - - -

- For those interested in the legacy Scala producer configs, information can be found - here. -

- -

3.3 Consumer Configs

- -In 0.9.0.0 we introduced the new Java consumer as a replacement for the older Scala-based simple and high-level consumers. -The configs for both new and old consumers are described below. - -

3.3.1 New Consumer Configs

-Below is the configuration for the new consumer: - - -

3.3.2 Old Consumer Configs

- -The essential old consumer configurations are the following: - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
PropertyDefaultDescription
group.idA string that uniquely identifies the group of consumer processes to which this consumer belongs. By setting the same group id multiple processes indicate that they are all part of the same consumer group.
zookeeper.connectSpecifies the ZooKeeper connection string in the form hostname:port where host and port are the host and port of a ZooKeeper server. To allow connecting through other ZooKeeper nodes when that ZooKeeper machine is down you can also specify multiple hosts in the form hostname1:port1,hostname2:port2,hostname3:port3. -

- The server may also have a ZooKeeper chroot path as part of its ZooKeeper connection string which puts its data under some path in the global ZooKeeper namespace. If so the consumer should use the same chroot path in its connection string. For example to give a chroot path of /chroot/path you would give the connection string as hostname1:port1,hostname2:port2,hostname3:port3/chroot/path.

consumer.idnull -

Generated automatically if not set.

-
socket.timeout.ms30 * 1000The socket timeout for network requests. The actual timeout set will be max.fetch.wait + socket.timeout.ms.
socket.receive.buffer.bytes64 * 1024The socket receive buffer for network requests
fetch.message.max.bytes1024 * 1024The number of bytes of messages to attempt to fetch for each topic-partition in each fetch request. These bytes will be read into memory for each partition, so this helps control the memory used by the consumer. The fetch request size must be at least as large as the maximum message size the server allows or else it is possible for the producer to send messages larger than the consumer can fetch.
num.consumer.fetchers1The number fetcher threads used to fetch data.
auto.commit.enabletrueIf 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.
auto.commit.interval.ms60 * 1000The frequency in ms that the consumer offsets are committed to zookeeper.
queued.max.message.chunks2Max number of message chunks buffered for consumption. Each chunk can be up to fetch.message.max.bytes.
rebalance.max.retries4When a new consumer joins a consumer group the set of consumers attempt to "rebalance" the load to assign partitions to each consumer. If the set of consumers changes while this assignment is taking place the rebalance will fail and retry. This setting controls the maximum number of attempts before giving up.
fetch.min.bytes1The minimum amount of data the server should return for a fetch request. If insufficient data is available the request will wait for that much data to accumulate before answering the request.
fetch.wait.max.ms100The maximum amount of time the server will block before answering the fetch request if there isn't sufficient data to immediately satisfy fetch.min.bytes
rebalance.backoff.ms2000Backoff time between retries during rebalance. If not set explicitly, the value in zookeeper.sync.time.ms is used. + + +
diff --git a/docs/connect.html b/docs/connect.html index d3fc7d7a1c500..23e168cae0309 100644 --- a/docs/connect.html +++ b/docs/connect.html @@ -15,411 +15,415 @@ ~ limitations under the License. ~--> -

8.1 Overview

+ + +
\ No newline at end of file diff --git a/docs/design.html b/docs/design.html index 57db13307a95e..21584321ff6a0 100644 --- a/docs/design.html +++ b/docs/design.html @@ -15,569 +15,573 @@ limitations under the License. --> -

4.1 Motivation

-

-We designed Kafka to be able to act as a unified platform for handling all the real-time data feeds a large company might have. To do this we had to think through a fairly broad set of use cases. -

-It would have to have high-throughput to support high volume event streams such as real-time log aggregation. -

-It would need to deal gracefully with large data backlogs to be able to support periodic data loads from offline systems. -

-It also meant the system would have to handle low-latency delivery to handle more traditional messaging use-cases. -

-We wanted to support partitioned, distributed, real-time processing of these feeds to create new, derived feeds. This motivated our partitioning and consumer model. -

-Finally in cases where the stream is fed into other data systems for serving, we knew the system would have to be able to guarantee fault-tolerance in the presence of machine failures. -

-Supporting these uses led us to a design with a number of unique elements, more akin to a database log than a traditional messaging system. We will outline some elements of the design in the following sections. - -

4.2 Persistence

-

Don't fear the filesystem!

-

-Kafka relies heavily on the filesystem for storing and caching messages. There is a general perception that "disks are slow" which makes people skeptical that a persistent structure can offer competitive performance. -In fact disks are both much slower and much faster than people expect depending on how they are used; and a properly designed disk structure can often be as fast as the network. -

-The key fact about disk performance is that the throughput of hard drives has been diverging from the latency of a disk seek for the last decade. As a result the performance of linear writes on a JBOD -configuration with six 7200rpm SATA RAID-5 array is about 600MB/sec but the performance of random writes is only about 100k/sec—a difference of over 6000X. These linear reads and writes are the most -predictable of all usage patterns, and are heavily optimized by the operating system. A modern operating system provides read-ahead and write-behind techniques that prefetch data in large block multiples and -group smaller logical writes into large physical writes. A further discussion of this issue can be found in this ACM Queue article; they actually find that -sequential disk access can in some cases be faster than random memory access! -

-To compensate for this performance divergence, modern operating systems have become increasingly aggressive in their use of main memory for disk caching. A modern OS will happily divert all free memory to -disk caching with little performance penalty when the memory is reclaimed. All disk reads and writes will go through this unified cache. This feature cannot easily be turned off without using direct I/O, so even -if a process maintains an in-process cache of the data, this data will likely be duplicated in OS pagecache, effectively storing everything twice. -

-Furthermore, we are building on top of the JVM, and anyone who has spent any time with Java memory usage knows two things: -

    -
  1. The memory overhead of objects is very high, often doubling the size of the data stored (or worse).
  2. -
  3. Java garbage collection becomes increasingly fiddly and slow as the in-heap data increases.
  4. -
-

-As a result of these factors using the filesystem and relying on pagecache is superior to maintaining an in-memory cache or other structure—we at least double the available cache by having automatic access -to all free memory, and likely double again by storing a compact byte structure rather than individual objects. Doing so will result in a cache of up to 28-30GB on a 32GB machine without GC penalties. -Furthermore, this cache will stay warm even if the service is restarted, whereas the in-process cache will need to be rebuilt in memory (which for a 10GB cache may take 10 minutes) or else it will need to start -with a completely cold cache (which likely means terrible initial performance). This also greatly simplifies the code as all logic for maintaining coherency between the cache and filesystem is now in the OS, -which tends to do so more efficiently and more correctly than one-off in-process attempts. If your disk usage favors linear reads then read-ahead is effectively pre-populating this cache with useful data on each -disk read. -

-This suggests a design which is very simple: rather than maintain as much as possible in-memory and flush it all out to the filesystem in a panic when we run out of space, we invert that. All data is immediately -written to a persistent log on the filesystem without necessarily flushing to disk. In effect this just means that it is transferred into the kernel's pagecache. -

-This style of pagecache-centric design is described in an article on the design of Varnish here (along with a healthy dose of arrogance). - -

Constant Time Suffices

-

-The persistent data structure used in messaging systems are often a per-consumer queue with an associated BTree or other general-purpose random access data structures to maintain metadata about messages. -BTrees are the most versatile data structure available, and make it possible to support a wide variety of transactional and non-transactional semantics in the messaging system. -They do come with a fairly high cost, though: Btree operations are O(log N). Normally O(log N) is considered essentially equivalent to constant time, but this is not true for disk operations. -Disk seeks come at 10 ms a pop, and each disk can do only one seek at a time so parallelism is limited. Hence even a handful of disk seeks leads to very high overhead. -Since storage systems mix very fast cached operations with very slow physical disk operations, the observed performance of tree structures is often superlinear as data increases with fixed cache--i.e. doubling -your data makes things much worse than twice as slow. -

-Intuitively a persistent queue could be built on simple reads and appends to files as is commonly the case with logging solutions. This structure has the advantage that all operations are O(1) and reads do not -block writes or each other. This has obvious performance advantages since the performance is completely decoupled from the data size—one server can now take full advantage of a number of cheap, -low-rotational speed 1+TB SATA drives. Though they have poor seek performance, these drives have acceptable performance for large reads and writes and come at 1/3 the price and 3x the capacity. -

-Having access to virtually unlimited disk space without any performance penalty means that we can provide some features not usually found in a messaging system. For example, in Kafka, instead of attempting to -delete messages as soon as they are consumed, we can retain messages for a relatively long period (say a week). This leads to a great deal of flexibility for consumers, as we will describe. - -

4.3 Efficiency

-

-We have put significant effort into efficiency. One of our primary use cases is handling web activity data, which is very high volume: each page view may generate dozens of writes. Furthermore, we assume each -message published is read by at least one consumer (often many), hence we strive to make consumption as cheap as possible. -

-We have also found, from experience building and running a number of similar systems, that efficiency is a key to effective multi-tenant operations. If the downstream infrastructure service can easily become a -bottleneck due to a small bump in usage by the application, such small changes will often create problems. By being very fast we help ensure that the application will tip-over under load before the infrastructure. -This is particularly important when trying to run a centralized service that supports dozens or hundreds of applications on a centralized cluster as changes in usage patterns are a near-daily occurrence. -

-We discussed disk efficiency in the previous section. Once poor disk access patterns have been eliminated, there are two common causes of inefficiency in this type of system: too many small I/O operations, and -excessive byte copying. -

-The small I/O problem happens both between the client and the server and in the server's own persistent operations. -

-To avoid this, our protocol is built around a "message set" abstraction that naturally groups messages together. This allows network requests to group messages together and amortize the overhead of the network -roundtrip rather than sending a single message at a time. The server in turn appends chunks of messages to its log in one go, and the consumer fetches large linear chunks at a time. -

-This simple optimization produces orders of magnitude speed up. Batching leads to larger network packets, larger sequential disk operations, contiguous memory blocks, and so on, all of which allows Kafka to turn -a bursty stream of random message writes into linear writes that flow to the consumers. -

-The other inefficiency is in byte copying. At low message rates this is not an issue, but under load the impact is significant. To avoid this we employ a standardized binary message format that is shared by the -producer, the broker, and the consumer (so data chunks can be transferred without modification between them). -

-The message log maintained by the broker is itself just a directory of files, each populated by a sequence of message sets that have been written to disk in the same format used by the producer and consumer. -Maintaining this common format allows optimization of the most important operation: network transfer of persistent log chunks. Modern unix operating systems offer a highly optimized code path for transferring data -out of pagecache to a socket; in Linux this is done with the sendfile system call. -

-To understand the impact of sendfile, it is important to understand the common data path for transfer of data from file to socket: -

    -
  1. The operating system reads data from the disk into pagecache in kernel space
  2. -
  3. The application reads the data from kernel space into a user-space buffer
  4. -
  5. The application writes the data back into kernel space into a socket buffer
  6. -
  7. The operating system copies the data from the socket buffer to the NIC buffer where it is sent over the network
  8. -
-

-This is clearly inefficient, there are four copies and two system calls. Using sendfile, this re-copying is avoided by allowing the OS to send the data from pagecache to the network directly. So in this optimized -path, only the final copy to the NIC buffer is needed. -

-We expect a common use case to be multiple consumers on a topic. Using the zero-copy optimization above, data is copied into pagecache exactly once and reused on each consumption instead of being stored in memory -and copied out to kernel space every time it is read. This allows messages to be consumed at a rate that approaches the limit of the network connection. -

-This combination of pagecache and sendfile means that on a Kafka cluster where the consumers are mostly caught up you will see no read activity on the disks whatsoever as they will be serving data entirely from cache. -

-For more background on the sendfile and zero-copy support in Java, see this article. - -

End-to-end Batch Compression

-

-In some cases the bottleneck is actually not CPU or disk but network bandwidth. This is particularly true for a data pipeline that needs to send messages between data centers over a wide-area network. Of course, -the user can always compress its messages one at a time without any support needed from Kafka, but this can lead to very poor compression ratios as much of the redundancy is due to repetition between messages of -the same type (e.g. field names in JSON or user agents in web logs or common string values). Efficient compression requires compressing multiple messages together rather than compressing each message individually. -

-Kafka supports this by allowing recursive message sets. A batch of messages can be clumped together compressed and sent to the server in this form. This batch of messages will be written in compressed form and will -remain compressed in the log and will only be decompressed by the consumer. -

-Kafka supports GZIP, Snappy and LZ4 compression protocols. More details on compression can be found here. - -

4.4 The Producer

- -

Load balancing

-

-The producer sends data directly to the broker that is the leader for the partition without any intervening routing tier. To help the producer do this all Kafka nodes can answer a request for metadata about which -servers are alive and where the leaders for the partitions of a topic are at any given time to allow the producer to appropriately direct its requests. -

-The client controls which partition it publishes messages to. This can be done at random, implementing a kind of random load balancing, or it can be done by some semantic partitioning function. We expose the interface -for semantic partitioning by allowing the user to specify a key to partition by and using this to hash to a partition (there is also an option to override the partition function if need be). For example if the key -chosen was a user id then all data for a given user would be sent to the same partition. This in turn will allow consumers to make locality assumptions about their consumption. This style of partitioning is explicitly -designed to allow locality-sensitive processing in consumers. - -

Asynchronous send

-

-Batching is one of the big drivers of efficiency, and to enable batching the Kafka producer will attempt to accumulate data in memory and to send out larger batches in a single request. The batching can be configured -to accumulate no more than a fixed number of messages and to wait no longer than some fixed latency bound (say 64k or 10 ms). This allows the accumulation of more bytes to send, and few larger I/O operations on the -servers. This buffering is configurable and gives a mechanism to trade off a small amount of additional latency for better throughput. -

-Details on configuration and the api for the producer can be found -elsewhere in the documentation. - -

4.5 The Consumer

- -The Kafka consumer works by issuing "fetch" requests to the brokers leading the partitions it wants to consume. The consumer specifies its offset in the log with each request and receives back a chunk of log -beginning from that position. The consumer thus has significant control over this position and can rewind it to re-consume data if need be. - -

Push vs. pull

-

-An initial question we considered is whether consumers should pull data from brokers or brokers should push data to the consumer. In this respect Kafka follows a more traditional design, shared by most messaging -systems, where data is pushed to the broker from the producer and pulled from the broker by the consumer. Some logging-centric systems, such as Scribe and -Apache Flume, follow a very different push-based path where data is pushed downstream. There are pros and cons to both approaches. However, a push-based system has difficulty -dealing with diverse consumers as the broker controls the rate at which data is transferred. The goal is generally for the consumer to be able to consume at the maximum possible rate; unfortunately, in a push -system this means the consumer tends to be overwhelmed when its rate of consumption falls below the rate of production (a denial of service attack, in essence). A pull-based system has the nicer property that -the consumer simply falls behind and catches up when it can. This can be mitigated with some kind of backoff protocol by which the consumer can indicate it is overwhelmed, but getting the rate of transfer to -fully utilize (but never over-utilize) the consumer is trickier than it seems. Previous attempts at building systems in this fashion led us to go with a more traditional pull model. -

-Another advantage of a pull-based system is that it lends itself to aggressive batching of data sent to the consumer. A push-based system must choose to either send a request immediately or accumulate more data -and then send it later without knowledge of whether the downstream consumer will be able to immediately process it. If tuned for low latency, this will result in sending a single message at a time only for the -transfer to end up being buffered anyway, which is wasteful. A pull-based design fixes this as the consumer always pulls all available messages after its current position in the log (or up to some configurable max -size). So one gets optimal batching without introducing unnecessary latency. -

-The deficiency of a naive pull-based system is that if the broker has no data the consumer may end up polling in a tight loop, effectively busy-waiting for data to arrive. To avoid this we have parameters in our -pull request that allow the consumer request to block in a "long poll" waiting until data arrives (and optionally waiting until a given number of bytes is available to ensure large transfer sizes). -

-You could imagine other possible designs which would be only pull, end-to-end. The producer would locally write to a local log, and brokers would pull from that with consumers pulling from them. A similar type of -"store-and-forward" producer is often proposed. This is intriguing but we felt not very suitable for our target use cases which have thousands of producers. Our experience running persistent data systems at -scale led us to feel that involving thousands of disks in the system across many applications would not actually make things more reliable and would be a nightmare to operate. And in practice we have found that we -can run a pipeline with strong SLAs at large scale without a need for producer persistence. - -

Consumer Position

-Keeping track of what has been consumed is, surprisingly, one of the key performance points of a messaging system. -

-Most messaging systems keep metadata about what messages have been consumed on the broker. That is, as a message is handed out to a consumer, the broker either records that fact locally immediately or it may wait -for acknowledgement from the consumer. This is a fairly intuitive choice, and indeed for a single machine server it is not clear where else this state could go. Since the data structures used for storage in many -messaging systems scale poorly, this is also a pragmatic choice--since the broker knows what is consumed it can immediately delete it, keeping the data size small. -

-What is perhaps not obvious is that getting the broker and consumer to come into agreement about what has been consumed is not a trivial problem. If the broker records a message as consumed immediately every -time it is handed out over the network, then if the consumer fails to process the message (say because it crashes or the request times out or whatever) that message will be lost. To solve this problem, many messaging -systems add an acknowledgement feature which means that messages are only marked as sent not consumed when they are sent; the broker waits for a specific acknowledgement from the consumer to record the -message as consumed. This strategy fixes the problem of losing messages, but creates new problems. First of all, if the consumer processes the message but fails before it can send an acknowledgement then the -message will be consumed twice. The second problem is around performance, now the broker must keep multiple states about every single message (first to lock it so it is not given out a second time, and then to mark -it as permanently consumed so that it can be removed). Tricky problems must be dealt with, like what to do with messages that are sent but never acknowledged. -

-Kafka handles this differently. Our topic is divided into a set of totally ordered partitions, each of which is consumed by exactly one consumer within each subscribing consumer group at any given time. This means -that the position of a consumer in each partition is just a single integer, the offset of the next message to consume. This makes the state about what has been consumed very small, just one number for each partition. -This state can be periodically checkpointed. This makes the equivalent of message acknowledgements very cheap. -

-There is a side benefit of this decision. A consumer can deliberately rewind back to an old offset and re-consume data. This violates the common contract of a queue, but turns out to be an essential feature -for many consumers. For example, if the consumer code has a bug and is discovered after some messages are consumed, the consumer can re-consume those messages once the bug is fixed. - -

Offline Data Load

- -Scalable persistence allows for the possibility of consumers that only periodically consume such as batch data loads that periodically bulk-load data into an offline system such as Hadoop or a relational data -warehouse. -

-In the case of Hadoop we parallelize the data load by splitting the load over individual map tasks, one for each node/topic/partition combination, allowing full parallelism in the loading. Hadoop provides the task -management, and tasks which fail can restart without danger of duplicate data—they simply restart from their original position. - -

4.6 Message Delivery Semantics

-

-Now that we understand a little about how producers and consumers work, let's discuss the semantic guarantees Kafka provides between producer and consumer. Clearly there are multiple possible message delivery -guarantees that could be provided: -

    -
  • - At most once—Messages may be lost but are never redelivered. -
  • -
  • - At least once—Messages are never lost but may be redelivered. -
  • -
  • - Exactly once—this is what people actually want, each message is delivered once and only once. -
  • -
- -It's worth noting that this breaks down into two problems: the durability guarantees for publishing a message and the guarantees when consuming a message. -

-Many systems claim to provide "exactly once" delivery semantics, but it is important to read the fine print, most of these claims are misleading (i.e. they don't translate to the case where consumers or producers -can fail, cases where there are multiple consumer processes, or cases where data written to disk can be lost). -

-Kafka's semantics are straight-forward. When publishing a message we have a notion of the message being "committed" to the log. Once a published message is committed it will not be lost as long as one broker that -replicates the partition to which this message was written remains "alive". The definition of alive as well as a description of which types of failures we attempt to handle will be described in more detail in the -next section. For now let's assume a perfect, lossless broker and try to understand the guarantees to the producer and consumer. If a producer attempts to publish a message and experiences a network error it cannot -be sure if this error happened before or after the message was committed. This is similar to the semantics of inserting into a database table with an autogenerated key. -

-These are not the strongest possible semantics for publishers. Although we cannot be sure of what happened in the case of a network error, it is possible to allow the producer to generate a sort of "primary key" that -makes retrying the produce request idempotent. This feature is not trivial for a replicated system because of course it must work even (or especially) in the case of a server failure. With this feature it would -suffice for the producer to retry until it receives acknowledgement of a successfully committed message at which point we would guarantee the message had been published exactly once. We hope to add this in a future -Kafka version. -

-Not all use cases require such strong guarantees. For uses which are latency sensitive we allow the producer to specify the durability level it desires. If the producer specifies that it wants to wait on the message -being committed this can take on the order of 10 ms. However the producer can also specify that it wants to perform the send completely asynchronously or that it wants to wait only until the leader (but not -necessarily the followers) have the message. -

-Now let's describe the semantics from the point-of-view of the consumer. All replicas have the exact same log with the same offsets. The consumer controls its position in this log. If the consumer never crashed it -could just store this position in memory, but if the consumer fails and we want this topic partition to be taken over by another process the new process will need to choose an appropriate position from which to start -processing. Let's say the consumer reads some messages -- it has several options for processing the messages and updating its position. -

    -
  1. It can read the messages, then save its position in the log, and finally process the messages. In this case there is a possibility that the consumer process crashes after saving its position but before saving - the output of its message processing. In this case the process that took over processing would start at the saved position even though a few messages prior to that position had not been processed. This corresponds - to "at-most-once" semantics as in the case of a consumer failure messages may not be processed. -
  2. It can read the messages, process the messages, and finally save its position. In this case there is a possibility that the consumer process crashes after processing messages but before saving its position. - In this case when the new process takes over the first few messages it receives will already have been processed. This corresponds to the "at-least-once" semantics in the case of consumer failure. In many cases - messages have a primary key and so the updates are idempotent (receiving the same message twice just overwrites a record with another copy of itself). -
  3. So what about exactly once semantics (i.e. the thing you actually want)? The limitation here is not actually a feature of the messaging system but rather the need to co-ordinate the consumer's position with - what is actually stored as output. The classic way of achieving this would be to introduce a two-phase commit between the storage for the consumer position and the storage of the consumers output. But this can be - handled more simply and generally by simply letting the consumer store its offset in the same place as its output. This is better because many of the output systems a consumer might want to write to will not - support a two-phase commit. As an example of this, our Hadoop ETL that populates data in HDFS stores its offsets in HDFS with the data it reads so that it is guaranteed that either data and offsets are both updated - or neither is. We follow similar patterns for many other data systems which require these stronger semantics and for which the messages do not have a primary key to allow for deduplication. -
-

-So effectively Kafka guarantees at-least-once delivery by default and allows the user to implement at most once delivery by disabling retries on the producer and committing its offset prior to processing a batch of -messages. Exactly-once delivery requires co-operation with the destination storage system but Kafka provides the offset which makes implementing this straight-forward. - -

4.7 Replication

-

-Kafka replicates the log for each topic's partitions across a configurable number of servers (you can set this replication factor on a topic-by-topic basis). This allows automatic failover to these replicas when a -server in the cluster fails so messages remain available in the presence of failures. -

-Other messaging systems provide some replication-related features, but, in our (totally biased) opinion, this appears to be a tacked-on thing, not heavily used, and with large downsides: slaves are inactive, -throughput is heavily impacted, it requires fiddly manual configuration, etc. Kafka is meant to be used with replication by default—in fact we implement un-replicated topics as replicated topics where the -replication factor is one. -

-The unit of replication is the topic partition. Under non-failure conditions, each partition in Kafka has a single leader and zero or more followers. The total number of replicas including the leader constitute the -replication factor. All reads and writes go to the leader of the partition. Typically, there are many more partitions than brokers and the leaders are evenly distributed among brokers. The logs on the followers are -identical to the leader's log—all have the same offsets and messages in the same order (though, of course, at any given time the leader may have a few as-yet unreplicated messages at the end of its log). -

-Followers consume messages from the leader just as a normal Kafka consumer would and apply them to their own log. Having the followers pull from the leader has the nice property of allowing the follower to naturally -batch together log entries they are applying to their log. -

-As with most distributed systems automatically handling failures requires having a precise definition of what it means for a node to be "alive". For Kafka node liveness has two conditions -

    -
  1. A node must be able to maintain its session with ZooKeeper (via ZooKeeper's heartbeat mechanism) -
  2. If it is a slave it must replicate the writes happening on the leader and not fall "too far" behind -
-We refer to nodes satisfying these two conditions as being "in sync" to avoid the vagueness of "alive" or "failed". The leader keeps track of the set of "in sync" nodes. If a follower dies, gets stuck, or falls -behind, the leader will remove it from the list of in sync replicas. The determination of stuck and lagging replicas is controlled by the replica.lag.time.max.ms configuration. -

-In distributed systems terminology we only attempt to handle a "fail/recover" model of failures where nodes suddenly cease working and then later recover (perhaps without knowing that they have died). Kafka does not -handle so-called "Byzantine" failures in which nodes produce arbitrary or malicious responses (perhaps due to bugs or foul play). -

-A message is considered "committed" when all in sync replicas for that partition have applied it to their log. Only committed messages are ever given out to the consumer. This means that the consumer need not worry -about potentially seeing a message that could be lost if the leader fails. Producers, on the other hand, have the option of either waiting for the message to be committed or not, depending on their preference for -tradeoff between latency and durability. This preference is controlled by the acks setting that the producer uses. -

-The guarantee that Kafka offers is that a committed message will not be lost, as long as there is at least one in sync replica alive, at all times. -

-Kafka will remain available in the presence of node failures after a short fail-over period, but may not remain available in the presence of network partitions. - -

Replicated Logs: Quorums, ISRs, and State Machines (Oh my!)

- -At its heart a Kafka partition is a replicated log. The replicated log is one of the most basic primitives in distributed data systems, and there are many approaches for implementing one. A replicated log can be -used by other systems as a primitive for implementing other distributed systems in the state-machine style. -

-A replicated log models the process of coming into consensus on the order of a series of values (generally numbering the log entries 0, 1, 2, ...). There are many ways to implement this, but the simplest and fastest -is with a leader who chooses the ordering of values provided to it. As long as the leader remains alive, all followers need to only copy the values and ordering the leader chooses. -

-Of course if leaders didn't fail we wouldn't need followers! When the leader does die we need to choose a new leader from among the followers. But followers themselves may fall behind or crash so we must ensure we -choose an up-to-date follower. The fundamental guarantee a log replication algorithm must provide is that if we tell the client a message is committed, and the leader fails, the new leader we elect must also have -that message. This yields a tradeoff: if the leader waits for more followers to acknowledge a message before declaring it committed then there will be more potentially electable leaders. -

-If you choose the number of acknowledgements required and the number of logs that must be compared to elect a leader such that there is guaranteed to be an overlap, then this is called a Quorum. -

-A common approach to this tradeoff is to use a majority vote for both the commit decision and the leader election. This is not what Kafka does, but let's explore it anyway to understand the tradeoffs. Let's say we -have 2f+1 replicas. If f+1 replicas must receive a message prior to a commit being declared by the leader, and if we elect a new leader by electing the follower with the most complete log from at least -f+1 replicas, then, with no more than f failures, the leader is guaranteed to have all committed messages. This is because among any f+1 replicas, there must be at least one replica that contains -all committed messages. That replica's log will be the most complete and therefore will be selected as the new leader. There are many remaining details that each algorithm must handle (such as precisely defined what -makes a log more complete, ensuring log consistency during leader failure or changing the set of servers in the replica set) but we will ignore these for now. -

-This majority vote approach has a very nice property: the latency is dependent on only the fastest servers. That is, if the replication factor is three, the latency is determined by the faster slave not the slower one. -

-There are a rich variety of algorithms in this family including ZooKeeper's -Zab, -Raft, -and Viewstamped Replication. -The most similar academic publication we are aware of to Kafka's actual implementation is -PacificA from Microsoft. -

-The downside of majority vote is that it doesn't take many failures to leave you with no electable leaders. To tolerate one failure requires three copies of the data, and to tolerate two failures requires five copies -of the data. In our experience having only enough redundancy to tolerate a single failure is not enough for a practical system, but doing every write five times, with 5x the disk space requirements and 1/5th the -throughput, is not very practical for large volume data problems. This is likely why quorum algorithms more commonly appear for shared cluster configuration such as ZooKeeper but are less common for primary data -storage. For example in HDFS the namenode's high-availability feature is built on a majority-vote-based journal, but this more -expensive approach is not used for the data itself. -

-Kafka takes a slightly different approach to choosing its quorum set. Instead of majority vote, Kafka dynamically maintains a set of in-sync replicas (ISR) that are caught-up to the leader. Only members of this set -are eligible for election as leader. A write to a Kafka partition is not considered committed until all in-sync replicas have received the write. This ISR set is persisted to ZooKeeper whenever it changes. -Because of this, any replica in the ISR is eligible to be elected leader. This is an important factor for Kafka's usage model where there are many partitions and ensuring leadership balance is important. -With this ISR model and f+1 replicas, a Kafka topic can tolerate f failures without losing committed messages. -

-For most use cases we hope to handle, we think this tradeoff is a reasonable one. In practice, to tolerate f failures, both the majority vote and the ISR approach will wait for the same number of replicas to -acknowledge before committing a message (e.g. to survive one failure a majority quorum needs three replicas and one acknowledgement and the ISR approach requires two replicas and one acknowledgement). -The ability to commit without the slowest servers is an advantage of the majority vote approach. However, we think it is ameliorated by allowing the client to choose whether they block on the message commit or not, -and the additional throughput and disk space due to the lower required replication factor is worth it. -

-Another important design distinction is that Kafka does not require that crashed nodes recover with all their data intact. It is not uncommon for replication algorithms in this space to depend on the existence of -"stable storage" that cannot be lost in any failure-recovery scenario without potential consistency violations. There are two primary problems with this assumption. First, disk errors are the most common problem we -observe in real operation of persistent data systems and they often do not leave data intact. Secondly, even if this were not a problem, we do not want to require the use of fsync on every write for our consistency -guarantees as this can reduce performance by two to three orders of magnitude. Our protocol for allowing a replica to rejoin the ISR ensures that before rejoining, it must fully re-sync again even if it lost unflushed -data in its crash. - -

Unclean leader election: What if they all die?

- -Note that Kafka's guarantee with respect to data loss is predicated on at least one replica remaining in sync. If all the nodes replicating a partition die, this guarantee no longer holds. -

-However a practical system needs to do something reasonable when all the replicas die. If you are unlucky enough to have this occur, it is important to consider what will happen. There are two behaviors that could be -implemented: -

    -
  1. Wait for a replica in the ISR to come back to life and choose this replica as the leader (hopefully it still has all its data). -
  2. Choose the first replica (not necessarily in the ISR) that comes back to life as the leader. -
-

-This is a simple tradeoff between availability and consistency. If we wait for replicas in the ISR, then we will remain unavailable as long as those replicas are down. If such replicas were destroyed or their data -was lost, then we are permanently down. If, on the other hand, a non-in-sync replica comes back to life and we allow it to become leader, then its log becomes the source of truth even though it is not guaranteed to -have every committed message. By default, Kafka chooses the second strategy and favor choosing a potentially inconsistent replica when all replicas in the ISR are dead. This behavior can be disabled using -configuration property unclean.leader.election.enable, to support use cases where downtime is preferable to inconsistency. -

-This dilemma is not specific to Kafka. It exists in any quorum-based scheme. For example in a majority voting scheme, if a majority of servers suffer a permanent failure, then you must either choose to lose 100% of -your data or violate consistency by taking what remains on an existing server as your new source of truth. - - -

Availability and Durability Guarantees

- -When writing to Kafka, producers can choose whether they wait for the message to be acknowledged by 0,1 or all (-1) replicas. -Note that "acknowledgement by all replicas" does not guarantee that the full set of assigned replicas have received the message. By default, when acks=all, acknowledgement happens as soon as all the current in-sync -replicas have received the message. For example, if a topic is configured with only two replicas and one fails (i.e., only one in sync replica remains), then writes that specify acks=all will succeed. However, these -writes could be lost if the remaining replica also fails. - -Although this ensures maximum availability of the partition, this behavior may be undesirable to some users who prefer durability over availability. Therefore, we provide two topic-level configurations that can be -used to prefer message durability over availability: -
    -
  1. Disable unclean leader election - if all replicas become unavailable, then the partition will remain unavailable until the most recent leader becomes available again. This effectively prefers unavailability - over the risk of message loss. See the previous section on Unclean Leader Election for clarification.
  2. -
  3. Specify a minimum ISR size - the partition will only accept writes if the size of the ISR is above a certain minimum, in order to prevent the loss of messages that were written to just a single replica, - which subsequently becomes unavailable. This setting only takes effect if the producer uses acks=all and guarantees that the message will be acknowledged by at least this many in-sync replicas. -This setting offers a trade-off between consistency and availability. A higher setting for minimum ISR size guarantees better consistency since the message is guaranteed to be written to more replicas which reduces -the probability that it will be lost. However, it reduces availability since the partition will be unavailable for writes if the number of in-sync replicas drops below the minimum threshold.
  4. -
- - -

Replica Management

- -The above discussion on replicated logs really covers only a single log, i.e. one topic partition. However a Kafka cluster will manage hundreds or thousands of these partitions. We attempt to balance partitions -within a cluster in a round-robin fashion to avoid clustering all partitions for high-volume topics on a small number of nodes. Likewise we try to balance leadership so that each node is the leader for a proportional -share of its partitions. -

-It is also important to optimize the leadership election process as that is the critical window of unavailability. A naive implementation of leader election would end up running an election per partition for all -partitions a node hosted when that node failed. Instead, we elect one of the brokers as the "controller". This controller detects failures at the broker level and is responsible for changing the leader of all -affected partitions in a failed broker. The result is that we are able to batch together many of the required leadership change notifications which makes the election process far cheaper and faster for a large number -of partitions. If the controller fails, one of the surviving brokers will become the new controller. - -

4.8 Log Compaction

- -Log compaction ensures that Kafka will always retain at least the last known value for each message key within the log of data for a single topic partition. It addresses use cases and scenarios such as restoring -state after application crashes or system failure, or reloading caches after application restarts during operational maintenance. Let's dive into these use cases in more detail and then describe how compaction works. -

-So far we have described only the simpler approach to data retention where old log data is discarded after a fixed period of time or when the log reaches some predetermined size. This works well for temporal event -data such as logging where each record stands alone. However an important class of data streams are the log of changes to keyed, mutable data (for example, the changes to a database table). -

-Let's discuss a concrete example of such a stream. Say we have a topic containing user email addresses; every time a user updates their email address we send a message to this topic using their user id as the -primary key. Now say we send the following messages over some time period for a user with id 123, each message corresponding to a change in email address (messages for other ids are omitted): -

-    123 => bill@microsoft.com
-            .
-            .
-            .
-    123 => bill@gatesfoundation.org
-            .
-            .
-            .
-    123 => bill@gmail.com
-
-Log compaction gives us a more granular retention mechanism so that we are guaranteed to retain at least the last update for each primary key (e.g. bill@gmail.com). By doing this we guarantee that the -log contains a full snapshot of the final value for every key not just keys that changed recently. This means downstream consumers can restore their own state off this topic without us having to retain a complete -log of all changes. -

-Let's start by looking at a few use cases where this is useful, then we'll see how it can be used. -

    -
  1. Database change subscription. It is often necessary to have a data set in multiple data systems, and often one of these systems is a database of some kind (either a RDBMS or perhaps a new-fangled key-value -store). For example you might have a database, a cache, a search cluster, and a Hadoop cluster. Each change to the database will need to be reflected in the cache, the search cluster, and eventually in Hadoop. -In the case that one is only handling the real-time updates you only need recent log. But if you want to be able to reload the cache or restore a failed search node you may need a complete data set. -
  2. Event sourcing. This is a style of application design which co-locates query processing with application design and uses a log of changes as the primary store for the application. -
  3. Journaling for high-availability. A process that does local computation can be made fault-tolerant by logging out changes that it makes to its local state so another process can reload these changes and -carry on if it should fail. A concrete example of this is handling counts, aggregations, and other "group by"-like processing in a stream query system. Samza, a real-time stream-processing framework, -uses this feature for exactly this purpose. -
-In each of these cases one needs primarily to handle the real-time feed of changes, but occasionally, when a machine crashes or data needs to be re-loaded or re-processed, one needs to do a full load. -Log compaction allows feeding both of these use cases off the same backing topic. - -This style of usage of a log is described in more detail in this blog post. -

-The general idea is quite simple. If we had infinite log retention, and we logged each change in the above cases, then we would have captured the state of the system at each time from when it first began. -Using this complete log, we could restore to any point in time by replaying the first N records in the log. This hypothetical complete log is not very practical for systems that update a single record many times -as the log will grow without bound even for a stable dataset. The simple log retention mechanism which throws away old updates will bound space but the log is no longer a way to restore the current state—now -restoring from the beginning of the log no longer recreates the current state as old updates may not be captured at all. -

-Log compaction is a mechanism to give finer-grained per-record retention, rather than the coarser-grained time-based retention. The idea is to selectively remove records where we have a more recent update with the -same primary key. This way the log is guaranteed to have at least the last state for each key. -

-This retention policy can be set per-topic, so a single cluster can have some topics where retention is enforced by size or time and other topics where retention is enforced by compaction. -

-This functionality is inspired by one of LinkedIn's oldest and most successful pieces of infrastructure—a database changelog caching service called Databus. -Unlike most log-structured storage systems Kafka is built for subscription and organizes data for fast linear reads and writes. Unlike Databus, Kafka acts as a source-of-truth store so it is useful even in -situations where the upstream data source would not otherwise be replayable. - -

Log Compaction Basics

- -Here is a high-level picture that shows the logical structure of a Kafka log with the offset for each message. -

- -

-The head of the log is identical to a traditional Kafka log. It has dense, sequential offsets and retains all messages. Log compaction adds an option for handling the tail of the log. The picture above shows a log -with a compacted tail. Note that the messages in the tail of the log retain the original offset assigned when they were first written—that never changes. Note also that all offsets remain valid positions in -the log, even if the message with that offset has been compacted away; in this case this position is indistinguishable from the next highest offset that does appear in the log. For example, in the picture above the -offsets 36, 37, and 38 are all equivalent positions and a read beginning at any of these offsets would return a message set beginning with 38. -

-Compaction also allows for deletes. A message with a key and a null payload will be treated as a delete from the log. This delete marker will cause any prior message with that key to be removed (as would any new -message with that key), but delete markers are special in that they will themselves be cleaned out of the log after a period of time to free up space. The point in time at which deletes are no longer retained is -marked as the "delete retention point" in the above diagram. -

-The compaction is done in the background by periodically recopying log segments. Cleaning does not block reads and can be throttled to use no more than a configurable amount of I/O throughput to avoid impacting -producers and consumers. The actual process of compacting a log segment looks something like this: -

- -

-

What guarantees does log compaction provide?

- -Log compaction guarantees the following: -
    -
  1. Any consumer that stays caught-up to within the head of the log will see every message that is written; these messages will have sequential offsets. The topic's min.compaction.lag.ms can be used to -guarantee the minimum length of time must pass after a message is written before it could be compacted. I.e. it provides a lower bound on how long each message will remain in the (uncompacted) head. -
  2. Ordering of messages is always maintained. Compaction will never re-order messages, just remove some. -
  3. The offset for a message never changes. It is the permanent identifier for a position in the log. -
  4. Any consumer progressing from the start of the log will see at least the final state of all records in the order they were written. Additionally, all delete markers for deleted records will be seen, provided -the consumer reaches the head of the log in a time period less than the topic's delete.retention.ms setting (the default is 24 hours). In other words: since the removal of delete markers happens -concurrently with reads, it is possible for a consumer to miss delete markers if it lags by more than delete.retention.ms. -
- -

Log Compaction Details

- -Log compaction is handled by the log cleaner, a pool of background threads that recopy log segment files, removing records whose key appears in the head of the log. Each compactor thread works as follows: -
    -
  1. It chooses the log that has the highest ratio of log head to log tail -
  2. It creates a succinct summary of the last offset for each key in the head of the log -
  3. It recopies the log from beginning to end removing keys which have a later occurrence in the log. New, clean segments are swapped into the log immediately so the additional disk space required is just one -additional log segment (not a fully copy of the log). -
  4. The summary of the log head is essentially just a space-compact hash table. It uses exactly 24 bytes per entry. As a result with 8GB of cleaner buffer one cleaner iteration can clean around 366GB of log head -(assuming 1k messages). -
-

-

Configuring The Log Cleaner

- -The log cleaner is enabled by default. This will start the pool of cleaner threads. -To enable log cleaning on a particular topic you can add the log-specific property -
  log.cleanup.policy=compact
-This can be done either at topic creation time or using the alter topic command. -

-The log cleaner can be configured to retain a minimum amount of the uncompacted "head" of the log. This is enabled by setting the compaction time lag. -

  log.cleaner.min.compaction.lag.ms
- -This can be used to prevent messages newer than a minimum message age from being subject to compaction. If not set, all log segments are eligible for compaction except for the last segment, i.e. the one currently -being written to. The active segment will not be compacted even if all of its messages are older than the minimum compaction time lag. -

-

-Further cleaner configurations are described here. - -

4.9 Quotas

-

- Starting in 0.9, the Kafka cluster has the ability to enforce quotas on produce and fetch requests. Quotas are basically byte-rate thresholds defined per group of clients sharing a quota. -

- -

Why are quotas necessary?

-

-It is possible for producers and consumers to produce/consume very high volumes of data and thus monopolize broker resources, cause network saturation and generally DOS other clients and the brokers themselves. -Having quotas protects against these issues and is all the more important in large multi-tenant clusters where a small set of badly behaved clients can degrade user experience for the well behaved ones. -In fact, when running Kafka as a service this even makes it possible to enforce API limits according to an agreed upon contract. -

-

Client groups

- The identity of Kafka clients is the user principal which represents an authenticated user in a secure cluster. In a cluster that supports unauthenticated clients, user principal is a grouping of unauthenticated - users - chosen by the broker using a configurable PrincipalBuilder. Client-id is a logical grouping of clients with a meaningful name chosen by the client application. The tuple (user, client-id) defines - a secure logical group of clients that share both user principal and client-id. -

- Quotas can be applied to (user, client-id), user or client-id groups. For a given connection, the most specific quota matching the connection is applied. All connections of a quota group share the quota configured for the group. - For example, if (user="test-user", client-id="test-client") has a produce quota of 10MB/sec, this is shared across all producer instances of user "test-user" with the client-id "test-client". -

-

Quota Configuration

-

- Quota configuration may be defined for (user, client-id), user and client-id groups. It is possible to override the default quota at any of the quota levels that needs a higher (or even lower) quota. - The mechanism is similar to the per-topic log config overrides. - User and (user, client-id) quota overrides are written to ZooKeeper under /config/users and client-id quota overrides are written under /config/clients. - These overrides are read by all brokers and are effective immediately. This lets us change quotas without having to do a rolling restart of the entire cluster. See here for details. - Default quotas for each group may also be updated dynamically using the same mechanism. -

-

- The order of precedence for quota configuration is: + + +

\ No newline at end of file diff --git a/docs/documentation.html b/docs/documentation.html index 75c04670631f4..e8516f909e40c 100644 --- a/docs/documentation.html +++ b/docs/documentation.html @@ -15,6 +15,8 @@ limitations under the License. --> + + diff --git a/docs/implementation.html b/docs/implementation.html index 60ca0a10992b3..48602f2630154 100644 --- a/docs/implementation.html +++ b/docs/implementation.html @@ -15,391 +15,395 @@ limitations under the License. --> -

5.1 API Design

- -

Producer APIs

- -

-The Producer API that wraps the 2 low-level producers - kafka.producer.SyncProducer and kafka.producer.async.AsyncProducer. -

-class Producer {
-
-  /* Sends the data, partitioned by key to the topic using either the */
-  /* synchronous or the asynchronous producer */
-  public void send(kafka.javaapi.producer.ProducerData<K,V> producerData);
-
-  /* Sends a list of data, partitioned by key to the topic using either */
-  /* the synchronous or the asynchronous producer */
-  public void send(java.util.List<kafka.javaapi.producer.ProducerData<K,V>> producerData);
-
-  /* Closes the producer and cleans up */
-  public void close();
-
-}
-
- -The goal is to expose all the producer functionality through a single API to the client. - -The Kafka producer -
    -
  • can handle queueing/buffering of multiple producer requests and asynchronous dispatch of the batched data: -

    kafka.producer.Producer provides the ability to batch multiple produce requests (producer.type=async), before serializing and dispatching them to the appropriate kafka broker partition. The size of the batch can be controlled by a few config parameters. As events enter a queue, they are buffered in a queue, until either queue.time or batch.size is reached. A background thread (kafka.producer.async.ProducerSendThread) dequeues the batch of data and lets the kafka.producer.EventHandler serialize and send the data to the appropriate kafka broker partition. A custom event handler can be plugged in through the event.handler config parameter. At various stages of this producer queue pipeline, it is helpful to be able to inject callbacks, either for plugging in custom logging/tracing code or custom monitoring logic. This is possible by implementing the kafka.producer.async.CallbackHandler interface and setting callback.handler config parameter to that class. -

    -
  • -
  • handles the serialization of data through a user-specified Encoder: -
    -interface Encoder<T> {
    -  public Message toMessage(T data);
    -}
    -
    -

    The default is the no-op kafka.serializer.DefaultEncoder

    -
  • -
  • provides software load balancing through an optionally user-specified Partitioner: -

    -The routing decision is influenced by the kafka.producer.Partitioner. -

    -interface Partitioner<T> {
    -   int partition(T key, int numPartitions);
    -}
    -
    -The partition API uses the key and the number of available broker partitions to return a partition id. This id is used as an index into a sorted list of broker_ids and partitions to pick a broker partition for the producer request. The default partitioning strategy is hash(key)%numPartitions. If the key is null, then a random broker partition is picked. A custom partitioning strategy can also be plugged in using the partitioner.class config parameter. -

    -
  • -
-

- -

Consumer APIs

-

-We have 2 levels of consumer APIs. The low-level "simple" API maintains a connection to a single broker and has a close correspondence to the network requests sent to the server. This API is completely stateless, with the offset being passed in on every request, allowing the user to maintain this metadata however they choose. -

-

-The high-level API hides the details of brokers from the consumer and allows consuming off the cluster of machines without concern for the underlying topology. It also maintains the state of what has been consumed. The high-level API also provides the ability to subscribe to topics that match a filter expression (i.e., either a whitelist or a blacklist regular expression). -

- -
Low-level API
-
-class SimpleConsumer {
-
-  /* Send fetch request to a broker and get back a set of messages. */
-  public ByteBufferMessageSet fetch(FetchRequest request);
-
-  /* Send a list of fetch requests to a broker and get back a response set. */
-  public MultiFetchResponse multifetch(List<FetchRequest> fetches);
-
-  /**
-   * Get a list of valid offsets (up to maxSize) before the given time.
-   * The result is a list of offsets, in descending order.
-   * @param time: time in millisecs,
-   *              if set to OffsetRequest$.MODULE$.LATEST_TIME(), get from the latest offset available.
-   *              if set to OffsetRequest$.MODULE$.EARLIEST_TIME(), get from the earliest offset available.
-   */
-  public long[] getOffsetsBefore(String topic, int partition, long time, int maxNumOffsets);
-}
-
- -The low-level API is used to implement the high-level API as well as being used directly for some of our offline consumers which have particular requirements around maintaining state. - -
High-level API
-
-
-/* create a connection to the cluster */
-ConsumerConnector connector = Consumer.create(consumerConfig);
-
-interface ConsumerConnector {
-
-  /**
-   * This method is used to get a list of KafkaStreams, which are iterators over
-   * MessageAndMetadata objects from which you can obtain messages and their
-   * associated metadata (currently only topic).
-   *  Input: a map of <topic, #streams>
-   *  Output: a map of <topic, list of message streams>
-   */
-  public Map<String,List<KafkaStream>> createMessageStreams(Map<String,Int> topicCountMap);
-
-  /**
-   * You can also obtain a list of KafkaStreams, that iterate over messages
-   * from topics that match a TopicFilter. (A TopicFilter encapsulates a
-   * whitelist or a blacklist which is a standard Java regex.)
-   */
-  public List<KafkaStream> createMessageStreamsByFilter(
-      TopicFilter topicFilter, int numStreams);
-
-  /* Commit the offsets of all messages consumed so far. */
-  public commitOffsets()
-
-  /* Shut down the connector */
-  public shutdown()
-}
-
-

-This API is centered around iterators, implemented by the KafkaStream class. Each KafkaStream represents the stream of messages from one or more partitions on one or more servers. Each stream is used for single threaded processing, so the client can provide the number of desired streams in the create call. Thus a stream may represent the merging of multiple server partitions (to correspond to the number of processing threads), but each partition only goes to one stream. -

-

-The createMessageStreams call registers the consumer for the topic, which results in rebalancing the consumer/broker assignment. The API encourages creating many topic streams in a single call in order to minimize this rebalancing. The createMessageStreamsByFilter call (additionally) registers watchers to discover new topics that match its filter. Note that each stream that createMessageStreamsByFilter returns may iterate over messages from multiple topics (i.e., if multiple topics are allowed by the filter). -

- -

5.2 Network Layer

-

-The network layer is a fairly straight-forward NIO server, and will not be described in great detail. The sendfile implementation is done by giving the MessageSet interface a writeTo method. This allows the file-backed message set to use the more efficient transferTo implementation instead of an in-process buffered write. The threading model is a single acceptor thread and N processor threads which handle a fixed number of connections each. This design has been pretty thoroughly tested elsewhere and found to be simple to implement and fast. The protocol is kept quite simple to allow for future implementation of clients in other languages. -

-

5.3 Messages

-

-Messages consist of a fixed-size header, a variable length opaque key byte array and a variable length opaque value byte array. The header contains the following fields: -

    -
  • A CRC32 checksum to detect corruption or truncation.
  • -
  • A format version.
  • -
  • An attributes identifier
  • -
  • A timestamp
  • -
-Leaving the key and value opaque is the right decision: there is a great deal of progress being made on serialization libraries right now, and any particular choice is unlikely to be right for all uses. Needless to say a particular application using Kafka would likely mandate a particular serialization type as part of its usage. The MessageSet interface is simply an iterator over messages with specialized methods for bulk reading and writing to an NIO Channel. - -

5.4 Message Format

- -
+
+
+
diff --git a/docs/introduction.html b/docs/introduction.html index d034683d4f34f..f1bf488641e99 100644 --- a/docs/introduction.html +++ b/docs/introduction.html @@ -14,186 +14,206 @@ See the License for the specific language governing permissions and limitations under the License. --> -

Kafka is a distributed streaming platform. What exactly does that mean?

-

We think of a streaming platform as having three key capabilities:

-
    -
  1. It lets you publish and subscribe to streams of records. In this respect it is similar to a message queue or enterprise messaging system. -
  2. It lets you store streams of records in a fault-tolerant way. -
  3. It lets you process streams of records as they occur. -
-

What is Kafka good for?

-

It gets used for two broad classes of application:

-
    -
  1. Building real-time streaming data pipelines that reliably get data between systems or applications -
  2. Building real-time streaming applications that transform or react to the streams of data -
-

To understand how Kafka does these things, let's dive in and explore Kafka's capabilities from the bottom up.

-

First a few concepts:

-
    -
  • Kafka is run as a cluster on one or more servers. -
  • The Kafka cluster stores streams of records in categories called topics. -
  • Each record consists of a key, a value, and a timestamp. -
-

Kafka has four core APIs:

-
-
    -
  • The Producer API allows an application to publish a stream records to one or more Kafka topics. -
  • The Consumer API allows an application to subscribe to one or more topics and process the stream of records produced to them. -
  • The Streams API allows an application to act as a stream processor, consuming an input stream from one or more topics and producing an output stream to one or more output topics, effectively transforming the input streams to output streams. -
  • The Connector API allows building and running reusable producers or consumers that connect Kafka topics to existing applications or data systems. For example, a connector to a relational database might capture every change to a table. -
- -
-

-In Kafka the communication between the clients and the servers is done with a simple, high-performance, language agnostic TCP protocol. This protocol is versioned and maintains backwards compatibility with older version. We provide a Java client for Kafka, but clients are available in many languages.

- -

Topics and Logs

-

Let's first dive into the core abstraction Kafka provides for a stream of records—the topic.

-

A topic is a category or feed name to which records are published. Topics in Kafka are always multi-subscriber; that is, a topic can have zero, one, or many consumers that subscribe to the data written to it.

-

For each topic, the Kafka cluster maintains a partitioned log that looks like this:

- - -

Each partition is an ordered, immutable sequence of records that is continually appended to—a structured commit log. The records in the partitions are each assigned a sequential id number called the offset that uniquely identifies each record within the partition. -

-

-The Kafka cluster retains all published records—whether or not they have been consumed—using a configurable retention period. For example, if the retention policy is set to two days, then for the two days after a record is published, it is available for consumption, after which it will be discarded to free up space. Kafka's performance is effectively constant with respect to data size so storing data for a long time is not a problem. -

- -

-In fact, the only metadata retained on a per-consumer basis is the offset or position of that consumer in the log. This offset is controlled by the consumer: normally a consumer will advance its offset linearly as it reads records, but, in fact, since the position is controlled by the consumer it can consume records in any order it likes. For example a consumer can reset to an older offset to reprocess data from the past or skip ahead to the most recent record and start consuming from "now". -

-

-This combination of features means that Kafka consumers are very cheap—they can come and go without much impact on the cluster or on other consumers. For example, you can use our command line tools to "tail" the contents of any topic without changing what is consumed by any existing consumers. -

-

-The partitions in the log serve several purposes. First, they allow the log to scale beyond a size that will fit on a single server. Each individual partition must fit on the servers that host it, but a topic may have many partitions so it can handle an arbitrary amount of data. Second they act as the unit of parallelism—more on that in a bit. -

- -

Distribution

- -

-The partitions of the log are distributed over the servers in the Kafka cluster with each server handling data and requests for a share of the partitions. Each partition is replicated across a configurable number of servers for fault tolerance. -

-

-Each partition has one server which acts as the "leader" and zero or more servers which act as "followers". The leader handles all read and write requests for the partition while the followers passively replicate the leader. If the leader fails, one of the followers will automatically become the new leader. Each server acts as a leader for some of its partitions and a follower for others so load is well balanced within the cluster. -

- -

Producers

-

-Producers publish data to the topics of their choice. The producer is responsible for choosing which record to assign to which partition within the topic. This can be done in a round-robin fashion simply to balance load or it can be done according to some semantic partition function (say based on some key in the record). More on the use of partitioning in a second! -

- -

Consumers

- -

-Consumers label themselves with a consumer group name, and each record published to a topic is delivered to one consumer instance within each subscribing consumer group. Consumer instances can be in separate processes or on separate machines. -

-

-If all the consumer instances have the same consumer group, then the records will effectively be load balanced over the consumer instances.

-

-If all the consumer instances have different consumer groups, then each record will be broadcast to all the consumer processes. -

- -

- A two server Kafka cluster hosting four partitions (P0-P3) with two consumer groups. Consumer group A has two consumer instances and group B has four. -

- -

-More commonly, however, we have found that topics have a small number of consumer groups, one for each "logical subscriber". Each group is composed of many consumer instances for scalability and fault tolerance. This is nothing more than publish-subscribe semantics where the subscriber is a cluster of consumers instead of a single process. -

-

-The way consumption is implemented in Kafka is by dividing up the partitions in the log over the consumer instances so that each instance is the exclusive consumer of a "fair share" of partitions at any point in time. This process of maintaining membership in the group is handled by the Kafka protocol dynamically. If new instances join the group they will take over some partitions from other members of the group; if an instance dies, its partitions will be distributed to the remaining instances. -

-

-Kafka only provides a total order over records within a partition, not between different partitions in a topic. Per-partition ordering combined with the ability to partition data by key is sufficient for most applications. However, if you require a total order over records this can be achieved with a topic that has only one partition, though this will mean only one consumer process per consumer group. -

-

Guarantees

-

-At a high-level Kafka gives the following guarantees: -

-
    -
  • Messages sent by a producer to a particular topic partition will be appended in the order they are sent. That is, if a record M1 is sent by the same producer as a record M2, and M1 is sent first, then M1 will have a lower offset than M2 and appear earlier in the log. -
  • A consumer instance sees records in the order they are stored in the log. -
  • For a topic with replication factor N, we will tolerate up to N-1 server failures without losing any records committed to the log. -
-

-More details on these guarantees are given in the design section of the documentation. -

-

Kafka as a Messaging System

-

-How does Kafka's notion of streams compare to a traditional enterprise messaging system? -

-

-Messaging traditionally has two models: queuing and publish-subscribe. In a queue, a pool of consumers may read from a server and each record goes to one of them; in publish-subscribe the record is broadcast to all consumers. Each of these two models has a strength and a weakness. The strength of queuing is that it allows you to divide up the processing of data over multiple consumer instances, which lets you scale your processing. Unfortunately, queues aren't multi-subscriber—once one process reads the data it's gone. Publish-subscribe allows you broadcast data to multiple processes, but has no way of scaling processing since every message goes to every subscriber. -

-

-The consumer group concept in Kafka generalizes these two concepts. As with a queue the consumer group allows you to divide up processing over a collection of processes (the members of the consumer group). As with publish-subscribe, Kafka allows you to broadcast messages to multiple consumer groups. -

-

-The advantage of Kafka's model is that every topic has both these properties—it can scale processing and is also multi-subscriber—there is no need to choose one or the other. -

-

-Kafka has stronger ordering guarantees than a traditional messaging system, too. -

-

-A traditional queue retains records in-order on the server, and if multiple consumers consume from the queue then the server hands out records in the order they are stored. However, although the server hands out records in order, the records are delivered asynchronously to consumers, so they may arrive out of order on different consumers. This effectively means the ordering of the records is lost in the presence of parallel consumption. Messaging systems often work around this by having a notion of "exclusive consumer" that allows only one process to consume from a queue, but of course this means that there is no parallelism in processing. -

-

-Kafka does it better. By having a notion of parallelism—the partition—within the topics, Kafka is able to provide both ordering guarantees and load balancing over a pool of consumer processes. This is achieved by assigning the partitions in the topic to the consumers in the consumer group so that each partition is consumed by exactly one consumer in the group. By doing this we ensure that the consumer is the only reader of that partition and consumes the data in order. Since there are many partitions this still balances the load over many consumer instances. Note however that there cannot be more consumer instances in a consumer group than partitions. -

- -

Kafka as a Storage System

- -

-Any message queue that allows publishing messages decoupled from consuming them is effectively acting as a storage system for the in-flight messages. What is different about Kafka is that it is a very good storage system. -

-

-Data written to Kafka is written to disk and replicated for fault-tolerance. Kafka allows producers to wait on acknowledgement so that a write isn't considered complete until it is fully replicated and guaranteed to persist even if the server written to fails. -

-

-The disk structures Kafka uses scale well—Kafka will perform the same whether you have 50 KB or 50 TB of persistent data on the server. -

-

-As a result of taking storage seriously and allowing the clients to control their read position, you can think of Kafka as a kind of special purpose distributed filesystem dedicated to high-performance, low-latency commit log storage, replication, and propagation. -

-

Kafka for Stream Processing

-

-It isn't enough to just read, write, and store streams of data, the purpose is to enable real-time processing of streams. -

-

-In Kafka a stream processor is anything that takes continual streams of data from input topics, performs some processing on this input, and produces continual streams of data to output topics. -

-

-For example, a retail application might take in input streams of sales and shipments, and output a stream of reorders and price adjustments computed off this data. -

-

-It is possible to do simple processing directly using the producer and consumer APIs. However for more complex transformations Kafka provides a fully integrated Streams API. This allows building applications that do non-trivial processing that compute aggregations off of streams or join streams together. -

-

-This facility helps solve the hard problems this type of application faces: handling out-of-order data, reprocessing input as code changes, performing stateful computations, etc. -

-

-The streams API builds on the core primitives Kafka provides: it uses the producer and consumer APIs for input, uses Kafka for stateful storage, and uses the same group mechanism for fault tolerance among the stream processor instances. -

-

Putting the Pieces Together

-

-This combination of messaging, storage, and stream processing may seem unusual but it is essential to Kafka's role as a streaming platform. -

-

-A distributed file system like HDFS allows storing static files for batch processing. Effectively a system like this allows storing and processing historical data from the past. -

-

-A traditional enterprise messaging system allows processing future messages that will arrive after you subscribe. Applications built in this way process future data as it arrives. -

-

-Kafka combines both of these capabilities, and the combination is critical both for Kafka usage as a platform for streaming applications as well as for streaming data pipelines. -

-

-By combining storage and low-latency subscriptions, streaming applications can treat both past and future data the same way. That is a single application can process historical, stored data but rather than ending when it reaches the last record it can keep processing as future data arrives. This is a generalized notion of stream processing that subsumes batch processing as well as message-driven applications. -

-

-Likewise for streaming data pipelines the combination of subscription to real-time events make it possible to use Kafka for very low-latency pipelines; but the ability to store data reliably make it possible to use it for critical data where the delivery of data must be guaranteed or for integration with offline systems that load data only periodically or may go down for extended periods of time for maintenance. The stream processing facilities make it possible to transform data as it arrives. -

-

-For more information on the guarantees, apis, and capabilities Kafka provides see the rest of the documentation. -

+ + + + + + + +
+ +
+
+
+
+ + + diff --git a/docs/js/templateData.js b/docs/js/templateData.js new file mode 100644 index 0000000000000..40c5da195b471 --- /dev/null +++ b/docs/js/templateData.js @@ -0,0 +1,21 @@ +/* +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. +*/ + +// Define variables for doc templates +var context={ + "version": "0101" +}; \ No newline at end of file diff --git a/docs/ops.html b/docs/ops.html index ef889a392353a..a034b49917a9f 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -15,1345 +15,1349 @@ limitations under the License. --> -Here is some information on actually running Kafka as a production system based on usage and experience at LinkedIn. Please send us any additional tips you know of. - -

6.1 Basic Kafka Operations

- -This section will review the most common operations you will perform on your Kafka cluster. All of the tools reviewed in this section are available under the bin/ directory of the Kafka distribution and each tool will print details on all possible commandline options if it is run with no arguments. - -

Adding and removing topics

- -You have the option of either adding topics manually or having them be created automatically when data is first published to a non-existent topic. If topics are auto-created then you may want to tune the default topic configurations used for auto-created topics. -

-Topics are added and modified using the topic tool: -

- > bin/kafka-topics.sh --zookeeper zk_host:port/chroot --create --topic my_topic_name
-       --partitions 20 --replication-factor 3 --config x=y
-
-The replication factor controls how many servers will replicate each message that is written. If you have a replication factor of 3 then up to 2 servers can fail before you will lose access to your data. We recommend you use a replication factor of 2 or 3 so that you can transparently bounce machines without interrupting data consumption. -

-The partition count controls how many logs the topic will be sharded into. There are several impacts of the partition count. First each partition must fit entirely on a single server. So if you have 20 partitions the full data set (and read and write load) will be handled by no more than 20 servers (no counting replicas). Finally the partition count impacts the maximum parallelism of your consumers. This is discussed in greater detail in the concepts section. -

-Each sharded partition log is placed into its own folder under the Kafka log directory. The name of such folders consists of the topic name, appended by a dash (-) and the partition id. Since a typical folder name can not be over 255 characters long, there will be a limitation on the length of topic names. We assume the number of partitions will not ever be above 100,000. Therefore, topic names cannot be longer than 249 characters. This leaves just enough room in the folder name for a dash and a potentially 5 digit long partition id. -

-The configurations added on the command line override the default settings the server has for things like the length of time data should be retained. The complete set of per-topic configurations is documented here. - -

Modifying topics

- -You can change the configuration or partitioning of a topic using the same topic tool. -

-To add partitions you can do -

- > bin/kafka-topics.sh --zookeeper zk_host:port/chroot --alter --topic my_topic_name
-       --partitions 40
-
-Be aware that one use case for partitions is to semantically partition data, and adding partitions doesn't change the partitioning of existing data so this may disturb consumers if they rely on that partition. That is if data is partitioned by hash(key) % number_of_partitions then this partitioning will potentially be shuffled by adding partitions but Kafka will not attempt to automatically redistribute data in any way. -

-To add configs: -

- > bin/kafka-topics.sh --zookeeper zk_host:port/chroot --alter --topic my_topic_name --config x=y
-
-To remove a config: -
- > bin/kafka-topics.sh --zookeeper zk_host:port/chroot --alter --topic my_topic_name --delete-config x
-
-And finally deleting a topic: -
- > bin/kafka-topics.sh --zookeeper zk_host:port/chroot --delete --topic my_topic_name
-
-Topic deletion option is disabled by default. To enable it set the server config -
delete.topic.enable=true
-

-Kafka does not currently support reducing the number of partitions for a topic. -

-Instructions for changing the replication factor of a topic can be found here. - -

Graceful shutdown

- -The Kafka cluster will automatically detect any broker shutdown or failure and elect new leaders for the partitions on that machine. This will occur whether a server fails or it is brought down intentionally for maintenance or configuration changes. For the latter cases Kafka supports a more graceful mechanism for stopping a server than just killing it. - -When a server is stopped gracefully it has two optimizations it will take advantage of: -
    -
  1. It will sync all its logs to disk to avoid needing to do any log recovery when it restarts (i.e. validating the checksum for all messages in the tail of the log). Log recovery takes time so this speeds up intentional restarts. -
  2. It will migrate any partitions the server is the leader for to other replicas prior to shutting down. This will make the leadership transfer faster and minimize the time each partition is unavailable to a few milliseconds. -
- -Syncing the logs will happen automatically whenever the server is stopped other than by a hard kill, but the controlled leadership migration requires using a special setting: -
-    controlled.shutdown.enable=true
-
-Note that controlled shutdown will only succeed if all the partitions hosted on the broker have replicas (i.e. the replication factor is greater than 1 and at least one of these replicas is alive). This is generally what you want since shutting down the last replica would make that topic partition unavailable. - -

Balancing leadership

- -Whenever a broker stops or crashes leadership for that broker's partitions transfers to other replicas. This means that by default when the broker is restarted it will only be a follower for all its partitions, meaning it will not be used for client reads and writes. -

-To avoid this imbalance, Kafka has a notion of preferred replicas. If the list of replicas for a partition is 1,5,9 then node 1 is preferred as the leader to either node 5 or 9 because it is earlier in the replica list. You can have the Kafka cluster try to restore leadership to the restored replicas by running the command: -

- > bin/kafka-preferred-replica-election.sh --zookeeper zk_host:port/chroot
-
- -Since running this command can be tedious you can also configure Kafka to do this automatically by setting the following configuration: -
-    auto.leader.rebalance.enable=true
-
- -

Balancing Replicas Across Racks

-The rack awareness feature spreads replicas of the same partition across different racks. This extends the guarantees Kafka provides for broker-failure to cover rack-failure, limiting the risk of data loss should all the brokers on a rack fail at once. The feature can also be applied to other broker groupings such as availability zones in EC2. -

-You can specify that a broker belongs to a particular rack by adding a property to the broker config: -
   broker.rack=my-rack-id
-When a topic is created, modified or replicas are redistributed, the rack constraint will be honoured, ensuring replicas span as many racks as they can (a partition will span min(#racks, replication-factor) different racks). -

-The algorithm used to assign replicas to brokers ensures that the number of leaders per broker will be constant, regardless of how brokers are distributed across racks. This ensures balanced throughput. -

-However if racks are assigned different numbers of brokers, the assignment of replicas will not be even. Racks with fewer brokers will get more replicas, meaning they will use more storage and put more resources into replication. Hence it is sensible to configure an equal number of brokers per rack. - -

Mirroring data between clusters

- -We refer to the process of replicating data between Kafka clusters "mirroring" to avoid confusion with the replication that happens amongst the nodes in a single cluster. Kafka comes with a tool for mirroring data between Kafka clusters. The tool consumes from a source cluster and produces to a destination cluster. - -A common use case for this kind of mirroring is to provide a replica in another datacenter. This scenario will be discussed in more detail in the next section. -

-You can run many such mirroring processes to increase throughput and for fault-tolerance (if one process dies, the others will take overs the additional load). -

-Data will be read from topics in the source cluster and written to a topic with the same name in the destination cluster. In fact the mirror maker is little more than a Kafka consumer and producer hooked together. -

-The source and destination clusters are completely independent entities: they can have different numbers of partitions and the offsets will not be the same. For this reason the mirror cluster is not really intended as a fault-tolerance mechanism (as the consumer position will be different); for that we recommend using normal in-cluster replication. The mirror maker process will, however, retain and use the message key for partitioning so order is preserved on a per-key basis. -

-Here is an example showing how to mirror a single topic (named my-topic) from an input cluster: -

- > bin/kafka-mirror-maker.sh
-       --consumer.config consumer.properties
-       --producer.config producer.properties --whitelist my-topic
-
-Note that we specify the list of topics with the --whitelist option. This option allows any regular expression using Java-style regular expressions. So you could mirror two topics named A and B using --whitelist 'A|B'. Or you could mirror all topics using --whitelist '*'. Make sure to quote any regular expression to ensure the shell doesn't try to expand it as a file path. For convenience we allow the use of ',' instead of '|' to specify a list of topics. -

-Sometimes it is easier to say what it is that you don't want. Instead of using --whitelist to say what you want -to mirror you can use --blacklist to say what to exclude. This also takes a regular expression argument. -However, --blacklist is not supported when the new consumer has been enabled (i.e. when bootstrap.servers -has been defined in the consumer configuration). -

-Combining mirroring with the configuration auto.create.topics.enable=true makes it possible to have a replica cluster that will automatically create and replicate all data in a source cluster even as new topics are added. - -

Checking consumer position

-Sometimes it's useful to see the position of your consumers. We have a tool that will show the position of all consumers in a consumer group as well as how far behind the end of the log they are. To run this tool on a consumer group named my-group consuming a topic named my-topic would look like this: -
- > bin/kafka-run-class.sh kafka.tools.ConsumerOffsetChecker --zookeeper localhost:2181 --group test
-Group           Topic                          Pid Offset          logSize         Lag             Owner
-my-group        my-topic                       0   0               0               0               test_jkreps-mn-1394154511599-60744496-0
-my-group        my-topic                       1   0               0               0               test_jkreps-mn-1394154521217-1a0be913-0
-
- - -NOTE: Since 0.9.0.0, the kafka.tools.ConsumerOffsetChecker tool has been deprecated. You should use the kafka.admin.ConsumerGroupCommand (or the bin/kafka-consumer-groups.sh script) to manage consumer groups, including consumers created with the new consumer API. - -

Managing Consumer Groups

- -With the ConsumerGroupCommand tool, we can list, describe, or delete consumer groups. Note that deletion is only available when the group metadata is stored in -ZooKeeper. When using the new consumer API (where -the broker handles coordination of partition handling and rebalance), the group is deleted when the last committed offset for that group expires. - -For example, to list all consumer groups across all topics: - -
- > bin/kafka-consumer-groups.sh --bootstrap-server broker1:9092 --list
-
-test-consumer-group
-
- -To view offsets as in the previous example with the ConsumerOffsetChecker, we "describe" the consumer group like this: - -
- > bin/kafka-consumer-groups.sh --bootstrap-server broker1:9092 --describe --group test-consumer-group
-
-GROUP                          TOPIC                          PARTITION  CURRENT-OFFSET  LOG-END-OFFSET  LAG             OWNER
-test-consumer-group            test-foo                       0          1               3               2               consumer-1_/127.0.0.1
-
- -If you are using the old high-level consumer and storing the group metadata in ZooKeeper (i.e. offsets.storage=zookeeper), pass ---zookeeper instead of bootstrap-server: - -
- > bin/kafka-consumer-groups.sh --zookeeper localhost:2181 --list
-
- -

Expanding your cluster

- -Adding servers to a Kafka cluster is easy, just assign them a unique broker id and start up Kafka on your new servers. However these new servers will not automatically be assigned any data partitions, so unless partitions are moved to them they won't be doing any work until new topics are created. So usually when you add machines to your cluster you will want to migrate some existing data to these machines. -

-The process of migrating data is manually initiated but fully automated. Under the covers what happens is that Kafka will add the new server as a follower of the partition it is migrating and allow it to fully replicate the existing data in that partition. When the new server has fully replicated the contents of this partition and joined the in-sync replica one of the existing replicas will delete their partition's data. -

-The partition reassignment tool can be used to move partitions across brokers. An ideal partition distribution would ensure even data load and partition sizes across all brokers. The partition reassignment tool does not have the capability to automatically study the data distribution in a Kafka cluster and move partitions around to attain an even load distribution. As such, the admin has to figure out which topics or partitions should be moved around. -

-The partition reassignment tool can run in 3 mutually exclusive modes: -

    -
  • --generate: In this mode, given a list of topics and a list of brokers, the tool generates a candidate reassignment to move all partitions of the specified topics to the new brokers. This option merely provides a convenient way to generate a partition reassignment plan given a list of topics and target brokers.
  • -
  • --execute: In this mode, the tool kicks off the reassignment of partitions based on the user provided reassignment plan. (using the --reassignment-json-file option). This can either be a custom reassignment plan hand crafted by the admin or provided by using the --generate option
  • -
  • --verify: In this mode, the tool verifies the status of the reassignment for all partitions listed during the last --execute. The status can be either of successfully completed, failed or in progress
  • -
-
Automatically migrating data to new machines
-The partition reassignment tool can be used to move some topics off of the current set of brokers to the newly added brokers. This is typically useful while expanding an existing cluster since it is easier to move entire topics to the new set of brokers, than moving one partition at a time. When used to do this, the user should provide a list of topics that should be moved to the new set of brokers and a target list of new brokers. The tool then evenly distributes all partitions for the given list of topics across the new set of brokers. During this move, the replication factor of the topic is kept constant. Effectively the replicas for all partitions for the input list of topics are moved from the old set of brokers to the newly added brokers. -

-For instance, the following example will move all partitions for topics foo1,foo2 to the new set of brokers 5,6. At the end of this move, all partitions for topics foo1 and foo2 will only exist on brokers 5,6. -

-Since the tool accepts the input list of topics as a json file, you first need to identify the topics you want to move and create the json file as follows: -

-> cat topics-to-move.json
-{"topics": [{"topic": "foo1"},
-            {"topic": "foo2"}],
- "version":1
-}
-
-Once the json file is ready, use the partition reassignment tool to generate a candidate assignment: -
-> bin/kafka-reassign-partitions.sh --zookeeper localhost:2181 --topics-to-move-json-file topics-to-move.json --broker-list "5,6" --generate
-Current partition replica assignment
-
-{"version":1,
- "partitions":[{"topic":"foo1","partition":2,"replicas":[1,2]},
-               {"topic":"foo1","partition":0,"replicas":[3,4]},
-               {"topic":"foo2","partition":2,"replicas":[1,2]},
-               {"topic":"foo2","partition":0,"replicas":[3,4]},
-               {"topic":"foo1","partition":1,"replicas":[2,3]},
-               {"topic":"foo2","partition":1,"replicas":[2,3]}]
-}
-
-Proposed partition reassignment configuration
-
-{"version":1,
- "partitions":[{"topic":"foo1","partition":2,"replicas":[5,6]},
-               {"topic":"foo1","partition":0,"replicas":[5,6]},
-               {"topic":"foo2","partition":2,"replicas":[5,6]},
-               {"topic":"foo2","partition":0,"replicas":[5,6]},
-               {"topic":"foo1","partition":1,"replicas":[5,6]},
-               {"topic":"foo2","partition":1,"replicas":[5,6]}]
-}
-
-

-The tool generates a candidate assignment that will move all partitions from topics foo1,foo2 to brokers 5,6. Note, however, that at this point, the partition movement has not started, it merely tells you the current assignment and the proposed new assignment. The current assignment should be saved in case you want to rollback to it. The new assignment should be saved in a json file (e.g. expand-cluster-reassignment.json) to be input to the tool with the --execute option as follows: -

-> bin/kafka-reassign-partitions.sh --zookeeper localhost:2181 --reassignment-json-file expand-cluster-reassignment.json --execute
-Current partition replica assignment
-
-{"version":1,
- "partitions":[{"topic":"foo1","partition":2,"replicas":[1,2]},
-               {"topic":"foo1","partition":0,"replicas":[3,4]},
-               {"topic":"foo2","partition":2,"replicas":[1,2]},
-               {"topic":"foo2","partition":0,"replicas":[3,4]},
-               {"topic":"foo1","partition":1,"replicas":[2,3]},
-               {"topic":"foo2","partition":1,"replicas":[2,3]}]
-}
-
-Save this to use as the --reassignment-json-file option during rollback
-Successfully started reassignment of partitions
-{"version":1,
- "partitions":[{"topic":"foo1","partition":2,"replicas":[5,6]},
-               {"topic":"foo1","partition":0,"replicas":[5,6]},
-               {"topic":"foo2","partition":2,"replicas":[5,6]},
-               {"topic":"foo2","partition":0,"replicas":[5,6]},
-               {"topic":"foo1","partition":1,"replicas":[5,6]},
-               {"topic":"foo2","partition":1,"replicas":[5,6]}]
-}
-
-

-Finally, the --verify option can be used with the tool to check the status of the partition reassignment. Note that the same expand-cluster-reassignment.json (used with the --execute option) should be used with the --verify option: -

-> bin/kafka-reassign-partitions.sh --zookeeper localhost:2181 --reassignment-json-file expand-cluster-reassignment.json --verify
-Status of partition reassignment:
-Reassignment of partition [foo1,0] completed successfully
-Reassignment of partition [foo1,1] is in progress
-Reassignment of partition [foo1,2] is in progress
-Reassignment of partition [foo2,0] completed successfully
-Reassignment of partition [foo2,1] completed successfully
-Reassignment of partition [foo2,2] completed successfully
-
- -
Custom partition assignment and migration
-The partition reassignment tool can also be used to selectively move replicas of a partition to a specific set of brokers. When used in this manner, it is assumed that the user knows the reassignment plan and does not require the tool to generate a candidate reassignment, effectively skipping the --generate step and moving straight to the --execute step -

-For instance, the following example moves partition 0 of topic foo1 to brokers 5,6 and partition 1 of topic foo2 to brokers 2,3: -

-The first step is to hand craft the custom reassignment plan in a json file: -

-> cat custom-reassignment.json
-{"version":1,"partitions":[{"topic":"foo1","partition":0,"replicas":[5,6]},{"topic":"foo2","partition":1,"replicas":[2,3]}]}
-
-Then, use the json file with the --execute option to start the reassignment process: -
-> bin/kafka-reassign-partitions.sh --zookeeper localhost:2181 --reassignment-json-file custom-reassignment.json --execute
-Current partition replica assignment
-
-{"version":1,
- "partitions":[{"topic":"foo1","partition":0,"replicas":[1,2]},
-               {"topic":"foo2","partition":1,"replicas":[3,4]}]
-}
-
-Save this to use as the --reassignment-json-file option during rollback
-Successfully started reassignment of partitions
-{"version":1,
- "partitions":[{"topic":"foo1","partition":0,"replicas":[5,6]},
-               {"topic":"foo2","partition":1,"replicas":[2,3]}]
-}
-
-

-The --verify option can be used with the tool to check the status of the partition reassignment. Note that the same expand-cluster-reassignment.json (used with the --execute option) should be used with the --verify option: -

-bin/kafka-reassign-partitions.sh --zookeeper localhost:2181 --reassignment-json-file custom-reassignment.json --verify
-Status of partition reassignment:
-Reassignment of partition [foo1,0] completed successfully
-Reassignment of partition [foo2,1] completed successfully
-
- -

Decommissioning brokers

-The partition reassignment tool does not have the ability to automatically generate a reassignment plan for decommissioning brokers yet. As such, the admin has to come up with a reassignment plan to move the replica for all partitions hosted on the broker to be decommissioned, to the rest of the brokers. This can be relatively tedious as the reassignment needs to ensure that all the replicas are not moved from the decommissioned broker to only one other broker. To make this process effortless, we plan to add tooling support for decommissioning brokers in the future. - -

Increasing replication factor

-Increasing the replication factor of an existing partition is easy. Just specify the extra replicas in the custom reassignment json file and use it with the --execute option to increase the replication factor of the specified partitions. -

-For instance, the following example increases the replication factor of partition 0 of topic foo from 1 to 3. Before increasing the replication factor, the partition's only replica existed on broker 5. As part of increasing the replication factor, we will add more replicas on brokers 6 and 7. -

-The first step is to hand craft the custom reassignment plan in a json file: -

-> cat increase-replication-factor.json
-{"version":1,
- "partitions":[{"topic":"foo","partition":0,"replicas":[5,6,7]}]}
-
-Then, use the json file with the --execute option to start the reassignment process: -
-> bin/kafka-reassign-partitions.sh --zookeeper localhost:2181 --reassignment-json-file increase-replication-factor.json --execute
-Current partition replica assignment
-
-{"version":1,
- "partitions":[{"topic":"foo","partition":0,"replicas":[5]}]}
-
-Save this to use as the --reassignment-json-file option during rollback
-Successfully started reassignment of partitions
-{"version":1,
- "partitions":[{"topic":"foo","partition":0,"replicas":[5,6,7]}]}
-
-

-The --verify option can be used with the tool to check the status of the partition reassignment. Note that the same increase-replication-factor.json (used with the --execute option) should be used with the --verify option: -

-bin/kafka-reassign-partitions.sh --zookeeper localhost:2181 --reassignment-json-file increase-replication-factor.json --verify
-Status of partition reassignment:
-Reassignment of partition [foo,0] completed successfully
-
-You can also verify the increase in replication factor with the kafka-topics tool: -
-> bin/kafka-topics.sh --zookeeper localhost:2181 --topic foo --describe
-Topic:foo	PartitionCount:1	ReplicationFactor:3	Configs:
-	Topic: foo	Partition: 0	Leader: 5	Replicas: 5,6,7	Isr: 5,6,7
-
- -

Limiting Bandwidth Usage during Data Migration

-Kafka lets you apply a throttle to replication traffic, setting an upper bound on the bandwidth used to move replicas from machine to machine. This is useful when rebalancing a cluster, bootstrapping a new broker or adding or removing brokers, as it limits the impact these data-intensive operations will have on users. -

-There are two interfaces that can be used to engage a throttle. The simplest, and safest, is to apply a throttle when invoking the kafka-reassign-partitions.sh, but kafka-configs.sh can also be used to view and alter the throttle values directly. -

-So for example, if you were to execute a rebalance, with the below command, it would move partitions at no more than 50MB/s. -
$ bin/kafka-reassign-partitions.sh --zookeeper myhost:2181--execute --reassignment-json-file bigger-cluster.json —throttle 50000000
-When you execute this script you will see the throttle engage: -
-The throttle limit was set to 50000000 B/s
-Successfully started reassignment of partitions.
-

Should you wish to alter the throttle, during a rebalance, say to increase the throughput so it completes quicker, you can do this by re-running the execute command passing the same reassignment-json-file:

-
$ bin/kafka-reassign-partitions.sh --zookeeper localhost:2181  --execute --reassignment-json-file bigger-cluster.json --throttle 700000000
-There is an existing assignment running.
-The throttle limit was set to 700000000 B/s
- -

Once the rebalance completes the administrator can check the status of the rebalance using the --verify option. - If the rebalance has completed, the throttle will be removed via the --verify command. It is important that - administrators remove the throttle in a timely manner once rebalancing completes by running the command with - the --verify option. Failure to do so could cause regular replication traffic to be throttled.

-

When the --verify option is executed, and the reassignment has completed, the script will confirm that the throttle was removed:

- -
$ bin/kafka-reassign-partitions.sh --zookeeper localhost:2181  --verify --reassignment-json-file bigger-cluster.json
-Status of partition reassignment:
-Reassignment of partition [my-topic,1] completed successfully
-Reassignment of partition [mytopic,0] completed successfully
-Throttle was removed.
- -

The administrator can also validate the assigned configs using the kafka-configs.sh. There are two pairs of throttle - configuration used to manage the throttling process. The throttle value itself. This is configured, at a broker - level, using the dynamic properties:

- -
leader.replication.throttled.rate
-follower.replication.throttled.rate
- -

There is also an enumerated set of throttled replicas:

- -
leader.replication.throttled.replicas
-follower.replication.throttled.replicas
- -

Which are configured per topic. All four config values are automatically assigned by kafka-reassign-partitions.sh - (discussed below).

-

To view the throttle limit configuration:

- -
$ bin/kafka-configs.sh --describe --zookeeper localhost:2181 --entity-type brokers
-Configs for brokers '2' are leader.replication.throttled.rate=700000000,follower.replication.throttled.rate=700000000
-Configs for brokers '1' are leader.replication.throttled.rate=700000000,follower.replication.throttled.rate=700000000
- -

This shows the throttle applied to both leader and follower side of the replication protocol. By default both sides - are assigned the same throttled throughput value.

- -

To view the list of throttled replicas:

- -
$ bin/kafka-configs.sh --describe --zookeeper localhost:2181 --entity-type topics
-Configs for topic 'my-topic' are leader.replication.throttled.replicas=1:102,0:101,
-    follower.replication.throttled.replicas=1:101,0:102
- -

Here we see the leader throttle is applied to partition 1 on broker 102 and partition 0 on broker 101. Likewise the - follower throttle is applied to partition 1 on - broker 101 and partition 0 on broker 102.

- -

By default kafka-reassign-partitions.sh will apply the leader throttle to all replicas that exist before the - rebalance, any one of which might be leader. - It will apply the follower throttle to all move destinations. So if there is a partition with replicas on brokers - 101,102, being reassigned to 102,103, a leader throttle, - for that partition, would be applied to 101,102 and a follower throttle would be applied to 103 only.

- - -

If required, you can also use the --alter switch on kafka-configs.sh to alter the throttle configurations manually. -

- -
Safe usage of throttled replication
- -

Some care should be taken when using throttled replication. In particular:

- -

(1) Throttle Removal:

-The throttle should be removed in a timely manner once reassignment completes (by running kafka-reassign-partitions -—verify). - -

(2) Ensuring Progress:

-

If the throttle is set too low, in comparison to the incoming write rate, it is possible for replication to not - make progress. This occurs when:

-
max(BytesInPerSec) > throttle
-

- Where BytesInPerSec is the metric that monitors the write throughput of producers into each broker.

-

The administrator can monitor whether replication is making progress, during the rebalance, using the metric:

- -
kafka.server:type=FetcherLagMetrics,name=ConsumerLag,clientId=([-.\w]+),topic=([-.\w]+),partition=([0-9]+)
- -

The lag should constantly decrease during replication. If the metric does not decrease the administrator should - increase the - throttle throughput as described above.

- - -

Setting quotas

-Quotas overrides and defaults may be configured at (user, client-id), user or client-id levels as described here. -By default, clients receive an unlimited quota. - -It is possible to set custom quotas for each (user, client-id), user or client-id group. -

-Configure custom quota for (user=user1, client-id=clientA): -

-> bin/kafka-configs.sh  --zookeeper localhost:2181 --alter --add-config 'producer_byte_rate=1024,consumer_byte_rate=2048' --entity-type users --entity-name user1 --entity-type clients --entity-name clientA
-Updated config for entity: user-principal 'user1', client-id 'clientA'.
-
- -Configure custom quota for user=user1: -
-> bin/kafka-configs.sh  --zookeeper localhost:2181 --alter --add-config 'producer_byte_rate=1024,consumer_byte_rate=2048' --entity-type users --entity-name user1
-Updated config for entity: user-principal 'user1'.
-
- -Configure custom quota for client-id=clientA: -
-> bin/kafka-configs.sh  --zookeeper localhost:2181 --alter --add-config 'producer_byte_rate=1024,consumer_byte_rate=2048' --entity-type clients --entity-name clientA
-Updated config for entity: client-id 'clientA'.
-
- -It is possible to set default quotas for each (user, client-id), user or client-id group by specifying --entity-default option instead of --entity-name. -

-Configure default client-id quota for user=userA: -

-> bin/kafka-configs.sh  --zookeeper localhost:2181 --alter --add-config 'producer_byte_rate=1024,consumer_byte_rate=2048' --entity-type users --entity-name user1 --entity-type clients --entity-default
-Updated config for entity: user-principal 'user1', default client-id.
-
- -Configure default quota for user: -
-> bin/kafka-configs.sh  --zookeeper localhost:2181 --alter --add-config 'producer_byte_rate=1024,consumer_byte_rate=2048' --entity-type users --entity-default
-Updated config for entity: default user-principal.
-
- -Configure default quota for client-id: -
-> bin/kafka-configs.sh  --zookeeper localhost:2181 --alter --add-config 'producer_byte_rate=1024,consumer_byte_rate=2048' --entity-type clients --entity-default
-Updated config for entity: default client-id.
-
- -Here's how to describe the quota for a given (user, client-id): -
-> bin/kafka-configs.sh  --zookeeper localhost:2181 --describe --entity-type users --entity-name user1 --entity-type clients --entity-name clientA
-Configs for user-principal 'user1', client-id 'clientA' are producer_byte_rate=1024,consumer_byte_rate=2048
-
-Describe quota for a given user: -
-> bin/kafka-configs.sh  --zookeeper localhost:2181 --describe --entity-type users --entity-name user1
-Configs for user-principal 'user1' are producer_byte_rate=1024,consumer_byte_rate=2048
-
-Describe quota for a given client-id: -
-> bin/kafka-configs.sh  --zookeeper localhost:2181 --describe --entity-type clients --entity-name clientA
-Configs for client-id 'clientA' are producer_byte_rate=1024,consumer_byte_rate=2048
-
-If entity name is not specified, all entities of the specified type are described. For example, describe all users: -
-> bin/kafka-configs.sh  --zookeeper localhost:2181 --describe --entity-type users
-Configs for user-principal 'user1' are producer_byte_rate=1024,consumer_byte_rate=2048
-Configs for default user-principal are producer_byte_rate=1024,consumer_byte_rate=2048
-
-Similarly for (user, client): -
-> bin/kafka-configs.sh  --zookeeper localhost:2181 --describe --entity-type users --entity-type clients
-Configs for user-principal 'user1', default client-id are producer_byte_rate=1024,consumer_byte_rate=2048
-Configs for user-principal 'user1', client-id 'clientA' are producer_byte_rate=1024,consumer_byte_rate=2048
-
-

-It is possible to set default quotas that apply to all client-ids by setting these configs on the brokers. These properties are applied only if quota overrides or defaults are not configured in Zookeeper. By default, each client-id receives an unlimited quota. The following sets the default quota per producer and consumer client-id to 10MB/sec. -

-  quota.producer.default=10485760
-  quota.consumer.default=10485760
-
-Note that these properties are being deprecated and may be removed in a future release. Defaults configured using kafka-configs.sh take precedence over these properties. - -

6.2 Datacenters

- -Some deployments will need to manage a data pipeline that spans multiple datacenters. Our recommended approach to this is to deploy a local Kafka cluster in each datacenter with application instances in each datacenter interacting only with their local cluster and mirroring between clusters (see the documentation on the mirror maker tool for how to do this). -

-This deployment pattern allows datacenters to act as independent entities and allows us to manage and tune inter-datacenter replication centrally. This allows each facility to stand alone and operate even if the inter-datacenter links are unavailable: when this occurs the mirroring falls behind until the link is restored at which time it catches up. -

-For applications that need a global view of all data you can use mirroring to provide clusters which have aggregate data mirrored from the local clusters in all datacenters. These aggregate clusters are used for reads by applications that require the full data set. -

-This is not the only possible deployment pattern. It is possible to read from or write to a remote Kafka cluster over the WAN, though obviously this will add whatever latency is required to get the cluster. -

-Kafka naturally batches data in both the producer and consumer so it can achieve high-throughput even over a high-latency connection. To allow this though it may be necessary to increase the TCP socket buffer sizes for the producer, consumer, and broker using the socket.send.buffer.bytes and socket.receive.buffer.bytes configurations. The appropriate way to set this is documented here. -

-It is generally not advisable to run a single Kafka cluster that spans multiple datacenters over a high-latency link. This will incur very high replication latency both for Kafka writes and ZooKeeper writes, and neither Kafka nor ZooKeeper will remain available in all locations if the network between locations is unavailable. - -

6.3 Kafka Configuration

- -

Important Client Configurations

-The most important producer configurations control -
    -
  • compression
  • -
  • sync vs async production
  • -
  • batch size (for async producers)
  • -
-The most important consumer configuration is the fetch size. -

-All configurations are documented in the configuration section. -

-

A Production Server Config

-Here is our production server configuration: -
-# Replication configurations
-num.replica.fetchers=4
-replica.fetch.max.bytes=1048576
-replica.fetch.wait.max.ms=500
-replica.high.watermark.checkpoint.interval.ms=5000
-replica.socket.timeout.ms=30000
-replica.socket.receive.buffer.bytes=65536
-replica.lag.time.max.ms=10000
-
-controller.socket.timeout.ms=30000
-controller.message.queue.size=10
-
-# Log configuration
-num.partitions=8
-message.max.bytes=1000000
-auto.create.topics.enable=true
-log.index.interval.bytes=4096
-log.index.size.max.bytes=10485760
-log.retention.hours=168
-log.flush.interval.ms=10000
-log.flush.interval.messages=20000
-log.flush.scheduler.interval.ms=2000
-log.roll.hours=168
-log.retention.check.interval.ms=300000
-log.segment.bytes=1073741824
-
-# ZK configuration
-zookeeper.connection.timeout.ms=6000
-zookeeper.sync.time.ms=2000
-
-# Socket server configuration
-num.io.threads=8
-num.network.threads=8
-socket.request.max.bytes=104857600
-socket.receive.buffer.bytes=1048576
-socket.send.buffer.bytes=1048576
-queued.max.requests=16
-fetch.purgatory.purge.interval.requests=100
-producer.purgatory.purge.interval.requests=100
-
- -Our client configuration varies a fair amount between different use cases. - -

6.4 Java Version

- -From a security perspective, we recommend you use the latest released version of JDK 1.8 as older freely available versions have disclosed security vulnerabilities. - -LinkedIn is currently running JDK 1.8 u5 (looking to upgrade to a newer version) with the G1 collector. If you decide to use the G1 collector (the current default) and you are still on JDK 1.7, make sure you are on u51 or newer. LinkedIn tried out u21 in testing, but they had a number of problems with the GC implementation in that version. - -LinkedIn's tuning looks like this: -
--Xmx6g -Xms6g -XX:MetaspaceSize=96m -XX:+UseG1GC
--XX:MaxGCPauseMillis=20 -XX:InitiatingHeapOccupancyPercent=35 -XX:G1HeapRegionSize=16M
--XX:MinMetaspaceFreeRatio=50 -XX:MaxMetaspaceFreeRatio=80
-
- -For reference, here are the stats on one of LinkedIn's busiest clusters (at peak): -
    -
  • 60 brokers
  • -
  • 50k partitions (replication factor 2)
  • -
  • 800k messages/sec in
  • -
  • 300 MB/sec inbound, 1 GB/sec+ outbound
  • -
- -The tuning looks fairly aggressive, but all of the brokers in that cluster have a 90% GC pause time of about 21ms, and they're doing less than 1 young GC per second. - -

6.5 Hardware and OS

-We are using dual quad-core Intel Xeon machines with 24GB of memory. -

-You need sufficient memory to buffer active readers and writers. You can do a back-of-the-envelope estimate of memory needs by assuming you want to be able to buffer for 30 seconds and compute your memory need as write_throughput*30. -

-The disk throughput is important. We have 8x7200 rpm SATA drives. In general disk throughput is the performance bottleneck, and more disks is better. Depending on how you configure flush behavior you may or may not benefit from more expensive disks (if you force flush often then higher RPM SAS drives may be better). - -

OS

-Kafka should run well on any unix system and has been tested on Linux and Solaris. -

-We have seen a few issues running on Windows and Windows is not currently a well supported platform though we would be happy to change that. -

-It is unlikely to require much OS-level tuning, but there are two potentially important OS-level configurations: -

    -
  • File descriptor limits: Kafka uses file descriptors for log segments and open connections. If a broker hosts many partitions, consider that the broker needs at least (number_of_partitions)*(partition_size/segment_size) to track all log segments in addition to the number of connections the broker makes. We recommend at least 100000 allowed file descriptors for the broker processes as a starting point. -
  • Max socket buffer size: can be increased to enable high-performance data transfer between data centers as described here. -
-

- -

Disks and Filesystem

-We recommend using multiple drives to get good throughput and not sharing the same drives used for Kafka data with application logs or other OS filesystem activity to ensure good latency. You can either RAID these drives together into a single volume or format and mount each drive as its own directory. Since Kafka has replication the redundancy provided by RAID can also be provided at the application level. This choice has several tradeoffs. -

-If you configure multiple data directories partitions will be assigned round-robin to data directories. Each partition will be entirely in one of the data directories. If data is not well balanced among partitions this can lead to load imbalance between disks. -

-RAID can potentially do better at balancing load between disks (although it doesn't always seem to) because it balances load at a lower level. The primary downside of RAID is that it is usually a big performance hit for write throughput and reduces the available disk space. -

-Another potential benefit of RAID is the ability to tolerate disk failures. However our experience has been that rebuilding the RAID array is so I/O intensive that it effectively disables the server, so this does not provide much real availability improvement. - -

Application vs. OS Flush Management

-Kafka always immediately writes all data to the filesystem and supports the ability to configure the flush policy that controls when data is forced out of the OS cache and onto disk using the flush. This flush policy can be controlled to force data to disk after a period of time or after a certain number of messages has been written. There are several choices in this configuration. -

-Kafka must eventually call fsync to know that data was flushed. When recovering from a crash for any log segment not known to be fsync'd Kafka will check the integrity of each message by checking its CRC and also rebuild the accompanying offset index file as part of the recovery process executed on startup. -

-Note that durability in Kafka does not require syncing data to disk, as a failed node will always recover from its replicas. -

-We recommend using the default flush settings which disable application fsync entirely. This means relying on the background flush done by the OS and Kafka's own background flush. This provides the best of all worlds for most uses: no knobs to tune, great throughput and latency, and full recovery guarantees. We generally feel that the guarantees provided by replication are stronger than sync to local disk, however the paranoid still may prefer having both and application level fsync policies are still supported. -

-The drawback of using application level flush settings is that it is less efficient in its disk usage pattern (it gives the OS less leeway to re-order writes) and it can introduce latency as fsync in most Linux filesystems blocks writes to the file whereas the background flushing does much more granular page-level locking. -

-In general you don't need to do any low-level tuning of the filesystem, but in the next few sections we will go over some of this in case it is useful. - -

Understanding Linux OS Flush Behavior

- -In Linux, data written to the filesystem is maintained in pagecache until it must be written out to disk (due to an application-level fsync or the OS's own flush policy). The flushing of data is done by a set of background threads called pdflush (or in post 2.6.32 kernels "flusher threads"). -

-Pdflush has a configurable policy that controls how much dirty data can be maintained in cache and for how long before it must be written back to disk. -This policy is described here. -When Pdflush cannot keep up with the rate of data being written it will eventually cause the writing process to block incurring latency in the writes to slow down the accumulation of data. -

-You can see the current state of OS memory usage by doing -

-  > cat /proc/meminfo
-
-The meaning of these values are described in the link above. -

-Using pagecache has several advantages over an in-process cache for storing data that will be written out to disk: -

    -
  • The I/O scheduler will batch together consecutive small writes into bigger physical writes which improves throughput. -
  • The I/O scheduler will attempt to re-sequence writes to minimize movement of the disk head which improves throughput. -
  • It automatically uses all the free memory on the machine -
- -

Filesystem Selection

-

Kafka uses regular files on disk, and as such it has no hard dependency on a specific filesystem. The two filesystems which have the most usage, however, are EXT4 and XFS. Historically, EXT4 has had more usage, but recent improvements to the XFS filesystem have shown it to have better performance characteristics for Kafka's workload with no compromise in stability.

-

Comparison testing was performed on a cluster with significant message loads, using a variety of filesystem creation and mount options. The primary metric in Kafka that was monitored was the "Request Local Time", indicating the amount of time append operations were taking. XFS resulted in much better local times (160ms vs. 250ms+ for the best EXT4 configuration), as well as lower average wait times. The XFS performance also showed less variability in disk performance.

-
General Filesystem Notes
-For any filesystem used for data directories, on Linux systems, the following options are recommended to be used at mount time: -
    -
  • noatime: This option disables updating of a file's atime (last access time) attribute when the file is read. This can eliminate a significant number of filesystem writes, especially in the case of bootstrapping consumers. Kafka does not rely on the atime attributes at all, so it is safe to disable this.
  • -
-
XFS Notes
-The XFS filesystem has a significant amount of auto-tuning in place, so it does not require any change in the default settings, either at filesystem creation time or at mount. The only tuning parameters worth considering are: -
    -
  • largeio: This affects the preferred I/O size reported by the stat call. While this can allow for higher performance on larger disk writes, in practice it had minimal or no effect on performance.
  • -
  • nobarrier: For underlying devices that have battery-backed cache, this option can provide a little more performance by disabling periodic write flushes. However, if the underlying device is well-behaved, it will report to the filesystem that it does not require flushes, and this option will have no effect.
  • -
-
EXT4 Notes
-EXT4 is a serviceable choice of filesystem for the Kafka data directories, however getting the most performance out of it will require adjusting several mount options. In addition, these options are generally unsafe in a failure scenario, and will result in much more data loss and corruption. For a single broker failure, this is not much of a concern as the disk can be wiped and the replicas rebuilt from the cluster. In a multiple-failure scenario, such as a power outage, this can mean underlying filesystem (and therefore data) corruption that is not easily recoverable. The following options can be adjusted: -
    -
  • data=writeback: Ext4 defaults to data=ordered which puts a strong order on some writes. Kafka does not require this ordering as it does very paranoid data recovery on all unflushed log. This setting removes the ordering constraint and seems to significantly reduce latency. -
  • Disabling journaling: Journaling is a tradeoff: it makes reboots faster after server crashes but it introduces a great deal of additional locking which adds variance to write performance. Those who don't care about reboot time and want to reduce a major source of write latency spikes can turn off journaling entirely. -
  • commit=num_secs: This tunes the frequency with which ext4 commits to its metadata journal. Setting this to a lower value reduces the loss of unflushed data during a crash. Setting this to a higher value will improve throughput. -
  • nobh: This setting controls additional ordering guarantees when using data=writeback mode. This should be safe with Kafka as we do not depend on write ordering and improves throughput and latency. -
  • delalloc: Delayed allocation means that the filesystem avoid allocating any blocks until the physical write occurs. This allows ext4 to allocate a large extent instead of smaller pages and helps ensure the data is written sequentially. This feature is great for throughput. It does seem to involve some locking in the filesystem which adds a bit of latency variance. -
- -

6.6 Monitoring

- -Kafka uses Yammer Metrics for metrics reporting in both the server and the client. This can be configured to report stats using pluggable stats reporters to hook up to your monitoring system. -

-The easiest way to see the available metrics is to fire up jconsole and point it at a running kafka client or server; this will allow browsing all metrics with JMX. -

-We do graphing and alerting on the following metrics: - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - + + +
diff --git a/docs/security.html b/docs/security.html index f58179ff5c63d..1f2e6f5470057 100644 --- a/docs/security.html +++ b/docs/security.html @@ -15,732 +15,736 @@ limitations under the License. --> -

7.1 Security Overview

-In release 0.9.0.0, the Kafka community added a number of features that, used either separately or together, increases security in a Kafka cluster. These features are considered to be of beta quality. The following security measures are currently supported: -
    -
  1. Authentication of connections to brokers from clients (producers and consumers), other brokers and tools, using either SSL or SASL (Kerberos). - SASL/PLAIN can also be used from release 0.10.0.0 onwards.
  2. -
  3. Authentication of connections from brokers to ZooKeeper
  4. -
  5. Encryption of data transferred between brokers and clients, between brokers, or between brokers and tools using SSL (Note that there is a performance degradation when SSL is enabled, the magnitude of which depends on the CPU type and the JVM implementation.)
  6. -
  7. Authorization of read / write operations by clients
  8. -
  9. Authorization is pluggable and integration with external authorization services is supported
  10. -
- -It's worth noting that security is optional - non-secured clusters are supported, as well as a mix of authenticated, unauthenticated, encrypted and non-encrypted clients. - -The guides below explain how to configure and use the security features in both clients and brokers. - -

7.2 Encryption and Authentication using SSL

-Apache Kafka allows clients to connect over SSL. By default, SSL is disabled but can be turned on as needed. - -
    -
  1. Generate SSL key and certificate for each Kafka broker

    - The first step of deploying HTTPS is to generate the key and the certificate for each machine in the cluster. You can use Java's keytool utility to accomplish this task. - We will generate the key into a temporary keystore initially so that we can export and sign it later with CA. -
    -        keytool -keystore server.keystore.jks -alias localhost -validity {validity} -genkey
    + + +
    \ No newline at end of file diff --git a/docs/streams.html b/docs/streams.html index 306b2a50a2c1f..dec17efa2d270 100644 --- a/docs/streams.html +++ b/docs/streams.html @@ -15,367 +15,410 @@ ~ limitations under the License. ~--> -

    9.1 Overview

    - -

    -Kafka Streams is a client library for processing and analyzing data stored in Kafka and either write the resulting data back to Kafka or send the final output to an external system. It builds upon important stream processing concepts such as properly distinguishing between event time and processing time, windowing support, and simple yet efficient management of application state. -Kafka Streams has a low barrier to entry: You can quickly write and run a small-scale proof-of-concept on a single machine; and you only need to run additional instances of your application on multiple machines to scale up to high-volume production workloads. Kafka Streams transparently handles the load balancing of multiple instances of the same application by leveraging Kafka's parallelism model. -

    -

    -Some highlights of Kafka Streams: -

    - -
      -
    • Designed as a simple and lightweight client library, which can be easily embedded in any Java application and integrated with any existing packaging, deployment and operational tools that users have for their streaming applications.
    • -
    • Has no external dependencies on systems other than Apache Kafka itself as the internal messaging layer; notably, it uses Kafka's partitioning model to horizontally scale processing while maintaining strong ordering guarantees.
    • -
    • Supports fault-tolerant local state, which enables very fast and efficient stateful operations like joins and windowed aggregations.
    • -
    • Employs one-record-at-a-time processing to achieve low processing latency, and supports event-time based windowing operations.
    • -
    • Offers necessary stream processing primitives, along with a high-level Streams DSL and a low-level Processor API.
    • - -
    - -

    9.2 Developer Guide

    - -

    -There is a quickstart example that provides how to run a stream processing program coded in the Kafka Streams library. -This section focuses on how to write, configure, and execute a Kafka Streams application. -

    - -

    Core Concepts

    - -

    -We first summarize the key concepts of Kafka Streams. -

    - -
    Stream Processing Topology
    - -
      -
    • A stream is the most important abstraction provided by Kafka Streams: it represents an unbounded, continuously updating data set. A stream is an ordered, replayable, and fault-tolerant sequence of immutable data records, where a data record is defined as a key-value pair.
    • -
    • A stream processing application written in Kafka Streams defines its computational logic through one or more processor topologies, where a processor topology is a graph of stream processors (nodes) that are connected by streams (edges).
    • -
    • A stream processor is a node in the processor topology; it represents a processing step to transform data in streams by receiving one input record at a time from its upstream processors in the topology, applying its operation to it, and may subsequently producing one or more output records to its downstream processors.
    • -
    - -

    -Kafka Streams offers two ways to define the stream processing topology: the Kafka Streams DSL provides -the most common data transformation operations such as map and filter; the lower-level Processor API allows -developers define and connect custom processors as well as to interact with state stores. -

    - -
    Time
    - -

    -A critical aspect in stream processing is the notion of time, and how it is modeled and integrated. -For example, some operations such as windowing are defined based on time boundaries. -

    -

    -Common notions of time in streams are: -

    - -
      -
    • Event time - The point in time when an event or data record occurred, i.e. was originally created "at the source".
    • -
    • Processing time - The point in time when the event or data record happens to be processed by the stream processing application, i.e. when the record is being consumed. The processing time may be milliseconds, hours, or days etc. later than the original event time.
    • -
    • Ingestion time - The point in time when an event or data record is stored in a topic partition by a Kafka broker. The difference to event time is that this ingestion timestamp is generated when the record is appended to the target topic by the Kafka broker, not when the record is created "at the source". The difference to processing time is that processing time is when the stream processing application processes the record. For example, if a record is never processed, there is no notion of processing time for it, but it still has an ingestion time. -
    -

    -The choice between event-time and ingestion-time is actually done through the configuration of Kafka (not Kafka Streams): From Kafka 0.10.x onwards, timestamps are automatically embedded into Kafka messages. Depending on Kafka's configuration these timestamps represent event-time or ingestion-time. The respective Kafka configuration setting can be specified on the broker level or per topic. The default timestamp extractor in Kafka Streams will retrieve these embedded timestamps as-is. Hence, the effective time semantics of your application depend on the effective Kafka configuration for these embedded timestamps. -

    -

    -Kafka Streams assigns a timestamp to every data record -via the TimestampExtractor interface. -Concrete implementations of this interface may retrieve or compute timestamps based on the actual contents of data records such as an embedded timestamp field -to provide event-time semantics, or use any other approach such as returning the current wall-clock time at the time of processing, -thereby yielding processing-time semantics to stream processing applications. -Developers can thus enforce different notions of time depending on their business needs. For example, -per-record timestamps describe the progress of a stream with regards to time (although records may be out-of-order within the stream) and -are leveraged by time-dependent operations such as joins. -

    - -

    - Finally, whenever a Kafka Streams application writes records to Kafka, then it will also assign timestamps to these new records. The way the timestamps are assigned depends on the context: -

      -
    • When new output records are generated via processing some input record, for example, context.forward() triggered in the process() function call, output record timestamps are inherited from input record timestamps directly.
    • -
    • When new output records are generated via periodic functions such as punctuate(), the output record timestamp is defined as the current internal time (obtained through context.timestamp()) of the stream task.
    • -
    • For aggregations, the timestamp of a resulting aggregate update record will be that of the latest arrived input record that triggered the update.
    • -
    -

    - -
    States
    - -

    -Some stream processing applications don't require state, which means the processing of a message is independent from -the processing of all other messages. -However, being able to maintain state opens up many possibilities for sophisticated stream processing applications: you -can join input streams, or group and aggregate data records. Many such stateful operators are provided by the Kafka Streams DSL. -

    -

    -Kafka Streams provides so-called state stores, which can be used by stream processing applications to store and query data. -This is an important capability when implementing stateful operations. -Every task in Kafka Streams embeds one or more state stores that can be accessed via APIs to store and query data required for processing. -These state stores can either be a persistent key-value store, an in-memory hashmap, or another convenient data structure. -Kafka Streams offers fault-tolerance and automatic recovery for local state stores. -

    -

    - Kafka Streams allows direct read-only queries of the state stores by methods, threads, processes or applications external to the stream processing application that created the state stores. This is provided through a feature called Interactive Queries. All stores are named and Interactive Queries exposes only the read operations of the underlying implementation. -

    -
    -

    -As we have mentioned above, the computational logic of a Kafka Streams application is defined as a processor topology. -Currently Kafka Streams provides two sets of APIs to define the processor topology, which will be described in the subsequent sections. -

    - -

    Low-Level Processor API

    - -
    Processor
    - -

    -Developers can define their customized processing logic by implementing the Processor interface, which -provides process and punctuate methods. The process method is performed on each -of the received record; and the punctuate method is performed periodically based on elapsed time. -In addition, the processor can maintain the current ProcessorContext instance variable initialized in the -init method, and use the context to schedule the punctuation period (context().schedule), to -forward the modified / new key-value pair to downstream processors (context().forward), to commit the current -processing progress (context().commit), etc. -

    - -
    -    public class MyProcessor extends Processor {
    -        private ProcessorContext context;
    -        private KeyValueStore kvStore;
    -
    -        @Override
    -        @SuppressWarnings("unchecked")
    -        public void init(ProcessorContext context) {
    -            this.context = context;
    -            this.context.schedule(1000);
    -            this.kvStore = (KeyValueStore) context.getStateStore("Counts");
    -        }
    -
    -        @Override
    -        public void process(String dummy, String line) {
    -            String[] words = line.toLowerCase().split(" ");
    -
    -            for (String word : words) {
    -                Integer oldValue = this.kvStore.get(word);
    -
    -                if (oldValue == null) {
    -                    this.kvStore.put(word, 1);
    -                } else {
    -                    this.kvStore.put(word, oldValue + 1);
    +
    +
    +
    +
    +
    +
    +
    + +
    + + +
    +
    +
    + +
DescriptionMbean nameNormal value
Message in ratekafka.server:type=BrokerTopicMetrics,name=MessagesInPerSec
Byte in ratekafka.server:type=BrokerTopicMetrics,name=BytesInPerSec
Request ratekafka.network:type=RequestMetrics,name=RequestsPerSec,request={Produce|FetchConsumer|FetchFollower}
Byte out ratekafka.server:type=BrokerTopicMetrics,name=BytesOutPerSec
Log flush rate and timekafka.log:type=LogFlushStats,name=LogFlushRateAndTimeMs
# of under replicated partitions (|ISR| < |all replicas|)kafka.server:type=ReplicaManager,name=UnderReplicatedPartitions0
Is controller active on brokerkafka.controller:type=KafkaController,name=ActiveControllerCountonly one broker in the cluster should have 1
Leader election ratekafka.controller:type=ControllerStats,name=LeaderElectionRateAndTimeMsnon-zero when there are broker failures
Unclean leader election ratekafka.controller:type=ControllerStats,name=UncleanLeaderElectionsPerSec0
Partition countskafka.server:type=ReplicaManager,name=PartitionCountmostly even across brokers
Leader replica countskafka.server:type=ReplicaManager,name=LeaderCountmostly even across brokers
ISR shrink ratekafka.server:type=ReplicaManager,name=IsrShrinksPerSecIf a broker goes down, ISR for some of the partitions will - shrink. When that broker is up again, ISR will be expanded - once the replicas are fully caught up. Other than that, the - expected value for both ISR shrink rate and expansion rate is 0.
ISR expansion ratekafka.server:type=ReplicaManager,name=IsrExpandsPerSecSee above
Max lag in messages btw follower and leader replicaskafka.server:type=ReplicaFetcherManager,name=MaxLag,clientId=Replicalag should be proportional to the maximum batch size of a produce request.
Lag in messages per follower replicakafka.server:type=FetcherLagMetrics,name=ConsumerLag,clientId=([-.\w]+),topic=([-.\w]+),partition=([0-9]+)lag should be proportional to the maximum batch size of a produce request.
Requests waiting in the producer purgatorykafka.server:type=DelayedOperationPurgatory,name=PurgatorySize,delayedOperation=Producenon-zero if ack=-1 is used
Requests waiting in the fetch purgatorykafka.server:type=DelayedOperationPurgatory,name=PurgatorySize,delayedOperation=Fetchsize depends on fetch.wait.max.ms in the consumer
Request total timekafka.network:type=RequestMetrics,name=TotalTimeMs,request={Produce|FetchConsumer|FetchFollower}broken into queue, local, remote and response send time
Time the request waits in the request queuekafka.network:type=RequestMetrics,name=RequestQueueTimeMs,request={Produce|FetchConsumer|FetchFollower}
Time the request is processed at the leaderkafka.network:type=RequestMetrics,name=LocalTimeMs,request={Produce|FetchConsumer|FetchFollower}
Time the request waits for the followerkafka.network:type=RequestMetrics,name=RemoteTimeMs,request={Produce|FetchConsumer|FetchFollower}non-zero for produce requests when ack=-1
Time the request waits in the response queuekafka.network:type=RequestMetrics,name=ResponseQueueTimeMs,request={Produce|FetchConsumer|FetchFollower}