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 9f4b603e98..2a71ed4158 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 @@ -96,6 +96,37 @@ 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_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. " + + "Example: remote.data.dir.weights: 1, 2"); + + 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. " + + "The default value is ROUND_ROBIN."); + public static final ConfigOption REMOTE_FS_WRITE_BUFFER_SIZE = key("remote.fs.write-buffer-size") .memoryType() @@ -1925,4 +1956,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 fa9c4274c9..7433eab5e2 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,14 @@ import org.apache.fluss.annotation.Internal; import org.apache.fluss.annotation.VisibleForTesting; +import org.apache.fluss.exception.IllegalConfigurationException; import java.lang.reflect.Field; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; /** Utilities of Fluss {@link ConfigOptions}. */ @Internal @@ -74,4 +76,82 @@ static Map> extractConfigOptions(String prefix) { } return options; } + + public static void validateCoordinatorConfigs(Configuration conf) { + validServerConfigs(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())); + } + + // validate remote.data.dirs + List remoteDataDirs = conf.get(ConfigOptions.REMOTE_DATA_DIRS); + 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.size() != weights.size()) { + throw new IllegalConfigurationException( + String.format( + "Invalid configuration for %s, its size must be equal to the size of %s.", + ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS.key(), + ConfigOptions.REMOTE_DATA_DIRS.key())); + } + } + } + + 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)); + } + + 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.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)); + } + } + + 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)); + } + } } diff --git a/fluss-common/src/main/java/org/apache/fluss/remote/RemoteLogSegment.java b/fluss-common/src/main/java/org/apache/fluss/remote/RemoteLogSegment.java index 39480e4d10..a799051179 100644 --- a/fluss-common/src/main/java/org/apache/fluss/remote/RemoteLogSegment.java +++ b/fluss-common/src/main/java/org/apache/fluss/remote/RemoteLogSegment.java @@ -18,6 +18,7 @@ package org.apache.fluss.remote; import org.apache.fluss.annotation.Internal; +import org.apache.fluss.fs.FsPath; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.TableBucket; @@ -50,6 +51,8 @@ public class RemoteLogSegment { private final int segmentSizeInBytes; + private final FsPath remoteLogDir; + private RemoteLogSegment( PhysicalTablePath physicalTablePath, TableBucket tableBucket, @@ -57,7 +60,8 @@ private RemoteLogSegment( long remoteLogStartOffset, long remoteLogEndOffset, long maxTimestamp, - int segmentSizeInBytes) { + int segmentSizeInBytes, + FsPath remoteLogDir) { this.physicalTablePath = checkNotNull(physicalTablePath); this.tableBucket = checkNotNull(tableBucket); this.remoteLogSegmentId = checkNotNull(remoteLogSegmentId); @@ -79,6 +83,7 @@ private RemoteLogSegment( this.remoteLogEndOffset = remoteLogEndOffset; this.maxTimestamp = maxTimestamp; this.segmentSizeInBytes = segmentSizeInBytes; + this.remoteLogDir = remoteLogDir; } public PhysicalTablePath physicalTablePath() { @@ -115,6 +120,10 @@ public int segmentSizeInBytes() { return segmentSizeInBytes; } + public FsPath remoteLogDir() { + return remoteLogDir; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -174,6 +183,7 @@ public static class Builder { private long remoteLogEndOffset; private long maxTimestamp; private int segmentSizeInBytes; + private FsPath remoteLogDir; public static Builder builder() { return new Builder(); @@ -214,6 +224,11 @@ public Builder tableBucket(TableBucket tableBucket) { return this; } + public Builder remoteLogDir(FsPath remoteLogDir) { + this.remoteLogDir = remoteLogDir; + return this; + } + public RemoteLogSegment build() { return new RemoteLogSegment( physicalTablePath, @@ -222,7 +237,8 @@ public RemoteLogSegment build() { remoteLogStartOffset, remoteLogEndOffset, maxTimestamp, - segmentSizeInBytes); + segmentSizeInBytes, + remoteLogDir); } } } 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 d04268fbaa..fc196f8070 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 @@ -401,6 +401,14 @@ public static UUID uuidFromRemoteIndexCacheFileName(String fileName) { fileName.substring(fileName.indexOf('_') + 1, fileName.indexOf('.'))); } + // ---------------------------------------------------------------------------------------- + // Remote Data Paths + // ---------------------------------------------------------------------------------------- + + public static FsPath remoteDataDir(Configuration conf) { + return new FsPath(conf.get(ConfigOptions.REMOTE_DATA_DIR)); + } + // ---------------------------------------------------------------------------------------- // Remote Log Paths // ---------------------------------------------------------------------------------------- @@ -418,6 +426,10 @@ public static FsPath remoteLogDir(Configuration conf) { return new FsPath(conf.get(ConfigOptions.REMOTE_DATA_DIR) + "/" + REMOTE_LOG_DIR_NAME); } + public static FsPath remoteLogDir(FsPath remoteDataDir) { + return new FsPath(remoteDataDir, REMOTE_LOG_DIR_NAME); + } + /** * Returns the remote directory path for storing log files for a log tablet. * @@ -584,6 +596,10 @@ public static FsPath remoteKvDir(Configuration conf) { return new FsPath(conf.get(ConfigOptions.REMOTE_DATA_DIR) + "/" + REMOTE_KV_DIR_NAME); } + public static FsPath remoteKvDir(FsPath remoteDataDir) { + return new FsPath(remoteDataDir, REMOTE_KV_DIR_NAME); + } + /** * Returns the remote directory path for storing kv snapshot files for a kv tablet. * diff --git a/fluss-common/src/main/java/org/apache/fluss/utils/RemoteDataDirConfigUtils.java b/fluss-common/src/main/java/org/apache/fluss/utils/RemoteDataDirConfigUtils.java new file mode 100644 index 0000000000..267852c157 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/utils/RemoteDataDirConfigUtils.java @@ -0,0 +1,171 @@ +/* + * 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.utils; + +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.config.Configuration; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.URI; +import java.util.List; + +/** + * Utility class for extracting path-specific configurations for remote data directories. + * + *

This class supports per-path authentication by allowing configuration keys to be prefixed with + * {@code remote.data.dir.} to override global settings for specific paths. + * + *

Example configuration: + * + *

