This is an automated email from the ASF dual-hosted git repository.
lzljs3620320 pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/paimon.git
The following commit(s) were added to refs/heads/master by this push:
new b5ee79a703 [core] RESTCatalog: support dlf token loader (#5285)
b5ee79a703 is described below
commit b5ee79a70301666f54485f97951d8db94d258eb3
Author: shyjsarah <[email protected]>
AuthorDate: Fri Mar 14 14:27:35 2025 +0800
[core] RESTCatalog: support dlf token loader (#5285)
---
.../org/apache/paimon/rest/RESTCatalogOptions.java | 6 ++
.../apache/paimon/rest/auth/DLFAuthProvider.java | 42 +++----------
.../paimon/rest/auth/DLFAuthProviderFactory.java | 14 +++--
.../paimon/rest/auth/DLFLocalFileTokenLoader.java | 64 +++++++++++++++++++
.../rest/auth/DLFLocalFileTokenLoaderFactory.java | 38 ++++++++++++
.../apache/paimon/rest/auth/DLFTokenLoader.java | 25 ++++++++
.../paimon/rest/auth/DLFTokenLoaderFactory.java | 38 ++++++++++++
.../services/org.apache.paimon.factories.Factory | 1 +
.../apache/paimon/rest/MockRESTCatalogTest.java | 10 ++-
.../apache/paimon/rest/auth/AuthSessionTest.java | 72 ++++++++++++++++++++++
.../paimon/rest/auth/CustomTestDLFTokenLoader.java | 42 +++++++++++++
.../rest/auth/CustomTestDLFTokenLoaderFactory.java | 42 +++++++++++++
.../services/org.apache.paimon.factories.Factory | 3 +-
13 files changed, 356 insertions(+), 41 deletions(-)
diff --git
a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java
b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java
index 0dd5bc2a00..3980108e7e 100644
--- a/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java
+++ b/paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java
@@ -79,4 +79,10 @@ public class RESTCatalogOptions {
.stringType()
.noDefaultValue()
.withDescription("REST Catalog auth DLF security token");
+
+ public static final ConfigOption<String> DLF_TOKEN_LOADER =
+ ConfigOptions.key("dlf.token-loader")
+ .stringType()
+ .noDefaultValue()
+ .withDescription("REST Catalog auth DLF token loader.");
}
diff --git
a/paimon-core/src/main/java/org/apache/paimon/rest/auth/DLFAuthProvider.java
b/paimon-core/src/main/java/org/apache/paimon/rest/auth/DLFAuthProvider.java
index 8ab8c59ca9..0f5371228d 100644
--- a/paimon-core/src/main/java/org/apache/paimon/rest/auth/DLFAuthProvider.java
+++ b/paimon-core/src/main/java/org/apache/paimon/rest/auth/DLFAuthProvider.java
@@ -18,14 +18,8 @@
package org.apache.paimon.rest.auth;
-import org.apache.paimon.utils.FileIOUtils;
-
import okhttp3.MediaType;
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.UncheckedIOException;
import java.time.LocalDateTime;
import java.time.ZoneOffset;
import java.time.ZonedDateTime;
@@ -34,8 +28,6 @@ import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
-import static org.apache.paimon.rest.RESTObjectMapper.OBJECT_MAPPER;
-
/** Auth provider for <b>Ali CLoud</b> DLF. */
public class DLFAuthProvider implements AuthProvider {
@@ -55,9 +47,8 @@ public class DLFAuthProvider implements AuthProvider {
public static final DateTimeFormatter AUTH_DATE_FORMATTER =
DateTimeFormatter.ofPattern("yyyyMMdd");
protected static final MediaType MEDIA_TYPE =
MediaType.parse("application/json");
- private static final long[] READ_TOKEN_FILE_BACKOFF_WAIT_TIME_MILLIS =
{1_000, 3_000, 5_000};
- private final String tokenFilePath;
+ private final DLFTokenLoader tokenLoader;
protected DLFToken token;
private final boolean keepRefreshed;
@@ -66,11 +57,11 @@ public class DLFAuthProvider implements AuthProvider {
private final String region;
public static DLFAuthProvider buildRefreshToken(
- String tokenFilePath, Long tokenRefreshInMills, String region) {
- DLFToken token = readToken(tokenFilePath, 0);
+ DLFTokenLoader tokenLoader, Long tokenRefreshInMills, String
region) {
+ DLFToken token = tokenLoader.loadToken();
Long expiresAtMillis = getExpirationInMills(token.getExpiration());
return new DLFAuthProvider(
- tokenFilePath, token, true, expiresAtMillis,
tokenRefreshInMills, region);
+ tokenLoader, token, true, expiresAtMillis,
tokenRefreshInMills, region);
}
public static DLFAuthProvider buildAKToken(
@@ -80,13 +71,13 @@ public class DLFAuthProvider implements AuthProvider {
}
public DLFAuthProvider(
- String tokenFilePath,
+ DLFTokenLoader tokenLoader,
DLFToken token,
boolean keepRefreshed,
Long expiresAtMillis,
Long tokenRefreshInMills,
String region) {
- this.tokenFilePath = tokenFilePath;
+ this.tokenLoader = tokenLoader;
this.token = token;
this.keepRefreshed = keepRefreshed;
this.expiresAtMillis = expiresAtMillis;
@@ -135,7 +126,7 @@ public class DLFAuthProvider implements AuthProvider {
@Override
public boolean refresh() {
long start = System.currentTimeMillis();
- DLFToken newToken = readToken(tokenFilePath, 0);
+ DLFToken newToken = tokenLoader.loadToken();
if (newToken == null) {
return false;
}
@@ -169,25 +160,6 @@ public class DLFAuthProvider implements AuthProvider {
return Optional.ofNullable(this.tokenRefreshInMills);
}
- protected static DLFToken readToken(String tokenFilePath, int retryTimes) {
- try {
- File tokenFile = new File(tokenFilePath);
- if (tokenFile.exists()) {
- String tokenStr = FileIOUtils.readFileUtf8(tokenFile);
- return OBJECT_MAPPER.readValue(tokenStr, DLFToken.class);
- } else if (retryTimes <
READ_TOKEN_FILE_BACKOFF_WAIT_TIME_MILLIS.length - 1) {
-
Thread.sleep(READ_TOKEN_FILE_BACKOFF_WAIT_TIME_MILLIS[retryTimes]);
- return readToken(tokenFilePath, retryTimes + 1);
- } else {
- throw new FileNotFoundException(tokenFilePath);
- }
- } catch (IOException e) {
- throw new UncheckedIOException(e);
- } catch (InterruptedException e) {
- throw new RuntimeException(e);
- }
- }
-
private static Long getExpirationInMills(String dateStr) {
try {
if (dateStr == null) {
diff --git
a/paimon-core/src/main/java/org/apache/paimon/rest/auth/DLFAuthProviderFactory.java
b/paimon-core/src/main/java/org/apache/paimon/rest/auth/DLFAuthProviderFactory.java
index d81f37bb91..1c8474fb13 100644
---
a/paimon-core/src/main/java/org/apache/paimon/rest/auth/DLFAuthProviderFactory.java
+++
b/paimon-core/src/main/java/org/apache/paimon/rest/auth/DLFAuthProviderFactory.java
@@ -24,7 +24,6 @@ import org.apache.paimon.rest.RESTCatalogOptions;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
-import static org.apache.paimon.rest.RESTCatalogOptions.DLF_TOKEN_PATH;
import static org.apache.paimon.rest.RESTCatalogOptions.TOKEN_REFRESH_TIME;
import static org.apache.paimon.rest.RESTCatalogOptions.URI;
@@ -41,10 +40,17 @@ public class DLFAuthProviderFactory implements
AuthProviderFactory {
String region =
options.getOptional(RESTCatalogOptions.DLF_REGION)
.orElseGet(() -> parseRegionFromUri(options.get(URI)));
- if
(options.getOptional(RESTCatalogOptions.DLF_TOKEN_PATH).isPresent()) {
- String tokenFilePath = options.get(DLF_TOKEN_PATH);
+ if
(options.getOptional(RESTCatalogOptions.DLF_TOKEN_LOADER).isPresent()) {
+ DLFTokenLoader dlfTokenLoader =
+ DLFTokenLoaderFactory.createDLFTokenLoader(
+ options.get(RESTCatalogOptions.DLF_TOKEN_LOADER),
options);
long tokenRefreshInMills =
options.get(TOKEN_REFRESH_TIME).toMillis();
- return DLFAuthProvider.buildRefreshToken(tokenFilePath,
tokenRefreshInMills, region);
+ return DLFAuthProvider.buildRefreshToken(dlfTokenLoader,
tokenRefreshInMills, region);
+ } else if
(options.getOptional(RESTCatalogOptions.DLF_TOKEN_PATH).isPresent()) {
+ DLFTokenLoader dlfTokenLoader =
+ DLFTokenLoaderFactory.createDLFTokenLoader("local_file",
options);
+ long tokenRefreshInMills =
options.get(TOKEN_REFRESH_TIME).toMillis();
+ return DLFAuthProvider.buildRefreshToken(dlfTokenLoader,
tokenRefreshInMills, region);
} else if
(options.getOptional(RESTCatalogOptions.DLF_ACCESS_KEY_ID).isPresent()
&&
options.getOptional(RESTCatalogOptions.DLF_ACCESS_KEY_SECRET).isPresent()) {
return DLFAuthProvider.buildAKToken(
diff --git
a/paimon-core/src/main/java/org/apache/paimon/rest/auth/DLFLocalFileTokenLoader.java
b/paimon-core/src/main/java/org/apache/paimon/rest/auth/DLFLocalFileTokenLoader.java
new file mode 100644
index 0000000000..c1aea84280
--- /dev/null
+++
b/paimon-core/src/main/java/org/apache/paimon/rest/auth/DLFLocalFileTokenLoader.java
@@ -0,0 +1,64 @@
+/*
+ * 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.paimon.rest.auth;
+
+import org.apache.paimon.utils.FileIOUtils;
+
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+
+import static org.apache.paimon.rest.RESTObjectMapper.OBJECT_MAPPER;
+
+/** DLF Token Loader for local file. */
+public class DLFLocalFileTokenLoader implements DLFTokenLoader {
+
+ private static final long[] READ_TOKEN_FILE_BACKOFF_WAIT_TIME_MILLIS =
{1_000, 3_000, 5_000};
+
+ private final String tokenFilePath;
+
+ public DLFLocalFileTokenLoader(String tokenFilePath) {
+ this.tokenFilePath = tokenFilePath;
+ }
+
+ @Override
+ public DLFToken loadToken() {
+ return readToken(tokenFilePath, 0);
+ }
+
+ protected static DLFToken readToken(String tokenFilePath, int retryTimes) {
+ try {
+ File tokenFile = new File(tokenFilePath);
+ if (tokenFile.exists()) {
+ String tokenStr = FileIOUtils.readFileUtf8(tokenFile);
+ return OBJECT_MAPPER.readValue(tokenStr, DLFToken.class);
+ } else if (retryTimes <
READ_TOKEN_FILE_BACKOFF_WAIT_TIME_MILLIS.length - 1) {
+
Thread.sleep(READ_TOKEN_FILE_BACKOFF_WAIT_TIME_MILLIS[retryTimes]);
+ return readToken(tokenFilePath, retryTimes + 1);
+ } else {
+ throw new FileNotFoundException(tokenFilePath);
+ }
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
+ } catch (InterruptedException e) {
+ throw new RuntimeException(e);
+ }
+ }
+}
diff --git
a/paimon-core/src/main/java/org/apache/paimon/rest/auth/DLFLocalFileTokenLoaderFactory.java
b/paimon-core/src/main/java/org/apache/paimon/rest/auth/DLFLocalFileTokenLoaderFactory.java
new file mode 100644
index 0000000000..88fba78ad4
--- /dev/null
+++
b/paimon-core/src/main/java/org/apache/paimon/rest/auth/DLFLocalFileTokenLoaderFactory.java
@@ -0,0 +1,38 @@
+/*
+ * 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.paimon.rest.auth;
+
+import org.apache.paimon.options.Options;
+
+import static org.apache.paimon.rest.RESTCatalogOptions.DLF_TOKEN_PATH;
+
+/** Factory for {@link DLFLocalFileTokenLoader}. */
+public class DLFLocalFileTokenLoaderFactory implements DLFTokenLoaderFactory {
+
+ @Override
+ public String identifier() {
+ return "local_file";
+ }
+
+ @Override
+ public DLFTokenLoader create(Options options) {
+ String tokenFilePath = options.get(DLF_TOKEN_PATH);
+ return new DLFLocalFileTokenLoader(tokenFilePath);
+ }
+}
diff --git
a/paimon-core/src/main/java/org/apache/paimon/rest/auth/DLFTokenLoader.java
b/paimon-core/src/main/java/org/apache/paimon/rest/auth/DLFTokenLoader.java
new file mode 100644
index 0000000000..fb2199e320
--- /dev/null
+++ b/paimon-core/src/main/java/org/apache/paimon/rest/auth/DLFTokenLoader.java
@@ -0,0 +1,25 @@
+/*
+ * 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.paimon.rest.auth;
+
+/** DLF Token Loader. */
+public interface DLFTokenLoader {
+
+ DLFToken loadToken();
+}
diff --git
a/paimon-core/src/main/java/org/apache/paimon/rest/auth/DLFTokenLoaderFactory.java
b/paimon-core/src/main/java/org/apache/paimon/rest/auth/DLFTokenLoaderFactory.java
new file mode 100644
index 0000000000..82108a65ef
--- /dev/null
+++
b/paimon-core/src/main/java/org/apache/paimon/rest/auth/DLFTokenLoaderFactory.java
@@ -0,0 +1,38 @@
+/*
+ * 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.paimon.rest.auth;
+
+import org.apache.paimon.factories.Factory;
+import org.apache.paimon.factories.FactoryUtil;
+import org.apache.paimon.options.Options;
+
+/** Factory for {@link DLFTokenLoader}. */
+public interface DLFTokenLoaderFactory extends Factory {
+
+ DLFTokenLoader create(Options options);
+
+ static DLFTokenLoader createDLFTokenLoader(String name, Options options) {
+ DLFTokenLoaderFactory factory =
+ FactoryUtil.discoverFactory(
+ DLFTokenLoaderFactory.class.getClassLoader(),
+ DLFTokenLoaderFactory.class,
+ name);
+ return factory.create(options);
+ }
+}
diff --git
a/paimon-core/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory
b/paimon-core/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory
index 81fc475d76..3789e93908 100644
---
a/paimon-core/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory
+++
b/paimon-core/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory
@@ -40,3 +40,4 @@ org.apache.paimon.rest.RESTCatalogFactory
org.apache.paimon.iceberg.migrate.IcebergMigrateHadoopMetadataFactory
org.apache.paimon.rest.auth.BearTokenAuthProviderFactory
org.apache.paimon.rest.auth.DLFAuthProviderFactory
+org.apache.paimon.rest.auth.DLFLocalFileTokenLoaderFactory
diff --git
a/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTCatalogTest.java
b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTCatalogTest.java
index ca23860488..e9370760f8 100644
--- a/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTCatalogTest.java
+++ b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTCatalogTest.java
@@ -29,6 +29,8 @@ import org.apache.paimon.rest.auth.AuthProvider;
import org.apache.paimon.rest.auth.AuthProviderEnum;
import org.apache.paimon.rest.auth.BearTokenAuthProvider;
import org.apache.paimon.rest.auth.DLFAuthProvider;
+import org.apache.paimon.rest.auth.DLFTokenLoader;
+import org.apache.paimon.rest.auth.DLFTokenLoaderFactory;
import org.apache.paimon.rest.auth.RESTAuthParameter;
import org.apache.paimon.rest.exceptions.NotAuthorizedException;
import org.apache.paimon.rest.responses.ConfigResponse;
@@ -130,8 +132,14 @@ class MockRESTCatalogTest extends RESTCatalogTestBase {
String region = "cn-hangzhou";
String tokenPath = dataPath + UUID.randomUUID();
generateTokenAndWriteToFile(tokenPath);
+ DLFTokenLoader tokenLoader =
+ DLFTokenLoaderFactory.createDLFTokenLoader(
+ "local_file",
+ new Options(
+ ImmutableMap.of(
+
RESTCatalogOptions.DLF_TOKEN_PATH.key(), tokenPath)));
DLFAuthProvider authProvider =
- DLFAuthProvider.buildRefreshToken(tokenPath, 1000_000L,
region);
+ DLFAuthProvider.buildRefreshToken(tokenLoader, 1000_000L,
region);
restCatalogServer =
new RESTCatalogServer(dataPath, authProvider, this.config,
restWarehouse);
restCatalogServer.start();
diff --git
a/paimon-core/src/test/java/org/apache/paimon/rest/auth/AuthSessionTest.java
b/paimon-core/src/test/java/org/apache/paimon/rest/auth/AuthSessionTest.java
index 54b88e74e4..b0ab1fd8dd 100644
--- a/paimon-core/src/test/java/org/apache/paimon/rest/auth/AuthSessionTest.java
+++ b/paimon-core/src/test/java/org/apache/paimon/rest/auth/AuthSessionTest.java
@@ -26,6 +26,7 @@ import org.apache.paimon.utils.ThreadPoolUtils;
import
org.apache.paimon.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.commons.io.FileUtils;
+import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
@@ -45,6 +46,7 @@ import static
org.apache.paimon.rest.RESTCatalogOptions.DLF_ACCESS_KEY_ID;
import static org.apache.paimon.rest.RESTCatalogOptions.DLF_ACCESS_KEY_SECRET;
import static org.apache.paimon.rest.RESTCatalogOptions.DLF_REGION;
import static org.apache.paimon.rest.RESTCatalogOptions.DLF_SECURITY_TOKEN;
+import static org.apache.paimon.rest.RESTCatalogOptions.DLF_TOKEN_LOADER;
import static org.apache.paimon.rest.RESTCatalogOptions.DLF_TOKEN_PATH;
import static org.apache.paimon.rest.RESTCatalogOptions.TOKEN;
import static org.apache.paimon.rest.RESTCatalogOptions.TOKEN_REFRESH_TIME;
@@ -228,6 +230,67 @@ public class AuthSessionTest {
AuthProviderEnum.DLF.identifier(), new
Options()));
}
+ @Test
+ public void testCreateDlfAuthProviderByDLFTokenLoader()
+ throws IOException, InterruptedException {
+ String fileName = UUID.randomUUID().toString();
+ Pair<File, String> tokenFile2Token =
generateTokenAndWriteToFile(fileName);
+ String theFirstGenerateToken = tokenFile2Token.getRight();
+ File tokenFile = tokenFile2Token.getLeft();
+ long tokenRefreshInMills = 1000;
+ // create options with token loader
+ Options options = new Options();
+ options.set(DLF_TOKEN_LOADER.key(), "local_file");
+ options.set(DLF_TOKEN_PATH.key(), folder.getRoot().getPath() + "/" +
fileName);
+ options.set(RESTCatalogOptions.URI.key(), "serverUrl");
+ options.set(DLF_REGION.key(), "cn-hangzhou");
+ options.set(TOKEN_REFRESH_TIME.key(), tokenRefreshInMills + "ms");
+ AuthProvider authProvider =
+
AuthProviderFactory.createAuthProvider(AuthProviderEnum.DLF.identifier(),
options);
+ ScheduledExecutorService executor =
+ ThreadPoolUtils.createScheduledThreadPool(1, "refresh-token");
+ AuthSession session = AuthSession.fromRefreshAuthProvider(executor,
authProvider);
+ DLFAuthProvider dlfAuthProvider = (DLFAuthProvider)
session.getAuthProvider();
+ String theFirstFetchToken =
+
OBJECT_MAPPER_INSTANCE.writeValueAsString(dlfAuthProvider.token);
+ assertEquals(theFirstFetchToken, theFirstGenerateToken);
+ tokenFile.delete();
+ tokenFile2Token = generateTokenAndWriteToFile(fileName);
+ String theSecondGenerateToken = tokenFile2Token.getRight();
+ Thread.sleep(tokenRefreshInMills * 2);
+ String theSecondFetchToken =
+
OBJECT_MAPPER_INSTANCE.writeValueAsString(dlfAuthProvider.token);
+ // if the second fetch token is not equal to the first fetch token, it
means refresh success
+ // as refresh maybe fail in test environment, so we need to check
whether refresh success
+ if (!theSecondFetchToken.equals(theFirstFetchToken)) {
+ assertEquals(theSecondGenerateToken, theSecondFetchToken);
+ }
+ }
+
+ @Test
+ public void testCreateDlfAuthProviderByCustomDLFTokenLoader()
+ throws IOException, InterruptedException {
+ DLFToken customToken = generateToken();
+ // create options with custom token loader
+ Options options = new Options();
+ options.set(DLF_TOKEN_LOADER.key(), "custom");
+ options.set(DLF_ACCESS_KEY_ID.key(), customToken.getAccessKeyId());
+ options.set(DLF_ACCESS_KEY_SECRET.key(),
customToken.getAccessKeySecret());
+ options.set(DLF_SECURITY_TOKEN.key(), customToken.getSecurityToken());
+ options.set(RESTCatalogOptions.URI.key(), "serverUrl");
+ options.set(DLF_REGION.key(), "cn-hangzhou");
+ AuthProvider authProvider =
+
AuthProviderFactory.createAuthProvider(AuthProviderEnum.DLF.identifier(),
options);
+ ScheduledExecutorService executor =
+ ThreadPoolUtils.createScheduledThreadPool(1, "refresh-token");
+ AuthSession session = AuthSession.fromRefreshAuthProvider(executor,
authProvider);
+ DLFAuthProvider dlfAuthProvider = (DLFAuthProvider)
session.getAuthProvider();
+ DLFToken fetchToken = dlfAuthProvider.token;
+ Assert.assertEquals(fetchToken.getAccessKeyId(),
customToken.getAccessKeyId());
+ Assert.assertEquals(fetchToken.getAccessKeySecret(),
customToken.getAccessKeySecret());
+ Assert.assertEquals(fetchToken.getSecurityToken(),
customToken.getSecurityToken());
+ }
+
@Test
public void testDLFAuthProviderAuthHeaderWhenDataIsNotEmpty() throws
Exception {
String fileName = UUID.randomUUID().toString();
@@ -283,6 +346,15 @@ public class AuthSessionTest {
return Pair.of(tokenFile, tokenStr);
}
+ private DLFToken generateToken() {
+ String accessKeyId = UUID.randomUUID().toString();
+ String accessKeySecret = UUID.randomUUID().toString();
+ String securityToken = UUID.randomUUID().toString();
+ ZonedDateTime now = ZonedDateTime.now(ZoneOffset.UTC);
+ String expiration = now.format(TOKEN_DATE_FORMATTER);
+ return new DLFToken(accessKeyId, accessKeySecret, securityToken,
expiration);
+ }
+
private AuthProvider generateDLFAuthProvider(
Optional<Long> tokenRefreshInMillsOpt, String fileName, String
serverUrl) {
Options options = new Options();
diff --git
a/paimon-core/src/test/java/org/apache/paimon/rest/auth/CustomTestDLFTokenLoader.java
b/paimon-core/src/test/java/org/apache/paimon/rest/auth/CustomTestDLFTokenLoader.java
new file mode 100644
index 0000000000..279c70080b
--- /dev/null
+++
b/paimon-core/src/test/java/org/apache/paimon/rest/auth/CustomTestDLFTokenLoader.java
@@ -0,0 +1,42 @@
+/*
+ * 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.paimon.rest.auth;
+
+import java.time.ZoneOffset;
+import java.time.ZonedDateTime;
+
+import static org.apache.paimon.rest.auth.DLFAuthProvider.TOKEN_DATE_FORMATTER;
+
+/** DLF Token Loader for custom. */
+public class CustomTestDLFTokenLoader implements DLFTokenLoader {
+
+ DLFToken token;
+
+ public CustomTestDLFTokenLoader(
+ String accessKeyId, String accessKeySecret, String securityToken) {
+ ZonedDateTime now = ZonedDateTime.now(ZoneOffset.UTC);
+ String expiration = now.format(TOKEN_DATE_FORMATTER);
+ this.token = new DLFToken(accessKeyId, accessKeySecret, securityToken,
expiration);
+ }
+
+ @Override
+ public DLFToken loadToken() {
+ return token;
+ }
+}
diff --git
a/paimon-core/src/test/java/org/apache/paimon/rest/auth/CustomTestDLFTokenLoaderFactory.java
b/paimon-core/src/test/java/org/apache/paimon/rest/auth/CustomTestDLFTokenLoaderFactory.java
new file mode 100644
index 0000000000..e3f5359e0f
--- /dev/null
+++
b/paimon-core/src/test/java/org/apache/paimon/rest/auth/CustomTestDLFTokenLoaderFactory.java
@@ -0,0 +1,42 @@
+/*
+ * 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.paimon.rest.auth;
+
+import org.apache.paimon.options.Options;
+
+import static org.apache.paimon.rest.RESTCatalogOptions.DLF_ACCESS_KEY_ID;
+import static org.apache.paimon.rest.RESTCatalogOptions.DLF_ACCESS_KEY_SECRET;
+import static org.apache.paimon.rest.RESTCatalogOptions.DLF_SECURITY_TOKEN;
+
+/** Factory for {@link CustomTestDLFTokenLoader}. */
+public class CustomTestDLFTokenLoaderFactory implements DLFTokenLoaderFactory {
+
+ @Override
+ public String identifier() {
+ return "custom";
+ }
+
+ @Override
+ public DLFTokenLoader create(Options options) {
+ String accessKeyId = options.get(DLF_ACCESS_KEY_ID);
+ String accessKeySecret = options.get(DLF_ACCESS_KEY_SECRET);
+ String securityToken = options.get(DLF_SECURITY_TOKEN);
+ return new CustomTestDLFTokenLoader(accessKeyId, accessKeySecret,
securityToken);
+ }
+}
diff --git
a/paimon-core/src/test/resources/META-INF/services/org.apache.paimon.factories.Factory
b/paimon-core/src/test/resources/META-INF/services/org.apache.paimon.factories.Factory
index 7eb517ab98..b44cf4490a 100644
---
a/paimon-core/src/test/resources/META-INF/services/org.apache.paimon.factories.Factory
+++
b/paimon-core/src/test/resources/META-INF/services/org.apache.paimon.factories.Factory
@@ -13,4 +13,5 @@
# See the License for the specific language governing permissions and
# limitations under the License.
-org.apache.paimon.mergetree.compact.aggregate.TestCustomAggFactory
\ No newline at end of file
+org.apache.paimon.mergetree.compact.aggregate.TestCustomAggFactory
+org.apache.paimon.rest.auth.CustomTestDLFTokenLoaderFactory
\ No newline at end of file