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..42bcd8e13b5c 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,32 @@ 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."); + @ExcludeFromDocumentation("Internal use only") - public static final ConfigOption PATH = + public static final ConfigOption TABLE_SCHEMA_PATH = key("path") .stringType() .noDefaultValue() - .withDescription("The file path of this table in the filesystem."); + .withDescription( + "The schema file path of this table in the filesystem. if " + + DATA_FILE_EXTERNAL_PATH.key() + + "is not set, the data file path will be the same as the schema file path."); + + @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." + + TABLE_SCHEMA_PATH.key()); public static final ConfigOption BRANCH = key("branch").stringType().defaultValue("main").withDescription("Specify branch name."); @@ -1548,8 +1568,12 @@ public int bucket() { return options.get(BUCKET); } - public Path path() { - return path(options.toMap()); + public Path schemaPath() { + return schemaPath(options.toMap()); + } + + public Path dataPath() { + return dataPath(options.toMap()); } public String branch() { @@ -1563,12 +1587,20 @@ public static String branch(Map options) { return BRANCH.defaultValue(); } - public static Path path(Map options) { - return new Path(options.get(PATH.key())); + public static Path schemaPath(Map options) { + return new Path(options.get(TABLE_SCHEMA_PATH.key())); + } + + public static Path schemaPath(Options options) { + return new Path(options.get(TABLE_SCHEMA_PATH)); } - 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() { @@ -2360,6 +2392,10 @@ public boolean asyncFileWrite() { return options.get(ASYNC_FILE_WRITE); } + public String getDataFileExternalPath() { + return options.get(DATA_FILE_EXTERNAL_PATH); + } + public boolean statsDenseStore() { return options.get(METADATA_STATS_DENSE_STORE); } diff --git a/paimon-common/src/main/java/org/apache/paimon/fs/HybridFileIO.java b/paimon-common/src/main/java/org/apache/paimon/fs/HybridFileIO.java new file mode 100644 index 000000000000..24dc54271e8f --- /dev/null +++ b/paimon-common/src/main/java/org/apache/paimon/fs/HybridFileIO.java @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.fs; + +import org.apache.paimon.CoreOptions; +import org.apache.paimon.catalog.CatalogContext; +import org.apache.paimon.options.Options; + +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +/** + * A hybrid implementation of {@link FileIO} that supports multiple file system schemas. It + * dynamically selects the appropriate {@link FileIO} based on the URI scheme of the given path. + */ +public class HybridFileIO implements FileIO { + + private static final long serialVersionUID = 1L; + + protected Options options; + + private Map fileIOMap; + private volatile FileIO fallbackFileIO; + + @Override + public boolean isObjectStore() { + if (options.get(CoreOptions.DATA_FILE_EXTERNAL_PATH) != null + && ((options.get(CoreOptions.DATA_FILE_EXTERNAL_PATH).startsWith("oss://") + || (options.get(CoreOptions.DATA_FILE_EXTERNAL_PATH) + .startsWith("s3://"))))) { + return true; + } + return false; + } + + @Override + public void configure(CatalogContext context) { + this.options = context.options(); + this.fileIOMap = new ConcurrentHashMap<>(); + } + + @Override + public SeekableInputStream newInputStream(Path path) throws IOException { + return wrap(() -> fileIO(path).newInputStream(path)); + } + + @Override + public PositionOutputStream newOutputStream(Path path, boolean overwrite) throws IOException { + return wrap(() -> fileIO(path).newOutputStream(path, overwrite)); + } + + @Override + public FileStatus getFileStatus(Path path) throws IOException { + return wrap(() -> fileIO(path).getFileStatus(path)); + } + + @Override + public FileStatus[] listStatus(Path path) throws IOException { + return wrap(() -> fileIO(path).listStatus(path)); + } + + @Override + public boolean exists(Path path) throws IOException { + return wrap(() -> fileIO(path).exists(path)); + } + + @Override + public boolean delete(Path path, boolean recursive) throws IOException { + return wrap(() -> fileIO(path).delete(path, recursive)); + } + + @Override + public boolean mkdirs(Path path) throws IOException { + return wrap(() -> fileIO(path).mkdirs(path)); + } + + @Override + public boolean rename(Path src, Path dst) throws IOException { + return wrap(() -> fileIO(src).rename(src, dst)); + } + + private FileIO fileIO(Path path) throws IOException { + String schema = path.toUri().getScheme(); + if (schema == null) { + if (fallbackFileIO == null) { + synchronized (this) { + if (fallbackFileIO == null) { + CatalogContext catalogContext = CatalogContext.create(options); + fallbackFileIO = FileIO.get(path, catalogContext); + } + } + } + return fallbackFileIO; + } + + if (!fileIOMap.containsKey(schema)) { + synchronized (this) { + if (!fileIOMap.containsKey(schema)) { + CatalogContext catalogContext = CatalogContext.create(options); + FileIO fileIO = FileIO.get(path, catalogContext); + fileIOMap.put(path.toUri().getScheme(), fileIO); + } + } + } + return fileIOMap.get(path.toUri().getScheme()); + } + + private T wrap(Func func) throws IOException { + ClassLoader cl = Thread.currentThread().getContextClassLoader(); + try { + Thread.currentThread().setContextClassLoader(HybridFileIO.class.getClassLoader()); + return func.apply(); + } finally { + Thread.currentThread().setContextClassLoader(cl); + } + } + + /** Apply function with wrapping classloader. */ + @FunctionalInterface + protected interface Func { + T apply() throws IOException; + } +} 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..960fb1798a13 100644 --- a/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java +++ b/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java @@ -110,7 +110,7 @@ public FileStorePathFactory pathFactory() { protected FileStorePathFactory pathFactory(String format) { return new FileStorePathFactory( - options.path(), + options.schemaPath(), partitionType, options.partitionDefaultName(), format, @@ -119,12 +119,13 @@ protected FileStorePathFactory pathFactory(String format) { options.legacyPartitionName(), options.fileSuffixIncludeCompression(), options.fileCompression(), - options.dataFilePathDirectory()); + options.dataFilePathDirectory(), + options.dataPath()); } @Override public SnapshotManager snapshotManager() { - return new SnapshotManager(fileIO, options.path(), options.branch(), snapshotCache); + return new SnapshotManager(fileIO, options.schemaPath(), options.branch(), snapshotCache); } @Override @@ -275,7 +276,7 @@ public ChangelogDeletion newChangelogDeletion() { @Override public TagManager newTagManager() { - return new TagManager(fileIO, options.path()); + return new TagManager(fileIO, options.schemaPath()); } @Override @@ -344,7 +345,7 @@ public List createTagCallbacks() { @Override public ServiceManager newServiceManager() { - return new ServiceManager(fileIO, options.path()); + return new ServiceManager(fileIO, options.schemaPath()); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/append/AppendOnlyWriter.java b/paimon-core/src/main/java/org/apache/paimon/append/AppendOnlyWriter.java index a3087e362864..e70410b5d371 100644 --- a/paimon-core/src/main/java/org/apache/paimon/append/AppendOnlyWriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/append/AppendOnlyWriter.java @@ -244,7 +244,7 @@ public void close() throws Exception { for (DataFileMeta file : compactAfter) { // appendOnlyCompactManager will rewrite the file and no file upgrade will occur, so we // can directly delete the file in compactAfter. - fileIO.deleteQuietly(pathFactory.toPath(file.fileName())); + fileIO.deleteQuietly(pathFactory.toPath(file.fileName(), file.externalPath())); } sinkWriter.close(); @@ -271,7 +271,7 @@ public void toBufferedWriter() throws Exception { } finally { // remove small files for (DataFileMeta file : files) { - fileIO.deleteQuietly(pathFactory.toPath(file.fileName())); + fileIO.deleteQuietly(pathFactory.toPath(file.fileName(), file.externalPath())); } } } diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java index d7447c37dd79..fc0e9f6cad9c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java +++ b/paimon-core/src/main/java/org/apache/paimon/catalog/AbstractCatalog.java @@ -577,7 +577,7 @@ private void validateAutoCreateClose(Map options) { } private void validateCustomTablePath(Map options) { - if (!allowCustomTablePath() && options.containsKey(CoreOptions.PATH.key())) { + if (!allowCustomTablePath() && options.containsKey(CoreOptions.TABLE_SCHEMA_PATH.key())) { throw new UnsupportedOperationException( String.format( "The current catalog %s does not support specifying the table path when creating a table.", diff --git a/paimon-core/src/main/java/org/apache/paimon/iceberg/AbstractIcebergCommitCallback.java b/paimon-core/src/main/java/org/apache/paimon/iceberg/AbstractIcebergCommitCallback.java index f561546e8bb3..adf9b2f583f5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/iceberg/AbstractIcebergCommitCallback.java +++ b/paimon-core/src/main/java/org/apache/paimon/iceberg/AbstractIcebergCommitCallback.java @@ -108,7 +108,8 @@ public AbstractIcebergCommitCallback(FileStoreTable table, String commitUser) { table.coreOptions().toConfiguration().get(IcebergOptions.METADATA_ICEBERG_STORAGE); switch (storageType) { case TABLE_LOCATION: - this.pathFactory = new IcebergPathFactory(new Path(table.location(), "metadata")); + this.pathFactory = + new IcebergPathFactory(new Path(table.tableDataPath(), "metadata")); break; case HADOOP_CATALOG: case HIVE_CATALOG: @@ -139,11 +140,12 @@ public AbstractIcebergCommitCallback(FileStoreTable table, String commitUser) { public static Path catalogTableMetadataPath(FileStoreTable table) { Path icebergDBPath = catalogDatabasePath(table); - return new Path(icebergDBPath, String.format("%s/metadata", table.location().getName())); + return new Path( + icebergDBPath, String.format("%s/metadata", table.tableDataPath().getName())); } public static Path catalogDatabasePath(FileStoreTable table) { - Path dbPath = table.location().getParent(); + Path dbPath = table.tableDataPath().getParent(); final String dbSuffix = ".db"; if (dbPath.getName().endsWith(dbSuffix)) { String dbName = @@ -250,7 +252,7 @@ private void createMetadataWithoutBase(long snapshotId) throws IOException { IcebergMetadata metadata = new IcebergMetadata( tableUuid, - table.location().toString(), + table.tableDataPath().toString(), snapshotId, icebergSchema.highestFieldId(), Collections.singletonList(icebergSchema), @@ -448,7 +450,7 @@ private boolean collectFileChanges( boolean isAddOnly = true; for (ManifestEntry entry : manifestEntries) { String path = - fileStorePathFactory.bucketPath(entry.partition(), entry.bucket()) + fileStorePathFactory.dataBucketPath(entry.partition(), entry.bucket()) + "/" + entry.fileName(); switch (entry.kind()) { @@ -753,7 +755,7 @@ private void expireAllBefore(long snapshotId) throws IOException { private class SchemaCache { - SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.location()); + SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.tableSchemaPath()); Map schemas = new HashMap<>(); private IcebergSchema get(long schemaId) { 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 3be09ea6c229..f1964a31639b 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 @@ -135,7 +135,8 @@ public static DataFileMeta forAppend( List extraFiles, @Nullable byte[] embeddedIndex, @Nullable FileSource fileSource, - @Nullable List valueStatsCols) { + @Nullable List valueStatsCols, + @Nullable String externalPath) { return new DataFileMeta( fileName, fileSize, @@ -154,7 +155,7 @@ public static DataFileMeta forAppend( embeddedIndex, fileSource, valueStatsCols, - null); + externalPath); } public DataFileMeta( @@ -173,7 +174,8 @@ public DataFileMeta( @Nullable Long deleteRowCount, @Nullable byte[] embeddedIndex, @Nullable FileSource fileSource, - @Nullable List valueStatsCols) { + @Nullable List valueStatsCols, + @Nullable String externalPath) { this( fileName, fileSize, @@ -192,7 +194,7 @@ public DataFileMeta( embeddedIndex, fileSource, valueStatsCols, - null); + externalPath); } public DataFileMeta( @@ -449,8 +451,8 @@ public DataFileMeta copyWithoutStats() { public List collectFiles(DataFilePathFactory pathFactory) { List paths = new ArrayList<>(); - paths.add(pathFactory.toPath(fileName)); - extraFiles.forEach(f -> paths.add(pathFactory.toPath(f))); + paths.add(pathFactory.toPath(fileName, externalPath)); + extraFiles.forEach(f -> paths.add(pathFactory.toPath(f, externalPath))); return paths; } 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 b632d44c9420..2cb7f73d1570 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 @@ -32,7 +32,8 @@ public class DataFilePathFactory { public static final String INDEX_PATH_SUFFIX = ".index"; - private final Path parent; + private final Path warehouseDataPath; + private final Path externalDataPath; private final String uuid; private final AtomicInteger pathCount; @@ -43,13 +44,14 @@ public class DataFilePathFactory { private final String fileCompression; public DataFilePathFactory( - Path parent, + Path warehouseDataPath, String formatIdentifier, String dataFilePrefix, String changelogFilePrefix, boolean fileSuffixIncludeCompression, - String fileCompression) { - this.parent = parent; + String fileCompression, + Path externalDataPath) { + this.warehouseDataPath = warehouseDataPath; this.uuid = UUID.randomUUID().toString(); this.pathCount = new AtomicInteger(0); this.formatIdentifier = formatIdentifier; @@ -57,6 +59,11 @@ public DataFilePathFactory( this.changelogFilePrefix = changelogFilePrefix; this.fileSuffixIncludeCompression = fileSuffixIncludeCompression; this.fileCompression = fileCompression; + if (externalDataPath == null) { + this.externalDataPath = warehouseDataPath; + } else { + this.externalDataPath = externalDataPath; + } } public Path newPath() { @@ -75,11 +82,15 @@ private Path newPath(String prefix) { extension = "." + formatIdentifier; } String name = prefix + uuid + "-" + pathCount.getAndIncrement() + extension; - return new Path(parent, name); + return new Path(externalDataPath, name); } - public Path toPath(String fileName) { - return new Path(parent + "/" + fileName); + public Path toPath(String fileName, String externalPath) { + if (externalPath == null) { + return new Path(warehouseDataPath + "/" + fileName); + } else { + return new Path(externalPath, fileName); + } } @VisibleForTesting diff --git a/paimon-core/src/main/java/org/apache/paimon/io/FileIndexEvaluator.java b/paimon-core/src/main/java/org/apache/paimon/io/FileIndexEvaluator.java index 530b87165322..48aa273c5be7 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/FileIndexEvaluator.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/FileIndexEvaluator.java @@ -62,7 +62,7 @@ public static FileIndexResult evaluate( // go to file index check try (FileIndexPredicate predicate = new FileIndexPredicate( - dataFilePathFactory.toPath(indexFiles.get(0)), + dataFilePathFactory.toPath(indexFiles.get(0), file.externalPath()), fileIO, dataSchema.logicalRowType())) { return predicate.evaluate( 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 651c6a6f7b56..d77f6b71e446 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 @@ -195,7 +195,8 @@ public DataFileMeta result() throws IOException { deleteRecordCount, indexResult.embeddedIndexBytes(), fileSource, - valueStatsPair.getKey()); + valueStatsPair.getKey(), + path.getParent().toString()); } abstract Pair fetchKeyValueStats(SimpleColStats[] rowStats); diff --git a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java index 7e272fc97c65..af55ae2eb00b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java +++ b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java @@ -97,16 +97,24 @@ private KeyValueFileReaderFactory( @Override public RecordReader createRecordReader(DataFileMeta file) throws IOException { - return createRecordReader(file.schemaId(), file.fileName(), file.fileSize(), file.level()); + return createRecordReader( + file.schemaId(), + file.fileName(), + file.fileSize(), + file.level(), + file.externalPath()); } public RecordReader createRecordReader( - long schemaId, String fileName, long fileSize, int level) throws IOException { + long schemaId, String fileName, long fileSize, int level, String externalPath) + throws IOException { if (fileSize >= asyncThreshold && fileName.endsWith(".orc")) { return new AsyncRecordReader<>( - () -> createRecordReader(schemaId, fileName, level, false, 2, fileSize)); + () -> + createRecordReader( + schemaId, fileName, level, false, 2, fileSize, externalPath)); } - return createRecordReader(schemaId, fileName, level, true, null, fileSize); + return createRecordReader(schemaId, fileName, level, true, null, fileSize, externalPath); } private FileRecordReader createRecordReader( @@ -115,7 +123,8 @@ private FileRecordReader createRecordReader( int level, boolean reuseFormat, @Nullable Integer orcPoolSize, - long fileSize) + long fileSize, + String externalPath) throws IOException { String formatIdentifier = DataFilePathFactory.formatIdentifier(fileName); @@ -132,7 +141,7 @@ private FileRecordReader createRecordReader( new FormatKey(schemaId, formatIdentifier), key -> formatSupplier.get()) : formatSupplier.get(); - Path filePath = pathFactory.toPath(fileName); + Path filePath = pathFactory.toPath(fileName, externalPath); FileRecordReader fileRecordReader = new DataFileRecordReader( 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 a6aae3985bd4..0f17b098ac2f 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 @@ -142,14 +142,15 @@ private KeyValueDataFileWriter createDataFileWriter( fileIndexOptions); } - public void deleteFile(String filename, int level) { - fileIO.deleteQuietly(formatContext.pathFactory(level).toPath(filename)); + public void deleteFile(String filename, int level, String externalPath) { + fileIO.deleteQuietly(formatContext.pathFactory(level).toPath(filename, externalPath)); } - public void copyFile(String sourceFileName, String targetFileName, int level) + public void copyFile( + String sourceFileName, String targetFileName, int level, String externalPath) throws IOException { - Path sourcePath = formatContext.pathFactory(level).toPath(sourceFileName); - Path targetPath = formatContext.pathFactory(level).toPath(targetFileName); + Path sourcePath = formatContext.pathFactory(level).toPath(sourceFileName, externalPath); + Path targetPath = formatContext.pathFactory(level).toPath(targetFileName, externalPath); fileIO.copyFile(sourcePath, targetPath, true); } 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 8c2e8ec9498c..ae5dfa07ad8a 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 @@ -124,6 +124,7 @@ public DataFileMeta result() throws IOException { : Collections.singletonList(indexResult.independentIndexFile()), indexResult.embeddedIndexBytes(), fileSource, - statsPair.getKey()); + statsPair.getKey(), + path.getParent().toString()); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/manifest/ExpireFileEntry.java b/paimon-core/src/main/java/org/apache/paimon/manifest/ExpireFileEntry.java index 060360623cd0..5d6d68144e1c 100644 --- a/paimon-core/src/main/java/org/apache/paimon/manifest/ExpireFileEntry.java +++ b/paimon-core/src/main/java/org/apache/paimon/manifest/ExpireFileEntry.java @@ -41,8 +41,19 @@ public ExpireFileEntry( @Nullable byte[] embeddedIndex, BinaryRow minKey, BinaryRow maxKey, - @Nullable FileSource fileSource) { - super(kind, partition, bucket, level, fileName, extraFiles, embeddedIndex, minKey, maxKey); + @Nullable FileSource fileSource, + @Nullable String externalPath) { + super( + kind, + partition, + bucket, + level, + fileName, + extraFiles, + embeddedIndex, + minKey, + maxKey, + externalPath); this.fileSource = fileSource; } @@ -61,7 +72,8 @@ public static ExpireFileEntry from(ManifestEntry entry) { entry.file().embeddedIndex(), entry.minKey(), entry.maxKey(), - entry.file().fileSource().orElse(null)); + entry.file().fileSource().orElse(null), + entry.externalPath()); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/manifest/FileEntry.java b/paimon-core/src/main/java/org/apache/paimon/manifest/FileEntry.java index a2569beac61c..0a05559c9bde 100644 --- a/paimon-core/src/main/java/org/apache/paimon/manifest/FileEntry.java +++ b/paimon-core/src/main/java/org/apache/paimon/manifest/FileEntry.java @@ -54,6 +54,8 @@ public interface FileEntry { String fileName(); + String externalPath(); + Identifier identifier(); BinaryRow minKey(); diff --git a/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestEntry.java b/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestEntry.java index 626e0a5d468f..aec6b9063003 100644 --- a/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestEntry.java +++ b/paimon-core/src/main/java/org/apache/paimon/manifest/ManifestEntry.java @@ -92,6 +92,11 @@ public String fileName() { return file.fileName(); } + @Override + public String externalPath() { + return file.externalPath(); + } + @Override public BinaryRow minKey() { return file.minKey(); diff --git a/paimon-core/src/main/java/org/apache/paimon/manifest/SimpleFileEntry.java b/paimon-core/src/main/java/org/apache/paimon/manifest/SimpleFileEntry.java index fdaed2b85aaf..df3b4652307b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/manifest/SimpleFileEntry.java +++ b/paimon-core/src/main/java/org/apache/paimon/manifest/SimpleFileEntry.java @@ -38,6 +38,7 @@ public class SimpleFileEntry implements FileEntry { @Nullable private final byte[] embeddedIndex; private final BinaryRow minKey; private final BinaryRow maxKey; + private final String externalPath; public SimpleFileEntry( FileKind kind, @@ -48,7 +49,8 @@ public SimpleFileEntry( List extraFiles, @Nullable byte[] embeddedIndex, BinaryRow minKey, - BinaryRow maxKey) { + BinaryRow maxKey, + String externalPath) { this.kind = kind; this.partition = partition; this.bucket = bucket; @@ -58,6 +60,7 @@ public SimpleFileEntry( this.embeddedIndex = embeddedIndex; this.minKey = minKey; this.maxKey = maxKey; + this.externalPath = externalPath; } public static SimpleFileEntry from(ManifestEntry entry) { @@ -70,7 +73,8 @@ public static SimpleFileEntry from(ManifestEntry entry) { entry.file().extraFiles(), entry.file().embeddedIndex(), entry.minKey(), - entry.maxKey()); + entry.maxKey(), + entry.externalPath()); } public static List from(List entries) { @@ -122,6 +126,10 @@ public List extraFiles() { return extraFiles; } + public String externalPath() { + return externalPath; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -138,12 +146,14 @@ public boolean equals(Object o) { && Objects.equals(fileName, that.fileName) && Objects.equals(extraFiles, that.extraFiles) && Objects.equals(minKey, that.minKey) - && Objects.equals(maxKey, that.maxKey); + && Objects.equals(maxKey, that.maxKey) + && Objects.equals(externalPath, that.externalPath); } @Override public int hashCode() { - return Objects.hash(kind, partition, bucket, level, fileName, extraFiles, minKey, maxKey); + return Objects.hash( + kind, partition, bucket, level, fileName, extraFiles, minKey, maxKey, externalPath); } @Override @@ -165,6 +175,8 @@ public String toString() { + minKey + ", maxKey=" + maxKey + + ", externalPath=" + + externalPath + '}'; } } diff --git a/paimon-core/src/main/java/org/apache/paimon/manifest/SimpleFileEntrySerializer.java b/paimon-core/src/main/java/org/apache/paimon/manifest/SimpleFileEntrySerializer.java index bdc89b8d4c3d..50065a93fadd 100644 --- a/paimon-core/src/main/java/org/apache/paimon/manifest/SimpleFileEntrySerializer.java +++ b/paimon-core/src/main/java/org/apache/paimon/manifest/SimpleFileEntrySerializer.java @@ -63,6 +63,7 @@ public SimpleFileEntry convertFrom(int version, InternalRow row) { fromStringArrayData(file.getArray(11)), file.isNullAt(14) ? null : file.getBinary(14), deserializeBinaryRow(file.getBinary(3)), - deserializeBinaryRow(file.getBinary(4))); + deserializeBinaryRow(file.getBinary(4)), + file.getString(17).toString()); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/mergetree/MergeTreeWriter.java b/paimon-core/src/main/java/org/apache/paimon/mergetree/MergeTreeWriter.java index f2a964bae16a..719f6721fdd5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/mergetree/MergeTreeWriter.java +++ b/paimon-core/src/main/java/org/apache/paimon/mergetree/MergeTreeWriter.java @@ -243,7 +243,11 @@ private void flushWriteBuffer(boolean waitForLatestCompaction, boolean forcedFul for (DataFileMeta dataMeta : dataMetas) { DataFileMeta changelogMeta = dataMeta.rename(writerFactory.newChangelogPath(0).getName()); - writerFactory.copyFile(dataMeta.fileName(), changelogMeta.fileName(), 0); + writerFactory.copyFile( + dataMeta.fileName(), + changelogMeta.fileName(), + 0, + dataMeta.externalPath()); changelogMetas.add(changelogMeta); } newFilesChangelog.addAll(changelogMetas); @@ -341,7 +345,7 @@ private void updateCompactResult(CompactResult result) { // 2. This file is not the input of upgraded. if (!compactBefore.containsKey(file.fileName()) && !afterFiles.contains(file.fileName())) { - writerFactory.deleteFile(file.fileName(), file.level()); + writerFactory.deleteFile(file.fileName(), file.level(), file.externalPath()); } } else { compactBefore.put(file.fileName(), file); @@ -375,7 +379,7 @@ public void close() throws Exception { deletedFiles.clear(); for (DataFileMeta file : newFilesChangelog) { - writerFactory.deleteFile(file.fileName(), file.level()); + writerFactory.deleteFile(file.fileName(), file.level(), file.externalPath()); } newFilesChangelog.clear(); @@ -390,12 +394,12 @@ public void close() throws Exception { compactAfter.clear(); for (DataFileMeta file : compactChangelog) { - writerFactory.deleteFile(file.fileName(), file.level()); + writerFactory.deleteFile(file.fileName(), file.level(), file.externalPath()); } compactChangelog.clear(); for (DataFileMeta file : delete) { - writerFactory.deleteFile(file.fileName(), file.level()); + writerFactory.deleteFile(file.fileName(), file.level(), file.externalPath()); } if (compactDeletionFile != null) { diff --git a/paimon-core/src/main/java/org/apache/paimon/migrate/FileMetaUtils.java b/paimon-core/src/main/java/org/apache/paimon/migrate/FileMetaUtils.java index 391c5f9bb615..526f05564aa1 100644 --- a/paimon-core/src/main/java/org/apache/paimon/migrate/FileMetaUtils.java +++ b/paimon-core/src/main/java/org/apache/paimon/migrate/FileMetaUtils.java @@ -169,7 +169,8 @@ private static DataFileMeta constructFileMeta( Collections.emptyList(), null, FileSource.APPEND, - null); + null, + path.getParent().toString()); } public static BinaryRow writePartitionValue( diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/FileDeletionBase.java b/paimon-core/src/main/java/org/apache/paimon/operation/FileDeletionBase.java index cfecd767b6fb..287e8c1355d2 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/FileDeletionBase.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/FileDeletionBase.java @@ -136,7 +136,7 @@ public void cleanEmptyDirectories() { List toDeleteEmptyDirectory = new ArrayList<>(); // try to delete bucket directories for (Integer bucket : entry.getValue()) { - toDeleteEmptyDirectory.add(pathFactory.bucketPath(entry.getKey(), bucket)); + toDeleteEmptyDirectory.add(pathFactory.dataBucketPath(entry.getKey(), bucket)); } deleteFiles(toDeleteEmptyDirectory, this::tryDeleteEmptyDirectory); @@ -217,7 +217,7 @@ protected void getDataFileToDelete( // we cannot delete a data file directly when we meet a DELETE entry, because that // file might be upgraded for (ExpireFileEntry entry : dataFileEntries) { - Path bucketPath = pathFactory.bucketPath(entry.partition(), entry.bucket()); + Path bucketPath = pathFactory.dataBucketPath(entry.partition(), entry.bucket()); Path dataFilePath = new Path(bucketPath, entry.fileName()); switch (entry.kind()) { case ADD: @@ -263,7 +263,7 @@ private void deleteAddedDataFiles(List manifestEntries) { if (entry.kind() == FileKind.ADD) { dataFileToDelete.add( new Path( - pathFactory.bucketPath(entry.partition(), entry.bucket()), + pathFactory.dataBucketPath(entry.partition(), entry.bucket()), entry.fileName())); recordDeletionBuckets(entry); } diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java index 001132e1671c..f5556218f8b4 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreCommitImpl.java @@ -593,7 +593,7 @@ public void abort(List commitMessages) { toDelete.addAll(commitMessage.compactIncrement().changelogFiles()); for (DataFileMeta file : toDelete) { - fileIO.deleteQuietly(pathFactory.toPath(file.fileName())); + fileIO.deleteQuietly(pathFactory.toPath(file.fileName(), file.externalPath())); } } } diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/OrphanFilesClean.java b/paimon-core/src/main/java/org/apache/paimon/operation/OrphanFilesClean.java index 274cdd52fe14..4d1e935885d3 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/OrphanFilesClean.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/OrphanFilesClean.java @@ -94,7 +94,7 @@ public OrphanFilesClean( this.table = table; this.fileIO = table.fileIO(); this.partitionKeysNum = table.partitionKeys().size(); - this.location = table.location(); + this.location = table.tableDataPath(); this.olderThanMillis = olderThanMillis; this.fileCleaner = fileCleaner; } diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/RawFileSplitRead.java b/paimon-core/src/main/java/org/apache/paimon/operation/RawFileSplitRead.java index 4fda82f4e88f..70452dd92ce0 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/RawFileSplitRead.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/RawFileSplitRead.java @@ -211,7 +211,7 @@ private FileRecordReader createFileReader( FormatReaderContext formatReaderContext = new FormatReaderContext( fileIO, - dataFilePathFactory.toPath(file.fileName()), + dataFilePathFactory.toPath(file.fileName(), file.externalPath()), file.fileSize(), fileIndexResult); FileRecordReader fileRecordReader = diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/TagDeletion.java b/paimon-core/src/main/java/org/apache/paimon/operation/TagDeletion.java index 2722ed0c7ec8..e1f4e5456a1b 100644 --- a/paimon-core/src/main/java/org/apache/paimon/operation/TagDeletion.java +++ b/paimon-core/src/main/java/org/apache/paimon/operation/TagDeletion.java @@ -80,7 +80,7 @@ public void cleanUnusedDataFiles(Snapshot taggedSnapshot, Predicate dataFileToDelete = new HashSet<>(); for (ExpireFileEntry entry : manifestEntries) { if (!skipper.test(entry)) { - Path bucketPath = pathFactory.bucketPath(entry.partition(), entry.bucket()); + Path bucketPath = pathFactory.dataBucketPath(entry.partition(), entry.bucket()); dataFileToDelete.add(new Path(bucketPath, entry.fileName())); for (String file : entry.extraFiles()) { dataFileToDelete.add(new Path(bucketPath, file)); diff --git a/paimon-core/src/main/java/org/apache/paimon/partition/actions/PartitionMarkDoneAction.java b/paimon-core/src/main/java/org/apache/paimon/partition/actions/PartitionMarkDoneAction.java index d6b13a25e270..3f000394ac64 100644 --- a/paimon-core/src/main/java/org/apache/paimon/partition/actions/PartitionMarkDoneAction.java +++ b/paimon-core/src/main/java/org/apache/paimon/partition/actions/PartitionMarkDoneAction.java @@ -45,7 +45,8 @@ static List createActions( switch (action) { case "success-file": return new SuccessFileMarkDoneAction( - fileStoreTable.fileIO(), fileStoreTable.location()); + fileStoreTable.fileIO(), + fileStoreTable.tableDataPath()); case "done-partition": return new AddDonePartitionAction( createMetastoreClient(fileStoreTable, options)); 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..dc0a7f924302 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.Path; 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 Path tableSchemaPath() { + return wrapped.tableSchemaPath(); + } + 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/schema/SchemaManager.java b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java index 9edbe901b6fa..04377260b6cc 100644 --- a/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java +++ b/paimon-core/src/main/java/org/apache/paimon/schema/SchemaManager.java @@ -264,7 +264,7 @@ private void checkSchemaForExternalTable(Schema existsSchema, Schema newSchema) (key, value) -> { // ignore `owner` and `path` if (!key.equals(Catalog.OWNER_PROP) - && !key.equals(CoreOptions.PATH.key()) + && !key.equals(CoreOptions.TABLE_SCHEMA_PATH.key()) && (!existsOptions.containsKey(key) || !existsOptions.get(key).equals(value))) { throw new RuntimeException( @@ -890,7 +890,7 @@ public static void checkResetTableOption(String key) { } public static void checkAlterTablePath(String key) { - if (CoreOptions.PATH.key().equalsIgnoreCase(key)) { + if (CoreOptions.TABLE_SCHEMA_PATH.key().equalsIgnoreCase(key)) { throw new UnsupportedOperationException("Change path is not supported yet."); } } 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..13e2e055b4fc 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 @@ -86,7 +86,8 @@ import java.util.SortedMap; 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.CoreOptions.TABLE_SCHEMA_PATH; import static org.apache.paimon.utils.Preconditions.checkArgument; /** Abstract {@link FileStoreTable}. */ @@ -97,7 +98,8 @@ abstract class AbstractFileStoreTable implements FileStoreTable { private static final String WATERMARK_PREFIX = "watermark-"; protected final FileIO fileIO; - protected final Path path; + protected final Path tableSchemaPath; + protected final Path tableDataPath; protected final TableSchema tableSchema; protected final CatalogEnvironment catalogEnvironment; @@ -107,19 +109,32 @@ abstract class AbstractFileStoreTable implements FileStoreTable { protected AbstractFileStoreTable( FileIO fileIO, - Path path, + Path tableSchemaPath, TableSchema tableSchema, - CatalogEnvironment catalogEnvironment) { + CatalogEnvironment catalogEnvironment, + Path tableDataPath) { this.fileIO = fileIO; - this.path = path; - if (!tableSchema.options().containsKey(PATH.key())) { + this.tableSchemaPath = tableSchemaPath; + if (!tableSchema.options().containsKey(TABLE_SCHEMA_PATH.key())) { // make sure table is always available Map newOptions = new HashMap<>(tableSchema.options()); - newOptions.put(PATH.key(), path.toString()); + newOptions.put(TABLE_SCHEMA_PATH.key(), tableSchemaPath.toString()); + tableSchema = tableSchema.copy(newOptions); + } + + if (tableDataPath != null && !tableSchema.options().containsKey(TABLE_DATA_PATH.key())) { + // set table external path + Map newOptions = new HashMap<>(tableSchema.options()); + newOptions.put(TABLE_DATA_PATH.key(), tableDataPath.toString()); tableSchema = tableSchema.copy(newOptions); } this.tableSchema = tableSchema; this.catalogEnvironment = catalogEnvironment; + if (tableDataPath == null) { + this.tableDataPath = tableSchemaPath; + } else { + this.tableDataPath = tableDataPath; + } } public String currentBranch() { @@ -183,7 +198,7 @@ public Identifier identifier() { Identifier identifier = catalogEnvironment.identifier(); return identifier == null ? SchemaManager.identifierFromPath( - location().toUri().toString(), true, currentBranch()) + tableSchemaPath().toUri().toString(), true, currentBranch()) : identifier; } @@ -334,7 +349,7 @@ private FileStoreTable copyInternal(Map dynamicOptions, boolean Options newOptions = Options.fromMap(options); // set path always - newOptions.set(PATH, path.toString()); + newOptions.set(TABLE_SCHEMA_PATH, tableSchemaPath.toString()); // set dynamic options with default values CoreOptions.setDefaultValues(newOptions); @@ -372,9 +387,17 @@ public FileStoreTable copy(TableSchema newTableSchema) { AbstractFileStoreTable copied = newTableSchema.primaryKeys().isEmpty() ? new AppendOnlyFileStoreTable( - fileIO, path, newTableSchema, catalogEnvironment) + fileIO, + tableSchemaPath, + newTableSchema, + catalogEnvironment, + tableDataPath) : new PrimaryKeyFileStoreTable( - fileIO, path, newTableSchema, catalogEnvironment); + fileIO, + tableSchemaPath, + newTableSchema, + catalogEnvironment, + tableDataPath); if (snapshotCache != null) { copied.setSnapshotCache(snapshotCache); } @@ -389,7 +412,7 @@ public FileStoreTable copy(TableSchema newTableSchema) { @Override public SchemaManager schemaManager() { - return new SchemaManager(fileIO(), path, currentBranch()); + return new SchemaManager(fileIO(), tableSchemaPath, currentBranch()); } @Override @@ -403,8 +426,8 @@ public FileIO fileIO() { } @Override - public Path location() { - return path; + public Path tableDataPath() { + return tableDataPath; } @Override @@ -454,7 +477,7 @@ public TableCommitImpl newCommit(String commitUser) { options.writeOnly() ? null : store().newTagCreationManager(), catalogEnvironment.lockFactory().create(), CoreOptions.fromMap(options()).consumerExpireTime(), - new ConsumerManager(fileIO, path, snapshotManager().branch()), + new ConsumerManager(fileIO, tableSchemaPath, snapshotManager().branch()), options.snapshotExpireExecutionMode(), name(), options.forceCreatingSnapshot()); @@ -712,12 +735,13 @@ public void rollbackTo(String tagName) { @Override public TagManager tagManager() { - return new TagManager(fileIO, path, currentBranch()); + return new TagManager(fileIO, tableSchemaPath, currentBranch()); } @Override public BranchManager branchManager() { - return new BranchManager(fileIO, path, snapshotManager(), tagManager(), schemaManager()); + return new BranchManager( + fileIO, tableSchemaPath, snapshotManager(), tagManager(), schemaManager()); } @Override @@ -729,7 +753,7 @@ public FileStoreTable switchToBranch(String branchName) { } Optional optionalSchema = - new SchemaManager(fileIO(), location(), targetBranch).latest(); + new SchemaManager(fileIO(), tableDataPath(), targetBranch).latest(); Preconditions.checkArgument( optionalSchema.isPresent(), "Branch " + targetBranch + " does not exist"); @@ -738,7 +762,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()); + fileIO(), tableDataPath(), branchSchema, new Options(), catalogEnvironment()); } private RollbackHelper rollbackHelper() { @@ -764,6 +788,7 @@ 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(tableSchemaPath, that.tableSchemaPath) + && 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 103fa64050aa..93a33f3365dd 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(), null); } AppendOnlyFileStoreTable( FileIO fileIO, - Path path, + Path tableSchemaPath, TableSchema tableSchema, - CatalogEnvironment catalogEnvironment) { - super(fileIO, path, tableSchema, catalogEnvironment); + CatalogEnvironment catalogEnvironment, + Path tableDataPath) { + super(fileIO, tableSchemaPath, tableSchema, catalogEnvironment, tableDataPath); } @Override @@ -173,4 +174,9 @@ protected List createCommitCallbacks(String commitUser) { return callbacks; } + + @Override + public Path tableSchemaPath() { + return tableSchemaPath; + } } 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..4121263260e1 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 @@ -52,7 +52,9 @@ public interface DataTable extends InnerTable { */ DataTable switchToBranch(String branchName); - Path location(); + Path tableSchemaPath(); + + Path tableDataPath(); 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..42de5dea6e3f 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 @@ -111,8 +111,13 @@ public BranchManager branchManager() { } @Override - public Path location() { - return wrapped.location(); + public Path tableDataPath() { + return wrapped.tableDataPath(); + } + + @Override + public Path tableSchemaPath() { + return wrapped.tableSchemaPath(); } @Override 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..3ca2b634ba14 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.Path; 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 Path tableSchemaPath() { + return wrapped.tableSchemaPath(); + } + private FileStoreTable switchWrappedToBranch(String branchName) { Optional optionalSchema = wrapped.schemaManager().copyWithBranch(branchName).latest(); @@ -112,10 +118,11 @@ private FileStoreTable switchWrappedToBranch(String branchName) { branchSchema = branchSchema.copy(branchOptions.toMap()); return FileStoreTableFactory.createWithoutFallbackBranch( wrapped.fileIO(), - wrapped.location(), + wrapped.tableSchemaPath(), branchSchema, new Options(), - wrapped.catalogEnvironment()); + wrapped.catalogEnvironment(), + wrapped.tableDataPath()); } 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..383608822130 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 @@ -21,40 +21,35 @@ import org.apache.paimon.CoreOptions; import org.apache.paimon.catalog.CatalogContext; import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.HybridFileIO; import org.apache.paimon.fs.Path; import org.apache.paimon.options.Options; import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.schema.TableSchema; import org.apache.paimon.utils.StringUtils; -import java.io.IOException; -import java.io.UncheckedIOException; import java.util.Optional; -import static org.apache.paimon.CoreOptions.PATH; +import static org.apache.paimon.CoreOptions.TABLE_SCHEMA_PATH; import static org.apache.paimon.utils.Preconditions.checkArgument; /** Factory to create {@link FileStoreTable}. */ public class FileStoreTableFactory { public static FileStoreTable create(CatalogContext context) { - FileIO fileIO; - try { - fileIO = FileIO.get(CoreOptions.path(context.options()), context); - } catch (IOException e) { - throw new UncheckedIOException(e); - } + FileIO fileIO = new HybridFileIO(); + fileIO.configure(context); return create(fileIO, context.options()); } public static FileStoreTable create(FileIO fileIO, Path path) { Options options = new Options(); - options.set(PATH, path.toString()); + options.set(TABLE_SCHEMA_PATH, path.toString()); return create(fileIO, options); } public static FileStoreTable create(FileIO fileIO, Options options) { - Path tablePath = CoreOptions.path(options); + Path tablePath = CoreOptions.schemaPath(options); String branchName = CoreOptions.branch(options.toMap()); TableSchema tableSchema = new SchemaManager(fileIO, tablePath, branchName) @@ -82,21 +77,33 @@ public static FileStoreTable create( public static FileStoreTable create( FileIO fileIO, - Path tablePath, + Path tableSchemaPath, TableSchema tableSchema, Options dynamicOptions, CatalogEnvironment catalogEnvironment) { + CoreOptions coreOptions = CoreOptions.fromMap(tableSchema.options()); + Path tableDataPath = null; + if (coreOptions.getDataFileExternalPath() != null) { + String dbAndTableName = + tableSchemaPath.getParent().getName() + "/" + tableSchemaPath.getName(); + tableDataPath = new Path(coreOptions.getDataFileExternalPath(), dbAndTableName); + } FileStoreTable table = createWithoutFallbackBranch( - fileIO, tablePath, tableSchema, dynamicOptions, catalogEnvironment); - + fileIO, + tableSchemaPath, + tableSchema, + dynamicOptions, + catalogEnvironment, + tableDataPath); Options options = new Options(table.options()); String fallbackBranch = options.get(CoreOptions.SCAN_FALLBACK_BRANCH); + if (!StringUtils.isNullOrWhitespaceOnly(fallbackBranch)) { Options branchOptions = new Options(dynamicOptions.toMap()); branchOptions.set(CoreOptions.BRANCH, fallbackBranch); Optional schema = - new SchemaManager(fileIO, tablePath, fallbackBranch).latest(); + new SchemaManager(fileIO, tableSchemaPath, fallbackBranch).latest(); checkArgument( schema.isPresent(), "Cannot set '%s' = '%s' because the branch '%s' isn't existed.", @@ -105,7 +112,12 @@ public static FileStoreTable create( fallbackBranch); FileStoreTable fallbackTable = createWithoutFallbackBranch( - fileIO, tablePath, schema.get(), branchOptions, catalogEnvironment); + fileIO, + tableSchemaPath, + schema.get(), + branchOptions, + catalogEnvironment, + tableDataPath); table = new FallbackReadFileStoreTable(table, fallbackTable); } @@ -114,16 +126,25 @@ public static FileStoreTable create( public static FileStoreTable createWithoutFallbackBranch( FileIO fileIO, - Path tablePath, + Path tableSchemaPath, TableSchema tableSchema, Options dynamicOptions, - CatalogEnvironment catalogEnvironment) { + CatalogEnvironment catalogEnvironment, + Path tableDataPath) { FileStoreTable table = tableSchema.primaryKeys().isEmpty() ? new AppendOnlyFileStoreTable( - fileIO, tablePath, tableSchema, catalogEnvironment) + fileIO, + tableSchemaPath, + tableSchema, + catalogEnvironment, + tableDataPath) : new PrimaryKeyFileStoreTable( - fileIO, tablePath, tableSchema, catalogEnvironment); + fileIO, + tableSchemaPath, + 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..b720cf848009 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(), null); } PrimaryKeyFileStoreTable( FileIO fileIO, - Path path, + Path tableSchemaPath, TableSchema tableSchema, - CatalogEnvironment catalogEnvironment) { - super(fileIO, path, tableSchema, catalogEnvironment); + CatalogEnvironment catalogEnvironment, + Path tableDataPath) { + super(fileIO, tableSchemaPath, tableSchema, catalogEnvironment, tableDataPath); } @Override @@ -191,4 +192,9 @@ protected List createCommitCallbacks(String commitUser) { return callbacks; } + + @Override + public Path tableSchemaPath() { + return tableSchemaPath; + } } 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..d9d28d45826a 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 @@ -19,6 +19,7 @@ package org.apache.paimon.table.object; import org.apache.paimon.fs.FileIO; +import org.apache.paimon.fs.Path; import org.apache.paimon.manifest.ManifestCacheFilter; import org.apache.paimon.schema.TableSchema; import org.apache.paimon.table.DelegatedFileStoreTable; @@ -215,5 +216,10 @@ public ObjectTable switchToBranch(String branchName) { return new ObjectTableImpl( wrapped.switchToBranch(branchName), objectFileIO, objectLocation); } + + @Override + public Path tableSchemaPath() { + return wrapped.tableSchemaPath(); + } } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/query/LocalTableQuery.java b/paimon-core/src/main/java/org/apache/paimon/table/query/LocalTableQuery.java index 8ff5ce7a6580..2bedb10cc8cd 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/query/LocalTableQuery.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/query/LocalTableQuery.java @@ -166,7 +166,8 @@ private void newLookupLevels(BinaryRow partition, int bucket, List file.schemaId(), file.fileName(), file.fileSize(), - file.level()); + file.level(), + file.externalPath()); if (cacheRowFilter != null) { reader = reader.filter( diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/IncrementalStartingScanner.java b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/IncrementalStartingScanner.java index 9bfb54f2cf60..85b74656b212 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/IncrementalStartingScanner.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/IncrementalStartingScanner.java @@ -133,7 +133,7 @@ public Result scan(SnapshotReader reader) { for (Map.Entry, List> entry : grouped.entrySet()) { BinaryRow partition = entry.getKey().getLeft(); int bucket = entry.getKey().getRight(); - String bucketPath = reader.pathFactory().bucketPath(partition, bucket).toString(); + String bucketPath = reader.pathFactory().dataBucketPath(partition, bucket).toString(); for (SplitGenerator.SplitGroup splitGroup : reader.splitGenerator().splitForBatch(entry.getValue())) { DataSplit.Builder dataSplitBuilder = diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java index bf19ba10c689..2ca57a3ac7f1 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/SnapshotReaderImpl.java @@ -346,7 +346,7 @@ private List generateSplits( Pair.of(partition, bucket), Collections.emptyList()); for (SplitGenerator.SplitGroup splitGroup : splitGroups) { List dataFiles = splitGroup.files; - String bucketPath = pathFactory.bucketPath(partition, bucket).toString(); + String bucketPath = pathFactory.dataBucketPath(partition, bucket).toString(); builder.withDataFiles(dataFiles) .rawConvertible(splitGroup.rawConvertible) .withBucketPath(bucketPath); @@ -447,7 +447,8 @@ private Plan toChangesPlan( .withBeforeFiles(before) .withDataFiles(data) .isStreaming(isStreaming) - .withBucketPath(pathFactory.bucketPath(part, bucket).toString()); + .withBucketPath( + pathFactory.dataBucketPath(part, bucket).toString()); if (deletionVectors) { builder.withBeforeDeletionFiles( getDeletionFiles( diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/AggregationFieldsTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/AggregationFieldsTable.java index 8c0eed4d6b8b..ed3cd8802090 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/AggregationFieldsTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/AggregationFieldsTable.java @@ -81,7 +81,7 @@ public class AggregationFieldsTable implements ReadonlyTable { public AggregationFieldsTable(FileStoreTable dataTable) { this.fileIO = dataTable.fileIO(); - this.location = dataTable.location(); + this.location = dataTable.tableDataPath(); this.dataTable = dataTable; } 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..686a6d377894 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 @@ -179,8 +179,8 @@ public CoreOptions coreOptions() { } @Override - public Path location() { - return wrapped.location(); + public Path tableDataPath() { + return wrapped.tableDataPath(); } @Override @@ -208,6 +208,11 @@ public DataTable switchToBranch(String branchName) { return new AuditLogTable(wrapped.switchToBranch(branchName)); } + @Override + public Path tableSchemaPath() { + return wrapped.tableSchemaPath(); + } + @Override public InnerTableRead newRead() { return new AuditLogRead(wrapped.newRead()); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/BranchesTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/BranchesTable.java index 384a2eee92c8..9f6600dee114 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/BranchesTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/BranchesTable.java @@ -85,7 +85,7 @@ public class BranchesTable implements ReadonlyTable { public BranchesTable(FileStoreTable dataTable) { this.fileIO = dataTable.fileIO(); - this.location = dataTable.location(); + this.location = dataTable.tableDataPath(); this.dataTable = dataTable; } 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..1cc5bfd3f32e 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 @@ -137,8 +137,8 @@ public SimpleFileReader indexManifestFileReader() { } @Override - public Path location() { - return wrapped.location(); + public Path tableDataPath() { + return wrapped.tableDataPath(); } @Override @@ -167,9 +167,14 @@ public DataTable switchToBranch(String branchName) { wrapped.switchToBranch(branchName), isContinuous, databaseName); } + @Override + public Path tableSchemaPath() { + return wrapped.tableSchemaPath(); + } + @Override public String name() { - return "__internal_buckets_" + wrapped.location().getName(); + return "__internal_buckets_" + wrapped.tableDataPath().getName(); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/ConsumersTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/ConsumersTable.java index 7e4816b13510..a4fb93d80a47 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/ConsumersTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/ConsumersTable.java @@ -78,7 +78,7 @@ public class ConsumersTable implements ReadonlyTable { public ConsumersTable(FileStoreTable dataTable) { this.fileIO = dataTable.fileIO(); - this.location = dataTable.location(); + this.location = dataTable.tableDataPath(); this.branch = CoreOptions.branch(dataTable.schema().options()); this.dataTable = dataTable; } 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..fcaaea417af8 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 @@ -123,8 +123,8 @@ public SimpleFileReader indexManifestFileReader() { } @Override - public Path location() { - return wrapped.location(); + public Path tableDataPath() { + return wrapped.tableDataPath(); } @Override @@ -152,9 +152,14 @@ public DataTable switchToBranch(String branchName) { return new FileMonitorTable(wrapped.switchToBranch(branchName)); } + @Override + public Path tableSchemaPath() { + return wrapped.tableSchemaPath(); + } + @Override public String name() { - return "__internal_file_monitor_" + wrapped.location().getName(); + return "__internal_file_monitor_" + wrapped.tableDataPath().getName(); } @Override diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/OptionsTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/OptionsTable.java index ed20896646b2..5808143603d0 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/OptionsTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/OptionsTable.java @@ -72,7 +72,7 @@ public class OptionsTable implements ReadonlyTable { public OptionsTable(FileStoreTable dataTable) { this.fileIO = dataTable.fileIO(); - this.location = dataTable.location(); + this.location = dataTable.tableDataPath(); this.dataTable = dataTable; } 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..607ddd516d60 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 @@ -157,8 +157,8 @@ public CoreOptions coreOptions() { } @Override - public Path location() { - return wrapped.location(); + public Path tableDataPath() { + return wrapped.tableDataPath(); } @Override @@ -186,6 +186,11 @@ public DataTable switchToBranch(String branchName) { return new ReadOptimizedTable(wrapped.switchToBranch(branchName)); } + @Override + public Path tableSchemaPath() { + return wrapped.tableSchemaPath(); + } + @Override public InnerTableRead newRead() { return wrapped.newRead(); diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/SchemasTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/SchemasTable.java index 3cb0ff4783e9..b8a9cbc9dbe5 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/SchemasTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/SchemasTable.java @@ -98,7 +98,7 @@ public class SchemasTable implements ReadonlyTable { private final FileStoreTable dataTable; public SchemasTable(FileStoreTable dataTable) { - this.location = dataTable.location(); + this.location = dataTable.tableDataPath(); this.dataTable = dataTable; } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/SnapshotsTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/SnapshotsTable.java index 2af13ee937bd..10c072be8e23 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/SnapshotsTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/SnapshotsTable.java @@ -115,7 +115,7 @@ public class SnapshotsTable implements ReadonlyTable { public SnapshotsTable(FileStoreTable dataTable) { this.fileIO = dataTable.fileIO(); - this.location = dataTable.location(); + this.location = dataTable.tableDataPath(); this.dataTable = dataTable; } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/StatisticTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/StatisticTable.java index f0180444988c..10f40efae896 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/StatisticTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/StatisticTable.java @@ -120,7 +120,7 @@ public InnerTableScan withFilter(Predicate predicate) { public Plan innerPlan() { return () -> Collections.singletonList( - new StatisticTable.StatisticSplit(dataTable.location())); + new StatisticTable.StatisticSplit(dataTable.tableDataPath())); } } diff --git a/paimon-core/src/main/java/org/apache/paimon/table/system/TagsTable.java b/paimon-core/src/main/java/org/apache/paimon/table/system/TagsTable.java index 9aafdb5983fd..93d503181901 100644 --- a/paimon-core/src/main/java/org/apache/paimon/table/system/TagsTable.java +++ b/paimon-core/src/main/java/org/apache/paimon/table/system/TagsTable.java @@ -99,7 +99,7 @@ public class TagsTable implements ReadonlyTable { public TagsTable(FileStoreTable dataTable) { this.fileIO = dataTable.fileIO(); - this.location = dataTable.location(); + this.location = dataTable.tableDataPath(); this.branch = CoreOptions.branch(dataTable.schema().options()); this.dataTable = dataTable; } 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..ff574c92fd91 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,8 @@ public class FileStorePathFactory { public static final String BUCKET_PATH_PREFIX = "bucket-"; - private final Path root; + private final Path schemaRoot; + private final Path dataRoot; private final String uuid; private final InternalRowPartitionComputer partitionComputer; private final String formatIdentifier; @@ -56,7 +57,7 @@ public class FileStorePathFactory { private final AtomicInteger statsFileCount; public FileStorePathFactory( - Path root, + Path schemaRoot, RowType partitionType, String defaultPartValue, String formatIdentifier, @@ -65,8 +66,9 @@ public FileStorePathFactory( boolean legacyPartitionName, boolean fileSuffixIncludeCompression, String fileCompression, - @Nullable String dataFilePathDirectory) { - this.root = root; + @Nullable String dataFilePathDirectory, + Path dataRoot) { + this.schemaRoot = schemaRoot; this.dataFilePathDirectory = dataFilePathDirectory; this.uuid = UUID.randomUUID().toString(); @@ -83,10 +85,11 @@ public FileStorePathFactory( this.indexManifestCount = new AtomicInteger(0); this.indexFileCount = new AtomicInteger(0); this.statsFileCount = new AtomicInteger(0); - } - - public Path root() { - return root; + if (dataRoot == null) { + this.dataRoot = schemaRoot; + } else { + this.dataRoot = dataRoot; + } } @VisibleForTesting @@ -99,12 +102,16 @@ public static InternalRowPartitionComputer getPartitionComputer( public Path newManifestFile() { return new Path( - root + "/manifest/manifest-" + uuid + "-" + manifestFileCount.getAndIncrement()); + schemaRoot + + "/manifest/manifest-" + + uuid + + "-" + + manifestFileCount.getAndIncrement()); } public Path newManifestList() { return new Path( - root + schemaRoot + "/manifest/manifest-list-" + uuid + "-" @@ -112,25 +119,30 @@ public Path newManifestList() { } public Path toManifestFilePath(String manifestFileName) { - return new Path(root + "/manifest/" + manifestFileName); + return new Path(schemaRoot + "/manifest/" + manifestFileName); } public Path toManifestListPath(String manifestListName) { - return new Path(root + "/manifest/" + manifestListName); + return new Path(schemaRoot + "/manifest/" + manifestListName); } public DataFilePathFactory createDataFilePathFactory(BinaryRow partition, int bucket) { return new DataFilePathFactory( - bucketPath(partition, bucket), + dataBucketPath(partition, bucket), formatIdentifier, dataFilePrefix, changelogFilePrefix, fileSuffixIncludeCompression, - fileCompression); + fileCompression, + warehouseBucketPath(partition, bucket)); } - public Path bucketPath(BinaryRow partition, int bucket) { - return new Path(root, relativeBucketPath(partition, bucket)); + public Path dataBucketPath(BinaryRow partition, int bucket) { + return new Path(dataRoot, relativeBucketPath(partition, bucket)); + } + + public Path warehouseBucketPath(BinaryRow partition, int bucket) { + return new Path(schemaRoot, relativeBucketPath(partition, bucket)); } public Path relativeBucketPath(BinaryRow partition, int bucket) { @@ -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(schemaRoot + "/" + p)) .collect(Collectors.toList()); } @@ -202,7 +214,7 @@ public PathFactory indexManifestFileFactory() { @Override public Path newPath() { return new Path( - root + schemaRoot + "/manifest/index-manifest-" + uuid + "-" @@ -211,7 +223,7 @@ public Path newPath() { @Override public Path toPath(String fileName) { - return new Path(root + "/manifest/" + fileName); + return new Path(schemaRoot + "/manifest/" + fileName); } }; } @@ -221,12 +233,16 @@ public PathFactory indexFileFactory() { @Override public Path newPath() { return new Path( - root + "/index/index-" + uuid + "-" + indexFileCount.getAndIncrement()); + schemaRoot + + "/index/index-" + + uuid + + "-" + + indexFileCount.getAndIncrement()); } @Override public Path toPath(String fileName) { - return new Path(root + "/index/" + fileName); + return new Path(schemaRoot + "/index/" + fileName); } }; } @@ -236,7 +252,7 @@ public PathFactory statsFileFactory() { @Override public Path newPath() { return new Path( - root + schemaRoot + "/statistics/stats-" + uuid + "-" @@ -245,7 +261,7 @@ public Path newPath() { @Override public Path toPath(String fileName) { - return new Path(root + "/statistics/" + fileName); + return new Path(schemaRoot + "/statistics/" + fileName); } }; } 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..a516c1c77421 100644 --- a/paimon-core/src/test/java/org/apache/paimon/TestAppendFileStore.java +++ b/paimon-core/src/test/java/org/apache/paimon/TestAppendFileStore.java @@ -155,7 +155,7 @@ public static TestAppendFileStore createAppendStore( FileIO fileIO = FileIOFinder.find(new Path(root)); SchemaManager schemaManage = new SchemaManager(new LocalFileIO(), path); - options.put(CoreOptions.PATH.key(), root); + options.put(CoreOptions.TABLE_SCHEMA_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..348cce069273 100644 --- a/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java +++ b/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java @@ -144,7 +144,7 @@ private TestFileStore( } private static SchemaManager schemaManager(String root, CoreOptions options) { - return new SchemaManager(FileIOFinder.find(new Path(root)), options.path()); + return new SchemaManager(FileIOFinder.find(new Path(root)), options.schemaPath()); } public AbstractFileStoreWrite newWrite() { @@ -159,7 +159,7 @@ public ExpireSnapshots newExpire(int numRetainedMin, int numRetainedMax, long mi return new ExpireSnapshotsImpl( snapshotManager(), newSnapshotDeletion(), - new TagManager(fileIO, options.path())) + new TagManager(fileIO, options.schemaPath())) .config( ExpireConfig.builder() .snapshotRetainMax(numRetainedMax) @@ -172,7 +172,7 @@ public ExpireSnapshots newExpire(ExpireConfig expireConfig) { return new ExpireSnapshotsImpl( snapshotManager(), newSnapshotDeletion(), - new TagManager(fileIO, options.path())) + new TagManager(fileIO, options.schemaPath())) .config(expireConfig); } @@ -180,7 +180,7 @@ public ExpireSnapshots newChangelogExpire(ExpireConfig config) { ExpireChangelogImpl impl = new ExpireChangelogImpl( snapshotManager(), - new TagManager(fileIO, options.path()), + new TagManager(fileIO, options.schemaPath()), newChangelogDeletion()); impl.config(config); return impl; @@ -532,7 +532,7 @@ public void assertCleaned() throws IOException { private Set getFilesInUse() { Set result = new HashSet<>(); - SchemaManager schemaManager = new SchemaManager(fileIO, options.path()); + SchemaManager schemaManager = new SchemaManager(fileIO, options.schemaPath()); schemaManager.listAllIds().forEach(id -> result.add(schemaManager.toSchemaPath(id))); SnapshotManager snapshotManager = snapshotManager(); @@ -646,7 +646,7 @@ private static Set getSnapshotFileInUse( for (ManifestEntry entry : entries) { result.add( new Path( - pathFactory.bucketPath(entry.partition(), entry.bucket()), + pathFactory.dataBucketPath(entry.partition(), entry.bucket()), entry.file().fileName())); } @@ -668,7 +668,7 @@ private static Set getSnapshotFileInUse( == FileSource.APPEND) { result.add( new Path( - pathFactory.bucketPath(entry.partition(), entry.bucket()), + pathFactory.dataBucketPath(entry.partition(), entry.bucket()), entry.file().fileName())); } } @@ -718,7 +718,7 @@ private static Set getChangelogFileInUse( if (entry.file().fileSource().orElse(FileSource.APPEND) == FileSource.APPEND) { result.add( new Path( - pathFactory.bucketPath(entry.partition(), entry.bucket()), + pathFactory.dataBucketPath(entry.partition(), entry.bucket()), entry.file().fileName())); } } @@ -733,7 +733,7 @@ private static Set getChangelogFileInUse( for (ManifestEntry entry : files) { result.add( new Path( - pathFactory.bucketPath(entry.partition(), entry.bucket()), + pathFactory.dataBucketPath(entry.partition(), entry.bucket()), entry.file().fileName())); } } @@ -797,7 +797,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_SCHEMA_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 a9012ed89b34..0e52cf9c27d2 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 @@ -125,7 +125,7 @@ public void testSingleWrite() throws Exception { DataFileMeta meta = increment.newFilesIncrement().newFiles().get(0); assertThat(meta).isNotNull(); - Path path = pathFactory.toPath(meta.fileName()); + Path path = pathFactory.toPath(meta.fileName(), null); assertThat(LocalFileIO.create().exists(path)).isTrue(); assertThat(meta.rowCount()).isEqualTo(1L); @@ -186,7 +186,7 @@ public void testMultipleCommits() throws Exception { assertThat(inc.newFilesIncrement().newFiles().size()).isEqualTo(1); DataFileMeta meta = inc.newFilesIncrement().newFiles().get(0); - Path path = pathFactory.toPath(meta.fileName()); + Path path = pathFactory.toPath(meta.fileName(), null); assertThat(LocalFileIO.create().exists(path)).isTrue(); assertThat(meta.rowCount()).isEqualTo(100L); @@ -227,7 +227,7 @@ public void testRollingWrite() throws Exception { int id = 0; for (DataFileMeta meta : firstInc.newFilesIncrement().newFiles()) { - Path path = pathFactory.toPath(meta.fileName()); + Path path = pathFactory.toPath(meta.fileName(), null); assertThat(LocalFileIO.create().exists(path)).isTrue(); assertThat(meta.rowCount()).isEqualTo(1000L); @@ -524,7 +524,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(), + null); } private AppendOnlyWriter createEmptyWriter(long targetFileSize) { @@ -647,7 +648,7 @@ private DataFileMeta generateCompactAfter(List toCompact) throws I long minSeq = toCompact.get(0).minSequenceNumber(); long maxSeq = toCompact.get(size - 1).maxSequenceNumber(); String fileName = "compact-" + UUID.randomUUID(); - LocalFileIO.create().newOutputStream(pathFactory.toPath(fileName), false).close(); + LocalFileIO.create().newOutputStream(pathFactory.toPath(fileName, null), false).close(); return DataFileMeta.forAppend( fileName, toCompact.stream().mapToLong(DataFileMeta::fileSize).sum(), @@ -680,6 +681,7 @@ private DataFileMeta generateCompactAfter(List toCompact) throws I Collections.emptyList(), null, FileSource.APPEND, + null, 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 c29519ce8b9b..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 @@ -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(), + 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/iceberg/IcebergCompatibilityTest.java b/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergCompatibilityTest.java index 7258a1dd4170..b1b98fb985f5 100644 --- a/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergCompatibilityTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/iceberg/IcebergCompatibilityTest.java @@ -208,7 +208,7 @@ public void testRetryCreateMetadata() throws Exception { assertThat(table.latestSnapshotId()).hasValue(3L); IcebergPathFactory pathFactory = - new IcebergPathFactory(new Path(table.location(), "metadata")); + new IcebergPathFactory(new Path(table.tableDataPath(), "metadata")); Path metadata3Path = pathFactory.toMetadataPath(3); assertThat(table.fileIO().exists(metadata3Path)).isTrue(); @@ -241,7 +241,7 @@ public void testSchemaChange() throws Exception { commit.commit(1, write.prepareCommit(false, 1)); assertThat(getIcebergResult()).containsExactlyInAnyOrder("Record(1, 10)", "Record(2, 20)"); - SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.location()); + SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.tableDataPath()); schemaManager.commitChanges(SchemaChange.addColumn("v2", DataTypes.STRING())); table = table.copyWithLatestSchema(); write.close(); @@ -288,7 +288,7 @@ public void testIcebergSnapshotExpire() throws Exception { FileIO fileIO = table.fileIO(); IcebergMetadata metadata = IcebergMetadata.fromPath( - fileIO, new Path(table.location(), "metadata/v1.metadata.json")); + fileIO, new Path(table.tableDataPath(), "metadata/v1.metadata.json")); assertThat(metadata.snapshots()).hasSize(1); assertThat(metadata.currentSnapshotId()).isEqualTo(1); @@ -299,7 +299,7 @@ public void testIcebergSnapshotExpire() throws Exception { assertThat(table.snapshotManager().latestSnapshotId()).isEqualTo(3L); metadata = IcebergMetadata.fromPath( - fileIO, new Path(table.location(), "metadata/v3.metadata.json")); + fileIO, new Path(table.tableDataPath(), "metadata/v3.metadata.json")); assertThat(metadata.snapshots()).hasSize(3); assertThat(metadata.currentSnapshotId()).isEqualTo(3); @@ -307,7 +307,7 @@ public void testIcebergSnapshotExpire() throws Exception { // are kept. So the first 2 Iceberg snapshots will be expired. IcebergPathFactory pathFactory = - new IcebergPathFactory(new Path(table.location(), "metadata")); + new IcebergPathFactory(new Path(table.tableDataPath(), "metadata")); IcebergManifestList manifestList = IcebergManifestList.create(table, pathFactory); assertThat(manifestList.compression()).isEqualTo("snappy"); @@ -360,7 +360,7 @@ public void testIcebergSnapshotExpire() throws Exception { assertThat(table.snapshotManager().latestSnapshotId()).isEqualTo(5L); metadata = IcebergMetadata.fromPath( - fileIO, new Path(table.location(), "metadata/v5.metadata.json")); + fileIO, new Path(table.tableDataPath(), "metadata/v5.metadata.json")); assertThat(metadata.snapshots()).hasSize(3); assertThat(metadata.currentSnapshotId()).isEqualTo(5); 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 d36966c55a0e..8259799a9a85 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(), + null); String uuid = pathFactory.uuid(); for (int i = 0; i < 20; i++) { @@ -55,7 +56,7 @@ public void testNoPartition() { + "." + CoreOptions.FILE_FORMAT.defaultValue())); } - assertThat(pathFactory.toPath("my-data-file-name")) + assertThat(pathFactory.toPath("my-data-file-name", null)) .isEqualTo(new Path(tempDir.toString() + "/bucket-123/my-data-file-name")); } @@ -68,7 +69,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(), + null); String uuid = pathFactory.uuid(); for (int i = 0; i < 20; i++) { @@ -83,7 +85,7 @@ public void testWithPartition() { + "." + CoreOptions.FILE_FORMAT.defaultValue())); } - assertThat(pathFactory.toPath("my-data-file-name")) + assertThat(pathFactory.toPath("my-data-file-name", null)) .isEqualTo( new Path(tempDir.toString() + "/dt=20211224/bucket-123/my-data-file-name")); } 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 e43cd898dbc2..8d16a753a151 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 @@ -78,7 +78,7 @@ public class KeyValueFileReadWriteTest { public void testReadNonExistentFile() { KeyValueFileReaderFactory readerFactory = createReaderFactory(tempDir.toString(), "avro", null, null); - assertThatThrownBy(() -> readerFactory.createRecordReader(0, "dummy_file.avro", 1, 0)) + assertThatThrownBy(() -> readerFactory.createRecordReader(0, "dummy_file.avro", 1, 0, null)) .hasMessageContaining( "you can configure 'snapshot.time-retained' option with a larger value."); } @@ -233,6 +233,7 @@ 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); int suggestedFileSize = ThreadLocalRandom.current().nextInt(8192) + 1024; FileIO fileIO = FileIOFinder.find(path); @@ -253,6 +254,7 @@ 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)); return KeyValueFileWriterFactory.builder( @@ -312,7 +314,8 @@ private void assertData( meta.schemaId(), meta.fileName(), meta.fileSize(), - meta.level())); + meta.level(), + meta.externalPath())); while (actualKvsIterator.hasNext()) { assertThat(expectedIterator.hasNext()).isTrue(); KeyValue actualKv = actualKvsIterator.next(); 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..ee358a087bf9 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(), + 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 19bd6a856bf9..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 @@ -151,6 +151,7 @@ protected ManifestFile createManifestFile(String pathStr) { CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), + null, null), Long.MAX_VALUE, null) 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..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 @@ -106,6 +106,7 @@ private ManifestFile createManifestFile(String pathStr) { CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), + null, null); int suggestedFileSize = ThreadLocalRandom.current().nextInt(8192) + 1024; FileIO fileIO = FileIOFinder.find(path); 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..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 @@ -110,6 +110,7 @@ private ManifestList createManifestList(String pathStr) { CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), + null, null); return new ManifestList.Factory(FileIOFinder.find(path), avro, "zstd", pathFactory, null) .create(); diff --git a/paimon-core/src/test/java/org/apache/paimon/mergetree/ContainsLevelsTest.java b/paimon-core/src/test/java/org/apache/paimon/mergetree/ContainsLevelsTest.java index be49311427a0..fa96765a4278 100644 --- a/paimon-core/src/test/java/org/apache/paimon/mergetree/ContainsLevelsTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/mergetree/ContainsLevelsTest.java @@ -198,7 +198,11 @@ private LookupLevels createContainsLevels(Levels levels, MemorySize max file -> createReaderFactory() .createRecordReader( - 0, file.fileName(), file.fileSize(), file.level()), + 0, + file.fileName(), + file.fileSize(), + file.level(), + file.externalPath()), file -> new File(tempDir.toFile(), LOOKUP_FILE_PREFIX + UUID.randomUUID()), new HashLookupStoreFactory( new CacheManager(MemorySize.ofMebiBytes(1)), diff --git a/paimon-core/src/test/java/org/apache/paimon/mergetree/LookupLevelsTest.java b/paimon-core/src/test/java/org/apache/paimon/mergetree/LookupLevelsTest.java index a678534042eb..56c45cfdc442 100644 --- a/paimon-core/src/test/java/org/apache/paimon/mergetree/LookupLevelsTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/mergetree/LookupLevelsTest.java @@ -275,7 +275,11 @@ private LookupLevels createLookupLevels(Levels levels, MemorySize maxD file -> createReaderFactory() .createRecordReader( - 0, file.fileName(), file.fileSize(), file.level()), + 0, + file.fileName(), + file.fileSize(), + file.level(), + file.externalPath()), file -> new File(tempDir.toFile(), LOOKUP_FILE_PREFIX + UUID.randomUUID()), new HashLookupStoreFactory( new CacheManager(MemorySize.ofMebiBytes(1)), diff --git a/paimon-core/src/test/java/org/apache/paimon/mergetree/MergeTreeTestBase.java b/paimon-core/src/test/java/org/apache/paimon/mergetree/MergeTreeTestBase.java index f2a9c44dd7ce..8b822c295219 100644 --- a/paimon-core/src/test/java/org/apache/paimon/mergetree/MergeTreeTestBase.java +++ b/paimon-core/src/test/java/org/apache/paimon/mergetree/MergeTreeTestBase.java @@ -114,7 +114,7 @@ public void beforeEach() throws IOException { pathFactory = createNonPartFactory(path); comparator = Comparator.comparingInt(o -> o.getInt(0)); recreateMergeTree(1024 * 1024); - Path bucketDir = writerFactory.pathFactory(0).toPath("ignore").getParent(); + Path bucketDir = writerFactory.pathFactory(0).toPath("ignore", null).getParent(); LocalFileIO.create().mkdirs(bucketDir); } @@ -418,7 +418,7 @@ private void doTestWriteRead(int batchNumber, int perBatch) throws Exception { writer.close(); - Path bucketDir = writerFactory.pathFactory(0).toPath("ignore").getParent(); + Path bucketDir = writerFactory.pathFactory(0).toPath("ignore", null).getParent(); Set files = Arrays.stream(LocalFileIO.create().listStatus(bucketDir)) .map(FileStatus::getPath) @@ -475,7 +475,7 @@ private void doTestWriteReadWithChangelog( writer.close(); - Path bucketDir = writerFactory.pathFactory(0).toPath("ignore").getParent(); + Path bucketDir = writerFactory.pathFactory(0).toPath("ignore", null).getParent(); Set files = Arrays.stream(LocalFileIO.create().listStatus(bucketDir)) .map(FileStatus::getPath) @@ -592,7 +592,7 @@ private void mergeCompacted( assertThat(remove).isTrue(); // See MergeTreeWriter.updateCompactResult if (!newFileNames.contains(file.fileName()) && !afterFiles.contains(file.fileName())) { - compactWriterFactory.deleteFile(file.fileName(), file.level()); + compactWriterFactory.deleteFile(file.fileName(), file.level(), file.externalPath()); } } compactedFiles.addAll(increment.compactIncrement().compactAfter()); diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/ExpireSnapshotsTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/ExpireSnapshotsTest.java index abff820b2cb4..b6a07460f5eb 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/ExpireSnapshotsTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/ExpireSnapshotsTest.java @@ -186,7 +186,7 @@ public void testExpireExtraFiles() throws IOException { // write test files BinaryRow partition = gen.getPartition(gen.next()); - Path bucketPath = store.pathFactory().bucketPath(partition, 0); + Path bucketPath = store.pathFactory().dataBucketPath(partition, 0); Path myDataFile = new Path(bucketPath, "myDataFile"); new LocalFileIO().tryToWriteAtomic(myDataFile, "1"); Path extra1 = new Path(bucketPath, "extra1"); @@ -429,7 +429,7 @@ public void testExpireWithUpgradedFile() throws Exception { ManifestEntry entry = entries.get(0); assertThat(entry.file().level()).isEqualTo(0); Path dataFilePath1 = - new Path(store.pathFactory().bucketPath(partition, 0), entry.file().fileName()); + new Path(store.pathFactory().dataBucketPath(partition, 0), entry.file().fileName()); FileStoreTestUtils.assertPathExists(fileIO, dataFilePath1); // compact & commit @@ -444,7 +444,7 @@ public void testExpireWithUpgradedFile() throws Exception { // data file has been upgraded due to compact assertThat(entry.file().level()).isEqualTo(5); Path dataFilePath2 = - new Path(store.pathFactory().bucketPath(partition, 0), entry.file().fileName()); + new Path(store.pathFactory().dataBucketPath(partition, 0), entry.file().fileName()); assertThat(dataFilePath1).isEqualTo(dataFilePath2); FileStoreTestUtils.assertPathExists(fileIO, dataFilePath2); diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/FileDeletionTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/FileDeletionTest.java index 3a5ee93daa37..a160c7275007 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/FileDeletionTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/FileDeletionTest.java @@ -138,7 +138,7 @@ public void testMultiPartitions(boolean cleanEmptyDirs) throws Exception { // check all paths exist for (BinaryRow partition : partitions) { for (int bucket : Arrays.asList(0, 1)) { - assertPathExists(fileIO, pathFactory.bucketPath(partition, bucket)); + assertPathExists(fileIO, pathFactory.dataBucketPath(partition, bucket)); } } @@ -210,14 +210,14 @@ public void testNoPartitions() throws Exception { cleanBucket(store, partition, 0); // check before expiring - assertPathExists(fileIO, pathFactory.bucketPath(partition, 0)); - assertPathExists(fileIO, pathFactory.bucketPath(partition, 1)); + assertPathExists(fileIO, pathFactory.dataBucketPath(partition, 0)); + assertPathExists(fileIO, pathFactory.dataBucketPath(partition, 1)); // check after expiring store.newExpire(1, 1, Long.MAX_VALUE).expire(); - assertPathNotExists(fileIO, pathFactory.bucketPath(partition, 0)); - assertPathExists(fileIO, pathFactory.bucketPath(partition, 1)); + assertPathNotExists(fileIO, pathFactory.dataBucketPath(partition, 0)); + assertPathExists(fileIO, pathFactory.dataBucketPath(partition, 1)); } /** @@ -253,7 +253,7 @@ public void testNoPartitions() throws Exception { @Test public void testExpireWithExistingTags() throws Exception { TestFileStore store = createStore(TestKeyValueGenerator.GeneratorMode.NON_PARTITIONED, 4); - tagManager = new TagManager(fileIO, store.options().path()); + tagManager = new TagManager(fileIO, store.options().schemaPath()); SnapshotManager snapshotManager = store.snapshotManager(); TestKeyValueGenerator gen = new TestKeyValueGenerator(TestKeyValueGenerator.GeneratorMode.NON_PARTITIONED); @@ -295,20 +295,20 @@ public void testExpireWithExistingTags() throws Exception { // check before expiring FileStorePathFactory pathFactory = store.pathFactory(); for (int i = 0; i < 4; i++) { - assertPathExists(fileIO, pathFactory.bucketPath(partition, i)); + assertPathExists(fileIO, pathFactory.dataBucketPath(partition, i)); } // check expiring results store.newExpire(1, 1, Long.MAX_VALUE).expire(); // expiring snapshot 1 will delete file A - assertPathNotExists(fileIO, pathFactory.bucketPath(partition, 0)); + assertPathNotExists(fileIO, pathFactory.dataBucketPath(partition, 0)); // expiring snapshot 2 & 3 won't delete file B - assertPathExists(fileIO, pathFactory.bucketPath(partition, 1)); + assertPathExists(fileIO, pathFactory.dataBucketPath(partition, 1)); // expiring snapshot 4 & 5 will delete file D - assertPathNotExists(fileIO, pathFactory.bucketPath(partition, 3)); + assertPathNotExists(fileIO, pathFactory.dataBucketPath(partition, 3)); // file C survives - assertPathExists(fileIO, pathFactory.bucketPath(partition, 2)); + assertPathExists(fileIO, pathFactory.dataBucketPath(partition, 2)); // check manifests ManifestList manifestList = store.manifestListFactory().create(); @@ -331,7 +331,7 @@ public void testExpireWithExistingTags() throws Exception { @Test public void testExpireWithUpgradeAndTags() throws Exception { TestFileStore store = createStore(TestKeyValueGenerator.GeneratorMode.NON_PARTITIONED); - tagManager = new TagManager(fileIO, store.options().path()); + tagManager = new TagManager(fileIO, store.options().schemaPath()); SnapshotManager snapshotManager = store.snapshotManager(); TestKeyValueGenerator gen = new TestKeyValueGenerator(TestKeyValueGenerator.GeneratorMode.NON_PARTITIONED); @@ -365,7 +365,7 @@ public void testExpireWithUpgradeAndTags() throws Exception { // check data file and manifests FileStorePathFactory pathFactory = store.pathFactory(); - assertPathExists(fileIO, pathFactory.bucketPath(partition, 0)); + assertPathExists(fileIO, pathFactory.dataBucketPath(partition, 0)); Snapshot tag1 = tagManager.taggedSnapshot("tag1"); ManifestList manifestList = store.manifestListFactory().create(); @@ -385,7 +385,7 @@ public void testExpireWithUpgradeAndTags() throws Exception { @Test public void testDeleteTagWithSnapshot() throws Exception { TestFileStore store = createStore(TestKeyValueGenerator.GeneratorMode.NON_PARTITIONED, 3); - tagManager = new TagManager(fileIO, store.options().path()); + tagManager = new TagManager(fileIO, store.options().schemaPath()); SnapshotManager snapshotManager = store.snapshotManager(); TestKeyValueGenerator gen = new TestKeyValueGenerator(TestKeyValueGenerator.GeneratorMode.NON_PARTITIONED); @@ -425,7 +425,7 @@ public void testDeleteTagWithSnapshot() throws Exception { // check before deleting tag1 FileStorePathFactory pathFactory = store.pathFactory(); for (int i = 0; i < 3; i++) { - assertPathExists(fileIO, pathFactory.bucketPath(partition, i)); + assertPathExists(fileIO, pathFactory.dataBucketPath(partition, i)); } for (ManifestFileMeta manifestFileMeta : snapshot1Data) { assertPathExists(fileIO, pathFactory.toManifestFilePath(manifestFileMeta.fileName())); @@ -438,9 +438,9 @@ public void testDeleteTagWithSnapshot() throws Exception { "tag1", store.newTagDeletion(), snapshotManager, Collections.emptyList()); // check data files - assertPathNotExists(fileIO, pathFactory.bucketPath(partition, 0)); - assertPathExists(fileIO, pathFactory.bucketPath(partition, 1)); - assertPathExists(fileIO, pathFactory.bucketPath(partition, 2)); + assertPathNotExists(fileIO, pathFactory.dataBucketPath(partition, 0)); + assertPathExists(fileIO, pathFactory.dataBucketPath(partition, 1)); + assertPathExists(fileIO, pathFactory.dataBucketPath(partition, 2)); // check manifests for (ManifestFileMeta manifestFileMeta : snapshot1Data) { @@ -459,7 +459,7 @@ public void testDeleteTagWithSnapshot() throws Exception { @Test public void testDeleteTagWithOtherTag() throws Exception { TestFileStore store = createStore(TestKeyValueGenerator.GeneratorMode.NON_PARTITIONED, 3); - tagManager = new TagManager(fileIO, store.options().path()); + tagManager = new TagManager(fileIO, store.options().schemaPath()); SnapshotManager snapshotManager = store.snapshotManager(); TestKeyValueGenerator gen = new TestKeyValueGenerator(TestKeyValueGenerator.GeneratorMode.NON_PARTITIONED); @@ -501,8 +501,8 @@ public void testDeleteTagWithOtherTag() throws Exception { // check before deleting tag2 FileStorePathFactory pathFactory = store.pathFactory(); - assertPathExists(fileIO, pathFactory.bucketPath(partition, 0)); - assertPathExists(fileIO, pathFactory.bucketPath(partition, 1)); + assertPathExists(fileIO, pathFactory.dataBucketPath(partition, 0)); + assertPathExists(fileIO, pathFactory.dataBucketPath(partition, 1)); for (ManifestFileMeta manifestFileMeta : snapshot2Data) { assertPathExists(fileIO, pathFactory.toManifestFilePath(manifestFileMeta.fileName())); @@ -515,8 +515,8 @@ public void testDeleteTagWithOtherTag() throws Exception { "tag2", store.newTagDeletion(), snapshotManager, Collections.emptyList()); // check data files - assertPathExists(fileIO, pathFactory.bucketPath(partition, 0)); - assertPathNotExists(fileIO, pathFactory.bucketPath(partition, 1)); + assertPathExists(fileIO, pathFactory.dataBucketPath(partition, 0)); + assertPathNotExists(fileIO, pathFactory.dataBucketPath(partition, 1)); // check manifests Snapshot tag1 = tagManager.taggedSnapshot("tag1"); @@ -574,7 +574,7 @@ public void testExpireWithMissingManifest() throws Exception { // check that there are one data file and get its path FileStorePathFactory pathFactory = store.pathFactory(); - Path bucket0 = pathFactory.bucketPath(partition, 0); + Path bucket0 = pathFactory.dataBucketPath(partition, 0); List datafiles = Files.walk(Paths.get(bucket0.toString())) .filter(Files::isRegularFile) @@ -645,7 +645,7 @@ public void testExpireWithMissingManifest() throws Exception { @Test public void testExpireWithDeletingTags() throws Exception { TestFileStore store = createStore(TestKeyValueGenerator.GeneratorMode.NON_PARTITIONED, 2); - tagManager = new TagManager(fileIO, store.options().path()); + tagManager = new TagManager(fileIO, store.options().schemaPath()); SnapshotManager snapshotManager = store.snapshotManager(); TestKeyValueGenerator gen = new TestKeyValueGenerator(TestKeyValueGenerator.GeneratorMode.NON_PARTITIONED); @@ -696,8 +696,8 @@ public void testExpireWithDeletingTags() throws Exception { assertThat(snapshotManager.snapshotCount()).isEqualTo(2); FileStorePathFactory pathFactory = store.pathFactory(); - assertPathExists(fileIO, pathFactory.bucketPath(partition, 0)); - assertPathExists(fileIO, pathFactory.bucketPath(partition, 1)); + assertPathExists(fileIO, pathFactory.dataBucketPath(partition, 0)); + assertPathExists(fileIO, pathFactory.dataBucketPath(partition, 1)); } private TestFileStore createStore(TestKeyValueGenerator.GeneratorMode mode) throws Exception { diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/LocalOrphanFilesCleanTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/LocalOrphanFilesCleanTest.java index 5139dd44957d..d1af8e8c5773 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/LocalOrphanFilesCleanTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/LocalOrphanFilesCleanTest.java @@ -684,7 +684,7 @@ private static String randomValue() { } private FileStoreTable createFileStoreTable(RowType rowType, Options conf) throws Exception { - conf.set(CoreOptions.PATH, tablePath.toString()); + conf.set(CoreOptions.TABLE_SCHEMA_PATH, tablePath.toString()); conf.set(CoreOptions.BUCKET, RANDOM.nextInt(3) + 1); TableSchema tableSchema = SchemaUtils.forceCommit( diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/PartitionExpireTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/PartitionExpireTest.java index 893fe1bf5762..f307c48502be 100644 --- a/paimon-core/src/test/java/org/apache/paimon/operation/PartitionExpireTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/operation/PartitionExpireTest.java @@ -70,7 +70,7 @@ import static org.apache.paimon.CoreOptions.PARTITION_EXPIRATION_CHECK_INTERVAL; import static org.apache.paimon.CoreOptions.PARTITION_EXPIRATION_TIME; import static org.apache.paimon.CoreOptions.PARTITION_TIMESTAMP_FORMATTER; -import static org.apache.paimon.CoreOptions.PATH; +import static org.apache.paimon.CoreOptions.TABLE_SCHEMA_PATH; import static org.apache.paimon.CoreOptions.WRITE_ONLY; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -92,8 +92,8 @@ public void beforeEach() { private void newTable() { LocalFileIO fileIO = LocalFileIO.create(); Options options = new Options(); - options.set(PATH, path.toString()); - Path tablePath = CoreOptions.path(options); + options.set(TABLE_SCHEMA_PATH, path.toString()); + Path tablePath = CoreOptions.schemaPath(options); String branchName = CoreOptions.branch(options.toMap()); TableSchema tableSchema = new SchemaManager(fileIO, tablePath, branchName).latest().get(); deletedPartitions = new ArrayList<>(); diff --git a/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyFileStoreTableTest.java b/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyFileStoreTableTest.java index 01d4e89af95d..ee45bda0c87d 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyFileStoreTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/AppendOnlyFileStoreTableTest.java @@ -112,7 +112,7 @@ public void testReadDeletedFiles() throws Exception { table.store() .pathFactory() .createDataFilePathFactory(split.partition(), split.bucket()) - .toPath(split.dataFiles().get(0).fileName()); + .toPath(split.dataFiles().get(0).fileName(), null); table.fileIO().deleteQuietly(path); // read @@ -1086,7 +1086,7 @@ private void writeBranchData(FileStoreTable table) throws Exception { protected FileStoreTable createFileStoreTable(Consumer configure, RowType rowType) throws Exception { Options conf = new Options(); - conf.set(CoreOptions.PATH, tablePath.toString()); + conf.set(CoreOptions.TABLE_SCHEMA_PATH, tablePath.toString()); configure.accept(conf); if (!conf.contains(BUCKET_KEY) && conf.get(BUCKET) != -1) { conf.set(BUCKET_KEY, "a"); @@ -1107,7 +1107,7 @@ protected FileStoreTable createFileStoreTable(Consumer configure, RowTy protected FileStoreTable createFileStoreTable(String branch, Consumer configure) throws Exception { Options conf = new Options(); - conf.set(CoreOptions.PATH, tablePath.toString()); + conf.set(CoreOptions.TABLE_SCHEMA_PATH, tablePath.toString()); conf.set(CoreOptions.BRANCH, branch); configure.accept(conf); if (!conf.contains(BUCKET_KEY) && conf.get(BUCKET) != -1) { @@ -1128,7 +1128,7 @@ protected FileStoreTable createFileStoreTable(String branch, Consumer c @Override protected FileStoreTable overwriteTestFileStoreTable() throws Exception { Options conf = new Options(); - conf.set(CoreOptions.PATH, tablePath.toString()); + conf.set(CoreOptions.TABLE_SCHEMA_PATH, tablePath.toString()); TableSchema tableSchema = SchemaUtils.forceCommit( new SchemaManager(LocalFileIO.create(), tablePath), @@ -1144,7 +1144,7 @@ protected FileStoreTable overwriteTestFileStoreTable() throws Exception { protected FileStoreTable createUnawareBucketFileStoreTable(Consumer configure) throws Exception { Options conf = new Options(); - conf.set(CoreOptions.PATH, tablePath.toString()); + conf.set(CoreOptions.TABLE_SCHEMA_PATH, tablePath.toString()); conf.set(CoreOptions.BUCKET, -1); configure.accept(conf); TableSchema tableSchema = @@ -1162,7 +1162,7 @@ protected FileStoreTable createUnawareBucketFileStoreTable(Consumer con protected FileStoreTable createUnawareBucketFileStoreTable( RowType rowType, Consumer configure) throws Exception { Options conf = new Options(); - conf.set(CoreOptions.PATH, tablePath.toString()); + conf.set(CoreOptions.TABLE_SCHEMA_PATH, tablePath.toString()); conf.set(CoreOptions.BUCKET, -1); configure.accept(conf); TableSchema tableSchema = diff --git a/paimon-core/src/test/java/org/apache/paimon/table/FileStoreTableTestBase.java b/paimon-core/src/test/java/org/apache/paimon/table/FileStoreTableTestBase.java index 75e284a68c3a..7ba2a7977280 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/FileStoreTableTestBase.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/FileStoreTableTestBase.java @@ -665,7 +665,7 @@ public void testWriteWithoutCompactionAndExpiration() throws Exception { } SnapshotManager snapshotManager = - new SnapshotManager(FileIOFinder.find(tablePath), table.location()); + new SnapshotManager(FileIOFinder.find(tablePath), table.tableDataPath()); Long latestSnapshotId = snapshotManager.latestSnapshotId(); assertThat(latestSnapshotId).isNotNull(); for (int i = 1; i <= latestSnapshotId; i++) { @@ -685,7 +685,7 @@ public void testCopyWithLatestSchema() throws Exception { write.write(rowData(1, 20, 200L)); commit.commit(0, write.prepareCommit(true, 0)); - SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.location()); + SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.tableDataPath()); schemaManager.commitChanges(SchemaChange.addColumn("added", DataTypes.INT())); table = table.copyWithLatestSchema(); assertThat(table.coreOptions().snapshotNumRetainMax()).isEqualTo(100); @@ -1582,13 +1582,14 @@ public void testExpireWithLimit() throws Exception { @Test public void testSchemaPathOption() throws Exception { String fakePath = "fake path"; - FileStoreTable table = createFileStoreTable(conf -> conf.set(CoreOptions.PATH, fakePath)); - String originSchemaPath = table.schema().options().get(CoreOptions.PATH.key()); + FileStoreTable table = + createFileStoreTable(conf -> conf.set(CoreOptions.TABLE_SCHEMA_PATH, fakePath)); + String originSchemaPath = table.schema().options().get(CoreOptions.TABLE_SCHEMA_PATH.key()); assertThat(originSchemaPath).isEqualTo(fakePath); // reset PATH of schema option to table location table = table.copy(Collections.emptyMap()); - String schemaPath = table.schema().options().get(CoreOptions.PATH.key()); - String tablePath = table.location().toString(); + String schemaPath = table.schema().options().get(CoreOptions.TABLE_SCHEMA_PATH.key()); + String tablePath = table.tableDataPath().toString(); assertThat(schemaPath).isEqualTo(tablePath); } diff --git a/paimon-core/src/test/java/org/apache/paimon/table/IndexFileExpireTableTest.java b/paimon-core/src/test/java/org/apache/paimon/table/IndexFileExpireTableTest.java index 4ad634a43351..00b4fc877690 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/IndexFileExpireTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/IndexFileExpireTableTest.java @@ -106,7 +106,7 @@ public void testIndexFileExpirationWithTag() throws Exception { assertThat(indexFileSize()).isEqualTo(5); assertThat(indexManifestSize()).isEqualTo(3); - TagManager tagManager = new TagManager(LocalFileIO.create(), table.location()); + TagManager tagManager = new TagManager(LocalFileIO.create(), table.tableDataPath()); checkIndexFiles(tagManager.taggedSnapshot("tag3")); checkIndexFiles(tagManager.taggedSnapshot("tag5")); } @@ -133,7 +133,7 @@ public void testIndexFileExpirationWhenDeletingTag() throws Exception { expire.expireUntil(1, 7); table.deleteTag("tag3"); - TagManager tagManager = new TagManager(LocalFileIO.create(), table.location()); + TagManager tagManager = new TagManager(LocalFileIO.create(), table.tableDataPath()); checkIndexFiles(7); checkIndexFiles(tagManager.taggedSnapshot("tag5")); assertThat(indexFileSize()).isEqualTo(4); @@ -241,12 +241,13 @@ private void checkIndexFiles(Snapshot snapshot) { } private long indexFileSize() throws IOException { - return LocalFileIO.create().listStatus(new Path(table.location(), "index")).length; + return LocalFileIO.create().listStatus(new Path(table.tableDataPath(), "index")).length; } private long indexManifestSize() throws IOException { return Arrays.stream( - LocalFileIO.create().listStatus(new Path(table.location(), "manifest"))) + LocalFileIO.create() + .listStatus(new Path(table.tableDataPath(), "manifest"))) .filter(s -> s.getPath().getName().startsWith("index-")) .count(); } diff --git a/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileStoreTableTest.java b/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileStoreTableTest.java index fa635e2ab666..130ee11ebcfc 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileStoreTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/PrimaryKeyFileStoreTableTest.java @@ -2187,7 +2187,7 @@ private void refreshTableService(LocalTableQuery query, List comm @Override protected FileStoreTable overwriteTestFileStoreTable() throws Exception { Options conf = new Options(); - conf.set(CoreOptions.PATH, tablePath.toString()); + conf.set(CoreOptions.TABLE_SCHEMA_PATH, tablePath.toString()); conf.set(BUCKET, 1); TableSchema tableSchema = SchemaUtils.forceCommit( @@ -2205,7 +2205,7 @@ protected FileStoreTable overwriteTestFileStoreTable() throws Exception { protected FileStoreTable createFileStoreTable(Consumer configure, RowType rowType) throws Exception { Options options = new Options(); - options.set(CoreOptions.PATH, tablePath.toString()); + options.set(CoreOptions.TABLE_SCHEMA_PATH, tablePath.toString()); options.set(BUCKET, 1); configure.accept(options); TableSchema tableSchema = @@ -2229,7 +2229,7 @@ protected FileStoreTable createFileStoreTable(String branch, Consumer c private FileStoreTable createFileStoreTable( String branch, Consumer configure, RowType rowType) throws Exception { Options options = new Options(); - options.set(CoreOptions.PATH, tablePath.toString()); + options.set(CoreOptions.TABLE_SCHEMA_PATH, tablePath.toString()); options.set(BUCKET, 1); options.set(BRANCH, branch); configure.accept(options); diff --git a/paimon-core/src/test/java/org/apache/paimon/table/SchemaEvolutionTableTestBase.java b/paimon-core/src/test/java/org/apache/paimon/table/SchemaEvolutionTableTestBase.java index 27789241037b..6cf921e33850 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/SchemaEvolutionTableTestBase.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/SchemaEvolutionTableTestBase.java @@ -130,7 +130,7 @@ public void before() throws Exception { tablePath = new Path(TraceableFileIO.SCHEME + "://" + tempDir.toString()); fileIO = FileIOFinder.find(tablePath); commitUser = UUID.randomUUID().toString(); - tableConfig.set(CoreOptions.PATH, tablePath.toString()); + tableConfig.set(CoreOptions.TABLE_SCHEMA_PATH, tablePath.toString()); tableConfig.set(CoreOptions.BUCKET, 1); } diff --git a/paimon-core/src/test/java/org/apache/paimon/table/WritePreemptMemoryTest.java b/paimon-core/src/test/java/org/apache/paimon/table/WritePreemptMemoryTest.java index 13f692ac7e4e..d353721c4cdd 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/WritePreemptMemoryTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/WritePreemptMemoryTest.java @@ -93,7 +93,7 @@ protected FileStoreTable createFileStoreTable(Consumer configure, RowTy throws Exception { Options options = new Options(); options.set(CoreOptions.BUCKET, 1); - options.set(CoreOptions.PATH, tablePath.toString()); + options.set(CoreOptions.TABLE_SCHEMA_PATH, tablePath.toString()); // Run with minimal memory to ensure a more intense preempt // Currently a writer needs at least one page int pages = 10; @@ -117,7 +117,7 @@ protected FileStoreTable createFileStoreTable(String branch, Consumer c throws Exception { Options options = new Options(); options.set(CoreOptions.BUCKET, 1); - options.set(CoreOptions.PATH, tablePath.toString()); + options.set(CoreOptions.TABLE_SCHEMA_PATH, tablePath.toString()); // Run with minimal memory to ensure a more intense preempt // Currently a writer needs at least one page int pages = 10; @@ -140,7 +140,7 @@ protected FileStoreTable createFileStoreTable(String branch, Consumer c @Override protected FileStoreTable overwriteTestFileStoreTable() throws Exception { Options conf = new Options(); - conf.set(CoreOptions.PATH, tablePath.toString()); + conf.set(CoreOptions.TABLE_SCHEMA_PATH, tablePath.toString()); conf.set(CoreOptions.BUCKET, 1); // Run with minimal memory to ensure a more intense preempt // Currently a writer needs at least one page diff --git a/paimon-core/src/test/java/org/apache/paimon/table/sink/TableCommitTest.java b/paimon-core/src/test/java/org/apache/paimon/table/sink/TableCommitTest.java index 227a3b58eeed..c9694d5e38ba 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/sink/TableCommitTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/sink/TableCommitTest.java @@ -103,7 +103,7 @@ private void testCommitCallbackWithFailureImpl(int bucket, int numIdentifiers, S new String[] {"k", "v"}); Options conf = new Options(); - conf.set(CoreOptions.PATH, path); + conf.set(CoreOptions.TABLE_SCHEMA_PATH, path); conf.set(CoreOptions.BUCKET, bucket); conf.set(CoreOptions.COMMIT_CALLBACKS, TestCommitCallback.class.getName()); conf.set( @@ -201,7 +201,7 @@ public void testRecoverDeletedFiles() throws Exception { new String[] {"k", "v"}); Options options = new Options(); - options.set(CoreOptions.PATH, path); + options.set(CoreOptions.TABLE_SCHEMA_PATH, path); options.set(CoreOptions.BUCKET, 1); TableSchema tableSchema = SchemaUtils.forceCommit( diff --git a/paimon-core/src/test/java/org/apache/paimon/table/source/SplitTest.java b/paimon-core/src/test/java/org/apache/paimon/table/source/SplitTest.java index a4e581b701ea..a088f40dab21 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/source/SplitTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/source/SplitTest.java @@ -447,6 +447,7 @@ private DataFileMeta newDataFile(long rowCount) { Collections.emptyList(), null, null, + null, null); } diff --git a/paimon-core/src/test/java/org/apache/paimon/table/source/StartupModeTest.java b/paimon-core/src/test/java/org/apache/paimon/table/source/StartupModeTest.java index 7197b9b0fb41..c883e964864d 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/source/StartupModeTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/source/StartupModeTest.java @@ -42,8 +42,8 @@ import java.util.Map; import java.util.UUID; -import static org.apache.paimon.CoreOptions.PATH; import static org.apache.paimon.CoreOptions.StartupMode; +import static org.apache.paimon.CoreOptions.TABLE_SCHEMA_PATH; 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; @@ -366,7 +366,7 @@ private void initializeTable(CoreOptions.StartupMode startupMode) throws Excepti private void initializeTable( CoreOptions.StartupMode startupMode, Map properties) throws Exception { Options options = new Options(); - options.set(PATH, tablePath.toString()); + options.set(TABLE_SCHEMA_PATH, tablePath.toString()); options.set(CoreOptions.SCAN_MODE, startupMode); for (Map.Entry property : properties.entrySet()) { options.set(property.getKey(), property.getValue()); diff --git a/paimon-core/src/test/java/org/apache/paimon/table/system/ConsumersTableTest.java b/paimon-core/src/test/java/org/apache/paimon/table/system/ConsumersTableTest.java index 6412db9df010..1536d3e38260 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/system/ConsumersTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/system/ConsumersTableTest.java @@ -62,7 +62,7 @@ public void before() throws Exception { .build(); catalog.createTable(identifier, schema, true); FileStoreTable table = (FileStoreTable) catalog.getTable(identifier); - manager = new ConsumerManager(table.fileIO(), table.location()); + manager = new ConsumerManager(table.fileIO(), table.tableDataPath()); manager.resetConsumer("id1", new Consumer(5)); manager.resetConsumer("id2", new Consumer(6)); consumersTable = (ConsumersTable) catalog.getTable(identifier(tableName + "$consumers")); diff --git a/paimon-core/src/test/java/org/apache/paimon/table/system/FilesTableTest.java b/paimon-core/src/test/java/org/apache/paimon/table/system/FilesTableTest.java index f0280560c267..e778b9399f51 100644 --- a/paimon-core/src/test/java/org/apache/paimon/table/system/FilesTableTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/table/system/FilesTableTest.java @@ -192,7 +192,7 @@ private List getExpectedResult(long snapshotId) { Arrays.toString(new String[] {partition1, partition2})), fileEntry.bucket(), BinaryString.fromString( - table.location() + table.tableDataPath() + "/pt1=" + partition1 + "/pt2=" 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 6ca15cf1503d..2c324d3ca7ca 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 @@ -72,7 +72,7 @@ public void testCreateDataFilePathFactoryNoPartition() { FileStorePathFactory pathFactory = createNonPartFactory(new Path(tempDir.toString())); DataFilePathFactory dataFilePathFactory = pathFactory.createDataFilePathFactory(new BinaryRow(0), 123); - assertThat(dataFilePathFactory.toPath("my-data-file-name")) + assertThat(dataFilePathFactory.toPath("my-data-file-name", null)) .isEqualTo(new Path(tempDir.toString() + "/bucket-123/my-data-file-name")); } @@ -91,6 +91,7 @@ public void testCreateDataFilePathFactoryWithPartition() { CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), + null, null); assertPartition("20211224", 16, pathFactory, "/dt=20211224/hr=16"); @@ -116,7 +117,7 @@ private void assertPartition( writer.complete(); DataFilePathFactory dataFilePathFactory = pathFactory.createDataFilePathFactory(partition, 123); - assertThat(dataFilePathFactory.toPath("my-data-file-name")) + assertThat(dataFilePathFactory.toPath("my-data-file-name", null)) .isEqualTo( new Path(tempDir.toString() + expected + "/bucket-123/my-data-file-name")); } @@ -132,6 +133,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); } } diff --git a/paimon-core/src/test/java/org/apache/paimon/utils/TagManagerTest.java b/paimon-core/src/test/java/org/apache/paimon/utils/TagManagerTest.java index 3e702b9b2cd0..6aab0614f737 100644 --- a/paimon-core/src/test/java/org/apache/paimon/utils/TagManagerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/utils/TagManagerTest.java @@ -73,7 +73,7 @@ public void setup() throws Exception { @Test public void testCreateTagWithoutTimeRetained() throws Exception { TestFileStore store = createStore(TestKeyValueGenerator.GeneratorMode.NON_PARTITIONED, 4); - tagManager = new TagManager(fileIO, store.options().path()); + tagManager = new TagManager(fileIO, store.options().schemaPath()); SnapshotManager snapshotManager = store.snapshotManager(); TestKeyValueGenerator gen = new TestKeyValueGenerator(TestKeyValueGenerator.GeneratorMode.NON_PARTITIONED); @@ -104,7 +104,7 @@ public void testCreateTagWithoutTimeRetained() throws Exception { @Test public void testCreateTagWithTimeRetained() throws Exception { TestFileStore store = createStore(TestKeyValueGenerator.GeneratorMode.NON_PARTITIONED, 4); - tagManager = new TagManager(fileIO, store.options().path()); + tagManager = new TagManager(fileIO, store.options().schemaPath()); SnapshotManager snapshotManager = store.snapshotManager(); TestKeyValueGenerator gen = new TestKeyValueGenerator(TestKeyValueGenerator.GeneratorMode.NON_PARTITIONED); diff --git a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/ResetConsumerProcedure.java b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/ResetConsumerProcedure.java index 7777ccda19de..66af955ad938 100644 --- a/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/ResetConsumerProcedure.java +++ b/paimon-flink/paimon-flink-1.18/src/main/java/org/apache/paimon/flink/procedure/ResetConsumerProcedure.java @@ -53,7 +53,7 @@ public String[] call( ConsumerManager consumerManager = new ConsumerManager( fileStoreTable.fileIO(), - fileStoreTable.location(), + fileStoreTable.tableSchemaPath(), fileStoreTable.snapshotManager().branch()); consumerManager.resetConsumer(consumerId, new Consumer(nextSnapshotId)); @@ -67,7 +67,7 @@ public String[] call(ProcedureContext procedureContext, String tableId, String c ConsumerManager consumerManager = new ConsumerManager( fileStoreTable.fileIO(), - fileStoreTable.location(), + fileStoreTable.tableSchemaPath(), fileStoreTable.snapshotManager().branch()); consumerManager.deleteConsumer(consumerId); diff --git a/paimon-flink/paimon-flink-1.18/src/test/java/org/apache/paimon/flink/RemoveOrphanFilesActionITCase.java b/paimon-flink/paimon-flink-1.18/src/test/java/org/apache/paimon/flink/RemoveOrphanFilesActionITCase.java index a168c3785c7c..33786394595f 100644 --- a/paimon-flink/paimon-flink-1.18/src/test/java/org/apache/paimon/flink/RemoveOrphanFilesActionITCase.java +++ b/paimon-flink/paimon-flink-1.18/src/test/java/org/apache/paimon/flink/RemoveOrphanFilesActionITCase.java @@ -93,7 +93,7 @@ private FileStoreTable createTableAndWriteData(String tableName) throws Exceptio } private Path getOrphanFilePath(FileStoreTable table, String orphanFile) { - return new Path(table.location(), orphanFile); + return new Path(table.tableDataPath(), orphanFile); } @Test @@ -201,14 +201,15 @@ public void testCleanWithBranch() throws Exception { // create first branch and write some data table.createBranch("br"); - SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.location(), "br"); + SchemaManager schemaManager = + new SchemaManager(table.fileIO(), table.tableDataPath(), "br"); TableSchema branchSchema = schemaManager.commitChanges(SchemaChange.addColumn("v2", DataTypes.INT())); Options branchOptions = new Options(branchSchema.options()); branchOptions.set(CoreOptions.BRANCH, "br"); branchSchema = branchSchema.copy(branchOptions.toMap()); FileStoreTable branchTable = - FileStoreTableFactory.create(table.fileIO(), table.location(), branchSchema); + FileStoreTableFactory.create(table.fileIO(), table.tableDataPath(), branchSchema); String commitUser = UUID.randomUUID().toString(); StreamTableWrite write = branchTable.newWrite(commitUser); @@ -219,14 +220,16 @@ public void testCleanWithBranch() throws Exception { commit.close(); // create orphan file in snapshot directory of first branch - Path orphanFile3 = new Path(table.location(), "branch/branch-br/snapshot/orphan_file3"); + Path orphanFile3 = + new Path(table.tableDataPath(), "branch/branch-br/snapshot/orphan_file3"); branchTable.fileIO().writeFile(orphanFile3, "x", true); // create second branch, which is empty table.createBranch("br2"); // create orphan file in snapshot directory of second branch - Path orphanFile4 = new Path(table.location(), "branch/branch-br2/snapshot/orphan_file4"); + Path orphanFile4 = + new Path(table.tableDataPath(), "branch/branch-br2/snapshot/orphan_file4"); branchTable.fileIO().writeFile(orphanFile4, "y", true); if (ThreadLocalRandom.current().nextBoolean()) { diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcSinkBuilder.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcSinkBuilder.java index 5c27db6ddf1b..01f99fb03479 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcSinkBuilder.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcSinkBuilder.java @@ -107,7 +107,8 @@ public DataStreamSink build() { parsed, CdcParsingProcessFunction.NEW_DATA_FIELD_LIST_OUTPUT_TAG) .process( new UpdatedDataFieldsProcessFunction( - new SchemaManager(dataTable.fileIO(), dataTable.location()), + new SchemaManager( + dataTable.fileIO(), dataTable.tableSchemaPath()), identifier, catalogLoader)) .name("Schema Evolution"); diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkBuilder.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkBuilder.java index bd18c7e7ad82..122c6d01b8a3 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkBuilder.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkBuilder.java @@ -199,7 +199,8 @@ private void buildDividedCdcSink() { .createUpdatedDataFieldsOutputTag(table.name())) .process( new UpdatedDataFieldsProcessFunction( - new SchemaManager(table.fileIO(), table.location()), + new SchemaManager( + table.fileIO(), table.tableSchemaPath()), Identifier.create(database, table.name()), catalogLoader)) .name("Schema Evolution"); diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/MultiTableUpdatedDataFieldsProcessFunction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/MultiTableUpdatedDataFieldsProcessFunction.java index dd612a52c2eb..fe3c17c05065 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/MultiTableUpdatedDataFieldsProcessFunction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/MultiTableUpdatedDataFieldsProcessFunction.java @@ -73,7 +73,7 @@ public void processElement( } catch (Catalog.TableNotExistException e) { return null; } - return new SchemaManager(table.fileIO(), table.location()); + return new SchemaManager(table.fileIO(), table.tableSchemaPath()); }); if (Objects.isNull(schemaManager)) { diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/SchemaEvolutionTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/SchemaEvolutionTest.java index 46c8e98fb639..e424c50f97d5 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/SchemaEvolutionTest.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/SchemaEvolutionTest.java @@ -208,7 +208,7 @@ public void testSchemaEvolution() throws Exception { upDataFieldStream .process( new UpdatedDataFieldsProcessFunction( - new SchemaManager(table.fileIO(), table.location()), + new SchemaManager(table.fileIO(), table.tableSchemaPath()), identifier, catalogLoader)) .name("Schema Evolution"); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java index 749d87eb0636..fbe1c5e28d21 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java @@ -679,7 +679,8 @@ private void testAllTypesImpl(Statement statement) throws Exception { waitForResult(newExpected, table, newRowType, Arrays.asList("pt", "_id")); } finally { statement.executeUpdate("ALTER TABLE all_types_table DROP COLUMN v"); - SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.location()); + SchemaManager schemaManager = + new SchemaManager(table.fileIO(), table.tableSchemaPath()); schemaManager.commitChanges(SchemaChange.dropColumn("v")); } } diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperatorTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperatorTest.java index 4436aa392d42..b2f937609a3e 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperatorTest.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperatorTest.java @@ -331,7 +331,7 @@ public void testSingleTableAddColumn() throws Exception { actual = runner.poll(1); assertThat(actual).isNull(); - SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.location()); + SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.tableDataPath()); schemaManager.commitChanges(SchemaChange.addColumn("v2", DataTypes.INT())); actual = runner.take(); assertThat(actual).isEqualTo(expected); @@ -398,7 +398,7 @@ public void testSingleTableUpdateColumnType() throws Exception { actual = runner.poll(1); assertThat(actual).isNull(); - SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.location()); + SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.tableDataPath()); schemaManager.commitChanges(SchemaChange.updateColumnType("v1", DataTypes.BIGINT())); actual = runner.take(); assertThat(actual).isEqualTo(expected); @@ -527,7 +527,7 @@ public void testMultiTableUpdateColumnType() throws Exception { actual = runner.poll(1); assertThat(actual).isNull(); - schemaManager = new SchemaManager(table1.fileIO(), table1.location()); + schemaManager = new SchemaManager(table1.fileIO(), table1.tableDataPath()); schemaManager.commitChanges(SchemaChange.updateColumnType("k", DataTypes.BIGINT())); actual = runner.take(); assertThat(actual).isEqualTo(expected); @@ -546,7 +546,7 @@ public void testMultiTableUpdateColumnType() throws Exception { actual = runner.poll(1); assertThat(actual).isNull(); - schemaManager = new SchemaManager(table2.fileIO(), table2.location()); + schemaManager = new SchemaManager(table2.fileIO(), table2.tableDataPath()); schemaManager.commitChanges(SchemaChange.updateColumnType("v1", DataTypes.BIGINT())); actual = runner.take(); assertThat(actual).isEqualTo(expected); @@ -567,7 +567,7 @@ public void testMultiTableUpdateColumnType() throws Exception { actual = runner.poll(1); assertThat(actual).isNull(); - schemaManager = new SchemaManager(table2.fileIO(), table2.location()); + schemaManager = new SchemaManager(table2.fileIO(), table2.tableDataPath()); schemaManager.commitChanges(SchemaChange.updateColumnType("v2", DataTypes.DOUBLE())); actual = runner.take(); assertThat(actual).isEqualTo(expected); @@ -587,7 +587,7 @@ public void testMultiTableUpdateColumnType() throws Exception { actual = runner.poll(1); assertThat(actual).isNull(); - schemaManager = new SchemaManager(table2.fileIO(), table2.location()); + schemaManager = new SchemaManager(table2.fileIO(), table2.tableDataPath()); schemaManager.commitChanges(SchemaChange.updateColumnType("v3", DataTypes.VARCHAR(10))); actual = runner.take(); assertThat(actual).isEqualTo(expected); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperatorTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperatorTest.java index f00229d99890..205ed7059639 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperatorTest.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperatorTest.java @@ -135,7 +135,7 @@ public void testAddColumn() throws Exception { actual = runner.poll(1); assertThat(actual).isNull(); - SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.location()); + SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.tableDataPath()); schemaManager.commitChanges(SchemaChange.addColumn("v2", DataTypes.INT())); actual = runner.take(); assertThat(actual).isEqualTo(expected); @@ -196,7 +196,7 @@ public void testUpdateColumnType() throws Exception { actual = runner.poll(1); assertThat(actual).isNull(); - SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.location()); + SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.tableDataPath()); schemaManager.commitChanges(SchemaChange.updateColumnType("v1", DataTypes.BIGINT())); actual = runner.take(); assertThat(actual).isEqualTo(expected); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkITCase.java index 35286e3a88d4..3d90b984c10a 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncDatabaseSinkITCase.java @@ -183,7 +183,7 @@ private void innerTestRandomCdcEvents(Supplier bucket, boolean unawareB FailingFileIO.reset(failingName, 0, 1); for (int i = 0; i < numTables; i++) { FileStoreTable table = fileStoreTables.get(i).copyWithLatestSchema(); - SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.location()); + SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.tableDataPath()); TableSchema schema = schemaManager.latest().get(); ReadBuilder readBuilder = table.newReadBuilder(); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncTableSinkITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncTableSinkITCase.java index 9fccaac99228..0f739e331985 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncTableSinkITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/FlinkCdcSyncTableSinkITCase.java @@ -178,7 +178,7 @@ private void innerTestRandomCdcEvents( FailingFileIO.reset(failingName, 0, 1); table = table.copyWithLatestSchema(); - SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.location()); + SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.tableDataPath()); TableSchema schema = schemaManager.latest().get(); ReadBuilder readBuilder = table.newReadBuilder(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java index ec3c4a47a69d..e5c36a48a51c 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkCatalog.java @@ -135,7 +135,7 @@ import static org.apache.paimon.CoreOptions.MATERIALIZED_TABLE_REFRESH_HANDLER_DESCRIPTION; import static org.apache.paimon.CoreOptions.MATERIALIZED_TABLE_REFRESH_MODE; import static org.apache.paimon.CoreOptions.MATERIALIZED_TABLE_REFRESH_STATUS; -import static org.apache.paimon.CoreOptions.PATH; +import static org.apache.paimon.CoreOptions.TABLE_SCHEMA_PATH; import static org.apache.paimon.catalog.Catalog.COMMENT_PROP; import static org.apache.paimon.catalog.Catalog.LAST_UPDATE_TIME_PROP; import static org.apache.paimon.catalog.Catalog.NUM_FILES_PROP; @@ -845,7 +845,7 @@ public void alterTable( continue; } - if (PATH.key().equalsIgnoreCase(key)) { + if (TABLE_SCHEMA_PATH.key().equalsIgnoreCase(key)) { throw new IllegalArgumentException("Illegal table path in table options: " + value); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkTableFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkTableFactory.java index d5c1ed043b56..39f96359e7cb 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkTableFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkTableFactory.java @@ -75,7 +75,7 @@ private void createTableIfNeeded(Context context) { Options options = Options.fromMap(table.getOptions()); if (options.get(AUTO_CREATE)) { try { - Path tablePath = CoreOptions.path(table.getOptions()); + Path tablePath = CoreOptions.schemaPath(table.getOptions()); String branch = CoreOptions.branch(table.getOptions()); SchemaManager schemaManager = new SchemaManager( diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ResetConsumerAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ResetConsumerAction.java index 9c8f800650e6..fd1746a8872f 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ResetConsumerAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/ResetConsumerAction.java @@ -51,7 +51,7 @@ public void run() throws Exception { ConsumerManager consumerManager = new ConsumerManager( dataTable.fileIO(), - dataTable.location(), + dataTable.tableSchemaPath(), dataTable.snapshotManager().branch()); if (Objects.isNull(nextSnapshotId)) { consumerManager.deleteConsumer(consumerId); 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 8bcaa2a2071f..d297c47f3ec8 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 @@ -136,7 +136,7 @@ public void processElement(StreamRecord streamRecord) throws Exce } private Path pathOfTable(Table table) { - return new Path(table.options().get(CoreOptions.PATH.key())); + return new Path(table.options().get(CoreOptions.TABLE_SCHEMA_PATH.key())); } @Override 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 67eecbc6f2ae..2609591fa60a 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 @@ -89,12 +89,13 @@ public void processElement(StreamRecord> streamRecord) th List result = toCloneFileInfos( PickFilesUtil.getUsedFilesForLatestSnapshot(sourceTable), - sourceTable.location(), + sourceTable.tableDataPath(), sourceIdentifierStr, targetIdentifierStr); if (LOG.isDebugEnabled()) { - LOG.debug("The CloneFileInfo of table {} is {} : ", sourceTable.location(), result); + LOG.debug( + "The CloneFileInfo of table {} is {} : ", sourceTable.tableDataPath(), result); } for (CloneFileInfo info : result) { @@ -110,7 +111,10 @@ private static Schema newSchemaFromTableSchema(TableSchema tableSchema) { ImmutableMap.copyOf( Iterables.filter( tableSchema.options().entrySet(), - entry -> !Objects.equals(entry.getKey(), CoreOptions.PATH.key()))), + entry -> + !Objects.equals( + entry.getKey(), + CoreOptions.TABLE_SCHEMA_PATH.key()))), tableSchema.comment()); } 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 f83b5cf8f9e3..009f4bd294b8 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 @@ -53,7 +53,7 @@ public static List getUsedFilesForLatestSnapshot(FileStoreTable table) { SnapshotManager snapshotManager = store.snapshotManager(); Snapshot snapshot = snapshotManager.latestSnapshot(); ManifestList manifestList = store.manifestListFactory().create(); - SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.location()); + SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.tableSchemaPath()); IndexFileHandler indexFileHandler = store.newIndexFileHandler(); List files = new ArrayList<>(); @@ -108,7 +108,7 @@ private static List getUsedFilesInternal( pathFactory .createDataFilePathFactory( simpleFileEntry.partition(), simpleFileEntry.bucket()) - .toPath(simpleFileEntry.fileName()); + .toPath(simpleFileEntry.fileName(), simpleFileEntry.externalPath()); dataFiles.add(dataFilePath); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactTask.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactTask.java index 6b95e369074b..5034e9293cf4 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactTask.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/compact/changelog/ChangelogCompactTask.java @@ -96,7 +96,7 @@ public List doCompact(FileStoreTable table) throws Exception { outputStream, results, table, - dataFilePathFactory.toPath(meta.fileName()), + dataFilePathFactory.toPath(meta.fileName(), meta.externalPath()), bucket, false, meta); @@ -111,7 +111,7 @@ public List doCompact(FileStoreTable table) throws Exception { outputStream, results, table, - dataFilePathFactory.toPath(meta.fileName()), + dataFilePathFactory.toPath(meta.fileName(), meta.externalPath()), bucket, true, meta); @@ -171,7 +171,8 @@ private List produceNewCommittables( realName + "." + CompactedChangelogReadOnlyFormat.getIdentifier( - baseResult.meta.fileFormat()))); + baseResult.meta.fileFormat()), + null)); List newCommittables = new ArrayList<>(); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/ResetConsumerProcedure.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/ResetConsumerProcedure.java index 934ce182a09c..98ab793c94f3 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/ResetConsumerProcedure.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/procedure/ResetConsumerProcedure.java @@ -64,7 +64,7 @@ public String[] call( ConsumerManager consumerManager = new ConsumerManager( fileStoreTable.fileIO(), - fileStoreTable.location(), + fileStoreTable.tableSchemaPath(), fileStoreTable.snapshotManager().branch()); if (nextSnapshotId != null) { fileStoreTable.snapshotManager().snapshot(nextSnapshotId); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RewriteFileIndexSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RewriteFileIndexSink.java index d9f863c6b919..9a8044cb6acf 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RewriteFileIndexSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RewriteFileIndexSink.java @@ -172,7 +172,8 @@ public FileIndexProcessor(FileStoreTable table) { this.pathFactory = table.store().pathFactory(); this.dataFilePathFactoryMap = new HashMap<>(); this.schemaInfoCache = - new SchemaCache(fileIndexOptions, new SchemaManager(fileIO, table.location())); + new SchemaCache( + fileIndexOptions, new SchemaManager(fileIO, table.tableSchemaPath())); this.sizeInMeta = table.coreOptions().fileIndexInManifestThreshold(); } @@ -198,16 +199,21 @@ public DataFileMeta process(BinaryRow partition, int bucket, DataFileMeta dataFi String indexFile = indexFiles.get(0); try (FileIndexFormat.Reader indexReader = FileIndexFormat.createReader( - fileIO.newInputStream(dataFilePathFactory.toPath(indexFile)), + fileIO.newInputStream( + dataFilePathFactory.toPath( + indexFile, dataFileMeta.externalPath())), schemaInfo.fileSchema)) { maintainers = indexReader.readAll(); } - newIndexPath = createNewFileIndexFilePath(dataFilePathFactory.toPath(indexFile)); + newIndexPath = + createNewFileIndexFilePath( + dataFilePathFactory.toPath(indexFile, dataFileMeta.externalPath())); } else { maintainers = new HashMap<>(); newIndexPath = dataFileToFileIndexPath( - dataFilePathFactory.toPath(dataFileMeta.fileName())); + dataFilePathFactory.toPath( + dataFileMeta.fileName(), dataFileMeta.externalPath())); } // remove unnecessary @@ -246,7 +252,7 @@ public DataFileMeta process(BinaryRow partition, int bucket, DataFileMeta dataFi .withBucket(bucket) .withBucketPath( pathFactory - .bucketPath(partition, bucket) + .dataBucketPath(partition, bucket) .toString()) .withDataFiles( Collections.singletonList(dataFileMeta)) diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BranchSqlITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BranchSqlITCase.java index 2566fbe92e4c..2e139d4ab726 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BranchSqlITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/BranchSqlITCase.java @@ -585,7 +585,8 @@ public void testReadBranchTableWithMultiSchemaIds() throws Exception { sql("CALL sys.create_branch('default.T', 'test', 'tag1')"); FileStoreTable table = paimonTable("T"); - SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.location(), "test"); + SchemaManager schemaManager = + new SchemaManager(table.fileIO(), table.tableDataPath(), "test"); List schemaIds = schemaManager.listAllIds(); assertThat(schemaIds.size()).isEqualTo(2); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreITCase.java index 5245114e80ee..db9386e8cd4d 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreITCase.java @@ -84,7 +84,7 @@ import static org.apache.paimon.CoreOptions.BUCKET; import static org.apache.paimon.CoreOptions.BUCKET_KEY; import static org.apache.paimon.CoreOptions.FILE_FORMAT; -import static org.apache.paimon.CoreOptions.PATH; +import static org.apache.paimon.CoreOptions.TABLE_SCHEMA_PATH; import static org.apache.paimon.flink.LogicalTypeConversion.toDataType; import static org.apache.paimon.utils.FailingFileIO.retryArtificialException; import static org.assertj.core.api.Assertions.assertThat; @@ -486,7 +486,7 @@ public static FileStoreTable buildFileStoreTable( boolean noFail, String temporaryPath, int[] partitions, int[] primaryKey) throws Exception { Options options = buildConfiguration(noFail, temporaryPath); - Path tablePath = new CoreOptions(options.toMap()).path(); + Path tablePath = new CoreOptions(options.toMap()).schemaPath(); if (primaryKey.length == 0) { options.set(BUCKET_KEY, "_k"); } @@ -512,11 +512,12 @@ public static Options buildConfiguration(boolean noFail, String temporaryPath) { Options options = new Options(); options.set(BUCKET, NUM_BUCKET); if (noFail) { - options.set(PATH, temporaryPath); + options.set(TABLE_SCHEMA_PATH, temporaryPath); } else { String failingName = UUID.randomUUID().toString(); FailingFileIO.reset(failingName, 3, 100); - options.set(PATH, FailingFileIO.getFailingPath(failingName, temporaryPath)); + options.set( + TABLE_SCHEMA_PATH, FailingFileIO.getFailingPath(failingName, temporaryPath)); } options.set(FILE_FORMAT, CoreOptions.FILE_FORMAT_AVRO); options.set(BRANCH, branch); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkBatchJobPartitionMarkdoneITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkBatchJobPartitionMarkdoneITCase.java index 340213aeae1b..36421b013949 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkBatchJobPartitionMarkdoneITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkBatchJobPartitionMarkdoneITCase.java @@ -57,7 +57,7 @@ import static org.apache.paimon.CoreOptions.BUCKET; import static org.apache.paimon.CoreOptions.BUCKET_KEY; import static org.apache.paimon.CoreOptions.FILE_FORMAT; -import static org.apache.paimon.CoreOptions.PATH; +import static org.apache.paimon.CoreOptions.TABLE_SCHEMA_PATH; import static org.apache.paimon.flink.LogicalTypeConversion.toDataType; import static org.apache.paimon.utils.FailingFileIO.retryArtificialException; import static org.assertj.core.api.Assertions.assertThat; @@ -111,11 +111,11 @@ protected List ddl() { private void validateResult(FileStoreTable table) throws Exception { LocalFileIO fileIO = new LocalFileIO(); - Path successPath1 = new Path(table.location(), "p=p1/_SUCCESS"); + Path successPath1 = new Path(table.tableDataPath(), "p=p1/_SUCCESS"); SuccessFile successFile1 = SuccessFile.safelyFromPath(fileIO, successPath1); assertThat(successFile1).isNotNull(); - Path successPath2 = new Path(table.location(), "p=p2/_SUCCESS"); + Path successPath2 = new Path(table.tableDataPath(), "p=p2/_SUCCESS"); SuccessFile successFile2 = SuccessFile.safelyFromPath(fileIO, successPath2); assertThat(successFile2).isNotNull(); } @@ -159,11 +159,11 @@ private FileStoreTable buildFileStoreTable(int[] partitions, int[] primaryKey) throws Exception { Options options = new Options(); options.set(BUCKET, 3); - options.set(PATH, getTempDirPath()); + options.set(TABLE_SCHEMA_PATH, getTempDirPath()); options.set(FILE_FORMAT, CoreOptions.FILE_FORMAT_AVRO); options.set(CoreOptions.PARTITION_MARK_DONE_WHEN_END_INPUT.key(), "true"); - Path tablePath = new CoreOptions(options.toMap()).path(); + Path tablePath = new CoreOptions(options.toMap()).schemaPath(); if (primaryKey.length == 0) { options.set(BUCKET_KEY, "_k"); } 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..31dd6506575d 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 @@ -90,8 +90,8 @@ import java.util.stream.Stream; 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_SCHEMA_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; @@ -353,7 +353,7 @@ public void testCreateFlinkTable(Map options) { public void testCreateFlinkTableWithPath() throws Exception { catalog.createDatabase(path1.getDatabaseName(), null, false); Map options = new HashMap<>(); - options.put(PATH.key(), "/unknown/path"); + options.put(TABLE_SCHEMA_PATH.key(), "/unknown/path"); CatalogTable table1 = createTable(options); assertThatThrownBy(() -> catalog.createTable(this.path1, table1, false)) .hasMessageContaining( @@ -899,7 +899,7 @@ private void checkEquals( .catalog() .getTable(FlinkCatalog.toIdentifier(path)) .options() - .get(PATH.key())); + .get(TABLE_SCHEMA_PATH.key())); } catch (org.apache.paimon.catalog.Catalog.TableNotExistException e) { throw new RuntimeException(e); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkEndInputPartitionExpireITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkEndInputPartitionExpireITCase.java index 63eebcd7d1f2..abaafe47246a 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkEndInputPartitionExpireITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkEndInputPartitionExpireITCase.java @@ -62,7 +62,7 @@ import static org.apache.paimon.CoreOptions.PARTITION_EXPIRATION_CHECK_INTERVAL; import static org.apache.paimon.CoreOptions.PARTITION_EXPIRATION_TIME; import static org.apache.paimon.CoreOptions.PARTITION_TIMESTAMP_FORMATTER; -import static org.apache.paimon.CoreOptions.PATH; +import static org.apache.paimon.CoreOptions.TABLE_SCHEMA_PATH; import static org.apache.paimon.flink.LogicalTypeConversion.toDataType; import static org.apache.paimon.utils.FailingFileIO.retryArtificialException; @@ -134,14 +134,14 @@ private FileStoreTable buildFileStoreTable(int[] partitions, int[] primaryKey) throws Exception { Options options = new Options(); options.set(BUCKET, 3); - options.set(PATH, getTempDirPath()); + options.set(TABLE_SCHEMA_PATH, getTempDirPath()); options.set(FILE_FORMAT, CoreOptions.FILE_FORMAT_AVRO); options.set(PARTITION_EXPIRATION_TIME, Duration.ofDays(2)); options.set(PARTITION_EXPIRATION_CHECK_INTERVAL, Duration.ofHours(1)); options.set(PARTITION_TIMESTAMP_FORMATTER, "yyyyMMdd"); options.set(END_INPUT_CHECK_PARTITION_EXPIRE, true); - Path tablePath = new CoreOptions(options.toMap()).path(); + Path tablePath = new CoreOptions(options.toMap()).schemaPath(); if (primaryKey.length == 0) { options.set(BUCKET_KEY, "_k"); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkEndInputWatermarkITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkEndInputWatermarkITCase.java index c85b570dd53a..ba1ad10946fd 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkEndInputWatermarkITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FlinkEndInputWatermarkITCase.java @@ -57,7 +57,7 @@ import static org.apache.paimon.CoreOptions.BUCKET; import static org.apache.paimon.CoreOptions.BUCKET_KEY; import static org.apache.paimon.CoreOptions.FILE_FORMAT; -import static org.apache.paimon.CoreOptions.PATH; +import static org.apache.paimon.CoreOptions.TABLE_SCHEMA_PATH; import static org.apache.paimon.flink.LogicalTypeConversion.toDataType; import static org.apache.paimon.utils.FailingFileIO.retryArtificialException; @@ -152,13 +152,13 @@ private FileStoreTable buildFileStoreTable(int[] partitions, int[] primaryKey) throws Exception { Options options = new Options(); options.set(BUCKET, 3); - options.set(PATH, getTempDirPath()); + options.set(TABLE_SCHEMA_PATH, getTempDirPath()); options.set(FILE_FORMAT, CoreOptions.FILE_FORMAT_AVRO); options.set( FlinkConnectorOptions.END_INPUT_WATERMARK.key(), String.valueOf(END_INPUT_WATERMARK)); - Path tablePath = new CoreOptions(options.toMap()).path(); + Path tablePath = new CoreOptions(options.toMap()).schemaPath(); if (primaryKey.length == 0) { options.set(BUCKET_KEY, "_k"); } 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..7e8ca4c12183 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 @@ -1855,7 +1855,7 @@ private void testSinkParallelism(Integer configParallelism, int expectedParallel false); // create table - Path path = CoreOptions.path(context.getCatalogTable().getOptions()); + Path path = CoreOptions.schemaPath(context.getCatalogTable().getOptions()); LocalFileIO.create().mkdirs(path); // update schema new SchemaManager(LocalFileIO.create(), path) diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/BranchActionITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/BranchActionITCase.java index a8286964832a..af595e8bec7a 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/BranchActionITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/BranchActionITCase.java @@ -70,7 +70,7 @@ void testCreateAndDeleteBranch() throws Exception { writeData(rowData(2L, BinaryString.fromString("Hello"))); writeData(rowData(3L, BinaryString.fromString("Paimon"))); - TagManager tagManager = new TagManager(table.fileIO(), table.location()); + TagManager tagManager = new TagManager(table.fileIO(), table.tableDataPath()); executeSQL( String.format( "CALL sys.create_tag('%s.%s', 'tag2', 2, '5 d')", database, tableName)); @@ -236,7 +236,7 @@ void testFastForward() throws Exception { writeData(rowData(3L, BinaryString.fromString("Paimon"))); // Create tag2 - TagManager tagManager = new TagManager(table.fileIO(), table.location()); + TagManager tagManager = new TagManager(table.fileIO(), table.tableDataPath()); executeSQL(String.format("CALL sys.create_tag('%s.%s', 'tag2', 2)", database, tableName)); assertThat(tagManager.tagExists("tag2")).isTrue(); // Create tag3 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..fb5d705b6b75 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 @@ -465,11 +465,11 @@ private void compareCloneFiles( Pair.of( absolutePath, getPathExcludeTableRoot( - absolutePath, targetTable.location()))) + absolutePath, targetTable.tableDataPath()))) .collect(Collectors.toList()); FileStoreTable sourceTable = getFileStoreTable(sourceWarehouse, sourceDb, sourceTableName); - Path tableLocation = sourceTable.location(); + Path tableLocation = sourceTable.tableDataPath(); for (Pair filesPathInfo : filesPathInfoList) { Path sourceTableFile = new Path(tableLocation.toString() + filesPathInfo.getRight()); assertThat(sourceTable.fileIO().exists(sourceTableFile)).isTrue(); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/ConsumerActionITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/ConsumerActionITCase.java index 6fb8c81eb744..44543ef48ce9 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/ConsumerActionITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/ConsumerActionITCase.java @@ -86,7 +86,8 @@ public void testResetConsumer(String invoker) throws Exception { changelogRow("+I", 2L, "Hello"), changelogRow("+I", 3L, "Paimon"))); - ConsumerManager consumerManager = new ConsumerManager(table.fileIO(), table.location()); + ConsumerManager consumerManager = + new ConsumerManager(table.fileIO(), table.tableDataPath()); while (!consumerManager.consumer("myid").isPresent()) { Thread.sleep(1000); } @@ -242,7 +243,7 @@ public void testResetBranchConsumer(String invoker) throws Exception { changelogRow("+I", 3L, "Paimon"))); ConsumerManager consumerManager = - new ConsumerManager(table.fileIO(), table.location(), branchName); + new ConsumerManager(table.fileIO(), table.tableDataPath(), branchName); while (!consumerManager.consumer("myid").isPresent()) { Thread.sleep(1000); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/MarkPartitionDoneActionITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/MarkPartitionDoneActionITCase.java index 3c1b73df8cb3..fef955c62152 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/MarkPartitionDoneActionITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/MarkPartitionDoneActionITCase.java @@ -96,7 +96,7 @@ public void testPartitionMarkDoneWithSinglePartitionKey(boolean hasPk, String in throw new UnsupportedOperationException(invoker); } - Path successPath = new Path(table.location(), "partKey0=0/_SUCCESS"); + Path successPath = new Path(table.tableDataPath(), "partKey0=0/_SUCCESS"); SuccessFile successFile = SuccessFile.safelyFromPath(table.fileIO(), successPath); assertThat(successFile).isNotNull(); } @@ -140,11 +140,11 @@ public void testDropPartitionWithMultiplePartitionKey(boolean hasPk, String invo throw new UnsupportedOperationException(invoker); } - Path successPath1 = new Path(table.location(), "partKey0=0/partKey1=1/_SUCCESS"); + Path successPath1 = new Path(table.tableDataPath(), "partKey0=0/partKey1=1/_SUCCESS"); SuccessFile successFile1 = SuccessFile.safelyFromPath(table.fileIO(), successPath1); assertThat(successFile1).isNotNull(); - Path successPath2 = new Path(table.location(), "partKey0=1/partKey1=0/_SUCCESS"); + Path successPath2 = new Path(table.tableDataPath(), "partKey0=1/partKey1=0/_SUCCESS"); SuccessFile successFile2 = SuccessFile.safelyFromPath(table.fileIO(), successPath2); assertThat(successFile2).isNotNull(); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/RemoveOrphanFilesActionITCaseBase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/RemoveOrphanFilesActionITCaseBase.java index 77f3be2f0c76..6d0757c4b847 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/RemoveOrphanFilesActionITCaseBase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/RemoveOrphanFilesActionITCaseBase.java @@ -93,7 +93,7 @@ private FileStoreTable createTableAndWriteData(String tableName) throws Exceptio } private Path getOrphanFilePath(FileStoreTable table, String orphanFile) { - return new Path(table.location(), orphanFile); + return new Path(table.tableDataPath(), orphanFile); } @ParameterizedTest @@ -225,14 +225,15 @@ public void testCleanWithBranch(boolean isNamedArgument) throws Exception { // create first branch and write some data table.createBranch("br"); - SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.location(), "br"); + SchemaManager schemaManager = + new SchemaManager(table.fileIO(), table.tableDataPath(), "br"); TableSchema branchSchema = schemaManager.commitChanges(SchemaChange.addColumn("v2", DataTypes.INT())); Options branchOptions = new Options(branchSchema.options()); branchOptions.set(CoreOptions.BRANCH, "br"); branchSchema = branchSchema.copy(branchOptions.toMap()); FileStoreTable branchTable = - FileStoreTableFactory.create(table.fileIO(), table.location(), branchSchema); + FileStoreTableFactory.create(table.fileIO(), table.tableDataPath(), branchSchema); String commitUser = UUID.randomUUID().toString(); StreamTableWrite write = branchTable.newWrite(commitUser); @@ -243,14 +244,16 @@ public void testCleanWithBranch(boolean isNamedArgument) throws Exception { commit.close(); // create orphan file in snapshot directory of first branch - Path orphanFile3 = new Path(table.location(), "branch/branch-br/snapshot/orphan_file3"); + Path orphanFile3 = + new Path(table.tableDataPath(), "branch/branch-br/snapshot/orphan_file3"); branchTable.fileIO().writeFile(orphanFile3, "x", true); // create second branch, which is empty table.createBranch("br2"); // create orphan file in snapshot directory of second branch - Path orphanFile4 = new Path(table.location(), "branch/branch-br2/snapshot/orphan_file4"); + Path orphanFile4 = + new Path(table.tableDataPath(), "branch/branch-br2/snapshot/orphan_file4"); branchTable.fileIO().writeFile(orphanFile4, "y", true); if (ThreadLocalRandom.current().nextBoolean()) { diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/RewriteFileIndexActionITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/RewriteFileIndexActionITCase.java index 242a7514168e..c27e1fc9d50c 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/RewriteFileIndexActionITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/RewriteFileIndexActionITCase.java @@ -102,7 +102,7 @@ public void testFileIndexAddIndex() throws Exception { table.store() .pathFactory() .createDataFilePathFactory(entry.partition(), entry.bucket()) - .toPath(file); + .toPath(file, null); try (FileIndexFormat.Reader reader = FileIndexFormat.createReader( table.fileIO().newInputStream(indexFilePath), table.rowType())) { diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/TagActionITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/TagActionITCase.java index 7310a68df7a2..1b7d60cc6ee0 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/TagActionITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/TagActionITCase.java @@ -71,7 +71,7 @@ public void testCreateAndDeleteTag(String invoker) throws Exception { writeData(rowData(2L, BinaryString.fromString("Hello"))); writeData(rowData(3L, BinaryString.fromString("Paimon"))); - TagManager tagManager = new TagManager(table.fileIO(), table.location()); + TagManager tagManager = new TagManager(table.fileIO(), table.tableDataPath()); switch (invoker) { case "action": @@ -266,7 +266,7 @@ public void testRenameTag(String invoker) throws Exception { writeData(rowData(2L, BinaryString.fromString("Hello"))); writeData(rowData(3L, BinaryString.fromString("Paimon"))); - TagManager tagManager = new TagManager(table.fileIO(), table.location()); + TagManager tagManager = new TagManager(table.fileIO(), table.tableDataPath()); switch (invoker) { case "action": createAction( @@ -361,7 +361,7 @@ public void testCreateLatestTag(String invoker) throws Exception { writeData(rowData(2L, BinaryString.fromString("Hello"))); writeData(rowData(3L, BinaryString.fromString("Paimon"))); - TagManager tagManager = new TagManager(table.fileIO(), table.location()); + TagManager tagManager = new TagManager(table.fileIO(), table.tableDataPath()); switch (invoker) { case "action": diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/RewriteFileIndexProcedureITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/RewriteFileIndexProcedureITCase.java index 1abfe355a566..fe3fa558cb1b 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/RewriteFileIndexProcedureITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/procedure/RewriteFileIndexProcedureITCase.java @@ -199,7 +199,7 @@ public void testFileIndexProcedureDropIndex(boolean isNamedArgument) throws Exce table.store() .pathFactory() .createDataFilePathFactory(entry.partition(), entry.bucket()) - .toPath(file); + .toPath(file, null); try (FileIndexFormat.Reader reader = FileIndexFormat.createReader( table.fileIO().newInputStream(indexFilePath), table.rowType())) { diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AppendOnlySingleTableCompactionWorkerOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AppendOnlySingleTableCompactionWorkerOperatorTest.java index 6238a9cbf3ea..817f34b30d32 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AppendOnlySingleTableCompactionWorkerOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/AppendOnlySingleTableCompactionWorkerOperatorTest.java @@ -161,7 +161,9 @@ public void testAsyncCompactionFileDeletedWhenShutdown() throws Exception { ((CommitMessageImpl) commitMessage).compactIncrement().compactAfter(); for (DataFileMeta fileMeta : fileMetas) { Assertions.assertThat( - localFileIO.exists(dataFilePathFactory.toPath(fileMeta.fileName()))) + localFileIO.exists( + dataFilePathFactory.toPath( + fileMeta.fileName(), fileMeta.externalPath()))) .isTrue(); } if (i++ > 2) { @@ -190,7 +192,8 @@ public void testAsyncCompactionFileDeletedWhenShutdown() throws Exception { for (DataFileMeta fileMeta : fileMetas) { Assertions.assertThat( localFileIO.exists( - dataFilePathFactory.toPath(fileMeta.fileName()))) + dataFilePathFactory.toPath( + fileMeta.fileName(), fileMeta.externalPath()))) .isFalse(); } } catch (Exception e) { diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommitterOperatorTestBase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommitterOperatorTestBase.java index a69f8dbd3a35..90a97cd99c3b 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommitterOperatorTestBase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CommitterOperatorTestBase.java @@ -98,7 +98,7 @@ protected FileStoreTable createFileStoreTable(Consumer setOptions) thro protected FileStoreTable createFileStoreTable( Consumer setOptions, List partitionKeys) throws Exception { Options conf = new Options(); - conf.set(CoreOptions.PATH, tablePath.toString()); + conf.set(CoreOptions.TABLE_SCHEMA_PATH, tablePath.toString()); conf.setString("bucket", "1"); conf.setString("bucket-key", "a"); setOptions.accept(conf); 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..4f815c0e4def 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 @@ -109,7 +109,7 @@ private boolean testSpillable( private FileStoreTable createFileStoreTable() throws Exception { org.apache.paimon.fs.Path tablePath = new org.apache.paimon.fs.Path(tempPath.toString()); Options options = new Options(); - options.set(CoreOptions.PATH, tablePath.toString()); + options.set(CoreOptions.TABLE_SCHEMA_PATH, tablePath.toString()); options.set(CoreOptions.BUCKET, 1); TableSchema tableSchema = SchemaUtils.forceCommit( diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreMultiCommitterTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreMultiCommitterTest.java index 53e3a6dcb79c..815a99c626d2 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreMultiCommitterTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/StoreMultiCommitterTest.java @@ -145,8 +145,8 @@ public void beforeEach() throws Exception { catalog, Tuple2.of(firstTable, firstTableSchema), Tuple2.of(secondTable, secondTableSchema)); - firstTablePath = ((FileStoreTable) catalog.getTable(firstTable)).location(); - secondTablePath = ((FileStoreTable) catalog.getTable(secondTable)).location(); + firstTablePath = ((FileStoreTable) catalog.getTable(firstTable)).tableDataPath(); + secondTablePath = ((FileStoreTable) catalog.getTable(secondTable)).tableDataPath(); } @AfterEach diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterOperatorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterOperatorTest.java index 83af15745078..2a6e7022a245 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterOperatorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterOperatorTest.java @@ -467,7 +467,7 @@ private void commitAll( private FileStoreTable createFileStoreTable( RowType rowType, List primaryKeys, List partitionKeys, Options conf) throws Exception { - conf.set(CoreOptions.PATH, tablePath.toString()); + conf.set(CoreOptions.TABLE_SCHEMA_PATH, tablePath.toString()); SchemaManager schemaManager = new SchemaManager(LocalFileIO.create(), tablePath); schemaManager.createTable( new Schema(rowType.getFields(), partitionKeys, primaryKeys, conf.toMap(), "")); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/PartitionMarkDoneTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/PartitionMarkDoneTest.java index 9e5fe7ff9ff7..c3c98a8bb472 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/PartitionMarkDoneTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/partition/PartitionMarkDoneTest.java @@ -83,7 +83,7 @@ private void innerTest(boolean deletionVectors) throws Exception { .build(); catalog.createTable(identifier, schema, true); FileStoreTable table = (FileStoreTable) catalog.getTable(identifier); - Path location = table.location(); + Path location = table.tableDataPath(); Path successFile = new Path(location, "a=0/_SUCCESS"); PartitionMarkDone markDone = PartitionMarkDone.create(false, false, new MockOperatorStateStore(), table).get(); 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..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 @@ -111,6 +111,7 @@ public TestChangelogDataReadWrite(String root) { CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), + null, null); this.snapshotManager = new SnapshotManager(LocalFileIO.create(), new Path(root)); this.commitUser = UUID.randomUUID().toString(); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/align/AlignedContinuousFileSplitEnumeratorTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/align/AlignedContinuousFileSplitEnumeratorTest.java index a5edc2804061..3291c7acb221 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/align/AlignedContinuousFileSplitEnumeratorTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/align/AlignedContinuousFileSplitEnumeratorTest.java @@ -184,7 +184,8 @@ public void testScanWithConsumerId() throws Exception { } enumerator.addSplits(splits); - ConsumerManager consumerManager = new ConsumerManager(table.fileIO(), table.location()); + ConsumerManager consumerManager = + new ConsumerManager(table.fileIO(), table.tableDataPath()); assertThat(consumerManager.consumer(CONSUMER_ID)).isEmpty(); enumerator.handleSplitRequest(0, "test-host"); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/statistics/AppendOnlyTableStatisticsTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/statistics/AppendOnlyTableStatisticsTest.java index 6b6ae8c3a515..b1f1950e7aff 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/statistics/AppendOnlyTableStatisticsTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/statistics/AppendOnlyTableStatisticsTest.java @@ -31,7 +31,7 @@ public class AppendOnlyTableStatisticsTest extends FileStoreTableStatisticsTestB @Override protected FileStoreTable createStoreTable() throws Exception { Options conf = new Options(); - conf.set(CoreOptions.PATH, tablePath.toString()); + conf.set(CoreOptions.TABLE_SCHEMA_PATH, tablePath.toString()); conf.set(CoreOptions.BUCKET, 1); TableSchema tableSchema = new SchemaManager(LocalFileIO.create(), tablePath) diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/statistics/PrimaryKeyTableStatisticsTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/statistics/PrimaryKeyTableStatisticsTest.java index ea47df2d9d72..e0cec38da881 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/statistics/PrimaryKeyTableStatisticsTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/statistics/PrimaryKeyTableStatisticsTest.java @@ -60,7 +60,7 @@ public void testTableFilterValueStatistics() throws Exception { @Override FileStoreTable createStoreTable() throws Exception { Options options = new Options(); - options.set(CoreOptions.PATH, tablePath.toString()); + options.set(CoreOptions.TABLE_SCHEMA_PATH, tablePath.toString()); options.set(CoreOptions.BUCKET, 1); Schema.Builder builder = schemaBuilder(); builder.options(options.toMap()); diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java index 0be872a58cbf..4cad5cc92a1f 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java @@ -217,9 +217,9 @@ private Pair initialTableLocation( Map tableOptions, Identifier identifier) { boolean externalTable; Path location; - if (tableOptions.containsKey(CoreOptions.PATH.key())) { + if (tableOptions.containsKey(CoreOptions.TABLE_SCHEMA_PATH.key())) { externalTable = true; - location = new Path(tableOptions.get(CoreOptions.PATH.key())); + location = new Path(tableOptions.get(CoreOptions.TABLE_SCHEMA_PATH.key())); } else { externalTable = usingExternalTable(tableOptions); location = getTableLocation(identifier, null); @@ -379,7 +379,7 @@ private boolean partitionExistsInOtherBranches( } Optional branchSchema = - tableSchemaInFileSystem(mainTable.location(), branchName); + tableSchemaInFileSystem(mainTable.tableDataPath(), branchName); if (!branchSchema.isPresent()) { continue; } diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/migrate/HiveMigrator.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/migrate/HiveMigrator.java index d1478830ac6d..48deca0b58e1 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/migrate/HiveMigrator.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/migrate/HiveMigrator.java @@ -322,7 +322,7 @@ private List importPartitionedTableTask( values, valueSetters, coreOptions.partitionDefaultName()); - Path path = paimonTable.store().pathFactory().bucketPath(partitionRow, 0); + Path path = paimonTable.store().pathFactory().dataBucketPath(partitionRow, 0); migrateTasks.add( new MigrateTask( @@ -338,7 +338,7 @@ public MigrateTask importUnPartitionedTableTask( Map rollback) { String format = parseFormat(sourceTable.getSd().getSerdeInfo().toString()); String location = sourceTable.getSd().getLocation(); - Path path = paimonTable.store().pathFactory().bucketPath(BinaryRow.EMPTY_ROW, 0); + Path path = paimonTable.store().pathFactory().dataBucketPath(BinaryRow.EMPTY_ROW, 0); return new MigrateTask( fileIO, format, location, paimonTable, BinaryRow.EMPTY_ROW, path, rollback); } diff --git a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java b/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java index bf6eb02f3e55..658e8d80f7a5 100644 --- a/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java +++ b/paimon-hive/paimon-hive-catalog/src/test/java/org/apache/paimon/hive/HiveCatalogTest.java @@ -390,7 +390,7 @@ public void testCreateExternalTableWithLocation(@TempDir java.nio.file.Path temp org.apache.paimon.table.Table table = externalWarehouseCatalog.getTable(identifier); assertThat(table.options()) - .extracting(CoreOptions.PATH.key()) + .extracting(CoreOptions.TABLE_SCHEMA_PATH.key()) .isEqualTo("file:" + externalTablePath); externalWarehouseCatalog.close(); diff --git a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/HiveSchema.java b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/HiveSchema.java index 108315a96103..5aad8e8875cd 100644 --- a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/HiveSchema.java +++ b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/HiveSchema.java @@ -199,7 +199,7 @@ private static Optional getExistingSchema( } Path path = new Path(location); Options options = HiveUtils.extractCatalogConfig(configuration); - options.set(CoreOptions.PATH, location); + options.set(CoreOptions.TABLE_SCHEMA_PATH, location); CatalogContext context = CatalogContext.create(options, configuration); try { return new SchemaManager(FileIO.get(path, context), path).latest(); diff --git a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/PaimonMetaHook.java b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/PaimonMetaHook.java index 5cc826b554bf..f80adf3dbd33 100644 --- a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/PaimonMetaHook.java +++ b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/PaimonMetaHook.java @@ -184,7 +184,7 @@ public void commitDropTable(Table table, boolean b) throws MetaException {} private CatalogContext catalogContext(Table table, String location) { Options options = HiveUtils.extractCatalogConfig(conf); - options.set(CoreOptions.PATH, location); + options.set(CoreOptions.TABLE_SCHEMA_PATH, location); table.getParameters().forEach(options::set); return CatalogContext.create(options, conf); } diff --git a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/mapred/PaimonOutputCommitter.java b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/mapred/PaimonOutputCommitter.java index 94bc4a675ae8..c05e33136e21 100644 --- a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/mapred/PaimonOutputCommitter.java +++ b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/mapred/PaimonOutputCommitter.java @@ -96,7 +96,7 @@ public void commitTask(TaskAttemptContext originalContext) throws IOException { createPreCommitFile( commitTables, generatePreCommitFileLocation( - table.location(), + table.tableDataPath(), attemptID.getJobID(), attemptID.getTaskID().getId()), table.fileIO()); @@ -144,7 +144,7 @@ public void commitJob(JobContext originalContext) throws IOException { if (table != null) { BatchWriteBuilder batchWriteBuilder = table.newBatchWriteBuilder(); List commitMessagesList = - getAllPreCommitMessage(table.location(), jobContext, table.fileIO()); + getAllPreCommitMessage(table.tableDataPath(), jobContext, table.fileIO()); try (BatchTableCommit batchTableCommit = batchWriteBuilder.newCommit()) { batchTableCommit.commit(commitMessagesList); } catch (Exception e) { @@ -152,7 +152,7 @@ public void commitJob(JobContext originalContext) throws IOException { } deleteTemporaryFile( jobContext, - generateJobLocation(table.location(), jobContext.getJobID()), + generateJobLocation(table.tableDataPath(), jobContext.getJobID()), table.fileIO()); } else { LOG.info("CommitJob not found table, Skipping job commit."); @@ -172,7 +172,7 @@ public void abortJob(JobContext originalContext, int status) throws IOException LOG.info("AbortJob {} has started", jobContext.getJobID()); List commitMessagesList = - getAllPreCommitMessage(table.location(), jobContext, table.fileIO()); + getAllPreCommitMessage(table.tableDataPath(), jobContext, table.fileIO()); BatchWriteBuilder batchWriteBuilder = table.newBatchWriteBuilder(); try (BatchTableCommit batchTableCommit = batchWriteBuilder.newCommit()) { batchTableCommit.abort(commitMessagesList); @@ -181,7 +181,7 @@ public void abortJob(JobContext originalContext, int status) throws IOException } deleteTemporaryFile( jobContext, - generateJobLocation(table.location(), jobContext.getJobID()), + generateJobLocation(table.tableDataPath(), jobContext.getJobID()), table.fileIO()); LOG.info("Job {} is aborted. preCommit file has deleted", jobContext.getJobID()); } diff --git a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/utils/HiveUtils.java b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/utils/HiveUtils.java index 789879b35e89..c6f46ed73357 100644 --- a/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/utils/HiveUtils.java +++ b/paimon-hive/paimon-hive-connector-common/src/main/java/org/apache/paimon/hive/utils/HiveUtils.java @@ -54,7 +54,7 @@ public class HiveUtils { public static FileStoreTable createFileStoreTable(JobConf jobConf) { Options options = extractCatalogConfig(jobConf); - options.set(CoreOptions.PATH, LocationKeyExtractor.getPaimonLocation(jobConf)); + options.set(CoreOptions.TABLE_SCHEMA_PATH, LocationKeyExtractor.getPaimonLocation(jobConf)); CatalogContext catalogContext; if (options.get(HADOOP_LOAD_DEFAULT_CONFIG)) { diff --git a/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/PaimonSinkTest.scala b/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/PaimonSinkTest.scala index ab4a9bcd9dbf..5b22703010dc 100644 --- a/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/PaimonSinkTest.scala +++ b/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/PaimonSinkTest.scala @@ -43,7 +43,7 @@ class PaimonSinkTest extends PaimonSparkTestBase with StreamTest { |CREATE TABLE T (a INT, b STRING) |TBLPROPERTIES ('primary-key'='a', 'bucket'='3') |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(Int, String)] val stream = inputData @@ -87,7 +87,7 @@ class PaimonSinkTest extends PaimonSparkTestBase with StreamTest { |CREATE TABLE T (a INT, b STRING) |TBLPROPERTIES ('primary-key'='a', 'bucket'='3') |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(Int, String)] val stream = inputData @@ -127,7 +127,7 @@ class PaimonSinkTest extends PaimonSparkTestBase with StreamTest { spark.sql(s""" |CREATE TABLE T (city String, population Long) |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(Int, String)] val stream = inputData.toDS @@ -171,7 +171,7 @@ class PaimonSinkTest extends PaimonSparkTestBase with StreamTest { |CREATE TABLE T (a INT, b STRING) |TBLPROPERTIES ('primary-key'='a', 'bucket'='3') |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(Int, String)] intercept[RuntimeException] { @@ -194,7 +194,7 @@ class PaimonSinkTest extends PaimonSparkTestBase with StreamTest { spark.sql(s""" |CREATE TABLE T (start Timestamp, stockId INT, avg_price DOUBLE) |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(Long, Int, Double)] val data = inputData.toDS @@ -245,7 +245,7 @@ class PaimonSinkTest extends PaimonSparkTestBase with StreamTest { |CREATE TABLE T (a INT, b STRING) |TBLPROPERTIES ('primary-key'='a', 'bucket'='3') |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val date = Date.valueOf("2023-08-10") spark.sql("INSERT INTO T VALUES (1, '2023-08-09'), (2, '2023-08-09')") diff --git a/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/CreateAndDeleteTagProcedureTest.scala b/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/CreateAndDeleteTagProcedureTest.scala index 3f59e897ec6c..3a0643e80ba4 100644 --- a/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/CreateAndDeleteTagProcedureTest.scala +++ b/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/CreateAndDeleteTagProcedureTest.scala @@ -37,7 +37,7 @@ class CreateAndDeleteTagProcedureTest extends PaimonSparkTestBase with StreamTes |CREATE TABLE T (a INT, b STRING) |TBLPROPERTIES ('primary-key'='a', 'bucket'='3') |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(Int, String)] val stream = inputData diff --git a/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/RollbackProcedureTest.scala b/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/RollbackProcedureTest.scala index 7a3a5730ed3b..9331f7b9bcad 100644 --- a/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/RollbackProcedureTest.scala +++ b/paimon-spark/paimon-spark-3.4/src/test/scala/org/apache/paimon/spark/sql/RollbackProcedureTest.scala @@ -37,7 +37,7 @@ class RollbackProcedureTest extends PaimonSparkTestBase with StreamTest { |CREATE TABLE T (a INT, b STRING) |TBLPROPERTIES ('primary-key'='a', 'bucket'='3') |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(Int, String)] val stream = inputData diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java index f32b87603f44..423909bd8c39 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/SparkCatalog.java @@ -392,7 +392,7 @@ private Schema toInitialSchema( normalizedProperties.remove(TableCatalog.PROP_COMMENT); if (normalizedProperties.containsKey(TableCatalog.PROP_LOCATION)) { String path = normalizedProperties.remove(TableCatalog.PROP_LOCATION); - normalizedProperties.put(CoreOptions.PATH.key(), path); + normalizedProperties.put(CoreOptions.TABLE_SCHEMA_PATH.key(), path); } String pkAsString = properties.get(PRIMARY_KEY_IDENTIFIER); List primaryKeys = diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/ResetConsumerProcedure.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/ResetConsumerProcedure.java index 0f7fabd05d13..31f6bf1832f0 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/ResetConsumerProcedure.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/ResetConsumerProcedure.java @@ -85,7 +85,7 @@ public InternalRow[] call(InternalRow args) { ConsumerManager consumerManager = new ConsumerManager( fileStoreTable.fileIO(), - fileStoreTable.location(), + fileStoreTable.tableDataPath(), fileStoreTable.snapshotManager().branch()); if (nextSnapshotId == null) { consumerManager.deleteConsumer(consumerId); diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkSource.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkSource.scala index d80d7350a655..cb1f17161578 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkSource.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkSource.scala @@ -82,7 +82,7 @@ class SparkSource } private def loadTable(options: JMap[String, String]): DataTable = { - val path = CoreOptions.path(options) + val path = CoreOptions.schemaPath(options) val catalogContext = CatalogContext.create( Options.fromMap( mergeSQLConfWithIdentifier( diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkTable.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkTable.scala index b9a90d8b5bef..de33358bde4d 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkTable.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/SparkTable.scala @@ -64,8 +64,10 @@ case class SparkTable(table: Table) if (table.comment.isPresent) { properties.put(TableCatalog.PROP_COMMENT, table.comment.get) } - if (properties.containsKey(CoreOptions.PATH.key())) { - properties.put(TableCatalog.PROP_LOCATION, properties.get(CoreOptions.PATH.key())) + if (properties.containsKey(CoreOptions.TABLE_SCHEMA_PATH.key())) { + properties.put( + TableCatalog.PROP_LOCATION, + properties.get(CoreOptions.TABLE_SCHEMA_PATH.key())) } properties case _ => Collections.emptyMap() diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonCommand.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonCommand.scala index 28ac1623fb59..ae8a5f79e648 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonCommand.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/PaimonCommand.scala @@ -86,7 +86,7 @@ trait PaimonCommand extends WithFileStoreTable with ExpressionHelper with SQLCon /** Gets a relative path against the table path. */ protected def relativePath(absolutePath: String): String = { - val location = table.location().toUri + val location = table.tableDataPath().toUri location.relativize(new URI(absolutePath)).toString } @@ -230,7 +230,7 @@ trait PaimonCommand extends WithFileStoreTable with ExpressionHelper with SQLCon dataFilePathToMeta.mapValues(meta => (meta.partition, meta.bucket)).toArray val my_table = table - val location = my_table.location + val location = my_table.tableDataPath dataWithMetadataColumns .select(FILE_PATH_COLUMN, ROW_INDEX_COLUMN) .as[(String, Long)] diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/SparkDataFileMeta.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/SparkDataFileMeta.scala index 569a84a74cf5..078c28a5ad6b 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/SparkDataFileMeta.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/SparkDataFileMeta.scala @@ -77,7 +77,7 @@ object SparkDataFileMeta { .withDataFiles(dataFiles.toList.asJava) .withDataDeletionFiles(deletionFiles.toList.asJava) .rawConvertible(rawConvertible) - .withBucketPath(pathFactory.bucketPath(partition, bucket).toString) + .withBucketPath(pathFactory.dataBucketPath(partition, bucket).toString) .build() } .toArray diff --git a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/WithFileStoreTable.scala b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/WithFileStoreTable.scala index 7b0907a1c803..a3c721977748 100644 --- a/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/WithFileStoreTable.scala +++ b/paimon-spark/paimon-spark-common/src/main/scala/org/apache/paimon/spark/commands/WithFileStoreTable.scala @@ -27,7 +27,7 @@ private[spark] trait WithFileStoreTable { def table: FileStoreTable - def root: Path = table.location() + def root: Path = table.tableDataPath() def withPrimaryKeys: Boolean = !table.primaryKeys().isEmpty 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 0360def685b6..32921a3b91f0 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,6 +161,7 @@ protected void foreachIndexReader(Consumer consumer) CoreOptions.PARTITION_GENERATE_LEGCY_NAME.defaultValue(), CoreOptions.FILE_SUFFIX_INCLUDE_COMPRESSION.defaultValue(), CoreOptions.FILE_COMPRESSION.defaultValue(), + null, null); Table table = fileSystemCatalog.getTable(Identifier.create("db", "T")); @@ -185,7 +186,7 @@ protected void foreachIndexReader(Consumer consumer) try (FileIndexFormat.Reader reader = FileIndexFormat.createReader( fileIO.newInputStream( - dataFilePathFactory.toPath(indexFiles.get(0))), + dataFilePathFactory.toPath(indexFiles.get(0), null)), tableSchema.logicalRowType())) { Optional fileIndexReader = reader.readColumnIndex("a").stream().findFirst(); diff --git a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkWriteITCase.java b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkWriteITCase.java index b0d5b380c1f2..4dfe1dffa972 100644 --- a/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkWriteITCase.java +++ b/paimon-spark/paimon-spark-ut/src/test/java/org/apache/paimon/spark/SparkWriteITCase.java @@ -307,7 +307,7 @@ public void testChangelogFilePrefixForPkTable() throws Exception { "CREATE TABLE T (a INT, b INT, c STRING) TBLPROPERTIES ('primary-key'='a', 'bucket' = '1', 'changelog-producer' = 'lookup')"); FileStoreTable table = getTable("T"); - Path tabLocation = table.location(); + Path tabLocation = table.tableDataPath(); FileIO fileIO = table.fileIO(); // default prefix "changelog-" @@ -334,7 +334,7 @@ public void testMarkDone() throws IOException { FileStoreTable table = getTable("T"); FileIO fileIO = table.fileIO(); - Path tabLocation = table.location(); + Path tabLocation = table.tableDataPath(); Assertions.assertTrue(fileIO.exists(new Path(tabLocation, "c=aa/_SUCCESS"))); } @@ -398,7 +398,7 @@ public void testChangelogFileSuffixName() throws Exception { + "'file.compression' = 'zstd')"); FileStoreTable table = getTable("T"); - Path tabLocation = table.location(); + Path tabLocation = table.tableDataPath(); FileIO fileIO = table.fileIO(); spark.sql("INSERT INTO T VALUES (1, 1, 'aa')"); diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonCDCSourceTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonCDCSourceTest.scala index e103429559ba..c720d3f76a6b 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonCDCSourceTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonCDCSourceTest.scala @@ -44,7 +44,7 @@ class PaimonCDCSourceTest extends PaimonSparkTestBase with StreamTest { spark.sql(s"INSERT INTO $tableName VALUES (2, 'v_2_new')") val table = loadTable(tableName) - val location = table.location().toString + val location = table.tableDataPath().toString val readStream = spark.readStream .format("paimon") @@ -95,7 +95,7 @@ class PaimonCDCSourceTest extends PaimonSparkTestBase with StreamTest { spark.sql(s"INSERT INTO $tableName VALUES (2, 'v_2_new')") val table = loadTable(tableName) - val location = table.location().toString + val location = table.tableDataPath().toString val readStream = spark.readStream .format("paimon") @@ -147,7 +147,7 @@ class PaimonCDCSourceTest extends PaimonSparkTestBase with StreamTest { |""".stripMargin) val table = loadTable(tableName) - val location = table.location().toString + val location = table.tableDataPath().toString // streaming write val inputData = MemoryStream[(Int, String)] diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonCommitTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonCommitTest.scala index c93e10ef90b9..c135e8159b18 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonCommitTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonCommitTest.scala @@ -37,7 +37,7 @@ class PaimonCommitTest extends PaimonSparkTestBase { |""".stripMargin) val table = loadTable("tb") - val location = table.location().toString + val location = table.tableDataPath().toString val _spark = spark import _spark.implicits._ diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonSinkTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonSinkTest.scala index 61bf5524942d..4657fa6ad2c3 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonSinkTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonSinkTest.scala @@ -43,7 +43,7 @@ class PaimonSinkTest extends PaimonSparkTestBase with StreamTest { |CREATE TABLE T (a INT, b STRING) |TBLPROPERTIES ('primary-key'='a', 'bucket'='3') |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(Int, String)] val stream = inputData @@ -87,7 +87,7 @@ class PaimonSinkTest extends PaimonSparkTestBase with StreamTest { |CREATE TABLE T (a INT, b STRING) |TBLPROPERTIES ('primary-key'='a', 'bucket'='3') |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(Int, String)] val stream = inputData @@ -127,7 +127,7 @@ class PaimonSinkTest extends PaimonSparkTestBase with StreamTest { spark.sql(s""" |CREATE TABLE T (city String, population Long) |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(Int, String)] val stream = inputData.toDS @@ -171,7 +171,7 @@ class PaimonSinkTest extends PaimonSparkTestBase with StreamTest { |CREATE TABLE T (a INT, b STRING) |TBLPROPERTIES ('primary-key'='a', 'bucket'='3') |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(Int, String)] intercept[RuntimeException] { @@ -195,7 +195,7 @@ class PaimonSinkTest extends PaimonSparkTestBase with StreamTest { |CREATE TABLE T (start Timestamp, stockId INT, avg_price DOUBLE) |TBLPROPERTIES ('bucket'='3', 'bucket-key'='stockId') |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(Long, Int, Double)] val data = inputData.toDS @@ -246,7 +246,7 @@ class PaimonSinkTest extends PaimonSparkTestBase with StreamTest { |CREATE TABLE T (a INT, b STRING) |TBLPROPERTIES ('primary-key'='a', 'bucket'='3') |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val date = Date.valueOf("2023-08-10") spark.sql("INSERT INTO T VALUES (1, '2023-08-09'), (2, '2023-08-09')") diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonSourceTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonSourceTest.scala index cd3fade60386..b933b2ed0b25 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonSourceTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/PaimonSourceTest.scala @@ -734,7 +734,7 @@ class PaimonSourceTest extends PaimonSparkTestBase with StreamTest { |TBLPROPERTIES ($primaryKeysProp 'bucket'='2', 'file.format'='parquet') |""".stripMargin) val table = loadTable(tableName) - val location = table.location().toString + val location = table.tableDataPath().toString val mergedData = scala.collection.mutable.TreeMap.empty[Int, String] val unmergedData = scala.collection.mutable.ArrayBuffer.empty[(Int, String)] diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/ScanHelperTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/ScanHelperTest.scala index a3223446f644..4997f65eaf0c 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/ScanHelperTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/ScanHelperTest.scala @@ -53,6 +53,7 @@ class ScanHelperTest extends PaimonSparkTestBase { new java.util.ArrayList[String](), null, FileSource.APPEND, + null, null) } @@ -89,6 +90,7 @@ class ScanHelperTest extends PaimonSparkTestBase { new java.util.ArrayList[String](), null, FileSource.APPEND, + null, null) ).asJava diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/AlterBranchProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/AlterBranchProcedureTest.scala index 316c36c40c56..6ec2141132f3 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/AlterBranchProcedureTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/AlterBranchProcedureTest.scala @@ -35,7 +35,7 @@ class AlterBranchProcedureTest extends PaimonSparkTestBase with StreamTest { |CREATE TABLE T (a INT, b STRING) |TBLPROPERTIES ('primary-key'='a', 'bucket'='3') |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(Int, String)] val stream = inputData diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/BranchProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/BranchProcedureTest.scala index 509001aabc6c..780557376d07 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/BranchProcedureTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/BranchProcedureTest.scala @@ -36,7 +36,7 @@ class BranchProcedureTest extends PaimonSparkTestBase with StreamTest { |CREATE TABLE T (a INT, b STRING) |TBLPROPERTIES ('primary-key'='a', 'bucket'='3') |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(Int, String)] val stream = inputData diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CompactProcedureTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CompactProcedureTestBase.scala index 31f78f61c20d..2f6c5ea09181 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CompactProcedureTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CompactProcedureTestBase.scala @@ -97,7 +97,7 @@ abstract class CompactProcedureTestBase extends PaimonSparkTestBase with StreamT |CREATE TABLE T (a INT, b INT) |TBLPROPERTIES ('bucket'='-1') |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(Int, Int)] val stream = inputData @@ -193,7 +193,7 @@ abstract class CompactProcedureTestBase extends PaimonSparkTestBase with StreamT |TBLPROPERTIES ('bucket'='-1') |PARTITIONED BY (p) |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(Int, Int, Int)] val stream = inputData @@ -350,7 +350,7 @@ abstract class CompactProcedureTestBase extends PaimonSparkTestBase with StreamT |CREATE TABLE T (a INT, b INT) |TBLPROPERTIES ('primary-key'='a,b', 'bucket'='1') |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(Int, Int)] val stream = inputData @@ -559,19 +559,19 @@ abstract class CompactProcedureTestBase extends PaimonSparkTestBase with StreamT Assertions.assertThat(lastSnapshotCommand(table).equals(CommitKind.COMPACT)).isTrue Assertions .assertThat( - fileIO.listStatus(new Path(table.location(), "dt=2024-01-01/hh=0/bucket-0")).length) + fileIO.listStatus(new Path(table.tableDataPath(), "dt=2024-01-01/hh=0/bucket-0")).length) .isEqualTo(2) Assertions .assertThat( - fileIO.listStatus(new Path(table.location(), "dt=2024-01-01/hh=1/bucket-0")).length) + fileIO.listStatus(new Path(table.tableDataPath(), "dt=2024-01-01/hh=1/bucket-0")).length) .isEqualTo(3) Assertions .assertThat( - fileIO.listStatus(new Path(table.location(), "dt=2024-01-02/hh=0/bucket-0")).length) + fileIO.listStatus(new Path(table.tableDataPath(), "dt=2024-01-02/hh=0/bucket-0")).length) .isEqualTo(2) Assertions .assertThat( - fileIO.listStatus(new Path(table.location(), "dt=2024-01-02/hh=1/bucket-0")).length) + fileIO.listStatus(new Path(table.tableDataPath(), "dt=2024-01-02/hh=1/bucket-0")).length) .isEqualTo(2) } diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CreateAndDeleteTagProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CreateAndDeleteTagProcedureTest.scala index 4a4c7ae215df..9268004a6073 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CreateAndDeleteTagProcedureTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CreateAndDeleteTagProcedureTest.scala @@ -37,7 +37,7 @@ class CreateAndDeleteTagProcedureTest extends PaimonSparkTestBase with StreamTes |CREATE TABLE T (a INT, b STRING) |TBLPROPERTIES ('primary-key'='a', 'bucket'='3') |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(Int, String)] val stream = inputData @@ -144,7 +144,7 @@ class CreateAndDeleteTagProcedureTest extends PaimonSparkTestBase with StreamTes |CREATE TABLE T (a INT, b STRING) |TBLPROPERTIES ('primary-key'='a', 'bucket'='3') |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(Int, String)] val stream = inputData diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CreateTagFromTimestampProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CreateTagFromTimestampProcedureTest.scala index 1da05843dcf6..c79feda3e9a4 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CreateTagFromTimestampProcedureTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/CreateTagFromTimestampProcedureTest.scala @@ -37,7 +37,7 @@ class CreateTagFromTimestampProcedureTest extends PaimonSparkTestBase with Strea |CREATE TABLE T (a INT, b STRING) |TBLPROPERTIES ('primary-key'='a', 'bucket'='3') |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(Int, String)] val stream = inputData @@ -114,7 +114,7 @@ class CreateTagFromTimestampProcedureTest extends PaimonSparkTestBase with Strea |CREATE TABLE T (a INT, b STRING) |TBLPROPERTIES ('primary-key'='a', 'bucket'='3') |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(Int, String)] val stream = inputData diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ExpirePartitionsProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ExpirePartitionsProcedureTest.scala index 9f0d23dc9379..40f2ec22e830 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ExpirePartitionsProcedureTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ExpirePartitionsProcedureTest.scala @@ -38,7 +38,7 @@ class ExpirePartitionsProcedureTest extends PaimonSparkTestBase with StreamTest |TBLPROPERTIES ('primary-key'='k,pt', 'bucket'='1') | PARTITIONED BY (pt) |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(String, String)] val stream = inputData @@ -90,7 +90,7 @@ class ExpirePartitionsProcedureTest extends PaimonSparkTestBase with StreamTest |TBLPROPERTIES ('primary-key'='k,pt,hm', 'bucket'='1') | PARTITIONED BY (pt,hm) |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(String, String, String)] val stream = inputData @@ -159,7 +159,7 @@ class ExpirePartitionsProcedureTest extends PaimonSparkTestBase with StreamTest |TBLPROPERTIES ('primary-key'='k,pt', 'bucket'='1') | PARTITIONED BY (pt) |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(String, String)] val stream = inputData @@ -215,7 +215,7 @@ class ExpirePartitionsProcedureTest extends PaimonSparkTestBase with StreamTest |TBLPROPERTIES ('primary-key'='k,pt', 'bucket'='1') | PARTITIONED BY (pt) |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(String, String)] val stream = inputData @@ -283,7 +283,7 @@ class ExpirePartitionsProcedureTest extends PaimonSparkTestBase with StreamTest |TBLPROPERTIES ('primary-key'='k,pt,hm', 'bucket'='1') | PARTITIONED BY (pt,hm) |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(String, String, String)] val stream = inputData @@ -349,7 +349,7 @@ class ExpirePartitionsProcedureTest extends PaimonSparkTestBase with StreamTest |TBLPROPERTIES ('primary-key'='k,pt', 'bucket'='1') | PARTITIONED BY (pt) |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(String, String)] val stream = inputData @@ -414,7 +414,7 @@ class ExpirePartitionsProcedureTest extends PaimonSparkTestBase with StreamTest |TBLPROPERTIES ('primary-key'='k,pt,hm', 'bucket'='1') | PARTITIONED BY (hm, pt) |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(String, String, String)] val stream = inputData @@ -484,7 +484,7 @@ class ExpirePartitionsProcedureTest extends PaimonSparkTestBase with StreamTest |TBLPROPERTIES ('primary-key'='k,pt,hm', 'bucket'='1') | PARTITIONED BY (pt,hm) |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(String, String, String)] val stream = inputData @@ -562,7 +562,7 @@ class ExpirePartitionsProcedureTest extends PaimonSparkTestBase with StreamTest |TBLPROPERTIES ('primary-key'='k,pt', 'bucket'='1', 'partition.expiration-max-num'='2') |PARTITIONED BY (pt) |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(String, String)] val stream = inputData diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ExpireSnapshotsProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ExpireSnapshotsProcedureTest.scala index da7be423108a..0be4f0a88e05 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ExpireSnapshotsProcedureTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/ExpireSnapshotsProcedureTest.scala @@ -41,7 +41,7 @@ class ExpireSnapshotsProcedureTest extends PaimonSparkTestBase with StreamTest { |CREATE TABLE T (a INT, b STRING) |TBLPROPERTIES ('primary-key'='a', 'bucket'='3') |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(Int, String)] val stream = inputData @@ -97,7 +97,7 @@ class ExpireSnapshotsProcedureTest extends PaimonSparkTestBase with StreamTest { |CREATE TABLE T (a INT, b STRING) |TBLPROPERTIES ('primary-key'='a', 'bucket'='3') |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(Int, String)] val stream = inputData diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/MarkPartitionDoneProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/MarkPartitionDoneProcedureTest.scala index 8abc7ddfdae8..9a3bf0c00d3a 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/MarkPartitionDoneProcedureTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/MarkPartitionDoneProcedureTest.scala @@ -48,11 +48,11 @@ class MarkPartitionDoneProcedureTest extends PaimonSparkTestBase { val table = loadTable("T") - val successPath1 = new Path(table.location, "day=2024-07-13/_SUCCESS") + val successPath1 = new Path(table.tableDataPath, "day=2024-07-13/_SUCCESS") val successFile1 = SuccessFile.safelyFromPath(table.fileIO, successPath1) assertThat(successFile1).isNotNull - val successPath2 = new Path(table.location, "day=2024-07-14/_SUCCESS") + val successPath2 = new Path(table.tableDataPath, "day=2024-07-14/_SUCCESS") val successFile2 = SuccessFile.safelyFromPath(table.fileIO, successPath2) assertThat(successFile2).isNotNull diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/RemoveOrphanFilesProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/RemoveOrphanFilesProcedureTest.scala index 3ffe7fba264f..8cb4a9cb2061 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/RemoveOrphanFilesProcedureTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/RemoveOrphanFilesProcedureTest.scala @@ -42,7 +42,7 @@ class RemoveOrphanFilesProcedureTest extends PaimonSparkTestBase { val table = loadTable("T") val fileIO = table.fileIO() - val tablePath = table.location() + val tablePath = table.tableDataPath() val orphanFile1 = new Path(tablePath, ORPHAN_FILE_1) val orphanFile2 = new Path(tablePath, ORPHAN_FILE_2) @@ -87,7 +87,7 @@ class RemoveOrphanFilesProcedureTest extends PaimonSparkTestBase { val table = loadTable("T") val fileIO = table.fileIO() - val tablePath = table.location() + val tablePath = table.tableDataPath() val orphanFile1 = new Path(tablePath, ORPHAN_FILE_1) val orphanFile2 = new Path(tablePath, ORPHAN_FILE_2) @@ -131,8 +131,8 @@ class RemoveOrphanFilesProcedureTest extends PaimonSparkTestBase { val table2 = loadTable("T2") val fileIO1 = table1.fileIO() val fileIO2 = table2.fileIO() - val tablePath1 = table1.location() - val tablePath2 = table2.location() + val tablePath1 = table1.tableDataPath() + val tablePath2 = table2.tableDataPath() val orphanFile11 = new Path(tablePath1, ORPHAN_FILE_1) val orphanFile12 = new Path(tablePath1, ORPHAN_FILE_2) @@ -183,7 +183,7 @@ class RemoveOrphanFilesProcedureTest extends PaimonSparkTestBase { val table = loadTable("T") val fileIO = table.fileIO() - val tablePath = table.location() + val tablePath = table.tableDataPath() val orphanFile1 = new Path(tablePath, ORPHAN_FILE_1) val orphanFile2 = new Path(tablePath, ORPHAN_FILE_2) diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/RollbackProcedureTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/RollbackProcedureTest.scala index 457c5ba513ec..28dfe08272d8 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/RollbackProcedureTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/procedure/RollbackProcedureTest.scala @@ -37,7 +37,7 @@ class RollbackProcedureTest extends PaimonSparkTestBase with StreamTest { |CREATE TABLE T (a INT, b STRING) |TBLPROPERTIES ('primary-key'='a', 'bucket'='3') |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(Int, String)] val stream = inputData @@ -103,7 +103,7 @@ class RollbackProcedureTest extends PaimonSparkTestBase with StreamTest { |CREATE TABLE T (a INT, b STRING) |TBLPROPERTIES ('primary-key'='a', 'bucket'='3') |""".stripMargin) - val location = loadTable("T").location().toString + val location = loadTable("T").tableDataPath().toString val inputData = MemoryStream[(Int, String)] val stream = inputData diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/AnalyzeTableTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/AnalyzeTableTestBase.scala index 4f8ccae22dd5..e07619e7d15e 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/AnalyzeTableTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/AnalyzeTableTestBase.scala @@ -368,7 +368,7 @@ abstract class AnalyzeTableTestBase extends PaimonSparkTestBase { |""".stripMargin) val table = loadTable("T") - val tableLocation = table.location() + val tableLocation = table.tableDataPath() val fileIO = table.fileIO() spark.sql(s"INSERT INTO T VALUES ('1', 'a')") diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala index 3ed2c98306fb..754b3f651280 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLTestBase.scala @@ -564,7 +564,7 @@ abstract class DDLTestBase extends PaimonSparkTestBase { sql("CREATE TABLE managed_tbl (id INT) USING paimon") val table = loadTable("managed_tbl") val fileIO = table.fileIO() - val tableLocation = table.location() + val tableLocation = table.tableDataPath() // drop managed table sql("DROP TABLE managed_tbl") diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala index 179020ae5b74..cb3ab0a86b73 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTestBase.scala @@ -336,11 +336,13 @@ abstract class DDLWithHiveCatalogTestBase extends PaimonHiveTestBase { assert(metastoreClient.listPartitions(dbName, tblName, 100).size() == 3) // check partitions in filesystem if (dataFilePathDir.isEmpty) { - assert(containsDir(table.location(), Array("pt=1", "pt=2", "pt=3"))) + assert(containsDir(table.tableDataPath(), Array("pt=1", "pt=2", "pt=3"))) } else { - assert(!containsDir(table.location(), Array("pt=1", "pt=2", "pt=3"))) + assert(!containsDir(table.tableDataPath(), Array("pt=1", "pt=2", "pt=3"))) assert( - containsDir(new Path(table.location(), "data"), Array("pt=1", "pt=2", "pt=3"))) + containsDir( + new Path(table.tableDataPath(), "data"), + Array("pt=1", "pt=2", "pt=3"))) } spark.sql(s"INSERT INTO $tblName VALUES (4, 3), (5, 4)") @@ -377,7 +379,7 @@ abstract class DDLWithHiveCatalogTestBase extends PaimonHiveTestBase { checkAnswer(spark.sql("SELECT * FROM external_tbl"), Row(1)) val table = loadTable("paimon_db", "external_tbl") val fileIO = table.fileIO() - val actualTbLocation = table.location() + val actualTbLocation = table.tableDataPath() assert(actualTbLocation.toString.split(':').apply(1).equals(expertTbLocation)) // drop external table @@ -392,7 +394,7 @@ abstract class DDLWithHiveCatalogTestBase extends PaimonHiveTestBase { // create managed table spark.sql(s"CREATE TABLE managed_tbl (id INT) USING paimon") - val managedTbLocation = loadTable("paimon_db", "managed_tbl").location() + val managedTbLocation = loadTable("paimon_db", "managed_tbl").tableDataPath() // drop managed table spark.sql("DROP TABLE managed_tbl") @@ -422,7 +424,7 @@ abstract class DDLWithHiveCatalogTestBase extends PaimonHiveTestBase { spark.sql( s"CREATE TABLE external_tbl (id INT) USING paimon LOCATION '$expertTbLocation'") spark.sql("INSERT INTO external_tbl VALUES (1)") - val actualTbLocation = loadTable("paimon_db", "external_tbl").location() + val actualTbLocation = loadTable("paimon_db", "external_tbl").tableDataPath() assert(actualTbLocation.toString.split(':').apply(1).equals(expertTbLocation)) // rename external table, location should not change @@ -435,7 +437,7 @@ abstract class DDLWithHiveCatalogTestBase extends PaimonHiveTestBase { // create managed table spark.sql(s"CREATE TABLE managed_tbl (id INT) USING paimon") spark.sql("INSERT INTO managed_tbl VALUES (1)") - val managedTbLocation = loadTable("paimon_db", "managed_tbl").location() + val managedTbLocation = loadTable("paimon_db", "managed_tbl").tableDataPath() // rename managed table, location should change spark.sql("ALTER TABLE managed_tbl RENAME TO managed_tbl_renamed") @@ -565,10 +567,11 @@ abstract class DDLWithHiveCatalogTestBase extends PaimonHiveTestBase { spark.sql("INSERT INTO managed_tbl VALUES (1)") checkAnswer(spark.sql("SELECT * FROM managed_tbl"), Row(1)) - val tablePath = loadTable("paimon_db", "managed_tbl").location().toString + val tablePath = loadTable("paimon_db", "managed_tbl").tableDataPath().toString spark.sql(s"CREATE TABLE external_tbl (id INT) USING paimon LOCATION '$tablePath'") checkAnswer(spark.sql("SELECT * FROM external_tbl"), Row(1)) - assert(loadTable("paimon_db", "external_tbl").location().toString.equals(tablePath)) + assert( + loadTable("paimon_db", "external_tbl").tableDataPath().toString.equals(tablePath)) } } } @@ -667,6 +670,6 @@ abstract class DDLWithHiveCatalogTestBase extends PaimonHiveTestBase { } def getActualTableLocation(dbName: String, tblName: String): String = { - loadTable(dbName, tblName).location().toString.split(':').apply(1) + loadTable(dbName, tblName).tableDataPath().toString.split(':').apply(1) } } diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DataFrameWriteTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DataFrameWriteTest.scala index edd092c85ce8..5671c67adae8 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DataFrameWriteTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DataFrameWriteTest.scala @@ -158,7 +158,7 @@ class DataFrameWriteTest extends PaimonSparkTestBase { |""".stripMargin) val paimonTable = loadTable("T") - val location = paimonTable.location().toString + val location = paimonTable.tableDataPath().toString val df1 = Seq((1, "a"), (2, "b")).toDF("a", "b") df1.write.format("paimon").mode("append").save(location) @@ -209,7 +209,7 @@ class DataFrameWriteTest extends PaimonSparkTestBase { |""".stripMargin) val paimonTable = loadTable("T") - val location = paimonTable.location().toString + val location = paimonTable.tableDataPath().toString val df1 = Seq((1, "a"), (2, "b")).toDF("a", "b") df1.write.format("paimon").mode("append").save(location) @@ -323,7 +323,7 @@ class DataFrameWriteTest extends PaimonSparkTestBase { |""".stripMargin) val paimonTable = loadTable("T") - val location = paimonTable.location().toString + val location = paimonTable.tableDataPath().toString val df1 = Seq((1, "2023-08-01"), (2, "2023-08-02")).toDF("a", "b") df1.write.format("paimon").mode("append").save(location) diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DescribeTableTestBase.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DescribeTableTestBase.scala index e020d5fbfeee..b1b8274ba7cd 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DescribeTableTestBase.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/DescribeTableTestBase.scala @@ -76,7 +76,7 @@ abstract class DescribeTableTestBase extends PaimonSparkTestBase { Assertions.assertEquals(information1("Provider"), "paimon") Assertions.assertEquals( information1("Location"), - loadTable(testDB, "s2").location().toString) + loadTable(testDB, "s2").tableDataPath().toString) // check table s2 partition info val error1 = intercept[Exception] { diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/LookupCompactionTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/LookupCompactionTest.scala index 66c0de02f01e..2ed4cc924304 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/LookupCompactionTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/LookupCompactionTest.scala @@ -44,7 +44,7 @@ class LookupCompactionTest extends PaimonSparkTestBase { |""".stripMargin) val table = loadTable("T") - val tabLocation = table.location() + val tabLocation = table.tableDataPath() val fileIO = table.fileIO() // First insert, file is upgraded to the max level when compaction, no need rewrite diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonOptionTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonOptionTest.scala index 44df3e54ca72..809aee7896cf 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonOptionTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonOptionTest.scala @@ -33,7 +33,7 @@ class PaimonOptionTest extends PaimonSparkTestBase { sql("CREATE TABLE T (id INT)") val table = loadTable("T") // check options in schema file directly - val fileStoreTable = FileStoreTableFactory.create(table.fileIO(), table.location()) + val fileStoreTable = FileStoreTableFactory.create(table.fileIO(), table.tableDataPath()) Assertions.assertNull(fileStoreTable.options().get("scan.snapshot-id")) } } @@ -48,7 +48,7 @@ class PaimonOptionTest extends PaimonSparkTestBase { .saveAsTable("T") val table = loadTable("T") // check options in schema file directly - val fileStoreTable = FileStoreTableFactory.create(table.fileIO(), table.location()) + val fileStoreTable = FileStoreTableFactory.create(table.fileIO(), table.tableDataPath()) Assertions.assertNull(fileStoreTable.options().get("scan.snapshot-id")) } } @@ -63,7 +63,7 @@ class PaimonOptionTest extends PaimonSparkTestBase { // query with mutable option withSparkSQLConf("spark.paimon.scan.snapshot-id" -> "1") { checkAnswer(sql("SELECT * FROM T ORDER BY id"), Row(1)) - checkAnswer(spark.read.format("paimon").load(table.location().toString), Row(1)) + checkAnswer(spark.read.format("paimon").load(table.tableDataPath().toString), Row(1)) } // query with immutable option @@ -72,7 +72,7 @@ class PaimonOptionTest extends PaimonSparkTestBase { sql("SELECT * FROM T ORDER BY id") } assertThrows[UnsupportedOperationException] { - spark.read.format("paimon").load(table.location().toString) + spark.read.format("paimon").load(table.tableDataPath().toString) } } } @@ -87,13 +87,13 @@ class PaimonOptionTest extends PaimonSparkTestBase { // query with global options withSparkSQLConf("spark.paimon.scan.snapshot-id" -> "1") { checkAnswer(sql("SELECT * FROM T ORDER BY id"), Row(1)) - checkAnswer(spark.read.format("paimon").load(table.location().toString), Row(1)) + checkAnswer(spark.read.format("paimon").load(table.tableDataPath().toString), Row(1)) } // query with table options withSparkSQLConf("spark.paimon.*.*.T.scan.snapshot-id" -> "1") { checkAnswer(sql("SELECT * FROM T ORDER BY id"), Row(1)) - checkAnswer(spark.read.format("paimon").load(table.location().toString), Row(1)) + checkAnswer(spark.read.format("paimon").load(table.tableDataPath().toString), Row(1)) } // query with both global and table options @@ -102,7 +102,7 @@ class PaimonOptionTest extends PaimonSparkTestBase { "spark.paimon.*.*.T.scan.snapshot-id" -> "2") { checkAnswer(sql("SELECT * FROM T ORDER BY id"), Row(1) :: Row(2) :: Nil) checkAnswer( - spark.read.format("paimon").load(table.location().toString), + spark.read.format("paimon").load(table.tableDataPath().toString), Row(1) :: Row(2) :: Nil) } } @@ -127,8 +127,8 @@ class PaimonOptionTest extends PaimonSparkTestBase { checkAnswer( spark.read .format("paimon") - .load(table1.location().toString) - .join(spark.read.format("paimon").load(table2.location().toString), "id"), + .load(table1.tableDataPath().toString) + .join(spark.read.format("paimon").load(table2.tableDataPath().toString), "id"), Row(1) ) } @@ -139,8 +139,8 @@ class PaimonOptionTest extends PaimonSparkTestBase { checkAnswer( spark.read .format("paimon") - .load(table1.location().toString) - .join(spark.read.format("paimon").load(table2.location().toString), "id"), + .load(table1.tableDataPath().toString) + .join(spark.read.format("paimon").load(table2.tableDataPath().toString), "id"), Row(1) ) } @@ -155,8 +155,8 @@ class PaimonOptionTest extends PaimonSparkTestBase { checkAnswer( spark.read .format("paimon") - .load(table1.location().toString) - .join(spark.read.format("paimon").load(table2.location().toString), "id"), + .load(table1.tableDataPath().toString) + .join(spark.read.format("paimon").load(table2.tableDataPath().toString), "id"), Row(1) :: Row(2) :: Nil ) } @@ -168,8 +168,8 @@ class PaimonOptionTest extends PaimonSparkTestBase { checkAnswer( spark.read .format("paimon") - .load(table1.location().toString) - .join(spark.read.format("paimon").load(table2.location().toString), "id"), + .load(table1.tableDataPath().toString) + .join(spark.read.format("paimon").load(table2.tableDataPath().toString), "id"), Row(1) ) } @@ -181,8 +181,8 @@ class PaimonOptionTest extends PaimonSparkTestBase { checkAnswer( spark.read .format("paimon") - .load(table1.location().toString) - .join(spark.read.format("paimon").load(table2.location().toString), "id"), + .load(table1.tableDataPath().toString) + .join(spark.read.format("paimon").load(table2.tableDataPath().toString), "id"), Row(1) ) } @@ -196,8 +196,8 @@ class PaimonOptionTest extends PaimonSparkTestBase { checkAnswer( spark.read .format("paimon") - .load(table1.location().toString) - .join(spark.read.format("paimon").load(table2.location().toString), "id"), + .load(table1.tableDataPath().toString) + .join(spark.read.format("paimon").load(table2.tableDataPath().toString), "id"), Row(1) :: Row(2) :: Nil ) } diff --git a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonQueryTest.scala b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonQueryTest.scala index 08f5275f01b5..c9e09e93986d 100644 --- a/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonQueryTest.scala +++ b/paimon-spark/paimon-spark-ut/src/test/scala/org/apache/paimon/spark/sql/PaimonQueryTest.scala @@ -49,7 +49,7 @@ class PaimonQueryTest extends PaimonSparkTestBase { |VALUES (1, 'x1'), (2, 'x3'), (3, 'x3'), (4, 'x4'), (5, 'x5') |""".stripMargin) - val location = loadTable("T").location().toUri.toString + val location = loadTable("T").tableDataPath().toUri.toString val res = spark.sql( s""" |SELECT SUM(cnt) @@ -129,7 +129,7 @@ class PaimonQueryTest extends PaimonSparkTestBase { |TBLPROPERTIES ('file.format'='$fileFormat' $bucketProp) |""".stripMargin) - val location = loadTable("T").location().toUri.toString + val location = loadTable("T").tableDataPath().toUri.toString spark.sql("INSERT INTO T VALUES (1, 'x1'), (3, 'x3')") @@ -176,7 +176,7 @@ class PaimonQueryTest extends PaimonSparkTestBase { |TBLPROPERTIES ('file.format'='$fileFormat' $bucketProp) |""".stripMargin) - val location = loadTable("T").location().toUri.toString + val location = loadTable("T").tableDataPath().toUri.toString spark.sql("INSERT INTO T VALUES (1, 'x1', '2024'), (3, 'x3', '2024')") @@ -373,7 +373,7 @@ class PaimonQueryTest extends PaimonSparkTestBase { partitions: Seq[String], partitionFilter: java.util.Map[String, String]): Array[String] = { val paimonTable = loadTable(tableName) - val location = paimonTable.location() + val location = paimonTable.tableDataPath() val files = paimonTable .newSnapshotReader()