Skip to content

Commit 065bb04

Browse files
authored
[core] Introduce ResolvingFileIO (#4891)
1 parent 7715c60 commit 065bb04

File tree

6 files changed

+328
-2
lines changed

6 files changed

+328
-2
lines changed

docs/layouts/shortcodes/generated/catalog_configuration.html

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -140,5 +140,11 @@
140140
<td>String</td>
141141
<td>The warehouse root path of catalog.</td>
142142
</tr>
143+
<tr>
144+
<td><h5>resolving-fileio.enabled</h5></td>
145+
<td style="word-wrap: break-word;">false</td>
146+
<td>Boolean</td>
147+
<td>Whether to enable resolving fileio, when this option is enabled, in conjunction with the table's property data-file.external-paths, Paimon can read and write to external storage paths, such as OSS or S3. In order to access these external paths correctly, you also need to configure the corresponding access key and secret key.</td>
148+
</tr>
143149
</tbody>
144150
</table>

paimon-common/src/main/java/org/apache/paimon/fs/FileIO.java

Lines changed: 11 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@
2222
import org.apache.paimon.catalog.CatalogContext;
2323
import org.apache.paimon.fs.hadoop.HadoopFileIOLoader;
2424
import org.apache.paimon.fs.local.LocalFileIO;
25+
import org.apache.paimon.options.CatalogOptions;
2526

2627
import org.apache.commons.io.IOUtils;
2728
import org.slf4j.Logger;
@@ -419,6 +420,16 @@ default Optional<String> readOverwrittenFileUtf8(Path path) throws IOException {
419420
* by the given path.
420421
*/
421422
static FileIO get(Path path, CatalogContext config) throws IOException {
423+
boolean resolvingFileIOEnabled =
424+
config.options().get(CatalogOptions.RESOLVING_FILEIO_ENABLED);
425+
if (resolvingFileIOEnabled) {
426+
FileIO fileIO = new ResolvingFileIO();
427+
// set to false to avoid infinite loop
428+
config.options().set(CatalogOptions.RESOLVING_FILEIO_ENABLED, false);
429+
fileIO.configure(config);
430+
return fileIO;
431+
}
432+
422433
URI uri = path.toUri();
423434
if (LOG.isDebugEnabled()) {
424435
LOG.debug("Getting FileIO by scheme {}.", uri.getScheme());
Lines changed: 159 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,159 @@
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.fs;
20+
21+
import org.apache.paimon.annotation.VisibleForTesting;
22+
import org.apache.paimon.catalog.CatalogContext;
23+
import org.apache.paimon.options.CatalogOptions;
24+
25+
import java.io.IOException;
26+
import java.io.Serializable;
27+
import java.util.Map;
28+
import java.util.Objects;
29+
import java.util.concurrent.ConcurrentHashMap;
30+
31+
/**
32+
* An implementation of {@link FileIO} that supports multiple file system schemas. It dynamically
33+
* selects the appropriate {@link FileIO} based on the URI scheme of the given path.
34+
*/
35+
public class ResolvingFileIO implements FileIO {
36+
37+
private static final long serialVersionUID = 1L;
38+
39+
private final Map<CacheKey, FileIO> fileIOMap = new ConcurrentHashMap<>();
40+
41+
private CatalogContext context;
42+
43+
// TODO, how to decide the real fileio is object store or not?
44+
@Override
45+
public boolean isObjectStore() {
46+
String warehouse = context.options().get(CatalogOptions.WAREHOUSE);
47+
if (warehouse == null) {
48+
return false;
49+
}
50+
Path path = new Path(warehouse);
51+
String scheme = path.toUri().getScheme();
52+
return scheme != null
53+
&& !scheme.equalsIgnoreCase("file")
54+
&& !scheme.equalsIgnoreCase("hdfs");
55+
}
56+
57+
@Override
58+
public void configure(CatalogContext context) {
59+
this.context = context;
60+
}
61+
62+
@Override
63+
public SeekableInputStream newInputStream(Path path) throws IOException {
64+
return wrap(() -> fileIO(path).newInputStream(path));
65+
}
66+
67+
@Override
68+
public PositionOutputStream newOutputStream(Path path, boolean overwrite) throws IOException {
69+
return wrap(() -> fileIO(path).newOutputStream(path, overwrite));
70+
}
71+
72+
@Override
73+
public FileStatus getFileStatus(Path path) throws IOException {
74+
return wrap(() -> fileIO(path).getFileStatus(path));
75+
}
76+
77+
@Override
78+
public FileStatus[] listStatus(Path path) throws IOException {
79+
return wrap(() -> fileIO(path).listStatus(path));
80+
}
81+
82+
@Override
83+
public boolean exists(Path path) throws IOException {
84+
return wrap(() -> fileIO(path).exists(path));
85+
}
86+
87+
@Override
88+
public boolean delete(Path path, boolean recursive) throws IOException {
89+
return wrap(() -> fileIO(path).delete(path, recursive));
90+
}
91+
92+
@Override
93+
public boolean mkdirs(Path path) throws IOException {
94+
return wrap(() -> fileIO(path).mkdirs(path));
95+
}
96+
97+
@Override
98+
public boolean rename(Path src, Path dst) throws IOException {
99+
return wrap(() -> fileIO(src).rename(src, dst));
100+
}
101+
102+
@VisibleForTesting
103+
public FileIO fileIO(Path path) throws IOException {
104+
CacheKey cacheKey = new CacheKey(path.toUri().getScheme(), path.toUri().getAuthority());
105+
return fileIOMap.computeIfAbsent(
106+
cacheKey,
107+
k -> {
108+
try {
109+
return FileIO.get(path, context);
110+
} catch (IOException e) {
111+
throw new RuntimeException(e);
112+
}
113+
});
114+
}
115+
116+
private <T> T wrap(Func<T> func) throws IOException {
117+
ClassLoader cl = Thread.currentThread().getContextClassLoader();
118+
try {
119+
Thread.currentThread().setContextClassLoader(ResolvingFileIO.class.getClassLoader());
120+
return func.apply();
121+
} finally {
122+
Thread.currentThread().setContextClassLoader(cl);
123+
}
124+
}
125+
126+
/** Apply function with wrapping classloader. */
127+
@FunctionalInterface
128+
protected interface Func<T> {
129+
T apply() throws IOException;
130+
}
131+
132+
private static class CacheKey implements Serializable {
133+
private final String scheme;
134+
private final String authority;
135+
136+
private CacheKey(String scheme, String authority) {
137+
this.scheme = scheme;
138+
this.authority = authority;
139+
}
140+
141+
@Override
142+
public boolean equals(Object o) {
143+
if (this == o) {
144+
return true;
145+
}
146+
if (o == null || getClass() != o.getClass()) {
147+
return false;
148+
}
149+
CacheKey cacheKey = (CacheKey) o;
150+
return Objects.equals(scheme, cacheKey.scheme)
151+
&& Objects.equals(authority, cacheKey.authority);
152+
}
153+
154+
@Override
155+
public int hashCode() {
156+
return Objects.hash(scheme, authority);
157+
}
158+
}
159+
}

paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java

Lines changed: 9 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -149,4 +149,13 @@ public class CatalogOptions {
149149
"Whether to support format tables, format table corresponds to a regular csv, parquet or orc table, allowing read and write operations. "
150150
+ "However, during these processes, it does not connect to the metastore; hence, newly added partitions will not be reflected in"
151151
+ " the metastore and need to be manually added as separate partition operations.");
152+
153+
public static final ConfigOption<Boolean> RESOLVING_FILEIO_ENABLED =
154+
ConfigOptions.key("resolving-fileio.enabled")
155+
.booleanType()
156+
.defaultValue(false)
157+
.withDescription(
158+
"Whether to enable resolving fileio, when this option is enabled, in conjunction with the table's property data-file.external-paths, "
159+
+ "Paimon can read and write to external storage paths, such as OSS or S3. "
160+
+ "In order to access these external paths correctly, you also need to configure the corresponding access key and secret key.");
152161
}
Lines changed: 140 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,140 @@
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.fs;
20+
21+
import org.apache.paimon.catalog.CatalogContext;
22+
import org.apache.paimon.fs.hadoop.HadoopFileIO;
23+
import org.apache.paimon.fs.local.LocalFileIO;
24+
import org.apache.paimon.options.Options;
25+
26+
import org.junit.jupiter.api.BeforeEach;
27+
import org.junit.jupiter.api.Test;
28+
29+
import java.io.IOException;
30+
import java.util.concurrent.ExecutorService;
31+
import java.util.concurrent.Executors;
32+
import java.util.concurrent.Future;
33+
34+
import static org.junit.jupiter.api.Assertions.assertEquals;
35+
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
36+
import static org.junit.jupiter.api.Assertions.assertNotNull;
37+
38+
/** Tests for {@link ResolvingFileIO}. */
39+
public class ResolvingFileIOTest {
40+
41+
private ResolvingFileIO resolvingFileIO;
42+
43+
@BeforeEach
44+
public void setUp() {
45+
resolvingFileIO = new ResolvingFileIO();
46+
Options options = new Options();
47+
CatalogContext catalogContext = CatalogContext.create(options);
48+
resolvingFileIO.configure(catalogContext);
49+
}
50+
51+
@Test
52+
public void testFileIONullSchemeReturnsFallbackFileIO() throws IOException {
53+
Path path = new Path("/path/to/file");
54+
FileIO result = resolvingFileIO.fileIO(path);
55+
assertNotNull(result);
56+
assertInstanceOf(LocalFileIO.class, result);
57+
}
58+
59+
@Test
60+
public void testFileIOReturnsLocalFileIO() throws IOException {
61+
Path path = new Path("file:///path/to/file");
62+
FileIO result = resolvingFileIO.fileIO(path);
63+
assertNotNull(result);
64+
assertInstanceOf(LocalFileIO.class, result);
65+
}
66+
67+
@Test
68+
public void testFileIOWithSchemeReturnsHdfsFileIO() throws IOException {
69+
Path path = new Path("hdfs:///path/to/file");
70+
FileIO result = resolvingFileIO.fileIO(path);
71+
assertNotNull(result);
72+
assertInstanceOf(HadoopFileIO.class, result);
73+
}
74+
75+
@Test
76+
public void testFileIOConcurrentAccessInitializesFallbackFileIO() throws Exception {
77+
Path fileSchemePath = new Path("file:///path/to/file");
78+
ExecutorService executorService = Executors.newFixedThreadPool(2);
79+
Future<FileIO> future1 =
80+
executorService.submit(() -> resolvingFileIO.fileIO(fileSchemePath));
81+
Future<FileIO> future2 =
82+
executorService.submit(() -> resolvingFileIO.fileIO(fileSchemePath));
83+
84+
FileIO result1 = future1.get();
85+
FileIO result2 = future2.get();
86+
87+
assertNotNull(result1);
88+
assertNotNull(result2);
89+
assertEquals(result1, result2);
90+
assertInstanceOf(LocalFileIO.class, result1);
91+
92+
Path noSchemePath = new Path("/path/to/file");
93+
future1 = executorService.submit(() -> resolvingFileIO.fileIO(noSchemePath));
94+
future2 = executorService.submit(() -> resolvingFileIO.fileIO(noSchemePath));
95+
96+
result1 = future1.get();
97+
result2 = future2.get();
98+
99+
assertNotNull(result1);
100+
assertNotNull(result2);
101+
assertEquals(result1, result2);
102+
assertInstanceOf(LocalFileIO.class, result1);
103+
104+
Path hdfsSchemePath = new Path("hdfs:///path/to/file");
105+
future1 = executorService.submit(() -> resolvingFileIO.fileIO(hdfsSchemePath));
106+
future2 = executorService.submit(() -> resolvingFileIO.fileIO(hdfsSchemePath));
107+
108+
result1 = future1.get();
109+
result2 = future2.get();
110+
111+
assertNotNull(result1);
112+
assertNotNull(result2);
113+
assertEquals(result1, result2);
114+
assertInstanceOf(HadoopFileIO.class, result1);
115+
}
116+
117+
@Test
118+
public void testFileIOMapStoresFileIOInstances() throws IOException {
119+
Path localPath = new Path("file:///path/to/local/file1");
120+
Path hdfsPath = new Path("hdfs:///path/to/hdfs/file1");
121+
122+
// First call should create new instances
123+
FileIO localFileIO = resolvingFileIO.fileIO(localPath);
124+
FileIO hdfsFileIO = resolvingFileIO.fileIO(hdfsPath);
125+
126+
assertNotNull(localFileIO);
127+
assertNotNull(hdfsFileIO);
128+
assertInstanceOf(LocalFileIO.class, localFileIO);
129+
assertInstanceOf(HadoopFileIO.class, hdfsFileIO);
130+
131+
// Second call should return the same instances from fileIOMap
132+
FileIO localFileIOAgain = resolvingFileIO.fileIO(new Path("file:///path/to/local/file2"));
133+
FileIO hdfsFileIOAgain = resolvingFileIO.fileIO(new Path("hdfs:///path/to/local/file2"));
134+
135+
assertNotNull(localFileIOAgain);
136+
assertNotNull(hdfsFileIOAgain);
137+
assertEquals(localFileIO, localFileIOAgain);
138+
assertEquals(hdfsFileIO, hdfsFileIOAgain);
139+
}
140+
}

paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,7 @@
2020

2121
import org.apache.paimon.CoreOptions;
2222
import org.apache.paimon.fs.FileIO;
23-
import org.apache.paimon.fs.Path;
23+
import org.apache.paimon.fs.ResolvingFileIO;
2424
import org.apache.paimon.options.CatalogOptions;
2525
import org.apache.paimon.options.Options;
2626
import org.apache.paimon.partition.Partition;
@@ -90,7 +90,8 @@ public void setUp() throws Exception {
9090
Options catalogOptions = new Options();
9191
catalogOptions.set(CatalogOptions.WAREHOUSE, warehouse);
9292
CatalogContext catalogContext = CatalogContext.create(catalogOptions);
93-
fileIO = FileIO.get(new Path(warehouse), catalogContext);
93+
fileIO = new ResolvingFileIO();
94+
fileIO.configure(catalogContext);
9495
}
9596

9697
@AfterEach

0 commit comments

Comments
 (0)