From 6718af4e686954f07feefa24d65e7b404ce13b17 Mon Sep 17 00:00:00 2001 From: HouliangQi Date: Tue, 24 Dec 2024 20:23:42 +0800 Subject: [PATCH 01/21] support write to the external path --- .../generated/core_configuration.html | 6 +++ .../java/org/apache/paimon/CoreOptions.java | 34 +++++++++++++ .../main/java/org/apache/paimon/fs/Path.java | 11 ++++ .../org/apache/paimon/AbstractFileStore.java | 3 +- .../apache/paimon/io/DataFilePathFactory.java | 25 ++++++--- .../paimon/io/KeyValueDataFileWriter.java | 3 +- .../apache/paimon/io/RowDataFileWriter.java | 3 +- .../paimon/table/AbstractFileStoreTable.java | 36 +++++++++++-- .../table/AppendOnlyFileStoreTable.java | 7 +-- .../org/apache/paimon/table/DataTable.java | 2 + .../paimon/table/DelegatedFileStoreTable.java | 5 ++ .../table/FallbackReadFileStoreTable.java | 3 +- .../paimon/table/FileStoreTableFactory.java | 51 +++++++++++++++---- .../table/PrimaryKeyFileStoreTable.java | 7 +-- .../paimon/table/system/AuditLogTable.java | 5 ++ .../table/system/CompactBucketsTable.java | 5 ++ .../paimon/table/system/FileMonitorTable.java | 5 ++ .../table/system/ReadOptimizedTable.java | 5 ++ .../paimon/utils/FileStorePathFactory.java | 20 ++++++-- .../paimon/append/AppendOnlyWriterTest.java | 3 +- .../paimon/format/FileFormatSuffixTest.java | 3 +- .../paimon/io/DataFilePathFactoryTest.java | 6 ++- .../paimon/io/KeyValueFileReadWriteTest.java | 6 ++- .../paimon/io/RollingFileWriterTest.java | 3 +- .../manifest/ManifestFileMetaTestBase.java | 3 +- .../paimon/manifest/ManifestFileTest.java | 3 +- .../paimon/manifest/ManifestListTest.java | 3 +- .../source/snapshot/ScannerTestBase.java | 2 +- .../source/snapshot/SnapshotReaderTest.java | 2 +- .../utils/FileStorePathFactoryTest.java | 6 ++- .../source/TestChangelogDataReadWrite.java | 3 +- .../paimon/spark/SparkFileIndexITCase.java | 3 +- 32 files changed, 231 insertions(+), 51 deletions(-) diff --git a/docs/layouts/shortcodes/generated/core_configuration.html b/docs/layouts/shortcodes/generated/core_configuration.html index 1133de289fa3..2a8ea1c0bf1e 100644 --- a/docs/layouts/shortcodes/generated/core_configuration.html +++ b/docs/layouts/shortcodes/generated/core_configuration.html @@ -1008,5 +1008,11 @@ Integer The bytes of types (CHAR, VARCHAR, BINARY, VARBINARY) devote to the zorder sort. + +
data-file.external-path
+ (none) + String + The path where the data of this table is currently written. + diff --git a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java index 6e1e9bba076b..de08a6c9c27d 100644 --- a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java +++ b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java @@ -124,6 +124,13 @@ public class CoreOptions implements Serializable { + "if there is no primary key, the full row will be used.") .build()); + public static final ConfigOption DATA_FILE_EXTERNAL_PATH = + key("data-file.external-path") + .stringType() + .noDefaultValue() + .withDescription("The path where the data of this table is currently written."); + + // todo, this path is the table schema path, the name will be changed in the later PR. @ExcludeFromDocumentation("Internal use only") public static final ConfigOption PATH = key("path") @@ -131,6 +138,17 @@ public class CoreOptions implements Serializable { .noDefaultValue() .withDescription("The file path of this table in the filesystem."); + @ExcludeFromDocumentation("Internal use only") + public static final ConfigOption TABLE_DATA_PATH = + key("table.data.path") + .stringType() + .noDefaultValue() + .withDescription( + "The data file path of this table in the filesystem. if " + + DATA_FILE_EXTERNAL_PATH.key() + + "is not set, it will be same with." + + PATH.key()); + public static final ConfigOption BRANCH = key("branch").stringType().defaultValue("main").withDescription("Specify branch name."); @@ -1552,6 +1570,10 @@ public Path path() { return path(options.toMap()); } + public Path dataPath() { + return dataPath(options.toMap()); + } + public String branch() { return branch(options.toMap()); } @@ -1571,6 +1593,14 @@ public static Path path(Options options) { return new Path(options.get(PATH)); } + public static Path dataPath(Map options) { + return new Path(options.get(TABLE_DATA_PATH.key())); + } + + public static Path dataPath(Options options) { + return new Path(options.get(TABLE_DATA_PATH)); + } + public TableType type() { return options.get(TYPE); } @@ -2364,6 +2394,10 @@ public boolean statsDenseStore() { return options.get(METADATA_STATS_DENSE_STORE); } + public String dataFileExternalPath() { + return options.get(DATA_FILE_EXTERNAL_PATH); + } + public boolean dataFileThinMode() { return options.get(DATA_FILE_THIN_MODE); } diff --git a/paimon-common/src/main/java/org/apache/paimon/fs/Path.java b/paimon-common/src/main/java/org/apache/paimon/fs/Path.java index 3cfeac5b6de4..fa72e1d40d0a 100644 --- a/paimon-common/src/main/java/org/apache/paimon/fs/Path.java +++ b/paimon-common/src/main/java/org/apache/paimon/fs/Path.java @@ -62,6 +62,8 @@ public class Path implements Comparable, Serializable { /** A hierarchical URI. */ private URI uri; + private boolean isExternalPath; + /** * Create a new Path based on the child path resolved against the parent path. * @@ -385,4 +387,13 @@ public int hashCode() { public int compareTo(Path that) { return this.uri.compareTo(that.uri); } + + public Path setExternalPath(boolean externalPath) { + this.isExternalPath = externalPath; + return this; + } + + public boolean isExternalPath() { + return isExternalPath; + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java b/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java index 1caff252a654..ac9c8b46d802 100644 --- a/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java +++ b/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java @@ -119,7 +119,8 @@ protected FileStorePathFactory pathFactory(String format) { options.legacyPartitionName(), options.fileSuffixIncludeCompression(), options.fileCompression(), - options.dataFilePathDirectory()); + options.dataFilePathDirectory(), + options.dataPath()); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/io/DataFilePathFactory.java b/paimon-core/src/main/java/org/apache/paimon/io/DataFilePathFactory.java index 19525ab6cd91..354d2d97a34a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/DataFilePathFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/DataFilePathFactory.java @@ -43,6 +43,7 @@ public class DataFilePathFactory { private final String changelogFilePrefix; private final boolean fileSuffixIncludeCompression; private final String fileCompression; + private final boolean isExternalPath; public DataFilePathFactory( Path parent, @@ -50,7 +51,8 @@ public DataFilePathFactory( String dataFilePrefix, String changelogFilePrefix, boolean fileSuffixIncludeCompression, - String fileCompression) { + String fileCompression, + boolean isExternalPath) { this.parent = parent; this.uuid = UUID.randomUUID().toString(); this.pathCount = new AtomicInteger(0); @@ -59,6 +61,7 @@ public DataFilePathFactory( this.changelogFilePrefix = changelogFilePrefix; this.fileSuffixIncludeCompression = fileSuffixIncludeCompression; this.fileCompression = fileCompression; + this.isExternalPath = isExternalPath; } public Path newPath() { @@ -74,7 +77,7 @@ public String newChangelogFileName() { } public Path newPath(String prefix) { - return new Path(parent, newFileName(prefix)); + return new Path(parent, newFileName(prefix)).setExternalPath(isExternalPath); } private String newFileName(String prefix) { @@ -88,21 +91,27 @@ private String newFileName(String prefix) { } public Path toPath(DataFileMeta file) { - return file.externalPath().map(Path::new).orElse(new Path(parent, file.fileName())); + return file.externalPath() + .map(Path::new) + .orElse(new Path(parent, file.fileName())) + .setExternalPath(file.externalPath().isPresent()); } public Path toPath(FileEntry file) { return Optional.ofNullable(file.externalPath()) .map(Path::new) - .orElse(new Path(parent, file.fileName())); + .orElse(new Path(parent, file.fileName())) + .setExternalPath(Optional.ofNullable(file.externalPath()).isPresent()); } public Path toAlignedPath(String fileName, DataFileMeta aligned) { - return new Path(aligned.externalPathDir().map(Path::new).orElse(parent), fileName); + return new Path(aligned.externalPathDir().map(Path::new).orElse(parent), fileName) + .setExternalPath(aligned.externalPathDir().isPresent()); } public static Path dataFileToFileIndexPath(Path dataFilePath) { - return new Path(dataFilePath.getParent(), dataFilePath.getName() + INDEX_PATH_SUFFIX); + return new Path(dataFilePath.getParent(), dataFilePath.getName() + INDEX_PATH_SUFFIX) + .setExternalPath(dataFilePath.isExternalPath()); } public static Path createNewFileIndexFilePath(Path filePath) { @@ -121,7 +130,9 @@ public static Path createNewFileIndexFilePath(Path filePath) { } } return new Path( - filePath.getParent(), fileName.substring(0, dot) + "-" + 1 + INDEX_PATH_SUFFIX); + filePath.getParent(), + fileName.substring(0, dot) + "-" + 1 + INDEX_PATH_SUFFIX) + .setExternalPath(filePath.isExternalPath()); } public static String formatIdentifier(String fileName) { diff --git a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileWriter.java b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileWriter.java index f78d7556487f..65ea06b8fb9d 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileWriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileWriter.java @@ -177,6 +177,7 @@ public DataFileMeta result() throws IOException { ? DataFileIndexWriter.EMPTY_RESULT : dataFileIndexWriter.result(); + String externalPath = path.isExternalPath() ? path.toString() : null; return new DataFileMeta( path.getName(), fileIO.getFileSize(path), @@ -196,7 +197,7 @@ public DataFileMeta result() throws IOException { indexResult.embeddedIndexBytes(), fileSource, valueStatsPair.getKey(), - null); + externalPath); } abstract Pair fetchKeyValueStats(SimpleColStats[] rowStats); diff --git a/paimon-core/src/main/java/org/apache/paimon/io/RowDataFileWriter.java b/paimon-core/src/main/java/org/apache/paimon/io/RowDataFileWriter.java index cd46d67e3b60..1db35eba1f2f 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/RowDataFileWriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/RowDataFileWriter.java @@ -111,6 +111,7 @@ public DataFileMeta result() throws IOException { dataFileIndexWriter == null ? DataFileIndexWriter.EMPTY_RESULT : dataFileIndexWriter.result(); + String externalPath = path.isExternalPath() ? path.toString() : null; return DataFileMeta.forAppend( path.getName(), fileIO.getFileSize(path), @@ -125,6 +126,6 @@ public DataFileMeta result() throws IOException { indexResult.embeddedIndexBytes(), fileSource, statsPair.getKey(), - null); + externalPath); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java index 7e008698c4fd..25e14b4a74e7 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java @@ -87,6 +87,7 @@ import java.util.function.BiConsumer; import static org.apache.paimon.CoreOptions.PATH; +import static org.apache.paimon.CoreOptions.TABLE_DATA_PATH; import static org.apache.paimon.utils.Preconditions.checkArgument; /** Abstract {@link FileStoreTable}. */ @@ -98,6 +99,7 @@ abstract class AbstractFileStoreTable implements FileStoreTable { protected final FileIO fileIO; protected final Path path; + protected final Path tableDataPath; protected final TableSchema tableSchema; protected final CatalogEnvironment catalogEnvironment; @@ -109,7 +111,8 @@ protected AbstractFileStoreTable( FileIO fileIO, Path path, TableSchema tableSchema, - CatalogEnvironment catalogEnvironment) { + CatalogEnvironment catalogEnvironment, + Path tableDataPath) { this.fileIO = fileIO; this.path = path; if (!tableSchema.options().containsKey(PATH.key())) { @@ -118,8 +121,16 @@ protected AbstractFileStoreTable( newOptions.put(PATH.key(), path.toString()); tableSchema = tableSchema.copy(newOptions); } + + if (!tableSchema.options().containsKey(TABLE_DATA_PATH.key())) { + Map newOptions = new HashMap<>(tableSchema.options()); + newOptions.put(TABLE_DATA_PATH.key(), tableDataPath.toString()); + tableSchema = tableSchema.copy(newOptions); + } + this.tableSchema = tableSchema; this.catalogEnvironment = catalogEnvironment; + this.tableDataPath = tableDataPath; } public String currentBranch() { @@ -336,6 +347,9 @@ private FileStoreTable copyInternal(Map dynamicOptions, boolean // set path always newOptions.set(PATH, path.toString()); + // set tableDataPath always + newOptions.set(TABLE_DATA_PATH, tableDataPath.toString()); + // set dynamic options with default values CoreOptions.setDefaultValues(newOptions); @@ -372,9 +386,9 @@ public FileStoreTable copy(TableSchema newTableSchema) { AbstractFileStoreTable copied = newTableSchema.primaryKeys().isEmpty() ? new AppendOnlyFileStoreTable( - fileIO, path, newTableSchema, catalogEnvironment) + fileIO, path, newTableSchema, catalogEnvironment, tableDataPath) : new PrimaryKeyFileStoreTable( - fileIO, path, newTableSchema, catalogEnvironment); + fileIO, path, newTableSchema, catalogEnvironment, tableDataPath); if (snapshotCache != null) { copied.setSnapshotCache(snapshotCache); } @@ -407,6 +421,11 @@ public Path location() { return path; } + @Override + public Path dataLocation() { + return tableDataPath; + } + @Override public TableSchema schema() { return tableSchema; @@ -738,7 +757,12 @@ public FileStoreTable switchToBranch(String branchName) { branchOptions.set(CoreOptions.BRANCH, targetBranch); branchSchema = branchSchema.copy(branchOptions.toMap()); return FileStoreTableFactory.create( - fileIO(), location(), branchSchema, new Options(), catalogEnvironment()); + fileIO(), + location(), + branchSchema, + new Options(), + catalogEnvironment(), + tableDataPath); } private RollbackHelper rollbackHelper() { @@ -764,6 +788,8 @@ public boolean equals(Object o) { return false; } AbstractFileStoreTable that = (AbstractFileStoreTable) o; - return Objects.equals(path, that.path) && Objects.equals(tableSchema, that.tableSchema); + return Objects.equals(path, that.path) + && Objects.equals(tableSchema, that.tableSchema) + && Objects.equals(tableDataPath, that.tableDataPath); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/AppendOnlyFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/AppendOnlyFileStoreTable.java index 103fa64050aa..1f0e395e1aeb 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/AppendOnlyFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/AppendOnlyFileStoreTable.java @@ -58,15 +58,16 @@ class AppendOnlyFileStoreTable extends AbstractFileStoreTable { private transient AppendOnlyFileStore lazyStore; AppendOnlyFileStoreTable(FileIO fileIO, Path path, TableSchema tableSchema) { - this(fileIO, path, tableSchema, CatalogEnvironment.empty()); + this(fileIO, path, tableSchema, CatalogEnvironment.empty(), path); } AppendOnlyFileStoreTable( FileIO fileIO, Path path, TableSchema tableSchema, - CatalogEnvironment catalogEnvironment) { - super(fileIO, path, tableSchema, catalogEnvironment); + CatalogEnvironment catalogEnvironment, + Path tableDataPath) { + super(fileIO, path, tableSchema, catalogEnvironment, tableDataPath); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/table/DataTable.java b/paimon-core/src/main/java/org/apache/paimon/table/DataTable.java index 7979daccf756..b87660d7c865 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/DataTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/DataTable.java @@ -54,5 +54,7 @@ public interface DataTable extends InnerTable { Path location(); + Path dataLocation(); + FileIO fileIO(); } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/DelegatedFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/DelegatedFileStoreTable.java index 0a548941bedc..a9cda655f371 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/DelegatedFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/DelegatedFileStoreTable.java @@ -115,6 +115,11 @@ public Path location() { return wrapped.location(); } + @Override + public Path dataLocation() { + return wrapped.dataLocation(); + } + @Override public FileIO fileIO() { return wrapped.fileIO(); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/FallbackReadFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/FallbackReadFileStoreTable.java index e3e290f06086..999f6ca6297c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/FallbackReadFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/FallbackReadFileStoreTable.java @@ -115,7 +115,8 @@ private FileStoreTable switchWrappedToBranch(String branchName) { wrapped.location(), branchSchema, new Options(), - wrapped.catalogEnvironment()); + wrapped.catalogEnvironment(), + wrapped.dataLocation()); } private Map rewriteFallbackOptions(Map options) { diff --git a/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTableFactory.java b/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTableFactory.java index 423dc1726319..f6fb08a73be0 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTableFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTableFactory.java @@ -65,11 +65,30 @@ public static FileStoreTable create(FileIO fileIO, Options options) { "Schema file not found in location " + tablePath + ". Please create table first.")); - return create(fileIO, tablePath, tableSchema, options, CatalogEnvironment.empty()); + + Path tableDataPath = getTableDataPath(tableSchema, tablePath); + return create( + fileIO, tablePath, tableSchema, options, CatalogEnvironment.empty(), tableDataPath); + } + + private static Path getTableDataPath(TableSchema tableSchema, Path tablePath) { + String externalPath = tableSchema.options().get(CoreOptions.DATA_FILE_EXTERNAL_PATH.key()); + if (externalPath == null || externalPath.isEmpty()) { + return tablePath; + } + String dbAndTablePath = tablePath.getParent().getName() + "/" + tablePath.getName(); + return new Path(externalPath, dbAndTablePath); } public static FileStoreTable create(FileIO fileIO, Path tablePath, TableSchema tableSchema) { - return create(fileIO, tablePath, tableSchema, new Options(), CatalogEnvironment.empty()); + Path tableDataPath = getTableDataPath(tableSchema, tablePath); + return create( + fileIO, + tablePath, + tableSchema, + new Options(), + CatalogEnvironment.empty(), + tableDataPath); } public static FileStoreTable create( @@ -77,7 +96,9 @@ public static FileStoreTable create( Path tablePath, TableSchema tableSchema, CatalogEnvironment catalogEnvironment) { - return create(fileIO, tablePath, tableSchema, new Options(), catalogEnvironment); + Path tableDataPath = getTableDataPath(tableSchema, tablePath); + return create( + fileIO, tablePath, tableSchema, new Options(), catalogEnvironment, tableDataPath); } public static FileStoreTable create( @@ -85,10 +106,16 @@ public static FileStoreTable create( Path tablePath, TableSchema tableSchema, Options dynamicOptions, - CatalogEnvironment catalogEnvironment) { + CatalogEnvironment catalogEnvironment, + Path tableDataPath) { FileStoreTable table = createWithoutFallbackBranch( - fileIO, tablePath, tableSchema, dynamicOptions, catalogEnvironment); + fileIO, + tablePath, + tableSchema, + dynamicOptions, + catalogEnvironment, + tableDataPath); Options options = new Options(table.options()); String fallbackBranch = options.get(CoreOptions.SCAN_FALLBACK_BRANCH); @@ -105,7 +132,12 @@ public static FileStoreTable create( fallbackBranch); FileStoreTable fallbackTable = createWithoutFallbackBranch( - fileIO, tablePath, schema.get(), branchOptions, catalogEnvironment); + fileIO, + tablePath, + schema.get(), + branchOptions, + catalogEnvironment, + tableDataPath); table = new FallbackReadFileStoreTable(table, fallbackTable); } @@ -117,13 +149,14 @@ public static FileStoreTable createWithoutFallbackBranch( Path tablePath, TableSchema tableSchema, Options dynamicOptions, - CatalogEnvironment catalogEnvironment) { + CatalogEnvironment catalogEnvironment, + Path tableDataPath) { FileStoreTable table = tableSchema.primaryKeys().isEmpty() ? new AppendOnlyFileStoreTable( - fileIO, tablePath, tableSchema, catalogEnvironment) + fileIO, tablePath, tableSchema, catalogEnvironment, tableDataPath) : new PrimaryKeyFileStoreTable( - fileIO, tablePath, tableSchema, catalogEnvironment); + fileIO, tablePath, tableSchema, catalogEnvironment, tableDataPath); return table.copy(dynamicOptions.toMap()); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java index 516ae766cef8..69820944a7c1 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java @@ -57,15 +57,16 @@ class PrimaryKeyFileStoreTable extends AbstractFileStoreTable { private transient KeyValueFileStore lazyStore; PrimaryKeyFileStoreTable(FileIO fileIO, Path path, TableSchema tableSchema) { - this(fileIO, path, tableSchema, CatalogEnvironment.empty()); + this(fileIO, path, tableSchema, CatalogEnvironment.empty(), path); } PrimaryKeyFileStoreTable( FileIO fileIO, Path path, TableSchema tableSchema, - CatalogEnvironment catalogEnvironment) { - super(fileIO, path, tableSchema, catalogEnvironment); + CatalogEnvironment catalogEnvironment, + Path tableDataPath) { + super(fileIO, path, tableSchema, catalogEnvironment, tableDataPath); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java index 1cb967f8d1e2..3fff9e2ca105 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java @@ -183,6 +183,11 @@ public Path location() { return wrapped.location(); } + @Override + public Path dataLocation() { + return wrapped.dataLocation(); + } + @Override public SnapshotManager snapshotManager() { return wrapped.snapshotManager(); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/CompactBucketsTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/CompactBucketsTable.java index 31cecbfb15c2..b2bcd63f5259 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/CompactBucketsTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/CompactBucketsTable.java @@ -141,6 +141,11 @@ public Path location() { return wrapped.location(); } + @Override + public Path dataLocation() { + return wrapped.dataLocation(); + } + @Override public SnapshotManager snapshotManager() { return wrapped.snapshotManager(); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/FileMonitorTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/FileMonitorTable.java index 522335aaa6c9..3ff111ea9617 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/FileMonitorTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/FileMonitorTable.java @@ -127,6 +127,11 @@ public Path location() { return wrapped.location(); } + @Override + public Path dataLocation() { + return wrapped.dataLocation(); + } + @Override public SnapshotManager snapshotManager() { return wrapped.snapshotManager(); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/ReadOptimizedTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/ReadOptimizedTable.java index 5308005053c8..3884798f9274 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/ReadOptimizedTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/ReadOptimizedTable.java @@ -161,6 +161,11 @@ public Path location() { return wrapped.location(); } + @Override + public Path dataLocation() { + return wrapped.dataLocation(); + } + @Override public SnapshotManager snapshotManager() { return wrapped.snapshotManager(); diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java b/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java index f255762cfd3c..12952afb582b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java @@ -38,7 +38,10 @@ public class FileStorePathFactory { public static final String BUCKET_PATH_PREFIX = "bucket-"; + // this is the table schema root path private final Path root; + // this is the table data root path + private final Path dataRoot; private final String uuid; private final InternalRowPartitionComputer partitionComputer; private final String formatIdentifier; @@ -54,6 +57,7 @@ public class FileStorePathFactory { private final AtomicInteger indexManifestCount; private final AtomicInteger indexFileCount; private final AtomicInteger statsFileCount; + private final boolean isExternalPath; public FileStorePathFactory( Path root, @@ -65,7 +69,8 @@ public FileStorePathFactory( boolean legacyPartitionName, boolean fileSuffixIncludeCompression, String fileCompression, - @Nullable String dataFilePathDirectory) { + @Nullable String dataFilePathDirectory, + Path dataRoot) { this.root = root; this.dataFilePathDirectory = dataFilePathDirectory; this.uuid = UUID.randomUUID().toString(); @@ -83,6 +88,8 @@ public FileStorePathFactory( this.indexManifestCount = new AtomicInteger(0); this.indexFileCount = new AtomicInteger(0); this.statsFileCount = new AtomicInteger(0); + this.dataRoot = dataRoot; + this.isExternalPath = !root.equals(dataRoot); } public Path root() { @@ -126,13 +133,18 @@ public DataFilePathFactory createDataFilePathFactory(BinaryRow partition, int bu dataFilePrefix, changelogFilePrefix, fileSuffixIncludeCompression, - fileCompression); + fileCompression, + isExternalPath); } public Path bucketPath(BinaryRow partition, int bucket) { - return new Path(root, relativeBucketPath(partition, bucket)); + return new Path(dataRoot, relativeBucketPath(partition, bucket)); } + // public Path dataBucketPath(BinaryRow partition, int bucket) { + // return new Path(dataRoot, relativeBucketPath(partition, bucket)); + // } + public Path relativeBucketPath(BinaryRow partition, int bucket) { Path relativeBucketPath = new Path(BUCKET_PATH_PREFIX + bucket); String partitionPath = getPartitionString(partition); @@ -160,7 +172,7 @@ public List getHierarchicalPartitionPath(BinaryRow partition) { partition, "Partition binary row is null. This is unexpected."))) .stream() - .map(p -> new Path(root + "/" + p)) + .map(p -> new Path(dataRoot + "/" + p)) .collect(Collectors.toList()); } diff --git a/paimon-core/src/test/java/org/apache/paimon/append/AppendOnlyWriterTest.java b/paimon-core/src/test/java/org/apache/paimon/append/AppendOnlyWriterTest.java index 77570205327e..0dbbc8585c44 100644 --- a/paimon-core/src/test/java/org/apache/paimon/append/AppendOnlyWriterTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/append/AppendOnlyWriterTest.java @@ -523,7 +523,8 @@ private DataFilePathFactory createPathFactory() { CoreOptions.DATA_FILE_PREFIX.defaultValue(), CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), - CoreOptions.FILE_COMPRESSION.defaultValue()); + CoreOptions.FILE_COMPRESSION.defaultValue(), + false); } private AppendOnlyWriter createEmptyWriter(long targetFileSize) { diff --git a/paimon-core/src/test/java/org/apache/paimon/format/FileFormatSuffixTest.java b/paimon-core/src/test/java/org/apache/paimon/format/FileFormatSuffixTest.java index c29519ce8b9b..d1f95c39c6d1 100644 --- a/paimon-core/src/test/java/org/apache/paimon/format/FileFormatSuffixTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/format/FileFormatSuffixTest.java @@ -72,7 +72,8 @@ public void testFileSuffix(@TempDir java.nio.file.Path tempDir) throws Exception CoreOptions.DATA_FILE_PREFIX.defaultValue(), CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), - CoreOptions.FILE_COMPRESSION.defaultValue()); + CoreOptions.FILE_COMPRESSION.defaultValue(), + false); FileFormat fileFormat = FileFormat.fromIdentifier(format, new Options()); LinkedList toCompact = new LinkedList<>(); CoreOptions options = new CoreOptions(new HashMap<>()); diff --git a/paimon-core/src/test/java/org/apache/paimon/io/DataFilePathFactoryTest.java b/paimon-core/src/test/java/org/apache/paimon/io/DataFilePathFactoryTest.java index 109f33c3dc1b..ddb76cfdcc52 100644 --- a/paimon-core/src/test/java/org/apache/paimon/io/DataFilePathFactoryTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/io/DataFilePathFactoryTest.java @@ -40,7 +40,8 @@ public void testNoPartition() { CoreOptions.DATA_FILE_PREFIX.defaultValue(), CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), - CoreOptions.FILE_COMPRESSION.defaultValue()); + CoreOptions.FILE_COMPRESSION.defaultValue(), + false); String uuid = pathFactory.uuid(); for (int i = 0; i < 20; i++) { @@ -69,7 +70,8 @@ public void testWithPartition() { CoreOptions.DATA_FILE_PREFIX.defaultValue(), CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), - CoreOptions.FILE_COMPRESSION.defaultValue()); + CoreOptions.FILE_COMPRESSION.defaultValue(), + false); String uuid = pathFactory.uuid(); for (int i = 0; i < 20; i++) { diff --git a/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java b/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java index 8f2c815404cf..8e234a82ac97 100644 --- a/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java @@ -237,7 +237,8 @@ protected KeyValueFileWriterFactory createWriterFactory(String pathStr, String f CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), - null); + null, + path); int suggestedFileSize = ThreadLocalRandom.current().nextInt(8192) + 1024; FileIO fileIO = FileIOFinder.find(path); Options options = new Options(); @@ -257,7 +258,8 @@ protected KeyValueFileWriterFactory createWriterFactory(String pathStr, String f CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), - null)); + null, + path)); return KeyValueFileWriterFactory.builder( fileIO, diff --git a/paimon-core/src/test/java/org/apache/paimon/io/RollingFileWriterTest.java b/paimon-core/src/test/java/org/apache/paimon/io/RollingFileWriterTest.java index 9e1de71451a8..bf50e23e1658 100644 --- a/paimon-core/src/test/java/org/apache/paimon/io/RollingFileWriterTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/io/RollingFileWriterTest.java @@ -84,7 +84,8 @@ public void initialize(String identifier, boolean statsDenseStore) { .defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION .defaultValue(), - CoreOptions.FILE_COMPRESSION.defaultValue()) + CoreOptions.FILE_COMPRESSION.defaultValue(), + false) .newPath(), SCHEMA, fileFormat diff --git a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTestBase.java b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTestBase.java index 19bd6a856bf9..f6a216513c6b 100644 --- a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTestBase.java +++ b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTestBase.java @@ -151,7 +151,8 @@ protected ManifestFile createManifestFile(String pathStr) { CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), - null), + null, + path), Long.MAX_VALUE, null) .create(); diff --git a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileTest.java b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileTest.java index 089e11656a99..8f9f4c768396 100644 --- a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileTest.java @@ -106,7 +106,8 @@ private ManifestFile createManifestFile(String pathStr) { CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), - null); + null, + path); int suggestedFileSize = ThreadLocalRandom.current().nextInt(8192) + 1024; FileIO fileIO = FileIOFinder.find(path); return new ManifestFile.Factory( diff --git a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestListTest.java b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestListTest.java index 5bf01f32cb07..1baf688a78d6 100644 --- a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestListTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestListTest.java @@ -110,7 +110,8 @@ private ManifestList createManifestList(String pathStr) { CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), - null); + null, + path); return new ManifestList.Factory(FileIOFinder.find(path), avro, "zstd", pathFactory, null) .create(); } diff --git a/paimon-core/src/test/java/org/apache/paimon/table/source/snapshot/ScannerTestBase.java b/paimon-core/src/test/java/org/apache/paimon/table/source/snapshot/ScannerTestBase.java index 2ed0d5c9b34b..ce53337a6cfd 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/source/snapshot/ScannerTestBase.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/source/snapshot/ScannerTestBase.java @@ -168,7 +168,7 @@ protected FileStoreTable createFileStoreTable( conf.toMap(), "")); return FileStoreTableFactory.create( - fileIO, tablePath, tableSchema, conf, CatalogEnvironment.empty()); + fileIO, tablePath, tableSchema, conf, CatalogEnvironment.empty(), tablePath); } protected List toSplits(List dataSplits) { diff --git a/paimon-core/src/test/java/org/apache/paimon/table/source/snapshot/SnapshotReaderTest.java b/paimon-core/src/test/java/org/apache/paimon/table/source/snapshot/SnapshotReaderTest.java index 02a170c362ef..48ad20f3e4c1 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/source/snapshot/SnapshotReaderTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/source/snapshot/SnapshotReaderTest.java @@ -368,6 +368,6 @@ private FileStoreTable createFileStoreTable( options.toMap(), "")); return FileStoreTableFactory.create( - fileIO, tablePath, tableSchema, options, CatalogEnvironment.empty()); + fileIO, tablePath, tableSchema, options, CatalogEnvironment.empty(), tablePath); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/utils/FileStorePathFactoryTest.java b/paimon-core/src/test/java/org/apache/paimon/utils/FileStorePathFactoryTest.java index c5cda2286dfb..5d5a546d704c 100644 --- a/paimon-core/src/test/java/org/apache/paimon/utils/FileStorePathFactoryTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/utils/FileStorePathFactoryTest.java @@ -92,7 +92,8 @@ public void testCreateDataFilePathFactoryWithPartition() { CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), - null); + null, + new Path(tempDir.toString())); assertPartition("20211224", 16, pathFactory, "/dt=20211224/hr=16"); assertPartition("20211224", null, pathFactory, "/dt=20211224/hr=default"); @@ -134,6 +135,7 @@ public static FileStorePathFactory createNonPartFactory(Path root) { CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), - null); + null, + root); } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java index d2bb9eb98274..cd9a45270a16 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java @@ -111,7 +111,8 @@ public TestChangelogDataReadWrite(String root) { CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), - null); + null, + tablePath); this.snapshotManager = new SnapshotManager(LocalFileIO.create(), new Path(root)); this.commitUser = UUID.randomUUID().toString(); } diff --git a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkFileIndexITCase.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkFileIndexITCase.java index 99e95cf40e5a..65424ba39987 100644 --- a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkFileIndexITCase.java +++ b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkFileIndexITCase.java @@ -161,7 +161,8 @@ protected void foreachIndexReader(Consumer consumer) CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), - null); + null, + tableRoot); Table table = fileSystemCatalog.getTable(Identifier.create("db", "T")); ReadBuilder readBuilder = table.newReadBuilder(); From 6edd28175c1e0f92d72435d57a1f8acce2c25de1 Mon Sep 17 00:00:00 2001 From: HouliangQi Date: Tue, 24 Dec 2024 20:52:52 +0800 Subject: [PATCH 02/21] fix compile error --- .../java/org/apache/paimon/utils/FileStorePathFactory.java | 4 ---- .../test/java/org/apache/paimon/flink/sink/FlinkSinkTest.java | 3 ++- 2 files changed, 2 insertions(+), 5 deletions(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java b/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java index 12952afb582b..1fde3048d87f 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java @@ -141,10 +141,6 @@ public Path bucketPath(BinaryRow partition, int bucket) { return new Path(dataRoot, relativeBucketPath(partition, bucket)); } - // public Path dataBucketPath(BinaryRow partition, int bucket) { - // return new Path(dataRoot, relativeBucketPath(partition, bucket)); - // } - public Path relativeBucketPath(BinaryRow partition, int bucket) { Path relativeBucketPath = new Path(BUCKET_PATH_PREFIX + bucket); String partitionPath = getPartitionString(partition); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/FlinkSinkTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/FlinkSinkTest.java index 5f21858e61a5..5fe33061a6a5 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/FlinkSinkTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/FlinkSinkTest.java @@ -125,6 +125,7 @@ private FileStoreTable createFileStoreTable() throws Exception { tablePath, tableSchema, options, - CatalogEnvironment.empty()); + CatalogEnvironment.empty(), + tablePath); } } From 098d08ee8d2f3b9b28c03a6a019668d0add404e1 Mon Sep 17 00:00:00 2001 From: HouliangQi Date: Tue, 24 Dec 2024 21:17:16 +0800 Subject: [PATCH 03/21] fix tests --- .../java/org/apache/paimon/CoreOptions.java | 4 ---- .../org/apache/paimon/TestAppendFileStore.java | 1 + .../java/org/apache/paimon/TestFileStore.java | 1 + .../apache/paimon/flink/FlinkCatalogTest.java | 18 +++++++++++------- 4 files changed, 13 insertions(+), 11 deletions(-) diff --git a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java index de08a6c9c27d..7e10f1eb9989 100644 --- a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java +++ b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java @@ -1597,10 +1597,6 @@ public static Path dataPath(Map options) { return new Path(options.get(TABLE_DATA_PATH.key())); } - public static Path dataPath(Options options) { - return new Path(options.get(TABLE_DATA_PATH)); - } - public TableType type() { return options.get(TYPE); } diff --git a/paimon-core/src/test/java/org/apache/paimon/TestAppendFileStore.java b/paimon-core/src/test/java/org/apache/paimon/TestAppendFileStore.java index e6db51589408..c45fa7ae7139 100644 --- a/paimon-core/src/test/java/org/apache/paimon/TestAppendFileStore.java +++ b/paimon-core/src/test/java/org/apache/paimon/TestAppendFileStore.java @@ -156,6 +156,7 @@ public static TestAppendFileStore createAppendStore( SchemaManager schemaManage = new SchemaManager(new LocalFileIO(), path); options.put(CoreOptions.PATH.key(), root); + options.put(CoreOptions.TABLE_DATA_PATH.key(), root); TableSchema tableSchema = SchemaUtils.forceCommit( schemaManage, diff --git a/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java b/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java index 0d8ea5f4a49a..1a5b68e11fb1 100644 --- a/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java +++ b/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java @@ -798,6 +798,7 @@ public TestFileStore build() { conf.set(CoreOptions.FILE_FORMAT, format); conf.set(CoreOptions.MANIFEST_FORMAT, format); conf.set(CoreOptions.PATH, root); + conf.set(CoreOptions.TABLE_DATA_PATH, root); conf.set(CoreOptions.BUCKET, numBuckets); conf.set(CoreOptions.CHANGELOG_PRODUCER, changelogProducer); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java index 4b8cf7912192..a79ab8ab3929 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java @@ -92,6 +92,7 @@ import static org.apache.flink.table.factories.FactoryUtil.CONNECTOR; import static org.apache.paimon.CoreOptions.PATH; import static org.apache.paimon.CoreOptions.SCAN_FILE_CREATION_TIME_MILLIS; +import static org.apache.paimon.CoreOptions.TABLE_DATA_PATH; import static org.apache.paimon.flink.FlinkCatalogOptions.DISABLE_CREATE_TABLE_IN_DEFAULT_DB; import static org.apache.paimon.flink.FlinkCatalogOptions.LOG_SYSTEM_AUTO_REGISTER; import static org.apache.paimon.flink.FlinkConnectorOptions.LOG_SYSTEM; @@ -709,6 +710,7 @@ public void testCreateTableWithColumnOptions() throws Exception { Map expected = got.getOptions(); expected.remove("path"); + expected.remove("table.data.path"); expected.remove(FlinkCatalogOptions.REGISTER_TIMEOUT.key()); assertThat(catalogTable.getOptions()).isEqualTo(expected); } @@ -892,19 +894,21 @@ private void checkEquals( Map optionsToAdd, Set optionsToRemove) { Path tablePath; + Path tableDataPath; try { - tablePath = - new Path( - ((FlinkCatalog) catalog) - .catalog() - .getTable(FlinkCatalog.toIdentifier(path)) - .options() - .get(PATH.key())); + Map options = + ((FlinkCatalog) catalog) + .catalog() + .getTable(FlinkCatalog.toIdentifier(path)) + .options(); + tablePath = new Path(options.get(PATH.key())); + tableDataPath = new Path(options.get(TABLE_DATA_PATH.key())); } catch (org.apache.paimon.catalog.Catalog.TableNotExistException e) { throw new RuntimeException(e); } Map options = new HashMap<>(t1.getOptions()); options.put("path", tablePath.toString()); + options.put("table.data.path", tableDataPath.toString()); options.putAll(optionsToAdd); optionsToRemove.forEach(options::remove); if (t1.getTableKind() == CatalogBaseTable.TableKind.TABLE) { From 9f8ecd2023ed012f07339f976c170c29bea6f268 Mon Sep 17 00:00:00 2001 From: HouliangQi Date: Tue, 24 Dec 2024 22:26:16 +0800 Subject: [PATCH 04/21] merge master --- .../src/test/java/org/apache/paimon/rest/MockRESTMessage.java | 1 + 1 file changed, 1 insertion(+) diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTMessage.java b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTMessage.java index 3e9f32ba08e1..575adf179c3f 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTMessage.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/MockRESTMessage.java @@ -214,6 +214,7 @@ private static TableSchema tableSchema() { options.put("option-2", "value-2"); // set path for test as if not set system will add one options.put(CoreOptions.PATH.key(), "/a/b/c"); + options.put(CoreOptions.TABLE_DATA_PATH.key(), "/a/b/c"); return new TableSchema(1, fields, 1, partitionKeys, primaryKeys, options, "comment"); } } From 3a00acc2f2b316d70ab0df7ac02c38bc902bdd7a Mon Sep 17 00:00:00 2001 From: HouliangQi Date: Wed, 25 Dec 2024 10:17:45 +0800 Subject: [PATCH 05/21] fix the clone tests --- .../org/apache/paimon/io/DataFileMeta.java | 22 +++++ .../apache/paimon/manifest/ManifestFile.java | 7 +- .../paimon/flink/clone/CloneFileInfo.java | 17 +++- .../paimon/flink/clone/CopyFileOperator.java | 63 ++++++++++---- .../apache/paimon/flink/clone/FileType.java | 32 +++++++ .../clone/PickFilesForCloneOperator.java | 21 +++-- .../paimon/flink/clone/PickFilesUtil.java | 87 +++++++++++-------- .../flink/action/CloneActionITCase.java | 14 ++- 8 files changed, 195 insertions(+), 68 deletions(-) create mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/FileType.java diff --git a/paimon-core/src/main/java/org/apache/paimon/io/DataFileMeta.java b/paimon-core/src/main/java/org/apache/paimon/io/DataFileMeta.java index b164b60fe525..176f49a6091a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/DataFileMeta.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/DataFileMeta.java @@ -484,6 +484,28 @@ public DataFileMeta copy(List newExtraFiles) { externalPath); } + public DataFileMeta copy(String newExternalPath) { + return new DataFileMeta( + fileName, + fileSize, + rowCount, + minKey, + maxKey, + keyStats, + valueStats, + minSequenceNumber, + maxSequenceNumber, + schemaId, + level, + extraFiles, + creationTime, + deleteRowCount, + embeddedIndex, + fileSource, + valueStatsCols, + newExternalPath); + } + public DataFileMeta copy(byte[] newEmbeddedIndex) { return new DataFileMeta( fileName, diff --git a/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestFile.java b/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestFile.java index 1aba2ef19561..03aa8a422411 100644 --- a/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestFile.java +++ b/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestFile.java @@ -116,7 +116,12 @@ public RollingFileWriter createRollingWriter() suggestedFileSize); } - private class ManifestEntryWriter extends SingleFileWriter { + public ManifestEntryWriter createManifestEntryWriter(Path manifestPath) { + return new ManifestEntryWriter(writerFactory, manifestPath, compression); + } + + /** Writer for {@link ManifestEntry}. */ + public class ManifestEntryWriter extends SingleFileWriter { private final SimpleStatsCollector partitionStatsCollector; private final SimpleStatsConverter partitionStatsSerializer; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CloneFileInfo.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CloneFileInfo.java index 5c0ac75e167f..56ae6c8300e2 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CloneFileInfo.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CloneFileInfo.java @@ -25,16 +25,19 @@ public class CloneFileInfo { private final String filePathExcludeTableRoot; private final String sourceIdentifier; private final String targetIdentifier; + private final FileType fileType; public CloneFileInfo( String sourceFilePath, String filePathExcludeTableRoot, String sourceIdentifier, - String targetIdentifier) { + String targetIdentifier, + FileType fileType) { this.sourceFilePath = sourceFilePath; this.filePathExcludeTableRoot = filePathExcludeTableRoot; this.sourceIdentifier = sourceIdentifier; this.targetIdentifier = targetIdentifier; + this.fileType = fileType; } public String getSourceFilePath() { @@ -53,10 +56,18 @@ public String getTargetIdentifier() { return targetIdentifier; } + public FileType getFileType() { + return fileType; + } + @Override public String toString() { return String.format( - "{ sourceFilePath: %s, filePathExcludeTableRoot: %s, sourceIdentifier: %s, targetIdentifier: %s }", - sourceFilePath, filePathExcludeTableRoot, sourceIdentifier, targetIdentifier); + "{ sourceFilePath: %s, filePathExcludeTableRoot: %s, sourceIdentifier: %s, targetIdentifier: %s, fileType: %s }", + sourceFilePath, + filePathExcludeTableRoot, + sourceIdentifier, + targetIdentifier, + fileType); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CopyFileOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CopyFileOperator.java index e7002cce1eec..674e9c6fff08 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CopyFileOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CopyFileOperator.java @@ -19,12 +19,18 @@ package org.apache.paimon.flink.clone; import org.apache.paimon.CoreOptions; +import org.apache.paimon.FileStore; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.catalog.Catalog.TableNotExistException; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.FlinkCatalogFactory; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; +import org.apache.paimon.manifest.ManifestEntry; +import org.apache.paimon.manifest.ManifestFile; +import org.apache.paimon.manifest.ManifestFile.ManifestEntryWriter; import org.apache.paimon.options.Options; +import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; import org.apache.paimon.utils.IOUtils; @@ -34,7 +40,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; +import java.util.ArrayList; import java.util.HashMap; +import java.util.List; import java.util.Map; /** A Operator to copy files. */ @@ -74,18 +83,6 @@ public void processElement(StreamRecord streamRecord) throws Exce FileIO sourceTableFileIO = sourceCatalog.fileIO(); FileIO targetTableFileIO = targetCatalog.fileIO(); - - Path sourceTableRootPath = - srcLocations.computeIfAbsent( - cloneFileInfo.getSourceIdentifier(), - key -> { - try { - return pathOfTable( - sourceCatalog.getTable(Identifier.fromString(key))); - } catch (Catalog.TableNotExistException e) { - throw new RuntimeException(e); - } - }); Path targetTableRootPath = targetLocations.computeIfAbsent( cloneFileInfo.getTargetIdentifier(), @@ -125,9 +122,14 @@ public void processElement(StreamRecord streamRecord) throws Exce if (LOG.isDebugEnabled()) { LOG.debug("Begin copy file from {} to {}.", sourcePath, targetPath); } - IOUtils.copyBytes( - sourceTableFileIO.newInputStream(sourcePath), - targetTableFileIO.newOutputStream(targetPath, true)); + + if (cloneFileInfo.getFileType() == FileType.MANIFEST_FILE) { + copyManifestFile(sourcePath, targetPath, cloneFileInfo); + } else { + IOUtils.copyBytes( + sourceTableFileIO.newInputStream(sourcePath), + targetTableFileIO.newOutputStream(targetPath, true)); + } if (LOG.isDebugEnabled()) { LOG.debug("End copy file from {} to {}.", sourcePath, targetPath); } @@ -139,6 +141,37 @@ private Path pathOfTable(Table table) { return new Path(table.options().get(CoreOptions.PATH.key())); } + private void copyManifestFile(Path sourcePath, Path targetPath, CloneFileInfo cloneFileInfo) + throws IOException, TableNotExistException { + Identifier sourceIdentifier = Identifier.fromString(cloneFileInfo.getSourceIdentifier()); + FileStoreTable sourceTable = (FileStoreTable) sourceCatalog.getTable(sourceIdentifier); + FileStore store = sourceTable.store(); + ManifestFile manifestFile = store.manifestFileFactory().create(); + + List manifestEntries = + manifestFile.readWithIOException(sourcePath.getName()); + List targetManifestEntries = new ArrayList<>(manifestEntries.size()); + + // clone job will clone the source path to target warehouse path, so the target external + // path is null + String newExternalPath = null; + for (ManifestEntry manifestEntry : manifestEntries) { + ManifestEntry newManifestEntry = + new ManifestEntry( + manifestEntry.kind(), + manifestEntry.partition(), + manifestEntry.bucket(), + manifestEntry.totalBuckets(), + manifestEntry.file().copy(newExternalPath)); + targetManifestEntries.add(newManifestEntry); + } + + ManifestEntryWriter manifestEntryWriter = + manifestFile.createManifestEntryWriter(targetPath); + manifestEntryWriter.write(targetManifestEntries); + manifestEntryWriter.close(); + } + @Override public void close() throws Exception { if (sourceCatalog != null) { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/FileType.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/FileType.java new file mode 100644 index 000000000000..f6a92c74c3f7 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/FileType.java @@ -0,0 +1,32 @@ +/* + * 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.flink.clone; + +/** The clone file type. */ +public enum FileType { + MANIFEST_FILE, + MANIFEST_LIST_FILE, + INDEX_FILE, + DATA_FILE, + SNAPSHOT_FILE, + SCHEMA_FILE, + CHANGELOG_MANIFEST_LIST_FILE, + STATISTICS_FILE, + OTHER_FILE +} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/PickFilesForCloneOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/PickFilesForCloneOperator.java index f58d3acafdb9..70695541f0e0 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/PickFilesForCloneOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/PickFilesForCloneOperator.java @@ -115,19 +115,22 @@ private static Schema newSchemaFromTableSchema(TableSchema tableSchema) { } private List toCloneFileInfos( - List files, + Map> filesMap, Path sourceTableRoot, String sourceIdentifier, String targetIdentifier) { List result = new ArrayList<>(); - for (Path file : files) { - Path relativePath = getPathExcludeTableRoot(file, sourceTableRoot); - result.add( - new CloneFileInfo( - file.toUri().toString(), - relativePath.toString(), - sourceIdentifier, - targetIdentifier)); + for (Map.Entry> entry : filesMap.entrySet()) { + for (Path file : entry.getValue()) { + Path relativePath = getPathExcludeTableRoot(file, sourceTableRoot); + result.add( + new CloneFileInfo( + file.toUri().toString(), + relativePath.toString(), + sourceIdentifier, + targetIdentifier, + entry.getKey())); + } } return result; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/PickFilesUtil.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/PickFilesUtil.java index 9de974d047f1..d3a5f8debf0e 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/PickFilesUtil.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/PickFilesUtil.java @@ -38,7 +38,9 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -48,7 +50,7 @@ public class PickFilesUtil { private static final int READ_FILE_RETRY_NUM = 3; private static final int READ_FILE_RETRY_INTERVAL = 5; - public static List getUsedFilesForLatestSnapshot(FileStoreTable table) { + public static Map> getUsedFilesForLatestSnapshot(FileStoreTable table) { FileStore store = table.store(); SnapshotManager snapshotManager = store.snapshotManager(); Snapshot snapshot = snapshotManager.latestSnapshot(); @@ -56,31 +58,33 @@ public static List getUsedFilesForLatestSnapshot(FileStoreTable table) { SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.location()); IndexFileHandler indexFileHandler = store.newIndexFileHandler(); - List files = new ArrayList<>(); + Map> filesMap = new HashMap<>(); if (snapshot != null) { - files.add(snapshotManager.snapshotPath(snapshot.id())); - files.addAll( - getUsedFilesInternal( - snapshot, - store.pathFactory(), - store.newScan(), - manifestList, - indexFileHandler)); + filesMap.computeIfAbsent(FileType.SNAPSHOT_FILE, k -> new ArrayList<>()) + .add(snapshotManager.snapshotPath(snapshot.id())); + getUsedFilesInternal( + snapshot, + store.pathFactory(), + store.newScan(), + manifestList, + indexFileHandler, + filesMap); } for (long id : schemaManager.listAllIds()) { - files.add(schemaManager.toSchemaPath(id)); + filesMap.computeIfAbsent(FileType.SCHEMA_FILE, k -> new ArrayList<>()) + .add(schemaManager.toSchemaPath(id)); } - return files; + return filesMap; } - private static List getUsedFilesInternal( + private static void getUsedFilesInternal( Snapshot snapshot, FileStorePathFactory pathFactory, FileStoreScan scan, ManifestList manifestList, - IndexFileHandler indexFileHandler) { - List files = new ArrayList<>(); - addManifestList(files, snapshot, pathFactory); + IndexFileHandler indexFileHandler, + Map> filesMap) { + addManifestList(filesMap, snapshot, pathFactory); try { // try to read manifests @@ -88,16 +92,17 @@ private static List getUsedFilesInternal( retryReadingFiles( () -> readAllManifestsWithIOException(snapshot, manifestList)); if (manifestFileMetas == null) { - return Collections.emptyList(); + return; } List manifestFileName = manifestFileMetas.stream() .map(ManifestFileMeta::fileName) .collect(Collectors.toList()); - files.addAll( - manifestFileName.stream() - .map(pathFactory::toManifestFilePath) - .collect(Collectors.toList())); + filesMap.computeIfAbsent(FileType.MANIFEST_FILE, k -> new ArrayList<>()) + .addAll( + manifestFileName.stream() + .map(pathFactory::toManifestFilePath) + .collect(Collectors.toList())); // try to read data files List dataFiles = new ArrayList<>(); @@ -119,44 +124,52 @@ private static List getUsedFilesInternal( // deleted. Older files however, are from previous partitions and should not be changed // very often. Collections.reverse(dataFiles); - files.addAll(dataFiles); + filesMap.computeIfAbsent(FileType.DATA_FILE, k -> new ArrayList<>()).addAll(dataFiles); // try to read index files String indexManifest = snapshot.indexManifest(); if (indexManifest != null && indexFileHandler.existsManifest(indexManifest)) { - files.add(pathFactory.indexManifestFileFactory().toPath(indexManifest)); + filesMap.computeIfAbsent(FileType.INDEX_FILE, k -> new ArrayList<>()) + .add(pathFactory.indexManifestFileFactory().toPath(indexManifest)); List indexManifestEntries = retryReadingFiles( () -> indexFileHandler.readManifestWithIOException(indexManifest)); - if (indexManifestEntries == null) { - return Collections.emptyList(); + if (indexManifestEntries != null) { + indexManifestEntries.stream() + .map(IndexManifestEntry::indexFile) + .map(indexFileHandler::filePath) + .forEach( + filePath -> + filesMap.computeIfAbsent( + FileType.INDEX_FILE, + k -> new ArrayList<>()) + .add(filePath)); } - - indexManifestEntries.stream() - .map(IndexManifestEntry::indexFile) - .map(indexFileHandler::filePath) - .forEach(files::add); } // add statistic file if (snapshot.statistics() != null) { - files.add(pathFactory.statsFileFactory().toPath(snapshot.statistics())); + filesMap.computeIfAbsent(FileType.STATISTICS_FILE, k -> new ArrayList<>()) + .add(pathFactory.statsFileFactory().toPath(snapshot.statistics())); } } catch (IOException e) { throw new RuntimeException(e); } - - return files; } private static void addManifestList( - List used, Snapshot snapshot, FileStorePathFactory pathFactory) { - used.add(pathFactory.toManifestListPath(snapshot.baseManifestList())); - used.add(pathFactory.toManifestListPath(snapshot.deltaManifestList())); + Map> filesMap, + Snapshot snapshot, + FileStorePathFactory pathFactory) { + filesMap.computeIfAbsent(FileType.MANIFEST_LIST_FILE, k -> new ArrayList<>()) + .add(pathFactory.toManifestListPath(snapshot.baseManifestList())); + filesMap.get(FileType.MANIFEST_LIST_FILE) + .add(pathFactory.toManifestListPath(snapshot.deltaManifestList())); String changelogManifestList = snapshot.changelogManifestList(); if (changelogManifestList != null) { - used.add(pathFactory.toManifestListPath(changelogManifestList)); + filesMap.computeIfAbsent(FileType.CHANGELOG_MANIFEST_LIST_FILE, k -> new ArrayList<>()) + .add(pathFactory.toManifestListPath(changelogManifestList)); } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CloneActionITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CloneActionITCase.java index a55b01cc203b..b177f98024d9 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CloneActionITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CloneActionITCase.java @@ -22,6 +22,7 @@ import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.CatalogFactory; import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.flink.clone.FileType; import org.apache.paimon.flink.clone.PickFilesUtil; import org.apache.paimon.fs.Path; import org.apache.paimon.table.FileStoreTable; @@ -40,6 +41,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; @@ -457,7 +459,10 @@ private void compareCloneFiles( String targetTableName) throws Exception { FileStoreTable targetTable = getFileStoreTable(targetWarehouse, targetDb, targetTableName); - List targetTableFiles = PickFilesUtil.getUsedFilesForLatestSnapshot(targetTable); + Map> filesMap = + PickFilesUtil.getUsedFilesForLatestSnapshot(targetTable); + List targetTableFiles = + filesMap.values().stream().flatMap(List::stream).collect(Collectors.toList()); List> filesPathInfoList = targetTableFiles.stream() .map( @@ -473,8 +478,11 @@ private void compareCloneFiles( for (Pair filesPathInfo : filesPathInfoList) { Path sourceTableFile = new Path(tableLocation.toString() + filesPathInfo.getRight()); assertThat(sourceTable.fileIO().exists(sourceTableFile)).isTrue(); - assertThat(targetTable.fileIO().getFileSize(filesPathInfo.getLeft())) - .isEqualTo(sourceTable.fileIO().getFileSize(sourceTableFile)); + // TODO, need to check the manifest file's content + if (!filesPathInfo.getLeft().toString().contains("/manifest/manifest-")) { + assertThat(targetTable.fileIO().getFileSize(filesPathInfo.getLeft())) + .isEqualTo(sourceTable.fileIO().getFileSize(sourceTableFile)); + } } } From e323a6384ed926f2ce0e00803219ecc119af4f2f Mon Sep 17 00:00:00 2001 From: HouliangQi Date: Fri, 27 Dec 2024 10:26:34 +0800 Subject: [PATCH 06/21] fix review comments --- .../src/main/java/org/apache/paimon/CoreOptions.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java index 7e10f1eb9989..dcbac5635bf7 100644 --- a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java +++ b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java @@ -2390,10 +2390,6 @@ public boolean statsDenseStore() { return options.get(METADATA_STATS_DENSE_STORE); } - public String dataFileExternalPath() { - return options.get(DATA_FILE_EXTERNAL_PATH); - } - public boolean dataFileThinMode() { return options.get(DATA_FILE_THIN_MODE); } From 9965db38af75e180ce12753fb61a5d295ce5d102 Mon Sep 17 00:00:00 2001 From: HouliangQi Date: Fri, 27 Dec 2024 11:47:53 +0800 Subject: [PATCH 07/21] fix review comments --- .../java/org/apache/paimon/flink/clone/CopyFileOperator.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CopyFileOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CopyFileOperator.java index 674e9c6fff08..0fee72935553 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CopyFileOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CopyFileOperator.java @@ -154,7 +154,6 @@ private void copyManifestFile(Path sourcePath, Path targetPath, CloneFileInfo cl // clone job will clone the source path to target warehouse path, so the target external // path is null - String newExternalPath = null; for (ManifestEntry manifestEntry : manifestEntries) { ManifestEntry newManifestEntry = new ManifestEntry( @@ -162,7 +161,7 @@ private void copyManifestFile(Path sourcePath, Path targetPath, CloneFileInfo cl manifestEntry.partition(), manifestEntry.bucket(), manifestEntry.totalBuckets(), - manifestEntry.file().copy(newExternalPath)); + manifestEntry.file().copy((String) null)); targetManifestEntries.add(newManifestEntry); } From a1f2af0889505ff5cea07efce32fc214a48fdfd2 Mon Sep 17 00:00:00 2001 From: HouliangQi Date: Thu, 2 Jan 2025 09:22:21 +0800 Subject: [PATCH 08/21] support external-paths.strategy --- .../java/org/apache/paimon/CoreOptions.java | 116 ++++++++++++--- .../paimon/fs/ExternalPathProvider.java | 138 ++++++++++++++++++ .../org/apache/paimon/AbstractFileStore.java | 8 +- .../apache/paimon/AppendOnlyFileStore.java | 14 +- .../org/apache/paimon/KeyValueFileStore.java | 14 +- .../apache/paimon/io/DataFilePathFactory.java | 18 ++- .../privilege/PrivilegedFileStoreTable.java | 6 + .../paimon/table/AbstractFileStoreTable.java | 40 +++-- .../table/AppendOnlyFileStoreTable.java | 15 +- .../org/apache/paimon/table/DataTable.java | 3 +- .../paimon/table/DelegatedFileStoreTable.java | 5 - .../table/FallbackReadFileStoreTable.java | 8 +- .../paimon/table/FileStoreTableFactory.java | 57 ++++++-- .../table/PrimaryKeyFileStoreTable.java | 15 +- .../paimon/table/object/ObjectTable.java | 6 + .../paimon/table/system/AuditLogTable.java | 5 +- .../table/system/CompactBucketsTable.java | 5 +- .../paimon/table/system/FileMonitorTable.java | 5 +- .../table/system/ReadOptimizedTable.java | 5 +- .../paimon/utils/FileStorePathFactory.java | 20 ++- .../apache/paimon/TestAppendFileStore.java | 5 +- .../java/org/apache/paimon/TestFileStore.java | 5 +- .../paimon/append/AppendOnlyWriterTest.java | 4 +- .../paimon/format/FileFormatSuffixTest.java | 4 +- .../paimon/io/DataFilePathFactoryTest.java | 7 +- .../paimon/io/KeyValueFileReadWriteTest.java | 5 +- .../paimon/io/RollingFileWriterTest.java | 4 +- .../manifest/ManifestFileMetaTestBase.java | 3 +- .../paimon/manifest/ManifestFileTest.java | 3 +- .../paimon/manifest/ManifestListTest.java | 3 +- .../apache/paimon/rest/RESTCatalogTest.java | 4 +- .../source/snapshot/ScannerTestBase.java | 8 +- .../source/snapshot/SnapshotReaderTest.java | 8 +- .../utils/FileStorePathFactoryTest.java | 5 +- .../flink/lookup/LookupFileStoreTable.java | 6 + .../apache/paimon/flink/FlinkCatalogTest.java | 3 - .../paimon/flink/sink/FlinkSinkTest.java | 3 +- .../source/TestChangelogDataReadWrite.java | 3 +- .../paimon/spark/SparkFileIndexITCase.java | 3 +- 39 files changed, 469 insertions(+), 120 deletions(-) create mode 100644 paimon-common/src/main/java/org/apache/paimon/fs/ExternalPathProvider.java diff --git a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java index dcbac5635bf7..3d8403a0b05e 100644 --- a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java +++ b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java @@ -130,6 +130,31 @@ public class CoreOptions implements Serializable { .noDefaultValue() .withDescription("The path where the data of this table is currently written."); + public static final ConfigOption DATA_FILE_EXTERNAL_PATHS = + key("data-file.external-paths") + .stringType() + .noDefaultValue() + .withDescription( + "The external paths where the data of this table will be written, " + + "multiple elements separated by commas."); + + public static final ConfigOption DATA_FILE_EXTERNAL_PATHS_STRATEGY = + key("data-file.external-paths.strategy") + .enumType(ExternalPathStrategy.class) + .defaultValue(ExternalPathStrategy.NONE) + .withDescription( + "The strategy of selecting an external path when writing data"); + + public static final ConfigOption DATA_FILE_EXTERNAL_PATHS_SPECIFIC_FS = + key("data-file.external-paths.specific-fs") + .enumType(ExternalFSStrategy.class) + .noDefaultValue() + .withDescription( + "The specific file system of the external path when " + + DATA_FILE_EXTERNAL_PATHS_STRATEGY.key() + + " is set to " + + ExternalPathStrategy.SPECIFIC_FS); + // todo, this path is the table schema path, the name will be changed in the later PR. @ExcludeFromDocumentation("Internal use only") public static final ConfigOption PATH = @@ -138,17 +163,6 @@ public class CoreOptions implements Serializable { .noDefaultValue() .withDescription("The file path of this table in the filesystem."); - @ExcludeFromDocumentation("Internal use only") - public static final ConfigOption TABLE_DATA_PATH = - key("table.data.path") - .stringType() - .noDefaultValue() - .withDescription( - "The data file path of this table in the filesystem. if " - + DATA_FILE_EXTERNAL_PATH.key() - + "is not set, it will be same with." - + PATH.key()); - public static final ConfigOption BRANCH = key("branch").stringType().defaultValue("main").withDescription("Specify branch name."); @@ -1570,10 +1584,6 @@ public Path path() { return path(options.toMap()); } - public Path dataPath() { - return dataPath(options.toMap()); - } - public String branch() { return branch(options.toMap()); } @@ -1593,10 +1603,6 @@ public static Path path(Options options) { return new Path(options.get(PATH)); } - public static Path dataPath(Map options) { - return new Path(options.get(TABLE_DATA_PATH.key())); - } - public TableType type() { return options.get(TYPE); } @@ -2197,6 +2203,18 @@ public PartitionExpireStrategy partitionExpireStrategy() { return options.get(PARTITION_EXPIRATION_STRATEGY); } + public String dataFileExternalPaths() { + return options.get(DATA_FILE_EXTERNAL_PATHS); + } + + public ExternalPathStrategy externalPathStrategy() { + return options.get(DATA_FILE_EXTERNAL_PATHS_STRATEGY); + } + + public ExternalFSStrategy externalSpecificFSStrategy() { + return options.get(DATA_FILE_EXTERNAL_PATHS_SPECIFIC_FS); + } + public String partitionTimestampFormatter() { return options.get(PARTITION_TIMESTAMP_FORMATTER); } @@ -3000,6 +3018,66 @@ public InlineElement getDescription() { } } + /** Specifies the strategy for selecting external storage paths. */ + public enum ExternalPathStrategy implements DescribedEnum { + NONE( + "none", + "Do not choose any external storage, data will still be written to the default warehouse path."), + + SPECIFIC_FS( + "specific-fs", + "Select a specific file system as the external path. Currently supported are S3 and OSS."), + + ROUND_ROBIN( + "round-robin", + "When writing a new file, a path is chosen from data-file.external-paths in turn."); + + private final String value; + + private final String description; + + ExternalPathStrategy(String value, String description) { + this.value = value; + this.description = description; + } + + @Override + public String toString() { + return value; + } + + @Override + public InlineElement getDescription() { + return text(description); + } + } + + /** Specifies the strategy for selecting specific filesystem storage paths. */ + public enum ExternalFSStrategy implements DescribedEnum { + S3("S3", "Select S3 as the write path for the external path."), + + OSS("OSS", "Select OSS as the write path for the external path."); + + private final String value; + + private final String description; + + ExternalFSStrategy(String value, String description) { + this.value = value; + this.description = description; + } + + @Override + public String toString() { + return value; + } + + @Override + public InlineElement getDescription() { + return text(description); + } + } + /** Specifies the local file type for lookup. */ public enum LookupLocalFileType implements DescribedEnum { SORT("sort", "Construct a sorted file for lookup."), diff --git a/paimon-common/src/main/java/org/apache/paimon/fs/ExternalPathProvider.java b/paimon-common/src/main/java/org/apache/paimon/fs/ExternalPathProvider.java new file mode 100644 index 000000000000..8dc1484566aa --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/fs/ExternalPathProvider.java @@ -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.CoreOptions.ExternalFSStrategy; +import org.apache.paimon.CoreOptions.ExternalPathStrategy; +import org.apache.paimon.annotation.VisibleForTesting; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +/** Provider for external paths. */ +public class ExternalPathProvider { + private final Map externalPathsMap; + private final List externalPathsList; + + private final ExternalPathStrategy externalPathStrategy; + private final ExternalFSStrategy externalFSStrategy; + private int currentIndex; + private boolean externalPathExists; + private final String dbAndTableRelativePath; + + @VisibleForTesting + public ExternalPathProvider() { + this.externalPathsMap = new HashMap<>(); + this.externalPathsList = new ArrayList<>(); + this.externalPathStrategy = ExternalPathStrategy.NONE; + this.externalFSStrategy = null; + this.dbAndTableRelativePath = null; + } + + public ExternalPathProvider( + String externalPaths, + ExternalPathStrategy externalPathStrategy, + ExternalFSStrategy externalFSStrategy, + String dbAndTableRelativePath) { + this.externalPathsMap = new HashMap<>(); + this.externalPathsList = new ArrayList<>(); + this.externalPathStrategy = externalPathStrategy; + this.externalFSStrategy = externalFSStrategy; + this.dbAndTableRelativePath = dbAndTableRelativePath; + this.currentIndex = 0; + initExternalPaths(externalPaths); + } + + private void initExternalPaths(String externalPaths) { + if (externalPaths == null) { + return; + } + String[] tmpArray = externalPaths.split(","); + for (String part : tmpArray) { + String path = part.trim(); + if (path.toLowerCase().startsWith("oss")) { + externalPathsMap.put(ExternalFSStrategy.OSS, new Path(path)); + externalPathsList.add(new Path(path)); + } else if (path.toLowerCase().startsWith("s3")) { + externalPathsMap.put(ExternalFSStrategy.S3, new Path(path)); + externalPathsList.add(new Path(path)); + } else { + throw new IllegalArgumentException("Unsupported external path: " + path); + } + } + + if (!externalPathsMap.isEmpty() + && !externalPathsList.isEmpty() + && externalPathStrategy != ExternalPathStrategy.NONE) { + externalPathExists = true; + } + } + + /** + * Get the next external path. + * + * @return the next external path + */ + public Optional getNextExternalPath() { + if (externalPathsMap == null || externalPathsMap.isEmpty()) { + return Optional.empty(); + } + + switch (externalPathStrategy) { + case NONE: + return Optional.empty(); + case SPECIFIC_FS: + return getSpecificFSExternalPath(); + case ROUND_ROBIN: + return getRoundRobinPath(); + default: + return Optional.empty(); + } + } + + private Optional getSpecificFSExternalPath() { + switch (externalFSStrategy) { + case S3: + return Optional.of( + new Path( + externalPathsMap.get(ExternalFSStrategy.S3), + dbAndTableRelativePath)); + case OSS: + return Optional.of( + new Path( + externalPathsMap.get(ExternalFSStrategy.OSS), + dbAndTableRelativePath)); + default: + throw new IllegalArgumentException( + "Unsupported external fs strategy: " + externalFSStrategy); + } + } + + private Optional getRoundRobinPath() { + currentIndex = (currentIndex + 1) % externalPathsList.size(); + return Optional.of(new Path(externalPathsList.get(currentIndex), dbAndTableRelativePath)); + } + + public boolean externalPathExists() { + return externalPathExists; + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java b/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java index ac9c8b46d802..1ee1d6a0cdfb 100644 --- a/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java +++ b/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java @@ -20,6 +20,7 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.deletionvectors.DeletionVectorsIndexFile; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.index.HashIndexFile; @@ -82,6 +83,7 @@ abstract class AbstractFileStore implements FileStore { @Nullable private final SegmentsCache writeManifestCache; @Nullable private SegmentsCache readManifestCache; @Nullable private Cache snapshotCache; + private final ExternalPathProvider externalPathProvider; protected AbstractFileStore( FileIO fileIO, @@ -90,7 +92,8 @@ protected AbstractFileStore( String tableName, CoreOptions options, RowType partitionType, - CatalogEnvironment catalogEnvironment) { + CatalogEnvironment catalogEnvironment, + ExternalPathProvider externalPathProvider) { this.fileIO = fileIO; this.schemaManager = schemaManager; this.schema = schema; @@ -101,6 +104,7 @@ protected AbstractFileStore( this.writeManifestCache = SegmentsCache.create( options.pageSize(), options.writeManifestCache(), Long.MAX_VALUE); + this.externalPathProvider = externalPathProvider; } @Override @@ -120,7 +124,7 @@ protected FileStorePathFactory pathFactory(String format) { options.fileSuffixIncludeCompression(), options.fileCompression(), options.dataFilePathDirectory(), - options.dataPath()); + externalPathProvider); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/AppendOnlyFileStore.java b/paimon-core/src/main/java/org/apache/paimon/AppendOnlyFileStore.java index a06b98d7b30c..e06ad35a5517 100644 --- a/paimon-core/src/main/java/org/apache/paimon/AppendOnlyFileStore.java +++ b/paimon-core/src/main/java/org/apache/paimon/AppendOnlyFileStore.java @@ -21,6 +21,7 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.deletionvectors.DeletionVectorsMaintainer; import org.apache.paimon.format.FileFormatDiscover; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.manifest.ManifestCacheFilter; import org.apache.paimon.operation.AppendOnlyFileStoreScan; @@ -59,8 +60,17 @@ public AppendOnlyFileStore( RowType bucketKeyType, RowType rowType, String tableName, - CatalogEnvironment catalogEnvironment) { - super(fileIO, schemaManager, schema, tableName, options, partitionType, catalogEnvironment); + CatalogEnvironment catalogEnvironment, + ExternalPathProvider externalPathProvider) { + super( + fileIO, + schemaManager, + schema, + tableName, + options, + partitionType, + catalogEnvironment, + externalPathProvider); this.bucketKeyType = bucketKeyType; this.rowType = rowType; } diff --git a/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java b/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java index 8cf45105c01b..f5626160b85e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java +++ b/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java @@ -22,6 +22,7 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.deletionvectors.DeletionVectorsMaintainer; import org.apache.paimon.format.FileFormatDiscover; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.index.HashIndexMaintainer; import org.apache.paimon.index.IndexMaintainer; @@ -84,8 +85,17 @@ public KeyValueFileStore( KeyValueFieldsExtractor keyValueFieldsExtractor, MergeFunctionFactory mfFactory, String tableName, - CatalogEnvironment catalogEnvironment) { - super(fileIO, schemaManager, schema, tableName, options, partitionType, catalogEnvironment); + CatalogEnvironment catalogEnvironment, + ExternalPathProvider externalPathProvider) { + super( + fileIO, + schemaManager, + schema, + tableName, + options, + partitionType, + catalogEnvironment, + externalPathProvider); this.crossPartitionUpdate = crossPartitionUpdate; this.bucketKeyType = bucketKeyType; this.keyType = keyType; diff --git a/paimon-core/src/main/java/org/apache/paimon/io/DataFilePathFactory.java b/paimon-core/src/main/java/org/apache/paimon/io/DataFilePathFactory.java index 354d2d97a34a..88d413d88ba1 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/DataFilePathFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/DataFilePathFactory.java @@ -19,6 +19,7 @@ package org.apache.paimon.io; import org.apache.paimon.annotation.VisibleForTesting; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.FileEntry; @@ -35,6 +36,7 @@ public class DataFilePathFactory { public static final String INDEX_PATH_SUFFIX = ".index"; private final Path parent; + private final Path relativePath; private final String uuid; private final AtomicInteger pathCount; @@ -43,7 +45,7 @@ public class DataFilePathFactory { private final String changelogFilePrefix; private final boolean fileSuffixIncludeCompression; private final String fileCompression; - private final boolean isExternalPath; + private final ExternalPathProvider externalPathProvider; public DataFilePathFactory( Path parent, @@ -52,7 +54,8 @@ public DataFilePathFactory( String changelogFilePrefix, boolean fileSuffixIncludeCompression, String fileCompression, - boolean isExternalPath) { + ExternalPathProvider externalPathProvider, + Path relativePath) { this.parent = parent; this.uuid = UUID.randomUUID().toString(); this.pathCount = new AtomicInteger(0); @@ -61,7 +64,8 @@ public DataFilePathFactory( this.changelogFilePrefix = changelogFilePrefix; this.fileSuffixIncludeCompression = fileSuffixIncludeCompression; this.fileCompression = fileCompression; - this.isExternalPath = isExternalPath; + this.externalPathProvider = externalPathProvider; + this.relativePath = relativePath; } public Path newPath() { @@ -77,7 +81,13 @@ public String newChangelogFileName() { } public Path newPath(String prefix) { - return new Path(parent, newFileName(prefix)).setExternalPath(isExternalPath); + return externalPathProvider + .getNextExternalPath() + .map( + externalPath -> + new Path(new Path(externalPath, relativePath), newFileName(prefix))) + .orElse(new Path(parent, newFileName(prefix))) + .setExternalPath(externalPathProvider.externalPathExists()); } private String newFileName(String prefix) { diff --git a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedFileStoreTable.java index 52c806c7c53b..c1f9ee5b85de 100644 --- a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedFileStoreTable.java @@ -21,6 +21,7 @@ import org.apache.paimon.FileStore; import org.apache.paimon.Snapshot; import org.apache.paimon.catalog.Identifier; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.manifest.ManifestCacheFilter; import org.apache.paimon.schema.TableSchema; import org.apache.paimon.stats.Statistics; @@ -287,6 +288,11 @@ public PrivilegedFileStoreTable switchToBranch(String branchName) { wrapped.switchToBranch(branchName), privilegeChecker, identifier); } + @Override + public ExternalPathProvider externalPathProvider() { + return wrapped.externalPathProvider(); + } + public static PrivilegedFileStoreTable wrap( FileStoreTable table, PrivilegeChecker privilegeChecker, Identifier identifier) { if (table instanceof ObjectTable) { diff --git a/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java index 25e14b4a74e7..65de00a556cf 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java @@ -22,6 +22,7 @@ import org.apache.paimon.Snapshot; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.consumer.ConsumerManager; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.IndexManifestEntry; @@ -87,7 +88,6 @@ import java.util.function.BiConsumer; import static org.apache.paimon.CoreOptions.PATH; -import static org.apache.paimon.CoreOptions.TABLE_DATA_PATH; import static org.apache.paimon.utils.Preconditions.checkArgument; /** Abstract {@link FileStoreTable}. */ @@ -99,7 +99,7 @@ abstract class AbstractFileStoreTable implements FileStoreTable { protected final FileIO fileIO; protected final Path path; - protected final Path tableDataPath; + protected final TableSchema tableSchema; protected final CatalogEnvironment catalogEnvironment; @@ -107,12 +107,14 @@ abstract class AbstractFileStoreTable implements FileStoreTable { @Nullable protected transient Cache snapshotCache; @Nullable protected transient Cache statsCache; + protected final ExternalPathProvider externalPathProvider; + protected AbstractFileStoreTable( FileIO fileIO, Path path, TableSchema tableSchema, CatalogEnvironment catalogEnvironment, - Path tableDataPath) { + ExternalPathProvider externalPathProvider) { this.fileIO = fileIO; this.path = path; if (!tableSchema.options().containsKey(PATH.key())) { @@ -122,15 +124,9 @@ protected AbstractFileStoreTable( tableSchema = tableSchema.copy(newOptions); } - if (!tableSchema.options().containsKey(TABLE_DATA_PATH.key())) { - Map newOptions = new HashMap<>(tableSchema.options()); - newOptions.put(TABLE_DATA_PATH.key(), tableDataPath.toString()); - tableSchema = tableSchema.copy(newOptions); - } - this.tableSchema = tableSchema; this.catalogEnvironment = catalogEnvironment; - this.tableDataPath = tableDataPath; + this.externalPathProvider = externalPathProvider; } public String currentBranch() { @@ -347,9 +343,6 @@ private FileStoreTable copyInternal(Map dynamicOptions, boolean // set path always newOptions.set(PATH, path.toString()); - // set tableDataPath always - newOptions.set(TABLE_DATA_PATH, tableDataPath.toString()); - // set dynamic options with default values CoreOptions.setDefaultValues(newOptions); @@ -386,9 +379,17 @@ public FileStoreTable copy(TableSchema newTableSchema) { AbstractFileStoreTable copied = newTableSchema.primaryKeys().isEmpty() ? new AppendOnlyFileStoreTable( - fileIO, path, newTableSchema, catalogEnvironment, tableDataPath) + fileIO, + path, + newTableSchema, + catalogEnvironment, + externalPathProvider) : new PrimaryKeyFileStoreTable( - fileIO, path, newTableSchema, catalogEnvironment, tableDataPath); + fileIO, + path, + newTableSchema, + catalogEnvironment, + externalPathProvider); if (snapshotCache != null) { copied.setSnapshotCache(snapshotCache); } @@ -421,11 +422,6 @@ public Path location() { return path; } - @Override - public Path dataLocation() { - return tableDataPath; - } - @Override public TableSchema schema() { return tableSchema; @@ -762,7 +758,7 @@ public FileStoreTable switchToBranch(String branchName) { branchSchema, new Options(), catalogEnvironment(), - tableDataPath); + externalPathProvider); } private RollbackHelper rollbackHelper() { @@ -790,6 +786,6 @@ public boolean equals(Object o) { AbstractFileStoreTable that = (AbstractFileStoreTable) o; return Objects.equals(path, that.path) && Objects.equals(tableSchema, that.tableSchema) - && Objects.equals(tableDataPath, that.tableDataPath); + && Objects.equals(externalPathProvider, that.externalPathProvider); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/AppendOnlyFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/AppendOnlyFileStoreTable.java index 1f0e395e1aeb..9d732effaf02 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/AppendOnlyFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/AppendOnlyFileStoreTable.java @@ -21,6 +21,7 @@ import org.apache.paimon.AppendOnlyFileStore; import org.apache.paimon.CoreOptions; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.iceberg.AppendOnlyIcebergCommitCallback; @@ -58,7 +59,7 @@ class AppendOnlyFileStoreTable extends AbstractFileStoreTable { private transient AppendOnlyFileStore lazyStore; AppendOnlyFileStoreTable(FileIO fileIO, Path path, TableSchema tableSchema) { - this(fileIO, path, tableSchema, CatalogEnvironment.empty(), path); + this(fileIO, path, tableSchema, CatalogEnvironment.empty(), new ExternalPathProvider()); } AppendOnlyFileStoreTable( @@ -66,8 +67,8 @@ class AppendOnlyFileStoreTable extends AbstractFileStoreTable { Path path, TableSchema tableSchema, CatalogEnvironment catalogEnvironment, - Path tableDataPath) { - super(fileIO, path, tableSchema, catalogEnvironment, tableDataPath); + ExternalPathProvider externalPathProvider) { + super(fileIO, path, tableSchema, catalogEnvironment, externalPathProvider); } @Override @@ -83,7 +84,8 @@ public AppendOnlyFileStore store() { tableSchema.logicalBucketKeyType(), tableSchema.logicalRowType(), name(), - catalogEnvironment); + catalogEnvironment, + externalPathProvider); } return lazyStore; } @@ -174,4 +176,9 @@ protected List createCommitCallbacks(String commitUser) { return callbacks; } + + @Override + public ExternalPathProvider externalPathProvider() { + return externalPathProvider; + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/DataTable.java b/paimon-core/src/main/java/org/apache/paimon/table/DataTable.java index b87660d7c865..f271912e33d2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/DataTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/DataTable.java @@ -19,6 +19,7 @@ package org.apache.paimon.table; import org.apache.paimon.CoreOptions; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.schema.SchemaManager; @@ -54,7 +55,7 @@ public interface DataTable extends InnerTable { Path location(); - Path dataLocation(); + ExternalPathProvider externalPathProvider(); FileIO fileIO(); } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/DelegatedFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/DelegatedFileStoreTable.java index a9cda655f371..0a548941bedc 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/DelegatedFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/DelegatedFileStoreTable.java @@ -115,11 +115,6 @@ public Path location() { return wrapped.location(); } - @Override - public Path dataLocation() { - return wrapped.dataLocation(); - } - @Override public FileIO fileIO() { return wrapped.fileIO(); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/FallbackReadFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/FallbackReadFileStoreTable.java index 999f6ca6297c..e5184b4af543 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/FallbackReadFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/FallbackReadFileStoreTable.java @@ -23,6 +23,7 @@ import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.disk.IOManager; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.manifest.PartitionEntry; import org.apache.paimon.metrics.MetricRegistry; import org.apache.paimon.options.Options; @@ -100,6 +101,11 @@ public FileStoreTable switchToBranch(String branchName) { return new FallbackReadFileStoreTable(switchWrappedToBranch(branchName), fallback); } + @Override + public ExternalPathProvider externalPathProvider() { + return wrapped.externalPathProvider(); + } + private FileStoreTable switchWrappedToBranch(String branchName) { Optional optionalSchema = wrapped.schemaManager().copyWithBranch(branchName).latest(); @@ -116,7 +122,7 @@ private FileStoreTable switchWrappedToBranch(String branchName) { branchSchema, new Options(), wrapped.catalogEnvironment(), - wrapped.dataLocation()); + wrapped.externalPathProvider()); } private Map rewriteFallbackOptions(Map options) { diff --git a/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTableFactory.java b/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTableFactory.java index f6fb08a73be0..534a4c1bf45b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTableFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTableFactory.java @@ -19,7 +19,10 @@ package org.apache.paimon.table; import org.apache.paimon.CoreOptions; +import org.apache.paimon.CoreOptions.ExternalFSStrategy; +import org.apache.paimon.CoreOptions.ExternalPathStrategy; import org.apache.paimon.catalog.CatalogContext; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.options.Options; @@ -66,9 +69,14 @@ public static FileStoreTable create(FileIO fileIO, Options options) { + tablePath + ". Please create table first.")); - Path tableDataPath = getTableDataPath(tableSchema, tablePath); + ExternalPathProvider externalPathProvider = getExternalPathProvider(tableSchema, tablePath); return create( - fileIO, tablePath, tableSchema, options, CatalogEnvironment.empty(), tableDataPath); + fileIO, + tablePath, + tableSchema, + options, + CatalogEnvironment.empty(), + externalPathProvider); } private static Path getTableDataPath(TableSchema tableSchema, Path tablePath) { @@ -80,15 +88,26 @@ private static Path getTableDataPath(TableSchema tableSchema, Path tablePath) { return new Path(externalPath, dbAndTablePath); } + private static ExternalPathProvider getExternalPathProvider( + TableSchema tableSchema, Path tablePath) { + CoreOptions coreOptions = CoreOptions.fromMap(tableSchema.options()); + String externalPaths = coreOptions.dataFileExternalPaths(); + ExternalPathStrategy externalPathStrategy = coreOptions.externalPathStrategy(); + ExternalFSStrategy externalSpecificFSStrategy = coreOptions.externalSpecificFSStrategy(); + String dbAndTablePath = tablePath.getParent().getName() + "/" + tablePath.getName(); + return new ExternalPathProvider( + externalPaths, externalPathStrategy, externalSpecificFSStrategy, dbAndTablePath); + } + public static FileStoreTable create(FileIO fileIO, Path tablePath, TableSchema tableSchema) { - Path tableDataPath = getTableDataPath(tableSchema, tablePath); + ExternalPathProvider externalPathProvider = getExternalPathProvider(tableSchema, tablePath); return create( fileIO, tablePath, tableSchema, new Options(), CatalogEnvironment.empty(), - tableDataPath); + externalPathProvider); } public static FileStoreTable create( @@ -96,9 +115,15 @@ public static FileStoreTable create( Path tablePath, TableSchema tableSchema, CatalogEnvironment catalogEnvironment) { - Path tableDataPath = getTableDataPath(tableSchema, tablePath); + // Path tableDataPath = getTableDataPath(tableSchema, tablePath); + ExternalPathProvider externalPathProvider = getExternalPathProvider(tableSchema, tablePath); return create( - fileIO, tablePath, tableSchema, new Options(), catalogEnvironment, tableDataPath); + fileIO, + tablePath, + tableSchema, + new Options(), + catalogEnvironment, + externalPathProvider); } public static FileStoreTable create( @@ -107,7 +132,7 @@ public static FileStoreTable create( TableSchema tableSchema, Options dynamicOptions, CatalogEnvironment catalogEnvironment, - Path tableDataPath) { + ExternalPathProvider externalPathProvider) { FileStoreTable table = createWithoutFallbackBranch( fileIO, @@ -115,7 +140,7 @@ public static FileStoreTable create( tableSchema, dynamicOptions, catalogEnvironment, - tableDataPath); + externalPathProvider); Options options = new Options(table.options()); String fallbackBranch = options.get(CoreOptions.SCAN_FALLBACK_BRANCH); @@ -137,7 +162,7 @@ public static FileStoreTable create( schema.get(), branchOptions, catalogEnvironment, - tableDataPath); + externalPathProvider); table = new FallbackReadFileStoreTable(table, fallbackTable); } @@ -150,13 +175,21 @@ public static FileStoreTable createWithoutFallbackBranch( TableSchema tableSchema, Options dynamicOptions, CatalogEnvironment catalogEnvironment, - Path tableDataPath) { + ExternalPathProvider externalPathProvider) { FileStoreTable table = tableSchema.primaryKeys().isEmpty() ? new AppendOnlyFileStoreTable( - fileIO, tablePath, tableSchema, catalogEnvironment, tableDataPath) + fileIO, + tablePath, + tableSchema, + catalogEnvironment, + externalPathProvider) : new PrimaryKeyFileStoreTable( - fileIO, tablePath, tableSchema, catalogEnvironment, tableDataPath); + fileIO, + tablePath, + tableSchema, + catalogEnvironment, + externalPathProvider); return table.copy(dynamicOptions.toMap()); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java index 69820944a7c1..6b1d86b666d5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java @@ -21,6 +21,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.KeyValue; import org.apache.paimon.KeyValueFileStore; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.iceberg.IcebergOptions; @@ -57,7 +58,7 @@ class PrimaryKeyFileStoreTable extends AbstractFileStoreTable { private transient KeyValueFileStore lazyStore; PrimaryKeyFileStoreTable(FileIO fileIO, Path path, TableSchema tableSchema) { - this(fileIO, path, tableSchema, CatalogEnvironment.empty(), path); + this(fileIO, path, tableSchema, CatalogEnvironment.empty(), new ExternalPathProvider()); } PrimaryKeyFileStoreTable( @@ -65,8 +66,8 @@ class PrimaryKeyFileStoreTable extends AbstractFileStoreTable { Path path, TableSchema tableSchema, CatalogEnvironment catalogEnvironment, - Path tableDataPath) { - super(fileIO, path, tableSchema, catalogEnvironment, tableDataPath); + ExternalPathProvider externalPathProvider) { + super(fileIO, path, tableSchema, catalogEnvironment, externalPathProvider); } @Override @@ -100,7 +101,8 @@ public KeyValueFileStore store() { extractor, mfFactory, name(), - catalogEnvironment); + catalogEnvironment, + externalPathProvider); } return lazyStore; } @@ -192,4 +194,9 @@ protected List createCommitCallbacks(String commitUser) { return callbacks; } + + @Override + public ExternalPathProvider externalPathProvider() { + return externalPathProvider; + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/object/ObjectTable.java b/paimon-core/src/main/java/org/apache/paimon/table/object/ObjectTable.java index 97acfe7299c5..9a8b8e449a96 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/object/ObjectTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/object/ObjectTable.java @@ -18,6 +18,7 @@ package org.apache.paimon.table.object; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.manifest.ManifestCacheFilter; import org.apache.paimon.schema.TableSchema; @@ -215,5 +216,10 @@ public ObjectTable switchToBranch(String branchName) { return new ObjectTableImpl( wrapped.switchToBranch(branchName), objectFileIO, objectLocation); } + + @Override + public ExternalPathProvider externalPathProvider() { + return wrapped.externalPathProvider(); + } } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java index 3fff9e2ca105..88ad3a71b153 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java @@ -25,6 +25,7 @@ import org.apache.paimon.data.BinaryString; import org.apache.paimon.data.InternalRow; import org.apache.paimon.disk.IOManager; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.BucketEntry; @@ -184,8 +185,8 @@ public Path location() { } @Override - public Path dataLocation() { - return wrapped.dataLocation(); + public ExternalPathProvider externalPathProvider() { + return wrapped.externalPathProvider(); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/CompactBucketsTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/CompactBucketsTable.java index b2bcd63f5259..b3db0e0aecb9 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/CompactBucketsTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/CompactBucketsTable.java @@ -24,6 +24,7 @@ import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.disk.IOManager; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.io.DataFileMeta; @@ -142,8 +143,8 @@ public Path location() { } @Override - public Path dataLocation() { - return wrapped.dataLocation(); + public ExternalPathProvider externalPathProvider() { + return wrapped.externalPathProvider(); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/FileMonitorTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/FileMonitorTable.java index 3ff111ea9617..42ffbf6e33a1 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/FileMonitorTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/FileMonitorTable.java @@ -25,6 +25,7 @@ import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.disk.IOManager; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.io.DataFileMeta; @@ -128,8 +129,8 @@ public Path location() { } @Override - public Path dataLocation() { - return wrapped.dataLocation(); + public ExternalPathProvider externalPathProvider() { + return wrapped.externalPathProvider(); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/ReadOptimizedTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/ReadOptimizedTable.java index 3884798f9274..5e5124630f70 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/ReadOptimizedTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/ReadOptimizedTable.java @@ -20,6 +20,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.Snapshot; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.IndexManifestEntry; @@ -162,8 +163,8 @@ public Path location() { } @Override - public Path dataLocation() { - return wrapped.dataLocation(); + public ExternalPathProvider externalPathProvider() { + return wrapped.externalPathProvider(); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java b/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java index 1fde3048d87f..614d65e6de46 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java @@ -20,6 +20,7 @@ import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.data.BinaryRow; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.Path; import org.apache.paimon.io.DataFilePathFactory; import org.apache.paimon.types.RowType; @@ -41,7 +42,7 @@ public class FileStorePathFactory { // this is the table schema root path private final Path root; // this is the table data root path - private final Path dataRoot; + // private final Path dataRoot; private final String uuid; private final InternalRowPartitionComputer partitionComputer; private final String formatIdentifier; @@ -57,7 +58,8 @@ public class FileStorePathFactory { private final AtomicInteger indexManifestCount; private final AtomicInteger indexFileCount; private final AtomicInteger statsFileCount; - private final boolean isExternalPath; + // private final boolean externalPathExists; + private final ExternalPathProvider externalPathProvider; public FileStorePathFactory( Path root, @@ -70,7 +72,7 @@ public FileStorePathFactory( boolean fileSuffixIncludeCompression, String fileCompression, @Nullable String dataFilePathDirectory, - Path dataRoot) { + ExternalPathProvider externalPathProvider) { this.root = root; this.dataFilePathDirectory = dataFilePathDirectory; this.uuid = UUID.randomUUID().toString(); @@ -88,8 +90,8 @@ public FileStorePathFactory( this.indexManifestCount = new AtomicInteger(0); this.indexFileCount = new AtomicInteger(0); this.statsFileCount = new AtomicInteger(0); - this.dataRoot = dataRoot; - this.isExternalPath = !root.equals(dataRoot); + this.externalPathProvider = externalPathProvider; + // this.externalPathExists = externalPathProvider.externalPathExists(); } public Path root() { @@ -134,11 +136,12 @@ public DataFilePathFactory createDataFilePathFactory(BinaryRow partition, int bu changelogFilePrefix, fileSuffixIncludeCompression, fileCompression, - isExternalPath); + externalPathProvider, + relativeBucketPath(partition, bucket)); } public Path bucketPath(BinaryRow partition, int bucket) { - return new Path(dataRoot, relativeBucketPath(partition, bucket)); + return new Path(root, relativeBucketPath(partition, bucket)); } public Path relativeBucketPath(BinaryRow partition, int bucket) { @@ -161,6 +164,7 @@ public String getPartitionString(BinaryRow partition) { partition, "Partition row data is null. This is unexpected."))); } + // @TODO, need to be changed public List getHierarchicalPartitionPath(BinaryRow partition) { return PartitionPathUtils.generateHierarchicalPartitionPaths( partitionComputer.generatePartValues( @@ -168,7 +172,7 @@ public List getHierarchicalPartitionPath(BinaryRow partition) { partition, "Partition binary row is null. This is unexpected."))) .stream() - .map(p -> new Path(dataRoot + "/" + p)) + .map(p -> new Path(root + "/" + p)) .collect(Collectors.toList()); } diff --git a/paimon-core/src/test/java/org/apache/paimon/TestAppendFileStore.java b/paimon-core/src/test/java/org/apache/paimon/TestAppendFileStore.java index c45fa7ae7139..345e7cdfc20e 100644 --- a/paimon-core/src/test/java/org/apache/paimon/TestAppendFileStore.java +++ b/paimon-core/src/test/java/org/apache/paimon/TestAppendFileStore.java @@ -22,6 +22,7 @@ import org.apache.paimon.deletionvectors.DeletionVectorsMaintainer; import org.apache.paimon.deletionvectors.append.AppendDeletionFileMaintainerHelper; import org.apache.paimon.deletionvectors.append.UnawareAppendDeletionFileMaintainer; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.FileIOFinder; import org.apache.paimon.fs.Path; @@ -80,7 +81,8 @@ public TestAppendFileStore( bucketType, rowType, tableName, - CatalogEnvironment.empty()); + CatalogEnvironment.empty(), + new ExternalPathProvider()); this.fileIO = fileIO; this.commitUser = UUID.randomUUID().toString(); @@ -156,7 +158,6 @@ public static TestAppendFileStore createAppendStore( SchemaManager schemaManage = new SchemaManager(new LocalFileIO(), path); options.put(CoreOptions.PATH.key(), root); - options.put(CoreOptions.TABLE_DATA_PATH.key(), root); TableSchema tableSchema = SchemaUtils.forceCommit( schemaManage, diff --git a/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java b/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java index 1a5b68e11fb1..05fed5a5ee0c 100644 --- a/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java +++ b/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java @@ -20,6 +20,7 @@ import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.serializer.InternalRowSerializer; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.FileIOFinder; import org.apache.paimon.fs.Path; @@ -133,7 +134,8 @@ private TestFileStore( keyValueFieldsExtractor, mfFactory, (new Path(root)).getName(), - CatalogEnvironment.empty()); + CatalogEnvironment.empty(), + new ExternalPathProvider()); this.root = root; this.fileIO = FileIOFinder.find(new Path(root)); this.keySerializer = new InternalRowSerializer(keyType); @@ -798,7 +800,6 @@ public TestFileStore build() { conf.set(CoreOptions.FILE_FORMAT, format); conf.set(CoreOptions.MANIFEST_FORMAT, format); conf.set(CoreOptions.PATH, root); - conf.set(CoreOptions.TABLE_DATA_PATH, root); conf.set(CoreOptions.BUCKET, numBuckets); conf.set(CoreOptions.CHANGELOG_PRODUCER, changelogProducer); diff --git a/paimon-core/src/test/java/org/apache/paimon/append/AppendOnlyWriterTest.java b/paimon-core/src/test/java/org/apache/paimon/append/AppendOnlyWriterTest.java index 0dbbc8585c44..dd6e2abca084 100644 --- a/paimon-core/src/test/java/org/apache/paimon/append/AppendOnlyWriterTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/append/AppendOnlyWriterTest.java @@ -31,6 +31,7 @@ import org.apache.paimon.fileindex.FileIndexOptions; import org.apache.paimon.format.FileFormat; import org.apache.paimon.format.SimpleColStats; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.io.DataFileMeta; @@ -524,7 +525,8 @@ private DataFilePathFactory createPathFactory() { CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), - false); + new ExternalPathProvider(), + null); } private AppendOnlyWriter createEmptyWriter(long targetFileSize) { diff --git a/paimon-core/src/test/java/org/apache/paimon/format/FileFormatSuffixTest.java b/paimon-core/src/test/java/org/apache/paimon/format/FileFormatSuffixTest.java index d1f95c39c6d1..e636822be1e7 100644 --- a/paimon-core/src/test/java/org/apache/paimon/format/FileFormatSuffixTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/format/FileFormatSuffixTest.java @@ -26,6 +26,7 @@ import org.apache.paimon.data.GenericRow; import org.apache.paimon.disk.IOManager; import org.apache.paimon.fileindex.FileIndexOptions; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.io.DataFileMeta; @@ -73,7 +74,8 @@ public void testFileSuffix(@TempDir java.nio.file.Path tempDir) throws Exception CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), - false); + new ExternalPathProvider(), + null); FileFormat fileFormat = FileFormat.fromIdentifier(format, new Options()); LinkedList toCompact = new LinkedList<>(); CoreOptions options = new CoreOptions(new HashMap<>()); diff --git a/paimon-core/src/test/java/org/apache/paimon/io/DataFilePathFactoryTest.java b/paimon-core/src/test/java/org/apache/paimon/io/DataFilePathFactoryTest.java index ddb76cfdcc52..76a6ed77a482 100644 --- a/paimon-core/src/test/java/org/apache/paimon/io/DataFilePathFactoryTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/io/DataFilePathFactoryTest.java @@ -19,6 +19,7 @@ package org.apache.paimon.io; import org.apache.paimon.CoreOptions; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.Path; import org.junit.jupiter.api.Test; @@ -41,7 +42,8 @@ public void testNoPartition() { CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), - false); + new ExternalPathProvider(), + null); String uuid = pathFactory.uuid(); for (int i = 0; i < 20; i++) { @@ -71,7 +73,8 @@ public void testWithPartition() { CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), - false); + new ExternalPathProvider(), + null); String uuid = pathFactory.uuid(); for (int i = 0; i < 20; i++) { diff --git a/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java b/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java index 8e234a82ac97..674c084a68b0 100644 --- a/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java @@ -28,6 +28,7 @@ import org.apache.paimon.deletionvectors.DeletionVector; import org.apache.paimon.format.FlushingFileFormat; import org.apache.paimon.format.SimpleColStats; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.FileIOFinder; import org.apache.paimon.fs.FileStatus; @@ -238,7 +239,7 @@ protected KeyValueFileWriterFactory createWriterFactory(String pathStr, String f CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), null, - path); + new ExternalPathProvider()); int suggestedFileSize = ThreadLocalRandom.current().nextInt(8192) + 1024; FileIO fileIO = FileIOFinder.find(path); Options options = new Options(); @@ -259,7 +260,7 @@ protected KeyValueFileWriterFactory createWriterFactory(String pathStr, String f CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), null, - path)); + new ExternalPathProvider())); return KeyValueFileWriterFactory.builder( fileIO, diff --git a/paimon-core/src/test/java/org/apache/paimon/io/RollingFileWriterTest.java b/paimon-core/src/test/java/org/apache/paimon/io/RollingFileWriterTest.java index bf50e23e1658..881b812072dc 100644 --- a/paimon-core/src/test/java/org/apache/paimon/io/RollingFileWriterTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/io/RollingFileWriterTest.java @@ -23,6 +23,7 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.fileindex.FileIndexOptions; import org.apache.paimon.format.FileFormat; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.manifest.FileSource; @@ -85,7 +86,8 @@ public void initialize(String identifier, boolean statsDenseStore) { CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION .defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), - false) + new ExternalPathProvider(), + null) .newPath(), SCHEMA, fileFormat diff --git a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTestBase.java b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTestBase.java index f6a216513c6b..0f6c732a064c 100644 --- a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTestBase.java +++ b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTestBase.java @@ -23,6 +23,7 @@ import org.apache.paimon.data.BinaryRowWriter; import org.apache.paimon.data.Timestamp; import org.apache.paimon.format.FileFormat; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.FileIOFinder; import org.apache.paimon.fs.Path; @@ -152,7 +153,7 @@ protected ManifestFile createManifestFile(String pathStr) { CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), null, - path), + new ExternalPathProvider()), Long.MAX_VALUE, null) .create(); diff --git a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileTest.java b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileTest.java index 8f9f4c768396..f574913f3b07 100644 --- a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileTest.java @@ -21,6 +21,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.format.FileFormat; import org.apache.paimon.format.SimpleColStats; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.FileIOFinder; import org.apache.paimon.fs.Path; @@ -107,7 +108,7 @@ private ManifestFile createManifestFile(String pathStr) { CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), null, - path); + new ExternalPathProvider()); int suggestedFileSize = ThreadLocalRandom.current().nextInt(8192) + 1024; FileIO fileIO = FileIOFinder.find(path); return new ManifestFile.Factory( diff --git a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestListTest.java b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestListTest.java index 1baf688a78d6..6f9201251c61 100644 --- a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestListTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestListTest.java @@ -21,6 +21,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.TestKeyValueGenerator; import org.apache.paimon.format.FileFormat; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIOFinder; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; @@ -111,7 +112,7 @@ private ManifestList createManifestList(String pathStr) { CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), null, - path); + new ExternalPathProvider()); return new ManifestList.Factory(FileIOFinder.find(path), avro, "zstd", pathFactory, null) .create(); } diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogTest.java index 39a43ff62176..222dafb3bcb3 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogTest.java @@ -243,8 +243,8 @@ public void testGetTable() throws Exception { GetTableResponse response = MockRESTMessage.getTableResponse(); mockResponse(mapper.writeValueAsString(response), 200); Table result = mockRestCatalog.getTable(Identifier.create(databaseName, "table")); - // catalog will add PATH and TABLE_DATA_PATH option - assertEquals(response.getSchema().options().size() + 2, result.options().size()); + // catalog will add PATH option + assertEquals(response.getSchema().options().size() + 1, result.options().size()); verify(mockRestCatalog, times(1)).getDataOrFormatTable(any()); } diff --git a/paimon-core/src/test/java/org/apache/paimon/table/source/snapshot/ScannerTestBase.java b/paimon-core/src/test/java/org/apache/paimon/table/source/snapshot/ScannerTestBase.java index ce53337a6cfd..46de16dd2032 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/source/snapshot/ScannerTestBase.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/source/snapshot/ScannerTestBase.java @@ -23,6 +23,7 @@ import org.apache.paimon.data.BinaryRowWriter; import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.FileIOFinder; import org.apache.paimon.fs.Path; @@ -168,7 +169,12 @@ protected FileStoreTable createFileStoreTable( conf.toMap(), "")); return FileStoreTableFactory.create( - fileIO, tablePath, tableSchema, conf, CatalogEnvironment.empty(), tablePath); + fileIO, + tablePath, + tableSchema, + conf, + CatalogEnvironment.empty(), + new ExternalPathProvider()); } protected List toSplits(List dataSplits) { diff --git a/paimon-core/src/test/java/org/apache/paimon/table/source/snapshot/SnapshotReaderTest.java b/paimon-core/src/test/java/org/apache/paimon/table/source/snapshot/SnapshotReaderTest.java index 48ad20f3e4c1..10b4b6158002 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/source/snapshot/SnapshotReaderTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/source/snapshot/SnapshotReaderTest.java @@ -23,6 +23,7 @@ import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.serializer.InternalRowSerializer; import org.apache.paimon.fileindex.bloomfilter.BloomFilterFileIndexFactory; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.FileIOFinder; import org.apache.paimon.fs.Path; @@ -368,6 +369,11 @@ private FileStoreTable createFileStoreTable( options.toMap(), "")); return FileStoreTableFactory.create( - fileIO, tablePath, tableSchema, options, CatalogEnvironment.empty(), tablePath); + fileIO, + tablePath, + tableSchema, + options, + CatalogEnvironment.empty(), + new ExternalPathProvider()); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/utils/FileStorePathFactoryTest.java b/paimon-core/src/test/java/org/apache/paimon/utils/FileStorePathFactoryTest.java index 5d5a546d704c..91f26422a02e 100644 --- a/paimon-core/src/test/java/org/apache/paimon/utils/FileStorePathFactoryTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/utils/FileStorePathFactoryTest.java @@ -22,6 +22,7 @@ import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.BinaryRowWriter; import org.apache.paimon.data.BinaryString; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.Path; import org.apache.paimon.io.DataFilePathFactory; import org.apache.paimon.types.DataType; @@ -93,7 +94,7 @@ public void testCreateDataFilePathFactoryWithPartition() { CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), null, - new Path(tempDir.toString())); + new ExternalPathProvider()); assertPartition("20211224", 16, pathFactory, "/dt=20211224/hr=16"); assertPartition("20211224", null, pathFactory, "/dt=20211224/hr=default"); @@ -136,6 +137,6 @@ public static FileStorePathFactory createNonPartFactory(Path root) { CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), null, - root); + new ExternalPathProvider()); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupFileStoreTable.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupFileStoreTable.java index 090399706330..a6bb9fcee776 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupFileStoreTable.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupFileStoreTable.java @@ -22,6 +22,7 @@ import org.apache.paimon.KeyValueFileStore; import org.apache.paimon.flink.FlinkConnectorOptions; import org.apache.paimon.flink.utils.TableScanUtils; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.operation.DefaultValueAssigner; import org.apache.paimon.options.Options; import org.apache.paimon.options.description.DescribedEnum; @@ -109,6 +110,11 @@ public FileStoreTable switchToBranch(String branchName) { return this; } + @Override + public ExternalPathProvider externalPathProvider() { + return wrapped.externalPathProvider(); + } + private LookupStreamScanMode lookupStreamScanMode(FileStoreTable table, List joinKeys) { Options options = Options.fromMap(table.options()); if (options.get(LOOKUP_CACHE_MODE) == FlinkConnectorOptions.LookupCacheMode.AUTO diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java index a79ab8ab3929..87cc9b7f040c 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkCatalogTest.java @@ -92,7 +92,6 @@ import static org.apache.flink.table.factories.FactoryUtil.CONNECTOR; import static org.apache.paimon.CoreOptions.PATH; import static org.apache.paimon.CoreOptions.SCAN_FILE_CREATION_TIME_MILLIS; -import static org.apache.paimon.CoreOptions.TABLE_DATA_PATH; import static org.apache.paimon.flink.FlinkCatalogOptions.DISABLE_CREATE_TABLE_IN_DEFAULT_DB; import static org.apache.paimon.flink.FlinkCatalogOptions.LOG_SYSTEM_AUTO_REGISTER; import static org.apache.paimon.flink.FlinkConnectorOptions.LOG_SYSTEM; @@ -902,13 +901,11 @@ private void checkEquals( .getTable(FlinkCatalog.toIdentifier(path)) .options(); tablePath = new Path(options.get(PATH.key())); - tableDataPath = new Path(options.get(TABLE_DATA_PATH.key())); } catch (org.apache.paimon.catalog.Catalog.TableNotExistException e) { throw new RuntimeException(e); } Map options = new HashMap<>(t1.getOptions()); options.put("path", tablePath.toString()); - options.put("table.data.path", tableDataPath.toString()); options.putAll(optionsToAdd); optionsToRemove.forEach(options::remove); if (t1.getTableKind() == CatalogBaseTable.TableKind.TABLE) { diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/FlinkSinkTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/FlinkSinkTest.java index 5fe33061a6a5..3d9430411439 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/FlinkSinkTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/FlinkSinkTest.java @@ -21,6 +21,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIOFinder; import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.operation.KeyValueFileStoreWrite; @@ -126,6 +127,6 @@ private FileStoreTable createFileStoreTable() throws Exception { tableSchema, options, CatalogEnvironment.empty(), - tablePath); + new ExternalPathProvider()); } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java index cd9a45270a16..52bad1c3995b 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java @@ -26,6 +26,7 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.format.FileFormat; import org.apache.paimon.format.FileFormatDiscover; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.io.DataFileMeta; @@ -112,7 +113,7 @@ public TestChangelogDataReadWrite(String root) { CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), null, - tablePath); + new ExternalPathProvider()); this.snapshotManager = new SnapshotManager(LocalFileIO.create(), new Path(root)); this.commitUser = UUID.randomUUID().toString(); } diff --git a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkFileIndexITCase.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkFileIndexITCase.java index 65424ba39987..09fdc1566dfe 100644 --- a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkFileIndexITCase.java +++ b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkFileIndexITCase.java @@ -28,6 +28,7 @@ import org.apache.paimon.fileindex.FileIndexReader; import org.apache.paimon.fileindex.FileIndexResult; import org.apache.paimon.fileindex.bitmap.BitmapIndexResult; +import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; @@ -162,7 +163,7 @@ protected void foreachIndexReader(Consumer consumer) CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), null, - tableRoot); + new ExternalPathProvider()); Table table = fileSystemCatalog.getTable(Identifier.create("db", "T")); ReadBuilder readBuilder = table.newReadBuilder(); From 63625dfbfc37b4f40222c38773bb2752ccd539c1 Mon Sep 17 00:00:00 2001 From: HouliangQi Date: Thu, 2 Jan 2025 14:24:33 +0800 Subject: [PATCH 09/21] remove useless codes --- .../shortcodes/generated/core_configuration.html | 16 ++++++++++++++-- .../main/java/org/apache/paimon/CoreOptions.java | 8 +------- .../paimon/table/AbstractFileStoreTable.java | 2 -- .../paimon/table/FileStoreTableFactory.java | 10 ---------- .../paimon/utils/FileStorePathFactory.java | 3 --- .../org/apache/paimon/rest/RESTCatalogTest.java | 2 +- 6 files changed, 16 insertions(+), 25 deletions(-) diff --git a/docs/layouts/shortcodes/generated/core_configuration.html b/docs/layouts/shortcodes/generated/core_configuration.html index 2a8ea1c0bf1e..910997e4977f 100644 --- a/docs/layouts/shortcodes/generated/core_configuration.html +++ b/docs/layouts/shortcodes/generated/core_configuration.html @@ -1009,10 +1009,22 @@ The bytes of types (CHAR, VARCHAR, BINARY, VARBINARY) devote to the zorder sort. -
data-file.external-path
+
data-file.external-paths
(none) String - The path where the data of this table is currently written. + The external paths where the data of this table will be written, multiple elements separated by commas. + + +
data-file.external-paths.strategy
+ (none) +

