satishd commented on code in PR #13049:
URL: https://github.com/apache/kafka/pull/13049#discussion_r1062049275


##########
storage/src/main/java/org/apache/kafka/server/log/internals/LogConfig.java:
##########
@@ -0,0 +1,504 @@
+/*
+ * 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.
+ */
+package org.apache.kafka.server.log.internals;
+
+import static java.util.Arrays.asList;
+import static org.apache.kafka.common.config.ConfigDef.Range.between;
+import static org.apache.kafka.common.config.ConfigDef.Type.BOOLEAN;
+import static org.apache.kafka.common.config.ConfigDef.Type.DOUBLE;
+import static org.apache.kafka.common.config.ConfigDef.Type.LIST;
+import static org.apache.kafka.common.config.ConfigDef.Type.LONG;
+import static org.apache.kafka.common.config.ConfigDef.Type.STRING;
+import static org.apache.kafka.server.common.MetadataVersion.IBP_3_0_IV1;
+import static org.apache.kafka.common.config.ConfigDef.Importance.LOW;
+import static org.apache.kafka.common.config.ConfigDef.Importance.MEDIUM;
+import static org.apache.kafka.common.config.ConfigDef.Range.atLeast;
+import static org.apache.kafka.common.config.ConfigDef.Type.INT;
+import static org.apache.kafka.common.config.ConfigDef.ValidString.in;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.stream.Collectors;
+import org.apache.kafka.common.config.AbstractConfig;
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigDef.ConfigKey;
+import org.apache.kafka.common.config.ConfigDef.Type;
+import org.apache.kafka.common.config.ConfigDef.ValidList;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.config.TopicConfig;
+import org.apache.kafka.common.errors.InvalidConfigurationException;
+import org.apache.kafka.common.record.LegacyRecord;
+import org.apache.kafka.common.record.RecordVersion;
+import org.apache.kafka.common.record.Records;
+import org.apache.kafka.common.record.TimestampType;
+import org.apache.kafka.common.utils.ConfigUtils;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.server.common.MetadataVersion;
+import org.apache.kafka.server.common.MetadataVersionValidator;
+import org.apache.kafka.server.config.ServerTopicConfigSynonyms;
+import org.apache.kafka.server.record.BrokerCompressionType;
+
+public class LogConfig extends AbstractConfig {
+
+    public static class MessageFormatVersion {
+        private final String messageFormatVersionString;
+        private final String interBrokerProtocolVersionString;
+        private final MetadataVersion messageFormatVersion;
+        private final MetadataVersion interBrokerProtocolVersion;
+
+        public MessageFormatVersion(String messageFormatVersionString, String 
interBrokerProtocolVersionString) {
+            this.messageFormatVersionString = messageFormatVersionString;
+            this.interBrokerProtocolVersionString = 
interBrokerProtocolVersionString;
+            this.messageFormatVersion = 
MetadataVersion.fromVersionString(messageFormatVersionString);
+            this.interBrokerProtocolVersion = 
MetadataVersion.fromVersionString(interBrokerProtocolVersionString);
+        }
+
+        public MetadataVersion messageFormatVersion() {
+            return messageFormatVersion;
+        }
+
+        public MetadataVersion interBrokerProtocolVersion() {
+            return interBrokerProtocolVersion;
+        }
+
+        public boolean shouldIgnore() {
+            return 
shouldIgnoreMessageFormatVersion(interBrokerProtocolVersion);
+        }
+
+        public boolean shouldWarn() {
+            return interBrokerProtocolVersion.isAtLeast(IBP_3_0_IV1)
+                && 
messageFormatVersion.highestSupportedRecordVersion().precedes(RecordVersion.V2);
+        }
+
+        @SuppressWarnings("deprecation")
+        public String topicWarningMessage(String topicName) {
+            return "Topic configuration " + 
TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG + " with value `"
+                + messageFormatVersionString + "` is ignored for `" + 
topicName + "` because the "
+                + "inter-broker protocol version `" + 
interBrokerProtocolVersionString + "` is greater or "
+                + "equal than 3.0. This configuration is deprecated and it 
will be removed in Apache Kafka 4.0.";
+        }
+    }
+
+    public static class RemoteLogConfig {
+
+        public final boolean remoteStorageEnable;
+
+        public final long localRetentionMs;
+        public final long localRetentionBytes;
+
+        private RemoteLogConfig(LogConfig config, long retentionMs, long 
retentionSize) {
+            this.remoteStorageEnable = 
config.getBoolean(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG);
+
+            long localLogRetentionMs = 
config.getLong(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG);
+
+            // -2 indicates to derive value from retentionMs property.
+            if (localLogRetentionMs == -2)
+                this.localRetentionMs = retentionMs;
+            else {
+                // Added validation here to check the effective value should 
not be more than RetentionMs.
+                if (localLogRetentionMs == -1 && retentionMs != -1)
+                    throw new 
ConfigException(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, localLogRetentionMs,
+                        "Value must not be -1 as " + 
TopicConfig.RETENTION_MS_CONFIG + " value is set as " + retentionMs);
+
+                if (localLogRetentionMs > retentionMs)
+                    throw new 
ConfigException(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, localLogRetentionMs,
+                        "Value must not be more than property: " + 
TopicConfig.RETENTION_MS_CONFIG + " value.");
+
+                this.localRetentionMs = localLogRetentionMs;
+            }
+
+            long localLogRetentionBytes = 
config.getLong(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG);
+
+            // -2 indicates to derive value from retentionSize property.
+            if (localLogRetentionBytes == -2)
+                this.localRetentionBytes = retentionSize;
+            else {
+                // Added validation here to check the effective value should 
not be more than RetentionBytes.
+                if (localLogRetentionBytes == -1 && retentionSize != -1)
+                    throw new 
ConfigException(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, 
localLogRetentionBytes,
+                        "Value must not be -1 as " + 
TopicConfig.RETENTION_BYTES_CONFIG + " value is set as " + retentionSize);
+
+                if (localLogRetentionBytes > retentionSize)
+                    throw new 
ConfigException(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, 
localLogRetentionBytes,
+                        "Value must not be more than property: " + 
TopicConfig.RETENTION_BYTES_CONFIG + " value.");
+
+                this.localRetentionBytes = localLogRetentionBytes;
+            }
+        }
+    }
+
+    // Visible for testing
+    public static class LogConfigDef extends ConfigDef {
+        public LogConfigDef() {
+            this(new ConfigDef());
+        }
+
+        public LogConfigDef(ConfigDef base) {
+            super(base);
+        }
+
+        @Override
+        public List<String> headers() {
+            return asList("Name", "Description", "Type", "Default", "Valid 
Values", SERVER_DEFAULT_HEADER_NAME, "Importance");
+        }
+
+        // Visible for testing
+        @Override
+        public String getConfigValue(ConfigKey key, String headerName) {
+            if (headerName.equals(SERVER_DEFAULT_HEADER_NAME))
+                return 
ServerTopicConfigSynonyms.TOPIC_CONFIG_SYNONYMS.get(key.name);
+            else
+                return super.getConfigValue(key, headerName);
+        }
+
+        public Optional<String> serverConfigName(String configName) {
+            return 
Optional.ofNullable(ServerTopicConfigSynonyms.TOPIC_CONFIG_SYNONYMS.get(configName));
+        }
+    }
+
+    // Visible for testing
+    public static final String SERVER_DEFAULT_HEADER_NAME = "Server Default 
Property";
+
+    public static final int DEFAULT_MAX_MESSAGE_BYTES = 1024 * 1024 + 
Records.LOG_OVERHEAD;
+    public static final int DEFAULT_SEGMENT_BYTES = 1024 * 1024 * 1024;
+    public static final long DEFAULT_SEGMENT_MS = 24 * 7 * 60 * 60 * 1000L;
+    public static final long DEFAULT_SEGMENT_JITTER_MS = 0;
+    public static final long DEFAULT_RETENTION_MS = 24 * 7 * 60 * 60 * 1000L;
+    public static final long DEFAULT_RETENTION_BYTES = -1L;
+    public static final int DEFAULT_SEGMENT_INDEX_BYTES = 10 * 1024 * 1024;
+    public static final int DEFAULT_INDEX_INTERVAL_BYTES = 4096;
+    public static final long DEFAULT_FILE_DELETE_DELAY_MS = 60000L;
+    public static final String DEFAULT_CLEANUP_POLICY = 
TopicConfig.CLEANUP_POLICY_DELETE;
+    public static final long DEFAULT_FLUSH_MESSAGES_INTERVAL = Long.MAX_VALUE;
+    public static final long DEFAULT_FLUSH_MS = Long.MAX_VALUE;
+    public static final long DEFAULT_DELETE_RETENTION_MS = 24 * 60 * 60 * 
1000L;
+    public static final long DEFAULT_MIN_COMPACTION_LAG_MS = 0;
+    public static final long DEFAULT_MAX_COMPACTION_LAG_MS = Long.MAX_VALUE;
+    public static final double DEFAULT_MIN_CLEANABLE_DIRTY_RATIO = 0.5;
+    public static final boolean DEFAULT_UNCLEAN_LEADER_ELECTION_ENABLE = false;
+    public static final int DEFAULT_MIN_IN_SYNC_REPLICAS = 1;
+    public static final String DEFAULT_COMPRESSION_TYPE = 
BrokerCompressionType.PRODUCER.name;
+    public static final boolean DEFAULT_PREALLOCATE = false;
+    public static final String DEFAULT_MESSAGE_TIMESTAMP_TYPE = "CreateTime";
+    public static final long DEFAULT_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS = 
Long.MAX_VALUE;
+    public static final boolean DEFAULT_MESSAGE_DOWNCONVERSION_ENABLE = true;
+
+    public static final boolean DEFAULT_REMOTE_STORAGE_ENABLE = false;
+    public static final int DEFAULT_LOCAL_RETENTION_BYTES = -2; // It 
indicates the value to be derived from RetentionBytes
+    public static final int DEFAULT_LOCAL_RETENTION_MS = -2; // It indicates 
the value to be derived from RetentionMs
+    public static final List<String> 
DEFAULT_LEADER_REPLICATION_THROTTLED_REPLICAS = Collections.emptyList();
+    public static final List<String> 
DEFAULT_FOLLOWER_REPLICATION_THROTTLED_REPLICAS = Collections.emptyList();
+
+    /* See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for details */
+    @Deprecated
+    public static final String DEFAULT_MESSAGE_FORMAT_VERSION = 
IBP_3_0_IV1.version();
+
+    // Leave these out of TopicConfig for now as they are replication quota 
configs
+    public static final String LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG = 
"leader.replication.throttled.replicas";
+    public static final String FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG 
= "follower.replication.throttled.replicas";
+
+    @SuppressWarnings("deprecation")
+    private static final String MESSAGE_FORMAT_VERSION_CONFIG = 
TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG;
+
+    // Visible for testing
+    public static final Set<String> CONFIGS_WITH_NO_SERVER_DEFAULTS = 
Collections.unmodifiableSet(Utils.mkSet(
+        TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG,
+        TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG,
+        TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG,
+        LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG,
+        FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG
+    ));
+
+    public static final String LEADER_REPLICATION_THROTTLED_REPLICAS_DOC = "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.";
+    public static final String FOLLOWER_REPLICATION_THROTTLED_REPLICAS_DOC = 
"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.";
+
+    @SuppressWarnings("deprecation")
+    private static final String MESSAGE_FORMAT_VERSION_DOC = 
TopicConfig.MESSAGE_FORMAT_VERSION_DOC;
+
+    @SuppressWarnings("deprecation")
+    private static final LogConfigDef CONFIG = new LogConfigDef();
+    static {
+        CONFIG.
+            define(TopicConfig.SEGMENT_BYTES_CONFIG, INT, 
DEFAULT_SEGMENT_BYTES, atLeast(LegacyRecord.RECORD_OVERHEAD_V0), MEDIUM,
+                TopicConfig.SEGMENT_BYTES_DOC)
+            .define(TopicConfig.SEGMENT_MS_CONFIG, LONG, DEFAULT_SEGMENT_MS, 
atLeast(1), MEDIUM, TopicConfig.SEGMENT_MS_DOC)
+            .define(TopicConfig.SEGMENT_JITTER_MS_CONFIG, LONG, 
DEFAULT_SEGMENT_JITTER_MS, atLeast(0), MEDIUM,
+                TopicConfig.SEGMENT_JITTER_MS_DOC)
+            .define(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG, INT, 
DEFAULT_SEGMENT_INDEX_BYTES, atLeast(4), MEDIUM,
+                TopicConfig.SEGMENT_INDEX_BYTES_DOC)
+            .define(TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG, LONG, 
DEFAULT_FLUSH_MESSAGES_INTERVAL, atLeast(1), MEDIUM,
+                TopicConfig.FLUSH_MESSAGES_INTERVAL_DOC)
+            .define(TopicConfig.FLUSH_MS_CONFIG, LONG, DEFAULT_FLUSH_MS, 
atLeast(0), MEDIUM,
+                TopicConfig.FLUSH_MS_DOC)
+            // can be negative. See 
kafka.log.LogManager.cleanupSegmentsToMaintainSize
+            .define(TopicConfig.RETENTION_BYTES_CONFIG, LONG, 
DEFAULT_RETENTION_BYTES, MEDIUM, TopicConfig.RETENTION_BYTES_DOC)
+            // can be negative. See kafka.log.LogManager.cleanupExpiredSegments
+            .define(TopicConfig.RETENTION_MS_CONFIG, LONG, 
DEFAULT_RETENTION_MS, atLeast(-1), MEDIUM,
+                TopicConfig.RETENTION_MS_DOC)
+            .define(TopicConfig.MAX_MESSAGE_BYTES_CONFIG, INT, 
DEFAULT_MAX_MESSAGE_BYTES, atLeast(0), MEDIUM,
+                TopicConfig.MAX_MESSAGE_BYTES_DOC)
+            .define(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG, INT, 
DEFAULT_INDEX_INTERVAL_BYTES, atLeast(0), MEDIUM,
+                TopicConfig.INDEX_INTERVAL_BYTES_DOC)
+            .define(TopicConfig.DELETE_RETENTION_MS_CONFIG, LONG, 
DEFAULT_DELETE_RETENTION_MS, atLeast(0), MEDIUM,
+                TopicConfig.DELETE_RETENTION_MS_DOC)
+            .define(TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG, LONG, 
DEFAULT_MIN_COMPACTION_LAG_MS, atLeast(0), MEDIUM,
+                TopicConfig.MIN_COMPACTION_LAG_MS_DOC)
+            .define(TopicConfig.MAX_COMPACTION_LAG_MS_CONFIG, LONG, 
DEFAULT_MAX_COMPACTION_LAG_MS, atLeast(1), MEDIUM,
+                TopicConfig.MAX_COMPACTION_LAG_MS_DOC)
+            .define(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG, LONG, 
DEFAULT_FILE_DELETE_DELAY_MS, atLeast(0), MEDIUM,
+                TopicConfig.FILE_DELETE_DELAY_MS_DOC)
+            .define(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG, DOUBLE, 
DEFAULT_MIN_CLEANABLE_DIRTY_RATIO, between(0, 1), MEDIUM,
+                TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_DOC)
+            .define(TopicConfig.CLEANUP_POLICY_CONFIG, LIST, 
DEFAULT_CLEANUP_POLICY, ValidList.in(TopicConfig.CLEANUP_POLICY_COMPACT,
+                TopicConfig.CLEANUP_POLICY_DELETE), MEDIUM, 
TopicConfig.CLEANUP_POLICY_DOC)
+            .define(TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG, 
BOOLEAN, DEFAULT_UNCLEAN_LEADER_ELECTION_ENABLE,
+                MEDIUM, TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_DOC)
+            .define(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG, INT, 
DEFAULT_MIN_IN_SYNC_REPLICAS, atLeast(1), MEDIUM,
+                TopicConfig.MIN_IN_SYNC_REPLICAS_DOC)
+            .define(TopicConfig.COMPRESSION_TYPE_CONFIG, STRING, 
DEFAULT_COMPRESSION_TYPE, in(BrokerCompressionType.names().toArray(new 
String[0])),
+                MEDIUM, TopicConfig.COMPRESSION_TYPE_DOC)
+            .define(TopicConfig.PREALLOCATE_CONFIG, BOOLEAN, 
DEFAULT_PREALLOCATE, MEDIUM, TopicConfig.PREALLOCATE_DOC)
+            .define(MESSAGE_FORMAT_VERSION_CONFIG, STRING, 
DEFAULT_MESSAGE_FORMAT_VERSION, new MetadataVersionValidator(), MEDIUM,
+                MESSAGE_FORMAT_VERSION_DOC)
+            .define(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG, STRING, 
DEFAULT_MESSAGE_TIMESTAMP_TYPE,
+                in("CreateTime", "LogAppendTime"), MEDIUM, 
TopicConfig.MESSAGE_TIMESTAMP_TYPE_DOC)
+            .define(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG, 
LONG, DEFAULT_MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS,
+                atLeast(0), MEDIUM, 
TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_DOC)
+            .define(LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, LIST, 
DEFAULT_LEADER_REPLICATION_THROTTLED_REPLICAS,
+                ThrottledReplicaListValidator.INSTANCE, MEDIUM, 
LEADER_REPLICATION_THROTTLED_REPLICAS_DOC)
+            .define(FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG, LIST, 
DEFAULT_FOLLOWER_REPLICATION_THROTTLED_REPLICAS,
+                ThrottledReplicaListValidator.INSTANCE, MEDIUM, 
FOLLOWER_REPLICATION_THROTTLED_REPLICAS_DOC)
+            .define(TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_CONFIG, BOOLEAN, 
DEFAULT_MESSAGE_DOWNCONVERSION_ENABLE, LOW,
+                TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_DOC)
+            .defineInternal(TopicConfig.REMOTE_LOG_STORAGE_ENABLE_CONFIG, 
BOOLEAN, DEFAULT_REMOTE_STORAGE_ENABLE, null,
+                MEDIUM, TopicConfig.REMOTE_LOG_STORAGE_ENABLE_DOC)
+            .defineInternal(TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, LONG, 
DEFAULT_LOCAL_RETENTION_MS, atLeast(-2), MEDIUM,
+                TopicConfig.LOCAL_LOG_RETENTION_MS_DOC)
+            .defineInternal(TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, 
LONG, DEFAULT_LOCAL_RETENTION_BYTES, atLeast(-2), MEDIUM,
+                TopicConfig.LOCAL_LOG_RETENTION_BYTES_DOC);
+    }
+
+    public final Set<String> overriddenConfigs;
+
+    /*
+     * Important note: Any configuration parameter that is passed along from 
KafkaConfig to LogConfig
+     * should also be in `KafkaConfig#extractLogConfigMap`.
+     */
+    public final int segmentSize;
+    public final long segmentMs;
+    public final long segmentJitterMs;
+    public final int maxIndexSize;
+    public final long flushInterval;
+    public final long flushMs;
+    public final long retentionSize;
+    public final long retentionMs;
+    public final int indexInterval;
+    public final long fileDeleteDelayMs;
+    public final long deleteRetentionMs;
+    public final long compactionLagMs;
+    public final long maxCompactionLagMs;
+    public final double minCleanableRatio;
+    public final boolean compact;
+    public final boolean delete;
+    public final boolean uncleanLeaderElectionEnable;
+    public final int minInSyncReplicas;
+    public final String compressionType;
+    public final boolean preallocate;
+
+    /* See `TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG` for details regarding 
the deprecation */
+    @Deprecated
+    public final MetadataVersion messageFormatVersion;
+
+    public final TimestampType messageTimestampType;
+    public final long messageTimestampDifferenceMaxMs;
+    public final List<String> leaderReplicationThrottledReplicas;
+    public final List<String> followerReplicationThrottledReplicas;
+    public final boolean messageDownConversionEnable;
+    public final RemoteLogConfig remoteLogConfig;
+
+    private final int maxMessageSize;
+    private final Map<?, ?> props;
+
+    public LogConfig(Map<?, ?> props) {
+        this(props, Collections.emptySet());
+    }
+
+    @SuppressWarnings("deprecation")
+    public LogConfig(Map<?, ?> props, Set<String> overriddenConfigs) {
+        super(CONFIG, props, false);
+        this.props = Collections.unmodifiableMap(props);
+        this.overriddenConfigs = 
Collections.unmodifiableSet(overriddenConfigs);
+
+        this.segmentSize = getInt(TopicConfig.SEGMENT_BYTES_CONFIG);
+        this.segmentMs = getLong(TopicConfig.SEGMENT_MS_CONFIG);
+        this.segmentJitterMs = getLong(TopicConfig.SEGMENT_JITTER_MS_CONFIG);
+        this.maxIndexSize = getInt(TopicConfig.SEGMENT_INDEX_BYTES_CONFIG);
+        this.flushInterval = 
getLong(TopicConfig.FLUSH_MESSAGES_INTERVAL_CONFIG);
+        this.flushMs = getLong(TopicConfig.FLUSH_MS_CONFIG);
+        this.retentionSize = getLong(TopicConfig.RETENTION_BYTES_CONFIG);
+        this.retentionMs = getLong(TopicConfig.RETENTION_MS_CONFIG);
+        this.maxMessageSize = getInt(TopicConfig.MAX_MESSAGE_BYTES_CONFIG);
+        this.indexInterval = getInt(TopicConfig.INDEX_INTERVAL_BYTES_CONFIG);
+        this.fileDeleteDelayMs = 
getLong(TopicConfig.FILE_DELETE_DELAY_MS_CONFIG);
+        this.deleteRetentionMs = 
getLong(TopicConfig.DELETE_RETENTION_MS_CONFIG);
+        this.compactionLagMs = 
getLong(TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG);
+        this.maxCompactionLagMs = 
getLong(TopicConfig.MAX_COMPACTION_LAG_MS_CONFIG);
+        this.minCleanableRatio = 
getDouble(TopicConfig.MIN_CLEANABLE_DIRTY_RATIO_CONFIG);
+        this.compact = getList(TopicConfig.CLEANUP_POLICY_CONFIG).stream()
+            .map(c -> c.toLowerCase(Locale.ROOT))
+            .collect(Collectors.toList())
+            .contains(TopicConfig.CLEANUP_POLICY_COMPACT);
+        this.delete = getList(TopicConfig.CLEANUP_POLICY_CONFIG).stream()
+            .map(c -> c.toLowerCase(Locale.ROOT))
+            .collect(Collectors.toList())
+            .contains(TopicConfig.CLEANUP_POLICY_DELETE);
+        this.uncleanLeaderElectionEnable = 
getBoolean(TopicConfig.UNCLEAN_LEADER_ELECTION_ENABLE_CONFIG);
+        this.minInSyncReplicas = 
getInt(TopicConfig.MIN_IN_SYNC_REPLICAS_CONFIG);
+        this.compressionType = 
getString(TopicConfig.COMPRESSION_TYPE_CONFIG).toLowerCase(Locale.ROOT);
+        this.preallocate = getBoolean(TopicConfig.PREALLOCATE_CONFIG);
+        this.messageFormatVersion = 
MetadataVersion.fromVersionString(getString(TopicConfig.MESSAGE_FORMAT_VERSION_CONFIG));
+        this.messageTimestampType = 
TimestampType.forName(getString(TopicConfig.MESSAGE_TIMESTAMP_TYPE_CONFIG));
+        this.messageTimestampDifferenceMaxMs = 
getLong(TopicConfig.MESSAGE_TIMESTAMP_DIFFERENCE_MAX_MS_CONFIG);
+        this.leaderReplicationThrottledReplicas = 
Collections.unmodifiableList(getList(LogConfig.LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG));
+        this.followerReplicationThrottledReplicas = 
Collections.unmodifiableList(getList(LogConfig.FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG));
+        this.messageDownConversionEnable = 
getBoolean(TopicConfig.MESSAGE_DOWNCONVERSION_ENABLE_CONFIG);
+
+        remoteLogConfig = new RemoteLogConfig(this, retentionMs, 
retentionSize);
+    }
+
+    @SuppressWarnings("deprecation")
+    public RecordVersion recordVersion() {
+        return messageFormatVersion.highestSupportedRecordVersion();
+    }
+
+    // Exposed as a method so it can be mocked
+    public int maxMessageSize() {
+        return maxMessageSize;
+    }
+
+    public long randomSegmentJitter() {
+        if (segmentJitterMs == 0)
+            return 0;
+        else
+            return Utils.abs(ThreadLocalRandom.current().nextInt()) % 
Math.min(segmentJitterMs, segmentMs);
+    }
+
+    public long maxSegmentMs() {
+        if (compact && maxCompactionLagMs > 0)
+            return Math.min(maxCompactionLagMs, segmentMs);
+        else
+            return segmentMs;
+    }
+
+    public int initFileSize() {
+        if (preallocate)
+            return segmentSize;
+        else
+            return 0;
+    }
+
+    public String overriddenConfigsAsLoggableString() {
+        Map<String, Object> overriddenTopicProps = new HashMap<>();
+        props.forEach((k, v) -> {
+            if (overriddenConfigs.contains(k))
+                overriddenTopicProps.put((String) k, v);
+        });
+        return ConfigUtils.configMapToRedactedString(overriddenTopicProps, 
CONFIG);
+    }
+
+    /**
+     * Create a log config instance using the given properties and defaults
+     */
+    public static LogConfig fromProps(Map<?, ?> defaults, Properties 
overrides) {
+        Properties props = new Properties();
+        defaults.forEach((k, v) -> props.put(k, v));
+        props.putAll(overrides);
+        Set<String> overriddenKeys = overrides.keySet().stream().map(k -> 
(String) k).collect(Collectors.toSet());
+        return new LogConfig(props, overriddenKeys);
+    }
+
+    // Visible for testing, return a copy since it's a mutable global variable
+    public static LogConfigDef configDefCopy() {
+        return new LogConfigDef(CONFIG);
+    }
+
+    public static boolean shouldIgnoreMessageFormatVersion(MetadataVersion 
interBrokerProtocolVersion) {
+        return interBrokerProtocolVersion.isAtLeast(IBP_3_0_IV1);
+    }
+
+    public static Optional<Type> configType(String configName) {
+        return Optional.ofNullable(CONFIG.configKeys().get(configName)).map(c 
-> c.type);
+    }
+
+    public static List<String> configNames() {
+        return CONFIG.names().stream().sorted().collect(Collectors.toList());
+    }
+
+    public static Optional<String> serverConfigName(String configName) {
+        return CONFIG.serverConfigName(configName);
+    }
+
+    public static Map<String, ConfigKey> configKeys() {
+        return Collections.unmodifiableMap(CONFIG.configKeys());
+    }
+
+    /**
+     * Check that property names are valid
+     */
+    public static void validateNames(Properties props) {
+        List<String> names = configNames();
+        for (Object name : props.keySet())
+            if (!names.contains(name))
+                throw new InvalidConfigurationException("Unknown topic config 
name: " + name);
+    }
+
+    public static void validateValues(Map<?, ?> props) {
+        long minCompactionLag = (Long) 
props.get(TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG);
+        long maxCompactionLag = (Long) 
props.get(TopicConfig.MAX_COMPACTION_LAG_MS_CONFIG);
+        if (minCompactionLag > maxCompactionLag) {
+            throw new InvalidConfigurationException("conflict topic config 
setting "
+                + TopicConfig.MIN_COMPACTION_LAG_MS_CONFIG + " (" + 
minCompactionLag + ") > "
+                + TopicConfig.MAX_COMPACTION_LAG_MS_CONFIG + " (" + 
maxCompactionLag + ")");
+        }
+    }
+
+    /**
+     * Check that the given properties contain only valid log config names and 
that all values can be parsed and are valid
+     */
+    public static void validate(Properties props) {
+        validateNames(props);
+        Map<?, ?> valueMaps = CONFIG.parse(props);
+        validateValues(valueMaps);
+    }
+
+    public static void main(String[] args) {

Review Comment:
   I observed this issue when I pulled the trunk changes in other PR and raised 
a fix for that in https://github.com/apache/kafka/pull/13079



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to