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()