diff --git a/docs/layouts/shortcodes/generated/catalog_configuration.html b/docs/layouts/shortcodes/generated/catalog_configuration.html
index 03efd178b8fc..8ccd4bf4a46e 100644
--- a/docs/layouts/shortcodes/generated/catalog_configuration.html
+++ b/docs/layouts/shortcodes/generated/catalog_configuration.html
@@ -140,5 +140,11 @@
String |
The warehouse root path of catalog. |
+
+ resolving-fileio.enabled |
+ false |
+ Boolean |
+ 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. |
+
diff --git a/paimon-common/src/main/java/org/apache/paimon/fs/FileIO.java b/paimon-common/src/main/java/org/apache/paimon/fs/FileIO.java
index 5ba16acfca0f..c557ece03529 100644
--- a/paimon-common/src/main/java/org/apache/paimon/fs/FileIO.java
+++ b/paimon-common/src/main/java/org/apache/paimon/fs/FileIO.java
@@ -22,6 +22,7 @@
import org.apache.paimon.catalog.CatalogContext;
import org.apache.paimon.fs.hadoop.HadoopFileIOLoader;
import org.apache.paimon.fs.local.LocalFileIO;
+import org.apache.paimon.options.CatalogOptions;
import org.apache.commons.io.IOUtils;
import org.slf4j.Logger;
@@ -411,6 +412,16 @@ default Optional readOverwrittenFileUtf8(Path path) throws IOException {
* by the given path.
*/
static FileIO get(Path path, CatalogContext config) throws IOException {
+ boolean resolvingFileIOEnabled =
+ config.options().get(CatalogOptions.RESOLVING_FILEIO_ENABLED);
+ if (resolvingFileIOEnabled) {
+ FileIO fileIO = new ResolvingFileIO();
+ // set to false to avoid infinite loop
+ config.options().set(CatalogOptions.RESOLVING_FILEIO_ENABLED, false);
+ fileIO.configure(config);
+ return fileIO;
+ }
+
URI uri = path.toUri();
if (LOG.isDebugEnabled()) {
LOG.debug("Getting FileIO by scheme {}.", uri.getScheme());
diff --git a/paimon-common/src/main/java/org/apache/paimon/fs/ResolvingFileIO.java b/paimon-common/src/main/java/org/apache/paimon/fs/ResolvingFileIO.java
new file mode 100644
index 000000000000..55c50e383804
--- /dev/null
+++ b/paimon-common/src/main/java/org/apache/paimon/fs/ResolvingFileIO.java
@@ -0,0 +1,159 @@
+/*
+ * 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.fs;
+
+import org.apache.paimon.annotation.VisibleForTesting;
+import org.apache.paimon.catalog.CatalogContext;
+import org.apache.paimon.options.CatalogOptions;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * An implementation of {@link FileIO} that supports multiple file system schemas. It dynamically
+ * selects the appropriate {@link FileIO} based on the URI scheme of the given path.
+ */
+public class ResolvingFileIO implements FileIO {
+
+ private static final long serialVersionUID = 1L;
+
+ private final Map fileIOMap = new ConcurrentHashMap<>();
+
+ private CatalogContext context;
+
+ // TODO, how to decide the real fileio is object store or not?
+ @Override
+ public boolean isObjectStore() {
+ String warehouse = context.options().get(CatalogOptions.WAREHOUSE);
+ if (warehouse == null) {
+ return false;
+ }
+ Path path = new Path(warehouse);
+ String scheme = path.toUri().getScheme();
+ return scheme != null
+ && !scheme.equalsIgnoreCase("file")
+ && !scheme.equalsIgnoreCase("hdfs");
+ }
+
+ @Override
+ public void configure(CatalogContext context) {
+ this.context = context;
+ }
+
+ @Override
+ public SeekableInputStream newInputStream(Path path) throws IOException {
+ return wrap(() -> fileIO(path).newInputStream(path));
+ }
+
+ @Override
+ public PositionOutputStream newOutputStream(Path path, boolean overwrite) throws IOException {
+ return wrap(() -> fileIO(path).newOutputStream(path, overwrite));
+ }
+
+ @Override
+ public FileStatus getFileStatus(Path path) throws IOException {
+ return wrap(() -> fileIO(path).getFileStatus(path));
+ }
+
+ @Override
+ public FileStatus[] listStatus(Path path) throws IOException {
+ return wrap(() -> fileIO(path).listStatus(path));
+ }
+
+ @Override
+ public boolean exists(Path path) throws IOException {
+ return wrap(() -> fileIO(path).exists(path));
+ }
+
+ @Override
+ public boolean delete(Path path, boolean recursive) throws IOException {
+ return wrap(() -> fileIO(path).delete(path, recursive));
+ }
+
+ @Override
+ public boolean mkdirs(Path path) throws IOException {
+ return wrap(() -> fileIO(path).mkdirs(path));
+ }
+
+ @Override
+ public boolean rename(Path src, Path dst) throws IOException {
+ return wrap(() -> fileIO(src).rename(src, dst));
+ }
+
+ @VisibleForTesting
+ public FileIO fileIO(Path path) throws IOException {
+ CacheKey cacheKey = new CacheKey(path.toUri().getScheme(), path.toUri().getAuthority());
+ return fileIOMap.computeIfAbsent(
+ cacheKey,
+ k -> {
+ try {
+ return FileIO.get(path, context);
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
+ });
+ }
+
+ private T wrap(Func func) throws IOException {
+ ClassLoader cl = Thread.currentThread().getContextClassLoader();
+ try {
+ Thread.currentThread().setContextClassLoader(ResolvingFileIO.class.getClassLoader());
+ return func.apply();
+ } finally {
+ Thread.currentThread().setContextClassLoader(cl);
+ }
+ }
+
+ /** Apply function with wrapping classloader. */
+ @FunctionalInterface
+ protected interface Func {
+ T apply() throws IOException;
+ }
+
+ private static class CacheKey implements Serializable {
+ private final String scheme;
+ private final String authority;
+
+ private CacheKey(String scheme, String authority) {
+ this.scheme = scheme;
+ this.authority = authority;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ CacheKey cacheKey = (CacheKey) o;
+ return Objects.equals(scheme, cacheKey.scheme)
+ && Objects.equals(authority, cacheKey.authority);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(scheme, authority);
+ }
+ }
+}
diff --git a/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java b/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java
index d0cfbeaf39ed..dd0cb6c5b218 100644
--- a/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java
+++ b/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java
@@ -149,4 +149,13 @@ public class CatalogOptions {
"Whether to support format tables, format table corresponds to a regular csv, parquet or orc table, allowing read and write operations. "
+ "However, during these processes, it does not connect to the metastore; hence, newly added partitions will not be reflected in"
+ " the metastore and need to be manually added as separate partition operations.");
+
+ public static final ConfigOption RESOLVING_FILEIO_ENABLED =
+ ConfigOptions.key("resolving-fileio.enabled")
+ .booleanType()
+ .defaultValue(false)
+ .withDescription(
+ "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.");
}
diff --git a/paimon-common/src/test/java/org/apache/paimon/fs/ResolvingFileIOTest.java b/paimon-common/src/test/java/org/apache/paimon/fs/ResolvingFileIOTest.java
new file mode 100644
index 000000000000..dae270eb873a
--- /dev/null
+++ b/paimon-common/src/test/java/org/apache/paimon/fs/ResolvingFileIOTest.java
@@ -0,0 +1,140 @@
+/*
+ * 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.fs;
+
+import org.apache.paimon.catalog.CatalogContext;
+import org.apache.paimon.fs.hadoop.HadoopFileIO;
+import org.apache.paimon.fs.local.LocalFileIO;
+import org.apache.paimon.options.Options;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertInstanceOf;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+
+/** Tests for {@link ResolvingFileIO}. */
+public class ResolvingFileIOTest {
+
+ private ResolvingFileIO resolvingFileIO;
+
+ @BeforeEach
+ public void setUp() {
+ resolvingFileIO = new ResolvingFileIO();
+ Options options = new Options();
+ CatalogContext catalogContext = CatalogContext.create(options);
+ resolvingFileIO.configure(catalogContext);
+ }
+
+ @Test
+ public void testFileIONullSchemeReturnsFallbackFileIO() throws IOException {
+ Path path = new Path("/path/to/file");
+ FileIO result = resolvingFileIO.fileIO(path);
+ assertNotNull(result);
+ assertInstanceOf(LocalFileIO.class, result);
+ }
+
+ @Test
+ public void testFileIOReturnsLocalFileIO() throws IOException {
+ Path path = new Path("file:///path/to/file");
+ FileIO result = resolvingFileIO.fileIO(path);
+ assertNotNull(result);
+ assertInstanceOf(LocalFileIO.class, result);
+ }
+
+ @Test
+ public void testFileIOWithSchemeReturnsHdfsFileIO() throws IOException {
+ Path path = new Path("hdfs:///path/to/file");
+ FileIO result = resolvingFileIO.fileIO(path);
+ assertNotNull(result);
+ assertInstanceOf(HadoopFileIO.class, result);
+ }
+
+ @Test
+ public void testFileIOConcurrentAccessInitializesFallbackFileIO() throws Exception {
+ Path fileSchemePath = new Path("file:///path/to/file");
+ ExecutorService executorService = Executors.newFixedThreadPool(2);
+ Future future1 =
+ executorService.submit(() -> resolvingFileIO.fileIO(fileSchemePath));
+ Future future2 =
+ executorService.submit(() -> resolvingFileIO.fileIO(fileSchemePath));
+
+ FileIO result1 = future1.get();
+ FileIO result2 = future2.get();
+
+ assertNotNull(result1);
+ assertNotNull(result2);
+ assertEquals(result1, result2);
+ assertInstanceOf(LocalFileIO.class, result1);
+
+ Path noSchemePath = new Path("/path/to/file");
+ future1 = executorService.submit(() -> resolvingFileIO.fileIO(noSchemePath));
+ future2 = executorService.submit(() -> resolvingFileIO.fileIO(noSchemePath));
+
+ result1 = future1.get();
+ result2 = future2.get();
+
+ assertNotNull(result1);
+ assertNotNull(result2);
+ assertEquals(result1, result2);
+ assertInstanceOf(LocalFileIO.class, result1);
+
+ Path hdfsSchemePath = new Path("hdfs:///path/to/file");
+ future1 = executorService.submit(() -> resolvingFileIO.fileIO(hdfsSchemePath));
+ future2 = executorService.submit(() -> resolvingFileIO.fileIO(hdfsSchemePath));
+
+ result1 = future1.get();
+ result2 = future2.get();
+
+ assertNotNull(result1);
+ assertNotNull(result2);
+ assertEquals(result1, result2);
+ assertInstanceOf(HadoopFileIO.class, result1);
+ }
+
+ @Test
+ public void testFileIOMapStoresFileIOInstances() throws IOException {
+ Path localPath = new Path("file:///path/to/local/file1");
+ Path hdfsPath = new Path("hdfs:///path/to/hdfs/file1");
+
+ // First call should create new instances
+ FileIO localFileIO = resolvingFileIO.fileIO(localPath);
+ FileIO hdfsFileIO = resolvingFileIO.fileIO(hdfsPath);
+
+ assertNotNull(localFileIO);
+ assertNotNull(hdfsFileIO);
+ assertInstanceOf(LocalFileIO.class, localFileIO);
+ assertInstanceOf(HadoopFileIO.class, hdfsFileIO);
+
+ // Second call should return the same instances from fileIOMap
+ FileIO localFileIOAgain = resolvingFileIO.fileIO(new Path("file:///path/to/local/file2"));
+ FileIO hdfsFileIOAgain = resolvingFileIO.fileIO(new Path("hdfs:///path/to/local/file2"));
+
+ assertNotNull(localFileIOAgain);
+ assertNotNull(hdfsFileIOAgain);
+ assertEquals(localFileIO, localFileIOAgain);
+ assertEquals(hdfsFileIO, hdfsFileIOAgain);
+ }
+}
diff --git a/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java b/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java
index b8da733e7740..453a38a9d901 100644
--- a/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java
+++ b/paimon-core/src/test/java/org/apache/paimon/catalog/CatalogTestBase.java
@@ -20,7 +20,7 @@
import org.apache.paimon.CoreOptions;
import org.apache.paimon.fs.FileIO;
-import org.apache.paimon.fs.Path;
+import org.apache.paimon.fs.ResolvingFileIO;
import org.apache.paimon.options.CatalogOptions;
import org.apache.paimon.options.Options;
import org.apache.paimon.partition.Partition;
@@ -90,7 +90,8 @@ public void setUp() throws Exception {
Options catalogOptions = new Options();
catalogOptions.set(CatalogOptions.WAREHOUSE, warehouse);
CatalogContext catalogContext = CatalogContext.create(catalogOptions);
- fileIO = FileIO.get(new Path(warehouse), catalogContext);
+ fileIO = new ResolvingFileIO();
+ fileIO.configure(catalogContext);
}
@AfterEach