Skip to content

Commit 4c2c90c

Browse files
committed
[FLINK-37166] Use concurrenthashmap in flink config manager
1 parent cd55bdc commit 4c2c90c

File tree

2 files changed

+45
-4
lines changed

2 files changed

+45
-4
lines changed

flink-kubernetes-operator/src/main/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManager.java

+4-4
Original file line numberDiff line numberDiff line change
@@ -50,13 +50,13 @@
5050

5151
import java.time.Duration;
5252
import java.util.ArrayList;
53-
import java.util.HashMap;
5453
import java.util.List;
5554
import java.util.Map;
5655
import java.util.Optional;
5756
import java.util.Set;
5857
import java.util.SortedMap;
5958
import java.util.TreeMap;
59+
import java.util.concurrent.ConcurrentHashMap;
6060
import java.util.concurrent.Executors;
6161
import java.util.concurrent.ScheduledExecutorService;
6262
import java.util.concurrent.TimeUnit;
@@ -87,7 +87,7 @@ public class FlinkConfigManager {
8787
private final AtomicLong defaultConfigVersion = new AtomicLong(0);
8888
private final LoadingCache<Key, Configuration> cache;
8989
private final Consumer<Set<String>> namespaceListener;
90-
private volatile Map<FlinkVersion, List<String>> relevantFlinkVersionPrefixes;
90+
private volatile ConcurrentHashMap<FlinkVersion, List<String>> relevantFlinkVersionPrefixes;
9191

9292
protected static final Pattern FLINK_VERSION_PATTERN =
9393
Pattern.compile(
@@ -114,7 +114,7 @@ public FlinkConfigManager(
114114
this.namespaceListener = namespaceListener;
115115
Duration cacheTimeout =
116116
defaultConfig.get(KubernetesOperatorConfigOptions.OPERATOR_CONFIG_CACHE_TIMEOUT);
117-
this.relevantFlinkVersionPrefixes = new HashMap<>();
117+
this.relevantFlinkVersionPrefixes = new ConcurrentHashMap<>();
118118
this.cache =
119119
CacheBuilder.newBuilder()
120120
.maximumSize(
@@ -189,7 +189,7 @@ public void updateDefaultConfig(Configuration newConf) {
189189
// We clear the cached relevant Flink version prefixes as the base config may include new
190190
// version overrides.
191191
// This will trigger a regeneration of the prefixes in the next call to getDefaultConfig.
192-
relevantFlinkVersionPrefixes = new HashMap<>();
192+
relevantFlinkVersionPrefixes = new ConcurrentHashMap<>();
193193
}
194194

195195
/**

flink-kubernetes-operator/src/test/java/org/apache/flink/kubernetes/operator/config/FlinkConfigManagerTest.java

+41
Original file line numberDiff line numberDiff line change
@@ -49,6 +49,7 @@
4949
import java.util.Map;
5050
import java.util.Optional;
5151
import java.util.Set;
52+
import java.util.concurrent.atomic.AtomicBoolean;
5253
import java.util.regex.Matcher;
5354

5455
import static org.apache.flink.kubernetes.operator.config.KubernetesOperatorConfigOptions.OPERATOR_WATCHED_NAMESPACES;
@@ -380,4 +381,44 @@ public void testVersionNamespaceDefaultConfs() {
380381
assertEquals("v3", observeConfig.get("conf3"));
381382
assertEquals("false", observeConfig.get("conf0"));
382383
}
384+
385+
@Test
386+
public void testConcurrentDefaultConfig() throws InterruptedException {
387+
var opConf = new Configuration();
388+
var configManager = new FlinkConfigManager(opConf);
389+
var completed1 = new AtomicBoolean();
390+
var completed2 = new AtomicBoolean();
391+
var completed3 = new AtomicBoolean();
392+
393+
var t1 =
394+
new Thread(
395+
() -> {
396+
configManager.getDefaultConfig("ns1", FlinkVersion.v1_18);
397+
completed1.set(true);
398+
});
399+
var t2 =
400+
new Thread(
401+
() -> {
402+
configManager.getDefaultConfig("ns1", FlinkVersion.v1_18);
403+
completed2.set(true);
404+
});
405+
var t3 =
406+
new Thread(
407+
() -> {
408+
configManager.getDefaultConfig("ns1", FlinkVersion.v1_18);
409+
completed3.set(true);
410+
});
411+
412+
t1.start();
413+
t2.start();
414+
t3.start();
415+
416+
t1.join();
417+
t2.join();
418+
t3.join();
419+
420+
assertTrue(completed1.get());
421+
assertTrue(completed2.get());
422+
assertTrue(completed3.get());
423+
}
383424
}

0 commit comments

Comments
 (0)