Skip to content
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

[Core]Introduce ResolvingFileIO #4891

Open
wants to merge 10 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion docs/layouts/shortcodes/generated/core_configuration.html
Original file line number Diff line number Diff line change
Expand Up @@ -1024,7 +1024,7 @@
<td><h5>write-buffer-spillable</h5></td>
<td style="word-wrap: break-word;">(none)</td>
<td>Boolean</td>
<td>Whether the write buffer can be spillable. Enabled by default when using object storage.</td>
<td>Whether the write buffer can be spillable.</td>
</tr>
<tr>
<td><h5>write-manifest-cache</h5></td>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -475,8 +475,7 @@ public class CoreOptions implements Serializable {
key("write-buffer-spillable")
.booleanType()
.noDefaultValue()
.withDescription(
"Whether the write buffer can be spillable. Enabled by default when using object storage.");
neuyilan marked this conversation as resolved.
Show resolved Hide resolved
.withDescription("Whether the write buffer can be spillable.");

public static final ConfigOption<Boolean> WRITE_BUFFER_FOR_APPEND =
key("write-buffer-for-append")
Expand Down
135 changes: 135 additions & 0 deletions paimon-common/src/main/java/org/apache/paimon/fs/HybridFileIO.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,135 @@
/*
* 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.Options;

import java.io.IOException;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;

/**
* A hybrid 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 HybridFileIO implements FileIO {
neuyilan marked this conversation as resolved.
Show resolved Hide resolved
private static final long serialVersionUID = 1L;

private Options options;

private Map<String, FileIO> fileIOMap;
private volatile FileIO fallbackFileIO;

// TODO, how to decide the real fileio is object store or not?
@Override
public boolean isObjectStore() {
return false;
}

@Override
public void configure(CatalogContext context) {
this.options = context.options();
this.fileIOMap = new ConcurrentHashMap<>();
}

@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 {
String scheme = path.toUri().getScheme();
if (scheme == null) {
if (fallbackFileIO == null) {
neuyilan marked this conversation as resolved.
Show resolved Hide resolved
synchronized (this) {
if (fallbackFileIO == null) {
CatalogContext catalogContext = CatalogContext.create(options);
fallbackFileIO = FileIO.get(path, catalogContext);
}
}
}
return fallbackFileIO;
}

if (!fileIOMap.containsKey(scheme)) {
synchronized (this) {
if (!fileIOMap.containsKey(scheme)) {
CatalogContext catalogContext = CatalogContext.create(options);
FileIO fileIO = FileIO.get(path, catalogContext);
fileIOMap.put(scheme, fileIO);
}
}
}
return fileIOMap.get(scheme);
}

private <T> T wrap(Func<T> func) throws IOException {
ClassLoader cl = Thread.currentThread().getContextClassLoader();
try {
Thread.currentThread().setContextClassLoader(HybridFileIO.class.getClassLoader());
return func.apply();
} finally {
Thread.currentThread().setContextClassLoader(cl);
}
}

/** Apply function with wrapping classloader. */
@FunctionalInterface
protected interface Func<T> {
T apply() throws IOException;
}
}
138 changes: 138 additions & 0 deletions paimon-common/src/test/java/org/apache/paimon/fs/HybridFileIOTest.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,138 @@
/*
* 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 HybridFileIO}. */
public class HybridFileIOTest {

private HybridFileIO hybridFileIO;

@BeforeEach
public void setUp() {
hybridFileIO = new HybridFileIO();
Options options = new Options();
CatalogContext catalogContext = CatalogContext.create(options);
hybridFileIO.configure(catalogContext);
}

@Test
public void testFileIONullSchemeReturnsFallbackFileIO() throws IOException {
Path path = new Path("/path/to/file");
FileIO result = hybridFileIO.fileIO(path);
assertNotNull(result);
assertInstanceOf(LocalFileIO.class, result);
}

@Test
public void testFileIOReturnsLocalFileIO() throws IOException {
Path path = new Path("file:///path/to/file");
FileIO result = hybridFileIO.fileIO(path);
assertNotNull(result);
assertInstanceOf(LocalFileIO.class, result);
}

@Test
public void testFileIOWithSchemeReturnsHdfsFileIO() throws IOException {
Path path = new Path("hdfs:///path/to/file");
FileIO result = hybridFileIO.fileIO(path);
assertNotNull(result);
assertInstanceOf(HadoopFileIO.class, result);
}

@Test
public void testFileIOConcurrentAccessInitializesFallbackFileIO() throws Exception {
neuyilan marked this conversation as resolved.
Show resolved Hide resolved
Path fileSchemePath = new Path("file:///path/to/file");
ExecutorService executorService = Executors.newFixedThreadPool(2);
Future<FileIO> future1 = executorService.submit(() -> hybridFileIO.fileIO(fileSchemePath));
Future<FileIO> future2 = executorService.submit(() -> hybridFileIO.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(() -> hybridFileIO.fileIO(noSchemePath));
future2 = executorService.submit(() -> hybridFileIO.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(() -> hybridFileIO.fileIO(hdfsSchemePath));
future2 = executorService.submit(() -> hybridFileIO.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 = hybridFileIO.fileIO(localPath);
FileIO hdfsFileIO = hybridFileIO.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 = hybridFileIO.fileIO(new Path("file:///path/to/local/file2"));
FileIO hdfsFileIOAgain = hybridFileIO.fileIO(new Path("hdfs:///path/to/local/file2"));

assertNotNull(localFileIOAgain);
assertNotNull(hdfsFileIOAgain);
assertEquals(localFileIO, localFileIOAgain);
assertEquals(hdfsFileIO, hdfsFileIOAgain);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,14 +21,13 @@
import org.apache.paimon.CoreOptions;
import org.apache.paimon.catalog.CatalogContext;
import org.apache.paimon.fs.FileIO;
import org.apache.paimon.fs.HybridFileIO;
import org.apache.paimon.fs.Path;
import org.apache.paimon.options.Options;
import org.apache.paimon.schema.SchemaManager;
import org.apache.paimon.schema.TableSchema;
import org.apache.paimon.utils.StringUtils;

import java.io.IOException;
import java.io.UncheckedIOException;
import java.util.Optional;

import static org.apache.paimon.CoreOptions.PATH;
Expand All @@ -38,12 +37,8 @@
public class FileStoreTableFactory {

public static FileStoreTable create(CatalogContext context) {
FileIO fileIO;
try {
fileIO = FileIO.get(CoreOptions.path(context.options()), context);
} catch (IOException e) {
throw new UncheckedIOException(e);
}
FileIO fileIO = new HybridFileIO();
fileIO.configure(context);
return create(fileIO, context.options());
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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.HybridFileIO;
import org.apache.paimon.options.CatalogOptions;
import org.apache.paimon.options.Options;
import org.apache.paimon.schema.Schema;
Expand Down Expand Up @@ -85,7 +85,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 HybridFileIO();
fileIO.configure(catalogContext);
}

@AfterEach
Expand Down
Loading