diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/RemoteLogDownloader.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/RemoteLogDownloader.java index 20b5ca5b84..44424cf165 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/RemoteLogDownloader.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/RemoteLogDownloader.java @@ -167,8 +167,9 @@ void fetchOnce() throws Exception { (bytes, throwable) -> { if (throwable != null) { LOG.error( - "Failed to download remote log segment file {}.", + "Failed to download remote log segment file {} for bucket {}.", fsPathAndFileName.getFileName(), + request.segment.tableBucket(), ExceptionUtils.stripExecutionException(throwable)); // release the semaphore for the failed request prefetchSemaphore.release(); @@ -178,8 +179,9 @@ void fetchOnce() throws Exception { scannerMetricGroup.remoteFetchErrorCount().inc(); } else { LOG.info( - "Successfully downloaded remote log segment file {} to local cost {} ms.", + "Successfully downloaded remote log segment file {} for bucket {} to local cost {} ms.", fsPathAndFileName.getFileName(), + request.segment.tableBucket(), System.currentTimeMillis() - startTime); File localFile = new File( 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..8379bf966a 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,44 @@ public class ConfigOptions { "The directory used for storing the kv snapshot data files and remote log for log tiered storage " + " in a Fluss supported filesystem."); + public static final ConfigOption> REMOTE_DATA_DIRS = + key("remote.data.dirs") + .stringType() + .asList() + .defaultValues() + .withDescription( + "The directories used for storing the kv snapshot data files and remote log for log tiered storage " + + " in a Fluss supported filesystem. " + + "This is a list of remote data directory paths. " + + "Example: `remote.data.dirs: oss://bucket1/fluss-remote-data, oss://bucket2/fluss-remote-data`."); + + public static final ConfigOption REMOTE_DATA_DIRS_STRATEGY = + key("remote.data.dirs.strategy") + .enumType(RemoteDataDirStrategy.class) + .defaultValue(RemoteDataDirStrategy.ROUND_ROBIN) + .withDescription( + "The strategy for selecting the remote data directory from `" + + REMOTE_DATA_DIRS.key() + + "`."); + + public static final ConfigOption> REMOTE_DATA_DIRS_WEIGHTS = + key("remote.data.dirs.weights") + .intType() + .asList() + .defaultValues() + .withDescription( + "The weights of the remote data directories. " + + "This is a list of weights corresponding to the `" + + REMOTE_DATA_DIRS.key() + + "` in the same order. When `" + + REMOTE_DATA_DIRS_STRATEGY.key() + + "` is set to `" + + RemoteDataDirStrategy.WEIGHTED_ROUND_ROBIN + + "`, this must be configured, and its size must be equal to `" + + REMOTE_DATA_DIRS.key() + + "`; otherwise, it will be ignored." + + "Example: `remote.data.dir.weights: 1, 2`"); + public static final ConfigOption REMOTE_FS_WRITE_BUFFER_SIZE = key("remote.fs.write-buffer-size") .memoryType() @@ -1925,4 +1963,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..d13e20bd70 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,97 @@ 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.isEmpty() && !weights.isEmpty()) { + if (remoteDataDirs.size() != weights.size()) { + throw new IllegalConfigurationException( + String.format( + "The size of '%s' (%d) must match the size of '%s' (%d) when using WEIGHTED_ROUND_ROBIN strategy.", + ConfigOptions.REMOTE_DATA_DIRS.key(), + remoteDataDirs.size(), + ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS.key(), + weights.size())); + } + // validate all weights are positive + for (int i = 0; i < weights.size(); i++) { + if (weights.get(i) < 0) { + throw new IllegalConfigurationException( + String.format( + "All weights in '%s' must be no less than 0, but found %d at index %d.", + ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS.key(), + weights.get(i), + i)); + } + } + } + } + } + + public static void validateTabletConfigs(Configuration conf) { + validServerConfigs(conf); + + Optional serverId = conf.getOptional(ConfigOptions.TABLET_SERVER_ID); + if (!serverId.isPresent()) { + throw new IllegalConfigurationException( + String.format("Configuration %s must be set.", ConfigOptions.TABLET_SERVER_ID)); + } + + 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/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..3de4301903 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 @@ -28,6 +28,7 @@ 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.DatabaseDescriptor; import org.apache.fluss.metadata.Schema; import org.apache.fluss.metadata.TableBucket; @@ -48,7 +49,9 @@ import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; 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.Collection; @@ -149,12 +152,16 @@ public class FlinkTestBase extends AbstractTestBase { protected static Configuration clientConf; protected static String bootstrapServers; + protected static @TempDir Path tempDir; + protected static FsPath remoteDataDir; + @BeforeAll protected static void beforeAll() { clientConf = FLUSS_CLUSTER_EXTENSION.getClientConfig(); bootstrapServers = FLUSS_CLUSTER_EXTENSION.getBootstrapServers(); conn = ConnectionFactory.createConnection(clientConf); admin = conn.getAdmin(); + remoteDataDir = new FsPath(tempDir.toAbsolutePath().toString()); } @BeforeEach @@ -247,6 +254,7 @@ public static Map createPartitions( partition, new PartitionAssignment( tableInfo.getTableId(), assignment.getBucketAssignments()), + remoteDataDir, 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 76a3fbd68c..5b5225a413 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; @@ -34,6 +33,7 @@ import org.apache.fluss.server.authorizer.Authorizer; import org.apache.fluss.server.authorizer.AuthorizerLoader; import org.apache.fluss.server.coordinator.rebalance.RebalanceManager; +import org.apache.fluss.server.coordinator.remote.RemoteDirDynamicLoader; import org.apache.fluss.server.metadata.CoordinatorMetadataCache; import org.apache.fluss.server.metadata.ServerMetadataCache; import org.apache.fluss.server.metrics.ServerMetricUtils; @@ -63,6 +63,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: * @@ -141,9 +143,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<>(); } @@ -173,10 +178,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(); @@ -207,6 +215,7 @@ protected void startServices() throws Exception { authorizer, lakeCatalogDynamicLoader, lakeTableTieringManager, + remoteDirDynamicLoader, dynamicConfigManager, ioExecutor); @@ -231,7 +240,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 @@ -444,6 +454,10 @@ CompletableFuture stopServices() { if (lakeCatalogDynamicLoader != null) { lakeCatalogDynamicLoader.close(); } + + if (remoteDirDynamicLoader != null) { + remoteDirDynamicLoader.close(); + } } catch (Throwable t) { exception = ExceptionUtils.firstOrSuppressed(t, exception); } @@ -513,31 +527,4 @@ public DynamicConfigManager getDynamicConfigManager() { public RebalanceManager getRebalanceManager() { return coordinatorEventProcessor.getRebalanceManager(); } - - private static void validateConfigs(Configuration conf) { - if (conf.get(ConfigOptions.DEFAULT_REPLICATION_FACTOR) < 1) { - throw new IllegalConfigurationException( - String.format( - "Invalid configuration for %s, it must be greater than or equal 1.", - ConfigOptions.DEFAULT_REPLICATION_FACTOR.key())); - } - if (conf.get(ConfigOptions.KV_MAX_RETAINED_SNAPSHOTS) < 1) { - throw new IllegalConfigurationException( - String.format( - "Invalid configuration for %s, it must be greater than or equal 1.", - ConfigOptions.KV_MAX_RETAINED_SNAPSHOTS.key())); - } - - if (conf.get(ConfigOptions.SERVER_IO_POOL_SIZE) < 1) { - throw new IllegalConfigurationException( - String.format( - "Invalid configuration for %s, it must be greater than or equal 1.", - ConfigOptions.SERVER_IO_POOL_SIZE.key())); - } - - if (conf.get(ConfigOptions.REMOTE_DATA_DIR) == null) { - throw new IllegalConfigurationException( - String.format("Configuration %s must be set.", ConfigOptions.REMOTE_DATA_DIR)); - } - } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorService.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorService.java index ee0092d1b8..69d15321ba 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 @@ -124,6 +124,7 @@ import org.apache.fluss.server.coordinator.event.RebalanceEvent; import org.apache.fluss.server.coordinator.event.RemoveServerTagEvent; import org.apache.fluss.server.coordinator.rebalance.goal.Goal; +import org.apache.fluss.server.coordinator.remote.RemoteDirDynamicLoader; import org.apache.fluss.server.entity.CommitKvSnapshotData; import org.apache.fluss.server.entity.LakeTieringTableInfo; import org.apache.fluss.server.entity.TablePropertyChanges; @@ -189,6 +190,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, @@ -200,6 +202,7 @@ public CoordinatorService( @Nullable Authorizer authorizer, LakeCatalogDynamicLoader lakeCatalogDynamicLoader, LakeTableTieringManager lakeTableTieringManager, + RemoteDirDynamicLoader remoteDirDynamicLoader, DynamicConfigManager dynamicConfigManager, ExecutorService ioExecutor) { super( @@ -224,6 +227,7 @@ public CoordinatorService( this.ioExecutor = ioExecutor; this.lakeTableHelper = new LakeTableHelper(zkClient, conf.getString(ConfigOptions.REMOTE_DATA_DIR)); + this.remoteDirDynamicLoader = remoteDirDynamicLoader; } @Override @@ -337,9 +341,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()); } @@ -540,9 +556,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()); @@ -596,6 +618,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 05c5d51de2..15a634bf8d 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); @@ -718,6 +725,7 @@ public Set getPartitions(TablePath tablePath) { public void createPartition( TablePath tablePath, long tableId, + FsPath remoteDataDir, PartitionAssignment partitionAssignment, ResolvedPartitionSpec partition, boolean ignoreIfExists) { @@ -777,7 +785,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) { @@ -826,7 +839,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..b7b0e27bc5 --- /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 no less than 0, but found %d at index %d.", + ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS.key(), + weights.get(i), + i)); + } + } + } + } + } + + @Override + public void reconfigure(Configuration newConfig) throws ConfigException { + // Check if any relevant configuration has changed + boolean strategyChanged = + hasConfigChanged(newConfig, ConfigOptions.REMOTE_DATA_DIRS_STRATEGY); + boolean remoteDirsChanged = hasConfigChanged(newConfig, ConfigOptions.REMOTE_DATA_DIRS); + boolean weightsChanged = + hasConfigChanged(newConfig, ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS); + + if (strategyChanged || remoteDirsChanged || weightsChanged) { + // Create a new container with the merged configuration + Configuration mergedConfig = mergeConfigurations(currentConfiguration, newConfig); + this.remoteDirContainer = new RemoteDirContainer(mergedConfig); + this.currentConfiguration = mergedConfig; + } + } + + /** + * Checks if a specific configuration option has changed in the new config. + * + * @param newConfig the new configuration + * @param option the configuration option to check + * @param the type of the configuration value + * @return true if the configuration has changed + */ + private boolean hasConfigChanged(Configuration newConfig, ConfigOption option) { + return newConfig + .getOptional(option) + .map(newValue -> !Objects.equals(newValue, currentConfiguration.get(option))) + .orElse(false); + } + + /** + * Merges the current configuration with new configuration values. + * + * @param current the current configuration + * @param updates the configuration updates to apply + * @return a new merged configuration + */ + private Configuration mergeConfigurations(Configuration current, Configuration updates) { + Configuration merged = new Configuration(current); + updates.toMap().forEach(merged::setString); + return merged; + } + + @Override + public void close() throws Exception { + // do nothing + } + + /** Container for managing remote data directories and selecting the next directory to use. */ + public static class RemoteDirContainer { + + private final 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..e1f878db94 --- /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() || totalWeight == 0) { + return defaultRemoteDataDir; + } + + synchronized (lock) { + int selectedIndex = -1; + int maxCurrentWeight = Integer.MIN_VALUE; + + // Step 1 & 2: Add weight to currentWeight and find the max + for (int i = 0; i < remoteDataDirs.size(); i++) { + currentWeights[i] += weights[i]; + if (currentWeights[i] > maxCurrentWeight) { + maxCurrentWeight = currentWeights[i]; + selectedIndex = i; + } + } + + // Step 3: Subtract total weight from selected node's current weight + currentWeights[selectedIndex] -= totalWeight; + + return remoteDataDirs.get(selectedIndex); + } + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/KvManager.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvManager.java index 1637af39d1..d155dadf71 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 = @@ -392,12 +384,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..10f46ca98b 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}. @@ -154,11 +155,13 @@ void deleteRemoteLogManifestSnapshot(FsPath remoteLogManifestPath) * Deletes the remote log data and metadata from remote storage for the input table bucket as * this table have been deleted. * + * @param remoteLogDir the remote log directory for the table bucket. * @param physicalTablePath the physical table path. * @param tableBucket the table bucket. * @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 f86778ebc3..5406516c06 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( @@ -830,9 +837,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 421db4a068..ac3203ad5a 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; @@ -1216,11 +1221,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(), @@ -1535,7 +1539,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()); } } @@ -1586,6 +1592,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 { @@ -1594,7 +1617,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 = @@ -1619,6 +1647,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 bd6bcad7f3..3e7c66f5f8 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 @@ -21,6 +21,7 @@ import org.apache.fluss.annotation.VisibleForTesting; 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; @@ -43,6 +44,7 @@ import org.apache.fluss.server.zk.data.DatabaseRegistration; import org.apache.fluss.server.zk.data.LeaderAndIsr; import org.apache.fluss.server.zk.data.PartitionAssignment; +import org.apache.fluss.server.zk.data.PartitionRegistration; import org.apache.fluss.server.zk.data.RebalanceTask; import org.apache.fluss.server.zk.data.RemoteLogManifestHandle; import org.apache.fluss.server.zk.data.ResourceAcl; @@ -646,7 +648,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())); } @@ -705,7 +707,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())); } @@ -751,7 +754,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); @@ -774,7 +777,7 @@ public Map getPartitionIds( return processGetDataResponses( responses, response -> path2PartitionPathMap.get(response.getPath()), - PartitionZNode::decode, + (byte[] data) -> PartitionZNode.decode(data).toTablePartition(), "partition"); } @@ -799,6 +802,7 @@ public void registerPartitionAssignmentAndMetadata( long partitionId, String partitionName, PartitionAssignment partitionAssignment, + FsPath remoteDataDir, TablePath tablePath, long tableId) throws Exception { @@ -844,12 +848,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 59a8644c8e..453cdcdf41 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/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 dfb4088e49..196e213179 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 @@ -48,6 +48,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; @@ -87,7 +88,6 @@ import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.api.io.TempDir; -import java.io.IOException; import java.nio.file.Path; import java.time.Duration; import java.util.Arrays; @@ -145,6 +145,7 @@ class CoordinatorEventProcessorTest { private static ZooKeeperClient zookeeperClient; private static MetadataManager metadataManager; + private static FsPath remoteDataDir; private CoordinatorEventProcessor eventProcessor; private final String defaultDatabase = "db"; @@ -155,7 +156,7 @@ class CoordinatorEventProcessorTest { private CoordinatorMetadataCache serverMetadataCache; @BeforeAll - static void baseBeforeAll() throws Exception { + static void baseBeforeAll(@TempDir Path tempDir) throws Exception { zookeeperClient = ZOO_KEEPER_EXTENSION_WRAPPER .getCustomExtension() @@ -181,18 +182,23 @@ static void baseBeforeAll() throws Exception { new Endpoint("host" + i, 1000, DEFAULT_LISTENER_NAME)), System.currentTimeMillis())); } + remoteDataDir = new FsPath(tempDir.toAbsolutePath().toString() + "/remote-data"); } @BeforeEach - void beforeEach() throws IOException { + void beforeEach() { serverMetadataCache = new CoordinatorMetadataCache(); // set a test channel manager for the context testCoordinatorChannelManager = new TestCoordinatorChannelManager(); - autoPartitionManager = - new AutoPartitionManager(serverMetadataCache, metadataManager, new Configuration()); lakeTableTieringManager = new LakeTableTieringManager(); 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( @@ -227,10 +233,14 @@ void testCreateAndDropTable() throws Exception { new TabletServerInfo(1, "rack1"), new TabletServerInfo(2, "rack2") }); - long t1Id = metadataManager.createTable(t1, tableDescriptor, tableAssignment, false); + long t1Id = + metadataManager.createTable( + t1, remoteDataDir, tableDescriptor, tableAssignment, false); TablePath t2 = TablePath.of(defaultDatabase, "create_drop_t2"); - long t2Id = metadataManager.createTable(t2, tableDescriptor, tableAssignment, false); + long t2Id = + metadataManager.createTable( + t2, remoteDataDir, tableDescriptor, tableAssignment, false); verifyTableCreated(t2Id, tableAssignment, nBuckets, replicationFactor); @@ -360,13 +370,15 @@ void testServerBecomeOnlineAndOfflineLine() throws Exception { TablePath table1Path = TablePath.of(defaultDatabase, "t1"); long table1Id = - metadataManager.createTable(table1Path, TEST_TABLE, table1Assignment, false); + metadataManager.createTable( + table1Path, remoteDataDir, TEST_TABLE, table1Assignment, false); TableAssignment table2Assignment = TableAssignment.builder().add(0, BucketAssignment.of(3)).build(); TablePath table2Path = TablePath.of(defaultDatabase, "t2"); long table2Id = - metadataManager.createTable(table2Path, TEST_TABLE, table2Assignment, false); + metadataManager.createTable( + table2Path, remoteDataDir, TEST_TABLE, table2Assignment, false); // retry until the table2 been created retryVerifyContext( @@ -471,7 +483,9 @@ void testRestartTriggerReplicaToOffline() throws Exception { .add(1, BucketAssignment.of(1, 2, 0)) .build(); TablePath tablePath = TablePath.of(defaultDatabase, "t_restart"); - long table1Id = metadataManager.createTable(tablePath, TEST_TABLE, tableAssignment, false); + long table1Id = + metadataManager.createTable( + tablePath, remoteDataDir, TEST_TABLE, tableAssignment, false); // let's restart initCoordinatorChannel(); @@ -595,7 +609,8 @@ void testCreateAndDropPartition() throws Exception { // create a partitioned table TableDescriptor tablePartitionTableDescriptor = getPartitionedTable(); long tableId = - metadataManager.createTable(tablePath, tablePartitionTableDescriptor, null, false); + metadataManager.createTable( + tablePath, remoteDataDir, tablePartitionTableDescriptor, null, false); int nBuckets = 3; int replicationFactor = 3; @@ -667,7 +682,8 @@ void testRestartResumeDropPartition() throws Exception { // create a partitioned table TableDescriptor tablePartitionTableDescriptor = getPartitionedTable(); long tableId = - metadataManager.createTable(tablePath, tablePartitionTableDescriptor, null, false); + metadataManager.createTable( + tablePath, remoteDataDir, tablePartitionTableDescriptor, null, false); int nBuckets = 3; int replicationFactor = 3; @@ -813,7 +829,8 @@ void testProcessAdjustIsr() throws Exception { new TabletServerInfo(1, "rack1"), new TabletServerInfo(2, "rack2") }); - long t1Id = metadataManager.createTable(t1, TEST_TABLE, tableAssignment, false); + long t1Id = + metadataManager.createTable(t1, remoteDataDir, TEST_TABLE, tableAssignment, false); verifyTableCreated(t1Id, tableAssignment, nBuckets, replicationFactor); // get the origin bucket leaderAndIsr @@ -867,7 +884,7 @@ void testSchemaChange() throws Exception { }); // create table List replicas = tableAssignment.getBucketAssignment(0).getReplicas(); - metadataManager.createTable(t1, TEST_TABLE, tableAssignment, false); + metadataManager.createTable(t1, remoteDataDir, TEST_TABLE, tableAssignment, false); TableInfo tableInfo = metadataManager.getTable(t1); retry( @@ -924,7 +941,11 @@ void testDoBucketReassignment() throws Exception { TableAssignment tableAssignment = new TableAssignment(bucketAssignments); long t1Id = metadataManager.createTable( - t1, CoordinatorEventProcessorTest.TEST_TABLE, tableAssignment, false); + t1, + remoteDataDir, + CoordinatorEventProcessorTest.TEST_TABLE, + tableAssignment, + false); TableBucket tb0 = new TableBucket(t1Id, 0); verifyIsr(tb0, 0, Arrays.asList(0, 1, 3)); @@ -1028,9 +1049,19 @@ private Tuple2 preparePartitionAssignment( String partition1Name = "2024"; String partition2Name = "2025"; zookeeperClient.registerPartitionAssignmentAndMetadata( - partition1Id, partition1Name, partitionAssignment, tablePath, tableId); + partition1Id, + partition1Name, + partitionAssignment, + remoteDataDir, + tablePath, + tableId); zookeeperClient.registerPartitionAssignmentAndMetadata( - partition2Id, partition2Name, partitionAssignment, tablePath, tableId); + partition2Id, + partition2Name, + partitionAssignment, + remoteDataDir, + tablePath, + tableId); return Tuple2.of( new PartitionIdName(partition1Id, partition1Name), @@ -1301,7 +1332,11 @@ private long createTable(TablePath tablePath, TabletServerInfo[] servers) { TableAssignment tableAssignment = generateAssignment(N_BUCKETS, REPLICATION_FACTOR, servers); return metadataManager.createTable( - tablePath, CoordinatorEventProcessorTest.TEST_TABLE, tableAssignment, false); + tablePath, + remoteDataDir, + CoordinatorEventProcessorTest.TEST_TABLE, + tableAssignment, + false); } private void alterTable(TablePath tablePath, List schemaChanges) { diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/MultiRemoteDirITCase.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/MultiRemoteDirITCase.java new file mode 100644 index 0000000000..f5593f2722 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/MultiRemoteDirITCase.java @@ -0,0 +1,285 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.coordinator; + +import org.apache.fluss.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.PartitionSpec; +import org.apache.fluss.metadata.Schema; +import org.apache.fluss.metadata.TableDescriptor; +import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.server.testutils.FlussClusterExtension; +import org.apache.fluss.server.testutils.RpcMessageTestUtils; +import org.apache.fluss.server.zk.ZooKeeperClient; +import org.apache.fluss.server.zk.data.PartitionRegistration; +import org.apache.fluss.server.zk.data.TableRegistration; +import org.apache.fluss.types.DataTypes; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import static org.apache.fluss.record.TestData.DATA1_PARTITIONED_TABLE_DESCRIPTOR; +import static org.apache.fluss.record.TestData.DATA1_TABLE_DESCRIPTOR; +import static org.apache.fluss.record.TestData.DATA1_TABLE_DESCRIPTOR_PK; +import static org.assertj.core.api.Assertions.assertThat; + +/** ITCase for multi remote data directories functionality. */ +class MultiRemoteDirITCase { + + private static final TableDescriptor DATA1_PARTITIONED_TABLE_DESCRIPTOR_PK = + TableDescriptor.builder() + .schema( + Schema.newBuilder() + .column("a", DataTypes.INT()) + .withComment("a is first column") + .column("b", DataTypes.STRING()) + .withComment("b is second column") + .primaryKey("a", "b") + .build()) + .distributedBy(3) + .partitionedBy("b") + .property(ConfigOptions.TABLE_AUTO_PARTITION_ENABLED, true) + .property( + ConfigOptions.TABLE_AUTO_PARTITION_TIME_UNIT, + AutoPartitionTimeUnit.YEAR) + .build(); + + private static final List REMOTE_DIR_NAMES = Arrays.asList("dir1", "dir2", "dir3"); + + @RegisterExtension + public static final FlussClusterExtension FLUSS_CLUSTER_EXTENSION = + FlussClusterExtension.builder() + .setNumOfTabletServers(3) + .setClusterConf(initConfig()) + .setRemoteDirNames(REMOTE_DIR_NAMES) + .build(); + + private ZooKeeperClient zkClient; + + private static Configuration initConfig() { + Configuration conf = new Configuration(); + conf.setInt(ConfigOptions.DEFAULT_REPLICATION_FACTOR, 3); + conf.set(ConfigOptions.AUTO_PARTITION_CHECK_INTERVAL, Duration.ofSeconds(1)); + + return conf; + } + + @BeforeEach + void setup() { + zkClient = FLUSS_CLUSTER_EXTENSION.getZooKeeperClient(); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testCreateMultipleTablesWithRoundRobin(boolean isPrimaryKeyTable) throws Exception { + // Create multiple tables and verify they use different remote dirs via round-robin + TableDescriptor tableDescriptor = + isPrimaryKeyTable ? DATA1_TABLE_DESCRIPTOR_PK : DATA1_TABLE_DESCRIPTOR; + String tablePrefix = isPrimaryKeyTable ? "pk_table_" : "non_pk_table_"; + + List remoteDirsUsed = new ArrayList<>(); + int tableCount = 6; // Create more tables than dirs to see round-robin in action + + for (int i = 0; i < tableCount; i++) { + TablePath tablePath = TablePath.of("test_db", tablePrefix + i); + RpcMessageTestUtils.createTable(FLUSS_CLUSTER_EXTENSION, tablePath, tableDescriptor); + + // Get the table registration to check remoteDataDir + Optional tableOpt = zkClient.getTable(tablePath); + assertThat(tableOpt).isPresent(); + TableRegistration table = tableOpt.get(); + + assertThat(table.remoteDataDir).isNotNull(); + remoteDirsUsed.add(table.remoteDataDir); + } + + // Verify round-robin distribution: each dir should be used at least once + Map dirUsageCount = new HashMap<>(); + for (FsPath dir : remoteDirsUsed) { + dirUsageCount.merge(dir.toString(), 1, Integer::sum); + } + + // With round-robin, all configured dirs should be used + assertThat(dirUsageCount.keySet()).hasSize(REMOTE_DIR_NAMES.size()); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testCreatePartitionsWithRoundRobin(boolean isPrimaryKeyTable) throws Exception { + // Create a partitioned table and add multiple partitions + // Each partition should get a different remoteDataDir via round-robin + String tablePrefix = isPrimaryKeyTable ? "pk_partitioned_" : "partitioned_"; + TablePath tablePath = TablePath.of("test_db", tablePrefix + "table_2"); + + TableDescriptor tableDescriptor = + isPrimaryKeyTable + ? DATA1_PARTITIONED_TABLE_DESCRIPTOR_PK + : DATA1_PARTITIONED_TABLE_DESCRIPTOR; + RpcMessageTestUtils.createTable(FLUSS_CLUSTER_EXTENSION, tablePath, tableDescriptor); + + Optional tableOpt = zkClient.getTable(tablePath); + assertThat(tableOpt).isPresent(); + TableRegistration table = tableOpt.get(); + // Partitioned table should NOT have remoteDataDir set at table level + assertThat(table.remoteDataDir).isNull(); + + // Create multiple partitions using partition column "b" + int partitionCount = 6; + List partitionNames = new ArrayList<>(); + for (int i = 0; i < partitionCount; i++) { + String partitionName = "p" + i; + partitionNames.add(partitionName); + PartitionSpec partitionSpec = + new PartitionSpec(Collections.singletonMap("b", partitionName)); + RpcMessageTestUtils.createPartition( + FLUSS_CLUSTER_EXTENSION, tablePath, partitionSpec, false); + } + + // Verify each partition has remoteDataDir set and round-robin is applied + Set usedRemoteDirs = new HashSet<>(); + for (String partitionName : partitionNames) { + Optional partitionOpt = + zkClient.getPartition(tablePath, partitionName); + assertThat(partitionOpt).isPresent(); + PartitionRegistration partition = partitionOpt.get(); + + assertThat(partition.getRemoteDataDir()).isNotNull(); + usedRemoteDirs.add(partition.getRemoteDataDir().toString()); + } + + // All configured dirs should be used + assertThat(usedRemoteDirs).hasSize(REMOTE_DIR_NAMES.size()); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testAutoPartitionWithMultipleRemoteDirs(boolean isPrimaryKeyTable) throws Exception { + // Create an auto-partitioned table and verify partitions use different remote dirs + String tablePrefix = isPrimaryKeyTable ? "auto_pk_partitioned_" : "auto_partitioned_"; + TablePath tablePath = TablePath.of("test_db", tablePrefix + "table"); + TableDescriptor tableDescriptor = + isPrimaryKeyTable + ? DATA1_PARTITIONED_TABLE_DESCRIPTOR_PK + : DATA1_PARTITIONED_TABLE_DESCRIPTOR; + + RpcMessageTestUtils.createTable(FLUSS_CLUSTER_EXTENSION, tablePath, tableDescriptor); + + // Wait for auto partitions to be created + Map partitions = + FLUSS_CLUSTER_EXTENSION.waitUntilPartitionAllReady(tablePath); + assertThat(partitions).isNotEmpty(); + + // Verify partitions use remote dirs + Set usedRemoteDirs = new HashSet<>(); + for (String partitionName : partitions.keySet()) { + Optional partitionOpt = + zkClient.getPartition(tablePath, partitionName); + assertThat(partitionOpt).isPresent(); + PartitionRegistration partition = partitionOpt.get(); + + assertThat(partition.getRemoteDataDir()).isNotNull(); + usedRemoteDirs.add(partition.getRemoteDataDir().toString()); + } + + // At least one remote dir should be used + assertThat(usedRemoteDirs).isNotEmpty(); + } + + @Test + void testMixedTableAndPartitionCreation() throws Exception { + // Create a mix of non-partitioned tables and partitioned table partitions + // to verify round-robin works correctly across both types + + // Create 2 non-partitioned tables + for (int i = 0; i < 2; i++) { + TablePath tablePath = TablePath.of("test_db", "mixed_non_pk_" + i); + RpcMessageTestUtils.createTable( + FLUSS_CLUSTER_EXTENSION, tablePath, DATA1_TABLE_DESCRIPTOR_PK); + } + + // Create a partitioned table + TablePath partitionedTablePath = TablePath.of("test_db", "mixed_partitioned"); + RpcMessageTestUtils.createTable( + FLUSS_CLUSTER_EXTENSION, partitionedTablePath, DATA1_PARTITIONED_TABLE_DESCRIPTOR); + + // Create partitions using partition column "b" + RpcMessageTestUtils.createPartition( + FLUSS_CLUSTER_EXTENSION, + partitionedTablePath, + new PartitionSpec(Collections.singletonMap("b", "p1")), + false); + RpcMessageTestUtils.createPartition( + FLUSS_CLUSTER_EXTENSION, + partitionedTablePath, + new PartitionSpec(Collections.singletonMap("b", "p2")), + false); + + // Create 2 more non-partitioned tables + for (int i = 2; i < 4; i++) { + TablePath tablePath = TablePath.of("test_db", "mixed_non_pk_" + i); + RpcMessageTestUtils.createTable( + FLUSS_CLUSTER_EXTENSION, tablePath, DATA1_TABLE_DESCRIPTOR_PK); + } + + // Collect all remote dirs used + Set allUsedDirs = new HashSet<>(); + + // Check non-partitioned tables + for (int i = 0; i < 4; i++) { + TablePath tablePath = TablePath.of("test_db", "mixed_non_pk_" + i); + Optional tableOpt = zkClient.getTable(tablePath); + assertThat(tableOpt).isPresent(); + assertThat(tableOpt.get().remoteDataDir).isNotNull(); + allUsedDirs.add(tableOpt.get().remoteDataDir.toString()); + } + + // Check partitions + for (String p : Arrays.asList("p1", "p2")) { + Optional partitionOpt = + zkClient.getPartition(partitionedTablePath, p); + assertThat(partitionOpt).isPresent(); + assertThat(partitionOpt.get().getRemoteDataDir()).isNotNull(); + allUsedDirs.add(partitionOpt.get().getRemoteDataDir().toString()); + } + + // All remote dirs should have been used (6 items, 3 dirs, round-robin) + assertThat(allUsedDirs).hasSize(REMOTE_DIR_NAMES.size()); + } + + @Test + void test() { + System.out.println(ConfigOptions.REMOTE_DATA_DIRS_STRATEGY.description()); + System.out.println(ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS.description()); + } +} 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..47bd48082e --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/RemoteDirDynamicLoaderTest.java @@ -0,0 +1,83 @@ +/* + * 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() throws Exception { + Configuration conf = createBaseConfiguration(); + conf.set(ConfigOptions.REMOTE_DATA_DIRS, Arrays.asList("hdfs://dir1", "hdfs://dir2")); + try (RemoteDirDynamicLoader loader = new RemoteDirDynamicLoader(conf)) { + RemoteDirDynamicLoader.RemoteDirContainer originalContainer = + loader.getRemoteDataDirContainer(); + + // 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() throws Exception { + Configuration conf = createBaseConfiguration(); + conf.set( + ConfigOptions.REMOTE_DATA_DIRS_STRATEGY, + ConfigOptions.RemoteDataDirStrategy.WEIGHTED_ROUND_ROBIN); + conf.set(ConfigOptions.REMOTE_DATA_DIRS, Arrays.asList("hdfs://dir1", "hdfs://dir2")); + conf.set(ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS, Arrays.asList(1, 2)); + + try (RemoteDirDynamicLoader loader = new RemoteDirDynamicLoader(conf)) { + RemoteDirDynamicLoader.RemoteDirContainer originalContainer = + loader.getRemoteDataDirContainer(); + + // 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..1f335e33ae --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/WeightedRoundRobinRemoteDirSelectorTest.java @@ -0,0 +1,270 @@ +/* + * 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); + } + + @Test + void testZeroWeights() { + // Test case 1: Some directories have zero weight - they should never be selected + List dirs = + Arrays.asList( + new FsPath("hdfs://cluster/data1"), + new FsPath("hdfs://cluster/data2"), + new FsPath("hdfs://cluster/data3")); + // weight 0 for data2 means it should never be selected + List weights = Arrays.asList(2, 0, 1); + + WeightedRoundRobinRemoteDirSelector selector = + new WeightedRoundRobinRemoteDirSelector(DEFAULT_DIR, dirs, weights); + + Map counts = new HashMap<>(); + int totalCalls = 30; // 10 complete cycles (total weight = 3) + + for (int i = 0; i < totalCalls; i++) { + FsPath selected = selector.nextDataDir(); + counts.merge(selected, 1, Integer::sum); + } + + // data1 should be selected 20 times (2/3 * 30) + assertThat(counts.get(dirs.get(0))).isEqualTo(20); + // data2 should never be selected (weight = 0) + assertThat(counts.get(dirs.get(1))).isNull(); + // data3 should be selected 10 times (1/3 * 30) + assertThat(counts.get(dirs.get(2))).isEqualTo(10); + + // Test case 2: All weights are zero - should always return default dir + List allZeroWeights = Arrays.asList(0, 0, 0); + WeightedRoundRobinRemoteDirSelector allZeroSelector = + new WeightedRoundRobinRemoteDirSelector(DEFAULT_DIR, dirs, allZeroWeights); + + assertThat(allZeroSelector.nextDataDir()).isEqualTo(DEFAULT_DIR); + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachineTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachineTest.java index bafb477c54..feb7c4eded 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.coordinator.statemachine.ReplicaLeaderElection.ControlledShutdownLeaderElection; import org.apache.fluss.server.coordinator.statemachine.TableBucketStateMachine.ElectionResult; import org.apache.fluss.server.metadata.CoordinatorMetadataCache; @@ -117,6 +118,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/kv/snapshot/KvSnapshotMultipleDirsITCase.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/snapshot/KvSnapshotMultipleDirsITCase.java new file mode 100644 index 0000000000..6c16e41dda --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/kv/snapshot/KvSnapshotMultipleDirsITCase.java @@ -0,0 +1,267 @@ +/* + * 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.kv.snapshot; + +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.fs.FileStatus; +import org.apache.fluss.fs.FsPath; +import org.apache.fluss.metadata.PartitionSpec; +import org.apache.fluss.metadata.PhysicalTablePath; +import org.apache.fluss.metadata.Schema; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TableDescriptor; +import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.record.KvRecordBatch; +import org.apache.fluss.rpc.gateway.TabletServerGateway; +import org.apache.fluss.rpc.messages.PutKvRequest; +import org.apache.fluss.server.testutils.FlussClusterExtension; +import org.apache.fluss.server.testutils.RpcMessageTestUtils; +import org.apache.fluss.server.zk.ZooKeeperClient; +import org.apache.fluss.server.zk.data.PartitionRegistration; +import org.apache.fluss.server.zk.data.TableRegistration; +import org.apache.fluss.types.DataTypes; +import org.apache.fluss.utils.FlussPaths; +import org.apache.fluss.utils.types.Tuple2; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import static org.apache.fluss.record.TestData.DATA1_SCHEMA_PK; +import static org.apache.fluss.server.testutils.RpcMessageTestUtils.createPartition; +import static org.apache.fluss.server.testutils.RpcMessageTestUtils.createTable; +import static org.apache.fluss.server.testutils.RpcMessageTestUtils.newPutKvRequest; +import static org.apache.fluss.testutils.DataTestUtils.genKvRecordBatch; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * ITCase for verifying KV snapshot can be uploaded to multiple remote directories with round-robin + * distribution. + */ +public class KvSnapshotMultipleDirsITCase { + + private static final List REMOTE_DIR_NAMES = Arrays.asList("dir1", "dir2", "dir3"); + + @RegisterExtension + public static final FlussClusterExtension FLUSS_CLUSTER_EXTENSION = + FlussClusterExtension.builder() + .setNumOfTabletServers(3) + .setClusterConf(initConfig()) + .setRemoteDirNames(REMOTE_DIR_NAMES) + .build(); + + private ZooKeeperClient zkClient; + + @BeforeEach + void setup() { + zkClient = FLUSS_CLUSTER_EXTENSION.getZooKeeperClient(); + } + + @Test + void testKvSnapshotToMultipleDirsForNonPartitionedTable() throws Exception { + // Create multiple tables (more than number of remote dirs) to ensure round-robin + // distribution. Each table's KV snapshot should be uploaded to different remote dirs. + int tableCount = 6; + List tablePaths = new ArrayList<>(); + List tableIds = new ArrayList<>(); + + // Create tables with primary key (KV tables) + TableDescriptor tableDescriptor = + TableDescriptor.builder() + .schema(DATA1_SCHEMA_PK) + .distributedBy(1) // Single bucket for simpler verification + .build(); + + for (int i = 0; i < tableCount; i++) { + TablePath tablePath = TablePath.of("test_db", String.format("kv_snapshot_table_%d", i)); + tablePaths.add(tablePath); + long tableId = + RpcMessageTestUtils.createTable( + FLUSS_CLUSTER_EXTENSION, tablePath, tableDescriptor); + tableIds.add(tableId); + } + + // Write data to each table to trigger KV snapshot + for (int t = 0; t < tableCount; t++) { + TableBucket tb = new TableBucket(tableIds.get(t), 0); + FLUSS_CLUSTER_EXTENSION.waitUntilAllReplicaReady(tb); + + int leaderId = FLUSS_CLUSTER_EXTENSION.waitAndGetLeader(tb); + TabletServerGateway leaderGateway = + FLUSS_CLUSTER_EXTENSION.newTabletServerClientForNode(leaderId); + + // Write KV data to trigger snapshot + KvRecordBatch kvRecordBatch = + genKvRecordBatch( + Tuple2.of("k1", new Object[] {1, "v1"}), + Tuple2.of("k2", new Object[] {2, "v2"})); + + PutKvRequest putKvRequest = newPutKvRequest(tableIds.get(t), 0, 1, kvRecordBatch); + leaderGateway.putKv(putKvRequest).get(); + } + + // Wait for all tables' snapshots to be completed + for (int t = 0; t < tableCount; t++) { + TableBucket tb = new TableBucket(tableIds.get(t), 0); + // Wait for snapshot 0 to be finished + FLUSS_CLUSTER_EXTENSION.waitUntilSnapshotFinished(tb, 0); + } + + // Collect the remote data directories used by each table + Set usedRemoteDataDirs = new HashSet<>(); + for (int t = 0; t < tableCount; t++) { + Optional tableOpt = zkClient.getTable(tablePaths.get(t)); + assertThat(tableOpt).isPresent(); + TableRegistration table = tableOpt.get(); + + assertThat(table.remoteDataDir).isNotNull(); + usedRemoteDataDirs.add(table.remoteDataDir.toString()); + + // Verify the remote KV snapshot files actually exist + FsPath remoteKvDir = FlussPaths.remoteKvDir(table.remoteDataDir); + TableBucket tb = new TableBucket(tableIds.get(t), 0); + FsPath remoteKvTabletDir = + FlussPaths.remoteKvTabletDir( + remoteKvDir, PhysicalTablePath.of(tablePaths.get(t)), tb); + assertThat(remoteKvTabletDir.getFileSystem().exists(remoteKvTabletDir)).isTrue(); + FileStatus[] fileStatuses = + remoteKvTabletDir.getFileSystem().listStatus(remoteKvTabletDir); + assertThat(fileStatuses).isNotEmpty(); + } + + // All configured remote dirs should be used due to round-robin distribution + assertThat(usedRemoteDataDirs).hasSameSizeAs(REMOTE_DIR_NAMES); + } + + @Test + void testKvSnapshotToMultipleDirsForPartitionedTable() throws Exception { + // Create a partitioned table and add multiple partitions (more than number of remote dirs) + // to ensure round-robin distribution. Each partition's KV snapshot should be uploaded to + // different remote dirs. + int partitionCount = 6; + TablePath tablePath = TablePath.of("test_db", "partitioned_kv_snapshot_table"); + + // Create partitioned table with primary key + Schema schema = + Schema.newBuilder() + .column("a", DataTypes.INT()) + .column("b", DataTypes.STRING()) + .primaryKey("a", "b") + .build(); + + TableDescriptor tableDescriptor = + TableDescriptor.builder() + .schema(schema) + .distributedBy(1) // Single bucket for simpler verification + .partitionedBy("b") + .build(); + + long tableId = createTable(FLUSS_CLUSTER_EXTENSION, tablePath, tableDescriptor); + + // Create partitions + List partitionNames = new ArrayList<>(); + for (int i = 0; i < partitionCount; i++) { + String partitionName = "p" + i; + partitionNames.add(partitionName); + PartitionSpec partitionSpec = + new PartitionSpec(Collections.singletonMap("b", partitionName)); + createPartition(FLUSS_CLUSTER_EXTENSION, tablePath, partitionSpec, false); + } + + // Get partition IDs from ZK + Map partitionNameToId = zkClient.getPartitionNameAndIds(tablePath); + assertThat(partitionNameToId).hasSize(partitionCount); + + // Wait for all partitions to be ready and write data to trigger KV snapshot + for (int p = 0; p < partitionCount; p++) { + String partitionName = partitionNames.get(p); + Long partitionId = partitionNameToId.get(partitionName); + TableBucket tb = new TableBucket(tableId, partitionId, 0); + FLUSS_CLUSTER_EXTENSION.waitUntilAllReplicaReady(tb); + + int leaderId = FLUSS_CLUSTER_EXTENSION.waitAndGetLeader(tb); + TabletServerGateway leaderGateway = + FLUSS_CLUSTER_EXTENSION.newTabletServerClientForNode(leaderId); + + // Write KV data to trigger snapshot + KvRecordBatch kvRecordBatch = + genKvRecordBatch( + Tuple2.of("k1", new Object[] {1, "v1"}), + Tuple2.of("k2", new Object[] {2, "v2"})); + + PutKvRequest putKvRequest = newPutKvRequest(tableId, partitionId, 0, 1, kvRecordBatch); + leaderGateway.putKv(putKvRequest).get(); + } + + // Wait for all partitions' snapshots to be completed + for (int p = 0; p < partitionCount; p++) { + Long partitionId = partitionNameToId.get(partitionNames.get(p)); + TableBucket tb = new TableBucket(tableId, partitionId, 0); + // Wait for snapshot 0 to be finished + FLUSS_CLUSTER_EXTENSION.waitUntilSnapshotFinished(tb, 0); + } + + // Collect the remote data directories used by each partition + Set usedRemoteDataDirs = new HashSet<>(); + for (int p = 0; p < partitionCount; p++) { + String partitionName = partitionNames.get(p); + Long partitionId = partitionNameToId.get(partitionName); + Optional partitionOpt = + zkClient.getPartition(tablePath, partitionName); + assertThat(partitionOpt).isPresent(); + PartitionRegistration partition = partitionOpt.get(); + + assertThat(partition.getRemoteDataDir()).isNotNull(); + usedRemoteDataDirs.add(partition.getRemoteDataDir().toString()); + + // Verify the remote KV snapshot files actually exist + FsPath remoteKvDir = FlussPaths.remoteKvDir(partition.getRemoteDataDir()); + TableBucket tb = new TableBucket(tableId, partitionId, 0); + FsPath remoteKvTabletDir = + FlussPaths.remoteKvTabletDir( + remoteKvDir, PhysicalTablePath.of(tablePath, partitionName), tb); + assertThat(remoteKvTabletDir.getFileSystem().exists(remoteKvTabletDir)).isTrue(); + FileStatus[] fileStatuses = + remoteKvTabletDir.getFileSystem().listStatus(remoteKvTabletDir); + assertThat(fileStatuses).isNotEmpty(); + } + + // All configured remote dirs should be used due to round-robin distribution + assertThat(usedRemoteDataDirs).hasSameSizeAs(REMOTE_DIR_NAMES); + } + + private static Configuration initConfig() { + Configuration conf = new Configuration(); + conf.setInt(ConfigOptions.DEFAULT_BUCKET_NUMBER, 1); + conf.setInt(ConfigOptions.DEFAULT_REPLICATION_FACTOR, 3); + // Set a shorter interval for testing purpose + conf.set(ConfigOptions.KV_SNAPSHOT_INTERVAL, Duration.ofSeconds(1)); + return conf; + } +} 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..cae6ca0b51 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 @@ -20,10 +20,15 @@ import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; import org.apache.fluss.config.MemorySize; +import org.apache.fluss.fs.FileStatus; import org.apache.fluss.fs.FileSystem; import org.apache.fluss.fs.FsPath; +import org.apache.fluss.metadata.PartitionSpec; import org.apache.fluss.metadata.PhysicalTablePath; +import org.apache.fluss.metadata.Schema; import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TableDescriptor; +import org.apache.fluss.metadata.TablePath; import org.apache.fluss.rpc.entity.FetchLogResultForBucket; import org.apache.fluss.rpc.gateway.CoordinatorGateway; import org.apache.fluss.rpc.gateway.TabletServerGateway; @@ -32,22 +37,36 @@ import org.apache.fluss.server.log.FetchParams; import org.apache.fluss.server.tablet.TabletServer; import org.apache.fluss.server.testutils.FlussClusterExtension; +import org.apache.fluss.server.testutils.RpcMessageTestUtils; +import org.apache.fluss.server.zk.ZooKeeperClient; +import org.apache.fluss.server.zk.data.PartitionRegistration; +import org.apache.fluss.server.zk.data.TableRegistration; import org.apache.fluss.utils.FlussPaths; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; +import java.util.HashSet; +import java.util.List; import java.util.Map; +import java.util.Optional; +import java.util.Set; import java.util.concurrent.CompletableFuture; import static org.apache.fluss.record.TestData.DATA1; +import static org.apache.fluss.record.TestData.DATA1_SCHEMA; +import static org.apache.fluss.record.TestData.DATA1_SCHEMA_PK; import static org.apache.fluss.record.TestData.DATA1_TABLE_DESCRIPTOR; import static org.apache.fluss.record.TestData.DATA1_TABLE_PATH; import static org.apache.fluss.server.testutils.RpcMessageTestUtils.assertProduceLogResponse; +import static org.apache.fluss.server.testutils.RpcMessageTestUtils.createPartition; import static org.apache.fluss.server.testutils.RpcMessageTestUtils.createTable; import static org.apache.fluss.server.testutils.RpcMessageTestUtils.newDropTableRequest; import static org.apache.fluss.server.testutils.RpcMessageTestUtils.newProduceLogRequest; @@ -58,13 +77,24 @@ /** ITCase for remote log. */ public class RemoteLogITCase { + + private static final List REMOTE_DIR_NAMES = Arrays.asList("dir1", "dir2", "dir3"); + @RegisterExtension public static final FlussClusterExtension FLUSS_CLUSTER_EXTENSION = FlussClusterExtension.builder() .setNumOfTabletServers(3) .setClusterConf(initConfig()) + .setRemoteDirNames(REMOTE_DIR_NAMES) .build(); + private ZooKeeperClient zkClient; + + @BeforeEach + void setup() { + zkClient = FLUSS_CLUSTER_EXTENSION.getZooKeeperClient(); + } + private TableBucket setupTableBucket() throws Exception { long tableId = createTable(FLUSS_CLUSTER_EXTENSION, DATA1_TABLE_PATH, DATA1_TABLE_DESCRIPTOR); @@ -117,9 +147,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); @@ -202,6 +230,204 @@ void testFollowerFetchAlreadyMoveToRemoteLog(boolean withWriterId) throws Except FLUSS_CLUSTER_EXTENSION.waitUntilReplicaExpandToIsr(tb, follower); } + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testRemoteLogTieredToMultipleDirsForNonPartitionedTable(boolean isPrimaryTable) + throws Exception { + // Create multiple tables (more than number of remote dirs) to ensure round-robin + // distribution. Each table's remote log should be uploaded to different remote dirs. + int tableCount = 6; + List tablePaths = new ArrayList<>(); + List tableIds = new ArrayList<>(); + + // Create tables + Schema schema = isPrimaryTable ? DATA1_SCHEMA_PK : DATA1_SCHEMA; + TableDescriptor tableDescriptor = + TableDescriptor.builder() + .schema(schema) + .distributedBy(1) // Single bucket for simpler verification + .build(); + + for (int i = 0; i < tableCount; i++) { + TablePath tablePath = + TablePath.of( + "test_db", + String.format("remote_%s_table_%d", isPrimaryTable ? "kv" : "log", i)); + tablePaths.add(tablePath); + long tableId = + RpcMessageTestUtils.createTable( + FLUSS_CLUSTER_EXTENSION, tablePath, tableDescriptor); + tableIds.add(tableId); + } + + // Write data to each table to trigger segment rollover and remote log copy + for (int t = 0; t < tableCount; t++) { + TableBucket tb = new TableBucket(tableIds.get(t), 0); + FLUSS_CLUSTER_EXTENSION.waitUntilAllReplicaReady(tb); + + int leaderId = FLUSS_CLUSTER_EXTENSION.waitAndGetLeader(tb); + TabletServerGateway leaderGateway = + FLUSS_CLUSTER_EXTENSION.newTabletServerClientForNode(leaderId); + + // Write enough data to create multiple segments (segment size is 1kb) + int batchCount = 10; + for (int i = 0; i < batchCount; i++) { + assertProduceLogResponse( + leaderGateway + .produceLog( + newProduceLogRequest( + tableIds.get(t), + 0, + 1, + genMemoryLogRecordsByObject(DATA1))) + .get(), + 0, + (long) i * DATA1.size()); + } + } + + // Wait for all tables' log segments to be copied to remote + for (int t = 0; t < tableCount; t++) { + TableBucket tb = new TableBucket(tableIds.get(t), 0); + FLUSS_CLUSTER_EXTENSION.waitUntilSomeLogSegmentsCopyToRemote(tb); + } + + // Collect the remote data directories used by each table + // The remote log dir is derived from table's remoteDataDir: {remoteDataDir}/log + Set usedRemoteDataDirs = new HashSet<>(); + for (int t = 0; t < tableCount; t++) { + Optional tableOpt = zkClient.getTable(tablePaths.get(t)); + assertThat(tableOpt).isPresent(); + TableRegistration table = tableOpt.get(); + + assertThat(table.remoteDataDir).isNotNull(); + usedRemoteDataDirs.add(table.remoteDataDir.toString()); + + // Verify the remote log files actually exist + FsPath remoteLogDir = FlussPaths.remoteLogDir(table.remoteDataDir); + TableBucket tb = new TableBucket(tableIds.get(t), 0); + FsPath remoteLogTabletDir = + FlussPaths.remoteLogTabletDir( + remoteLogDir, PhysicalTablePath.of(tablePaths.get(t)), tb); + assertThat(remoteLogTabletDir.getFileSystem().exists(remoteLogTabletDir)).isTrue(); + FileStatus[] fileStatuses = + remoteLogTabletDir.getFileSystem().listStatus(remoteLogTabletDir); + assertThat(fileStatuses).isNotEmpty(); + } + + assertThat(usedRemoteDataDirs).hasSameSizeAs(REMOTE_DIR_NAMES); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testRemoteLogTieredToMultipleDirsForPartitionedTable(boolean isPrimaryTable) + throws Exception { + // Create a partitioned table and add multiple partitions (more than number of remote dirs) + // to ensure round-robin distribution. Each partition's remote log should be uploaded to + // different remote dirs. + int partitionCount = 6; + String tablePrefix = isPrimaryTable ? "partitioned_pk_" : "partitioned_log_"; + TablePath tablePath = TablePath.of("test_db", tablePrefix + "remote_table"); + + // Create partitioned table + Schema.Builder schemaBuilder = + Schema.newBuilder() + .column("a", org.apache.fluss.types.DataTypes.INT()) + .column("b", org.apache.fluss.types.DataTypes.STRING()) + .column("c", org.apache.fluss.types.DataTypes.STRING()); + if (isPrimaryTable) { + schemaBuilder.primaryKey("a", "c"); + } + Schema schema = schemaBuilder.build(); + + TableDescriptor tableDescriptor = + TableDescriptor.builder() + .schema(schema) + .distributedBy(1) // Single bucket for simpler verification + .partitionedBy("c") + .build(); + + long tableId = createTable(FLUSS_CLUSTER_EXTENSION, tablePath, tableDescriptor); + + // Create partitions + List partitionNames = new ArrayList<>(); + for (int i = 0; i < partitionCount; i++) { + String partitionName = "p" + i; + partitionNames.add(partitionName); + PartitionSpec partitionSpec = + new PartitionSpec(Collections.singletonMap("c", partitionName)); + createPartition(FLUSS_CLUSTER_EXTENSION, tablePath, partitionSpec, false); + } + + // Get partition IDs from ZK + Map partitionNameToId = zkClient.getPartitionNameAndIds(tablePath); + assertThat(partitionNameToId).hasSize(partitionCount); + + // Wait for all partitions to be ready and write data to trigger segment rollover + for (int p = 0; p < partitionCount; p++) { + String partitionName = partitionNames.get(p); + Long partitionId = partitionNameToId.get(partitionName); + TableBucket tb = new TableBucket(tableId, partitionId, 0); + FLUSS_CLUSTER_EXTENSION.waitUntilAllReplicaReady(tb); + + int leaderId = FLUSS_CLUSTER_EXTENSION.waitAndGetLeader(tb); + TabletServerGateway leaderGateway = + FLUSS_CLUSTER_EXTENSION.newTabletServerClientForNode(leaderId); + + // Write enough data to create multiple segments (segment size is 1kb) + int batchCount = 10; + for (int i = 0; i < batchCount; i++) { + assertProduceLogResponse( + leaderGateway + .produceLog( + newProduceLogRequest( + tableId, + partitionId, + 0, + 1, + genMemoryLogRecordsByObject(DATA1))) + .get(), + 0, + (long) i * DATA1.size()); + } + } + + // Wait for all partitions' log segments to be copied to remote + for (int p = 0; p < partitionCount; p++) { + Long partitionId = partitionNameToId.get(partitionNames.get(p)); + TableBucket tb = new TableBucket(tableId, partitionId, 0); + FLUSS_CLUSTER_EXTENSION.waitUntilSomeLogSegmentsCopyToRemote(tb); + } + + // Collect the remote data directories used by each partition + Set usedRemoteDataDirs = new HashSet<>(); + for (int p = 0; p < partitionCount; p++) { + String partitionName = partitionNames.get(p); + Long partitionId = partitionNameToId.get(partitionName); + Optional partitionOpt = + zkClient.getPartition(tablePath, partitionName); + assertThat(partitionOpt).isPresent(); + PartitionRegistration partition = partitionOpt.get(); + + assertThat(partition.getRemoteDataDir()).isNotNull(); + usedRemoteDataDirs.add(partition.getRemoteDataDir().toString()); + + // Verify the remote log files actually exist + FsPath remoteLogDir = FlussPaths.remoteLogDir(partition.getRemoteDataDir()); + TableBucket tb = new TableBucket(tableId, partitionId, 0); + FsPath remoteLogTabletDir = + FlussPaths.remoteLogTabletDir( + remoteLogDir, PhysicalTablePath.of(tablePath, partitionName), tb); + assertThat(remoteLogTabletDir.getFileSystem().exists(remoteLogTabletDir)).isTrue(); + FileStatus[] fileStatuses = + remoteLogTabletDir.getFileSystem().listStatus(remoteLogTabletDir); + assertThat(fileStatuses).isNotEmpty(); + } + + // All configured remote dirs should be used due to round-robin distribution + assertThat(usedRemoteDataDirs).hasSameSizeAs(REMOTE_DIR_NAMES); + } + private static Configuration initConfig() { Configuration conf = new Configuration(); conf.setInt(ConfigOptions.DEFAULT_BUCKET_NUMBER, 1); 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/log/remote/TestingRemoteLogStorage.java b/fluss-server/src/test/java/org/apache/fluss/server/log/remote/TestingRemoteLogStorage.java index 75802dc028..2fd3a9cce3 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/log/remote/TestingRemoteLogStorage.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/log/remote/TestingRemoteLogStorage.java @@ -21,7 +21,6 @@ import org.apache.fluss.exception.RemoteStorageException; import org.apache.fluss.fs.FsPath; -import java.io.IOException; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicBoolean; @@ -33,8 +32,7 @@ public class TestingRemoteLogStorage extends DefaultRemoteLogStorage { public final AtomicBoolean writeManifestFail = new AtomicBoolean(false); - public TestingRemoteLogStorage(Configuration conf, ExecutorService ioExecutor) - throws IOException { + public TestingRemoteLogStorage(Configuration conf, ExecutorService ioExecutor) { super(conf, ioExecutor); } 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/KvReplicaRestoreITCase.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/KvReplicaRestoreITCase.java index 932c17f238..1bfb03092d 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/KvReplicaRestoreITCase.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/KvReplicaRestoreITCase.java @@ -32,6 +32,8 @@ import org.apache.fluss.rpc.messages.PutKvRequest; import org.apache.fluss.server.kv.snapshot.ZooKeeperCompletedSnapshotHandleStore; import org.apache.fluss.server.testutils.FlussClusterExtension; +import org.apache.fluss.server.zk.ZooKeeperClient; +import org.apache.fluss.server.zk.data.TableRegistration; import org.apache.fluss.utils.ExceptionUtils; import org.apache.fluss.utils.FileUtils; import org.apache.fluss.utils.types.Tuple2; @@ -42,7 +44,9 @@ import java.time.Duration; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; +import java.util.Optional; import java.util.Random; import java.util.concurrent.atomic.AtomicInteger; @@ -56,21 +60,27 @@ import static org.apache.fluss.testutils.DataTestUtils.getKeyValuePairs; import static org.apache.fluss.testutils.DataTestUtils.toKvRecordBatch; import static org.apache.fluss.testutils.common.CommonTestUtils.waitUntil; +import static org.assertj.core.api.Assertions.assertThat; /** The IT case for the restoring of kv replica. */ class KvReplicaRestoreITCase { + private static final List REMOTE_DIR_NAMES = Arrays.asList("dir1", "dir2", "dir3"); + @RegisterExtension public static final FlussClusterExtension FLUSS_CLUSTER_EXTENSION = FlussClusterExtension.builder() .setNumOfTabletServers(3) .setClusterConf(initConfig()) + .setRemoteDirNames(REMOTE_DIR_NAMES) .build(); + private ZooKeeperClient zkClient; private ZooKeeperCompletedSnapshotHandleStore completedSnapshotHandleStore; @BeforeEach void beforeEach() { + zkClient = FLUSS_CLUSTER_EXTENSION.getZooKeeperClient(); completedSnapshotHandleStore = new ZooKeeperCompletedSnapshotHandleStore( FLUSS_CLUSTER_EXTENSION.getZooKeeperClient()); @@ -94,6 +104,13 @@ void testRestore() throws Exception { for (int bucket = 0; bucket < bucketNum; bucket++) { tableBuckets.add(new TableBucket(tableId, bucket)); } + + Optional tableOpt = zkClient.getTable(tablePath); + assertThat(tableOpt).isPresent(); + TableRegistration table = tableOpt.get(); + + assertThat(table.remoteDataDir).isNotNull(); + assertThat(REMOTE_DIR_NAMES).contains(table.remoteDataDir.getName()); } for (TableBucket tableBucket : tableBuckets) { 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/testutils/FlussClusterExtension.java b/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java index b63138ea8c..1b71afb572 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java @@ -131,6 +131,7 @@ public final class FlussClusterExtension private final Map tabletServerInfos; private final Configuration clusterConf; private final Clock clock; + private final List remoteDirNames; /** Creates a new {@link Builder} for {@link FlussClusterExtension}. */ public static Builder builder() { @@ -142,7 +143,8 @@ private FlussClusterExtension( String coordinatorServerListeners, String tabletServerListeners, Configuration clusterConf, - Clock clock) { + Clock clock, + List remoteDirNames) { this.initialNumOfTabletServers = numOfTabletServers; this.tabletServers = new HashMap<>(numOfTabletServers); this.coordinatorServerListeners = coordinatorServerListeners; @@ -150,6 +152,7 @@ private FlussClusterExtension( this.tabletServerInfos = new HashMap<>(); this.clusterConf = clusterConf; this.clock = clock; + this.remoteDirNames = remoteDirNames; } @Override @@ -258,6 +261,7 @@ public void startCoordinatorServer() throws Exception { conf.setString(ConfigOptions.ZOOKEEPER_ADDRESS, zooKeeperServer.getConnectString()); conf.setString(ConfigOptions.BIND_LISTENERS, coordinatorServerListeners); setRemoteDataDir(conf); + setRemoteDataDirs(conf); coordinatorServer = new CoordinatorServer(conf); coordinatorServer.start(); coordinatorServerInfo = @@ -361,12 +365,26 @@ private void setRemoteDataDir(Configuration conf) { conf.set(ConfigOptions.REMOTE_DATA_DIR, getRemoteDataDir()); } + private void setRemoteDataDirs(Configuration conf) { + if (!remoteDirNames.isEmpty()) { + List remoteDataDirs = + remoteDirNames.stream() + .map(this::getRemoteDataDir) + .collect(Collectors.toList()); + conf.set(ConfigOptions.REMOTE_DATA_DIRS, remoteDataDirs); + } + } + public String getRemoteDataDir() { + return getRemoteDataDir("remote-data-dir"); + } + + public String getRemoteDataDir(String dirName) { return LocalFileSystem.getLocalFsURI().getScheme() + "://" + tempDir.getAbsolutePath() + File.separator - + "remote-data-dir"; + + dirName; } /** Stop a tablet server. */ @@ -846,6 +864,7 @@ public static class Builder { private String tabletServerListeners = DEFAULT_LISTENERS; private String coordinatorServerListeners = DEFAULT_LISTENERS; private Clock clock = SystemClock.getInstance(); + private List remoteDirNames = Collections.emptyList(); private final Configuration clusterConf = new Configuration(); @@ -885,13 +904,19 @@ public Builder setClock(Clock clock) { return this; } + public Builder setRemoteDirNames(List remoteDirNames) { + this.remoteDirNames = remoteDirNames; + return this; + } + public FlussClusterExtension build() { return new FlussClusterExtension( numOfTabletServers, coordinatorServerListeners, tabletServerListeners, clusterConf, - clock); + clock, + remoteDirNames); } } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/testutils/RpcMessageTestUtils.java b/fluss-server/src/test/java/org/apache/fluss/server/testutils/RpcMessageTestUtils.java index b567eff1f7..7a55954ac5 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/testutils/RpcMessageTestUtils.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/testutils/RpcMessageTestUtils.java @@ -175,6 +175,11 @@ public static MetadataRequest newMetadataRequest(List tablePaths) { public static ProduceLogRequest newProduceLogRequest( long tableId, int bucketId, int acks, MemoryLogRecords records) { + return newProduceLogRequest(tableId, null, bucketId, acks, records); + } + + public static ProduceLogRequest newProduceLogRequest( + long tableId, Long partitionId, int bucketId, int acks, MemoryLogRecords records) { ProduceLogRequest produceRequest = new ProduceLogRequest(); produceRequest.setTableId(tableId).setAcks(acks).setTimeoutMs(10000); PbProduceLogReqForBucket pbProduceLogReqForBucket = new PbProduceLogReqForBucket(); @@ -185,16 +190,27 @@ public static ProduceLogRequest newProduceLogRequest( records.getMemorySegment(), records.getPosition(), records.sizeInBytes())); + if (partitionId != null) { + pbProduceLogReqForBucket.setPartitionId(partitionId); + } produceRequest.addAllBucketsReqs(Collections.singletonList(pbProduceLogReqForBucket)); return produceRequest; } public static PutKvRequest newPutKvRequest( long tableId, int bucketId, int acks, KvRecordBatch kvRecordBatch) { + return newPutKvRequest(tableId, null, bucketId, acks, kvRecordBatch); + } + + public static PutKvRequest newPutKvRequest( + long tableId, Long partitionId, int bucketId, int acks, KvRecordBatch kvRecordBatch) { PutKvRequest putKvRequest = new PutKvRequest(); putKvRequest.setTableId(tableId).setAcks(acks).setTimeoutMs(10000); PbPutKvReqForBucket pbPutKvReqForBucket = new PbPutKvReqForBucket(); pbPutKvReqForBucket.setBucketId(bucketId); + if (partitionId != null) { + pbPutKvReqForBucket.setPartitionId(partitionId); + } if (kvRecordBatch instanceof DefaultKvRecordBatch) { DefaultKvRecordBatch batch = (DefaultKvRecordBatch) kvRecordBatch; pbPutKvReqForBucket.setRecords( 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 f6caffd19d..14bd869603 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.RebalanceTask; 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); diff --git a/website/docs/maintenance/configuration.md b/website/docs/maintenance/configuration.md index 97da3457d0..3b2fc422e9 100644 --- a/website/docs/maintenance/configuration.md +++ b/website/docs/maintenance/configuration.md @@ -26,29 +26,32 @@ during the Fluss cluster working. ## Common -| Option | Type | Default | Description | -|-----------------------------------------------------|--------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| bind.listeners | String | (None) | The network address and port to which the server binds for accepting connections. This defines the interface and port where the server will listen for incoming requests. The format is `{listener_name}://{host}:{port}`, and multiple addresses can be specified, separated by commas. Use `0.0.0.0` for the `host` to bind to all available interfaces which is dangerous on production and not suggested for production usage. The `listener_name` serves as an identifier for the address in the configuration. For example, `internal.listener.name` specifies the address used for internal server communication. If multiple addresses are configured, ensure that the `listener_name` values are unique. | -| advertised.listeners | String | (None) | The externally advertised address and port for client connections. Required in distributed environments when the bind address is not publicly reachable. Format matches `bind.listeners` (`{listener_name}://{host}:{port}`). Defaults to the value of `bind.listeners` if not explicitly configured. | -| internal.listener.name | String | FLUSS | The listener for server internal communication. | -| security.protocol.map | Map | (none) | A map defining the authentication protocol for each listener. The format is `listenerName1:protocol1,listenerName2:protocol2`, e.g., `INTERNAL:PLAINTEXT,CLIENT:GSSAPI`. Each listener can be associated with a specific authentication protocol. Listeners not included in the map will use PLAINTEXT by default, which does not require authentication. | -| `security.${protocol}.*` | String | (none) | Protocol-specific configuration properties. For example, security.sasl.jaas.config for SASL authentication settings. | -| default.bucket.number | Integer | 1 | The default number of buckets for a table in Fluss cluster. It's a cluster-level parameter and all the tables without specifying bucket number in the cluster will use the value as the bucket number. | -| default.replication.factor | Integer | 1 | The default replication factor for the log of a table in Fluss cluster. It's a cluster-level parameter, and all the tables without specifying replication factor in the cluster will use the value as replication factor. | -| remote.data.dir | String | (None) | The directory used for storing the kv snapshot data files and remote log for log tiered storage in a Fluss supported filesystem. | -| remote.fs.write-buffer-size | MemorySize | 4kb | The default size of the write buffer for writing the local files to remote file systems. | -| plugin.classloader.parent-first-patterns.additional | List<String> | (None) | A (semicolon-separated) list of patterns that specifies which classes should always be resolved through the plugin parent ClassLoader first. A pattern is a simple prefix that is checked against the fully qualified class name. These patterns are appended to `classloader.parent-first-patterns.default`. | -| plugin.classloader.parent-first-patterns.default | String | java.,
org.apache.fluss.,
javax.annotation.,
org.slf4j,
org.apache.log4j,
org.apache.logging,
org.apache.commons.logging,
ch.qos.logback | A (semicolon-separated) list of patterns that specifies which classes should always be resolved through the plugin parent ClassLoader first. A pattern is a simple prefix that is checked against the fully qualified class name. This setting should generally not be modified. | -| auto-partition.check.interval | Duration | 10min | The interval of auto partition check. The default value is 10 minutes. | -| allow.create.log.tables | Boolean | true | Whether to allow creation of log tables. When set to false, attempts to create log tables (tables without primary key) will be rejected. The default value is true. | -| allow.create.kv.tables | Boolean | true | Whether to allow creation of kv tables (primary key tables). When set to false, attempts to create kv tables (tables with primary key) will be rejected. The default value is true. | -| max.partition.num | Integer | 1000 | Limits the maximum number of partitions that can be created for a partitioned table to avoid creating too many partitions. | -| max.bucket.num | Integer | 128000 | The maximum number of buckets that can be created for a table. The default value is 128000. | -| acl.notification.expiration-time | Duration | 15min | The duration for which ACL notifications are valid before they expire. This configuration determines the time window during which an ACL notification is considered active. After this duration, the notification will no longer be valid and will be discarded. The default value is 15 minutes. This setting is important to ensure that ACL changes are propagated in a timely manner and do not remain active longer than necessary. | -| authorizer.enabled | Boolean | false | Specifies whether to enable the authorization feature. If enabled, access control is enforced based on the authorization rules defined in the configuration. If disabled, all operations and resources are accessible to all users. | -| authorizer.type | String | default | Specifies the type of authorizer to be used for access control. This value corresponds to the identifier of the authorization plugin. The default value is `default`, which indicates the built-in authorizer implementation. Custom authorizers can be implemented by providing a matching plugin identifier. | -| super.users | String | (None) | A semicolon-separated list of superusers who have unrestricted access to all operations and resources. Note that the delimiter is semicolon since SSL user names may contain comma, and each super user should be specified in the format `principal_type:principal_name`, e.g., `User:admin;User:bob`. This configuration is critical for defining administrative privileges in the system. | -| server.io-pool.size | Integer | 10 | The size of the IO thread pool to run blocking operations for both coordinator and tablet servers. This includes discard unnecessary snapshot files, transfer kv snapshot files, and transfer remote log files. Increase this value if you experience slow IO operations. The default value is 10. | +| Option | Type | Default | Description | +|-----------------------------------------------------|---------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| bind.listeners | String | (None) | The network address and port to which the server binds for accepting connections. This defines the interface and port where the server will listen for incoming requests. The format is `{listener_name}://{host}:{port}`, and multiple addresses can be specified, separated by commas. Use `0.0.0.0` for the `host` to bind to all available interfaces which is dangerous on production and not suggested for production usage. The `listener_name` serves as an identifier for the address in the configuration. For example, `internal.listener.name` specifies the address used for internal server communication. If multiple addresses are configured, ensure that the `listener_name` values are unique. | +| advertised.listeners | String | (None) | The externally advertised address and port for client connections. Required in distributed environments when the bind address is not publicly reachable. Format matches `bind.listeners` (`{listener_name}://{host}:{port}`). Defaults to the value of `bind.listeners` if not explicitly configured. | +| internal.listener.name | String | FLUSS | The listener for server internal communication. | +| security.protocol.map | Map | (none) | A map defining the authentication protocol for each listener. The format is `listenerName1:protocol1,listenerName2:protocol2`, e.g., `INTERNAL:PLAINTEXT,CLIENT:GSSAPI`. Each listener can be associated with a specific authentication protocol. Listeners not included in the map will use PLAINTEXT by default, which does not require authentication. | +| `security.${protocol}.*` | String | (none) | Protocol-specific configuration properties. For example, security.sasl.jaas.config for SASL authentication settings. | +| default.bucket.number | Integer | 1 | The default number of buckets for a table in Fluss cluster. It's a cluster-level parameter and all the tables without specifying bucket number in the cluster will use the value as the bucket number. | +| default.replication.factor | Integer | 1 | The default replication factor for the log of a table in Fluss cluster. It's a cluster-level parameter, and all the tables without specifying replication factor in the cluster will use the value as replication factor. | +| remote.data.dir | String | (None) | The directory used for storing the kv snapshot data files and remote log for log tiered storage in a Fluss supported filesystem. | +| remote.data.dirs | List<String> | (None) | The directories used for storing the kv snapshot data files and remote log for log tiered storage in a Fluss supported filesystem. This is a list of remote data directory paths. Example: `remote.data.dirs: oss://bucket1/fluss-remote-data, oss://bucket2/fluss-remote-data`. | +| remote.data.dirs.strategy | Enum | ROUND_ROBIN | The strategy for selecting the remote data directory from `remote.data.dirs`. | +| remote.data.dirs.weights | List<Integer> | (None) | The weights of the remote data directories. This is a list of weights corresponding to the `remote.data.dirs` in the same order. When `remote.data.dirs.strategy` is set to `WEIGHTED_ROUND_ROBIN`, this must be configured, and its size must be equal to `remote.data.dirs`; otherwise, it will be ignored.Example: `remote.data.dir.weights: 1, 2` | +| remote.fs.write-buffer-size | MemorySize | 4kb | The default size of the write buffer for writing the local files to remote file systems. | +| plugin.classloader.parent-first-patterns.additional | List<String> | (None) | A (semicolon-separated) list of patterns that specifies which classes should always be resolved through the plugin parent ClassLoader first. A pattern is a simple prefix that is checked against the fully qualified class name. These patterns are appended to `classloader.parent-first-patterns.default`. | +| plugin.classloader.parent-first-patterns.default | String | java.,
org.apache.fluss.,
javax.annotation.,
org.slf4j,
org.apache.log4j,
org.apache.logging,
org.apache.commons.logging,
ch.qos.logback | A (semicolon-separated) list of patterns that specifies which classes should always be resolved through the plugin parent ClassLoader first. A pattern is a simple prefix that is checked against the fully qualified class name. This setting should generally not be modified. | +| auto-partition.check.interval | Duration | 10min | The interval of auto partition check. The default value is 10 minutes. | +| allow.create.log.tables | Boolean | true | Whether to allow creation of log tables. When set to false, attempts to create log tables (tables without primary key) will be rejected. The default value is true. | +| allow.create.kv.tables | Boolean | true | Whether to allow creation of kv tables (primary key tables). When set to false, attempts to create kv tables (tables with primary key) will be rejected. The default value is true. | +| max.partition.num | Integer | 1000 | Limits the maximum number of partitions that can be created for a partitioned table to avoid creating too many partitions. | +| max.bucket.num | Integer | 128000 | The maximum number of buckets that can be created for a table. The default value is 128000. | +| acl.notification.expiration-time | Duration | 15min | The duration for which ACL notifications are valid before they expire. This configuration determines the time window during which an ACL notification is considered active. After this duration, the notification will no longer be valid and will be discarded. The default value is 15 minutes. This setting is important to ensure that ACL changes are propagated in a timely manner and do not remain active longer than necessary. | +| authorizer.enabled | Boolean | false | Specifies whether to enable the authorization feature. If enabled, access control is enforced based on the authorization rules defined in the configuration. If disabled, all operations and resources are accessible to all users. | +| authorizer.type | String | default | Specifies the type of authorizer to be used for access control. This value corresponds to the identifier of the authorization plugin. The default value is `default`, which indicates the built-in authorizer implementation. Custom authorizers can be implemented by providing a matching plugin identifier. | +| super.users | String | (None) | A semicolon-separated list of superusers who have unrestricted access to all operations and resources. Note that the delimiter is semicolon since SSL user names may contain comma, and each super user should be specified in the format `principal_type:principal_name`, e.g., `User:admin;User:bob`. This configuration is critical for defining administrative privileges in the system. | +| server.io-pool.size | Integer | 10 | The size of the IO thread pool to run blocking operations for both coordinator and tablet servers. This includes discard unnecessary snapshot files, transfer kv snapshot files, and transfer remote log files. Increase this value if you experience slow IO operations. The default value is 10. | ## CoordinatorServer