- 3.2 Topic-Level Configs
- cleanup.policy
- compression.type
- delete.retention.ms
- file.delete.delay.ms
- flush.messages
- flush.ms
- follower.replication.throttled.replicas
- index.interval.bytes
- leader.replication.throttled.replicas
- max.compaction.lag.ms
- max.message.bytes
- message.format.version
- message.timestamp.difference.max.ms
- message.timestamp.type
- min.cleanable.dirty.ratio
- min.compaction.lag.ms
- min.insync.replicas
- preallocate
- retention.bytes
- retention.ms
- segment.bytes
- segment.index.bytes
- segment.jitter.ms
- segment.ms
- unclean.leader.election.enable
- message.downconversion.enable
3.2 Topic-Level Configs
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 --bootstrap-server localhost:9092 --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 --bootstrap-server localhost:9092 --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 --bootstrap-server localhost:9092 --entity-type topics --entity-name my-topic --describe
To remove an override you can do
> bin/kafka-configs.sh --bootstrap-server localhost:9092 --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.
cleanup.policy
A string that is either “delete” or “compact” or both. 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 log compaction on the topic.
Type: list Default: delete Valid Values: [compact, delete] Server Default Property: log.cleanup.policy Importance: medium compression.type
Specify the final compression type for a given topic. This configuration accepts the standard compression codecs (‘gzip’, ‘snappy’, ‘lz4’, ‘zstd’). It additionally accepts ‘uncompressed’ which is equivalent to no compression; and ‘producer’ which means retain the original compression codec set by the producer.
Type: string Default: producer Valid Values: [uncompressed, zstd, lz4, snappy, gzip, producer] Server Default Property: compression.type Importance: medium delete.retention.ms
The amount of time to retain delete tombstone markers for log compacted 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).
Type: long Default: 86400000 (1 day) Valid Values: [0,…] Server Default Property: log.cleaner.delete.retention.ms Importance: medium file.delete.delay.ms
The time to wait before deleting a file from the filesystem
Type: long Default: 60000 (1 minute) Valid Values: [0,…] Server Default Property: log.segment.delete.delay.ms Importance: medium flush.messages
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 the per-topic configuration section).
Type: long Default: 9223372036854775807 Valid Values: [0,…] Server Default Property: log.flush.interval.messages Importance: medium flush.ms
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.
Type: long Default: 9223372036854775807 Valid Values: [0,…] Server Default Property: log.flush.interval.ms Importance: medium follower.replication.throttled.replicas
A list of replicas for which log replication should be throttled on the follower side. The list should describe a set of replicas in the form [PartitionId]:[BrokerId],[PartitionId]:[BrokerId]:… or alternatively the wildcard ‘*‘ can be used to throttle all replicas for this topic.
Type: list Default: “” Valid Values: [partitionId]:[brokerId],[partitionId]:[brokerId],… Server Default Property: follower.replication.throttled.replicas Importance: medium index.interval.bytes
This setting controls how frequently Kafka adds an index entry to its 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.
Type: int Default: 4096 (4 kibibytes) Valid Values: [0,…] Server Default Property: log.index.interval.bytes Importance: medium leader.replication.throttled.replicas
A list of replicas for which log replication should be throttled on the leader side. The list should describe a set of replicas in the form [PartitionId]:[BrokerId],[PartitionId]:[BrokerId]:… or alternatively the wildcard ‘*‘ can be used to throttle all replicas for this topic.
Type: list Default: “” Valid Values: [partitionId]:[brokerId],[partitionId]:[brokerId],… Server Default Property: leader.replication.throttled.replicas Importance: medium max.compaction.lag.ms
The maximum time a message will remain ineligible for compaction in the log. Only applicable for logs that are being compacted.
Type: long Default: 9223372036854775807 Valid Values: [1,…] Server Default Property: log.cleaner.max.compaction.lag.ms Importance: medium max.message.bytes
The largest record batch size allowed by Kafka (after compression if compression is enabled). If this is increased and there are consumers older than 0.10.2, the consumers’ fetch size must also be increased so that they can fetch record batches this large. In the latest message format version, records are always grouped into batches for efficiency. In previous message format versions, uncompressed records are not grouped into batches and this limit only applies to a single record in that case.
Type: int Default: 1048588 Valid Values: [0,…] Server Default Property: message.max.bytes Importance: medium message.format.version
Specify the message format version the broker will use to append messages to the logs. The value should be a valid ApiVersion. Some examples are: 0.8.2, 0.9.0.0, 0.10.0, check ApiVersion for more details. By setting a particular message format version, the user is certifying that all the existing messages on disk are smaller or equal than the specified version. Setting this value incorrectly will cause consumers with older versions to break as they will receive messages with a format that they don’t understand.
Type: string Default: 2.7-IV2 Valid Values: [0.8.0, 0.8.1, 0.8.2, 0.9.0, 0.10.0-IV0, 0.10.0-IV1, 0.10.1-IV0, 0.10.1-IV1, 0.10.1-IV2, 0.10.2-IV0, 0.11.0-IV0, 0.11.0-IV1, 0.11.0-IV2, 1.0-IV0, 1.1-IV0, 2.0-IV0, 2.0-IV1, 2.1-IV0, 2.1-IV1, 2.1-IV2, 2.2-IV0, 2.2-IV1, 2.3-IV0, 2.3-IV1, 2.4-IV0, 2.4-IV1, 2.5-IV0, 2.6-IV0, 2.7-IV0, 2.7-IV1, 2.7-IV2] Server Default Property: log.message.format.version Importance: medium message.timestamp.difference.max.ms
The maximum difference allowed between the timestamp when a broker receives a message and the timestamp specified in the message. If message.timestamp.type=CreateTime, a message will be rejected if the difference in timestamp exceeds this threshold. This configuration is ignored if message.timestamp.type=LogAppendTime.
Type: long Default: 9223372036854775807 Valid Values: [0,…] Server Default Property: log.message.timestamp.difference.max.ms Importance: medium message.timestamp.type
Define whether the timestamp in the message is message create time or log append time. The value should be either `CreateTime` or `LogAppendTime`
Type: string Default: CreateTime Valid Values: [CreateTime, LogAppendTime] Server Default Property: log.message.timestamp.type Importance: medium min.cleanable.dirty.ratio
This configuration controls how frequently the log compactor will attempt to clean the log (assuming log compaction 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. If the max.compaction.lag.ms or the min.compaction.lag.ms configurations are also specified, then the log compactor considers the log to be eligible for compaction as soon as either: (i) the dirty ratio threshold has been met and the log has had dirty (uncompacted) records for at least the min.compaction.lag.ms duration, or (ii) if the log has had dirty (uncompacted) records for at most the max.compaction.lag.ms period.
Type: double Default: 0.5 Valid Values: [0,…,1] Server Default Property: log.cleaner.min.cleanable.ratio Importance: medium min.compaction.lag.ms
The minimum time a message will remain uncompacted in the log. Only applicable for logs that are being compacted.
Type: long Default: 0 Valid Values: [0,…] Server Default Property: log.cleaner.min.compaction.lag.ms Importance: medium min.insync.replicas
When a producer sets acks to “all” (or “-1”), this configuration 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).
When used together,min.insync.replicas
andacks
allow you to enforce greater durability guarantees. A typical scenario would be to create a topic with a replication factor of 3, setmin.insync.replicas
to 2, and produce withacks
of “all”. This will ensure that the producer raises an exception if a majority of replicas do not receive a write.Type: int Default: 1 Valid Values: [1,…] Server Default Property: min.insync.replicas Importance: medium preallocate
True if we should preallocate the file on disk when creating a new log segment.
Type: boolean Default: false Valid Values: Server Default Property: log.preallocate Importance: medium retention.bytes
This configuration controls the maximum size a partition (which consists of log segments) 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. Since this limit is enforced at the partition level, multiply it by the number of partitions to compute the topic retention in bytes.
Type: long Default: -1 Valid Values: Server Default Property: log.retention.bytes Importance: medium retention.ms
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. If set to -1, no time limit is applied.
Type: long Default: 604800000 (7 days) Valid Values: [-1,…] Server Default Property: log.retention.ms Importance: medium segment.bytes
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.
Type: int Default: 1073741824 (1 gibibyte) Valid Values: [14,…] Server Default Property: log.segment.bytes Importance: medium segment.index.bytes
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.
Type: int Default: 10485760 (10 mebibytes) Valid Values: [0,…] Server Default Property: log.index.size.max.bytes Importance: medium segment.jitter.ms
The maximum random jitter subtracted from the scheduled segment roll time to avoid thundering herds of segment rolling
Type: long Default: 0 Valid Values: [0,…] Server Default Property: log.roll.jitter.ms Importance: medium segment.ms
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.
Type: long Default: 604800000 (7 days) Valid Values: [1,…] Server Default Property: log.roll.ms Importance: medium unclean.leader.election.enable
Indicates whether to enable replicas not in the ISR set to be elected as leader as a last resort, even though doing so may result in data loss.
Type: boolean Default: false Valid Values: Server Default Property: unclean.leader.election.enable Importance: medium message.downconversion.enable
This configuration controls whether down-conversion of message formats is enabled to satisfy consume requests. When set to
false
, broker will not perform down-conversion for consumers expecting an older message format. The broker responds withUNSUPPORTED_VERSION
error for consume requests from such older clients. This configurationdoes not apply to any message format conversion that might be required for replication to followers.Type: boolean Default: true Valid Values: Server Default Property: log.message.downconversion.enable Importance: low