wuchong commented on code in PR #1567:
URL: https://github.com/apache/fluss/pull/1567#discussion_r2419788176
##########
fluss-client/src/test/java/org/apache/fluss/client/security/acl/FlussAuthorizationITCase.java:
##########
@@ -653,6 +657,70 @@ void testProduceAndConsumer() throws Exception {
}
}
+ @Test
+ void testDynamicConfigs() throws ExecutionException, InterruptedException {
+ assertThatThrownBy(() -> guestAdmin.describeClusterConfigs().get())
+ .rootCause()
+ .hasMessageContaining(
+ String.format(
+ "Principal %s have no authorization to operate
DESCRIBE on resource Resource{type=CLUSTER, name='fluss-cluster'}",
+ guestPrincipal));
+ rootAdmin
+ .createAcls(
+ Collections.singletonList(
+ new AclBinding(
+ Resource.cluster(),
+ new AccessControlEntry(
+ guestPrincipal,
+ "*",
+ OperationType.DESCRIBE,
+ PermissionType.ALLOW))))
+ .all()
+ .get();
+ Collection<ConfigEntry> configToResourceConfigs =
guestAdmin.describeClusterConfigs().get();
+ assertThat(configToResourceConfigs)
+ .contains(
+ new ConfigEntry(
+ DATALAKE_FORMAT.key(),
+ "paimon",
+
ConfigEntry.ConfigSource.INITIAL_SERVER_CONFIG));
+
+ assertThatThrownBy(
+ () ->
+ guestAdmin
+ .alterClusterConfigs(
+ Collections.singletonList(
+ new AlterConfig(
+
DATALAKE_FORMAT.key(),
+ null,
+
AlterConfigOpType.SET)))
+ .get())
+ .rootCause()
+ .hasMessageContaining(
+ String.format(
+ "Principal %s have no authorization to operate
ALTER on resource Resource{type=CLUSTER, name='fluss-cluster'}",
+ guestPrincipal));
+
+ rootAdmin
+ .createAcls(
+ Collections.singletonList(
+ new AclBinding(
+ Resource.cluster(),
+ new AccessControlEntry(
+ guestPrincipal,
+ "*",
+ OperationType.ALTER,
+ PermissionType.ALLOW))))
+ .all()
+ .get();
+ guestAdmin
+ .alterClusterConfigs(
+ Collections.singletonList(
+ new AlterConfig(
+ DATALAKE_FORMAT.key(), null,
AlterConfigOpType.SET)))
+ .get();
Review Comment:
assert the new cluster config has been changed?
##########
fluss-server/src/main/java/org/apache/fluss/server/coordinator/LakeCatalogDynamicLoader.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.fluss.server.coordinator;
+
+import org.apache.fluss.cluster.ServerReconfigurable;
+import org.apache.fluss.config.ConfigOptions;
+import org.apache.fluss.config.Configuration;
+import org.apache.fluss.exception.ConfigException;
+import org.apache.fluss.lake.lakestorage.LakeCatalog;
+import org.apache.fluss.lake.lakestorage.LakeStorage;
+import org.apache.fluss.lake.lakestorage.LakeStoragePlugin;
+import org.apache.fluss.lake.lakestorage.LakeStoragePluginSetUp;
+import org.apache.fluss.metadata.DataLakeFormat;
+import org.apache.fluss.plugin.PluginManager;
+import org.apache.fluss.server.utils.LakeStorageUtils;
+import org.apache.fluss.utils.IOUtils;
+
+import javax.annotation.Nullable;
+
+import java.util.Map;
+
+import static org.apache.fluss.config.ConfigOptions.DATALAKE_FORMAT;
+import static
org.apache.fluss.server.utils.LakeStorageUtils.extractLakeProperties;
+import static org.apache.fluss.utils.Preconditions.checkNotNull;
+
+/**
+ * A dynamic loader for lake catalog. Each time when the datalake format is
changed, the lake
+ * catalog will be changed.
+ */
+public class LakeCatalogDynamicLoader implements ServerReconfigurable,
AutoCloseable {
+ private volatile LakeCatalogContainer lakeCatalogContainer;
+ private Configuration currentConfiguration;
+ private final PluginManager pluginManager;
+ private final boolean isCoordinator;
+
+ public LakeCatalogDynamicLoader(
+ Configuration configuration, PluginManager pluginManager, boolean
isCoordinator) {
+ this.isCoordinator = isCoordinator;
+ this.currentConfiguration = configuration;
+ this.lakeCatalogContainer =
+ new LakeCatalogContainer(configuration, pluginManager,
isCoordinator);
+ this.pluginManager = pluginManager;
+ }
+
+ @Override
+ public void validate(Configuration newConfig) throws ConfigException {
+ final DataLakeFormat newDatalakeFormat =
+ newConfig.getOptional(DATALAKE_FORMAT).isPresent()
+ ? newConfig.get(DATALAKE_FORMAT)
+ : currentConfiguration.get(DATALAKE_FORMAT);
+ Map<String, String> configMap = newConfig.toMap();
+ String datalakePrefix = "datalake." + newDatalakeFormat + ".";
+ final DataLakeFormat finalDatalakeFormat = newDatalakeFormat;
Review Comment:
No need for a new variable, use `newDatalakeFormat` is fine?
##########
fluss-common/src/main/java/org/apache/fluss/cluster/AlterConfig.java:
##########
@@ -0,0 +1,84 @@
+/*
+ * 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.fluss.cluster;
Review Comment:
Move to `org.apache.fluss.config.cluster`? Currently, the
`org.apache.fluss.cluster` package only contains metadata cache classes.
##########
fluss-server/src/main/java/org/apache/fluss/server/coordinator/LakeCatalogDynamicLoader.java:
##########
@@ -0,0 +1,162 @@
+/*
+ * 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.fluss.server.coordinator;
+
+import org.apache.fluss.cluster.ServerReconfigurable;
+import org.apache.fluss.config.ConfigOptions;
+import org.apache.fluss.config.Configuration;
+import org.apache.fluss.exception.ConfigException;
+import org.apache.fluss.lake.lakestorage.LakeCatalog;
+import org.apache.fluss.lake.lakestorage.LakeStorage;
+import org.apache.fluss.lake.lakestorage.LakeStoragePlugin;
+import org.apache.fluss.lake.lakestorage.LakeStoragePluginSetUp;
+import org.apache.fluss.metadata.DataLakeFormat;
+import org.apache.fluss.plugin.PluginManager;
+import org.apache.fluss.server.utils.LakeStorageUtils;
+import org.apache.fluss.utils.IOUtils;
+
+import javax.annotation.Nullable;
+
+import java.util.Map;
+
+import static org.apache.fluss.config.ConfigOptions.DATALAKE_FORMAT;
+import static
org.apache.fluss.server.utils.LakeStorageUtils.extractLakeProperties;
+import static org.apache.fluss.utils.Preconditions.checkNotNull;
+
+/**
+ * A dynamic loader for lake catalog. Each time when the datalake format is
changed, the lake
+ * catalog will be changed.
+ */
+public class LakeCatalogDynamicLoader implements ServerReconfigurable,
AutoCloseable {
+ private volatile LakeCatalogContainer lakeCatalogContainer;
+ private Configuration currentConfiguration;
+ private final PluginManager pluginManager;
+ private final boolean isCoordinator;
+
+ public LakeCatalogDynamicLoader(
+ Configuration configuration, PluginManager pluginManager, boolean
isCoordinator) {
+ this.isCoordinator = isCoordinator;
+ this.currentConfiguration = configuration;
+ this.lakeCatalogContainer =
+ new LakeCatalogContainer(configuration, pluginManager,
isCoordinator);
+ this.pluginManager = pluginManager;
+ }
+
+ @Override
+ public void validate(Configuration newConfig) throws ConfigException {
+ final DataLakeFormat newDatalakeFormat =
+ newConfig.getOptional(DATALAKE_FORMAT).isPresent()
+ ? newConfig.get(DATALAKE_FORMAT)
+ : currentConfiguration.get(DATALAKE_FORMAT);
+ Map<String, String> configMap = newConfig.toMap();
+ String datalakePrefix = "datalake." + newDatalakeFormat + ".";
+ final DataLakeFormat finalDatalakeFormat = newDatalakeFormat;
+ configMap.forEach(
+ (key, value) -> {
+ if (!key.equals(DATALAKE_FORMAT.key())
+ && key.startsWith("datalake.")
+ && !key.startsWith(datalakePrefix)) {
+ throw new ConfigException(
+ String.format(
+ "Invalid configuration %s for %s
datalake format",
+ newConfig, finalDatalakeFormat));
Review Comment:
The `newConfig` contains all the configuration of the cluster, this exposes
too many sensitive inforamtion to users
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]