+ * remote.data.dirs: [oss://bucket1/path1, oss://bucket2/path2]
+ * # Global OSS configuration
+ * fs.oss.endpoint: oss-cn-hangzhou.aliyuncs.com
+ * fs.oss.region: cn-hangzhou
+ * # Path-specific configuration for the first path (index 0)
+ * remote.data.dir.0.fs.oss.accessKeyId: ak1
+ * remote.data.dir.0.fs.oss.accessKeySecret: sk1
+ * # Path-specific configuration for the second path (index 1)
+ * remote.data.dir.1.fs.oss.accessKeyId: ak2
+ * remote.data.dir.1.fs.oss.accessKeySecret: sk2
+ * 
+ */ +public class RemoteDataDirConfigUtils { + + private static final Logger LOG = LoggerFactory.getLogger(RemoteDataDirConfigUtils.class); + + private static final String REMOTE_DATA_DIR_PREFIX = "remote.data.dir."; + + /** + * Creates a merged configuration for a given URI by combining global configuration with + * path-specific overrides. + * + *

If the URI matches one of the configured remote data directories, path-specific + * configurations (prefixed with {@code remote.data.dir.}) will override the global + * configuration. + * + * @param uri the URI to get configuration for + * @param globalConfig the global configuration + * @return a new Configuration object with path-specific overrides applied + */ + public static Configuration getMergedConfigForUri(URI uri, Configuration globalConfig) { + Configuration mergedConfig = new Configuration(globalConfig); + + // Find the index of the URI in remote.data.dirs + Integer pathIndex = findPathIndex(uri, globalConfig); + if (pathIndex == null) { + // URI doesn't match any configured remote data directory, return global config + return mergedConfig; + } + + // Apply path-specific configuration overrides + String pathPrefix = REMOTE_DATA_DIR_PREFIX + pathIndex + "."; + for (String key : globalConfig.keySet()) { + if (key.startsWith(pathPrefix)) { + String originalKey = key.substring(pathPrefix.length()); + String value = + globalConfig.getString( + org.apache.fluss.config.ConfigBuilder.key(key) + .stringType() + .noDefaultValue(), + null); + if (value != null) { + mergedConfig.setString(originalKey, value); + LOG.debug( + "Applied path-specific config override: {} = {} (from {})", + originalKey, + maskSensitiveValue(originalKey, value), + key); + } + } + } + + return mergedConfig; + } + + /** + * Finds the index of the given URI in the remote.data.dirs configuration. + * + * @param uri the URI to find + * @param config the configuration + * @return the index if found, null otherwise + */ + public static Integer findPathIndex(URI uri, Configuration config) { + List remoteDataDirs = config.get(ConfigOptions.REMOTE_DATA_DIRS); + if (remoteDataDirs == null || remoteDataDirs.isEmpty()) { + return null; + } + + String uriString = uri.toString(); + // Normalize URI by removing trailing slashes for comparison + String normalizedUri = normalizeUri(uriString); + + for (int i = 0; i < remoteDataDirs.size(); i++) { + String remoteDataDir = remoteDataDirs.get(i); + String normalizedDir = normalizeUri(remoteDataDir); + // Check if the URI starts with the remote data dir path + if (normalizedUri.startsWith(normalizedDir)) { + LOG.debug( + "Found URI {} matches remote.data.dirs[{}] = {}", + uriString, + i, + remoteDataDir); + return i; + } + } + + return null; + } + + /** + * Normalizes a URI string by removing trailing slashes. + * + * @param uriString the URI string to normalize + * @return the normalized URI string + */ + private static String normalizeUri(String uriString) { + String normalized = uriString; + while (normalized.endsWith("/")) { + normalized = normalized.substring(0, normalized.length() - 1); + } + return normalized; + } + + /** + * Masks sensitive configuration values in log messages. + * + * @param key the configuration key + * @param value the configuration value + * @return the masked value if the key is sensitive, otherwise the original value + */ + private static String maskSensitiveValue(String key, String value) { + if (key == null || value == null) { + return value; + } + String lowerKey = key.toLowerCase(); + if (lowerKey.contains("secret") + || lowerKey.contains("password") + || lowerKey.contains("key") + || lowerKey.contains("token")) { + return "***"; + } + return value; + } +} 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..a6af9fdbb8 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,72 @@ 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 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); + } } 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 32088399e2..eac4497a97 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 @@ -247,6 +247,7 @@ public static Map createPartitions( partition, new PartitionAssignment( tableInfo.getTableId(), assignment.getBucketAssignments()), + null, tablePath, tableInfo.getTableId()); } 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 b12bb787c7..2fca5a3d0a 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; @@ -343,7 +343,7 @@ public CompletableFuture getLatestKvSnapshots( } private long getPartitionId(TablePath tablePath, String partitionName) { - Optional optTablePartition; + Optional optTablePartition; try { optTablePartition = zkClient.getPartition(tablePath, partitionName); } catch (Exception e) { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/TabletManagerBase.java b/fluss-server/src/main/java/org/apache/fluss/server/TabletManagerBase.java index 6525df1c45..fd5268692d 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/TabletManagerBase.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/TabletManagerBase.java @@ -20,6 +20,7 @@ import org.apache.fluss.config.Configuration; import org.apache.fluss.exception.KvStorageException; import org.apache.fluss.exception.LogStorageException; +import org.apache.fluss.exception.PartitionNotExistException; import org.apache.fluss.exception.SchemaNotExistException; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.SchemaInfo; @@ -29,6 +30,7 @@ import org.apache.fluss.server.kv.KvManager; import org.apache.fluss.server.log.LogManager; 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.utils.FileUtils; import org.apache.fluss.utils.FlussPaths; @@ -200,6 +202,26 @@ protected File getTabletDir(PhysicalTablePath tablePath, TableBucket tableBucket // TODO: we should support get table info from local properties file instead of from zk public static TableInfo getTableInfo(ZooKeeperClient zkClient, TablePath tablePath) throws Exception { + SchemaInfo schemaInfo = getSchemaInfo(zkClient, tablePath); + + TableRegistration tableRegistration = getTableRegistration(zkClient, tablePath); + + return tableRegistration.toTableInfo(tablePath, schemaInfo); + } + + public static TableRegistration getTableRegistration( + ZooKeeperClient zkClient, TablePath tablePath) throws Exception { + return zkClient.getTable(tablePath) + .orElseThrow( + () -> + new LogStorageException( + String.format( + "Failed to load table '%s': table info not found in zookeeper metadata.", + tablePath))); + } + + public static SchemaInfo getSchemaInfo(ZooKeeperClient zkClient, TablePath tablePath) + throws Exception { int schemaId = zkClient.getCurrentSchemaId(tablePath); Optional schemaInfoOpt = zkClient.getSchemaById(tablePath, schemaId); SchemaInfo schemaInfo; @@ -211,17 +233,20 @@ public static TableInfo getTableInfo(ZooKeeperClient zkClient, TablePath tablePa } else { schemaInfo = schemaInfoOpt.get(); } + return schemaInfo; + } - TableRegistration tableRegistration = - zkClient.getTable(tablePath) - .orElseThrow( - () -> - new LogStorageException( - String.format( - "Failed to load table '%s': table info not found in zookeeper metadata.", - tablePath))); - - return tableRegistration.toTableInfo(tablePath, schemaInfo); + public static PartitionRegistration getPartitionRegistration( + ZooKeeperClient zkClient, PhysicalTablePath physicalTablePath) throws Exception { + return zkClient.getPartition( + physicalTablePath.getTablePath(), physicalTablePath.getPartitionName()) + .orElseThrow( + () -> + new PartitionNotExistException( + String.format( + "Failed to load partition '%s' for table %s: partition info not found in zookeeper metadata.", + physicalTablePath.getPartitionName(), + physicalTablePath.getTablePath()))); } /** Create a tablet directory in the given dir. */ 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..51b42ebd39 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 @@ -26,9 +26,11 @@ import org.apache.fluss.exception.PartitionNotExistException; import org.apache.fluss.exception.TooManyBucketsException; import org.apache.fluss.exception.TooManyPartitionsException; +import org.apache.fluss.fs.FsPath; 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 +87,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 +111,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 +127,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 +356,11 @@ private void createPartitions( PartitionAssignment partitionAssignment = new PartitionAssignment(tableInfo.getTableId(), bucketAssignments); + // select a remote data dir for the partition + FsPath 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/CoordinatorServer.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java index cd546f4479..b7b5ccaf53 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; @@ -33,6 +32,7 @@ import org.apache.fluss.server.ServerBase; import org.apache.fluss.server.authorizer.Authorizer; import org.apache.fluss.server.authorizer.AuthorizerLoader; +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; @@ -62,6 +62,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: * @@ -140,9 +142,12 @@ public class CoordinatorServer extends ServerBase { @GuardedBy("lock") private LakeCatalogDynamicLoader lakeCatalogDynamicLoader; + @GuardedBy("lock") + private RemoteDirDynamicLoader remoteDirDynamicLoader; + public CoordinatorServer(Configuration conf) { super(conf); - validateConfigs(conf); + validateCoordinatorConfigs(conf); this.terminationFuture = new CompletableFuture<>(); } @@ -172,10 +177,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(); @@ -206,6 +214,7 @@ protected void startServices() throws Exception { authorizer, lakeCatalogDynamicLoader, lakeTableTieringManager, + remoteDirDynamicLoader, dynamicConfigManager, ioExecutor); @@ -230,7 +239,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 @@ -443,6 +453,10 @@ CompletableFuture stopServices() { if (lakeCatalogDynamicLoader != null) { lakeCatalogDynamicLoader.close(); } + + if (remoteDirDynamicLoader != null) { + remoteDirDynamicLoader.close(); + } } catch (Throwable t) { exception = ExceptionUtils.firstOrSuppressed(t, exception); } @@ -507,31 +521,4 @@ public ServerMetadataCache getMetadataCache() { public DynamicConfigManager getDynamicConfigManager() { return dynamicConfigManager; } - - 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 7fc53b96f8..26567a9311 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 @@ -119,6 +119,7 @@ import org.apache.fluss.server.coordinator.event.ControlledShutdownEvent; import org.apache.fluss.server.coordinator.event.EventManager; import org.apache.fluss.server.coordinator.event.RemoveServerTagEvent; +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; @@ -183,6 +184,7 @@ public final class CoordinatorService extends RpcServiceBase implements Coordina private final LakeCatalogDynamicLoader lakeCatalogDynamicLoader; private final ExecutorService ioExecutor; private final LakeTableHelper lakeTableHelper; + private final RemoteDirDynamicLoader remoteDirDynamicLoader; public CoordinatorService( Configuration conf, @@ -194,6 +196,7 @@ public CoordinatorService( @Nullable Authorizer authorizer, LakeCatalogDynamicLoader lakeCatalogDynamicLoader, LakeTableTieringManager lakeTableTieringManager, + RemoteDirDynamicLoader remoteDirDynamicLoader, DynamicConfigManager dynamicConfigManager, ExecutorService ioExecutor) { super( @@ -218,6 +221,7 @@ public CoordinatorService( this.ioExecutor = ioExecutor; this.lakeTableHelper = new LakeTableHelper(zkClient, conf.getString(ConfigOptions.REMOTE_DATA_DIR)); + this.remoteDirDynamicLoader = remoteDirDynamicLoader; } @Override @@ -331,9 +335,21 @@ public CompletableFuture createTable(CreateTableRequest req } } + // select remote data dir for non-partitioned table + FsPath remoteDataDir = null; + if (!tableDescriptor.isPartitioned()) { + RemoteDirDynamicLoader.RemoteDirContainer remoteDataDirContainer = + remoteDirDynamicLoader.getRemoteDataDirContainer(); + remoteDataDir = remoteDataDirContainer.nextDataDir(); + } + // then create table; metadataManager.createTable( - tablePath, tableDescriptor, tableAssignment, request.isIgnoreIfExists()); + tablePath, + remoteDataDir, + tableDescriptor, + tableAssignment, + request.isIgnoreIfExists()); return CompletableFuture.completedFuture(new CreateTableResponse()); } @@ -534,9 +550,15 @@ public CompletableFuture createPartition( PartitionAssignment partitionAssignment = new PartitionAssignment(table.tableId, bucketAssignments); + // select remote data dir for partition + RemoteDirDynamicLoader.RemoteDirContainer remoteDataDirContainer = + remoteDirDynamicLoader.getRemoteDataDirContainer(); + FsPath remoteDataDir = remoteDataDirContainer.nextDataDir(); + metadataManager.createPartition( tablePath, table.tableId, + remoteDataDir, partitionAssignment, partitionToCreate, request.isIgnoreIfNotExists()); @@ -591,6 +613,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 9d5dc34790..6e7eefa315 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 @@ -34,6 +34,7 @@ import org.apache.fluss.exception.TableNotPartitionedException; import org.apache.fluss.exception.TooManyBucketsException; import org.apache.fluss.exception.TooManyPartitionsException; +import org.apache.fluss.fs.FsPath; import org.apache.fluss.lake.lakestorage.LakeCatalog; import org.apache.fluss.metadata.DatabaseDescriptor; import org.apache.fluss.metadata.DatabaseInfo; @@ -49,6 +50,7 @@ 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; @@ -267,6 +269,8 @@ 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, will be null when the table is partitioned + * table * @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 @@ -274,6 +278,7 @@ public void completeDeletePartition(long partitionId) { */ public long createTable( TablePath tablePath, + @Nullable FsPath remoteDataDir, TableDescriptor tableToCreate, @Nullable TableAssignment tableAssignment, boolean ignoreIfExists) @@ -315,7 +320,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); @@ -714,6 +721,7 @@ public Set getPartitions(TablePath tablePath) { public void createPartition( TablePath tablePath, long tableId, + FsPath remoteDataDir, PartitionAssignment partitionAssignment, ResolvedPartitionSpec partition, boolean ignoreIfExists) { @@ -773,7 +781,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) { @@ -822,7 +835,9 @@ public void dropPartition( private Optional getOptionalTablePartition( TablePath tablePath, String partitionName) { try { - return zookeeperClient.getPartition(tablePath, partitionName); + return zookeeperClient + .getPartition(tablePath, partitionName) + .map(PartitionRegistration::toTablePartition); } catch (Exception e) { throw new FlussRuntimeException( String.format( 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 09c273e7d0..cc515e4d57 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 @@ -139,7 +139,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()); + TablePartition partition = + PartitionZNode.decode(oldData.getData()).toTablePartition(); eventManager.put( new DropPartitionEvent( partition.getTableId(), @@ -213,7 +214,8 @@ private void processCreateTable(TablePath tablePath, ChildData tableData) { private void processCreatePartition( TablePath tablePath, String partitionName, ChildData partitionData) { - TablePartition partition = PartitionZNode.decode(partitionData.getData()); + TablePartition partition = + PartitionZNode.decode(partitionData.getData()).toTablePartition(); 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..0b0b20d6be --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/remote/RemoteDirDynamicLoader.java @@ -0,0 +1,209 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.coordinator.remote; + +import org.apache.fluss.config.ConfigOption; +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.config.cluster.ServerReconfigurable; +import org.apache.fluss.exception.ConfigException; +import org.apache.fluss.fs.FsPath; + +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; + +/** + * 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 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)); + + // Get remote data dirs and weights + List remoteDataDirs = + newConfig + .getOptional(ConfigOptions.REMOTE_DATA_DIRS) + .orElseGet(() -> currentConfiguration.get(ConfigOptions.REMOTE_DATA_DIRS)); + List weights = + newConfig + .getOptional(ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS) + .orElseGet( + () -> + currentConfiguration.get( + ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS)); + + // Validate weighted round-robin strategy configuration + if (strategy == ConfigOptions.RemoteDataDirStrategy.WEIGHTED_ROUND_ROBIN) { + 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 positive, 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 RemoteDirSelector remoteDirSelector; + + public RemoteDirContainer(Configuration conf) { + this.remoteDirSelector = + createRemoteDirSelector( + conf.get(ConfigOptions.REMOTE_DATA_DIRS_STRATEGY), + new FsPath(conf.get(ConfigOptions.REMOTE_DATA_DIR)), + conf.get(ConfigOptions.REMOTE_DATA_DIRS).stream() + .map(FsPath::new) + .collect(Collectors.toList()), + conf.get(ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS)); + } + + private RemoteDirSelector createRemoteDirSelector( + ConfigOptions.RemoteDataDirStrategy strategy, + FsPath 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 FsPath nextDataDir() { + return remoteDirSelector.nextDataDir(); + } + } +} 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..3c6b5853e3 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/remote/RemoteDirSelector.java @@ -0,0 +1,51 @@ +/* + * 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.fs.FsPath; + +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}. + */ + FsPath 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..f98b57ee0c --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/remote/RoundRobinRemoteDirSelector.java @@ -0,0 +1,63 @@ +/* + * 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.fs.FsPath; + +import java.util.List; + +/** + * 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 FsPath defaultRemoteDataDir; + private final List remoteDataDirs; + + // Current position in the round-robin cycle. + private int position; + + // Lock object for thread safety + private final Object lock = new Object(); + + public RoundRobinRemoteDirSelector(FsPath defaultRemoteDataDir, List remoteDataDirs) { + this.defaultRemoteDataDir = defaultRemoteDataDir; + this.remoteDataDirs = remoteDataDirs; + this.position = 0; + } + + @Override + public FsPath nextDataDir() { + if (remoteDataDirs.isEmpty()) { + return defaultRemoteDataDir; + } + + synchronized (lock) { + int selectedIndex = position++; + if (position == remoteDataDirs.size()) { + position = 0; + } + return remoteDataDirs.get(selectedIndex); + } + } +} 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..ae48bd3f6f --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/remote/WeightedRoundRobinRemoteDirSelector.java @@ -0,0 +1,101 @@ +/* + * 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.fs.FsPath; + +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 FsPath 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( + FsPath defaultRemoteDataDir, List remoteDataDirs, List weights) { + this.defaultRemoteDataDir = defaultRemoteDataDir; + this.remoteDataDirs = 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 FsPath nextDataDir() { + if (remoteDataDirs.isEmpty()) { + 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/kv/KvManager.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvManager.java index 0986c67f95..b681a082df 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/KvManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvManager.java @@ -25,7 +25,6 @@ import org.apache.fluss.config.cluster.ServerReconfigurable; import org.apache.fluss.exception.ConfigException; import org.apache.fluss.exception.KvStorageException; -import org.apache.fluss.fs.FileSystem; import org.apache.fluss.fs.FsPath; import org.apache.fluss.memory.LazyMemorySegmentPool; import org.apache.fluss.memory.MemorySegmentPool; @@ -124,10 +123,6 @@ public static RateLimiter getDefaultRateLimiter() { /** The memory segment pool to allocate memorySegment. */ private final MemorySegmentPool memorySegmentPool; - private final FsPath remoteKvDir; - - private final FileSystem remoteFileSystem; - /** * The shared rate limiter for all RocksDB instances to control flush and compaction write rate. */ @@ -144,15 +139,12 @@ private KvManager( ZooKeeperClient zkClient, int recoveryThreadsPerDataDir, LogManager logManager, - TabletServerMetricGroup tabletServerMetricGroup) - throws IOException { + TabletServerMetricGroup tabletServerMetricGroup) { super(TabletType.KV, dataDir, conf, recoveryThreadsPerDataDir); this.logManager = logManager; this.arrowBufferAllocator = new RootAllocator(Long.MAX_VALUE); this.memorySegmentPool = LazyMemorySegmentPool.createServerBufferPool(conf); this.zkClient = zkClient; - this.remoteKvDir = FlussPaths.remoteKvDir(conf); - this.remoteFileSystem = remoteKvDir.getFileSystem(); this.serverMetricGroup = tabletServerMetricGroup; this.sharedRocksDBRateLimiter = createSharedRateLimiter(conf); this.currentSharedRateLimitBytesPerSec = @@ -391,12 +383,13 @@ public KvTablet loadKv(File tabletDir, SchemaGetter schemaGetter) throws Excepti } public void deleteRemoteKvSnapshot( - PhysicalTablePath physicalTablePath, TableBucket tableBucket) { + FsPath remoteDataDir, PhysicalTablePath physicalTablePath, TableBucket tableBucket) { FsPath remoteKvTabletDir = - FlussPaths.remoteKvTabletDir(remoteKvDir, physicalTablePath, tableBucket); + FlussPaths.remoteKvTabletDir( + FlussPaths.remoteKvDir(remoteDataDir), physicalTablePath, tableBucket); try { - if (remoteFileSystem.exists(remoteKvTabletDir)) { - remoteFileSystem.delete(remoteKvTabletDir, true); + if (remoteDataDir.getFileSystem().exists(remoteKvTabletDir)) { + remoteDataDir.getFileSystem().delete(remoteKvTabletDir, true); LOG.info("Delete table's remote bucket snapshot dir of {} success.", tableBucket); } } catch (Exception e) { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/DefaultSnapshotContext.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/DefaultSnapshotContext.java index a6894851bf..8d54005262 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/DefaultSnapshotContext.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/DefaultSnapshotContext.java @@ -19,11 +19,9 @@ import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; -import org.apache.fluss.fs.FsPath; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.server.kv.KvSnapshotResource; import org.apache.fluss.server.zk.ZooKeeperClient; -import org.apache.fluss.utils.FlussPaths; import org.apache.fluss.utils.function.FunctionWithException; import java.util.Optional; @@ -50,8 +48,6 @@ public class DefaultSnapshotContext implements SnapshotContext { private final int maxFetchLogSizeInRecoverKv; - private final FsPath remoteKvDir; - private DefaultSnapshotContext( ZooKeeperClient zooKeeperClient, CompletedKvSnapshotCommitter completedKvSnapshotCommitter, @@ -61,7 +57,6 @@ private DefaultSnapshotContext( KvSnapshotDataDownloader kvSnapshotDataDownloader, long kvSnapshotIntervalMs, int writeBufferSizeInBytes, - FsPath remoteKvDir, CompletedSnapshotHandleStore completedSnapshotHandleStore, int maxFetchLogSizeInRecoverKv) { this.zooKeeperClient = zooKeeperClient; @@ -72,7 +67,6 @@ private DefaultSnapshotContext( this.kvSnapshotDataDownloader = kvSnapshotDataDownloader; this.kvSnapshotIntervalMs = kvSnapshotIntervalMs; this.writeBufferSizeInBytes = writeBufferSizeInBytes; - this.remoteKvDir = remoteKvDir; this.completedSnapshotHandleStore = completedSnapshotHandleStore; this.maxFetchLogSizeInRecoverKv = maxFetchLogSizeInRecoverKv; @@ -92,19 +86,21 @@ public static DefaultSnapshotContext create( kvSnapshotResource.getKvSnapshotDataDownloader(), conf.get(ConfigOptions.KV_SNAPSHOT_INTERVAL).toMillis(), (int) conf.get(ConfigOptions.REMOTE_FS_WRITE_BUFFER_SIZE).getBytes(), - FlussPaths.remoteKvDir(conf), new ZooKeeperCompletedSnapshotHandleStore(zkClient), (int) conf.get(ConfigOptions.KV_RECOVER_LOG_RECORD_BATCH_MAX_SIZE).getBytes()); } + @Override public ZooKeeperClient getZooKeeperClient() { return zooKeeperClient; } + @Override public ExecutorService getAsyncOperationsThreadPool() { return asyncOperationsThreadPool; } + @Override public KvSnapshotDataUploader getSnapshotDataUploader() { return kvSnapshotDataUploader; } @@ -114,6 +110,7 @@ public KvSnapshotDataDownloader getSnapshotDataDownloader() { return kvSnapshotDataDownloader; } + @Override public ScheduledExecutorService getSnapshotScheduler() { return snapshotScheduler; } @@ -133,10 +130,6 @@ public int getSnapshotFsWriteBufferSize() { return writeBufferSizeInBytes; } - public FsPath getRemoteKvDir() { - return remoteKvDir; - } - @Override public FunctionWithException getLatestCompletedSnapshotProvider() { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/SnapshotContext.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/SnapshotContext.java index 8f528bc473..efc7d7889c 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/SnapshotContext.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/snapshot/SnapshotContext.java @@ -17,7 +17,6 @@ package org.apache.fluss.server.kv.snapshot; -import org.apache.fluss.fs.FsPath; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.utils.function.FunctionWithException; @@ -54,9 +53,6 @@ public interface SnapshotContext { /** Get the size of the write buffer for writing the kv snapshot file to remote filesystem. */ int getSnapshotFsWriteBufferSize(); - /** Get the remote root path to store kv snapshot files. */ - FsPath getRemoteKvDir(); - /** * Get the provider of latest CompletedSnapshot for a table bucket. When no completed snapshot * exists, the CompletedSnapshot provided will be null. diff --git a/fluss-server/src/main/java/org/apache/fluss/server/log/remote/DefaultRemoteLogStorage.java b/fluss-server/src/main/java/org/apache/fluss/server/log/remote/DefaultRemoteLogStorage.java index 2dedc01da0..aaeac2d721 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/log/remote/DefaultRemoteLogStorage.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/log/remote/DefaultRemoteLogStorage.java @@ -66,20 +66,17 @@ public class DefaultRemoteLogStorage implements RemoteLogStorage { private static final int READ_BUFFER_SIZE = 16 * 1024; private final FsPath remoteLogDir; - private final FileSystem fileSystem; private final ExecutorService ioExecutor; private final int writeBufferSize; - public DefaultRemoteLogStorage(Configuration conf, ExecutorService ioExecutor) - throws IOException { + public DefaultRemoteLogStorage(Configuration conf, ExecutorService ioExecutor) { this.remoteLogDir = FlussPaths.remoteLogDir(conf); - this.fileSystem = remoteLogDir.getFileSystem(); this.writeBufferSize = (int) conf.get(ConfigOptions.REMOTE_FS_WRITE_BUFFER_SIZE).getBytes(); this.ioExecutor = ioExecutor; } @Override - public FsPath getRemoteLogDir() { + public FsPath getDefaultRemoteLogDir() { return remoteLogDir; } @@ -141,6 +138,7 @@ public void deleteLogSegmentFiles(RemoteLogSegment remoteLogSegment) throws RemoteStorageException { LOG.debug("Deleting log segment and indexes for : {}", remoteLogSegment); try { + FsPath remoteLogDir = remoteLogSegment.remoteLogDir(); FsPath segmentDir = remoteLogSegmentDir(remoteLogDir, remoteLogSegment); long baseOffset = remoteLogSegment.remoteLogStartOffset(); FsPath logFile = remoteLogSegmentFile(segmentDir, baseOffset); @@ -154,7 +152,7 @@ public void deleteLogSegmentFiles(RemoteLogSegment remoteLogSegment) // delete dir at last for (FsPath path : Arrays.asList(logFile, offsetIndex, timeIndex, writerSnapshot, segmentDir)) { - fileSystem.delete(path, false); + remoteLogDir.getFileSystem().delete(path, false); } LOG.debug("Successful delete log segment and indexes for : {}", remoteLogSegment); } catch (IOException e) { @@ -166,6 +164,8 @@ public void deleteLogSegmentFiles(RemoteLogSegment remoteLogSegment) @Override public InputStream fetchIndex(RemoteLogSegment remoteLogSegment, IndexType indexType) throws RemoteStorageException { + FsPath remoteLogDir = remoteLogSegment.remoteLogDir(); + FsPath remoteLogSegmentIndexFile; if (indexType == IndexType.WRITER_ID_SNAPSHOT) { remoteLogSegmentIndexFile = @@ -178,7 +178,7 @@ public InputStream fetchIndex(RemoteLogSegment remoteLogSegment, IndexType index } try { - return fileSystem.open(remoteLogSegmentIndexFile); + return remoteLogDir.getFileSystem().open(remoteLogSegmentIndexFile); } catch (IOException e) { throw new RemoteStorageException( "Failed to fetch index file type: " @@ -195,7 +195,7 @@ public RemoteLogManifest readRemoteLogManifestSnapshot(FsPath remoteLogManifestP FSDataInputStream inputStream = null; ByteArrayOutputStream outputStream = null; try { - inputStream = fileSystem.open(remoteLogManifestPath); + inputStream = remoteLogManifestPath.getFileSystem().open(remoteLogManifestPath); outputStream = new ByteArrayOutputStream(); IOUtils.copyBytes(inputStream, outputStream, false); return RemoteLogManifest.fromJsonBytes(outputStream.toByteArray()); @@ -215,7 +215,7 @@ public void deleteRemoteLogManifestSnapshot(FsPath remoteLogManifestPath) throws RemoteStorageException { LOG.debug("Deleting remote log segment manifest: {}", remoteLogManifestPath); try { - fileSystem.delete(remoteLogManifestPath, false); + remoteLogManifestPath.getFileSystem().delete(remoteLogManifestPath, false); LOG.debug("Successful delete log segment manifest: {}", remoteLogManifestPath); } catch (IOException e) { throw new RemoteStorageException( @@ -227,6 +227,7 @@ public void deleteRemoteLogManifestSnapshot(FsPath remoteLogManifestPath) @Override public FsPath writeRemoteLogManifestSnapshot(RemoteLogManifest manifest) throws RemoteStorageException { + FsPath remoteLogDir = manifest.getRemoteLogDir(); FsPath manifestFile = FlussPaths.remoteLogManifestFile( FlussPaths.remoteLogTabletDir( @@ -249,11 +250,13 @@ public FsPath writeRemoteLogManifestSnapshot(RemoteLogManifest manifest) } @Override - public void deleteTableBucket(PhysicalTablePath physicalTablePath, TableBucket tableBucket) + public void deleteTableBucket( + FsPath remoteLogDir, PhysicalTablePath physicalTablePath, TableBucket tableBucket) throws RemoteStorageException { FsPath remoteLogTabletDir = FlussPaths.remoteLogTabletDir(remoteLogDir, physicalTablePath, tableBucket); try { + FileSystem fileSystem = remoteLogDir.getFileSystem(); if (fileSystem.exists(remoteLogTabletDir)) { fileSystem.delete(remoteLogTabletDir, true); } @@ -322,8 +325,9 @@ private List> createUploadFutures( } private FsPath createRemoteLogSegmentDir(RemoteLogSegment remoteLogSegment) throws IOException { + FsPath remoteLogDir = remoteLogSegment.remoteLogDir(); FsPath remoteLogSegmentDir = remoteLogSegmentDir(remoteLogDir, remoteLogSegment); - fileSystem.mkdirs(remoteLogSegmentDir); + remoteLogDir.getFileSystem().mkdirs(remoteLogSegmentDir); return remoteLogSegmentDir; } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/log/remote/LogTieringTask.java b/fluss-server/src/main/java/org/apache/fluss/server/log/remote/LogTieringTask.java index cdde3842dc..28537ae6c1 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/log/remote/LogTieringTask.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/log/remote/LogTieringTask.java @@ -270,6 +270,7 @@ private long copyLogSegmentFilesToRemote( .remoteLogEndOffset(endOffset) .maxTimestamp(segment.maxTimestampSoFar()) .segmentSizeInBytes(sizeInBytes) + .remoteLogDir(remoteLog.getRemoteLogDir()) .build(); try { remoteLogStorage.copyLogSegmentFiles(copyRemoteLogSegment, logSegmentFiles); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/log/remote/RemoteLogManager.java b/fluss-server/src/main/java/org/apache/fluss/server/log/remote/RemoteLogManager.java index 9f30067473..069d4c293e 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/log/remote/RemoteLogManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/log/remote/RemoteLogManager.java @@ -30,6 +30,7 @@ import org.apache.fluss.server.replica.Replica; import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.server.zk.data.RemoteLogManifestHandle; +import org.apache.fluss.utils.FlussPaths; import org.apache.fluss.utils.IOUtils; import org.apache.fluss.utils.MapUtils; import org.apache.fluss.utils.clock.Clock; @@ -125,10 +126,6 @@ public RemoteLogStorage getRemoteLogStorage() { return remoteLogStorage; } - public FsPath remoteLogDir() { - return remoteLogStorage.getRemoteLogDir(); - } - /** Restore the remote log manifest and start the log tiering task for the given replica. */ public void startLogTiering(Replica replica) throws Exception { if (remoteDisabled()) { @@ -137,8 +134,10 @@ public void startLogTiering(Replica replica) throws Exception { TableBucket tableBucket = replica.getTableBucket(); PhysicalTablePath physicalTablePath = replica.getPhysicalTablePath(); LogTablet log = replica.getLogTablet(); + FsPath remoteLogDir = FlussPaths.remoteLogDir(replica.getRemoteDataDir()); RemoteLogTablet remoteLog = - new RemoteLogTablet(physicalTablePath, tableBucket, replica.getLogTTLMs()); + new RemoteLogTablet( + physicalTablePath, tableBucket, replica.getLogTTLMs(), remoteLogDir); Optional remoteLogManifestHandleOpt = zkClient.getRemoteLogManifestHandle(tableBucket); if (remoteLogManifestHandleOpt.isPresent()) { @@ -147,6 +146,15 @@ public void startLogTiering(Replica replica) throws Exception { RemoteLogManifest manifest = remoteLogStorage.readRemoteLogManifestSnapshot( remoteLogManifestHandleOpt.get().getRemoteLogManifestPath()); + + // If the RemoteLogManifest does not include remoteLogDir, it means the manifest was + // generated by an old version that does not support remote.data.dirs. + // We set remoteLogDir manually here, so subsequent usage will be safe to directly use + // it. + if (manifest.getRemoteLogDir() == null) { + manifest = manifest.newManifest(remoteLogDir); + } + remoteLog.loadRemoteLogManifest(manifest); } remoteLog.getRemoteLogEndOffset().ifPresent(log::updateRemoteLogEndOffset); @@ -208,8 +216,9 @@ public void stopReplica(Replica replica, boolean deleteRemote) { if (deleteRemote) { LOG.info("Deleting the remote log segments for table-bucket: {}", tb); + FsPath remoteLogDir = FlussPaths.remoteLogDir(replica.getRemoteDataDir()); // delete the remote log of the table bucket. - deleteRemoteLog(physicalTablePath, tb); + deleteRemoteLog(remoteLogDir, physicalTablePath, tb); } } @@ -260,11 +269,12 @@ private boolean remoteDisabled() { *

Note: the zk path for {@link RemoteLogManifestHandle} will be deleted by coordinator while * table delete. */ - private void deleteRemoteLog(PhysicalTablePath physicalTablePath, TableBucket tableBucket) { + private void deleteRemoteLog( + FsPath remoteLogDir, PhysicalTablePath physicalTablePath, TableBucket tableBucket) { // delete the file in remote storage. try { // TODO: maybe need to optimize to delete on specific file path - remoteLogStorage.deleteTableBucket(physicalTablePath, tableBucket); + remoteLogStorage.deleteTableBucket(remoteLogDir, physicalTablePath, tableBucket); } catch (RemoteStorageException e) { LOG.error( "Error occurred while deleting remote log for table-bucket: {}", diff --git a/fluss-server/src/main/java/org/apache/fluss/server/log/remote/RemoteLogManifest.java b/fluss-server/src/main/java/org/apache/fluss/server/log/remote/RemoteLogManifest.java index e1478dec4c..d437083e69 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/log/remote/RemoteLogManifest.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/log/remote/RemoteLogManifest.java @@ -18,6 +18,7 @@ package org.apache.fluss.server.log.remote; import org.apache.fluss.annotation.VisibleForTesting; +import org.apache.fluss.fs.FsPath; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.remote.RemoteLogSegment; @@ -39,13 +40,17 @@ public class RemoteLogManifest { private final TableBucket tableBucket; private final List remoteLogSegmentList; + private final FsPath remoteLogDir; + public RemoteLogManifest( PhysicalTablePath physicalTablePath, TableBucket tableBucket, - List remoteLogSegmentList) { + List remoteLogSegmentList, + FsPath remoteLogDir) { this.physicalTablePath = physicalTablePath; this.tableBucket = tableBucket; this.remoteLogSegmentList = Collections.unmodifiableList(remoteLogSegmentList); + this.remoteLogDir = remoteLogDir; // sanity check for (RemoteLogSegment remoteLogSegment : remoteLogSegmentList) { @@ -73,7 +78,7 @@ public RemoteLogManifest trimAndMerge( } } newSegments.addAll(addedSegments); - return new RemoteLogManifest(physicalTablePath, tableBucket, newSegments); + return new RemoteLogManifest(physicalTablePath, tableBucket, newSegments, remoteLogDir); } public long getRemoteLogStartOffset() { @@ -120,11 +125,36 @@ public TableBucket getTableBucket() { return tableBucket; } + public FsPath getRemoteLogDir() { + return remoteLogDir; + } + @VisibleForTesting public List getRemoteLogSegmentList() { return remoteLogSegmentList; } + public RemoteLogManifest newManifest(FsPath remoteLogDir) { + List newRemoteLogSegments = new ArrayList<>(remoteLogSegmentList.size()); + for (RemoteLogSegment remoteLogSegment : remoteLogSegmentList) { + newRemoteLogSegments.add( + RemoteLogSegment.Builder.builder() + .physicalTablePath(remoteLogSegment.physicalTablePath()) + .tableBucket(remoteLogSegment.tableBucket()) + .remoteLogSegmentId(remoteLogSegment.remoteLogSegmentId()) + .remoteLogStartOffset(remoteLogSegment.remoteLogStartOffset()) + .remoteLogEndOffset(remoteLogSegment.remoteLogEndOffset()) + .maxTimestamp(remoteLogSegment.maxTimestamp()) + .segmentSizeInBytes(remoteLogSegment.segmentSizeInBytes()) + // We set remoteLogDir manually here, so subsequent usage will be safe + // to directly use it. + .remoteLogDir(remoteLogDir) + .build()); + } + return new RemoteLogManifest( + physicalTablePath, tableBucket, newRemoteLogSegments, remoteLogDir); + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/log/remote/RemoteLogManifestJsonSerde.java b/fluss-server/src/main/java/org/apache/fluss/server/log/remote/RemoteLogManifestJsonSerde.java index 27c5488490..81130b6107 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/log/remote/RemoteLogManifestJsonSerde.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/log/remote/RemoteLogManifestJsonSerde.java @@ -17,6 +17,7 @@ package org.apache.fluss.server.log.remote; +import org.apache.fluss.fs.FsPath; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.remote.RemoteLogSegment; @@ -50,6 +51,7 @@ public class RemoteLogManifestJsonSerde private static final String END_OFFSET_FIELD = "end_offset"; private static final String MAX_TIMESTAMP_FIELD = "max_timestamp"; private static final String SEGMENT_SIZE_IN_BYTES_FIELD = "size_in_bytes"; + private static final String REMOTE_LOG_DIR_FIELD = "remote_log_dir"; private static final int SNAPSHOT_VERSION = 1; @Override @@ -85,9 +87,14 @@ public void serialize(RemoteLogManifest manifest, JsonGenerator generator) throw generator.writeNumberField(MAX_TIMESTAMP_FIELD, remoteLogSegment.maxTimestamp()); generator.writeNumberField( SEGMENT_SIZE_IN_BYTES_FIELD, remoteLogSegment.segmentSizeInBytes()); + generator.writeStringField( + REMOTE_LOG_DIR_FIELD, remoteLogSegment.remoteLogDir().toString()); generator.writeEndObject(); } generator.writeEndArray(); + + generator.writeStringField(REMOTE_LOG_DIR_FIELD, manifest.getRemoteLogDir().toString()); + generator.writeEndObject(); } @@ -119,6 +126,11 @@ public RemoteLogManifest deserialize(JsonNode node) { long endOffset = entryJson.get(END_OFFSET_FIELD).asLong(); long maxTimestamp = entryJson.get(MAX_TIMESTAMP_FIELD).asLong(); int segmentSizeInBytes = entryJson.get(SEGMENT_SIZE_IN_BYTES_FIELD).asInt(); + // backward compatibility for existing RemoteLogSegment which does not have remoteLogDir + FsPath remoteLogDir = null; + if (entryJson.has(REMOTE_LOG_DIR_FIELD)) { + remoteLogDir = new FsPath(entryJson.get(REMOTE_LOG_DIR_FIELD).asText()); + } snapshotEntries.add( RemoteLogSegment.Builder.builder() .physicalTablePath(physicalTablePath) @@ -128,10 +140,17 @@ public RemoteLogManifest deserialize(JsonNode node) { .remoteLogEndOffset(endOffset) .maxTimestamp(maxTimestamp) .segmentSizeInBytes(segmentSizeInBytes) + .remoteLogDir(remoteLogDir) .build()); } - return new RemoteLogManifest(physicalTablePath, tableBucket, snapshotEntries); + // backward compatibility for existing RemoteLogManifest which does not have remoteLogDir + FsPath remoteLogDir = null; + if (node.has(REMOTE_LOG_DIR_FIELD)) { + remoteLogDir = new FsPath(node.get(REMOTE_LOG_DIR_FIELD).asText()); + } + + return new RemoteLogManifest(physicalTablePath, tableBucket, snapshotEntries, remoteLogDir); } public static RemoteLogManifest fromJson(byte[] json) { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/log/remote/RemoteLogStorage.java b/fluss-server/src/main/java/org/apache/fluss/server/log/remote/RemoteLogStorage.java index 6c47c214b2..3693e7edf9 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/log/remote/RemoteLogStorage.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/log/remote/RemoteLogStorage.java @@ -67,11 +67,12 @@ public static String getFileSuffix(IndexType indexType) { } /** - * Returns the remote log directory. + * Returns the default remote log directory, configured via {@link + * org.apache.fluss.config.ConfigOptions#REMOTE_DATA_DIR}. * * @return the remote log directory. */ - FsPath getRemoteLogDir(); + FsPath getDefaultRemoteLogDir(); /** * Copies the given {@link LogSegmentFiles} provided for the given {@link RemoteLogSegment}. @@ -159,6 +160,7 @@ void deleteRemoteLogManifestSnapshot(FsPath remoteLogManifestPath) * @throws RemoteStorageException if there are any errors while delete remote log data and * metadata. */ - void deleteTableBucket(PhysicalTablePath physicalTablePath, TableBucket tableBucket) + void deleteTableBucket( + FsPath remoteLogDir, PhysicalTablePath physicalTablePath, TableBucket tableBucket) throws RemoteStorageException; } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/log/remote/RemoteLogTablet.java b/fluss-server/src/main/java/org/apache/fluss/server/log/remote/RemoteLogTablet.java index 75892ed55f..daa81a5a3c 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/log/remote/RemoteLogTablet.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/log/remote/RemoteLogTablet.java @@ -18,6 +18,7 @@ package org.apache.fluss.server.log.remote; import org.apache.fluss.annotation.VisibleForTesting; +import org.apache.fluss.fs.FsPath; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metrics.MetricNames; @@ -56,6 +57,8 @@ public class RemoteLogTablet { private final PhysicalTablePath physicalTablePath; + private final FsPath remoteLogDir; + /** * It contains all the segment-id to {@link RemoteLogSegment} mappings which did not delete in * remote storage. @@ -102,12 +105,17 @@ public class RemoteLogTablet { private volatile boolean closed = false; public RemoteLogTablet( - PhysicalTablePath physicalTablePath, TableBucket tableBucket, long ttlMs) { + PhysicalTablePath physicalTablePath, + TableBucket tableBucket, + long ttlMs, + FsPath remoteLogDir) { this.tableBucket = tableBucket; this.physicalTablePath = physicalTablePath; + this.remoteLogDir = remoteLogDir; this.ttlMs = ttlMs; this.currentManifest = - new RemoteLogManifest(physicalTablePath, tableBucket, new ArrayList<>()); + new RemoteLogManifest( + physicalTablePath, tableBucket, new ArrayList<>(), remoteLogDir); reset(); } @@ -237,6 +245,10 @@ public OptionalLong getRemoteLogEndOffset() { : OptionalLong.of(remoteLogEndOffset); } + public FsPath getRemoteLogDir() { + return remoteLogDir; + } + /** * Gets the snapshot of current remote log segment manifest. The snapshot including the exists * remoteLogSegment already committed. @@ -328,7 +340,8 @@ public void addAndDeleteLogSegments( new RemoteLogManifest( physicalTablePath, tableBucket, - new ArrayList<>(idToRemoteLogSegment.values())); + new ArrayList<>(idToRemoteLogSegment.values()), + remoteLogDir); }); } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java index 1669e004df..fee5332934 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java @@ -144,6 +144,7 @@ public final class Replica { private final PhysicalTablePath physicalPath; private final TableBucket tableBucket; + private final FsPath remoteDataDir; private final LogManager logManager; private final LogTablet logTablet; @@ -219,6 +220,7 @@ public Replica( FatalErrorHandler fatalErrorHandler, BucketMetricGroup bucketMetricGroup, TableInfo tableInfo, + FsPath remoteDataDir, Clock clock) throws Exception { this.physicalPath = physicalPath; @@ -248,6 +250,7 @@ public Replica( this.closeableRegistry = new CloseableRegistry(); this.logTablet = createLog(lazyHighWatermarkCheckpoint); + this.remoteDataDir = remoteDataDir; this.clock = clock; registerMetrics(); } @@ -378,6 +381,10 @@ public LogFormat getLogFormat() { return logFormat; } + public FsPath getRemoteDataDir() { + return remoteDataDir; + } + public void makeLeader(NotifyLeaderAndIsrData data) throws IOException { boolean leaderHWIncremented = inWriteLock( @@ -820,9 +827,10 @@ private void startPeriodicKvSnapshot(@Nullable CompletedSnapshot completedSnapsh // instead of a separate class Supplier bucketLeaderEpochSupplier = () -> leaderEpoch; Supplier coordinatorEpochSupplier = () -> coordinatorEpoch; + + FsPath remoteKvDir = FlussPaths.remoteKvDir(remoteDataDir); FsPath remoteKvTabletDir = - FlussPaths.remoteKvTabletDir( - snapshotContext.getRemoteKvDir(), physicalPath, tableBucket); + FlussPaths.remoteKvTabletDir(remoteKvDir, physicalPath, tableBucket); kvTabletSnapshotTarget = new KvTabletSnapshotTarget( diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java index d9350257dc..ad18005c12 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java @@ -32,6 +32,7 @@ import org.apache.fluss.fs.FsPath; import org.apache.fluss.metadata.LogFormat; import org.apache.fluss.metadata.PhysicalTablePath; +import org.apache.fluss.metadata.SchemaInfo; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePath; @@ -98,6 +99,8 @@ import org.apache.fluss.server.replica.fetcher.ReplicaFetcherManager; import org.apache.fluss.server.utils.FatalErrorHandler; 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.server.zk.data.lake.LakeTableSnapshot; import org.apache.fluss.utils.FileUtils; import org.apache.fluss.utils.FlussPaths; @@ -130,7 +133,9 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.apache.fluss.server.TabletManagerBase.getTableInfo; +import static org.apache.fluss.server.TabletManagerBase.getPartitionRegistration; +import static org.apache.fluss.server.TabletManagerBase.getSchemaInfo; +import static org.apache.fluss.server.TabletManagerBase.getTableRegistration; import static org.apache.fluss.utils.FileUtils.isDirectoryEmpty; import static org.apache.fluss.utils.Preconditions.checkState; import static org.apache.fluss.utils.concurrent.LockUtils.inLock; @@ -1205,11 +1210,10 @@ private boolean canFetchFromRemoteLog(Replica replica, long fetchOffset) { remoteLogManager.lookupPositionForOffset( remoteLogSegmentList.get(0), fetchOffset); PhysicalTablePath physicalTablePath = replica.getPhysicalTablePath(); + FsPath remoteLogDir = FlussPaths.remoteLogDir(replica.getRemoteDataDir()); FsPath remoteLogTabletDir = FlussPaths.remoteLogTabletDir( - remoteLogManager.remoteLogDir(), - physicalTablePath, - replica.getTableBucket()); + remoteLogDir, physicalTablePath, replica.getTableBucket()); return new RemoteLogFetchInfo( remoteLogTabletDir.toString(), physicalTablePath.getPartitionName(), @@ -1524,7 +1528,9 @@ private StopReplicaResultForBucket stopReplica( remoteLogManager.stopReplica(replicaToDelete, delete && replicaToDelete.isLeader()); if (delete && replicaToDelete.isLeader()) { kvManager.deleteRemoteKvSnapshot( - replicaToDelete.getPhysicalTablePath(), replicaToDelete.getTableBucket()); + replicaToDelete.getRemoteDataDir(), + replicaToDelete.getPhysicalTablePath(), + replicaToDelete.getTableBucket()); } } @@ -1575,6 +1581,23 @@ private void dropEmptyTableOrPartitionDir(Path dir, long id, String dirType) { } } + private FsPath getRemoteDataDir( + TableBucket tb, + PhysicalTablePath physicalTablePath, + TableRegistration tableRegistration) + throws Exception { + if (tb.getPartitionId() != null) { + PartitionRegistration partitionRegistration = + getPartitionRegistration(zkClient, physicalTablePath); + if (partitionRegistration.getRemoteDataDir() != null) { + return partitionRegistration.getRemoteDataDir(); + } + } else if (tableRegistration.remoteDataDir != null) { + return tableRegistration.remoteDataDir; + } + return new FsPath(conf.get(ConfigOptions.REMOTE_DATA_DIR)); + } + protected Optional maybeCreateReplica(NotifyLeaderAndIsrData data) { Optional replicaOpt = Optional.empty(); try { @@ -1583,7 +1606,12 @@ protected Optional maybeCreateReplica(NotifyLeaderAndIsrData data) { if (hostedReplica instanceof NoneReplica) { PhysicalTablePath physicalTablePath = data.getPhysicalTablePath(); TablePath tablePath = physicalTablePath.getTablePath(); - TableInfo tableInfo = getTableInfo(zkClient, tablePath); + + TableRegistration tableRegistration = getTableRegistration(zkClient, tablePath); + SchemaInfo schemaInfo = getSchemaInfo(zkClient, tablePath); + TableInfo tableInfo = tableRegistration.toTableInfo(tablePath, schemaInfo); + + FsPath remoteDataDir = getRemoteDataDir(tb, physicalTablePath, tableRegistration); boolean isKvTable = tableInfo.hasPrimaryKey(); BucketMetricGroup bucketMetricGroup = @@ -1608,6 +1636,7 @@ protected Optional maybeCreateReplica(NotifyLeaderAndIsrData data) { fatalErrorHandler, bucketMetricGroup, tableInfo, + remoteDataDir, clock); allReplicas.put(tb, new OnlineReplica(replica)); replicaOpt = Optional.of(replica); 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/zk/ZooKeeperClient.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java index 40bffe8171..9ebebb5067 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 @@ -20,6 +20,7 @@ import org.apache.fluss.annotation.Internal; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; +import org.apache.fluss.fs.FsPath; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.ResolvedPartitionSpec; import org.apache.fluss.metadata.Schema; @@ -42,6 +43,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.RebalancePlan; import org.apache.fluss.server.zk.data.RemoteLogManifestHandle; import org.apache.fluss.server.zk.data.ResourceAcl; @@ -635,7 +637,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())); } @@ -694,7 +696,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())); } @@ -740,7 +743,7 @@ 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); @@ -763,7 +766,7 @@ public Map getPartitionIds( return processGetDataResponses( responses, response -> path2PartitionPathMap.get(response.getPath()), - PartitionZNode::decode, + (byte[] data) -> PartitionZNode.decode(data).toTablePartition(), "partition"); } @@ -788,6 +791,7 @@ public void registerPartitionAssignmentAndMetadata( long partitionId, String partitionName, PartitionAssignment partitionAssignment, + FsPath remoteDataDir, TablePath tablePath, long tableId) throws Exception { @@ -833,12 +837,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..5d21c9cc26 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/PartitionRegistration.java @@ -0,0 +1,91 @@ +/* + * 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.fs.FsPath; +import org.apache.fluss.metadata.TablePartition; + +import javax.annotation.Nullable; + +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; + + private final @Nullable FsPath remoteDataDir; + + public PartitionRegistration(long tableId, long partitionId, @Nullable FsPath remoteDataDir) { + this.tableId = tableId; + this.partitionId = partitionId; + this.remoteDataDir = remoteDataDir; + } + + public long getTableId() { + return tableId; + } + + public long getPartitionId() { + return partitionId; + } + + @Nullable + public FsPath getRemoteDataDir() { + return remoteDataDir; + } + + public TablePartition toTablePartition() { + return new TablePartition(tableId, partitionId); + } + + @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..20b16e1494 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/PartitionRegistrationJsonSerde.java @@ -0,0 +1,69 @@ +/* + * 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.fs.FsPath; +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 javax.annotation.Nullable; + +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()); + if (registration.getRemoteDataDir() != null) { + generator.writeStringField( + REMOTE_DATA_DIR_KEY, registration.getRemoteDataDir().toString()); + } + generator.writeEndObject(); + } + + @Override + public PartitionRegistration deserialize(JsonNode node) { + long tableId = node.get(TABLE_ID_KEY).asLong(); + long partitionId = node.get(PARTITION_ID_KEY).asLong(); + @Nullable FsPath remoteDataDir = null; + if (node.has(REMOTE_DATA_DIR_KEY)) { + remoteDataDir = new FsPath(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..1789eb53c8 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 @@ -20,6 +20,7 @@ import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; import org.apache.fluss.config.TableConfig; +import org.apache.fluss.fs.FsPath; import org.apache.fluss.metadata.Schema; import org.apache.fluss.metadata.SchemaInfo; import org.apache.fluss.metadata.TableDescriptor; @@ -52,6 +53,7 @@ public class TableRegistration { public final int bucketCount; public final Map properties; public final Map customProperties; + public final @Nullable FsPath remoteDataDir; public final long createdTime; public final long modifiedTime; @@ -62,6 +64,7 @@ public TableRegistration( TableDistribution tableDistribution, Map properties, Map customProperties, + @Nullable FsPath remoteDataDir, long createdTime, long modifiedTime) { checkArgument( @@ -74,6 +77,7 @@ public TableRegistration( this.bucketKeys = tableDistribution.getBucketKeys(); this.properties = properties; this.customProperties = customProperties; + this.remoteDataDir = remoteDataDir; this.createdTime = createdTime; this.modifiedTime = modifiedTime; } @@ -116,7 +120,8 @@ public TableInfo toTableInfo( this.modifiedTime); } - public static TableRegistration newTable(long tableId, TableDescriptor tableDescriptor) { + public static TableRegistration newTable( + long tableId, @Nullable FsPath remoteDataDir, TableDescriptor tableDescriptor) { checkArgument( tableDescriptor.getTableDistribution().isPresent(), "Table distribution is required for table registration."); @@ -128,6 +133,7 @@ public static TableRegistration newTable(long tableId, TableDescriptor tableDesc tableDescriptor.getTableDistribution().get(), tableDescriptor.getProperties(), tableDescriptor.getCustomProperties(), + remoteDataDir, currentMillis, currentMillis); } @@ -142,6 +148,7 @@ public TableRegistration newProperties( new TableDistribution(bucketCount, bucketKeys), newProperties, newCustomProperties, + remoteDataDir, createdTime, currentMillis); } 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..fc400f9845 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 @@ -18,6 +18,7 @@ package org.apache.fluss.server.zk.data; import org.apache.fluss.annotation.Internal; +import org.apache.fluss.fs.FsPath; import org.apache.fluss.metadata.TableDescriptor.TableDistribution; import org.apache.fluss.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; import org.apache.fluss.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; @@ -45,6 +46,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 +102,11 @@ public void serialize(TableRegistration tableReg, JsonGenerator generator) throw } generator.writeEndObject(); + // serialize remote data dir + if (tableReg.remoteDataDir != null) { + generator.writeStringField(REMOTE_DATA_DIR, tableReg.remoteDataDir.toString()); + } + // serialize createdTime generator.writeNumberField(CREATED_TIME, tableReg.createdTime); @@ -141,6 +148,11 @@ public TableRegistration deserialize(JsonNode node) { Map customProperties = deserializeProperties(node.get(CUSTOM_PROPERTIES_NAME)); + FsPath remoteDataDir = null; + if (node.has(REMOTE_DATA_DIR)) { + remoteDataDir = new FsPath(node.get(REMOTE_DATA_DIR).asText()); + } + long createdTime = node.get(CREATED_TIME).asLong(); long modifiedTime = node.get(MODIFIED_TIME).asLong(); @@ -151,6 +163,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 4798623a74..16974ebeaf 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; @@ -232,12 +231,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/main/resources/META-INF/services/org.apache.fluss.server.coordinator.remote.RemoteLogPathProviderPlugin b/fluss-server/src/main/resources/META-INF/services/org.apache.fluss.server.coordinator.remote.RemoteLogPathProviderPlugin new file mode 100644 index 0000000000..796ebef697 --- /dev/null +++ b/fluss-server/src/main/resources/META-INF/services/org.apache.fluss.server.coordinator.remote.RemoteLogPathProviderPlugin @@ -0,0 +1,20 @@ +# +# 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. +# + +org.apache.fluss.server.coordinator.remote.DefaultRemoteLogPathProviderPlugin + 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..81ee67860f 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 @@ -21,16 +21,19 @@ import org.apache.fluss.config.AutoPartitionTimeUnit; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; +import org.apache.fluss.fs.FsPath; import org.apache.fluss.metadata.Schema; 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 +52,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; @@ -68,6 +77,10 @@ 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() { zookeeperClient = @@ -79,6 +92,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, "/dir"); + 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 +321,7 @@ void testAddPartitionedTable(TestParams params) throws Exception { zookeeperClient, new Configuration(), new LakeCatalogDynamicLoader(new Configuration(), null, true)), + remoteDirDynamicLoader, new Configuration(), clock, periodicExecutor); @@ -312,6 +337,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 +359,7 @@ void testAddPartitionedTable(TestParams params) throws Exception { metadataManager.createPartition( tablePath, tableId, + new FsPath(remoteDataDir), partitionAssignment, fromPartitionName(table.getPartitionKeys(), partitionName), false); @@ -353,11 +380,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 +416,7 @@ void testMaxPartitions() throws Exception { new AutoPartitionManager( new TestingServerMetadataCache(3), metadataManager, + remoteDirDynamicLoader, new Configuration(), clock, periodicExecutor); @@ -424,6 +454,7 @@ void testMaxPartitions() throws Exception { metadataManager.createPartition( tablePath, tableId, + new FsPath(remoteDataDir), partitionAssignment, fromPartitionName(table.getPartitionKeys(), i + ""), false); @@ -463,6 +494,7 @@ void testAutoCreateDayPartitionShouldJitter() throws Exception { new AutoPartitionManager( new TestingServerMetadataCache(3), metadataManager, + remoteDirDynamicLoader, new Configuration(), clock, periodicExecutor); @@ -527,6 +559,7 @@ void testMaxBucketNum() throws Exception { new AutoPartitionManager( new TestingServerMetadataCache(3), metadataManager, + remoteDirDynamicLoader, config, clock, periodicExecutor); @@ -698,6 +731,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); + FsPath remoteDataDir = partition.get().getRemoteDataDir(); + assertThat(remoteDataDir).isNotNull(); + assertThat(allRemoteDataDirs).contains(remoteDataDir.toString()); + } + } + private TableInfo createPartitionedTable( int partitionRetentionNum, int partitionPreCreateNum, AutoPartitionTimeUnit timeUnit) throws Exception { @@ -753,7 +799,7 @@ private TableInfo createPartitionedTable( long currentMillis = System.currentTimeMillis(); TableInfo tableInfo = TableInfo.of(tablePath, tableId, 1, descriptor, currentMillis, currentMillis); - TableRegistration registration = TableRegistration.newTable(tableId, descriptor); + TableRegistration registration = TableRegistration.newTable(tableId, null, descriptor); zookeeperClient.registerTable(tablePath, registration); return tableInfo; } @@ -795,7 +841,7 @@ private TableInfo createPartitionedTableWithBuckets( long currentMillis = System.currentTimeMillis(); TableInfo tableInfo = TableInfo.of(tablePath, tableId, 1, descriptor, currentMillis, currentMillis); - TableRegistration registration = TableRegistration.newTable(tableId, descriptor); + TableRegistration registration = TableRegistration.newTable(tableId, null, descriptor); zookeeperClient.registerTable(tablePath, registration); return tableInfo; } 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 dcfa6b5b3a..8885a89116 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 @@ -46,6 +46,7 @@ import org.apache.fluss.server.coordinator.event.CommitKvSnapshotEvent; import org.apache.fluss.server.coordinator.event.CommitRemoteLogManifestEvent; import org.apache.fluss.server.coordinator.event.CoordinatorEventManager; +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; @@ -185,11 +186,15 @@ void beforeEach() throws IOException { serverMetadataCache = new CoordinatorMetadataCache(); // set a test channel manager for the context testCoordinatorChannelManager = new TestCoordinatorChannelManager(); - autoPartitionManager = - new AutoPartitionManager(serverMetadataCache, metadataManager, new Configuration()); lakeTableTieringManager = new LakeTableTieringManager(); Configuration conf = new Configuration(); conf.setString(ConfigOptions.REMOTE_DATA_DIR, "/tmp/fluss/remote-data"); + autoPartitionManager = + new AutoPartitionManager( + serverMetadataCache, + metadataManager, + new RemoteDirDynamicLoader(conf), + new Configuration()); eventProcessor = buildCoordinatorEventProcessor(); eventProcessor.startup(); metadataManager.createDatabase( @@ -224,10 +229,10 @@ 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, null, tableDescriptor, tableAssignment, false); TablePath t2 = TablePath.of(defaultDatabase, "create_drop_t2"); - long t2Id = metadataManager.createTable(t2, tableDescriptor, tableAssignment, false); + long t2Id = metadataManager.createTable(t2, null, tableDescriptor, tableAssignment, false); verifyTableCreated(t2Id, tableAssignment, nBuckets, replicationFactor); @@ -357,13 +362,13 @@ void testServerBecomeOnlineAndOfflineLine() throws Exception { TablePath table1Path = TablePath.of(defaultDatabase, "t1"); long table1Id = - metadataManager.createTable(table1Path, TEST_TABLE, table1Assignment, false); + metadataManager.createTable(table1Path, null, 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, null, TEST_TABLE, table2Assignment, false); // retry until the table2 been created retryVerifyContext( @@ -465,7 +470,8 @@ 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, null, TEST_TABLE, tableAssignment, false); // let's restart initCoordinatorChannel(); @@ -589,7 +595,8 @@ void testCreateAndDropPartition() throws Exception { // create a partitioned table TableDescriptor tablePartitionTableDescriptor = getPartitionedTable(); long tableId = - metadataManager.createTable(tablePath, tablePartitionTableDescriptor, null, false); + metadataManager.createTable( + tablePath, null, tablePartitionTableDescriptor, null, false); int nBuckets = 3; int replicationFactor = 3; @@ -661,7 +668,8 @@ void testRestartResumeDropPartition() throws Exception { // create a partitioned table TableDescriptor tablePartitionTableDescriptor = getPartitionedTable(); long tableId = - metadataManager.createTable(tablePath, tablePartitionTableDescriptor, null, false); + metadataManager.createTable( + tablePath, null, tablePartitionTableDescriptor, null, false); int nBuckets = 3; int replicationFactor = 3; @@ -807,7 +815,7 @@ 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, null, TEST_TABLE, tableAssignment, false); verifyTableCreated(t1Id, tableAssignment, nBuckets, replicationFactor); // get the origin bucket leaderAndIsr @@ -861,7 +869,7 @@ void testSchemaChange() throws Exception { }); // create table List replicas = tableAssignment.getBucketAssignment(0).getReplicas(); - metadataManager.createTable(t1, TEST_TABLE, tableAssignment, false); + metadataManager.createTable(t1, null, TEST_TABLE, tableAssignment, false); TableInfo tableInfo = metadataManager.getTable(t1); retry( @@ -953,9 +961,9 @@ private Tuple2 preparePartitionAssignment( String partition1Name = "2024"; String partition2Name = "2025"; zookeeperClient.registerPartitionAssignmentAndMetadata( - partition1Id, partition1Name, partitionAssignment, tablePath, tableId); + partition1Id, partition1Name, partitionAssignment, null, tablePath, tableId); zookeeperClient.registerPartitionAssignmentAndMetadata( - partition2Id, partition2Name, partitionAssignment, tablePath, tableId); + partition2Id, partition2Name, partitionAssignment, null, tablePath, tableId); return Tuple2.of( new PartitionIdName(partition1Id, partition1Name), @@ -1226,7 +1234,7 @@ 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, null, 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/TableManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TableManagerTest.java index 3c417efe8d..3f66dfe721 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 @@ -280,7 +280,7 @@ void testCreateAndDropPartition() throws Exception { String partitionName = "2024"; long partitionId = zookeeperClient.getPartitionIdAndIncrement(); zookeeperClient.registerPartitionAssignmentAndMetadata( - partitionId, partitionName, partitionAssignment, DATA1_TABLE_PATH, tableId); + partitionId, partitionName, partitionAssignment, null, 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 bb9a25d1d2..609e9e59d3 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 @@ -20,6 +20,7 @@ import org.apache.fluss.cluster.TabletServerInfo; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; +import org.apache.fluss.fs.FsPath; import org.apache.fluss.metadata.DatabaseDescriptor; import org.apache.fluss.metadata.Schema; import org.apache.fluss.metadata.SchemaInfo; @@ -49,7 +50,9 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; +import java.nio.file.Path; import java.time.Duration; import java.util.ArrayList; import java.util.Collections; @@ -79,6 +82,8 @@ class TableChangeWatcherTest { private TestingEventManager eventManager; private TableChangeWatcher tableChangeWatcher; private static MetadataManager metadataManager; + private @TempDir Path tempDir; + private FsPath remoteDataDir; @BeforeAll static void beforeAll() { @@ -99,6 +104,7 @@ void before() { eventManager = new TestingEventManager(); tableChangeWatcher = new TableChangeWatcher(zookeeperClient, eventManager); tableChangeWatcher.start(); + remoteDataDir = new FsPath(tempDir.toAbsolutePath().toString()); } @AfterEach @@ -124,7 +130,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( @@ -184,7 +191,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 @@ -216,9 +225,9 @@ void testPartitionedTable() throws Exception { .getBucketAssignments()); // register assignment and metadata zookeeperClient.registerPartitionAssignmentAndMetadata( - 1L, "2011", partitionAssignment, tablePath, tableId); + 1L, "2011", partitionAssignment, null, tablePath, tableId); zookeeperClient.registerPartitionAssignmentAndMetadata( - 2L, "2022", partitionAssignment, tablePath, tableId); + 2L, "2022", partitionAssignment, null, tablePath, tableId); // create partitions events expectedEvents.add( @@ -263,7 +272,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( 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..0f6afe5231 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/RemoteDirDynamicLoaderTest.java @@ -0,0 +1,82 @@ +/* + * 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.junit.jupiter.api.Test; + +import java.util.Arrays; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link RemoteDirDynamicLoader}. */ +class RemoteDirDynamicLoaderTest { + + private static final String DEFAULT_REMOTE_DIR = "hdfs://cluster/default"; + + @Test + void testReconfigureWithStrategyChange() { + Configuration conf = createBaseConfiguration(); + conf.set(ConfigOptions.REMOTE_DATA_DIRS, Arrays.asList("hdfs://dir1", "hdfs://dir2")); + RemoteDirDynamicLoader loader = new RemoteDirDynamicLoader(conf); + + RemoteDirDynamicLoader.RemoteDirContainer originalContainer = + loader.getRemoteDataDirContainer(); + + // 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); + } + + @Test + void testReconfigureWithWeightsChange() { + 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)); + RemoteDirDynamicLoader loader = new RemoteDirDynamicLoader(conf); + + RemoteDirDynamicLoader.RemoteDirContainer originalContainer = + loader.getRemoteDataDirContainer(); + + // 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); + } + + 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/RoundRobinRemoteDirSelectorTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/RoundRobinRemoteDirSelectorTest.java new file mode 100644 index 0000000000..12233bcdf0 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/RoundRobinRemoteDirSelectorTest.java @@ -0,0 +1,173 @@ +/* + * 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.fs.FsPath; + +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 FsPath DEFAULT_DIR = new FsPath("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() { + FsPath dir = new FsPath("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( + new FsPath("hdfs://cluster/data1"), + new FsPath("hdfs://cluster/data2"), + new FsPath("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( + new FsPath("hdfs://cluster/data1"), + new FsPath("hdfs://cluster/data2"), + new FsPath("hdfs://cluster/data3")); + + RoundRobinRemoteDirSelector selector = new RoundRobinRemoteDirSelector(DEFAULT_DIR, dirs); + + Map counts = new HashMap<>(); + int totalCalls = 30; + + for (int i = 0; i < totalCalls; i++) { + FsPath 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( + new FsPath("hdfs://cluster/data1"), new FsPath("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( + new FsPath("hdfs://cluster/data1"), + new FsPath("hdfs://cluster/data2"), + new FsPath("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( + new FsPath("hdfs://cluster/data1"), + new FsPath("hdfs://cluster/data2"), + new FsPath("hdfs://cluster/data3"), + new FsPath("hdfs://cluster/data4"), + new FsPath("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..30916300a0 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/WeightedRoundRobinRemoteDirSelectorTest.java @@ -0,0 +1,233 @@ +/* + * 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.fs.FsPath; + +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 FsPath DEFAULT_DIR = new FsPath("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() { + FsPath dir = new FsPath("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( + new FsPath("hdfs://cluster/data1"), + new FsPath("hdfs://cluster/data2"), + new FsPath("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++) { + FsPath 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( + new FsPath("hdfs://cluster/data1"), + new FsPath("hdfs://cluster/data2"), + new FsPath("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++) { + FsPath 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( + new FsPath("hdfs://cluster/A"), + new FsPath("hdfs://cluster/B"), + new FsPath("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( + new FsPath("hdfs://cluster/data1"), new FsPath("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( + new FsPath("hdfs://cluster/data1"), new FsPath("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( + new FsPath("hdfs://cluster/data1"), new FsPath("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++) { + FsPath 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); + } +} 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 c57c9950e5..d6970ed197 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 @@ -35,6 +35,7 @@ import org.apache.fluss.server.coordinator.MetadataManager; import org.apache.fluss.server.coordinator.TestCoordinatorChannelManager; import org.apache.fluss.server.coordinator.event.CoordinatorEventManager; +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; @@ -115,6 +116,7 @@ void beforeEach() throws IOException { zookeeperClient, new Configuration(), new LakeCatalogDynamicLoader(new Configuration(), null, true)), + new RemoteDirDynamicLoader(conf), new Configuration()); lakeTableTieringManager = new LakeTableTieringManager(); } 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..7de0b9fcec 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 @@ -20,6 +20,7 @@ import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; import org.apache.fluss.config.TableConfig; +import org.apache.fluss.fs.FsPath; import org.apache.fluss.metadata.KvFormat; import org.apache.fluss.metadata.LogFormat; import org.apache.fluss.metadata.PhysicalTablePath; @@ -71,6 +72,7 @@ final class DroppedTableRecoveryTest extends LogTestBase { private TableBucket tableBucket; private LogManager logManager; private KvManager kvManager; + private FsPath remoteDataDir; @BeforeAll static void baseBeforeAll() { @@ -103,12 +105,15 @@ public void setup() throws Exception { KvManager.create( conf, zkClient, logManager, TestingMetricGroups.TABLET_SERVER_METRICS); kvManager.startup(); + + remoteDataDir = new FsPath(tempDir.getAbsolutePath() + "/remote_dir"); } 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, remoteDataDir, 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..1eba6525a9 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 @@ -18,6 +18,7 @@ package org.apache.fluss.server.log; import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.fs.FsPath; import org.apache.fluss.metadata.LogFormat; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.TableBucket; @@ -78,6 +79,7 @@ final class LogManagerTest extends LogTestBase { private TableBucket tableBucket1; private TableBucket tableBucket2; private LogManager logManager; + private FsPath remoteDataDir; // TODO add more tests refer to kafka's LogManagerTest. @@ -108,15 +110,19 @@ public void setup() throws Exception { SystemClock.getInstance(), TestingMetricGroups.TABLET_SERVER_METRICS); logManager.startup(); + + remoteDataDir = new FsPath(tempDir.getAbsolutePath() + "/remote_dir"); } 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, remoteDataDir, 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, remoteDataDir, DATA2_TABLE_DESCRIPTOR)); zkClient.registerFirstSchema(tablePath2, DATA2_SCHEMA); } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/log/remote/DefaultRemoteLogStorageTest.java b/fluss-server/src/test/java/org/apache/fluss/server/log/remote/DefaultRemoteLogStorageTest.java index a450295a7f..27ee9da1ab 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/log/remote/DefaultRemoteLogStorageTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/log/remote/DefaultRemoteLogStorageTest.java @@ -184,13 +184,14 @@ void testDeleteTable(boolean partitionTable) throws Exception { File remoteDirForBucket = new File( FlussPaths.remoteLogTabletDir( - remoteLogStorageManager.getRemoteLogDir(), + remoteLogSegment.remoteLogDir(), physicalTablePath, tableBucket) .toString()); assertThat(remoteDirForBucket.exists()).isTrue(); - remoteLogStorageManager.deleteTableBucket(physicalTablePath, tableBucket); + remoteLogStorageManager.deleteTableBucket( + remoteLogSegment.remoteLogDir(), physicalTablePath, tableBucket); assertThat(remoteDirForBucket.exists()).isFalse(); assertThatThrownBy( () -> @@ -204,7 +205,7 @@ private File getTestingRemoteLogSegmentDir(RemoteLogSegment remoteLogSegment) { return new File( FlussPaths.remoteLogSegmentDir( FlussPaths.remoteLogTabletDir( - remoteLogStorageManager.getRemoteLogDir(), + remoteLogSegment.remoteLogDir(), remoteLogSegment.physicalTablePath(), remoteLogSegment.tableBucket()), remoteLogSegment.remoteLogSegmentId()) diff --git a/fluss-server/src/test/java/org/apache/fluss/server/log/remote/RemoteLogITCase.java b/fluss-server/src/test/java/org/apache/fluss/server/log/remote/RemoteLogITCase.java index 3187e5db58..7a244e4c25 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/log/remote/RemoteLogITCase.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/log/remote/RemoteLogITCase.java @@ -117,9 +117,7 @@ public void remoteLogMiscTest() throws Exception { // test create: verify remote log created FsPath fsPath = FlussPaths.remoteLogTabletDir( - tabletServer.getReplicaManager().getRemoteLogManager().remoteLogDir(), - PhysicalTablePath.of(DATA1_TABLE_PATH), - tb); + manifest.getRemoteLogDir(), PhysicalTablePath.of(DATA1_TABLE_PATH), tb); FileSystem fileSystem = fsPath.getFileSystem(); assertThat(fileSystem.exists(fsPath)).isTrue(); assertThat(fileSystem.listStatus(fsPath).length).isGreaterThan(0); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/log/remote/RemoteLogManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/log/remote/RemoteLogManagerTest.java index e2e5be650f..c4cdc903f0 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/log/remote/RemoteLogManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/log/remote/RemoteLogManagerTest.java @@ -43,7 +43,6 @@ import static org.apache.fluss.record.TestData.DATA1_PHYSICAL_TABLE_PATH; import static org.apache.fluss.record.TestData.DATA1_SCHEMA; -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.server.zk.data.LeaderAndIsr.INITIAL_LEADER_EPOCH; @@ -356,7 +355,7 @@ void testFetchRecordsFromRemote(boolean partitionTable) throws Exception { @ParameterizedTest @ValueSource(booleans = {true, false}) void testCleanupLocalSegments(boolean partitionTable) throws Exception { - TableBucket tb = makeTableBucket(partitionTable); + TableBucket tb = makeTableBucket(partitionTable, true); // Need to make leader by ReplicaManager. makeKvTableAsLeader(tb, DATA1_TABLE_PATH_PK, INITIAL_LEADER_EPOCH, partitionTable); LogTablet logTablet = replicaManager.getReplicaOrException(tb).getLogTablet(); @@ -528,16 +527,4 @@ void testLookupOffsetForTimestamp(boolean partitionTable) throws Exception { assertThat(remoteLogManager.lookupOffsetForTimestamp(tb, startTimestamp + 5000)) .isEqualTo(-1L); } - - private TableBucket makeTableBucket(boolean partitionTable) { - return makeTableBucket(DATA1_TABLE_ID, partitionTable); - } - - private TableBucket makeTableBucket(long tableId, boolean partitionTable) { - if (partitionTable) { - return new TableBucket(tableId, 0L, 0); - } else { - return new TableBucket(tableId, 0); - } - } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/log/remote/RemoteLogManifestJsonSerdeTest.java b/fluss-server/src/test/java/org/apache/fluss/server/log/remote/RemoteLogManifestJsonSerdeTest.java index da4024ffc4..0bc50033da 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/log/remote/RemoteLogManifestJsonSerdeTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/log/remote/RemoteLogManifestJsonSerdeTest.java @@ -17,15 +17,20 @@ package org.apache.fluss.server.log.remote; +import org.apache.fluss.fs.FsPath; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.remote.RemoteLogSegment; import org.apache.fluss.utils.json.JsonSerdeTestBase; +import org.junit.jupiter.api.Test; + import java.util.Arrays; import java.util.UUID; +import static org.assertj.core.api.Assertions.assertThat; + /** Tests of {@link org.apache.fluss.server.log.remote.RemoteLogManifestJsonSerde}. */ class RemoteLogManifestJsonSerdeTest extends JsonSerdeTestBase { private static final PhysicalTablePath TABLE_PATH1 = @@ -36,6 +41,8 @@ class RemoteLogManifestJsonSerdeTest extends JsonSerdeTestBase createRemoteLogSegmentList(LogTablet logTablet) { + protected List createRemoteLogSegmentList(LogTablet logTablet) { return logTablet.getSegments().stream() .map( segment -> { @@ -146,6 +149,7 @@ protected static List createRemoteLogSegmentList(LogTablet log segment.getFileLogRecords().sizeInBytes()) .tableBucket(logTablet.getTableBucket()) .physicalTablePath(logTablet.getPhysicalTablePath()) + .remoteLogDir(FlussPaths.remoteLogDir(conf)) .build(); } catch (IOException e) { throw new RuntimeException(e); 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..21ee2178e6 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 @@ -100,7 +100,7 @@ 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, null, desc, tableAssignment, false); // Create leader and isr for buckets TableBucket tableBucket0 = new TableBucket(tableId, 0); @@ -161,7 +161,7 @@ void testGetPartitionMetadataFromZk() throws Exception { new PartitionAssignment(tableId, bucketAssignments); zookeeperClient.registerPartitionAssignmentAndMetadata( - partitionId, partitionName, partitionAssignment, tablePath, tableId); + partitionId, partitionName, partitionAssignment, null, tablePath, tableId); // Create leader and isr for partition buckets TableBucket partitionBucket0 = new TableBucket(tableId, partitionId, 0); @@ -233,9 +233,9 @@ void testBatchGetPartitionMetadataFromZkAsync() throws Exception { tableId1, Collections.singletonMap(1, BucketAssignment.of(2, 3))); zookeeperClient.registerPartitionAssignmentAndMetadata( - partitionId1, partitionName1, partitionAssignment1, tablePath1, tableId1); + partitionId1, partitionName1, partitionAssignment1, null, tablePath1, tableId1); zookeeperClient.registerPartitionAssignmentAndMetadata( - partitionId2, partitionName2, partitionAssignment2, tablePath1, tableId1); + partitionId2, partitionName2, partitionAssignment2, null, tablePath1, tableId1); // Create partition for table2 long partitionId3 = 21L; @@ -246,7 +246,7 @@ void testBatchGetPartitionMetadataFromZkAsync() throws Exception { tableId2, Collections.singletonMap(0, BucketAssignment.of(1, 3))); zookeeperClient.registerPartitionAssignmentAndMetadata( - partitionId3, partitionName3, partitionAssignment3, tablePath2, tableId2); + partitionId3, partitionName3, partitionAssignment3, null, tablePath2, tableId2); // Create leader and isr for all partition buckets TableBucket bucket1 = new TableBucket(tableId1, partitionId1, 0); @@ -338,6 +338,7 @@ private TableRegistration createTestTableRegistration( new TableDescriptor.TableDistribution(3, Collections.singletonList("a")), options, Collections.emptyMap(), + null, currentMillis, currentMillis); } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/NotifyReplicaLakeTableOffsetTest.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/NotifyReplicaLakeTableOffsetTest.java index 0d0b25204c..d31b3c2a97 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/NotifyReplicaLakeTableOffsetTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/NotifyReplicaLakeTableOffsetTest.java @@ -29,8 +29,6 @@ import java.util.Collections; import java.util.concurrent.CompletableFuture; -import static org.apache.fluss.record.TestData.DATA1_TABLE_ID; - /** Test for notify replica lakehouse data info. */ class NotifyReplicaLakeTableOffsetTest extends ReplicaTestBase { @@ -76,18 +74,6 @@ private void verifyLakeTableOffset( .isEqualTo(maxTimestamp); } - private TableBucket makeTableBucket(boolean partitionTable) { - return makeTableBucket(DATA1_TABLE_ID, partitionTable); - } - - private TableBucket makeTableBucket(long tableId, boolean partitionTable) { - if (partitionTable) { - return new TableBucket(tableId, 0L, 0); - } else { - return new TableBucket(tableId, 0); - } - } - private NotifyLakeTableOffsetData getNotifyLakeTableOffset( TableBucket tableBucket, long snapshotId, 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 7db23130fb..4db0a5ad97 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 @@ -942,7 +942,7 @@ void testLimitScanLogTable() throws Exception { @ParameterizedTest @ValueSource(booleans = {true, false}) void testListOffsets(boolean isPartitioned) throws Exception { - TableBucket tb = new TableBucket(DATA1_TABLE_ID, isPartitioned ? 10L : null, 1); + TableBucket tb = makeTableBucket(DATA1_TABLE_ID, isPartitioned ? 10L : null, false); makeLogTableAsLeader(tb, isPartitioned); // produce one batch to this bucket. @@ -1619,11 +1619,12 @@ void testUpdateMetadata() throws Exception { // register table to zk. zkClient.registerTable( nonePartitionTablePath, - TableRegistration.newTable(nonePartitionTableId, DATA1_TABLE_DESCRIPTOR)); + TableRegistration.newTable(nonePartitionTableId, null, DATA1_TABLE_DESCRIPTOR)); zkClient.registerFirstSchema(nonePartitionTablePath, DATA1_TABLE_DESCRIPTOR.getSchema()); zkClient.registerTable( partitionTablePath, - TableRegistration.newTable(partitionTableId, DATA1_PARTITIONED_TABLE_DESCRIPTOR)); + TableRegistration.newTable( + partitionTableId, null, DATA1_PARTITIONED_TABLE_DESCRIPTOR)); zkClient.registerFirstSchema( partitionTablePath, DATA1_PARTITIONED_TABLE_DESCRIPTOR.getSchema()); 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 559353df63..ab293d2af8 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 @@ -57,10 +57,12 @@ import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.server.zk.ZooKeeperExtension; import org.apache.fluss.server.zk.data.LeaderAndIsr; +import org.apache.fluss.server.zk.data.PartitionAssignment; import org.apache.fluss.server.zk.data.TableRegistration; import org.apache.fluss.testutils.common.AllCallbackWrapper; import org.apache.fluss.testutils.common.ManuallyTriggeredScheduledExecutorService; import org.apache.fluss.utils.CloseableRegistry; +import org.apache.fluss.utils.FlussPaths; import org.apache.fluss.utils.clock.ManualClock; import org.apache.fluss.utils.concurrent.FlussScheduler; import org.apache.fluss.utils.function.FunctionWithException; @@ -255,16 +257,21 @@ 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, FlussPaths.remoteDataDir(conf), 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, + FlussPaths.remoteDataDir(conf), + 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, FlussPaths.remoteDataDir(conf), DATA2_TABLE_DESCRIPTOR)); zkClient.registerFirstSchema(DATA2_TABLE_PATH, DATA2_SCHEMA); } @@ -283,7 +290,10 @@ protected long registerTableInZkClient( if (zkClient.tableExist(tablePath)) { zkClient.deleteTable(tablePath); } - zkClient.registerTable(tablePath, TableRegistration.newTable(tableId, tableDescriptor)); + zkClient.registerTable( + tablePath, + TableRegistration.newTable( + tableId, FlussPaths.remoteDataDir(conf), tableDescriptor)); zkClient.registerFirstSchema(tablePath, schema); return tableId; } @@ -481,6 +491,7 @@ private Replica makeReplica( NOPErrorHandler.INSTANCE, metricGroup, DATA1_TABLE_INFO, + ((TestSnapshotContext) snapshotContext).getRemoteKvDir().getParent(), manualClock); } @@ -556,6 +567,50 @@ protected Set listRemoteLogFiles(TableBucket tableBucket) throws IOExcep .collect(Collectors.toSet()); } + protected TableBucket makeTableBucket(boolean partitionTable) throws Exception { + return makeTableBucket(DATA1_TABLE_ID, partitionTable); + } + + protected TableBucket makeTableBucket(boolean partitionTable, boolean kvTable) + throws Exception { + long tableId = kvTable ? DATA1_TABLE_ID_PK : DATA1_TABLE_ID; + Long partitionId = partitionTable ? 0L : null; + return makeTableBucket(tableId, partitionId, kvTable); + } + + protected TableBucket makeTableBucket(long tableId, boolean partitionTable) throws Exception { + Long partitionId = partitionTable ? 0L : null; + return makeTableBucket(tableId, partitionId, false); + } + + protected TableBucket makeTableBucket(long tableId, Long partitionId, boolean kvTable) + throws Exception { + int bucketId = 0; + boolean partitionTable = partitionId != null; + if (partitionTable) { + if (kvTable) { + zkClient.registerPartitionAssignmentAndMetadata( + partitionId, + DATA1_PHYSICAL_TABLE_PATH_PK_PA_2024.getPartitionName(), + new PartitionAssignment(tableId, Collections.emptyMap()), + FlussPaths.remoteDataDir(conf), + DATA1_TABLE_PATH_PK, + tableId); + } else { + zkClient.registerPartitionAssignmentAndMetadata( + partitionId, + DATA1_PHYSICAL_TABLE_PATH_PA_2024.getPartitionName(), + new PartitionAssignment(tableId, Collections.emptyMap()), + FlussPaths.remoteDataDir(conf), + DATA1_TABLE_PATH, + tableId); + } + return new TableBucket(tableId, partitionId, bucketId); + } else { + return new TableBucket(tableId, bucketId); + } + } + /** An implementation of {@link SnapshotContext} for test purpose. */ protected class TestSnapshotContext implements SnapshotContext { @@ -643,7 +698,6 @@ public int getSnapshotFsWriteBufferSize() { return 1024; } - @Override public FsPath getRemoteKvDir() { return remoteKvTabletDir; } 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..9ad21d2aa8 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 @@ -22,6 +22,7 @@ import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; import org.apache.fluss.exception.FlussRuntimeException; +import org.apache.fluss.fs.FsPath; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.rpc.RpcClient; @@ -93,6 +94,7 @@ public class ReplicaFetcherThreadTest { private static ZooKeeperClient zkClient; private ManualClock manualClock; private @TempDir File tempDir; + private FsPath remoteDataDir; private TableBucket tb; private final int leaderServerId = 1; private final int followerServerId = 2; @@ -115,6 +117,7 @@ public void setup() throws Exception { ZOO_KEEPER_EXTENSION_WRAPPER.getCustomExtension().cleanupRoot(); manualClock = new ManualClock(System.currentTimeMillis()); Configuration conf = new Configuration(); + remoteDataDir = new FsPath(tempDir.getAbsolutePath() + "/remote_data_dir"); tb = new TableBucket(DATA1_TABLE_ID, 0); leaderRM = createReplicaManager(leaderServerId); followerRM = createReplicaManager(followerServerId); @@ -377,7 +380,7 @@ 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, remoteDataDir, DATA1_TABLE_DESCRIPTOR)); zkClient.registerFirstSchema(DATA1_TABLE_PATH, DATA1_SCHEMA); } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/utils/ConfigurationParserUtilsTest.java b/fluss-server/src/test/java/org/apache/fluss/server/utils/ConfigurationParserUtilsTest.java index a4a75f66d1..6d37c49b8f 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/utils/ConfigurationParserUtilsTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/utils/ConfigurationParserUtilsTest.java @@ -30,6 +30,7 @@ import java.nio.file.Files; import java.nio.file.Path; import java.util.Collections; +import java.util.List; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -37,6 +38,21 @@ /** Test for {@link ConfigurationParserUtils}. */ public class ConfigurationParserUtilsTest { + @Test + void test(@TempDir Path tempFolder) throws Exception { + Path yamlFile = tempFolder.resolve("server.yaml"); + Files.write(yamlFile, Collections.singleton("remote.data.dir:path")); + String confDir = tempFolder.toAbsolutePath().toString(); + + final String[] args = {"--configDir", confDir}; + Configuration configuration = + ConfigurationParserUtils.loadCommonConfiguration( + args, ConfigurationParserUtilsTest.class.getSimpleName()); + + List strings = configuration.get(ConfigOptions.REMOTE_DATA_DIRS); + System.out.println(strings); + } + @Test void testLoadCommonConfiguration(@TempDir Path tempFolder) throws Exception { Path yamlFile = tempFolder.resolve("server.yaml"); 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 64da4b95cb..9b041d690e 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 @@ -23,11 +23,11 @@ import org.apache.fluss.cluster.rebalance.ServerTag; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; +import org.apache.fluss.fs.FsPath; import org.apache.fluss.metadata.Schema; 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; @@ -35,6 +35,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.RebalancePlan; import org.apache.fluss.server.zk.data.ServerTags; import org.apache.fluss.server.zk.data.TableAssignment; @@ -312,6 +313,8 @@ void testTable() throws Exception { TablePath tablePath1 = TablePath.of("db", "tb1"); TablePath tablePath2 = TablePath.of("db", "tb2"); + FsPath remoteDataDir = new FsPath("file://testTable"); + assertThat(zookeeperClient.getTable(tablePath1)).isEmpty(); assertThat(zookeeperClient.getTable(tablePath2)).isEmpty(); @@ -328,6 +331,7 @@ void testTable() throws Exception { new TableDescriptor.TableDistribution(16, Collections.singletonList("a")), options, Collections.singletonMap("custom-1", "100"), + remoteDataDir, currentMillis, currentMillis); TableRegistration tableReg2 = @@ -338,6 +342,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); @@ -363,6 +368,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); @@ -508,6 +514,7 @@ void testPartition() throws Exception { // first create a table TablePath tablePath = TablePath.of("db", "tb"); long tableId = 12; + FsPath remoteDataDir = new FsPath("file://testPartition"); long currentMillis = System.currentTimeMillis(); TableRegistration tableReg = new TableRegistration( @@ -517,6 +524,7 @@ void testPartition() throws Exception { new TableDescriptor.TableDistribution(16, Collections.singletonList("a")), Collections.emptyMap(), Collections.emptyMap(), + remoteDataDir, currentMillis, currentMillis); zookeeperClient.registerTable(tablePath, tableReg); @@ -538,14 +546,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); 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..356760dbb9 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/PartitionRegistrationJsonSerdeTest.java @@ -0,0 +1,51 @@ +/* + * 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.fs.FsPath; +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[2]; + + // with remote data dir + partitionRegistrations[0] = + new PartitionRegistration(1234L, 5678L, new FsPath("file://local/remote")); + + // without remote data dir + partitionRegistrations[1] = new PartitionRegistration(1234L, 5678L, null); + + return partitionRegistrations; + } + + @Override + protected String[] expectedJsons() { + return new String[] { + "{\"version\":1,\"table_id\":1234,\"partition_id\":5678,\"remote_data_dir\":\"file://local/remote\"}", + "{\"version\":1,\"table_id\":1234,\"partition_id\":5678}" + }; + } +} 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..5654a0c4f6 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 @@ -17,6 +17,7 @@ package org.apache.fluss.server.zk.data; +import org.apache.fluss.fs.FsPath; import org.apache.fluss.metadata.TableDescriptor; import org.apache.fluss.metadata.TableDescriptor.TableDistribution; import org.apache.fluss.record.TestData; @@ -48,6 +49,7 @@ void testInvalidTableRegistration() { new TableDistribution(null, Arrays.asList("b", "c")), Maps.newHashMap(), Collections.singletonMap("custom-3", "\"300\""), + null, 1735538268L, 1735538268L)) .isInstanceOf(IllegalArgumentException.class) @@ -58,6 +60,7 @@ void testInvalidTableRegistration() { () -> TableRegistration.newTable( 11, + null, TableDescriptor.builder() .schema(TestData.DATA1_SCHEMA) .build())) @@ -77,6 +80,7 @@ protected TableRegistration[] createObjects() { new TableDistribution(16, Arrays.asList("b", "c")), Maps.newHashMap(), Collections.singletonMap("custom-3", "\"300\""), + new FsPath("file://local/remote"), 1735538268L, 1735538268L); @@ -88,6 +92,7 @@ protected TableRegistration[] createObjects() { new TableDistribution(32, Collections.emptyList()), Collections.singletonMap("option-3", "300"), Maps.newHashMap(), + null, -1, -1); @@ -98,7 +103,7 @@ 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}", + + "\"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\":{},\"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 f55c1b94ad..ed4f66a15b 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 @@ -97,6 +97,7 @@ void testRegisterLakeTableSnapshotCompatibility(@TempDir Path tempDir) throws Ex 1, Collections.singletonList("a")), Collections.emptyMap(), Collections.emptyMap(), + null, System.currentTimeMillis(), System.currentTimeMillis()); zookeeperClient.registerTable(tablePath, tableReg);