This is an automated email from the ASF dual-hosted git repository.

hongshun pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/fluss.git


The following commit(s) were added to refs/heads/main by this push:
     new 94c131f41 [server] Skip datalake prefix validation when datalake 
format is not null (#2919)
94c131f41 is described below

commit 94c131f4180894514b7149e9aab1bd6cbb363ca0
Author: Hongshun Wang <[email protected]>
AuthorDate: Wed Mar 25 14:04:14 2026 +0800

    [server] Skip datalake prefix validation when datalake format is not null 
(#2919)
---
 .../apache/fluss/server/DynamicServerConfig.java   |  1 +
 .../coordinator/LakeCatalogDynamicLoader.java      |  6 ++++
 .../fluss/server/DynamicConfigChangeTest.java      | 41 ++++++++++++++++++++++
 3 files changed, 48 insertions(+)

diff --git 
a/fluss-server/src/main/java/org/apache/fluss/server/DynamicServerConfig.java 
b/fluss-server/src/main/java/org/apache/fluss/server/DynamicServerConfig.java
index ac9c2e401..d4430bb50 100644
--- 
a/fluss-server/src/main/java/org/apache/fluss/server/DynamicServerConfig.java
+++ 
b/fluss-server/src/main/java/org/apache/fluss/server/DynamicServerConfig.java
@@ -210,6 +210,7 @@ class DynamicServerConfig {
             throws ConfigException {
         for (String configKey : dynamicConfigs.keySet()) {
             if (newDynamicConfigs.containsKey(configKey)) {
+                effectiveChanges.put(configKey, 
newDynamicConfigs.get(configKey));
                 continue; // Not deleted
             }
 
diff --git 
a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/LakeCatalogDynamicLoader.java
 
b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/LakeCatalogDynamicLoader.java
index c8c4f2eba..db295922b 100644
--- 
a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/LakeCatalogDynamicLoader.java
+++ 
b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/LakeCatalogDynamicLoader.java
@@ -63,6 +63,12 @@ public class LakeCatalogDynamicLoader implements 
ServerReconfigurable, AutoClose
                 newConfig.getOptional(DATALAKE_FORMAT).isPresent()
                         ? newConfig.get(DATALAKE_FORMAT)
                         : currentConfiguration.get(DATALAKE_FORMAT);
+        // If datalake format is not set, skip prefix validation so that users 
can disable or enable
+        // datalake format without re-supplying all datalake-prefixed 
properties.
+        if (newDatalakeFormat == null) {
+            return;
+        }
+
         Map<String, String> configMap = newConfig.toMap();
         String datalakePrefix = "datalake." + newDatalakeFormat + ".";
         configMap.forEach(
diff --git 
a/fluss-server/src/test/java/org/apache/fluss/server/DynamicConfigChangeTest.java
 
b/fluss-server/src/test/java/org/apache/fluss/server/DynamicConfigChangeTest.java
index 423afa9cd..b20f39bc5 100644
--- 
a/fluss-server/src/test/java/org/apache/fluss/server/DynamicConfigChangeTest.java
+++ 
b/fluss-server/src/test/java/org/apache/fluss/server/DynamicConfigChangeTest.java
@@ -186,6 +186,47 @@ public class DynamicConfigChangeTest {
         }
     }
 
+    @Test
+    void testDatalakePrefixValidationSkippedWhenFormatIsNull() throws 
Exception {
+        Configuration configuration = new Configuration();
+        try (LakeCatalogDynamicLoader lakeCatalogDynamicLoader =
+                new LakeCatalogDynamicLoader(configuration, null, true)) {
+            DynamicConfigManager dynamicConfigManager =
+                    new DynamicConfigManager(zookeeperClient, configuration, 
true);
+            dynamicConfigManager.register(lakeCatalogDynamicLoader);
+            dynamicConfigManager.startup();
+
+            // Setting `datalake.paimon.*` without setting `datalake.format` 
should pass because
+            // prefix validation is skipped.
+            assertThatCode(
+                            () ->
+                                    dynamicConfigManager.alterConfigs(
+                                            Collections.singletonList(
+                                                    new AlterConfig(
+                                                            
"datalake.iceberg.type",
+                                                            "rest",
+                                                            
AlterConfigOpType.SET))))
+                    .doesNotThrowAnyException();
+
+            
assertThat(lakeCatalogDynamicLoader.getLakeCatalogContainer().getDataLakeFormat())
+                    .isNull();
+            assertThatThrownBy(
+                            () ->
+                                    dynamicConfigManager.alterConfigs(
+                                            Arrays.asList(
+                                                    new AlterConfig(
+                                                            
"datalake.iceberg.type",
+                                                            "rest",
+                                                            
AlterConfigOpType.SET),
+                                                    new AlterConfig(
+                                                            "datalake.format",
+                                                            "paimon",
+                                                            
AlterConfigOpType.SET))))
+                    .hasMessageContaining(
+                            "Invalid configuration 'datalake.iceberg.type' for 
'paimon' datalake format");
+        }
+    }
+
     @Test
     void testWrongLakeFormatPrefix() throws Exception {
         Configuration configuration = new Configuration();

Reply via email to