|
18 | 18 |
|
19 | 19 | import org.apache.kafka.common.config.AbstractConfig; |
20 | 20 | import org.apache.kafka.common.config.ConfigDef; |
| 21 | +import org.apache.kafka.common.config.ConfigException; |
21 | 22 |
|
22 | 23 | import java.util.Collections; |
23 | 24 | import java.util.Map; |
@@ -93,20 +94,29 @@ public final class RemoteLogManagerConfig { |
93 | 94 | public static final long DEFAULT_REMOTE_LOG_INDEX_FILE_CACHE_TOTAL_SIZE_BYTES = 1024 * 1024 * 1024L; |
94 | 95 |
|
95 | 96 | public static final String REMOTE_LOG_MANAGER_THREAD_POOL_SIZE_PROP = "remote.log.manager.thread.pool.size"; |
96 | | - public static final String REMOTE_LOG_MANAGER_THREAD_POOL_SIZE_DOC = "Size of the thread pool used in scheduling tasks to copy " + |
| 97 | + public static final String REMOTE_LOG_MANAGER_THREAD_POOL_SIZE_DOC = "Deprecated. Size of the thread pool used in scheduling tasks to copy " + |
97 | 98 | "segments, fetch remote log indexes and clean up remote log segments."; |
98 | 99 | public static final int DEFAULT_REMOTE_LOG_MANAGER_THREAD_POOL_SIZE = 10; |
99 | 100 |
|
| 101 | + private static final String REMOTE_LOG_MANAGER_THREAD_POOL_FALLBACK = "The default value of -1 means that this will be set to the configured value of " + |
| 102 | + REMOTE_LOG_MANAGER_THREAD_POOL_SIZE_PROP + ", if available; otherwise, it defaults to " + DEFAULT_REMOTE_LOG_MANAGER_THREAD_POOL_SIZE + "."; |
| 103 | + private static final ConfigDef.Validator REMOTE_LOG_MANAGER_THREAD_POOL_SIZE_VALIDATOR = ConfigDef.LambdaValidator.with( |
| 104 | + (name, value) -> { |
| 105 | + if ((int) value < -1 || (int) value == 0) throw new ConfigException(name, value, "Value can be -1 or greater than 0"); |
| 106 | + }, |
| 107 | + () -> REMOTE_LOG_MANAGER_THREAD_POOL_FALLBACK |
| 108 | + ); |
| 109 | + |
100 | 110 | public static final String REMOTE_LOG_MANAGER_COPIER_THREAD_POOL_SIZE_PROP = "remote.log.manager.copier.thread.pool.size"; |
101 | | - public static final String REMOTE_LOG_MANAGER_COPIER_THREAD_POOL_SIZE_DOC = "Size of the thread pool used in " + |
102 | | - "scheduling tasks to copy segments."; |
103 | | - public static final int DEFAULT_REMOTE_LOG_MANAGER_COPIER_THREAD_POOL_SIZE = 10; |
| 111 | + public static final String REMOTE_LOG_MANAGER_COPIER_THREAD_POOL_SIZE_DOC = "Size of the thread pool used in scheduling tasks " + |
| 112 | + "to copy segments. " + REMOTE_LOG_MANAGER_THREAD_POOL_FALLBACK; |
| 113 | + public static final int DEFAULT_REMOTE_LOG_MANAGER_COPIER_THREAD_POOL_SIZE = -1; |
104 | 114 |
|
105 | 115 | public static final String REMOTE_LOG_MANAGER_EXPIRATION_THREAD_POOL_SIZE_PROP = "remote.log.manager.expiration.thread.pool.size"; |
106 | | - public static final String REMOTE_LOG_MANAGER_EXPIRATION_THREAD_POOL_SIZE_DOC = "Size of the thread pool used in" + |
107 | | - " scheduling tasks to clean up remote log segments."; |
108 | | - public static final int DEFAULT_REMOTE_LOG_MANAGER_EXPIRATION_THREAD_POOL_SIZE = 10; |
109 | | - |
| 116 | + public static final String REMOTE_LOG_MANAGER_EXPIRATION_THREAD_POOL_SIZE_DOC = "Size of the thread pool used in scheduling tasks " + |
| 117 | + "to clean up remote log segments. " + REMOTE_LOG_MANAGER_THREAD_POOL_FALLBACK; |
| 118 | + public static final int DEFAULT_REMOTE_LOG_MANAGER_EXPIRATION_THREAD_POOL_SIZE = -1; |
| 119 | + |
110 | 120 | public static final String REMOTE_LOG_MANAGER_TASK_INTERVAL_MS_PROP = "remote.log.manager.task.interval.ms"; |
111 | 121 | public static final String REMOTE_LOG_MANAGER_TASK_INTERVAL_MS_DOC = "Interval at which remote log manager runs the scheduled tasks like copy " + |
112 | 122 | "segments, and clean up remote log segments."; |
@@ -257,16 +267,16 @@ public static ConfigDef configDef() { |
257 | 267 | atLeast(1), |
258 | 268 | MEDIUM, |
259 | 269 | REMOTE_LOG_MANAGER_THREAD_POOL_SIZE_DOC) |
260 | | - .defineInternal(REMOTE_LOG_MANAGER_COPIER_THREAD_POOL_SIZE_PROP, |
| 270 | + .define(REMOTE_LOG_MANAGER_COPIER_THREAD_POOL_SIZE_PROP, |
261 | 271 | INT, |
262 | 272 | DEFAULT_REMOTE_LOG_MANAGER_COPIER_THREAD_POOL_SIZE, |
263 | | - atLeast(1), |
| 273 | + REMOTE_LOG_MANAGER_THREAD_POOL_SIZE_VALIDATOR, |
264 | 274 | MEDIUM, |
265 | 275 | REMOTE_LOG_MANAGER_COPIER_THREAD_POOL_SIZE_DOC) |
266 | | - .defineInternal(REMOTE_LOG_MANAGER_EXPIRATION_THREAD_POOL_SIZE_PROP, |
| 276 | + .define(REMOTE_LOG_MANAGER_EXPIRATION_THREAD_POOL_SIZE_PROP, |
267 | 277 | INT, |
268 | 278 | DEFAULT_REMOTE_LOG_MANAGER_EXPIRATION_THREAD_POOL_SIZE, |
269 | | - atLeast(1), |
| 279 | + REMOTE_LOG_MANAGER_THREAD_POOL_SIZE_VALIDATOR, |
270 | 280 | MEDIUM, |
271 | 281 | REMOTE_LOG_MANAGER_EXPIRATION_THREAD_POOL_SIZE_DOC) |
272 | 282 | .define(REMOTE_LOG_MANAGER_TASK_INTERVAL_MS_PROP, |
@@ -395,11 +405,13 @@ public int remoteLogManagerThreadPoolSize() { |
395 | 405 | } |
396 | 406 |
|
397 | 407 | public int remoteLogManagerCopierThreadPoolSize() { |
398 | | - return config.getInt(REMOTE_LOG_MANAGER_COPIER_THREAD_POOL_SIZE_PROP); |
| 408 | + int size = config.getInt(REMOTE_LOG_MANAGER_COPIER_THREAD_POOL_SIZE_PROP); |
| 409 | + return size == -1 ? remoteLogManagerThreadPoolSize() : size; |
399 | 410 | } |
400 | 411 |
|
401 | 412 | public int remoteLogManagerExpirationThreadPoolSize() { |
402 | | - return config.getInt(REMOTE_LOG_MANAGER_EXPIRATION_THREAD_POOL_SIZE_PROP); |
| 413 | + int size = config.getInt(REMOTE_LOG_MANAGER_EXPIRATION_THREAD_POOL_SIZE_PROP); |
| 414 | + return size == -1 ? remoteLogManagerThreadPoolSize() : size; |
403 | 415 | } |
404 | 416 |
|
405 | 417 | public long remoteLogManagerTaskIntervalMs() { |
|
0 commit comments