mirror of https://github.com/apache/kafka.git
KAFKA-13610: Deprecate log.cleaner.enable configuration (#19472)
JIRA: KAFKA-13610 This patch deprecates the `log.cleaner.enable` configuration. It's part of [KIP-1148](https://cwiki.apache.org/confluence/x/XAyWF). Reviewers: Chia-Ping Tsai <chia7712@gmail.com>, PoAn Yang <payang@apache.org>, Ken Huang <s7133700@gmail.com>, Jhen-Yung Hsu <jhenyunghsu@gmail.com>
This commit is contained in:
parent
fb2ce76b49
commit
73afcc9b69
|
@ -341,7 +341,6 @@ class KafkaConfig private(doLog: Boolean, val props: util.Map[_, _])
|
|||
def logCleanerMaxCompactionLagMs = getLong(CleanerConfig.LOG_CLEANER_MAX_COMPACTION_LAG_MS_PROP)
|
||||
def logCleanerBackoffMs = getLong(CleanerConfig.LOG_CLEANER_BACKOFF_MS_PROP)
|
||||
def logCleanerMinCleanRatio = getDouble(CleanerConfig.LOG_CLEANER_MIN_CLEAN_RATIO_PROP)
|
||||
val logCleanerEnable = getBoolean(CleanerConfig.LOG_CLEANER_ENABLE_PROP)
|
||||
def logIndexSizeMaxBytes = getInt(ServerLogConfigs.LOG_INDEX_SIZE_MAX_BYTES_CONFIG)
|
||||
def logIndexIntervalBytes = getInt(ServerLogConfigs.LOG_INDEX_INTERVAL_BYTES_CONFIG)
|
||||
def logDeleteDelayMs = getLong(ServerLogConfigs.LOG_DELETE_DELAY_MS_CONFIG)
|
||||
|
|
|
@ -32,6 +32,14 @@
|
|||
</li>
|
||||
</ul>
|
||||
</li>
|
||||
<li><b>Broker</b>
|
||||
<ul>
|
||||
<li>
|
||||
The configuration <code>log.cleaner.enable</code> is deprecated. Users should no longer set it to <code>false</code> to prepare for future removal.
|
||||
For further details, please refer to <a href="https://cwiki.apache.org/confluence/x/XAyWF">KIP-1148</a>.
|
||||
</li>
|
||||
</ul>
|
||||
</li>
|
||||
<li><b>Producer</b>
|
||||
<ul>
|
||||
<li>The <code>flush</code> method now detects potential deadlocks and prohibits its use inside a callback. This change prevents unintended blocking behavior, which was a known risk in earlier versions.
|
||||
|
|
|
@ -50,6 +50,7 @@ public class CleanerConfig {
|
|||
public static final String LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_PROP = ServerTopicConfigSynonyms.LOG_CLEANER_PREFIX + "io.buffer.load.factor";
|
||||
public static final String LOG_CLEANER_BACKOFF_MS_PROP = ServerTopicConfigSynonyms.LOG_CLEANER_PREFIX + "backoff.ms";
|
||||
public static final String LOG_CLEANER_MIN_CLEAN_RATIO_PROP = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG);
|
||||
@Deprecated(since = "4.1", forRemoval = true)
|
||||
public static final String LOG_CLEANER_ENABLE_PROP = ServerTopicConfigSynonyms.LOG_CLEANER_PREFIX + "enable";
|
||||
public static final String LOG_CLEANER_DELETE_RETENTION_MS_PROP = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.DELETE_RETENTION_MS_CONFIG);
|
||||
public static final String LOG_CLEANER_MIN_COMPACTION_LAG_MS_PROP = ServerTopicConfigSynonyms.serverSynonym(TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG);
|
||||
|
@ -68,7 +69,9 @@ public class CleanerConfig {
|
|||
public static final String LOG_CLEANER_DEDUPE_BUFFER_LOAD_FACTOR_DOC = "Log cleaner dedupe buffer load factor. The percentage full the dedupe buffer can become. A higher value " +
|
||||
"will allow more log to be cleaned at once but will lead to more hash collisions";
|
||||
public static final String LOG_CLEANER_BACKOFF_MS_DOC = "The amount of time to sleep when there are no logs to clean";
|
||||
public static final String LOG_CLEANER_ENABLE_DOC = "Enable the log cleaner process to run on the server. Should be enabled if using any topics with a cleanup.policy=compact including the internal offsets topic. If disabled those topics will not be compacted and continually grow in size.";
|
||||
@Deprecated(since = "4.1", forRemoval = true)
|
||||
public static final String LOG_CLEANER_ENABLE_DOC = "This configuration has been deprecated and will be removed in Kafka 5.0. Users should not set it to false to prepare for its future removal. " +
|
||||
"Enable the log cleaner process to run on the server. Should be enabled if using any topics with a cleanup.policy=compact including the internal offsets topic. If disabled those topics will not be compacted and continually grow in size.";
|
||||
public static final String LOG_CLEANER_DELETE_RETENTION_MS_DOC = "The amount of time to retain tombstone message 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 " +
|
||||
"tombstones messages may be collected before a consumer completes their scan).";
|
||||
|
|
Loading…
Reference in New Issue