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.
-->
Kafka uses key-value pairs in the <ahref="http://en.wikipedia.org/wiki/.properties">property file format</a> for configuration. These values can be supplied either from a file or programmatically.
Configurations pertinent to topics have both a global default as well an optional per-topic override. If no per-topic configuration is given the global default is used. The override can be set at topic creation time by giving one or more <code>--config</code> options. This example creates a topic named <i>my-topic</i> with a custom max message size and flush rate:
The following are the topic-level configurations. The server's default configuration for this property is given under the Server Default Property heading, setting this default in the server config allows you to change the default given to topics that have no override specified.
<tableclass="data-table">
<tbody>
<tr>
<th>Property</th>
<th>Default</th>
<th>Server Default Property</th>
<th>Description</th>
</tr>
<tr>
<td>cleanup.policy</td>
<td>delete</td>
<td>log.cleanup.policy</td>
<td>A string that is either "delete" or "compact". This string designates the retention policy to use on old log segments. The default policy ("delete") will discard old segments when their retention time or size limit has been reached. The "compact" setting will enable <ahref="#compaction">log compaction</a> on the topic.</td>
</tr>
<tr>
<td>delete.retention.ms</td>
<td>86400000 (24 hours)</td>
<td>log.cleaner.delete.retention.ms</td>
<td>The amount of time to retain delete tombstone markers for <ahref="#compaction">log compacted</a> topics. This setting also gives a bound on the time in which a consumer must complete a read if they begin from offset 0 to ensure that they get a valid snapshot of the final stage (otherwise delete tombstones may be collected before they complete their scan).</td>
</tr>
<tr>
<td>flush.messages</td>
<td>None</td>
<td>log.flush.interval.messages</td>
<td>This setting allows specifying an interval at which we will force an fsync of data written to the log. For example if this was set to 1 we would fsync after every message; if it were 5 we would fsync after every five messages. In general we recommend you not set this and use replication for durability and allow the operating system's background flush capabilities as it is more efficient. This setting can be overridden on a per-topic basis (see <ahref="#topic-config">the per-topic configuration section</a>).</td>
</tr>
<tr>
<td>flush.ms</td>
<td>None</td>
<td>log.flush.interval.ms</td>
<td>This setting allows specifying a time interval at which we will force an fsync of data written to the log. For example if this was set to 1000 we would fsync after 1000 ms had passed. In general we recommend you not set this and use replication for durability and allow the operating system's background flush capabilities as it is more efficient.</td>
</tr>
<tr>
<td>index.interval.bytes</td>
<td>4096</td>
<td>log.index.interval.bytes</td>
<td>This setting controls how frequently Kafka adds an index entry to it's offset index. The default setting ensures that we index a message roughly every 4096 bytes. More indexing allows reads to jump closer to the exact position in the log but makes the index larger. You probably don't need to change this.</td>
</tr>
<tr>
<td>max.message.bytes</td>
<td>1,000,000</td>
<td>message.max.bytes</td>
<td>This is largest message size Kafka will allow to be appended to this topic. Note that if you increase this size you must also increase your consumer's fetch size so they can fetch messages this large.</td>
</tr>
<tr>
<td>min.cleanable.dirty.ratio</td>
<td>0.5</td>
<td>log.cleaner.min.cleanable.ratio</td>
<td>This configuration controls how frequently the log compactor will attempt to clean the log (assuming <ahref="#compaction">log compaction</a> is enabled). By default we will avoid cleaning a log where more than 50% of the log has been compacted. This ratio bounds the maximum space wasted in the log by duplicates (at 50% at most 50% of the log could be duplicates). A higher ratio will mean fewer, more efficient cleanings but will mean more wasted space in the log.</td>
</tr>
<tr>
<td>min.insync.replicas</td>
<td>1</td>
<td>min.insync.replicas</td>
<td>When a producer sets request.required.acks to -1, min.insync.replicas specifies the minimum number of replicas that must acknowledge a write for the write to be considered successful. If this minimum cannot be met, then the producer will raise an exception (either NotEnoughReplicas or NotEnoughReplicasAfterAppend). </br>
When used together, min.insync.replicas and request.required.acks allow you to enforce greater durability guarantees. A typical scenario would be to create a topic with a replication factor of 3, set min.insync.replicas to 2, and produce with request.required.acks of -1. This will ensure that the producer raises an exception if a majority of replicas do not receive a write.</td>
</tr>
<tr>
<td>retention.bytes</td>
<td>None</td>
<td>log.retention.bytes</td>
<td>This configuration controls the maximum size a log can grow to before we will discard old log segments to free up space if we are using the "delete" retention policy. By default there is no size limit only a time limit.</td>
</tr>
<tr>
<td>retention.ms</td>
<td>7 days</td>
<td>log.retention.minutes</td>
<td>This configuration controls the maximum time we will retain a log before we will discard old log segments to free up space if we are using the "delete" retention policy. This represents an SLA on how soon consumers must read their data.</td>
</tr>
<tr>
<td>segment.bytes</td>
<td>1 GB</td>
<td>log.segment.bytes</td>
<td>This configuration controls the segment file size for the log. Retention and cleaning is always done a file at a time so a larger segment size means fewer files but less granular control over retention.</td>
</tr>
<tr>
<td>segment.index.bytes</td>
<td>10 MB</td>
<td>log.index.size.max.bytes</td>
<td>This configuration controls the size of the index that maps offsets to file positions. We preallocate this index file and shrink it only after log rolls. You generally should not need to change this setting.</td>
</tr>
<tr>
<td>segment.ms</td>
<td>7 days</td>
<td>log.roll.hours</td>
<td>This configuration controls the period of time after which Kafka will force the log to roll even if the segment file isn't full to ensure that retention can delete or compact old data.</td>
</tr>
<tr>
<td>segment.jitter.ms</td>
<td>0</td>
<td>log.roll.jitter.{ms,hours}</td>
<td>The maximum jitter to subtract from logRollTimeMillis.</td>
The essential consumer configurations are the following:
<ul>
<li><code>group.id</code>
<li><code>zookeeper.connect</code>
</ul>
<tableclass="data-table">
<tbody><tr>
<th>Property</th>
<th>Default</th>
<th>Description</th>
</tr>
<tr>
<td>group.id</td>
<tdcolspan="1"></td>
<td>A 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.</td>
</tr>
<tr>
<td>zookeeper.connect</td>
<tdcolspan="1"></td>
<td>Specifies the ZooKeeper connection string in the form <code>hostname:port</code> 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 <code>hostname1:port1,hostname2:port2,hostname3:port3</code>.
<p>
The server may also have a ZooKeeper chroot path as part of it's 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 <code>/chroot/path</code> you would give the connection string as <code>hostname1:port1,hostname2:port2,hostname3:port3/chroot/path</code>.</td>
</tr>
<tr>
<td>consumer.id</td>
<tdcolspan="1">null</td>
<td>
<p>Generated automatically if not set.</p>
</td>
</tr>
<tr>
<td>socket.timeout.ms</td>
<tdcolspan="1">30 * 1000</td>
<td>The socket timeout for network requests. The actual timeout set will be max.fetch.wait + socket.timeout.ms.</td>
</tr>
<tr>
<td>socket.receive.buffer.bytes</td>
<tdcolspan="1">64 * 1024</td>
<td>The socket receive buffer for network requests</td>
</tr>
<tr>
<td>fetch.message.max.bytes</td>
<tdnowrap>1024 * 1024</td>
<td>The number of byes 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.</td>
</tr>
<tr>
<td>num.consumer.fetchers</td>
<tdcolspan="1">1</td>
<td>The number fetcher threads used to fetch data.</td>
</tr>
<tr>
<td>auto.commit.enable</td>
<tdcolspan="1">true</td>
<td>If true, periodically commit to ZooKeeper the offset of messages already fetched by the consumer. This committed offset will be used when the process fails as the position from which the new consumer will begin.</td>
</tr>
<tr>
<td>auto.commit.interval.ms</td>
<tdcolspan="1">60 * 1000</td>
<td>The frequency in ms that the consumer offsets are committed to zookeeper.</td>
</tr>
<tr>
<td>queued.max.message.chunks</td>
<tdcolspan="1">2</td>
<td>Max number of message chunks buffered for consumption. Each chunk can be up to fetch.message.max.bytes.</td>
</tr>
<tr>
<td>rebalance.max.retries</td>
<tdcolspan="1">4</td>
<td>When 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.</td>
</tr>
<tr>
<td>fetch.min.bytes</td>
<tdcolspan="1">1</td>
<td>The 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.</td>
</tr>
<tr>
<td>fetch.wait.max.ms</td>
<tdcolspan="1">100</td>
<td>The 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</td>
</tr>
<tr>
<td>rebalance.backoff.ms</td>
<td>2000</td>
<td>Backoff time between retries during rebalance.</td>
</tr>
<tr>
<td>refresh.leader.backoff.ms</td>
<tdcolspan="1">200</td>
<td>Backoff time to wait before trying to determine the leader of a partition that has just lost its leader.</td>
</tr>
<tr>
<td>auto.offset.reset</td>
<tdcolspan="1">largest</td>
<td>
<p>What to do when there is no initial offset in ZooKeeper or if an offset is out of range:<br/>* smallest : automatically reset the offset to the smallest offset<br/>* largest : automatically reset the offset to the largest offset<br/>* anything else: throw exception to the consumer</p>
</td>
</tr>
<tr>
<td>consumer.timeout.ms</td>
<tdcolspan="1">-1</td>
<td>Throw a timeout exception to the consumer if no message is available for consumption after the specified interval</td>
</tr>
<tr>
<td>exclude.internal.topics</td>
<tdcolspan="1">true</td>
<td>Whether messages from internal topics (such as offsets) should be exposed to the consumer.</td>
</tr>
<tr>
<td>partition.assignment.strategy</td>
<tdcolspan="1">range</td>
<td>Select a strategy for assigning partitions to consumer streams. Possible values: range, roundrobin.</td>
</tr>
<tr>
<td>client.id</td>
<tdcolspan="1">group id value</td>
<td>The client id is a user-specified string sent in each request to help trace calls. It should logically identify the application making the request.</td>
</tr>
<tr>
<td>zookeeper.session.timeout.ms</td>
<tdcolspan="1">6000</td>
<td>ZooKeeper session timeout. If the consumer fails to heartbeat to ZooKeeper for this period of time it is considered dead and a rebalance will occur.</td>
</tr>
<tr>
<td>zookeeper.connection.timeout.ms</td>
<tdcolspan="1">6000</td>
<td>The max time that the client waits while establishing a connection to zookeeper.</td>
</tr>
<tr>
<td>zookeeper.sync.time.ms</td>
<tdcolspan="1">2000</td>
<td>How far a ZK follower can be behind a ZK leader</td>
</tr>
<tr>
<td>offsets.storage</td>
<tdcolspan="1">zookeeper</td>
<td>Select where offsets should be stored (zookeeper or kafka).</td>
</tr>
<tr>
<td>offsets.channel.backoff.ms</td>
<tdcolspan="1">1000</td>
<td>The backoff period when reconnecting the offsets channel or retrying failed offset fetch/commit requests.</td>
</tr>
<tr>
<td>offsets.channel.socket.timeout.ms</td>
<tdcolspan="1">10000</td>
<td>Socket timeout when reading responses for offset fetch/commit requests. This timeout is also used for ConsumerMetadata requests that are used to query for the offset manager.</td>
</tr>
<tr>
<td>offsets.commit.max.retries</td>
<tdcolspan="1">5</td>
<td>Retry the offset commit up to this many times on failure. This retry count only applies to offset commits during shut-down. It does not apply to commits originating from the auto-commit thread. It also does not apply to attempts to query for the offset coordinator before committing offsets. i.e., if a consumer metadata request fails for any reason, it will be retried and that retry does not count toward this limit.</td>
</tr>
<tr>
<td>dual.commit.enabled</td>
<tdcolspan="1">true</td>
<td>If you are using "kafka" as offsets.storage, you can dual commit offsets to ZooKeeper (in addition to Kafka). This is required during migration from zookeeper-based offset storage to kafka-based offset storage. With respect to any given consumer group, it is safe to turn this off after all instances within that group have been migrated to the new version that commits offsets to the broker (instead of directly to ZooKeeper).</td>
</tr>
<tr>
<td>partition.assignment.strategy</td>
<tdcolspan="1">range</td>
<td><p>Select between the "range" or "roundrobin" strategy for assigning partitions to consumer streams.<p>The round-robin partition assignor lays out all the available partitions and all the available consumer threads. It then proceeds to do a round-robin assignment from partition to consumer thread. If the subscriptions of all consumer instances are identical, then the partitions will be uniformly distributed. (i.e., the partition ownership counts will be within a delta of exactly one across all consumer threads.) Round-robin assignment is permitted only if: (a) Every topic has the same number of streams within a consumer instance (b) The set of subscribed topics is identical for every consumer instance within the group.<p> Range partitioning works on a per-topic basis. For each topic, we lay out the available partitions in numeric order and the consumer threads in lexicographic order. We then divide the number of partitions by the total number of consumer streams (threads) to determine the number of partitions to assign to each consumer. If it does not evenly divide, then the first few consumers will have one extra partition.</td>
</tr>
</tbody>
</table>
<p>More details about consumer configuration can be found in the scala class <code>kafka.consumer.ConsumerConfig</code>.</p>
<h3><aid="newconsumerconfigs">3.3 New Consumer Configs</a></h3>
Since 0.9.0.0 we have been working on a replacement for our existing simple and high-level consumers. The code can be considered beta quality. Below is the configuration for the new consumer: </p>
We recommend using KafkaProducer as SyncProducer and AsyncProducer will be deprecated in the next release. Below is the configuration for the new producer:</p>
<!--#include virtual="producer_config.html" -->
<h3><aid="oldproducerconfigs">3.5 Old Producer Configs</a></h3>
We recommend not to use the Scala SyncProducer and AsyncProducer as they will be deprecated in the next release. Below you can see their configuration:
<p>This is for bootstrapping and the producer will only use it for getting metadata (topics, partitions and replicas). The socket connections for sending the actual data will be established based on the broker information returned in the metadata. The format is host1:port1,host2:port2, and the list can be a subset of brokers or a VIP pointing to a subset of brokers.</p>
</td>
</tr>
<tr>
<td>request.required.acks</td>
<tdcolspan="1">0</td>
<td>
<p>This value controls when a produce request is considered completed. Specifically, how many other brokers must have committed the data to their log and acknowledged this to the leader? Typical values are
<ul>
<li>0, which means that the producer never waits for an acknowledgement from the broker (the same behavior as 0.7). This option provides the lowest latency but the weakest durability guarantees (some data will be lost when a server fails).
<li> 1, which means that the producer gets an acknowledgement after the leader replica has received the data. This option provides better durability as the client waits until the server acknowledges the request as successful (only messages that were written to the now-dead leader but not yet replicated will be lost).
<li> -1, The producer gets an acknowledgement after all in-sync replicas have received the data. This option provides the greatest level of durability. However, it does not completely eliminate the risk of message loss because the number of in sync replicas may, in rare cases, shrink to 1. If you want to ensure that some minimum number of replicas (typically a majority) receive a write, then you must set the topic-level min.insync.replicas setting. Please read the Replication section of the design documentation for a more in-depth discussion.
</ul>
</p>
</td>
</tr>
<tr>
<td>request.timeout.ms</td>
<tdcolspan="1">10000</td>
<td>The amount of time the broker will wait trying to meet the request.required.acks requirement before sending back an error to the client.</td>
</tr>
<tr>
<td>producer.type</td>
<tdcolspan="1">sync</td>
<td>
<p>This parameter specifies whether the messages are sent asynchronously in a background thread. Valid values are (1) async for asynchronous send and (2) sync for synchronous send. By setting the producer to async we allow batching together of requests (which is great for throughput) but open the possibility of a failure of the client machine dropping unsent data.</p>
<td>The partitioner class for partitioning messages amongst sub-topics. The default partitioner is based on the hash of the key.</td>
</tr>
<tr>
<td>compression.codec</td>
<tdcolspan="1">none</td>
<td>
<p>This parameter allows you to specify the compression codec for all data generated by this producer. Valid values are "none", "gzip" and "snappy".</p>
</td>
</tr>
<tr>
<td>compressed.topics</td>
<tdcolspan="1">null</td>
<td>
<p>This parameter allows you to set whether compression should be turned on for particular topics. If the compression codec is anything other than NoCompressionCodec, enable compression only for specified topics if any. If the list of compressed topics is empty, then enable the specified compression codec for all topics. If the compression codec is NoCompressionCodec, compression is disabled for all topics</p>
</td>
</tr>
<tr>
<td>message.send.max.retries</td>
<tdcolspan="1">3</td>
<td>
<p>This property will cause the producer to automatically retry a failed send request. This property specifies the number of retries when such failures occur. Note that setting a non-zero value here can lead to duplicates in the case of network errors that cause a message to be sent but the acknowledgement to be lost.</p>
</td>
</tr>
<tr>
<td>retry.backoff.ms</td>
<tdcolspan="1">100</td>
<td>
<p>Before each retry, the producer refreshes the metadata of relevant topics to see if a new leader has been elected. Since leader election takes a bit of time, this property specifies the amount of time that the producer waits before refreshing the metadata.</p>
</td>
</tr>
<tr>
<td>topic.metadata.refresh.interval.ms</td>
<tdcolspan="1">600 * 1000</td>
<td>
<p>The producer generally refreshes the topic metadata from brokers when there is a failure (partition missing, leader not available...). It will also poll regularly (default: every 10min so 600000ms). If you set this to a negative value, metadata will only get refreshed on failure. If you set this to zero, the metadata will get refreshed after each message sent (not recommended). Important note: the refresh happen only AFTER the message is sent, so if the producer never sends a message the metadata is never refreshed</p>
</td>
</tr>
<tr>
<td>queue.buffering.max.ms</td>
<tdcolspan="1">5000</td>
<td>Maximum time to buffer data when using async mode. For example a setting of 100 will try to batch together 100ms of messages to send at once. This will improve throughput but adds message delivery latency due to the buffering.</td>
</tr>
<tr>
<td>queue.buffering.max.messages</td>
<tdcolspan="1">10000</td>
<td>The maximum number of unsent messages that can be queued up the producer when using async mode before either the producer must be blocked or data must be dropped.</td>
</tr>
<tr>
<td>queue.enqueue.timeout.ms</td>
<tdcolspan="1">-1</td>
<td>
<p>The amount of time to block before dropping messages when running in async mode and the buffer has reached queue.buffering.max.messages. If set to 0 events will be enqueued immediately or dropped if the queue is full (the producer send call will never block). If set to -1 the producer will block indefinitely and never willingly drop a send.</p>
</td>
</tr>
<tr>
<td>batch.num.messages</td>
<tdcolspan="1">200</td>
<td>The number of messages to send in one batch when using async mode. The producer will wait until either this number of messages are ready to send or queue.buffer.max.ms is reached.</td>
</tr>
<tr>
<td>send.buffer.bytes</td>
<tdcolspan="1">100 * 1024</td>
<td>Socket write buffer size</td>
</tr>
<tr>
<td>client.id</td>
<tdcolspan="1">""</td>
<td>The client id is a user-specified string sent in each request to help trace calls. It should logically identify the application making the request.</td>