diff --git a/fluss-client/src/main/java/org/apache/fluss/client/admin/FlussAdmin.java b/fluss-client/src/main/java/org/apache/fluss/client/admin/FlussAdmin.java index 8f39a6e420..0d98cfedc4 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/admin/FlussAdmin.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/admin/FlussAdmin.java @@ -298,6 +298,9 @@ public CompletableFuture getTableInfo(TablePath tablePath) { r.getTableId(), r.getSchemaId(), TableDescriptor.fromJsonBytes(r.getTableJson()), + // For backward compatibility, results returned by old + // clusters do not include the remote data dir + r.hasRemoteDataDir() ? r.getRemoteDataDir() : null, r.getCreatedTime(), r.getModifiedTime())); } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/token/DefaultSecurityTokenManager.java b/fluss-client/src/main/java/org/apache/fluss/client/token/DefaultSecurityTokenManager.java index c104235285..f6f5dc917f 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/token/DefaultSecurityTokenManager.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/token/DefaultSecurityTokenManager.java @@ -31,6 +31,7 @@ import javax.annotation.concurrent.GuardedBy; import java.time.Clock; +import java.util.List; import java.util.Optional; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -95,7 +96,7 @@ public void start() throws Exception { void startTokensUpdate() { try { LOG.info("Starting tokens update task"); - AtomicReference tokenContainer = new AtomicReference<>(); + AtomicReference> tokenContainer = new AtomicReference<>(); Optional nextRenewal = obtainSecurityTokensAndGetNextRenewal(tokenContainer); if (tokenContainer.get() != null) { @@ -133,17 +134,17 @@ void startTokensUpdate() { } protected Optional obtainSecurityTokensAndGetNextRenewal( - AtomicReference tokenContainer) { + AtomicReference> tokenContainer) { try { - LOG.debug("Obtaining security token."); - ObtainedSecurityToken token = securityTokenProvider.obtainSecurityToken(); - tokenContainer.set(token); - checkNotNull(token, "Obtained security tokens must not be null"); - LOG.debug("Obtained security token successfully"); - return token.getValidUntil(); + LOG.debug("Obtaining security tokens."); + List tokens = securityTokenProvider.obtainSecurityTokens(); + tokenContainer.set(tokens); + checkNotNull(tokens, "Obtained security tokens must not be null"); + LOG.debug("Obtained security tokens successfully"); + return tokens.get(0).getValidUntil(); } catch (Exception e) { Throwable t = ExceptionUtils.stripExecutionException(e); - LOG.error("Failed to obtain security token.", t); + LOG.error("Failed to obtain security tokens.", t); throw new FlussRuntimeException(t); } } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/token/DefaultSecurityTokenProvider.java b/fluss-client/src/main/java/org/apache/fluss/client/token/DefaultSecurityTokenProvider.java index c0ac0da14e..4a191cbb13 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/token/DefaultSecurityTokenProvider.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/token/DefaultSecurityTokenProvider.java @@ -22,6 +22,8 @@ import org.apache.fluss.rpc.gateway.AdminReadOnlyGateway; import org.apache.fluss.rpc.messages.GetFileSystemSecurityTokenRequest; +import java.util.List; + /** A default implementation of {@link SecurityTokenProvider} to get token from server. */ public class DefaultSecurityTokenProvider implements SecurityTokenProvider { @@ -32,10 +34,10 @@ public DefaultSecurityTokenProvider(AdminReadOnlyGateway adminReadOnlyGateway) { } @Override - public ObtainedSecurityToken obtainSecurityToken() throws Exception { + public List obtainSecurityTokens() throws Exception { return adminReadOnlyGateway .getFileSystemSecurityToken(new GetFileSystemSecurityTokenRequest()) - .thenApply(ClientRpcMessageUtils::toSecurityToken) + .thenApply(ClientRpcMessageUtils::toSecurityTokens) .get(); } } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/token/SecurityTokenProvider.java b/fluss-client/src/main/java/org/apache/fluss/client/token/SecurityTokenProvider.java index c082fa43f6..19911fc751 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/token/SecurityTokenProvider.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/token/SecurityTokenProvider.java @@ -19,13 +19,15 @@ import org.apache.fluss.fs.token.ObtainedSecurityToken; +import java.util.List; + /** Security token provider API. */ public interface SecurityTokenProvider { /** - * Obtain security token. + * Obtain security tokens for all file systems configured in the target cluster. * - * @return the obtained security token. + * @return the obtained security tokens. */ - ObtainedSecurityToken obtainSecurityToken() throws Exception; + List obtainSecurityTokens() throws Exception; } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/token/SecurityTokenReceiverRepository.java b/fluss-client/src/main/java/org/apache/fluss/client/token/SecurityTokenReceiverRepository.java index 6dac499abe..1b875a80e2 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/token/SecurityTokenReceiverRepository.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/token/SecurityTokenReceiverRepository.java @@ -25,6 +25,7 @@ import org.slf4j.LoggerFactory; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.ServiceLoader; import java.util.function.Consumer; @@ -71,23 +72,25 @@ private Map loadReceivers() { } /** - * Callback function when new security token obtained. + * Callback function when new security tokens obtained. * - * @param token security token obtained. The token will be forwarded to the appropriate {@link - * SecurityTokenReceiver} based on scheme name. + * @param tokens security tokens obtained. The tokens will be forwarded to the appropriate + * {@link SecurityTokenReceiver} based on scheme name. */ - void onNewTokensObtained(ObtainedSecurityToken token) { - String schemeName = token.getScheme(); - LOG.info("New security tokens arrived, sending them to receiver"); - if (!securityTokenReceivers.containsKey(schemeName)) { - throw new IllegalStateException( - "Token arrived for service but no receiver found for it: " + schemeName); + void onNewTokensObtained(List tokens) { + for (ObtainedSecurityToken token : tokens) { + String schemeName = token.getScheme(); + LOG.info("New security tokens arrived, sending them to receiver"); + if (!securityTokenReceivers.containsKey(schemeName)) { + throw new IllegalStateException( + "Tokens arrived for service but no receiver found for them: " + schemeName); + } + try { + securityTokenReceivers.get(schemeName).onNewTokensObtained(token); + } catch (Exception e) { + LOG.warn("Failed to send tokens to security token receiver {}", schemeName, e); + } + LOG.info("Security tokens sent to receiver"); } - try { - securityTokenReceivers.get(schemeName).onNewTokensObtained(token); - } catch (Exception e) { - LOG.warn("Failed to send token to security token receiver {}", schemeName, e); - } - LOG.info("Security token sent to receiver"); } } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java b/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java index a7dbec47ce..563d2e95cd 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java @@ -291,11 +291,34 @@ public static List toFsPathAndFileName( public static ObtainedSecurityToken toSecurityToken( GetFileSystemSecurityTokenResponse response) { String scheme = response.getSchema(); + String authority = response.hasAuthority() ? response.getAuthority() : null; byte[] tokens = response.getToken(); Long validUntil = response.hasExpirationTime() ? response.getExpirationTime() : null; Map additionInfo = toKeyValueMap(response.getAdditionInfosList()); - return new ObtainedSecurityToken(scheme, tokens, validUntil, additionInfo); + return new ObtainedSecurityToken(scheme, authority, tokens, validUntil, additionInfo); + } + + public static List toSecurityTokens( + GetFileSystemSecurityTokenResponse response) { + List obtainedSecurityTokens = new ArrayList<>(); + + obtainedSecurityTokens.add(toSecurityToken(response)); + + response.getTokensList() + .forEach( + token -> + obtainedSecurityTokens.add( + new ObtainedSecurityToken( + token.getSchema(), + token.hasAuthority() ? token.getAuthority() : null, + token.getToken(), + token.hasExpirationTime() + ? token.getExpirationTime() + : null, + toKeyValueMap(token.getAdditionInfosList())))); + + return obtainedSecurityTokens; } public static MetadataRequest makeMetadataRequest( @@ -555,8 +578,12 @@ public static List toPartitionInfos(ListPartitionInfosResponse re pbPartitionInfo -> new PartitionInfo( pbPartitionInfo.getPartitionId(), - toResolvedPartitionSpec( - pbPartitionInfo.getPartitionSpec()))) + toResolvedPartitionSpec(pbPartitionInfo.getPartitionSpec()), + // For backward compatibility, results returned by old + // clusters do not include the remote data dir + pbPartitionInfo.hasRemoteDataDir() + ? pbPartitionInfo.getRemoteDataDir() + : null)) .collect(Collectors.toList()); } diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/DefaultCompletedFetchTest.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/DefaultCompletedFetchTest.java index 9f50fe59e4..c4624617df 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/DefaultCompletedFetchTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/DefaultCompletedFetchTest.java @@ -67,6 +67,7 @@ import static org.apache.fluss.record.TestData.DATA2_TABLE_ID; import static org.apache.fluss.record.TestData.DATA2_TABLE_INFO; import static org.apache.fluss.record.TestData.DATA2_TABLE_PATH; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.record.TestData.DEFAULT_SCHEMA_ID; import static org.apache.fluss.row.BinaryString.fromString; import static org.apache.fluss.rpc.util.CommonRpcMessageUtils.toByteBuffer; @@ -169,6 +170,7 @@ void testProjection(LogFormat logFormat, byte magic) throws Exception { .distributedBy(3) .logFormat(logFormat) .build(), + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis()); long fetchOffset = 0L; @@ -313,6 +315,7 @@ void testComplexTypeFetch() throws Exception { .distributedBy(3) .logFormat(LogFormat.ARROW) .build(), + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis()); long fetchOffset = 0L; diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherITCase.java index b6817b327b..73c9b40fb6 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherITCase.java @@ -172,6 +172,7 @@ void testFetchWithSchemaChange() throws Exception { DATA1_TABLE_INFO.getNumBuckets(), DATA1_TABLE_INFO.getProperties(), DATA1_TABLE_INFO.getCustomProperties(), + DATA1_TABLE_INFO.getRemoteDataDir(), DATA1_TABLE_INFO.getComment().orElse(null), DATA1_TABLE_INFO.getCreatedTime(), DATA1_TABLE_INFO.getModifiedTime()), diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetchTest.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetchTest.java index f3022c91b4..e5fe7290cf 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetchTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteCompletedFetchTest.java @@ -61,6 +61,7 @@ import static org.apache.fluss.record.TestData.DATA2_TABLE_ID; import static org.apache.fluss.record.TestData.DATA2_TABLE_INFO; import static org.apache.fluss.record.TestData.DATA2_TABLE_PATH; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.record.TestData.DEFAULT_SCHEMA_ID; import static org.apache.fluss.testutils.DataTestUtils.genLogFile; import static org.apache.fluss.utils.FlussPaths.remoteLogSegmentDir; @@ -218,6 +219,7 @@ void testProjection(String format) throws Exception { .distributedBy(3) .logFormat(logFormat) .build(), + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis()); long fetchOffset = 0L; diff --git a/fluss-client/src/test/java/org/apache/fluss/client/token/DefaultSecurityTokenManagerTest.java b/fluss-client/src/test/java/org/apache/fluss/client/token/DefaultSecurityTokenManagerTest.java index df82463307..888d52f30f 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/token/DefaultSecurityTokenManagerTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/token/DefaultSecurityTokenManagerTest.java @@ -25,6 +25,7 @@ import java.time.Clock; import java.time.Duration; import java.time.ZoneId; +import java.util.Arrays; import static java.time.Instant.ofEpochMilli; import static org.apache.fluss.config.ConfigOptions.FILESYSTEM_SECURITY_TOKEN_RENEWAL_RETRY_BACKOFF; @@ -37,7 +38,7 @@ class DefaultSecurityTokenManagerTest { @Test void startTokensUpdateShouldScheduleRenewal() { TestingSecurityTokenProvider testingSecurityTokenProvider = - new TestingSecurityTokenProvider("token1"); + new TestingSecurityTokenProvider(Arrays.asList("token1-1", "token1-2", "token1-3")); // set small token renew backoff Configuration configuration = new Configuration(); @@ -54,23 +55,31 @@ void startTokensUpdateShouldScheduleRenewal() { Duration.ofMinutes(1), () -> assertThat(testingSecurityTokenProvider.getHistoryTokens()) - .containsExactly("token1")); + .containsExactly( + Arrays.asList("token1-1", "token1-2", "token1-3"))); // token history should be token1, token2 - testingSecurityTokenProvider.setCurrentToken("token2"); + testingSecurityTokenProvider.setCurrentTokens( + Arrays.asList("token2-1", "token2-2", "token2-3")); retry( Duration.ofMinutes(1), () -> assertThat(testingSecurityTokenProvider.getHistoryTokens()) - .containsExactly("token1", "token2")); + .containsExactly( + Arrays.asList("token1-1", "token1-2", "token1-3"), + Arrays.asList("token2-1", "token2-2", "token2-3"))); // token history should be token1, token2, token3 - testingSecurityTokenProvider.setCurrentToken("token3"); + testingSecurityTokenProvider.setCurrentTokens( + Arrays.asList("token3-1", "token3-2", "token3-3")); retry( Duration.ofMinutes(1), () -> assertThat(testingSecurityTokenProvider.getHistoryTokens()) - .containsExactly("token1", "token2", "token3")); + .containsExactly( + Arrays.asList("token1-1", "token1-2", "token1-3"), + Arrays.asList("token2-1", "token2-2", "token2-3"), + Arrays.asList("token3-1", "token3-2", "token3-3"))); securityTokenManager.stopTokensUpdate(); diff --git a/fluss-client/src/test/java/org/apache/fluss/client/token/TestingSecurityTokenProvider.java b/fluss-client/src/test/java/org/apache/fluss/client/token/TestingSecurityTokenProvider.java index a6a3b40f3e..f1b53a55b4 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/token/TestingSecurityTokenProvider.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/token/TestingSecurityTokenProvider.java @@ -21,49 +21,65 @@ import java.time.Clock; import java.util.Collections; +import java.util.List; import java.util.Queue; import java.util.concurrent.LinkedBlockingDeque; +import java.util.stream.Collectors; /** A {@link SecurityTokenProvider} for testing purpose. */ public class TestingSecurityTokenProvider implements SecurityTokenProvider { - private final Queue historyTokens = new LinkedBlockingDeque<>(); - private volatile String currentToken; + private final Queue> historyTokens = new LinkedBlockingDeque<>(); + private volatile List currentTokens; - public TestingSecurityTokenProvider(String currentToken) { - this.currentToken = currentToken; + public TestingSecurityTokenProvider(List currentTokens) { + this.currentTokens = currentTokens; } - public void setCurrentToken(String currentToken) { + public void setCurrentTokens(List currentTokens) { synchronized (this) { - this.currentToken = currentToken; + this.currentTokens = currentTokens; } } @Override - public ObtainedSecurityToken obtainSecurityToken() { + public List obtainSecurityTokens() { synchronized (this) { - String previousToken = historyTokens.peek(); + List previousTokens = historyTokens.peek(); long currentTime = Clock.systemDefaultZone().millis(); // we set expire time to 2s later, should be large enough for testing. // if it's too small, DefaultSecurityTokenManager#calculateRenewalDelay will // get a negative value by formula ‘Math.round(tokensRenewalTimeRatio * (nextRenewal - - // now))’ which causes never renewal token + // now))’ which causes never renewal tokens long expireTime = currentTime + 2000; - if (previousToken != null && previousToken.equals(currentToken)) { - // just return the previous one token - return new ObtainedSecurityToken( - "testing", previousToken.getBytes(), expireTime, Collections.emptyMap()); + if (previousTokens != null && previousTokens.equals(currentTokens)) { + // just return the previous tokens + return previousTokens.stream() + .map( + t -> + new ObtainedSecurityToken( + "testing", + t.getBytes(), + expireTime, + Collections.emptyMap())) + .collect(Collectors.toList()); } else { - // return the current token and push back to the queue - historyTokens.add(currentToken); - return new ObtainedSecurityToken( - "testing", currentToken.getBytes(), expireTime, Collections.emptyMap()); + // return the current tokens and push back to the queue + historyTokens.add(currentTokens); + return currentTokens.stream() + .map( + t -> + new ObtainedSecurityToken( + "testing", + t.getBytes(), + expireTime, + Collections.emptyMap())) + .collect(Collectors.toList()); } } } - public Queue getHistoryTokens() { + public Queue> getHistoryTokens() { return historyTokens; } } diff --git a/fluss-client/src/test/java/org/apache/fluss/client/write/RecordAccumulatorTest.java b/fluss-client/src/test/java/org/apache/fluss/client/write/RecordAccumulatorTest.java index 746d251d81..773c12a445 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/write/RecordAccumulatorTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/write/RecordAccumulatorTest.java @@ -76,6 +76,7 @@ import static org.apache.fluss.record.TestData.DATA1_TABLE_ID; import static org.apache.fluss.record.TestData.DATA1_TABLE_INFO; import static org.apache.fluss.record.TestData.DATA1_TABLE_PATH; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.testutils.DataTestUtils.indexedRow; import static org.apache.fluss.testutils.DataTestUtils.row; import static org.assertj.core.api.Assertions.assertThat; @@ -96,6 +97,7 @@ class RecordAccumulatorTest { .distributedBy(3) .property(ConfigOptions.TABLE_LOG_ARROW_COMPRESSION_TYPE.key(), "zstd") .build(), + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis()); diff --git a/fluss-common/src/main/java/org/apache/fluss/config/FlussConfigUtils.java b/fluss-common/src/main/java/org/apache/fluss/config/FlussConfigUtils.java index b3a1d84f92..eff9938f87 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 @@ -59,6 +59,25 @@ public static boolean isAlterableTableOption(String key) { return ALTERABLE_TABLE_OPTIONS.contains(key); } + /** + * Returns the default remote data directory from the configuration. Used as a fallback for + * tables or partitions that do not contain remote data directory metadata. + * + * @param conf the Fluss configuration + * @return the default remote data directory path, never {@code null} if the configuration is + * valid (i.e., at least one of {@code remote.data.dir} or {@code remote.data.dirs} is set) + * @see ConfigOptions#REMOTE_DATA_DIR + * @see ConfigOptions#REMOTE_DATA_DIRS + */ + public static String getDefaultRemoteDataDir(Configuration conf) { + List remoteDataDirs = conf.get(ConfigOptions.REMOTE_DATA_DIRS); + if (!remoteDataDirs.isEmpty()) { + return remoteDataDirs.get(0); + } + + return conf.get(ConfigOptions.REMOTE_DATA_DIR); + } + @VisibleForTesting static Map> extractConfigOptions(String prefix) { Map> options = new HashMap<>(); @@ -97,34 +116,9 @@ public static void validateTabletConfigs(Configuration conf) { validMinValue(ConfigOptions.TABLET_SERVER_ID, serverId.get(), 0); } - /** Validate common server configs. */ - protected static void validateServerConfigs(Configuration conf) { - // Validate remote.data.dir and remote.data.dirs - String remoteDataDir = conf.get(ConfigOptions.REMOTE_DATA_DIR); - List remoteDataDirs = conf.get(ConfigOptions.REMOTE_DATA_DIRS); - if (conf.get(ConfigOptions.REMOTE_DATA_DIR) == null - && conf.get(ConfigOptions.REMOTE_DATA_DIRS).isEmpty()) { - throw new IllegalConfigurationException( - String.format( - "Either %s or %s must be configured.", - ConfigOptions.REMOTE_DATA_DIR.key(), - ConfigOptions.REMOTE_DATA_DIRS.key())); - } - - if (remoteDataDir != null) { - // Must validate that remote.data.dir is a valid FsPath - try { - new FsPath(conf.get(ConfigOptions.REMOTE_DATA_DIR)); - } catch (Exception e) { - throw new IllegalConfigurationException( - String.format( - "Invalid configuration for %s.", - ConfigOptions.REMOTE_DATA_DIR.key()), - e); - } - } - + public static void validateRemoteDataDirs(Configuration conf) { // Validate remote.data.dirs + List remoteDataDirs = conf.get(ConfigOptions.REMOTE_DATA_DIRS); for (int i = 0; i < remoteDataDirs.size(); i++) { String dir = remoteDataDirs.get(i); try { @@ -154,20 +148,57 @@ protected static void validateServerConfigs(Configuration conf) { weights.size())); } - // Validate all weights are no less than 0 + // Verify that each weight is non-negative and that the total weight is greater than + // 0. + int totalWeight = 0; for (int i = 0; i < weights.size(); i++) { - if (weights.get(i) < 0) { + int weight = weights.get(i); + if (weight < 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)); + ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS.key(), weight, i)); } + totalWeight += weight; } + if (totalWeight <= 0) { + throw new IllegalConfigurationException( + String.format( + "The sum of all weights in '%s' must be greater than 0, but the current sum is %d.", + ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS.key(), totalWeight)); + } + } + } + } + + /** Validate common server configs. */ + protected static void validateServerConfigs(Configuration conf) { + // Validate remote.data.dir and remote.data.dirs + String remoteDataDir = conf.get(ConfigOptions.REMOTE_DATA_DIR); + List remoteDataDirs = conf.get(ConfigOptions.REMOTE_DATA_DIRS); + if (remoteDataDir == null && remoteDataDirs.isEmpty()) { + throw new IllegalConfigurationException( + String.format( + "Either %s or %s must be configured.", + ConfigOptions.REMOTE_DATA_DIR.key(), + ConfigOptions.REMOTE_DATA_DIRS.key())); + } + + if (remoteDataDir != null) { + // Must validate that remote.data.dir is a valid FsPath + try { + new FsPath(conf.get(ConfigOptions.REMOTE_DATA_DIR)); + } catch (Exception e) { + throw new IllegalConfigurationException( + String.format( + "Invalid configuration for %s.", + ConfigOptions.REMOTE_DATA_DIR.key()), + e); } } + validateRemoteDataDirs(conf); + validMinValue(conf, ConfigOptions.DEFAULT_REPLICATION_FACTOR, 1); validMinValue(conf, ConfigOptions.KV_MAX_RETAINED_SNAPSHOTS, 1); validMinValue(conf, ConfigOptions.SERVER_IO_POOL_SIZE, 1); diff --git a/fluss-common/src/main/java/org/apache/fluss/fs/FileSystem.java b/fluss-common/src/main/java/org/apache/fluss/fs/FileSystem.java index 7410b54096..bb9d56e947 100644 --- a/fluss-common/src/main/java/org/apache/fluss/fs/FileSystem.java +++ b/fluss-common/src/main/java/org/apache/fluss/fs/FileSystem.java @@ -590,7 +590,7 @@ private static void addAllPluginsToList( // ------------------------------------------------------------------------ /** An identifier of a file system, via its scheme and its authority. */ - private static final class FSKey { + public static final class FSKey { /** The scheme of the file system. */ private final String scheme; diff --git a/fluss-common/src/main/java/org/apache/fluss/fs/token/ObtainedSecurityToken.java b/fluss-common/src/main/java/org/apache/fluss/fs/token/ObtainedSecurityToken.java index 93aa7c6cc7..1ef8725e05 100644 --- a/fluss-common/src/main/java/org/apache/fluss/fs/token/ObtainedSecurityToken.java +++ b/fluss-common/src/main/java/org/apache/fluss/fs/token/ObtainedSecurityToken.java @@ -31,6 +31,9 @@ public final class ObtainedSecurityToken { /** The scheme of filesystem the token is for. */ private final String scheme; + /** The authority of the file system the token is for. */ + @Nullable private final String authority; + /** Additional information for accessing filesystem. */ private final Map additionInfos; @@ -42,7 +45,17 @@ public ObtainedSecurityToken( byte[] token, @Nullable Long validUntil, Map additionInfos) { + this(scheme, null, token, validUntil, additionInfos); + } + + public ObtainedSecurityToken( + String scheme, + @Nullable String authority, + byte[] token, + @Nullable Long validUntil, + Map additionInfos) { this.scheme = scheme; + this.authority = authority; this.token = token; this.validUntil = validUntil; this.additionInfos = additionInfos; @@ -52,6 +65,10 @@ public String getScheme() { return scheme; } + public Optional getAuthority() { + return Optional.ofNullable(authority); + } + public byte[] getToken() { return token; } diff --git a/fluss-common/src/main/java/org/apache/fluss/metadata/PartitionInfo.java b/fluss-common/src/main/java/org/apache/fluss/metadata/PartitionInfo.java index c6c9fb2585..1462513ff5 100644 --- a/fluss-common/src/main/java/org/apache/fluss/metadata/PartitionInfo.java +++ b/fluss-common/src/main/java/org/apache/fluss/metadata/PartitionInfo.java @@ -22,8 +22,8 @@ import java.util.Objects; /** - * Information of a partition metadata, includes the partition's name and the partition id that - * represents the unique identifier of the partition. + * Information of a partition metadata, includes partition id (unique identifier of the partition), + * partition name, remote data dir for partitioned data storage, etc. * * @since 0.2 */ @@ -31,10 +31,13 @@ public class PartitionInfo { private final long partitionId; private final ResolvedPartitionSpec partitionSpec; + private final String remoteDataDir; - public PartitionInfo(long partitionId, ResolvedPartitionSpec partitionSpec) { + public PartitionInfo( + long partitionId, ResolvedPartitionSpec partitionSpec, String remoteDataDir) { this.partitionId = partitionId; this.partitionSpec = partitionSpec; + this.remoteDataDir = remoteDataDir; } /** Get the partition id. The id is globally unique in the Fluss cluster. */ @@ -58,6 +61,10 @@ public PartitionSpec getPartitionSpec() { return partitionSpec.toPartitionSpec(); } + public String getRemoteDataDir() { + return remoteDataDir; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -67,16 +74,25 @@ public boolean equals(Object o) { return false; } PartitionInfo that = (PartitionInfo) o; - return partitionId == that.partitionId && Objects.equals(partitionSpec, that.partitionSpec); + return partitionId == that.partitionId + && Objects.equals(partitionSpec, that.partitionSpec) + && Objects.equals(remoteDataDir, that.remoteDataDir); } @Override public int hashCode() { - return Objects.hash(partitionId, partitionSpec); + return Objects.hash(partitionId, partitionSpec, remoteDataDir); } @Override public String toString() { - return "Partition{name='" + getPartitionName() + '\'' + ", id=" + partitionId + '}'; + return "Partition{name='" + + getPartitionName() + + '\'' + + ", id=" + + partitionId + + ", remoteDataDir=" + + remoteDataDir + + '}'; } } diff --git a/fluss-common/src/main/java/org/apache/fluss/metadata/TableInfo.java b/fluss-common/src/main/java/org/apache/fluss/metadata/TableInfo.java index be894ee790..afda3a28a0 100644 --- a/fluss-common/src/main/java/org/apache/fluss/metadata/TableInfo.java +++ b/fluss-common/src/main/java/org/apache/fluss/metadata/TableInfo.java @@ -59,6 +59,7 @@ public final class TableInfo { private final Configuration properties; private final TableConfig tableConfig; private final Configuration customProperties; + private final String remoteDataDir; private final @Nullable String comment; private final long createdTime; @@ -74,6 +75,7 @@ public TableInfo( int numBuckets, Configuration properties, Configuration customProperties, + String remoteDataDir, @Nullable String comment, long createdTime, long modifiedTime) { @@ -90,6 +92,7 @@ public TableInfo( this.properties = properties; this.tableConfig = new TableConfig(properties); this.customProperties = customProperties; + this.remoteDataDir = remoteDataDir; this.comment = comment; this.createdTime = createdTime; this.modifiedTime = modifiedTime; @@ -263,6 +266,11 @@ public Configuration getCustomProperties() { return customProperties; } + /** Returns the remote data directory of the table. */ + public String getRemoteDataDir() { + return remoteDataDir; + } + /** Returns the comment/description of the table. */ public Optional getComment() { return Optional.ofNullable(comment); @@ -314,6 +322,7 @@ public static TableInfo of( long tableId, int schemaId, TableDescriptor tableDescriptor, + String remoteDataDir, long createdTime, long modifiedTime) { Schema schema = tableDescriptor.getSchema(); @@ -335,6 +344,7 @@ public static TableInfo of( numBuckets, Configuration.fromMap(tableDescriptor.getProperties()), Configuration.fromMap(tableDescriptor.getCustomProperties()), + remoteDataDir, tableDescriptor.getComment().orElse(null), createdTime, modifiedTime); @@ -358,6 +368,7 @@ public boolean equals(Object o) { && Objects.equals(partitionKeys, that.partitionKeys) && Objects.equals(properties, that.properties) && Objects.equals(customProperties, that.customProperties) + && Objects.equals(remoteDataDir, that.remoteDataDir) && Objects.equals(comment, that.comment); } @@ -376,6 +387,7 @@ public int hashCode() { numBuckets, properties, customProperties, + remoteDataDir, comment); } @@ -402,6 +414,8 @@ public String toString() { + properties + ", customProperties=" + customProperties + + ", remoteDataDir=" + + remoteDataDir + ", comment='" + comment + '\'' diff --git a/fluss-common/src/main/java/org/apache/fluss/remote/RemoteLogSegment.java b/fluss-common/src/main/java/org/apache/fluss/remote/RemoteLogSegment.java index 39480e4d10..0d30ae47cd 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) { @@ -130,7 +139,8 @@ public boolean equals(Object o) { && maxTimestamp == that.maxTimestamp && Objects.equals(remoteLogSegmentId, that.remoteLogSegmentId) && Objects.equals(physicalTablePath, that.physicalTablePath) - && Objects.equals(tableBucket, that.tableBucket); + && Objects.equals(tableBucket, that.tableBucket) + && Objects.equals(remoteLogDir, that.remoteLogDir); } @Override @@ -142,7 +152,8 @@ public int hashCode() { remoteLogStartOffset, remoteLogEndOffset, maxTimestamp, - segmentSizeInBytes); + segmentSizeInBytes, + remoteLogDir); } @Override @@ -162,6 +173,8 @@ public String toString() { + maxTimestamp + ", segmentSizeInBytes=" + segmentSizeInBytes + + ", remoteLogDir=" + + remoteLogDir + '}'; } @@ -174,6 +187,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 +228,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 +241,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 9a0659f180..7db9f7fc01 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 @@ -426,6 +426,19 @@ public static FsPath remoteLogDir(Configuration conf) { return new FsPath(conf.get(ConfigOptions.REMOTE_DATA_DIR) + "/" + REMOTE_LOG_DIR_NAME); } + /** + * Returns the remote root directory path for storing log files. + * + *

The path contract: + * + *

+     * {$remote.data.dir}/log
+     * 
+ */ + public static FsPath remoteLogDir(String remoteDataDir) { + return new FsPath(remoteDataDir, REMOTE_LOG_DIR_NAME); + } + /** * Returns the remote directory path for storing log files for a log tablet. * @@ -592,6 +605,19 @@ public static FsPath remoteKvDir(Configuration conf) { return new FsPath(conf.get(ConfigOptions.REMOTE_DATA_DIR) + "/" + REMOTE_KV_DIR_NAME); } + /** + * Returns the remote root directory path for storing kv snapshot files. + * + *

The path contract: + * + *

+     * {$remote.data.dir}/kv
+     * 
+ */ + public static FsPath remoteKvDir(String 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 e2690c54ba..2783eea7af 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 @@ -137,6 +137,20 @@ void testValidateCoordinatorConfigs() { .hasMessageContaining( "All weights in 'remote.data.dirs.weights' must be no less than 0"); + // Test all zero weights + Configuration zeroWeightsConf = new Configuration(); + zeroWeightsConf.set( + ConfigOptions.REMOTE_DATA_DIRS_STRATEGY, + ConfigOptions.RemoteDataDirStrategy.WEIGHTED_ROUND_ROBIN); + zeroWeightsConf.set( + ConfigOptions.REMOTE_DATA_DIRS, Arrays.asList("s3://bucket1", "s3://bucket2")); + zeroWeightsConf.set(ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS, Arrays.asList(0, 0)); + assertThatThrownBy(() -> validateCoordinatorConfigs(zeroWeightsConf)) + .isInstanceOf(IllegalConfigurationException.class) + .hasMessageContaining("The sum of all weights") + .hasMessageContaining(ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS.key()) + .hasMessageContaining("must be greater than 0"); + // Test invalid DEFAULT_REPLICATION_FACTOR Configuration invalidReplicationConf = new Configuration(); invalidReplicationConf.set(ConfigOptions.REMOTE_DATA_DIR, "s3://bucket/path"); diff --git a/fluss-common/src/test/java/org/apache/fluss/record/TestData.java b/fluss-common/src/test/java/org/apache/fluss/record/TestData.java index 28ab9ececc..a4071d2aaa 100644 --- a/fluss-common/src/test/java/org/apache/fluss/record/TestData.java +++ b/fluss-common/src/test/java/org/apache/fluss/record/TestData.java @@ -39,6 +39,7 @@ public final class TestData { public static final short DEFAULT_SCHEMA_ID = 1; public static final long BASE_OFFSET = 0L; public static final byte DEFAULT_MAGIC = CURRENT_LOG_MAGIC_VALUE; + public static final String DEFAULT_REMOTE_DATA_DIR = "/tmp/fluss/remote-data"; // ---------------------------- data1 and related table info begin --------------------------- public static final List DATA1 = Arrays.asList( @@ -93,6 +94,7 @@ public final class TestData { DATA1_TABLE_ID, 1, DATA1_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, currentMillis, currentMillis); @@ -118,6 +120,7 @@ public final class TestData { PARTITION_TABLE_ID, 1, DATA1_PARTITIONED_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis()); @@ -148,6 +151,7 @@ public final class TestData { DATA1_TABLE_ID_PK, 1, DATA1_TABLE_DESCRIPTOR_PK, + DEFAULT_REMOTE_DATA_DIR, currentMillis, currentMillis); @@ -217,6 +221,7 @@ public final class TestData { DATA2_TABLE_ID, 1, DATA2_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis()); // -------------------------------- data2 info end ------------------------------------ diff --git a/fluss-common/src/test/java/org/apache/fluss/utils/PartitionUtilsTest.java b/fluss-common/src/test/java/org/apache/fluss/utils/PartitionUtilsTest.java index 686e5ce479..dae176265e 100644 --- a/fluss-common/src/test/java/org/apache/fluss/utils/PartitionUtilsTest.java +++ b/fluss-common/src/test/java/org/apache/fluss/utils/PartitionUtilsTest.java @@ -40,6 +40,7 @@ import static org.apache.fluss.metadata.TablePath.detectInvalidName; import static org.apache.fluss.record.TestData.DATA1_SCHEMA; import static org.apache.fluss.record.TestData.DATA1_TABLE_PATH; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.utils.PartitionUtils.convertValueOfType; import static org.apache.fluss.utils.PartitionUtils.generateAutoPartition; import static org.apache.fluss.utils.PartitionUtils.validatePartitionSpec; @@ -84,7 +85,8 @@ void testValidatePartitionValues() { ConfigOptions.TABLE_AUTO_PARTITION_TIME_UNIT, AutoPartitionTimeUnit.YEAR) .build(); - TableInfo tableInfo = TableInfo.of(DATA1_TABLE_PATH, 1L, 1, descriptor, 1L, 1L); + TableInfo tableInfo = + TableInfo.of(DATA1_TABLE_PATH, 1L, 1, descriptor, DEFAULT_REMOTE_DATA_DIR, 1L, 1L); assertThatThrownBy( () -> validatePartitionSpec( diff --git a/fluss-filesystems/fluss-fs-oss/src/main/java/org/apache/fluss/fs/oss/OSSFileSystem.java b/fluss-filesystems/fluss-fs-oss/src/main/java/org/apache/fluss/fs/oss/OSSFileSystem.java index 6dcb583abc..b266f0fca0 100644 --- a/fluss-filesystems/fluss-fs-oss/src/main/java/org/apache/fluss/fs/oss/OSSFileSystem.java +++ b/fluss-filesystems/fluss-fs-oss/src/main/java/org/apache/fluss/fs/oss/OSSFileSystem.java @@ -39,18 +39,21 @@ class OSSFileSystem extends HadoopFileSystem { private final Configuration conf; private volatile OSSSecurityTokenProvider ossSecurityTokenProvider; private final String scheme; + private final String authority; - OSSFileSystem(FileSystem hadoopFileSystem, String scheme, Configuration conf) { + OSSFileSystem( + FileSystem hadoopFileSystem, String scheme, String authority, Configuration conf) { super(hadoopFileSystem); this.scheme = scheme; this.conf = conf; + this.authority = authority; } @Override public ObtainedSecurityToken obtainSecurityToken() throws IOException { try { mayCreateSecurityTokenProvider(); - return ossSecurityTokenProvider.obtainSecurityToken(scheme); + return ossSecurityTokenProvider.obtainSecurityToken(scheme, authority); } catch (Exception e) { throw new IOException(e); } diff --git a/fluss-filesystems/fluss-fs-oss/src/main/java/org/apache/fluss/fs/oss/OSSFileSystemPlugin.java b/fluss-filesystems/fluss-fs-oss/src/main/java/org/apache/fluss/fs/oss/OSSFileSystemPlugin.java index e3d3f88b58..4883bcc674 100644 --- a/fluss-filesystems/fluss-fs-oss/src/main/java/org/apache/fluss/fs/oss/OSSFileSystemPlugin.java +++ b/fluss-filesystems/fluss-fs-oss/src/main/java/org/apache/fluss/fs/oss/OSSFileSystemPlugin.java @@ -80,7 +80,7 @@ public FileSystem create(URI fsUri, Configuration flussConfig) throws IOExceptio // no ak, no credentialsProvider, // set default credential provider which will get token from // OSSSecurityTokenReceiver - setDefaultCredentialProvider(flussConfig, hadoopConfig); + setDefaultCredentialProvider(fsUri, flussConfig, hadoopConfig); } } else { LOG.info("{} is set, using provided access key id and secret.", ACCESS_KEY_ID); @@ -99,7 +99,7 @@ public FileSystem create(URI fsUri, Configuration flussConfig) throws IOExceptio } org.apache.hadoop.fs.FileSystem fileSystem = initFileSystem(fsUri, hadoopConfig); - return new OSSFileSystem(fileSystem, getScheme(), hadoopConfig); + return new OSSFileSystem(fileSystem, getScheme(), authority, hadoopConfig); } protected org.apache.hadoop.fs.FileSystem initFileSystem( @@ -111,9 +111,11 @@ protected org.apache.hadoop.fs.FileSystem initFileSystem( } protected void setDefaultCredentialProvider( - Configuration flussConfig, org.apache.hadoop.conf.Configuration hadoopConfig) { + URI fsUri, + Configuration flussConfig, + org.apache.hadoop.conf.Configuration hadoopConfig) { // use OSSSecurityTokenReceiver to update hadoop config to set credentialsProvider - OSSSecurityTokenReceiver.updateHadoopConfig(hadoopConfig); + OSSSecurityTokenReceiver.updateHadoopConfig(fsUri, hadoopConfig); } @VisibleForTesting diff --git a/fluss-filesystems/fluss-fs-oss/src/main/java/org/apache/fluss/fs/oss/token/DynamicTemporaryOssCredentialsProvider.java b/fluss-filesystems/fluss-fs-oss/src/main/java/org/apache/fluss/fs/oss/token/DynamicTemporaryOssCredentialsProvider.java index 7b2f670163..ac95b44e8f 100644 --- a/fluss-filesystems/fluss-fs-oss/src/main/java/org/apache/fluss/fs/oss/token/DynamicTemporaryOssCredentialsProvider.java +++ b/fluss-filesystems/fluss-fs-oss/src/main/java/org/apache/fluss/fs/oss/token/DynamicTemporaryOssCredentialsProvider.java @@ -23,10 +23,13 @@ import com.aliyun.oss.common.auth.Credentials; import com.aliyun.oss.common.auth.CredentialsProvider; import com.aliyun.oss.common.auth.InvalidCredentialsException; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.aliyun.oss.AliyunOSSFileSystem; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.net.URI; + /** * Support dynamic session credentials for authenticating with OSS. It'll get credentials from * {@link OSSSecurityTokenReceiver}. It implements oss native {@link CredentialsProvider} to work @@ -40,6 +43,12 @@ public class DynamicTemporaryOssCredentialsProvider implements CredentialsProvid public static final String NAME = DynamicTemporaryOssCredentialsProvider.class.getName(); + private final URI uri; + + public DynamicTemporaryOssCredentialsProvider(URI uri, Configuration conf) { + this.uri = uri; + } + @Override public void setCredentials(Credentials credentials) { // do nothing @@ -47,7 +56,7 @@ public void setCredentials(Credentials credentials) { @Override public Credentials getCredentials() { - Credentials credentials = OSSSecurityTokenReceiver.getCredentials(); + Credentials credentials = OSSSecurityTokenReceiver.getCredentials(uri); if (credentials == null) { throw new InvalidCredentialsException("Credentials is not ready."); } diff --git a/fluss-filesystems/fluss-fs-oss/src/main/java/org/apache/fluss/fs/oss/token/OSSSecurityTokenProvider.java b/fluss-filesystems/fluss-fs-oss/src/main/java/org/apache/fluss/fs/oss/token/OSSSecurityTokenProvider.java index 229123a621..693f4f2343 100644 --- a/fluss-filesystems/fluss-fs-oss/src/main/java/org/apache/fluss/fs/oss/token/OSSSecurityTokenProvider.java +++ b/fluss-filesystems/fluss-fs-oss/src/main/java/org/apache/fluss/fs/oss/token/OSSSecurityTokenProvider.java @@ -17,9 +17,11 @@ package org.apache.fluss.fs.oss.token; +import org.apache.fluss.fs.FileSystem.FSKey; import org.apache.fluss.fs.token.Credentials; import org.apache.fluss.fs.token.CredentialsJsonSerde; import org.apache.fluss.fs.token.ObtainedSecurityToken; +import org.apache.fluss.utils.StringUtils; import com.aliyun.oss.common.auth.DefaultCredentials; import com.aliyuncs.DefaultAcsClient; @@ -47,35 +49,37 @@ public class OSSSecurityTokenProvider { private static final String ROLE_ARN_KEY = "fs.oss.roleArn"; private static final String STS_ENDPOINT_KEY = "fs.oss.sts.endpoint"; + /** Prefix for all OSS properties: {@value}. */ + private static final String FS_OSS_PREFIX = "fs.oss."; + /** Prefix for OSS bucket-specific properties: {@value}. */ + private static final String FS_OSS_BUCKET_PREFIX = "fs.oss.bucket."; - private final String endpoint; - private final String region; - private final DefaultAcsClient acsClient; - private final String roleArn; - - public OSSSecurityTokenProvider(Configuration conf) throws IOException { - endpoint = AliyunOSSUtils.getValueWithKey(conf, ENDPOINT_KEY); - String accessKeyId = AliyunOSSUtils.getValueWithKey(conf, ACCESS_KEY_ID); - String accessKeySecret = AliyunOSSUtils.getValueWithKey(conf, ACCESS_KEY_SECRET); - String endpoint = AliyunOSSUtils.getValueWithKey(conf, STS_ENDPOINT_KEY); - region = AliyunOSSUtils.getValueWithKey(conf, REGION_KEY); - // don't need set region id - DefaultProfile.addEndpoint("", "Sts", endpoint); - IClientProfile profile = DefaultProfile.getProfile("", accessKeyId, accessKeySecret); - acsClient = new DefaultAcsClient(profile); - roleArn = conf.get(ROLE_ARN_KEY); + protected final Configuration conf; + + protected final Map clientInfoMap = new HashMap<>(); + + public OSSSecurityTokenProvider(Configuration conf) { + this.conf = conf; } - public ObtainedSecurityToken obtainSecurityToken(String scheme) throws Exception { + public ObtainedSecurityToken obtainSecurityToken(String scheme, String authority) + throws Exception { + FSKey fsKey = new FSKey(scheme, authority); + if (!clientInfoMap.containsKey(fsKey)) { + clientInfoMap.put(fsKey, initOSSClientInfo(authority)); + } + + OSSClientInfo info = clientInfoMap.get(fsKey); + final AssumeRoleRequest request = new AssumeRoleRequest(); request.setSysMethod(MethodType.POST); - request.setRoleArn(roleArn); + request.setRoleArn(info.roleArn); // session name is used for audit, in here, we just generate a unique session name // todo: may consider use meaningful session name request.setRoleSessionName("fluss-" + UUID.randomUUID()); // todo: may consider make token duration time configurable, we don't set it now // token duration time is 1 hour by default - final AssumeRoleResponse response = acsClient.getAcsResponse(request); + final AssumeRoleResponse response = info.acsClient.getAcsResponse(request); AssumeRoleResponse.Credentials credentials = response.getCredentials(); DefaultCredentials defaultCredentials = @@ -86,16 +90,34 @@ public ObtainedSecurityToken obtainSecurityToken(String scheme) throws Exception Map additionInfo = new HashMap<>(); // we need to put endpoint as addition info - additionInfo.put(ENDPOINT_KEY, endpoint); - additionInfo.put(REGION_KEY, region); + additionInfo.put(ENDPOINT_KEY, info.endpoint); + additionInfo.put(REGION_KEY, info.region); return new ObtainedSecurityToken( scheme, + authority, toJson(defaultCredentials), Instant.parse(credentials.getExpiration()).toEpochMilli(), additionInfo); } + private OSSClientInfo initOSSClientInfo(String bucket) throws IOException { + OSSClientInfo info = new OSSClientInfo(); + + info.endpoint = getBucketOption(conf, bucket, ENDPOINT_KEY); + String accessKeyId = getBucketOption(conf, bucket, ACCESS_KEY_ID); + String accessKeySecret = getBucketOption(conf, bucket, ACCESS_KEY_SECRET); + String endpoint = getBucketOption(conf, bucket, STS_ENDPOINT_KEY); + info.region = getBucketOption(conf, bucket, REGION_KEY); + // don't need set region id + DefaultProfile.addEndpoint("", "Sts", endpoint); + IClientProfile profile = DefaultProfile.getProfile("", accessKeyId, accessKeySecret); + info.acsClient = new DefaultAcsClient(profile); + info.roleArn = getBucketOption(conf, bucket, ROLE_ARN_KEY); + + return info; + } + private byte[] toJson(DefaultCredentials defaultCredentials) { Credentials credentials = new Credentials( @@ -104,4 +126,34 @@ private byte[] toJson(DefaultCredentials defaultCredentials) { defaultCredentials.getSecurityToken()); return CredentialsJsonSerde.toJson(credentials); } + + /** + * Get a bucket-specific property. If the generic key passed in has an {@code fs.oss.prefix}, + * that's stripped off. + * + * @param conf configuration to get + * @param bucket bucket name + * @param genericKey key; can start with "fs.oss." + * @return the bucket option, null if there is none + */ + public static String getBucketOption(Configuration conf, String bucket, String genericKey) + throws IOException { + final String baseKey = + genericKey.startsWith(FS_OSS_PREFIX) + ? genericKey.substring(FS_OSS_PREFIX.length()) + : genericKey; + String value = + AliyunOSSUtils.getValueWithKey(conf, FS_OSS_BUCKET_PREFIX + bucket + '.' + baseKey); + if (StringUtils.isNullOrWhitespaceOnly(value)) { + value = AliyunOSSUtils.getValueWithKey(conf, FS_OSS_PREFIX + baseKey); + } + return value; + } + + protected static class OSSClientInfo { + public String endpoint; + public String region; + public DefaultAcsClient acsClient; + public String roleArn; + } } diff --git a/fluss-filesystems/fluss-fs-oss/src/main/java/org/apache/fluss/fs/oss/token/OSSSecurityTokenReceiver.java b/fluss-filesystems/fluss-fs-oss/src/main/java/org/apache/fluss/fs/oss/token/OSSSecurityTokenReceiver.java index ea3ade52eb..4e797b3044 100644 --- a/fluss-filesystems/fluss-fs-oss/src/main/java/org/apache/fluss/fs/oss/token/OSSSecurityTokenReceiver.java +++ b/fluss-filesystems/fluss-fs-oss/src/main/java/org/apache/fluss/fs/oss/token/OSSSecurityTokenReceiver.java @@ -21,6 +21,7 @@ import org.apache.fluss.fs.token.CredentialsJsonSerde; import org.apache.fluss.fs.token.ObtainedSecurityToken; import org.apache.fluss.fs.token.SecurityTokenReceiver; +import org.apache.fluss.utils.MapUtils; import com.aliyun.oss.common.auth.Credentials; import com.aliyun.oss.common.auth.DefaultCredentials; @@ -29,22 +30,28 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.net.URI; import java.util.Map; +import static org.apache.fluss.fs.FileSystem.FSKey; + /** Security token receiver for OSS filesystem. */ public class OSSSecurityTokenReceiver implements SecurityTokenReceiver { private static final Logger LOG = LoggerFactory.getLogger(OSSSecurityTokenReceiver.class); - static volatile Credentials credentials; - static volatile Map additionInfos; + static Map credentialsCache = MapUtils.newConcurrentHashMap(); + static Map> additionInfosCache = MapUtils.newConcurrentHashMap(); - public static void updateHadoopConfig(org.apache.hadoop.conf.Configuration hadoopConfig) { - updateHadoopConfig(hadoopConfig, DynamicTemporaryOssCredentialsProvider.NAME); + public static void updateHadoopConfig( + URI fsUri, org.apache.hadoop.conf.Configuration hadoopConfig) { + updateHadoopConfig(fsUri, hadoopConfig, DynamicTemporaryOssCredentialsProvider.NAME); } protected static void updateHadoopConfig( - org.apache.hadoop.conf.Configuration hadoopConfig, String credentialsProviderName) { + URI fsUri, + org.apache.hadoop.conf.Configuration hadoopConfig, + String credentialsProviderName) { LOG.info("Updating Hadoop configuration"); String providers = hadoopConfig.get(Constants.CREDENTIALS_PROVIDER_KEY, ""); @@ -62,6 +69,8 @@ protected static void updateHadoopConfig( LOG.debug("Provider already exists"); } + FSKey fsKey = new FSKey(fsUri.getScheme(), fsUri.getAuthority()); + Map additionInfos = additionInfosCache.get(fsKey); // then, set addition info if (additionInfos == null) { // if addition info is null, it also means we have not received any token, @@ -90,19 +99,23 @@ public void onNewTokensObtained(ObtainedSecurityToken token) { org.apache.fluss.fs.token.Credentials flussCredentials = CredentialsJsonSerde.fromJson(tokenBytes); - credentials = + FSKey fsKey = new FSKey(token.getScheme(), token.getAuthority().orElse(null)); + + Credentials credentials = new DefaultCredentials( flussCredentials.getAccessKeyId(), flussCredentials.getSecretAccessKey(), flussCredentials.getSecurityToken()); - additionInfos = token.getAdditionInfos(); + credentialsCache.put(fsKey, credentials); + + additionInfosCache.put(fsKey, token.getAdditionInfos()); LOG.info( "Session credentials updated successfully with access key: {}.", credentials.getAccessKeyId()); } - public static Credentials getCredentials() { - return credentials; + public static Credentials getCredentials(URI uri) { + return credentialsCache.get(new FSKey(uri.getScheme(), uri.getAuthority())); } } diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/undo/RecoveryOffsetManagerTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/undo/RecoveryOffsetManagerTest.java index f39e288e95..2e072ef757 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/undo/RecoveryOffsetManagerTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/sink/undo/RecoveryOffsetManagerTest.java @@ -44,6 +44,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -82,6 +83,7 @@ private static TableInfo createTableInfo(int numBuckets, boolean isPartitioned) numBuckets, new Configuration(), new Configuration(), + DEFAULT_REMOTE_DATA_DIR, null, // comment System.currentTimeMillis(), System.currentTimeMillis()); @@ -89,7 +91,7 @@ private static TableInfo createTableInfo(int numBuckets, boolean isPartitioned) private static PartitionInfo createPartitionInfo(long partitionId, String partitionName) { ResolvedPartitionSpec spec = ResolvedPartitionSpec.fromPartitionValue("pt", partitionName); - return new PartitionInfo(partitionId, spec); + return new PartitionInfo(partitionId, spec, DEFAULT_REMOTE_DATA_DIR); } // ==================== FRESH_START Tests ==================== diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumeratorTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumeratorTest.java index c895142438..ca4722c9e6 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumeratorTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/source/enumerator/FlinkSourceEnumeratorTest.java @@ -76,6 +76,7 @@ import java.util.stream.Collectors; import static org.apache.fluss.client.table.scanner.log.LogScanner.EARLIEST_OFFSET; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.testutils.DataTestUtils.row; import static org.assertj.core.api.Assertions.assertThat; @@ -646,13 +647,15 @@ void testPartitionsExpiredInFlussButExistInLake( ResolvedPartitionSpec.fromPartitionName( Collections.singletonList(isPrimaryKeyTable ? "date" : "name"), partitionName); - lakePartitionInfos.add(new PartitionInfo(partitionId, partitionSpec)); + lakePartitionInfos.add( + new PartitionInfo(partitionId, partitionSpec, DEFAULT_REMOTE_DATA_DIR)); } ResolvedPartitionSpec partitionSpec = ResolvedPartitionSpec.fromPartitionName( Collections.singletonList(isPrimaryKeyTable ? "date" : "name"), hybridPartitionName); - lakePartitionInfos.add(new PartitionInfo(hybridPartitionId, partitionSpec)); + lakePartitionInfos.add( + new PartitionInfo(hybridPartitionId, partitionSpec, DEFAULT_REMOTE_DATA_DIR)); LakeSource lakeSource = new TestingLakeSource(DEFAULT_BUCKET_NUM, lakePartitionInfos); diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlinkConversionsTest.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlinkConversionsTest.java index a467bb14a2..efc386eb64 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlinkConversionsTest.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlinkConversionsTest.java @@ -57,6 +57,7 @@ import static org.apache.fluss.flink.FlinkConnectorOptions.BUCKET_NUMBER; import static org.apache.fluss.flink.utils.CatalogTableTestUtils.addOptions; import static org.apache.fluss.flink.utils.CatalogTableTestUtils.checkEqualsIgnoreSchema; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.types.DataTypes.FIELD; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -257,6 +258,7 @@ void testTableConversion() { 1L, 1, flussTable.withBucketCount(1), + DEFAULT_REMOTE_DATA_DIR, currentMillis, currentMillis); // get the converted flink table @@ -446,6 +448,7 @@ void testFlinkMaterializedTableConversions() { 1L, 1, flussTable.withBucketCount(1), + DEFAULT_REMOTE_DATA_DIR, currentMillis, currentMillis); // get the converted flink table @@ -508,6 +511,7 @@ void testAggregationFunctionRoundTrip() { 1L, 1, flussTable.withBucketCount(1), + DEFAULT_REMOTE_DATA_DIR, currentMillis, currentMillis); CatalogTable convertedFlinkTable = (CatalogTable) FlinkConversions.toFlinkTable(tableInfo); diff --git a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlinkTestBase.java b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlinkTestBase.java index 7c80bbc0ca..daf5c0813c 100644 --- a/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlinkTestBase.java +++ b/fluss-flink/fluss-flink-common/src/test/java/org/apache/fluss/flink/utils/FlinkTestBase.java @@ -237,6 +237,7 @@ public static Map createPartitions( partition, new PartitionAssignment( tableInfo.getTableId(), assignment.getBucketAssignments()), + zkClient.getDefaultRemoteDataDir(), tablePath, tableInfo.getTableId()); } diff --git a/fluss-lake/fluss-lake-iceberg/src/test/java/org/apache/fluss/lake/iceberg/tiering/IcebergTieringTest.java b/fluss-lake/fluss-lake-iceberg/src/test/java/org/apache/fluss/lake/iceberg/tiering/IcebergTieringTest.java index eb24994cc4..d5255005c7 100644 --- a/fluss-lake/fluss-lake-iceberg/src/test/java/org/apache/fluss/lake/iceberg/tiering/IcebergTieringTest.java +++ b/fluss-lake/fluss-lake-iceberg/src/test/java/org/apache/fluss/lake/iceberg/tiering/IcebergTieringTest.java @@ -77,6 +77,7 @@ import static org.apache.fluss.record.ChangeType.INSERT; import static org.apache.fluss.record.ChangeType.UPDATE_AFTER; import static org.apache.fluss.record.ChangeType.UPDATE_BEFORE; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.iceberg.expressions.Expressions.equal; import static org.assertj.core.api.Assertions.assertThat; @@ -134,7 +135,8 @@ void testTieringWriteTable(boolean isPrimaryKeyTable, boolean isPartitionedTable .distributedBy(BUCKET_NUM) .property(ConfigOptions.TABLE_DATALAKE_ENABLED, true) .build(); - TableInfo tableInfo = TableInfo.of(tablePath, 0, 1, descriptor, 1L, 1L); + TableInfo tableInfo = + TableInfo.of(tablePath, 0, 1, descriptor, DEFAULT_REMOTE_DATA_DIR, 1L, 1L); Table icebergTable = icebergCatalog.loadTable(toIceberg(tablePath)); diff --git a/fluss-lake/fluss-lake-lance/src/test/java/org/apache/fluss/lake/lance/tiering/LanceTieringTest.java b/fluss-lake/fluss-lake-lance/src/test/java/org/apache/fluss/lake/lance/tiering/LanceTieringTest.java index a7a13ff5e2..28aeba37bf 100644 --- a/fluss-lake/fluss-lake-lance/src/test/java/org/apache/fluss/lake/lance/tiering/LanceTieringTest.java +++ b/fluss-lake/fluss-lake-lance/src/test/java/org/apache/fluss/lake/lance/tiering/LanceTieringTest.java @@ -68,6 +68,7 @@ import java.util.stream.Stream; import static org.apache.fluss.lake.committer.LakeCommitter.FLUSS_LAKE_SNAP_BUCKET_OFFSET_PROPERTY; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.assertj.core.api.Assertions.assertThat; /** The UT for tiering to Lance via {@link LanceLakeTieringFactory}. */ @@ -114,7 +115,8 @@ void testTieringWriteTable(boolean isPartitioned) throws Exception { .property(ConfigOptions.TABLE_DATALAKE_ENABLED, true) .customProperties(customProperties) .build(); - TableInfo tableInfo = TableInfo.of(tablePath, 0, 1, descriptor, 1L, 1L); + TableInfo tableInfo = + TableInfo.of(tablePath, 0, 1, descriptor, DEFAULT_REMOTE_DATA_DIR, 1L, 1L); List lanceWriteResults = new ArrayList<>(); SimpleVersionedSerializer writeResultSerializer = diff --git a/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/tiering/PaimonTieringTest.java b/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/tiering/PaimonTieringTest.java index 600c3ba661..958d134b77 100644 --- a/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/tiering/PaimonTieringTest.java +++ b/fluss-lake/fluss-lake-paimon/src/test/java/org/apache/fluss/lake/paimon/tiering/PaimonTieringTest.java @@ -80,6 +80,7 @@ import static org.apache.fluss.record.ChangeType.INSERT; import static org.apache.fluss.record.ChangeType.UPDATE_AFTER; import static org.apache.fluss.record.ChangeType.UPDATE_BEFORE; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.assertj.core.api.Assertions.assertThat; /** The UT for tiering to Paimon via {@link PaimonLakeTieringFactory}. */ @@ -147,7 +148,8 @@ void testTieringWriteTable(boolean isPrimaryKeyTable, boolean isPartitioned) thr .distributedBy(bucketNum) .property(ConfigOptions.TABLE_DATALAKE_ENABLED, true) .build(); - TableInfo tableInfo = TableInfo.of(tablePath, 0, 1, descriptor, 1L, 1L); + TableInfo tableInfo = + TableInfo.of(tablePath, 0, 1, descriptor, DEFAULT_REMOTE_DATA_DIR, 1L, 1L); try (LakeCommitter lakeCommitter = createLakeCommitter(tablePath, tableInfo, new Configuration())) { @@ -219,7 +221,8 @@ void testMultiPartitionTiering() throws Exception { .distributedBy(1) .property(ConfigOptions.TABLE_DATALAKE_ENABLED, true) .build(); - TableInfo tableInfo = TableInfo.of(tablePath, 0, 1, descriptor, 1L, 1L); + TableInfo tableInfo = + TableInfo.of(tablePath, 0, 1, descriptor, DEFAULT_REMOTE_DATA_DIR, 1L, 1L); Map> recordsByPartition = new HashMap<>(); List paimonWriteResults = new ArrayList<>(); @@ -293,7 +296,8 @@ void testThreePartitionTiering() throws Exception { .distributedBy(1) .property(ConfigOptions.TABLE_DATALAKE_ENABLED, true) .build(); - TableInfo tableInfo = TableInfo.of(tablePath, 0, 1, descriptor, 1L, 1L); + TableInfo tableInfo = + TableInfo.of(tablePath, 0, 1, descriptor, DEFAULT_REMOTE_DATA_DIR, 1L, 1L); Map> recordsByPartition = new HashMap<>(); List paimonWriteResults = new ArrayList<>(); @@ -382,7 +386,8 @@ void testSnapshotExpiration( ConfigOptions.TABLE_DATALAKE_AUTO_EXPIRE_SNAPSHOT, isTableAutoExpireSnapshot) .build(); - TableInfo tableInfo = TableInfo.of(tablePath, 0, 1, descriptor, 1L, 1L); + TableInfo tableInfo = + TableInfo.of(tablePath, 0, 1, descriptor, DEFAULT_REMOTE_DATA_DIR, 1L, 1L); // Get the FileStoreTable to verify snapshots FileStoreTable fileStoreTable = (FileStoreTable) paimonCatalog.getTable(toPaimon(tablePath)); diff --git a/fluss-rpc/src/main/proto/FlussApi.proto b/fluss-rpc/src/main/proto/FlussApi.proto index f04b307565..ef5d2eed68 100644 --- a/fluss-rpc/src/main/proto/FlussApi.proto +++ b/fluss-rpc/src/main/proto/FlussApi.proto @@ -139,6 +139,7 @@ message GetTableInfoResponse { required bytes table_json = 3; required int64 created_time = 4; required int64 modified_time = 5; + optional string remote_data_dir = 6; } // list tables request and response @@ -433,10 +434,24 @@ message GetFileSystemSecurityTokenRequest { } message GetFileSystemSecurityTokenResponse { + // security token for the default remote data dir configured via remote.data.dir required string schema = 1; required bytes token = 2; optional int64 expiration_time = 3; repeated PbKeyValue addition_info = 4; + // optional for backward compatibility + optional string authority = 5; + + // security tokens for the remote data dirs configured in remote.data.dirs + repeated PbFileSystemSecurityToken tokens = 6; +} + +message PbFileSystemSecurityToken { + required string schema = 1; + required string authority = 2; + required bytes token = 3; + optional int64 expiration_time = 4; + repeated PbKeyValue addition_info = 5; } // init writer request and response @@ -762,6 +777,7 @@ message PbTableMetadata { repeated PbBucketMetadata bucket_metadata = 5; required int64 created_time = 6; required int64 modified_time = 7; + optional string remote_data_dir = 8; // TODO add a new filed 'deleted_table' to indicate this table is deleted in UpdateMetadataRequest. // trace by: https://github.com/apache/fluss/issues/981 @@ -1006,6 +1022,7 @@ message PbRemoteLogSegment { message PbPartitionInfo { required int64 partition_id = 1; required PbPartitionSpec partition_spec = 2; + optional string remote_data_dir = 3; } message PbPartitionSpec { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/DynamicServerConfig.java b/fluss-server/src/main/java/org/apache/fluss/server/DynamicServerConfig.java index f3ae90496f..0820abd088 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/DynamicServerConfig.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/DynamicServerConfig.java @@ -44,6 +44,9 @@ import static org.apache.fluss.config.ConfigOptions.DATALAKE_FORMAT; import static org.apache.fluss.config.ConfigOptions.KV_SHARED_RATE_LIMITER_BYTES_PER_SEC; import static org.apache.fluss.config.ConfigOptions.KV_SNAPSHOT_INTERVAL; +import static org.apache.fluss.config.ConfigOptions.REMOTE_DATA_DIRS; +import static org.apache.fluss.config.ConfigOptions.REMOTE_DATA_DIRS_STRATEGY; +import static org.apache.fluss.config.ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS; import static org.apache.fluss.utils.concurrent.LockUtils.inReadLock; import static org.apache.fluss.utils.concurrent.LockUtils.inWriteLock; @@ -62,7 +65,11 @@ class DynamicServerConfig { Arrays.asList( DATALAKE_FORMAT.key(), KV_SHARED_RATE_LIMITER_BYTES_PER_SEC.key(), - KV_SNAPSHOT_INTERVAL.key())); + KV_SNAPSHOT_INTERVAL.key(), + // Config options for remote.data.dirs + REMOTE_DATA_DIRS.key(), + REMOTE_DATA_DIRS_STRATEGY.key(), + REMOTE_DATA_DIRS_WEIGHTS.key())); private static final Set ALLOWED_CONFIG_PREFIXES = Collections.singleton("datalake."); private final ReadWriteLock lock = new ReentrantReadWriteLock(); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/RpcServiceBase.java b/fluss-server/src/main/java/org/apache/fluss/server/RpcServiceBase.java index 2946fb93bc..253aefa097 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 @@ -29,6 +29,8 @@ import org.apache.fluss.exception.SecurityTokenException; import org.apache.fluss.exception.TableNotPartitionedException; import org.apache.fluss.fs.FileSystem; +import org.apache.fluss.fs.FileSystem.FSKey; +import org.apache.fluss.fs.FsPath; import org.apache.fluss.fs.token.ObtainedSecurityToken; import org.apache.fluss.metadata.DatabaseInfo; import org.apache.fluss.metadata.PhysicalTablePath; @@ -36,7 +38,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; @@ -84,6 +85,7 @@ import org.apache.fluss.server.authorizer.Authorizer; import org.apache.fluss.server.coordinator.CoordinatorService; import org.apache.fluss.server.coordinator.MetadataManager; +import org.apache.fluss.server.coordinator.remote.RemoteDirDynamicLoader; import org.apache.fluss.server.kv.snapshot.CompletedSnapshot; import org.apache.fluss.server.metadata.MetadataProvider; import org.apache.fluss.server.metadata.PartitionMetadata; @@ -93,6 +95,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; @@ -102,6 +105,7 @@ import java.util.ArrayList; import java.util.Collection; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -142,9 +146,11 @@ public abstract class RpcServiceBase extends RpcGatewayService implements AdminR protected final MetadataManager metadataManager; protected final @Nullable Authorizer authorizer; protected final DynamicConfigManager dynamicConfigManager; + protected final RemoteDirDynamicLoader remoteDirDynamicLoader; private long tokenLastUpdateTimeMs = 0; private ObtainedSecurityToken securityToken = null; + private final Map securityTokenMap = new HashMap<>(); private final ExecutorService ioExecutor; @@ -155,6 +161,7 @@ public RpcServiceBase( MetadataManager metadataManager, @Nullable Authorizer authorizer, DynamicConfigManager dynamicConfigManager, + RemoteDirDynamicLoader remoteDirDynamicLoader, ExecutorService ioExecutor) { this.remoteFileSystem = remoteFileSystem; this.provider = provider; @@ -163,6 +170,7 @@ public RpcServiceBase( this.metadataManager = metadataManager; this.authorizer = authorizer; this.dynamicConfigManager = dynamicConfigManager; + this.remoteDirDynamicLoader = remoteDirDynamicLoader; this.ioExecutor = ioExecutor; } @@ -362,21 +370,21 @@ public CompletableFuture getLatestKvSnapshots( } private long getPartitionId(TablePath tablePath, String partitionName) { - Optional optTablePartition; + Optional optPartitionRegistration; try { - optTablePartition = zkClient.getPartition(tablePath, partitionName); + optPartitionRegistration = zkClient.getPartition(tablePath, partitionName); } catch (Exception e) { throw new FlussRuntimeException( String.format("Failed to get latest kv snapshots for table '%s'", tablePath), e); } - if (!optTablePartition.isPresent()) { + if (!optPartitionRegistration.isPresent()) { throw new PartitionNotExistException( String.format( "The partition '%s' of table '%s' does not exist.", partitionName, tablePath)); } - return optTablePartition.get().getPartitionId(); + return optPartitionRegistration.get().getPartitionId(); } @Override @@ -415,15 +423,33 @@ public CompletableFuture getFileSystemSecuri try { // In order to avoid repeatedly obtaining security token, cache it for a while. long currentTimeMs = System.currentTimeMillis(); + List remoteDataDirs = + remoteDirDynamicLoader.getRemoteDataDirs().stream() + .map(FsPath::new) + .collect(Collectors.toList()); + List remoteFSKeys = + remoteDataDirs.stream() + .map(FsPath::toUri) + .map(uri -> new FSKey(uri.getScheme(), uri.getAuthority())) + .collect(Collectors.toList()); if (securityToken == null - || currentTimeMs - tokenLastUpdateTimeMs > TOKEN_EXPIRATION_TIME_MS) { + || currentTimeMs - tokenLastUpdateTimeMs > TOKEN_EXPIRATION_TIME_MS + || !securityTokenMap.keySet().containsAll(remoteFSKeys)) { securityToken = remoteFileSystem.obtainSecurityToken(); tokenLastUpdateTimeMs = currentTimeMs; + + securityTokenMap.clear(); + for (int i = 0; i < remoteDataDirs.size(); i++) { + FsPath remoteDataDir = remoteDataDirs.get(i); + FSKey fsKey = remoteFSKeys.get(i); + ObtainedSecurityToken token = + remoteDataDir.getFileSystem().obtainSecurityToken(); + securityTokenMap.put(fsKey, token); + } } return CompletableFuture.completedFuture( - toGetFileSystemSecurityTokenResponse( - remoteFileSystem.getUri().getScheme(), securityToken)); + toGetFileSystemSecurityTokenResponse(securityToken, securityTokenMap.values())); } catch (Exception e) { throw new SecurityTokenException( "Failed to get file access security token: " + e.getMessage()); 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..ec5688d68c 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/AutoPartitionManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/AutoPartitionManager.java @@ -29,6 +29,7 @@ import org.apache.fluss.metadata.ResolvedPartitionSpec; import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.server.coordinator.remote.RemoteDirDynamicLoader; import org.apache.fluss.server.metadata.ServerMetadataCache; import org.apache.fluss.server.zk.data.BucketAssignment; import org.apache.fluss.server.zk.data.PartitionAssignment; @@ -85,6 +86,7 @@ public class AutoPartitionManager implements AutoCloseable { private final ServerMetadataCache metadataCache; private final MetadataManager metadataManager; + private final RemoteDirDynamicLoader remoteDirDynamicLoader; private final Clock clock; private final long periodicInterval; @@ -108,10 +110,12 @@ public class AutoPartitionManager implements AutoCloseable { public AutoPartitionManager( ServerMetadataCache metadataCache, MetadataManager metadataManager, + RemoteDirDynamicLoader remoteDirDynamicLoader, Configuration conf) { this( metadataCache, metadataManager, + remoteDirDynamicLoader, conf, SystemClock.getInstance(), Executors.newScheduledThreadPool( @@ -122,11 +126,13 @@ public AutoPartitionManager( AutoPartitionManager( ServerMetadataCache metadataCache, MetadataManager metadataManager, + RemoteDirDynamicLoader remoteDirDynamicLoader, Configuration conf, Clock clock, ScheduledExecutorService periodicExecutor) { this.metadataCache = metadataCache; this.metadataManager = metadataManager; + this.remoteDirDynamicLoader = remoteDirDynamicLoader; this.clock = clock; this.periodicExecutor = periodicExecutor; this.periodicInterval = conf.get(ConfigOptions.AUTO_PARTITION_CHECK_INTERVAL).toMillis(); @@ -349,8 +355,10 @@ private void createPartitions( PartitionAssignment partitionAssignment = new PartitionAssignment(tableInfo.getTableId(), bucketAssignments); + // select a remote data dir for the partition + String remoteDataDir = remoteDirDynamicLoader.getRemoteDirSelector().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/CoordinatorContext.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorContext.java index bece7d9dc3..231571fb28 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorContext.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorContext.java @@ -19,6 +19,7 @@ import org.apache.fluss.annotation.VisibleForTesting; import org.apache.fluss.cluster.rebalance.ServerTag; +import org.apache.fluss.metadata.PartitionInfo; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableBucketReplica; @@ -84,6 +85,7 @@ public class CoordinatorContext { // a map from partition_id -> physicalTablePath private final Map pathByPartitionId = new HashMap<>(); private final Map partitionIdByPath = new HashMap<>(); + private final Map partitionInfoById = new HashMap<>(); // a map from table_id to the table path private final Map tablePathById = new HashMap<>(); @@ -267,9 +269,11 @@ public void putTableInfo(TableInfo tableInfo) { this.tableInfoById.put(tableInfo.getTableId(), tableInfo); } - public void putPartition(long partitionId, PhysicalTablePath physicalTablePath) { + public void putPartition( + long partitionId, PhysicalTablePath physicalTablePath, PartitionInfo partitionInfo) { this.pathByPartitionId.put(partitionId, physicalTablePath); this.partitionIdByPath.put(physicalTablePath, partitionId); + this.partitionInfoById.put(partitionId, partitionInfo); } public TableInfo getTableInfoById(long tableId) { @@ -309,6 +313,10 @@ public Optional getPartitionId(PhysicalTablePath physicalTablePath) { return Optional.ofNullable(partitionIdByPath.get(physicalTablePath)); } + public PartitionInfo getPartitionInfoById(long partitionId) { + return partitionInfoById.get(partitionId); + } + public Map> getTableAssignment(long tableId) { return tableAssignments.getOrDefault(tableId, Collections.emptyMap()); } @@ -442,6 +450,11 @@ public Set getAllReplicasForPartition(long tableId, long par return allReplicas; } + public boolean hasPartitionsToDelete(long tableId) { + return partitionsToBeDeleted.stream() + .anyMatch(partition -> partition.getTableId() == tableId); + } + /** * Pick up the replicas that should retry delete and replicas that considered as success delete. * diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java index 909fa503dc..58996cdd73 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java @@ -40,7 +40,9 @@ import org.apache.fluss.exception.TabletServerNotAvailableException; import org.apache.fluss.exception.UnknownServerException; import org.apache.fluss.exception.UnknownTableOrBucketException; +import org.apache.fluss.metadata.PartitionInfo; import org.apache.fluss.metadata.PhysicalTablePath; +import org.apache.fluss.metadata.ResolvedPartitionSpec; import org.apache.fluss.metadata.SchemaInfo; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableBucketReplica; @@ -90,6 +92,7 @@ import org.apache.fluss.server.coordinator.event.watcher.TabletServerChangeWatcher; import org.apache.fluss.server.coordinator.lease.KvSnapshotLeaseManager; import org.apache.fluss.server.coordinator.rebalance.RebalanceManager; +import org.apache.fluss.server.coordinator.remote.RemoteStorageCleaner; import org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElection.ControlledShutdownLeaderElection; import org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElection.ReassignmentLeaderElection; import org.apache.fluss.server.coordinator.statemachine.ReplicaStateMachine; @@ -109,6 +112,7 @@ import org.apache.fluss.server.zk.data.BucketAssignment; 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.ServerTags; @@ -397,8 +401,8 @@ private void initCoordinatorContext() throws Exception { .filter(entry -> entry.getValue().isPartitioned()) .map(Map.Entry::getKey) .collect(Collectors.toList()); - Map> tablePathMap = - zooKeeperClient.getPartitionNameAndIdsForTables(partitionedTablePathList); + Map> tablePathMap = + zooKeeperClient.getPartitionNameAndRegistrationsForTables(partitionedTablePathList); for (TablePath tablePath : tablePathSet) { TableInfo tableInfo = tablePath2TableInfoMap.get(tablePath); coordinatorContext.putTablePath(tableInfo.getTableId(), tablePath); @@ -409,13 +413,22 @@ private void initCoordinatorContext() throws Exception { lakeTables.add(Tuple2.of(tableInfo, System.currentTimeMillis())); } if (tableInfo.isPartitioned()) { - Map partitions = tablePathMap.get(tablePath); + Map partitions = tablePathMap.get(tablePath); if (partitions != null) { - for (Map.Entry partition : partitions.entrySet()) { + for (Map.Entry partition : + partitions.entrySet()) { + long partitionId = partition.getValue().getPartitionId(); // put partition info to coordinator context + PartitionInfo partitionInfo = + new PartitionInfo( + partitionId, + ResolvedPartitionSpec.fromPartitionName( + tableInfo.getPartitionKeys(), partition.getKey()), + partition.getValue().getRemoteDataDir()); coordinatorContext.putPartition( - partition.getValue(), - PhysicalTablePath.of(tableInfo.getTablePath(), partition.getKey())); + partitionId, + PhysicalTablePath.of(tableInfo.getTablePath(), partition.getKey()), + partitionInfo); } } // if the table is auto partition, put the partitions info @@ -716,6 +729,7 @@ private void processSchemaChange(SchemaChangeEvent schemaChangeEvent) { oldTableInfo.getNumBuckets(), oldTableInfo.getProperties(), oldTableInfo.getCustomProperties(), + oldTableInfo.getRemoteDataDir(), oldTableInfo.getComment().orElse(null), oldTableInfo.getCreatedTime(), System.currentTimeMillis())); @@ -796,12 +810,14 @@ private void processCreatePartition(CreatePartitionEvent createPartitionEvent) { TablePath tablePath = createPartitionEvent.getTablePath(); String partitionName = createPartitionEvent.getPartitionName(); PartitionAssignment partitionAssignment = createPartitionEvent.getPartitionAssignment(); - tableManager.onCreateNewPartition( - tablePath, - tableId, - createPartitionEvent.getPartitionId(), - partitionName, - partitionAssignment); + TableInfo tableInfo = coordinatorContext.getTableInfoById(tableId); + PartitionInfo partitionInfo = + new PartitionInfo( + partitionId, + ResolvedPartitionSpec.fromPartitionName( + tableInfo.getPartitionKeys(), partitionName), + createPartitionEvent.getRemoteDataDir()); + tableManager.onCreateNewPartition(tablePath, tableId, partitionInfo, partitionAssignment); autoPartitionManager.addPartition(tableId, partitionName); Set tableBuckets = new HashSet<>(); @@ -845,6 +861,12 @@ private void processDropTable(DropTableEvent dropTableEvent) { lakeTableTieringManager.removeLakeTable(tableId); } + // for partitioned table, if all the partitions of this table have been deleted, + // there's no event to trigger resumeTableDeletions, so we need to trigger here + if (dropTableInfo.isPartitioned()) { + tableManager.resumeTableDeletions(); + } + // send update metadata request. updateTabletServerMetadataCache( new HashSet<>(coordinatorContext.getLiveTabletServers().values()), diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorRequestBatch.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorRequestBatch.java index 063ba58786..60eaa988c9 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorRequestBatch.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorRequestBatch.java @@ -83,6 +83,7 @@ public class CoordinatorRequestBatch { private static final Schema EMPTY_SCHEMA = Schema.newBuilder().build(); private static final TableDescriptor EMPTY_TABLE_DESCRIPTOR = TableDescriptor.builder().schema(EMPTY_SCHEMA).distributedBy(0).build(); + private static final String EMPTY_REMOTE_DATA_DIR = ""; // a map from tablet server to notify the leader and isr for each bucket. private final Map> @@ -694,7 +695,13 @@ private TableInfo getTableInfo(long tableId) { if (tableInfo == null) { if (tableQueuedForDeletion) { return TableInfo.of( - DELETED_TABLE_PATH, tableId, 0, EMPTY_TABLE_DESCRIPTOR, -1L, -1L); + DELETED_TABLE_PATH, + tableId, + 0, + EMPTY_TABLE_DESCRIPTOR, + EMPTY_REMOTE_DATA_DIR, + -1L, + -1L); } else { // it may happen that the table is dropped, but the partition still exists // when coordinator restarts, it won't consider it as deleted table, @@ -710,6 +717,7 @@ private TableInfo getTableInfo(long tableId) { DELETED_TABLE_ID, 0, EMPTY_TABLE_DESCRIPTOR, + EMPTY_REMOTE_DATA_DIR, -1L, -1L) : tableInfo; diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java index 1f9f3306e5..f07e9ee00e 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 @@ -34,6 +34,7 @@ import org.apache.fluss.server.authorizer.AuthorizerLoader; import org.apache.fluss.server.coordinator.lease.KvSnapshotLeaseManager; import org.apache.fluss.server.coordinator.rebalance.RebalanceManager; +import org.apache.fluss.server.coordinator.remote.RemoteDirDynamicLoader; import org.apache.fluss.server.metadata.CoordinatorMetadataCache; import org.apache.fluss.server.metadata.ServerMetadataCache; import org.apache.fluss.server.metrics.ServerMetricUtils; @@ -147,6 +148,9 @@ public class CoordinatorServer extends ServerBase { @GuardedBy("lock") private LakeCatalogDynamicLoader lakeCatalogDynamicLoader; + @GuardedBy("lock") + private RemoteDirDynamicLoader remoteDirDynamicLoader; + @GuardedBy("lock") private KvSnapshotLeaseManager kvSnapshotLeaseManager; @@ -187,10 +191,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(); @@ -235,6 +242,7 @@ protected void startServices() throws Exception { authorizer, lakeCatalogDynamicLoader, lakeTableTieringManager, + remoteDirDynamicLoader, dynamicConfigManager, ioExecutor, kvSnapshotLeaseManager); @@ -260,7 +268,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 @@ -480,6 +489,10 @@ CompletableFuture stopServices() { lakeCatalogDynamicLoader.close(); } + if (remoteDirDynamicLoader != null) { + remoteDirDynamicLoader.close(); + } + if (kvSnapshotLeaseManager != null) { kvSnapshotLeaseManager.close(); } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorService.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorService.java index 388f1816f4..df325443f1 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorService.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorService.java @@ -149,6 +149,7 @@ import org.apache.fluss.server.coordinator.lease.KvSnapshotLeaseManager; import org.apache.fluss.server.coordinator.producer.ProducerOffsetsManager; import org.apache.fluss.server.coordinator.rebalance.goal.Goal; +import org.apache.fluss.server.coordinator.remote.RemoteDirDynamicLoader; import org.apache.fluss.server.entity.CommitKvSnapshotData; import org.apache.fluss.server.entity.LakeTieringTableInfo; import org.apache.fluss.server.entity.TablePropertyChanges; @@ -243,6 +244,7 @@ public CoordinatorService( @Nullable Authorizer authorizer, LakeCatalogDynamicLoader lakeCatalogDynamicLoader, LakeTableTieringManager lakeTableTieringManager, + RemoteDirDynamicLoader remoteDirDynamicLoader, DynamicConfigManager dynamicConfigManager, ExecutorService ioExecutor, KvSnapshotLeaseManager kvSnapshotLeaseManager) { @@ -253,6 +255,7 @@ public CoordinatorService( metadataManager, authorizer, dynamicConfigManager, + remoteDirDynamicLoader, ioExecutor); this.defaultBucketNumber = conf.getInt(ConfigOptions.DEFAULT_BUCKET_NUMBER); this.defaultReplicationFactor = conf.getInt(ConfigOptions.DEFAULT_REPLICATION_FACTOR); @@ -432,9 +435,18 @@ public CompletableFuture createTable(CreateTableRequest req } } + // select remote data dir for table. + // remote data dir will be used to store table data for non-partitioned table and metadata + // (such as lake snapshot offset file) for partitioned table + String remoteDataDir = remoteDirDynamicLoader.getRemoteDirSelector().nextDataDir(); + // then create table; metadataManager.createTable( - tablePath, tableDescriptor, tableAssignment, request.isIgnoreIfExists()); + tablePath, + remoteDataDir, + tableDescriptor, + tableAssignment, + request.isIgnoreIfExists()); return CompletableFuture.completedFuture(new CreateTableResponse()); } @@ -637,9 +649,13 @@ public CompletableFuture createPartition( PartitionAssignment partitionAssignment = new PartitionAssignment(table.tableId, bucketAssignments); + // select remote data dir for partition + String remoteDataDir = remoteDirDynamicLoader.getRemoteDirSelector().nextDataDir(); + metadataManager.createPartition( tablePath, table.tableId, + remoteDataDir, partitionAssignment, partitionToCreate, request.isIgnoreIfNotExists()); @@ -688,6 +704,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 cc3808d871..9ddf39bc29 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/MetadataManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/MetadataManager.java @@ -44,13 +44,13 @@ import org.apache.fluss.metadata.TableChange; import org.apache.fluss.metadata.TableDescriptor; import org.apache.fluss.metadata.TableInfo; -import org.apache.fluss.metadata.TablePartition; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.security.acl.FlussPrincipal; import org.apache.fluss.server.entity.TablePropertyChanges; import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.server.zk.data.DatabaseRegistration; import org.apache.fluss.server.zk.data.PartitionAssignment; +import org.apache.fluss.server.zk.data.PartitionRegistration; import org.apache.fluss.server.zk.data.TableAssignment; import org.apache.fluss.server.zk.data.TableRegistration; import org.apache.fluss.shaded.zookeeper3.org.apache.zookeeper.KeeperException; @@ -275,6 +275,7 @@ public void completeDeletePartition(long partitionId) { * Returns -1 if the table already exists and ignoreIfExists is true. * * @param tablePath the table path + * @param remoteDataDir the remote data directory * @param tableToCreate the table descriptor describing the table to create * @param tableAssignment the table assignment, will be null when the table is partitioned table * @param ignoreIfExists whether to ignore if the table already exists @@ -282,6 +283,7 @@ public void completeDeletePartition(long partitionId) { */ public long createTable( TablePath tablePath, + String remoteDataDir, TableDescriptor tableToCreate, @Nullable TableAssignment tableAssignment, boolean ignoreIfExists) @@ -323,7 +325,9 @@ public long createTable( } // register the table zookeeperClient.registerTable( - tablePath, TableRegistration.newTable(tableId, tableToCreate), false); + tablePath, + TableRegistration.newTable(tableId, remoteDataDir, tableToCreate), + false); return tableId; }, "Fail to create table " + tablePath); @@ -689,13 +693,14 @@ public Set getPartitions(TablePath tablePath) { public void createPartition( TablePath tablePath, long tableId, + String remoteDataDir, PartitionAssignment partitionAssignment, ResolvedPartitionSpec partition, boolean ignoreIfExists) { String partitionName = partition.getPartitionName(); - Optional optionalTablePartition = - getOptionalTablePartition(tablePath, partitionName); - if (optionalTablePartition.isPresent()) { + Optional optionalPartitionRegistration = + getOptionalPartitionRegistration(tablePath, partitionName); + if (optionalPartitionRegistration.isPresent()) { if (ignoreIfExists) { return; } @@ -748,7 +753,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) { @@ -770,9 +780,9 @@ public void createPartition( public void dropPartition( TablePath tablePath, ResolvedPartitionSpec partition, boolean ignoreIfNotExists) { String partitionName = partition.getPartitionName(); - Optional optionalTablePartition = - getOptionalTablePartition(tablePath, partitionName); - if (!optionalTablePartition.isPresent()) { + Optional optionalPartitionRegistration = + getOptionalPartitionRegistration(tablePath, partitionName); + if (!optionalPartitionRegistration.isPresent()) { if (ignoreIfNotExists) { return; } @@ -794,7 +804,7 @@ public void dropPartition( } } - private Optional getOptionalTablePartition( + private Optional getOptionalPartitionRegistration( TablePath tablePath, String partitionName) { try { return zookeeperClient.getPartition(tablePath, partitionName); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/TableManager.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/TableManager.java index b92a2dad6e..c0bd62c9ae 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/TableManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/TableManager.java @@ -17,12 +17,14 @@ package org.apache.fluss.server.coordinator; +import org.apache.fluss.metadata.PartitionInfo; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableBucketReplica; import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePartition; import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.server.coordinator.remote.RemoteStorageCleaner; import org.apache.fluss.server.coordinator.statemachine.BucketState; import org.apache.fluss.server.coordinator.statemachine.ReplicaState; import org.apache.fluss.server.coordinator.statemachine.ReplicaStateMachine; @@ -114,16 +116,16 @@ public void onCreateNewTable( * * @param tablePath the table path * @param tableId the table id - * @param partitionId the id for the created partition - * @param partitionName the name for the created partition + * @param partitionInfo the info for the created partition * @param partitionAssignment the assignment for the created partition. */ public void onCreateNewPartition( TablePath tablePath, long tableId, - long partitionId, - String partitionName, + PartitionInfo partitionInfo, PartitionAssignment partitionAssignment) { + String partitionName = partitionInfo.getPartitionName(); + long partitionId = partitionInfo.getPartitionId(); LOG.info( "New partition {} with partition id {} and assignment {} for table {}.", partitionName, @@ -140,7 +142,7 @@ public void onCreateNewPartition( TableBucket tableBucket = new TableBucket(tableId, partitionId, bucket); coordinatorContext.updateBucketReplicaAssignment(tableBucket, replicas); coordinatorContext.putPartition( - partitionId, PhysicalTablePath.of(tablePath, partitionName)); + partitionId, PhysicalTablePath.of(tablePath, partitionName), partitionInfo); newTableBuckets.add(tableBucket); } onCreateNewTableBucket(tableId, newTableBuckets); @@ -206,7 +208,7 @@ public void resumeDeletions() { resumePartitionDeletions(); } - private void resumeTableDeletions() { + public void resumeTableDeletions() { Set tablesToBeDeleted = new HashSet<>(coordinatorContext.getTablesToBeDeleted()); Set eligibleTableDeletion = new HashSet<>(); @@ -214,6 +216,16 @@ private void resumeTableDeletions() { // if all replicas are marked as deleted successfully, then table deletion is done if (coordinatorContext.areAllReplicasInState( tableId, ReplicaState.ReplicaDeletionSuccessful)) { + TableInfo tableInfo = coordinatorContext.getTableInfoById(tableId); + if (tableInfo != null && tableInfo.isPartitioned()) { + if (coordinatorContext.hasPartitionsToDelete(tableId)) { + // Skip table deletion until all partitions are deleted + LOG.debug( + "Table {} has partitions still being deleted, skip table deletion for now.", + tableId); + continue; + } + } completeDeleteTable(tableId); LOG.info("Deletion of table with id {} successfully completed.", tableId); } @@ -232,12 +244,14 @@ private void resumePartitionDeletions() { Set partitionsToDelete = new HashSet<>(coordinatorContext.getPartitionsToBeDeleted()); Set eligiblePartitionDeletion = new HashSet<>(); + boolean hasPartitionCompleted = false; for (TablePartition partition : partitionsToDelete) { // if all replicas are marked as deleted successfully, then partition deletion is done if (coordinatorContext.areAllReplicasInState( partition, ReplicaState.ReplicaDeletionSuccessful)) { completeDeletePartition(partition); + hasPartitionCompleted = true; LOG.info("Deletion of partition {} successfully completed.", partition); } if (isEligibleForDeletion(partition)) { @@ -249,6 +263,12 @@ private void resumePartitionDeletions() { onDeletePartition(partition.getTableId(), partition.getPartitionId()); } } + + // If any partition was completed, check if we can now complete the table deletion + // for partitioned tables that were waiting for all partitions to be deleted. + if (hasPartitionCompleted) { + resumeTableDeletions(); + } } private void completeDeleteTable(long tableId) { @@ -275,7 +295,10 @@ private void asyncDeleteRemoteDirectory(long tableId) { TableInfo tableInfo = coordinatorContext.getTableInfoById(tableId); if (tableInfo != null) { remoteStorageCleaner.asyncDeleteTableRemoteDir( - tableInfo.getTablePath(), tableInfo.hasPrimaryKey(), tableId); + tableInfo.getRemoteDataDir(), + tableInfo.getTablePath(), + tableInfo.hasPrimaryKey(), + tableId); } } @@ -286,8 +309,11 @@ private void asyncDeleteRemoteDirectory(TablePartition tablePartition) { if (tableInfo != null) { String partitionName = coordinatorContext.getPartitionName(tablePartition.getPartitionId()); + PartitionInfo partitionInfo = + coordinatorContext.getPartitionInfoById(tablePartition.getPartitionId()); if (partitionName != null) { remoteStorageCleaner.asyncDeletePartitionRemoteDir( + partitionInfo.getRemoteDataDir(), PhysicalTablePath.of(tableInfo.getTablePath(), partitionName), tableInfo.hasPrimaryKey(), tablePartition); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/CreatePartitionEvent.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/CreatePartitionEvent.java index 646bfb1082..a6d8b5d2cd 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/CreatePartitionEvent.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/CreatePartitionEvent.java @@ -31,18 +31,21 @@ public class CreatePartitionEvent implements CoordinatorEvent { private final String partitionName; private final long partitionId; private final PartitionAssignment partitionAssignment; + private final String remoteDataDir; public CreatePartitionEvent( TablePath tablePath, long tableId, long partitionId, String partitionName, - PartitionAssignment partitionAssignment) { + PartitionAssignment partitionAssignment, + String remoteDataDir) { this.tablePath = tablePath; this.tableId = tableId; this.partitionId = partitionId; this.partitionName = partitionName; this.partitionAssignment = partitionAssignment; + this.remoteDataDir = remoteDataDir; } public TablePath getTablePath() { @@ -65,6 +68,10 @@ public PartitionAssignment getPartitionAssignment() { return partitionAssignment; } + public String getRemoteDataDir() { + return remoteDataDir; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -78,12 +85,14 @@ public boolean equals(Object o) { && partitionId == that.partitionId && Objects.equals(tablePath, that.tablePath) && Objects.equals(partitionName, that.partitionName) - && Objects.equals(partitionAssignment, that.partitionAssignment); + && Objects.equals(partitionAssignment, that.partitionAssignment) + && Objects.equals(remoteDataDir, that.remoteDataDir); } @Override public int hashCode() { - return Objects.hash(tablePath, tableId, partitionName, partitionId, partitionAssignment); + return Objects.hash( + tablePath, tableId, partitionName, partitionId, partitionAssignment, remoteDataDir); } @Override @@ -100,6 +109,8 @@ public String toString() { + partitionId + ", partitionAssignment=" + partitionAssignment + + ", remoteDataDir=" + + remoteDataDir + '}'; } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/TableChangeWatcher.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/TableChangeWatcher.java index 5b4821f194..ecfd7591d4 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/TableChangeWatcher.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/watcher/TableChangeWatcher.java @@ -22,7 +22,6 @@ import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.SchemaInfo; import org.apache.fluss.metadata.TableInfo; -import org.apache.fluss.metadata.TablePartition; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.server.coordinator.event.CreatePartitionEvent; import org.apache.fluss.server.coordinator.event.CreateTableEvent; @@ -33,6 +32,7 @@ import org.apache.fluss.server.coordinator.event.TableRegistrationChangeEvent; import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.server.zk.data.PartitionAssignment; +import org.apache.fluss.server.zk.data.PartitionRegistration; import org.apache.fluss.server.zk.data.TableAssignment; import org.apache.fluss.server.zk.data.TableRegistration; import org.apache.fluss.server.zk.data.ZkData.DatabasesZNode; @@ -149,7 +149,8 @@ public void event(Type type, ChildData oldData, ChildData newData) { PartitionZNode.parsePath(oldData.getPath()); if (physicalTablePath != null) { // it's for deletion of a table partition node - TablePartition partition = PartitionZNode.decode(oldData.getData()); + PartitionRegistration partition = + PartitionZNode.decode(oldData.getData()); eventManager.put( new DropPartitionEvent( partition.getTableId(), @@ -223,7 +224,7 @@ private void processCreateTable(TablePath tablePath, ChildData tableData) { private void processCreatePartition( TablePath tablePath, String partitionName, ChildData partitionData) { - TablePartition partition = PartitionZNode.decode(partitionData.getData()); + PartitionRegistration partition = PartitionZNode.decode(partitionData.getData()); long partitionId = partition.getPartitionId(); long tableId = partition.getTableId(); PartitionAssignment partitionAssignment; @@ -247,9 +248,19 @@ private void processCreatePartition( e); return; } + // Use default remote data dir configured by ConfigOptions.REMOTE_DATA_DIR for + // PartitionRegistration created by old cluster. + String remoteDataDir = + Optional.ofNullable(partition.getRemoteDataDir()) + .orElseGet(zooKeeperClient::getDefaultRemoteDataDir); eventManager.put( new CreatePartitionEvent( - tablePath, tableId, partitionId, partitionName, partitionAssignment)); + tablePath, + tableId, + partitionId, + partitionName, + partitionAssignment, + remoteDataDir)); } private void processTableRegistrationChange(TablePath tablePath, ChildData newData) { 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..27a1276de1 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/remote/RemoteDirDynamicLoader.java @@ -0,0 +1,171 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.coordinator.remote; + +import org.apache.fluss.config.ConfigOption; +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.config.cluster.ServerReconfigurable; +import org.apache.fluss.exception.ConfigException; +import org.apache.fluss.exception.IllegalConfigurationException; + +import java.util.HashSet; +import java.util.List; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; + +import static org.apache.fluss.config.FlussConfigUtils.validateRemoteDataDirs; + +/** + * Dynamic loader for remote data directories that supports runtime reconfiguration. + * + *

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

When creating a new table or partition, the coordinator server uses this loader to select an + * appropriate remote data directory based on the configured selection strategy (see {@link + * org.apache.fluss.config.ConfigOptions#REMOTE_DATA_DIRS_STRATEGY}). + */ +public class RemoteDirDynamicLoader implements ServerReconfigurable, AutoCloseable { + + private volatile RemoteDirSelector remoteDirSelector; + private Configuration currentConfiguration; + + public RemoteDirDynamicLoader(Configuration configuration) { + this.currentConfiguration = configuration; + this.remoteDirSelector = createRemoteDirSelector(configuration); + } + + public RemoteDirSelector getRemoteDirSelector() { + return remoteDirSelector; + } + + public List getRemoteDataDirs() { + return currentConfiguration.get(ConfigOptions.REMOTE_DATA_DIRS); + } + + @Override + public void validate(Configuration newConfig) throws ConfigException { + // Validate new remote data dirs contain all old remote data dirs + Optional> newRemoteDataDirsOp = + newConfig.getOptional(ConfigOptions.REMOTE_DATA_DIRS); + if (newRemoteDataDirsOp.isPresent()) { + List oldRemoteDataDirs = + currentConfiguration.get(ConfigOptions.REMOTE_DATA_DIRS); + Set newRemoteDataDirs = new HashSet<>(newRemoteDataDirsOp.get()); + if (!newRemoteDataDirs.containsAll(oldRemoteDataDirs)) { + throw new ConfigException( + String.format( + "New %s: %s must contain all old %s: %s. " + + "If you want the Fluss cluster to stop transferring data to a certain path, " + + "keep it in %s and set its weight to 0 in %s.", + ConfigOptions.REMOTE_DATA_DIRS.key(), + newRemoteDataDirsOp.get(), + ConfigOptions.REMOTE_DATA_DIRS.key(), + oldRemoteDataDirs, + ConfigOptions.REMOTE_DATA_DIRS.key(), + ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS.key())); + } + } + + Configuration mergedConfig = mergeConfigurations(currentConfiguration, newConfig); + try { + validateRemoteDataDirs(mergedConfig); + } catch (IllegalConfigurationException e) { + throw new ConfigException(e.getMessage()); + } + } + + @Override + public void reconfigure(Configuration newConfig) throws ConfigException { + if (strategyChanged(newConfig) + || remoteDataDirsChanged(newConfig) + || weightsChanged(newConfig)) { + // Create a new container with the merged configuration + Configuration mergedConfig = mergeConfigurations(currentConfiguration, newConfig); + this.remoteDirSelector = createRemoteDirSelector(mergedConfig); + this.currentConfiguration = mergedConfig; + } + } + + private RemoteDirSelector createRemoteDirSelector(Configuration conf) { + ConfigOptions.RemoteDataDirStrategy strategy = + conf.get(ConfigOptions.REMOTE_DATA_DIRS_STRATEGY); + String remoteDataDir = conf.get(ConfigOptions.REMOTE_DATA_DIR); + List remoteDataDirs = conf.get(ConfigOptions.REMOTE_DATA_DIRS); + List weights = conf.get(ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS); + + switch (strategy) { + case ROUND_ROBIN: + return new RoundRobinRemoteDirSelector(remoteDataDir, remoteDataDirs); + case WEIGHTED_ROUND_ROBIN: + return new WeightedRoundRobinRemoteDirSelector( + remoteDataDir, remoteDataDirs, weights); + default: + throw new IllegalArgumentException( + "Unsupported remote data directory select strategy: " + strategy); + } + } + + private boolean strategyChanged(Configuration newConfig) { + return hasConfigChanged(newConfig, ConfigOptions.REMOTE_DATA_DIRS_STRATEGY); + } + + private boolean remoteDataDirsChanged(Configuration newConfig) { + return hasConfigChanged(newConfig, ConfigOptions.REMOTE_DATA_DIRS); + } + + private boolean weightsChanged(Configuration newConfig) { + return hasConfigChanged(newConfig, ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS); + } + + /** + * Checks if a specific configuration option has changed in the new config. + * + * @param newConfig the new configuration + * @param option the configuration option to check + * @param 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 + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/remote/RemoteDirSelector.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/remote/RemoteDirSelector.java new file mode 100644 index 0000000000..f98dcaa623 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/remote/RemoteDirSelector.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.coordinator.remote; + +import javax.annotation.concurrent.ThreadSafe; + +/** + * Interface for selecting remote data directories from a list of available directories. + * + *

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

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

This method should implement the selection strategy (e.g., round-robin, weighted + * round-robin) to choose from the available remote data directories. + * + * @return the next remote data directory path to use. If {@link + * org.apache.fluss.config.ConfigOptions#REMOTE_DATA_DIRS} is empty, should always return + * {@link org.apache.fluss.config.ConfigOptions#REMOTE_DATA_DIR}. + */ + String nextDataDir(); +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/RemoteStorageCleaner.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/remote/RemoteStorageCleaner.java similarity index 74% rename from fluss-server/src/main/java/org/apache/fluss/server/coordinator/RemoteStorageCleaner.java rename to fluss-server/src/main/java/org/apache/fluss/server/coordinator/remote/RemoteStorageCleaner.java index d206f07d2c..6e98374933 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/RemoteStorageCleaner.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/remote/RemoteStorageCleaner.java @@ -15,11 +15,9 @@ * limitations under the License. */ -package org.apache.fluss.server.coordinator; +package org.apache.fluss.server.coordinator.remote; -import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; -import org.apache.fluss.exception.FlussRuntimeException; import org.apache.fluss.fs.FileSystem; import org.apache.fluss.fs.FsPath; import org.apache.fluss.metadata.PhysicalTablePath; @@ -33,38 +31,25 @@ import java.io.IOException; import java.util.concurrent.ExecutorService; -/** A cleaner for cleaning kv snapshots and log segments files of table. */ +/** A cleaner for cleaning kv snapshots, log segments files and lake metadata of table. */ public class RemoteStorageCleaner { private static final Logger LOG = LoggerFactory.getLogger(RemoteStorageCleaner.class); - private final FsPath remoteKvDir; - - private final FsPath remoteLogDir; - - private final String remoteDataDir; - - private final FileSystem remoteFileSystem; - private final ExecutorService ioExecutor; public RemoteStorageCleaner(Configuration configuration, ExecutorService ioExecutor) { - this.remoteKvDir = FlussPaths.remoteKvDir(configuration); - this.remoteLogDir = FlussPaths.remoteLogDir(configuration); - this.remoteDataDir = configuration.getString(ConfigOptions.REMOTE_DATA_DIR); this.ioExecutor = ioExecutor; - try { - this.remoteFileSystem = remoteKvDir.getFileSystem(); - } catch (IOException e) { - throw new FlussRuntimeException( - "Fail to get remote file system for path " + remoteKvDir, e); - } } - public void asyncDeleteTableRemoteDir(TablePath tablePath, boolean isKvTable, long tableId) { + public void asyncDeleteTableRemoteDir( + String remoteDataDir, TablePath tablePath, boolean isKvTable, long tableId) { if (isKvTable) { + FsPath remoteKvDir = FlussPaths.remoteKvDir(remoteDataDir); asyncDeleteDir(FlussPaths.remoteTableDir(remoteKvDir, tablePath, tableId)); } + + FsPath remoteLogDir = FlussPaths.remoteLogDir(remoteDataDir); asyncDeleteDir(FlussPaths.remoteTableDir(remoteLogDir, tablePath, tableId)); // Always delete lake snapshot metadata directory, regardless of isLakeEnabled flag. @@ -76,11 +61,17 @@ public void asyncDeleteTableRemoteDir(TablePath tablePath, boolean isKvTable, lo } public void asyncDeletePartitionRemoteDir( - PhysicalTablePath physicalTablePath, boolean isKvTable, TablePartition tablePartition) { + String remoteDataDir, + PhysicalTablePath physicalTablePath, + boolean isKvTable, + TablePartition tablePartition) { if (isKvTable) { + FsPath remoteKvDir = FlussPaths.remoteKvDir(remoteDataDir); asyncDeleteDir( FlussPaths.remotePartitionDir(remoteKvDir, physicalTablePath, tablePartition)); } + + FsPath remoteLogDir = FlussPaths.remoteLogDir(remoteDataDir); asyncDeleteDir( FlussPaths.remotePartitionDir(remoteLogDir, physicalTablePath, tablePartition)); } @@ -89,6 +80,7 @@ private void asyncDeleteDir(FsPath fsPath) { ioExecutor.submit( () -> { try { + FileSystem remoteFileSystem = fsPath.getFileSystem(); if (remoteFileSystem.exists(fsPath)) { remoteFileSystem.delete(fsPath, true); } 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..b627ce4fe7 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/remote/RoundRobinRemoteDirSelector.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.coordinator.remote; + +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Round-robin remote data dir selector. + * + *

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

Example: For directories [A, B, C], the selection sequence would be: A, B, C, A, B, C, ... + */ +public class RoundRobinRemoteDirSelector implements RemoteDirSelector { + + private final String remoteDataDir; + private final List remoteDataDirs; + + // Current position in the round-robin cycle. + private final AtomicInteger position; + + public RoundRobinRemoteDirSelector(String remoteDataDir, List remoteDataDirs) { + this.remoteDataDir = remoteDataDir; + this.remoteDataDirs = Collections.unmodifiableList(remoteDataDirs); + this.position = new AtomicInteger(0); + } + + @Override + public String nextDataDir() { + if (remoteDataDirs.isEmpty()) { + return remoteDataDir; + } + + int index = position.getAndUpdate(i -> (i + 1) % remoteDataDirs.size()); + return remoteDataDirs.get(index); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/remote/WeightedRoundRobinRemoteDirSelector.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/remote/WeightedRoundRobinRemoteDirSelector.java new file mode 100644 index 0000000000..302ebe3162 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/remote/WeightedRoundRobinRemoteDirSelector.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.coordinator.remote; + +import java.util.Collections; +import java.util.List; + +/** + * Weighted Round-robin remote data dir selector using Interleaved (Smooth) Weighted Round-Robin + * algorithm. + * + *

This implementation uses the smooth weighted round-robin algorithm (also known as interleaved + * weighted round-robin), which distributes selections more evenly compared to traditional weighted + * round-robin. Instead of selecting the same node consecutively based on its weight, it interleaves + * selections to achieve a smoother distribution. + * + *

Algorithm: + * + *

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

Example: For nodes A, B, C with weights 5, 1, 1 (total=7), the selection sequence would be: A, + * A, B, A, C, A, A (instead of A, A, A, A, A, B, C in traditional WRR). + */ +public class WeightedRoundRobinRemoteDirSelector implements RemoteDirSelector { + + private final String remoteDataDir; + private final List remoteDataDirs; + private final int[] weights; + private final int totalWeight; + + // Current weights for each node, used in smooth weighted round-robin + private final int[] currentWeights; + + // Lock object for thread safety + private final Object lock = new Object(); + + public WeightedRoundRobinRemoteDirSelector( + String remoteDataDir, List remoteDataDirs, List weights) { + this.remoteDataDir = remoteDataDir; + this.remoteDataDirs = Collections.unmodifiableList(remoteDataDirs); + + // Convert weights list to array for better performance + this.weights = new int[weights.size()]; + int sum = 0; + for (int i = 0; i < weights.size(); i++) { + this.weights[i] = weights.get(i); + sum += this.weights[i]; + } + this.totalWeight = sum; + + // Initialize current weights to 0 + this.currentWeights = new int[remoteDataDirs.size()]; + } + + @Override + public String nextDataDir() { + if (remoteDataDirs.isEmpty()) { + return remoteDataDir; + } + + 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 1f0c9c2689..e26d955888 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; @@ -126,10 +125,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. */ @@ -146,15 +141,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 = @@ -409,12 +401,13 @@ public KvTablet loadKv(File tabletDir, SchemaGetter schemaGetter) throws Excepti } public void deleteRemoteKvSnapshot( - PhysicalTablePath physicalTablePath, TableBucket tableBucket) { + String 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 (remoteKvTabletDir.getFileSystem().exists(remoteKvTabletDir)) { + remoteKvTabletDir.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 9bec8977cc..932b9f81ac 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 @@ -21,11 +21,9 @@ 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 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 org.slf4j.Logger; @@ -57,8 +55,6 @@ public class DefaultSnapshotContext implements SnapshotContext, ServerReconfigur private final int maxFetchLogSizeInRecoverKv; - private final FsPath remoteKvDir; - private DefaultSnapshotContext( ZooKeeperClient zooKeeperClient, CompletedKvSnapshotCommitter completedKvSnapshotCommitter, @@ -68,7 +64,6 @@ private DefaultSnapshotContext( KvSnapshotDataDownloader kvSnapshotDataDownloader, long kvSnapshotIntervalMs, int writeBufferSizeInBytes, - FsPath remoteKvDir, CompletedSnapshotHandleStore completedSnapshotHandleStore, int maxFetchLogSizeInRecoverKv) { this.zooKeeperClient = zooKeeperClient; @@ -79,7 +74,6 @@ private DefaultSnapshotContext( this.kvSnapshotDataDownloader = kvSnapshotDataDownloader; this.kvSnapshotIntervalMs = kvSnapshotIntervalMs; this.writeBufferSizeInBytes = writeBufferSizeInBytes; - this.remoteKvDir = remoteKvDir; this.completedSnapshotHandleStore = completedSnapshotHandleStore; this.maxFetchLogSizeInRecoverKv = maxFetchLogSizeInRecoverKv; @@ -99,19 +93,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; } @@ -121,6 +117,7 @@ public KvSnapshotDataDownloader getSnapshotDataDownloader() { return kvSnapshotDataDownloader; } + @Override public ScheduledExecutorService getSnapshotScheduler() { return snapshotScheduler; } @@ -140,10 +137,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..6ce87b35a4 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 @@ -65,24 +65,14 @@ 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 { - this.remoteLogDir = FlussPaths.remoteLogDir(conf); - this.fileSystem = remoteLogDir.getFileSystem(); + public DefaultRemoteLogStorage(Configuration conf, ExecutorService ioExecutor) { this.writeBufferSize = (int) conf.get(ConfigOptions.REMOTE_FS_WRITE_BUFFER_SIZE).getBytes(); this.ioExecutor = ioExecutor; } - @Override - public FsPath getRemoteLogDir() { - return remoteLogDir; - } - /** * Copy log segments to remote path. * @@ -141,6 +131,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 +145,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 +157,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 +171,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 +188,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 +208,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 +220,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 +243,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 +318,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 8c7d0d8832..5d1d16b53d 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 @@ -281,6 +281,7 @@ private long copyLogSegmentFilesToRemote( .remoteLogEndOffset(segmentEndOffset) .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 ba34143af6..da5c1a90e3 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; @@ -129,10 +130,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()) { @@ -141,8 +138,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()) { @@ -151,6 +150,20 @@ 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) { + LOG.info( + "RemoteLogManifest loaded from old version without remoteLogDir, " + + "setting remoteLogDir to {} for bucket {}", + remoteLogDir, + manifest.getTableBucket()); + manifest = manifest.newManifest(remoteLogDir); + } + remoteLog.loadRemoteLogManifest(manifest); } remoteLog.getRemoteLogEndOffset().ifPresent(log::updateRemoteLogEndOffset); @@ -212,8 +225,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); } } @@ -264,11 +278,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..31b9c88afc 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 @@ -66,13 +66,6 @@ public static String getFileSuffix(IndexType indexType) { } } - /** - * Returns the remote log directory. - * - * @return the remote log directory. - */ - FsPath getRemoteLogDir(); - /** * Copies the given {@link LogSegmentFiles} provided for the given {@link RemoteLogSegment}. * This includes log segment and its auxiliary indexes like offset index and writer id snapshot @@ -154,11 +147,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 953fb7e17f..1b89b34186 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(); } @@ -258,6 +266,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. @@ -349,7 +361,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 f73bcdda25..e9491b335b 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 @@ -147,6 +147,7 @@ public final class Replica { private final PhysicalTablePath physicalPath; private final TableBucket tableBucket; + private final String remoteDataDir; private final LogManager logManager; private final LogTablet logTablet; @@ -223,6 +224,7 @@ public Replica( FatalErrorHandler fatalErrorHandler, BucketMetricGroup bucketMetricGroup, TableInfo tableInfo, + String remoteDataDir, Clock clock) throws Exception { this.physicalPath = physicalPath; @@ -254,6 +256,7 @@ public Replica( this.logTablet = createLog(lazyHighWatermarkCheckpoint); this.logTablet.updateIsDataLakeEnabled(tableConfig.isDataLakeEnabled()); + this.remoteDataDir = remoteDataDir; this.clock = clock; registerMetrics(); } @@ -388,6 +391,10 @@ public LogFormat getLogFormat() { return logFormat; } + public String getRemoteDataDir() { + return remoteDataDir; + } + public void makeLeader(NotifyLeaderAndIsrData data) throws IOException { boolean leaderHWIncremented = inWriteLock( @@ -905,9 +912,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 3fc0242092..e780b7645c 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 @@ -35,6 +35,7 @@ import org.apache.fluss.metadata.LogFormat; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.Schema; +import org.apache.fluss.metadata.SchemaInfo; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePath; @@ -105,6 +106,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; @@ -139,7 +142,9 @@ import java.util.stream.Stream; import static org.apache.fluss.config.ConfigOptions.KV_FORMAT_VERSION_2; -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.checkArgument; import static org.apache.fluss.utils.Preconditions.checkNotNull; @@ -1460,11 +1465,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(), @@ -1793,7 +1797,9 @@ private StopReplicaResultForBucket stopReplica( replicaToDelete, deleteRemote && replicaToDelete.isLeader()); if (deleteRemote && replicaToDelete.isLeader()) { kvManager.deleteRemoteKvSnapshot( - replicaToDelete.getPhysicalTablePath(), replicaToDelete.getTableBucket()); + replicaToDelete.getRemoteDataDir(), + replicaToDelete.getPhysicalTablePath(), + replicaToDelete.getTableBucket()); } } @@ -1844,6 +1850,20 @@ private void dropEmptyTableOrPartitionDir(Path dir, long id, String dirType) { } } + private String getRemoteDataDir( + TableBucket tb, + PhysicalTablePath physicalTablePath, + TableRegistration tableRegistration) + throws Exception { + if (tb.getPartitionId() != null) { + PartitionRegistration partitionRegistration = + getPartitionRegistration(zkClient, physicalTablePath); + return partitionRegistration.getRemoteDataDir(); + } else { + return tableRegistration.remoteDataDir; + } + } + protected Optional maybeCreateReplica(NotifyLeaderAndIsrData data) { Optional replicaOpt = Optional.empty(); try { @@ -1852,7 +1872,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); + + String remoteDataDir = getRemoteDataDir(tb, physicalTablePath, tableRegistration); boolean isKvTable = tableInfo.hasPrimaryKey(); BucketMetricGroup bucketMetricGroup = @@ -1877,6 +1902,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 93e8951b7f..64a06a9a7f 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 @@ -39,6 +39,7 @@ import org.apache.fluss.server.authorizer.AuthorizerLoader; import org.apache.fluss.server.coordinator.LakeCatalogDynamicLoader; import org.apache.fluss.server.coordinator.MetadataManager; +import org.apache.fluss.server.coordinator.remote.RemoteDirDynamicLoader; import org.apache.fluss.server.kv.KvManager; import org.apache.fluss.server.kv.snapshot.DefaultCompletedKvSnapshotCommitter; import org.apache.fluss.server.log.LogManager; @@ -164,6 +165,9 @@ public class TabletServer extends ServerBase { @GuardedBy("lock") private LakeCatalogDynamicLoader lakeCatalogDynamicLoader; + @GuardedBy("lock") + private RemoteDirDynamicLoader remoteDirDynamicLoader; + @GuardedBy("lock") private CoordinatorGateway coordinatorGateway; @@ -215,10 +219,12 @@ protected void startServices() throws Exception { this.lakeCatalogDynamicLoader = new LakeCatalogDynamicLoader(conf, pluginManager, false); + this.remoteDirDynamicLoader = new RemoteDirDynamicLoader(conf); MetadataManager metadataManager = new MetadataManager(zkClient, conf, lakeCatalogDynamicLoader); this.dynamicConfigManager = new DynamicConfigManager(zkClient, conf, false); dynamicConfigManager.register(lakeCatalogDynamicLoader); + dynamicConfigManager.register(remoteDirDynamicLoader); this.metadataCache = new TabletServerMetadataCache(metadataManager); @@ -288,6 +294,7 @@ protected void startServices() throws Exception { metadataManager, authorizer, dynamicConfigManager, + remoteDirDynamicLoader, ioExecutor); RequestsMetrics requestsMetrics = diff --git a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java index b511f21785..1f79fa9ba5 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/tablet/TabletService.java @@ -71,6 +71,7 @@ import org.apache.fluss.server.RpcServiceBase; import org.apache.fluss.server.authorizer.Authorizer; import org.apache.fluss.server.coordinator.MetadataManager; +import org.apache.fluss.server.coordinator.remote.RemoteDirDynamicLoader; import org.apache.fluss.server.entity.FetchReqInfo; import org.apache.fluss.server.entity.NotifyLeaderAndIsrData; import org.apache.fluss.server.entity.UserContext; @@ -142,6 +143,7 @@ public TabletService( MetadataManager metadataManager, @Nullable Authorizer authorizer, DynamicConfigManager dynamicConfigManager, + RemoteDirDynamicLoader remoteDirDynamicLoader, ExecutorService ioExecutor) { super( remoteFileSystem, @@ -150,6 +152,7 @@ public TabletService( metadataManager, authorizer, dynamicConfigManager, + remoteDirDynamicLoader, ioExecutor); this.serviceName = "server-" + serverId; this.replicaManager = replicaManager; diff --git a/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java b/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java index 6f5672ff46..42f954774e 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java @@ -112,6 +112,7 @@ import org.apache.fluss.rpc.messages.PbFetchLogReqForTable; import org.apache.fluss.rpc.messages.PbFetchLogRespForBucket; import org.apache.fluss.rpc.messages.PbFetchLogRespForTable; +import org.apache.fluss.rpc.messages.PbFileSystemSecurityToken; import org.apache.fluss.rpc.messages.PbKeyValue; import org.apache.fluss.rpc.messages.PbKvSnapshot; import org.apache.fluss.rpc.messages.PbKvSnapshotLeaseForBucket; @@ -530,6 +531,7 @@ private static PbTableMetadata toPbTableMetadata(TableMetadata tableMetadata) { .setTableId(tableInfo.getTableId()) .setSchemaId(tableInfo.getSchemaId()) .setTableJson(tableInfo.toTableDescriptor().toJsonBytes()) + .setRemoteDataDir(tableInfo.getRemoteDataDir()) .setCreatedTime(tableInfo.getCreatedTime()) .setModifiedTime(tableInfo.getModifiedTime()); TablePath tablePath = tableInfo.getTablePath(); @@ -588,6 +590,14 @@ private static TableMetadata toTableMetaData(PbTableMetadata pbTableMetadata) { tableId, pbTableMetadata.getSchemaId(), TableDescriptor.fromJsonBytes(pbTableMetadata.getTableJson()), + // For backword capability. When an older Coordinator sends an + // UpdateMetadataRequest to a newer TabletServer, the remoteDataDir will be + // missing. In this case, setting it to null is acceptable because the + // TabletServerMetadataCache does not maintain any remoteDataDir + // information. + pbTableMetadata.hasRemoteDataDir() + ? pbTableMetadata.getRemoteDataDir() + : null, pbTableMetadata.getCreatedTime(), pbTableMetadata.getModifiedTime()); @@ -1469,26 +1479,49 @@ private static List toPbSnapshotFileHandles( } public static GetFileSystemSecurityTokenResponse toGetFileSystemSecurityTokenResponse( - String filesystemSchema, ObtainedSecurityToken obtainedSecurityToken) { + ObtainedSecurityToken obtainedSecurityToken, + Collection obtainedSecurityTokens) { GetFileSystemSecurityTokenResponse getFileSystemSecurityTokenResponse = new GetFileSystemSecurityTokenResponse() .setToken(obtainedSecurityToken.getToken()) - .setSchema(filesystemSchema); + .setSchema(obtainedSecurityToken.getScheme()); + + obtainedSecurityToken + .getAuthority() + .ifPresent(getFileSystemSecurityTokenResponse::setAuthority); obtainedSecurityToken .getValidUntil() .ifPresent(getFileSystemSecurityTokenResponse::setExpirationTime); - List pbKeyValues = - new ArrayList<>(obtainedSecurityToken.getAdditionInfos().size()); - for (Map.Entry entry : - obtainedSecurityToken.getAdditionInfos().entrySet()) { - pbKeyValues.add(new PbKeyValue().setKey(entry.getKey()).setValue(entry.getValue())); + getFileSystemSecurityTokenResponse.addAllAdditionInfos( + toPbKeyValues(obtainedSecurityToken.getAdditionInfos())); + + List securityTokens = new ArrayList<>(); + for (ObtainedSecurityToken token : obtainedSecurityTokens) { + PbFileSystemSecurityToken pbFileSystemSecurityToken = + new PbFileSystemSecurityToken() + .setToken(token.getToken()) + .setSchema(token.getScheme()); + + token.getAuthority().ifPresent(pbFileSystemSecurityToken::setAuthority); + token.getValidUntil().ifPresent(pbFileSystemSecurityToken::setExpirationTime); + pbFileSystemSecurityToken.addAllAdditionInfos(toPbKeyValues(token.getAdditionInfos())); } - getFileSystemSecurityTokenResponse.addAllAdditionInfos(pbKeyValues); + + getFileSystemSecurityTokenResponse.addAllTokens(securityTokens); + return getFileSystemSecurityTokenResponse; } + private static List toPbKeyValues(Map keyValues) { + List pbKeyValues = new ArrayList<>(keyValues.size()); + for (Map.Entry entry : keyValues.entrySet()) { + pbKeyValues.add(new PbKeyValue().setKey(entry.getKey()).setValue(entry.getValue())); + } + return pbKeyValues; + } + public static CommitRemoteLogManifestData getCommitRemoteLogManifestData( CommitRemoteLogManifestRequest request) { return new CommitRemoteLogManifestData( diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java index d088e103ce..01ae79ebd6 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.config.FlussConfigUtils; import org.apache.fluss.metadata.DatabaseSummary; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.ResolvedPartitionSpec; @@ -46,6 +47,7 @@ import org.apache.fluss.server.zk.data.DatabaseRegistration; import org.apache.fluss.server.zk.data.LeaderAndIsr; import org.apache.fluss.server.zk.data.PartitionAssignment; +import org.apache.fluss.server.zk.data.PartitionRegistration; import org.apache.fluss.server.zk.data.RebalanceTask; import org.apache.fluss.server.zk.data.RemoteLogManifestHandle; import org.apache.fluss.server.zk.data.ResourceAcl; @@ -150,6 +152,8 @@ public class ZooKeeperClient implements AutoCloseable { private final Semaphore inFlightRequests; private final Configuration configuration; + private final String defaultRemoteDataDir; + public ZooKeeperClient( CuratorFrameworkWithUnhandledErrorListener curatorFrameworkWrapper, Configuration configuration) { @@ -164,6 +168,8 @@ public ZooKeeperClient( configuration.getInt(ConfigOptions.ZOOKEEPER_MAX_INFLIGHT_REQUESTS); this.inFlightRequests = new Semaphore(maxInFlightRequests); this.configuration = configuration; + + this.defaultRemoteDataDir = FlussConfigUtils.getDefaultRemoteDataDir(configuration); } public Optional getOrEmpty(String path) throws Exception { @@ -174,6 +180,10 @@ public Optional getOrEmpty(String path) throws Exception { } } + public String getDefaultRemoteDataDir() { + return defaultRemoteDataDir; + } + // -------------------------------------------------------------------------------------------- // Coordinator server // -------------------------------------------------------------------------------------------- @@ -576,7 +586,11 @@ public void registerTable( /** Get the table in ZK. */ public Optional getTable(TablePath tablePath) throws Exception { Optional bytes = getOrEmpty(TableZNode.path(tablePath)); - return bytes.map(TableZNode::decode); + Optional tableRegistration = bytes.map(TableZNode::decode); + // Set the default remote data dir for a node generated by an older version which does not + // have remote data dir + return tableRegistration.map( + t -> t.remoteDataDir == null ? t.newRemoteDataDir(defaultRemoteDataDir) : t); } /** Get the tables in ZK. */ @@ -685,17 +699,17 @@ 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())); } return partitions; } - /** Get the partition and the id for the partitions of tables in ZK. */ - public Map> getPartitionNameAndIdsForTables( - List tablePaths) throws Exception { - Map> result = new HashMap<>(); + /** Get the partition name and registrations for the partitions of tables in ZK. */ + public Map> + getPartitionNameAndRegistrationsForTables(List tablePaths) throws Exception { + Map> result = new HashMap<>(); Map> tablePath2Partitions = getPartitionsForTables(tablePaths); @@ -718,9 +732,10 @@ public Map> getPartitionNameAndIdsForTables( String zkPath = response.getPath(); TablePath tablePath = zkPath2TablePath.get(zkPath); String partitionName = zkPath2PartitionName.get(zkPath); - long partitionId = PartitionZNode.decode(response.getData()).getPartitionId(); + PartitionRegistration partitionRegistration = + PartitionZNode.decode(response.getData()); result.computeIfAbsent(tablePath, k -> new HashMap<>()) - .put(partitionName, partitionId); + .put(partitionName, partitionRegistration); } else { LOG.warn( "Failed to get data for path {}: {}", @@ -744,7 +759,8 @@ public Map getPartitionNameAndIds( fromPartitionName(partitionKeys, partitionName); boolean contains = resolvedPartitionSpec.contains(partialPartitionSpec); if (contains) { - Optional optPartition = getPartition(tablePath, partitionName); + Optional optPartition = + getPartition(tablePath, partitionName); optPartition.ifPresent( partition -> partitions.put(partitionName, partition.getPartitionId())); } @@ -790,10 +806,15 @@ public Map getPartitionIdAndPaths(Collection } /** Get a partition of a table in ZK. */ - public Optional getPartition(TablePath tablePath, String partitionName) + public Optional getPartition(TablePath tablePath, String partitionName) throws Exception { String path = PartitionZNode.path(tablePath, partitionName); - return getOrEmpty(path).map(PartitionZNode::decode); + Optional partitionRegistration = + getOrEmpty(path).map(PartitionZNode::decode); + // Set the default remote data dir for a node generated by an older version which does not + // have remote data dir + return partitionRegistration.map( + p -> p.getRemoteDataDir() == null ? p.newRemoteDataDir(defaultRemoteDataDir) : p); } /** Get partition id and table id for each partition in a batch async way. */ @@ -813,7 +834,7 @@ public Map getPartitionIds( return processGetDataResponses( responses, response -> path2PartitionPathMap.get(response.getPath()), - PartitionZNode::decode, + (byte[] data) -> PartitionZNode.decode(data).toTablePartition(), "partition"); } @@ -838,6 +859,7 @@ public void registerPartitionAssignmentAndMetadata( long partitionId, String partitionName, PartitionAssignment partitionAssignment, + String remoteDataDir, TablePath tablePath, long tableId) throws Exception { @@ -883,12 +905,15 @@ public void registerPartitionAssignmentAndMetadata( .withMode(CreateMode.PERSISTENT) .forPath( metadataPath, - PartitionZNode.encode(new TablePartition(tableId, partitionId))); + PartitionZNode.encode( + new PartitionRegistration( + tableId, partitionId, remoteDataDir))); ops.add(tabletServerPartitionNode); ops.add(metadataPartitionNode); zkClient.transaction().forOperations(ops); } + // -------------------------------------------------------------------------------------------- // Schema // -------------------------------------------------------------------------------------------- diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/PartitionRegistration.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/PartitionRegistration.java new file mode 100644 index 0000000000..c1a6307a63 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/PartitionRegistration.java @@ -0,0 +1,108 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.zk.data; + +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.metadata.TablePartition; + +import java.util.Objects; + +/** + * The registration information of partition in {@link ZkData.PartitionZNode}. It is used to store + * the partition information in zookeeper. + * + * @see PartitionRegistrationJsonSerde for json serialization and deserialization. + */ +public class PartitionRegistration { + + private final long tableId; + private final long partitionId; + + /** + * The remote data directory of the partition. It is null if and only if it is deserialized by + * {@link PartitionRegistrationJsonSerde} from an existing node produced by an older version + * that does not support multiple remote paths. But immediately after that, we will set it as + * the default remote file path configured by {@link ConfigOptions#REMOTE_DATA_DIR} (see {@link + * org.apache.fluss.server.zk.ZooKeeperClient#getPartition}). This unifies subsequent usage and + * eliminates the need to account for differences between versions. + */ + private final String remoteDataDir; + + public PartitionRegistration(long tableId, long partitionId, String remoteDataDir) { + this.tableId = tableId; + this.partitionId = partitionId; + this.remoteDataDir = remoteDataDir; + } + + public long getTableId() { + return tableId; + } + + public long getPartitionId() { + return partitionId; + } + + public String getRemoteDataDir() { + return remoteDataDir; + } + + public TablePartition toTablePartition() { + return new TablePartition(tableId, partitionId); + } + + /** + * Returns a new registration with the given remote data directory. Should only be called by + * {@link org.apache.fluss.server.zk.ZooKeeperClient#getPartition} when deserialize an old + * PartitionRegistration node without remote data dir configured. + * + * @param remoteDataDir the remote data directory + * @return a new registration with the given remote data directory + */ + public PartitionRegistration newRemoteDataDir(String remoteDataDir) { + return new PartitionRegistration(tableId, partitionId, remoteDataDir); + } + + @Override + public boolean equals(Object o) { + if (o == null || getClass() != o.getClass()) { + return false; + } + PartitionRegistration that = (PartitionRegistration) o; + return tableId == that.tableId + && partitionId == that.partitionId + && Objects.equals(remoteDataDir, that.remoteDataDir); + } + + @Override + public int hashCode() { + return Objects.hash(tableId, partitionId, remoteDataDir); + } + + @Override + public String toString() { + return "PartitionRegistration{" + + "tableId=" + + tableId + + ", partitionId=" + + partitionId + + ", remoteDataDir='" + + remoteDataDir + + '\'' + + '}'; + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/PartitionRegistrationJsonSerde.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/PartitionRegistrationJsonSerde.java new file mode 100644 index 0000000000..e3115f7cc9 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/PartitionRegistrationJsonSerde.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.zk.data; + +import org.apache.fluss.annotation.Internal; +import org.apache.fluss.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; +import org.apache.fluss.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.fluss.utils.json.JsonDeserializer; +import org.apache.fluss.utils.json.JsonSerializer; + +import java.io.IOException; + +/** Json serializer and deserializer for {@link PartitionRegistration}. */ +@Internal +public class PartitionRegistrationJsonSerde + implements JsonSerializer, JsonDeserializer { + + public static final PartitionRegistrationJsonSerde INSTANCE = + new PartitionRegistrationJsonSerde(); + + private static final String VERSION_KEY = "version"; + private static final String TABLE_ID_KEY = "table_id"; + private static final String PARTITION_ID_KEY = "partition_id"; + private static final String REMOTE_DATA_DIR_KEY = "remote_data_dir"; + private static final int VERSION = 1; + + @Override + public void serialize(PartitionRegistration registration, JsonGenerator generator) + throws IOException { + generator.writeStartObject(); + generator.writeNumberField(VERSION_KEY, VERSION); + generator.writeNumberField(TABLE_ID_KEY, registration.getTableId()); + generator.writeNumberField(PARTITION_ID_KEY, registration.getPartitionId()); + generator.writeStringField(REMOTE_DATA_DIR_KEY, registration.getRemoteDataDir()); + generator.writeEndObject(); + } + + @Override + public PartitionRegistration deserialize(JsonNode node) { + long tableId = node.get(TABLE_ID_KEY).asLong(); + long partitionId = node.get(PARTITION_ID_KEY).asLong(); + // When deserialize from an old version, the remote data dir may not exist. + // But we will fill it with ConfigOptions.REMOTE_DATA_DIR immediately. + String remoteDataDir = null; + if (node.has(REMOTE_DATA_DIR_KEY)) { + remoteDataDir = node.get(REMOTE_DATA_DIR_KEY).asText(); + } + return new PartitionRegistration(tableId, partitionId, remoteDataDir); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/TableRegistration.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/TableRegistration.java index 547326f6a1..422f725518 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/TableRegistration.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/TableRegistration.java @@ -52,6 +52,17 @@ public class TableRegistration { public final int bucketCount; public final Map properties; public final Map customProperties; + + /** + * The remote data directory of the table. It is null if and only if it is deserialized by + * {@link TableRegistrationJsonSerde} from an existing node produced by an older version that + * does not support multiple remote paths. But immediately after that, we will set it as the + * default remote file path configured by {@link ConfigOptions#REMOTE_DATA_DIR} (see {@link + * org.apache.fluss.server.zk.ZooKeeperClient#getTable}). This unifies subsequent usage and + * eliminates the need to account for differences between versions. + */ + public final String remoteDataDir; + public final long createdTime; public final long modifiedTime; @@ -62,6 +73,7 @@ public TableRegistration( TableDistribution tableDistribution, Map properties, Map customProperties, + String remoteDataDir, long createdTime, long modifiedTime) { checkArgument( @@ -74,6 +86,7 @@ public TableRegistration( this.bucketKeys = tableDistribution.getBucketKeys(); this.properties = properties; this.customProperties = customProperties; + this.remoteDataDir = remoteDataDir; this.createdTime = createdTime; this.modifiedTime = modifiedTime; } @@ -111,12 +124,14 @@ public TableInfo toTableInfo( this.bucketCount, properties, Configuration.fromMap(this.customProperties), + this.remoteDataDir, this.comment, this.createdTime, this.modifiedTime); } - public static TableRegistration newTable(long tableId, TableDescriptor tableDescriptor) { + public static TableRegistration newTable( + long tableId, String remoteDataDir, TableDescriptor tableDescriptor) { checkArgument( tableDescriptor.getTableDistribution().isPresent(), "Table distribution is required for table registration."); @@ -128,6 +143,7 @@ public static TableRegistration newTable(long tableId, TableDescriptor tableDesc tableDescriptor.getTableDistribution().get(), tableDescriptor.getProperties(), tableDescriptor.getCustomProperties(), + remoteDataDir, currentMillis, currentMillis); } @@ -142,10 +158,32 @@ public TableRegistration newProperties( new TableDistribution(bucketCount, bucketKeys), newProperties, newCustomProperties, + remoteDataDir, createdTime, currentMillis); } + /** + * Returns a new registration with the given remote data directory. Should only be called by + * {@link org.apache.fluss.server.zk.ZooKeeperClient#getTable} when deserialize an old + * TableRegistration node without remote data dir configured. + * + * @param remoteDataDir the remote data directory + * @return a new registration with the given remote data directory + */ + public TableRegistration newRemoteDataDir(String remoteDataDir) { + return new TableRegistration( + tableId, + comment, + partitionKeys, + new TableDistribution(bucketCount, bucketKeys), + properties, + customProperties, + remoteDataDir, + createdTime, + modifiedTime); + } + @Override public boolean equals(Object o) { if (this == o) { @@ -164,7 +202,8 @@ public boolean equals(Object o) { && Objects.equals(bucketCount, that.bucketCount) && Objects.equals(bucketKeys, that.bucketKeys) && Objects.equals(properties, that.properties) - && Objects.equals(customProperties, that.customProperties); + && Objects.equals(customProperties, that.customProperties) + && Objects.equals(remoteDataDir, that.remoteDataDir); } @Override @@ -177,6 +216,7 @@ public int hashCode() { bucketKeys, properties, customProperties, + remoteDataDir, createdTime, modifiedTime); } @@ -199,6 +239,8 @@ public String toString() { + properties + ", customProperties=" + customProperties + + ", remoteDataDir=" + + remoteDataDir + ", createdTime=" + createdTime + ", modifiedTime=" diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/TableRegistrationJsonSerde.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/TableRegistrationJsonSerde.java index 122d081e40..9a155a372a 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/TableRegistrationJsonSerde.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/TableRegistrationJsonSerde.java @@ -45,6 +45,7 @@ public class TableRegistrationJsonSerde static final String BUCKET_COUNT_NAME = "bucket_count"; static final String PROPERTIES_NAME = "properties"; static final String CUSTOM_PROPERTIES_NAME = "custom_properties"; + static final String REMOTE_DATA_DIR = "remote_data_dir"; static final String CREATED_TIME = "created_time"; static final String MODIFIED_TIME = "modified_time"; private static final String VERSION_KEY = "version"; @@ -100,6 +101,9 @@ public void serialize(TableRegistration tableReg, JsonGenerator generator) throw } generator.writeEndObject(); + // serialize remote data dir + generator.writeStringField(REMOTE_DATA_DIR, tableReg.remoteDataDir); + // serialize createdTime generator.writeNumberField(CREATED_TIME, tableReg.createdTime); @@ -141,6 +145,13 @@ public TableRegistration deserialize(JsonNode node) { Map customProperties = deserializeProperties(node.get(CUSTOM_PROPERTIES_NAME)); + // When deserialize from an old version, the remote data dir may not exist. + // But we will fill it with ConfigOptions.REMOTE_DATA_DIR immediately. + String remoteDataDir = null; + if (node.has(REMOTE_DATA_DIR)) { + remoteDataDir = node.get(REMOTE_DATA_DIR).asText(); + } + long createdTime = node.get(CREATED_TIME).asLong(); long modifiedTime = node.get(MODIFIED_TIME).asLong(); @@ -151,6 +162,7 @@ public TableRegistration deserialize(JsonNode node) { distribution, properties, customProperties, + remoteDataDir, createdTime, modifiedTime); } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java index b1e0e9493c..aed5de0b31 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java @@ -20,7 +20,6 @@ import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.Schema; import org.apache.fluss.metadata.TableBucket; -import org.apache.fluss.metadata.TablePartition; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.security.acl.Resource; import org.apache.fluss.security.acl.ResourceType; @@ -236,12 +235,13 @@ public static String path(TablePath tablePath, String partitionName) { return PartitionsZNode.path(tablePath) + "/" + partitionName; } - public static byte[] encode(TablePartition partition) { - return partition.toJsonBytes(); + public static byte[] encode(PartitionRegistration partitionRegistration) { + return JsonSerdeUtils.writeValueAsBytes( + partitionRegistration, PartitionRegistrationJsonSerde.INSTANCE); } - public static TablePartition decode(byte[] json) { - return TablePartition.fromJsonBytes(json); + public static PartitionRegistration decode(byte[] json) { + return JsonSerdeUtils.readValue(json, PartitionRegistrationJsonSerde.INSTANCE); } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/DynamicConfigChangeTest.java b/fluss-server/src/test/java/org/apache/fluss/server/DynamicConfigChangeTest.java index aa55372f73..7f413581ac 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/DynamicConfigChangeTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/DynamicConfigChangeTest.java @@ -24,6 +24,9 @@ import org.apache.fluss.config.cluster.ServerReconfigurable; import org.apache.fluss.exception.ConfigException; import org.apache.fluss.server.coordinator.LakeCatalogDynamicLoader; +import org.apache.fluss.server.coordinator.remote.RemoteDirDynamicLoader; +import org.apache.fluss.server.coordinator.remote.RoundRobinRemoteDirSelector; +import org.apache.fluss.server.coordinator.remote.WeightedRoundRobinRemoteDirSelector; import org.apache.fluss.server.zk.NOPErrorHandler; import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.server.zk.ZooKeeperExtension; @@ -412,4 +415,56 @@ public void reconfigure(Configuration newConfig) { // Verify the reconfigurable was notified with the new value assertThat(reconfiguredInterval.get()).isEqualTo(Duration.ofMinutes(5)); } + + @Test + void testDynamicReconfigurationOfRemoteDataDirs() throws Exception { + Configuration configuration = new Configuration(); + configuration.set(ConfigOptions.REMOTE_DATA_DIR, "hdfs://default-dir"); + configuration.set( + ConfigOptions.REMOTE_DATA_DIRS, Arrays.asList("hdfs://dir1", "hdfs://dir2")); + + try (RemoteDirDynamicLoader remoteDirDynamicLoader = + new RemoteDirDynamicLoader(configuration)) { + DynamicConfigManager dynamicConfigManager = + new DynamicConfigManager(zookeeperClient, configuration, true); + dynamicConfigManager.register(remoteDirDynamicLoader); + dynamicConfigManager.startup(); + + // Verify initial selector is RoundRobin (default strategy) + assertThat(remoteDirDynamicLoader.getRemoteDirSelector()) + .isInstanceOf(RoundRobinRemoteDirSelector.class); + + // Change multiple configs - generic validation applies to all + dynamicConfigManager.alterConfigs( + Arrays.asList( + new AlterConfig( + ConfigOptions.REMOTE_DATA_DIRS.key(), + "hdfs://dir1,hdfs://dir2,hdfs://dir3", + AlterConfigOpType.SET), + new AlterConfig( + ConfigOptions.REMOTE_DATA_DIRS_STRATEGY.key(), + ConfigOptions.RemoteDataDirStrategy.WEIGHTED_ROUND_ROBIN.name(), + AlterConfigOpType.SET), + new AlterConfig( + ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS.key(), + "1,2,3", + AlterConfigOpType.SET))); + + // Verify both configs were applied + Map zkConfig = zookeeperClient.fetchEntityConfig(); + assertThat(zkConfig.get(ConfigOptions.REMOTE_DATA_DIRS.key())) + .isEqualTo("hdfs://dir1,hdfs://dir2,hdfs://dir3"); + assertThat(zkConfig.get(ConfigOptions.REMOTE_DATA_DIRS_STRATEGY.key())) + .isEqualTo(ConfigOptions.RemoteDataDirStrategy.WEIGHTED_ROUND_ROBIN.name()); + assertThat(zkConfig.get(ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS.key())) + .isEqualTo("1,2,3"); + + // Wait for config change to propagate via ZK watcher + retry( + Duration.ofMinutes(1), + () -> + assertThat(remoteDirDynamicLoader.getRemoteDirSelector()) + .isInstanceOf(WeightedRoundRobinRemoteDirSelector.class)); + } + } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/AutoPartitionManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/AutoPartitionManagerTest.java index 933ecb4379..54e2e5232d 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/AutoPartitionManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/AutoPartitionManagerTest.java @@ -25,12 +25,14 @@ import org.apache.fluss.metadata.TableDescriptor; import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.server.coordinator.remote.RemoteDirDynamicLoader; import org.apache.fluss.server.testutils.TestingServerMetadataCache; import org.apache.fluss.server.zk.NOPErrorHandler; import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.server.zk.ZooKeeperExtension; import org.apache.fluss.server.zk.data.BucketAssignment; import org.apache.fluss.server.zk.data.PartitionAssignment; +import org.apache.fluss.server.zk.data.PartitionRegistration; import org.apache.fluss.server.zk.data.TableRegistration; import org.apache.fluss.testutils.common.AllCallbackWrapper; import org.apache.fluss.testutils.common.ManuallyTriggeredScheduledExecutorService; @@ -49,7 +51,13 @@ import java.time.LocalDateTime; import java.time.ZoneId; import java.time.ZonedDateTime; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; import java.util.Map; +import java.util.Optional; +import java.util.Set; import java.util.UUID; import java.util.function.Function; import java.util.stream.Stream; @@ -67,6 +75,9 @@ class AutoPartitionManagerTest { protected static ZooKeeperClient zookeeperClient; private static MetadataManager metadataManager; + private static String remoteDataDir; + private static List remoteDataDirs; + private static RemoteDirDynamicLoader remoteDirDynamicLoader; @BeforeAll static void beforeAll() { @@ -79,6 +90,17 @@ static void beforeAll() { zookeeperClient, new Configuration(), new LakeCatalogDynamicLoader(new Configuration(), null, true)); + + remoteDataDir = "/dir"; + remoteDataDirs = Arrays.asList("/dir1", "/dir2", "/dir3", "/dir4"); + Configuration conf = new Configuration(); + conf.set(ConfigOptions.REMOTE_DATA_DIR, remoteDataDir); + conf.set(ConfigOptions.REMOTE_DATA_DIRS, remoteDataDirs); + conf.set(ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS, Arrays.asList(1, 1, 1, 1)); + conf.set( + ConfigOptions.REMOTE_DATA_DIRS_STRATEGY, + ConfigOptions.RemoteDataDirStrategy.WEIGHTED_ROUND_ROBIN); + remoteDirDynamicLoader = new RemoteDirDynamicLoader(conf); } @AfterEach @@ -297,6 +319,7 @@ void testAddPartitionedTable(TestParams params) throws Exception { zookeeperClient, new Configuration(), new LakeCatalogDynamicLoader(new Configuration(), null, true)), + remoteDirDynamicLoader, new Configuration(), clock, periodicExecutor); @@ -312,6 +335,7 @@ void testAddPartitionedTable(TestParams params) throws Exception { Map partitions = zookeeperClient.getPartitionNameAndIds(tablePath); // pre-create 4 partitions including current partition assertThat(partitions.keySet()).containsExactlyInAnyOrder(params.expectedPartitions); + verifyPartitionsRemoteDataDir(tablePath, partitions.keySet()); int replicaFactor = table.getTableConfig().getReplicationFactor(); Map bucketAssignments = @@ -333,6 +357,7 @@ void testAddPartitionedTable(TestParams params) throws Exception { metadataManager.createPartition( tablePath, tableId, + remoteDataDir, partitionAssignment, fromPartitionName(table.getPartitionKeys(), partitionName), false); @@ -353,11 +378,13 @@ void testAddPartitionedTable(TestParams params) throws Exception { partitions = zookeeperClient.getPartitionNameAndIds(tablePath); assertThat(partitions.keySet()) .containsExactlyInAnyOrder(params.expectedPartitionsAfterAdvance); + verifyPartitionsRemoteDataDir(tablePath, partitions.keySet()); clock.advanceTime(params.advanceDuration2); periodicExecutor.triggerPeriodicScheduledTasks(); partitions = zookeeperClient.getPartitionNameAndIds(tablePath); assertThat(partitions.keySet()).containsExactlyInAnyOrder(params.expectedPartitionsFinal); + verifyPartitionsRemoteDataDir(tablePath, partitions.keySet()); // trigger again at the same time, should be nothing changes periodicExecutor.triggerPeriodicScheduledTasks(); @@ -387,6 +414,7 @@ void testMaxPartitions() throws Exception { new AutoPartitionManager( new TestingServerMetadataCache(3), metadataManager, + remoteDirDynamicLoader, new Configuration(), clock, periodicExecutor); @@ -424,6 +452,7 @@ void testMaxPartitions() throws Exception { metadataManager.createPartition( tablePath, tableId, + remoteDataDir, partitionAssignment, fromPartitionName(table.getPartitionKeys(), i + ""), false); @@ -463,6 +492,7 @@ void testAutoCreateDayPartitionShouldJitter() throws Exception { new AutoPartitionManager( new TestingServerMetadataCache(3), metadataManager, + remoteDirDynamicLoader, new Configuration(), clock, periodicExecutor); @@ -527,6 +557,7 @@ void testMaxBucketNum() throws Exception { new AutoPartitionManager( new TestingServerMetadataCache(3), metadataManager, + remoteDirDynamicLoader, config, clock, periodicExecutor); @@ -698,6 +729,19 @@ public TestParams build() { // ------------------------------------------------------------------------------------------- + private void verifyPartitionsRemoteDataDir( + TablePath tablePath, Collection partitionNames) throws Exception { + Set allRemoteDataDirs = new HashSet<>(remoteDataDirs); + allRemoteDataDirs.add(remoteDataDir); + for (String partitionName : partitionNames) { + Optional partition = + zookeeperClient.getPartition(tablePath, partitionName); + String remoteDataDir = partition.get().getRemoteDataDir(); + assertThat(remoteDataDir).isNotNull(); + assertThat(allRemoteDataDirs).contains(remoteDataDir); + } + } + private TableInfo createPartitionedTable( int partitionRetentionNum, int partitionPreCreateNum, AutoPartitionTimeUnit timeUnit) throws Exception { @@ -752,8 +796,16 @@ private TableInfo createPartitionedTable( .build(); long currentMillis = System.currentTimeMillis(); TableInfo tableInfo = - TableInfo.of(tablePath, tableId, 1, descriptor, currentMillis, currentMillis); - TableRegistration registration = TableRegistration.newTable(tableId, descriptor); + TableInfo.of( + tablePath, + tableId, + 1, + descriptor, + remoteDataDir, + currentMillis, + currentMillis); + TableRegistration registration = + TableRegistration.newTable(tableId, remoteDataDir, descriptor); zookeeperClient.registerTable(tablePath, registration); return tableInfo; } @@ -794,8 +846,16 @@ private TableInfo createPartitionedTableWithBuckets( .build(); long currentMillis = System.currentTimeMillis(); TableInfo tableInfo = - TableInfo.of(tablePath, tableId, 1, descriptor, currentMillis, currentMillis); - TableRegistration registration = TableRegistration.newTable(tableId, descriptor); + TableInfo.of( + tablePath, + tableId, + 1, + descriptor, + remoteDataDir, + currentMillis, + currentMillis); + TableRegistration registration = + TableRegistration.newTable(tableId, remoteDataDir, descriptor); zookeeperClient.registerTable(tablePath, registration); return tableInfo; } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorContextTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorContextTest.java index 778ac45e15..5c321dfffa 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorContextTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorContextTest.java @@ -29,6 +29,7 @@ import java.time.Duration; import static org.apache.fluss.config.ConfigOptions.TABLE_DATALAKE_ENABLED; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.assertj.core.api.Assertions.assertThat; /** Test for {@link CoordinatorContext}. */ @@ -82,6 +83,7 @@ private TableInfo createTableInfo(long tableId, TablePath tablePath, boolean isL tableId, 1, tableDescriptor, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis()); } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java index 74fd01c5e9..e95a6db30a 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java @@ -49,6 +49,7 @@ import org.apache.fluss.server.coordinator.event.CommitRemoteLogManifestEvent; import org.apache.fluss.server.coordinator.event.CoordinatorEventManager; import org.apache.fluss.server.coordinator.lease.KvSnapshotLeaseManager; +import org.apache.fluss.server.coordinator.remote.RemoteDirDynamicLoader; import org.apache.fluss.server.coordinator.statemachine.BucketState; import org.apache.fluss.server.coordinator.statemachine.ReplicaState; import org.apache.fluss.server.entity.AdjustIsrResultForBucket; @@ -90,7 +91,6 @@ import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.api.io.TempDir; -import java.io.IOException; import java.nio.file.Path; import java.time.Duration; import java.util.Arrays; @@ -157,6 +157,7 @@ class CoordinatorEventProcessorTest { private CompletedSnapshotStoreManager completedSnapshotStoreManager; private CoordinatorMetadataCache serverMetadataCache; private KvSnapshotLeaseManager kvSnapshotLeaseManager; + private String remoteDataDir; @BeforeAll static void baseBeforeAll() throws Exception { @@ -188,17 +189,21 @@ static void baseBeforeAll() throws Exception { } @BeforeEach - void beforeEach() throws IOException { + void beforeEach() { serverMetadataCache = new CoordinatorMetadataCache(); // set a test channel manager for the context testCoordinatorChannelManager = new TestCoordinatorChannelManager(); - autoPartitionManager = - new AutoPartitionManager(serverMetadataCache, metadataManager, new Configuration()); lakeTableTieringManager = new LakeTableTieringManager(TestingMetricGroups.LAKE_TIERING_METRICS); + remoteDataDir = zookeeperClient.getDefaultRemoteDataDir(); Configuration conf = new Configuration(); - String remoteDataDir = "/tmp/fluss/remote-data"; conf.setString(ConfigOptions.REMOTE_DATA_DIR, remoteDataDir); + autoPartitionManager = + new AutoPartitionManager( + serverMetadataCache, + metadataManager, + new RemoteDirDynamicLoader(conf), + new Configuration()); kvSnapshotLeaseManager = new KvSnapshotLeaseManager( Duration.ofMinutes(10).toMillis(), @@ -242,10 +247,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); @@ -375,13 +384,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( @@ -486,7 +497,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(); @@ -610,7 +623,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; @@ -682,7 +696,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; @@ -828,7 +843,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 @@ -882,7 +898,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( @@ -941,7 +957,7 @@ void testTableRegistrationChange() throws Exception { }); // create table List replicas = tableAssignment.getBucketAssignment(0).getReplicas(); - metadataManager.createTable(t1, TEST_TABLE, tableAssignment, false); + metadataManager.createTable(t1, remoteDataDir, TEST_TABLE, tableAssignment, false); TableInfo tableInfo = metadataManager.getTable(t1); retry( @@ -1003,7 +1019,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)); @@ -1055,6 +1075,8 @@ private void verifyIsr(TableBucket tb, int expectedLeader, List expecte } private CoordinatorEventProcessor buildCoordinatorEventProcessor() { + Configuration conf = new Configuration(); + conf.set(ConfigOptions.REMOTE_DATA_DIR, remoteDataDir); return new CoordinatorEventProcessor( zookeeperClient, serverMetadataCache, @@ -1063,7 +1085,7 @@ private CoordinatorEventProcessor buildCoordinatorEventProcessor() { autoPartitionManager, lakeTableTieringManager, TestingMetricGroups.COORDINATOR_METRICS, - new Configuration(), + conf, Executors.newFixedThreadPool(1, new ExecutorThreadFactory("test-coordinator-io")), metadataManager, kvSnapshotLeaseManager); @@ -1108,9 +1130,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), @@ -1381,7 +1413,11 @@ private long createTable(TablePath tablePath, TabletServerInfo[] servers) { TableAssignment tableAssignment = generateAssignment(N_BUCKETS, REPLICATION_FACTOR, servers); return metadataManager.createTable( - tablePath, CoordinatorEventProcessorTest.TEST_TABLE, tableAssignment, false); + tablePath, + remoteDataDir, + CoordinatorEventProcessorTest.TEST_TABLE, + tableAssignment, + false); } private void alterTable(TablePath tablePath, List schemaChanges) { diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/LakeTableTieringManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/LakeTableTieringManagerTest.java index f0ab38f78c..62da35290d 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/LakeTableTieringManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/LakeTableTieringManagerTest.java @@ -51,12 +51,14 @@ class LakeTableTieringManagerTest { private LakeTableTieringManager tableTieringManager; private ManualClock manualClock; private ManuallyTriggeredScheduledExecutorService lakeTieringServiceTimeoutChecker; + private String remoteDataDir; @BeforeEach void beforeEach() { manualClock = new ManualClock(); lakeTieringServiceTimeoutChecker = new ManuallyTriggeredScheduledExecutorService(); tableTieringManager = createLakeTableTieringManager(); + remoteDataDir = "/tmp/fluss/remote-data"; } private LakeTableTieringManager createLakeTableTieringManager() { @@ -315,6 +317,7 @@ private TableInfo createTableInfo(long tableId, TablePath tablePath, Duration fr tableId, 1, tableDescriptor, + remoteDataDir, System.currentTimeMillis(), System.currentTimeMillis()); } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TableManagerITCase.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TableManagerITCase.java index e49636cf43..928474bf07 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TableManagerITCase.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TableManagerITCase.java @@ -711,6 +711,7 @@ void testSchemaEvolution() throws Exception { pbTableMetadata.getTableId(), pbTableMetadata.getSchemaId(), TableDescriptor.fromJsonBytes(pbTableMetadata.getTableJson()), + pbTableMetadata.getRemoteDataDir(), pbTableMetadata.getCreatedTime(), pbTableMetadata.getModifiedTime()); List columns = tableInfo.getSchema().getColumns(); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TableManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TableManagerTest.java index 3c417efe8d..2e525fb14f 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 @@ -19,6 +19,8 @@ import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; +import org.apache.fluss.metadata.PartitionInfo; +import org.apache.fluss.metadata.ResolvedPartitionSpec; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableBucketReplica; import org.apache.fluss.metadata.TableInfo; @@ -26,6 +28,7 @@ import org.apache.fluss.server.coordinator.event.CoordinatorEvent; import org.apache.fluss.server.coordinator.event.DeleteReplicaResponseReceivedEvent; import org.apache.fluss.server.coordinator.event.TestingEventManager; +import org.apache.fluss.server.coordinator.remote.RemoteStorageCleaner; import org.apache.fluss.server.coordinator.statemachine.ReplicaStateMachine; import org.apache.fluss.server.coordinator.statemachine.TableBucketStateMachine; import org.apache.fluss.server.entity.DeleteReplicaResultForBucket; @@ -58,11 +61,13 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +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.apache.fluss.record.TestData.DATA1_TABLE_ID; import static org.apache.fluss.record.TestData.DATA1_TABLE_PATH; import static org.apache.fluss.record.TestData.DATA1_TABLE_PATH_PK; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.server.coordinator.statemachine.BucketState.OnlineBucket; import static org.apache.fluss.server.coordinator.statemachine.ReplicaState.OnlineReplica; import static org.apache.fluss.server.coordinator.statemachine.ReplicaState.ReplicaDeletionSuccessful; @@ -162,6 +167,7 @@ void testCreateTable() throws Exception { tableId, 0, DATA1_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis())); tableManager.onCreateNewTable(DATA1_TABLE_PATH, tableId, assignment); @@ -185,6 +191,7 @@ void testDeleteTable() throws Exception { tableId, 0, DATA1_TABLE_DESCRIPTOR_PK, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis())); tableManager.onCreateNewTable(DATA1_TABLE_PATH_PK, tableId, assignment); @@ -224,6 +231,7 @@ void testResumeDeletionAfterRestart() throws Exception { tableId, 0, DATA1_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis())); tableManager.onCreateNewTable(DATA1_TABLE_PATH, tableId, assignment); @@ -271,6 +279,7 @@ void testCreateAndDropPartition() throws Exception { tableId, 0, DATA1_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis())); tableManager.onCreateNewTable(DATA1_TABLE_PATH, tableId, assignment); @@ -280,11 +289,23 @@ void testCreateAndDropPartition() throws Exception { String partitionName = "2024"; long partitionId = zookeeperClient.getPartitionIdAndIncrement(); zookeeperClient.registerPartitionAssignmentAndMetadata( - partitionId, partitionName, partitionAssignment, DATA1_TABLE_PATH, tableId); + partitionId, + partitionName, + partitionAssignment, + DEFAULT_REMOTE_DATA_DIR, + DATA1_TABLE_PATH, + tableId); // create partition + PartitionInfo partitionInfo = + new PartitionInfo( + partitionId, + ResolvedPartitionSpec.fromPartitionName( + DATA1_PARTITIONED_TABLE_DESCRIPTOR.getPartitionKeys(), + partitionName), + DEFAULT_REMOTE_DATA_DIR); tableManager.onCreateNewPartition( - DATA1_TABLE_PATH, tableId, partitionId, partitionName, partitionAssignment); + DATA1_TABLE_PATH, tableId, partitionInfo, partitionAssignment); // all replicas should be online checkReplicaOnline(tableId, partitionId, partitionAssignment); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/event/watcher/TableChangeWatcherTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/event/watcher/TableChangeWatcherTest.java index 76634b970c..355a238d89 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 @@ -58,6 +58,7 @@ import java.util.Collections; import java.util.List; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.server.utils.TableAssignmentUtils.generateAssignment; import static org.apache.fluss.testutils.common.CommonTestUtils.retry; import static org.assertj.core.api.Assertions.assertThat; @@ -134,7 +135,8 @@ void testTableChanges() { new TabletServerInfo(2, "rack2") }); long tableId = - metadataManager.createTable(tablePath, TEST_TABLE, tableAssignment, false); + metadataManager.createTable( + tablePath, DEFAULT_REMOTE_DATA_DIR, TEST_TABLE, tableAssignment, false); SchemaInfo schemaInfo = metadataManager.getLatestSchema(tablePath); long currentMillis = System.currentTimeMillis(); expectedEvents.add( @@ -144,6 +146,7 @@ void testTableChanges() { tableId, schemaInfo.getSchemaId(), TEST_TABLE, + DEFAULT_REMOTE_DATA_DIR, currentMillis, currentMillis), tableAssignment)); @@ -194,7 +197,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, DEFAULT_REMOTE_DATA_DIR, partitionedTable, null, false); List expectedEvents = new ArrayList<>(); SchemaInfo schemaInfo = metadataManager.getLatestSchema(tablePath); // create table event @@ -206,6 +211,7 @@ void testPartitionedTable() throws Exception { tableId, schemaInfo.getSchemaId(), partitionedTable, + DEFAULT_REMOTE_DATA_DIR, currentMillis, currentMillis), TableAssignment.builder().build())); @@ -226,15 +232,27 @@ void testPartitionedTable() throws Exception { .getBucketAssignments()); // register assignment and metadata zookeeperClient.registerPartitionAssignmentAndMetadata( - 1L, "2011", partitionAssignment, tablePath, tableId); + 1L, "2011", partitionAssignment, DEFAULT_REMOTE_DATA_DIR, tablePath, tableId); zookeeperClient.registerPartitionAssignmentAndMetadata( - 2L, "2022", partitionAssignment, tablePath, tableId); + 2L, "2022", partitionAssignment, DEFAULT_REMOTE_DATA_DIR, tablePath, tableId); // create partitions events expectedEvents.add( - new CreatePartitionEvent(tablePath, tableId, 1L, "2011", partitionAssignment)); + new CreatePartitionEvent( + tablePath, + tableId, + 1L, + "2011", + partitionAssignment, + DEFAULT_REMOTE_DATA_DIR)); expectedEvents.add( - new CreatePartitionEvent(tablePath, tableId, 2L, "2022", partitionAssignment)); + new CreatePartitionEvent( + tablePath, + tableId, + 2L, + "2022", + partitionAssignment, + DEFAULT_REMOTE_DATA_DIR)); retry( Duration.ofMinutes(1), @@ -273,7 +291,8 @@ void testSchemaChanges() { new TabletServerInfo(2, "rack2") }); long tableId = - metadataManager.createTable(tablePath, TEST_TABLE, tableAssignment, false); + metadataManager.createTable( + tablePath, DEFAULT_REMOTE_DATA_DIR, TEST_TABLE, tableAssignment, false); SchemaInfo schemaInfo = metadataManager.getLatestSchema(tablePath); long currentMillis = System.currentTimeMillis(); expectedEvents.add( @@ -283,6 +302,7 @@ void testSchemaChanges() { tableId, schemaInfo.getSchemaId(), TEST_TABLE, + DEFAULT_REMOTE_DATA_DIR, currentMillis, currentMillis), tableAssignment)); @@ -345,7 +365,9 @@ void testTableRegistrationChange() { new TabletServerInfo(1, "rack1"), new TabletServerInfo(2, "rack2") }); - long tableId = metadataManager.createTable(tablePath, TEST_TABLE, tableAssignment, false); + long tableId = + metadataManager.createTable( + tablePath, DEFAULT_REMOTE_DATA_DIR, TEST_TABLE, tableAssignment, false); SchemaInfo schemaInfo = metadataManager.getLatestSchema(tablePath); long currentMillis = System.currentTimeMillis(); @@ -357,6 +379,7 @@ void testTableRegistrationChange() { tableId, schemaInfo.getSchemaId(), TEST_TABLE, + DEFAULT_REMOTE_DATA_DIR, currentMillis, currentMillis), tableAssignment)); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/RebalanceManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/RebalanceManagerTest.java index a961dc393e..4a5caf82cd 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/RebalanceManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/RebalanceManagerTest.java @@ -18,6 +18,7 @@ package org.apache.fluss.server.coordinator.rebalance; import org.apache.fluss.cluster.rebalance.RebalanceStatus; +import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; import org.apache.fluss.server.coordinator.AutoPartitionManager; import org.apache.fluss.server.coordinator.CoordinatorContext; @@ -27,6 +28,7 @@ import org.apache.fluss.server.coordinator.MetadataManager; import org.apache.fluss.server.coordinator.TestCoordinatorChannelManager; import org.apache.fluss.server.coordinator.lease.KvSnapshotLeaseManager; +import org.apache.fluss.server.coordinator.remote.RemoteDirDynamicLoader; import org.apache.fluss.server.metadata.CoordinatorMetadataCache; import org.apache.fluss.server.metrics.group.TestingMetricGroups; import org.apache.fluss.server.zk.NOPErrorHandler; @@ -80,8 +82,10 @@ static void baseBeforeAll() { void beforeEach() { serverMetadataCache = new CoordinatorMetadataCache(); testCoordinatorChannelManager = new TestCoordinatorChannelManager(); - String remoteDataDir = "/tmp/fluss/remote-data"; + Configuration conf = new Configuration(); + conf.set(ConfigOptions.REMOTE_DATA_DIR, remoteDataDir); + kvSnapshotLeaseManager = new KvSnapshotLeaseManager( Duration.ofMinutes(10).toMillis(), @@ -92,10 +96,14 @@ void beforeEach() { kvSnapshotLeaseManager.start(); autoPartitionManager = - new AutoPartitionManager(serverMetadataCache, metadataManager, new Configuration()); + new AutoPartitionManager( + serverMetadataCache, + metadataManager, + new RemoteDirDynamicLoader(conf), + conf); lakeTableTieringManager = new LakeTableTieringManager(TestingMetricGroups.LAKE_TIERING_METRICS); - CoordinatorEventProcessor eventProcessor = buildCoordinatorEventProcessor(); + CoordinatorEventProcessor eventProcessor = buildCoordinatorEventProcessor(conf); rebalanceManager = new RebalanceManager(eventProcessor, zookeeperClient); rebalanceManager.startup(); } @@ -131,7 +139,7 @@ void testRebalanceWithoutTask() throws Exception { .hasValue(new RebalanceTask(rebalanceId, COMPLETED, new HashMap<>())); } - private CoordinatorEventProcessor buildCoordinatorEventProcessor() { + private CoordinatorEventProcessor buildCoordinatorEventProcessor(Configuration conf) { return new CoordinatorEventProcessor( zookeeperClient, serverMetadataCache, @@ -140,7 +148,7 @@ private CoordinatorEventProcessor buildCoordinatorEventProcessor() { autoPartitionManager, lakeTableTieringManager, TestingMetricGroups.COORDINATOR_METRICS, - new Configuration(), + conf, Executors.newFixedThreadPool(1, new ExecutorThreadFactory("test-coordinator-io")), metadataManager, kvSnapshotLeaseManager); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/RemoteDirDynamicLoaderTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/RemoteDirDynamicLoaderTest.java new file mode 100644 index 0000000000..ddbdf51ba8 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/RemoteDirDynamicLoaderTest.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.coordinator.remote; + +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.exception.ConfigException; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Tests for {@link RemoteDirDynamicLoader}. */ +class RemoteDirDynamicLoaderTest { + + @Test + void testReconfigureWithStrategyChange() throws Exception { + Configuration conf = new Configuration(); + conf.set(ConfigOptions.REMOTE_DATA_DIRS, Arrays.asList("hdfs://dir1", "hdfs://dir2")); + try (RemoteDirDynamicLoader loader = new RemoteDirDynamicLoader(conf)) { + RemoteDirSelector originalSelector = loader.getRemoteDirSelector(); + assertThat(originalSelector).isInstanceOf(RoundRobinRemoteDirSelector.class); + + // Reconfigure with strategy change + Configuration newConfig = new Configuration(); + newConfig.set( + ConfigOptions.REMOTE_DATA_DIRS_STRATEGY, + ConfigOptions.RemoteDataDirStrategy.WEIGHTED_ROUND_ROBIN); + newConfig.set(ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS, Arrays.asList(1, 2)); + loader.reconfigure(newConfig); + + // Container should be replaced + assertThat(loader.getRemoteDirSelector()).isNotSameAs(originalSelector); + assertThat(loader.getRemoteDirSelector()) + .isInstanceOf(WeightedRoundRobinRemoteDirSelector.class); + } + } + + @Test + void testReconfigureWithWeightsChange() throws Exception { + Configuration conf = new Configuration(); + 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)) { + RemoteDirSelector originalSelector = loader.getRemoteDirSelector(); + assertThat(originalSelector).isInstanceOf(WeightedRoundRobinRemoteDirSelector.class); + + // Reconfigure with weights change + Configuration newConfig = new Configuration(); + newConfig.set(ConfigOptions.REMOTE_DATA_DIRS_WEIGHTS, Arrays.asList(3, 4)); + loader.reconfigure(newConfig); + + // Container should be replaced + assertThat(loader.getRemoteDirSelector()).isNotSameAs(originalSelector); + assertThat(loader.getRemoteDirSelector()) + .isInstanceOf(WeightedRoundRobinRemoteDirSelector.class); + } + } + + @Test + void testReconfigureWithRemoteDataDirsChange() throws Exception { + // Test new dirs must contain all old dirs + Configuration conf1 = new Configuration(); + conf1.set(ConfigOptions.REMOTE_DATA_DIRS, Arrays.asList("hdfs://dir1", "hdfs://dir2")); + try (RemoteDirDynamicLoader loader = new RemoteDirDynamicLoader(conf1)) { + Configuration newConfig = new Configuration(); + newConfig.set( + ConfigOptions.REMOTE_DATA_DIRS, Arrays.asList("hdfs://dir2", "hdfs://dir3")); + + assertThatThrownBy(() -> loader.validate(newConfig)) + .isInstanceOf(ConfigException.class) + .hasMessageContaining("must contain all old remote.data.dirs"); + } + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/RemoteDirsITCase.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/RemoteDirsITCase.java new file mode 100644 index 0000000000..1a445808cc --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/RemoteDirsITCase.java @@ -0,0 +1,278 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.coordinator.remote; + +import org.apache.fluss.config.AutoPartitionTimeUnit; +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.metadata.PartitionSpec; +import org.apache.fluss.metadata.Schema; +import org.apache.fluss.metadata.TableDescriptor; +import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.server.testutils.FlussClusterExtension; +import org.apache.fluss.server.testutils.RpcMessageTestUtils; +import org.apache.fluss.server.zk.ZooKeeperClient; +import org.apache.fluss.server.zk.data.PartitionRegistration; +import org.apache.fluss.server.zk.data.TableRegistration; +import org.apache.fluss.types.DataTypes; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +import static org.apache.fluss.record.TestData.DATA1_PARTITIONED_TABLE_DESCRIPTOR; +import static org.apache.fluss.record.TestData.DATA1_TABLE_DESCRIPTOR; +import static org.apache.fluss.record.TestData.DATA1_TABLE_DESCRIPTOR_PK; +import static org.assertj.core.api.Assertions.assertThat; + +/** ITCase for multi remote data directories functionality. */ +class RemoteDirsITCase { + + private static final TableDescriptor DATA1_PARTITIONED_TABLE_DESCRIPTOR_PK = + TableDescriptor.builder() + .schema( + Schema.newBuilder() + .column("a", DataTypes.INT()) + .withComment("a is first column") + .column("b", DataTypes.STRING()) + .withComment("b is second column") + .primaryKey("a", "b") + .build()) + .distributedBy(3) + .partitionedBy("b") + .property(ConfigOptions.TABLE_AUTO_PARTITION_ENABLED, true) + .property( + ConfigOptions.TABLE_AUTO_PARTITION_TIME_UNIT, + AutoPartitionTimeUnit.YEAR) + .build(); + + private static final List REMOTE_DIR_NAMES = Arrays.asList("dir1", "dir2", "dir3"); + + @RegisterExtension + public static final FlussClusterExtension FLUSS_CLUSTER_EXTENSION = + FlussClusterExtension.builder() + .setNumOfTabletServers(3) + .setClusterConf(initConfig()) + .setRemoteDirNames(REMOTE_DIR_NAMES) + .build(); + + private ZooKeeperClient zkClient; + + private static Configuration initConfig() { + Configuration conf = new Configuration(); + conf.setInt(ConfigOptions.DEFAULT_REPLICATION_FACTOR, 3); + conf.set(ConfigOptions.AUTO_PARTITION_CHECK_INTERVAL, Duration.ofSeconds(1)); + + return conf; + } + + @BeforeEach + void setup() { + zkClient = FLUSS_CLUSTER_EXTENSION.getZooKeeperClient(); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testCreateMultipleTablesWithRoundRobin(boolean isPrimaryKeyTable) throws Exception { + // Create multiple tables and verify they use different remote dirs via round-robin + TableDescriptor tableDescriptor = + isPrimaryKeyTable ? DATA1_TABLE_DESCRIPTOR_PK : DATA1_TABLE_DESCRIPTOR; + String tablePrefix = isPrimaryKeyTable ? "pk_table_" : "non_pk_table_"; + + List remoteDirsUsed = new ArrayList<>(); + int tableCount = 6; // Create more tables than dirs to see round-robin in action + + for (int i = 0; i < tableCount; i++) { + TablePath tablePath = TablePath.of("test_db", tablePrefix + i); + RpcMessageTestUtils.createTable(FLUSS_CLUSTER_EXTENSION, tablePath, tableDescriptor); + + // Get the table registration to check remoteDataDir + Optional tableOpt = zkClient.getTable(tablePath); + assertThat(tableOpt).isPresent(); + TableRegistration table = tableOpt.get(); + + assertThat(table.remoteDataDir).isNotNull(); + remoteDirsUsed.add(table.remoteDataDir); + } + + // Verify round-robin distribution: each dir should be used at least once + Map dirUsageCount = new HashMap<>(); + for (String dir : remoteDirsUsed) { + dirUsageCount.merge(dir, 1, Integer::sum); + } + + // With round-robin, all configured dirs should be used + assertThat(dirUsageCount.keySet()).hasSize(REMOTE_DIR_NAMES.size()); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testCreatePartitionsWithRoundRobin(boolean isPrimaryKeyTable) throws Exception { + // Create a partitioned table and add multiple partitions + // Each partition should get a different remoteDataDir via round-robin + String tablePrefix = isPrimaryKeyTable ? "pk_partitioned_" : "partitioned_"; + TablePath tablePath = TablePath.of("test_db", tablePrefix + "table_2"); + + TableDescriptor tableDescriptor = + isPrimaryKeyTable + ? DATA1_PARTITIONED_TABLE_DESCRIPTOR_PK + : DATA1_PARTITIONED_TABLE_DESCRIPTOR; + RpcMessageTestUtils.createTable(FLUSS_CLUSTER_EXTENSION, tablePath, tableDescriptor); + + Optional tableOpt = zkClient.getTable(tablePath); + assertThat(tableOpt).isPresent(); + TableRegistration table = tableOpt.get(); + // Partitioned table should NOT have remoteDataDir set at table level + assertThat(table.remoteDataDir).isNotNull(); + + // Create multiple partitions using partition column "b" + int partitionCount = 6; + List partitionNames = new ArrayList<>(); + for (int i = 0; i < partitionCount; i++) { + String partitionName = "p" + i; + partitionNames.add(partitionName); + PartitionSpec partitionSpec = + new PartitionSpec(Collections.singletonMap("b", partitionName)); + RpcMessageTestUtils.createPartition( + FLUSS_CLUSTER_EXTENSION, tablePath, partitionSpec, false); + } + + // Verify each partition has remoteDataDir set and round-robin is applied + Set usedRemoteDirs = new HashSet<>(); + for (String partitionName : partitionNames) { + Optional partitionOpt = + zkClient.getPartition(tablePath, partitionName); + assertThat(partitionOpt).isPresent(); + PartitionRegistration partition = partitionOpt.get(); + + assertThat(partition.getRemoteDataDir()).isNotNull(); + usedRemoteDirs.add(partition.getRemoteDataDir()); + } + + // All configured dirs should be used + assertThat(usedRemoteDirs).hasSize(REMOTE_DIR_NAMES.size()); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testAutoPartitionWithMultipleRemoteDirs(boolean isPrimaryKeyTable) throws Exception { + // Create an auto-partitioned table and verify partitions use different remote dirs + String tablePrefix = isPrimaryKeyTable ? "auto_pk_partitioned_" : "auto_partitioned_"; + TablePath tablePath = TablePath.of("test_db", tablePrefix + "table"); + TableDescriptor tableDescriptor = + isPrimaryKeyTable + ? DATA1_PARTITIONED_TABLE_DESCRIPTOR_PK + : DATA1_PARTITIONED_TABLE_DESCRIPTOR; + + RpcMessageTestUtils.createTable(FLUSS_CLUSTER_EXTENSION, tablePath, tableDescriptor); + + // Wait for auto partitions to be created + Map partitions = + FLUSS_CLUSTER_EXTENSION.waitUntilPartitionAllReady(tablePath); + assertThat(partitions).isNotEmpty(); + + // Verify partitions use remote dirs + Set usedRemoteDirs = new HashSet<>(); + for (String partitionName : partitions.keySet()) { + Optional partitionOpt = + zkClient.getPartition(tablePath, partitionName); + assertThat(partitionOpt).isPresent(); + PartitionRegistration partition = partitionOpt.get(); + + assertThat(partition.getRemoteDataDir()).isNotNull(); + usedRemoteDirs.add(partition.getRemoteDataDir()); + } + + // At least one remote dir should be used + assertThat(usedRemoteDirs).isNotEmpty(); + } + + @Test + void testMixedTableAndPartitionCreation() throws Exception { + // Create a mix of non-partitioned tables and partitioned table partitions + // to verify round-robin works correctly across both types + + // Create 2 non-partitioned tables + for (int i = 0; i < 2; i++) { + TablePath tablePath = TablePath.of("test_db", "mixed_non_pk_" + i); + RpcMessageTestUtils.createTable( + FLUSS_CLUSTER_EXTENSION, tablePath, DATA1_TABLE_DESCRIPTOR_PK); + } + + // Create a partitioned table + TablePath partitionedTablePath = TablePath.of("test_db", "mixed_partitioned"); + RpcMessageTestUtils.createTable( + FLUSS_CLUSTER_EXTENSION, partitionedTablePath, DATA1_PARTITIONED_TABLE_DESCRIPTOR); + + // Create partitions using partition column "b" + RpcMessageTestUtils.createPartition( + FLUSS_CLUSTER_EXTENSION, + partitionedTablePath, + new PartitionSpec(Collections.singletonMap("b", "p1")), + false); + RpcMessageTestUtils.createPartition( + FLUSS_CLUSTER_EXTENSION, + partitionedTablePath, + new PartitionSpec(Collections.singletonMap("b", "p2")), + false); + + // Create 2 more non-partitioned tables + for (int i = 2; i < 4; i++) { + TablePath tablePath = TablePath.of("test_db", "mixed_non_pk_" + i); + RpcMessageTestUtils.createTable( + FLUSS_CLUSTER_EXTENSION, tablePath, DATA1_TABLE_DESCRIPTOR_PK); + } + + // Collect all remote dirs used + Set allUsedDirs = new HashSet<>(); + + // Check non-partitioned tables + for (int i = 0; i < 4; i++) { + TablePath tablePath = TablePath.of("test_db", "mixed_non_pk_" + i); + Optional tableOpt = zkClient.getTable(tablePath); + assertThat(tableOpt).isPresent(); + assertThat(tableOpt.get().remoteDataDir).isNotNull(); + allUsedDirs.add(tableOpt.get().remoteDataDir); + } + + // Check partitions + for (String p : Arrays.asList("p1", "p2")) { + Optional partitionOpt = + zkClient.getPartition(partitionedTablePath, p); + assertThat(partitionOpt).isPresent(); + assertThat(partitionOpt.get().getRemoteDataDir()).isNotNull(); + allUsedDirs.add(partitionOpt.get().getRemoteDataDir()); + } + + // All remote dirs should have been used (6 items, 3 dirs, round-robin) + assertThat(allUsedDirs).hasSize(REMOTE_DIR_NAMES.size()); + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/RemoteStorageCleanerITCase.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/RemoteStorageCleanerITCase.java new file mode 100644 index 0000000000..2e48cb39d7 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/RemoteStorageCleanerITCase.java @@ -0,0 +1,391 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.coordinator.remote; + +import org.apache.fluss.config.AutoPartitionTimeUnit; +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.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.TablePartition; +import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.rpc.gateway.CoordinatorGateway; +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.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import java.io.File; +import java.time.Duration; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +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.apache.fluss.server.testutils.RpcMessageTestUtils.newDropPartitionRequest; +import static org.apache.fluss.server.testutils.RpcMessageTestUtils.newDropTableRequest; +import static org.apache.fluss.testutils.common.CommonTestUtils.retry; +import static org.assertj.core.api.Assertions.assertThat; + +/** ITCase for {@link RemoteStorageCleaner}. */ +class RemoteStorageCleanerITCase { + + 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_DATA_DIRS = Arrays.asList("dir1", "dir2", "dir3"); + + @RegisterExtension + public static final FlussClusterExtension FLUSS_CLUSTER_EXTENSION = + FlussClusterExtension.builder() + .setNumOfTabletServers(3) + .setClusterConf(new Configuration()) + .setRemoteDirNames(REMOTE_DATA_DIRS) + .build(); + + private ZooKeeperClient zkClient; + private CoordinatorGateway coordinatorGateway; + + @BeforeEach + void beforeEach() { + zkClient = FLUSS_CLUSTER_EXTENSION.getZooKeeperClient(); + coordinatorGateway = FLUSS_CLUSTER_EXTENSION.newCoordinatorClient(); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testDeleteNonPartitionedTable(boolean isPkTable) throws Exception { + // Create table + String tablePrefix = isPkTable ? "pk_table_cleaner_" : "log_table_cleaner_"; + TablePath tablePath = TablePath.of("test_db", tablePrefix + "1"); + TableDescriptor tableDescriptor = + isPkTable ? DATA1_TABLE_DESCRIPTOR_PK : DATA1_TABLE_DESCRIPTOR; + + long tableId = + RpcMessageTestUtils.createTable( + FLUSS_CLUSTER_EXTENSION, tablePath, tableDescriptor); + TableBucket tb = new TableBucket(tableId, 0); + FLUSS_CLUSTER_EXTENSION.waitUntilAllReplicaReady(tb); + + // Get remote data dir for table + Optional tableOpt = zkClient.getTable(tablePath); + assertThat(tableOpt).isPresent(); + String remoteTableDataDir = tableOpt.get().remoteDataDir; + assertThat(remoteTableDataDir).isNotNull(); + + // Mock remote log dirs + FsPath remoteLogTableDir = + FlussPaths.remoteTableDir( + FlussPaths.remoteLogDir(remoteTableDataDir), tablePath, tableId); + remoteLogTableDir.getFileSystem().mkdirs(remoteLogTableDir); + assertThat(remoteLogTableDir.getFileSystem().exists(remoteLogTableDir)).isTrue(); + + // Mock remote kv dirs + FsPath remoteKvTableDir; + if (isPkTable) { + remoteKvTableDir = + FlussPaths.remoteTableDir( + FlussPaths.remoteKvDir(remoteTableDataDir), tablePath, tableId); + remoteKvTableDir.getFileSystem().mkdirs(remoteKvTableDir); + assertThat(remoteKvTableDir.getFileSystem().exists(remoteKvTableDir)).isTrue(); + } else { + remoteKvTableDir = null; + } + + // Mock remote lake dirs + FsPath remoteLakeDir = + FlussPaths.remoteLakeTableSnapshotDir(remoteTableDataDir, tablePath, tableId); + remoteLakeDir.getFileSystem().mkdirs(remoteLakeDir); + assertThat(remoteLakeDir.getFileSystem().exists(remoteLakeDir)).isTrue(); + + // Drop table + coordinatorGateway + .dropTable( + newDropTableRequest( + tablePath.getDatabaseName(), tablePath.getTableName(), false)) + .get(); + assertThat(zkClient.tableExist(tablePath)).isFalse(); + + // Verify remote log dir is deleted + retry( + Duration.ofMinutes(1), + () -> + assertThat(remoteLogTableDir.getFileSystem().exists(remoteLogTableDir)) + .isFalse()); + + // Verify remote kv dir is deleted for PK table + if (isPkTable) { + retry( + Duration.ofMinutes(1), + () -> + assertThat(remoteKvTableDir.getFileSystem().exists(remoteKvTableDir)) + .isFalse()); + } + + // Verify remote lake dir is deleted + retry( + Duration.ofMinutes(1), + () -> assertThat(remoteLakeDir.getFileSystem().exists(remoteLakeDir)).isFalse()); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testDeletePartitionedTable(boolean isPkTable) throws Exception { + // Create partitioned table + String tableName = isPkTable ? "pk_partitioned_cleaner" : "partitioned_cleaner"; + TablePath tablePath = TablePath.of("test_db", tableName); + TableDescriptor tableDescriptor = + isPkTable + ? DATA1_PARTITIONED_TABLE_DESCRIPTOR_PK + : DATA1_PARTITIONED_TABLE_DESCRIPTOR; + + long tableId = + RpcMessageTestUtils.createTable( + FLUSS_CLUSTER_EXTENSION, tablePath, tableDescriptor); + + // Get remote data dir for table + Optional tableOpt = zkClient.getTable(tablePath); + assertThat(tableOpt).isPresent(); + String remoteTableDataDir = tableOpt.get().remoteDataDir; + assertThat(remoteTableDataDir).isNotNull(); + + // Wait for auto partitions to be created + Map partitions = + FLUSS_CLUSTER_EXTENSION.waitUntilPartitionAllReady(tablePath); + assertThat(partitions).isNotEmpty(); + + // Get one partition's remote dir for verification + String firstPartitionName = partitions.keySet().iterator().next(); + Long firstPartitionId = partitions.get(firstPartitionName); + Optional partitionOpt = + zkClient.getPartition(tablePath, firstPartitionName); + assertThat(partitionOpt).isPresent(); + String partitionRemoteDataDir = partitionOpt.get().getRemoteDataDir(); + assertThat(partitionRemoteDataDir).isNotNull(); + + PhysicalTablePath physicalTablePath = PhysicalTablePath.of(tablePath, firstPartitionName); + TablePartition tablePartition = new TablePartition(tableId, firstPartitionId); + + // Mock remote log dirs + FsPath remoteLogPartitionDir = + FlussPaths.remotePartitionDir( + FlussPaths.remoteLogDir(partitionRemoteDataDir), + physicalTablePath, + tablePartition); + remoteLogPartitionDir.getFileSystem().mkdirs(remoteLogPartitionDir); + assertThat(remoteLogPartitionDir.getFileSystem().exists(remoteLogPartitionDir)).isTrue(); + + // Mock remote kv dirs + FsPath remoteKvPartitionDir; + if (isPkTable) { + remoteKvPartitionDir = + FlussPaths.remotePartitionDir( + FlussPaths.remoteKvDir(partitionRemoteDataDir), + physicalTablePath, + tablePartition); + remoteKvPartitionDir.getFileSystem().mkdirs(remoteKvPartitionDir); + assertThat(remoteKvPartitionDir.getFileSystem().exists(remoteKvPartitionDir)).isTrue(); + } else { + remoteKvPartitionDir = null; + } + + // Mock remote lake dirs + FsPath remoteLakeDir = + FlussPaths.remoteLakeTableSnapshotDir(remoteTableDataDir, tablePath, tableId); + remoteLakeDir.getFileSystem().mkdirs(remoteLakeDir); + assertThat(remoteLakeDir.getFileSystem().exists(remoteLakeDir)).isTrue(); + + // Drop table + coordinatorGateway + .dropTable( + newDropTableRequest( + tablePath.getDatabaseName(), tablePath.getTableName(), false)) + .get(); + assertThat(zkClient.tableExist(tablePath)).isFalse(); + + // Verify remote log partition dir is deleted + retry( + Duration.ofMinutes(2), + () -> + assertThat( + remoteLogPartitionDir + .getFileSystem() + .exists(remoteLogPartitionDir)) + .isFalse()); + + // Verify remote kv partition dir is deleted for PK table + if (isPkTable) { + retry( + Duration.ofMinutes(2), + () -> + assertThat( + remoteKvPartitionDir + .getFileSystem() + .exists(remoteKvPartitionDir)) + .isFalse()); + } + + // Verify remote lake dir is deleted + retry( + Duration.ofMinutes(2), + () -> assertThat(remoteLakeDir.getFileSystem().exists(remoteLakeDir)).isFalse()); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + void testDeleteSinglePartition(boolean isPkTable) throws Exception { + // Create partitioned table + String tableName = + isPkTable ? "pk_partition_drop_cleaner_table" : "partition_drop_cleaner_table"; + TablePath tablePath = TablePath.of("test_db", tableName); + TableDescriptor tableDescriptor = + isPkTable + ? DATA1_PARTITIONED_TABLE_DESCRIPTOR_PK + : DATA1_PARTITIONED_TABLE_DESCRIPTOR; + + long tableId = + RpcMessageTestUtils.createTable( + FLUSS_CLUSTER_EXTENSION, tablePath, tableDescriptor); + + // Get remote data dir for table + Optional tableOpt = zkClient.getTable(tablePath); + assertThat(tableOpt).isPresent(); + String remoteTableDataDir = tableOpt.get().remoteDataDir; + assertThat(remoteTableDataDir).isNotNull(); + + // Create a manual partition + String partitionName = "manual_partition"; + PartitionSpec partitionSpec = + new PartitionSpec(Collections.singletonMap("b", partitionName)); + RpcMessageTestUtils.createPartition( + FLUSS_CLUSTER_EXTENSION, tablePath, partitionSpec, false); + + // Wait for the partition to be ready + Optional partitionOpt = + zkClient.getPartition(tablePath, partitionName); + assertThat(partitionOpt).isPresent(); + long partitionId = partitionOpt.get().getPartitionId(); + TableBucket tb = new TableBucket(tableId, partitionId, 0); + FLUSS_CLUSTER_EXTENSION.waitUntilAllReplicaReady(tb); + + // Get remote partition dir + String partitionRemoteDataDir = partitionOpt.get().getRemoteDataDir(); + assertThat(partitionRemoteDataDir).isNotNull(); + + PhysicalTablePath physicalTablePath = PhysicalTablePath.of(tablePath, partitionName); + TablePartition tablePartition = new TablePartition(tableId, partitionId); + + // Mock remote log dirs + FsPath remoteLogPartitionDir = + FlussPaths.remotePartitionDir( + FlussPaths.remoteLogDir(partitionRemoteDataDir), + physicalTablePath, + tablePartition); + remoteLogPartitionDir.getFileSystem().mkdirs(remoteLogPartitionDir); + assertThat(remoteLogPartitionDir.getFileSystem().exists(remoteLogPartitionDir)).isTrue(); + + // Mock remote kv dirs + FsPath remoteKvPartitionDir; + if (isPkTable) { + remoteKvPartitionDir = + FlussPaths.remotePartitionDir( + FlussPaths.remoteKvDir(partitionRemoteDataDir), + physicalTablePath, + tablePartition); + remoteKvPartitionDir.getFileSystem().mkdirs(remoteKvPartitionDir); + assertThat(remoteKvPartitionDir.getFileSystem().exists(remoteKvPartitionDir)).isTrue(); + } else { + remoteKvPartitionDir = null; + } + + // Mock remote lake dirs + FsPath remoteLakeDir = + FlussPaths.remoteLakeTableSnapshotDir(remoteTableDataDir, tablePath, tableId); + remoteLakeDir.getFileSystem().mkdirs(remoteLakeDir); + File remoteLakeDirFile = new File(remoteLakeDir.getPath()); + assertThat(remoteLakeDirFile.exists()).isTrue(); + + // Drop partition + coordinatorGateway + .dropPartition(newDropPartitionRequest(tablePath, partitionSpec, false)) + .get(); + + // Wait partition is dropped + FLUSS_CLUSTER_EXTENSION.waitUntilPartitionsDropped( + tablePath, Collections.singletonList(partitionName)); + + // Verify remote log partition dir is deleted + retry( + Duration.ofMinutes(1), + () -> + assertThat( + remoteLogPartitionDir + .getFileSystem() + .exists(remoteLogPartitionDir)) + .isFalse()); + + // Verify remote kv partition dir is deleted for PK table + if (isPkTable) { + retry( + Duration.ofMinutes(1), + () -> + assertThat( + remoteKvPartitionDir + .getFileSystem() + .exists(remoteKvPartitionDir)) + .isFalse()); + } + + // Verify remote lake dir still exists (only partition is deleted, not table) + assertThat(remoteLakeDirFile.exists()).isTrue(); + + // Verify table still exists + assertThat(zkClient.tableExist(tablePath)).isTrue(); + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/RoundRobinRemoteDirSelectorTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/RoundRobinRemoteDirSelectorTest.java new file mode 100644 index 0000000000..b94d3831a8 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/RoundRobinRemoteDirSelectorTest.java @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.coordinator.remote; + +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link RoundRobinRemoteDirSelector}. */ +class RoundRobinRemoteDirSelectorTest { + + private static final String DEFAULT_DIR = "hdfs://default/data"; + + @Test + void testEmptyRemoteDirsShouldReturnDefault() { + RoundRobinRemoteDirSelector selector = + new RoundRobinRemoteDirSelector(DEFAULT_DIR, Collections.emptyList()); + + // Should always return default when remoteDataDirs is empty + for (int i = 0; i < 10; i++) { + assertThat(selector.nextDataDir()).isEqualTo(DEFAULT_DIR); + } + } + + @Test + void testSingleDirShouldAlwaysReturnSame() { + String dir = "hdfs://cluster/data1"; + RoundRobinRemoteDirSelector selector = + new RoundRobinRemoteDirSelector(DEFAULT_DIR, Collections.singletonList(dir)); + + // Should always return the single directory + for (int i = 0; i < 10; i++) { + assertThat(selector.nextDataDir()).isEqualTo(dir); + } + } + + @Test + void testRoundRobinOrder() { + List dirs = + Arrays.asList( + "hdfs://cluster/data1", "hdfs://cluster/data2", "hdfs://cluster/data3"); + + RoundRobinRemoteDirSelector selector = new RoundRobinRemoteDirSelector(DEFAULT_DIR, dirs); + + // Collect selections for multiple cycles + List selections = new ArrayList<>(); + for (int i = 0; i < 9; i++) { + selections.add(selector.nextDataDir()); + } + + // Verify round-robin pattern: each cycle should contain all dirs in order + // First cycle + assertThat(selections.subList(0, 3)).containsExactlyElementsOf(dirs); + // Second cycle + assertThat(selections.subList(3, 6)).containsExactlyElementsOf(dirs); + // Third cycle + assertThat(selections.subList(6, 9)).containsExactlyElementsOf(dirs); + } + + @Test + void testEvenDistribution() { + List dirs = + Arrays.asList( + "hdfs://cluster/data1", "hdfs://cluster/data2", "hdfs://cluster/data3"); + + RoundRobinRemoteDirSelector selector = new RoundRobinRemoteDirSelector(DEFAULT_DIR, dirs); + + Map counts = new HashMap<>(); + int totalCalls = 30; + + for (int i = 0; i < totalCalls; i++) { + String selected = selector.nextDataDir(); + counts.merge(selected, 1, Integer::sum); + } + + // Each directory should be selected equally + assertThat(counts.get(dirs.get(0))).isEqualTo(10); + assertThat(counts.get(dirs.get(1))).isEqualTo(10); + assertThat(counts.get(dirs.get(2))).isEqualTo(10); + } + + @Test + void testTwoDirs() { + List dirs = Arrays.asList("hdfs://cluster/data1", "hdfs://cluster/data2"); + + RoundRobinRemoteDirSelector selector = new RoundRobinRemoteDirSelector(DEFAULT_DIR, dirs); + + // Verify alternating pattern + assertThat(selector.nextDataDir()).isEqualTo(dirs.get(0)); + assertThat(selector.nextDataDir()).isEqualTo(dirs.get(1)); + assertThat(selector.nextDataDir()).isEqualTo(dirs.get(0)); + assertThat(selector.nextDataDir()).isEqualTo(dirs.get(1)); + } + + @Test + void testCycleWrapsCorrectly() { + List dirs = + Arrays.asList( + "hdfs://cluster/data1", "hdfs://cluster/data2", "hdfs://cluster/data3"); + + RoundRobinRemoteDirSelector selector = new RoundRobinRemoteDirSelector(DEFAULT_DIR, dirs); + + // Collect first cycle + List firstCycle = new ArrayList<>(); + for (int i = 0; i < 3; i++) { + firstCycle.add(selector.nextDataDir()); + } + + // Collect second cycle + List secondCycle = new ArrayList<>(); + for (int i = 0; i < 3; i++) { + secondCycle.add(selector.nextDataDir()); + } + + // Both cycles should have same sequence + assertThat(secondCycle).isEqualTo(firstCycle); + } + + @Test + void testAllDirsSelectedInOneCycle() { + List dirs = + Arrays.asList( + "hdfs://cluster/data1", + "hdfs://cluster/data2", + "hdfs://cluster/data3", + "hdfs://cluster/data4", + "hdfs://cluster/data5"); + + RoundRobinRemoteDirSelector selector = new RoundRobinRemoteDirSelector(DEFAULT_DIR, dirs); + + Set selectedInCycle = new HashSet<>(); + for (int i = 0; i < dirs.size(); i++) { + selectedInCycle.add(selector.nextDataDir()); + } + + // All directories should be selected exactly once in one cycle + assertThat(selectedInCycle).containsExactlyInAnyOrderElementsOf(dirs); + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/WeightedRoundRobinRemoteDirSelectorTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/WeightedRoundRobinRemoteDirSelectorTest.java new file mode 100644 index 0000000000..44fe690a7d --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/remote/WeightedRoundRobinRemoteDirSelectorTest.java @@ -0,0 +1,246 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.coordinator.remote; + +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Tests for {@link WeightedRoundRobinRemoteDirSelector}. */ +class WeightedRoundRobinRemoteDirSelectorTest { + + private static final String DEFAULT_DIR = "hdfs://default/data"; + + @Test + void testEmptyRemoteDirsShouldReturnDefault() { + WeightedRoundRobinRemoteDirSelector selector = + new WeightedRoundRobinRemoteDirSelector( + DEFAULT_DIR, Collections.emptyList(), Collections.emptyList()); + + // Should always return default when remoteDataDirs is empty + for (int i = 0; i < 10; i++) { + assertThat(selector.nextDataDir()).isEqualTo(DEFAULT_DIR); + } + } + + @Test + void testSingleDirShouldAlwaysReturnSame() { + String dir = "hdfs://cluster/data1"; + WeightedRoundRobinRemoteDirSelector selector = + new WeightedRoundRobinRemoteDirSelector( + DEFAULT_DIR, Collections.singletonList(dir), Collections.singletonList(5)); + + // Should always return the single directory + for (int i = 0; i < 10; i++) { + assertThat(selector.nextDataDir()).isEqualTo(dir); + } + } + + @Test + void testEqualWeightsShouldDistributeEvenly() { + List dirs = + Arrays.asList( + "hdfs://cluster/data1", "hdfs://cluster/data2", "hdfs://cluster/data3"); + List weights = Arrays.asList(1, 1, 1); + + WeightedRoundRobinRemoteDirSelector selector = + new WeightedRoundRobinRemoteDirSelector(DEFAULT_DIR, dirs, weights); + + Map counts = new HashMap<>(); + int totalCalls = 30; + + for (int i = 0; i < totalCalls; i++) { + String selected = selector.nextDataDir(); + counts.merge(selected, 1, Integer::sum); + } + + // Each directory should be selected equally + assertThat(counts.get(dirs.get(0))).isEqualTo(10); + assertThat(counts.get(dirs.get(1))).isEqualTo(10); + assertThat(counts.get(dirs.get(2))).isEqualTo(10); + } + + @Test + void testWeightedDistribution() { + List dirs = + Arrays.asList( + "hdfs://cluster/data1", "hdfs://cluster/data2", "hdfs://cluster/data3"); + // weights: 5, 1, 1 -> total = 7 + List weights = Arrays.asList(5, 1, 1); + + WeightedRoundRobinRemoteDirSelector selector = + new WeightedRoundRobinRemoteDirSelector(DEFAULT_DIR, dirs, weights); + + Map counts = new HashMap<>(); + int totalCalls = 70; // 10 complete cycles + + for (int i = 0; i < totalCalls; i++) { + String selected = selector.nextDataDir(); + counts.merge(selected, 1, Integer::sum); + } + + // Distribution should match weights ratio: 5:1:1 + assertThat(counts.get(dirs.get(0))).isEqualTo(50); // 5/7 * 70 = 50 + assertThat(counts.get(dirs.get(1))).isEqualTo(10); // 1/7 * 70 = 10 + assertThat(counts.get(dirs.get(2))).isEqualTo(10); // 1/7 * 70 = 10 + } + + @Test + void testInterleavedDistribution() { + // Verify that selections are interleaved, not consecutive + List dirs = + Arrays.asList("hdfs://cluster/A", "hdfs://cluster/B", "hdfs://cluster/C"); + // weights: 5, 1, 1 -> total = 7 + List weights = Arrays.asList(5, 1, 1); + + WeightedRoundRobinRemoteDirSelector selector = + new WeightedRoundRobinRemoteDirSelector(DEFAULT_DIR, dirs, weights); + + List sequence = new ArrayList<>(); + for (int i = 0; i < 7; i++) { + sequence.add(selector.nextDataDir()); + } + + // Expected interleaved sequence for weights 5,1,1: + // The smooth WRR should produce: A, A, B, A, C, A, A (or similar interleaved pattern) + // Instead of traditional WRR: A, A, A, A, A, B, C + + // Count consecutive same selections - should be less than weight + int maxConsecutive = 0; + int currentConsecutive = 1; + for (int i = 1; i < sequence.size(); i++) { + if (sequence.get(i).equals(sequence.get(i - 1))) { + currentConsecutive++; + } else { + maxConsecutive = Math.max(maxConsecutive, currentConsecutive); + currentConsecutive = 1; + } + } + maxConsecutive = Math.max(maxConsecutive, currentConsecutive); + + // With smooth WRR, max consecutive selections should be <= 2 for this weight distribution + // (In traditional WRR, A would be selected 5 times consecutively) + assertThat(maxConsecutive).isLessThanOrEqualTo(2); + + // Verify all directories are selected at least once within one cycle + assertThat(sequence).contains(dirs.get(0), dirs.get(1), dirs.get(2)); + } + + @Test + void testTwoDirsWithDifferentWeights() { + List dirs = Arrays.asList("hdfs://cluster/data1", "hdfs://cluster/data2"); + // weights: 3, 1 -> total = 4 + List weights = Arrays.asList(3, 1); + + WeightedRoundRobinRemoteDirSelector selector = + new WeightedRoundRobinRemoteDirSelector(DEFAULT_DIR, dirs, weights); + + List sequence = new ArrayList<>(); + for (int i = 0; i < 8; i++) { + sequence.add(selector.nextDataDir()); + } + + // Count selections + long dir1Count = sequence.stream().filter(d -> d.equals(dirs.get(0))).count(); + long dir2Count = sequence.stream().filter(d -> d.equals(dirs.get(1))).count(); + + // Should follow 3:1 ratio + assertThat(dir1Count).isEqualTo(6); // 3/4 * 8 = 6 + assertThat(dir2Count).isEqualTo(2); // 1/4 * 8 = 2 + } + + @Test + void testCycleRepeatsCorrectly() { + List dirs = Arrays.asList("hdfs://cluster/data1", "hdfs://cluster/data2"); + List weights = Arrays.asList(2, 1); + + WeightedRoundRobinRemoteDirSelector selector = + new WeightedRoundRobinRemoteDirSelector(DEFAULT_DIR, dirs, weights); + + // Collect first cycle (3 selections) + List firstCycle = new ArrayList<>(); + for (int i = 0; i < 3; i++) { + firstCycle.add(selector.nextDataDir()); + } + + // Collect second cycle (3 selections) + List secondCycle = new ArrayList<>(); + for (int i = 0; i < 3; i++) { + secondCycle.add(selector.nextDataDir()); + } + + // Both cycles should have same sequence + assertThat(secondCycle).isEqualTo(firstCycle); + } + + @Test + void testLargeWeights() { + List dirs = Arrays.asList("hdfs://cluster/data1", "hdfs://cluster/data2"); + List weights = Arrays.asList(100, 1); + + WeightedRoundRobinRemoteDirSelector selector = + new WeightedRoundRobinRemoteDirSelector(DEFAULT_DIR, dirs, weights); + + Map counts = new HashMap<>(); + int totalCalls = 101; // One complete cycle + + for (int i = 0; i < totalCalls; i++) { + String selected = selector.nextDataDir(); + counts.merge(selected, 1, Integer::sum); + } + + // Should follow 100:1 ratio + assertThat(counts.get(dirs.get(0))).isEqualTo(100); + assertThat(counts.get(dirs.get(1))).isEqualTo(1); + } + + @Test + void testZeroWeights() { + // Test case 1: Some directories have zero weight - they should never be selected + List dirs = + Arrays.asList( + "hdfs://cluster/data1", "hdfs://cluster/data2", "hdfs://cluster/data3"); + // weight 0 for data2 means it should never be selected + List weights = Arrays.asList(2, 0, 1); + + WeightedRoundRobinRemoteDirSelector selector = + new WeightedRoundRobinRemoteDirSelector(DEFAULT_DIR, dirs, weights); + + Map counts = new HashMap<>(); + int totalCalls = 30; // 10 complete cycles (total weight = 3) + + for (int i = 0; i < totalCalls; i++) { + String selected = selector.nextDataDir(); + counts.merge(selected, 1, Integer::sum); + } + + // data1 should be selected 20 times (2/3 * 30) + assertThat(counts.get(dirs.get(0))).isEqualTo(20); + // data2 should never be selected (weight = 0) + assertThat(counts.get(dirs.get(1))).isNull(); + // data3 should be selected 10 times (1/3 * 30) + assertThat(counts.get(dirs.get(2))).isEqualTo(10); + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachineTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachineTest.java index 454ec5de4b..70578b6144 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachineTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachineTest.java @@ -54,6 +54,7 @@ import static org.apache.fluss.record.TestData.DATA1_TABLE_DESCRIPTOR; import static org.apache.fluss.record.TestData.DATA1_TABLE_PATH; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.server.coordinator.statemachine.ReplicaState.NewReplica; import static org.apache.fluss.server.coordinator.statemachine.ReplicaState.OfflineReplica; import static org.apache.fluss.server.coordinator.statemachine.ReplicaState.OnlineReplica; @@ -202,6 +203,7 @@ void testOfflineReplicasShouldBeRemovedFromIsr() throws Exception { tableId, 0, DATA1_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis())); coordinatorContext.putTablePath(tableId, DATA1_TABLE_PATH); @@ -240,6 +242,7 @@ void testOfflineReplicaShouldBeRemovedFromIsr() throws Exception { tableId, 0, DATA1_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis())); coordinatorContext.putTablePath(tableId, DATA1_TABLE_PATH); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachineTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachineTest.java index b905ff9694..6d28a139dc 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachineTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachineTest.java @@ -36,6 +36,7 @@ import org.apache.fluss.server.coordinator.TestCoordinatorChannelManager; import org.apache.fluss.server.coordinator.event.CoordinatorEventManager; import org.apache.fluss.server.coordinator.lease.KvSnapshotLeaseManager; +import org.apache.fluss.server.coordinator.remote.RemoteDirDynamicLoader; import org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElection.ControlledShutdownLeaderElection; import org.apache.fluss.server.coordinator.statemachine.TableBucketStateMachine.ElectionResult; import org.apache.fluss.server.metadata.CoordinatorMetadataCache; @@ -64,6 +65,7 @@ import java.util.stream.Collectors; import static org.apache.fluss.record.TestData.DATA1_TABLE_DESCRIPTOR; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.server.coordinator.CoordinatorTestUtils.createServers; import static org.apache.fluss.server.coordinator.CoordinatorTestUtils.makeSendLeaderAndStopRequestAlwaysSuccess; import static org.apache.fluss.server.coordinator.statemachine.BucketState.NewBucket; @@ -121,6 +123,7 @@ void beforeEach() throws IOException { zookeeperClient, new Configuration(), new LakeCatalogDynamicLoader(new Configuration(), null, true)), + new RemoteDirDynamicLoader(conf), new Configuration()); lakeTableTieringManager = new LakeTableTieringManager(TestingMetricGroups.LAKE_TIERING_METRICS); @@ -216,6 +219,7 @@ void testStateChangeToOnline() throws Exception { tableId, 0, DATA1_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis())); coordinatorContext.putTablePath(tableId, fakeTablePath); @@ -361,6 +365,7 @@ void testStateChangeForTabletServerControlledShutdown() { tableId, 0, DATA1_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis())); coordinatorContext.putTablePath(tableId, fakeTablePath); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/kv/snapshot/KvSnapshotMultipleDirsITCase.java b/fluss-server/src/test/java/org/apache/fluss/server/kv/snapshot/KvSnapshotMultipleDirsITCase.java new file mode 100644 index 0000000000..648bdb0d80 --- /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); + + // 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()); + + // 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..021e9d1b04 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/log/DroppedTableRecoveryTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/log/DroppedTableRecoveryTest.java @@ -55,6 +55,7 @@ import static org.apache.fluss.record.TestData.DATA1_SCHEMA_PK; import static org.apache.fluss.record.TestData.DATA1_TABLE_DESCRIPTOR; import static org.apache.fluss.record.TestData.DATA1_TABLE_ID; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.testutils.DataTestUtils.genMemoryLogRecordsByObject; import static org.assertj.core.api.Assertions.assertThat; @@ -108,7 +109,9 @@ public void setup() throws Exception { private void registerTableInZkClient() throws Exception { ZOO_KEEPER_EXTENSION_WRAPPER.getCustomExtension().cleanupRoot(); zkClient.registerTable( - tablePath, TableRegistration.newTable(DATA1_TABLE_ID, DATA1_TABLE_DESCRIPTOR)); + tablePath, + TableRegistration.newTable( + DATA1_TABLE_ID, DEFAULT_REMOTE_DATA_DIR, DATA1_TABLE_DESCRIPTOR)); zkClient.registerFirstSchema(tablePath, DATA1_SCHEMA); } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/log/LogManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/log/LogManagerTest.java index 80e99c2f56..bdaa5d6464 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/log/LogManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/log/LogManagerTest.java @@ -60,6 +60,7 @@ import static org.apache.fluss.record.TestData.DATA2_SCHEMA; import static org.apache.fluss.record.TestData.DATA2_TABLE_DESCRIPTOR; import static org.apache.fluss.record.TestData.DATA2_TABLE_ID; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.server.log.LogManager.CLEAN_SHUTDOWN_FILE; import static org.apache.fluss.testutils.DataTestUtils.assertLogRecordsEquals; import static org.apache.fluss.testutils.DataTestUtils.genMemoryLogRecordsByObject; @@ -113,10 +114,14 @@ public void setup() throws Exception { private void registerTableInZkClient() throws Exception { ZOO_KEEPER_EXTENSION_WRAPPER.getCustomExtension().cleanupRoot(); zkClient.registerTable( - tablePath1, TableRegistration.newTable(DATA1_TABLE_ID, DATA1_TABLE_DESCRIPTOR)); + tablePath1, + TableRegistration.newTable( + DATA1_TABLE_ID, DEFAULT_REMOTE_DATA_DIR, DATA1_TABLE_DESCRIPTOR)); zkClient.registerFirstSchema(tablePath1, DATA1_SCHEMA); zkClient.registerTable( - tablePath2, TableRegistration.newTable(DATA2_TABLE_ID, DATA2_TABLE_DESCRIPTOR)); + tablePath2, + TableRegistration.newTable( + DATA2_TABLE_ID, DEFAULT_REMOTE_DATA_DIR, DATA2_TABLE_DESCRIPTOR)); zkClient.registerFirstSchema(tablePath2, DATA2_SCHEMA); } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/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 90dfa0914e..3f72b2f3e0 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,13 @@ 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.DataLakeFormat; +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; @@ -38,27 +41,39 @@ import org.apache.fluss.server.replica.Replica; 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.apache.fluss.utils.clock.ManualClock; +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.Comparator; +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 java.util.stream.Collectors; 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.newAlterTableRequest; import static org.apache.fluss.server.testutils.RpcMessageTestUtils.newDropTableRequest; @@ -73,14 +88,24 @@ public class RemoteLogITCase { private static final ManualClock MANUAL_CLOCK = new ManualClock(System.currentTimeMillis()); + 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()) .setClock(MANUAL_CLOCK) + .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); @@ -138,9 +163,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); @@ -223,6 +246,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); + } + @Test void testRemoteLogTTLWithDynamicLakeToggle() throws Exception { TablePath tablePath = TablePath.of("fluss", "test_remote_log_ttl_dynamic_lake"); 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 2ac4e20967..38906ea3ec 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 @@ -50,7 +50,6 @@ import static org.apache.fluss.record.TestData.DATA1_PHYSICAL_TABLE_PATH; import static org.apache.fluss.record.TestData.DATA1_PHYSICAL_TABLE_PATH_PA_2024; 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; @@ -363,7 +362,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(); @@ -692,18 +691,6 @@ void testCopySegmentPartialFailureCommitsSuccessfulOnes(boolean partitionTable) .collect(Collectors.toSet())); } - 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 static Stream stopArgs() { return Stream.of( Arguments.of(false, false), 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 s.remoteLogSegmentId().toString()) .collect(Collectors.toSet())); } - - private TableBucket makeTableBucket(boolean partitionTable) { - if (partitionTable) { - return new TableBucket(DATA1_TABLE_ID, 0L, 0); - } else { - return new TableBucket(DATA1_TABLE_ID, 0); - } - } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/log/remote/RemoteLogTTLTest.java b/fluss-server/src/test/java/org/apache/fluss/server/log/remote/RemoteLogTTLTest.java index d23667431f..62603e8e93 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/log/remote/RemoteLogTTLTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/log/remote/RemoteLogTTLTest.java @@ -33,7 +33,6 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; -import static org.apache.fluss.record.TestData.DATA1_TABLE_ID; import static org.assertj.core.api.Assertions.assertThat; /** Test for remote log ttl in {@link RemoteLogManager}. */ @@ -47,12 +46,7 @@ public void setup() throws Exception { @ParameterizedTest @ValueSource(booleans = {true, false}) void testRemoteLogTTL(boolean partitionTable) throws Exception { - TableBucket tb; - if (partitionTable) { - tb = new TableBucket(DATA1_TABLE_ID, 0L, 0); - } else { - tb = new TableBucket(DATA1_TABLE_ID, 0); - } + TableBucket tb = makeTableBucket(partitionTable); // Need to make leader by ReplicaManager. makeLogTableAsLeader(tb, partitionTable); LogTablet logTablet = replicaManager.getReplicaOrException(tb).getLogTablet(); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/log/remote/RemoteLogTestBase.java b/fluss-server/src/test/java/org/apache/fluss/server/log/remote/RemoteLogTestBase.java index 13ab2f7ba7..cfd517a27c 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/log/remote/RemoteLogTestBase.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/log/remote/RemoteLogTestBase.java @@ -20,6 +20,7 @@ import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; import org.apache.fluss.config.MemorySize; +import org.apache.fluss.fs.FsPath; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.remote.RemoteLogSegment; @@ -29,6 +30,7 @@ import org.apache.fluss.server.replica.Replica; import org.apache.fluss.server.replica.ReplicaTestBase; import org.apache.fluss.server.zk.data.LeaderAndIsr; +import org.apache.fluss.utils.FlussPaths; import org.junit.jupiter.api.BeforeEach; @@ -43,6 +45,7 @@ import static org.apache.fluss.record.TestData.DATA1_PHYSICAL_TABLE_PATH; import static org.apache.fluss.record.TestData.DATA1_PHYSICAL_TABLE_PATH_PA_2024; import static org.apache.fluss.record.TestData.DATA1_TABLE_ID; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.assertj.core.api.Assertions.assertThat; /** Test base for remote log. */ @@ -93,7 +96,7 @@ private Replica makeReplicaAndAddSegments( } protected static RemoteLogSegment copyLogSegmentToRemote( - LogTablet logTablet, RemoteLogStorage remoteLogStorage, int segmentIndex) + LogTablet logTablet, DefaultRemoteLogStorage remoteLogStorage, int segmentIndex) throws Exception { PhysicalTablePath tp = logTablet.getPhysicalTablePath(); TableBucket tb = logTablet.getTableBucket(); @@ -120,6 +123,7 @@ protected static RemoteLogSegment copyLogSegmentToRemote( .segmentSizeInBytes(segment.getFileLogRecords().sizeInBytes()) .tableBucket(tb) .physicalTablePath(tp) + .remoteLogDir(new FsPath(DEFAULT_REMOTE_DATA_DIR)) .build(); remoteLogStorage.copyLogSegmentFiles(remoteLogSegment, logSegmentFiles); @@ -130,10 +134,11 @@ protected RemoteLogTablet buildRemoteLogTablet(LogTablet logTablet) { return new RemoteLogTablet( logTablet.getPhysicalTablePath(), logTablet.getTableBucket(), - conf.get(ConfigOptions.TABLE_LOG_TTL).toMillis()); + conf.get(ConfigOptions.TABLE_LOG_TTL).toMillis(), + FlussPaths.remoteLogDir(conf)); } - protected static List createRemoteLogSegmentList(LogTablet logTablet) { + protected List createRemoteLogSegmentList(LogTablet logTablet) { return logTablet.getSegments().stream() .map( segment -> { @@ -147,6 +152,7 @@ protected static List createRemoteLogSegmentList(LogTablet log segment.getFileLogRecords().sizeInBytes()) .tableBucket(logTablet.getTableBucket()) .physicalTablePath(logTablet.getPhysicalTablePath()) + .remoteLogDir(FlussPaths.remoteLogDir(conf)) .build(); } catch (IOException e) { throw new RuntimeException(e); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/metadata/ServerSchemaCacheTest.java b/fluss-server/src/test/java/org/apache/fluss/server/metadata/ServerSchemaCacheTest.java index c42c7b296c..66656f5b2a 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/metadata/ServerSchemaCacheTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/metadata/ServerSchemaCacheTest.java @@ -41,6 +41,7 @@ import static org.apache.fluss.record.TestData.DATA1_TABLE_PATH; import static org.apache.fluss.record.TestData.DATA2_SCHEMA; import static org.apache.fluss.record.TestData.DATA2_TABLE_DESCRIPTOR; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -89,6 +90,7 @@ void testGetHistorySchema() { DATA1_TABLE_ID, 2, DATA2_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis()); ServerSchemaCache manager = diff --git a/fluss-server/src/test/java/org/apache/fluss/server/metadata/TabletServerMetadataCacheTest.java b/fluss-server/src/test/java/org/apache/fluss/server/metadata/TabletServerMetadataCacheTest.java index 6ba37e44d4..8783657648 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/metadata/TabletServerMetadataCacheTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/metadata/TabletServerMetadataCacheTest.java @@ -45,6 +45,7 @@ import static org.apache.fluss.record.TestData.DATA1_TABLE_ID; import static org.apache.fluss.record.TestData.DATA1_TABLE_INFO; import static org.apache.fluss.record.TestData.DATA1_TABLE_PATH; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.server.metadata.PartitionMetadata.DELETED_PARTITION_ID; import static org.apache.fluss.server.metadata.TableMetadata.DELETED_TABLE_ID; import static org.apache.fluss.server.zk.data.LeaderAndIsr.NO_LEADER; @@ -69,6 +70,7 @@ public class TabletServerMetadataCacheTest { partitionTableId, 0, DATA1_PARTITIONED_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, 100L, 100L); private List tableMetadataList; @@ -217,6 +219,7 @@ void testUpdateClusterMetadataRequest() { // deletion. 1, DATA1_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis()), changedBucket1BucketMetadata)), diff --git a/fluss-server/src/test/java/org/apache/fluss/server/metadata/ZkBasedMetadataProviderTest.java b/fluss-server/src/test/java/org/apache/fluss/server/metadata/ZkBasedMetadataProviderTest.java index 1ebb0c3e03..e6a0064f15 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/metadata/ZkBasedMetadataProviderTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/metadata/ZkBasedMetadataProviderTest.java @@ -51,6 +51,7 @@ import java.util.Map; import java.util.stream.Collectors; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -100,7 +101,9 @@ void testGetTableMetadataFromZk() throws Exception { .add(1, BucketAssignment.of(2, 3, 4)) .build(); metadataManager.createDatabase("test_db", DatabaseDescriptor.EMPTY, true); - long tableId = metadataManager.createTable(tablePath, desc, tableAssignment, false); + long tableId = + metadataManager.createTable( + tablePath, DEFAULT_REMOTE_DATA_DIR, desc, tableAssignment, false); // Create leader and isr for buckets TableBucket tableBucket0 = new TableBucket(tableId, 0); @@ -161,7 +164,12 @@ void testGetPartitionMetadataFromZk() throws Exception { new PartitionAssignment(tableId, bucketAssignments); zookeeperClient.registerPartitionAssignmentAndMetadata( - partitionId, partitionName, partitionAssignment, tablePath, tableId); + partitionId, + partitionName, + partitionAssignment, + DEFAULT_REMOTE_DATA_DIR, + tablePath, + tableId); // Create leader and isr for partition buckets TableBucket partitionBucket0 = new TableBucket(tableId, partitionId, 0); @@ -233,9 +241,19 @@ void testBatchGetPartitionMetadataFromZkAsync() throws Exception { tableId1, Collections.singletonMap(1, BucketAssignment.of(2, 3))); zookeeperClient.registerPartitionAssignmentAndMetadata( - partitionId1, partitionName1, partitionAssignment1, tablePath1, tableId1); + partitionId1, + partitionName1, + partitionAssignment1, + DEFAULT_REMOTE_DATA_DIR, + tablePath1, + tableId1); zookeeperClient.registerPartitionAssignmentAndMetadata( - partitionId2, partitionName2, partitionAssignment2, tablePath1, tableId1); + partitionId2, + partitionName2, + partitionAssignment2, + DEFAULT_REMOTE_DATA_DIR, + tablePath1, + tableId1); // Create partition for table2 long partitionId3 = 21L; @@ -246,7 +264,12 @@ void testBatchGetPartitionMetadataFromZkAsync() throws Exception { tableId2, Collections.singletonMap(0, BucketAssignment.of(1, 3))); zookeeperClient.registerPartitionAssignmentAndMetadata( - partitionId3, partitionName3, partitionAssignment3, tablePath2, tableId2); + partitionId3, + partitionName3, + partitionAssignment3, + DEFAULT_REMOTE_DATA_DIR, + tablePath2, + tableId2); // Create leader and isr for all partition buckets TableBucket bucket1 = new TableBucket(tableId1, partitionId1, 0); @@ -338,6 +361,7 @@ private TableRegistration createTestTableRegistration( new TableDescriptor.TableDistribution(3, Collections.singletonList("a")), options, Collections.emptyMap(), + DEFAULT_REMOTE_DATA_DIR, currentMillis, currentMillis); } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/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 bd63f3cbfd..e6d8125458 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java @@ -123,6 +123,7 @@ import static org.apache.fluss.record.TestData.DATA3_TABLE_ID_PK_AUTO_INC; import static org.apache.fluss.record.TestData.DATA3_TABLE_PATH_PK_AUTO_INC; import static org.apache.fluss.record.TestData.DATA_1_WITH_KEY_AND_VALUE; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.record.TestData.DEFAULT_SCHEMA_ID; import static org.apache.fluss.record.TestData.EXPECTED_LOG_RESULTS_FOR_DATA_1_WITH_PK; import static org.apache.fluss.server.coordinator.CoordinatorContext.INITIAL_COORDINATOR_EPOCH; @@ -1250,7 +1251,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. @@ -1903,6 +1904,7 @@ void testUpdateMetadata() throws Exception { nonePartitionTableId, 1, DATA1_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis()); TableInfo partitionTableInfo = @@ -1911,6 +1913,7 @@ void testUpdateMetadata() throws Exception { partitionTableId, 1, DATA1_PARTITIONED_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis()); TableMetadata tableMetadata1 = @@ -1935,11 +1938,15 @@ void testUpdateMetadata() throws Exception { // register table to zk. zkClient.registerTable( nonePartitionTablePath, - TableRegistration.newTable(nonePartitionTableId, DATA1_TABLE_DESCRIPTOR)); + TableRegistration.newTable( + nonePartitionTableId, DEFAULT_REMOTE_DATA_DIR, DATA1_TABLE_DESCRIPTOR)); zkClient.registerFirstSchema(nonePartitionTablePath, DATA1_TABLE_DESCRIPTOR.getSchema()); zkClient.registerTable( partitionTablePath, - TableRegistration.newTable(partitionTableId, DATA1_PARTITIONED_TABLE_DESCRIPTOR)); + TableRegistration.newTable( + partitionTableId, + DEFAULT_REMOTE_DATA_DIR, + DATA1_PARTITIONED_TABLE_DESCRIPTOR)); zkClient.registerFirstSchema( partitionTablePath, DATA1_PARTITIONED_TABLE_DESCRIPTOR.getSchema()); @@ -1978,6 +1985,7 @@ void testUpdateMetadata() throws Exception { DELETED_TABLE_ID, // mark as deleted. 1, DATA1_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis()), Collections.emptyList())), @@ -2065,6 +2073,7 @@ void testGetReplicaOrException() { tableId, 1, DATA1_TABLE_DESCRIPTOR, + DEFAULT_REMOTE_DATA_DIR, System.currentTimeMillis(), System.currentTimeMillis()); TableBucket tableBucket1 = new TableBucket(tableId, 1); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTestBase.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTestBase.java index 3fb01a7e25..77dbc8b3f5 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,6 +57,7 @@ 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; @@ -110,6 +111,7 @@ import static org.apache.fluss.record.TestData.DATA3_TABLE_DESCRIPTOR_PK_AUTO_INC; import static org.apache.fluss.record.TestData.DATA3_TABLE_ID_PK_AUTO_INC; import static org.apache.fluss.record.TestData.DATA3_TABLE_PATH_PK_AUTO_INC; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.server.coordinator.CoordinatorContext.INITIAL_COORDINATOR_EPOCH; import static org.apache.fluss.server.replica.ReplicaManager.HIGH_WATERMARK_CHECKPOINT_FILE_NAME; import static org.apache.fluss.server.zk.data.LeaderAndIsr.INITIAL_BUCKET_EPOCH; @@ -130,6 +132,7 @@ public class ReplicaTestBase { protected static final int TABLET_SERVER_ID = 1; private static final String TABLET_SERVER_RACK = "rack1"; protected static ZooKeeperClient zkClient; + protected static String remoteDataDir; // to register all should be closed after each test private final CloseableRegistry closeableRegistry = new CloseableRegistry(); @@ -164,6 +167,7 @@ static void baseBeforeAll() { ZOO_KEEPER_EXTENSION_WRAPPER .getCustomExtension() .getZooKeeperClient(NOPErrorHandler.INSTANCE); + remoteDataDir = zkClient.getDefaultRemoteDataDir(); } @BeforeEach @@ -259,22 +263,33 @@ 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, + conf.get(ConfigOptions.REMOTE_DATA_DIR), + data1NonPkTableDescriptor)); zkClient.registerFirstSchema(DATA1_TABLE_PATH, DATA1_SCHEMA); zkClient.registerTable( DATA1_TABLE_PATH_PK, - TableRegistration.newTable(DATA1_TABLE_ID_PK, DATA1_TABLE_DESCRIPTOR_PK)); + TableRegistration.newTable( + DATA1_TABLE_ID_PK, + conf.get(ConfigOptions.REMOTE_DATA_DIR), + DATA1_TABLE_DESCRIPTOR_PK)); zkClient.registerFirstSchema(DATA1_TABLE_PATH_PK, DATA1_SCHEMA_PK); zkClient.registerTable( DATA2_TABLE_PATH, - TableRegistration.newTable(DATA2_TABLE_ID, DATA2_TABLE_DESCRIPTOR)); + TableRegistration.newTable( + DATA2_TABLE_ID, + conf.get(ConfigOptions.REMOTE_DATA_DIR), + DATA2_TABLE_DESCRIPTOR)); zkClient.registerFirstSchema(DATA2_TABLE_PATH, DATA2_SCHEMA); zkClient.registerTable( DATA3_TABLE_PATH_PK_AUTO_INC, TableRegistration.newTable( - DATA3_TABLE_ID_PK_AUTO_INC, DATA3_TABLE_DESCRIPTOR_PK_AUTO_INC)); + DATA3_TABLE_ID_PK_AUTO_INC, + conf.get(ConfigOptions.REMOTE_DATA_DIR), + DATA3_TABLE_DESCRIPTOR_PK_AUTO_INC)); zkClient.registerFirstSchema(DATA3_TABLE_PATH_PK_AUTO_INC, DATA3_SCHEMA_PK_AUTO_INC); } @@ -293,7 +308,9 @@ protected long registerTableInZkClient( if (zkClient.tableExist(tablePath)) { zkClient.deleteTable(tablePath); } - zkClient.registerTable(tablePath, TableRegistration.newTable(tableId, tableDescriptor)); + zkClient.registerTable( + tablePath, + TableRegistration.newTable(tableId, DEFAULT_REMOTE_DATA_DIR, tableDescriptor)); zkClient.registerFirstSchema(tablePath, schema); return tableId; } @@ -491,6 +508,7 @@ private Replica makeReplica( NOPErrorHandler.INSTANCE, metricGroup, DATA1_TABLE_INFO, + remoteDataDir, manualClock); } @@ -566,6 +584,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()), + conf.get(ConfigOptions.REMOTE_DATA_DIR), + DATA1_TABLE_PATH_PK, + tableId); + } else { + zkClient.registerPartitionAssignmentAndMetadata( + partitionId, + DATA1_PHYSICAL_TABLE_PATH_PA_2024.getPartitionName(), + new PartitionAssignment(tableId, Collections.emptyMap()), + conf.get(ConfigOptions.REMOTE_DATA_DIR), + 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 { @@ -653,11 +715,6 @@ public int getSnapshotFsWriteBufferSize() { return 1024; } - @Override - public FsPath getRemoteKvDir() { - return remoteKvTabletDir; - } - @Override public FunctionWithException getLatestCompletedSnapshotProvider() { diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherThreadTest.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherThreadTest.java index c857d84ded..0501d0374b 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherThreadTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherThreadTest.java @@ -75,6 +75,7 @@ import static org.apache.fluss.record.TestData.DATA1_TABLE_DESCRIPTOR; import static org.apache.fluss.record.TestData.DATA1_TABLE_ID; import static org.apache.fluss.record.TestData.DATA1_TABLE_PATH; +import static org.apache.fluss.record.TestData.DEFAULT_REMOTE_DATA_DIR; import static org.apache.fluss.server.coordinator.CoordinatorContext.INITIAL_COORDINATOR_EPOCH; import static org.apache.fluss.server.metrics.group.TestingMetricGroups.USER_METRICS; import static org.apache.fluss.server.zk.data.LeaderAndIsr.INITIAL_BUCKET_EPOCH; @@ -377,7 +378,8 @@ private void registerTableInZkClient() throws Exception { ZOO_KEEPER_EXTENSION_WRAPPER.getCustomExtension().cleanupRoot(); zkClient.registerTable( DATA1_TABLE_PATH, - TableRegistration.newTable(DATA1_TABLE_ID, DATA1_TABLE_DESCRIPTOR)); + TableRegistration.newTable( + DATA1_TABLE_ID, DEFAULT_REMOTE_DATA_DIR, DATA1_TABLE_DESCRIPTOR)); zkClient.registerFirstSchema(DATA1_TABLE_PATH, DATA1_SCHEMA); } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java b/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java index 305a4269aa..b583499414 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java @@ -137,6 +137,7 @@ public final class FlussClusterExtension private final Configuration clusterConf; private final Clock clock; private final String[] racks; + private final List remoteDirNames; /** Creates a new {@link Builder} for {@link FlussClusterExtension}. */ public static Builder builder() { @@ -149,7 +150,8 @@ private FlussClusterExtension( String tabletServerListeners, Configuration clusterConf, Clock clock, - String[] racks) { + String[] racks, + List remoteDirNames) { this.initialNumOfTabletServers = numOfTabletServers; this.tabletServers = new HashMap<>(numOfTabletServers); this.coordinatorServerListeners = coordinatorServerListeners; @@ -161,6 +163,7 @@ private FlussClusterExtension( racks != null && racks.length == numOfTabletServers, "racks must be not null and have the same length as numOfTabletServers"); this.racks = racks; + this.remoteDirNames = remoteDirNames; } @Override @@ -203,15 +206,19 @@ public void afterEach(ExtensionContext extensionContext) throws Exception { public void start() throws Exception { tempDir = Files.createTempDirectory("fluss-testing-cluster").toFile(); + Configuration conf = new Configuration(); + setRemoteDataDir(conf); + setRemoteDataDirs(conf); zooKeeperServer = ZooKeeperTestUtils.createAndStartZookeeperTestingServer(); zooKeeperClient = - createZooKeeperClient(zooKeeperServer.getConnectString(), NOPErrorHandler.INSTANCE); + createZooKeeperClient( + conf, zooKeeperServer.getConnectString(), NOPErrorHandler.INSTANCE); metadataManager = new MetadataManager( zooKeeperClient, clusterConf, new LakeCatalogDynamicLoader(clusterConf, null, true)); - Configuration conf = new Configuration(); + rpcClient = RpcClient.create( conf, @@ -261,6 +268,7 @@ public void startCoordinatorServer() throws Exception { conf.setString(ConfigOptions.ZOOKEEPER_ADDRESS, zooKeeperServer.getConnectString()); conf.setString(ConfigOptions.BIND_LISTENERS, coordinatorServerListeners); setRemoteDataDir(conf); + setRemoteDataDirs(conf); coordinatorServer = new CoordinatorServer(conf, clock); coordinatorServer.start(); coordinatorServerInfo = @@ -371,12 +379,26 @@ private void setRemoteDataDir(Configuration conf) { conf.set(ConfigOptions.REMOTE_DATA_DIR, getRemoteDataDir()); } + private void setRemoteDataDirs(Configuration conf) { + if (!remoteDirNames.isEmpty()) { + List remoteDataDirs = + remoteDirNames.stream() + .map(this::getRemoteDataDir) + .collect(Collectors.toList()); + conf.set(ConfigOptions.REMOTE_DATA_DIRS, remoteDataDirs); + } + } + public String getRemoteDataDir() { + return getRemoteDataDir("remote-data-dir"); + } + + public String getRemoteDataDir(String dirName) { return LocalFileSystem.getLocalFsURI().getScheme() + "://" + tempDir.getAbsolutePath() + File.separator - + "remote-data-dir"; + + dirName; } /** Stop a tablet server. */ @@ -774,7 +796,7 @@ private Long triggerSnapshot(TableBucket tableBucket) { } } - private CompletedSnapshot waitUntilSnapshotFinished(TableBucket tableBucket, long snapshotId) { + public CompletedSnapshot waitUntilSnapshotFinished(TableBucket tableBucket, long snapshotId) { ZooKeeperClient zkClient = getZooKeeperClient(); return waitValue( () -> { @@ -940,6 +962,7 @@ public static class Builder { private String coordinatorServerListeners = DEFAULT_LISTENERS; private Clock clock = SystemClock.getInstance(); private String[] racks = new String[] {"rack-0"}; + private List remoteDirNames = Collections.emptyList(); private final Configuration clusterConf = new Configuration(); @@ -985,6 +1008,11 @@ public Builder setRacks(String[] racks) { return this; } + public Builder setRemoteDirNames(List remoteDirNames) { + this.remoteDirNames = remoteDirNames; + return this; + } + public FlussClusterExtension build() { if (numOfTabletServers > 1 && racks.length == 1) { String[] racks = new String[numOfTabletServers]; @@ -1000,7 +1028,8 @@ public FlussClusterExtension build() { tabletServerListeners, clusterConf, clock, - racks); + racks, + remoteDirNames); } } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/testutils/RpcMessageTestUtils.java b/fluss-server/src/test/java/org/apache/fluss/server/testutils/RpcMessageTestUtils.java index 8562163827..e82963e54e 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 @@ -193,6 +193,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(); @@ -203,16 +208,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 4cf0a511e7..cce759c385 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java @@ -29,7 +29,6 @@ import org.apache.fluss.metadata.SchemaInfo; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableDescriptor; -import org.apache.fluss.metadata.TablePartition; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.server.entity.RegisterTableBucketLeadAndIsrInfo; import org.apache.fluss.server.zk.data.BucketAssignment; @@ -37,6 +36,7 @@ import org.apache.fluss.server.zk.data.CoordinatorAddress; import org.apache.fluss.server.zk.data.LeaderAndIsr; import org.apache.fluss.server.zk.data.PartitionAssignment; +import org.apache.fluss.server.zk.data.PartitionRegistration; import org.apache.fluss.server.zk.data.RebalanceTask; import org.apache.fluss.server.zk.data.ServerTags; import org.apache.fluss.server.zk.data.TableAssignment; @@ -84,6 +84,7 @@ class ZooKeeperClientTest { new AllCallbackWrapper<>(new ZooKeeperExtension()); private static ZooKeeperClient zookeeperClient; + private static String remoteDataDir; @BeforeAll static void beforeAll() { @@ -91,6 +92,7 @@ static void beforeAll() { ZOO_KEEPER_EXTENSION_WRAPPER .getCustomExtension() .getZooKeeperClient(NOPErrorHandler.INSTANCE); + remoteDataDir = zookeeperClient.getDefaultRemoteDataDir(); } @AfterEach @@ -331,6 +333,7 @@ void testTable() throws Exception { new TableDescriptor.TableDistribution(16, Collections.singletonList("a")), options, Collections.singletonMap("custom-1", "100"), + remoteDataDir, currentMillis, currentMillis); TableRegistration tableReg2 = @@ -341,6 +344,7 @@ void testTable() throws Exception { new TableDescriptor.TableDistribution(16, Collections.singletonList("a")), options, Collections.singletonMap("custom-2", "200"), + remoteDataDir, currentMillis, currentMillis); zookeeperClient.registerTable(tablePath1, tableReg1); @@ -366,6 +370,7 @@ void testTable() throws Exception { new TableDescriptor.TableDistribution(16, Collections.singletonList("a")), options, Collections.singletonMap("custom-3", "300"), + remoteDataDir, currentMillis, currentMillis); zookeeperClient.updateTable(tablePath1, tableReg1); @@ -550,6 +555,7 @@ void testPartition() throws Exception { new TableDescriptor.TableDistribution(16, Collections.singletonList("a")), Collections.emptyMap(), Collections.emptyMap(), + remoteDataDir, currentMillis, currentMillis); zookeeperClient.registerTable(tablePath, tableReg); @@ -571,14 +577,14 @@ void testPartition() throws Exception { }) .getBucketAssignments()); zookeeperClient.registerPartitionAssignmentAndMetadata( - 1L, "p1", partitionAssignment, tablePath, tableId); + 1L, "p1", partitionAssignment, remoteDataDir, tablePath, tableId); zookeeperClient.registerPartitionAssignmentAndMetadata( - 2L, "p2", partitionAssignment, tablePath, tableId); + 2L, "p2", partitionAssignment, remoteDataDir, tablePath, tableId); // check created partitions partitions = zookeeperClient.getPartitions(tablePath); assertThat(partitions).containsExactly("p1", "p2"); - TablePartition partition = zookeeperClient.getPartition(tablePath, "p1").get(); + PartitionRegistration partition = zookeeperClient.getPartition(tablePath, "p1").get(); assertThat(partition.getPartitionId()).isEqualTo(1L); partition = zookeeperClient.getPartition(tablePath, "p2").get(); assertThat(partition.getPartitionId()).isEqualTo(2L); @@ -677,6 +683,7 @@ void testZookeeperConfigPath() throws Exception { config.setString( ConfigOptions.ZOOKEEPER_ADDRESS, ZOO_KEEPER_EXTENSION_WRAPPER.getCustomExtension().getConnectString()); + config.set(ConfigOptions.REMOTE_DATA_DIR, remoteDataDir.toString()); config.setString(ConfigOptions.ZOOKEEPER_CONFIG_PATH, "./no-file.properties"); assertThatThrownBy( () -> ZooKeeperUtils.startZookeeperClient(config, NOPErrorHandler.INSTANCE)) @@ -720,6 +727,7 @@ void testGetDatabaseSummary() throws Exception { new TableDescriptor.TableDistribution(16, Collections.singletonList("a")), Collections.emptyMap(), Collections.emptyMap(), + remoteDataDir, beforeCreateTime, beforeCreateTime); zookeeperClient.registerTable(tablePath, tableReg1); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperExtension.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperExtension.java index 18516960ae..41bb965d73 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperExtension.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperExtension.java @@ -17,6 +17,9 @@ package org.apache.fluss.server.zk; +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.fs.local.LocalFileSystem; import org.apache.fluss.server.utils.FatalErrorHandler; import org.apache.fluss.shaded.zookeeper3.org.apache.zookeeper.KeeperException; import org.apache.fluss.testutils.common.CustomExtension; @@ -30,7 +33,9 @@ import javax.annotation.Nullable; +import java.io.File; import java.io.IOException; +import java.nio.file.Files; import static org.apache.fluss.utils.Preconditions.checkNotNull; import static org.apache.fluss.utils.Preconditions.checkState; @@ -44,6 +49,8 @@ public class ZooKeeperExtension implements CustomExtension { @Nullable private ZooKeeperClient zooKeeperClient; + private File tempDir; + @Override public void before(ExtensionContext context) throws Exception { close(); @@ -96,7 +103,27 @@ public ZooKeeperClient getZooKeeperClient(FatalErrorHandler fatalErrorHandler) { } public ZooKeeperClient createZooKeeperClient(FatalErrorHandler fatalErrorHandler) { - return ZooKeeperTestUtils.createZooKeeperClient(getConnectString(), fatalErrorHandler); + try { + tempDir = Files.createTempDirectory(null).toFile(); + Configuration conf = new Configuration(); + setRemoteDataDir(conf); + return ZooKeeperTestUtils.createZooKeeperClient( + conf, getConnectString(), fatalErrorHandler); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + private void setRemoteDataDir(Configuration conf) { + conf.set(ConfigOptions.REMOTE_DATA_DIR, getRemoteDataDir()); + } + + public String getRemoteDataDir() { + return LocalFileSystem.getLocalFsURI().getScheme() + + "://" + + tempDir.getAbsolutePath() + + File.separator + + "remote-data-dir"; } public void restart() throws Exception { diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperTestUtils.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperTestUtils.java index 9d287bd63d..d850b2194d 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperTestUtils.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperTestUtils.java @@ -43,8 +43,7 @@ public static TestingServer createAndStartZookeeperTestingServer() throws Except /** Create a {@link ZooKeeperClient} client using provided connect string. */ public static ZooKeeperClient createZooKeeperClient( - String connectString, FatalErrorHandler fatalErrorHandler) { - final Configuration config = new Configuration(); + Configuration config, String connectString, FatalErrorHandler fatalErrorHandler) { config.setString(ConfigOptions.ZOOKEEPER_ADDRESS, connectString); return ZooKeeperUtils.startZookeeperClient(config, fatalErrorHandler); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/PartitionRegistrationJsonSerdeTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/PartitionRegistrationJsonSerdeTest.java new file mode 100644 index 0000000000..26ba77ac62 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/PartitionRegistrationJsonSerdeTest.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.zk.data; + +import org.apache.fluss.utils.json.JsonSerdeTestBase; + +/** Test for {@link PartitionRegistrationJsonSerde}. */ +class PartitionRegistrationJsonSerdeTest extends JsonSerdeTestBase { + + PartitionRegistrationJsonSerdeTest() { + super(PartitionRegistrationJsonSerde.INSTANCE); + } + + @Override + protected PartitionRegistration[] createObjects() { + PartitionRegistration[] partitionRegistrations = new PartitionRegistration[1]; + + partitionRegistrations[0] = new PartitionRegistration(1234L, 5678L, "file://local/remote"); + + return partitionRegistrations; + } + + @Override + protected String[] expectedJsons() { + return new String[] { + "{\"version\":1,\"table_id\":1234,\"partition_id\":5678,\"remote_data_dir\":\"file://local/remote\"}" + }; + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/TableRegistrationJsonSerdeTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/TableRegistrationJsonSerdeTest.java index 348b20cfca..0d6a3f2fc0 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/TableRegistrationJsonSerdeTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/TableRegistrationJsonSerdeTest.java @@ -48,6 +48,7 @@ void testInvalidTableRegistration() { new TableDistribution(null, Arrays.asList("b", "c")), Maps.newHashMap(), Collections.singletonMap("custom-3", "\"300\""), + "file://local/remote", 1735538268L, 1735538268L)) .isInstanceOf(IllegalArgumentException.class) @@ -58,6 +59,7 @@ void testInvalidTableRegistration() { () -> TableRegistration.newTable( 11, + "file://local/remote", TableDescriptor.builder() .schema(TestData.DATA1_SCHEMA) .build())) @@ -77,6 +79,7 @@ protected TableRegistration[] createObjects() { new TableDistribution(16, Arrays.asList("b", "c")), Maps.newHashMap(), Collections.singletonMap("custom-3", "\"300\""), + "file://local/remote", 1735538268L, 1735538268L); @@ -88,6 +91,7 @@ protected TableRegistration[] createObjects() { new TableDistribution(32, Collections.emptyList()), Collections.singletonMap("option-3", "300"), Maps.newHashMap(), + "file://local/remote", -1, -1); @@ -98,8 +102,8 @@ protected TableRegistration[] createObjects() { protected String[] expectedJsons() { return new String[] { "{\"version\":1,\"table_id\":1234,\"comment\":\"first-table\",\"partition_key\":[\"a\",\"b\"]," - + "\"bucket_key\":[\"b\",\"c\"],\"bucket_count\":16,\"properties\":{},\"custom_properties\":{\"custom-3\":\"\\\"300\\\"\"},\"created_time\":1735538268,\"modified_time\":1735538268}", - "{\"version\":1,\"table_id\":1234,\"comment\":\"second-table\",\"bucket_count\":32,\"properties\":{\"option-3\":\"300\"},\"custom_properties\":{},\"created_time\":-1,\"modified_time\":-1}", + + "\"bucket_key\":[\"b\",\"c\"],\"bucket_count\":16,\"properties\":{},\"custom_properties\":{\"custom-3\":\"\\\"300\\\"\"},\"remote_data_dir\":\"file://local/remote\",\"created_time\":1735538268,\"modified_time\":1735538268}", + "{\"version\":1,\"table_id\":1234,\"comment\":\"second-table\",\"bucket_count\":32,\"properties\":{\"option-3\":\"300\"},\"custom_properties\":{},\"remote_data_dir\":\"file://local/remote\",\"created_time\":-1,\"modified_time\":-1}", }; } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableHelperTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableHelperTest.java index 1f759d3fac..731f89a480 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableHelperTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/lake/LakeTableHelperTest.java @@ -59,6 +59,7 @@ class LakeTableHelperTest { new AllCallbackWrapper<>(new ZooKeeperExtension()); private static ZooKeeperClient zookeeperClient; + private static String remoteDataDir; @BeforeAll static void beforeAll() { @@ -66,6 +67,7 @@ static void beforeAll() { ZOO_KEEPER_EXTENSION_WRAPPER .getCustomExtension() .getZooKeeperClient(NOPErrorHandler.INSTANCE); + remoteDataDir = zookeeperClient.getDefaultRemoteDataDir(); } @AfterEach @@ -267,6 +269,7 @@ private TableRegistration createTableReg(long tableId) { new TableDescriptor.TableDistribution(1, Collections.singletonList("a")), Collections.emptyMap(), Collections.emptyMap(), + remoteDataDir, System.currentTimeMillis(), System.currentTimeMillis()); }