Skip to content

Commit b5ee79a

Browse files
authored
[core] RESTCatalog: support dlf token loader (apache#5285)
1 parent e6ab8eb commit b5ee79a

File tree

13 files changed

+356
-41
lines changed

13 files changed

+356
-41
lines changed

paimon-core/src/main/java/org/apache/paimon/rest/RESTCatalogOptions.java

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -79,4 +79,10 @@ public class RESTCatalogOptions {
7979
.stringType()
8080
.noDefaultValue()
8181
.withDescription("REST Catalog auth DLF security token");
82+
83+
public static final ConfigOption<String> DLF_TOKEN_LOADER =
84+
ConfigOptions.key("dlf.token-loader")
85+
.stringType()
86+
.noDefaultValue()
87+
.withDescription("REST Catalog auth DLF token loader.");
8288
}

paimon-core/src/main/java/org/apache/paimon/rest/auth/DLFAuthProvider.java

Lines changed: 7 additions & 35 deletions
Original file line numberDiff line numberDiff line change
@@ -18,14 +18,8 @@
1818

1919
package org.apache.paimon.rest.auth;
2020

21-
import org.apache.paimon.utils.FileIOUtils;
22-
2321
import okhttp3.MediaType;
2422

25-
import java.io.File;
26-
import java.io.FileNotFoundException;
27-
import java.io.IOException;
28-
import java.io.UncheckedIOException;
2923
import java.time.LocalDateTime;
3024
import java.time.ZoneOffset;
3125
import java.time.ZonedDateTime;
@@ -34,8 +28,6 @@
3428
import java.util.Map;
3529
import java.util.Optional;
3630

37-
import static org.apache.paimon.rest.RESTObjectMapper.OBJECT_MAPPER;
38-
3931
/** Auth provider for <b>Ali CLoud</b> DLF. */
4032
public class DLFAuthProvider implements AuthProvider {
4133

@@ -55,9 +47,8 @@ public class DLFAuthProvider implements AuthProvider {
5547
public static final DateTimeFormatter AUTH_DATE_FORMATTER =
5648
DateTimeFormatter.ofPattern("yyyyMMdd");
5749
protected static final MediaType MEDIA_TYPE = MediaType.parse("application/json");
58-
private static final long[] READ_TOKEN_FILE_BACKOFF_WAIT_TIME_MILLIS = {1_000, 3_000, 5_000};
5950

60-
private final String tokenFilePath;
51+
private final DLFTokenLoader tokenLoader;
6152

6253
protected DLFToken token;
6354
private final boolean keepRefreshed;
@@ -66,11 +57,11 @@ public class DLFAuthProvider implements AuthProvider {
6657
private final String region;
6758

6859
public static DLFAuthProvider buildRefreshToken(
69-
String tokenFilePath, Long tokenRefreshInMills, String region) {
70-
DLFToken token = readToken(tokenFilePath, 0);
60+
DLFTokenLoader tokenLoader, Long tokenRefreshInMills, String region) {
61+
DLFToken token = tokenLoader.loadToken();
7162
Long expiresAtMillis = getExpirationInMills(token.getExpiration());
7263
return new DLFAuthProvider(
73-
tokenFilePath, token, true, expiresAtMillis, tokenRefreshInMills, region);
64+
tokenLoader, token, true, expiresAtMillis, tokenRefreshInMills, region);
7465
}
7566

7667
public static DLFAuthProvider buildAKToken(
@@ -80,13 +71,13 @@ public static DLFAuthProvider buildAKToken(
8071
}
8172

8273
public DLFAuthProvider(
83-
String tokenFilePath,
74+
DLFTokenLoader tokenLoader,
8475
DLFToken token,
8576
boolean keepRefreshed,
8677
Long expiresAtMillis,
8778
Long tokenRefreshInMills,
8879
String region) {
89-
this.tokenFilePath = tokenFilePath;
80+
this.tokenLoader = tokenLoader;
9081
this.token = token;
9182
this.keepRefreshed = keepRefreshed;
9283
this.expiresAtMillis = expiresAtMillis;
@@ -135,7 +126,7 @@ public static Map<String, String> generateSignHeaders(
135126
@Override
136127
public boolean refresh() {
137128
long start = System.currentTimeMillis();
138-
DLFToken newToken = readToken(tokenFilePath, 0);
129+
DLFToken newToken = tokenLoader.loadToken();
139130
if (newToken == null) {
140131
return false;
141132
}
@@ -169,25 +160,6 @@ public Optional<Long> tokenRefreshInMills() {
169160
return Optional.ofNullable(this.tokenRefreshInMills);
170161
}
171162

172-
protected static DLFToken readToken(String tokenFilePath, int retryTimes) {
173-
try {
174-
File tokenFile = new File(tokenFilePath);
175-
if (tokenFile.exists()) {
176-
String tokenStr = FileIOUtils.readFileUtf8(tokenFile);
177-
return OBJECT_MAPPER.readValue(tokenStr, DLFToken.class);
178-
} else if (retryTimes < READ_TOKEN_FILE_BACKOFF_WAIT_TIME_MILLIS.length - 1) {
179-
Thread.sleep(READ_TOKEN_FILE_BACKOFF_WAIT_TIME_MILLIS[retryTimes]);
180-
return readToken(tokenFilePath, retryTimes + 1);
181-
} else {
182-
throw new FileNotFoundException(tokenFilePath);
183-
}
184-
} catch (IOException e) {
185-
throw new UncheckedIOException(e);
186-
} catch (InterruptedException e) {
187-
throw new RuntimeException(e);
188-
}
189-
}
190-
191163
private static Long getExpirationInMills(String dateStr) {
192164
try {
193165
if (dateStr == null) {

paimon-core/src/main/java/org/apache/paimon/rest/auth/DLFAuthProviderFactory.java

Lines changed: 10 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,6 @@
2424
import java.util.regex.Matcher;
2525
import java.util.regex.Pattern;
2626

27-
import static org.apache.paimon.rest.RESTCatalogOptions.DLF_TOKEN_PATH;
2827
import static org.apache.paimon.rest.RESTCatalogOptions.TOKEN_REFRESH_TIME;
2928
import static org.apache.paimon.rest.RESTCatalogOptions.URI;
3029

@@ -41,10 +40,17 @@ public AuthProvider create(Options options) {
4140
String region =
4241
options.getOptional(RESTCatalogOptions.DLF_REGION)
4342
.orElseGet(() -> parseRegionFromUri(options.get(URI)));
44-
if (options.getOptional(RESTCatalogOptions.DLF_TOKEN_PATH).isPresent()) {
45-
String tokenFilePath = options.get(DLF_TOKEN_PATH);
43+
if (options.getOptional(RESTCatalogOptions.DLF_TOKEN_LOADER).isPresent()) {
44+
DLFTokenLoader dlfTokenLoader =
45+
DLFTokenLoaderFactory.createDLFTokenLoader(
46+
options.get(RESTCatalogOptions.DLF_TOKEN_LOADER), options);
4647
long tokenRefreshInMills = options.get(TOKEN_REFRESH_TIME).toMillis();
47-
return DLFAuthProvider.buildRefreshToken(tokenFilePath, tokenRefreshInMills, region);
48+
return DLFAuthProvider.buildRefreshToken(dlfTokenLoader, tokenRefreshInMills, region);
49+
} else if (options.getOptional(RESTCatalogOptions.DLF_TOKEN_PATH).isPresent()) {
50+
DLFTokenLoader dlfTokenLoader =
51+
DLFTokenLoaderFactory.createDLFTokenLoader("local_file", options);
52+
long tokenRefreshInMills = options.get(TOKEN_REFRESH_TIME).toMillis();
53+
return DLFAuthProvider.buildRefreshToken(dlfTokenLoader, tokenRefreshInMills, region);
4854
} else if (options.getOptional(RESTCatalogOptions.DLF_ACCESS_KEY_ID).isPresent()
4955
&& options.getOptional(RESTCatalogOptions.DLF_ACCESS_KEY_SECRET).isPresent()) {
5056
return DLFAuthProvider.buildAKToken(
Lines changed: 64 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,64 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
19+
package org.apache.paimon.rest.auth;
20+
21+
import org.apache.paimon.utils.FileIOUtils;
22+
23+
import java.io.File;
24+
import java.io.FileNotFoundException;
25+
import java.io.IOException;
26+
import java.io.UncheckedIOException;
27+
28+
import static org.apache.paimon.rest.RESTObjectMapper.OBJECT_MAPPER;
29+
30+
/** DLF Token Loader for local file. */
31+
public class DLFLocalFileTokenLoader implements DLFTokenLoader {
32+
33+
private static final long[] READ_TOKEN_FILE_BACKOFF_WAIT_TIME_MILLIS = {1_000, 3_000, 5_000};
34+
35+
private final String tokenFilePath;
36+
37+
public DLFLocalFileTokenLoader(String tokenFilePath) {
38+
this.tokenFilePath = tokenFilePath;
39+
}
40+
41+
@Override
42+
public DLFToken loadToken() {
43+
return readToken(tokenFilePath, 0);
44+
}
45+
46+
protected static DLFToken readToken(String tokenFilePath, int retryTimes) {
47+
try {
48+
File tokenFile = new File(tokenFilePath);
49+
if (tokenFile.exists()) {
50+
String tokenStr = FileIOUtils.readFileUtf8(tokenFile);
51+
return OBJECT_MAPPER.readValue(tokenStr, DLFToken.class);
52+
} else if (retryTimes < READ_TOKEN_FILE_BACKOFF_WAIT_TIME_MILLIS.length - 1) {
53+
Thread.sleep(READ_TOKEN_FILE_BACKOFF_WAIT_TIME_MILLIS[retryTimes]);
54+
return readToken(tokenFilePath, retryTimes + 1);
55+
} else {
56+
throw new FileNotFoundException(tokenFilePath);
57+
}
58+
} catch (IOException e) {
59+
throw new UncheckedIOException(e);
60+
} catch (InterruptedException e) {
61+
throw new RuntimeException(e);
62+
}
63+
}
64+
}
Lines changed: 38 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,38 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
19+
package org.apache.paimon.rest.auth;
20+
21+
import org.apache.paimon.options.Options;
22+
23+
import static org.apache.paimon.rest.RESTCatalogOptions.DLF_TOKEN_PATH;
24+
25+
/** Factory for {@link DLFLocalFileTokenLoader}. */
26+
public class DLFLocalFileTokenLoaderFactory implements DLFTokenLoaderFactory {
27+
28+
@Override
29+
public String identifier() {
30+
return "local_file";
31+
}
32+
33+
@Override
34+
public DLFTokenLoader create(Options options) {
35+
String tokenFilePath = options.get(DLF_TOKEN_PATH);
36+
return new DLFLocalFileTokenLoader(tokenFilePath);
37+
}
38+
}
Lines changed: 25 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,25 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
19+
package org.apache.paimon.rest.auth;
20+
21+
/** DLF Token Loader. */
22+
public interface DLFTokenLoader {
23+
24+
DLFToken loadToken();
25+
}
Lines changed: 38 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,38 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
19+
package org.apache.paimon.rest.auth;
20+
21+
import org.apache.paimon.factories.Factory;
22+
import org.apache.paimon.factories.FactoryUtil;
23+
import org.apache.paimon.options.Options;
24+
25+
/** Factory for {@link DLFTokenLoader}. */
26+
public interface DLFTokenLoaderFactory extends Factory {
27+
28+
DLFTokenLoader create(Options options);
29+
30+
static DLFTokenLoader createDLFTokenLoader(String name, Options options) {
31+
DLFTokenLoaderFactory factory =
32+
FactoryUtil.discoverFactory(
33+
DLFTokenLoaderFactory.class.getClassLoader(),
34+
DLFTokenLoaderFactory.class,
35+
name);
36+
return factory.create(options);
37+
}
38+
}

paimon-core/src/main/resources/META-INF/services/org.apache.paimon.factories.Factory

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -40,3 +40,4 @@ org.apache.paimon.rest.RESTCatalogFactory
4040
org.apache.paimon.iceberg.migrate.IcebergMigrateHadoopMetadataFactory
4141
org.apache.paimon.rest.auth.BearTokenAuthProviderFactory
4242
org.apache.paimon.rest.auth.DLFAuthProviderFactory
43+
org.apache.paimon.rest.auth.DLFLocalFileTokenLoaderFactory

paimon-core/src/test/java/org/apache/paimon/rest/MockRESTCatalogTest.java

Lines changed: 9 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -29,6 +29,8 @@
2929
import org.apache.paimon.rest.auth.AuthProviderEnum;
3030
import org.apache.paimon.rest.auth.BearTokenAuthProvider;
3131
import org.apache.paimon.rest.auth.DLFAuthProvider;
32+
import org.apache.paimon.rest.auth.DLFTokenLoader;
33+
import org.apache.paimon.rest.auth.DLFTokenLoaderFactory;
3234
import org.apache.paimon.rest.auth.RESTAuthParameter;
3335
import org.apache.paimon.rest.exceptions.NotAuthorizedException;
3436
import org.apache.paimon.rest.responses.ConfigResponse;
@@ -130,8 +132,14 @@ void testDlfStSTokenPathAuth() throws Exception {
130132
String region = "cn-hangzhou";
131133
String tokenPath = dataPath + UUID.randomUUID();
132134
generateTokenAndWriteToFile(tokenPath);
135+
DLFTokenLoader tokenLoader =
136+
DLFTokenLoaderFactory.createDLFTokenLoader(
137+
"local_file",
138+
new Options(
139+
ImmutableMap.of(
140+
RESTCatalogOptions.DLF_TOKEN_PATH.key(), tokenPath)));
133141
DLFAuthProvider authProvider =
134-
DLFAuthProvider.buildRefreshToken(tokenPath, 1000_000L, region);
142+
DLFAuthProvider.buildRefreshToken(tokenLoader, 1000_000L, region);
135143
restCatalogServer =
136144
new RESTCatalogServer(dataPath, authProvider, this.config, restWarehouse);
137145
restCatalogServer.start();

0 commit comments

Comments
 (0)