Copilot commented on code in PR #2769:
URL: https://github.com/apache/fluss/pull/2769#discussion_r3015683805


##########
fluss-server/src/main/java/org/apache/fluss/server/coordinator/remote/RemoteDirDynamicLoader.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.remote;
+
+import org.apache.fluss.config.ConfigOption;
+import org.apache.fluss.config.ConfigOptions;
+import org.apache.fluss.config.Configuration;
+import org.apache.fluss.config.cluster.ServerReconfigurable;
+import org.apache.fluss.exception.ConfigException;
+import org.apache.fluss.exception.IllegalConfigurationException;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+
+import static org.apache.fluss.config.FlussConfigUtils.validateRemoteDataDirs;
+
+/**
+ * Dynamic loader for remote data directories that supports runtime 
reconfiguration.
+ *
+ * <p>This class manages the lifecycle of remote data directories and provides 
a selector for
+ * selecting remote data directories. It implements {@link 
ServerReconfigurable} to support dynamic
+ * configuration updates at runtime without requiring a server restart.
+ *
+ * <p>When creating a new table or partition, the coordinator server uses this 
loader to select an
+ * appropriate remote data directory based on the configured selection 
strategy (see {@link
+ * org.apache.fluss.config.ConfigOptions#REMOTE_DATA_DIRS_STRATEGY}).
+ */
+public class RemoteDirDynamicLoader implements ServerReconfigurable, 
AutoCloseable {
+
+    private volatile RemoteDirSelector remoteDirSelector;
+    private Configuration currentConfiguration;
+
+    public RemoteDirDynamicLoader(Configuration configuration) {
+        this.currentConfiguration = configuration;
+        this.remoteDirSelector = createRemoteDirSelector(configuration);
+    }
+
+    public RemoteDirSelector getRemoteDirSelector() {
+        return remoteDirSelector;
+    }
+
+    @Override
+    public void validate(Configuration newConfig) throws ConfigException {
+        // Validate new remote data dirs contain all old remote data dirs
+        Optional<List<String>> newRemoteDataDirsOp =
+                newConfig.getOptional(ConfigOptions.REMOTE_DATA_DIRS);
+        if (newRemoteDataDirsOp.isPresent()) {
+            List<String> oldRemoteDataDirs =
+                    currentConfiguration.get(ConfigOptions.REMOTE_DATA_DIRS);
+            Set<String> newRemoteDataDirs = new 
HashSet<>(newRemoteDataDirsOp.get());
+            if (!newRemoteDataDirs.containsAll(oldRemoteDataDirs)) {
+                throw new ConfigException(
+                        String.format(
+                                "New %s: %s must contain all old %s: %s. "
+                                        + "If you want the Fluss cluster to 
stop transferring data to a certain path, "
+                                        + "keep it in %s and set its weight to 
0 in %s.",
+                                ConfigOptions.REMOTE_DATA_DIRS.key(),
+                                newRemoteDataDirsOp.get(),
+                                ConfigOptions.REMOTE_DATA_DIRS.key(),
+                                oldRemoteDataDirs,
+                                ConfigOptions.REMOTE_DATA_DIRS.key(),
+                                ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS.key()));
+            }
+        }
+
+        Configuration mergedConfig = mergeConfigurations(currentConfiguration, 
newConfig);
+        try {
+            validateRemoteDataDirs(mergedConfig);
+        } catch (IllegalConfigurationException e) {
+            throw new ConfigException(e.getMessage());
+        }
+    }
+
+    @Override
+    public void reconfigure(Configuration newConfig) throws ConfigException {
+        if (strategyChanged(newConfig)
+                || remoteDataDirsChanged(newConfig)
+                || weightsChanged(newConfig)) {
+            // Create a new container with the merged configuration
+            Configuration mergedConfig = 
mergeConfigurations(currentConfiguration, newConfig);
+            this.remoteDirSelector = createRemoteDirSelector(mergedConfig);
+            this.currentConfiguration = mergedConfig;
+        }

Review Comment:
   RemoteDirDynamicLoader assumes validate/reconfigure receive only incremental 
updates and merges configs manually. In production, DynamicServerConfig passes 
a full merged Configuration; using mergeConfigurations(current, newConfig) can 
incorrectly preserve keys that were removed (e.g., deleting 
remote.data.dirs/weights/strategy), so the selector may not reflect the 
effective configuration. Consider treating the provided newConfig as 
authoritative (no merge), or explicitly handling deletions so removed keys 
revert to defaults rather than being retained from currentConfiguration.



##########
fluss-server/src/main/java/org/apache/fluss/server/coordinator/remote/RemoteDirDynamicLoader.java:
##########
@@ -0,0 +1,167 @@
+/*
+ * 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.remote;
+
+import org.apache.fluss.config.ConfigOption;
+import org.apache.fluss.config.ConfigOptions;
+import org.apache.fluss.config.Configuration;
+import org.apache.fluss.config.cluster.ServerReconfigurable;
+import org.apache.fluss.exception.ConfigException;
+import org.apache.fluss.exception.IllegalConfigurationException;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+
+import static org.apache.fluss.config.FlussConfigUtils.validateRemoteDataDirs;
+
+/**
+ * Dynamic loader for remote data directories that supports runtime 
reconfiguration.
+ *
+ * <p>This class manages the lifecycle of remote data directories and provides 
a selector for
+ * selecting remote data directories. It implements {@link 
ServerReconfigurable} to support dynamic
+ * configuration updates at runtime without requiring a server restart.
+ *
+ * <p>When creating a new table or partition, the coordinator server uses this 
loader to select an
+ * appropriate remote data directory based on the configured selection 
strategy (see {@link
+ * org.apache.fluss.config.ConfigOptions#REMOTE_DATA_DIRS_STRATEGY}).
+ */
+public class RemoteDirDynamicLoader implements ServerReconfigurable, 
AutoCloseable {
+
+    private volatile RemoteDirSelector remoteDirSelector;
+    private Configuration currentConfiguration;
+
+    public RemoteDirDynamicLoader(Configuration configuration) {
+        this.currentConfiguration = configuration;
+        this.remoteDirSelector = createRemoteDirSelector(configuration);
+    }
+
+    public RemoteDirSelector getRemoteDirSelector() {
+        return remoteDirSelector;
+    }
+
+    @Override
+    public void validate(Configuration newConfig) throws ConfigException {
+        // Validate new remote data dirs contain all old remote data dirs
+        Optional<List<String>> newRemoteDataDirsOp =
+                newConfig.getOptional(ConfigOptions.REMOTE_DATA_DIRS);
+        if (newRemoteDataDirsOp.isPresent()) {
+            List<String> oldRemoteDataDirs =
+                    currentConfiguration.get(ConfigOptions.REMOTE_DATA_DIRS);
+            Set<String> newRemoteDataDirs = new 
HashSet<>(newRemoteDataDirsOp.get());
+            if (!newRemoteDataDirs.containsAll(oldRemoteDataDirs)) {
+                throw new ConfigException(
+                        String.format(
+                                "New %s: %s must contain all old %s: %s. "
+                                        + "If you want the Fluss cluster to 
stop transferring data to a certain path, "
+                                        + "keep it in %s and set its weight to 
0 in %s.",
+                                ConfigOptions.REMOTE_DATA_DIRS.key(),
+                                newRemoteDataDirsOp.get(),
+                                ConfigOptions.REMOTE_DATA_DIRS.key(),
+                                oldRemoteDataDirs,
+                                ConfigOptions.REMOTE_DATA_DIRS.key(),
+                                ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS.key()));
+            }
+        }
+
+        Configuration mergedConfig = mergeConfigurations(currentConfiguration, 
newConfig);
+        try {
+            validateRemoteDataDirs(mergedConfig);
+        } catch (IllegalConfigurationException e) {
+            throw new ConfigException(e.getMessage());
+        }
+    }
+
+    @Override
+    public void reconfigure(Configuration newConfig) throws ConfigException {
+        if (strategyChanged(newConfig)
+                || remoteDataDirsChanged(newConfig)
+                || weightsChanged(newConfig)) {
+            // Create a new container with the merged configuration
+            Configuration mergedConfig = 
mergeConfigurations(currentConfiguration, newConfig);
+            this.remoteDirSelector = createRemoteDirSelector(mergedConfig);
+            this.currentConfiguration = mergedConfig;
+        }
+    }
+
+    private RemoteDirSelector createRemoteDirSelector(Configuration conf) {
+        ConfigOptions.RemoteDataDirStrategy strategy =
+                conf.get(ConfigOptions.REMOTE_DATA_DIRS_STRATEGY);
+        String remoteDataDir = conf.get(ConfigOptions.REMOTE_DATA_DIR);
+        List<String> remoteDataDirs = conf.get(ConfigOptions.REMOTE_DATA_DIRS);
+        List<Integer> weights = 
conf.get(ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS);
+
+        switch (strategy) {
+            case ROUND_ROBIN:
+                return new RoundRobinRemoteDirSelector(remoteDataDir, 
remoteDataDirs);
+            case WEIGHTED_ROUND_ROBIN:
+                return new WeightedRoundRobinRemoteDirSelector(
+                        remoteDataDir, remoteDataDirs, weights);
+            default:
+                throw new IllegalArgumentException(
+                        "Unsupported remote data directory select strategy: " 
+ strategy);
+        }
+    }
+
+    private boolean strategyChanged(Configuration newConfig) {
+        return hasConfigChanged(newConfig, 
ConfigOptions.REMOTE_DATA_DIRS_STRATEGY);
+    }
+
+    private boolean remoteDataDirsChanged(Configuration newConfig) {
+        return hasConfigChanged(newConfig, ConfigOptions.REMOTE_DATA_DIRS);
+    }
+
+    private boolean weightsChanged(Configuration newConfig) {
+        return hasConfigChanged(newConfig, 
ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS);
+    }
+
+    /**
+     * Checks if a specific configuration option has changed in the new config.
+     *
+     * @param newConfig the new configuration
+     * @param option the configuration option to check
+     * @param <T> the type of the configuration value
+     * @return true if the configuration has changed
+     */
+    private <T> boolean hasConfigChanged(Configuration newConfig, 
ConfigOption<T> option) {
+        return newConfig
+                .getOptional(option)
+                .map(newValue -> !Objects.equals(newValue, 
currentConfiguration.get(option)))
+                .orElse(false);

Review Comment:
   hasConfigChanged() uses newConfig.getOptional(option) and returns false when 
a key is absent. This means deleting a dynamic config (e.g., removing 
remote.data.dirs to fall back to remote.data.dir) will not be detected as a 
change, so RemoteDirDynamicLoader will keep using the old selector/config. 
Comparing effective values (e.g., newConfig.get(option) vs 
currentConfiguration.get(option)) or checking raw key presence/removal would 
correctly handle deletions.
   ```suggestion
           boolean newHasOption = newConfig.contains(option);
           boolean currentHasOption = currentConfiguration.contains(option);
   
           // If the presence of the option changed (added or removed), this is 
a config change.
           if (newHasOption != currentHasOption) {
               return true;
           }
   
           // If both have (or both don't have) the option, compare effective 
values.
           T newValue = newHasOption ? newConfig.get(option) : null;
           T currentValue = currentHasOption ? currentConfiguration.get(option) 
: null;
   
           return !Objects.equals(newValue, currentValue);
   ```



##########
fluss-server/src/main/java/org/apache/fluss/server/coordinator/remote/RoundRobinRemoteDirSelector.java:
##########
@@ -0,0 +1,55 @@
+/*
+ * 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.remote;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Round-robin remote data dir selector.
+ *
+ * <p>This implementation cycles through the available remote data directories 
in order, ensuring
+ * each directory is selected once before repeating.
+ *
+ * <p>Example: For directories [A, B, C], the selection sequence would be: A, 
B, C, A, B, C, ...
+ */
+public class RoundRobinRemoteDirSelector implements RemoteDirSelector {
+
+    private final String remoteDataDir;
+    private final List<String> remoteDataDirs;
+
+    // Current position in the round-robin cycle.
+    private final AtomicInteger position;
+
+    public RoundRobinRemoteDirSelector(String remoteDataDir, List<String> 
remoteDataDirs) {
+        this.remoteDataDir = remoteDataDir;
+        this.remoteDataDirs = Collections.unmodifiableList(remoteDataDirs);
+        this.position = new AtomicInteger(0);
+    }

Review Comment:
   RoundRobinRemoteDirSelector wraps the provided remoteDataDirs with 
Collections.unmodifiableList(...) without copying. If the caller later mutates 
the original list, selector behavior can change unexpectedly and may even break 
modulo logic. Consider copying into a new ArrayList before wrapping to ensure 
immutability of the selector’s directory set.



##########
fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java:
##########
@@ -162,6 +164,7 @@ private FlussClusterExtension(
                 racks != null && racks.length == numOfTabletServers,
                 "racks must be not null and have the same length as 
numOfTabletServers");
         this.racks = racks;
+        this.remoteDirNames = remoteDirNames;

Review Comment:
   FlussClusterExtension assumes remoteDirNames is non-null (setRemoteDataDirs 
calls remoteDirNames.isEmpty()), but Builder.setRemoteDirNames does not 
null-check and the constructor doesn’t enforce non-null either. Passing null 
will cause a NullPointerException during cluster start. Consider defaulting 
null to Collections.emptyList() and/or validating the argument in 
setRemoteDirNames()/constructor.
   ```suggestion
           this.remoteDirNames =
                   remoteDirNames != null ? remoteDirNames : 
Collections.emptyList();
   ```



##########
fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorService.java:
##########
@@ -275,6 +278,7 @@ public CoordinatorService(
         this.ioExecutor = ioExecutor;
         this.lakeTableHelper =
                 new LakeTableHelper(zkClient, 
conf.getString(ConfigOptions.REMOTE_DATA_DIR));
+        this.remoteDirDynamicLoader = remoteDirDynamicLoader;

Review Comment:
   LakeTableHelper is constructed with conf.getString(REMOTE_DATA_DIR), but 
with multi-dir support remote.data.dir may be unset (remote.data.dirs only), 
and tables/partitions now have a selected per-table remoteDataDir. This can 
cause PrepareLakeTableSnapshot offsets files to be written under the wrong/NULL 
base directory instead of the table’s assigned remoteDataDir. Consider using 
FlussConfigUtils.getDefaultRemoteDataDir(conf) as a safe fallback, and/or 
resolve the table’s remoteDataDir from metadata when storing lake snapshot 
offset files.



##########
fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/RemoteDirsITCase.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.remote;
+
+import org.apache.fluss.config.AutoPartitionTimeUnit;
+import org.apache.fluss.config.ConfigOptions;
+import org.apache.fluss.config.Configuration;
+import org.apache.fluss.metadata.PartitionSpec;
+import org.apache.fluss.metadata.Schema;
+import org.apache.fluss.metadata.TableDescriptor;
+import org.apache.fluss.metadata.TablePath;
+import org.apache.fluss.server.testutils.FlussClusterExtension;
+import org.apache.fluss.server.testutils.RpcMessageTestUtils;
+import org.apache.fluss.server.zk.ZooKeeperClient;
+import org.apache.fluss.server.zk.data.PartitionRegistration;
+import org.apache.fluss.server.zk.data.TableRegistration;
+import org.apache.fluss.types.DataTypes;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+import static 
org.apache.fluss.record.TestData.DATA1_PARTITIONED_TABLE_DESCRIPTOR;
+import static org.apache.fluss.record.TestData.DATA1_TABLE_DESCRIPTOR;
+import static org.apache.fluss.record.TestData.DATA1_TABLE_DESCRIPTOR_PK;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** ITCase for multi remote data directories functionality. */
+class RemoteDirsITCase {
+
+    private static final TableDescriptor DATA1_PARTITIONED_TABLE_DESCRIPTOR_PK 
=
+            TableDescriptor.builder()
+                    .schema(
+                            Schema.newBuilder()
+                                    .column("a", DataTypes.INT())
+                                    .withComment("a is first column")
+                                    .column("b", DataTypes.STRING())
+                                    .withComment("b is second column")
+                                    .primaryKey("a", "b")
+                                    .build())
+                    .distributedBy(3)
+                    .partitionedBy("b")
+                    .property(ConfigOptions.TABLE_AUTO_PARTITION_ENABLED, true)
+                    .property(
+                            ConfigOptions.TABLE_AUTO_PARTITION_TIME_UNIT,
+                            AutoPartitionTimeUnit.YEAR)
+                    .build();
+
+    private static final List<String> REMOTE_DIR_NAMES = Arrays.asList("dir1", 
"dir2", "dir3");
+
+    @RegisterExtension
+    public static final FlussClusterExtension FLUSS_CLUSTER_EXTENSION =
+            FlussClusterExtension.builder()
+                    .setNumOfTabletServers(3)
+                    .setClusterConf(initConfig())
+                    .setRemoteDirNames(REMOTE_DIR_NAMES)
+                    .build();
+
+    private ZooKeeperClient zkClient;
+
+    private static Configuration initConfig() {
+        Configuration conf = new Configuration();
+        conf.setInt(ConfigOptions.DEFAULT_REPLICATION_FACTOR, 3);
+        conf.set(ConfigOptions.AUTO_PARTITION_CHECK_INTERVAL, 
Duration.ofSeconds(1));
+
+        return conf;
+    }
+
+    @BeforeEach
+    void setup() {
+        zkClient = FLUSS_CLUSTER_EXTENSION.getZooKeeperClient();
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    void testCreateMultipleTablesWithRoundRobin(boolean isPrimaryKeyTable) 
throws Exception {
+        // Create multiple tables and verify they use different remote dirs 
via round-robin
+        TableDescriptor tableDescriptor =
+                isPrimaryKeyTable ? DATA1_TABLE_DESCRIPTOR_PK : 
DATA1_TABLE_DESCRIPTOR;
+        String tablePrefix = isPrimaryKeyTable ? "pk_table_" : "non_pk_table_";
+
+        List<String> remoteDirsUsed = new ArrayList<>();
+        int tableCount = 6; // Create more tables than dirs to see round-robin 
in action
+
+        for (int i = 0; i < tableCount; i++) {
+            TablePath tablePath = TablePath.of("test_db", tablePrefix + i);
+            RpcMessageTestUtils.createTable(FLUSS_CLUSTER_EXTENSION, 
tablePath, tableDescriptor);
+
+            // Get the table registration to check remoteDataDir
+            Optional<TableRegistration> tableOpt = 
zkClient.getTable(tablePath);
+            assertThat(tableOpt).isPresent();
+            TableRegistration table = tableOpt.get();
+
+            assertThat(table.remoteDataDir).isNotNull();
+            remoteDirsUsed.add(table.remoteDataDir);
+        }
+
+        // Verify round-robin distribution: each dir should be used at least 
once
+        Map<String, Integer> dirUsageCount = new HashMap<>();
+        for (String dir : remoteDirsUsed) {
+            dirUsageCount.merge(dir, 1, Integer::sum);
+        }
+
+        // With round-robin, all configured dirs should be used
+        assertThat(dirUsageCount.keySet()).hasSize(REMOTE_DIR_NAMES.size());
+    }
+
+    @ParameterizedTest
+    @ValueSource(booleans = {true, false})
+    void testCreatePartitionsWithRoundRobin(boolean isPrimaryKeyTable) throws 
Exception {
+        // Create a partitioned table and add multiple partitions
+        // Each partition should get a different remoteDataDir via round-robin
+        String tablePrefix = isPrimaryKeyTable ? "pk_partitioned_" : 
"partitioned_";
+        TablePath tablePath = TablePath.of("test_db", tablePrefix + "table_2");
+
+        TableDescriptor tableDescriptor =
+                isPrimaryKeyTable
+                        ? DATA1_PARTITIONED_TABLE_DESCRIPTOR_PK
+                        : DATA1_PARTITIONED_TABLE_DESCRIPTOR;
+        RpcMessageTestUtils.createTable(FLUSS_CLUSTER_EXTENSION, tablePath, 
tableDescriptor);
+
+        Optional<TableRegistration> tableOpt = zkClient.getTable(tablePath);
+        assertThat(tableOpt).isPresent();
+        TableRegistration table = tableOpt.get();
+        // Partitioned table should NOT have remoteDataDir set at table level

Review Comment:
   The comment says the partitioned table should NOT have remoteDataDir set at 
table level, but the assertion immediately below expects it to be non-null. 
Given table creation now always stores a selected remoteDataDir (used for 
metadata), the comment should be updated to match the actual expected behavior.
   ```suggestion
           // Partitioned table should have remoteDataDir set at table level 
(used for metadata)
   ```



##########
fluss-server/src/main/java/org/apache/fluss/server/coordinator/remote/WeightedRoundRobinRemoteDirSelector.java:
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.remote;
+
+import java.util.Collections;
+import java.util.List;
+
+/**
+ * Weighted Round-robin remote data dir selector using Interleaved (Smooth) 
Weighted Round-Robin
+ * algorithm.
+ *
+ * <p>This implementation uses the smooth weighted round-robin algorithm (also 
known as interleaved
+ * weighted round-robin), which distributes selections more evenly compared to 
traditional weighted
+ * round-robin. Instead of selecting the same node consecutively based on its 
weight, it interleaves
+ * selections to achieve a smoother distribution.
+ *
+ * <p>Algorithm:
+ *
+ * <ol>
+ *   <li>Each node maintains a currentWeight initialized to 0
+ *   <li>On each selection: add the node's configured weight to its 
currentWeight
+ *   <li>Select the node with the highest currentWeight
+ *   <li>Subtract the total weight sum from the selected node's currentWeight
+ * </ol>
+ *
+ * <p>Example: For nodes A, B, C with weights 5, 1, 1 (total=7), the selection 
sequence would be: A,
+ * A, B, A, C, A, A (instead of A, A, A, A, A, B, C in traditional WRR).
+ */
+public class WeightedRoundRobinRemoteDirSelector implements RemoteDirSelector {
+
+    private final String remoteDataDir;
+    private final List<String> remoteDataDirs;
+    private final int[] weights;
+    private final int totalWeight;
+
+    // Current weights for each node, used in smooth weighted round-robin
+    private final int[] currentWeights;
+
+    // Lock object for thread safety
+    private final Object lock = new Object();
+
+    public WeightedRoundRobinRemoteDirSelector(
+            String remoteDataDir, List<String> remoteDataDirs, List<Integer> 
weights) {
+        this.remoteDataDir = remoteDataDir;
+        this.remoteDataDirs = Collections.unmodifiableList(remoteDataDirs);
+
+        // Convert weights list to array for better performance
+        this.weights = new int[weights.size()];
+        int sum = 0;
+        for (int i = 0; i < weights.size(); i++) {
+            this.weights[i] = weights.get(i);
+            sum += this.weights[i];
+        }
+        this.totalWeight = sum;
+
+        // Initialize current weights to 0
+        this.currentWeights = new int[remoteDataDirs.size()];
+    }

Review Comment:
   WeightedRoundRobinRemoteDirSelector assumes remoteDataDirs.size() == 
weights.size(); otherwise nextDataDir() will throw 
ArrayIndexOutOfBoundsException when indexing weights[i]. Even if config 
validation normally enforces this, it’s a public class and can be constructed 
directly in tests/other code. Add a fail-fast argument check (and ideally 
defensive copies of the input lists) so misuse produces a clear 
IllegalArgumentException instead of a runtime indexing error.



-- 
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]

Reply via email to