diff --git a/docs/layouts/shortcodes/generated/core_configuration.html b/docs/layouts/shortcodes/generated/core_configuration.html
index 52645cf8cc78e..9cd26567e2eee 100644
--- a/docs/layouts/shortcodes/generated/core_configuration.html
+++ b/docs/layouts/shortcodes/generated/core_configuration.html
@@ -731,5 +731,11 @@
Integer |
The bytes of types (CHAR, VARCHAR, BINARY, VARBINARY) devote to the zorder sort. |
+
+ branch |
+ main |
+ String |
+ Specify branch name. |
+
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 19554296cd534..b150d4c16d099 100644
--- a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java
+++ b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java
@@ -101,6 +101,9 @@ public class CoreOptions implements Serializable {
.noDefaultValue()
.withDescription("The file path of this table in the filesystem.");
+ public static final ConfigOption BRANCH =
+ key("branch").stringType().defaultValue("main").withDescription("Specify branch name.");
+
public static final ConfigOption FILE_FORMAT =
key("file.format")
.enumType(FileFormatType.class)
@@ -1114,6 +1117,17 @@ public Path path() {
return path(options.toMap());
}
+ public String branch() {
+ return branch(options.toMap());
+ }
+
+ public static String branch(Map options) {
+ if (options.containsKey(BRANCH.key())) {
+ return options.get(BRANCH.key());
+ }
+ return BRANCH.defaultValue();
+ }
+
public static Path path(Map options) {
return new Path(options.get(PATH.key()));
}
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 87cc4e65c5442..8a99b990589d7 100644
--- a/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java
+++ b/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java
@@ -104,7 +104,7 @@ public FileStorePathFactory pathFactory() {
@Override
public SnapshotManager snapshotManager() {
- return new SnapshotManager(fileIO, options.path());
+ return new SnapshotManager(fileIO, options.path(), options.branch());
}
@Override
diff --git a/paimon-core/src/main/java/org/apache/paimon/AppendOnlyFileStore.java b/paimon-core/src/main/java/org/apache/paimon/AppendOnlyFileStore.java
index 0d546e2154d55..6ff2e01d15afe 100644
--- a/paimon-core/src/main/java/org/apache/paimon/AppendOnlyFileStore.java
+++ b/paimon-core/src/main/java/org/apache/paimon/AppendOnlyFileStore.java
@@ -144,8 +144,7 @@ public void pushdown(Predicate predicate) {
manifestListFactory(forWrite),
options.bucket(),
forWrite,
- options.scanManifestParallelism(),
- branchName);
+ options.scanManifestParallelism());
}
@Override
diff --git a/paimon-core/src/main/java/org/apache/paimon/FileStore.java b/paimon-core/src/main/java/org/apache/paimon/FileStore.java
index cd38d20611fcf..30e9a18ee2f01 100644
--- a/paimon-core/src/main/java/org/apache/paimon/FileStore.java
+++ b/paimon-core/src/main/java/org/apache/paimon/FileStore.java
@@ -63,8 +63,6 @@ public interface FileStore extends Serializable {
FileStoreScan newScan();
- FileStoreScan newScan(String branchName);
-
ManifestList.Factory manifestListFactory();
ManifestFile.Factory manifestFileFactory();
diff --git a/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java b/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java
index d80bd6d3979de..167211cbd9ecf 100644
--- a/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java
+++ b/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java
@@ -39,6 +39,7 @@
import org.apache.paimon.table.BucketMode;
import org.apache.paimon.table.CatalogEnvironment;
import org.apache.paimon.types.RowType;
+import org.apache.paimon.utils.BranchManager;
import org.apache.paimon.utils.FileStorePathFactory;
import org.apache.paimon.utils.KeyComparatorSupplier;
import org.apache.paimon.utils.UserDefinedSeqComparator;
@@ -55,7 +56,6 @@
import static org.apache.paimon.predicate.PredicateBuilder.and;
import static org.apache.paimon.predicate.PredicateBuilder.pickTransformFieldMapping;
import static org.apache.paimon.predicate.PredicateBuilder.splitAnd;
-import static org.apache.paimon.utils.BranchManager.DEFAULT_MAIN_BRANCH;
import static org.apache.paimon.utils.Preconditions.checkArgument;
/** {@link FileStore} for querying and updating {@link KeyValue}s. */
@@ -111,11 +111,7 @@ public BucketMode bucketMode() {
@Override
public KeyValueFileStoreScan newScan() {
- return newScan(DEFAULT_MAIN_BRANCH);
- }
-
- public KeyValueFileStoreScan newScan(String branchName) {
- return newScan(false, branchName);
+ return newScan(false, BranchManager.DEFAULT_MAIN_BRANCH);
}
@Override
@@ -173,7 +169,7 @@ public KeyValueFileStoreWrite newWrite(String commitUser, ManifestCacheFilter ma
pathFactory(),
format2PathFactory(),
snapshotManager(),
- newScan(true, DEFAULT_MAIN_BRANCH).withManifestCacheFilter(manifestFilter),
+ newScan(true, BranchManager.DEFAULT_MAIN_BRANCH).withManifestCacheFilter(manifestFilter),
indexFactory,
deletionVectorsMaintainerFactory,
options,
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 c69a72b0db6a5..3e8927400a271 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
@@ -36,6 +36,7 @@
import org.apache.paimon.table.Table;
import org.apache.paimon.table.sink.BatchWriteBuilder;
import org.apache.paimon.table.system.SystemTableLoader;
+import org.apache.paimon.utils.BranchManager;
import org.apache.paimon.utils.StringUtils;
import javax.annotation.Nullable;
@@ -66,6 +67,7 @@ public abstract class AbstractCatalog implements Catalog {
protected final FileIO fileIO;
protected final Map tableDefaultOptions;
protected final Options catalogOptions;
+ protected final String branchName;
@Nullable protected final LineageMetaFactory lineageMetaFactory;
@@ -74,6 +76,7 @@ protected AbstractCatalog(FileIO fileIO) {
this.lineageMetaFactory = null;
this.tableDefaultOptions = new HashMap<>();
this.catalogOptions = new Options();
+ branchName = BranchManager.DEFAULT_MAIN_BRANCH;
}
protected AbstractCatalog(FileIO fileIO, Options options) {
@@ -83,6 +86,7 @@ protected AbstractCatalog(FileIO fileIO, Options options) {
this.tableDefaultOptions =
convertToPropertiesPrefixKey(options.toMap(), TABLE_DEFAULT_OPTION_PREFIX);
this.catalogOptions = options;
+ this.branchName = options.get(CoreOptions.BRANCH);
if (lockEnabled()) {
checkArgument(options.contains(LOCK_TYPE), "No lock type when lock is enabled.");
@@ -320,7 +324,12 @@ public Table getTable(Identifier identifier) throws TableNotExistException {
}
return table;
} else {
- return getDataTable(identifier);
+ Table table = getDataTable(identifier);
+ // Override branch option
+ if (!branchName.equals(BranchManager.DEFAULT_MAIN_BRANCH)) {
+ table.options().put(CoreOptions.BRANCH.key(), branchName);
+ }
+ return table;
}
}
diff --git a/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java b/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java
index e71c92dc4007f..0f7411093a91f 100644
--- a/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java
+++ b/paimon-core/src/main/java/org/apache/paimon/catalog/FileSystemCatalog.java
@@ -118,7 +118,7 @@ public boolean tableExists(Identifier identifier) {
}
private boolean tableExists(Path tablePath) {
- return new SchemaManager(fileIO, tablePath).listAllIds().size() > 0;
+ return new SchemaManager(fileIO, tablePath, branchName).listAllIds().size() > 0;
}
@Override
@@ -152,7 +152,7 @@ private SchemaManager schemaManager(Identifier identifier) {
new RuntimeException(
"No lock context when lock is enabled."))))
.orElse(null);
- return new SchemaManager(fileIO, path)
+ return new SchemaManager(fileIO, path, branchName)
.withLock(catalogLock == null ? null : Lock.fromCatalog(catalogLock, identifier));
}
diff --git a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java
index 689a93ee91f74..670c9f614bde2 100644
--- a/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java
+++ b/paimon-core/src/main/java/org/apache/paimon/jdbc/JdbcCatalog.java
@@ -291,7 +291,7 @@ protected void renameTableImpl(Identifier fromTable, Identifier toTable) {
updateTable(connections, catalogKey, fromTable, toTable);
Path fromPath = getDataTableLocation(fromTable);
- if (new SchemaManager(fileIO, fromPath).listAllIds().size() > 0) {
+ if (new SchemaManager(fileIO, fromPath, branchName).listAllIds().size() > 0) {
// Rename the file system's table directory. Maintain consistency between tables in
// the file system and tables in the Hive Metastore.
Path toPath = getDataTableLocation(toTable);
@@ -326,7 +326,7 @@ protected TableSchema getDataTableSchema(Identifier identifier) throws TableNotE
throw new TableNotExistException(identifier);
}
Path tableLocation = getDataTableLocation(identifier);
- return new SchemaManager(fileIO, tableLocation)
+ return new SchemaManager(fileIO, tableLocation, branchName)
.latest()
.orElseThrow(
() -> new RuntimeException("There is no paimon table in " + tableLocation));
@@ -369,7 +369,7 @@ public void close() throws Exception {
}
private SchemaManager getSchemaManager(Identifier identifier) {
- return new SchemaManager(fileIO, getDataTableLocation(identifier))
+ return new SchemaManager(fileIO, getDataTableLocation(identifier), branchName)
.withLock(lock(identifier));
}
diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java b/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java
index 52983f4b6e2b7..fbbd63d3fd633 100644
--- a/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java
+++ b/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreScan.java
@@ -74,7 +74,6 @@ public abstract class AbstractFileStoreScan implements FileStoreScan {
private final SchemaManager schemaManager;
private final TableSchema schema;
protected final ScanBucketFilter bucketKeyFilter;
- private final String branchName;
private PartitionPredicate partitionFilter;
private Snapshot specifiedSnapshot = null;
@@ -97,8 +96,7 @@ public AbstractFileStoreScan(
ManifestList.Factory manifestListFactory,
int numOfBuckets,
boolean checkNumOfBuckets,
- Integer scanManifestParallelism,
- String branchName) {
+ Integer scanManifestParallelism) {
this.partitionType = partitionType;
this.bucketKeyFilter = bucketKeyFilter;
this.snapshotManager = snapshotManager;
@@ -110,7 +108,6 @@ public AbstractFileStoreScan(
this.checkNumOfBuckets = checkNumOfBuckets;
this.tableSchemas = new ConcurrentHashMap<>();
this.scanManifestParallelism = scanManifestParallelism;
- this.branchName = branchName;
}
@Override
@@ -367,7 +364,7 @@ private Pair> readManifests() {
if (manifests == null) {
snapshot =
specifiedSnapshot == null
- ? snapshotManager.latestSnapshot(branchName)
+ ? snapshotManager.latestSnapshot()
: specifiedSnapshot;
if (snapshot == null) {
manifests = Collections.emptyList();
diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreScan.java b/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreScan.java
index 866c87d75f667..ccbdb8d67542b 100644
--- a/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreScan.java
+++ b/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreScan.java
@@ -50,8 +50,7 @@ public AppendOnlyFileStoreScan(
ManifestList.Factory manifestListFactory,
int numOfBuckets,
boolean checkNumOfBuckets,
- Integer scanManifestParallelism,
- String branchName) {
+ Integer scanManifestParallelism) {
super(
partitionType,
bucketFilter,
@@ -62,8 +61,7 @@ public AppendOnlyFileStoreScan(
manifestListFactory,
numOfBuckets,
checkNumOfBuckets,
- scanManifestParallelism,
- branchName);
+ scanManifestParallelism);
this.fieldStatsConverters =
new FieldStatsConverters(sid -> scanTableSchema(sid).fields(), schema.id());
}
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 46cfceb145c7e..9645a4ffd9f4a 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
@@ -243,7 +243,7 @@ public void commit(ManifestCommittable committable, Map properti
// we can skip conflict checking in tryCommit method.
// This optimization is mainly used to decrease the number of times we read from
// files.
- latestSnapshot = snapshotManager.latestSnapshot(branchName);
+ latestSnapshot = snapshotManager.latestSnapshot();
if (latestSnapshot != null) {
// it is possible that some partitions only have compact changes,
// so we need to contain all changes
@@ -639,7 +639,7 @@ private int tryCommit(
@Nullable String statsFileName) {
int cnt = 0;
while (true) {
- Snapshot latestSnapshot = snapshotManager.latestSnapshot(branchName);
+ Snapshot latestSnapshot = snapshotManager.latestSnapshot();
cnt++;
if (tryCommitOnce(
tableFiles,
@@ -739,7 +739,7 @@ public boolean tryCommitOnce(
Path newSnapshotPath =
branchName.equals(DEFAULT_MAIN_BRANCH)
? snapshotManager.snapshotPath(newSnapshotId)
- : snapshotManager.branchSnapshotPath(branchName, newSnapshotId);
+ : snapshotManager.snapshotPath(branchName, newSnapshotId);
if (LOG.isDebugEnabled()) {
LOG.debug("Ready to commit table files to snapshot #" + newSnapshotId);
@@ -821,7 +821,7 @@ public boolean tryCommitOnce(
newIndexManifest = indexManifest;
}
- long latestSchemaId = schemaManager.latest(branchName).get().id();
+ long latestSchemaId = schemaManager.latest().get().id();
// write new stats or inherit from the previous snapshot
String statsFileName = null;
diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreScan.java b/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreScan.java
index 0f34cac5a138e..fc1de0a4d7e0c 100644
--- a/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreScan.java
+++ b/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreScan.java
@@ -70,8 +70,7 @@ public KeyValueFileStoreScan(
manifestListFactory,
numOfBuckets,
checkNumOfBuckets,
- scanManifestParallelism,
- branchName);
+ scanManifestParallelism);
this.fieldKeyStatsConverters =
new FieldStatsConverters(
sid -> keyValueFieldsExtractor.keyFields(scanTableSchema(sid)),
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 a6d274688aea0..0fe576b34b633 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
@@ -44,6 +44,7 @@
import org.apache.paimon.types.RowType;
import org.apache.paimon.utils.JsonSerdeUtil;
import org.apache.paimon.utils.Preconditions;
+import org.apache.paimon.utils.StringUtils;
import javax.annotation.Nullable;
import javax.annotation.concurrent.ThreadSafe;
@@ -77,12 +78,18 @@ public class SchemaManager implements Serializable {
private final FileIO fileIO;
private final Path tableRoot;
-
@Nullable private transient Lock lock;
+ private final String branch;
public SchemaManager(FileIO fileIO, Path tableRoot) {
+ this(fileIO, tableRoot, DEFAULT_MAIN_BRANCH);
+ }
+
+ /** Specify the default branch for data writing. */
+ public SchemaManager(FileIO fileIO, Path tableRoot, String branch) {
this.fileIO = fileIO;
this.tableRoot = tableRoot;
+ this.branch = StringUtils.isBlank(branch) ? DEFAULT_MAIN_BRANCH : branch;
}
public SchemaManager withLock(@Nullable Lock lock) {
@@ -90,44 +97,53 @@ public SchemaManager withLock(@Nullable Lock lock) {
return this;
}
- /** @return latest schema. */
public Optional latest() {
- return latest(DEFAULT_MAIN_BRANCH);
+ return latest(branch);
}
- public Optional latest(String branchName) {
- Path directoryPath =
- branchName.equals(DEFAULT_MAIN_BRANCH)
- ? schemaDirectory()
- : branchSchemaDirectory(branchName);
+ public Optional latest(String branch) {
try {
- return listVersionedFiles(fileIO, directoryPath, SCHEMA_PREFIX)
+ return listVersionedFiles(fileIO, schemaDirectory(branch), SCHEMA_PREFIX)
.reduce(Math::max)
- .map(this::schema);
+ .map(id -> schema(id, branch));
} catch (IOException e) {
throw new UncheckedIOException(e);
}
}
- /** List all schema. */
public List listAll() {
- return listAllIds().stream().map(this::schema).collect(Collectors.toList());
+ return listAll(branch);
+ }
+
+ public List listAll(String branch) {
+ return listAllIds(branch).stream()
+ .map(id -> schema(id, branch))
+ .collect(Collectors.toList());
}
- /** List all schema IDs. */
public List listAllIds() {
+ return listAllIds(branch);
+ }
+
+ /** List all schema IDs. */
+ public List listAllIds(String branch) {
try {
- return listVersionedFiles(fileIO, schemaDirectory(), SCHEMA_PREFIX)
+ return listVersionedFiles(fileIO, schemaDirectory(branch), SCHEMA_PREFIX)
.collect(Collectors.toList());
} catch (IOException e) {
throw new UncheckedIOException(e);
}
}
- /** Create a new schema from {@link Schema}. */
public TableSchema createTable(Schema schema) throws Exception {
+ return createTable(branch, schema);
+ }
+
+ /** Create a new schema from {@link Schema}. */
+ public TableSchema createTable(String branch, Schema schema) throws Exception {
while (true) {
- latest().ifPresent(
+ latest(branch)
+ .ifPresent(
latest -> {
throw new IllegalStateException(
"Schema in filesystem exists, please use updating,"
@@ -151,25 +167,36 @@ public TableSchema createTable(Schema schema) throws Exception {
options,
schema.comment());
- boolean success = commit(newSchema);
+ boolean success = commit(branch, newSchema);
if (success) {
return newSchema;
}
}
}
- /** Update {@link SchemaChange}s. */
public TableSchema commitChanges(SchemaChange... changes) throws Exception {
- return commitChanges(Arrays.asList(changes));
+ return commitChanges(branch, changes);
}
/** Update {@link SchemaChange}s. */
+ public TableSchema commitChanges(String branch, SchemaChange... changes) throws Exception {
+ return commitChanges(branch, Arrays.asList(changes));
+ }
+
public TableSchema commitChanges(List changes)
+ throws Catalog.ColumnAlreadyExistException, Catalog.TableNotExistException,
+ Catalog.ColumnNotExistException {
+ return commitChanges(branch, changes);
+ }
+
+ /** Update {@link SchemaChange}s. */
+ public TableSchema commitChanges(String branch, List changes)
throws Catalog.TableNotExistException, Catalog.ColumnAlreadyExistException,
Catalog.ColumnNotExistException {
while (true) {
TableSchema schema =
- latest().orElseThrow(
+ latest(branch)
+ .orElseThrow(
() ->
new Catalog.TableNotExistException(
fromPath(tableRoot.toString(), true)));
@@ -361,7 +388,7 @@ public TableSchema commitChanges(List changes)
newComment);
try {
- boolean success = commit(newSchema);
+ boolean success = commit(branch, newSchema);
if (success) {
return newSchema;
}
@@ -372,8 +399,13 @@ public TableSchema commitChanges(List changes)
}
public boolean mergeSchema(RowType rowType, boolean allowExplicitCast) {
+ return mergeSchema(branch, rowType, allowExplicitCast);
+ }
+
+ public boolean mergeSchema(String branch, RowType rowType, boolean allowExplicitCast) {
TableSchema current =
- latest().orElseThrow(
+ latest(branch)
+ .orElseThrow(
() ->
new RuntimeException(
"It requires that the current schema to exist when calling 'mergeSchema'"));
@@ -382,7 +414,7 @@ public boolean mergeSchema(RowType rowType, boolean allowExplicitCast) {
return false;
} else {
try {
- return commit(update);
+ return commit(branch, update);
} catch (Exception e) {
throw new RuntimeException("Failed to commit the schema.", e);
}
@@ -455,9 +487,13 @@ private void updateColumn(
@VisibleForTesting
boolean commit(TableSchema newSchema) throws Exception {
- SchemaValidation.validateTableSchema(newSchema);
+ return commit(branch, newSchema);
+ }
- Path schemaPath = toSchemaPath(newSchema.id());
+ @VisibleForTesting
+ boolean commit(String branchName, TableSchema newSchema) throws Exception {
+ SchemaValidation.validateTableSchema(newSchema);
+ Path schemaPath = toSchemaPath(branchName, newSchema.id());
Callable callable = () -> fileIO.writeFileUtf8(schemaPath, newSchema.toString());
if (lock == null) {
return callable.call();
@@ -465,10 +501,15 @@ boolean commit(TableSchema newSchema) throws Exception {
return lock.runWithLock(callable);
}
- /** Read schema for schema id. */
public TableSchema schema(long id) {
+ return schema(id, branch);
+ }
+
+ /** Read schema for schema id. */
+ public TableSchema schema(long id, String branch) {
try {
- return JsonSerdeUtil.fromJson(fileIO.readFileUtf8(toSchemaPath(id)), TableSchema.class);
+ return JsonSerdeUtil.fromJson(
+ fileIO.readFileUtf8(toSchemaPath(branch, id)), TableSchema.class);
} catch (IOException e) {
throw new UncheckedIOException(e);
}
@@ -486,18 +527,34 @@ private Path schemaDirectory() {
return new Path(tableRoot + "/schema");
}
+ public Path schemaDirectory(String branchName) {
+ return branchName.equals(DEFAULT_MAIN_BRANCH)
+ ? schemaDirectory()
+ : new Path(getBranchPath(tableRoot, branchName) + "/schema");
+ }
+
@VisibleForTesting
public Path toSchemaPath(long id) {
return new Path(tableRoot + "/schema/" + SCHEMA_PREFIX + id);
}
- public Path branchSchemaDirectory(String branchName) {
- return new Path(getBranchPath(tableRoot, branchName) + "/schema");
+ public Path toSchemaPath(String branchName, long schemaId) {
+ return branchName.equals(DEFAULT_MAIN_BRANCH)
+ ? toSchemaPath(schemaId)
+ : new Path(
+ getBranchPath(tableRoot, branchName)
+ + "/schema/"
+ + SCHEMA_PREFIX
+ + schemaId);
}
- public Path branchSchemaPath(String branchName, long schemaId) {
- return new Path(
- getBranchPath(tableRoot, branchName) + "/schema/" + SCHEMA_PREFIX + schemaId);
+ /**
+ * Delete schema with specific id.
+ *
+ * @param schemaId the schema id to delete.
+ */
+ public void deleteSchema(long schemaId) {
+ deleteSchema(branch, schemaId);
}
/**
@@ -505,8 +562,8 @@ public Path branchSchemaPath(String branchName, long schemaId) {
*
* @param schemaId the schema id to delete.
*/
- public void deleteSchema(long schemaId) {
- fileIO.deleteQuietly(toSchemaPath(schemaId));
+ public void deleteSchema(String branch, long schemaId) {
+ fileIO.deleteQuietly(toSchemaPath(branch, schemaId));
}
public static void checkAlterTableOption(String key) {
diff --git a/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java b/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java
index 398aa98d128d7..1438e81de2efb 100644
--- a/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java
+++ b/paimon-core/src/main/java/org/apache/paimon/schema/TableSchema.java
@@ -104,7 +104,8 @@ public TableSchema(
this.highestFieldId = highestFieldId;
this.partitionKeys = partitionKeys;
this.primaryKeys = primaryKeys;
- this.options = Collections.unmodifiableMap(options);
+ Objects.requireNonNull(options);
+ this.options = options;
this.comment = comment;
this.timeMillis = timeMillis;
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 39b368982ae10..740c593942744 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
@@ -141,7 +141,7 @@ public SnapshotReader newSnapshotReader() {
@Override
public SnapshotReader newSnapshotReader(String branchName) {
return new SnapshotReaderImpl(
- store().newScan(branchName),
+ store().newScan(),
tableSchema,
coreOptions(),
snapshotManager(),
@@ -244,7 +244,8 @@ private FileStoreTable copyInternal(Map dynamicOptions, boolean
@Override
public FileStoreTable copyWithLatestSchema() {
Map options = tableSchema.options();
- SchemaManager schemaManager = new SchemaManager(fileIO(), location());
+ SchemaManager schemaManager =
+ new SchemaManager(fileIO(), location(), CoreOptions.branch(options()));
Optional optionalLatestSchema = schemaManager.latest();
if (optionalLatestSchema.isPresent()) {
TableSchema newTableSchema = optionalLatestSchema.get();
@@ -257,7 +258,7 @@ public FileStoreTable copyWithLatestSchema() {
}
protected SchemaManager schemaManager() {
- return new SchemaManager(fileIO(), path);
+ return new SchemaManager(fileIO(), path, CoreOptions.branch(options()));
}
@Override
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 6382535858676..e941a89e01b3f 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
@@ -53,8 +53,9 @@ public static FileStoreTable create(FileIO fileIO, Path path) {
public static FileStoreTable create(FileIO fileIO, Options options) {
Path tablePath = CoreOptions.path(options);
+ String branchName = CoreOptions.branch(options.toMap());
TableSchema tableSchema =
- new SchemaManager(fileIO, tablePath)
+ new SchemaManager(fileIO, tablePath, branchName)
.latest()
.orElseThrow(
() ->
diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/BranchManager.java b/paimon-core/src/main/java/org/apache/paimon/utils/BranchManager.java
index 6564bd4e56dcd..e7209d11c96e4 100644
--- a/paimon-core/src/main/java/org/apache/paimon/utils/BranchManager.java
+++ b/paimon-core/src/main/java/org/apache/paimon/utils/BranchManager.java
@@ -99,13 +99,13 @@ public void createBranch(String branchName, String tagName) {
try {
// Copy the corresponding tag, snapshot and schema files into the branch directory
fileIO.copyFileUtf8(
- tagManager.tagPath(tagName), tagManager.branchTagPath(branchName, tagName));
+ tagManager.tagPath(tagName), tagManager.tagPath(branchName, tagName));
fileIO.copyFileUtf8(
snapshotManager.snapshotPath(snapshot.id()),
- snapshotManager.branchSnapshotPath(branchName, snapshot.id()));
+ snapshotManager.snapshotPath(branchName, snapshot.id()));
fileIO.copyFileUtf8(
schemaManager.toSchemaPath(snapshot.schemaId()),
- schemaManager.branchSchemaPath(branchName, snapshot.schemaId()));
+ schemaManager.toSchemaPath(branchName, snapshot.schemaId()));
} catch (IOException e) {
throw new RuntimeException(
String.format(
diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/SnapshotManager.java b/paimon-core/src/main/java/org/apache/paimon/utils/SnapshotManager.java
index b330fc30389f4..924d67f74bb6f 100644
--- a/paimon-core/src/main/java/org/apache/paimon/utils/SnapshotManager.java
+++ b/paimon-core/src/main/java/org/apache/paimon/utils/SnapshotManager.java
@@ -60,10 +60,17 @@ public class SnapshotManager implements Serializable {
private final FileIO fileIO;
private final Path tablePath;
+ private final String branch;
public SnapshotManager(FileIO fileIO, Path tablePath) {
+ this(fileIO, tablePath, DEFAULT_MAIN_BRANCH);
+ }
+
+ /** Specify the default branch for data writing. */
+ public SnapshotManager(FileIO fileIO, Path tablePath, String branchName) {
this.fileIO = fileIO;
this.tablePath = tablePath;
+ this.branch = StringUtils.isBlank(branchName) ? DEFAULT_MAIN_BRANCH : branchName;
}
public FileIO fileIO() {
@@ -74,6 +81,10 @@ public Path tablePath() {
return tablePath;
}
+ public String getBranch() {
+ return branch;
+ }
+
public Path snapshotDirectory() {
return new Path(tablePath + "/snapshot");
}
@@ -82,38 +93,37 @@ public Path snapshotPath(long snapshotId) {
return new Path(tablePath + "/snapshot/" + SNAPSHOT_PREFIX + snapshotId);
}
- public Path branchSnapshotDirectory(String branchName) {
- return new Path(getBranchPath(tablePath, branchName) + "/snapshot");
- }
-
- public Path branchSnapshotPath(String branchName, long snapshotId) {
- return new Path(
- getBranchPath(tablePath, branchName) + "/snapshot/" + SNAPSHOT_PREFIX + snapshotId);
- }
-
- public Path snapshotPathByBranch(String branchName, long snapshotId) {
+ public Path snapshotPath(String branchName, long snapshotId) {
return branchName.equals(DEFAULT_MAIN_BRANCH)
? snapshotPath(snapshotId)
- : branchSnapshotPath(branchName, snapshotId);
+ : new Path(
+ getBranchPath(tablePath, branchName)
+ + "/snapshot/"
+ + SNAPSHOT_PREFIX
+ + snapshotId);
}
- public Path snapshotDirByBranch(String branchName) {
+ public Path snapshotDirectory(String branchName) {
return branchName.equals(DEFAULT_MAIN_BRANCH)
? snapshotDirectory()
- : branchSnapshotDirectory(branchName);
+ : new Path(getBranchPath(tablePath, branchName) + "/snapshot");
}
public Snapshot snapshot(long snapshotId) {
- return snapshot(DEFAULT_MAIN_BRANCH, snapshotId);
+ return snapshot(snapshotId, branch);
}
- public Snapshot snapshot(String branchName, long snapshotId) {
- Path snapshotPath = snapshotPathByBranch(branchName, snapshotId);
+ public Snapshot snapshot(long snapshotId, String branch) {
+ Path snapshotPath = snapshotPath(branch, snapshotId);
return Snapshot.fromPath(fileIO, snapshotPath);
}
public boolean snapshotExists(long snapshotId) {
- Path path = snapshotPath(snapshotId);
+ return snapshotExists(snapshotId, branch);
+ }
+
+ public boolean snapshotExists(long snapshotId, String branch) {
+ Path path = snapshotPath(branch, snapshotId);
try {
return fileIO.exists(path);
} catch (IOException e) {
@@ -124,33 +134,37 @@ public boolean snapshotExists(long snapshotId) {
}
public @Nullable Snapshot latestSnapshot() {
- return latestSnapshot(DEFAULT_MAIN_BRANCH);
+ return latestSnapshot(branch);
}
- public @Nullable Snapshot latestSnapshot(String branchName) {
- Long snapshotId = latestSnapshotId(branchName);
- return snapshotId == null ? null : snapshot(branchName, snapshotId);
+ public @Nullable Snapshot latestSnapshot(String branch) {
+ Long snapshotId = latestSnapshotId(branch);
+ return snapshotId == null ? null : snapshot(snapshotId);
}
public @Nullable Long latestSnapshotId() {
- return latestSnapshotId(DEFAULT_MAIN_BRANCH);
+ return latestSnapshotId(branch);
}
- public @Nullable Long latestSnapshotId(String branchName) {
+ public @Nullable Long latestSnapshotId(String branch) {
try {
- return findLatest(branchName);
+ return findLatest(branch);
} catch (IOException e) {
throw new RuntimeException("Failed to find latest snapshot id", e);
}
}
public @Nullable Snapshot earliestSnapshot() {
- Long snapshotId = earliestSnapshotId();
- return snapshotId == null ? null : snapshot(snapshotId);
+ return earliestSnapshot(branch);
+ }
+
+ public @Nullable Snapshot earliestSnapshot(String branch) {
+ Long snapshotId = earliestSnapshotId(branch);
+ return snapshotId == null ? null : snapshot(snapshotId, branch);
}
public @Nullable Long earliestSnapshotId() {
- return earliestSnapshotId(DEFAULT_MAIN_BRANCH);
+ return earliestSnapshotId(branch);
}
public @Nullable Long earliestSnapshotId(String branchName) {
@@ -162,15 +176,19 @@ public boolean snapshotExists(long snapshotId) {
}
public @Nullable Long pickOrLatest(Predicate predicate) {
- Long latestId = latestSnapshotId();
- Long earliestId = earliestSnapshotId();
+ return pickOrLatest(branch, predicate);
+ }
+
+ public @Nullable Long pickOrLatest(String branch, Predicate predicate) {
+ Long latestId = latestSnapshotId(branch);
+ Long earliestId = earliestSnapshotId(branch);
if (latestId == null || earliestId == null) {
return null;
}
for (long snapshotId = latestId; snapshotId >= earliestId; snapshotId--) {
- if (snapshotExists(snapshotId)) {
- Snapshot snapshot = snapshot(snapshotId);
+ if (snapshotExists(snapshotId, branch)) {
+ Snapshot snapshot = snapshot(snapshotId, branch);
if (predicate.test(snapshot)) {
return snapshot.id();
}
@@ -180,25 +198,29 @@ public boolean snapshotExists(long snapshotId) {
return latestId;
}
+ public @Nullable Long earlierThanTimeMills(long timestampMills) {
+ return earlierThanTimeMills(branch, timestampMills);
+ }
+
/**
* Returns the latest snapshot earlier than the timestamp mills. A non-existent snapshot may be
* returned if all snapshots are equal to or later than the timestamp mills.
*/
- public @Nullable Long earlierThanTimeMills(long timestampMills) {
- Long earliest = earliestSnapshotId();
- Long latest = latestSnapshotId();
+ public @Nullable Long earlierThanTimeMills(String branch, long timestampMills) {
+ Long earliest = earliestSnapshotId(branch);
+ Long latest = latestSnapshotId(branch);
if (earliest == null || latest == null) {
return null;
}
- if (snapshot(earliest).timeMillis() >= timestampMills) {
+ if (snapshot(earliest, branch).timeMillis() >= timestampMills) {
return earliest - 1;
}
while (earliest < latest) {
long mid = (earliest + latest + 1) / 2;
- if (snapshot(mid).timeMillis() < timestampMills) {
+ if (snapshot(mid, branch).timeMillis() < timestampMills) {
earliest = mid;
} else {
latest = mid - 1;
@@ -207,24 +229,28 @@ public boolean snapshotExists(long snapshotId) {
return earliest;
}
+ public @Nullable Snapshot earlierOrEqualTimeMills(long timestampMills) {
+ return earlierOrEqualTimeMills(branch, timestampMills);
+ }
+
/**
* Returns a {@link Snapshot} whoes commit time is earlier than or equal to given timestamp
* mills. If there is no such a snapshot, returns null.
*/
- public @Nullable Snapshot earlierOrEqualTimeMills(long timestampMills) {
- Long earliest = earliestSnapshotId();
- Long latest = latestSnapshotId();
+ public @Nullable Snapshot earlierOrEqualTimeMills(String branch, long timestampMills) {
+ Long earliest = earliestSnapshotId(branch);
+ Long latest = latestSnapshotId(branch);
if (earliest == null || latest == null) {
return null;
}
- if (snapshot(earliest).timeMillis() > timestampMills) {
+ if (snapshot(earliest, branch).timeMillis() > timestampMills) {
return null;
}
Snapshot finalSnapshot = null;
while (earliest <= latest) {
long mid = earliest + (latest - earliest) / 2; // Avoid overflow
- Snapshot snapshot = snapshot(mid);
+ Snapshot snapshot = snapshot(mid, branch);
long commitTime = snapshot.timeMillis();
if (commitTime > timestampMills) {
latest = mid - 1; // Search in the left half
@@ -240,23 +266,35 @@ public boolean snapshotExists(long snapshotId) {
}
public long snapshotCount() throws IOException {
- return listVersionedFiles(fileIO, snapshotDirectory(), SNAPSHOT_PREFIX).count();
+ return snapshotCount(branch);
+ }
+
+ public long snapshotCount(String branch) throws IOException {
+ return listVersionedFiles(fileIO, snapshotDirectory(branch), SNAPSHOT_PREFIX).count();
}
public Iterator snapshots() throws IOException {
- return listVersionedFiles(fileIO, snapshotDirectory(), SNAPSHOT_PREFIX)
+ return snapshots(branch);
+ }
+
+ public Iterator snapshots(String branch) throws IOException {
+ return listVersionedFiles(fileIO, snapshotDirectory(branch), SNAPSHOT_PREFIX)
.map(this::snapshot)
.sorted(Comparator.comparingLong(Snapshot::id))
.iterator();
}
+ public List safelyGetAllSnapshots() throws IOException {
+ return safelyGetAllSnapshots(branch);
+ }
+
/**
* If {@link FileNotFoundException} is thrown when reading the snapshot file, this snapshot may
* be deleted by other processes, so just skip this snapshot.
*/
- public List safelyGetAllSnapshots() throws IOException {
+ public List safelyGetAllSnapshots(String branch) throws IOException {
List paths =
- listVersionedFiles(fileIO, snapshotDirectory(), SNAPSHOT_PREFIX)
+ listVersionedFiles(fileIO, snapshotDirectory(branch), SNAPSHOT_PREFIX)
.map(this::snapshotPath)
.collect(Collectors.toList());
@@ -268,13 +306,18 @@ public List safelyGetAllSnapshots() throws IOException {
return snapshots;
}
+ public List tryGetNonSnapshotFiles(Predicate fileStatusFilter) {
+ return tryGetNonSnapshotFiles(branch, fileStatusFilter);
+ }
+
/**
* Try to get non snapshot files. If any error occurred, just ignore it and return an empty
* result.
*/
- public List tryGetNonSnapshotFiles(Predicate fileStatusFilter) {
+ public List tryGetNonSnapshotFiles(
+ String branch, Predicate fileStatusFilter) {
try {
- FileStatus[] statuses = fileIO.listStatus(snapshotDirectory());
+ FileStatus[] statuses = fileIO.listStatus(snapshotDirectory(branch));
if (statuses == null) {
return Collections.emptyList();
}
@@ -299,18 +342,22 @@ private Predicate nonSnapshotFileFilter() {
}
public Optional latestSnapshotOfUser(String user) {
- Long latestId = latestSnapshotId();
+ return latestSnapshotOfUser(branch, user);
+ }
+
+ public Optional latestSnapshotOfUser(String branch, String user) {
+ Long latestId = latestSnapshotId(branch);
if (latestId == null) {
return Optional.empty();
}
long earliestId =
Preconditions.checkNotNull(
- earliestSnapshotId(),
+ earliestSnapshotId(branch),
"Latest snapshot id is not null, but earliest snapshot id is null. "
+ "This is unexpected.");
for (long id = latestId; id >= earliestId; id--) {
- Snapshot snapshot = snapshot(id);
+ Snapshot snapshot = snapshot(id, branch);
if (user.equals(snapshot.commitUser())) {
return Optional.of(snapshot);
}
@@ -318,19 +365,24 @@ public Optional latestSnapshotOfUser(String user) {
return Optional.empty();
}
- /** Find the snapshot of the specified identifiers written by the specified user. */
public List findSnapshotsForIdentifiers(
@Nonnull String user, List identifiers) {
+ return findSnapshotsForIdentifiers(user, identifiers, branch);
+ }
+
+ /** Find the snapshot of the specified identifiers written by the specified user. */
+ public List findSnapshotsForIdentifiers(
+ @Nonnull String user, List identifiers, String branch) {
if (identifiers.isEmpty()) {
return Collections.emptyList();
}
- Long latestId = latestSnapshotId();
+ Long latestId = latestSnapshotId(branch);
if (latestId == null) {
return Collections.emptyList();
}
long earliestId =
Preconditions.checkNotNull(
- earliestSnapshotId(),
+ earliestSnapshotId(branch),
"Latest snapshot id is not null, but earliest snapshot id is null. "
+ "This is unexpected.");
@@ -338,7 +390,7 @@ public List findSnapshotsForIdentifiers(
List matchedSnapshots = new ArrayList<>();
Set remainingIdentifiers = new HashSet<>(identifiers);
for (long id = latestId; id >= earliestId && !remainingIdentifiers.isEmpty(); id--) {
- Snapshot snapshot = snapshot(id);
+ Snapshot snapshot = snapshot(id, branch);
if (user.equals(snapshot.commitUser())) {
if (remainingIdentifiers.remove(snapshot.commitIdentifier())) {
matchedSnapshots.add(snapshot);
@@ -351,18 +403,23 @@ public List findSnapshotsForIdentifiers(
return matchedSnapshots;
}
+ @Nullable
+ public Snapshot traversalSnapshotsFromLatestSafely(Filter checker) {
+ return traversalSnapshotsFromLatestSafely(checker, branch);
+ }
+
/**
* Traversal snapshots from latest to earliest safely, this is applied on the writer side
* because the committer may delete obsolete snapshots, which may cause the writer to encounter
* unreadable snapshots.
*/
@Nullable
- public Snapshot traversalSnapshotsFromLatestSafely(Filter checker) {
- Long latestId = latestSnapshotId();
+ public Snapshot traversalSnapshotsFromLatestSafely(Filter checker, String branch) {
+ Long latestId = latestSnapshotId(branch);
if (latestId == null) {
return null;
}
- Long earliestId = earliestSnapshotId();
+ Long earliestId = earliestSnapshotId(branch);
if (earliestId == null) {
return null;
}
@@ -370,9 +427,9 @@ public Snapshot traversalSnapshotsFromLatestSafely(Filter checker) {
for (long id = latestId; id >= earliestId; id--) {
Snapshot snapshot;
try {
- snapshot = snapshot(id);
+ snapshot = snapshot(id, branch);
} catch (Exception e) {
- Long newEarliestId = earliestSnapshotId();
+ Long newEarliestId = earliestSnapshotId(branch);
if (newEarliestId == null) {
return null;
}
@@ -393,13 +450,13 @@ public Snapshot traversalSnapshotsFromLatestSafely(Filter checker) {
return null;
}
- private @Nullable Long findLatest(String branchName) throws IOException {
- Path snapshotDir = snapshotDirByBranch(branchName);
+ private @Nullable Long findLatest(String branch) throws IOException {
+ Path snapshotDir = snapshotDirectory(branch);
if (!fileIO.exists(snapshotDir)) {
return null;
}
- Long snapshotId = readHint(LATEST, branchName);
+ Long snapshotId = readHint(LATEST, branch);
if (snapshotId != null) {
long nextSnapshot = snapshotId + 1;
// it is the latest only there is no next one
@@ -408,30 +465,34 @@ public Snapshot traversalSnapshotsFromLatestSafely(Filter checker) {
}
}
- return findByListFiles(Math::max, branchName);
+ return findByListFiles(Math::max);
}
- private @Nullable Long findEarliest(String branchName) throws IOException {
- Path snapshotDir = snapshotDirByBranch(branchName);
+ private @Nullable Long findEarliest() throws IOException {
+ return findEarliest(branch);
+ }
+
+ private @Nullable Long findEarliest(String branch) throws IOException {
+ Path snapshotDir = snapshotDirectory(branch);
if (!fileIO.exists(snapshotDir)) {
return null;
}
- Long snapshotId = readHint(EARLIEST, branchName);
+ Long snapshotId = readHint(EARLIEST, branch);
// null and it is the earliest only it exists
if (snapshotId != null && snapshotExists(snapshotId)) {
return snapshotId;
}
- return findByListFiles(Math::min, branchName);
+ return findByListFiles(Math::min);
}
public Long readHint(String fileName) {
- return readHint(fileName, DEFAULT_MAIN_BRANCH);
+ return readHint(fileName, branch);
}
public Long readHint(String fileName, String branchName) {
- Path snapshotDir = snapshotDirByBranch(branchName);
+ Path snapshotDir = snapshotDirectory(branchName);
Path path = new Path(snapshotDir, fileName);
int retryNumber = 0;
while (retryNumber++ < READ_HINT_RETRY_NUM) {
@@ -449,16 +510,15 @@ public Long readHint(String fileName, String branchName) {
return null;
}
- private Long findByListFiles(BinaryOperator reducer, String branchName)
- throws IOException {
- Path snapshotDir = snapshotDirByBranch(branchName);
+ private Long findByListFiles(BinaryOperator reducer) throws IOException {
+ Path snapshotDir = snapshotDirectory(branch);
return listVersionedFiles(fileIO, snapshotDir, SNAPSHOT_PREFIX)
.reduce(reducer)
.orElse(null);
}
public void commitLatestHint(long snapshotId) throws IOException {
- commitLatestHint(snapshotId, DEFAULT_MAIN_BRANCH);
+ commitLatestHint(snapshotId, branch);
}
public void commitLatestHint(long snapshotId, String branchName) throws IOException {
@@ -466,7 +526,7 @@ public void commitLatestHint(long snapshotId, String branchName) throws IOExcept
}
public void commitEarliestHint(long snapshotId) throws IOException {
- commitEarliestHint(snapshotId, DEFAULT_MAIN_BRANCH);
+ commitEarliestHint(snapshotId, branch);
}
public void commitEarliestHint(long snapshotId, String branchName) throws IOException {
@@ -475,7 +535,7 @@ public void commitEarliestHint(long snapshotId, String branchName) throws IOExce
private void commitHint(long snapshotId, String fileName, String branchName)
throws IOException {
- Path snapshotDir = snapshotDirByBranch(branchName);
+ Path snapshotDir = snapshotDirectory(branchName);
Path hintFile = new Path(snapshotDir, fileName);
fileIO.overwriteFileUtf8(hintFile, String.valueOf(snapshotId));
}
diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/TagManager.java b/paimon-core/src/main/java/org/apache/paimon/utils/TagManager.java
index a29a3e151c766..4ee52703681b9 100644
--- a/paimon-core/src/main/java/org/apache/paimon/utils/TagManager.java
+++ b/paimon-core/src/main/java/org/apache/paimon/utils/TagManager.java
@@ -40,6 +40,7 @@
import java.util.function.Predicate;
import java.util.stream.Collectors;
+import static org.apache.paimon.utils.BranchManager.DEFAULT_MAIN_BRANCH;
import static org.apache.paimon.utils.BranchManager.getBranchPath;
import static org.apache.paimon.utils.FileUtils.listVersionedFileStatus;
import static org.apache.paimon.utils.Preconditions.checkArgument;
@@ -53,10 +54,17 @@ public class TagManager {
private final FileIO fileIO;
private final Path tablePath;
+ private final String branch;
public TagManager(FileIO fileIO, Path tablePath) {
+ this(fileIO, tablePath, DEFAULT_MAIN_BRANCH);
+ }
+
+ /** Specify the default branch for data writing. */
+ public TagManager(FileIO fileIO, Path tablePath, String branch) {
this.fileIO = fileIO;
this.tablePath = tablePath;
+ this.branch = StringUtils.isBlank(branch) ? DEFAULT_MAIN_BRANCH : branch;
}
/** Return the root Directory of tags. */
@@ -64,22 +72,36 @@ public Path tagDirectory() {
return new Path(tablePath + "/tag");
}
+ /** Return the root Directory of tags. */
+ public Path tagDirectory(String branch) {
+ return branch.equals(DEFAULT_MAIN_BRANCH)
+ ? tagDirectory()
+ : new Path(getBranchPath(tablePath, branch) + "/tag");
+ }
+
/** Return the path of a tag. */
public Path tagPath(String tagName) {
return new Path(tablePath + "/tag/" + TAG_PREFIX + tagName);
}
/** Return the path of a tag in branch. */
- public Path branchTagPath(String branchName, String tagName) {
- return new Path(getBranchPath(tablePath, branchName) + "/tag/" + TAG_PREFIX + tagName);
+ public Path tagPath(String branch, String tagName) {
+ return branch.equals(DEFAULT_MAIN_BRANCH)
+ ? tagPath(tagName)
+ : new Path(getBranchPath(tablePath, branch) + "/tag/" + TAG_PREFIX + tagName);
}
- /** Create a tag from given snapshot and save it in the storage. */
public void createTag(Snapshot snapshot, String tagName, List callbacks) {
+ createTag(snapshot, tagName, callbacks, branch);
+ }
+
+ /** Create a tag from given snapshot and save it in the storage. */
+ public void createTag(
+ Snapshot snapshot, String tagName, List callbacks, String branch) {
checkArgument(!StringUtils.isBlank(tagName), "Tag name '%s' is blank.", tagName);
- checkArgument(!tagExists(tagName), "Tag name '%s' already exists.", tagName);
+ checkArgument(!tagExists(branch, tagName), "Tag name '%s' already exists.", tagName);
- Path newTagPath = tagPath(tagName);
+ Path newTagPath = tagPath(branch, tagName);
try {
fileIO.writeFileUtf8(newTagPath, snapshot.toJson());
} catch (IOException e) {
@@ -102,8 +124,11 @@ public void createTag(Snapshot snapshot, String tagName, List callb
/** Make sure the tagNames are ALL tags of one snapshot. */
public void deleteAllTagsOfOneSnapshot(
- List tagNames, TagDeletion tagDeletion, SnapshotManager snapshotManager) {
- Snapshot taggedSnapshot = taggedSnapshot(tagNames.get(0));
+ List tagNames,
+ TagDeletion tagDeletion,
+ SnapshotManager snapshotManager,
+ String branch) {
+ Snapshot taggedSnapshot = taggedSnapshot(branch, tagNames.get(0));
List taggedSnapshots;
// skip file deletion if snapshot exists
@@ -112,29 +137,37 @@ public void deleteAllTagsOfOneSnapshot(
return;
} else {
// FileIO discovers tags by tag file, so we should read all tags before we delete tag
- taggedSnapshots = taggedSnapshots();
- tagNames.forEach(tagName -> fileIO.deleteQuietly(tagPath(tagName)));
+ taggedSnapshots = taggedSnapshots(branch);
+ tagNames.forEach(tagName -> fileIO.deleteQuietly(tagPath(branch, tagName)));
}
- doClean(taggedSnapshot, taggedSnapshots, snapshotManager, tagDeletion);
+ doClean(taggedSnapshot, taggedSnapshots, snapshotManager, tagDeletion, branch);
}
public void deleteTag(
String tagName, TagDeletion tagDeletion, SnapshotManager snapshotManager) {
+ deleteTag(tagName, tagDeletion, snapshotManager, branch);
+ }
+
+ public void deleteTag(
+ String tagName,
+ TagDeletion tagDeletion,
+ SnapshotManager snapshotManager,
+ String branch) {
checkArgument(!StringUtils.isBlank(tagName), "Tag name '%s' is blank.", tagName);
- checkArgument(tagExists(tagName), "Tag '%s' doesn't exist.", tagName);
+ checkArgument(tagExists(branch, tagName), "Tag '%s' doesn't exist.", tagName);
- Snapshot taggedSnapshot = taggedSnapshot(tagName);
+ Snapshot taggedSnapshot = taggedSnapshot(branch, tagName);
List taggedSnapshots;
// skip file deletion if snapshot exists
if (snapshotManager.snapshotExists(taggedSnapshot.id())) {
- fileIO.deleteQuietly(tagPath(tagName));
+ fileIO.deleteQuietly(tagPath(branch, tagName));
return;
} else {
// FileIO discovers tags by tag file, so we should read all tags before we delete tag
SortedMap> tags = tags();
- fileIO.deleteQuietly(tagPath(tagName));
+ fileIO.deleteQuietly(tagPath(branch, tagName));
// skip data file clean if more than 1 tags are created based on this snapshot
if (tags.get(taggedSnapshot).size() > 1) {
@@ -143,14 +176,15 @@ public void deleteTag(
taggedSnapshots = new ArrayList<>(tags.keySet());
}
- doClean(taggedSnapshot, taggedSnapshots, snapshotManager, tagDeletion);
+ doClean(taggedSnapshot, taggedSnapshots, snapshotManager, tagDeletion, branch);
}
private void doClean(
Snapshot taggedSnapshot,
List taggedSnapshots,
SnapshotManager snapshotManager,
- TagDeletion tagDeletion) {
+ TagDeletion tagDeletion,
+ String branch) {
// collect skipping sets from the left neighbor tag and the nearest right neighbor (either
// the earliest snapshot or right neighbor tag)
List skippedSnapshots = new ArrayList<>();
@@ -161,7 +195,7 @@ private void doClean(
skippedSnapshots.add(taggedSnapshots.get(index - 1));
}
// the nearest right neighbor
- Snapshot right = snapshotManager.earliestSnapshot();
+ Snapshot right = snapshotManager.earliestSnapshot(branch);
if (index + 1 < taggedSnapshots.size()) {
Snapshot rightTag = taggedSnapshots.get(index + 1);
right = right.id() < rightTag.id() ? right : rightTag;
@@ -191,9 +225,9 @@ private void doClean(
taggedSnapshot, tagDeletion.manifestSkippingSet(skippedSnapshots));
}
- /** Check if a tag exists. */
- public boolean tagExists(String tagName) {
- Path path = tagPath(tagName);
+ /** Check if a branch tag exists. */
+ public boolean tagExists(String branch, String tagName) {
+ Path path = tagPath(branch, tagName);
try {
return fileIO.exists(path);
} catch (IOException e) {
@@ -204,10 +238,28 @@ public boolean tagExists(String tagName) {
}
}
- /** Get the tagged snapshot by name. */
+ /** Check if a tag exists. */
+ public boolean tagExists(String tagName) {
+ return tagExists(DEFAULT_MAIN_BRANCH, tagName);
+ }
+
+ /** Get the branch tagged snapshot by name. */
public Snapshot taggedSnapshot(String tagName) {
- checkArgument(tagExists(tagName), "Tag '%s' doesn't exist.", tagName);
- return Snapshot.fromPath(fileIO, tagPath(tagName));
+ return taggedSnapshot(DEFAULT_MAIN_BRANCH, tagName);
+ }
+
+ /** Get the tagged snapshot by name. */
+ public Snapshot taggedSnapshot(String branch, String tagName) {
+ checkArgument(tagExists(branch, tagName), "Tag '%s' doesn't exist.", tagName);
+ return Snapshot.fromPath(fileIO, tagPath(branch, tagName));
+ }
+
+ public long tagCount(String branch) {
+ try {
+ return listVersionedFileStatus(fileIO, tagDirectory(branch), TAG_PREFIX).count();
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
}
public long tagCount() {
@@ -223,11 +275,25 @@ public List taggedSnapshots() {
return new ArrayList<>(tags().keySet());
}
+ /** Get all tagged snapshots sorted by snapshot id. */
+ public List taggedSnapshots(String branch) {
+ return new ArrayList<>(tags(branch).keySet());
+ }
+
+ /** Get all tagged snapshots with names sorted by snapshot id. */
+ public SortedMap> tags(String branch) {
+ return tags(branch, tagName -> true);
+ }
+
/** Get all tagged snapshots with names sorted by snapshot id. */
public SortedMap> tags() {
return tags(tagName -> true);
}
+ public SortedMap> tags(Predicate filter) {
+ return tags(DEFAULT_MAIN_BRANCH, filter);
+ }
+
/**
* Retrieves a sorted map of snapshots filtered based on a provided predicate. The predicate
* determines which tag names should be included in the result. Only snapshots with tag names
@@ -239,12 +305,12 @@ public SortedMap> tags() {
* name.
* @throws RuntimeException if an IOException occurs during retrieval of snapshots.
*/
- public SortedMap> tags(Predicate filter) {
+ public SortedMap> tags(String branch, Predicate filter) {
TreeMap> tags =
new TreeMap<>(Comparator.comparingLong(Snapshot::id));
try {
List paths =
- listVersionedFileStatus(fileIO, tagDirectory(), TAG_PREFIX)
+ listVersionedFileStatus(fileIO, tagDirectory(branch), TAG_PREFIX)
.map(FileStatus::getPath)
.collect(Collectors.toList());
@@ -269,11 +335,15 @@ public SortedMap> tags(Predicate filter) {
}
public List sortTagsOfOneSnapshot(List tagNames) {
+ return sortTagsOfOneSnapshot(branch, tagNames);
+ }
+
+ public List sortTagsOfOneSnapshot(String branch, List tagNames) {
return tagNames.stream()
.map(
name -> {
try {
- return fileIO.getFileStatus(tagPath(name));
+ return fileIO.getFileStatus(tagPath(branch, name));
} catch (IOException e) {
throw new RuntimeException(e);
}
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 43e0297fd921c..8ddb9bebab45f 100644
--- a/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java
+++ b/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java
@@ -52,10 +52,12 @@
import org.apache.paimon.table.source.DataSplit;
import org.apache.paimon.table.source.ScanMode;
import org.apache.paimon.types.RowType;
+import org.apache.paimon.utils.BranchManager;
import org.apache.paimon.utils.CommitIncrement;
import org.apache.paimon.utils.FileStorePathFactory;
import org.apache.paimon.utils.RecordWriter;
import org.apache.paimon.utils.SnapshotManager;
+import org.apache.paimon.utils.StringUtils;
import org.apache.paimon.utils.TagManager;
import org.slf4j.Logger;
@@ -97,6 +99,8 @@ public class TestFileStore extends KeyValueFileStore {
private long commitIdentifier;
+ private String branch;
+
private TestFileStore(
String root,
CoreOptions options,
@@ -105,7 +109,7 @@ private TestFileStore(
RowType valueType,
KeyValueFieldsExtractor keyValueFieldsExtractor,
MergeFunctionFactory mfFactory,
- TableSchema tableSchema) {
+ TableSchema tableSchema, String branch) {
super(
FileIOFinder.find(new Path(root)),
schemaManager(root, options),
@@ -136,6 +140,7 @@ private TestFileStore(
this.commitUser = UUID.randomUUID().toString();
this.commitIdentifier = 0L;
+ this.branch = branch;
}
private static SchemaManager schemaManager(String root, CoreOptions options) {
@@ -300,7 +305,7 @@ public List commitDataImpl(
.write(kv);
}
- FileStoreCommit commit = newCommit(commitUser);
+ FileStoreCommit commit = newCommit(commitUser, branch);
ManifestCommittable committable =
new ManifestCommittable(
identifier == null ? commitIdentifier++ : identifier, watermark);
@@ -321,12 +326,12 @@ public List commitDataImpl(
}
SnapshotManager snapshotManager = snapshotManager();
- Long snapshotIdBeforeCommit = snapshotManager.latestSnapshotId();
+ Long snapshotIdBeforeCommit = snapshotManager.latestSnapshotId(branch);
if (snapshotIdBeforeCommit == null) {
snapshotIdBeforeCommit = Snapshot.FIRST_SNAPSHOT_ID - 1;
}
commitFunction.accept(commit, committable);
- Long snapshotIdAfterCommit = snapshotManager.latestSnapshotId();
+ Long snapshotIdAfterCommit = snapshotManager.latestSnapshotId(branch);
if (snapshotIdAfterCommit == null) {
snapshotIdAfterCommit = Snapshot.FIRST_SNAPSHOT_ID - 1;
}
@@ -466,18 +471,18 @@ public void assertCleaned() throws IOException {
// - latest should < true_latest
// - earliest should < true_earliest
SnapshotManager snapshotManager = snapshotManager();
- Path snapshotDir = snapshotManager.snapshotDirectory();
+ Path snapshotDir = snapshotManager.snapshotDirectory(branch);
Path earliest = new Path(snapshotDir, SnapshotManager.EARLIEST);
Path latest = new Path(snapshotDir, SnapshotManager.LATEST);
if (actualFiles.remove(earliest)) {
- long earliestId = snapshotManager.readHint(SnapshotManager.EARLIEST);
+ long earliestId = snapshotManager.readHint(SnapshotManager.EARLIEST, branch);
fileIO.delete(earliest, false);
- assertThat(earliestId <= snapshotManager.earliestSnapshotId()).isTrue();
+ assertThat(earliestId <= snapshotManager.earliestSnapshotId(branch)).isTrue();
}
if (actualFiles.remove(latest)) {
- long latestId = snapshotManager.readHint(SnapshotManager.LATEST);
+ long latestId = snapshotManager.readHint(SnapshotManager.LATEST, branch);
fileIO.delete(latest, false);
- assertThat(latestId <= snapshotManager.latestSnapshotId()).isTrue();
+ assertThat(latestId <= snapshotManager.latestSnapshotId(branch)).isTrue();
}
actualFiles.remove(latest);
@@ -496,7 +501,9 @@ private Set getFilesInUse() {
Set result = new HashSet<>();
SchemaManager schemaManager = new SchemaManager(fileIO, options.path());
- schemaManager.listAllIds().forEach(id -> result.add(schemaManager.toSchemaPath(id)));
+ schemaManager
+ .listAllIds(branch)
+ .forEach(id -> result.add(schemaManager.toSchemaPath(branch, id)));
SnapshotManager snapshotManager = snapshotManager();
Long latestSnapshotId = snapshotManager.latestSnapshotId();
@@ -581,6 +588,7 @@ public static class Builder {
private final TableSchema tableSchema;
private CoreOptions.ChangelogProducer changelogProducer;
+ private final String branch;
public Builder(
String format,
@@ -592,6 +600,30 @@ public Builder(
KeyValueFieldsExtractor keyValueFieldsExtractor,
MergeFunctionFactory mfFactory,
TableSchema tableSchema) {
+ this(
+ format,
+ root,
+ numBuckets,
+ partitionType,
+ keyType,
+ valueType,
+ keyValueFieldsExtractor,
+ mfFactory,
+ tableSchema,
+ BranchManager.DEFAULT_MAIN_BRANCH);
+ }
+
+ public Builder(
+ String format,
+ String root,
+ int numBuckets,
+ RowType partitionType,
+ RowType keyType,
+ RowType valueType,
+ KeyValueFieldsExtractor keyValueFieldsExtractor,
+ MergeFunctionFactory mfFactory,
+ TableSchema tableSchema,
+ String branch) {
this.format = format;
this.root = root;
this.numBuckets = numBuckets;
@@ -603,6 +635,7 @@ public Builder(
this.tableSchema = tableSchema;
this.changelogProducer = CoreOptions.ChangelogProducer.NONE;
+ this.branch = StringUtils.isEmpty(branch) ? BranchManager.DEFAULT_MAIN_BRANCH : branch;
}
public Builder changelogProducer(CoreOptions.ChangelogProducer changelogProducer) {
@@ -630,6 +663,7 @@ public TestFileStore build() {
// disable dynamic-partition-overwrite in FileStoreCommit layer test
conf.set(CoreOptions.DYNAMIC_PARTITION_OVERWRITE, false);
+ conf.set(CoreOptions.BRANCH, branch);
return new TestFileStore(
root,
@@ -639,7 +673,8 @@ public TestFileStore build() {
valueType,
keyValueFieldsExtractor,
mfFactory,
- tableSchema);
+ tableSchema,
+ branch);
}
}
}
diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreReadBranchTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreReadBranchTest.java
new file mode 100644
index 0000000000000..d353429bf9eff
--- /dev/null
+++ b/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreReadBranchTest.java
@@ -0,0 +1,356 @@
+/*
+ * 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.operation;
+
+import org.apache.paimon.KeyValue;
+import org.apache.paimon.TestFileStore;
+import org.apache.paimon.TestKeyValueGenerator;
+import org.apache.paimon.data.BinaryRow;
+import org.apache.paimon.data.GenericRow;
+import org.apache.paimon.data.InternalRow;
+import org.apache.paimon.data.serializer.InternalRowSerializer;
+import org.apache.paimon.fs.FileIOFinder;
+import org.apache.paimon.fs.Path;
+import org.apache.paimon.manifest.ManifestEntry;
+import org.apache.paimon.mergetree.compact.DeduplicateMergeFunction;
+import org.apache.paimon.mergetree.compact.MergeFunction;
+import org.apache.paimon.mergetree.compact.MergeFunctionFactory;
+import org.apache.paimon.reader.RecordReader;
+import org.apache.paimon.reader.RecordReaderIterator;
+import org.apache.paimon.schema.KeyValueFieldsExtractor;
+import org.apache.paimon.schema.Schema;
+import org.apache.paimon.schema.SchemaManager;
+import org.apache.paimon.schema.TableSchema;
+import org.apache.paimon.table.source.DataSplit;
+import org.apache.paimon.types.BigIntType;
+import org.apache.paimon.types.DataField;
+import org.apache.paimon.types.DataType;
+import org.apache.paimon.types.IntType;
+import org.apache.paimon.types.RowKind;
+import org.apache.paimon.types.RowType;
+import org.apache.paimon.types.VarCharType;
+
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static org.apache.paimon.utils.Preconditions.checkArgument;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for {@link KeyValueFileStoreRead}. */
+public class KeyValueFileStoreReadBranchTest {
+
+ @TempDir java.nio.file.Path tempDir;
+
+ private String branch = UUID.randomUUID() + "-branch";
+
+ @Test
+ public void testKeyProjection() throws Exception {
+ // (a, b, c) -> (b, a), c is the partition, all integers are in range [0, 2]
+
+ ThreadLocalRandom random = ThreadLocalRandom.current();
+ int numRecords = random.nextInt(1000) + 1;
+ List data = new ArrayList<>();
+ Map expected = new HashMap<>();
+ for (int i = 0; i < numRecords; i++) {
+ int a = random.nextInt(3);
+ int b = random.nextInt(3);
+ int c = random.nextInt(3);
+ long delta = random.nextLong(21) - 10;
+ // count number of occurrence of (b, a)
+ expected.compute(b * 10 + a, (k, v) -> v == null ? delta : v + delta);
+ data.add(
+ new KeyValue()
+ .replace(
+ GenericRow.of(a, b, c),
+ i,
+ RowKind.INSERT,
+ GenericRow.of(delta)));
+ }
+ // remove zero occurrence, it might be merged and discarded by the merge tree
+ expected.entrySet().removeIf(e -> e.getValue() == 0);
+
+ RowType partitionType = RowType.of(new DataType[] {new IntType(false)}, new String[] {"c"});
+ InternalRowSerializer partitionSerializer = new InternalRowSerializer(partitionType);
+ List keyNames = Arrays.asList("a", "b", "c");
+ RowType keyType =
+ RowType.of(
+ new DataType[] {new IntType(false), new IntType(false), new IntType(false)},
+ keyNames.toArray(new String[0]));
+ RowType projectedKeyType = RowType.of(new IntType(false), new IntType(false));
+ InternalRowSerializer projectedKeySerializer = new InternalRowSerializer(projectedKeyType);
+ RowType valueType =
+ RowType.of(new DataType[] {new BigIntType(false)}, new String[] {"count"});
+ InternalRowSerializer valueSerializer = new InternalRowSerializer(valueType);
+
+ TestFileStore store =
+ createStore(
+ partitionType,
+ keyType,
+ valueType,
+ new KeyValueFieldsExtractor() {
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public List keyFields(TableSchema schema) {
+ return schema.fields().stream()
+ .filter(f -> keyNames.contains(f.name()))
+ .collect(Collectors.toList());
+ }
+
+ @Override
+ public List valueFields(TableSchema schema) {
+ return Collections.singletonList(
+ new DataField(0, "count", new BigIntType()));
+ }
+ },
+ TestValueCountMergeFunction.factory());
+ List readData =
+ writeThenRead(
+ data,
+ new int[][] {new int[] {1}, new int[] {0}},
+ null,
+ projectedKeySerializer,
+ valueSerializer,
+ store,
+ kv ->
+ partitionSerializer
+ .toBinaryRow(GenericRow.of(kv.key().getInt(2)))
+ .copy());
+ Map actual = new HashMap<>();
+ for (KeyValue kv : readData) {
+ assertThat(kv.key().getFieldCount()).isEqualTo(2);
+ int key = kv.key().getInt(0) * 10 + kv.key().getInt(1);
+ long delta = kv.value().getLong(0);
+ actual.compute(key, (k, v) -> v == null ? delta : v + delta);
+ }
+ actual.entrySet().removeIf(e -> e.getValue() == 0);
+ assertThat(actual).isEqualTo(expected);
+ }
+
+ @Test
+ public void testValueProjection() throws Exception {
+ // (dt, hr, shopId, orderId, itemId, priceAmount, comment) -> (shopId, itemId, dt, hr)
+
+ TestKeyValueGenerator gen = new TestKeyValueGenerator();
+ int numRecords = ThreadLocalRandom.current().nextInt(1000) + 1;
+ List data = new ArrayList<>();
+ for (int i = 0; i < numRecords; i++) {
+ data.add(gen.next());
+ }
+ TestFileStore store =
+ createStore(
+ TestKeyValueGenerator.DEFAULT_PART_TYPE,
+ TestKeyValueGenerator.KEY_TYPE,
+ TestKeyValueGenerator.DEFAULT_ROW_TYPE,
+ TestKeyValueGenerator.TestKeyValueFieldsExtractor.EXTRACTOR,
+ DeduplicateMergeFunction.factory());
+
+ InternalRowSerializer projectedValueSerializer =
+ new InternalRowSerializer(
+ new IntType(false),
+ new BigIntType(),
+ new VarCharType(false, 8),
+ new IntType(false));
+ Map expected = store.toKvMap(data);
+ expected.replaceAll(
+ (k, v) ->
+ projectedValueSerializer
+ .toBinaryRow(
+ GenericRow.of(
+ v.getInt(2),
+ v.isNullAt(4) ? null : v.getLong(4),
+ v.getString(0),
+ v.getInt(1)))
+ .copy());
+
+ List readData =
+ writeThenRead(
+ data,
+ null,
+ new int[][] {new int[] {2}, new int[] {4}, new int[] {0}, new int[] {1}},
+ TestKeyValueGenerator.KEY_SERIALIZER,
+ projectedValueSerializer,
+ store,
+ gen::getPartition);
+ for (KeyValue kv : readData) {
+ assertThat(kv.value().getFieldCount()).isEqualTo(4);
+ BinaryRow key = TestKeyValueGenerator.KEY_SERIALIZER.toBinaryRow(kv.key());
+ BinaryRow value = projectedValueSerializer.toBinaryRow(kv.value());
+ assertThat(expected).containsKey(key);
+ assertThat(value).isEqualTo(expected.get(key));
+ }
+ }
+
+ private List writeThenRead(
+ List data,
+ int[][] keyProjection,
+ int[][] valueProjection,
+ InternalRowSerializer projectedKeySerializer,
+ InternalRowSerializer projectedValueSerializer,
+ TestFileStore store,
+ Function partitionCalculator)
+ throws Exception {
+ store.commitData(data, partitionCalculator, kv -> 0);
+ FileStoreScan scan = store.newScan();
+ Long snapshotId = store.snapshotManager().latestSnapshotId();
+ Map> filesGroupedByPartition =
+ scan.withSnapshot(snapshotId).plan().files().stream()
+ .collect(Collectors.groupingBy(ManifestEntry::partition));
+ KeyValueFileStoreRead read = store.newRead();
+ if (keyProjection != null) {
+ read.withKeyProjection(keyProjection);
+ }
+ if (valueProjection != null) {
+ read.withValueProjection(valueProjection);
+ }
+
+ List result = new ArrayList<>();
+ for (Map.Entry> entry : filesGroupedByPartition.entrySet()) {
+ RecordReader reader =
+ read.createReader(
+ DataSplit.builder()
+ .withSnapshot(snapshotId)
+ .withPartition(entry.getKey())
+ .withBucket(0)
+ .withDataFiles(
+ entry.getValue().stream()
+ .map(ManifestEntry::file)
+ .collect(Collectors.toList()))
+ .build());
+ RecordReaderIterator actualIterator = new RecordReaderIterator<>(reader);
+ while (actualIterator.hasNext()) {
+ result.add(
+ actualIterator
+ .next()
+ .copy(projectedKeySerializer, projectedValueSerializer));
+ }
+ }
+ return result;
+ }
+
+ private TestFileStore createStore(
+ RowType partitionType,
+ RowType keyType,
+ RowType valueType,
+ KeyValueFieldsExtractor extractor,
+ MergeFunctionFactory mfFactory)
+ throws Exception {
+ Path path = new Path(tempDir.toUri());
+ SchemaManager schemaManager = new SchemaManager(FileIOFinder.find(path), path, branch);
+ boolean valueCountMode = mfFactory.create() instanceof TestValueCountMergeFunction;
+ schemaManager.createTable(
+ new Schema(
+ (valueCountMode ? keyType : valueType).getFields(),
+ partitionType.getFieldNames(),
+ valueCountMode
+ ? Collections.emptyList()
+ : Stream.concat(
+ keyType.getFieldNames().stream()
+ .map(field -> field.replace("key_", "")),
+ partitionType.getFieldNames().stream())
+ .collect(Collectors.toList()),
+ Collections.emptyMap(),
+ null));
+ return new TestFileStore.Builder(
+ "avro",
+ tempDir.toString(),
+ 1,
+ partitionType,
+ keyType,
+ valueType,
+ extractor,
+ mfFactory,
+ branch)
+ .build();
+ }
+
+ private static class TestValueCountMergeFunction implements MergeFunction {
+
+ private KeyValue latestKv;
+ private long total;
+ private KeyValue reused;
+
+ protected TestValueCountMergeFunction() {}
+
+ @Override
+ public void reset() {
+ latestKv = null;
+ total = 0;
+ }
+
+ @Override
+ public void add(KeyValue kv) {
+ checkArgument(
+ kv.valueKind() == RowKind.INSERT,
+ "In value count mode, only insert records come. This is a bug. Please file an issue.");
+ latestKv = kv;
+ total += count(kv.value());
+ }
+
+ @Override
+ @Nullable
+ public KeyValue getResult() {
+ if (total == 0) {
+ return null;
+ }
+
+ if (reused == null) {
+ reused = new KeyValue();
+ }
+ return reused.replace(
+ latestKv.key(),
+ latestKv.sequenceNumber(),
+ RowKind.INSERT,
+ GenericRow.of(total));
+ }
+
+ private long count(InternalRow value) {
+ checkArgument(!value.isNullAt(0), "Value count should not be null.");
+ return value.getLong(0);
+ }
+
+ public static MergeFunctionFactory factory() {
+ return new Factory();
+ }
+
+ private static class Factory implements MergeFunctionFactory {
+
+ private static final long serialVersionUID = 1L;
+
+ @Override
+ public MergeFunction create(@Nullable int[][] projection) {
+ return new TestValueCountMergeFunction();
+ }
+ }
+ }
+}
diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreReadTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreReadTest.java
index 07bf705e2bba6..2ea76128b8dd9 100644
--- a/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreReadTest.java
+++ b/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreReadTest.java
@@ -127,10 +127,7 @@ public List keyFields(TableSchema schema) {
@Override
public List valueFields(TableSchema schema) {
return Collections.singletonList(
- new DataField(
- 0,
- "count",
- new org.apache.paimon.types.BigIntType()));
+ new DataField(0, "count", new BigIntType()));
}
},
TestValueCountMergeFunction.factory());
diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreScanBranchTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreScanBranchTest.java
new file mode 100644
index 0000000000000..8ccbb40042538
--- /dev/null
+++ b/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreScanBranchTest.java
@@ -0,0 +1,346 @@
+/*
+ * 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.operation;
+
+import org.apache.paimon.KeyValue;
+import org.apache.paimon.Snapshot;
+import org.apache.paimon.TestFileStore;
+import org.apache.paimon.TestKeyValueGenerator;
+import org.apache.paimon.data.BinaryRow;
+import org.apache.paimon.data.BinaryRowWriter;
+import org.apache.paimon.data.BinaryString;
+import org.apache.paimon.fs.Path;
+import org.apache.paimon.fs.local.LocalFileIO;
+import org.apache.paimon.manifest.ManifestEntry;
+import org.apache.paimon.manifest.ManifestFileMeta;
+import org.apache.paimon.manifest.ManifestList;
+import org.apache.paimon.mergetree.compact.DeduplicateMergeFunction;
+import org.apache.paimon.predicate.PredicateBuilder;
+import org.apache.paimon.schema.Schema;
+import org.apache.paimon.schema.SchemaManager;
+import org.apache.paimon.types.IntType;
+import org.apache.paimon.types.RowType;
+import org.apache.paimon.utils.SnapshotManager;
+
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.stream.Collectors;
+
+import static org.assertj.core.api.Assertions.assertThat;
+
+/** Tests for {@link KeyValueFileStoreScan}. */
+public class KeyValueFileStoreScanBranchTest {
+
+ private static final int NUM_BUCKETS = 10;
+
+ private TestKeyValueGenerator gen;
+ @TempDir java.nio.file.Path tempDir;
+ private TestFileStore store;
+ private SnapshotManager snapshotManager;
+
+ private String branch = UUID.randomUUID() + "-branch";
+
+ @BeforeEach
+ public void beforeEach() throws Exception {
+ gen = new TestKeyValueGenerator();
+ store =
+ new TestFileStore.Builder(
+ "avro",
+ tempDir.toString(),
+ NUM_BUCKETS,
+ TestKeyValueGenerator.DEFAULT_PART_TYPE,
+ TestKeyValueGenerator.KEY_TYPE,
+ TestKeyValueGenerator.DEFAULT_ROW_TYPE,
+ TestKeyValueGenerator.TestKeyValueFieldsExtractor.EXTRACTOR,
+ DeduplicateMergeFunction.factory(),
+ branch)
+ .build();
+ snapshotManager = store.snapshotManager();
+
+ LocalFileIO localFile = LocalFileIO.create();
+
+ SchemaManager schemaManager =
+ new SchemaManager(localFile, new Path(tempDir.toUri()), branch);
+ schemaManager.createTable(
+ new Schema(
+ TestKeyValueGenerator.DEFAULT_ROW_TYPE.getFields(),
+ TestKeyValueGenerator.DEFAULT_PART_TYPE.getFieldNames(),
+ TestKeyValueGenerator.getPrimaryKeys(
+ TestKeyValueGenerator.GeneratorMode.MULTI_PARTITIONED),
+ Collections.emptyMap(),
+ null));
+ }
+
+ @Test
+ public void testWithPartitionFilter() throws Exception {
+ ThreadLocalRandom random = ThreadLocalRandom.current();
+ List data = generateData(random.nextInt(1000) + 1);
+ List partitions =
+ data.stream()
+ .map(kv -> gen.getPartition(kv))
+ .distinct()
+ .collect(Collectors.toList());
+ Snapshot snapshot = writeData(data);
+
+ Set wantedPartitions = new HashSet<>();
+ for (int i = random.nextInt(partitions.size() + 1); i > 0; i--) {
+ wantedPartitions.add(partitions.get(random.nextInt(partitions.size())));
+ }
+
+ FileStoreScan scan = store.newScan();
+ scan.withSnapshot(snapshot.id());
+ scan.withPartitionFilter(new ArrayList<>(wantedPartitions));
+
+ Map expected =
+ store.toKvMap(
+ wantedPartitions.isEmpty()
+ ? data
+ : data.stream()
+ .filter(
+ kv ->
+ wantedPartitions.contains(
+ gen.getPartition(kv)))
+ .collect(Collectors.toList()));
+ runTestExactMatch(scan, snapshot.id(), expected);
+ }
+
+ @Test
+ public void testWithKeyFilter() throws Exception {
+ ThreadLocalRandom random = ThreadLocalRandom.current();
+ List data = generateData(random.nextInt(1000) + 1);
+ Snapshot snapshot = writeData(data);
+
+ int wantedShopId = data.get(random.nextInt(data.size())).key().getInt(0);
+
+ KeyValueFileStoreScan scan = store.newScan();
+ scan.withSnapshot(snapshot.id());
+ scan.withKeyFilter(
+ new PredicateBuilder(RowType.of(new IntType(false))).equal(0, wantedShopId));
+
+ Map expected =
+ store.toKvMap(
+ data.stream()
+ .filter(kv -> kv.key().getInt(0) == wantedShopId)
+ .collect(Collectors.toList()));
+ runTestContainsAll(scan, snapshot.id(), expected);
+ }
+
+ @Test
+ public void testWithValueFilter() throws Exception {
+ ThreadLocalRandom random = ThreadLocalRandom.current();
+ List data = generateData(100, Math.abs(random.nextInt(1000)));
+ writeData(data, 0);
+ data = generateData(100, Math.abs(random.nextInt(1000)) + 1000);
+ writeData(data, 1);
+ data = generateData(100, Math.abs(random.nextInt(1000)) + 2000);
+ writeData(data, 2);
+ generateData(100, Math.abs(random.nextInt(1000)) + 3000);
+ Snapshot snapshot = writeData(data, 3);
+
+ KeyValueFileStoreScan scan = store.newScan();
+ scan.withSnapshot(snapshot.id());
+ List files = scan.plan().files();
+
+ scan = store.newScan();
+ scan.withSnapshot(snapshot.id());
+ scan.withValueFilter(
+ new PredicateBuilder(TestKeyValueGenerator.DEFAULT_ROW_TYPE)
+ .between(1, 1000, 2000));
+
+ List filesFiltered = scan.plan().files();
+
+ assertThat(files.size()).isEqualTo(4);
+ assertThat(filesFiltered.size()).isEqualTo(1);
+ }
+
+ @Test
+ public void testWithValuePartitionFilter() throws Exception {
+ ThreadLocalRandom random = ThreadLocalRandom.current();
+ List data = generateData(100, Math.abs(random.nextInt(1000)));
+ writeData(data, "0", 0);
+ data = generateData(100, Math.abs(random.nextInt(1000)) + 1000);
+ writeData(data, "1", 0);
+ data = generateData(100, Math.abs(random.nextInt(1000)) + 2000);
+ writeData(data, "2", 0);
+ generateData(100, Math.abs(random.nextInt(1000)) + 3000);
+ Snapshot snapshot = writeData(data, "3", 0);
+
+ KeyValueFileStoreScan scan = store.newScan();
+ scan.withSnapshot(snapshot.id());
+ List files = scan.plan().files();
+
+ scan = store.newScan();
+ scan.withSnapshot(snapshot.id());
+ scan.withValueFilter(
+ new PredicateBuilder(TestKeyValueGenerator.DEFAULT_ROW_TYPE)
+ .between(1, 1000, 2000));
+
+ List filesFiltered = scan.plan().files();
+
+ assertThat(files.size()).isEqualTo(4);
+ assertThat(filesFiltered.size()).isEqualTo(1);
+ }
+
+ @Test
+ public void testWithBucket() throws Exception {
+ ThreadLocalRandom random = ThreadLocalRandom.current();
+ List data = generateData(random.nextInt(1000) + 1);
+ Snapshot snapshot = writeData(data);
+
+ int wantedBucket = random.nextInt(NUM_BUCKETS);
+
+ FileStoreScan scan = store.newScan();
+ scan.withSnapshot(snapshot.id());
+ scan.withBucket(wantedBucket);
+
+ Map expected =
+ store.toKvMap(
+ data.stream()
+ .filter(kv -> getBucket(kv) == wantedBucket)
+ .collect(Collectors.toList()));
+ runTestExactMatch(scan, snapshot.id(), expected);
+ }
+
+ @Test
+ public void testWithSnapshot() throws Exception {
+ ThreadLocalRandom random = ThreadLocalRandom.current();
+ int numCommits = random.nextInt(10) + 1;
+ int wantedCommit = random.nextInt(numCommits);
+
+ List snapshots = new ArrayList<>();
+ List> allData = new ArrayList<>();
+ for (int i = 0; i < numCommits; i++) {
+ List data = generateData(random.nextInt(100) + 1);
+ snapshots.add(writeData(data));
+ allData.add(data);
+ }
+ long wantedSnapshot = snapshots.get(wantedCommit).id();
+
+ FileStoreScan scan = store.newScan();
+ scan.withSnapshot(wantedSnapshot);
+
+ Map expected =
+ store.toKvMap(
+ allData.subList(0, wantedCommit + 1).stream()
+ .flatMap(Collection::stream)
+ .collect(Collectors.toList()));
+ runTestExactMatch(scan, wantedSnapshot, expected);
+ }
+
+ @Test
+ public void testWithManifestList() throws Exception {
+ ThreadLocalRandom random = ThreadLocalRandom.current();
+ int numCommits = random.nextInt(10) + 1;
+ for (int i = 0; i < numCommits; i++) {
+ List data = generateData(random.nextInt(100) + 1);
+ writeData(data);
+ }
+
+ ManifestList manifestList = store.manifestListFactory().create();
+ long wantedSnapshotId = random.nextLong(snapshotManager.latestSnapshotId(branch)) + 1;
+ Snapshot wantedSnapshot = snapshotManager.snapshot(wantedSnapshotId);
+ List wantedManifests = wantedSnapshot.dataManifests(manifestList);
+
+ FileStoreScan scan = store.newScan();
+ scan.withManifestList(wantedManifests);
+
+ List expectedKvs = store.readKvsFromSnapshot(wantedSnapshotId);
+ gen.sort(expectedKvs);
+ Map expected = store.toKvMap(expectedKvs);
+ runTestExactMatch(scan, null, expected);
+ }
+
+ private void runTestExactMatch(
+ FileStoreScan scan, Long expectedSnapshotId, Map expected)
+ throws Exception {
+ Map actual = getActualKvMap(scan, expectedSnapshotId);
+ assertThat(actual).isEqualTo(expected);
+ }
+
+ private void runTestContainsAll(
+ FileStoreScan scan, Long expectedSnapshotId, Map expected)
+ throws Exception {
+ Map actual = getActualKvMap(scan, expectedSnapshotId);
+ for (Map.Entry entry : expected.entrySet()) {
+ assertThat(actual).containsKey(entry.getKey());
+ assertThat(actual.get(entry.getKey())).isEqualTo(entry.getValue());
+ }
+ }
+
+ private Map getActualKvMap(FileStoreScan scan, Long expectedSnapshotId)
+ throws Exception {
+ FileStoreScan.Plan plan = scan.plan();
+ assertThat(plan.snapshotId()).isEqualTo(expectedSnapshotId);
+
+ List actualKvs = store.readKvsFromManifestEntries(plan.files(), false);
+ gen.sort(actualKvs);
+ return store.toKvMap(actualKvs);
+ }
+
+ private List generateData(int numRecords) {
+ List data = new ArrayList<>();
+ for (int i = 0; i < numRecords; i++) {
+ data.add(gen.next());
+ }
+ return data;
+ }
+
+ private List generateData(int numRecords, int hr) {
+ List data = new ArrayList<>();
+ for (int i = 0; i < numRecords; i++) {
+ data.add(gen.nextInsert("", hr, null, null, null));
+ }
+ return data;
+ }
+
+ private Snapshot writeData(List kvs) throws Exception {
+ List snapshots = store.commitData(kvs, gen::getPartition, this::getBucket);
+ return snapshots.get(snapshots.size() - 1);
+ }
+
+ private Snapshot writeData(List kvs, int bucket) throws Exception {
+ List snapshots = store.commitData(kvs, gen::getPartition, b -> bucket);
+ return snapshots.get(snapshots.size() - 1);
+ }
+
+ private Snapshot writeData(List kvs, String partition, int bucket) throws Exception {
+ BinaryRow binaryRow = new BinaryRow(2);
+ BinaryRowWriter binaryRowWriter = new BinaryRowWriter(binaryRow);
+ binaryRowWriter.writeString(0, BinaryString.fromString(partition));
+ binaryRowWriter.writeInt(1, 0);
+ binaryRowWriter.complete();
+ List snapshots = store.commitData(kvs, p -> binaryRow, b -> bucket);
+ return snapshots.get(snapshots.size() - 1);
+ }
+
+ private int getBucket(KeyValue kv) {
+ return (kv.key().hashCode() % NUM_BUCKETS + NUM_BUCKETS) % NUM_BUCKETS;
+ }
+}
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 4cd019568c41f..03d6bc5ead59e 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
@@ -949,22 +949,21 @@ public void testCreateBranch() throws Exception {
// verify test-tag in test-branch is equal to snapshot 2
Snapshot branchTag =
Snapshot.fromPath(
- new TraceableFileIO(), tagManager.branchTagPath("test-branch", "test-tag"));
+ new TraceableFileIO(), tagManager.tagPath("test-branch", "test-tag"));
assertThat(branchTag.equals(snapshot2)).isTrue();
// verify snapshot in test-branch is equal to snapshot 2
SnapshotManager snapshotManager = new SnapshotManager(new TraceableFileIO(), tablePath);
Snapshot branchSnapshot =
Snapshot.fromPath(
- new TraceableFileIO(),
- snapshotManager.branchSnapshotPath("test-branch", 2));
+ new TraceableFileIO(), snapshotManager.snapshotPath("test-branch", 2));
assertThat(branchSnapshot.equals(snapshot2)).isTrue();
// verify schema in test-branch is equal to schema 0
SchemaManager schemaManager = new SchemaManager(new TraceableFileIO(), tablePath);
TableSchema branchSchema =
SchemaManager.fromPath(
- new TraceableFileIO(), schemaManager.branchSchemaPath("test-branch", 0));
+ new TraceableFileIO(), schemaManager.toSchemaPath("test-branch", 0));
TableSchema schema0 = schemaManager.schema(0);
assertThat(branchSchema.equals(schema0)).isTrue();
}
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 a55bda9118a56..f5874bed7eac7 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
@@ -502,12 +502,6 @@ public Optional latest() {
.orElseThrow(IllegalStateException::new)));
}
- @Override
- public Optional latest(String branchName) {
- // for compatibility test
- return latest();
- }
-
@Override
public List listAll() {
return new ArrayList<>(tableSchemas.values());
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 0c60ea26524fd..d279da0c581ae 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
@@ -18,6 +18,7 @@
package org.apache.paimon.flink.sink.cdc;
+import org.apache.paimon.CoreOptions;
import org.apache.paimon.annotation.Experimental;
import org.apache.paimon.catalog.Catalog;
import org.apache.paimon.catalog.Identifier;
@@ -26,6 +27,7 @@
import org.apache.paimon.table.BucketMode;
import org.apache.paimon.table.FileStoreTable;
import org.apache.paimon.table.Table;
+import org.apache.paimon.utils.BranchManager;
import org.apache.paimon.utils.Preconditions;
import org.apache.flink.streaming.api.datastream.DataStream;
@@ -64,6 +66,7 @@ public CdcSinkBuilder withParserFactory(EventParser.Factory parserFactory)
public CdcSinkBuilder withTable(Table table) {
this.table = table;
+ this.table.options().put(CoreOptions.BRANCH.key(), BranchManager.DEFAULT_MAIN_BRANCH);
return this;
}
@@ -106,7 +109,10 @@ 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.location(),
+ CoreOptions.branch(dataTable.options())),
identifier,
catalogLoader));
schemaChangeProcessFunction.getTransformation().setParallelism(1);
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 6d9a227f1355b..4285ebd362416 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
@@ -18,6 +18,7 @@
package org.apache.paimon.flink.sink.cdc;
+import org.apache.paimon.CoreOptions;
import org.apache.paimon.catalog.Catalog;
import org.apache.paimon.catalog.Identifier;
import org.apache.paimon.flink.FlinkConnectorOptions;
@@ -190,7 +191,10 @@ private void buildDividedCdcSink() {
.createUpdatedDataFieldsOutputTag(table.name()))
.process(
new UpdatedDataFieldsProcessFunction(
- new SchemaManager(table.fileIO(), table.location()),
+ new SchemaManager(
+ table.fileIO(),
+ table.location(),
+ CoreOptions.branch(table.options())),
Identifier.create(database, table.name()),
catalogLoader));
schemaChangeProcessFunction.getTransformation().setParallelism(1);
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 0ad412e47d349..8d822dfe89dd0 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
@@ -18,6 +18,7 @@
package org.apache.paimon.flink.sink.cdc;
+import org.apache.paimon.CoreOptions;
import org.apache.paimon.catalog.Catalog;
import org.apache.paimon.catalog.Identifier;
import org.apache.paimon.schema.SchemaChange;
@@ -72,7 +73,10 @@ public void processElement(
} catch (Catalog.TableNotExistException e) {
return null;
}
- return new SchemaManager(table.fileIO(), table.location());
+ return new SchemaManager(
+ table.fileIO(),
+ table.location(),
+ CoreOptions.branch(table.options()));
});
if (Objects.isNull(schemaManager)) {
diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java
index 7a621030217a6..6ff67fb5aa455 100644
--- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java
+++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java
@@ -54,6 +54,7 @@
import java.util.Map;
import java.util.concurrent.ThreadLocalRandom;
+import static org.apache.paimon.CoreOptions.BRANCH;
import static org.assertj.core.api.Assertions.assertThat;
/** CDC IT case base. */
@@ -72,6 +73,16 @@ public void setEnv() {
env.setRestartStrategy(RestartStrategies.noRestart());
}
+ protected Map getCatalogOptions(Map catalogOptions) {
+ catalogOptions.put(BRANCH.key(), branch);
+ return catalogOptions;
+ }
+
+ protected Map getTableConfig(Map tableConfig) {
+ tableConfig.put(BRANCH.key(), branch);
+ return tableConfig;
+ }
+
@AfterEach
public void closeEnv() throws Exception {
env.close();
diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncDatabaseToBranchActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncDatabaseToBranchActionITCase.java
new file mode 100644
index 0000000000000..53246624744f2
--- /dev/null
+++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncDatabaseToBranchActionITCase.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.paimon.flink.action.cdc.kafka;
+
+import org.junit.jupiter.api.BeforeEach;
+
+import java.io.IOException;
+
+/** IT cases for {@link KafkaSyncDatabaseAction}. */
+public class KafkaDebeziumSyncDatabaseToBranchActionITCase
+ extends KafkaDebeziumSyncDatabaseActionITCase {
+ @BeforeEach
+ public void before() throws IOException {
+ branch = "testKafkaDebeziumSyncDatabaseBranch";
+ super.before();
+ }
+}
diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncTableToBranchActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncTableToBranchActionITCase.java
new file mode 100644
index 0000000000000..77860bbb37e8b
--- /dev/null
+++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncTableToBranchActionITCase.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.paimon.flink.action.cdc.kafka;
+
+import org.junit.jupiter.api.BeforeEach;
+
+import java.io.IOException;
+
+/** IT cases for {@link KafkaSyncTableAction}. */
+public class KafkaDebeziumSyncTableToBranchActionITCase extends KafkaDebeziumSyncTableActionITCase {
+
+ @BeforeEach
+ public void before() throws IOException {
+ branch = "testKafkaDebeziumSyncTableBranch";
+ super.before();
+ }
+}
diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java
index cdad175ea15fa..cf474ea60f318 100644
--- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java
+++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java
@@ -28,6 +28,7 @@
import java.util.Arrays;
import java.util.Collections;
+import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.UUID;
@@ -69,7 +70,8 @@ protected void testSchemaEvolutionMultiTopic(String format) throws Exception {
kafkaConfig.put(TOPIC.key(), String.join(";", topics));
KafkaSyncDatabaseAction action =
syncDatabaseActionBuilder(kafkaConfig)
- .withTableConfig(getBasicTableConfig())
+ .withTableConfig(getTableConfig(getBasicTableConfig()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
.build();
runActionWithDefaultEnv(action);
@@ -103,7 +105,8 @@ protected void testSchemaEvolutionOneTopic(String format) throws Exception {
kafkaConfig.put(TOPIC.key(), String.join(";", topics));
KafkaSyncDatabaseAction action =
syncDatabaseActionBuilder(kafkaConfig)
- .withTableConfig(getBasicTableConfig())
+ .withTableConfig(getTableConfig(getBasicTableConfig()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
.build();
runActionWithDefaultEnv(action);
@@ -112,6 +115,7 @@ protected void testSchemaEvolutionOneTopic(String format) throws Exception {
private void testSchemaEvolutionImpl(
List topics, boolean writeOne, int fileCount, String format) throws Exception {
+
waitingTables("t1", "t2");
FileStoreTable table1 = getFileStoreTable("t1");
@@ -212,7 +216,11 @@ protected void testTopicIsEmpty(String format) {
Map kafkaConfig = getBasicKafkaConfig();
kafkaConfig.put(VALUE_FORMAT.key(), format + "-json");
- KafkaSyncDatabaseAction action = syncDatabaseActionBuilder(kafkaConfig).build();
+ KafkaSyncDatabaseAction action =
+ syncDatabaseActionBuilder(kafkaConfig)
+ .withTableConfig(getTableConfig(new HashMap<>()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
+ .build();
assertThatThrownBy(action::run)
.satisfies(
@@ -267,7 +275,8 @@ protected void testTableAffixMultiTopic(String format) throws Exception {
syncDatabaseActionBuilder(kafkaConfig)
.withTablePrefix("test_prefix_")
.withTableSuffix("_test_suffix")
- .withTableConfig(getBasicTableConfig())
+ .withTableConfig(getTableConfig(getBasicTableConfig()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
// test including check with affix
.includingTables(ThreadLocalRandom.current().nextBoolean() ? "t1|t2" : ".*")
.build();
@@ -321,7 +330,8 @@ protected void testTableAffixOneTopic(String format) throws Exception {
syncDatabaseActionBuilder(kafkaConfig)
.withTablePrefix("test_prefix_")
.withTableSuffix("_test_suffix")
- .withTableConfig(getBasicTableConfig())
+ .withTableConfig(getTableConfig(getBasicTableConfig()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
// test including check with affix
.includingTables(ThreadLocalRandom.current().nextBoolean() ? "t1|t2" : ".*")
.build();
@@ -490,7 +500,8 @@ private void includingAndExcludingTablesImpl(
syncDatabaseActionBuilder(kafkaConfig)
.includingTables(includingTables)
.excludingTables(excludingTables)
- .withTableConfig(getBasicTableConfig())
+ .withTableConfig(getTableConfig(getBasicTableConfig()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
.build();
runActionWithDefaultEnv(action);
@@ -516,12 +527,13 @@ protected void testCaseInsensitive(String format) throws Exception {
kafkaConfig.put(VALUE_FORMAT.key(), format + "-json");
kafkaConfig.put(TOPIC.key(), topic);
+ Map catalogConfig = new HashMap<>();
+ catalogConfig.put(FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false");
+
KafkaSyncDatabaseAction action =
syncDatabaseActionBuilder(kafkaConfig)
- .withTableConfig(getBasicTableConfig())
- .withCatalogConfig(
- Collections.singletonMap(
- FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false"))
+ .withTableConfig(getTableConfig(getBasicTableConfig()))
+ .withCatalogConfig(getCatalogOptions(catalogConfig))
.build();
runActionWithDefaultEnv(action);
diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java
index f2f8fc246682d..e6e6310b5130f 100644
--- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java
+++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java
@@ -33,6 +33,7 @@
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
+import java.util.HashMap;
import java.util.List;
import java.util.Map;
@@ -74,7 +75,8 @@ protected void runSingleTableSchemaEvolution(String sourceDir, String format) th
KafkaSyncTableAction action =
syncTableActionBuilder(kafkaConfig)
.withPrimaryKeys("id")
- .withTableConfig(getBasicTableConfig())
+ .withTableConfig(getTableConfig(getBasicTableConfig()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
.build();
runActionWithDefaultEnv(action);
@@ -177,7 +179,8 @@ public void testNotSupportFormat(String format) throws Exception {
KafkaSyncTableAction action =
syncTableActionBuilder(kafkaConfig)
.withPrimaryKeys("id")
- .withTableConfig(getBasicTableConfig())
+ .withTableConfig(getTableConfig(getBasicTableConfig()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
.build();
assertThatThrownBy(action::run)
@@ -216,7 +219,8 @@ protected void testAssertSchemaCompatible(String format) throws Exception {
KafkaSyncTableAction action =
syncTableActionBuilder(kafkaConfig)
.withPrimaryKeys("id")
- .withTableConfig(getBasicTableConfig())
+ .withTableConfig(getTableConfig(getBasicTableConfig()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
.build();
assertThatThrownBy(action::run)
@@ -248,7 +252,8 @@ protected void testStarUpOptionSpecific(String format) throws Exception {
KafkaSyncTableAction action =
syncTableActionBuilder(kafkaConfig)
.withPrimaryKeys("id")
- .withTableConfig(getBasicTableConfig())
+ .withTableConfig(getTableConfig(getBasicTableConfig()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
.build();
runActionWithDefaultEnv(action);
@@ -289,7 +294,8 @@ protected void testStarUpOptionLatest(String format) throws Exception {
KafkaSyncTableAction action =
syncTableActionBuilder(kafkaConfig)
.withPrimaryKeys("id")
- .withTableConfig(getBasicTableConfig())
+ .withTableConfig(getTableConfig(getBasicTableConfig()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
.build();
runActionWithDefaultEnv(action);
@@ -344,7 +350,8 @@ public void testStarUpOptionTimestamp(String format) throws Exception {
KafkaSyncTableAction action =
syncTableActionBuilder(kafkaConfig)
.withPrimaryKeys("id")
- .withTableConfig(getBasicTableConfig())
+ .withTableConfig(getTableConfig(getBasicTableConfig()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
.build();
runActionWithDefaultEnv(action);
@@ -396,7 +403,8 @@ public void testStarUpOptionEarliest(String format) throws Exception {
KafkaSyncTableAction action =
syncTableActionBuilder(kafkaConfig)
.withPrimaryKeys("id")
- .withTableConfig(getBasicTableConfig())
+ .withTableConfig(getTableConfig(getBasicTableConfig()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
.build();
runActionWithDefaultEnv(action);
@@ -450,7 +458,8 @@ public void testStarUpOptionGroup(String format) throws Exception {
KafkaSyncTableAction action =
syncTableActionBuilder(kafkaConfig)
.withPrimaryKeys("id")
- .withTableConfig(getBasicTableConfig())
+ .withTableConfig(getTableConfig(getBasicTableConfig()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
.build();
runActionWithDefaultEnv(action);
@@ -506,7 +515,8 @@ public void testComputedColumn(String format) throws Exception {
.withPartitionKeys("_year")
.withPrimaryKeys("_id", "_year")
.withComputedColumnArgs("_year=year(_date)")
- .withTableConfig(getBasicTableConfig())
+ .withTableConfig(getTableConfig(getBasicTableConfig()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
.build();
runActionWithDefaultEnv(action);
@@ -542,7 +552,8 @@ protected void testCDCOperations(String format) throws Exception {
KafkaSyncTableAction action =
syncTableActionBuilder(kafkaConfig)
.withPrimaryKeys("id")
- .withTableConfig(getBasicTableConfig())
+ .withTableConfig(getTableConfig(getBasicTableConfig()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
.build();
runActionWithDefaultEnv(action);
@@ -648,14 +659,17 @@ public void testWaterMarkSyncTable(String format) throws Exception {
config.put("scan.watermark.alignment.update-interval", "1 s");
KafkaSyncTableAction action =
- syncTableActionBuilder(kafkaConfig).withTableConfig(config).build();
+ syncTableActionBuilder(kafkaConfig)
+ .withTableConfig(getTableConfig(config))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
+ .build();
runActionWithDefaultEnv(action);
FileStoreTable table =
(FileStoreTable) catalog.getTable(new Identifier(database, tableName));
while (true) {
- if (table.snapshotManager().snapshotCount() > 0
- && table.snapshotManager().latestSnapshot().watermark()
+ if (table.snapshotManager().snapshotCount(branch) > 0
+ && table.snapshotManager().latestSnapshot(branch).watermark()
!= -9223372036854775808L) {
return;
}
@@ -680,7 +694,8 @@ public void testSchemaIncludeRecord(String format) throws Exception {
KafkaSyncTableAction action =
syncTableActionBuilder(kafkaConfig)
.withPrimaryKeys("id")
- .withTableConfig(getBasicTableConfig())
+ .withTableConfig(getTableConfig(getBasicTableConfig()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
.build();
runActionWithDefaultEnv(action);
@@ -717,7 +732,8 @@ public void testAllTypesWithSchemaImpl(String format) throws Exception {
syncTableActionBuilder(kafkaConfig)
.withPartitionKeys("pt")
.withPrimaryKeys("pt", "_id")
- .withTableConfig(getBasicTableConfig())
+ .withTableConfig(getTableConfig(getBasicTableConfig()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
.build();
runActionWithDefaultEnv(action);
@@ -969,7 +985,8 @@ protected void testTableFiledValNull(String format) throws Exception {
KafkaSyncTableAction action =
syncTableActionBuilder(kafkaConfig)
.withPrimaryKeys("id")
- .withTableConfig(getBasicTableConfig())
+ .withTableConfig(getTableConfig(getBasicTableConfig()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
.build();
runActionWithDefaultEnv(action);
diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseActionITCase.java
index 84169ffe02b76..a00abd48369f1 100644
--- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseActionITCase.java
+++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseActionITCase.java
@@ -79,7 +79,8 @@ public void testSchemaEvolution() throws Exception {
MySqlSyncDatabaseAction action =
syncDatabaseActionBuilder(mySqlConfig)
- .withTableConfig(getBasicTableConfig())
+ .withTableConfig(getTableConfig(getBasicTableConfig()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
.build();
runActionWithDefaultEnv(action);
@@ -207,7 +208,11 @@ public void testSpecifiedMySqlTable() {
mySqlConfig.put("database-name", "paimon_sync_database");
mySqlConfig.put("table-name", "my_table");
- MySqlSyncDatabaseAction action = syncDatabaseActionBuilder(mySqlConfig).build();
+ MySqlSyncDatabaseAction action =
+ syncDatabaseActionBuilder(mySqlConfig)
+ .withTableConfig(getTableConfig(new HashMap<>()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
+ .build();
assertThatThrownBy(action::run)
.isInstanceOf(IllegalArgumentException.class)
@@ -222,7 +227,11 @@ public void testInvalidDatabase() {
Map mySqlConfig = getBasicMySqlConfig();
mySqlConfig.put("database-name", "invalid");
- MySqlSyncDatabaseAction action = syncDatabaseActionBuilder(mySqlConfig).build();
+ MySqlSyncDatabaseAction action =
+ syncDatabaseActionBuilder(mySqlConfig)
+ .withTableConfig(getTableConfig(new HashMap<>()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
+ .build();
assertThatThrownBy(action::run)
.isInstanceOf(IllegalArgumentException.class)
@@ -249,7 +258,8 @@ public void testIgnoreIncompatibleTables() throws Exception {
MySqlSyncDatabaseAction action =
syncDatabaseActionBuilder(mySqlConfig)
- .withTableConfig(getBasicTableConfig())
+ .withTableConfig(getTableConfig(getBasicTableConfig()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
.ignoreIncompatible(true)
.build();
runActionWithDefaultEnv(action);
@@ -296,7 +306,8 @@ public void testTableAffix() throws Exception {
MySqlSyncDatabaseAction action =
syncDatabaseActionBuilder(mySqlConfig)
- .withTableConfig(getBasicTableConfig())
+ .withTableConfig(getTableConfig(getBasicTableConfig()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
.withTablePrefix("test_prefix_")
.withTableSuffix("_test_suffix")
// test including check with affix
@@ -453,7 +464,8 @@ private void includingAndExcludingTablesImpl(
MySqlSyncDatabaseAction action =
syncDatabaseActionBuilder(mySqlConfig)
- .withTableConfig(getBasicTableConfig())
+ .withTableConfig(getTableConfig(getBasicTableConfig()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
.includingTables(includingTables)
.excludingTables(excludingTables)
.build();
@@ -470,12 +482,13 @@ public void testIgnoreCase() throws Exception {
Map mySqlConfig = getBasicMySqlConfig();
mySqlConfig.put("database-name", "paimon_ignore_CASE");
+ Map catalogConfig = new HashMap<>();
+ catalogConfig.put(FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false");
+
MySqlSyncDatabaseAction action =
syncDatabaseActionBuilder(mySqlConfig)
- .withCatalogConfig(
- Collections.singletonMap(
- FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false"))
- .withTableConfig(getBasicTableConfig())
+ .withTableConfig(getTableConfig(getBasicTableConfig()))
+ .withCatalogConfig(getCatalogOptions(catalogConfig))
.build();
runActionWithDefaultEnv(action);
@@ -569,6 +582,8 @@ public void testAddIgnoredTable() throws Exception {
MySqlSyncDatabaseAction action =
syncDatabaseActionBuilder(mySqlConfig)
.withTableConfig(getBasicTableConfig())
+ .withTableConfig(getTableConfig(getBasicTableConfig()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
.includingTables("t.+")
.excludingTables(".*a$")
.withMode(COMBINED.configString())
@@ -846,16 +861,15 @@ private JobClient buildSyncDatabaseActionWithNewlyAddedTables(
mySqlConfig.put("database-name", databaseName);
mySqlConfig.put("scan.incremental.snapshot.chunk.size", "1");
- Map catalogConfig =
- testSchemaChange
- ? Collections.singletonMap(
- CatalogOptions.METASTORE.key(), "test-alter-table")
- : Collections.emptyMap();
+ Map catalogConfig = new HashMap<>();
+ if (testSchemaChange) {
+ catalogConfig.put(CatalogOptions.METASTORE.key(), "test-alter-table");
+ }
MySqlSyncDatabaseAction action =
syncDatabaseActionBuilder(mySqlConfig)
- .withCatalogConfig(catalogConfig)
- .withTableConfig(getBasicTableConfig())
+ .withTableConfig(getTableConfig(getBasicTableConfig()))
+ .withCatalogConfig(getCatalogOptions(catalogConfig))
.includingTables("t.+")
.withMode(COMBINED.configString())
.build();
@@ -895,7 +909,8 @@ public void testSyncManyTableWithLimitedMemory() throws Exception {
MySqlSyncDatabaseAction action =
syncDatabaseActionBuilder(mySqlConfig)
- .withTableConfig(tableConfig)
+ .withTableConfig(getTableConfig(tableConfig))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
.withMode(COMBINED.configString())
.build();
runActionWithDefaultEnv(action);
@@ -942,7 +957,8 @@ public void testSyncMultipleShards() throws Exception {
MultiTablesSinkMode mode = ThreadLocalRandom.current().nextBoolean() ? DIVIDED : COMBINED;
MySqlSyncDatabaseAction action =
syncDatabaseActionBuilder(mySqlConfig)
- .withTableConfig(getBasicTableConfig())
+ .withTableConfig(getTableConfig(getBasicTableConfig()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
.withMode(mode.configString())
.build();
runActionWithDefaultEnv(action);
@@ -1050,7 +1066,8 @@ public void testSyncMultipleShardsWithoutMerging() throws Exception {
MultiTablesSinkMode mode = ThreadLocalRandom.current().nextBoolean() ? DIVIDED : COMBINED;
MySqlSyncDatabaseAction action =
syncDatabaseActionBuilder(mySqlConfig)
- .withTableConfig(getBasicTableConfig())
+ .withTableConfig(getTableConfig(getBasicTableConfig()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
.mergeShards(false)
.withMode(mode.configString())
.build();
@@ -1165,6 +1182,8 @@ public void testMonitoredAndExcludedTablesWithMering() throws Exception {
MySqlSyncDatabaseAction action =
syncDatabaseActionBuilder(mySqlConfig)
+ .withTableConfig(getTableConfig(new HashMap<>()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
.ignoreIncompatible(true)
.withMode(COMBINED.configString())
.build();
@@ -1202,7 +1221,8 @@ public void testNewlyAddedTablesOptionsChange() throws Exception {
MySqlSyncDatabaseAction action1 =
syncDatabaseActionBuilder(mySqlConfig)
- .withTableConfig(tableConfig)
+ .withTableConfig(getTableConfig(tableConfig))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
.withMode(COMBINED.configString())
.build();
@@ -1228,7 +1248,10 @@ public void testNewlyAddedTablesOptionsChange() throws Exception {
}
MySqlSyncDatabaseAction action2 =
- syncDatabaseActionBuilder(mySqlConfig).withTableConfig(tableConfig).build();
+ syncDatabaseActionBuilder(mySqlConfig)
+ .withTableConfig(getTableConfig(tableConfig))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
+ .build();
runActionWithDefaultEnv(action2);
waitingTables("t2");
@@ -1238,15 +1261,20 @@ public void testNewlyAddedTablesOptionsChange() throws Exception {
@Test
public void testCatalogAndTableConfig() {
+ Map catalogConfig = new HashMap<>();
+ catalogConfig.put("catalog-key", "catalog-value");
+
+ Map tableConfig = new HashMap<>();
+ tableConfig.put("table-key", "table-value");
+
MySqlSyncDatabaseAction action =
syncDatabaseActionBuilder(getBasicMySqlConfig())
- .withCatalogConfig(Collections.singletonMap("catalog-key", "catalog-value"))
- .withTableConfig(Collections.singletonMap("table-key", "table-value"))
+ .withTableConfig(getTableConfig(tableConfig))
+ .withCatalogConfig(getCatalogOptions(catalogConfig))
.build();
assertThat(action.catalogConfig()).containsEntry("catalog-key", "catalog-value");
- assertThat(action.tableConfig())
- .containsExactlyEntriesOf(Collections.singletonMap("table-key", "table-value"));
+ assertThat(action.tableConfig()).containsExactlyEntriesOf(getTableConfig(tableConfig));
}
@Test
@@ -1258,7 +1286,8 @@ public void testMetadataColumns() throws Exception {
MultiTablesSinkMode mode = ThreadLocalRandom.current().nextBoolean() ? DIVIDED : COMBINED;
MySqlSyncDatabaseAction action =
syncDatabaseActionBuilder(mySqlConfig)
- .withTableConfig(getBasicTableConfig())
+ .withTableConfig(getTableConfig(getBasicTableConfig()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
.withMode(mode.configString())
.withMetadataColumn(Arrays.asList("table_name", "database_name"))
.build();
diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseActionToBranchITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseActionToBranchITCase.java
new file mode 100644
index 0000000000000..e6cf68455bb10
--- /dev/null
+++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseActionToBranchITCase.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.paimon.flink.action.cdc.mysql;
+
+import org.junit.jupiter.api.BeforeEach;
+
+import java.io.IOException;
+
+/** IT cases for {@link MySqlSyncDatabaseAction}. */
+public class MySqlSyncDatabaseActionToBranchITCase extends MySqlSyncDatabaseActionITCase {
+
+ @BeforeEach
+ @Override
+ public void before() throws IOException {
+ this.branch = "testMySqlSyncDatabaseActionBranch";
+ super.before();
+ }
+}
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 6cd35eec83284..bfa5020f47776 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
@@ -77,12 +77,13 @@ public void testSchemaEvolution() throws Exception {
mySqlConfig.put("database-name", DATABASE_NAME);
mySqlConfig.put("table-name", "schema_evolution_\\d+");
+ Map catalogConfig = getBasicMySqlConfig();
+ catalogConfig.put(CatalogOptions.METASTORE.key(), "test-alter-table");
+
MySqlSyncTableAction action =
syncTableActionBuilder(mySqlConfig)
- .withCatalogConfig(
- Collections.singletonMap(
- CatalogOptions.METASTORE.key(), "test-alter-table"))
- .withTableConfig(getBasicTableConfig())
+ .withCatalogConfig(getCatalogOptions(catalogConfig))
+ .withTableConfig(getTableConfig(getBasicTableConfig()))
.withPartitionKeys("pt")
.withPrimaryKeys("pt", "_id")
.build();
@@ -256,7 +257,11 @@ public void testMultipleSchemaEvolutions() throws Exception {
mySqlConfig.put("database-name", DATABASE_NAME);
mySqlConfig.put("table-name", "schema_evolution_multiple");
- MySqlSyncTableAction action = syncTableActionBuilder(mySqlConfig).build();
+ MySqlSyncTableAction action =
+ syncTableActionBuilder(mySqlConfig)
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
+ .withTableConfig(getTableConfig(new HashMap<>()))
+ .build();
runActionWithDefaultEnv(action);
checkTableSchema(
@@ -321,7 +326,7 @@ private void testSchemaEvolutionMultipleImpl(Statement statement) throws Excepti
}
@Test
- @Timeout(90)
+ @Timeout(180)
public void testAllTypes() throws Exception {
// the first round checks for table creation
// the second round checks for running the action on an existing table
@@ -330,13 +335,15 @@ public void testAllTypes() throws Exception {
}
}
- private void testAllTypesOnce() throws Exception {
+ protected void testAllTypesOnce() throws Exception {
Map mySqlConfig = getBasicMySqlConfig();
mySqlConfig.put("database-name", DATABASE_NAME);
mySqlConfig.put("table-name", "all_types_table");
MySqlSyncTableAction action =
syncTableActionBuilder(mySqlConfig)
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
+ .withTableConfig(getTableConfig(new HashMap<>()))
.withPartitionKeys("pt")
.withPrimaryKeys("pt", "_id")
.build();
@@ -616,7 +623,7 @@ private void testAllTypesImpl(Statement statement) throws Exception {
} finally {
statement.executeUpdate("ALTER TABLE all_types_table DROP COLUMN v");
SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.location());
- schemaManager.commitChanges(SchemaChange.dropColumn("v"));
+ schemaManager.commitChanges(branch, SchemaChange.dropColumn("v"));
}
}
@@ -626,7 +633,11 @@ public void testIncompatibleMySqlTable() {
mySqlConfig.put("database-name", DATABASE_NAME);
mySqlConfig.put("table-name", "incompatible_field_\\d+");
- MySqlSyncTableAction action = syncTableActionBuilder(mySqlConfig).build();
+ MySqlSyncTableAction action =
+ syncTableActionBuilder(mySqlConfig)
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
+ .withTableConfig(getTableConfig(new HashMap<>()))
+ .build();
assertThatThrownBy(action::run)
.satisfies(
@@ -652,7 +663,11 @@ public void testIncompatiblePaimonTable() throws Exception {
new HashMap<>());
MySqlSyncTableAction action =
- syncTableActionBuilder(mySqlConfig).withPrimaryKeys("a").build();
+ syncTableActionBuilder(mySqlConfig)
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
+ .withTableConfig(getTableConfig(new HashMap<>()))
+ .withPrimaryKeys("a")
+ .build();
assertThatThrownBy(action::run)
.satisfies(
@@ -668,7 +683,11 @@ public void testInvalidPrimaryKey() {
mySqlConfig.put("table-name", "schema_evolution_\\d+");
MySqlSyncTableAction action =
- syncTableActionBuilder(mySqlConfig).withPrimaryKeys("pk").build();
+ syncTableActionBuilder(mySqlConfig)
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
+ .withTableConfig(getTableConfig(new HashMap<>()))
+ .withPrimaryKeys("pk")
+ .build();
assertThatThrownBy(action::run)
.satisfies(
@@ -683,7 +702,11 @@ public void testNoPrimaryKey() {
mySqlConfig.put("database-name", DATABASE_NAME);
mySqlConfig.put("table-name", "incompatible_pk_\\d+");
- MySqlSyncTableAction action = syncTableActionBuilder(mySqlConfig).build();
+ MySqlSyncTableAction action =
+ syncTableActionBuilder(mySqlConfig)
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
+ .withTableConfig(getTableConfig(new HashMap<>()))
+ .build();
assertThatThrownBy(action::run)
.satisfies(
@@ -695,7 +718,7 @@ public void testNoPrimaryKey() {
}
@Test
- @Timeout(60)
+ @Timeout(240)
public void testComputedColumn() throws Exception {
// the first round checks for table creation
// the second round checks for running the action on an existing table
@@ -738,6 +761,8 @@ private void innerTestComputedColumn(boolean executeMysql) throws Exception {
MySqlSyncTableAction action =
syncTableActionBuilder(mySqlConfig)
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
+ .withTableConfig(getTableConfig(new HashMap<>()))
.withPartitionKeys("_year_date")
.withPrimaryKeys("pk", "_year_date")
.withComputedColumnArgs(computedColumnDefs)
@@ -854,6 +879,8 @@ public void testTemporalToIntWithEpochTime() throws Exception {
MySqlSyncTableAction action =
syncTableActionBuilder(mySqlConfig)
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
+ .withTableConfig(getTableConfig(new HashMap<>()))
.withComputedColumnArgs(computedColumnDef)
.build();
runActionWithDefaultEnv(action);
@@ -921,6 +948,8 @@ public void testDateFormatWithEpochTime() throws Exception {
MySqlSyncTableAction action =
syncTableActionBuilder(mySqlConfig)
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
+ .withTableConfig(getTableConfig(new HashMap<>()))
.withComputedColumnArgs(computedColumnDefs)
.build();
runActionWithDefaultEnv(action);
@@ -1010,6 +1039,8 @@ public void testSyncShards() throws Exception {
MySqlSyncTableAction action =
syncTableActionBuilder(mySqlConfig)
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
+ .withTableConfig(getTableConfig(new HashMap<>()))
.withPartitionKeys("pt")
.withPrimaryKeys("pk", "pt")
.withComputedColumnArgs("pt=substring(_date,5)")
@@ -1061,7 +1092,8 @@ public void testOptionsChange() throws Exception {
.withPartitionKeys("pt")
.withPrimaryKeys("pk", "pt")
.withComputedColumnArgs("pt=substring(_date,5)")
- .withTableConfig(tableConfig)
+ .withTableConfig(getTableConfig(tableConfig))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
.build();
JobClient jobClient = runActionWithDefaultEnv(action1);
try (Statement statement = getStatement()) {
@@ -1089,7 +1121,8 @@ public void testOptionsChange() throws Exception {
.withPartitionKeys("pt")
.withPrimaryKeys("pk", "pt")
.withComputedColumnArgs("pt=substring(_date,5)")
- .withTableConfig(tableConfig)
+ .withTableConfig(getTableConfig(tableConfig))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
.build();
runActionWithDefaultEnv(action2);
@@ -1111,6 +1144,8 @@ public void testMetadataColumns() throws Exception {
MySqlSyncTableAction action =
syncTableActionBuilder(mySqlConfig)
+ .withTableConfig(getTableConfig(new HashMap<>()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
.withPrimaryKeys("pk")
.withMetadataColumns("table_name", "database_name", "op_ts")
.build();
@@ -1148,15 +1183,20 @@ public void testMetadataColumns() throws Exception {
@Test
public void testCatalogAndTableConfig() {
+ Map catalogOptions = new HashMap<>();
+ catalogOptions.put("catalog-key", "catalog-value");
+
+ Map tableConfig = new HashMap<>();
+ tableConfig.put("table-key", "table-value");
+
MySqlSyncTableAction action =
syncTableActionBuilder(getBasicMySqlConfig())
- .withCatalogConfig(Collections.singletonMap("catalog-key", "catalog-value"))
- .withTableConfig(Collections.singletonMap("table-key", "table-value"))
+ .withTableConfig(getTableConfig(tableConfig))
+ .withCatalogConfig(getCatalogOptions(catalogOptions))
.build();
assertThat(action.catalogConfig()).containsEntry("catalog-key", "catalog-value");
- assertThat(action.tableConfig())
- .containsExactlyEntriesOf(Collections.singletonMap("table-key", "table-value"));
+ assertThat(action.tableConfig()).containsExactlyEntriesOf(getTableConfig(tableConfig));
}
private FileStoreTable getFileStoreTable() throws Exception {
@@ -1173,7 +1213,11 @@ public void testDefaultCheckpointInterval() throws Exception {
StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
env.setRestartStrategy(RestartStrategies.noRestart());
- MySqlSyncTableAction action = syncTableActionBuilder(mySqlConfig).build();
+ MySqlSyncTableAction action =
+ syncTableActionBuilder(mySqlConfig)
+ .withTableConfig(getTableConfig(new HashMap<>()))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
+ .build();
action.withStreamExecutionEnvironment(env);
Thread thread =
@@ -1204,11 +1248,13 @@ public void testComputedColumnWithCaseInsensitive() throws Exception {
mySqlConfig.put("database-name", "computed_column_with_case_insensitive");
mySqlConfig.put("table-name", "t");
+ Map catalogConfig = new HashMap<>();
+ catalogConfig.put(FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false");
+
MySqlSyncTableAction action =
syncTableActionBuilder(mySqlConfig)
- .withCatalogConfig(
- Collections.singletonMap(
- FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false"))
+ .withCatalogConfig(getCatalogOptions(catalogConfig))
+ .withTableConfig(getTableConfig(new HashMap<>()))
.withComputedColumnArgs("SUBSTRING=substring(UPPERCASE_STRING,2)")
.build();
runActionWithDefaultEnv(action);
@@ -1240,11 +1286,13 @@ public void testSpecifyKeysWithCaseInsensitive() throws Exception {
mySqlConfig.put("database-name", "specify_key_with_case_insensitive");
mySqlConfig.put("table-name", "t");
+ Map catalogConfig = new HashMap<>();
+ catalogConfig.put(FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false");
+
MySqlSyncTableAction action =
syncTableActionBuilder(mySqlConfig)
- .withCatalogConfig(
- Collections.singletonMap(
- FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false"))
+ .withCatalogConfig(getCatalogOptions(catalogConfig))
+ .withTableConfig(getTableConfig(new HashMap<>()))
.withPrimaryKeys("ID1", "PART")
.withPartitionKeys("PART")
.build();
@@ -1268,9 +1316,13 @@ public void testInvalidAlterBucket() throws Exception {
mySqlConfig.put("database-name", "invalid_alter_bucket");
mySqlConfig.put("table-name", "t");
+ Map tableConfig = new HashMap<>();
+ tableConfig.put(BUCKET.key(), "2");
+
MySqlSyncTableAction action =
syncTableActionBuilder(mySqlConfig)
- .withTableConfig(Collections.singletonMap(BUCKET.key(), "2"))
+ .withTableConfig(getTableConfig(tableConfig))
+ .withCatalogConfig(getCatalogOptions(new HashMap<>()))
.build();
assertThatCode(action::build).doesNotThrowAnyException();
diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionToBranchITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionToBranchITCase.java
new file mode 100644
index 0000000000000..db541ad31a16b
--- /dev/null
+++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionToBranchITCase.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.paimon.flink.action.cdc.mysql;
+
+import org.junit.jupiter.api.BeforeEach;
+
+import java.io.IOException;
+
+/** IT cases for {@link MySqlSyncTableAction}. */
+public class MySqlSyncTableActionToBranchITCase extends MySqlSyncTableActionITCase {
+ @BeforeEach
+ public void before() throws IOException {
+ super.branch = "testMySqlSyncTableActionBranch";
+ super.before();
+ }
+}
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 4c9f8ff9b55de..1617b4cc33278 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
@@ -58,9 +58,12 @@ private void createTableIfNeeded(Context context) {
if (options.get(AUTO_CREATE)) {
try {
Path tablePath = CoreOptions.path(table.getOptions());
+ String branch = CoreOptions.branch(table.getOptions());
SchemaManager schemaManager =
new SchemaManager(
- FileIO.get(tablePath, createCatalogContext(context)), tablePath);
+ FileIO.get(tablePath, createCatalogContext(context)),
+ tablePath,
+ branch);
if (!schemaManager.latest().isPresent()) {
schemaManager.createTable(FlinkCatalog.fromCatalogTable(table));
}
diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperator.java
index da3425e9b0d35..6f81de25959dd 100644
--- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperator.java
+++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperator.java
@@ -21,6 +21,7 @@
import org.apache.paimon.Snapshot;
import org.apache.paimon.operation.TagDeletion;
import org.apache.paimon.table.sink.TagCallback;
+import org.apache.paimon.utils.BranchManager;
import org.apache.paimon.utils.SerializableSupplier;
import org.apache.paimon.utils.SnapshotManager;
import org.apache.paimon.utils.TagManager;
@@ -86,6 +87,8 @@ public class AutoTagForSavepointCommitterOperator
private transient ListState identifiersForTagsState;
+ private transient String branchName;
+
public AutoTagForSavepointCommitterOperator(
CommitterOperator commitOperator,
SerializableSupplier snapshotManagerFactory,
@@ -98,6 +101,23 @@ public AutoTagForSavepointCommitterOperator(
this.tagDeletionFactory = tagDeletionFactory;
this.callbacksSupplier = callbacksSupplier;
this.identifiersForTags = new TreeSet<>();
+ this.branchName = BranchManager.DEFAULT_MAIN_BRANCH;
+ }
+
+ public AutoTagForSavepointCommitterOperator(
+ CommitterOperator commitOperator,
+ SerializableSupplier snapshotManagerFactory,
+ SerializableSupplier tagManagerFactory,
+ SerializableSupplier tagDeletionFactory,
+ SerializableSupplier> callbacksSupplier,
+ String branchName) {
+ this.commitOperator = commitOperator;
+ this.tagManagerFactory = tagManagerFactory;
+ this.snapshotManagerFactory = snapshotManagerFactory;
+ this.tagDeletionFactory = tagDeletionFactory;
+ this.callbacksSupplier = callbacksSupplier;
+ this.identifiersForTags = new TreeSet<>();
+ this.branchName = branchName;
}
@Override
@@ -155,8 +175,8 @@ public void notifyCheckpointAborted(long checkpointId) throws Exception {
commitOperator.notifyCheckpointAborted(checkpointId);
identifiersForTags.remove(checkpointId);
String tagName = SAVEPOINT_TAG_PREFIX + checkpointId;
- if (tagManager.tagExists(tagName)) {
- tagManager.deleteTag(tagName, tagDeletion, snapshotManager);
+ if (tagManager.tagExists(branchName, tagName)) {
+ tagManager.deleteTag(tagName, tagDeletion, snapshotManager, branchName);
}
}
@@ -166,8 +186,8 @@ private void createTagForIdentifiers(List identifiers) {
commitOperator.getCommitUser(), identifiers);
for (Snapshot snapshot : snapshotForTags) {
String tagName = SAVEPOINT_TAG_PREFIX + snapshot.commitIdentifier();
- if (!tagManager.tagExists(tagName)) {
- tagManager.createTag(snapshot, tagName, callbacks);
+ if (!tagManager.tagExists(branchName, tagName)) {
+ tagManager.createTag(snapshot, tagName, callbacks, branchName);
}
}
}
diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperator.java
index d65ab74140fe5..45087773b1459 100644
--- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperator.java
+++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperator.java
@@ -21,6 +21,7 @@
import org.apache.paimon.Snapshot;
import org.apache.paimon.operation.TagDeletion;
import org.apache.paimon.table.FileStoreTable;
+import org.apache.paimon.utils.BranchManager;
import org.apache.paimon.utils.SnapshotManager;
import org.apache.paimon.utils.TagManager;
@@ -65,10 +66,22 @@ public class BatchWriteGeneratorTagOperator
protected final FileStoreTable table;
+ protected final String branchName;
+
public BatchWriteGeneratorTagOperator(
CommitterOperator commitOperator, FileStoreTable table) {
this.table = table;
this.commitOperator = commitOperator;
+ this.branchName = BranchManager.DEFAULT_MAIN_BRANCH;
+ }
+
+ public BatchWriteGeneratorTagOperator(
+ CommitterOperator commitOperator,
+ FileStoreTable table,
+ String branchName) {
+ this.table = table;
+ this.commitOperator = commitOperator;
+ this.branchName = branchName;
}
@Override
@@ -113,11 +126,11 @@ private void createTag() {
+ localDateTime.format(DateTimeFormatter.ofPattern("yyyy-MM-dd"));
try {
// If the tag already exists, delete the tag
- if (tagManager.tagExists(tagName)) {
- tagManager.deleteTag(tagName, tagDeletion, snapshotManager);
+ if (tagManager.tagExists(branchName, tagName)) {
+ tagManager.deleteTag(tagName, tagDeletion, snapshotManager, branchName);
}
// Create a new tag
- tagManager.createTag(snapshot, tagName, table.store().createTagCallbacks());
+ tagManager.createTag(snapshot, tagName, table.store().createTagCallbacks(), branchName);
// Expire the tag
expireTag();
} catch (Exception e) {
@@ -136,18 +149,20 @@ private void expireTag() {
}
TagManager tagManager = table.tagManager();
TagDeletion tagDeletion = table.store().newTagDeletion();
- long tagCount = tagManager.tagCount();
+ long tagCount = tagManager.tagCount(branchName);
while (tagCount > tagNumRetainedMax) {
- for (List tagNames : tagManager.tags().values()) {
+ for (List tagNames : tagManager.tags(branchName).values()) {
if (tagCount - tagNames.size() >= tagNumRetainedMax) {
tagManager.deleteAllTagsOfOneSnapshot(
- tagNames, tagDeletion, snapshotManager);
+ tagNames, tagDeletion, snapshotManager, branchName);
tagCount = tagCount - tagNames.size();
} else {
- List sortedTagNames = tagManager.sortTagsOfOneSnapshot(tagNames);
+ List sortedTagNames =
+ tagManager.sortTagsOfOneSnapshot(branchName, tagNames);
for (String toBeDeleted : sortedTagNames) {
- tagManager.deleteTag(toBeDeleted, tagDeletion, snapshotManager);
+ tagManager.deleteTag(
+ toBeDeleted, tagDeletion, snapshotManager, branchName);
tagCount--;
if (tagCount == tagNumRetainedMax) {
break;
diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactorSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactorSink.java
index 8e08824567ad3..22104159a4e45 100644
--- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactorSink.java
+++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactorSink.java
@@ -42,7 +42,9 @@ protected OneInputStreamOperator createWriteOperator(
@Override
protected Committer.Factory createCommitterFactory(
boolean streamingCheckpointEnabled) {
- return (user, metricGroup) -> new StoreCommitter(table.newCommit(user), metricGroup);
+ return (user, metricGroup) ->
+ new StoreCommitter(
+ table.newCommit(user, table.coreOptions().branch()), metricGroup);
}
@Override
diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java
index 545bd7f070729..49da609222d52 100644
--- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java
+++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java
@@ -18,6 +18,7 @@
package org.apache.paimon.flink.sink;
+import org.apache.paimon.CoreOptions;
import org.apache.paimon.CoreOptions.ChangelogProducer;
import org.apache.paimon.CoreOptions.TagCreationMode;
import org.apache.paimon.flink.utils.StreamExecutionEnvironmentUtils;
@@ -238,6 +239,8 @@ protected DataStreamSink> doCommit(DataStream written, String com
commitUser,
createCommitterFactory(streamingCheckpointEnabled),
createCommittableStateManager());
+
+ String branch = CoreOptions.branch(table.options());
if (Options.fromMap(table.options()).get(SINK_AUTO_TAG_FOR_SAVEPOINT)) {
committerOperator =
new AutoTagForSavepointCommitterOperator<>(
@@ -245,14 +248,16 @@ protected DataStreamSink> doCommit(DataStream written, String com
table::snapshotManager,
table::tagManager,
() -> table.store().newTagDeletion(),
- () -> table.store().createTagCallbacks());
+ () -> table.store().createTagCallbacks(),
+ branch);
}
if (conf.get(ExecutionOptions.RUNTIME_MODE) == RuntimeExecutionMode.BATCH
&& table.coreOptions().tagCreationMode() == TagCreationMode.BATCH) {
committerOperator =
new BatchWriteGeneratorTagOperator<>(
(CommitterOperator) committerOperator,
- table);
+ table,
+ branch);
}
SingleOutputStreamOperator> committed =
written.transform(
diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkTableSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkTableSink.java
index 50bc45b752f8d..1628daac588ef 100644
--- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkTableSink.java
+++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkTableSink.java
@@ -18,6 +18,7 @@
package org.apache.paimon.flink.sink;
+import org.apache.paimon.CoreOptions;
import org.apache.paimon.flink.log.LogStoreTableFactory;
import org.apache.paimon.operation.FileStoreCommit;
import org.apache.paimon.table.FileStoreTable;
@@ -47,7 +48,10 @@ public FlinkTableSink(
@Override
public void executeTruncation() {
FileStoreCommit commit =
- ((FileStoreTable) table).store().newCommit(UUID.randomUUID().toString());
+ ((FileStoreTable) table)
+ .store()
+ .newCommit(
+ UUID.randomUUID().toString(), CoreOptions.branch(table.options()));
long identifier = BatchWriteBuilder.COMMIT_IDENTIFIER;
commit.purgeTable(identifier);
}
diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkWriteSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkWriteSink.java
index b812c04912b12..039d798c98539 100644
--- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkWriteSink.java
+++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkWriteSink.java
@@ -18,6 +18,7 @@
package org.apache.paimon.flink.sink;
+import org.apache.paimon.CoreOptions;
import org.apache.paimon.flink.VersionedSerializerWrapper;
import org.apache.paimon.manifest.ManifestCommittable;
import org.apache.paimon.manifest.ManifestCommittableSerializer;
@@ -48,7 +49,7 @@ protected Committer.Factory createCommitterFac
// a restart.
return (user, metricGroup) ->
new StoreCommitter(
- table.newCommit(user)
+ table.newCommit(user, CoreOptions.branch(table.options()))
.withOverwrite(overwritePartition)
.ignoreEmptyCommit(!streamingCheckpointEnabled),
metricGroup);
diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/UnawareBucketCompactionSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/UnawareBucketCompactionSink.java
index cf825cec7ba41..8be3b937c0b8e 100644
--- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/UnawareBucketCompactionSink.java
+++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/UnawareBucketCompactionSink.java
@@ -18,6 +18,7 @@
package org.apache.paimon.flink.sink;
+import org.apache.paimon.CoreOptions;
import org.apache.paimon.append.AppendOnlyCompactionTask;
import org.apache.paimon.manifest.ManifestCommittable;
import org.apache.paimon.table.FileStoreTable;
@@ -50,7 +51,9 @@ protected OneInputStreamOperator createWr
@Override
protected Committer.Factory createCommitterFactory(
boolean streamingCheckpointEnabled) {
- return (s, metricGroup) -> new StoreCommitter(table.newCommit(s), metricGroup);
+ return (s, metricGroup) ->
+ new StoreCommitter(
+ table.newCommit(s, CoreOptions.branch(table.options())), metricGroup);
}
@Override
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 b7c739e96e68d..7a70723a06e52 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
@@ -33,6 +33,7 @@
import org.apache.paimon.table.FileStoreTable;
import org.apache.paimon.table.FileStoreTableFactory;
import org.apache.paimon.utils.BlockingIterator;
+import org.apache.paimon.utils.BranchManager;
import org.apache.paimon.utils.FailingFileIO;
import org.apache.flink.api.common.RuntimeExecutionMode;
@@ -60,6 +61,7 @@
import org.apache.flink.types.Row;
import org.apache.flink.types.RowKind;
import org.apache.flink.util.CloseableIterator;
+import org.junit.jupiter.api.BeforeAll;
import org.junit.jupiter.api.TestTemplate;
import org.junit.jupiter.api.extension.ExtendWith;
@@ -74,6 +76,7 @@
import java.util.stream.Collectors;
import java.util.stream.Stream;
+import static org.apache.paimon.CoreOptions.BRANCH;
import static org.apache.paimon.CoreOptions.BUCKET;
import static org.apache.paimon.CoreOptions.FILE_FORMAT;
import static org.apache.paimon.CoreOptions.PATH;
@@ -122,11 +125,18 @@ public class FileStoreITCase extends AbstractTestBase {
private final StreamExecutionEnvironment env;
+ protected static String branch;
+
public FileStoreITCase(boolean isBatch) {
this.isBatch = isBatch;
this.env = isBatch ? buildBatchEnv() : buildStreamEnv();
}
+ @BeforeAll
+ public static void before() {
+ branch = BranchManager.DEFAULT_MAIN_BRANCH;
+ }
+
@Parameters(name = "isBatch-{0}")
public static List getVarSeg() {
return Arrays.asList(true, false);
@@ -429,7 +439,7 @@ public static FileStoreTable buildFileStoreTable(
"");
return retryArtificialException(
() -> {
- new SchemaManager(LocalFileIO.create(), tablePath).createTable(schema);
+ new SchemaManager(LocalFileIO.create(), tablePath, branch).createTable(schema);
return FileStoreTableFactory.create(LocalFileIO.create(), options);
});
}
@@ -445,6 +455,7 @@ public static Options buildConfiguration(boolean noFail, String temporaryPath) {
options.set(PATH, FailingFileIO.getFailingPath(failingName, temporaryPath));
}
options.set(FILE_FORMAT, CoreOptions.FileFormatType.AVRO);
+ options.set(BRANCH, branch);
return options;
}
diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreWithBranchITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreWithBranchITCase.java
new file mode 100644
index 0000000000000..8a2374b3c367b
--- /dev/null
+++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreWithBranchITCase.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.paimon.flink;
+
+import org.apache.paimon.flink.sink.FixedBucketSink;
+import org.apache.paimon.flink.source.ContinuousFileStoreSource;
+import org.apache.paimon.flink.source.StaticFileStoreSource;
+
+import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.extension.ExtendWith;
+
+/**
+ * ITCase for {@link StaticFileStoreSource}, {@link ContinuousFileStoreSource} and {@link
+ * FixedBucketSink}.
+ */
+@ExtendWith(ParameterizedTestExtension.class)
+public class FileStoreWithBranchITCase extends FileStoreITCase {
+ public FileStoreWithBranchITCase(boolean isBatch) {
+ super(isBatch);
+ }
+
+ @BeforeAll
+ public static void before() {
+ branch = "testBranch";
+ }
+}
diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/ActionITCaseBase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/ActionITCaseBase.java
index 80cf2cc767119..524b837ca3d22 100644
--- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/ActionITCaseBase.java
+++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/ActionITCaseBase.java
@@ -27,6 +27,7 @@
import org.apache.paimon.data.InternalRow;
import org.apache.paimon.flink.util.AbstractTestBase;
import org.apache.paimon.fs.Path;
+import org.apache.paimon.options.Options;
import org.apache.paimon.reader.RecordReader;
import org.apache.paimon.schema.Schema;
import org.apache.paimon.table.FileStoreTable;
@@ -35,6 +36,7 @@
import org.apache.paimon.table.source.Split;
import org.apache.paimon.table.source.TableRead;
import org.apache.paimon.types.RowType;
+import org.apache.paimon.utils.BranchManager;
import org.apache.flink.api.common.RuntimeExecutionMode;
import org.apache.flink.api.common.restartstrategy.RestartStrategies;
@@ -57,6 +59,9 @@
import java.util.UUID;
import java.util.concurrent.ThreadLocalRandom;
+import static org.apache.paimon.CoreOptions.BRANCH;
+import static org.apache.paimon.options.CatalogOptions.WAREHOUSE;
+
/** {@link Action} test base. */
public abstract class ActionITCaseBase extends AbstractTestBase {
@@ -68,6 +73,7 @@ public abstract class ActionITCaseBase extends AbstractTestBase {
protected StreamTableCommit commit;
protected Catalog catalog;
private long incrementalIdentifier;
+ protected String branch = BranchManager.DEFAULT_MAIN_BRANCH;
@BeforeEach
public void before() throws IOException {
@@ -76,7 +82,12 @@ public void before() throws IOException {
tableName = "test_table_" + UUID.randomUUID();
commitUser = UUID.randomUUID().toString();
incrementalIdentifier = 0;
- catalog = CatalogFactory.createCatalog(CatalogContext.create(new Path(warehouse)));
+ Map options = new HashMap<>();
+ options.put(WAREHOUSE.key(), new Path(warehouse).toUri().toString());
+ if (!branch.equals(BranchManager.DEFAULT_MAIN_BRANCH)) {
+ options.put(BRANCH.key(), branch);
+ }
+ catalog = CatalogFactory.createCatalog(CatalogContext.create(Options.fromMap(options)));
}
@AfterEach
diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceReaderFromBranchTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceReaderFromBranchTest.java
new file mode 100644
index 0000000000000..87632aa4b073c
--- /dev/null
+++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceReaderFromBranchTest.java
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.paimon.flink.source;
+
+import org.apache.paimon.fs.Path;
+import org.apache.paimon.fs.local.LocalFileIO;
+import org.apache.paimon.schema.Schema;
+import org.apache.paimon.schema.SchemaManager;
+import org.apache.paimon.types.BigIntType;
+import org.apache.paimon.types.DataField;
+import org.apache.paimon.types.IntType;
+import org.apache.paimon.types.RowType;
+
+import org.junit.jupiter.api.BeforeEach;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.UUID;
+
+/** Unit tests for the {@link FileStoreSourceReader}. */
+public class FileStoreSourceReaderFromBranchTest extends FileStoreSourceReaderTest {
+
+ @BeforeEach
+ public void beforeEach() throws Exception {
+ branch = "testBranch-" + UUID.randomUUID();
+ SchemaManager schemaManager =
+ new SchemaManager(LocalFileIO.create(), new Path(tempDir.toUri()), branch);
+ schemaManager.createTable(
+ new Schema(
+ new RowType(
+ Arrays.asList(
+ new DataField(0, "k", new BigIntType()),
+ new DataField(1, "v", new BigIntType()),
+ new DataField(2, "default", new IntType())))
+ .getFields(),
+ Collections.singletonList("default"),
+ Arrays.asList("k", "default"),
+ Collections.emptyMap(),
+ null));
+ }
+}
diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceReaderTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceReaderTest.java
index 882763cf74da2..33662b95bda34 100644
--- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceReaderTest.java
+++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceReaderTest.java
@@ -28,6 +28,7 @@
import org.apache.paimon.types.DataField;
import org.apache.paimon.types.IntType;
import org.apache.paimon.types.RowType;
+import org.apache.paimon.utils.BranchManager;
import org.apache.flink.api.connector.source.SourceEvent;
import org.apache.flink.connector.testutils.source.reader.TestingReaderContext;
@@ -57,6 +58,8 @@ public class FileStoreSourceReaderTest {
@TempDir protected java.nio.file.Path tempDir;
+ protected String branch = BranchManager.DEFAULT_MAIN_BRANCH;
+
@BeforeEach
public void beforeEach() throws Exception {
SchemaManager schemaManager =
@@ -138,7 +141,7 @@ public void testReaderOnSplitFinished() throws Exception {
protected FileStoreSourceReader createReader(TestingReaderContext context) {
return new FileStoreSourceReader(
context,
- new TestChangelogDataReadWrite(tempDir.toString()).createReadWithKey(),
+ new TestChangelogDataReadWrite(tempDir.toString()).createReadWithKey(branch),
new FileStoreSourceReaderMetrics(new DummyMetricGroup()),
IOManager.create(tempDir.toString()),
null);
diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceSplitReaderFromBranchTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceSplitReaderFromBranchTest.java
new file mode 100644
index 0000000000000..92ac692df101f
--- /dev/null
+++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceSplitReaderFromBranchTest.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.paimon.flink.source;
+
+import org.junit.jupiter.api.BeforeEach;
+
+import java.util.UUID;
+
+/** Test for {@link FileStoreSourceSplitReader}. */
+public class FileStoreSourceSplitReaderFromBranchTest extends FileStoreSourceSplitReaderTest {
+
+ @BeforeEach
+ public void beforeEach() throws Exception {
+ branch = "testBranch-" + UUID.randomUUID();
+ super.beforeEach();
+ }
+}
diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceSplitReaderTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceSplitReaderTest.java
index 67b3c3e603f84..853cab2cccea5 100644
--- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceSplitReaderTest.java
+++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceSplitReaderTest.java
@@ -28,13 +28,13 @@
import org.apache.paimon.schema.Schema;
import org.apache.paimon.schema.SchemaManager;
import org.apache.paimon.table.source.TableRead;
+import org.apache.paimon.utils.BranchManager;
import org.apache.paimon.utils.RecordWriter;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition;
import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange;
-import org.apache.flink.connector.file.src.reader.BulkFormat;
import org.apache.flink.connector.file.src.reader.BulkFormat.RecordIterator;
import org.apache.flink.connector.file.src.util.RecordAndPosition;
import org.apache.flink.table.data.RowData;
@@ -67,10 +67,12 @@ public class FileStoreSourceSplitReaderTest {
@TempDir java.nio.file.Path tempDir;
+ protected String branch = BranchManager.DEFAULT_MAIN_BRANCH;
+
@BeforeEach
public void beforeEach() throws Exception {
SchemaManager schemaManager =
- new SchemaManager(LocalFileIO.create(), new Path(tempDir.toUri()));
+ new SchemaManager(LocalFileIO.create(), new Path(tempDir.toUri()), branch);
schemaManager.createTable(
new Schema(
toDataType(
@@ -106,10 +108,10 @@ private FileStoreSourceSplitReader createReader(TableRead tableRead, @Nullable L
private void innerTestOnce(int skip) throws Exception {
TestChangelogDataReadWrite rw = new TestChangelogDataReadWrite(tempDir.toString());
- FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(), null);
+ FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(branch), null);
List> input = kvs();
- List files = rw.writeFiles(row(1), 0, input);
+ List files = rw.writeFiles(row(1), 0, input, branch);
assignSplit(reader, newSourceSplit("id1", row(1), 0, files, skip));
@@ -132,10 +134,10 @@ private void innerTestOnce(int skip) throws Exception {
@Test
public void testPrimaryKeyWithDelete() throws Exception {
TestChangelogDataReadWrite rw = new TestChangelogDataReadWrite(tempDir.toString());
- FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(), null);
+ FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(branch), null);
List> input = kvs();
- RecordWriter writer = rw.createMergeTreeWriter(row(1), 0);
+ RecordWriter writer = rw.createMergeTreeWriter(row(1), 0, branch);
for (Tuple2 tuple2 : input) {
writer.write(
new KeyValue()
@@ -154,7 +156,7 @@ public void testPrimaryKeyWithDelete() throws Exception {
writer.close();
assignSplit(reader, newSourceSplit("id1", row(1), 0, files, true));
- RecordsWithSplitIds> records = reader.fetch();
+ RecordsWithSplitIds> records = reader.fetch();
List> expected =
input.stream()
@@ -174,18 +176,18 @@ public void testPrimaryKeyWithDelete() throws Exception {
@Test
public void testMultipleBatchInSplit() throws Exception {
TestChangelogDataReadWrite rw = new TestChangelogDataReadWrite(tempDir.toString());
- FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(), null);
+ FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(branch), null);
List> input1 = kvs();
- List files = rw.writeFiles(row(1), 0, input1);
+ List files = rw.writeFiles(row(1), 0, input1, branch);
List> input2 = kvs(6);
- List files2 = rw.writeFiles(row(1), 0, input2);
+ List files2 = rw.writeFiles(row(1), 0, input2, branch);
files.addAll(files2);
assignSplit(reader, newSourceSplit("id1", row(1), 0, files));
- RecordsWithSplitIds> records = reader.fetch();
+ RecordsWithSplitIds> records = reader.fetch();
assertRecords(
records,
null,
@@ -210,14 +212,14 @@ public void testMultipleBatchInSplit() throws Exception {
@Test
public void testRestore() throws Exception {
TestChangelogDataReadWrite rw = new TestChangelogDataReadWrite(tempDir.toString());
- FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(), null);
+ FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(branch), null);
List> input = kvs();
- List files = rw.writeFiles(row(1), 0, input);
+ List files = rw.writeFiles(row(1), 0, input, branch);
assignSplit(reader, newSourceSplit("id1", row(1), 0, files, 3));
- RecordsWithSplitIds> records = reader.fetch();
+ RecordsWithSplitIds> records = reader.fetch();
assertRecords(
records,
null,
@@ -236,18 +238,18 @@ public void testRestore() throws Exception {
@Test
public void testRestoreMultipleBatchInSplit() throws Exception {
TestChangelogDataReadWrite rw = new TestChangelogDataReadWrite(tempDir.toString());
- FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(), null);
+ FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(branch), null);
List> input1 = kvs();
- List files = rw.writeFiles(row(1), 0, input1);
+ List files = rw.writeFiles(row(1), 0, input1, branch);
List> input2 = kvs(6);
- List files2 = rw.writeFiles(row(1), 0, input2);
+ List files2 = rw.writeFiles(row(1), 0, input2, branch);
files.addAll(files2);
assignSplit(reader, newSourceSplit("id1", row(1), 0, files, 7));
- RecordsWithSplitIds> records = reader.fetch();
+ RecordsWithSplitIds> records = reader.fetch();
assertRecords(
records,
null,
@@ -267,17 +269,17 @@ public void testRestoreMultipleBatchInSplit() throws Exception {
@Test
public void testMultipleSplits() throws Exception {
TestChangelogDataReadWrite rw = new TestChangelogDataReadWrite(tempDir.toString());
- FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(), null);
+ FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(branch), null);
List> input1 = kvs();
- List files1 = rw.writeFiles(row(1), 0, input1);
+ List files1 = rw.writeFiles(row(1), 0, input1, branch);
assignSplit(reader, newSourceSplit("id1", row(1), 0, files1));
List> input2 = kvs();
- List files2 = rw.writeFiles(row(2), 1, input2);
+ List files2 = rw.writeFiles(row(2), 1, input2, branch);
assignSplit(reader, newSourceSplit("id2", row(2), 1, files2));
- RecordsWithSplitIds> records = reader.fetch();
+ RecordsWithSplitIds> records = reader.fetch();
assertRecords(
records,
null,
@@ -305,7 +307,7 @@ public void testMultipleSplits() throws Exception {
@Test
public void testNoSplit() throws Exception {
TestChangelogDataReadWrite rw = new TestChangelogDataReadWrite(tempDir.toString());
- FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(), null);
+ FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(branch), null);
assertThatThrownBy(reader::fetch).hasMessageContaining("no split remaining");
reader.close();
}
@@ -313,14 +315,14 @@ public void testNoSplit() throws Exception {
@Test
public void testLimit() throws Exception {
TestChangelogDataReadWrite rw = new TestChangelogDataReadWrite(tempDir.toString());
- FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(), 2L);
+ FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(branch), 2L);
List> input = kvs();
- List files = rw.writeFiles(row(1), 0, input);
+ List