diff --git a/fluss-client/src/main/java/org/apache/fluss/client/admin/FlussAdmin.java b/fluss-client/src/main/java/org/apache/fluss/client/admin/FlussAdmin.java index 8f39a6e420..0d98cfedc4 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/admin/FlussAdmin.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/admin/FlussAdmin.java @@ -298,6 +298,9 @@ public CompletableFuture getTableInfo(TablePath tablePath) { r.getTableId(), r.getSchemaId(), TableDescriptor.fromJsonBytes(r.getTableJson()), + // For backward compatibility, results returned by old + // clusters do not include the remote data dir + r.hasRemoteDataDir() ? r.getRemoteDataDir() : null, r.getCreatedTime(), r.getModifiedTime())); } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java b/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java index a7dbec47ce..89d88a8606 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java @@ -555,8 +555,12 @@ public static List toPartitionInfos(ListPartitionInfosResponse re pbPartitionInfo -> new PartitionInfo( pbPartitionInfo.getPartitionId(), - toResolvedPartitionSpec( - pbPartitionInfo.getPartitionSpec()))) + toResolvedPartitionSpec(pbPartitionInfo.getPartitionSpec()), + // For backward compatibility, results returned by old + // clusters do not include the remote data dir + pbPartitionInfo.hasRemoteDataDir() + ? pbPartitionInfo.getRemoteDataDir() + : null)) .collect(Collectors.toList()); } diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/DefaultCompletedFetchTest.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/DefaultCompletedFetchTest.java index 9f50fe59e4..c4624617df 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/DefaultCompletedFetchTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/DefaultCompletedFetchTest.java @@ -67,6 +67,7 @@ import static org.apache.fluss.record.TestData.DATA2_TABLE_ID; import static org.apache.fluss.record.TestData.DATA2_TABLE_INFO; import static org.apache.fluss.record.TestData.DATA2_TABLE_PATH; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.record.TestData.DEFAULT_SCHEMA_ID; import static org.apache.fluss.row.BinaryString.fromString; import static org.apache.fluss.rpc.util.CommonRpcMessageUtils.toByteBuffer; @@ -169,6 +170,7 @@ void testProjection(LogFormat logFormat, byte magic) throws Exception { .distributedBy(3) .logFormat(logFormat) .build(), + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis()); long fetchOffset = 0L; @@ -313,6 +315,7 @@ void testComplexTypeFetch() throws Exception { .distributedBy(3) .logFormat(LogFormat.ARROW) .build(), + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis()); long fetchOffset = 0L; diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherITCase.java index b6817b327b..73c9b40fb6 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherITCase.java @@ -172,6 +172,7 @@ void testFetchWithSchemaChange() throws Exception { DATA1_TABLE_INFO.getNumBuckets(), DATA1_TABLE_INFO.getProperties(), DATA1_TABLE_INFO.getCustomProperties(), + DATA1_TABLE_INFO.getRemoteDataDir(), DATA1_TABLE_INFO.getComment().orElse(null), DATA1_TABLE_INFO.getCreatedTime(), DATA1_TABLE_INFO.getModifiedTime()), diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetchTest.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetchTest.java index f3022c91b4..e5fe7290cf 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetchTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetchTest.java @@ -61,6 +61,7 @@ import static org.apache.fluss.record.TestData.DATA2_TABLE_ID; import static org.apache.fluss.record.TestData.DATA2_TABLE_INFO; import static org.apache.fluss.record.TestData.DATA2_TABLE_PATH; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.record.TestData.DEFAULT_SCHEMA_ID; import static org.apache.fluss.testutils.DataTestUtils.genLogFile; import static org.apache.fluss.utils.FlussPaths.remoteLogSegmentDir; @@ -218,6 +219,7 @@ void testProjection(String format) throws Exception { .distributedBy(3) .logFormat(logFormat) .build(), + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis()); long fetchOffset = 0L; diff --git a/fluss-client/src/test/java/org/apache/fluss/client/write/RecordAccumulatorTest.java b/fluss-client/src/test/java/org/apache/fluss/client/write/RecordAccumulatorTest.java index 746d251d81..773c12a445 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/write/RecordAccumulatorTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/write/RecordAccumulatorTest.java @@ -76,6 +76,7 @@ import static org.apache.fluss.record.TestData.DATA1_TABLE_ID; import static org.apache.fluss.record.TestData.DATA1_TABLE_INFO; import static org.apache.fluss.record.TestData.DATA1_TABLE_PATH; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.testutils.DataTestUtils.indexedRow; import static org.apache.fluss.testutils.DataTestUtils.row; import static org.assertj.core.api.Assertions.assertThat; @@ -96,6 +97,7 @@ class RecordAccumulatorTest { .distributedBy(3) .property(ConfigOptions.TABLE_LOG_ARROW_COMPRESSION_TYPE.key(), "zstd") .build(), + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis()); diff --git a/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java b/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java index 08742ebc2e..cab3267595 100644 --- a/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java +++ b/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java @@ -99,6 +99,44 @@ public class ConfigOptions { "The directory used for storing the kv snapshot data files and remote log for log tiered storage " + " in a Fluss supported filesystem."); + public static final ConfigOption> REMOTE_DATA_DIRS = + key("remote.data.dirs") + .stringType() + .asList() + .defaultValues() + .withDescription( + "The directories used for storing the kv snapshot data files and remote log for log tiered storage " + + " in a Fluss supported filesystem. " + + "This is a list of remote data directory paths. " + + "Example: `remote.data.dirs: oss://bucket1/fluss-remote-data, oss://bucket2/fluss-remote-data`."); + + public static final ConfigOption REMOTE_DATA_DIRS_STRATEGY = + key("remote.data.dirs.strategy") + .enumType(RemoteDataDirStrategy.class) + .defaultValue(RemoteDataDirStrategy.ROUND_ROBIN) + .withDescription( + "The strategy for selecting the remote data directory from `" + + REMOTE_DATA_DIRS.key() + + "`."); + + public static final ConfigOption> REMOTE_DATA_DIRS_WEIGHTS = + key("remote.data.dirs.weights") + .intType() + .asList() + .defaultValues() + .withDescription( + "The weights of the remote data directories. " + + "This is a list of weights corresponding to the `" + + REMOTE_DATA_DIRS.key() + + "` in the same order. When `" + + REMOTE_DATA_DIRS_STRATEGY.key() + + "` is set to `" + + RemoteDataDirStrategy.WEIGHTED_ROUND_ROBIN + + "`, this must be configured, and its size must be equal to `" + + REMOTE_DATA_DIRS.key() + + "`; otherwise, it will be ignored." + + "Example: `remote.data.dir.weights: 1, 2`"); + public static final ConfigOption REMOTE_FS_WRITE_BUFFER_SIZE = key("remote.fs.write-buffer-size") .memoryType() @@ -2066,4 +2104,10 @@ private static class ConfigOptionsHolder { public static ConfigOption getConfigOption(String key) { return ConfigOptionsHolder.CONFIG_OPTIONS_BY_KEY.get(key); } + + /** Remote data dir select strategy for Fluss. */ + public enum RemoteDataDirStrategy { + ROUND_ROBIN, + WEIGHTED_ROUND_ROBIN + } } diff --git a/fluss-common/src/main/java/org/apache/fluss/config/FlussConfigUtils.java b/fluss-common/src/main/java/org/apache/fluss/config/FlussConfigUtils.java index 08b97256c4..9891a58df4 100644 --- a/fluss-common/src/main/java/org/apache/fluss/config/FlussConfigUtils.java +++ b/fluss-common/src/main/java/org/apache/fluss/config/FlussConfigUtils.java @@ -19,12 +19,16 @@ import org.apache.fluss.annotation.Internal; import org.apache.fluss.annotation.VisibleForTesting; +import org.apache.fluss.exception.IllegalConfigurationException; +import org.apache.fluss.fs.FsPath; +import org.apache.fluss.utils.FlussPaths; import java.lang.reflect.Field; import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; /** Utilities of Fluss {@link ConfigOptions}. */ @Internal @@ -77,4 +81,109 @@ static Map> extractConfigOptions(String prefix) { } return options; } + + public static void validateCoordinatorConfigs(Configuration conf) { + validServerConfigs(conf); + + validMinValue(conf, ConfigOptions.DEFAULT_REPLICATION_FACTOR, 1); + validMinValue(conf, ConfigOptions.KV_MAX_RETAINED_SNAPSHOTS, 1); + validMinValue(conf, ConfigOptions.SERVER_IO_POOL_SIZE, 1); + + // Validate remote.data.dirs + List remoteDataDirs = conf.get(ConfigOptions.REMOTE_DATA_DIRS); + for (int i = 0; i < remoteDataDirs.size(); i++) { + String remoteDataDir = remoteDataDirs.get(i); + try { + new FsPath(remoteDataDir); + } catch (Exception e) { + throw new IllegalConfigurationException( + String.format( + "Invalid remote path for %s at index %d.", + ConfigOptions.REMOTE_DATA_DIRS.key(), i), + e); + } + } + + // Validate remote.data.dirs.strategy + ConfigOptions.RemoteDataDirStrategy remoteDataDirStrategy = + conf.get(ConfigOptions.REMOTE_DATA_DIRS_STRATEGY); + if (remoteDataDirStrategy == ConfigOptions.RemoteDataDirStrategy.WEIGHTED_ROUND_ROBIN) { + List weights = conf.get(ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS); + if (!remoteDataDirs.isEmpty() && !weights.isEmpty()) { + if (remoteDataDirs.size() != weights.size()) { + throw new IllegalConfigurationException( + String.format( + "The size of '%s' (%d) must match the size of '%s' (%d) when using WEIGHTED_ROUND_ROBIN strategy.", + ConfigOptions.REMOTE_DATA_DIRS.key(), + remoteDataDirs.size(), + ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS.key(), + weights.size())); + } + // Validate all weights are positive + for (int i = 0; i < weights.size(); i++) { + if (weights.get(i) < 0) { + throw new IllegalConfigurationException( + String.format( + "All weights in '%s' must be no less than 0, but found %d at index %d.", + ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS.key(), + weights.get(i), + i)); + } + } + } + } + } + + public static void validateTabletConfigs(Configuration conf) { + validServerConfigs(conf); + + Optional serverId = conf.getOptional(ConfigOptions.TABLET_SERVER_ID); + if (!serverId.isPresent()) { + throw new IllegalConfigurationException( + String.format("Configuration %s must be set.", ConfigOptions.TABLET_SERVER_ID)); + } + validMinValue(ConfigOptions.TABLET_SERVER_ID, serverId.get(), 0); + + validMinValue(conf, ConfigOptions.BACKGROUND_THREADS, 1); + + if (conf.get(ConfigOptions.LOG_SEGMENT_FILE_SIZE).getBytes() > Integer.MAX_VALUE) { + throw new IllegalConfigurationException( + String.format( + "Invalid configuration for %s, it must be less than or equal %d bytes.", + ConfigOptions.LOG_SEGMENT_FILE_SIZE.key(), Integer.MAX_VALUE)); + } + } + + /** Validate common server configs. */ + private static void validServerConfigs(Configuration conf) { + if (conf.get(ConfigOptions.REMOTE_DATA_DIR) == null) { + throw new IllegalConfigurationException( + String.format("Configuration %s must be set.", ConfigOptions.REMOTE_DATA_DIR)); + } else { + // Must validate that remote.data.dir is a valid FsPath + try { + FlussPaths.remoteDataDir(conf); + } catch (Exception e) { + throw new IllegalConfigurationException( + String.format( + "Invalid configuration for %s.", + ConfigOptions.REMOTE_DATA_DIR.key()), + e); + } + } + } + + private static void validMinValue( + Configuration conf, ConfigOption option, int minValue) { + validMinValue(option, conf.get(option), minValue); + } + + private static void validMinValue(ConfigOption option, int value, int minValue) { + if (value < minValue) { + throw new IllegalConfigurationException( + String.format( + "Invalid configuration for %s, it must be greater than or equal %d.", + option.key(), minValue)); + } + } } diff --git a/fluss-common/src/main/java/org/apache/fluss/metadata/PartitionInfo.java b/fluss-common/src/main/java/org/apache/fluss/metadata/PartitionInfo.java index c6c9fb2585..1462513ff5 100644 --- a/fluss-common/src/main/java/org/apache/fluss/metadata/PartitionInfo.java +++ b/fluss-common/src/main/java/org/apache/fluss/metadata/PartitionInfo.java @@ -22,8 +22,8 @@ import java.util.Objects; /** - * Information of a partition metadata, includes the partition's name and the partition id that - * represents the unique identifier of the partition. + * Information of a partition metadata, includes partition id (unique identifier of the partition), + * partition name, remote data dir for partitioned data storage, etc. * * @since 0.2 */ @@ -31,10 +31,13 @@ public class PartitionInfo { private final long partitionId; private final ResolvedPartitionSpec partitionSpec; + private final String remoteDataDir; - public PartitionInfo(long partitionId, ResolvedPartitionSpec partitionSpec) { + public PartitionInfo( + long partitionId, ResolvedPartitionSpec partitionSpec, String remoteDataDir) { this.partitionId = partitionId; this.partitionSpec = partitionSpec; + this.remoteDataDir = remoteDataDir; } /** Get the partition id. The id is globally unique in the Fluss cluster. */ @@ -58,6 +61,10 @@ public PartitionSpec getPartitionSpec() { return partitionSpec.toPartitionSpec(); } + public String getRemoteDataDir() { + return remoteDataDir; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -67,16 +74,25 @@ public boolean equals(Object o) { return false; } PartitionInfo that = (PartitionInfo) o; - return partitionId == that.partitionId && Objects.equals(partitionSpec, that.partitionSpec); + return partitionId == that.partitionId + && Objects.equals(partitionSpec, that.partitionSpec) + && Objects.equals(remoteDataDir, that.remoteDataDir); } @Override public int hashCode() { - return Objects.hash(partitionId, partitionSpec); + return Objects.hash(partitionId, partitionSpec, remoteDataDir); } @Override public String toString() { - return "Partition{name='" + getPartitionName() + '\'' + ", id=" + partitionId + '}'; + return "Partition{name='" + + getPartitionName() + + '\'' + + ", id=" + + partitionId + + ", remoteDataDir=" + + remoteDataDir + + '}'; } } diff --git a/fluss-common/src/main/java/org/apache/fluss/metadata/TableInfo.java b/fluss-common/src/main/java/org/apache/fluss/metadata/TableInfo.java index be894ee790..afda3a28a0 100644 --- a/fluss-common/src/main/java/org/apache/fluss/metadata/TableInfo.java +++ b/fluss-common/src/main/java/org/apache/fluss/metadata/TableInfo.java @@ -59,6 +59,7 @@ public final class TableInfo { private final Configuration properties; private final TableConfig tableConfig; private final Configuration customProperties; + private final String remoteDataDir; private final @Nullable String comment; private final long createdTime; @@ -74,6 +75,7 @@ public TableInfo( int numBuckets, Configuration properties, Configuration customProperties, + String remoteDataDir, @Nullable String comment, long createdTime, long modifiedTime) { @@ -90,6 +92,7 @@ public TableInfo( this.properties = properties; this.tableConfig = new TableConfig(properties); this.customProperties = customProperties; + this.remoteDataDir = remoteDataDir; this.comment = comment; this.createdTime = createdTime; this.modifiedTime = modifiedTime; @@ -263,6 +266,11 @@ public Configuration getCustomProperties() { return customProperties; } + /** Returns the remote data directory of the table. */ + public String getRemoteDataDir() { + return remoteDataDir; + } + /** Returns the comment/description of the table. */ public Optional getComment() { return Optional.ofNullable(comment); @@ -314,6 +322,7 @@ public static TableInfo of( long tableId, int schemaId, TableDescriptor tableDescriptor, + String remoteDataDir, long createdTime, long modifiedTime) { Schema schema = tableDescriptor.getSchema(); @@ -335,6 +344,7 @@ public static TableInfo of( numBuckets, Configuration.fromMap(tableDescriptor.getProperties()), Configuration.fromMap(tableDescriptor.getCustomProperties()), + remoteDataDir, tableDescriptor.getComment().orElse(null), createdTime, modifiedTime); @@ -358,6 +368,7 @@ public boolean equals(Object o) { && Objects.equals(partitionKeys, that.partitionKeys) && Objects.equals(properties, that.properties) && Objects.equals(customProperties, that.customProperties) + && Objects.equals(remoteDataDir, that.remoteDataDir) && Objects.equals(comment, that.comment); } @@ -376,6 +387,7 @@ public int hashCode() { numBuckets, properties, customProperties, + remoteDataDir, comment); } @@ -402,6 +414,8 @@ public String toString() { + properties + ", customProperties=" + customProperties + + ", remoteDataDir=" + + remoteDataDir + ", comment='" + comment + '\'' diff --git a/fluss-common/src/main/java/org/apache/fluss/utils/FlussPaths.java b/fluss-common/src/main/java/org/apache/fluss/utils/FlussPaths.java index 9a0659f180..d6d3978964 100644 --- a/fluss-common/src/main/java/org/apache/fluss/utils/FlussPaths.java +++ b/fluss-common/src/main/java/org/apache/fluss/utils/FlussPaths.java @@ -409,6 +409,23 @@ public static UUID uuidFromRemoteIndexCacheFileName(String fileName) { fileName.substring(fileName.indexOf('_') + 1, fileName.indexOf('.'))); } + // ---------------------------------------------------------------------------------------- + // Remote Data Paths + // ---------------------------------------------------------------------------------------- + + /** + * Returns the remote root directory path for storing data files. + * + *

The path contract: + * + *

+     * {$remote.data.dir}
+     * 
+ */ + public static FsPath remoteDataDir(Configuration conf) { + return new FsPath(conf.get(ConfigOptions.REMOTE_DATA_DIR)); + } + // ---------------------------------------------------------------------------------------- // Remote Log Paths // ---------------------------------------------------------------------------------------- diff --git a/fluss-common/src/test/java/org/apache/fluss/config/FlussConfigUtilsTest.java b/fluss-common/src/test/java/org/apache/fluss/config/FlussConfigUtilsTest.java index e24bc121aa..7ff0d69e06 100644 --- a/fluss-common/src/test/java/org/apache/fluss/config/FlussConfigUtilsTest.java +++ b/fluss-common/src/test/java/org/apache/fluss/config/FlussConfigUtilsTest.java @@ -17,14 +17,20 @@ package org.apache.fluss.config; +import org.apache.fluss.exception.IllegalConfigurationException; + import org.junit.jupiter.api.Test; +import java.util.Arrays; +import java.util.Collections; import java.util.Map; import static org.apache.fluss.config.FlussConfigUtils.CLIENT_OPTIONS; import static org.apache.fluss.config.FlussConfigUtils.TABLE_OPTIONS; import static org.apache.fluss.config.FlussConfigUtils.extractConfigOptions; +import static org.apache.fluss.config.FlussConfigUtils.validateCoordinatorConfigs; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; /** Test for {@link FlussConfigUtils}. */ class FlussConfigUtilsTest { @@ -49,4 +55,105 @@ void testExtractOptions() { }); assertThat(clientOptions.size()).isEqualTo(CLIENT_OPTIONS.size()); } + + @Test + void testValidateCoordinatorConfigs() { + // Test valid configuration + Configuration validConf = new Configuration(); + validConf.set(ConfigOptions.REMOTE_DATA_DIR, "s3://bucket/path"); + validateCoordinatorConfigs(validConf); + + // Test invalid DEFAULT_REPLICATION_FACTOR + Configuration invalidReplicationConf = new Configuration(); + invalidReplicationConf.set(ConfigOptions.REMOTE_DATA_DIR, "s3://bucket/path"); + invalidReplicationConf.set(ConfigOptions.DEFAULT_REPLICATION_FACTOR, 0); + assertThatThrownBy(() -> validateCoordinatorConfigs(invalidReplicationConf)) + .isInstanceOf(IllegalConfigurationException.class) + .hasMessageContaining(ConfigOptions.DEFAULT_REPLICATION_FACTOR.key()) + .hasMessageContaining("must be greater than or equal 1"); + + // Test invalid KV_MAX_RETAINED_SNAPSHOTS + Configuration invalidSnapshotConf = new Configuration(); + invalidSnapshotConf.set(ConfigOptions.REMOTE_DATA_DIR, "s3://bucket/path"); + invalidSnapshotConf.set(ConfigOptions.KV_MAX_RETAINED_SNAPSHOTS, 0); + assertThatThrownBy(() -> validateCoordinatorConfigs(invalidSnapshotConf)) + .isInstanceOf(IllegalConfigurationException.class) + .hasMessageContaining(ConfigOptions.KV_MAX_RETAINED_SNAPSHOTS.key()) + .hasMessageContaining("must be greater than or equal 1"); + + // Test invalid SERVER_IO_POOL_SIZE + Configuration invalidIoPoolConf = new Configuration(); + invalidIoPoolConf.set(ConfigOptions.REMOTE_DATA_DIR, "s3://bucket/path"); + invalidIoPoolConf.set(ConfigOptions.SERVER_IO_POOL_SIZE, 0); + assertThatThrownBy(() -> validateCoordinatorConfigs(invalidIoPoolConf)) + .isInstanceOf(IllegalConfigurationException.class) + .hasMessageContaining(ConfigOptions.SERVER_IO_POOL_SIZE.key()) + .hasMessageContaining("must be greater than or equal 1"); + + // Test REMOTE_DATA_DIR not set + Configuration noRemoteDirConf = new Configuration(); + assertThatThrownBy(() -> validateCoordinatorConfigs(noRemoteDirConf)) + .isInstanceOf(IllegalConfigurationException.class) + .hasMessageContaining(ConfigOptions.REMOTE_DATA_DIR.key()) + .hasMessageContaining("must be set"); + + // Test invalid REMOTE_DATA_DIR + Configuration invalidRemoteDirConf = new Configuration(); + invalidRemoteDirConf.set(ConfigOptions.REMOTE_DATA_DIR, "123://invalid.com"); + assertThatThrownBy(() -> validateCoordinatorConfigs(invalidRemoteDirConf)) + .isInstanceOf(IllegalConfigurationException.class) + .hasMessageContaining(ConfigOptions.REMOTE_DATA_DIR.key()) + .hasMessageContaining("Invalid configuration for remote.data.dir"); + + // Test REMOTE_DATA_DIRS contains invalid path + Configuration invalidRemoteDirsConf = new Configuration(); + invalidRemoteDirsConf.set(ConfigOptions.REMOTE_DATA_DIR, "s3://bucket/path"); + invalidRemoteDirsConf.set( + ConfigOptions.REMOTE_DATA_DIRS, Arrays.asList("s3://bucket1", "123://invalid.com")); + assertThatThrownBy(() -> validateCoordinatorConfigs(invalidRemoteDirsConf)) + .isInstanceOf(IllegalConfigurationException.class) + .hasMessageContaining(ConfigOptions.REMOTE_DATA_DIRS.key()) + .hasMessageContaining("Invalid remote path for remote.data.dirs"); + + // Test WEIGHTED_ROUND_ROBIN with mismatched sizes + Configuration mismatchedWeightsConf = new Configuration(); + mismatchedWeightsConf.set(ConfigOptions.REMOTE_DATA_DIR, "s3://bucket/path"); + mismatchedWeightsConf.set( + ConfigOptions.REMOTE_DATA_DIRS_STRATEGY, + ConfigOptions.RemoteDataDirStrategy.WEIGHTED_ROUND_ROBIN); + mismatchedWeightsConf.set( + ConfigOptions.REMOTE_DATA_DIRS, Arrays.asList("s3://bucket1", "s3://bucket2")); + mismatchedWeightsConf.set( + ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS, Collections.singletonList(1)); + assertThatThrownBy(() -> validateCoordinatorConfigs(mismatchedWeightsConf)) + .isInstanceOf(IllegalConfigurationException.class) + .hasMessageContaining(ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS.key()) + .hasMessageContaining(ConfigOptions.REMOTE_DATA_DIRS.key()); + + // Test WEIGHTED_ROUND_ROBIN with matched sizes + Configuration matchedWeightsConf = new Configuration(); + matchedWeightsConf.set(ConfigOptions.REMOTE_DATA_DIR, "s3://bucket/path"); + matchedWeightsConf.set( + ConfigOptions.REMOTE_DATA_DIRS_STRATEGY, + ConfigOptions.RemoteDataDirStrategy.WEIGHTED_ROUND_ROBIN); + matchedWeightsConf.set( + ConfigOptions.REMOTE_DATA_DIRS, Arrays.asList("s3://bucket1", "s3://bucket2")); + matchedWeightsConf.set(ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS, Arrays.asList(1, 2)); + validateCoordinatorConfigs(matchedWeightsConf); + + // Test negative weight + Configuration negativeWeightConf = new Configuration(); + negativeWeightConf.set(ConfigOptions.REMOTE_DATA_DIR, "s3://bucket/path"); + negativeWeightConf.set( + ConfigOptions.REMOTE_DATA_DIRS_STRATEGY, + ConfigOptions.RemoteDataDirStrategy.WEIGHTED_ROUND_ROBIN); + negativeWeightConf.set( + ConfigOptions.REMOTE_DATA_DIRS, Arrays.asList("s3://bucket1", "s3://bucket2")); + negativeWeightConf.set(ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS, Arrays.asList(-1, 2)); + assertThatThrownBy(() -> validateCoordinatorConfigs(negativeWeightConf)) + .isInstanceOf(IllegalConfigurationException.class) + .hasMessageContaining(ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS.key()) + .hasMessageContaining( + "All weights in 'remote.data.dirs.weights' must be no less than 0"); + } } diff --git a/fluss-common/src/test/java/org/apache/fluss/record/TestData.java b/fluss-common/src/test/java/org/apache/fluss/record/TestData.java index 28ab9ececc..a4071d2aaa 100644 --- a/fluss-common/src/test/java/org/apache/fluss/record/TestData.java +++ b/fluss-common/src/test/java/org/apache/fluss/record/TestData.java @@ -39,6 +39,7 @@ public final class TestData { public static final short DEFAULT_SCHEMA_ID = 1; public static final long BASE_OFFSET = 0L; public static final byte DEFAULT_MAGIC = CURRENT_LOG_MAGIC_VALUE; + public static final String DEFAULT_REMOTE_DATA_DIR = "/tmp/fluss/remote-data"; // ---------------------------- data1 and related table info begin --------------------------- public static final List DATA1 = Arrays.asList( @@ -93,6 +94,7 @@ public final class TestData { DATA1_TABLE_ID, 1, DATA1_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, currentMillis, currentMillis); @@ -118,6 +120,7 @@ public final class TestData { PARTITION_TABLE_ID, 1, DATA1_PARTITIONED_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis()); @@ -148,6 +151,7 @@ public final class TestData { DATA1_TABLE_ID_PK, 1, DATA1_TABLE_DESCRIPTOR_PK, + DEFAULT_REMOTE_DATA_DIR, currentMillis, currentMillis); @@ -217,6 +221,7 @@ public final class TestData { DATA2_TABLE_ID, 1, DATA2_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis()); // -------------------------------- data2 info end ------------------------------------ diff --git a/fluss-common/src/test/java/org/apache/fluss/utils/PartitionUtilsTest.java b/fluss-common/src/test/java/org/apache/fluss/utils/PartitionUtilsTest.java index 686e5ce479..dae176265e 100644 --- a/fluss-common/src/test/java/org/apache/fluss/utils/PartitionUtilsTest.java +++ b/fluss-common/src/test/java/org/apache/fluss/utils/PartitionUtilsTest.java @@ -40,6 +40,7 @@ import static org.apache.fluss.metadata.TablePath.detectInvalidName; import static org.apache.fluss.record.TestData.DATA1_SCHEMA; import static org.apache.fluss.record.TestData.DATA1_TABLE_PATH; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.utils.PartitionUtils.convertValueOfType; import static org.apache.fluss.utils.PartitionUtils.generateAutoPartition; import static org.apache.fluss.utils.PartitionUtils.validatePartitionSpec; @@ -84,7 +85,8 @@ void testValidatePartitionValues() { ConfigOptions.TABLE_AUTO_PARTITION_TIME_UNIT, AutoPartitionTimeUnit.YEAR) .build(); - TableInfo tableInfo = TableInfo.of(DATA1_TABLE_PATH, 1L, 1, descriptor, 1L, 1L); + TableInfo tableInfo = + TableInfo.of(DATA1_TABLE_PATH, 1L, 1, descriptor, DEFAULT_REMOTE_DATA_DIR, 1L, 1L); assertThatThrownBy( () -> validatePartitionSpec( diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/undo/RecoveryOffsetManagerTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/undo/RecoveryOffsetManagerTest.java index f39e288e95..2e072ef757 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/undo/RecoveryOffsetManagerTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/undo/RecoveryOffsetManagerTest.java @@ -44,6 +44,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -82,6 +83,7 @@ private static TableInfo createTableInfo(int numBuckets, boolean isPartitioned) numBuckets, new Configuration(), new Configuration(), + DEFAULT_REMOTE_DATA_DIR, null, // comment System.currentTimeMillis(), System.currentTimeMillis()); @@ -89,7 +91,7 @@ private static TableInfo createTableInfo(int numBuckets, boolean isPartitioned) private static PartitionInfo createPartitionInfo(long partitionId, String partitionName) { ResolvedPartitionSpec spec = ResolvedPartitionSpec.fromPartitionValue("pt", partitionName); - return new PartitionInfo(partitionId, spec); + return new PartitionInfo(partitionId, spec, DEFAULT_REMOTE_DATA_DIR); } // ==================== FRESH_START Tests ==================== diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumeratorTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumeratorTest.java index c895142438..ca4722c9e6 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumeratorTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumeratorTest.java @@ -76,6 +76,7 @@ import java.util.stream.Collectors; import static org.apache.fluss.client.table.scanner.log.LogScanner.EARLIEST_OFFSET; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.testutils.DataTestUtils.row; import static org.assertj.core.api.Assertions.assertThat; @@ -646,13 +647,15 @@ void testPartitionsExpiredInFlussButExistInLake( ResolvedPartitionSpec.fromPartitionName( Collections.singletonList(isPrimaryKeyTable ? "date" : "name"), partitionName); - lakePartitionInfos.add(new PartitionInfo(partitionId, partitionSpec)); + lakePartitionInfos.add( + new PartitionInfo(partitionId, partitionSpec, DEFAULT_REMOTE_DATA_DIR)); } ResolvedPartitionSpec partitionSpec = ResolvedPartitionSpec.fromPartitionName( Collections.singletonList(isPrimaryKeyTable ? "date" : "name"), hybridPartitionName); - lakePartitionInfos.add(new PartitionInfo(hybridPartitionId, partitionSpec)); + lakePartitionInfos.add( + new PartitionInfo(hybridPartitionId, partitionSpec, DEFAULT_REMOTE_DATA_DIR)); LakeSource lakeSource = new TestingLakeSource(DEFAULT_BUCKET_NUM, lakePartitionInfos); diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlinkConversionsTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlinkConversionsTest.java index a467bb14a2..efc386eb64 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlinkConversionsTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlinkConversionsTest.java @@ -57,6 +57,7 @@ import static org.apache.fluss.flink.FlinkConnectorOptions.BUCKET_NUMBER; import static org.apache.fluss.flink.utils.CatalogTableTestUtils.addOptions; import static org.apache.fluss.flink.utils.CatalogTableTestUtils.checkEqualsIgnoreSchema; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.types.DataTypes.FIELD; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -257,6 +258,7 @@ void testTableConversion() { 1L, 1, flussTable.withBucketCount(1), + DEFAULT_REMOTE_DATA_DIR, currentMillis, currentMillis); // get the converted flink table @@ -446,6 +448,7 @@ void testFlinkMaterializedTableConversions() { 1L, 1, flussTable.withBucketCount(1), + DEFAULT_REMOTE_DATA_DIR, currentMillis, currentMillis); // get the converted flink table @@ -508,6 +511,7 @@ void testAggregationFunctionRoundTrip() { 1L, 1, flussTable.withBucketCount(1), + DEFAULT_REMOTE_DATA_DIR, currentMillis, currentMillis); CatalogTable convertedFlinkTable = (CatalogTable) FlinkConversions.toFlinkTable(tableInfo); diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlinkTestBase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlinkTestBase.java index 7c80bbc0ca..5d1963729c 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlinkTestBase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlinkTestBase.java @@ -237,6 +237,7 @@ public static Map createPartitions( partition, new PartitionAssignment( tableInfo.getTableId(), assignment.getBucketAssignments()), + zkClient.getRemoteDataDir(), tablePath, tableInfo.getTableId()); } diff --git a/fluss-lake/fluss-lake-iceberg/src/test/java/org/apache/fluss/lake/iceberg/tiering/IcebergTieringTest.java b/fluss-lake/fluss-lake-iceberg/src/test/java/org/apache/fluss/lake/iceberg/tiering/IcebergTieringTest.java index eb24994cc4..d5255005c7 100644 --- a/fluss-lake/fluss-lake-iceberg/src/test/java/org/apache/fluss/lake/iceberg/tiering/IcebergTieringTest.java +++ b/fluss-lake/fluss-lake-iceberg/src/test/java/org/apache/fluss/lake/iceberg/tiering/IcebergTieringTest.java @@ -77,6 +77,7 @@ import static org.apache.fluss.record.ChangeType.INSERT; import static org.apache.fluss.record.ChangeType.UPDATE_AFTER; import static org.apache.fluss.record.ChangeType.UPDATE_BEFORE; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.iceberg.expressions.Expressions.equal; import static org.assertj.core.api.Assertions.assertThat; @@ -134,7 +135,8 @@ void testTieringWriteTable(boolean isPrimaryKeyTable, boolean isPartitionedTable .distributedBy(BUCKET_NUM) .property(ConfigOptions.TABLE_DATALAKE_ENABLED, true) .build(); - TableInfo tableInfo = TableInfo.of(tablePath, 0, 1, descriptor, 1L, 1L); + TableInfo tableInfo = + TableInfo.of(tablePath, 0, 1, descriptor, DEFAULT_REMOTE_DATA_DIR, 1L, 1L); Table icebergTable = icebergCatalog.loadTable(toIceberg(tablePath)); diff --git a/fluss-lake/fluss-lake-lance/src/test/java/org/apache/fluss/lake/lance/tiering/LanceTieringTest.java b/fluss-lake/fluss-lake-lance/src/test/java/org/apache/fluss/lake/lance/tiering/LanceTieringTest.java index a7a13ff5e2..28aeba37bf 100644 --- a/fluss-lake/fluss-lake-lance/src/test/java/org/apache/fluss/lake/lance/tiering/LanceTieringTest.java +++ b/fluss-lake/fluss-lake-lance/src/test/java/org/apache/fluss/lake/lance/tiering/LanceTieringTest.java @@ -68,6 +68,7 @@ import java.util.stream.Stream; import static org.apache.fluss.lake.committer.LakeCommitter.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.assertj.core.api.Assertions.assertThat; /** The UT for tiering to Lance via {@link LanceLakeTieringFactory}. */ @@ -114,7 +115,8 @@ void testTieringWriteTable(boolean isPartitioned) throws Exception { .property(ConfigOptions.TABLE_DATALAKE_ENABLED, true) .customProperties(customProperties) .build(); - TableInfo tableInfo = TableInfo.of(tablePath, 0, 1, descriptor, 1L, 1L); + TableInfo tableInfo = + TableInfo.of(tablePath, 0, 1, descriptor, DEFAULT_REMOTE_DATA_DIR, 1L, 1L); List lanceWriteResults = new ArrayList<>(); SimpleVersionedSerializer writeResultSerializer = diff --git a/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/tiering/PaimonTieringTest.java b/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/tiering/PaimonTieringTest.java index 600c3ba661..958d134b77 100644 --- a/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/tiering/PaimonTieringTest.java +++ b/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/tiering/PaimonTieringTest.java @@ -80,6 +80,7 @@ import static org.apache.fluss.record.ChangeType.INSERT; import static org.apache.fluss.record.ChangeType.UPDATE_AFTER; import static org.apache.fluss.record.ChangeType.UPDATE_BEFORE; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.assertj.core.api.Assertions.assertThat; /** The UT for tiering to Paimon via {@link PaimonLakeTieringFactory}. */ @@ -147,7 +148,8 @@ void testTieringWriteTable(boolean isPrimaryKeyTable, boolean isPartitioned) thr .distributedBy(bucketNum) .property(ConfigOptions.TABLE_DATALAKE_ENABLED, true) .build(); - TableInfo tableInfo = TableInfo.of(tablePath, 0, 1, descriptor, 1L, 1L); + TableInfo tableInfo = + TableInfo.of(tablePath, 0, 1, descriptor, DEFAULT_REMOTE_DATA_DIR, 1L, 1L); try (LakeCommitter lakeCommitter = createLakeCommitter(tablePath, tableInfo, new Configuration())) { @@ -219,7 +221,8 @@ void testMultiPartitionTiering() throws Exception { .distributedBy(1) .property(ConfigOptions.TABLE_DATALAKE_ENABLED, true) .build(); - TableInfo tableInfo = TableInfo.of(tablePath, 0, 1, descriptor, 1L, 1L); + TableInfo tableInfo = + TableInfo.of(tablePath, 0, 1, descriptor, DEFAULT_REMOTE_DATA_DIR, 1L, 1L); Map> recordsByPartition = new HashMap<>(); List paimonWriteResults = new ArrayList<>(); @@ -293,7 +296,8 @@ void testThreePartitionTiering() throws Exception { .distributedBy(1) .property(ConfigOptions.TABLE_DATALAKE_ENABLED, true) .build(); - TableInfo tableInfo = TableInfo.of(tablePath, 0, 1, descriptor, 1L, 1L); + TableInfo tableInfo = + TableInfo.of(tablePath, 0, 1, descriptor, DEFAULT_REMOTE_DATA_DIR, 1L, 1L); Map> recordsByPartition = new HashMap<>(); List paimonWriteResults = new ArrayList<>(); @@ -382,7 +386,8 @@ void testSnapshotExpiration( ConfigOptions.TABLE_DATALAKE_AUTO_EXPIRE_SNAPSHOT, isTableAutoExpireSnapshot) .build(); - TableInfo tableInfo = TableInfo.of(tablePath, 0, 1, descriptor, 1L, 1L); + TableInfo tableInfo = + TableInfo.of(tablePath, 0, 1, descriptor, DEFAULT_REMOTE_DATA_DIR, 1L, 1L); // Get the FileStoreTable to verify snapshots FileStoreTable fileStoreTable = (FileStoreTable) paimonCatalog.getTable(toPaimon(tablePath)); diff --git a/fluss-rpc/src/main/proto/FlussApi.proto b/fluss-rpc/src/main/proto/FlussApi.proto index f04b307565..1de027e8f4 100644 --- a/fluss-rpc/src/main/proto/FlussApi.proto +++ b/fluss-rpc/src/main/proto/FlussApi.proto @@ -139,6 +139,7 @@ message GetTableInfoResponse { required bytes table_json = 3; required int64 created_time = 4; required int64 modified_time = 5; + optional string remote_data_dir = 6; } // list tables request and response @@ -762,6 +763,7 @@ message PbTableMetadata { repeated PbBucketMetadata bucket_metadata = 5; required int64 created_time = 6; required int64 modified_time = 7; + optional string remote_data_dir = 8; // TODO add a new filed 'deleted_table' to indicate this table is deleted in UpdateMetadataRequest. // trace by: https://github.com/apache/fluss/issues/981 @@ -1006,6 +1008,7 @@ message PbRemoteLogSegment { message PbPartitionInfo { required int64 partition_id = 1; required PbPartitionSpec partition_spec = 2; + optional string remote_data_dir = 3; } message PbPartitionSpec { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/RpcServiceBase.java b/fluss-server/src/main/java/org/apache/fluss/server/RpcServiceBase.java index 2946fb93bc..e0af1b6c8e 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/RpcServiceBase.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/RpcServiceBase.java @@ -36,7 +36,6 @@ import org.apache.fluss.metadata.SchemaInfo; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableInfo; -import org.apache.fluss.metadata.TablePartition; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.rpc.RpcGatewayService; import org.apache.fluss.rpc.gateway.AdminReadOnlyGateway; @@ -93,6 +92,7 @@ import org.apache.fluss.server.utils.ServerRpcMessageUtils; import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.server.zk.data.BucketSnapshot; +import org.apache.fluss.server.zk.data.PartitionRegistration; import org.apache.fluss.server.zk.data.lake.LakeTableSnapshot; import org.slf4j.Logger; @@ -362,21 +362,21 @@ public CompletableFuture getLatestKvSnapshots( } private long getPartitionId(TablePath tablePath, String partitionName) { - Optional optTablePartition; + Optional optPartitionRegistration; try { - optTablePartition = zkClient.getPartition(tablePath, partitionName); + optPartitionRegistration = zkClient.getPartition(tablePath, partitionName); } catch (Exception e) { throw new FlussRuntimeException( String.format("Failed to get latest kv snapshots for table '%s'", tablePath), e); } - if (!optTablePartition.isPresent()) { + if (!optPartitionRegistration.isPresent()) { throw new PartitionNotExistException( String.format( "The partition '%s' of table '%s' does not exist.", partitionName, tablePath)); } - return optTablePartition.get().getPartitionId(); + return optPartitionRegistration.get().getPartitionId(); } @Override diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/AutoPartitionManager.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/AutoPartitionManager.java index 5fb39037ff..d5d50d0932 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/AutoPartitionManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/AutoPartitionManager.java @@ -29,6 +29,7 @@ import org.apache.fluss.metadata.ResolvedPartitionSpec; import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.server.coordinator.remote.RemoteDirDynamicLoader; import org.apache.fluss.server.metadata.ServerMetadataCache; import org.apache.fluss.server.zk.data.BucketAssignment; import org.apache.fluss.server.zk.data.PartitionAssignment; @@ -85,6 +86,7 @@ public class AutoPartitionManager implements AutoCloseable { private final ServerMetadataCache metadataCache; private final MetadataManager metadataManager; + private final RemoteDirDynamicLoader remoteDirDynamicLoader; private final Clock clock; private final long periodicInterval; @@ -108,10 +110,12 @@ public class AutoPartitionManager implements AutoCloseable { public AutoPartitionManager( ServerMetadataCache metadataCache, MetadataManager metadataManager, + RemoteDirDynamicLoader remoteDirDynamicLoader, Configuration conf) { this( metadataCache, metadataManager, + remoteDirDynamicLoader, conf, SystemClock.getInstance(), Executors.newScheduledThreadPool( @@ -122,11 +126,13 @@ public AutoPartitionManager( AutoPartitionManager( ServerMetadataCache metadataCache, MetadataManager metadataManager, + RemoteDirDynamicLoader remoteDirDynamicLoader, Configuration conf, Clock clock, ScheduledExecutorService periodicExecutor) { this.metadataCache = metadataCache; this.metadataManager = metadataManager; + this.remoteDirDynamicLoader = remoteDirDynamicLoader; this.clock = clock; this.periodicExecutor = periodicExecutor; this.periodicInterval = conf.get(ConfigOptions.AUTO_PARTITION_CHECK_INTERVAL).toMillis(); @@ -349,8 +355,11 @@ private void createPartitions( PartitionAssignment partitionAssignment = new PartitionAssignment(tableInfo.getTableId(), bucketAssignments); + // select a remote data dir for the partition + String remoteDataDir = + remoteDirDynamicLoader.getRemoteDataDirContainer().nextDataDir(); metadataManager.createPartition( - tablePath, tableId, partitionAssignment, partition, false); + tablePath, tableId, remoteDataDir, partitionAssignment, partition, false); // only single partition key table supports automatic creation of partitions currentPartitions.put(partition.getPartitionName(), null); LOG.info( diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java index 909fa503dc..808795e852 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java @@ -716,6 +716,7 @@ private void processSchemaChange(SchemaChangeEvent schemaChangeEvent) { oldTableInfo.getNumBuckets(), oldTableInfo.getProperties(), oldTableInfo.getCustomProperties(), + oldTableInfo.getRemoteDataDir(), oldTableInfo.getComment().orElse(null), oldTableInfo.getCreatedTime(), System.currentTimeMillis())); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorRequestBatch.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorRequestBatch.java index 063ba58786..60eaa988c9 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorRequestBatch.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorRequestBatch.java @@ -83,6 +83,7 @@ public class CoordinatorRequestBatch { private static final Schema EMPTY_SCHEMA = Schema.newBuilder().build(); private static final TableDescriptor EMPTY_TABLE_DESCRIPTOR = TableDescriptor.builder().schema(EMPTY_SCHEMA).distributedBy(0).build(); + private static final String EMPTY_REMOTE_DATA_DIR = ""; // a map from tablet server to notify the leader and isr for each bucket. private final Map> @@ -694,7 +695,13 @@ private TableInfo getTableInfo(long tableId) { if (tableInfo == null) { if (tableQueuedForDeletion) { return TableInfo.of( - DELETED_TABLE_PATH, tableId, 0, EMPTY_TABLE_DESCRIPTOR, -1L, -1L); + DELETED_TABLE_PATH, + tableId, + 0, + EMPTY_TABLE_DESCRIPTOR, + EMPTY_REMOTE_DATA_DIR, + -1L, + -1L); } else { // it may happen that the table is dropped, but the partition still exists // when coordinator restarts, it won't consider it as deleted table, @@ -710,6 +717,7 @@ private TableInfo getTableInfo(long tableId) { DELETED_TABLE_ID, 0, EMPTY_TABLE_DESCRIPTOR, + EMPTY_REMOTE_DATA_DIR, -1L, -1L) : tableInfo; diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java index ca3708bfea..53b98f2889 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java @@ -22,7 +22,6 @@ import org.apache.fluss.cluster.ServerType; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; -import org.apache.fluss.exception.IllegalConfigurationException; import org.apache.fluss.metadata.DatabaseDescriptor; import org.apache.fluss.metrics.registry.MetricRegistry; import org.apache.fluss.rpc.RpcClient; @@ -35,6 +34,7 @@ import org.apache.fluss.server.authorizer.AuthorizerLoader; import org.apache.fluss.server.coordinator.lease.KvSnapshotLeaseManager; import org.apache.fluss.server.coordinator.rebalance.RebalanceManager; +import org.apache.fluss.server.coordinator.remote.RemoteDirDynamicLoader; import org.apache.fluss.server.metadata.CoordinatorMetadataCache; import org.apache.fluss.server.metadata.ServerMetadataCache; import org.apache.fluss.server.metrics.ServerMetricUtils; @@ -66,6 +66,8 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import static org.apache.fluss.config.FlussConfigUtils.validateCoordinatorConfigs; + /** * Coordinator server implementation. The coordinator server is responsible to: * @@ -145,6 +147,9 @@ public class CoordinatorServer extends ServerBase { @GuardedBy("lock") private LakeCatalogDynamicLoader lakeCatalogDynamicLoader; + @GuardedBy("lock") + private RemoteDirDynamicLoader remoteDirDynamicLoader; + @GuardedBy("lock") private KvSnapshotLeaseManager kvSnapshotLeaseManager; @@ -154,7 +159,7 @@ public CoordinatorServer(Configuration conf) { public CoordinatorServer(Configuration conf, Clock clock) { super(conf); - validateConfigs(conf); + validateCoordinatorConfigs(conf); this.terminationFuture = new CompletableFuture<>(); this.clock = clock; } @@ -185,10 +190,13 @@ protected void startServices() throws Exception { this.zkClient = ZooKeeperUtils.startZookeeperClient(conf, this); this.lakeCatalogDynamicLoader = new LakeCatalogDynamicLoader(conf, pluginManager, true); + this.remoteDirDynamicLoader = new RemoteDirDynamicLoader(conf); + this.dynamicConfigManager = new DynamicConfigManager(zkClient, conf, true); // Register server reconfigurable components dynamicConfigManager.register(lakeCatalogDynamicLoader); + dynamicConfigManager.register(remoteDirDynamicLoader); dynamicConfigManager.startup(); @@ -231,6 +239,7 @@ protected void startServices() throws Exception { authorizer, lakeCatalogDynamicLoader, lakeTableTieringManager, + remoteDirDynamicLoader, dynamicConfigManager, ioExecutor, kvSnapshotLeaseManager); @@ -256,7 +265,8 @@ protected void startServices() throws Exception { this.coordinatorChannelManager = new CoordinatorChannelManager(rpcClient); this.autoPartitionManager = - new AutoPartitionManager(metadataCache, metadataManager, conf); + new AutoPartitionManager( + metadataCache, metadataManager, remoteDirDynamicLoader, conf); autoPartitionManager.start(); // start coordinator event processor after we register coordinator leader to zk @@ -476,6 +486,10 @@ CompletableFuture stopServices() { lakeCatalogDynamicLoader.close(); } + if (remoteDirDynamicLoader != null) { + remoteDirDynamicLoader.close(); + } + if (kvSnapshotLeaseManager != null) { kvSnapshotLeaseManager.close(); } @@ -549,31 +563,4 @@ public DynamicConfigManager getDynamicConfigManager() { public RebalanceManager getRebalanceManager() { return coordinatorEventProcessor.getRebalanceManager(); } - - private static void validateConfigs(Configuration conf) { - if (conf.get(ConfigOptions.DEFAULT_REPLICATION_FACTOR) < 1) { - throw new IllegalConfigurationException( - String.format( - "Invalid configuration for %s, it must be greater than or equal 1.", - ConfigOptions.DEFAULT_REPLICATION_FACTOR.key())); - } - if (conf.get(ConfigOptions.KV_MAX_RETAINED_SNAPSHOTS) < 1) { - throw new IllegalConfigurationException( - String.format( - "Invalid configuration for %s, it must be greater than or equal 1.", - ConfigOptions.KV_MAX_RETAINED_SNAPSHOTS.key())); - } - - if (conf.get(ConfigOptions.SERVER_IO_POOL_SIZE) < 1) { - throw new IllegalConfigurationException( - String.format( - "Invalid configuration for %s, it must be greater than or equal 1.", - ConfigOptions.SERVER_IO_POOL_SIZE.key())); - } - - if (conf.get(ConfigOptions.REMOTE_DATA_DIR) == null) { - throw new IllegalConfigurationException( - String.format("Configuration %s must be set.", ConfigOptions.REMOTE_DATA_DIR)); - } - } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorService.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorService.java index 388f1816f4..18c9712aa8 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorService.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorService.java @@ -149,6 +149,7 @@ import org.apache.fluss.server.coordinator.lease.KvSnapshotLeaseManager; import org.apache.fluss.server.coordinator.producer.ProducerOffsetsManager; import org.apache.fluss.server.coordinator.rebalance.goal.Goal; +import org.apache.fluss.server.coordinator.remote.RemoteDirDynamicLoader; import org.apache.fluss.server.entity.CommitKvSnapshotData; import org.apache.fluss.server.entity.LakeTieringTableInfo; import org.apache.fluss.server.entity.TablePropertyChanges; @@ -232,6 +233,7 @@ public final class CoordinatorService extends RpcServiceBase implements Coordina private final LakeTableHelper lakeTableHelper; private final ProducerOffsetsManager producerOffsetsManager; private final KvSnapshotLeaseManager kvSnapshotLeaseManager; + private final RemoteDirDynamicLoader remoteDirDynamicLoader; public CoordinatorService( Configuration conf, @@ -243,6 +245,7 @@ public CoordinatorService( @Nullable Authorizer authorizer, LakeCatalogDynamicLoader lakeCatalogDynamicLoader, LakeTableTieringManager lakeTableTieringManager, + RemoteDirDynamicLoader remoteDirDynamicLoader, DynamicConfigManager dynamicConfigManager, ExecutorService ioExecutor, KvSnapshotLeaseManager kvSnapshotLeaseManager) { @@ -268,6 +271,7 @@ public CoordinatorService( this.ioExecutor = ioExecutor; this.lakeTableHelper = new LakeTableHelper(zkClient, conf.getString(ConfigOptions.REMOTE_DATA_DIR)); + this.remoteDirDynamicLoader = remoteDirDynamicLoader; // Initialize and start the producer snapshot manager this.producerOffsetsManager = new ProducerOffsetsManager(conf, zkClient); @@ -432,9 +436,20 @@ public CompletableFuture createTable(CreateTableRequest req } } + // select remote data dir for table. + // remote data dir will be used to store table data for non-partitioned table and metadata + // (such as lake snapshot offset file) for partitioned table + RemoteDirDynamicLoader.RemoteDirContainer remoteDataDirContainer = + remoteDirDynamicLoader.getRemoteDataDirContainer(); + String remoteDataDir = remoteDataDirContainer.nextDataDir(); + // then create table; metadataManager.createTable( - tablePath, tableDescriptor, tableAssignment, request.isIgnoreIfExists()); + tablePath, + remoteDataDir, + tableDescriptor, + tableAssignment, + request.isIgnoreIfExists()); return CompletableFuture.completedFuture(new CreateTableResponse()); } @@ -637,9 +652,15 @@ public CompletableFuture createPartition( PartitionAssignment partitionAssignment = new PartitionAssignment(table.tableId, bucketAssignments); + // select remote data dir for partition + RemoteDirDynamicLoader.RemoteDirContainer remoteDataDirContainer = + remoteDirDynamicLoader.getRemoteDataDirContainer(); + String remoteDataDir = remoteDataDirContainer.nextDataDir(); + metadataManager.createPartition( tablePath, table.tableId, + remoteDataDir, partitionAssignment, partitionToCreate, request.isIgnoreIfNotExists()); @@ -688,6 +709,7 @@ public CompletableFuture metadata(MetadataRequest request) { return metadataResponseAccessContextEvent.getResultFuture(); } + @Override public CompletableFuture adjustIsr(AdjustIsrRequest request) { CompletableFuture response = new CompletableFuture<>(); eventManagerSupplier diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/MetadataManager.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/MetadataManager.java index c9537136f8..187309fb16 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/MetadataManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/MetadataManager.java @@ -44,13 +44,13 @@ import org.apache.fluss.metadata.TableChange; import org.apache.fluss.metadata.TableDescriptor; import org.apache.fluss.metadata.TableInfo; -import org.apache.fluss.metadata.TablePartition; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.security.acl.FlussPrincipal; import org.apache.fluss.server.entity.TablePropertyChanges; import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.server.zk.data.DatabaseRegistration; import org.apache.fluss.server.zk.data.PartitionAssignment; +import org.apache.fluss.server.zk.data.PartitionRegistration; import org.apache.fluss.server.zk.data.TableAssignment; import org.apache.fluss.server.zk.data.TableRegistration; import org.apache.fluss.shaded.zookeeper3.org.apache.zookeeper.KeeperException; @@ -275,6 +275,7 @@ public void completeDeletePartition(long partitionId) { * Returns -1 if the table already exists and ignoreIfExists is true. * * @param tablePath the table path + * @param remoteDataDir the remote data directory * @param tableToCreate the table descriptor describing the table to create * @param tableAssignment the table assignment, will be null when the table is partitioned table * @param ignoreIfExists whether to ignore if the table already exists @@ -282,6 +283,7 @@ public void completeDeletePartition(long partitionId) { */ public long createTable( TablePath tablePath, + String remoteDataDir, TableDescriptor tableToCreate, @Nullable TableAssignment tableAssignment, boolean ignoreIfExists) @@ -323,7 +325,9 @@ public long createTable( } // register the table zookeeperClient.registerTable( - tablePath, TableRegistration.newTable(tableId, tableToCreate), false); + tablePath, + TableRegistration.newTable(tableId, remoteDataDir, tableToCreate), + false); return tableId; }, "Fail to create table " + tablePath); @@ -692,13 +696,14 @@ public Set getPartitions(TablePath tablePath) { public void createPartition( TablePath tablePath, long tableId, + String remoteDataDir, PartitionAssignment partitionAssignment, ResolvedPartitionSpec partition, boolean ignoreIfExists) { String partitionName = partition.getPartitionName(); - Optional optionalTablePartition = - getOptionalTablePartition(tablePath, partitionName); - if (optionalTablePartition.isPresent()) { + Optional optionalPartitionRegistration = + getOptionalPartitionRegistration(tablePath, partitionName); + if (optionalPartitionRegistration.isPresent()) { if (ignoreIfExists) { return; } @@ -751,7 +756,12 @@ public void createPartition( long partitionId = zookeeperClient.getPartitionIdAndIncrement(); // register partition assignments and partition metadata to zk in transaction zookeeperClient.registerPartitionAssignmentAndMetadata( - partitionId, partitionName, partitionAssignment, tablePath, tableId); + partitionId, + partitionName, + partitionAssignment, + remoteDataDir, + tablePath, + tableId); LOG.info( "Register partition {} to zookeeper for table [{}].", partitionName, tablePath); } catch (KeeperException.NodeExistsException nodeExistsException) { @@ -773,9 +783,9 @@ public void createPartition( public void dropPartition( TablePath tablePath, ResolvedPartitionSpec partition, boolean ignoreIfNotExists) { String partitionName = partition.getPartitionName(); - Optional optionalTablePartition = - getOptionalTablePartition(tablePath, partitionName); - if (!optionalTablePartition.isPresent()) { + Optional optionalPartitionRegistration = + getOptionalPartitionRegistration(tablePath, partitionName); + if (!optionalPartitionRegistration.isPresent()) { if (ignoreIfNotExists) { return; } @@ -797,7 +807,7 @@ public void dropPartition( } } - private Optional getOptionalTablePartition( + private Optional getOptionalPartitionRegistration( TablePath tablePath, String partitionName) { try { return zookeeperClient.getPartition(tablePath, partitionName); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/TableChangeWatcher.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/TableChangeWatcher.java index 5b4821f194..51e934c2a4 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/TableChangeWatcher.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/TableChangeWatcher.java @@ -22,7 +22,6 @@ import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.SchemaInfo; import org.apache.fluss.metadata.TableInfo; -import org.apache.fluss.metadata.TablePartition; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.server.coordinator.event.CreatePartitionEvent; import org.apache.fluss.server.coordinator.event.CreateTableEvent; @@ -33,6 +32,7 @@ import org.apache.fluss.server.coordinator.event.TableRegistrationChangeEvent; import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.server.zk.data.PartitionAssignment; +import org.apache.fluss.server.zk.data.PartitionRegistration; import org.apache.fluss.server.zk.data.TableAssignment; import org.apache.fluss.server.zk.data.TableRegistration; import org.apache.fluss.server.zk.data.ZkData.DatabasesZNode; @@ -149,7 +149,8 @@ public void event(Type type, ChildData oldData, ChildData newData) { PartitionZNode.parsePath(oldData.getPath()); if (physicalTablePath != null) { // it's for deletion of a table partition node - TablePartition partition = PartitionZNode.decode(oldData.getData()); + PartitionRegistration partition = + PartitionZNode.decode(oldData.getData()); eventManager.put( new DropPartitionEvent( partition.getTableId(), @@ -223,7 +224,7 @@ private void processCreateTable(TablePath tablePath, ChildData tableData) { private void processCreatePartition( TablePath tablePath, String partitionName, ChildData partitionData) { - TablePartition partition = PartitionZNode.decode(partitionData.getData()); + PartitionRegistration partition = PartitionZNode.decode(partitionData.getData()); long partitionId = partition.getPartitionId(); long tableId = partition.getTableId(); PartitionAssignment partitionAssignment; diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/remote/RemoteDirDynamicLoader.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/remote/RemoteDirDynamicLoader.java new file mode 100644 index 0000000000..0de230c5da --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/remote/RemoteDirDynamicLoader.java @@ -0,0 +1,230 @@ +/* + * 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.annotation.VisibleForTesting; +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.fs.FsPath; + +import java.util.List; +import java.util.Objects; + +/** + * Dynamic loader for remote data directories that supports runtime reconfiguration. + * + *

This class manages the lifecycle of remote data directories and provides a container for + * selecting remote data directories. It implements {@link ServerReconfigurable} to support dynamic + * configuration updates at runtime without requiring a server restart. + * + *

The remote data directories are used for storing tiered storage data, including: + * + *

    + *
  • KV snapshot data files for primary key tables + *
  • Remote log segments for log tiered storage + *
+ * + *

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 RemoteDirContainer remoteDirContainer; + private Configuration currentConfiguration; + + public RemoteDirDynamicLoader(Configuration configuration) { + this.currentConfiguration = configuration; + this.remoteDirContainer = new RemoteDirContainer(configuration); + } + + /** + * Gets a container for managing and selecting remote data directories. + * + *

The container encapsulates the remote data directories and the selector strategy used to + * choose directories. + * + * @return a container for remote data directories + */ + public RemoteDirContainer getRemoteDataDirContainer() { + return remoteDirContainer; + } + + @Override + public void validate(Configuration newConfig) throws ConfigException { + // Get and valid remote data dirs + List remoteDataDirs = + newConfig + .getOptional(ConfigOptions.REMOTE_DATA_DIRS) + .orElseGet(() -> currentConfiguration.get(ConfigOptions.REMOTE_DATA_DIRS)); + for (int i = 0; i < remoteDataDirs.size(); i++) { + String remoteDataDir = remoteDataDirs.get(i); + try { + new FsPath(remoteDataDir); + } catch (Exception e) { + throw new ConfigException( + String.format( + "Invalid remote path for %s at index %d.", + ConfigOptions.REMOTE_DATA_DIRS.key(), i)); + } + } + + // Get the strategy from new config or fall back to current config + ConfigOptions.RemoteDataDirStrategy strategy = + newConfig + .getOptional(ConfigOptions.REMOTE_DATA_DIRS_STRATEGY) + .orElseGet( + () -> + currentConfiguration.get( + ConfigOptions.REMOTE_DATA_DIRS_STRATEGY)); + + // Validate weighted round-robin strategy configuration + if (strategy == ConfigOptions.RemoteDataDirStrategy.WEIGHTED_ROUND_ROBIN) { + List weights = + newConfig + .getOptional(ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS) + .orElseGet( + () -> + currentConfiguration.get( + ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS)); + + if (!remoteDataDirs.isEmpty() && !weights.isEmpty()) { + if (remoteDataDirs.size() != weights.size()) { + throw new ConfigException( + String.format( + "The size of '%s' (%d) must match the size of '%s' (%d) when using WEIGHTED_ROUND_ROBIN strategy.", + ConfigOptions.REMOTE_DATA_DIRS.key(), + remoteDataDirs.size(), + ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS.key(), + weights.size())); + } + // Validate all weights are positive + for (int i = 0; i < weights.size(); i++) { + if (weights.get(i) < 0) { + throw new ConfigException( + String.format( + "All weights in '%s' must be no less than 0, but found %d at index %d.", + ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS.key(), + weights.get(i), + i)); + } + } + } + } + } + + @Override + public void reconfigure(Configuration newConfig) throws ConfigException { + // Check if any relevant configuration has changed + boolean strategyChanged = + hasConfigChanged(newConfig, ConfigOptions.REMOTE_DATA_DIRS_STRATEGY); + boolean remoteDirsChanged = hasConfigChanged(newConfig, ConfigOptions.REMOTE_DATA_DIRS); + boolean weightsChanged = + hasConfigChanged(newConfig, ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS); + + if (strategyChanged || remoteDirsChanged || weightsChanged) { + // Create a new container with the merged configuration + Configuration mergedConfig = mergeConfigurations(currentConfiguration, newConfig); + this.remoteDirContainer = new RemoteDirContainer(mergedConfig); + this.currentConfiguration = mergedConfig; + } + } + + /** + * 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 the type of the configuration value + * @return true if the configuration has changed + */ + private boolean hasConfigChanged(Configuration newConfig, ConfigOption option) { + return newConfig + .getOptional(option) + .map(newValue -> !Objects.equals(newValue, currentConfiguration.get(option))) + .orElse(false); + } + + /** + * Merges the current configuration with new configuration values. + * + * @param current the current configuration + * @param updates the configuration updates to apply + * @return a new merged configuration + */ + private Configuration mergeConfigurations(Configuration current, Configuration updates) { + Configuration merged = new Configuration(current); + updates.toMap().forEach(merged::setString); + return merged; + } + + @Override + public void close() throws Exception { + // do nothing + } + + /** Container for managing remote data directories and selecting the next directory to use. */ + public static class RemoteDirContainer { + + private final String defaultRemoteDataDir; + private final RemoteDirSelector remoteDirSelector; + + public RemoteDirContainer(Configuration conf) { + this.defaultRemoteDataDir = conf.get(ConfigOptions.REMOTE_DATA_DIR); + this.remoteDirSelector = + createRemoteDirSelector( + conf.get(ConfigOptions.REMOTE_DATA_DIRS_STRATEGY), + defaultRemoteDataDir, + conf.get(ConfigOptions.REMOTE_DATA_DIRS), + conf.get(ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS)); + } + + public String getDefaultRemoteDataDir() { + return defaultRemoteDataDir; + } + + private RemoteDirSelector createRemoteDirSelector( + ConfigOptions.RemoteDataDirStrategy strategy, + String defaultRemoteDataDir, + List remoteDataDirs, + List weights) { + switch (strategy) { + case ROUND_ROBIN: + return new RoundRobinRemoteDirSelector(defaultRemoteDataDir, remoteDataDirs); + case WEIGHTED_ROUND_ROBIN: + return new WeightedRoundRobinRemoteDirSelector( + defaultRemoteDataDir, remoteDataDirs, weights); + default: + throw new IllegalArgumentException( + "Unsupported remote data directory select strategy: " + strategy); + } + } + + public String nextDataDir() { + return remoteDirSelector.nextDataDir(); + } + + @VisibleForTesting + protected RemoteDirSelector getRemoteDirSelector() { + return remoteDirSelector; + } + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/remote/RemoteDirSelector.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/remote/RemoteDirSelector.java new file mode 100644 index 0000000000..f98dcaa623 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/remote/RemoteDirSelector.java @@ -0,0 +1,49 @@ +/* + * 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 javax.annotation.concurrent.ThreadSafe; + +/** + * Interface for selecting remote data directories from a list of available directories. + * + *

This interface is used to implement different selection strategies for choosing remote data + * directories when creating tables or partitions. The selection strategy can be configured via + * {@link org.apache.fluss.config.ConfigOptions#REMOTE_DATA_DIRS_STRATEGY}. + * + *

Implementations of this interface should be thread-safe as they may be accessed concurrently + * from multiple threads. + * + * @see RoundRobinRemoteDirSelector + * @see WeightedRoundRobinRemoteDirSelector + */ +@ThreadSafe +public interface RemoteDirSelector { + + /** + * Returns the next remote data directory path to use. + * + *

This method should implement the selection strategy (e.g., round-robin, weighted + * round-robin) to choose from the available remote data directories. + * + * @return the next remote data directory path to use. If {@link + * org.apache.fluss.config.ConfigOptions#REMOTE_DATA_DIRS} is empty, should always return + * {@link org.apache.fluss.config.ConfigOptions#REMOTE_DATA_DIR}. + */ + String nextDataDir(); +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/remote/RoundRobinRemoteDirSelector.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/remote/RoundRobinRemoteDirSelector.java new file mode 100644 index 0000000000..99c2dfd5b6 --- /dev/null +++ b/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. + * + *

This implementation cycles through the available remote data directories in order, ensuring + * each directory is selected once before repeating. + * + *

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 defaultRemoteDataDir; + private final List remoteDataDirs; + + // Current position in the round-robin cycle. + private final AtomicInteger position; + + public RoundRobinRemoteDirSelector(String defaultRemoteDataDir, List remoteDataDirs) { + this.defaultRemoteDataDir = defaultRemoteDataDir; + this.remoteDataDirs = Collections.unmodifiableList(remoteDataDirs); + this.position = new AtomicInteger(0); + } + + @Override + public String nextDataDir() { + if (remoteDataDirs.isEmpty()) { + return defaultRemoteDataDir; + } + + int index = position.getAndUpdate(i -> (i + 1) % remoteDataDirs.size()); + return remoteDataDirs.get(index); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/remote/WeightedRoundRobinRemoteDirSelector.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/remote/WeightedRoundRobinRemoteDirSelector.java new file mode 100644 index 0000000000..8c0ae3ac0a --- /dev/null +++ b/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. + * + *

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. + * + *

Algorithm: + * + *

    + *
  1. Each node maintains a currentWeight initialized to 0 + *
  2. On each selection: add the node's configured weight to its currentWeight + *
  3. Select the node with the highest currentWeight + *
  4. Subtract the total weight sum from the selected node's currentWeight + *
+ * + *

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 defaultRemoteDataDir; + private final List 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 defaultRemoteDataDir, List remoteDataDirs, List weights) { + this.defaultRemoteDataDir = defaultRemoteDataDir; + 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()]; + } + + @Override + public String nextDataDir() { + if (remoteDataDirs.isEmpty() || totalWeight == 0) { + return defaultRemoteDataDir; + } + + synchronized (lock) { + int selectedIndex = -1; + int maxCurrentWeight = Integer.MIN_VALUE; + + // Step 1 & 2: Add weight to currentWeight and find the max + for (int i = 0; i < remoteDataDirs.size(); i++) { + currentWeights[i] += weights[i]; + if (currentWeights[i] > maxCurrentWeight) { + maxCurrentWeight = currentWeights[i]; + selectedIndex = i; + } + } + + // Step 3: Subtract total weight from selected node's current weight + currentWeights[selectedIndex] -= totalWeight; + + return remoteDataDirs.get(selectedIndex); + } + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletServer.java b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletServer.java index 8eed63c844..d108a63372 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletServer.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletServer.java @@ -22,7 +22,6 @@ import org.apache.fluss.cluster.ServerType; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; -import org.apache.fluss.exception.IllegalConfigurationException; import org.apache.fluss.exception.InvalidServerRackInfoException; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metrics.registry.MetricRegistry; @@ -71,7 +70,6 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; -import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -79,6 +77,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import static org.apache.fluss.config.ConfigOptions.BACKGROUND_THREADS; +import static org.apache.fluss.config.FlussConfigUtils.validateTabletConfigs; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.toTableBucket; /** @@ -177,7 +176,7 @@ public TabletServer(Configuration conf) { public TabletServer(Configuration conf, Clock clock) { super(conf); - validateConfigs(conf); + validateTabletConfigs(conf); this.terminationFuture = new CompletableFuture<>(); this.serverId = conf.getInt(ConfigOptions.TABLET_SERVER_ID); this.rack = conf.getString(ConfigOptions.TABLET_SERVER_RACK); @@ -558,40 +557,6 @@ public ReplicaManager getReplicaManager() { return authorizer; } - private static void validateConfigs(Configuration conf) { - Optional serverId = conf.getOptional(ConfigOptions.TABLET_SERVER_ID); - if (!serverId.isPresent()) { - throw new IllegalConfigurationException( - String.format("Configuration %s must be set.", ConfigOptions.TABLET_SERVER_ID)); - } - - if (serverId.get() < 0) { - throw new IllegalConfigurationException( - String.format( - "Invalid configuration for %s, it must be greater than or equal 0.", - ConfigOptions.TABLET_SERVER_ID.key())); - } - - if (conf.get(ConfigOptions.BACKGROUND_THREADS) < 1) { - throw new IllegalConfigurationException( - String.format( - "Invalid configuration for %s, it must be greater than or equal 1.", - ConfigOptions.BACKGROUND_THREADS.key())); - } - - if (conf.get(ConfigOptions.REMOTE_DATA_DIR) == null) { - throw new IllegalConfigurationException( - String.format("Configuration %s must be set.", ConfigOptions.REMOTE_DATA_DIR)); - } - - if (conf.get(ConfigOptions.LOG_SEGMENT_FILE_SIZE).getBytes() > Integer.MAX_VALUE) { - throw new IllegalConfigurationException( - String.format( - "Invalid configuration for %s, it must be less than or equal %d bytes.", - ConfigOptions.LOG_SEGMENT_FILE_SIZE.key(), Integer.MAX_VALUE)); - } - } - @VisibleForTesting public RpcServer getRpcServer() { return rpcServer; diff --git a/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java b/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java index 6f5672ff46..d36cbe06c8 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java @@ -530,6 +530,7 @@ private static PbTableMetadata toPbTableMetadata(TableMetadata tableMetadata) { .setTableId(tableInfo.getTableId()) .setSchemaId(tableInfo.getSchemaId()) .setTableJson(tableInfo.toTableDescriptor().toJsonBytes()) + .setRemoteDataDir(tableInfo.getRemoteDataDir()) .setCreatedTime(tableInfo.getCreatedTime()) .setModifiedTime(tableInfo.getModifiedTime()); TablePath tablePath = tableInfo.getTablePath(); @@ -588,6 +589,14 @@ private static TableMetadata toTableMetaData(PbTableMetadata pbTableMetadata) { tableId, pbTableMetadata.getSchemaId(), TableDescriptor.fromJsonBytes(pbTableMetadata.getTableJson()), + // For backword capability. When an older Coordinator sends an + // UpdateMetadataRequest to a newer TabletServer, the remoteDataDir will be + // missing. In this case, setting it to null is acceptable because the + // TabletServerMetadataCache does not maintain any remoteDataDir + // information. + pbTableMetadata.hasRemoteDataDir() + ? pbTableMetadata.getRemoteDataDir() + : null, pbTableMetadata.getCreatedTime(), pbTableMetadata.getModifiedTime()); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java index d088e103ce..10bbe6b6f3 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java @@ -46,6 +46,7 @@ import org.apache.fluss.server.zk.data.DatabaseRegistration; import org.apache.fluss.server.zk.data.LeaderAndIsr; import org.apache.fluss.server.zk.data.PartitionAssignment; +import org.apache.fluss.server.zk.data.PartitionRegistration; import org.apache.fluss.server.zk.data.RebalanceTask; import org.apache.fluss.server.zk.data.RemoteLogManifestHandle; import org.apache.fluss.server.zk.data.ResourceAcl; @@ -150,6 +151,8 @@ public class ZooKeeperClient implements AutoCloseable { private final Semaphore inFlightRequests; private final Configuration configuration; + private final String remoteDataDir; + public ZooKeeperClient( CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper, Configuration configuration) { @@ -164,6 +167,8 @@ public ZooKeeperClient( configuration.getInt(ConfigOptions.ZOOKEEPER_MAX_INFLIGHT_REQUESTS); this.inFlightRequests = new Semaphore(maxInFlightRequests); this.configuration = configuration; + + this.remoteDataDir = configuration.get(ConfigOptions.REMOTE_DATA_DIR); } public Optional getOrEmpty(String path) throws Exception { @@ -174,6 +179,10 @@ public Optional getOrEmpty(String path) throws Exception { } } + public String getRemoteDataDir() { + return remoteDataDir; + } + // -------------------------------------------------------------------------------------------- // Coordinator server // -------------------------------------------------------------------------------------------- @@ -576,7 +585,11 @@ public void registerTable( /** Get the table in ZK. */ public Optional getTable(TablePath tablePath) throws Exception { Optional bytes = getOrEmpty(TableZNode.path(tablePath)); - return bytes.map(TableZNode::decode); + Optional tableRegistration = bytes.map(TableZNode::decode); + // Set the default remote data dir for a node generated by an older version which does not + // have remote data dir + return tableRegistration.map( + t -> t.remoteDataDir == null ? t.newRemoteDataDir(remoteDataDir) : t); } /** Get the tables in ZK. */ @@ -685,7 +698,7 @@ public Map> getPartitionsForTables(Collection public Map getPartitionNameAndIds(TablePath tablePath) throws Exception { Map partitions = new HashMap<>(); for (String partitionName : getPartitions(tablePath)) { - Optional optPartition = getPartition(tablePath, partitionName); + Optional optPartition = getPartition(tablePath, partitionName); optPartition.ifPresent( partition -> partitions.put(partitionName, partition.getPartitionId())); } @@ -744,7 +757,8 @@ public Map getPartitionNameAndIds( fromPartitionName(partitionKeys, partitionName); boolean contains = resolvedPartitionSpec.contains(partialPartitionSpec); if (contains) { - Optional optPartition = getPartition(tablePath, partitionName); + Optional optPartition = + getPartition(tablePath, partitionName); optPartition.ifPresent( partition -> partitions.put(partitionName, partition.getPartitionId())); } @@ -790,10 +804,15 @@ public Map getPartitionIdAndPaths(Collection } /** Get a partition of a table in ZK. */ - public Optional getPartition(TablePath tablePath, String partitionName) + public Optional getPartition(TablePath tablePath, String partitionName) throws Exception { String path = PartitionZNode.path(tablePath, partitionName); - return getOrEmpty(path).map(PartitionZNode::decode); + Optional partitionRegistration = + getOrEmpty(path).map(PartitionZNode::decode); + // Set the default remote data dir for a node generated by an older version which does not + // have remote data dir + return partitionRegistration.map( + p -> p.getRemoteDataDir() == null ? p.newRemoteDataDir(remoteDataDir) : p); } /** Get partition id and table id for each partition in a batch async way. */ @@ -813,7 +832,7 @@ public Map getPartitionIds( return processGetDataResponses( responses, response -> path2PartitionPathMap.get(response.getPath()), - PartitionZNode::decode, + (byte[] data) -> PartitionZNode.decode(data).toTablePartition(), "partition"); } @@ -838,6 +857,7 @@ public void registerPartitionAssignmentAndMetadata( long partitionId, String partitionName, PartitionAssignment partitionAssignment, + String remoteDataDir, TablePath tablePath, long tableId) throws Exception { @@ -883,12 +903,15 @@ public void registerPartitionAssignmentAndMetadata( .withMode(CreateMode.PERSISTENT) .forPath( metadataPath, - PartitionZNode.encode(new TablePartition(tableId, partitionId))); + PartitionZNode.encode( + new PartitionRegistration( + tableId, partitionId, remoteDataDir))); ops.add(tabletServerPartitionNode); ops.add(metadataPartitionNode); zkClient.transaction().forOperations(ops); } + // -------------------------------------------------------------------------------------------- // Schema // -------------------------------------------------------------------------------------------- diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/PartitionRegistration.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/PartitionRegistration.java new file mode 100644 index 0000000000..c1a6307a63 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/PartitionRegistration.java @@ -0,0 +1,108 @@ +/* + * 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.zk.data; + +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.metadata.TablePartition; + +import java.util.Objects; + +/** + * The registration information of partition in {@link ZkData.PartitionZNode}. It is used to store + * the partition information in zookeeper. + * + * @see PartitionRegistrationJsonSerde for json serialization and deserialization. + */ +public class PartitionRegistration { + + private final long tableId; + private final long partitionId; + + /** + * The remote data directory of the partition. It is null if and only if it is deserialized by + * {@link PartitionRegistrationJsonSerde} from an existing node produced by an older version + * that does not support multiple remote paths. But immediately after that, we will set it as + * the default remote file path configured by {@link ConfigOptions#REMOTE_DATA_DIR} (see {@link + * org.apache.fluss.server.zk.ZooKeeperClient#getPartition}). This unifies subsequent usage and + * eliminates the need to account for differences between versions. + */ + private final String remoteDataDir; + + public PartitionRegistration(long tableId, long partitionId, String remoteDataDir) { + this.tableId = tableId; + this.partitionId = partitionId; + this.remoteDataDir = remoteDataDir; + } + + public long getTableId() { + return tableId; + } + + public long getPartitionId() { + return partitionId; + } + + public String getRemoteDataDir() { + return remoteDataDir; + } + + public TablePartition toTablePartition() { + return new TablePartition(tableId, partitionId); + } + + /** + * Returns a new registration with the given remote data directory. Should only be called by + * {@link org.apache.fluss.server.zk.ZooKeeperClient#getPartition} when deserialize an old + * PartitionRegistration node without remote data dir configured. + * + * @param remoteDataDir the remote data directory + * @return a new registration with the given remote data directory + */ + public PartitionRegistration newRemoteDataDir(String remoteDataDir) { + return new PartitionRegistration(tableId, partitionId, remoteDataDir); + } + + @Override + public boolean equals(Object o) { + if (o == null || getClass() != o.getClass()) { + return false; + } + PartitionRegistration that = (PartitionRegistration) o; + return tableId == that.tableId + && partitionId == that.partitionId + && Objects.equals(remoteDataDir, that.remoteDataDir); + } + + @Override + public int hashCode() { + return Objects.hash(tableId, partitionId, remoteDataDir); + } + + @Override + public String toString() { + return "PartitionRegistration{" + + "tableId=" + + tableId + + ", partitionId=" + + partitionId + + ", remoteDataDir='" + + remoteDataDir + + '\'' + + '}'; + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/PartitionRegistrationJsonSerde.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/PartitionRegistrationJsonSerde.java new file mode 100644 index 0000000000..e3115f7cc9 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/PartitionRegistrationJsonSerde.java @@ -0,0 +1,65 @@ +/* + * 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.zk.data; + +import org.apache.fluss.annotation.Internal; +import org.apache.fluss.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; +import org.apache.fluss.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.fluss.utils.json.JsonDeserializer; +import org.apache.fluss.utils.json.JsonSerializer; + +import java.io.IOException; + +/** Json serializer and deserializer for {@link PartitionRegistration}. */ +@Internal +public class PartitionRegistrationJsonSerde + implements JsonSerializer, JsonDeserializer { + + public static final PartitionRegistrationJsonSerde INSTANCE = + new PartitionRegistrationJsonSerde(); + + private static final String VERSION_KEY = "version"; + private static final String TABLE_ID_KEY = "table_id"; + private static final String PARTITION_ID_KEY = "partition_id"; + private static final String REMOTE_DATA_DIR_KEY = "remote_data_dir"; + private static final int VERSION = 1; + + @Override + public void serialize(PartitionRegistration registration, JsonGenerator generator) + throws IOException { + generator.writeStartObject(); + generator.writeNumberField(VERSION_KEY, VERSION); + generator.writeNumberField(TABLE_ID_KEY, registration.getTableId()); + generator.writeNumberField(PARTITION_ID_KEY, registration.getPartitionId()); + generator.writeStringField(REMOTE_DATA_DIR_KEY, registration.getRemoteDataDir()); + generator.writeEndObject(); + } + + @Override + public PartitionRegistration deserialize(JsonNode node) { + long tableId = node.get(TABLE_ID_KEY).asLong(); + long partitionId = node.get(PARTITION_ID_KEY).asLong(); + // When deserialize from an old version, the remote data dir may not exist. + // But we will fill it with ConfigOptions.REMOTE_DATA_DIR immediately. + String remoteDataDir = null; + if (node.has(REMOTE_DATA_DIR_KEY)) { + remoteDataDir = node.get(REMOTE_DATA_DIR_KEY).asText(); + } + return new PartitionRegistration(tableId, partitionId, remoteDataDir); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/TableRegistration.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/TableRegistration.java index 547326f6a1..422f725518 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/TableRegistration.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/TableRegistration.java @@ -52,6 +52,17 @@ public class TableRegistration { public final int bucketCount; public final Map properties; public final Map customProperties; + + /** + * The remote data directory of the table. It is null if and only if it is deserialized by + * {@link TableRegistrationJsonSerde} from an existing node produced by an older version that + * does not support multiple remote paths. But immediately after that, we will set it as the + * default remote file path configured by {@link ConfigOptions#REMOTE_DATA_DIR} (see {@link + * org.apache.fluss.server.zk.ZooKeeperClient#getTable}). This unifies subsequent usage and + * eliminates the need to account for differences between versions. + */ + public final String remoteDataDir; + public final long createdTime; public final long modifiedTime; @@ -62,6 +73,7 @@ public TableRegistration( TableDistribution tableDistribution, Map properties, Map customProperties, + String remoteDataDir, long createdTime, long modifiedTime) { checkArgument( @@ -74,6 +86,7 @@ public TableRegistration( this.bucketKeys = tableDistribution.getBucketKeys(); this.properties = properties; this.customProperties = customProperties; + this.remoteDataDir = remoteDataDir; this.createdTime = createdTime; this.modifiedTime = modifiedTime; } @@ -111,12 +124,14 @@ public TableInfo toTableInfo( this.bucketCount, properties, Configuration.fromMap(this.customProperties), + this.remoteDataDir, this.comment, this.createdTime, this.modifiedTime); } - public static TableRegistration newTable(long tableId, TableDescriptor tableDescriptor) { + public static TableRegistration newTable( + long tableId, String remoteDataDir, TableDescriptor tableDescriptor) { checkArgument( tableDescriptor.getTableDistribution().isPresent(), "Table distribution is required for table registration."); @@ -128,6 +143,7 @@ public static TableRegistration newTable(long tableId, TableDescriptor tableDesc tableDescriptor.getTableDistribution().get(), tableDescriptor.getProperties(), tableDescriptor.getCustomProperties(), + remoteDataDir, currentMillis, currentMillis); } @@ -142,10 +158,32 @@ public TableRegistration newProperties( new TableDistribution(bucketCount, bucketKeys), newProperties, newCustomProperties, + remoteDataDir, createdTime, currentMillis); } + /** + * Returns a new registration with the given remote data directory. Should only be called by + * {@link org.apache.fluss.server.zk.ZooKeeperClient#getTable} when deserialize an old + * TableRegistration node without remote data dir configured. + * + * @param remoteDataDir the remote data directory + * @return a new registration with the given remote data directory + */ + public TableRegistration newRemoteDataDir(String remoteDataDir) { + return new TableRegistration( + tableId, + comment, + partitionKeys, + new TableDistribution(bucketCount, bucketKeys), + properties, + customProperties, + remoteDataDir, + createdTime, + modifiedTime); + } + @Override public boolean equals(Object o) { if (this == o) { @@ -164,7 +202,8 @@ public boolean equals(Object o) { && Objects.equals(bucketCount, that.bucketCount) && Objects.equals(bucketKeys, that.bucketKeys) && Objects.equals(properties, that.properties) - && Objects.equals(customProperties, that.customProperties); + && Objects.equals(customProperties, that.customProperties) + && Objects.equals(remoteDataDir, that.remoteDataDir); } @Override @@ -177,6 +216,7 @@ public int hashCode() { bucketKeys, properties, customProperties, + remoteDataDir, createdTime, modifiedTime); } @@ -199,6 +239,8 @@ public String toString() { + properties + ", customProperties=" + customProperties + + ", remoteDataDir=" + + remoteDataDir + ", createdTime=" + createdTime + ", modifiedTime=" diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/TableRegistrationJsonSerde.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/TableRegistrationJsonSerde.java index 122d081e40..9a155a372a 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/TableRegistrationJsonSerde.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/TableRegistrationJsonSerde.java @@ -45,6 +45,7 @@ public class TableRegistrationJsonSerde static final String BUCKET_COUNT_NAME = "bucket_count"; static final String PROPERTIES_NAME = "properties"; static final String CUSTOM_PROPERTIES_NAME = "custom_properties"; + static final String REMOTE_DATA_DIR = "remote_data_dir"; static final String CREATED_TIME = "created_time"; static final String MODIFIED_TIME = "modified_time"; private static final String VERSION_KEY = "version"; @@ -100,6 +101,9 @@ public void serialize(TableRegistration tableReg, JsonGenerator generator) throw } generator.writeEndObject(); + // serialize remote data dir + generator.writeStringField(REMOTE_DATA_DIR, tableReg.remoteDataDir); + // serialize createdTime generator.writeNumberField(CREATED_TIME, tableReg.createdTime); @@ -141,6 +145,13 @@ public TableRegistration deserialize(JsonNode node) { Map customProperties = deserializeProperties(node.get(CUSTOM_PROPERTIES_NAME)); + // When deserialize from an old version, the remote data dir may not exist. + // But we will fill it with ConfigOptions.REMOTE_DATA_DIR immediately. + String remoteDataDir = null; + if (node.has(REMOTE_DATA_DIR)) { + remoteDataDir = node.get(REMOTE_DATA_DIR).asText(); + } + long createdTime = node.get(CREATED_TIME).asLong(); long modifiedTime = node.get(MODIFIED_TIME).asLong(); @@ -151,6 +162,7 @@ public TableRegistration deserialize(JsonNode node) { distribution, properties, customProperties, + remoteDataDir, createdTime, modifiedTime); } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java index b1e0e9493c..aed5de0b31 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java @@ -20,7 +20,6 @@ import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.Schema; import org.apache.fluss.metadata.TableBucket; -import org.apache.fluss.metadata.TablePartition; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.security.acl.Resource; import org.apache.fluss.security.acl.ResourceType; @@ -236,12 +235,13 @@ public static String path(TablePath tablePath, String partitionName) { return PartitionsZNode.path(tablePath) + "/" + partitionName; } - public static byte[] encode(TablePartition partition) { - return partition.toJsonBytes(); + public static byte[] encode(PartitionRegistration partitionRegistration) { + return JsonSerdeUtils.writeValueAsBytes( + partitionRegistration, PartitionRegistrationJsonSerde.INSTANCE); } - public static TablePartition decode(byte[] json) { - return TablePartition.fromJsonBytes(json); + public static PartitionRegistration decode(byte[] json) { + return JsonSerdeUtils.readValue(json, PartitionRegistrationJsonSerde.INSTANCE); } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/AutoPartitionManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/AutoPartitionManagerTest.java index 933ecb4379..54e2e5232d 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/AutoPartitionManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/AutoPartitionManagerTest.java @@ -25,12 +25,14 @@ import org.apache.fluss.metadata.TableDescriptor; import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.server.coordinator.remote.RemoteDirDynamicLoader; import org.apache.fluss.server.testutils.TestingServerMetadataCache; import org.apache.fluss.server.zk.NOPErrorHandler; import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.server.zk.ZooKeeperExtension; import org.apache.fluss.server.zk.data.BucketAssignment; import org.apache.fluss.server.zk.data.PartitionAssignment; +import org.apache.fluss.server.zk.data.PartitionRegistration; import org.apache.fluss.server.zk.data.TableRegistration; import org.apache.fluss.testutils.common.AllCallbackWrapper; import org.apache.fluss.testutils.common.ManuallyTriggeredScheduledExecutorService; @@ -49,7 +51,13 @@ import java.time.LocalDateTime; import java.time.ZoneId; import java.time.ZonedDateTime; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; import java.util.Map; +import java.util.Optional; +import java.util.Set; import java.util.UUID; import java.util.function.Function; import java.util.stream.Stream; @@ -67,6 +75,9 @@ class AutoPartitionManagerTest { protected static ZooKeeperClient zookeeperClient; private static MetadataManager metadataManager; + private static String remoteDataDir; + private static List remoteDataDirs; + private static RemoteDirDynamicLoader remoteDirDynamicLoader; @BeforeAll static void beforeAll() { @@ -79,6 +90,17 @@ static void beforeAll() { zookeeperClient, new Configuration(), new LakeCatalogDynamicLoader(new Configuration(), null, true)); + + remoteDataDir = "/dir"; + remoteDataDirs = Arrays.asList("/dir1", "/dir2", "/dir3", "/dir4"); + Configuration conf = new Configuration(); + conf.set(ConfigOptions.REMOTE_DATA_DIR, remoteDataDir); + conf.set(ConfigOptions.REMOTE_DATA_DIRS, remoteDataDirs); + conf.set(ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS, Arrays.asList(1, 1, 1, 1)); + conf.set( + ConfigOptions.REMOTE_DATA_DIRS_STRATEGY, + ConfigOptions.RemoteDataDirStrategy.WEIGHTED_ROUND_ROBIN); + remoteDirDynamicLoader = new RemoteDirDynamicLoader(conf); } @AfterEach @@ -297,6 +319,7 @@ void testAddPartitionedTable(TestParams params) throws Exception { zookeeperClient, new Configuration(), new LakeCatalogDynamicLoader(new Configuration(), null, true)), + remoteDirDynamicLoader, new Configuration(), clock, periodicExecutor); @@ -312,6 +335,7 @@ void testAddPartitionedTable(TestParams params) throws Exception { Map partitions = zookeeperClient.getPartitionNameAndIds(tablePath); // pre-create 4 partitions including current partition assertThat(partitions.keySet()).containsExactlyInAnyOrder(params.expectedPartitions); + verifyPartitionsRemoteDataDir(tablePath, partitions.keySet()); int replicaFactor = table.getTableConfig().getReplicationFactor(); Map bucketAssignments = @@ -333,6 +357,7 @@ void testAddPartitionedTable(TestParams params) throws Exception { metadataManager.createPartition( tablePath, tableId, + remoteDataDir, partitionAssignment, fromPartitionName(table.getPartitionKeys(), partitionName), false); @@ -353,11 +378,13 @@ void testAddPartitionedTable(TestParams params) throws Exception { partitions = zookeeperClient.getPartitionNameAndIds(tablePath); assertThat(partitions.keySet()) .containsExactlyInAnyOrder(params.expectedPartitionsAfterAdvance); + verifyPartitionsRemoteDataDir(tablePath, partitions.keySet()); clock.advanceTime(params.advanceDuration2); periodicExecutor.triggerPeriodicScheduledTasks(); partitions = zookeeperClient.getPartitionNameAndIds(tablePath); assertThat(partitions.keySet()).containsExactlyInAnyOrder(params.expectedPartitionsFinal); + verifyPartitionsRemoteDataDir(tablePath, partitions.keySet()); // trigger again at the same time, should be nothing changes periodicExecutor.triggerPeriodicScheduledTasks(); @@ -387,6 +414,7 @@ void testMaxPartitions() throws Exception { new AutoPartitionManager( new TestingServerMetadataCache(3), metadataManager, + remoteDirDynamicLoader, new Configuration(), clock, periodicExecutor); @@ -424,6 +452,7 @@ void testMaxPartitions() throws Exception { metadataManager.createPartition( tablePath, tableId, + remoteDataDir, partitionAssignment, fromPartitionName(table.getPartitionKeys(), i + ""), false); @@ -463,6 +492,7 @@ void testAutoCreateDayPartitionShouldJitter() throws Exception { new AutoPartitionManager( new TestingServerMetadataCache(3), metadataManager, + remoteDirDynamicLoader, new Configuration(), clock, periodicExecutor); @@ -527,6 +557,7 @@ void testMaxBucketNum() throws Exception { new AutoPartitionManager( new TestingServerMetadataCache(3), metadataManager, + remoteDirDynamicLoader, config, clock, periodicExecutor); @@ -698,6 +729,19 @@ public TestParams build() { // ------------------------------------------------------------------------------------------- + private void verifyPartitionsRemoteDataDir( + TablePath tablePath, Collection partitionNames) throws Exception { + Set allRemoteDataDirs = new HashSet<>(remoteDataDirs); + allRemoteDataDirs.add(remoteDataDir); + for (String partitionName : partitionNames) { + Optional partition = + zookeeperClient.getPartition(tablePath, partitionName); + String remoteDataDir = partition.get().getRemoteDataDir(); + assertThat(remoteDataDir).isNotNull(); + assertThat(allRemoteDataDirs).contains(remoteDataDir); + } + } + private TableInfo createPartitionedTable( int partitionRetentionNum, int partitionPreCreateNum, AutoPartitionTimeUnit timeUnit) throws Exception { @@ -752,8 +796,16 @@ private TableInfo createPartitionedTable( .build(); long currentMillis = System.currentTimeMillis(); TableInfo tableInfo = - TableInfo.of(tablePath, tableId, 1, descriptor, currentMillis, currentMillis); - TableRegistration registration = TableRegistration.newTable(tableId, descriptor); + TableInfo.of( + tablePath, + tableId, + 1, + descriptor, + remoteDataDir, + currentMillis, + currentMillis); + TableRegistration registration = + TableRegistration.newTable(tableId, remoteDataDir, descriptor); zookeeperClient.registerTable(tablePath, registration); return tableInfo; } @@ -794,8 +846,16 @@ private TableInfo createPartitionedTableWithBuckets( .build(); long currentMillis = System.currentTimeMillis(); TableInfo tableInfo = - TableInfo.of(tablePath, tableId, 1, descriptor, currentMillis, currentMillis); - TableRegistration registration = TableRegistration.newTable(tableId, descriptor); + TableInfo.of( + tablePath, + tableId, + 1, + descriptor, + remoteDataDir, + currentMillis, + currentMillis); + TableRegistration registration = + TableRegistration.newTable(tableId, remoteDataDir, descriptor); zookeeperClient.registerTable(tablePath, registration); return tableInfo; } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorContextTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorContextTest.java index 778ac45e15..5c321dfffa 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorContextTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorContextTest.java @@ -29,6 +29,7 @@ import java.time.Duration; import static org.apache.fluss.config.ConfigOptions.TABLE_DATALAKE_ENABLED; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.assertj.core.api.Assertions.assertThat; /** Test for {@link CoordinatorContext}. */ @@ -82,6 +83,7 @@ private TableInfo createTableInfo(long tableId, TablePath tablePath, boolean isL tableId, 1, tableDescriptor, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis()); } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java index 1f19d4da7f..452407a256 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java @@ -49,6 +49,7 @@ import org.apache.fluss.server.coordinator.event.CommitRemoteLogManifestEvent; import org.apache.fluss.server.coordinator.event.CoordinatorEventManager; import org.apache.fluss.server.coordinator.lease.KvSnapshotLeaseManager; +import org.apache.fluss.server.coordinator.remote.RemoteDirDynamicLoader; import org.apache.fluss.server.coordinator.statemachine.BucketState; import org.apache.fluss.server.coordinator.statemachine.ReplicaState; import org.apache.fluss.server.entity.AdjustIsrResultForBucket; @@ -90,7 +91,6 @@ import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.api.io.TempDir; -import java.io.IOException; import java.nio.file.Path; import java.time.Duration; import java.util.Arrays; @@ -157,6 +157,7 @@ class CoordinatorEventProcessorTest { private CompletedSnapshotStoreManager completedSnapshotStoreManager; private CoordinatorMetadataCache serverMetadataCache; private KvSnapshotLeaseManager kvSnapshotLeaseManager; + private String remoteDataDir; @BeforeAll static void baseBeforeAll() throws Exception { @@ -188,16 +189,20 @@ static void baseBeforeAll() throws Exception { } @BeforeEach - void beforeEach() throws IOException { + void beforeEach() { serverMetadataCache = new CoordinatorMetadataCache(); // set a test channel manager for the context testCoordinatorChannelManager = new TestCoordinatorChannelManager(); - autoPartitionManager = - new AutoPartitionManager(serverMetadataCache, metadataManager, new Configuration()); lakeTableTieringManager = new LakeTableTieringManager(); + remoteDataDir = zookeeperClient.getRemoteDataDir(); Configuration conf = new Configuration(); - String remoteDataDir = "/tmp/fluss/remote-data"; conf.setString(ConfigOptions.REMOTE_DATA_DIR, remoteDataDir); + autoPartitionManager = + new AutoPartitionManager( + serverMetadataCache, + metadataManager, + new RemoteDirDynamicLoader(conf), + new Configuration()); kvSnapshotLeaseManager = new KvSnapshotLeaseManager( Duration.ofMinutes(10).toMillis(), @@ -241,10 +246,14 @@ void testCreateAndDropTable() throws Exception { new TabletServerInfo(1, "rack1"), new TabletServerInfo(2, "rack2") }); - long t1Id = metadataManager.createTable(t1, tableDescriptor, tableAssignment, false); + long t1Id = + metadataManager.createTable( + t1, remoteDataDir, tableDescriptor, tableAssignment, false); TablePath t2 = TablePath.of(defaultDatabase, "create_drop_t2"); - long t2Id = metadataManager.createTable(t2, tableDescriptor, tableAssignment, false); + long t2Id = + metadataManager.createTable( + t2, remoteDataDir, tableDescriptor, tableAssignment, false); verifyTableCreated(t2Id, tableAssignment, nBuckets, replicationFactor); @@ -374,13 +383,15 @@ void testServerBecomeOnlineAndOfflineLine() throws Exception { TablePath table1Path = TablePath.of(defaultDatabase, "t1"); long table1Id = - metadataManager.createTable(table1Path, TEST_TABLE, table1Assignment, false); + metadataManager.createTable( + table1Path, remoteDataDir, TEST_TABLE, table1Assignment, false); TableAssignment table2Assignment = TableAssignment.builder().add(0, BucketAssignment.of(3)).build(); TablePath table2Path = TablePath.of(defaultDatabase, "t2"); long table2Id = - metadataManager.createTable(table2Path, TEST_TABLE, table2Assignment, false); + metadataManager.createTable( + table2Path, remoteDataDir, TEST_TABLE, table2Assignment, false); // retry until the table2 been created retryVerifyContext( @@ -485,7 +496,9 @@ void testRestartTriggerReplicaToOffline() throws Exception { .add(1, BucketAssignment.of(1, 2, 0)) .build(); TablePath tablePath = TablePath.of(defaultDatabase, "t_restart"); - long table1Id = metadataManager.createTable(tablePath, TEST_TABLE, tableAssignment, false); + long table1Id = + metadataManager.createTable( + tablePath, remoteDataDir, TEST_TABLE, tableAssignment, false); // let's restart initCoordinatorChannel(); @@ -609,7 +622,8 @@ void testCreateAndDropPartition() throws Exception { // create a partitioned table TableDescriptor tablePartitionTableDescriptor = getPartitionedTable(); long tableId = - metadataManager.createTable(tablePath, tablePartitionTableDescriptor, null, false); + metadataManager.createTable( + tablePath, remoteDataDir, tablePartitionTableDescriptor, null, false); int nBuckets = 3; int replicationFactor = 3; @@ -681,7 +695,8 @@ void testRestartResumeDropPartition() throws Exception { // create a partitioned table TableDescriptor tablePartitionTableDescriptor = getPartitionedTable(); long tableId = - metadataManager.createTable(tablePath, tablePartitionTableDescriptor, null, false); + metadataManager.createTable( + tablePath, remoteDataDir, tablePartitionTableDescriptor, null, false); int nBuckets = 3; int replicationFactor = 3; @@ -827,7 +842,8 @@ void testProcessAdjustIsr() throws Exception { new TabletServerInfo(1, "rack1"), new TabletServerInfo(2, "rack2") }); - long t1Id = metadataManager.createTable(t1, TEST_TABLE, tableAssignment, false); + long t1Id = + metadataManager.createTable(t1, remoteDataDir, TEST_TABLE, tableAssignment, false); verifyTableCreated(t1Id, tableAssignment, nBuckets, replicationFactor); // get the origin bucket leaderAndIsr @@ -881,7 +897,7 @@ void testSchemaChange() throws Exception { }); // create table List replicas = tableAssignment.getBucketAssignment(0).getReplicas(); - metadataManager.createTable(t1, TEST_TABLE, tableAssignment, false); + metadataManager.createTable(t1, remoteDataDir, TEST_TABLE, tableAssignment, false); TableInfo tableInfo = metadataManager.getTable(t1); retry( @@ -940,7 +956,7 @@ void testTableRegistrationChange() throws Exception { }); // create table List replicas = tableAssignment.getBucketAssignment(0).getReplicas(); - metadataManager.createTable(t1, TEST_TABLE, tableAssignment, false); + metadataManager.createTable(t1, remoteDataDir, TEST_TABLE, tableAssignment, false); TableInfo tableInfo = metadataManager.getTable(t1); retry( @@ -1002,7 +1018,11 @@ void testDoBucketReassignment() throws Exception { TableAssignment tableAssignment = new TableAssignment(bucketAssignments); long t1Id = metadataManager.createTable( - t1, CoordinatorEventProcessorTest.TEST_TABLE, tableAssignment, false); + t1, + remoteDataDir, + CoordinatorEventProcessorTest.TEST_TABLE, + tableAssignment, + false); TableBucket tb0 = new TableBucket(t1Id, 0); verifyIsr(tb0, 0, Arrays.asList(0, 1, 3)); @@ -1054,6 +1074,8 @@ private void verifyIsr(TableBucket tb, int expectedLeader, List expecte } private CoordinatorEventProcessor buildCoordinatorEventProcessor() { + Configuration conf = new Configuration(); + conf.set(ConfigOptions.REMOTE_DATA_DIR, remoteDataDir); return new CoordinatorEventProcessor( zookeeperClient, serverMetadataCache, @@ -1062,7 +1084,7 @@ private CoordinatorEventProcessor buildCoordinatorEventProcessor() { autoPartitionManager, lakeTableTieringManager, TestingMetricGroups.COORDINATOR_METRICS, - new Configuration(), + conf, Executors.newFixedThreadPool(1, new ExecutorThreadFactory("test-coordinator-io")), metadataManager, kvSnapshotLeaseManager); @@ -1107,9 +1129,19 @@ private Tuple2 preparePartitionAssignment( String partition1Name = "2024"; String partition2Name = "2025"; zookeeperClient.registerPartitionAssignmentAndMetadata( - partition1Id, partition1Name, partitionAssignment, tablePath, tableId); + partition1Id, + partition1Name, + partitionAssignment, + remoteDataDir, + tablePath, + tableId); zookeeperClient.registerPartitionAssignmentAndMetadata( - partition2Id, partition2Name, partitionAssignment, tablePath, tableId); + partition2Id, + partition2Name, + partitionAssignment, + remoteDataDir, + tablePath, + tableId); return Tuple2.of( new PartitionIdName(partition1Id, partition1Name), @@ -1380,7 +1412,11 @@ private long createTable(TablePath tablePath, TabletServerInfo[] servers) { TableAssignment tableAssignment = generateAssignment(N_BUCKETS, REPLICATION_FACTOR, servers); return metadataManager.createTable( - tablePath, CoordinatorEventProcessorTest.TEST_TABLE, tableAssignment, false); + tablePath, + remoteDataDir, + CoordinatorEventProcessorTest.TEST_TABLE, + tableAssignment, + false); } private void alterTable(TablePath tablePath, List schemaChanges) { diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/LakeTableTieringManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/LakeTableTieringManagerTest.java index 80b589dc06..34f7a57518 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/LakeTableTieringManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/LakeTableTieringManagerTest.java @@ -50,12 +50,14 @@ class LakeTableTieringManagerTest { private LakeTableTieringManager tableTieringManager; private ManualClock manualClock; private ManuallyTriggeredScheduledExecutorService lakeTieringServiceTimeoutChecker; + private String remoteDataDir; @BeforeEach void beforeEach() { manualClock = new ManualClock(); lakeTieringServiceTimeoutChecker = new ManuallyTriggeredScheduledExecutorService(); tableTieringManager = createLakeTableTieringManager(); + remoteDataDir = "/tmp/fluss/remote-data"; } private LakeTableTieringManager createLakeTableTieringManager() { @@ -275,6 +277,7 @@ private TableInfo createTableInfo(long tableId, TablePath tablePath, Duration fr tableId, 1, tableDescriptor, + remoteDataDir, System.currentTimeMillis(), System.currentTimeMillis()); } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TableManagerITCase.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TableManagerITCase.java index e49636cf43..928474bf07 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TableManagerITCase.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TableManagerITCase.java @@ -711,6 +711,7 @@ void testSchemaEvolution() throws Exception { pbTableMetadata.getTableId(), pbTableMetadata.getSchemaId(), TableDescriptor.fromJsonBytes(pbTableMetadata.getTableJson()), + pbTableMetadata.getRemoteDataDir(), pbTableMetadata.getCreatedTime(), pbTableMetadata.getModifiedTime()); List columns = tableInfo.getSchema().getColumns(); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TableManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TableManagerTest.java index 3c417efe8d..024b12b336 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TableManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TableManagerTest.java @@ -63,6 +63,7 @@ import static org.apache.fluss.record.TestData.DATA1_TABLE_ID; import static org.apache.fluss.record.TestData.DATA1_TABLE_PATH; import static org.apache.fluss.record.TestData.DATA1_TABLE_PATH_PK; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.server.coordinator.statemachine.BucketState.OnlineBucket; import static org.apache.fluss.server.coordinator.statemachine.ReplicaState.OnlineReplica; import static org.apache.fluss.server.coordinator.statemachine.ReplicaState.ReplicaDeletionSuccessful; @@ -162,6 +163,7 @@ void testCreateTable() throws Exception { tableId, 0, DATA1_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis())); tableManager.onCreateNewTable(DATA1_TABLE_PATH, tableId, assignment); @@ -185,6 +187,7 @@ void testDeleteTable() throws Exception { tableId, 0, DATA1_TABLE_DESCRIPTOR_PK, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis())); tableManager.onCreateNewTable(DATA1_TABLE_PATH_PK, tableId, assignment); @@ -224,6 +227,7 @@ void testResumeDeletionAfterRestart() throws Exception { tableId, 0, DATA1_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis())); tableManager.onCreateNewTable(DATA1_TABLE_PATH, tableId, assignment); @@ -271,6 +275,7 @@ void testCreateAndDropPartition() throws Exception { tableId, 0, DATA1_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis())); tableManager.onCreateNewTable(DATA1_TABLE_PATH, tableId, assignment); @@ -280,7 +285,12 @@ void testCreateAndDropPartition() throws Exception { String partitionName = "2024"; long partitionId = zookeeperClient.getPartitionIdAndIncrement(); zookeeperClient.registerPartitionAssignmentAndMetadata( - partitionId, partitionName, partitionAssignment, DATA1_TABLE_PATH, tableId); + partitionId, + partitionName, + partitionAssignment, + DEFAULT_REMOTE_DATA_DIR, + DATA1_TABLE_PATH, + tableId); // create partition tableManager.onCreateNewPartition( diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/event/watcher/TableChangeWatcherTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/event/watcher/TableChangeWatcherTest.java index 76634b970c..aef520cb8f 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/event/watcher/TableChangeWatcherTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/event/watcher/TableChangeWatcherTest.java @@ -79,6 +79,7 @@ class TableChangeWatcherTest { new AllCallbackWrapper<>(new ZooKeeperExtension()); private static ZooKeeperClient zookeeperClient; + private static String remoteDataDir; private TestingEventManager eventManager; private TableChangeWatcher tableChangeWatcher; private static MetadataManager metadataManager; @@ -89,6 +90,7 @@ static void beforeAll() { ZOO_KEEPER_EXTENSION_WRAPPER .getCustomExtension() .getZooKeeperClient(NOPErrorHandler.INSTANCE); + remoteDataDir = zookeeperClient.getRemoteDataDir(); metadataManager = new MetadataManager( zookeeperClient, @@ -134,7 +136,8 @@ void testTableChanges() { new TabletServerInfo(2, "rack2") }); long tableId = - metadataManager.createTable(tablePath, TEST_TABLE, tableAssignment, false); + metadataManager.createTable( + tablePath, remoteDataDir, TEST_TABLE, tableAssignment, false); SchemaInfo schemaInfo = metadataManager.getLatestSchema(tablePath); long currentMillis = System.currentTimeMillis(); expectedEvents.add( @@ -144,6 +147,7 @@ void testTableChanges() { tableId, schemaInfo.getSchemaId(), TEST_TABLE, + remoteDataDir, currentMillis, currentMillis), tableAssignment)); @@ -194,7 +198,9 @@ void testPartitionedTable() throws Exception { .property(ConfigOptions.TABLE_AUTO_PARTITION_TIME_UNIT.key(), "DAY") .build() .withReplicationFactor(3); - long tableId = metadataManager.createTable(tablePath, partitionedTable, null, false); + long tableId = + metadataManager.createTable( + tablePath, remoteDataDir, partitionedTable, null, false); List expectedEvents = new ArrayList<>(); SchemaInfo schemaInfo = metadataManager.getLatestSchema(tablePath); // create table event @@ -206,6 +212,7 @@ void testPartitionedTable() throws Exception { tableId, schemaInfo.getSchemaId(), partitionedTable, + remoteDataDir, currentMillis, currentMillis), TableAssignment.builder().build())); @@ -226,9 +233,9 @@ void testPartitionedTable() throws Exception { .getBucketAssignments()); // register assignment and metadata zookeeperClient.registerPartitionAssignmentAndMetadata( - 1L, "2011", partitionAssignment, tablePath, tableId); + 1L, "2011", partitionAssignment, remoteDataDir, tablePath, tableId); zookeeperClient.registerPartitionAssignmentAndMetadata( - 2L, "2022", partitionAssignment, tablePath, tableId); + 2L, "2022", partitionAssignment, remoteDataDir, tablePath, tableId); // create partitions events expectedEvents.add( @@ -273,7 +280,8 @@ void testSchemaChanges() { new TabletServerInfo(2, "rack2") }); long tableId = - metadataManager.createTable(tablePath, TEST_TABLE, tableAssignment, false); + metadataManager.createTable( + tablePath, remoteDataDir, TEST_TABLE, tableAssignment, false); SchemaInfo schemaInfo = metadataManager.getLatestSchema(tablePath); long currentMillis = System.currentTimeMillis(); expectedEvents.add( @@ -283,6 +291,7 @@ void testSchemaChanges() { tableId, schemaInfo.getSchemaId(), TEST_TABLE, + remoteDataDir, currentMillis, currentMillis), tableAssignment)); @@ -345,7 +354,9 @@ void testTableRegistrationChange() { new TabletServerInfo(1, "rack1"), new TabletServerInfo(2, "rack2") }); - long tableId = metadataManager.createTable(tablePath, TEST_TABLE, tableAssignment, false); + long tableId = + metadataManager.createTable( + tablePath, remoteDataDir, TEST_TABLE, tableAssignment, false); SchemaInfo schemaInfo = metadataManager.getLatestSchema(tablePath); long currentMillis = System.currentTimeMillis(); @@ -357,6 +368,7 @@ void testTableRegistrationChange() { tableId, schemaInfo.getSchemaId(), TEST_TABLE, + remoteDataDir, currentMillis, currentMillis), tableAssignment)); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/RebalanceManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/RebalanceManagerTest.java index 43ed3ae78a..f6d920542e 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/RebalanceManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/RebalanceManagerTest.java @@ -18,6 +18,7 @@ package org.apache.fluss.server.coordinator.rebalance; import org.apache.fluss.cluster.rebalance.RebalanceStatus; +import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; import org.apache.fluss.server.coordinator.AutoPartitionManager; import org.apache.fluss.server.coordinator.CoordinatorContext; @@ -27,6 +28,7 @@ import org.apache.fluss.server.coordinator.MetadataManager; import org.apache.fluss.server.coordinator.TestCoordinatorChannelManager; import org.apache.fluss.server.coordinator.lease.KvSnapshotLeaseManager; +import org.apache.fluss.server.coordinator.remote.RemoteDirDynamicLoader; import org.apache.fluss.server.metadata.CoordinatorMetadataCache; import org.apache.fluss.server.metrics.group.TestingMetricGroups; import org.apache.fluss.server.zk.NOPErrorHandler; @@ -80,8 +82,10 @@ static void baseBeforeAll() { void beforeEach() { serverMetadataCache = new CoordinatorMetadataCache(); testCoordinatorChannelManager = new TestCoordinatorChannelManager(); - String remoteDataDir = "/tmp/fluss/remote-data"; + Configuration conf = new Configuration(); + conf.set(ConfigOptions.REMOTE_DATA_DIR, remoteDataDir); + kvSnapshotLeaseManager = new KvSnapshotLeaseManager( Duration.ofMinutes(10).toMillis(), @@ -92,9 +96,13 @@ void beforeEach() { kvSnapshotLeaseManager.start(); autoPartitionManager = - new AutoPartitionManager(serverMetadataCache, metadataManager, new Configuration()); + new AutoPartitionManager( + serverMetadataCache, + metadataManager, + new RemoteDirDynamicLoader(conf), + conf); lakeTableTieringManager = new LakeTableTieringManager(); - CoordinatorEventProcessor eventProcessor = buildCoordinatorEventProcessor(); + CoordinatorEventProcessor eventProcessor = buildCoordinatorEventProcessor(conf); rebalanceManager = new RebalanceManager(eventProcessor, zookeeperClient); rebalanceManager.startup(); } @@ -130,7 +138,7 @@ void testRebalanceWithoutTask() throws Exception { .hasValue(new RebalanceTask(rebalanceId, COMPLETED, new HashMap<>())); } - private CoordinatorEventProcessor buildCoordinatorEventProcessor() { + private CoordinatorEventProcessor buildCoordinatorEventProcessor(Configuration conf) { return new CoordinatorEventProcessor( zookeeperClient, serverMetadataCache, @@ -139,7 +147,7 @@ private CoordinatorEventProcessor buildCoordinatorEventProcessor() { autoPartitionManager, lakeTableTieringManager, TestingMetricGroups.COORDINATOR_METRICS, - new Configuration(), + conf, Executors.newFixedThreadPool(1, new ExecutorThreadFactory("test-coordinator-io")), metadataManager, kvSnapshotLeaseManager); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/RemoteDirDynamicLoaderTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/RemoteDirDynamicLoaderTest.java new file mode 100644 index 0000000000..0f48485d98 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/RemoteDirDynamicLoaderTest.java @@ -0,0 +1,147 @@ +/* + * 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.ConfigOptions; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.exception.ConfigException; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Tests for {@link RemoteDirDynamicLoader}. */ +class RemoteDirDynamicLoaderTest { + + private static final String DEFAULT_REMOTE_DIR = "hdfs://cluster/default"; + + @Test + void testReconfigureWithStrategyChange() throws Exception { + Configuration conf = createBaseConfiguration(); + conf.set(ConfigOptions.REMOTE_DATA_DIRS, Arrays.asList("hdfs://dir1", "hdfs://dir2")); + try (RemoteDirDynamicLoader loader = new RemoteDirDynamicLoader(conf)) { + RemoteDirDynamicLoader.RemoteDirContainer originalContainer = + loader.getRemoteDataDirContainer(); + assertThat(originalContainer.getRemoteDirSelector()) + .isInstanceOf(RoundRobinRemoteDirSelector.class); + + // Reconfigure with strategy change + Configuration newConfig = new Configuration(); + newConfig.set( + ConfigOptions.REMOTE_DATA_DIRS_STRATEGY, + ConfigOptions.RemoteDataDirStrategy.WEIGHTED_ROUND_ROBIN); + newConfig.set(ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS, Arrays.asList(1, 2)); + loader.reconfigure(newConfig); + + // Container should be replaced + assertThat(loader.getRemoteDataDirContainer()).isNotSameAs(originalContainer); + assertThat(loader.getRemoteDataDirContainer().getRemoteDirSelector()) + .isInstanceOf(WeightedRoundRobinRemoteDirSelector.class); + } + } + + @Test + void testReconfigureWithWeightsChange() throws Exception { + Configuration conf = createBaseConfiguration(); + conf.set( + ConfigOptions.REMOTE_DATA_DIRS_STRATEGY, + ConfigOptions.RemoteDataDirStrategy.WEIGHTED_ROUND_ROBIN); + conf.set(ConfigOptions.REMOTE_DATA_DIRS, Arrays.asList("hdfs://dir1", "hdfs://dir2")); + conf.set(ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS, Arrays.asList(1, 2)); + + try (RemoteDirDynamicLoader loader = new RemoteDirDynamicLoader(conf)) { + RemoteDirDynamicLoader.RemoteDirContainer originalContainer = + loader.getRemoteDataDirContainer(); + assertThat(originalContainer.getRemoteDirSelector()) + .isInstanceOf(WeightedRoundRobinRemoteDirSelector.class); + + // Reconfigure with weights change + Configuration newConfig = new Configuration(); + newConfig.set(ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS, Arrays.asList(3, 4)); + loader.reconfigure(newConfig); + + // Container should be replaced + assertThat(loader.getRemoteDataDirContainer()).isNotSameAs(originalContainer); + assertThat(loader.getRemoteDataDirContainer().getRemoteDirSelector()) + .isInstanceOf(WeightedRoundRobinRemoteDirSelector.class); + } + } + + @Test + void testValidateInvalidRemotePathAndWeights() throws Exception { + // Test invalid remote path + Configuration conf1 = createBaseConfiguration(); + conf1.set(ConfigOptions.REMOTE_DATA_DIRS, Arrays.asList("hdfs://dir1", "hdfs://dir2")); + try (RemoteDirDynamicLoader loader = new RemoteDirDynamicLoader(conf1)) { + Configuration newConfig = new Configuration(); + newConfig.set( + ConfigOptions.REMOTE_DATA_DIRS, Arrays.asList("456://invalid", "hdfs://dir2")); + + assertThatThrownBy(() -> loader.validate(newConfig)) + .isInstanceOf(ConfigException.class) + .hasMessageContaining("Invalid remote path") + .hasMessageContaining(ConfigOptions.REMOTE_DATA_DIRS.key()) + .hasMessageContaining("at index 0"); + } + + // Test mismatched weights size + Configuration conf2 = createBaseConfiguration(); + conf2.set(ConfigOptions.REMOTE_DATA_DIRS, Arrays.asList("hdfs://dir1", "hdfs://dir2")); + conf2.set( + ConfigOptions.REMOTE_DATA_DIRS_STRATEGY, + ConfigOptions.RemoteDataDirStrategy.WEIGHTED_ROUND_ROBIN); + conf2.set(ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS, Arrays.asList(1, 2)); + try (RemoteDirDynamicLoader loader = new RemoteDirDynamicLoader(conf2)) { + Configuration newConfig = new Configuration(); + newConfig.set(ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS, Arrays.asList(1, 2, 3)); + + assertThatThrownBy(() -> loader.validate(newConfig)) + .isInstanceOf(ConfigException.class) + .hasMessageContaining(ConfigOptions.REMOTE_DATA_DIRS.key()) + .hasMessageContaining(ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS.key()) + .hasMessageContaining("must match the size"); + } + + // Test negative weight + Configuration conf3 = createBaseConfiguration(); + conf3.set(ConfigOptions.REMOTE_DATA_DIRS, Arrays.asList("hdfs://dir1", "hdfs://dir2")); + conf3.set( + ConfigOptions.REMOTE_DATA_DIRS_STRATEGY, + ConfigOptions.RemoteDataDirStrategy.WEIGHTED_ROUND_ROBIN); + conf3.set(ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS, Arrays.asList(1, 2)); + try (RemoteDirDynamicLoader loader = new RemoteDirDynamicLoader(conf3)) { + Configuration newConfig = new Configuration(); + newConfig.set(ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS, Arrays.asList(-1, 2)); + + assertThatThrownBy(() -> loader.validate(newConfig)) + .isInstanceOf(ConfigException.class) + .hasMessageContaining(ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS.key()) + .hasMessageContaining("must be no less than 0") + .hasMessageContaining("at index 0"); + } + } + + private Configuration createBaseConfiguration() { + Configuration conf = new Configuration(); + conf.set(ConfigOptions.REMOTE_DATA_DIR, DEFAULT_REMOTE_DIR); + return conf; + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/RemoteDirsITCase.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/RemoteDirsITCase.java new file mode 100644 index 0000000000..1a445808cc --- /dev/null +++ b/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 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 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 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 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 tableOpt = zkClient.getTable(tablePath); + assertThat(tableOpt).isPresent(); + TableRegistration table = tableOpt.get(); + // Partitioned table should NOT have remoteDataDir set at table level + assertThat(table.remoteDataDir).isNotNull(); + + // Create multiple partitions using partition column "b" + int partitionCount = 6; + List partitionNames = new ArrayList<>(); + for (int i = 0; i < partitionCount; i++) { + String partitionName = "p" + i; + partitionNames.add(partitionName); + PartitionSpec partitionSpec = + new PartitionSpec(Collections.singletonMap("b", partitionName)); + RpcMessageTestUtils.createPartition( + FLUSS_CLUSTER_EXTENSION, tablePath, partitionSpec, false); + } + + // Verify each partition has remoteDataDir set and round-robin is applied + Set usedRemoteDirs = new HashSet<>(); + for (String partitionName : partitionNames) { + Optional partitionOpt = + zkClient.getPartition(tablePath, partitionName); + assertThat(partitionOpt).isPresent(); + PartitionRegistration partition = partitionOpt.get(); + + assertThat(partition.getRemoteDataDir()).isNotNull(); + usedRemoteDirs.add(partition.getRemoteDataDir()); + } + + // All configured dirs should be used + assertThat(usedRemoteDirs).hasSize(REMOTE_DIR_NAMES.size()); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testAutoPartitionWithMultipleRemoteDirs(boolean isPrimaryKeyTable) throws Exception { + // Create an auto-partitioned table and verify partitions use different remote dirs + String tablePrefix = isPrimaryKeyTable ? "auto_pk_partitioned_" : "auto_partitioned_"; + TablePath tablePath = TablePath.of("test_db", tablePrefix + "table"); + TableDescriptor tableDescriptor = + isPrimaryKeyTable + ? DATA1_PARTITIONED_TABLE_DESCRIPTOR_PK + : DATA1_PARTITIONED_TABLE_DESCRIPTOR; + + RpcMessageTestUtils.createTable(FLUSS_CLUSTER_EXTENSION, tablePath, tableDescriptor); + + // Wait for auto partitions to be created + Map partitions = + FLUSS_CLUSTER_EXTENSION.waitUntilPartitionAllReady(tablePath); + assertThat(partitions).isNotEmpty(); + + // Verify partitions use remote dirs + Set usedRemoteDirs = new HashSet<>(); + for (String partitionName : partitions.keySet()) { + Optional partitionOpt = + zkClient.getPartition(tablePath, partitionName); + assertThat(partitionOpt).isPresent(); + PartitionRegistration partition = partitionOpt.get(); + + assertThat(partition.getRemoteDataDir()).isNotNull(); + usedRemoteDirs.add(partition.getRemoteDataDir()); + } + + // At least one remote dir should be used + assertThat(usedRemoteDirs).isNotEmpty(); + } + + @Test + void testMixedTableAndPartitionCreation() throws Exception { + // Create a mix of non-partitioned tables and partitioned table partitions + // to verify round-robin works correctly across both types + + // Create 2 non-partitioned tables + for (int i = 0; i < 2; i++) { + TablePath tablePath = TablePath.of("test_db", "mixed_non_pk_" + i); + RpcMessageTestUtils.createTable( + FLUSS_CLUSTER_EXTENSION, tablePath, DATA1_TABLE_DESCRIPTOR_PK); + } + + // Create a partitioned table + TablePath partitionedTablePath = TablePath.of("test_db", "mixed_partitioned"); + RpcMessageTestUtils.createTable( + FLUSS_CLUSTER_EXTENSION, partitionedTablePath, DATA1_PARTITIONED_TABLE_DESCRIPTOR); + + // Create partitions using partition column "b" + RpcMessageTestUtils.createPartition( + FLUSS_CLUSTER_EXTENSION, + partitionedTablePath, + new PartitionSpec(Collections.singletonMap("b", "p1")), + false); + RpcMessageTestUtils.createPartition( + FLUSS_CLUSTER_EXTENSION, + partitionedTablePath, + new PartitionSpec(Collections.singletonMap("b", "p2")), + false); + + // Create 2 more non-partitioned tables + for (int i = 2; i < 4; i++) { + TablePath tablePath = TablePath.of("test_db", "mixed_non_pk_" + i); + RpcMessageTestUtils.createTable( + FLUSS_CLUSTER_EXTENSION, tablePath, DATA1_TABLE_DESCRIPTOR_PK); + } + + // Collect all remote dirs used + Set allUsedDirs = new HashSet<>(); + + // Check non-partitioned tables + for (int i = 0; i < 4; i++) { + TablePath tablePath = TablePath.of("test_db", "mixed_non_pk_" + i); + Optional tableOpt = zkClient.getTable(tablePath); + assertThat(tableOpt).isPresent(); + assertThat(tableOpt.get().remoteDataDir).isNotNull(); + allUsedDirs.add(tableOpt.get().remoteDataDir); + } + + // Check partitions + for (String p : Arrays.asList("p1", "p2")) { + Optional partitionOpt = + zkClient.getPartition(partitionedTablePath, p); + assertThat(partitionOpt).isPresent(); + assertThat(partitionOpt.get().getRemoteDataDir()).isNotNull(); + allUsedDirs.add(partitionOpt.get().getRemoteDataDir()); + } + + // All remote dirs should have been used (6 items, 3 dirs, round-robin) + assertThat(allUsedDirs).hasSize(REMOTE_DIR_NAMES.size()); + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/RoundRobinRemoteDirSelectorTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/RoundRobinRemoteDirSelectorTest.java new file mode 100644 index 0000000000..b94d3831a8 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/RoundRobinRemoteDirSelectorTest.java @@ -0,0 +1,163 @@ +/* + * 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.junit.jupiter.api.Test; + +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.Set; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link RoundRobinRemoteDirSelector}. */ +class RoundRobinRemoteDirSelectorTest { + + private static final String DEFAULT_DIR = "hdfs://default/data"; + + @Test + void testEmptyRemoteDirsShouldReturnDefault() { + RoundRobinRemoteDirSelector selector = + new RoundRobinRemoteDirSelector(DEFAULT_DIR, Collections.emptyList()); + + // Should always return default when remoteDataDirs is empty + for (int i = 0; i < 10; i++) { + assertThat(selector.nextDataDir()).isEqualTo(DEFAULT_DIR); + } + } + + @Test + void testSingleDirShouldAlwaysReturnSame() { + String dir = "hdfs://cluster/data1"; + RoundRobinRemoteDirSelector selector = + new RoundRobinRemoteDirSelector(DEFAULT_DIR, Collections.singletonList(dir)); + + // Should always return the single directory + for (int i = 0; i < 10; i++) { + assertThat(selector.nextDataDir()).isEqualTo(dir); + } + } + + @Test + void testRoundRobinOrder() { + List dirs = + Arrays.asList( + "hdfs://cluster/data1", "hdfs://cluster/data2", "hdfs://cluster/data3"); + + RoundRobinRemoteDirSelector selector = new RoundRobinRemoteDirSelector(DEFAULT_DIR, dirs); + + // Collect selections for multiple cycles + List selections = new ArrayList<>(); + for (int i = 0; i < 9; i++) { + selections.add(selector.nextDataDir()); + } + + // Verify round-robin pattern: each cycle should contain all dirs in order + // First cycle + assertThat(selections.subList(0, 3)).containsExactlyElementsOf(dirs); + // Second cycle + assertThat(selections.subList(3, 6)).containsExactlyElementsOf(dirs); + // Third cycle + assertThat(selections.subList(6, 9)).containsExactlyElementsOf(dirs); + } + + @Test + void testEvenDistribution() { + List dirs = + Arrays.asList( + "hdfs://cluster/data1", "hdfs://cluster/data2", "hdfs://cluster/data3"); + + RoundRobinRemoteDirSelector selector = new RoundRobinRemoteDirSelector(DEFAULT_DIR, dirs); + + Map counts = new HashMap<>(); + int totalCalls = 30; + + for (int i = 0; i < totalCalls; i++) { + String selected = selector.nextDataDir(); + counts.merge(selected, 1, Integer::sum); + } + + // Each directory should be selected equally + assertThat(counts.get(dirs.get(0))).isEqualTo(10); + assertThat(counts.get(dirs.get(1))).isEqualTo(10); + assertThat(counts.get(dirs.get(2))).isEqualTo(10); + } + + @Test + void testTwoDirs() { + List dirs = Arrays.asList("hdfs://cluster/data1", "hdfs://cluster/data2"); + + RoundRobinRemoteDirSelector selector = new RoundRobinRemoteDirSelector(DEFAULT_DIR, dirs); + + // Verify alternating pattern + assertThat(selector.nextDataDir()).isEqualTo(dirs.get(0)); + assertThat(selector.nextDataDir()).isEqualTo(dirs.get(1)); + assertThat(selector.nextDataDir()).isEqualTo(dirs.get(0)); + assertThat(selector.nextDataDir()).isEqualTo(dirs.get(1)); + } + + @Test + void testCycleWrapsCorrectly() { + List dirs = + Arrays.asList( + "hdfs://cluster/data1", "hdfs://cluster/data2", "hdfs://cluster/data3"); + + RoundRobinRemoteDirSelector selector = new RoundRobinRemoteDirSelector(DEFAULT_DIR, dirs); + + // Collect first cycle + List firstCycle = new ArrayList<>(); + for (int i = 0; i < 3; i++) { + firstCycle.add(selector.nextDataDir()); + } + + // Collect second cycle + List secondCycle = new ArrayList<>(); + for (int i = 0; i < 3; i++) { + secondCycle.add(selector.nextDataDir()); + } + + // Both cycles should have same sequence + assertThat(secondCycle).isEqualTo(firstCycle); + } + + @Test + void testAllDirsSelectedInOneCycle() { + List dirs = + Arrays.asList( + "hdfs://cluster/data1", + "hdfs://cluster/data2", + "hdfs://cluster/data3", + "hdfs://cluster/data4", + "hdfs://cluster/data5"); + + RoundRobinRemoteDirSelector selector = new RoundRobinRemoteDirSelector(DEFAULT_DIR, dirs); + + Set selectedInCycle = new HashSet<>(); + for (int i = 0; i < dirs.size(); i++) { + selectedInCycle.add(selector.nextDataDir()); + } + + // All directories should be selected exactly once in one cycle + assertThat(selectedInCycle).containsExactlyInAnyOrderElementsOf(dirs); + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/WeightedRoundRobinRemoteDirSelectorTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/WeightedRoundRobinRemoteDirSelectorTest.java new file mode 100644 index 0000000000..10893ebec3 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/WeightedRoundRobinRemoteDirSelectorTest.java @@ -0,0 +1,253 @@ +/* + * 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.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link WeightedRoundRobinRemoteDirSelector}. */ +class WeightedRoundRobinRemoteDirSelectorTest { + + private static final String DEFAULT_DIR = "hdfs://default/data"; + + @Test + void testEmptyRemoteDirsShouldReturnDefault() { + WeightedRoundRobinRemoteDirSelector selector = + new WeightedRoundRobinRemoteDirSelector( + DEFAULT_DIR, Collections.emptyList(), Collections.emptyList()); + + // Should always return default when remoteDataDirs is empty + for (int i = 0; i < 10; i++) { + assertThat(selector.nextDataDir()).isEqualTo(DEFAULT_DIR); + } + } + + @Test + void testSingleDirShouldAlwaysReturnSame() { + String dir = "hdfs://cluster/data1"; + WeightedRoundRobinRemoteDirSelector selector = + new WeightedRoundRobinRemoteDirSelector( + DEFAULT_DIR, Collections.singletonList(dir), Collections.singletonList(5)); + + // Should always return the single directory + for (int i = 0; i < 10; i++) { + assertThat(selector.nextDataDir()).isEqualTo(dir); + } + } + + @Test + void testEqualWeightsShouldDistributeEvenly() { + List dirs = + Arrays.asList( + "hdfs://cluster/data1", "hdfs://cluster/data2", "hdfs://cluster/data3"); + List weights = Arrays.asList(1, 1, 1); + + WeightedRoundRobinRemoteDirSelector selector = + new WeightedRoundRobinRemoteDirSelector(DEFAULT_DIR, dirs, weights); + + Map counts = new HashMap<>(); + int totalCalls = 30; + + for (int i = 0; i < totalCalls; i++) { + String selected = selector.nextDataDir(); + counts.merge(selected, 1, Integer::sum); + } + + // Each directory should be selected equally + assertThat(counts.get(dirs.get(0))).isEqualTo(10); + assertThat(counts.get(dirs.get(1))).isEqualTo(10); + assertThat(counts.get(dirs.get(2))).isEqualTo(10); + } + + @Test + void testWeightedDistribution() { + List dirs = + Arrays.asList( + "hdfs://cluster/data1", "hdfs://cluster/data2", "hdfs://cluster/data3"); + // weights: 5, 1, 1 -> total = 7 + List weights = Arrays.asList(5, 1, 1); + + WeightedRoundRobinRemoteDirSelector selector = + new WeightedRoundRobinRemoteDirSelector(DEFAULT_DIR, dirs, weights); + + Map counts = new HashMap<>(); + int totalCalls = 70; // 10 complete cycles + + for (int i = 0; i < totalCalls; i++) { + String selected = selector.nextDataDir(); + counts.merge(selected, 1, Integer::sum); + } + + // Distribution should match weights ratio: 5:1:1 + assertThat(counts.get(dirs.get(0))).isEqualTo(50); // 5/7 * 70 = 50 + assertThat(counts.get(dirs.get(1))).isEqualTo(10); // 1/7 * 70 = 10 + assertThat(counts.get(dirs.get(2))).isEqualTo(10); // 1/7 * 70 = 10 + } + + @Test + void testInterleavedDistribution() { + // Verify that selections are interleaved, not consecutive + List dirs = + Arrays.asList("hdfs://cluster/A", "hdfs://cluster/B", "hdfs://cluster/C"); + // weights: 5, 1, 1 -> total = 7 + List weights = Arrays.asList(5, 1, 1); + + WeightedRoundRobinRemoteDirSelector selector = + new WeightedRoundRobinRemoteDirSelector(DEFAULT_DIR, dirs, weights); + + List sequence = new ArrayList<>(); + for (int i = 0; i < 7; i++) { + sequence.add(selector.nextDataDir()); + } + + // Expected interleaved sequence for weights 5,1,1: + // The smooth WRR should produce: A, A, B, A, C, A, A (or similar interleaved pattern) + // Instead of traditional WRR: A, A, A, A, A, B, C + + // Count consecutive same selections - should be less than weight + int maxConsecutive = 0; + int currentConsecutive = 1; + for (int i = 1; i < sequence.size(); i++) { + if (sequence.get(i).equals(sequence.get(i - 1))) { + currentConsecutive++; + } else { + maxConsecutive = Math.max(maxConsecutive, currentConsecutive); + currentConsecutive = 1; + } + } + maxConsecutive = Math.max(maxConsecutive, currentConsecutive); + + // With smooth WRR, max consecutive selections should be <= 2 for this weight distribution + // (In traditional WRR, A would be selected 5 times consecutively) + assertThat(maxConsecutive).isLessThanOrEqualTo(2); + + // Verify all directories are selected at least once within one cycle + assertThat(sequence).contains(dirs.get(0), dirs.get(1), dirs.get(2)); + } + + @Test + void testTwoDirsWithDifferentWeights() { + List dirs = Arrays.asList("hdfs://cluster/data1", "hdfs://cluster/data2"); + // weights: 3, 1 -> total = 4 + List weights = Arrays.asList(3, 1); + + WeightedRoundRobinRemoteDirSelector selector = + new WeightedRoundRobinRemoteDirSelector(DEFAULT_DIR, dirs, weights); + + List sequence = new ArrayList<>(); + for (int i = 0; i < 8; i++) { + sequence.add(selector.nextDataDir()); + } + + // Count selections + long dir1Count = sequence.stream().filter(d -> d.equals(dirs.get(0))).count(); + long dir2Count = sequence.stream().filter(d -> d.equals(dirs.get(1))).count(); + + // Should follow 3:1 ratio + assertThat(dir1Count).isEqualTo(6); // 3/4 * 8 = 6 + assertThat(dir2Count).isEqualTo(2); // 1/4 * 8 = 2 + } + + @Test + void testCycleRepeatsCorrectly() { + List dirs = Arrays.asList("hdfs://cluster/data1", "hdfs://cluster/data2"); + List weights = Arrays.asList(2, 1); + + WeightedRoundRobinRemoteDirSelector selector = + new WeightedRoundRobinRemoteDirSelector(DEFAULT_DIR, dirs, weights); + + // Collect first cycle (3 selections) + List firstCycle = new ArrayList<>(); + for (int i = 0; i < 3; i++) { + firstCycle.add(selector.nextDataDir()); + } + + // Collect second cycle (3 selections) + List secondCycle = new ArrayList<>(); + for (int i = 0; i < 3; i++) { + secondCycle.add(selector.nextDataDir()); + } + + // Both cycles should have same sequence + assertThat(secondCycle).isEqualTo(firstCycle); + } + + @Test + void testLargeWeights() { + List dirs = Arrays.asList("hdfs://cluster/data1", "hdfs://cluster/data2"); + List weights = Arrays.asList(100, 1); + + WeightedRoundRobinRemoteDirSelector selector = + new WeightedRoundRobinRemoteDirSelector(DEFAULT_DIR, dirs, weights); + + Map counts = new HashMap<>(); + int totalCalls = 101; // One complete cycle + + for (int i = 0; i < totalCalls; i++) { + String selected = selector.nextDataDir(); + counts.merge(selected, 1, Integer::sum); + } + + // Should follow 100:1 ratio + assertThat(counts.get(dirs.get(0))).isEqualTo(100); + assertThat(counts.get(dirs.get(1))).isEqualTo(1); + } + + @Test + void testZeroWeights() { + // Test case 1: Some directories have zero weight - they should never be selected + List dirs = + Arrays.asList( + "hdfs://cluster/data1", "hdfs://cluster/data2", "hdfs://cluster/data3"); + // weight 0 for data2 means it should never be selected + List weights = Arrays.asList(2, 0, 1); + + WeightedRoundRobinRemoteDirSelector selector = + new WeightedRoundRobinRemoteDirSelector(DEFAULT_DIR, dirs, weights); + + Map counts = new HashMap<>(); + int totalCalls = 30; // 10 complete cycles (total weight = 3) + + for (int i = 0; i < totalCalls; i++) { + String selected = selector.nextDataDir(); + counts.merge(selected, 1, Integer::sum); + } + + // data1 should be selected 20 times (2/3 * 30) + assertThat(counts.get(dirs.get(0))).isEqualTo(20); + // data2 should never be selected (weight = 0) + assertThat(counts.get(dirs.get(1))).isNull(); + // data3 should be selected 10 times (1/3 * 30) + assertThat(counts.get(dirs.get(2))).isEqualTo(10); + + // Test case 2: All weights are zero - should always return default dir + List allZeroWeights = Arrays.asList(0, 0, 0); + WeightedRoundRobinRemoteDirSelector allZeroSelector = + new WeightedRoundRobinRemoteDirSelector(DEFAULT_DIR, dirs, allZeroWeights); + + assertThat(allZeroSelector.nextDataDir()).isEqualTo(DEFAULT_DIR); + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachineTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachineTest.java index 454ec5de4b..70578b6144 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachineTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachineTest.java @@ -54,6 +54,7 @@ import static org.apache.fluss.record.TestData.DATA1_TABLE_DESCRIPTOR; import static org.apache.fluss.record.TestData.DATA1_TABLE_PATH; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.server.coordinator.statemachine.ReplicaState.NewReplica; import static org.apache.fluss.server.coordinator.statemachine.ReplicaState.OfflineReplica; import static org.apache.fluss.server.coordinator.statemachine.ReplicaState.OnlineReplica; @@ -202,6 +203,7 @@ void testOfflineReplicasShouldBeRemovedFromIsr() throws Exception { tableId, 0, DATA1_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis())); coordinatorContext.putTablePath(tableId, DATA1_TABLE_PATH); @@ -240,6 +242,7 @@ void testOfflineReplicaShouldBeRemovedFromIsr() throws Exception { tableId, 0, DATA1_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis())); coordinatorContext.putTablePath(tableId, DATA1_TABLE_PATH); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachineTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachineTest.java index 26dfeb8dc9..796645b0ff 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachineTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachineTest.java @@ -36,6 +36,7 @@ import org.apache.fluss.server.coordinator.TestCoordinatorChannelManager; import org.apache.fluss.server.coordinator.event.CoordinatorEventManager; import org.apache.fluss.server.coordinator.lease.KvSnapshotLeaseManager; +import org.apache.fluss.server.coordinator.remote.RemoteDirDynamicLoader; import org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElection.ControlledShutdownLeaderElection; import org.apache.fluss.server.coordinator.statemachine.TableBucketStateMachine.ElectionResult; import org.apache.fluss.server.metadata.CoordinatorMetadataCache; @@ -64,6 +65,7 @@ import java.util.stream.Collectors; import static org.apache.fluss.record.TestData.DATA1_TABLE_DESCRIPTOR; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.server.coordinator.CoordinatorTestUtils.createServers; import static org.apache.fluss.server.coordinator.CoordinatorTestUtils.makeSendLeaderAndStopRequestAlwaysSuccess; import static org.apache.fluss.server.coordinator.statemachine.BucketState.NewBucket; @@ -121,6 +123,7 @@ void beforeEach() throws IOException { zookeeperClient, new Configuration(), new LakeCatalogDynamicLoader(new Configuration(), null, true)), + new RemoteDirDynamicLoader(conf), new Configuration()); lakeTableTieringManager = new LakeTableTieringManager(); @@ -215,6 +218,7 @@ void testStateChangeToOnline() throws Exception { tableId, 0, DATA1_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis())); coordinatorContext.putTablePath(tableId, fakeTablePath); @@ -360,6 +364,7 @@ void testStateChangeForTabletServerControlledShutdown() { tableId, 0, DATA1_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis())); coordinatorContext.putTablePath(tableId, fakeTablePath); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/log/DroppedTableRecoveryTest.java b/fluss-server/src/test/java/org/apache/fluss/server/log/DroppedTableRecoveryTest.java index cea1513c55..021e9d1b04 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/log/DroppedTableRecoveryTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/log/DroppedTableRecoveryTest.java @@ -55,6 +55,7 @@ import static org.apache.fluss.record.TestData.DATA1_SCHEMA_PK; import static org.apache.fluss.record.TestData.DATA1_TABLE_DESCRIPTOR; import static org.apache.fluss.record.TestData.DATA1_TABLE_ID; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.testutils.DataTestUtils.genMemoryLogRecordsByObject; import static org.assertj.core.api.Assertions.assertThat; @@ -108,7 +109,9 @@ public void setup() throws Exception { private void registerTableInZkClient() throws Exception { ZOO_KEEPER_EXTENSION_WRAPPER.getCustomExtension().cleanupRoot(); zkClient.registerTable( - tablePath, TableRegistration.newTable(DATA1_TABLE_ID, DATA1_TABLE_DESCRIPTOR)); + tablePath, + TableRegistration.newTable( + DATA1_TABLE_ID, DEFAULT_REMOTE_DATA_DIR, DATA1_TABLE_DESCRIPTOR)); zkClient.registerFirstSchema(tablePath, DATA1_SCHEMA); } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/log/LogManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/log/LogManagerTest.java index 80e99c2f56..bdaa5d6464 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/log/LogManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/log/LogManagerTest.java @@ -60,6 +60,7 @@ import static org.apache.fluss.record.TestData.DATA2_SCHEMA; import static org.apache.fluss.record.TestData.DATA2_TABLE_DESCRIPTOR; import static org.apache.fluss.record.TestData.DATA2_TABLE_ID; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.server.log.LogManager.CLEAN_SHUTDOWN_FILE; import static org.apache.fluss.testutils.DataTestUtils.assertLogRecordsEquals; import static org.apache.fluss.testutils.DataTestUtils.genMemoryLogRecordsByObject; @@ -113,10 +114,14 @@ public void setup() throws Exception { private void registerTableInZkClient() throws Exception { ZOO_KEEPER_EXTENSION_WRAPPER.getCustomExtension().cleanupRoot(); zkClient.registerTable( - tablePath1, TableRegistration.newTable(DATA1_TABLE_ID, DATA1_TABLE_DESCRIPTOR)); + tablePath1, + TableRegistration.newTable( + DATA1_TABLE_ID, DEFAULT_REMOTE_DATA_DIR, DATA1_TABLE_DESCRIPTOR)); zkClient.registerFirstSchema(tablePath1, DATA1_SCHEMA); zkClient.registerTable( - tablePath2, TableRegistration.newTable(DATA2_TABLE_ID, DATA2_TABLE_DESCRIPTOR)); + tablePath2, + TableRegistration.newTable( + DATA2_TABLE_ID, DEFAULT_REMOTE_DATA_DIR, DATA2_TABLE_DESCRIPTOR)); zkClient.registerFirstSchema(tablePath2, DATA2_SCHEMA); } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/metadata/ServerSchemaCacheTest.java b/fluss-server/src/test/java/org/apache/fluss/server/metadata/ServerSchemaCacheTest.java index c42c7b296c..66656f5b2a 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/metadata/ServerSchemaCacheTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/metadata/ServerSchemaCacheTest.java @@ -41,6 +41,7 @@ import static org.apache.fluss.record.TestData.DATA1_TABLE_PATH; import static org.apache.fluss.record.TestData.DATA2_SCHEMA; import static org.apache.fluss.record.TestData.DATA2_TABLE_DESCRIPTOR; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -89,6 +90,7 @@ void testGetHistorySchema() { DATA1_TABLE_ID, 2, DATA2_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis()); ServerSchemaCache manager = diff --git a/fluss-server/src/test/java/org/apache/fluss/server/metadata/TabletServerMetadataCacheTest.java b/fluss-server/src/test/java/org/apache/fluss/server/metadata/TabletServerMetadataCacheTest.java index 6ba37e44d4..8783657648 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/metadata/TabletServerMetadataCacheTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/metadata/TabletServerMetadataCacheTest.java @@ -45,6 +45,7 @@ import static org.apache.fluss.record.TestData.DATA1_TABLE_ID; import static org.apache.fluss.record.TestData.DATA1_TABLE_INFO; import static org.apache.fluss.record.TestData.DATA1_TABLE_PATH; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.server.metadata.PartitionMetadata.DELETED_PARTITION_ID; import static org.apache.fluss.server.metadata.TableMetadata.DELETED_TABLE_ID; import static org.apache.fluss.server.zk.data.LeaderAndIsr.NO_LEADER; @@ -69,6 +70,7 @@ public class TabletServerMetadataCacheTest { partitionTableId, 0, DATA1_PARTITIONED_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, 100L, 100L); private List tableMetadataList; @@ -217,6 +219,7 @@ void testUpdateClusterMetadataRequest() { // deletion. 1, DATA1_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis()), changedBucket1BucketMetadata)), diff --git a/fluss-server/src/test/java/org/apache/fluss/server/metadata/ZkBasedMetadataProviderTest.java b/fluss-server/src/test/java/org/apache/fluss/server/metadata/ZkBasedMetadataProviderTest.java index 1ebb0c3e03..e6a0064f15 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/metadata/ZkBasedMetadataProviderTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/metadata/ZkBasedMetadataProviderTest.java @@ -51,6 +51,7 @@ import java.util.Map; import java.util.stream.Collectors; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -100,7 +101,9 @@ void testGetTableMetadataFromZk() throws Exception { .add(1, BucketAssignment.of(2, 3, 4)) .build(); metadataManager.createDatabase("test_db", DatabaseDescriptor.EMPTY, true); - long tableId = metadataManager.createTable(tablePath, desc, tableAssignment, false); + long tableId = + metadataManager.createTable( + tablePath, DEFAULT_REMOTE_DATA_DIR, desc, tableAssignment, false); // Create leader and isr for buckets TableBucket tableBucket0 = new TableBucket(tableId, 0); @@ -161,7 +164,12 @@ void testGetPartitionMetadataFromZk() throws Exception { new PartitionAssignment(tableId, bucketAssignments); zookeeperClient.registerPartitionAssignmentAndMetadata( - partitionId, partitionName, partitionAssignment, tablePath, tableId); + partitionId, + partitionName, + partitionAssignment, + DEFAULT_REMOTE_DATA_DIR, + tablePath, + tableId); // Create leader and isr for partition buckets TableBucket partitionBucket0 = new TableBucket(tableId, partitionId, 0); @@ -233,9 +241,19 @@ void testBatchGetPartitionMetadataFromZkAsync() throws Exception { tableId1, Collections.singletonMap(1, BucketAssignment.of(2, 3))); zookeeperClient.registerPartitionAssignmentAndMetadata( - partitionId1, partitionName1, partitionAssignment1, tablePath1, tableId1); + partitionId1, + partitionName1, + partitionAssignment1, + DEFAULT_REMOTE_DATA_DIR, + tablePath1, + tableId1); zookeeperClient.registerPartitionAssignmentAndMetadata( - partitionId2, partitionName2, partitionAssignment2, tablePath1, tableId1); + partitionId2, + partitionName2, + partitionAssignment2, + DEFAULT_REMOTE_DATA_DIR, + tablePath1, + tableId1); // Create partition for table2 long partitionId3 = 21L; @@ -246,7 +264,12 @@ void testBatchGetPartitionMetadataFromZkAsync() throws Exception { tableId2, Collections.singletonMap(0, BucketAssignment.of(1, 3))); zookeeperClient.registerPartitionAssignmentAndMetadata( - partitionId3, partitionName3, partitionAssignment3, tablePath2, tableId2); + partitionId3, + partitionName3, + partitionAssignment3, + DEFAULT_REMOTE_DATA_DIR, + tablePath2, + tableId2); // Create leader and isr for all partition buckets TableBucket bucket1 = new TableBucket(tableId1, partitionId1, 0); @@ -338,6 +361,7 @@ private TableRegistration createTestTableRegistration( new TableDescriptor.TableDistribution(3, Collections.singletonList("a")), options, Collections.emptyMap(), + DEFAULT_REMOTE_DATA_DIR, currentMillis, currentMillis); } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java index bd63f3cbfd..0d9f5bc565 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java @@ -123,6 +123,7 @@ import static org.apache.fluss.record.TestData.DATA3_TABLE_ID_PK_AUTO_INC; import static org.apache.fluss.record.TestData.DATA3_TABLE_PATH_PK_AUTO_INC; import static org.apache.fluss.record.TestData.DATA_1_WITH_KEY_AND_VALUE; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.record.TestData.DEFAULT_SCHEMA_ID; import static org.apache.fluss.record.TestData.EXPECTED_LOG_RESULTS_FOR_DATA_1_WITH_PK; import static org.apache.fluss.server.coordinator.CoordinatorContext.INITIAL_COORDINATOR_EPOCH; @@ -1903,6 +1904,7 @@ void testUpdateMetadata() throws Exception { nonePartitionTableId, 1, DATA1_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis()); TableInfo partitionTableInfo = @@ -1911,6 +1913,7 @@ void testUpdateMetadata() throws Exception { partitionTableId, 1, DATA1_PARTITIONED_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis()); TableMetadata tableMetadata1 = @@ -1935,11 +1938,15 @@ void testUpdateMetadata() throws Exception { // register table to zk. zkClient.registerTable( nonePartitionTablePath, - TableRegistration.newTable(nonePartitionTableId, DATA1_TABLE_DESCRIPTOR)); + TableRegistration.newTable( + nonePartitionTableId, DEFAULT_REMOTE_DATA_DIR, DATA1_TABLE_DESCRIPTOR)); zkClient.registerFirstSchema(nonePartitionTablePath, DATA1_TABLE_DESCRIPTOR.getSchema()); zkClient.registerTable( partitionTablePath, - TableRegistration.newTable(partitionTableId, DATA1_PARTITIONED_TABLE_DESCRIPTOR)); + TableRegistration.newTable( + partitionTableId, + DEFAULT_REMOTE_DATA_DIR, + DATA1_PARTITIONED_TABLE_DESCRIPTOR)); zkClient.registerFirstSchema( partitionTablePath, DATA1_PARTITIONED_TABLE_DESCRIPTOR.getSchema()); @@ -1978,6 +1985,7 @@ void testUpdateMetadata() throws Exception { DELETED_TABLE_ID, // mark as deleted. 1, DATA1_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis()), Collections.emptyList())), @@ -2065,6 +2073,7 @@ void testGetReplicaOrException() { tableId, 1, DATA1_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis()); TableBucket tableBucket1 = new TableBucket(tableId, 1); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTestBase.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTestBase.java index 3fb01a7e25..8a2b107c5f 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTestBase.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTestBase.java @@ -110,6 +110,7 @@ import static org.apache.fluss.record.TestData.DATA3_TABLE_DESCRIPTOR_PK_AUTO_INC; import static org.apache.fluss.record.TestData.DATA3_TABLE_ID_PK_AUTO_INC; import static org.apache.fluss.record.TestData.DATA3_TABLE_PATH_PK_AUTO_INC; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.server.coordinator.CoordinatorContext.INITIAL_COORDINATOR_EPOCH; import static org.apache.fluss.server.replica.ReplicaManager.HIGH_WATERMARK_CHECKPOINT_FILE_NAME; import static org.apache.fluss.server.zk.data.LeaderAndIsr.INITIAL_BUCKET_EPOCH; @@ -259,22 +260,27 @@ private void registerTableInZkClient() throws Exception { TableDescriptor.builder().schema(DATA1_SCHEMA).distributedBy(3).build(); zkClient.registerTable( DATA1_TABLE_PATH, - TableRegistration.newTable(DATA1_TABLE_ID, data1NonPkTableDescriptor)); + TableRegistration.newTable( + DATA1_TABLE_ID, DEFAULT_REMOTE_DATA_DIR, data1NonPkTableDescriptor)); zkClient.registerFirstSchema(DATA1_TABLE_PATH, DATA1_SCHEMA); zkClient.registerTable( DATA1_TABLE_PATH_PK, - TableRegistration.newTable(DATA1_TABLE_ID_PK, DATA1_TABLE_DESCRIPTOR_PK)); + TableRegistration.newTable( + DATA1_TABLE_ID_PK, DEFAULT_REMOTE_DATA_DIR, DATA1_TABLE_DESCRIPTOR_PK)); zkClient.registerFirstSchema(DATA1_TABLE_PATH_PK, DATA1_SCHEMA_PK); zkClient.registerTable( DATA2_TABLE_PATH, - TableRegistration.newTable(DATA2_TABLE_ID, DATA2_TABLE_DESCRIPTOR)); + TableRegistration.newTable( + DATA2_TABLE_ID, DEFAULT_REMOTE_DATA_DIR, DATA2_TABLE_DESCRIPTOR)); zkClient.registerFirstSchema(DATA2_TABLE_PATH, DATA2_SCHEMA); zkClient.registerTable( DATA3_TABLE_PATH_PK_AUTO_INC, TableRegistration.newTable( - DATA3_TABLE_ID_PK_AUTO_INC, DATA3_TABLE_DESCRIPTOR_PK_AUTO_INC)); + DATA3_TABLE_ID_PK_AUTO_INC, + DEFAULT_REMOTE_DATA_DIR, + DATA3_TABLE_DESCRIPTOR_PK_AUTO_INC)); zkClient.registerFirstSchema(DATA3_TABLE_PATH_PK_AUTO_INC, DATA3_SCHEMA_PK_AUTO_INC); } @@ -293,7 +299,9 @@ protected long registerTableInZkClient( if (zkClient.tableExist(tablePath)) { zkClient.deleteTable(tablePath); } - zkClient.registerTable(tablePath, TableRegistration.newTable(tableId, tableDescriptor)); + zkClient.registerTable( + tablePath, + TableRegistration.newTable(tableId, DEFAULT_REMOTE_DATA_DIR, tableDescriptor)); zkClient.registerFirstSchema(tablePath, schema); return tableId; } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherThreadTest.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherThreadTest.java index c857d84ded..0501d0374b 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherThreadTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherThreadTest.java @@ -75,6 +75,7 @@ import static org.apache.fluss.record.TestData.DATA1_TABLE_DESCRIPTOR; import static org.apache.fluss.record.TestData.DATA1_TABLE_ID; import static org.apache.fluss.record.TestData.DATA1_TABLE_PATH; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.server.coordinator.CoordinatorContext.INITIAL_COORDINATOR_EPOCH; import static org.apache.fluss.server.metrics.group.TestingMetricGroups.USER_METRICS; import static org.apache.fluss.server.zk.data.LeaderAndIsr.INITIAL_BUCKET_EPOCH; @@ -377,7 +378,8 @@ private void registerTableInZkClient() throws Exception { ZOO_KEEPER_EXTENSION_WRAPPER.getCustomExtension().cleanupRoot(); zkClient.registerTable( DATA1_TABLE_PATH, - TableRegistration.newTable(DATA1_TABLE_ID, DATA1_TABLE_DESCRIPTOR)); + TableRegistration.newTable( + DATA1_TABLE_ID, DEFAULT_REMOTE_DATA_DIR, DATA1_TABLE_DESCRIPTOR)); zkClient.registerFirstSchema(DATA1_TABLE_PATH, DATA1_SCHEMA); } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java b/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java index 305a4269aa..b583499414 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java @@ -137,6 +137,7 @@ public final class FlussClusterExtension private final Configuration clusterConf; private final Clock clock; private final String[] racks; + private final List remoteDirNames; /** Creates a new {@link Builder} for {@link FlussClusterExtension}. */ public static Builder builder() { @@ -149,7 +150,8 @@ private FlussClusterExtension( String tabletServerListeners, Configuration clusterConf, Clock clock, - String[] racks) { + String[] racks, + List remoteDirNames) { this.initialNumOfTabletServers = numOfTabletServers; this.tabletServers = new HashMap<>(numOfTabletServers); this.coordinatorServerListeners = coordinatorServerListeners; @@ -161,6 +163,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; } @Override @@ -203,15 +206,19 @@ public void afterEach(ExtensionContext extensionContext) throws Exception { public void start() throws Exception { tempDir = Files.createTempDirectory("fluss-testing-cluster").toFile(); + Configuration conf = new Configuration(); + setRemoteDataDir(conf); + setRemoteDataDirs(conf); zooKeeperServer = ZooKeeperTestUtils.createAndStartZookeeperTestingServer(); zooKeeperClient = - createZooKeeperClient(zooKeeperServer.getConnectString(), NOPErrorHandler.INSTANCE); + createZooKeeperClient( + conf, zooKeeperServer.getConnectString(), NOPErrorHandler.INSTANCE); metadataManager = new MetadataManager( zooKeeperClient, clusterConf, new LakeCatalogDynamicLoader(clusterConf, null, true)); - Configuration conf = new Configuration(); + rpcClient = RpcClient.create( conf, @@ -261,6 +268,7 @@ public void startCoordinatorServer() throws Exception { conf.setString(ConfigOptions.ZOOKEEPER_ADDRESS, zooKeeperServer.getConnectString()); conf.setString(ConfigOptions.BIND_LISTENERS, coordinatorServerListeners); setRemoteDataDir(conf); + setRemoteDataDirs(conf); coordinatorServer = new CoordinatorServer(conf, clock); coordinatorServer.start(); coordinatorServerInfo = @@ -371,12 +379,26 @@ private void setRemoteDataDir(Configuration conf) { conf.set(ConfigOptions.REMOTE_DATA_DIR, getRemoteDataDir()); } + private void setRemoteDataDirs(Configuration conf) { + if (!remoteDirNames.isEmpty()) { + List remoteDataDirs = + remoteDirNames.stream() + .map(this::getRemoteDataDir) + .collect(Collectors.toList()); + conf.set(ConfigOptions.REMOTE_DATA_DIRS, remoteDataDirs); + } + } + public String getRemoteDataDir() { + return getRemoteDataDir("remote-data-dir"); + } + + public String getRemoteDataDir(String dirName) { return LocalFileSystem.getLocalFsURI().getScheme() + "://" + tempDir.getAbsolutePath() + File.separator - + "remote-data-dir"; + + dirName; } /** Stop a tablet server. */ @@ -774,7 +796,7 @@ private Long triggerSnapshot(TableBucket tableBucket) { } } - private CompletedSnapshot waitUntilSnapshotFinished(TableBucket tableBucket, long snapshotId) { + public CompletedSnapshot waitUntilSnapshotFinished(TableBucket tableBucket, long snapshotId) { ZooKeeperClient zkClient = getZooKeeperClient(); return waitValue( () -> { @@ -940,6 +962,7 @@ public static class Builder { private String coordinatorServerListeners = DEFAULT_LISTENERS; private Clock clock = SystemClock.getInstance(); private String[] racks = new String[] {"rack-0"}; + private List remoteDirNames = Collections.emptyList(); private final Configuration clusterConf = new Configuration(); @@ -985,6 +1008,11 @@ public Builder setRacks(String[] racks) { return this; } + public Builder setRemoteDirNames(List remoteDirNames) { + this.remoteDirNames = remoteDirNames; + return this; + } + public FlussClusterExtension build() { if (numOfTabletServers > 1 && racks.length == 1) { String[] racks = new String[numOfTabletServers]; @@ -1000,7 +1028,8 @@ public FlussClusterExtension build() { tabletServerListeners, clusterConf, clock, - racks); + racks, + remoteDirNames); } } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java index 4cf0a511e7..20e9a49a3e 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java @@ -29,7 +29,6 @@ import org.apache.fluss.metadata.SchemaInfo; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableDescriptor; -import org.apache.fluss.metadata.TablePartition; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.server.entity.RegisterTableBucketLeadAndIsrInfo; import org.apache.fluss.server.zk.data.BucketAssignment; @@ -37,6 +36,7 @@ import org.apache.fluss.server.zk.data.CoordinatorAddress; import org.apache.fluss.server.zk.data.LeaderAndIsr; import org.apache.fluss.server.zk.data.PartitionAssignment; +import org.apache.fluss.server.zk.data.PartitionRegistration; import org.apache.fluss.server.zk.data.RebalanceTask; import org.apache.fluss.server.zk.data.ServerTags; import org.apache.fluss.server.zk.data.TableAssignment; @@ -84,6 +84,7 @@ class ZooKeeperClientTest { new AllCallbackWrapper<>(new ZooKeeperExtension()); private static ZooKeeperClient zookeeperClient; + private static String remoteDataDir; @BeforeAll static void beforeAll() { @@ -91,6 +92,7 @@ static void beforeAll() { ZOO_KEEPER_EXTENSION_WRAPPER .getCustomExtension() .getZooKeeperClient(NOPErrorHandler.INSTANCE); + remoteDataDir = zookeeperClient.getRemoteDataDir(); } @AfterEach @@ -331,6 +333,7 @@ void testTable() throws Exception { new TableDescriptor.TableDistribution(16, Collections.singletonList("a")), options, Collections.singletonMap("custom-1", "100"), + remoteDataDir, currentMillis, currentMillis); TableRegistration tableReg2 = @@ -341,6 +344,7 @@ void testTable() throws Exception { new TableDescriptor.TableDistribution(16, Collections.singletonList("a")), options, Collections.singletonMap("custom-2", "200"), + remoteDataDir, currentMillis, currentMillis); zookeeperClient.registerTable(tablePath1, tableReg1); @@ -366,6 +370,7 @@ void testTable() throws Exception { new TableDescriptor.TableDistribution(16, Collections.singletonList("a")), options, Collections.singletonMap("custom-3", "300"), + remoteDataDir, currentMillis, currentMillis); zookeeperClient.updateTable(tablePath1, tableReg1); @@ -550,6 +555,7 @@ void testPartition() throws Exception { new TableDescriptor.TableDistribution(16, Collections.singletonList("a")), Collections.emptyMap(), Collections.emptyMap(), + remoteDataDir, currentMillis, currentMillis); zookeeperClient.registerTable(tablePath, tableReg); @@ -571,14 +577,14 @@ void testPartition() throws Exception { }) .getBucketAssignments()); zookeeperClient.registerPartitionAssignmentAndMetadata( - 1L, "p1", partitionAssignment, tablePath, tableId); + 1L, "p1", partitionAssignment, remoteDataDir, tablePath, tableId); zookeeperClient.registerPartitionAssignmentAndMetadata( - 2L, "p2", partitionAssignment, tablePath, tableId); + 2L, "p2", partitionAssignment, remoteDataDir, tablePath, tableId); // check created partitions partitions = zookeeperClient.getPartitions(tablePath); assertThat(partitions).containsExactly("p1", "p2"); - TablePartition partition = zookeeperClient.getPartition(tablePath, "p1").get(); + PartitionRegistration partition = zookeeperClient.getPartition(tablePath, "p1").get(); assertThat(partition.getPartitionId()).isEqualTo(1L); partition = zookeeperClient.getPartition(tablePath, "p2").get(); assertThat(partition.getPartitionId()).isEqualTo(2L); @@ -677,6 +683,7 @@ void testZookeeperConfigPath() throws Exception { config.setString( ConfigOptions.ZOOKEEPER_ADDRESS, ZOO_KEEPER_EXTENSION_WRAPPER.getCustomExtension().getConnectString()); + config.set(ConfigOptions.REMOTE_DATA_DIR, remoteDataDir.toString()); config.setString(ConfigOptions.ZOOKEEPER_CONFIG_PATH, "./no-file.properties"); assertThatThrownBy( () -> ZooKeeperUtils.startZookeeperClient(config, NOPErrorHandler.INSTANCE)) @@ -720,6 +727,7 @@ void testGetDatabaseSummary() throws Exception { new TableDescriptor.TableDistribution(16, Collections.singletonList("a")), Collections.emptyMap(), Collections.emptyMap(), + remoteDataDir, beforeCreateTime, beforeCreateTime); zookeeperClient.registerTable(tablePath, tableReg1); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperExtension.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperExtension.java index 18516960ae..41bb965d73 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperExtension.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperExtension.java @@ -17,6 +17,9 @@ package org.apache.fluss.server.zk; +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.fs.local.LocalFileSystem; import org.apache.fluss.server.utils.FatalErrorHandler; import org.apache.fluss.shaded.zookeeper3.org.apache.zookeeper.KeeperException; import org.apache.fluss.testutils.common.CustomExtension; @@ -30,7 +33,9 @@ import javax.annotation.Nullable; +import java.io.File; import java.io.IOException; +import java.nio.file.Files; import static org.apache.fluss.utils.Preconditions.checkNotNull; import static org.apache.fluss.utils.Preconditions.checkState; @@ -44,6 +49,8 @@ public class ZooKeeperExtension implements CustomExtension { @Nullable private ZooKeeperClient zooKeeperClient; + private File tempDir; + @Override public void before(ExtensionContext context) throws Exception { close(); @@ -96,7 +103,27 @@ public ZooKeeperClient getZooKeeperClient(FatalErrorHandler fatalErrorHandler) { } public ZooKeeperClient createZooKeeperClient(FatalErrorHandler fatalErrorHandler) { - return ZooKeeperTestUtils.createZooKeeperClient(getConnectString(), fatalErrorHandler); + try { + tempDir = Files.createTempDirectory(null).toFile(); + Configuration conf = new Configuration(); + setRemoteDataDir(conf); + return ZooKeeperTestUtils.createZooKeeperClient( + conf, getConnectString(), fatalErrorHandler); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + private void setRemoteDataDir(Configuration conf) { + conf.set(ConfigOptions.REMOTE_DATA_DIR, getRemoteDataDir()); + } + + public String getRemoteDataDir() { + return LocalFileSystem.getLocalFsURI().getScheme() + + "://" + + tempDir.getAbsolutePath() + + File.separator + + "remote-data-dir"; } public void restart() throws Exception { diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperTestUtils.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperTestUtils.java index 9d287bd63d..d850b2194d 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperTestUtils.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperTestUtils.java @@ -43,8 +43,7 @@ public static TestingServer createAndStartZookeeperTestingServer() throws Except /** Create a {@link ZooKeeperClient} client using provided connect string. */ public static ZooKeeperClient createZooKeeperClient( - String connectString, FatalErrorHandler fatalErrorHandler) { - final Configuration config = new Configuration(); + Configuration config, String connectString, FatalErrorHandler fatalErrorHandler) { config.setString(ConfigOptions.ZOOKEEPER_ADDRESS, connectString); return ZooKeeperUtils.startZookeeperClient(config, fatalErrorHandler); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/PartitionRegistrationJsonSerdeTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/PartitionRegistrationJsonSerdeTest.java new file mode 100644 index 0000000000..26ba77ac62 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/PartitionRegistrationJsonSerdeTest.java @@ -0,0 +1,44 @@ +/* + * 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.zk.data; + +import org.apache.fluss.utils.json.JsonSerdeTestBase; + +/** Test for {@link PartitionRegistrationJsonSerde}. */ +class PartitionRegistrationJsonSerdeTest extends JsonSerdeTestBase { + + PartitionRegistrationJsonSerdeTest() { + super(PartitionRegistrationJsonSerde.INSTANCE); + } + + @Override + protected PartitionRegistration[] createObjects() { + PartitionRegistration[] partitionRegistrations = new PartitionRegistration[1]; + + partitionRegistrations[0] = new PartitionRegistration(1234L, 5678L, "file://local/remote"); + + return partitionRegistrations; + } + + @Override + protected String[] expectedJsons() { + return new String[] { + "{\"version\":1,\"table_id\":1234,\"partition_id\":5678,\"remote_data_dir\":\"file://local/remote\"}" + }; + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/TableRegistrationJsonSerdeTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/TableRegistrationJsonSerdeTest.java index 348b20cfca..0d6a3f2fc0 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/TableRegistrationJsonSerdeTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/TableRegistrationJsonSerdeTest.java @@ -48,6 +48,7 @@ void testInvalidTableRegistration() { new TableDistribution(null, Arrays.asList("b", "c")), Maps.newHashMap(), Collections.singletonMap("custom-3", "\"300\""), + "file://local/remote", 1735538268L, 1735538268L)) .isInstanceOf(IllegalArgumentException.class) @@ -58,6 +59,7 @@ void testInvalidTableRegistration() { () -> TableRegistration.newTable( 11, + "file://local/remote", TableDescriptor.builder() .schema(TestData.DATA1_SCHEMA) .build())) @@ -77,6 +79,7 @@ protected TableRegistration[] createObjects() { new TableDistribution(16, Arrays.asList("b", "c")), Maps.newHashMap(), Collections.singletonMap("custom-3", "\"300\""), + "file://local/remote", 1735538268L, 1735538268L); @@ -88,6 +91,7 @@ protected TableRegistration[] createObjects() { new TableDistribution(32, Collections.emptyList()), Collections.singletonMap("option-3", "300"), Maps.newHashMap(), + "file://local/remote", -1, -1); @@ -98,8 +102,8 @@ protected TableRegistration[] createObjects() { protected String[] expectedJsons() { return new String[] { "{\"version\":1,\"table_id\":1234,\"comment\":\"first-table\",\"partition_key\":[\"a\",\"b\"]," - + "\"bucket_key\":[\"b\",\"c\"],\"bucket_count\":16,\"properties\":{},\"custom_properties\":{\"custom-3\":\"\\\"300\\\"\"},\"created_time\":1735538268,\"modified_time\":1735538268}", - "{\"version\":1,\"table_id\":1234,\"comment\":\"second-table\",\"bucket_count\":32,\"properties\":{\"option-3\":\"300\"},\"custom_properties\":{},\"created_time\":-1,\"modified_time\":-1}", + + "\"bucket_key\":[\"b\",\"c\"],\"bucket_count\":16,\"properties\":{},\"custom_properties\":{\"custom-3\":\"\\\"300\\\"\"},\"remote_data_dir\":\"file://local/remote\",\"created_time\":1735538268,\"modified_time\":1735538268}", + "{\"version\":1,\"table_id\":1234,\"comment\":\"second-table\",\"bucket_count\":32,\"properties\":{\"option-3\":\"300\"},\"custom_properties\":{},\"remote_data_dir\":\"file://local/remote\",\"created_time\":-1,\"modified_time\":-1}", }; } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableHelperTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableHelperTest.java index 1f759d3fac..ab88b7b042 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableHelperTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableHelperTest.java @@ -59,6 +59,7 @@ class LakeTableHelperTest { new AllCallbackWrapper<>(new ZooKeeperExtension()); private static ZooKeeperClient zookeeperClient; + private static String remoteDataDir; @BeforeAll static void beforeAll() { @@ -66,6 +67,7 @@ static void beforeAll() { ZOO_KEEPER_EXTENSION_WRAPPER .getCustomExtension() .getZooKeeperClient(NOPErrorHandler.INSTANCE); + remoteDataDir = zookeeperClient.getRemoteDataDir(); } @AfterEach @@ -267,6 +269,7 @@ private TableRegistration createTableReg(long tableId) { new TableDescriptor.TableDistribution(1, Collections.singletonList("a")), Collections.emptyMap(), Collections.emptyMap(), + remoteDataDir, System.currentTimeMillis(), System.currentTimeMillis()); } diff --git a/website/docs/maintenance/configuration.md b/website/docs/maintenance/configuration.md index 80d40fddbc..9566a96849 100644 --- a/website/docs/maintenance/configuration.md +++ b/website/docs/maintenance/configuration.md @@ -36,6 +36,9 @@ during the Fluss cluster working. | default.bucket.number | Integer | 1 | The default number of buckets for a table in Fluss cluster. It's a cluster-level parameter and all the tables without specifying bucket number in the cluster will use the value as the bucket number. | | default.replication.factor | Integer | 1 | The default replication factor for the log of a table in Fluss cluster. It's a cluster-level parameter, and all the tables without specifying replication factor in the cluster will use the value as replication factor. | | remote.data.dir | String | (None) | The directory used for storing the kv snapshot data files and remote log for log tiered storage in a Fluss supported filesystem. | +| remote.data.dirs | List<String> | (None) | The directories used for storing the kv snapshot data files and remote log for log tiered storage in a Fluss supported filesystem. This is a list of remote data directory paths. Example: `remote.data.dirs: oss://bucket1/fluss-remote-data, oss://bucket2/fluss-remote-data`. | +| remote.data.dirs.strategy | Enum | ROUND_ROBIN | The strategy for selecting the remote data directory from `remote.data.dirs`. | +| remote.data.dirs.weights | List<Integer>| (None) | The weights of the remote data directories. This is a list of weights corresponding to the `remote.data.dirs` in the same order. When `remote.data.dirs.strategy` is set to `WEIGHTED_ROUND_ROBIN`, this must be configured, and its size must be equal to `remote.data.dirs`; otherwise, it will be ignored.Example: `remote.data.dir.weights: 1, 2` | | remote.fs.write-buffer-size | MemorySize | 4kb | The default size of the write buffer for writing the local files to remote file systems. | | plugin.classloader.parent-first-patterns.additional | List<String> | (None) | A (semicolon-separated) list of patterns that specifies which classes should always be resolved through the plugin parent ClassLoader first. A pattern is a simple prefix that is checked against the fully qualified class name. These patterns are appended to `classloader.parent-first-patterns.default`. | | plugin.classloader.parent-first-patterns.default | String | java.,
org.apache.fluss.,
javax.annotation.,
org.slf4j,
org.apache.log4j,
org.apache.logging,
org.apache.commons.logging,
ch.qos.logback | A (semicolon-separated) list of patterns that specifies which classes should always be resolved through the plugin parent ClassLoader first. A pattern is a simple prefix that is checked against the fully qualified class name. This setting should generally not be modified. |