Enum

+ The strategy of selecting an external path when writing data.

Possible values:
  • "none": Do not choose any external storage, data will still be written to the default warehouse path.
  • "specific-fs": Select a specific file system as the external path. Currently supported are S3 and OSS.
  • "round-robin": When writing a new file, a path is chosen from data-file.external-paths in turn.
+ + +
data-file.external-paths.specific-fs
+ (none) +

Enum

+ The specific file system of the external path when data-file.external-paths.strategy is set to specific-fs.

Possible values:"S3": Select S3 as the write path for the external path.
  • "OSS": Select OSS as the write path for the external path.
  • diff --git a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java index 3d8403a0b05e..a908562c1882 100644 --- a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java +++ b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java @@ -124,12 +124,6 @@ public class CoreOptions implements Serializable { + "if there is no primary key, the full row will be used.") .build()); - public static final ConfigOption DATA_FILE_EXTERNAL_PATH = - key("data-file.external-path") - .stringType() - .noDefaultValue() - .withDescription("The path where the data of this table is currently written."); - public static final ConfigOption DATA_FILE_EXTERNAL_PATHS = key("data-file.external-paths") .stringType() @@ -143,7 +137,7 @@ public class CoreOptions implements Serializable { .enumType(ExternalPathStrategy.class) .defaultValue(ExternalPathStrategy.NONE) .withDescription( - "The strategy of selecting an external path when writing data"); + "The strategy of selecting an external path when writing data."); public static final ConfigOption DATA_FILE_EXTERNAL_PATHS_SPECIFIC_FS = key("data-file.external-paths.specific-fs") diff --git a/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java index 65de00a556cf..367d310ab47c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java @@ -99,7 +99,6 @@ abstract class AbstractFileStoreTable implements FileStoreTable { protected final FileIO fileIO; protected final Path path; - protected final TableSchema tableSchema; protected final CatalogEnvironment catalogEnvironment; @@ -123,7 +122,6 @@ protected AbstractFileStoreTable( newOptions.put(PATH.key(), path.toString()); tableSchema = tableSchema.copy(newOptions); } - this.tableSchema = tableSchema; this.catalogEnvironment = catalogEnvironment; this.externalPathProvider = externalPathProvider; diff --git a/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTableFactory.java b/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTableFactory.java index 534a4c1bf45b..bf5d4ba5bb30 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTableFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTableFactory.java @@ -79,15 +79,6 @@ public static FileStoreTable create(FileIO fileIO, Options options) { externalPathProvider); } - private static Path getTableDataPath(TableSchema tableSchema, Path tablePath) { - String externalPath = tableSchema.options().get(CoreOptions.DATA_FILE_EXTERNAL_PATH.key()); - if (externalPath == null || externalPath.isEmpty()) { - return tablePath; - } - String dbAndTablePath = tablePath.getParent().getName() + "/" + tablePath.getName(); - return new Path(externalPath, dbAndTablePath); - } - private static ExternalPathProvider getExternalPathProvider( TableSchema tableSchema, Path tablePath) { CoreOptions coreOptions = CoreOptions.fromMap(tableSchema.options()); @@ -115,7 +106,6 @@ public static FileStoreTable create( Path tablePath, TableSchema tableSchema, CatalogEnvironment catalogEnvironment) { - // Path tableDataPath = getTableDataPath(tableSchema, tablePath); ExternalPathProvider externalPathProvider = getExternalPathProvider(tableSchema, tablePath); return create( fileIO, diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java b/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java index 614d65e6de46..42d4f7a59276 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java @@ -41,8 +41,6 @@ public class FileStorePathFactory { // this is the table schema root path private final Path root; - // this is the table data root path - // private final Path dataRoot; private final String uuid; private final InternalRowPartitionComputer partitionComputer; private final String formatIdentifier; @@ -91,7 +89,6 @@ public FileStorePathFactory( this.indexFileCount = new AtomicInteger(0); this.statsFileCount = new AtomicInteger(0); this.externalPathProvider = externalPathProvider; - // this.externalPathExists = externalPathProvider.externalPathExists(); } public Path root() { diff --git a/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogTest.java b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogTest.java index 222dafb3bcb3..95a991a74a68 100644 --- a/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/rest/RESTCatalogTest.java @@ -243,7 +243,7 @@ public void testGetTable() throws Exception { GetTableResponse response = MockRESTMessage.getTableResponse(); mockResponse(mapper.writeValueAsString(response), 200); Table result = mockRestCatalog.getTable(Identifier.create(databaseName, "table")); - // catalog will add PATH option + // catalog will add path option assertEquals(response.getSchema().options().size() + 1, result.options().size()); verify(mockRestCatalog, times(1)).getDataOrFormatTable(any()); } From 20affd2d6b40daf34ba164e13eb91745c74b1bd3 Mon Sep 17 00:00:00 2001 From: HouliangQi Date: Thu, 2 Jan 2025 14:35:31 +0800 Subject: [PATCH 10/21] make ExternalPathProvider implements Serializable --- .../main/java/org/apache/paimon/fs/ExternalPathProvider.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/paimon-common/src/main/java/org/apache/paimon/fs/ExternalPathProvider.java b/paimon-common/src/main/java/org/apache/paimon/fs/ExternalPathProvider.java index 8dc1484566aa..9df62123e7db 100644 --- a/paimon-common/src/main/java/org/apache/paimon/fs/ExternalPathProvider.java +++ b/paimon-common/src/main/java/org/apache/paimon/fs/ExternalPathProvider.java @@ -22,6 +22,7 @@ import org.apache.paimon.CoreOptions.ExternalPathStrategy; import org.apache.paimon.annotation.VisibleForTesting; +import java.io.Serializable; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -29,7 +30,7 @@ import java.util.Optional; /** Provider for external paths. */ -public class ExternalPathProvider { +public class ExternalPathProvider implements Serializable { private final Map externalPathsMap; private final List externalPathsList; From 823adcf3fc26b66cc74778c890bb02c81e40bb00 Mon Sep 17 00:00:00 2001 From: HouliangQi Date: Thu, 2 Jan 2025 15:24:34 +0800 Subject: [PATCH 11/21] add toString and hash method in ExternalPathProvider --- .../paimon/fs/ExternalPathProvider.java | 53 +++++++++++++++++++ 1 file changed, 53 insertions(+) diff --git a/paimon-common/src/main/java/org/apache/paimon/fs/ExternalPathProvider.java b/paimon-common/src/main/java/org/apache/paimon/fs/ExternalPathProvider.java index 9df62123e7db..37d49f1fb3fb 100644 --- a/paimon-common/src/main/java/org/apache/paimon/fs/ExternalPathProvider.java +++ b/paimon-common/src/main/java/org/apache/paimon/fs/ExternalPathProvider.java @@ -27,6 +27,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; /** Provider for external paths. */ @@ -136,4 +137,56 @@ private Optional getRoundRobinPath() { public boolean externalPathExists() { return externalPathExists; } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + ExternalPathProvider that = (ExternalPathProvider) o; + return currentIndex == that.currentIndex + && externalPathExists == that.externalPathExists + && externalPathsMap.equals(that.externalPathsMap) + && externalPathsList.equals(that.externalPathsList) + && externalPathStrategy == that.externalPathStrategy + && externalFSStrategy == that.externalFSStrategy + && Objects.equals(dbAndTableRelativePath, that.dbAndTableRelativePath); + } + + @Override + public String toString() { + return "ExternalPathProvider{" + + " externalPathsMap=" + + externalPathsMap + + ", externalPathsList=" + + externalPathsList + + ", externalPathStrategy=" + + externalPathStrategy + + ", externalFSStrategy=" + + externalFSStrategy + + ", currentIndex=" + + currentIndex + + ", externalPathExists=" + + externalPathExists + + ", dbAndTableRelativePath='" + + dbAndTableRelativePath + + '\'' + + "}"; + } + + @Override + public int hashCode() { + return Objects.hash( + externalPathsMap, + externalPathsList, + externalPathStrategy, + externalFSStrategy, + currentIndex, + externalPathExists, + dbAndTableRelativePath); + } } From f981331d1ea1b50f4e6dc23250344287183eafa8 Mon Sep 17 00:00:00 2001 From: HouliangQi Date: Thu, 2 Jan 2025 19:28:28 +0800 Subject: [PATCH 12/21] fix docs description --- docs/layouts/shortcodes/generated/core_configuration.html | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/layouts/shortcodes/generated/core_configuration.html b/docs/layouts/shortcodes/generated/core_configuration.html index 333279341582..f89fc9b1ad42 100644 --- a/docs/layouts/shortcodes/generated/core_configuration.html +++ b/docs/layouts/shortcodes/generated/core_configuration.html @@ -1022,7 +1022,7 @@
    data-file.external-paths.strategy
    - (none) + none

    Enum

    The strategy of selecting an external path when writing data.

    Possible values:
    • "none": Do not choose any external storage, data will still be written to the default warehouse path.
    • "specific-fs": Select a specific file system as the external path. Currently supported are S3 and OSS.
    • "round-robin": When writing a new file, a path is chosen from data-file.external-paths in turn.
    @@ -1030,7 +1030,7 @@
    data-file.external-paths.specific-fs
    (none)

    Enum

    - The specific file system of the external path when data-file.external-paths.strategy is set to specific-fs.

    Possible values:"S3": Select S3 as the write path for the external path.
  • "OSS": Select OSS as the write path for the external path.
  • + The specific file system of the external path when data-file.external-paths.strategy is set to specific-fs

    Possible values:
    • "S3": Select S3 as the write path for the external path.
    • "OSS": Select OSS as the write path for the external path.
    From 1637325e6751e7af26cb19c7b909cd205a4d534d Mon Sep 17 00:00:00 2001 From: HouliangQi Date: Fri, 3 Jan 2025 10:22:49 +0800 Subject: [PATCH 13/21] add ExternalPathProviderTest --- .../paimon/fs/ExternalPathProvider.java | 16 +++ .../paimon/fs/ExternalPathProviderTest.java | 118 ++++++++++++++++++ 2 files changed, 134 insertions(+) create mode 100644 paimon-common/src/test/java/org/apache/paimon/fs/ExternalPathProviderTest.java diff --git a/paimon-common/src/main/java/org/apache/paimon/fs/ExternalPathProvider.java b/paimon-common/src/main/java/org/apache/paimon/fs/ExternalPathProvider.java index 37d49f1fb3fb..be22c8e65810 100644 --- a/paimon-common/src/main/java/org/apache/paimon/fs/ExternalPathProvider.java +++ b/paimon-common/src/main/java/org/apache/paimon/fs/ExternalPathProvider.java @@ -114,11 +114,17 @@ public Optional getNextExternalPath() { private Optional getSpecificFSExternalPath() { switch (externalFSStrategy) { case S3: + if (!externalPathsMap.containsKey(ExternalFSStrategy.S3)) { + return Optional.empty(); + } return Optional.of( new Path( externalPathsMap.get(ExternalFSStrategy.S3), dbAndTableRelativePath)); case OSS: + if (!externalPathsMap.containsKey(ExternalFSStrategy.OSS)) { + return Optional.empty(); + } return Optional.of( new Path( externalPathsMap.get(ExternalFSStrategy.OSS), @@ -138,6 +144,16 @@ public boolean externalPathExists() { return externalPathExists; } + @VisibleForTesting + public Map getExternalPathsMap() { + return externalPathsMap; + } + + @VisibleForTesting + public List getExternalPathsList() { + return externalPathsList; + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/paimon-common/src/test/java/org/apache/paimon/fs/ExternalPathProviderTest.java b/paimon-common/src/test/java/org/apache/paimon/fs/ExternalPathProviderTest.java new file mode 100644 index 000000000000..d6ce755484c1 --- /dev/null +++ b/paimon-common/src/test/java/org/apache/paimon/fs/ExternalPathProviderTest.java @@ -0,0 +1,118 @@ +/* + * 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.CoreOptions.ExternalFSStrategy; +import org.apache.paimon.CoreOptions.ExternalPathStrategy; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.Optional; + +import static org.junit.jupiter.api.Assertions.*; + +public class ExternalPathProviderTest { + + private ExternalPathProvider provider; + + @BeforeEach + public void setUp() { + provider = + new ExternalPathProvider( + "oss://bucket1/path1,s3://bucket2/path2", + ExternalPathStrategy.ROUND_ROBIN, + null, + "db/table"); + } + + @Test + public void testInitExternalPaths() { + assertTrue(provider.externalPathExists()); + assertEquals(2, provider.getExternalPathsMap().size()); + assertEquals(2, provider.getExternalPathsList().size()); + } + + @Test + public void testGetNextExternalPathRoundRobin() { + Optional path1 = provider.getNextExternalPath(); + assertTrue(path1.isPresent()); + assertEquals("s3://bucket2/path2/db/table", path1.get().toString()); + + Optional path2 = provider.getNextExternalPath(); + assertTrue(path2.isPresent()); + assertEquals("oss://bucket1/path1/db/table", path2.get().toString()); + + Optional path3 = provider.getNextExternalPath(); + assertTrue(path3.isPresent()); + assertEquals("s3://bucket2/path2/db/table", path3.get().toString()); + } + + @Test + public void testGetNextExternalPathSpecificFS() { + provider = + new ExternalPathProvider( + "oss://bucket1/path1,s3://bucket2/path2", + ExternalPathStrategy.SPECIFIC_FS, + ExternalFSStrategy.OSS, + "db/table"); + + Optional path = provider.getNextExternalPath(); + assertTrue(path.isPresent()); + assertEquals("oss://bucket1/path1/db/table", path.get().toString()); + } + + @Test + public void testGetNextExternalPathNone() { + provider = + new ExternalPathProvider( + "oss://bucket1/path1,s3://bucket2/path2", + ExternalPathStrategy.NONE, + ExternalFSStrategy.OSS, + "db/table"); + + Optional path = provider.getNextExternalPath(); + assertFalse(path.isPresent()); + } + + @Test + public void testUnsupportedExternalPath() { + assertThrows( + IllegalArgumentException.class, + () -> { + new ExternalPathProvider( + "hdfs://bucket1/path1", + ExternalPathStrategy.ROUND_ROBIN, + ExternalFSStrategy.OSS, + "db/table"); + }); + } + + @Test + public void testUnsupportedExternalFSStrategy() { + provider = + new ExternalPathProvider( + "oss://bucket1/path1", + ExternalPathStrategy.SPECIFIC_FS, + ExternalFSStrategy.S3, + "db/table"); + Optional path = provider.getNextExternalPath(); + assertFalse(path.isPresent()); + } +} From a025e654649fda0c865c5cb6b74e6efa2bc4f8ed Mon Sep 17 00:00:00 2001 From: HouliangQi Date: Fri, 3 Jan 2025 10:36:35 +0800 Subject: [PATCH 14/21] add doc for ExternalPathProviderTest --- .../paimon/fs/ExternalPathProviderTest.java | 48 ++++++++++--------- 1 file changed, 25 insertions(+), 23 deletions(-) diff --git a/paimon-common/src/test/java/org/apache/paimon/fs/ExternalPathProviderTest.java b/paimon-common/src/test/java/org/apache/paimon/fs/ExternalPathProviderTest.java index d6ce755484c1..5e98c9f87e28 100644 --- a/paimon-common/src/test/java/org/apache/paimon/fs/ExternalPathProviderTest.java +++ b/paimon-common/src/test/java/org/apache/paimon/fs/ExternalPathProviderTest.java @@ -21,13 +21,15 @@ import org.apache.paimon.CoreOptions.ExternalFSStrategy; import org.apache.paimon.CoreOptions.ExternalPathStrategy; +import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import java.util.Optional; -import static org.junit.jupiter.api.Assertions.*; +import static org.assertj.core.api.Assertions.assertThat; +/** Test for {@link ExternalPathProvider}. */ public class ExternalPathProviderTest { private ExternalPathProvider provider; @@ -44,24 +46,24 @@ public void setUp() { @Test public void testInitExternalPaths() { - assertTrue(provider.externalPathExists()); - assertEquals(2, provider.getExternalPathsMap().size()); - assertEquals(2, provider.getExternalPathsList().size()); + assertThat(provider.externalPathExists()).isTrue(); + assertThat(provider.getExternalPathsMap().size()).isEqualTo(2); + assertThat(provider.getExternalPathsList().size()).isEqualTo(2); } @Test public void testGetNextExternalPathRoundRobin() { Optional path1 = provider.getNextExternalPath(); - assertTrue(path1.isPresent()); - assertEquals("s3://bucket2/path2/db/table", path1.get().toString()); + assertThat(path1.isPresent()).isTrue(); + assertThat(path1.get().toString()).isEqualTo("s3://bucket2/path2/db/table"); Optional path2 = provider.getNextExternalPath(); - assertTrue(path2.isPresent()); - assertEquals("oss://bucket1/path1/db/table", path2.get().toString()); + assertThat(path2.isPresent()).isTrue(); + assertThat(path2.get().toString()).isEqualTo("oss://bucket1/path1/db/table"); Optional path3 = provider.getNextExternalPath(); - assertTrue(path3.isPresent()); - assertEquals("s3://bucket2/path2/db/table", path3.get().toString()); + assertThat(path3.isPresent()).isTrue(); + assertThat(path3.get().toString()).isEqualTo("s3://bucket2/path2/db/table"); } @Test @@ -74,8 +76,8 @@ public void testGetNextExternalPathSpecificFS() { "db/table"); Optional path = provider.getNextExternalPath(); - assertTrue(path.isPresent()); - assertEquals("oss://bucket1/path1/db/table", path.get().toString()); + assertThat(path.isPresent()).isTrue(); + assertThat(path.get().toString()).isEqualTo("oss://bucket1/path1/db/table"); } @Test @@ -88,20 +90,20 @@ public void testGetNextExternalPathNone() { "db/table"); Optional path = provider.getNextExternalPath(); - assertFalse(path.isPresent()); + assertThat(path.isPresent()).isFalse(); } @Test public void testUnsupportedExternalPath() { - assertThrows( - IllegalArgumentException.class, - () -> { - new ExternalPathProvider( - "hdfs://bucket1/path1", - ExternalPathStrategy.ROUND_ROBIN, - ExternalFSStrategy.OSS, - "db/table"); - }); + Assertions.assertThatThrownBy( + () -> { + new ExternalPathProvider( + "hdfs://bucket1/path1", + ExternalPathStrategy.ROUND_ROBIN, + ExternalFSStrategy.OSS, + "db/table"); + }) + .isInstanceOf(IllegalArgumentException.class); } @Test @@ -113,6 +115,6 @@ public void testUnsupportedExternalFSStrategy() { ExternalFSStrategy.S3, "db/table"); Optional path = provider.getNextExternalPath(); - assertFalse(path.isPresent()); + assertThat(path.isPresent()).isFalse(); } } From 80b872184254f6720506163198b641bf825c82fc Mon Sep 17 00:00:00 2001 From: HouliangQi Date: Fri, 3 Jan 2025 10:53:54 +0800 Subject: [PATCH 15/21] remove useless codes --- .../main/java/org/apache/paimon/utils/FileStorePathFactory.java | 1 - 1 file changed, 1 deletion(-) diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java b/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java index 42d4f7a59276..18d05577dd13 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java @@ -56,7 +56,6 @@ public class FileStorePathFactory { private final AtomicInteger indexManifestCount; private final AtomicInteger indexFileCount; private final AtomicInteger statsFileCount; - // private final boolean externalPathExists; private final ExternalPathProvider externalPathProvider; public FileStorePathFactory( From bcb2be121d2f6cdc25b1c78b3fb4a58f56da87f3 Mon Sep 17 00:00:00 2001 From: HouliangQi Date: Fri, 3 Jan 2025 14:09:54 +0800 Subject: [PATCH 16/21] add isExternalPath field in SingleFileWriter --- .../main/java/org/apache/paimon/fs/Path.java | 11 -------- .../iceberg/manifest/IcebergManifestFile.java | 1 + .../apache/paimon/io/DataFilePathFactory.java | 27 +++++++++---------- .../paimon/io/KeyValueDataFileWriter.java | 8 +++--- .../paimon/io/KeyValueDataFileWriterImpl.java | 6 +++-- .../paimon/io/KeyValueFileWriterFactory.java | 16 +++++++---- .../io/KeyValueThinDataFileWriterImpl.java | 6 +++-- .../apache/paimon/io/RowDataFileWriter.java | 8 +++--- .../paimon/io/RowDataRollingFileWriter.java | 3 ++- .../apache/paimon/io/SingleFileWriter.java | 5 +++- .../io/StatsCollectingSingleFileWriter.java | 5 ++-- .../apache/paimon/manifest/ManifestFile.java | 2 +- .../paimon/io/RollingFileWriterTest.java | 3 ++- 13 files changed, 54 insertions(+), 47 deletions(-) diff --git a/paimon-common/src/main/java/org/apache/paimon/fs/Path.java b/paimon-common/src/main/java/org/apache/paimon/fs/Path.java index fa72e1d40d0a..3cfeac5b6de4 100644 --- a/paimon-common/src/main/java/org/apache/paimon/fs/Path.java +++ b/paimon-common/src/main/java/org/apache/paimon/fs/Path.java @@ -62,8 +62,6 @@ public class Path implements Comparable, Serializable { /** A hierarchical URI. */ private URI uri; - private boolean isExternalPath; - /** * Create a new Path based on the child path resolved against the parent path. * @@ -387,13 +385,4 @@ public int hashCode() { public int compareTo(Path that) { return this.uri.compareTo(that.uri); } - - public Path setExternalPath(boolean externalPath) { - this.isExternalPath = externalPath; - return this; - } - - public boolean isExternalPath() { - return isExternalPath; - } } diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFile.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFile.java index 5955da6220f8..d4cfc0b5ece0 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFile.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/manifest/IcebergManifestFile.java @@ -154,6 +154,7 @@ private class IcebergManifestEntryWriter path, serializer::toRow, fileCompression, + false, false); this.partitionStatsCollector = new SimpleStatsCollector(partitionType); this.sequenceNumber = sequenceNumber; diff --git a/paimon-core/src/main/java/org/apache/paimon/io/DataFilePathFactory.java b/paimon-core/src/main/java/org/apache/paimon/io/DataFilePathFactory.java index 88d413d88ba1..ef9a7ae25a71 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/DataFilePathFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/DataFilePathFactory.java @@ -46,6 +46,7 @@ public class DataFilePathFactory { private final boolean fileSuffixIncludeCompression; private final String fileCompression; private final ExternalPathProvider externalPathProvider; + private final boolean isExternalPath; public DataFilePathFactory( Path parent, @@ -66,6 +67,7 @@ public DataFilePathFactory( this.fileCompression = fileCompression; this.externalPathProvider = externalPathProvider; this.relativePath = relativePath; + this.isExternalPath = externalPathProvider.externalPathExists(); } public Path newPath() { @@ -86,8 +88,7 @@ public Path newPath(String prefix) { .map( externalPath -> new Path(new Path(externalPath, relativePath), newFileName(prefix))) - .orElse(new Path(parent, newFileName(prefix))) - .setExternalPath(externalPathProvider.externalPathExists()); + .orElse(new Path(parent, newFileName(prefix))); } private String newFileName(String prefix) { @@ -101,27 +102,21 @@ private String newFileName(String prefix) { } public Path toPath(DataFileMeta file) { - return file.externalPath() - .map(Path::new) - .orElse(new Path(parent, file.fileName())) - .setExternalPath(file.externalPath().isPresent()); + return file.externalPath().map(Path::new).orElse(new Path(parent, file.fileName())); } public Path toPath(FileEntry file) { return Optional.ofNullable(file.externalPath()) .map(Path::new) - .orElse(new Path(parent, file.fileName())) - .setExternalPath(Optional.ofNullable(file.externalPath()).isPresent()); + .orElse(new Path(parent, file.fileName())); } public Path toAlignedPath(String fileName, DataFileMeta aligned) { - return new Path(aligned.externalPathDir().map(Path::new).orElse(parent), fileName) - .setExternalPath(aligned.externalPathDir().isPresent()); + return new Path(aligned.externalPathDir().map(Path::new).orElse(parent), fileName); } public static Path dataFileToFileIndexPath(Path dataFilePath) { - return new Path(dataFilePath.getParent(), dataFilePath.getName() + INDEX_PATH_SUFFIX) - .setExternalPath(dataFilePath.isExternalPath()); + return new Path(dataFilePath.getParent(), dataFilePath.getName() + INDEX_PATH_SUFFIX); } public static Path createNewFileIndexFilePath(Path filePath) { @@ -140,9 +135,7 @@ public static Path createNewFileIndexFilePath(Path filePath) { } } return new Path( - filePath.getParent(), - fileName.substring(0, dot) + "-" + 1 + INDEX_PATH_SUFFIX) - .setExternalPath(filePath.isExternalPath()); + filePath.getParent(), fileName.substring(0, dot) + "-" + 1 + INDEX_PATH_SUFFIX); } public static String formatIdentifier(String fileName) { @@ -154,6 +147,10 @@ public static String formatIdentifier(String fileName) { return fileName.substring(index + 1); } + public boolean isExternalPath() { + return isExternalPath; + } + @VisibleForTesting String uuid() { return uuid; diff --git a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileWriter.java b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileWriter.java index 65ea06b8fb9d..359855260e78 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileWriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileWriter.java @@ -91,7 +91,8 @@ public KeyValueDataFileWriter( String compression, CoreOptions options, FileSource fileSource, - FileIndexOptions fileIndexOptions) { + FileIndexOptions fileIndexOptions, + boolean isExternalPath) { super( fileIO, factory, @@ -102,7 +103,8 @@ public KeyValueDataFileWriter( compression, StatsCollectorFactories.createStatsFactories( options, writeRowType.getFieldNames(), keyType.getFieldNames()), - options.asyncFileWrite()); + options.asyncFileWrite(), + isExternalPath); this.keyType = keyType; this.valueType = valueType; @@ -177,7 +179,7 @@ public DataFileMeta result() throws IOException { ? DataFileIndexWriter.EMPTY_RESULT : dataFileIndexWriter.result(); - String externalPath = path.isExternalPath() ? path.toString() : null; + String externalPath = isExternalPath ? path.toString() : null; return new DataFileMeta( path.getName(), fileIO.getFileSize(path), diff --git a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileWriterImpl.java b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileWriterImpl.java index 27a1aef64e36..5bfc322a50bc 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileWriterImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueDataFileWriterImpl.java @@ -52,7 +52,8 @@ public KeyValueDataFileWriterImpl( String compression, CoreOptions options, FileSource fileSource, - FileIndexOptions fileIndexOptions) { + FileIndexOptions fileIndexOptions, + boolean isExternalPath) { super( fileIO, factory, @@ -67,7 +68,8 @@ public KeyValueDataFileWriterImpl( compression, options, fileSource, - fileIndexOptions); + fileIndexOptions, + isExternalPath); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileWriterFactory.java b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileWriterFactory.java index 7b6f4f0e3c56..feee72438985 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileWriterFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileWriterFactory.java @@ -95,7 +95,10 @@ public RollingFileWriter createRollingMergeTreeFileWrite return new RollingFileWriter<>( () -> createDataFileWriter( - formatContext.pathFactory(level).newPath(), level, fileSource), + formatContext.pathFactory(level).newPath(), + level, + fileSource, + formatContext.pathFactory(level).isExternalPath()), suggestedFileSize); } @@ -105,12 +108,13 @@ public RollingFileWriter createRollingChangelogFileWrite createDataFileWriter( formatContext.pathFactory(level).newChangelogPath(), level, - FileSource.APPEND), + FileSource.APPEND, + formatContext.pathFactory(level).isExternalPath()), suggestedFileSize); } private KeyValueDataFileWriter createDataFileWriter( - Path path, int level, FileSource fileSource) { + Path path, int level, FileSource fileSource, boolean isExternalPath) { return formatContext.thinModeEnabled() ? new KeyValueThinDataFileWriterImpl( fileIO, @@ -125,7 +129,8 @@ private KeyValueDataFileWriter createDataFileWriter( formatContext.compression(level), options, fileSource, - fileIndexOptions) + fileIndexOptions, + isExternalPath) : new KeyValueDataFileWriterImpl( fileIO, formatContext.writerFactory(level), @@ -139,7 +144,8 @@ private KeyValueDataFileWriter createDataFileWriter( formatContext.compression(level), options, fileSource, - fileIndexOptions); + fileIndexOptions, + isExternalPath); } public void deleteFile(DataFileMeta file) { diff --git a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueThinDataFileWriterImpl.java b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueThinDataFileWriterImpl.java index dd7ebb006764..4b5e1c842c8d 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueThinDataFileWriterImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueThinDataFileWriterImpl.java @@ -77,7 +77,8 @@ public KeyValueThinDataFileWriterImpl( String compression, CoreOptions options, FileSource fileSource, - FileIndexOptions fileIndexOptions) { + FileIndexOptions fileIndexOptions, + boolean isExternalPath) { super( fileIO, factory, @@ -92,7 +93,8 @@ public KeyValueThinDataFileWriterImpl( compression, options, fileSource, - fileIndexOptions); + fileIndexOptions, + isExternalPath); Map idToIndex = new HashMap<>(valueType.getFieldCount()); for (int i = 0; i < valueType.getFieldCount(); i++) { idToIndex.put(valueType.getFields().get(i).id(), i); diff --git a/paimon-core/src/main/java/org/apache/paimon/io/RowDataFileWriter.java b/paimon-core/src/main/java/org/apache/paimon/io/RowDataFileWriter.java index 1db35eba1f2f..97af4d067a3a 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/RowDataFileWriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/RowDataFileWriter.java @@ -66,7 +66,8 @@ public RowDataFileWriter( FileIndexOptions fileIndexOptions, FileSource fileSource, boolean asyncFileWrite, - boolean statsDenseStore) { + boolean statsDenseStore, + boolean isExternalPath) { super( fileIO, factory, @@ -76,7 +77,8 @@ public RowDataFileWriter( simpleStatsExtractor, fileCompression, statsCollectors, - asyncFileWrite); + asyncFileWrite, + isExternalPath); this.schemaId = schemaId; this.seqNumCounter = seqNumCounter; this.statsArraySerializer = new SimpleStatsConverter(writeSchema, statsDenseStore); @@ -111,7 +113,7 @@ public DataFileMeta result() throws IOException { dataFileIndexWriter == null ? DataFileIndexWriter.EMPTY_RESULT : dataFileIndexWriter.result(); - String externalPath = path.isExternalPath() ? path.toString() : null; + String externalPath = isExternalPath ? path.toString() : null; return DataFileMeta.forAppend( path.getName(), fileIO.getFileSize(path), diff --git a/paimon-core/src/main/java/org/apache/paimon/io/RowDataRollingFileWriter.java b/paimon-core/src/main/java/org/apache/paimon/io/RowDataRollingFileWriter.java index b929a4ae22af..3b4b05243d50 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/RowDataRollingFileWriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/RowDataRollingFileWriter.java @@ -64,7 +64,8 @@ public RowDataRollingFileWriter( fileIndexOptions, fileSource, asyncFileWrite, - statsDenseStore), + statsDenseStore, + pathFactory.isExternalPath()), targetFileSize); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/io/SingleFileWriter.java b/paimon-core/src/main/java/org/apache/paimon/io/SingleFileWriter.java index f303e8597870..7a3cc4143ba7 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/SingleFileWriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/SingleFileWriter.java @@ -54,6 +54,7 @@ public abstract class SingleFileWriter implements FileWriter { private long recordCount; protected boolean closed; + protected boolean isExternalPath; public SingleFileWriter( FileIO fileIO, @@ -61,7 +62,8 @@ public SingleFileWriter( Path path, Function converter, String compression, - boolean asyncWrite) { + boolean asyncWrite, + boolean isExternalPath) { this.fileIO = fileIO; this.path = path; this.converter = converter; @@ -84,6 +86,7 @@ public SingleFileWriter( this.recordCount = 0; this.closed = false; + this.isExternalPath = isExternalPath; } public Path path() { diff --git a/paimon-core/src/main/java/org/apache/paimon/io/StatsCollectingSingleFileWriter.java b/paimon-core/src/main/java/org/apache/paimon/io/StatsCollectingSingleFileWriter.java index 67a3fa6d1ace..0ab2de8c2bbb 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/StatsCollectingSingleFileWriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/StatsCollectingSingleFileWriter.java @@ -59,8 +59,9 @@ public StatsCollectingSingleFileWriter( @Nullable SimpleStatsExtractor simpleStatsExtractor, String compression, SimpleColStatsCollector.Factory[] statsCollectors, - boolean asyncWrite) { - super(fileIO, factory, path, converter, compression, asyncWrite); + boolean asyncWrite, + boolean isExternalPath) { + super(fileIO, factory, path, converter, compression, asyncWrite, isExternalPath); this.simpleStatsExtractor = simpleStatsExtractor; if (this.simpleStatsExtractor == null) { this.simpleStatsCollector = new SimpleStatsCollector(writeSchema, statsCollectors); diff --git a/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestFile.java b/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestFile.java index 03aa8a422411..b691eafc1161 100644 --- a/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestFile.java +++ b/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestFile.java @@ -137,8 +137,8 @@ public class ManifestEntryWriter extends SingleFileWriter Date: Fri, 3 Jan 2025 17:29:23 +0800 Subject: [PATCH 17/21] change data-file.external-paths.specific-fs to string --- .../generated/core_configuration.html | 4 +- .../java/org/apache/paimon/CoreOptions.java | 35 ++----------- .../paimon/fs/ExternalPathProvider.java | 51 ++++++++++--------- .../paimon/fs/ExternalPathProviderTest.java | 26 ++++++---- .../paimon/table/FileStoreTableFactory.java | 3 +- 5 files changed, 53 insertions(+), 66 deletions(-) diff --git a/docs/layouts/shortcodes/generated/core_configuration.html b/docs/layouts/shortcodes/generated/core_configuration.html index f89fc9b1ad42..bf83eb687bdf 100644 --- a/docs/layouts/shortcodes/generated/core_configuration.html +++ b/docs/layouts/shortcodes/generated/core_configuration.html @@ -1029,8 +1029,8 @@
    data-file.external-paths.specific-fs
    (none) -

    Enum

    - The specific file system of the external path when data-file.external-paths.strategy is set to specific-fs

    Possible values:
    • "S3": Select S3 as the write path for the external path.
    • "OSS": Select OSS as the write path for the external path.
    + String + The specific file system of the external path when data-file.external-paths.strategy is set to specific-fs, should be the prefix scheme of the external path, now supported are s3 and oss. diff --git a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java index 74cb95388a2f..05d9c0491687 100644 --- a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java +++ b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java @@ -139,15 +139,16 @@ public class CoreOptions implements Serializable { .withDescription( "The strategy of selecting an external path when writing data."); - public static final ConfigOption DATA_FILE_EXTERNAL_PATHS_SPECIFIC_FS = + public static final ConfigOption DATA_FILE_EXTERNAL_PATHS_SPECIFIC_FS = key("data-file.external-paths.specific-fs") - .enumType(ExternalFSStrategy.class) + .stringType() .noDefaultValue() .withDescription( "The specific file system of the external path when " + DATA_FILE_EXTERNAL_PATHS_STRATEGY.key() + " is set to " - + ExternalPathStrategy.SPECIFIC_FS); + + ExternalPathStrategy.SPECIFIC_FS + + ", should be the prefix scheme of the external path, now supported are s3 and oss."); // todo, this path is the table schema path, the name will be changed in the later PR. @ExcludeFromDocumentation("Internal use only") @@ -2215,7 +2216,7 @@ public ExternalPathStrategy externalPathStrategy() { return options.get(DATA_FILE_EXTERNAL_PATHS_STRATEGY); } - public ExternalFSStrategy externalSpecificFSStrategy() { + public String externalSpecificFSStrategy() { return options.get(DATA_FILE_EXTERNAL_PATHS_SPECIFIC_FS); } @@ -3060,32 +3061,6 @@ public InlineElement getDescription() { } } - /** Specifies the strategy for selecting specific filesystem storage paths. */ - public enum ExternalFSStrategy implements DescribedEnum { - S3("S3", "Select S3 as the write path for the external path."), - - OSS("OSS", "Select OSS as the write path for the external path."); - - private final String value; - - private final String description; - - ExternalFSStrategy(String value, String description) { - this.value = value; - this.description = description; - } - - @Override - public String toString() { - return value; - } - - @Override - public InlineElement getDescription() { - return text(description); - } - } - /** Specifies the local file type for lookup. */ public enum LookupLocalFileType implements DescribedEnum { SORT("sort", "Construct a sorted file for lookup."), diff --git a/paimon-common/src/main/java/org/apache/paimon/fs/ExternalPathProvider.java b/paimon-common/src/main/java/org/apache/paimon/fs/ExternalPathProvider.java index be22c8e65810..188a5af001cb 100644 --- a/paimon-common/src/main/java/org/apache/paimon/fs/ExternalPathProvider.java +++ b/paimon-common/src/main/java/org/apache/paimon/fs/ExternalPathProvider.java @@ -18,7 +18,6 @@ package org.apache.paimon.fs; -import org.apache.paimon.CoreOptions.ExternalFSStrategy; import org.apache.paimon.CoreOptions.ExternalPathStrategy; import org.apache.paimon.annotation.VisibleForTesting; @@ -32,11 +31,14 @@ /** Provider for external paths. */ public class ExternalPathProvider implements Serializable { - private final Map externalPathsMap; + private static final String S3 = "s3"; + private static final String OSS = "oss"; + + private final Map externalPathsMap; private final List externalPathsList; private final ExternalPathStrategy externalPathStrategy; - private final ExternalFSStrategy externalFSStrategy; + private final String externalFSStrategy; private int currentIndex; private boolean externalPathExists; private final String dbAndTableRelativePath; @@ -53,7 +55,7 @@ public ExternalPathProvider() { public ExternalPathProvider( String externalPaths, ExternalPathStrategy externalPathStrategy, - ExternalFSStrategy externalFSStrategy, + String externalFSStrategy, String dbAndTableRelativePath) { this.externalPathsMap = new HashMap<>(); this.externalPathsList = new ArrayList<>(); @@ -68,14 +70,22 @@ private void initExternalPaths(String externalPaths) { if (externalPaths == null) { return; } + + if (externalPathStrategy != null + && externalPathStrategy.equals(ExternalPathStrategy.SPECIFIC_FS)) { + if (externalFSStrategy == null) { + throw new IllegalArgumentException("external fs strategy should not be null: "); + } + } + String[] tmpArray = externalPaths.split(","); for (String part : tmpArray) { String path = part.trim(); - if (path.toLowerCase().startsWith("oss")) { - externalPathsMap.put(ExternalFSStrategy.OSS, new Path(path)); + if (path.toLowerCase().startsWith(OSS)) { + externalPathsMap.put(OSS, new Path(path)); externalPathsList.add(new Path(path)); - } else if (path.toLowerCase().startsWith("s3")) { - externalPathsMap.put(ExternalFSStrategy.S3, new Path(path)); + } else if (path.toLowerCase().startsWith(S3)) { + externalPathsMap.put(S3, new Path(path)); externalPathsList.add(new Path(path)); } else { throw new IllegalArgumentException("Unsupported external path: " + path); @@ -112,23 +122,17 @@ public Optional getNextExternalPath() { } private Optional getSpecificFSExternalPath() { - switch (externalFSStrategy) { + switch (externalFSStrategy.toLowerCase()) { case S3: - if (!externalPathsMap.containsKey(ExternalFSStrategy.S3)) { + if (!externalPathsMap.containsKey(S3)) { return Optional.empty(); } - return Optional.of( - new Path( - externalPathsMap.get(ExternalFSStrategy.S3), - dbAndTableRelativePath)); + return Optional.of(new Path(externalPathsMap.get(S3), dbAndTableRelativePath)); case OSS: - if (!externalPathsMap.containsKey(ExternalFSStrategy.OSS)) { + if (!externalPathsMap.containsKey(OSS)) { return Optional.empty(); } - return Optional.of( - new Path( - externalPathsMap.get(ExternalFSStrategy.OSS), - dbAndTableRelativePath)); + return Optional.of(new Path(externalPathsMap.get(OSS), dbAndTableRelativePath)); default: throw new IllegalArgumentException( "Unsupported external fs strategy: " + externalFSStrategy); @@ -145,7 +149,7 @@ public boolean externalPathExists() { } @VisibleForTesting - public Map getExternalPathsMap() { + public Map getExternalPathsMap() { return externalPathsMap; } @@ -169,21 +173,22 @@ public boolean equals(Object o) { && externalPathsMap.equals(that.externalPathsMap) && externalPathsList.equals(that.externalPathsList) && externalPathStrategy == that.externalPathStrategy - && externalFSStrategy == that.externalFSStrategy + && Objects.equals(externalFSStrategy, that.externalFSStrategy) && Objects.equals(dbAndTableRelativePath, that.dbAndTableRelativePath); } @Override public String toString() { return "ExternalPathProvider{" - + " externalPathsMap=" + + ", externalPathsMap=" + externalPathsMap + ", externalPathsList=" + externalPathsList + ", externalPathStrategy=" + externalPathStrategy - + ", externalFSStrategy=" + + ", externalFSStrategy='" + externalFSStrategy + + '\'' + ", currentIndex=" + currentIndex + ", externalPathExists=" diff --git a/paimon-common/src/test/java/org/apache/paimon/fs/ExternalPathProviderTest.java b/paimon-common/src/test/java/org/apache/paimon/fs/ExternalPathProviderTest.java index 5e98c9f87e28..9b767c6f98e9 100644 --- a/paimon-common/src/test/java/org/apache/paimon/fs/ExternalPathProviderTest.java +++ b/paimon-common/src/test/java/org/apache/paimon/fs/ExternalPathProviderTest.java @@ -18,7 +18,6 @@ package org.apache.paimon.fs; -import org.apache.paimon.CoreOptions.ExternalFSStrategy; import org.apache.paimon.CoreOptions.ExternalPathStrategy; import org.assertj.core.api.Assertions; @@ -31,7 +30,6 @@ /** Test for {@link ExternalPathProvider}. */ public class ExternalPathProviderTest { - private ExternalPathProvider provider; @BeforeEach @@ -72,7 +70,7 @@ public void testGetNextExternalPathSpecificFS() { new ExternalPathProvider( "oss://bucket1/path1,s3://bucket2/path2", ExternalPathStrategy.SPECIFIC_FS, - ExternalFSStrategy.OSS, + "OSS", "db/table"); Optional path = provider.getNextExternalPath(); @@ -86,7 +84,7 @@ public void testGetNextExternalPathNone() { new ExternalPathProvider( "oss://bucket1/path1,s3://bucket2/path2", ExternalPathStrategy.NONE, - ExternalFSStrategy.OSS, + "OSS", "db/table"); Optional path = provider.getNextExternalPath(); @@ -100,7 +98,7 @@ public void testUnsupportedExternalPath() { new ExternalPathProvider( "hdfs://bucket1/path1", ExternalPathStrategy.ROUND_ROBIN, - ExternalFSStrategy.OSS, + "oss", "db/table"); }) .isInstanceOf(IllegalArgumentException.class); @@ -110,11 +108,21 @@ public void testUnsupportedExternalPath() { public void testUnsupportedExternalFSStrategy() { provider = new ExternalPathProvider( - "oss://bucket1/path1", - ExternalPathStrategy.SPECIFIC_FS, - ExternalFSStrategy.S3, - "db/table"); + "oss://bucket1/path1", ExternalPathStrategy.SPECIFIC_FS, "S3", "db/table"); Optional path = provider.getNextExternalPath(); assertThat(path.isPresent()).isFalse(); } + + @Test + public void testExternalFSStrategyNull() { + Assertions.assertThatThrownBy( + () -> { + new ExternalPathProvider( + "oss://bucket1/path1", + ExternalPathStrategy.SPECIFIC_FS, + null, + "db/table"); + }) + .isInstanceOf(IllegalArgumentException.class); + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTableFactory.java b/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTableFactory.java index bf5d4ba5bb30..b0a965de3787 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTableFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTableFactory.java @@ -19,7 +19,6 @@ package org.apache.paimon.table; import org.apache.paimon.CoreOptions; -import org.apache.paimon.CoreOptions.ExternalFSStrategy; import org.apache.paimon.CoreOptions.ExternalPathStrategy; import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.fs.ExternalPathProvider; @@ -84,7 +83,7 @@ private static ExternalPathProvider getExternalPathProvider( CoreOptions coreOptions = CoreOptions.fromMap(tableSchema.options()); String externalPaths = coreOptions.dataFileExternalPaths(); ExternalPathStrategy externalPathStrategy = coreOptions.externalPathStrategy(); - ExternalFSStrategy externalSpecificFSStrategy = coreOptions.externalSpecificFSStrategy(); + String externalSpecificFSStrategy = coreOptions.externalSpecificFSStrategy(); String dbAndTablePath = tablePath.getParent().getName() + "/" + tablePath.getName(); return new ExternalPathProvider( externalPaths, externalPathStrategy, externalSpecificFSStrategy, dbAndTablePath); From 3a6385735b2d138155c33f4ebf1dac589d54a030 Mon Sep 17 00:00:00 2001 From: HouliangQi Date: Fri, 3 Jan 2025 18:48:08 +0800 Subject: [PATCH 18/21] Revert "fix the clone tests" This reverts commit 3a00acc2f2b316d70ab0df7ac02c38bc902bdd7a. --- .../org/apache/paimon/io/DataFileMeta.java | 22 ----- .../apache/paimon/manifest/ManifestFile.java | 7 +- .../paimon/flink/clone/CloneFileInfo.java | 17 +--- .../paimon/flink/clone/CopyFileOperator.java | 62 ++++--------- .../apache/paimon/flink/clone/FileType.java | 32 ------- .../clone/PickFilesForCloneOperator.java | 21 ++--- .../paimon/flink/clone/PickFilesUtil.java | 87 ++++++++----------- .../flink/action/CloneActionITCase.java | 14 +-- 8 files changed, 68 insertions(+), 194 deletions(-) delete mode 100644 paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/FileType.java diff --git a/paimon-core/src/main/java/org/apache/paimon/io/DataFileMeta.java b/paimon-core/src/main/java/org/apache/paimon/io/DataFileMeta.java index 176f49a6091a..b164b60fe525 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/DataFileMeta.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/DataFileMeta.java @@ -484,28 +484,6 @@ public DataFileMeta copy(List newExtraFiles) { externalPath); } - public DataFileMeta copy(String newExternalPath) { - return new DataFileMeta( - fileName, - fileSize, - rowCount, - minKey, - maxKey, - keyStats, - valueStats, - minSequenceNumber, - maxSequenceNumber, - schemaId, - level, - extraFiles, - creationTime, - deleteRowCount, - embeddedIndex, - fileSource, - valueStatsCols, - newExternalPath); - } - public DataFileMeta copy(byte[] newEmbeddedIndex) { return new DataFileMeta( fileName, diff --git a/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestFile.java b/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestFile.java index b691eafc1161..34bf77345a86 100644 --- a/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestFile.java +++ b/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestFile.java @@ -116,12 +116,7 @@ public RollingFileWriter createRollingWriter() suggestedFileSize); } - public ManifestEntryWriter createManifestEntryWriter(Path manifestPath) { - return new ManifestEntryWriter(writerFactory, manifestPath, compression); - } - - /** Writer for {@link ManifestEntry}. */ - public class ManifestEntryWriter extends SingleFileWriter { + private class ManifestEntryWriter extends SingleFileWriter { private final SimpleStatsCollector partitionStatsCollector; private final SimpleStatsConverter partitionStatsSerializer; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CloneFileInfo.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CloneFileInfo.java index 56ae6c8300e2..5c0ac75e167f 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CloneFileInfo.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CloneFileInfo.java @@ -25,19 +25,16 @@ public class CloneFileInfo { private final String filePathExcludeTableRoot; private final String sourceIdentifier; private final String targetIdentifier; - private final FileType fileType; public CloneFileInfo( String sourceFilePath, String filePathExcludeTableRoot, String sourceIdentifier, - String targetIdentifier, - FileType fileType) { + String targetIdentifier) { this.sourceFilePath = sourceFilePath; this.filePathExcludeTableRoot = filePathExcludeTableRoot; this.sourceIdentifier = sourceIdentifier; this.targetIdentifier = targetIdentifier; - this.fileType = fileType; } public String getSourceFilePath() { @@ -56,18 +53,10 @@ public String getTargetIdentifier() { return targetIdentifier; } - public FileType getFileType() { - return fileType; - } - @Override public String toString() { return String.format( - "{ sourceFilePath: %s, filePathExcludeTableRoot: %s, sourceIdentifier: %s, targetIdentifier: %s, fileType: %s }", - sourceFilePath, - filePathExcludeTableRoot, - sourceIdentifier, - targetIdentifier, - fileType); + "{ sourceFilePath: %s, filePathExcludeTableRoot: %s, sourceIdentifier: %s, targetIdentifier: %s }", + sourceFilePath, filePathExcludeTableRoot, sourceIdentifier, targetIdentifier); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CopyFileOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CopyFileOperator.java index 0fee72935553..e7002cce1eec 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CopyFileOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/CopyFileOperator.java @@ -19,18 +19,12 @@ package org.apache.paimon.flink.clone; import org.apache.paimon.CoreOptions; -import org.apache.paimon.FileStore; import org.apache.paimon.catalog.Catalog; -import org.apache.paimon.catalog.Catalog.TableNotExistException; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.flink.FlinkCatalogFactory; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; -import org.apache.paimon.manifest.ManifestEntry; -import org.apache.paimon.manifest.ManifestFile; -import org.apache.paimon.manifest.ManifestFile.ManifestEntryWriter; import org.apache.paimon.options.Options; -import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; import org.apache.paimon.utils.IOUtils; @@ -40,10 +34,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.ArrayList; import java.util.HashMap; -import java.util.List; import java.util.Map; /** A Operator to copy files. */ @@ -83,6 +74,18 @@ public void processElement(StreamRecord streamRecord) throws Exce FileIO sourceTableFileIO = sourceCatalog.fileIO(); FileIO targetTableFileIO = targetCatalog.fileIO(); + + Path sourceTableRootPath = + srcLocations.computeIfAbsent( + cloneFileInfo.getSourceIdentifier(), + key -> { + try { + return pathOfTable( + sourceCatalog.getTable(Identifier.fromString(key))); + } catch (Catalog.TableNotExistException e) { + throw new RuntimeException(e); + } + }); Path targetTableRootPath = targetLocations.computeIfAbsent( cloneFileInfo.getTargetIdentifier(), @@ -122,14 +125,9 @@ public void processElement(StreamRecord streamRecord) throws Exce if (LOG.isDebugEnabled()) { LOG.debug("Begin copy file from {} to {}.", sourcePath, targetPath); } - - if (cloneFileInfo.getFileType() == FileType.MANIFEST_FILE) { - copyManifestFile(sourcePath, targetPath, cloneFileInfo); - } else { - IOUtils.copyBytes( - sourceTableFileIO.newInputStream(sourcePath), - targetTableFileIO.newOutputStream(targetPath, true)); - } + IOUtils.copyBytes( + sourceTableFileIO.newInputStream(sourcePath), + targetTableFileIO.newOutputStream(targetPath, true)); if (LOG.isDebugEnabled()) { LOG.debug("End copy file from {} to {}.", sourcePath, targetPath); } @@ -141,36 +139,6 @@ private Path pathOfTable(Table table) { return new Path(table.options().get(CoreOptions.PATH.key())); } - private void copyManifestFile(Path sourcePath, Path targetPath, CloneFileInfo cloneFileInfo) - throws IOException, TableNotExistException { - Identifier sourceIdentifier = Identifier.fromString(cloneFileInfo.getSourceIdentifier()); - FileStoreTable sourceTable = (FileStoreTable) sourceCatalog.getTable(sourceIdentifier); - FileStore store = sourceTable.store(); - ManifestFile manifestFile = store.manifestFileFactory().create(); - - List manifestEntries = - manifestFile.readWithIOException(sourcePath.getName()); - List targetManifestEntries = new ArrayList<>(manifestEntries.size()); - - // clone job will clone the source path to target warehouse path, so the target external - // path is null - for (ManifestEntry manifestEntry : manifestEntries) { - ManifestEntry newManifestEntry = - new ManifestEntry( - manifestEntry.kind(), - manifestEntry.partition(), - manifestEntry.bucket(), - manifestEntry.totalBuckets(), - manifestEntry.file().copy((String) null)); - targetManifestEntries.add(newManifestEntry); - } - - ManifestEntryWriter manifestEntryWriter = - manifestFile.createManifestEntryWriter(targetPath); - manifestEntryWriter.write(targetManifestEntries); - manifestEntryWriter.close(); - } - @Override public void close() throws Exception { if (sourceCatalog != null) { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/FileType.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/FileType.java deleted file mode 100644 index f6a92c74c3f7..000000000000 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/FileType.java +++ /dev/null @@ -1,32 +0,0 @@ -/* - * 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.flink.clone; - -/** The clone file type. */ -public enum FileType { - MANIFEST_FILE, - MANIFEST_LIST_FILE, - INDEX_FILE, - DATA_FILE, - SNAPSHOT_FILE, - SCHEMA_FILE, - CHANGELOG_MANIFEST_LIST_FILE, - STATISTICS_FILE, - OTHER_FILE -} diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/PickFilesForCloneOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/PickFilesForCloneOperator.java index 70695541f0e0..f58d3acafdb9 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/PickFilesForCloneOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/PickFilesForCloneOperator.java @@ -115,22 +115,19 @@ private static Schema newSchemaFromTableSchema(TableSchema tableSchema) { } private List toCloneFileInfos( - Map> filesMap, + List files, Path sourceTableRoot, String sourceIdentifier, String targetIdentifier) { List result = new ArrayList<>(); - for (Map.Entry> entry : filesMap.entrySet()) { - for (Path file : entry.getValue()) { - Path relativePath = getPathExcludeTableRoot(file, sourceTableRoot); - result.add( - new CloneFileInfo( - file.toUri().toString(), - relativePath.toString(), - sourceIdentifier, - targetIdentifier, - entry.getKey())); - } + for (Path file : files) { + Path relativePath = getPathExcludeTableRoot(file, sourceTableRoot); + result.add( + new CloneFileInfo( + file.toUri().toString(), + relativePath.toString(), + sourceIdentifier, + targetIdentifier)); } return result; } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/PickFilesUtil.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/PickFilesUtil.java index d3a5f8debf0e..9de974d047f1 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/PickFilesUtil.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/clone/PickFilesUtil.java @@ -38,9 +38,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -50,7 +48,7 @@ public class PickFilesUtil { private static final int READ_FILE_RETRY_NUM = 3; private static final int READ_FILE_RETRY_INTERVAL = 5; - public static Map> getUsedFilesForLatestSnapshot(FileStoreTable table) { + public static List getUsedFilesForLatestSnapshot(FileStoreTable table) { FileStore store = table.store(); SnapshotManager snapshotManager = store.snapshotManager(); Snapshot snapshot = snapshotManager.latestSnapshot(); @@ -58,33 +56,31 @@ public static Map> getUsedFilesForLatestSnapshot(FileStoreT SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.location()); IndexFileHandler indexFileHandler = store.newIndexFileHandler(); - Map> filesMap = new HashMap<>(); + List files = new ArrayList<>(); if (snapshot != null) { - filesMap.computeIfAbsent(FileType.SNAPSHOT_FILE, k -> new ArrayList<>()) - .add(snapshotManager.snapshotPath(snapshot.id())); - getUsedFilesInternal( - snapshot, - store.pathFactory(), - store.newScan(), - manifestList, - indexFileHandler, - filesMap); + files.add(snapshotManager.snapshotPath(snapshot.id())); + files.addAll( + getUsedFilesInternal( + snapshot, + store.pathFactory(), + store.newScan(), + manifestList, + indexFileHandler)); } for (long id : schemaManager.listAllIds()) { - filesMap.computeIfAbsent(FileType.SCHEMA_FILE, k -> new ArrayList<>()) - .add(schemaManager.toSchemaPath(id)); + files.add(schemaManager.toSchemaPath(id)); } - return filesMap; + return files; } - private static void getUsedFilesInternal( + private static List getUsedFilesInternal( Snapshot snapshot, FileStorePathFactory pathFactory, FileStoreScan scan, ManifestList manifestList, - IndexFileHandler indexFileHandler, - Map> filesMap) { - addManifestList(filesMap, snapshot, pathFactory); + IndexFileHandler indexFileHandler) { + List files = new ArrayList<>(); + addManifestList(files, snapshot, pathFactory); try { // try to read manifests @@ -92,17 +88,16 @@ private static void getUsedFilesInternal( retryReadingFiles( () -> readAllManifestsWithIOException(snapshot, manifestList)); if (manifestFileMetas == null) { - return; + return Collections.emptyList(); } List manifestFileName = manifestFileMetas.stream() .map(ManifestFileMeta::fileName) .collect(Collectors.toList()); - filesMap.computeIfAbsent(FileType.MANIFEST_FILE, k -> new ArrayList<>()) - .addAll( - manifestFileName.stream() - .map(pathFactory::toManifestFilePath) - .collect(Collectors.toList())); + files.addAll( + manifestFileName.stream() + .map(pathFactory::toManifestFilePath) + .collect(Collectors.toList())); // try to read data files List dataFiles = new ArrayList<>(); @@ -124,52 +119,44 @@ private static void getUsedFilesInternal( // deleted. Older files however, are from previous partitions and should not be changed // very often. Collections.reverse(dataFiles); - filesMap.computeIfAbsent(FileType.DATA_FILE, k -> new ArrayList<>()).addAll(dataFiles); + files.addAll(dataFiles); // try to read index files String indexManifest = snapshot.indexManifest(); if (indexManifest != null && indexFileHandler.existsManifest(indexManifest)) { - filesMap.computeIfAbsent(FileType.INDEX_FILE, k -> new ArrayList<>()) - .add(pathFactory.indexManifestFileFactory().toPath(indexManifest)); + files.add(pathFactory.indexManifestFileFactory().toPath(indexManifest)); List indexManifestEntries = retryReadingFiles( () -> indexFileHandler.readManifestWithIOException(indexManifest)); - if (indexManifestEntries != null) { - indexManifestEntries.stream() - .map(IndexManifestEntry::indexFile) - .map(indexFileHandler::filePath) - .forEach( - filePath -> - filesMap.computeIfAbsent( - FileType.INDEX_FILE, - k -> new ArrayList<>()) - .add(filePath)); + if (indexManifestEntries == null) { + return Collections.emptyList(); } + + indexManifestEntries.stream() + .map(IndexManifestEntry::indexFile) + .map(indexFileHandler::filePath) + .forEach(files::add); } // add statistic file if (snapshot.statistics() != null) { - filesMap.computeIfAbsent(FileType.STATISTICS_FILE, k -> new ArrayList<>()) - .add(pathFactory.statsFileFactory().toPath(snapshot.statistics())); + files.add(pathFactory.statsFileFactory().toPath(snapshot.statistics())); } } catch (IOException e) { throw new RuntimeException(e); } + + return files; } private static void addManifestList( - Map> filesMap, - Snapshot snapshot, - FileStorePathFactory pathFactory) { - filesMap.computeIfAbsent(FileType.MANIFEST_LIST_FILE, k -> new ArrayList<>()) - .add(pathFactory.toManifestListPath(snapshot.baseManifestList())); - filesMap.get(FileType.MANIFEST_LIST_FILE) - .add(pathFactory.toManifestListPath(snapshot.deltaManifestList())); + List used, Snapshot snapshot, FileStorePathFactory pathFactory) { + used.add(pathFactory.toManifestListPath(snapshot.baseManifestList())); + used.add(pathFactory.toManifestListPath(snapshot.deltaManifestList())); String changelogManifestList = snapshot.changelogManifestList(); if (changelogManifestList != null) { - filesMap.computeIfAbsent(FileType.CHANGELOG_MANIFEST_LIST_FILE, k -> new ArrayList<>()) - .add(pathFactory.toManifestListPath(changelogManifestList)); + used.add(pathFactory.toManifestListPath(changelogManifestList)); } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CloneActionITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CloneActionITCase.java index b177f98024d9..a55b01cc203b 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CloneActionITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/CloneActionITCase.java @@ -22,7 +22,6 @@ import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.catalog.CatalogFactory; import org.apache.paimon.catalog.Identifier; -import org.apache.paimon.flink.clone.FileType; import org.apache.paimon.flink.clone.PickFilesUtil; import org.apache.paimon.fs.Path; import org.apache.paimon.table.FileStoreTable; @@ -41,7 +40,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.Map; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; @@ -459,10 +457,7 @@ private void compareCloneFiles( String targetTableName) throws Exception { FileStoreTable targetTable = getFileStoreTable(targetWarehouse, targetDb, targetTableName); - Map> filesMap = - PickFilesUtil.getUsedFilesForLatestSnapshot(targetTable); - List targetTableFiles = - filesMap.values().stream().flatMap(List::stream).collect(Collectors.toList()); + List targetTableFiles = PickFilesUtil.getUsedFilesForLatestSnapshot(targetTable); List> filesPathInfoList = targetTableFiles.stream() .map( @@ -478,11 +473,8 @@ private void compareCloneFiles( for (Pair filesPathInfo : filesPathInfoList) { Path sourceTableFile = new Path(tableLocation.toString() + filesPathInfo.getRight()); assertThat(sourceTable.fileIO().exists(sourceTableFile)).isTrue(); - // TODO, need to check the manifest file's content - if (!filesPathInfo.getLeft().toString().contains("/manifest/manifest-")) { - assertThat(targetTable.fileIO().getFileSize(filesPathInfo.getLeft())) - .isEqualTo(sourceTable.fileIO().getFileSize(sourceTableFile)); - } + assertThat(targetTable.fileIO().getFileSize(filesPathInfo.getLeft())) + .isEqualTo(sourceTable.fileIO().getFileSize(sourceTableFile)); } } From 928a02dc095b5c25f4bd6a770d263d6e8332b494 Mon Sep 17 00:00:00 2001 From: HouliangQi Date: Mon, 6 Jan 2025 00:11:11 +0800 Subject: [PATCH 19/21] fix review comments and add some IT tests --- .../java/org/apache/paimon/CoreOptions.java | 5 +- .../fs/DataFileExternalPathProvider.java | 82 +++++++ ...er.java => TableExternalPathProvider.java} | 105 ++++----- .../paimon/fs/ExternalPathProviderTest.java | 128 ----------- .../fs/TableExternalPathProviderTest.java | 194 ++++++++++++++++ .../org/apache/paimon/AbstractFileStore.java | 18 +- .../apache/paimon/AppendOnlyFileStore.java | 14 +- .../org/apache/paimon/KeyValueFileStore.java | 14 +- .../apache/paimon/io/DataFilePathFactory.java | 28 +-- .../privilege/PrivilegedFileStoreTable.java | 6 - .../paimon/table/AbstractFileStoreTable.java | 30 +-- .../table/AppendOnlyFileStoreTable.java | 16 +- .../org/apache/paimon/table/DataTable.java | 3 - .../table/FallbackReadFileStoreTable.java | 9 +- .../paimon/table/FileStoreTableFactory.java | 72 +----- .../table/PrimaryKeyFileStoreTable.java | 16 +- .../paimon/table/object/ObjectTable.java | 6 - .../paimon/table/system/AuditLogTable.java | 6 - .../table/system/CompactBucketsTable.java | 6 - .../paimon/table/system/FileMonitorTable.java | 6 - .../table/system/ReadOptimizedTable.java | 6 - .../paimon/utils/FileStorePathFactory.java | 18 +- .../apache/paimon/TestAppendFileStore.java | 4 +- .../java/org/apache/paimon/TestFileStore.java | 4 +- .../paimon/append/AppendOnlyWriterTest.java | 2 - .../paimon/format/FileFormatSuffixTest.java | 2 - .../paimon/io/DataFilePathFactoryTest.java | 3 - .../paimon/io/KeyValueFileReadWriteTest.java | 5 +- .../paimon/io/RollingFileWriterTest.java | 2 - .../manifest/ManifestFileMetaTestBase.java | 3 +- .../paimon/manifest/ManifestFileTest.java | 3 +- .../paimon/manifest/ManifestListTest.java | 3 +- .../source/snapshot/ScannerTestBase.java | 8 +- .../source/snapshot/SnapshotReaderTest.java | 8 +- .../utils/FileStorePathFactoryTest.java | 5 +- .../flink/lookup/LookupFileStoreTable.java | 6 - .../paimon/flink/AppendOnlyTableITCase.java | 210 ++++++++++++++++++ .../flink/PrimaryKeyFileStoreTableITCase.java | 133 +++++++++++ .../paimon/flink/sink/FlinkSinkTest.java | 4 +- .../source/TestChangelogDataReadWrite.java | 3 +- .../paimon/spark/SparkFileIndexITCase.java | 3 +- 41 files changed, 744 insertions(+), 455 deletions(-) create mode 100644 paimon-common/src/main/java/org/apache/paimon/fs/DataFileExternalPathProvider.java rename paimon-common/src/main/java/org/apache/paimon/fs/{ExternalPathProvider.java => TableExternalPathProvider.java} (62%) delete mode 100644 paimon-common/src/test/java/org/apache/paimon/fs/ExternalPathProviderTest.java create mode 100644 paimon-common/src/test/java/org/apache/paimon/fs/TableExternalPathProviderTest.java diff --git a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java index 05d9c0491687..8c59cdcd4169 100644 --- a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java +++ b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java @@ -2208,15 +2208,18 @@ public PartitionExpireStrategy partitionExpireStrategy() { return options.get(PARTITION_EXPIRATION_STRATEGY); } + @Nullable public String dataFileExternalPaths() { return options.get(DATA_FILE_EXTERNAL_PATHS); } + @Nullable public ExternalPathStrategy externalPathStrategy() { return options.get(DATA_FILE_EXTERNAL_PATHS_STRATEGY); } - public String externalSpecificFSStrategy() { + @Nullable + public String externalSpecificFS() { return options.get(DATA_FILE_EXTERNAL_PATHS_SPECIFIC_FS); } diff --git a/paimon-common/src/main/java/org/apache/paimon/fs/DataFileExternalPathProvider.java b/paimon-common/src/main/java/org/apache/paimon/fs/DataFileExternalPathProvider.java new file mode 100644 index 000000000000..71e6f78184cd --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/fs/DataFileExternalPathProvider.java @@ -0,0 +1,82 @@ +/* + * 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 javax.annotation.Nullable; + +import java.io.Serializable; +import java.util.Objects; +import java.util.Optional; + +/** Provider for external data paths. */ +public class DataFileExternalPathProvider implements Serializable { + @Nullable private final TableExternalPathProvider tableExternalPathProvider; + private final Path relativeBucketPath; + + public DataFileExternalPathProvider( + @Nullable TableExternalPathProvider tableExternalPathProvider, + Path relativeBucketPath) { + this.tableExternalPathProvider = tableExternalPathProvider; + this.relativeBucketPath = relativeBucketPath; + } + + /** + * Get the next external data path. + * + * @return the next external data path + */ + public Optional getNextExternalDataPath() { + return Optional.ofNullable(tableExternalPathProvider) + .flatMap(TableExternalPathProvider::getNextExternalPath) + .map(path -> new Path(path, relativeBucketPath)); + } + + public boolean externalPathExists() { + return tableExternalPathProvider != null && tableExternalPathProvider.externalPathExists(); + } + + @Override + public final boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof DataFileExternalPathProvider)) { + return false; + } + + DataFileExternalPathProvider that = (DataFileExternalPathProvider) o; + return Objects.equals(tableExternalPathProvider, that.tableExternalPathProvider) + && Objects.equals(relativeBucketPath, that.relativeBucketPath); + } + + @Override + public int hashCode() { + return Objects.hash(tableExternalPathProvider, relativeBucketPath); + } + + @Override + public String toString() { + return "DataFileExternalPathProvider{" + + " externalPathProvider=" + + tableExternalPathProvider + + ", relativeBucketPath=" + + relativeBucketPath + + "}"; + } +} diff --git a/paimon-common/src/main/java/org/apache/paimon/fs/ExternalPathProvider.java b/paimon-common/src/main/java/org/apache/paimon/fs/TableExternalPathProvider.java similarity index 62% rename from paimon-common/src/main/java/org/apache/paimon/fs/ExternalPathProvider.java rename to paimon-common/src/main/java/org/apache/paimon/fs/TableExternalPathProvider.java index 188a5af001cb..8ae94e22b72d 100644 --- a/paimon-common/src/main/java/org/apache/paimon/fs/ExternalPathProvider.java +++ b/paimon-common/src/main/java/org/apache/paimon/fs/TableExternalPathProvider.java @@ -28,42 +28,34 @@ import java.util.Map; import java.util.Objects; import java.util.Optional; +import java.util.Random; /** Provider for external paths. */ -public class ExternalPathProvider implements Serializable { - private static final String S3 = "s3"; - private static final String OSS = "oss"; - +public class TableExternalPathProvider implements Serializable { private final Map externalPathsMap; private final List externalPathsList; private final ExternalPathStrategy externalPathStrategy; - private final String externalFSStrategy; - private int currentIndex; + private final String externalSpecificFS; + private int currentIndex = 0; private boolean externalPathExists; - private final String dbAndTableRelativePath; - - @VisibleForTesting - public ExternalPathProvider() { - this.externalPathsMap = new HashMap<>(); - this.externalPathsList = new ArrayList<>(); - this.externalPathStrategy = ExternalPathStrategy.NONE; - this.externalFSStrategy = null; - this.dbAndTableRelativePath = null; - } - public ExternalPathProvider( + public TableExternalPathProvider( String externalPaths, ExternalPathStrategy externalPathStrategy, - String externalFSStrategy, - String dbAndTableRelativePath) { + String externalSpecificFS) { this.externalPathsMap = new HashMap<>(); this.externalPathsList = new ArrayList<>(); this.externalPathStrategy = externalPathStrategy; - this.externalFSStrategy = externalFSStrategy; - this.dbAndTableRelativePath = dbAndTableRelativePath; - this.currentIndex = 0; + if (externalSpecificFS != null) { + this.externalSpecificFS = externalSpecificFS.toLowerCase(); + } else { + this.externalSpecificFS = null; + } initExternalPaths(externalPaths); + if (!externalPathsList.isEmpty()) { + this.currentIndex = new Random().nextInt(externalPathsList.size()); + } } private void initExternalPaths(String externalPaths) { @@ -71,24 +63,27 @@ private void initExternalPaths(String externalPaths) { return; } + String[] tmpArray = externalPaths.split(","); + for (String s : tmpArray) { + Path path = new Path(s.trim()); + String scheme = path.toUri().getScheme(); + if (scheme == null) { + throw new IllegalArgumentException("scheme should not be null: " + path); + } + scheme = scheme.toLowerCase(); + externalPathsMap.put(scheme, path); + externalPathsList.add(path); + } + if (externalPathStrategy != null && externalPathStrategy.equals(ExternalPathStrategy.SPECIFIC_FS)) { - if (externalFSStrategy == null) { - throw new IllegalArgumentException("external fs strategy should not be null: "); + if (externalSpecificFS == null) { + throw new IllegalArgumentException("external specific fs should not be null: "); } - } - String[] tmpArray = externalPaths.split(","); - for (String part : tmpArray) { - String path = part.trim(); - if (path.toLowerCase().startsWith(OSS)) { - externalPathsMap.put(OSS, new Path(path)); - externalPathsList.add(new Path(path)); - } else if (path.toLowerCase().startsWith(S3)) { - externalPathsMap.put(S3, new Path(path)); - externalPathsList.add(new Path(path)); - } else { - throw new IllegalArgumentException("Unsupported external path: " + path); + if (!externalPathsMap.containsKey(externalSpecificFS)) { + throw new IllegalArgumentException( + "external specific fs not found: " + externalSpecificFS); } } @@ -122,26 +117,15 @@ public Optional getNextExternalPath() { } private Optional getSpecificFSExternalPath() { - switch (externalFSStrategy.toLowerCase()) { - case S3: - if (!externalPathsMap.containsKey(S3)) { - return Optional.empty(); - } - return Optional.of(new Path(externalPathsMap.get(S3), dbAndTableRelativePath)); - case OSS: - if (!externalPathsMap.containsKey(OSS)) { - return Optional.empty(); - } - return Optional.of(new Path(externalPathsMap.get(OSS), dbAndTableRelativePath)); - default: - throw new IllegalArgumentException( - "Unsupported external fs strategy: " + externalFSStrategy); + if (!externalPathsMap.containsKey(externalSpecificFS)) { + return Optional.empty(); } + return Optional.of(externalPathsMap.get(externalSpecificFS)); } private Optional getRoundRobinPath() { currentIndex = (currentIndex + 1) % externalPathsList.size(); - return Optional.of(new Path(externalPathsList.get(currentIndex), dbAndTableRelativePath)); + return Optional.of(externalPathsList.get(currentIndex)); } public boolean externalPathExists() { @@ -167,35 +151,31 @@ public boolean equals(Object o) { return false; } - ExternalPathProvider that = (ExternalPathProvider) o; + TableExternalPathProvider that = (TableExternalPathProvider) o; return currentIndex == that.currentIndex && externalPathExists == that.externalPathExists && externalPathsMap.equals(that.externalPathsMap) && externalPathsList.equals(that.externalPathsList) && externalPathStrategy == that.externalPathStrategy - && Objects.equals(externalFSStrategy, that.externalFSStrategy) - && Objects.equals(dbAndTableRelativePath, that.dbAndTableRelativePath); + && Objects.equals(externalSpecificFS, that.externalSpecificFS); } @Override public String toString() { return "ExternalPathProvider{" - + ", externalPathsMap=" + + " externalPathsMap=" + externalPathsMap + ", externalPathsList=" + externalPathsList + ", externalPathStrategy=" + externalPathStrategy - + ", externalFSStrategy='" - + externalFSStrategy + + ", externalSpecificFS='" + + externalSpecificFS + '\'' + ", currentIndex=" + currentIndex + ", externalPathExists=" + externalPathExists - + ", dbAndTableRelativePath='" - + dbAndTableRelativePath - + '\'' + "}"; } @@ -205,9 +185,8 @@ public int hashCode() { externalPathsMap, externalPathsList, externalPathStrategy, - externalFSStrategy, + externalSpecificFS, currentIndex, - externalPathExists, - dbAndTableRelativePath); + externalPathExists); } } diff --git a/paimon-common/src/test/java/org/apache/paimon/fs/ExternalPathProviderTest.java b/paimon-common/src/test/java/org/apache/paimon/fs/ExternalPathProviderTest.java deleted file mode 100644 index 9b767c6f98e9..000000000000 --- a/paimon-common/src/test/java/org/apache/paimon/fs/ExternalPathProviderTest.java +++ /dev/null @@ -1,128 +0,0 @@ -/* - * 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.CoreOptions.ExternalPathStrategy; - -import org.assertj.core.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -import java.util.Optional; - -import static org.assertj.core.api.Assertions.assertThat; - -/** Test for {@link ExternalPathProvider}. */ -public class ExternalPathProviderTest { - private ExternalPathProvider provider; - - @BeforeEach - public void setUp() { - provider = - new ExternalPathProvider( - "oss://bucket1/path1,s3://bucket2/path2", - ExternalPathStrategy.ROUND_ROBIN, - null, - "db/table"); - } - - @Test - public void testInitExternalPaths() { - assertThat(provider.externalPathExists()).isTrue(); - assertThat(provider.getExternalPathsMap().size()).isEqualTo(2); - assertThat(provider.getExternalPathsList().size()).isEqualTo(2); - } - - @Test - public void testGetNextExternalPathRoundRobin() { - Optional path1 = provider.getNextExternalPath(); - assertThat(path1.isPresent()).isTrue(); - assertThat(path1.get().toString()).isEqualTo("s3://bucket2/path2/db/table"); - - Optional path2 = provider.getNextExternalPath(); - assertThat(path2.isPresent()).isTrue(); - assertThat(path2.get().toString()).isEqualTo("oss://bucket1/path1/db/table"); - - Optional path3 = provider.getNextExternalPath(); - assertThat(path3.isPresent()).isTrue(); - assertThat(path3.get().toString()).isEqualTo("s3://bucket2/path2/db/table"); - } - - @Test - public void testGetNextExternalPathSpecificFS() { - provider = - new ExternalPathProvider( - "oss://bucket1/path1,s3://bucket2/path2", - ExternalPathStrategy.SPECIFIC_FS, - "OSS", - "db/table"); - - Optional path = provider.getNextExternalPath(); - assertThat(path.isPresent()).isTrue(); - assertThat(path.get().toString()).isEqualTo("oss://bucket1/path1/db/table"); - } - - @Test - public void testGetNextExternalPathNone() { - provider = - new ExternalPathProvider( - "oss://bucket1/path1,s3://bucket2/path2", - ExternalPathStrategy.NONE, - "OSS", - "db/table"); - - Optional path = provider.getNextExternalPath(); - assertThat(path.isPresent()).isFalse(); - } - - @Test - public void testUnsupportedExternalPath() { - Assertions.assertThatThrownBy( - () -> { - new ExternalPathProvider( - "hdfs://bucket1/path1", - ExternalPathStrategy.ROUND_ROBIN, - "oss", - "db/table"); - }) - .isInstanceOf(IllegalArgumentException.class); - } - - @Test - public void testUnsupportedExternalFSStrategy() { - provider = - new ExternalPathProvider( - "oss://bucket1/path1", ExternalPathStrategy.SPECIFIC_FS, "S3", "db/table"); - Optional path = provider.getNextExternalPath(); - assertThat(path.isPresent()).isFalse(); - } - - @Test - public void testExternalFSStrategyNull() { - Assertions.assertThatThrownBy( - () -> { - new ExternalPathProvider( - "oss://bucket1/path1", - ExternalPathStrategy.SPECIFIC_FS, - null, - "db/table"); - }) - .isInstanceOf(IllegalArgumentException.class); - } -} diff --git a/paimon-common/src/test/java/org/apache/paimon/fs/TableExternalPathProviderTest.java b/paimon-common/src/test/java/org/apache/paimon/fs/TableExternalPathProviderTest.java new file mode 100644 index 000000000000..0a444768cf6e --- /dev/null +++ b/paimon-common/src/test/java/org/apache/paimon/fs/TableExternalPathProviderTest.java @@ -0,0 +1,194 @@ +/* + * 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.CoreOptions.ExternalPathStrategy; + +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; + +import static org.apache.paimon.testutils.assertj.PaimonAssertions.anyCauseMatches; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Test for {@link TableExternalPathProvider}. */ +public class TableExternalPathProviderTest { + private TableExternalPathProvider provider; + + @BeforeEach + public void setUp() { + provider = + new TableExternalPathProvider( + "oss://bucket1/path1,s3://bucket2/path2", + ExternalPathStrategy.ROUND_ROBIN, + null); + } + + @Test + public void testInitExternalPaths() { + assertThat(provider.externalPathExists()).isTrue(); + assertThat(provider.getExternalPathsMap().size()).isEqualTo(2); + assertThat(provider.getExternalPathsList().size()).isEqualTo(2); + } + + @Test + public void testGetNextExternalPathRoundRobinSimple() { + String path1 = "s3://bucket2/path2"; + String path2 = "oss://bucket1/path1"; + List expectedPaths = new ArrayList(); + expectedPaths.add(path1); + expectedPaths.add(path2); + String externalPaths = path1 + "," + path2; + provider = + new TableExternalPathProvider( + externalPaths, ExternalPathStrategy.ROUND_ROBIN, null); + + // Collect the paths returned by getNextExternalPath + List actualPaths = new ArrayList<>(); + int expectIndex = 0; + for (int i = 0; i < 6; i++) { // Collect more paths to ensure all are covered + Optional path = provider.getNextExternalPath(); + assertThat(path.isPresent()).isTrue(); + actualPaths.add(path.get().toString()); + if (i == 0) { + if (path.get().toString().equals(path1)) { + expectIndex = 0; + } else if (path.get().toString().equals(path2)) { + expectIndex = 1; + } + } + + expectIndex = (expectIndex) % expectedPaths.size(); + assertThat(path.get().toString().equals(expectedPaths.get(expectIndex))).isTrue(); + expectIndex++; + } + + // Check that all expected paths are present in the actual paths + for (String expectedPath : expectedPaths) { + assertThat(actualPaths).contains(expectedPath); + } + } + + @Test + public void testGetNextExternalPathRoundRobinComplex() { + List expectedPathsList = new ArrayList(); + for (int i = 0; i < 20; i++) { + if (i % 2 == 0) { + expectedPathsList.add("oss://bucket1/path" + i); + } else { + expectedPathsList.add("s3://bucket2/path" + i); + } + } + String externalPaths = String.join(",", expectedPathsList); + provider = + new TableExternalPathProvider( + externalPaths, ExternalPathStrategy.ROUND_ROBIN, null); + + // Collect the paths returned by getNextExternalPath + List actualPaths = new ArrayList<>(); + int expectIndex = 0; + for (int i = 0; i < 40; i++) { // Collect more paths to ensure all are covered + Optional path = provider.getNextExternalPath(); + assertThat(path.isPresent()).isTrue(); + actualPaths.add(path.get().toString()); + if (i == 0) { + for (int j = 0; j < expectedPathsList.size(); j++) { + if (path.get().toString().equals(expectedPathsList.get(j))) { + expectIndex = j; + break; + } + } + } + expectIndex = (expectIndex) % expectedPathsList.size(); + assertThat(path.get().toString().equals(expectedPathsList.get(expectIndex))).isTrue(); + expectIndex++; + } + + // Check that all expected paths are present in the actual paths + for (String expectedPath : expectedPathsList) { + assertThat(actualPaths).contains(expectedPath); + } + } + + @Test + public void testGetNextExternalPathSpecificFS() { + provider = + new TableExternalPathProvider( + "oss://bucket1/path1,s3://bucket2/path2", + ExternalPathStrategy.SPECIFIC_FS, + "OSS"); + + Optional path = provider.getNextExternalPath(); + assertThat(path.isPresent()).isTrue(); + assertThat(path.get().toString()).isEqualTo("oss://bucket1/path1"); + } + + @Test + public void testGetNextExternalPathNone() { + provider = + new TableExternalPathProvider( + "oss://bucket1/path1,s3://bucket2/path2", ExternalPathStrategy.NONE, "OSS"); + + Optional path = provider.getNextExternalPath(); + assertThat(path.isPresent()).isFalse(); + } + + @Test + public void testUnsupportedExternalPath() { + Assertions.assertThatThrownBy( + () -> { + new TableExternalPathProvider( + "hdfs://bucket1/path1", + ExternalPathStrategy.SPECIFIC_FS, + "oss"); + }) + .isInstanceOf(IllegalArgumentException.class); + } + + @Test + public void testUnsupportedExternalSpecificFS() { + assertThatThrownBy( + () -> { + provider = + new TableExternalPathProvider( + "oss://bucket1/path1", + ExternalPathStrategy.SPECIFIC_FS, + "S3"); + }) + .satisfies( + anyCauseMatches( + IllegalArgumentException.class, + "external specific fs not found: s3")); + } + + @Test + public void testExternalSpecificFSNull() { + Assertions.assertThatThrownBy( + () -> { + new TableExternalPathProvider( + "oss://bucket1/path1", ExternalPathStrategy.SPECIFIC_FS, null); + }) + .isInstanceOf(IllegalArgumentException.class); + } +} diff --git a/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java b/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java index 6f6e091621c9..655d75edfa96 100644 --- a/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java +++ b/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java @@ -18,11 +18,12 @@ package org.apache.paimon; +import org.apache.paimon.CoreOptions.ExternalPathStrategy; import org.apache.paimon.data.InternalRow; import org.apache.paimon.deletionvectors.DeletionVectorsIndexFile; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; +import org.apache.paimon.fs.TableExternalPathProvider; import org.apache.paimon.index.HashIndexFile; import org.apache.paimon.index.IndexFileHandler; import org.apache.paimon.manifest.IndexManifestFile; @@ -83,7 +84,6 @@ abstract class AbstractFileStore implements FileStore { @Nullable private final SegmentsCache writeManifestCache; @Nullable private SegmentsCache readManifestCache; @Nullable private Cache snapshotCache; - private final ExternalPathProvider externalPathProvider; protected AbstractFileStore( FileIO fileIO, @@ -92,8 +92,7 @@ protected AbstractFileStore( String tableName, CoreOptions options, RowType partitionType, - CatalogEnvironment catalogEnvironment, - ExternalPathProvider externalPathProvider) { + CatalogEnvironment catalogEnvironment) { this.fileIO = fileIO; this.schemaManager = schemaManager; this.schema = schema; @@ -104,7 +103,6 @@ protected AbstractFileStore( this.writeManifestCache = SegmentsCache.create( options.pageSize(), options.writeManifestCache(), Long.MAX_VALUE); - this.externalPathProvider = externalPathProvider; } @Override @@ -124,7 +122,15 @@ protected FileStorePathFactory pathFactory(String format) { options.fileSuffixIncludeCompression(), options.fileCompression(), options.dataFilePathDirectory(), - externalPathProvider); + getExternalPathProvider()); + } + + private TableExternalPathProvider getExternalPathProvider() { + String externalPaths = options.dataFileExternalPaths(); + ExternalPathStrategy externalPathStrategy = options.externalPathStrategy(); + String externalSpecificFS = options.externalSpecificFS(); + return new TableExternalPathProvider( + externalPaths, externalPathStrategy, externalSpecificFS); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/AppendOnlyFileStore.java b/paimon-core/src/main/java/org/apache/paimon/AppendOnlyFileStore.java index e06ad35a5517..a06b98d7b30c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/AppendOnlyFileStore.java +++ b/paimon-core/src/main/java/org/apache/paimon/AppendOnlyFileStore.java @@ -21,7 +21,6 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.deletionvectors.DeletionVectorsMaintainer; import org.apache.paimon.format.FileFormatDiscover; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.manifest.ManifestCacheFilter; import org.apache.paimon.operation.AppendOnlyFileStoreScan; @@ -60,17 +59,8 @@ public AppendOnlyFileStore( RowType bucketKeyType, RowType rowType, String tableName, - CatalogEnvironment catalogEnvironment, - ExternalPathProvider externalPathProvider) { - super( - fileIO, - schemaManager, - schema, - tableName, - options, - partitionType, - catalogEnvironment, - externalPathProvider); + CatalogEnvironment catalogEnvironment) { + super(fileIO, schemaManager, schema, tableName, options, partitionType, catalogEnvironment); this.bucketKeyType = bucketKeyType; this.rowType = rowType; } diff --git a/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java b/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java index 30580cedd442..a969fca03777 100644 --- a/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java +++ b/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java @@ -22,7 +22,6 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.deletionvectors.DeletionVectorsMaintainer; import org.apache.paimon.format.FileFormatDiscover; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.index.HashIndexMaintainer; import org.apache.paimon.index.IndexMaintainer; @@ -85,17 +84,8 @@ public KeyValueFileStore( KeyValueFieldsExtractor keyValueFieldsExtractor, MergeFunctionFactory mfFactory, String tableName, - CatalogEnvironment catalogEnvironment, - ExternalPathProvider externalPathProvider) { - super( - fileIO, - schemaManager, - schema, - tableName, - options, - partitionType, - catalogEnvironment, - externalPathProvider); + CatalogEnvironment catalogEnvironment) { + super(fileIO, schemaManager, schema, tableName, options, partitionType, catalogEnvironment); this.crossPartitionUpdate = crossPartitionUpdate; this.bucketKeyType = bucketKeyType; this.keyType = keyType; diff --git a/paimon-core/src/main/java/org/apache/paimon/io/DataFilePathFactory.java b/paimon-core/src/main/java/org/apache/paimon/io/DataFilePathFactory.java index ef9a7ae25a71..be32afa644e9 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/DataFilePathFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/DataFilePathFactory.java @@ -19,10 +19,11 @@ package org.apache.paimon.io; import org.apache.paimon.annotation.VisibleForTesting; -import org.apache.paimon.fs.ExternalPathProvider; +import org.apache.paimon.fs.DataFileExternalPathProvider; import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.FileEntry; +import javax.annotation.Nullable; import javax.annotation.concurrent.ThreadSafe; import java.util.Optional; @@ -36,7 +37,6 @@ public class DataFilePathFactory { public static final String INDEX_PATH_SUFFIX = ".index"; private final Path parent; - private final Path relativePath; private final String uuid; private final AtomicInteger pathCount; @@ -45,7 +45,7 @@ public class DataFilePathFactory { private final String changelogFilePrefix; private final boolean fileSuffixIncludeCompression; private final String fileCompression; - private final ExternalPathProvider externalPathProvider; + @Nullable private final DataFileExternalPathProvider dataFileExternalPathProvider; private final boolean isExternalPath; public DataFilePathFactory( @@ -55,8 +55,7 @@ public DataFilePathFactory( String changelogFilePrefix, boolean fileSuffixIncludeCompression, String fileCompression, - ExternalPathProvider externalPathProvider, - Path relativePath) { + @Nullable DataFileExternalPathProvider dataFileExternalPathProvider) { this.parent = parent; this.uuid = UUID.randomUUID().toString(); this.pathCount = new AtomicInteger(0); @@ -65,9 +64,12 @@ public DataFilePathFactory( this.changelogFilePrefix = changelogFilePrefix; this.fileSuffixIncludeCompression = fileSuffixIncludeCompression; this.fileCompression = fileCompression; - this.externalPathProvider = externalPathProvider; - this.relativePath = relativePath; - this.isExternalPath = externalPathProvider.externalPathExists(); + this.dataFileExternalPathProvider = dataFileExternalPathProvider; + if (dataFileExternalPathProvider != null) { + this.isExternalPath = dataFileExternalPathProvider.externalPathExists(); + } else { + this.isExternalPath = false; + } } public Path newPath() { @@ -83,12 +85,10 @@ public String newChangelogFileName() { } public Path newPath(String prefix) { - return externalPathProvider - .getNextExternalPath() - .map( - externalPath -> - new Path(new Path(externalPath, relativePath), newFileName(prefix))) - .orElse(new Path(parent, newFileName(prefix))); + return Optional.ofNullable(dataFileExternalPathProvider) + .flatMap(DataFileExternalPathProvider::getNextExternalDataPath) + .map(path -> new Path(path, newFileName(prefix))) + .orElseGet(() -> new Path(parent, newFileName(prefix))); } private String newFileName(String prefix) { diff --git a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedFileStoreTable.java index c1f9ee5b85de..52c806c7c53b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/privilege/PrivilegedFileStoreTable.java @@ -21,7 +21,6 @@ import org.apache.paimon.FileStore; import org.apache.paimon.Snapshot; import org.apache.paimon.catalog.Identifier; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.manifest.ManifestCacheFilter; import org.apache.paimon.schema.TableSchema; import org.apache.paimon.stats.Statistics; @@ -288,11 +287,6 @@ public PrivilegedFileStoreTable switchToBranch(String branchName) { wrapped.switchToBranch(branchName), privilegeChecker, identifier); } - @Override - public ExternalPathProvider externalPathProvider() { - return wrapped.externalPathProvider(); - } - public static PrivilegedFileStoreTable wrap( FileStoreTable table, PrivilegeChecker privilegeChecker, Identifier identifier) { if (table instanceof ObjectTable) { diff --git a/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java index 367d310ab47c..7e008698c4fd 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/AbstractFileStoreTable.java @@ -22,7 +22,6 @@ import org.apache.paimon.Snapshot; import org.apache.paimon.catalog.Identifier; import org.apache.paimon.consumer.ConsumerManager; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.IndexManifestEntry; @@ -106,14 +105,11 @@ abstract class AbstractFileStoreTable implements FileStoreTable { @Nullable protected transient Cache snapshotCache; @Nullable protected transient Cache statsCache; - protected final ExternalPathProvider externalPathProvider; - protected AbstractFileStoreTable( FileIO fileIO, Path path, TableSchema tableSchema, - CatalogEnvironment catalogEnvironment, - ExternalPathProvider externalPathProvider) { + CatalogEnvironment catalogEnvironment) { this.fileIO = fileIO; this.path = path; if (!tableSchema.options().containsKey(PATH.key())) { @@ -124,7 +120,6 @@ protected AbstractFileStoreTable( } this.tableSchema = tableSchema; this.catalogEnvironment = catalogEnvironment; - this.externalPathProvider = externalPathProvider; } public String currentBranch() { @@ -377,17 +372,9 @@ public FileStoreTable copy(TableSchema newTableSchema) { AbstractFileStoreTable copied = newTableSchema.primaryKeys().isEmpty() ? new AppendOnlyFileStoreTable( - fileIO, - path, - newTableSchema, - catalogEnvironment, - externalPathProvider) + fileIO, path, newTableSchema, catalogEnvironment) : new PrimaryKeyFileStoreTable( - fileIO, - path, - newTableSchema, - catalogEnvironment, - externalPathProvider); + fileIO, path, newTableSchema, catalogEnvironment); if (snapshotCache != null) { copied.setSnapshotCache(snapshotCache); } @@ -751,12 +738,7 @@ public FileStoreTable switchToBranch(String branchName) { branchOptions.set(CoreOptions.BRANCH, targetBranch); branchSchema = branchSchema.copy(branchOptions.toMap()); return FileStoreTableFactory.create( - fileIO(), - location(), - branchSchema, - new Options(), - catalogEnvironment(), - externalPathProvider); + fileIO(), location(), branchSchema, new Options(), catalogEnvironment()); } private RollbackHelper rollbackHelper() { @@ -782,8 +764,6 @@ public boolean equals(Object o) { return false; } AbstractFileStoreTable that = (AbstractFileStoreTable) o; - return Objects.equals(path, that.path) - && Objects.equals(tableSchema, that.tableSchema) - && Objects.equals(externalPathProvider, that.externalPathProvider); + return Objects.equals(path, that.path) && Objects.equals(tableSchema, that.tableSchema); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/AppendOnlyFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/AppendOnlyFileStoreTable.java index 9d732effaf02..103fa64050aa 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/AppendOnlyFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/AppendOnlyFileStoreTable.java @@ -21,7 +21,6 @@ import org.apache.paimon.AppendOnlyFileStore; import org.apache.paimon.CoreOptions; import org.apache.paimon.data.InternalRow; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.iceberg.AppendOnlyIcebergCommitCallback; @@ -59,16 +58,15 @@ class AppendOnlyFileStoreTable extends AbstractFileStoreTable { private transient AppendOnlyFileStore lazyStore; AppendOnlyFileStoreTable(FileIO fileIO, Path path, TableSchema tableSchema) { - this(fileIO, path, tableSchema, CatalogEnvironment.empty(), new ExternalPathProvider()); + this(fileIO, path, tableSchema, CatalogEnvironment.empty()); } AppendOnlyFileStoreTable( FileIO fileIO, Path path, TableSchema tableSchema, - CatalogEnvironment catalogEnvironment, - ExternalPathProvider externalPathProvider) { - super(fileIO, path, tableSchema, catalogEnvironment, externalPathProvider); + CatalogEnvironment catalogEnvironment) { + super(fileIO, path, tableSchema, catalogEnvironment); } @Override @@ -84,8 +82,7 @@ public AppendOnlyFileStore store() { tableSchema.logicalBucketKeyType(), tableSchema.logicalRowType(), name(), - catalogEnvironment, - externalPathProvider); + catalogEnvironment); } return lazyStore; } @@ -176,9 +173,4 @@ protected List createCommitCallbacks(String commitUser) { return callbacks; } - - @Override - public ExternalPathProvider externalPathProvider() { - return externalPathProvider; - } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/DataTable.java b/paimon-core/src/main/java/org/apache/paimon/table/DataTable.java index f271912e33d2..7979daccf756 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/DataTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/DataTable.java @@ -19,7 +19,6 @@ package org.apache.paimon.table; import org.apache.paimon.CoreOptions; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.schema.SchemaManager; @@ -55,7 +54,5 @@ public interface DataTable extends InnerTable { Path location(); - ExternalPathProvider externalPathProvider(); - FileIO fileIO(); } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/FallbackReadFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/FallbackReadFileStoreTable.java index e5184b4af543..e3e290f06086 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/FallbackReadFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/FallbackReadFileStoreTable.java @@ -23,7 +23,6 @@ import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.disk.IOManager; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.manifest.PartitionEntry; import org.apache.paimon.metrics.MetricRegistry; import org.apache.paimon.options.Options; @@ -101,11 +100,6 @@ public FileStoreTable switchToBranch(String branchName) { return new FallbackReadFileStoreTable(switchWrappedToBranch(branchName), fallback); } - @Override - public ExternalPathProvider externalPathProvider() { - return wrapped.externalPathProvider(); - } - private FileStoreTable switchWrappedToBranch(String branchName) { Optional optionalSchema = wrapped.schemaManager().copyWithBranch(branchName).latest(); @@ -121,8 +115,7 @@ private FileStoreTable switchWrappedToBranch(String branchName) { wrapped.location(), branchSchema, new Options(), - wrapped.catalogEnvironment(), - wrapped.externalPathProvider()); + wrapped.catalogEnvironment()); } private Map rewriteFallbackOptions(Map options) { diff --git a/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTableFactory.java b/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTableFactory.java index b0a965de3787..afd6ce7da96b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTableFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTableFactory.java @@ -19,9 +19,7 @@ package org.apache.paimon.table; import org.apache.paimon.CoreOptions; -import org.apache.paimon.CoreOptions.ExternalPathStrategy; import org.apache.paimon.catalog.CatalogContext; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.options.Options; @@ -68,36 +66,11 @@ public static FileStoreTable create(FileIO fileIO, Options options) { + tablePath + ". Please create table first.")); - ExternalPathProvider externalPathProvider = getExternalPathProvider(tableSchema, tablePath); - return create( - fileIO, - tablePath, - tableSchema, - options, - CatalogEnvironment.empty(), - externalPathProvider); - } - - private static ExternalPathProvider getExternalPathProvider( - TableSchema tableSchema, Path tablePath) { - CoreOptions coreOptions = CoreOptions.fromMap(tableSchema.options()); - String externalPaths = coreOptions.dataFileExternalPaths(); - ExternalPathStrategy externalPathStrategy = coreOptions.externalPathStrategy(); - String externalSpecificFSStrategy = coreOptions.externalSpecificFSStrategy(); - String dbAndTablePath = tablePath.getParent().getName() + "/" + tablePath.getName(); - return new ExternalPathProvider( - externalPaths, externalPathStrategy, externalSpecificFSStrategy, dbAndTablePath); + return create(fileIO, tablePath, tableSchema, options, CatalogEnvironment.empty()); } public static FileStoreTable create(FileIO fileIO, Path tablePath, TableSchema tableSchema) { - ExternalPathProvider externalPathProvider = getExternalPathProvider(tableSchema, tablePath); - return create( - fileIO, - tablePath, - tableSchema, - new Options(), - CatalogEnvironment.empty(), - externalPathProvider); + return create(fileIO, tablePath, tableSchema, new Options(), CatalogEnvironment.empty()); } public static FileStoreTable create( @@ -105,14 +78,7 @@ public static FileStoreTable create( Path tablePath, TableSchema tableSchema, CatalogEnvironment catalogEnvironment) { - ExternalPathProvider externalPathProvider = getExternalPathProvider(tableSchema, tablePath); - return create( - fileIO, - tablePath, - tableSchema, - new Options(), - catalogEnvironment, - externalPathProvider); + return create(fileIO, tablePath, tableSchema, new Options(), catalogEnvironment); } public static FileStoreTable create( @@ -120,16 +86,10 @@ public static FileStoreTable create( Path tablePath, TableSchema tableSchema, Options dynamicOptions, - CatalogEnvironment catalogEnvironment, - ExternalPathProvider externalPathProvider) { + CatalogEnvironment catalogEnvironment) { FileStoreTable table = createWithoutFallbackBranch( - fileIO, - tablePath, - tableSchema, - dynamicOptions, - catalogEnvironment, - externalPathProvider); + fileIO, tablePath, tableSchema, dynamicOptions, catalogEnvironment); Options options = new Options(table.options()); String fallbackBranch = options.get(CoreOptions.SCAN_FALLBACK_BRANCH); @@ -146,12 +106,7 @@ public static FileStoreTable create( fallbackBranch); FileStoreTable fallbackTable = createWithoutFallbackBranch( - fileIO, - tablePath, - schema.get(), - branchOptions, - catalogEnvironment, - externalPathProvider); + fileIO, tablePath, schema.get(), branchOptions, catalogEnvironment); table = new FallbackReadFileStoreTable(table, fallbackTable); } @@ -163,22 +118,13 @@ public static FileStoreTable createWithoutFallbackBranch( Path tablePath, TableSchema tableSchema, Options dynamicOptions, - CatalogEnvironment catalogEnvironment, - ExternalPathProvider externalPathProvider) { + CatalogEnvironment catalogEnvironment) { FileStoreTable table = tableSchema.primaryKeys().isEmpty() ? new AppendOnlyFileStoreTable( - fileIO, - tablePath, - tableSchema, - catalogEnvironment, - externalPathProvider) + fileIO, tablePath, tableSchema, catalogEnvironment) : new PrimaryKeyFileStoreTable( - fileIO, - tablePath, - tableSchema, - catalogEnvironment, - externalPathProvider); + fileIO, tablePath, tableSchema, catalogEnvironment); return table.copy(dynamicOptions.toMap()); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java index 6b1d86b666d5..516ae766cef8 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java @@ -21,7 +21,6 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.KeyValue; import org.apache.paimon.KeyValueFileStore; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.iceberg.IcebergOptions; @@ -58,16 +57,15 @@ class PrimaryKeyFileStoreTable extends AbstractFileStoreTable { private transient KeyValueFileStore lazyStore; PrimaryKeyFileStoreTable(FileIO fileIO, Path path, TableSchema tableSchema) { - this(fileIO, path, tableSchema, CatalogEnvironment.empty(), new ExternalPathProvider()); + this(fileIO, path, tableSchema, CatalogEnvironment.empty()); } PrimaryKeyFileStoreTable( FileIO fileIO, Path path, TableSchema tableSchema, - CatalogEnvironment catalogEnvironment, - ExternalPathProvider externalPathProvider) { - super(fileIO, path, tableSchema, catalogEnvironment, externalPathProvider); + CatalogEnvironment catalogEnvironment) { + super(fileIO, path, tableSchema, catalogEnvironment); } @Override @@ -101,8 +99,7 @@ public KeyValueFileStore store() { extractor, mfFactory, name(), - catalogEnvironment, - externalPathProvider); + catalogEnvironment); } return lazyStore; } @@ -194,9 +191,4 @@ protected List createCommitCallbacks(String commitUser) { return callbacks; } - - @Override - public ExternalPathProvider externalPathProvider() { - return externalPathProvider; - } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/object/ObjectTable.java b/paimon-core/src/main/java/org/apache/paimon/table/object/ObjectTable.java index 9a8b8e449a96..97acfe7299c5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/object/ObjectTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/object/ObjectTable.java @@ -18,7 +18,6 @@ package org.apache.paimon.table.object; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.manifest.ManifestCacheFilter; import org.apache.paimon.schema.TableSchema; @@ -216,10 +215,5 @@ public ObjectTable switchToBranch(String branchName) { return new ObjectTableImpl( wrapped.switchToBranch(branchName), objectFileIO, objectLocation); } - - @Override - public ExternalPathProvider externalPathProvider() { - return wrapped.externalPathProvider(); - } } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java index 88ad3a71b153..1cb967f8d1e2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/AuditLogTable.java @@ -25,7 +25,6 @@ import org.apache.paimon.data.BinaryString; import org.apache.paimon.data.InternalRow; import org.apache.paimon.disk.IOManager; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.BucketEntry; @@ -184,11 +183,6 @@ public Path location() { return wrapped.location(); } - @Override - public ExternalPathProvider externalPathProvider() { - return wrapped.externalPathProvider(); - } - @Override public SnapshotManager snapshotManager() { return wrapped.snapshotManager(); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/CompactBucketsTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/CompactBucketsTable.java index b3db0e0aecb9..31cecbfb15c2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/CompactBucketsTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/CompactBucketsTable.java @@ -24,7 +24,6 @@ import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.disk.IOManager; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.io.DataFileMeta; @@ -142,11 +141,6 @@ public Path location() { return wrapped.location(); } - @Override - public ExternalPathProvider externalPathProvider() { - return wrapped.externalPathProvider(); - } - @Override public SnapshotManager snapshotManager() { return wrapped.snapshotManager(); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/FileMonitorTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/FileMonitorTable.java index 42ffbf6e33a1..522335aaa6c9 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/FileMonitorTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/FileMonitorTable.java @@ -25,7 +25,6 @@ import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; import org.apache.paimon.disk.IOManager; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.io.DataFileMeta; @@ -128,11 +127,6 @@ public Path location() { return wrapped.location(); } - @Override - public ExternalPathProvider externalPathProvider() { - return wrapped.externalPathProvider(); - } - @Override public SnapshotManager snapshotManager() { return wrapped.snapshotManager(); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/ReadOptimizedTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/ReadOptimizedTable.java index 5e5124630f70..5308005053c8 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/ReadOptimizedTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/ReadOptimizedTable.java @@ -20,7 +20,6 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.Snapshot; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.IndexManifestEntry; @@ -162,11 +161,6 @@ public Path location() { return wrapped.location(); } - @Override - public ExternalPathProvider externalPathProvider() { - return wrapped.externalPathProvider(); - } - @Override public SnapshotManager snapshotManager() { return wrapped.snapshotManager(); diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java b/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java index 18d05577dd13..7a9a7c4dbe3e 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/utils/FileStorePathFactory.java @@ -20,8 +20,9 @@ import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.data.BinaryRow; -import org.apache.paimon.fs.ExternalPathProvider; +import org.apache.paimon.fs.DataFileExternalPathProvider; import org.apache.paimon.fs.Path; +import org.apache.paimon.fs.TableExternalPathProvider; import org.apache.paimon.io.DataFilePathFactory; import org.apache.paimon.types.RowType; @@ -56,7 +57,7 @@ public class FileStorePathFactory { private final AtomicInteger indexManifestCount; private final AtomicInteger indexFileCount; private final AtomicInteger statsFileCount; - private final ExternalPathProvider externalPathProvider; + @Nullable private final TableExternalPathProvider tableExternalPathProvider; public FileStorePathFactory( Path root, @@ -69,7 +70,7 @@ public FileStorePathFactory( boolean fileSuffixIncludeCompression, String fileCompression, @Nullable String dataFilePathDirectory, - ExternalPathProvider externalPathProvider) { + @Nullable TableExternalPathProvider tableExternalPathProvider) { this.root = root; this.dataFilePathDirectory = dataFilePathDirectory; this.uuid = UUID.randomUUID().toString(); @@ -87,7 +88,7 @@ public FileStorePathFactory( this.indexManifestCount = new AtomicInteger(0); this.indexFileCount = new AtomicInteger(0); this.statsFileCount = new AtomicInteger(0); - this.externalPathProvider = externalPathProvider; + this.tableExternalPathProvider = tableExternalPathProvider; } public Path root() { @@ -132,8 +133,13 @@ public DataFilePathFactory createDataFilePathFactory(BinaryRow partition, int bu changelogFilePrefix, fileSuffixIncludeCompression, fileCompression, - externalPathProvider, - relativeBucketPath(partition, bucket)); + getDataFileExternalPathProvider( + tableExternalPathProvider, relativeBucketPath(partition, bucket))); + } + + private DataFileExternalPathProvider getDataFileExternalPathProvider( + TableExternalPathProvider tableExternalPathProvider, Path relativeBucketPath) { + return new DataFileExternalPathProvider(tableExternalPathProvider, relativeBucketPath); } public Path bucketPath(BinaryRow partition, int bucket) { diff --git a/paimon-core/src/test/java/org/apache/paimon/TestAppendFileStore.java b/paimon-core/src/test/java/org/apache/paimon/TestAppendFileStore.java index 345e7cdfc20e..e6db51589408 100644 --- a/paimon-core/src/test/java/org/apache/paimon/TestAppendFileStore.java +++ b/paimon-core/src/test/java/org/apache/paimon/TestAppendFileStore.java @@ -22,7 +22,6 @@ import org.apache.paimon.deletionvectors.DeletionVectorsMaintainer; import org.apache.paimon.deletionvectors.append.AppendDeletionFileMaintainerHelper; import org.apache.paimon.deletionvectors.append.UnawareAppendDeletionFileMaintainer; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.FileIOFinder; import org.apache.paimon.fs.Path; @@ -81,8 +80,7 @@ public TestAppendFileStore( bucketType, rowType, tableName, - CatalogEnvironment.empty(), - new ExternalPathProvider()); + CatalogEnvironment.empty()); this.fileIO = fileIO; this.commitUser = UUID.randomUUID().toString(); diff --git a/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java b/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java index 05fed5a5ee0c..0d8ea5f4a49a 100644 --- a/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java +++ b/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java @@ -20,7 +20,6 @@ import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.serializer.InternalRowSerializer; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.FileIOFinder; import org.apache.paimon.fs.Path; @@ -134,8 +133,7 @@ private TestFileStore( keyValueFieldsExtractor, mfFactory, (new Path(root)).getName(), - CatalogEnvironment.empty(), - new ExternalPathProvider()); + CatalogEnvironment.empty()); this.root = root; this.fileIO = FileIOFinder.find(new Path(root)); this.keySerializer = new InternalRowSerializer(keyType); diff --git a/paimon-core/src/test/java/org/apache/paimon/append/AppendOnlyWriterTest.java b/paimon-core/src/test/java/org/apache/paimon/append/AppendOnlyWriterTest.java index dd6e2abca084..3e54835a0ef7 100644 --- a/paimon-core/src/test/java/org/apache/paimon/append/AppendOnlyWriterTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/append/AppendOnlyWriterTest.java @@ -31,7 +31,6 @@ import org.apache.paimon.fileindex.FileIndexOptions; import org.apache.paimon.format.FileFormat; import org.apache.paimon.format.SimpleColStats; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.io.DataFileMeta; @@ -525,7 +524,6 @@ private DataFilePathFactory createPathFactory() { CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), - new ExternalPathProvider(), null); } diff --git a/paimon-core/src/test/java/org/apache/paimon/format/FileFormatSuffixTest.java b/paimon-core/src/test/java/org/apache/paimon/format/FileFormatSuffixTest.java index e636822be1e7..1a7202e91125 100644 --- a/paimon-core/src/test/java/org/apache/paimon/format/FileFormatSuffixTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/format/FileFormatSuffixTest.java @@ -26,7 +26,6 @@ import org.apache.paimon.data.GenericRow; import org.apache.paimon.disk.IOManager; import org.apache.paimon.fileindex.FileIndexOptions; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.io.DataFileMeta; @@ -74,7 +73,6 @@ public void testFileSuffix(@TempDir java.nio.file.Path tempDir) throws Exception CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), - new ExternalPathProvider(), null); FileFormat fileFormat = FileFormat.fromIdentifier(format, new Options()); LinkedList toCompact = new LinkedList<>(); diff --git a/paimon-core/src/test/java/org/apache/paimon/io/DataFilePathFactoryTest.java b/paimon-core/src/test/java/org/apache/paimon/io/DataFilePathFactoryTest.java index 76a6ed77a482..c406ae24f7c0 100644 --- a/paimon-core/src/test/java/org/apache/paimon/io/DataFilePathFactoryTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/io/DataFilePathFactoryTest.java @@ -19,7 +19,6 @@ package org.apache.paimon.io; import org.apache.paimon.CoreOptions; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.Path; import org.junit.jupiter.api.Test; @@ -42,7 +41,6 @@ public void testNoPartition() { CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), - new ExternalPathProvider(), null); String uuid = pathFactory.uuid(); @@ -73,7 +71,6 @@ public void testWithPartition() { CoreOptions.CHANGELOG_FILE_PREFIX.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), - new ExternalPathProvider(), null); String uuid = pathFactory.uuid(); diff --git a/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java b/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java index 674c084a68b0..2d0efdb6d10a 100644 --- a/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/io/KeyValueFileReadWriteTest.java @@ -28,7 +28,6 @@ import org.apache.paimon.deletionvectors.DeletionVector; import org.apache.paimon.format.FlushingFileFormat; import org.apache.paimon.format.SimpleColStats; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.FileIOFinder; import org.apache.paimon.fs.FileStatus; @@ -239,7 +238,7 @@ protected KeyValueFileWriterFactory createWriterFactory(String pathStr, String f CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), null, - new ExternalPathProvider()); + null); int suggestedFileSize = ThreadLocalRandom.current().nextInt(8192) + 1024; FileIO fileIO = FileIOFinder.find(path); Options options = new Options(); @@ -260,7 +259,7 @@ protected KeyValueFileWriterFactory createWriterFactory(String pathStr, String f CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), null, - new ExternalPathProvider())); + null)); return KeyValueFileWriterFactory.builder( fileIO, diff --git a/paimon-core/src/test/java/org/apache/paimon/io/RollingFileWriterTest.java b/paimon-core/src/test/java/org/apache/paimon/io/RollingFileWriterTest.java index 7157b893bf4a..1a58bfe46573 100644 --- a/paimon-core/src/test/java/org/apache/paimon/io/RollingFileWriterTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/io/RollingFileWriterTest.java @@ -23,7 +23,6 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.fileindex.FileIndexOptions; import org.apache.paimon.format.FileFormat; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.manifest.FileSource; @@ -86,7 +85,6 @@ public void initialize(String identifier, boolean statsDenseStore) { CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION .defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), - new ExternalPathProvider(), null) .newPath(), SCHEMA, diff --git a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTestBase.java b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTestBase.java index 0f6c732a064c..d27ab823544c 100644 --- a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTestBase.java +++ b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileMetaTestBase.java @@ -23,7 +23,6 @@ import org.apache.paimon.data.BinaryRowWriter; import org.apache.paimon.data.Timestamp; import org.apache.paimon.format.FileFormat; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.FileIOFinder; import org.apache.paimon.fs.Path; @@ -153,7 +152,7 @@ protected ManifestFile createManifestFile(String pathStr) { CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), null, - new ExternalPathProvider()), + null), Long.MAX_VALUE, null) .create(); diff --git a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileTest.java b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileTest.java index f574913f3b07..4c949eae961d 100644 --- a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestFileTest.java @@ -21,7 +21,6 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.format.FileFormat; import org.apache.paimon.format.SimpleColStats; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.FileIOFinder; import org.apache.paimon.fs.Path; @@ -108,7 +107,7 @@ private ManifestFile createManifestFile(String pathStr) { CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), null, - new ExternalPathProvider()); + null); int suggestedFileSize = ThreadLocalRandom.current().nextInt(8192) + 1024; FileIO fileIO = FileIOFinder.find(path); return new ManifestFile.Factory( diff --git a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestListTest.java b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestListTest.java index 6f9201251c61..f8b69a04b096 100644 --- a/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestListTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/manifest/ManifestListTest.java @@ -21,7 +21,6 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.TestKeyValueGenerator; import org.apache.paimon.format.FileFormat; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIOFinder; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; @@ -112,7 +111,7 @@ private ManifestList createManifestList(String pathStr) { CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), null, - new ExternalPathProvider()); + null); return new ManifestList.Factory(FileIOFinder.find(path), avro, "zstd", pathFactory, null) .create(); } diff --git a/paimon-core/src/test/java/org/apache/paimon/table/source/snapshot/ScannerTestBase.java b/paimon-core/src/test/java/org/apache/paimon/table/source/snapshot/ScannerTestBase.java index 46de16dd2032..2ed0d5c9b34b 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/source/snapshot/ScannerTestBase.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/source/snapshot/ScannerTestBase.java @@ -23,7 +23,6 @@ import org.apache.paimon.data.BinaryRowWriter; import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.FileIOFinder; import org.apache.paimon.fs.Path; @@ -169,12 +168,7 @@ protected FileStoreTable createFileStoreTable( conf.toMap(), "")); return FileStoreTableFactory.create( - fileIO, - tablePath, - tableSchema, - conf, - CatalogEnvironment.empty(), - new ExternalPathProvider()); + fileIO, tablePath, tableSchema, conf, CatalogEnvironment.empty()); } protected List toSplits(List dataSplits) { diff --git a/paimon-core/src/test/java/org/apache/paimon/table/source/snapshot/SnapshotReaderTest.java b/paimon-core/src/test/java/org/apache/paimon/table/source/snapshot/SnapshotReaderTest.java index 10b4b6158002..02a170c362ef 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/source/snapshot/SnapshotReaderTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/source/snapshot/SnapshotReaderTest.java @@ -23,7 +23,6 @@ import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.serializer.InternalRowSerializer; import org.apache.paimon.fileindex.bloomfilter.BloomFilterFileIndexFactory; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.FileIOFinder; import org.apache.paimon.fs.Path; @@ -369,11 +368,6 @@ private FileStoreTable createFileStoreTable( options.toMap(), "")); return FileStoreTableFactory.create( - fileIO, - tablePath, - tableSchema, - options, - CatalogEnvironment.empty(), - new ExternalPathProvider()); + fileIO, tablePath, tableSchema, options, CatalogEnvironment.empty()); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/utils/FileStorePathFactoryTest.java b/paimon-core/src/test/java/org/apache/paimon/utils/FileStorePathFactoryTest.java index 91f26422a02e..bf5653603c0c 100644 --- a/paimon-core/src/test/java/org/apache/paimon/utils/FileStorePathFactoryTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/utils/FileStorePathFactoryTest.java @@ -22,7 +22,6 @@ import org.apache.paimon.data.BinaryRow; import org.apache.paimon.data.BinaryRowWriter; import org.apache.paimon.data.BinaryString; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.Path; import org.apache.paimon.io.DataFilePathFactory; import org.apache.paimon.types.DataType; @@ -94,7 +93,7 @@ public void testCreateDataFilePathFactoryWithPartition() { CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), null, - new ExternalPathProvider()); + null); assertPartition("20211224", 16, pathFactory, "/dt=20211224/hr=16"); assertPartition("20211224", null, pathFactory, "/dt=20211224/hr=default"); @@ -137,6 +136,6 @@ public static FileStorePathFactory createNonPartFactory(Path root) { CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), null, - new ExternalPathProvider()); + null); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupFileStoreTable.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupFileStoreTable.java index a6bb9fcee776..090399706330 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupFileStoreTable.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/lookup/LookupFileStoreTable.java @@ -22,7 +22,6 @@ import org.apache.paimon.KeyValueFileStore; import org.apache.paimon.flink.FlinkConnectorOptions; import org.apache.paimon.flink.utils.TableScanUtils; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.operation.DefaultValueAssigner; import org.apache.paimon.options.Options; import org.apache.paimon.options.description.DescribedEnum; @@ -110,11 +109,6 @@ public FileStoreTable switchToBranch(String branchName) { return this; } - @Override - public ExternalPathProvider externalPathProvider() { - return wrapped.externalPathProvider(); - } - private LookupStreamScanMode lookupStreamScanMode(FileStoreTable table, List joinKeys) { Options options = Options.fromMap(table.options()); if (options.get(LOOKUP_CACHE_MODE) == FlinkConnectorOptions.LookupCacheMode.AUTO diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/AppendOnlyTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/AppendOnlyTableITCase.java index e29f8ab56ad7..9be91449c245 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/AppendOnlyTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/AppendOnlyTableITCase.java @@ -21,13 +21,18 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.Snapshot; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.fs.FileStatus; import org.apache.paimon.utils.BlockingIterator; +import org.apache.paimon.utils.TraceableFileIO; import org.apache.flink.table.planner.factories.TestValuesTableFactory; import org.apache.flink.types.Row; import org.apache.flink.types.RowKind; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import java.io.IOException; +import java.nio.file.Path; import java.time.LocalDateTime; import java.time.ZoneId; import java.util.Arrays; @@ -37,9 +42,12 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.fail; /** Test case for append-only managed table. */ public class AppendOnlyTableITCase extends CatalogITCaseBase { + @TempDir Path tempExternalPath1; + @TempDir Path tempExternalPath2; @Test public void testCreateUnawareBucketTableWithBucketKey() { @@ -87,6 +95,208 @@ public void testReadWrite() { assertThat(rows).containsExactlyInAnyOrder(Row.of("AAA"), Row.of("BBB")); } + @Test + public void testReadWriteWithExternalPathRoundRobinStrategy1() { + String externalPaths = + TraceableFileIO.SCHEME + + "://" + + tempExternalPath1.toString() + + "," + + TraceableFileIO.SCHEME + + "://" + + tempExternalPath2.toString(); + batchSql( + "ALTER TABLE append_table SET ('data-file.external-paths' = '" + + externalPaths + + "')"); + batchSql( + "ALTER TABLE append_table SET ('data-file.external-paths.strategy' = 'round-robin')"); + + batchSql("INSERT INTO append_table VALUES (1, 'AAA'), (2, 'BBB')"); + List rows = batchSql("SELECT * FROM append_table"); + assertThat(rows.size()).isEqualTo(2); + assertThat(rows).containsExactlyInAnyOrder(Row.of(1, "AAA"), Row.of(2, "BBB")); + + rows = batchSql("SELECT id FROM append_table"); + assertThat(rows.size()).isEqualTo(2); + assertThat(rows).containsExactlyInAnyOrder(Row.of(1), Row.of(2)); + + rows = batchSql("SELECT data from append_table"); + assertThat(rows.size()).isEqualTo(2); + assertThat(rows).containsExactlyInAnyOrder(Row.of("AAA"), Row.of("BBB")); + + batchSql("INSERT INTO append_table VALUES (3, 'CCC')"); + rows = batchSql("SELECT * FROM append_table"); + assertThat(rows.size()).isEqualTo(3); + assertThat(rows) + .containsExactlyInAnyOrder(Row.of(1, "AAA"), Row.of(2, "BBB"), Row.of(3, "CCC")); + + batchSql("INSERT INTO append_table VALUES (4, 'DDD')"); + rows = batchSql("SELECT * FROM append_table"); + assertThat(rows.size()).isEqualTo(4); + assertThat(rows) + .containsExactlyInAnyOrder( + Row.of(1, "AAA"), Row.of(2, "BBB"), Row.of(3, "CCC"), Row.of(4, "DDD")); + + TraceableFileIO traceableFileIO = new TraceableFileIO(); + try { + FileStatus[] fileStatuses = + traceableFileIO.listStatus( + new org.apache.paimon.fs.Path( + tempExternalPath1.toString() + "/bucket-0")); + assertThat(fileStatuses.length > 0).isTrue(); + + fileStatuses = + traceableFileIO.listStatus( + new org.apache.paimon.fs.Path( + tempExternalPath2.toString() + "/bucket-0")); + assertThat(fileStatuses.length > 0).isTrue(); + } catch (IOException e) { + fail(); + } + } + + @Test + public void testReadWriteWithExternalPathRoundRobinStrategy2() { + batchSql("INSERT INTO append_table VALUES (1, 'AAA'), (2, 'BBB')"); + List rows = batchSql("SELECT * FROM append_table"); + assertThat(rows.size()).isEqualTo(2); + assertThat(rows).containsExactlyInAnyOrder(Row.of(1, "AAA"), Row.of(2, "BBB")); + + rows = batchSql("SELECT id FROM append_table"); + assertThat(rows.size()).isEqualTo(2); + assertThat(rows).containsExactlyInAnyOrder(Row.of(1), Row.of(2)); + + rows = batchSql("SELECT data from append_table"); + assertThat(rows.size()).isEqualTo(2); + assertThat(rows).containsExactlyInAnyOrder(Row.of("AAA"), Row.of("BBB")); + + String externalPaths = + TraceableFileIO.SCHEME + + "://" + + tempExternalPath1.toString() + + "," + + TraceableFileIO.SCHEME + + "://" + + tempExternalPath2.toString(); + batchSql( + "ALTER TABLE append_table SET ('data-file.external-paths' = '" + + externalPaths + + "')"); + batchSql( + "ALTER TABLE append_table SET ('data-file.external-paths.strategy' = 'round-robin')"); + + batchSql("INSERT INTO append_table VALUES (3, 'CCC')"); + rows = batchSql("SELECT * FROM append_table"); + assertThat(rows.size()).isEqualTo(3); + assertThat(rows) + .containsExactlyInAnyOrder(Row.of(1, "AAA"), Row.of(2, "BBB"), Row.of(3, "CCC")); + + batchSql("INSERT INTO append_table VALUES (4, 'DDD')"); + rows = batchSql("SELECT * FROM append_table"); + assertThat(rows.size()).isEqualTo(4); + assertThat(rows) + .containsExactlyInAnyOrder( + Row.of(1, "AAA"), Row.of(2, "BBB"), Row.of(3, "CCC"), Row.of(4, "DDD")); + } + + @Test + public void testReadWriteWithExternalPathSpecificFSStrategy() { + String externalPaths = TraceableFileIO.SCHEME + "://" + tempExternalPath1.toString(); + batchSql( + "ALTER TABLE append_table SET ('data-file.external-paths' = '" + + externalPaths + + "')"); + batchSql( + "ALTER TABLE append_table SET ('data-file.external-paths.strategy' = 'specific-fs')"); + batchSql( + "ALTER TABLE append_table SET ('data-file.external-paths.specific-fs' = 'traceable')"); + + batchSql("INSERT INTO append_table VALUES (1, 'AAA'), (2, 'BBB')"); + List rows = batchSql("SELECT * FROM append_table"); + assertThat(rows.size()).isEqualTo(2); + assertThat(rows).containsExactlyInAnyOrder(Row.of(1, "AAA"), Row.of(2, "BBB")); + + rows = batchSql("SELECT id FROM append_table"); + assertThat(rows.size()).isEqualTo(2); + assertThat(rows).containsExactlyInAnyOrder(Row.of(1), Row.of(2)); + + rows = batchSql("SELECT data from append_table"); + assertThat(rows.size()).isEqualTo(2); + assertThat(rows).containsExactlyInAnyOrder(Row.of("AAA"), Row.of("BBB")); + + batchSql("INSERT INTO append_table VALUES (3, 'CCC')"); + rows = batchSql("SELECT * FROM append_table"); + assertThat(rows.size()).isEqualTo(3); + assertThat(rows) + .containsExactlyInAnyOrder(Row.of(1, "AAA"), Row.of(2, "BBB"), Row.of(3, "CCC")); + + batchSql("INSERT INTO append_table VALUES (4, 'DDD')"); + rows = batchSql("SELECT * FROM append_table"); + assertThat(rows.size()).isEqualTo(4); + assertThat(rows) + .containsExactlyInAnyOrder( + Row.of(1, "AAA"), Row.of(2, "BBB"), Row.of(3, "CCC"), Row.of(4, "DDD")); + + TraceableFileIO traceableFileIO = new TraceableFileIO(); + try { + FileStatus[] fileStatuses = + traceableFileIO.listStatus( + new org.apache.paimon.fs.Path( + tempExternalPath1.toString() + "/bucket-0")); + assertThat(fileStatuses.length > 0).isTrue(); + } catch (IOException e) { + fail(); + } + } + + @Test + public void testReadWriteWithExternalPathNoneStrategy() { + String externalPaths = TraceableFileIO.SCHEME + "://" + tempExternalPath1.toString(); + batchSql( + "ALTER TABLE append_table SET ('data-file.external-paths' = '" + + externalPaths + + "')"); + batchSql("ALTER TABLE append_table SET ('data-file.external-paths.strategy' = 'none')"); + + batchSql("INSERT INTO append_table VALUES (1, 'AAA'), (2, 'BBB')"); + List rows = batchSql("SELECT * FROM append_table"); + assertThat(rows.size()).isEqualTo(2); + assertThat(rows).containsExactlyInAnyOrder(Row.of(1, "AAA"), Row.of(2, "BBB")); + + rows = batchSql("SELECT id FROM append_table"); + assertThat(rows.size()).isEqualTo(2); + assertThat(rows).containsExactlyInAnyOrder(Row.of(1), Row.of(2)); + + rows = batchSql("SELECT data from append_table"); + assertThat(rows.size()).isEqualTo(2); + assertThat(rows).containsExactlyInAnyOrder(Row.of("AAA"), Row.of("BBB")); + + batchSql("INSERT INTO append_table VALUES (3, 'CCC')"); + rows = batchSql("SELECT * FROM append_table"); + assertThat(rows.size()).isEqualTo(3); + assertThat(rows) + .containsExactlyInAnyOrder(Row.of(1, "AAA"), Row.of(2, "BBB"), Row.of(3, "CCC")); + + batchSql("INSERT INTO append_table VALUES (4, 'DDD')"); + rows = batchSql("SELECT * FROM append_table"); + assertThat(rows.size()).isEqualTo(4); + assertThat(rows) + .containsExactlyInAnyOrder( + Row.of(1, "AAA"), Row.of(2, "BBB"), Row.of(3, "CCC"), Row.of(4, "DDD")); + + TraceableFileIO traceableFileIO = new TraceableFileIO(); + try { + FileStatus[] fileStatuses = + traceableFileIO.listStatus( + new org.apache.paimon.fs.Path( + tempExternalPath1.toString() + "/bucket-0")); + assertThat(fileStatuses.length).isEqualTo(0); + } catch (IOException e) { + fail(); + } + } + @Test public void testReadUnwareBucketTableWithRebalanceShuffle() throws Exception { batchSql( diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PrimaryKeyFileStoreTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PrimaryKeyFileStoreTableITCase.java index 4a90415c191a..6f1beb44244f 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PrimaryKeyFileStoreTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PrimaryKeyFileStoreTableITCase.java @@ -21,9 +21,11 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.flink.action.CompactAction; import org.apache.paimon.flink.util.AbstractTestBase; +import org.apache.paimon.fs.FileStatus; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.utils.FailingFileIO; +import org.apache.paimon.utils.TraceableFileIO; import org.apache.flink.api.common.JobStatus; import org.apache.flink.core.execution.JobClient; @@ -55,6 +57,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatCode; +import static org.junit.jupiter.api.Assertions.fail; /** Tests for changelog table with primary keys. */ public class PrimaryKeyFileStoreTableITCase extends AbstractTestBase { @@ -66,10 +69,14 @@ public class PrimaryKeyFileStoreTableITCase extends AbstractTestBase { // ------------------------------------------------------------------------ private String path; private Map tableDefaultProperties; + private String externalPath1; + private String externalPath2; @BeforeEach public void before() throws IOException { path = getTempDirPath(); + externalPath1 = getTempDirPath(); + externalPath2 = getTempDirPath(); ThreadLocalRandom random = ThreadLocalRandom.current(); tableDefaultProperties = new HashMap<>(); @@ -207,6 +214,132 @@ public void testLookupChangelog() throws Exception { innerTestChangelogProducing(Collections.singletonList("'changelog-producer' = 'lookup'")); } + @Test + public void testTableReadWriteWithExternalPathRoundRobin() throws Exception { + TableEnvironment sEnv = + tableEnvironmentBuilder() + .streamingMode() + .checkpointIntervalMs(ThreadLocalRandom.current().nextInt(900) + 100) + .parallelism(1) + .build(); + + sEnv.executeSql(createCatalogSql("testCatalog", path + "/warehouse")); + sEnv.executeSql("USE CATALOG testCatalog"); + String externalPaths = + TraceableFileIO.SCHEME + + "://" + + externalPath1.toString() + + "," + + TraceableFileIO.SCHEME + + "://" + + externalPath2.toString(); + sEnv.executeSql( + "CREATE TABLE T2 ( k INT, v STRING, PRIMARY KEY (k) NOT ENFORCED ) " + + "WITH ( " + + "'bucket' = '2'," + + "'data-file.external-paths' = '" + + externalPaths + + "'," + + "'data-file.external-paths.strategy' = 'round-robin'" + + ")"); + + CloseableIterator it = collect(sEnv.executeSql("SELECT * FROM T2")); + + // insert data + sEnv.executeSql("INSERT INTO T2 VALUES (1, 'A')").await(); + // read initial data + List actual = new ArrayList<>(); + for (int i = 0; i < 1; i++) { + actual.add(it.next().toString()); + } + assertThat(actual).containsExactlyInAnyOrder("+I[1, A]"); + + // insert data + sEnv.executeSql("INSERT INTO T2 VALUES (2, 'B'), (3, 'C')").await(); + + for (int i = 0; i < 2; i++) { + actual.add(it.next().toString()); + } + assertThat(actual).containsExactlyInAnyOrder("+I[1, A]", "+I[2, B]", "+I[3, C]"); + + TraceableFileIO traceableFileIO = new TraceableFileIO(); + try { + FileStatus[] fileStatuses = + traceableFileIO.listStatus( + new org.apache.paimon.fs.Path(externalPath1.toString() + "/bucket-0")); + assertThat(fileStatuses.length > 0).isTrue(); + fileStatuses = + traceableFileIO.listStatus( + new org.apache.paimon.fs.Path(externalPath2.toString() + "/bucket-0")); + assertThat(fileStatuses.length > 0).isTrue(); + } catch (IOException e) { + fail(); + } + } + + @Test + public void testTableReadWriteWithExternalPathSpecificFS() throws Exception { + TableEnvironment sEnv = + tableEnvironmentBuilder() + .streamingMode() + .checkpointIntervalMs(ThreadLocalRandom.current().nextInt(900) + 100) + .parallelism(1) + .build(); + + sEnv.executeSql(createCatalogSql("testCatalog", path + "/warehouse")); + sEnv.executeSql("USE CATALOG testCatalog"); + String externalPaths = + TraceableFileIO.SCHEME + + "://" + + externalPath1.toString() + + "," + + "fake://" + + externalPath2.toString(); + sEnv.executeSql( + "CREATE TABLE T2 ( k INT, v STRING, PRIMARY KEY (k) NOT ENFORCED ) " + + "WITH ( " + + "'bucket' = '2'," + + "'data-file.external-paths' = '" + + externalPaths + + "'," + + "'data-file.external-paths.strategy' = 'specific-fs'," + + "'data-file.external-paths.specific-fs' = 'traceable'" + + ")"); + + CloseableIterator it = collect(sEnv.executeSql("SELECT * FROM T2")); + + // insert data + sEnv.executeSql("INSERT INTO T2 VALUES (1, 'A')").await(); + // read initial data + List actual = new ArrayList<>(); + for (int i = 0; i < 1; i++) { + actual.add(it.next().toString()); + } + assertThat(actual).containsExactlyInAnyOrder("+I[1, A]"); + + // insert data + sEnv.executeSql("INSERT INTO T2 VALUES (2, 'B'), (3, 'C')").await(); + + for (int i = 0; i < 2; i++) { + actual.add(it.next().toString()); + } + assertThat(actual).containsExactlyInAnyOrder("+I[1, A]", "+I[2, B]", "+I[3, C]"); + + TraceableFileIO traceableFileIO = new TraceableFileIO(); + try { + FileStatus[] fileStatuses = + traceableFileIO.listStatus( + new org.apache.paimon.fs.Path(externalPath1.toString() + "/bucket-0")); + assertThat(fileStatuses.length > 0).isTrue(); + fileStatuses = + traceableFileIO.listStatus( + new org.apache.paimon.fs.Path(externalPath2.toString() + "/bucket-0")); + assertThat(fileStatuses.length).isEqualTo(0); + } catch (IOException e) { + fail(); + } + } + @Test public void testTableReadWriteBranch() throws Exception { TableEnvironment sEnv = diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/FlinkSinkTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/FlinkSinkTest.java index 3d9430411439..5f21858e61a5 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/FlinkSinkTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/FlinkSinkTest.java @@ -21,7 +21,6 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.data.GenericRow; import org.apache.paimon.data.InternalRow; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIOFinder; import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.operation.KeyValueFileStoreWrite; @@ -126,7 +125,6 @@ private FileStoreTable createFileStoreTable() throws Exception { tablePath, tableSchema, options, - CatalogEnvironment.empty(), - new ExternalPathProvider()); + CatalogEnvironment.empty()); } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java index 52bad1c3995b..a9107a78fcf4 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java @@ -26,7 +26,6 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.format.FileFormat; import org.apache.paimon.format.FileFormatDiscover; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.io.DataFileMeta; @@ -113,7 +112,7 @@ public TestChangelogDataReadWrite(String root) { CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), null, - new ExternalPathProvider()); + null); this.snapshotManager = new SnapshotManager(LocalFileIO.create(), new Path(root)); this.commitUser = UUID.randomUUID().toString(); } diff --git a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkFileIndexITCase.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkFileIndexITCase.java index 09fdc1566dfe..cb35fa507016 100644 --- a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkFileIndexITCase.java +++ b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkFileIndexITCase.java @@ -28,7 +28,6 @@ import org.apache.paimon.fileindex.FileIndexReader; import org.apache.paimon.fileindex.FileIndexResult; import org.apache.paimon.fileindex.bitmap.BitmapIndexResult; -import org.apache.paimon.fs.ExternalPathProvider; import org.apache.paimon.fs.FileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; @@ -163,7 +162,7 @@ protected void foreachIndexReader(Consumer consumer) CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), null, - new ExternalPathProvider()); + null); Table table = fileSystemCatalog.getTable(Identifier.create("db", "T")); ReadBuilder readBuilder = table.newReadBuilder(); From 48482982961681b1fb69c2fb6b86cc7fbf4c42c1 Mon Sep 17 00:00:00 2001 From: HouliangQi Date: Mon, 6 Jan 2025 09:40:40 +0800 Subject: [PATCH 20/21] fix IT --- .../paimon/flink/AppendOnlyTableITCase.java | 42 ------------------ .../flink/PrimaryKeyFileStoreTableITCase.java | 44 +++++-------------- 2 files changed, 11 insertions(+), 75 deletions(-) diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/AppendOnlyTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/AppendOnlyTableITCase.java index 9be91449c245..d6257379c98c 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/AppendOnlyTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/AppendOnlyTableITCase.java @@ -21,7 +21,6 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.Snapshot; import org.apache.paimon.catalog.Catalog; -import org.apache.paimon.fs.FileStatus; import org.apache.paimon.utils.BlockingIterator; import org.apache.paimon.utils.TraceableFileIO; @@ -31,7 +30,6 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; -import java.io.IOException; import java.nio.file.Path; import java.time.LocalDateTime; import java.time.ZoneId; @@ -42,7 +40,6 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.junit.jupiter.api.Assertions.fail; /** Test case for append-only managed table. */ public class AppendOnlyTableITCase extends CatalogITCaseBase { @@ -137,23 +134,6 @@ public void testReadWriteWithExternalPathRoundRobinStrategy1() { assertThat(rows) .containsExactlyInAnyOrder( Row.of(1, "AAA"), Row.of(2, "BBB"), Row.of(3, "CCC"), Row.of(4, "DDD")); - - TraceableFileIO traceableFileIO = new TraceableFileIO(); - try { - FileStatus[] fileStatuses = - traceableFileIO.listStatus( - new org.apache.paimon.fs.Path( - tempExternalPath1.toString() + "/bucket-0")); - assertThat(fileStatuses.length > 0).isTrue(); - - fileStatuses = - traceableFileIO.listStatus( - new org.apache.paimon.fs.Path( - tempExternalPath2.toString() + "/bucket-0")); - assertThat(fileStatuses.length > 0).isTrue(); - } catch (IOException e) { - fail(); - } } @Test @@ -237,17 +217,6 @@ public void testReadWriteWithExternalPathSpecificFSStrategy() { assertThat(rows) .containsExactlyInAnyOrder( Row.of(1, "AAA"), Row.of(2, "BBB"), Row.of(3, "CCC"), Row.of(4, "DDD")); - - TraceableFileIO traceableFileIO = new TraceableFileIO(); - try { - FileStatus[] fileStatuses = - traceableFileIO.listStatus( - new org.apache.paimon.fs.Path( - tempExternalPath1.toString() + "/bucket-0")); - assertThat(fileStatuses.length > 0).isTrue(); - } catch (IOException e) { - fail(); - } } @Test @@ -284,17 +253,6 @@ public void testReadWriteWithExternalPathNoneStrategy() { assertThat(rows) .containsExactlyInAnyOrder( Row.of(1, "AAA"), Row.of(2, "BBB"), Row.of(3, "CCC"), Row.of(4, "DDD")); - - TraceableFileIO traceableFileIO = new TraceableFileIO(); - try { - FileStatus[] fileStatuses = - traceableFileIO.listStatus( - new org.apache.paimon.fs.Path( - tempExternalPath1.toString() + "/bucket-0")); - assertThat(fileStatuses.length).isEqualTo(0); - } catch (IOException e) { - fail(); - } } @Test diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PrimaryKeyFileStoreTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PrimaryKeyFileStoreTableITCase.java index 6f1beb44244f..d9e471b75c66 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PrimaryKeyFileStoreTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PrimaryKeyFileStoreTableITCase.java @@ -21,7 +21,6 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.flink.action.CompactAction; import org.apache.paimon.flink.util.AbstractTestBase; -import org.apache.paimon.fs.FileStatus; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; import org.apache.paimon.utils.FailingFileIO; @@ -57,7 +56,6 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatCode; -import static org.junit.jupiter.api.Assertions.fail; /** Tests for changelog table with primary keys. */ public class PrimaryKeyFileStoreTableITCase extends AbstractTestBase { @@ -236,7 +234,7 @@ public void testTableReadWriteWithExternalPathRoundRobin() throws Exception { sEnv.executeSql( "CREATE TABLE T2 ( k INT, v STRING, PRIMARY KEY (k) NOT ENFORCED ) " + "WITH ( " - + "'bucket' = '2'," + + "'bucket' = '1'," + "'data-file.external-paths' = '" + externalPaths + "'," @@ -255,26 +253,20 @@ public void testTableReadWriteWithExternalPathRoundRobin() throws Exception { assertThat(actual).containsExactlyInAnyOrder("+I[1, A]"); // insert data - sEnv.executeSql("INSERT INTO T2 VALUES (2, 'B'), (3, 'C')").await(); + sEnv.executeSql("INSERT INTO T2 VALUES (2, 'B')").await(); - for (int i = 0; i < 2; i++) { + for (int i = 0; i < 1; i++) { actual.add(it.next().toString()); } - assertThat(actual).containsExactlyInAnyOrder("+I[1, A]", "+I[2, B]", "+I[3, C]"); - TraceableFileIO traceableFileIO = new TraceableFileIO(); - try { - FileStatus[] fileStatuses = - traceableFileIO.listStatus( - new org.apache.paimon.fs.Path(externalPath1.toString() + "/bucket-0")); - assertThat(fileStatuses.length > 0).isTrue(); - fileStatuses = - traceableFileIO.listStatus( - new org.apache.paimon.fs.Path(externalPath2.toString() + "/bucket-0")); - assertThat(fileStatuses.length > 0).isTrue(); - } catch (IOException e) { - fail(); + // insert data + sEnv.executeSql("INSERT INTO T2 VALUES (3, 'C')").await(); + + for (int i = 0; i < 1; i++) { + actual.add(it.next().toString()); } + + assertThat(actual).containsExactlyInAnyOrder("+I[1, A]", "+I[2, B]", "+I[3, C]"); } @Test @@ -298,7 +290,7 @@ public void testTableReadWriteWithExternalPathSpecificFS() throws Exception { sEnv.executeSql( "CREATE TABLE T2 ( k INT, v STRING, PRIMARY KEY (k) NOT ENFORCED ) " + "WITH ( " - + "'bucket' = '2'," + + "'bucket' = '1'," + "'data-file.external-paths' = '" + externalPaths + "'," @@ -324,20 +316,6 @@ public void testTableReadWriteWithExternalPathSpecificFS() throws Exception { actual.add(it.next().toString()); } assertThat(actual).containsExactlyInAnyOrder("+I[1, A]", "+I[2, B]", "+I[3, C]"); - - TraceableFileIO traceableFileIO = new TraceableFileIO(); - try { - FileStatus[] fileStatuses = - traceableFileIO.listStatus( - new org.apache.paimon.fs.Path(externalPath1.toString() + "/bucket-0")); - assertThat(fileStatuses.length > 0).isTrue(); - fileStatuses = - traceableFileIO.listStatus( - new org.apache.paimon.fs.Path(externalPath2.toString() + "/bucket-0")); - assertThat(fileStatuses.length).isEqualTo(0); - } catch (IOException e) { - fail(); - } } @Test From 701b79adeb46abbdeb950aec341461d732bc9416 Mon Sep 17 00:00:00 2001 From: HouliangQi Date: Mon, 6 Jan 2025 13:20:42 +0800 Subject: [PATCH 21/21] add external path IT --- .../paimon/flink/AppendOnlyTableITCase.java | 5 +- .../flink/PrimaryKeyFileStoreTableITCase.java | 3 +- .../paimon/flink/ReadWriteTableITCase.java | 118 ++++++++++++++++++ .../flink/util/ReadWriteTableTestUtil.java | 16 +++ 4 files changed, 139 insertions(+), 3 deletions(-) diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/AppendOnlyTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/AppendOnlyTableITCase.java index d6257379c98c..d05eeb16b299 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/AppendOnlyTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/AppendOnlyTableITCase.java @@ -21,6 +21,7 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.Snapshot; import org.apache.paimon.catalog.Catalog; +import org.apache.paimon.fs.local.LocalFileIOLoader; import org.apache.paimon.utils.BlockingIterator; import org.apache.paimon.utils.TraceableFileIO; @@ -99,7 +100,7 @@ public void testReadWriteWithExternalPathRoundRobinStrategy1() { + "://" + tempExternalPath1.toString() + "," - + TraceableFileIO.SCHEME + + LocalFileIOLoader.SCHEME + "://" + tempExternalPath2.toString(); batchSql( @@ -156,7 +157,7 @@ public void testReadWriteWithExternalPathRoundRobinStrategy2() { + "://" + tempExternalPath1.toString() + "," - + TraceableFileIO.SCHEME + + LocalFileIOLoader.SCHEME + "://" + tempExternalPath2.toString(); batchSql( diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PrimaryKeyFileStoreTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PrimaryKeyFileStoreTableITCase.java index d9e471b75c66..d0c7d35d778b 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PrimaryKeyFileStoreTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/PrimaryKeyFileStoreTableITCase.java @@ -23,6 +23,7 @@ import org.apache.paimon.flink.util.AbstractTestBase; import org.apache.paimon.fs.Path; import org.apache.paimon.fs.local.LocalFileIO; +import org.apache.paimon.fs.local.LocalFileIOLoader; import org.apache.paimon.utils.FailingFileIO; import org.apache.paimon.utils.TraceableFileIO; @@ -228,7 +229,7 @@ public void testTableReadWriteWithExternalPathRoundRobin() throws Exception { + "://" + externalPath1.toString() + "," - + TraceableFileIO.SCHEME + + LocalFileIOLoader.SCHEME + "://" + externalPath2.toString(); sEnv.executeSql( diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ReadWriteTableITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ReadWriteTableITCase.java index 732e96454236..dc07817b45fd 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ReadWriteTableITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/ReadWriteTableITCase.java @@ -54,6 +54,7 @@ import org.apache.flink.util.CollectionUtil; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; import org.junit.jupiter.params.provider.ValueSource; @@ -87,6 +88,7 @@ import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.buildQuery; import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.buildQueryWithTableOptions; import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.buildSimpleQuery; +import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.checkExternalFileStorePath; import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.checkFileStorePath; import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.createTable; import static org.apache.paimon.flink.util.ReadWriteTableTestUtil.createTemporaryTable; @@ -114,6 +116,8 @@ public class ReadWriteTableITCase extends AbstractTestBase { private final Map staticPartitionOverwrite = Collections.singletonMap(CoreOptions.DYNAMIC_PARTITION_OVERWRITE.key(), "false"); + @TempDir public static java.nio.file.Path externalPath1; + @BeforeEach public void setUp() { init(getTempDirPath()); @@ -215,6 +219,120 @@ public void testBatchReadWriteWithPartitionedRecordsWithPk() throws Exception { changelogRow("+I", "Euro", "2022-01-01"))); } + @Test + public void testBatchReadWriteWithPartitionedRecordsWithPkWithExternalPathRoundRobinStrategy() + throws Exception { + Map options = new HashMap<>(); + options.put( + CoreOptions.DATA_FILE_EXTERNAL_PATHS.key(), "file://" + externalPath1.toString()); + options.put(CoreOptions.DATA_FILE_EXTERNAL_PATHS_STRATEGY.key(), "ROUND-ROBIN"); + checkExternalPathTestResult(options, externalPath1.toString()); + } + + @Test + public void testBatchReadWriteWithPartitionedRecordsWithPkWithExternalPathSpecificFStrategy() + throws Exception { + Map options = new HashMap<>(); + options.put( + CoreOptions.DATA_FILE_EXTERNAL_PATHS.key(), "file://" + externalPath1.toString()); + options.put(CoreOptions.DATA_FILE_EXTERNAL_PATHS_STRATEGY.key(), "specific-fs"); + options.put(CoreOptions.DATA_FILE_EXTERNAL_PATHS_SPECIFIC_FS.key(), "file"); + checkExternalPathTestResult(options, externalPath1.toString()); + } + + public void checkExternalPathTestResult(Map options, String externalPath) + throws Exception { + List initialRecords = + Arrays.asList( + // part = 2022-01-01 + changelogRow("+I", "US Dollar", 114L, "2022-01-01"), + changelogRow("+I", "Yen", 1L, "2022-01-01"), + changelogRow("+I", "Euro", 114L, "2022-01-01"), + // part = 2022-01-02 + changelogRow("+I", "Euro", 119L, "2022-01-02")); + + String table = + createTable( + Arrays.asList("currency STRING", "rate BIGINT", "dt String"), + Arrays.asList("currency", "dt"), + Collections.emptyList(), + Collections.singletonList("dt"), + options); + + insertInto( + table, + "('US Dollar', 114, '2022-01-01')", + "('Yen', 1, '2022-01-01')", + "('Euro', 114, '2022-01-01')", + "('Euro', 119, '2022-01-02')"); + + checkExternalFileStorePath(Arrays.asList("dt=2022-01-01", "dt=2022-01-02"), externalPath); + + testBatchRead(buildSimpleQuery(table), initialRecords); + + insertOverwritePartition( + table, "PARTITION (dt = '2022-01-02')", "('Euro', 100)", "('Yen', 1)"); + + // batch read to check partition refresh + testBatchRead( + buildQuery(table, "*", "WHERE dt IN ('2022-01-02')"), + Arrays.asList( + // part = 2022-01-02 + changelogRow("+I", "Euro", 100L, "2022-01-02"), + changelogRow("+I", "Yen", 1L, "2022-01-02"))); + + // test partition filter + List expectedPartitionRecords = + Arrays.asList( + changelogRow("+I", "Yen", 1L, "2022-01-01"), + changelogRow("+I", "Euro", 114L, "2022-01-01"), + changelogRow("+I", "US Dollar", 114L, "2022-01-01")); + + testBatchRead(buildQuery(table, "*", "WHERE dt <> '2022-01-02'"), expectedPartitionRecords); + + testBatchRead( + buildQuery(table, "*", "WHERE dt IN ('2022-01-01')"), expectedPartitionRecords); + + // test field filter + testBatchRead( + buildQuery(table, "*", "WHERE rate >= 100"), + Arrays.asList( + changelogRow("+I", "US Dollar", 114L, "2022-01-01"), + changelogRow("+I", "Euro", 114L, "2022-01-01"), + changelogRow("+I", "Euro", 100L, "2022-01-02"))); + + // test partition and field filter + testBatchRead( + buildQuery(table, "*", "WHERE dt = '2022-01-02' AND rate >= 100"), + Collections.singletonList(changelogRow("+I", "Euro", 100L, "2022-01-02"))); + + // test projection + testBatchRead( + buildQuery(table, "dt", ""), + Arrays.asList( + changelogRow("+I", "2022-01-01"), + changelogRow("+I", "2022-01-01"), + changelogRow("+I", "2022-01-01"), + changelogRow("+I", "2022-01-02"), + changelogRow("+I", "2022-01-02"))); + + testBatchRead( + buildQuery(table, "dt, currency, rate", ""), + Arrays.asList( + changelogRow("+I", "2022-01-01", "US Dollar", 114L), + changelogRow("+I", "2022-01-01", "Yen", 1L), + changelogRow("+I", "2022-01-01", "Euro", 114L), + changelogRow("+I", "2022-01-02", "Euro", 100L), + changelogRow("+I", "2022-01-02", "Yen", 1L))); + + // test projection and filter + testBatchRead( + buildQuery(table, "currency, dt", "WHERE rate = 114"), + Arrays.asList( + changelogRow("+I", "US Dollar", "2022-01-01"), + changelogRow("+I", "Euro", "2022-01-01"))); + } + @Test public void testNaNType() throws Exception { bEnv.executeSql( diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/ReadWriteTableTestUtil.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/ReadWriteTableTestUtil.java index 0eac2ed2936e..1383e3f4edff 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/ReadWriteTableTestUtil.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/util/ReadWriteTableTestUtil.java @@ -273,6 +273,22 @@ public static void checkFileStorePath(String table, List partitionSpec) }); } + public static void checkExternalFileStorePath(List partitionSpec, String externalPath) { + // check data file path + if (partitionSpec.isEmpty()) { + partitionSpec = Collections.singletonList(""); + } + partitionSpec.stream() + .map(str -> str.replaceAll(",", "/")) + .map(str -> str.replaceAll("null", "__DEFAULT_PARTITION__")) + .forEach( + partition -> { + assertThat(Paths.get(externalPath, partition)).exists(); + // at least exists bucket-0 + assertThat(Paths.get(externalPath, partition, "bucket-0")).exists(); + }); + } + public static void testBatchRead(String query, List expected) throws Exception { CloseableIterator resultItr = bEnv.executeSql(query).collect(); try (BlockingIterator iterator = BlockingIterator.of(resultItr)) {