-
Notifications
You must be signed in to change notification settings - Fork 1.3k
[Core]Introduce ResolvingFileIO #4891
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
Merged
Changes from all commits
Commits
Show all changes
16 commits
Select commit
Hold shift + click to select a range
419cf97
introduce hybyidfileio
neuyilan 079ce2d
spotless
neuyilan a39faf0
fix review comments
neuyilan c99cfec
fix review comments
neuyilan ef85c9e
fix review comments
neuyilan bbbfdf3
change to ResolvingFileIO
neuyilan 95daab2
Merge branch 'master' into dev/qhl/hybird_fileio
neuyilan d207bce
fix null exception
neuyilan 48bf1d5
fix review comments
neuyilan 907c577
fix review comments
neuyilan c29e715
add data-file.external-paths.enabled catalogoptions
neuyilan 867dc03
fix docs description
neuyilan 7519f5d
rename the options
neuyilan abc1f9a
spotless
neuyilan a898543
fix re
neuyilan 2e5ecb5
fix review comments
neuyilan File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
159 changes: 159 additions & 0 deletions
159
paimon-common/src/main/java/org/apache/paimon/fs/ResolvingFileIO.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -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<CacheKey, FileIO> 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> T wrap(Func<T> 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> { | ||
| 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); | ||
| } | ||
| } | ||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
140 changes: 140 additions & 0 deletions
140
paimon-common/src/test/java/org/apache/paimon/fs/ResolvingFileIOTest.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -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<FileIO> future1 = | ||
| executorService.submit(() -> resolvingFileIO.fileIO(fileSchemePath)); | ||
| Future<FileIO> 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); | ||
| } | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.