diff --git a/docs/layouts/shortcodes/generated/catalog_configuration.html b/docs/layouts/shortcodes/generated/catalog_configuration.html
index cab6e731e8511..d0f9966a22fb4 100644
--- a/docs/layouts/shortcodes/generated/catalog_configuration.html
+++ b/docs/layouts/shortcodes/generated/catalog_configuration.html
@@ -92,5 +92,11 @@
String |
The warehouse root path of catalog. |
+
+ branch |
+ main |
+ String |
+ Specify the branch submitted by the schema. |
+
diff --git a/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java b/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java
index f00a35a750940..455f27a446530 100644
--- a/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java
+++ b/paimon-common/src/main/java/org/apache/paimon/options/CatalogOptions.java
@@ -110,4 +110,10 @@ public class CatalogOptions {
TextElement.text(
"\"custom\": You can implement LineageMetaFactory and LineageMeta to store lineage information in customized storage."))
.build());
+
+ public static final ConfigOption BRANCH =
+ key("branch")
+ .stringType()
+ .defaultValue("main")
+ .withDescription("Specify the branch submitted by the schema.");
}
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 6413cd088edeb..30a894c0cca43 100644
--- a/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java
+++ b/paimon-core/src/main/java/org/apache/paimon/AbstractFileStore.java
@@ -188,7 +188,7 @@ public FileStoreCommitImpl newCommit(String commitUser, String branchName) {
manifestFileFactory(),
manifestListFactory(),
indexManifestFileFactory(),
- newScan(),
+ newScan(branchName),
options.bucket(),
options.manifestTargetSize(),
options.manifestFullCompactionThresholdSize(),
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 8be8f817841db..d509f32bfd1f4 100644
--- a/paimon-core/src/main/java/org/apache/paimon/AppendOnlyFileStore.java
+++ b/paimon-core/src/main/java/org/apache/paimon/AppendOnlyFileStore.java
@@ -25,6 +25,7 @@
import org.apache.paimon.operation.AppendOnlyFileStoreRead;
import org.apache.paimon.operation.AppendOnlyFileStoreScan;
import org.apache.paimon.operation.AppendOnlyFileStoreWrite;
+import org.apache.paimon.operation.FileStoreWrite;
import org.apache.paimon.operation.ScanBucketFilter;
import org.apache.paimon.predicate.Predicate;
import org.apache.paimon.schema.SchemaManager;
@@ -88,14 +89,37 @@ public AppendOnlyFileStoreRead newRead() {
pathFactory());
}
+ @Override
+ public AppendOnlyFileStoreRead newRead(String branchName) {
+ return new AppendOnlyFileStoreRead(
+ fileIO,
+ schemaManager,
+ schemaId,
+ rowType,
+ FileFormatDiscover.of(options),
+ pathFactory(),
+ branchName);
+ }
+
@Override
public AppendOnlyFileStoreWrite newWrite(String commitUser) {
- return newWrite(commitUser, null);
+ return newWrite(commitUser, null, DEFAULT_MAIN_BRANCH);
+ }
+
+ @Override
+ public FileStoreWrite newWrite(String commitUser, String branch) {
+ return newWrite(commitUser, null, branch);
}
@Override
public AppendOnlyFileStoreWrite newWrite(
String commitUser, ManifestCacheFilter manifestFilter) {
+ return newWrite(commitUser, manifestFilter, DEFAULT_MAIN_BRANCH);
+ }
+
+ @Override
+ public AppendOnlyFileStoreWrite newWrite(
+ String commitUser, ManifestCacheFilter manifestFilter, String branchName) {
return new AppendOnlyFileStoreWrite(
fileIO,
newRead(),
@@ -104,9 +128,10 @@ public AppendOnlyFileStoreWrite newWrite(
rowType,
pathFactory(),
snapshotManager(),
- newScan(true, DEFAULT_MAIN_BRANCH).withManifestCacheFilter(manifestFilter),
+ newScan(true, branchName).withManifestCacheFilter(manifestFilter),
options,
- tableName);
+ tableName,
+ branchName);
}
private AppendOnlyFileStoreScan newScan(boolean forWrite, String branchName) {
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..0eda3da9c46b8 100644
--- a/paimon-core/src/main/java/org/apache/paimon/FileStore.java
+++ b/paimon-core/src/main/java/org/apache/paimon/FileStore.java
@@ -75,10 +75,17 @@ public interface FileStore extends Serializable {
FileStoreRead newRead();
+ FileStoreRead newRead(String branchName);
+
FileStoreWrite newWrite(String commitUser);
+ FileStoreWrite newWrite(String commitUser, String branch);
+
FileStoreWrite newWrite(String commitUser, ManifestCacheFilter manifestFilter);
+ FileStoreWrite newWrite(
+ String commitUser, ManifestCacheFilter manifestFilter, String branchName);
+
FileStoreCommit newCommit(String commitUser);
FileStoreCommit newCommit(String commitUser, String branchName);
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 e9b4cc5fd00ef..63cc4ae7b997f 100644
--- a/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java
+++ b/paimon-core/src/main/java/org/apache/paimon/KeyValueFileStore.java
@@ -130,6 +130,20 @@ public KeyValueFileStoreRead newRead() {
newReaderFactoryBuilder());
}
+ @Override
+ public KeyValueFileStoreRead newRead(String branchName) {
+ return new KeyValueFileStoreRead(
+ options,
+ schemaManager,
+ schemaId,
+ keyType,
+ valueType,
+ newKeyComparator(),
+ mfFactory,
+ newReaderFactoryBuilder(branchName),
+ branchName);
+ }
+
public KeyValueFileReaderFactory.Builder newReaderFactoryBuilder() {
return KeyValueFileReaderFactory.builder(
fileIO,
@@ -143,13 +157,38 @@ public KeyValueFileReaderFactory.Builder newReaderFactoryBuilder() {
options);
}
+ public KeyValueFileReaderFactory.Builder newReaderFactoryBuilder(String branch) {
+ return KeyValueFileReaderFactory.builder(
+ fileIO,
+ schemaManager,
+ schemaId,
+ keyType,
+ valueType,
+ FileFormatDiscover.of(options),
+ pathFactory(),
+ keyValueFieldsExtractor,
+ options,
+ branch);
+ }
+
@Override
public KeyValueFileStoreWrite newWrite(String commitUser) {
- return newWrite(commitUser, null);
+ return newWrite(commitUser, null, DEFAULT_MAIN_BRANCH);
+ }
+
+ @Override
+ public KeyValueFileStoreWrite newWrite(String commitUser, String branch) {
+ return newWrite(commitUser, null, branch);
}
@Override
public KeyValueFileStoreWrite newWrite(String commitUser, ManifestCacheFilter manifestFilter) {
+ return newWrite(commitUser, manifestFilter, DEFAULT_MAIN_BRANCH);
+ }
+
+ @Override
+ public KeyValueFileStoreWrite newWrite(
+ String commitUser, ManifestCacheFilter manifestFilter, String branchName) {
IndexMaintainer.Factory indexFactory = null;
if (bucketMode() == BucketMode.DYNAMIC) {
indexFactory = new HashIndexMaintainer.Factory(newIndexFileHandler());
@@ -173,12 +212,13 @@ public KeyValueFileStoreWrite newWrite(String commitUser, ManifestCacheFilter ma
pathFactory(),
format2PathFactory(),
snapshotManager(),
- newScan(true, DEFAULT_MAIN_BRANCH).withManifestCacheFilter(manifestFilter),
+ newScan(true, branchName).withManifestCacheFilter(manifestFilter),
indexFactory,
deletionVectorsMaintainerFactory,
options,
keyValueFieldsExtractor,
- tableName);
+ tableName,
+ branchName);
}
private Map format2PathFactory() {
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..0f7750d69b154 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;
@@ -50,6 +51,7 @@
import java.util.UUID;
import java.util.stream.Collectors;
+import static org.apache.paimon.options.CatalogOptions.BRANCH;
import static org.apache.paimon.options.CatalogOptions.LINEAGE_META;
import static org.apache.paimon.options.CatalogOptions.LOCK_ENABLED;
import static org.apache.paimon.options.CatalogOptions.LOCK_TYPE;
@@ -66,6 +68,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 +77,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 +87,7 @@ protected AbstractCatalog(FileIO fileIO, Options options) {
this.tableDefaultOptions =
convertToPropertiesPrefixKey(options.toMap(), TABLE_DEFAULT_OPTION_PREFIX);
this.catalogOptions = options;
+ this.branchName = options.get(BRANCH);
if (lockEnabled()) {
checkArgument(options.contains(LOCK_TYPE), "No lock type when lock is enabled.");
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..2a330bc3e6bf3 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,13 +118,13 @@ public boolean tableExists(Identifier identifier) {
}
private boolean tableExists(Path tablePath) {
- return new SchemaManager(fileIO, tablePath).listAllIds().size() > 0;
+ return new SchemaManager(fileIO, tablePath).listAllIds(branchName).size() > 0;
}
@Override
public TableSchema getDataTableSchema(Identifier identifier) throws TableNotExistException {
return schemaManager(identifier)
- .latest()
+ .latest(branchName)
.orElseThrow(() -> new TableNotExistException(identifier));
}
@@ -136,7 +136,7 @@ protected void dropTableImpl(Identifier identifier) {
@Override
public void createTableImpl(Identifier identifier, Schema schema) {
- uncheck(() -> schemaManager(identifier).createTable(schema));
+ uncheck(() -> schemaManager(identifier).createTable(schema, branchName));
}
private SchemaManager schemaManager(Identifier identifier) {
@@ -166,7 +166,7 @@ public void renameTableImpl(Identifier fromTable, Identifier toTable) {
@Override
protected void alterTableImpl(Identifier identifier, List changes)
throws TableNotExistException, ColumnAlreadyExistException, ColumnNotExistException {
- schemaManager(identifier).commitChanges(changes);
+ schemaManager(identifier).commitChanges(branchName, changes);
}
private static T uncheck(Callable callable) {
diff --git a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java
index 63fef31fc1425..f639d615be1c2 100644
--- a/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java
+++ b/paimon-core/src/main/java/org/apache/paimon/io/KeyValueFileReaderFactory.java
@@ -33,6 +33,7 @@
import org.apache.paimon.schema.SchemaManager;
import org.apache.paimon.types.RowType;
import org.apache.paimon.utils.AsyncRecordReader;
+import org.apache.paimon.utils.BranchManager;
import org.apache.paimon.utils.BulkFormatMapping;
import org.apache.paimon.utils.FileStorePathFactory;
import org.apache.paimon.utils.Projection;
@@ -63,6 +64,7 @@ public class KeyValueFileReaderFactory {
private final Map bulkFormatMappings;
private final BinaryRow partition;
private final DeletionVector.Factory dvFactory;
+ private final String branch;
private KeyValueFileReaderFactory(
FileIO fileIO,
@@ -74,7 +76,8 @@ private KeyValueFileReaderFactory(
DataFilePathFactory pathFactory,
long asyncThreshold,
BinaryRow partition,
- DeletionVector.Factory dvFactory) {
+ DeletionVector.Factory dvFactory,
+ String branch) {
this.fileIO = fileIO;
this.schemaManager = schemaManager;
this.schemaId = schemaId;
@@ -86,6 +89,7 @@ private KeyValueFileReaderFactory(
this.partition = partition;
this.bulkFormatMappings = new HashMap<>();
this.dvFactory = dvFactory;
+ this.branch = branch;
}
public RecordReader createRecordReader(
@@ -110,8 +114,8 @@ private RecordReader createRecordReader(
() ->
bulkFormatMappingBuilder.build(
formatIdentifier,
- schemaManager.schema(this.schemaId),
- schemaManager.schema(schemaId));
+ schemaManager.schema(branch, this.schemaId),
+ schemaManager.schema(branch, schemaId));
BulkFormatMapping bulkFormatMapping =
reuseFormat
@@ -148,6 +152,30 @@ public static Builder builder(
FileStorePathFactory pathFactory,
KeyValueFieldsExtractor extractor,
CoreOptions options) {
+ return builder(
+ fileIO,
+ schemaManager,
+ schemaId,
+ keyType,
+ valueType,
+ formatDiscover,
+ pathFactory,
+ extractor,
+ options,
+ BranchManager.DEFAULT_MAIN_BRANCH);
+ }
+
+ public static Builder builder(
+ FileIO fileIO,
+ SchemaManager schemaManager,
+ long schemaId,
+ RowType keyType,
+ RowType valueType,
+ FileFormatDiscover formatDiscover,
+ FileStorePathFactory pathFactory,
+ KeyValueFieldsExtractor extractor,
+ CoreOptions options,
+ String branch) {
return new Builder(
fileIO,
schemaManager,
@@ -157,7 +185,8 @@ public static Builder builder(
formatDiscover,
pathFactory,
extractor,
- options);
+ options,
+ branch);
}
/** Builder for {@link KeyValueFileReaderFactory}. */
@@ -178,6 +207,7 @@ public static class Builder {
private int[][] valueProjection;
private RowType projectedKeyType;
private RowType projectedValueType;
+ private String branch;
private Builder(
FileIO fileIO,
@@ -188,7 +218,8 @@ private Builder(
FileFormatDiscover formatDiscover,
FileStorePathFactory pathFactory,
KeyValueFieldsExtractor extractor,
- CoreOptions options) {
+ CoreOptions options,
+ String branch) {
this.fileIO = fileIO;
this.schemaManager = schemaManager;
this.schemaId = schemaId;
@@ -202,6 +233,7 @@ private Builder(
this.options = options;
this.keyProjection = fullKeyProjection;
this.valueProjection = Projection.range(0, valueType.getFieldCount()).toNestedIndexes();
+ this.branch = branch;
applyProjection();
}
@@ -215,7 +247,8 @@ public Builder copyWithoutProjection() {
formatDiscover,
pathFactory,
extractor,
- options);
+ options,
+ branch);
}
public Builder withKeyProjection(int[][] projection) {
@@ -263,7 +296,8 @@ public KeyValueFileReaderFactory build(
pathFactory.createDataFilePathFactory(partition, bucket),
options.fileReaderAsyncThreshold().getBytes(),
partition,
- dvFactory);
+ dvFactory,
+ branch);
}
private void applyProjection() {
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 e6c95e8854b5d..474a73348a1ef 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
@@ -157,7 +157,7 @@ public FileStoreScan withPartitionBucket(BinaryRow partition, int bucket) {
@Override
public FileStoreScan withSnapshot(long snapshotId) {
checkState(specifiedManifests == null, "Cannot set both snapshot and manifests.");
- this.specifiedSnapshot = snapshotManager.snapshot(snapshotId);
+ this.specifiedSnapshot = snapshotManager.snapshot(branchName, snapshotId);
return this;
}
@@ -406,8 +406,8 @@ private List readManifests(Snapshot snapshot) {
// ------------------------------------------------------------------------
/** Note: Keep this thread-safe. */
- protected TableSchema scanTableSchema(long id) {
- return tableSchemas.computeIfAbsent(id, key -> schemaManager.schema(id));
+ protected TableSchema scanTableSchema(long id, String branch) {
+ return tableSchemas.computeIfAbsent(id, key -> schemaManager.schema(branch, id));
}
/** Note: Keep this thread-safe. */
diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreWrite.java b/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreWrite.java
index 30ee9c6c1d071..e7b15fe1f4d05 100644
--- a/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreWrite.java
+++ b/paimon-core/src/main/java/org/apache/paimon/operation/AbstractFileStoreWrite.java
@@ -79,6 +79,7 @@ public abstract class AbstractFileStoreWrite implements FileStoreWrite {
protected CompactionMetrics compactionMetrics = null;
protected final String tableName;
+ private final String branchName;
protected AbstractFileStoreWrite(
String commitUser,
@@ -87,7 +88,8 @@ protected AbstractFileStoreWrite(
@Nullable IndexMaintainer.Factory indexFactory,
@Nullable DeletionVectorsMaintainer.Factory deletionVectorsMaintainerFactory,
String tableName,
- int writerNumberMax) {
+ int writerNumberMax,
+ String branchName) {
this.commitUser = commitUser;
this.snapshotManager = snapshotManager;
this.scan = scan;
@@ -96,6 +98,7 @@ protected AbstractFileStoreWrite(
this.writers = new HashMap<>();
this.tableName = tableName;
this.writerNumberMax = writerNumberMax;
+ this.branchName = branchName;
}
@Override
@@ -174,7 +177,7 @@ public List prepareCommit(boolean waitCompaction, long commitIden
} else {
latestCommittedIdentifier =
snapshotManager
- .latestSnapshotOfUser(commitUser)
+ .latestSnapshotOfUser(branchName, commitUser)
.map(Snapshot::commitIdentifier)
.orElse(Long.MIN_VALUE);
}
@@ -362,7 +365,7 @@ public WriterContainer createWriterContainer(
}
}
- Long latestSnapshotId = snapshotManager.latestSnapshotId();
+ Long latestSnapshotId = snapshotManager.latestSnapshotId(branchName);
List restoreFiles = new ArrayList<>();
if (!ignorePreviousFiles && latestSnapshotId != null) {
restoreFiles = scanExistingFileMetas(latestSnapshotId, partition, bucket);
diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreRead.java b/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreRead.java
index c06cce45899a2..d358a96107a65 100644
--- a/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreRead.java
+++ b/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreRead.java
@@ -37,6 +37,7 @@
import org.apache.paimon.schema.TableSchema;
import org.apache.paimon.table.source.DataSplit;
import org.apache.paimon.types.RowType;
+import org.apache.paimon.utils.BranchManager;
import org.apache.paimon.utils.BulkFormatMapping;
import org.apache.paimon.utils.FileStorePathFactory;
import org.apache.paimon.utils.Pair;
@@ -71,6 +72,8 @@ public class AppendOnlyFileStoreRead implements FileStoreRead {
@Nullable private List filters;
+ private String branch;
+
public AppendOnlyFileStoreRead(
FileIO fileIO,
SchemaManager schemaManager,
@@ -78,6 +81,24 @@ public AppendOnlyFileStoreRead(
RowType rowType,
FileFormatDiscover formatDiscover,
FileStorePathFactory pathFactory) {
+ this(
+ fileIO,
+ schemaManager,
+ schemaId,
+ rowType,
+ formatDiscover,
+ pathFactory,
+ BranchManager.DEFAULT_MAIN_BRANCH);
+ }
+
+ public AppendOnlyFileStoreRead(
+ FileIO fileIO,
+ SchemaManager schemaManager,
+ long schemaId,
+ RowType rowType,
+ FileFormatDiscover formatDiscover,
+ FileStorePathFactory pathFactory,
+ String branch) {
this.fileIO = fileIO;
this.schemaManager = schemaManager;
this.schemaId = schemaId;
@@ -86,6 +107,7 @@ public AppendOnlyFileStoreRead(
this.bulkFormatMappings = new HashMap<>();
this.projection = Projection.range(0, rowType.getFieldCount()).toNestedIndexes();
+ this.branch = branch;
}
public FileStoreRead withProjection(int[][] projectedFields) {
@@ -113,8 +135,9 @@ public RecordReader createReader(DataSplit split) throws IOExceptio
bulkFormatMappings.computeIfAbsent(
new FormatKey(file.schemaId(), formatIdentifier),
key -> {
- TableSchema tableSchema = schemaManager.schema(this.schemaId);
- TableSchema dataSchema = schemaManager.schema(key.schemaId);
+ TableSchema tableSchema =
+ schemaManager.schema(branch, this.schemaId);
+ TableSchema dataSchema = schemaManager.schema(branch, key.schemaId);
// projection to data schema
int[][] dataProjection =
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 2cec4e064975c..32b682c1bba35 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
@@ -63,7 +63,8 @@ public AppendOnlyFileStoreScan(
scanManifestParallelism,
branchName);
this.fieldStatsConverters =
- new FieldStatsConverters(sid -> scanTableSchema(sid).fields(), schemaId);
+ new FieldStatsConverters(
+ sid -> scanTableSchema(sid, branchName).fields(), schemaId);
}
public AppendOnlyFileStoreScan withFilter(Predicate predicate) {
diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreWrite.java b/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreWrite.java
index fbb51960a06dc..eea6a34b27df1 100644
--- a/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreWrite.java
+++ b/paimon-core/src/main/java/org/apache/paimon/operation/AppendOnlyFileStoreWrite.java
@@ -85,8 +85,9 @@ public AppendOnlyFileStoreWrite(
SnapshotManager snapshotManager,
FileStoreScan scan,
CoreOptions options,
- String tableName) {
- super(commitUser, snapshotManager, scan, options, null, null, tableName);
+ String tableName,
+ String branchName) {
+ super(commitUser, snapshotManager, scan, options, null, null, tableName, branchName);
this.fileIO = fileIO;
this.read = read;
this.schemaId = schemaId;
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..13c44b4e36d59 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
@@ -74,7 +74,6 @@
import static org.apache.paimon.deletionvectors.DeletionVectorsIndexFile.DELETION_VECTORS_INDEX;
import static org.apache.paimon.index.HashIndexFile.HASH_INDEX;
-import static org.apache.paimon.utils.BranchManager.DEFAULT_MAIN_BRANCH;
/**
* Default implementation of {@link FileStoreCommit}.
@@ -187,7 +186,8 @@ public Set filterCommitted(Set commitIdentifiers) {
return commitIdentifiers;
}
- Optional latestSnapshot = snapshotManager.latestSnapshotOfUser(commitUser);
+ Optional latestSnapshot =
+ snapshotManager.latestSnapshotOfUser(branchName, commitUser);
if (latestSnapshot.isPresent()) {
Set result = new HashSet<>();
for (Long identifier : commitIdentifiers) {
@@ -668,7 +668,7 @@ private int tryOverwrite(
Map logOffsets) {
int cnt = 0;
while (true) {
- Snapshot latestSnapshot = snapshotManager.latestSnapshot();
+ Snapshot latestSnapshot = snapshotManager.latestSnapshot(branchName);
cnt++;
List changesWithOverwrite = new ArrayList<>();
@@ -736,10 +736,7 @@ public boolean tryCommitOnce(
@Nullable String newStatsFileName) {
long newSnapshotId =
latestSnapshot == null ? Snapshot.FIRST_SNAPSHOT_ID : latestSnapshot.id() + 1;
- Path newSnapshotPath =
- branchName.equals(DEFAULT_MAIN_BRANCH)
- ? snapshotManager.snapshotPath(newSnapshotId)
- : snapshotManager.branchSnapshotPath(branchName, newSnapshotId);
+ Path newSnapshotPath = snapshotManager.snapshotPath(branchName, newSnapshotId);
if (LOG.isDebugEnabled()) {
LOG.debug("Ready to commit table files to snapshot #" + newSnapshotId);
diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreRead.java b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreRead.java
index 2d3e121b1c458..cd8cac218d4a3 100644
--- a/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreRead.java
+++ b/paimon-core/src/main/java/org/apache/paimon/operation/FileStoreRead.java
@@ -30,7 +30,6 @@
* @param type of record to read.
*/
public interface FileStoreRead {
-
FileStoreRead withFilter(Predicate predicate);
/** Create a {@link RecordReader} from split. */
diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreRead.java b/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreRead.java
index db4c0ac926ca5..3b5d6c829ac6d 100644
--- a/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreRead.java
+++ b/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreRead.java
@@ -46,8 +46,10 @@
import org.apache.paimon.table.source.DataSplit;
import org.apache.paimon.table.source.DeletionFile;
import org.apache.paimon.types.RowType;
+import org.apache.paimon.utils.BranchManager;
import org.apache.paimon.utils.ProjectedRow;
import org.apache.paimon.utils.Projection;
+import org.apache.paimon.utils.StringUtils;
import org.apache.paimon.utils.UserDefinedSeqComparator;
import javax.annotation.Nullable;
@@ -95,8 +97,12 @@ public KeyValueFileStoreRead(
RowType valueType,
Comparator keyComparator,
MergeFunctionFactory mfFactory,
- KeyValueFileReaderFactory.Builder readerFactoryBuilder) {
- this.tableSchema = schemaManager.schema(schemaId);
+ KeyValueFileReaderFactory.Builder readerFactoryBuilder,
+ String branch) {
+ this.tableSchema =
+ StringUtils.isBlank(branch)
+ ? schemaManager.schema(schemaId)
+ : schemaManager.schema(branch, schemaId);
this.readerFactoryBuilder = readerFactoryBuilder;
this.fileIO = readerFactoryBuilder.fileIO();
this.keyComparator = keyComparator;
@@ -107,6 +113,27 @@ public KeyValueFileStoreRead(
this.sequenceFields = options.sequenceField();
}
+ public KeyValueFileStoreRead(
+ CoreOptions options,
+ SchemaManager schemaManager,
+ long schemaId,
+ RowType keyType,
+ RowType valueType,
+ Comparator keyComparator,
+ MergeFunctionFactory mfFactory,
+ KeyValueFileReaderFactory.Builder readerFactoryBuilder) {
+ this(
+ options,
+ schemaManager,
+ schemaId,
+ keyType,
+ valueType,
+ keyComparator,
+ mfFactory,
+ readerFactoryBuilder,
+ BranchManager.DEFAULT_MAIN_BRANCH);
+ }
+
public KeyValueFileStoreRead withKeyProjection(@Nullable int[][] projectedFields) {
readerFactoryBuilder.withKeyProjection(projectedFields);
this.keyProjectedFields = projectedFields;
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 90d2c1b184e40..20837c3f62fb4 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,10 +70,14 @@ public KeyValueFileStoreScan(
branchName);
this.fieldKeyStatsConverters =
new FieldStatsConverters(
- sid -> keyValueFieldsExtractor.keyFields(scanTableSchema(sid)), schemaId);
+ sid -> keyValueFieldsExtractor.keyFields(scanTableSchema(sid, branchName)),
+ schemaId);
this.fieldValueStatsConverters =
new FieldStatsConverters(
- sid -> keyValueFieldsExtractor.valueFields(scanTableSchema(sid)), schemaId);
+ sid ->
+ keyValueFieldsExtractor.valueFields(
+ scanTableSchema(sid, branchName)),
+ schemaId);
}
public KeyValueFileStoreScan withKeyFilter(Predicate predicate) {
diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreWrite.java b/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreWrite.java
index 41b77d612885f..a944609a237e3 100644
--- a/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreWrite.java
+++ b/paimon-core/src/main/java/org/apache/paimon/operation/KeyValueFileStoreWrite.java
@@ -118,7 +118,8 @@ public KeyValueFileStoreWrite(
@Nullable DeletionVectorsMaintainer.Factory deletionVectorsMaintainerFactory,
CoreOptions options,
KeyValueFieldsExtractor extractor,
- String tableName) {
+ String tableName,
+ String branchName) {
super(
commitUser,
snapshotManager,
@@ -126,7 +127,8 @@ public KeyValueFileStoreWrite(
options,
indexFactory,
deletionVectorsMaintainerFactory,
- tableName);
+ tableName,
+ branchName);
this.fileIO = fileIO;
this.keyType = keyType;
this.valueType = valueType;
@@ -141,7 +143,8 @@ public KeyValueFileStoreWrite(
FileFormatDiscover.of(options),
pathFactory,
extractor,
- options);
+ options,
+ branchName);
this.writerFactoryBuilder =
KeyValueFileWriterFactory.builder(
fileIO,
diff --git a/paimon-core/src/main/java/org/apache/paimon/operation/MemoryFileStoreWrite.java b/paimon-core/src/main/java/org/apache/paimon/operation/MemoryFileStoreWrite.java
index 0b2bd719ff96f..094ef5bab1dd7 100644
--- a/paimon-core/src/main/java/org/apache/paimon/operation/MemoryFileStoreWrite.java
+++ b/paimon-core/src/main/java/org/apache/paimon/operation/MemoryFileStoreWrite.java
@@ -62,7 +62,8 @@ public MemoryFileStoreWrite(
CoreOptions options,
@Nullable IndexMaintainer.Factory indexFactory,
@Nullable DeletionVectorsMaintainer.Factory deletionVectorsMaintainerFactory,
- String tableName) {
+ String tableName,
+ String branchName) {
super(
commitUser,
snapshotManager,
@@ -70,7 +71,8 @@ public MemoryFileStoreWrite(
indexFactory,
deletionVectorsMaintainerFactory,
tableName,
- options.writeMaxWritersToSpill());
+ options.writeMaxWritersToSpill(),
+ branchName);
this.options = options;
this.cacheManager = new CacheManager(options.lookupCacheMaxMemory());
}
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..4e039c7ff24b1 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
@@ -96,14 +96,10 @@ public Optional latest() {
}
public Optional latest(String branchName) {
- Path directoryPath =
- branchName.equals(DEFAULT_MAIN_BRANCH)
- ? schemaDirectory()
- : branchSchemaDirectory(branchName);
try {
- return listVersionedFiles(fileIO, directoryPath, SCHEMA_PREFIX)
+ return listVersionedFiles(fileIO, schemaDirectory(branchName), SCHEMA_PREFIX)
.reduce(Math::max)
- .map(this::schema);
+ .map(id -> schema(branchName, id));
} catch (IOException e) {
throw new UncheckedIOException(e);
}
@@ -111,23 +107,36 @@ public Optional latest(String branchName) {
/** List all schema. */
public List listAll() {
- return listAllIds().stream().map(this::schema).collect(Collectors.toList());
+ return listAll(DEFAULT_MAIN_BRANCH);
+ }
+
+ public List listAll(String branchName) {
+ return listAllIds(branchName).stream().map(this::schema).collect(Collectors.toList());
}
- /** List all schema IDs. */
public List listAllIds() {
+ return listAllIds(DEFAULT_MAIN_BRANCH);
+ }
+
+ /** List all schema IDs. */
+ public List listAllIds(String branchName) {
try {
- return listVersionedFiles(fileIO, schemaDirectory(), SCHEMA_PREFIX)
+ return listVersionedFiles(fileIO, schemaDirectory(branchName), 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(schema, DEFAULT_MAIN_BRANCH);
+ }
+
+ /** Create a new schema from {@link Schema}. */
+ public TableSchema createTable(Schema schema, String branchName) throws Exception {
while (true) {
- latest().ifPresent(
+ latest(branchName)
+ .ifPresent(
latest -> {
throw new IllegalStateException(
"Schema in filesystem exists, please use updating,"
@@ -151,25 +160,36 @@ public TableSchema createTable(Schema schema) throws Exception {
options,
schema.comment());
- boolean success = commit(newSchema);
+ boolean success = commit(branchName, newSchema);
if (success) {
return newSchema;
}
}
}
- /** Update {@link SchemaChange}s. */
public TableSchema commitChanges(SchemaChange... changes) throws Exception {
- return commitChanges(Arrays.asList(changes));
+ return commitChanges(DEFAULT_MAIN_BRANCH, changes);
}
/** Update {@link SchemaChange}s. */
+ public TableSchema commitChanges(String branchName, SchemaChange... changes) throws Exception {
+ return commitChanges(branchName, Arrays.asList(changes));
+ }
+
public TableSchema commitChanges(List changes)
+ throws Catalog.ColumnAlreadyExistException, Catalog.TableNotExistException,
+ Catalog.ColumnNotExistException {
+ return commitChanges(DEFAULT_MAIN_BRANCH, changes);
+ }
+
+ /** Update {@link SchemaChange}s. */
+ public TableSchema commitChanges(String branchName, List changes)
throws Catalog.TableNotExistException, Catalog.ColumnAlreadyExistException,
Catalog.ColumnNotExistException {
while (true) {
TableSchema schema =
- latest().orElseThrow(
+ latest(branchName)
+ .orElseThrow(
() ->
new Catalog.TableNotExistException(
fromPath(tableRoot.toString(), true)));
@@ -361,7 +381,7 @@ public TableSchema commitChanges(List changes)
newComment);
try {
- boolean success = commit(newSchema);
+ boolean success = commit(branchName, newSchema);
if (success) {
return newSchema;
}
@@ -455,9 +475,13 @@ private void updateColumn(
@VisibleForTesting
boolean commit(TableSchema newSchema) throws Exception {
- SchemaValidation.validateTableSchema(newSchema);
+ return commit(DEFAULT_MAIN_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();
@@ -474,6 +498,16 @@ public TableSchema schema(long id) {
}
}
+ /** Read schema for branch and schema id. */
+ public TableSchema schema(String branchName, long id) {
+ try {
+ return JsonSerdeUtil.fromJson(
+ fileIO.readFileUtf8(toSchemaPath(branchName, id)), TableSchema.class);
+ } catch (IOException e) {
+ throw new UncheckedIOException(e);
+ }
+ }
+
public static TableSchema fromPath(FileIO fileIO, Path path) {
try {
return JsonSerdeUtil.fromJson(fileIO.readFileUtf8(path), TableSchema.class);
@@ -486,18 +520,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(String branchName, long schemaId) {
+ fileIO.deleteQuietly(toSchemaPath(branchName, schemaId));
}
/**
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..25cb918bb79dd 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
@@ -39,8 +39,10 @@
import org.apache.paimon.table.sink.CommitCallback;
import org.apache.paimon.table.sink.DynamicBucketRowKeyExtractor;
import org.apache.paimon.table.sink.FixedBucketRowKeyExtractor;
+import org.apache.paimon.table.sink.InnerTableWrite;
import org.apache.paimon.table.sink.RowKeyExtractor;
import org.apache.paimon.table.sink.TableCommitImpl;
+import org.apache.paimon.table.sink.TableWriteImpl;
import org.apache.paimon.table.sink.UnawareBucketRowKeyExtractor;
import org.apache.paimon.table.source.InnerStreamTableScan;
import org.apache.paimon.table.source.InnerStreamTableScanImpl;
@@ -159,6 +161,15 @@ public InnerTableScan newScan() {
coreOptions(), newSnapshotReader(), DefaultValueAssigner.create(tableSchema));
}
+ @Override
+ public InnerTableScan newScan(String branch) {
+ return new InnerTableScanImpl(
+ coreOptions(),
+ newSnapshotReader(branch),
+ DefaultValueAssigner.create(tableSchema),
+ branch);
+ }
+
@Override
public InnerStreamTableScan newStreamScan() {
return new InnerStreamTableScanImpl(
@@ -169,6 +180,17 @@ public InnerStreamTableScan newStreamScan() {
DefaultValueAssigner.create(tableSchema));
}
+ @Override
+ public InnerStreamTableScan newStreamScan(String branch) {
+ return new InnerStreamTableScanImpl(
+ coreOptions(),
+ newSnapshotReader(branch),
+ snapshotManager(),
+ supportStreamingReadOverwrite(),
+ DefaultValueAssigner.create(tableSchema),
+ branch);
+ }
+
protected abstract SplitGenerator splitGenerator();
protected abstract BiConsumer nonPartitionFilterConsumer();
@@ -241,6 +263,21 @@ private FileStoreTable copyInternal(Map dynamicOptions, boolean
return copy(newTableSchema);
}
+ @Override
+ public FileStoreTable copyWithLatestSchema(String branch) {
+ Map options = tableSchema.options();
+ SchemaManager schemaManager = new SchemaManager(fileIO(), location());
+ Optional optionalLatestSchema = schemaManager.latest(branch);
+ if (optionalLatestSchema.isPresent()) {
+ TableSchema newTableSchema = optionalLatestSchema.get();
+ newTableSchema = newTableSchema.copy(options);
+ SchemaValidation.validateTableSchema(newTableSchema);
+ return copy(newTableSchema);
+ } else {
+ return this;
+ }
+ }
+
@Override
public FileStoreTable copyWithLatestSchema() {
Map options = tableSchema.options();
@@ -300,6 +337,16 @@ public TableCommitImpl newCommit(String commitUser) {
return newCommit(commitUser, DEFAULT_MAIN_BRANCH);
}
+ @Override
+ public TableWriteImpl> newWrite(String commitUser) {
+ return newWrite(commitUser, null, BranchManager.DEFAULT_MAIN_BRANCH);
+ }
+
+ @Override
+ public InnerTableWrite newWrite(String commitUser, String branch) {
+ return newWrite(commitUser, null, branch);
+ }
+
public TableCommitImpl newCommit(String commitUser, String branchName) {
CoreOptions options = coreOptions();
Runnable snapshotExpire = null;
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/AppendOnlyFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/AppendOnlyFileStoreTable.java
index 9c97d406cb7e6..3a3a3253fc63e 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/AppendOnlyFileStoreTable.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/AppendOnlyFileStoreTable.java
@@ -33,6 +33,7 @@
import org.apache.paimon.reader.RecordReader;
import org.apache.paimon.schema.TableSchema;
import org.apache.paimon.table.query.LocalTableQuery;
+import org.apache.paimon.table.sink.InnerTableWrite;
import org.apache.paimon.table.sink.TableWriteImpl;
import org.apache.paimon.table.source.AbstractDataTableRead;
import org.apache.paimon.table.source.AppendOnlySplitGenerator;
@@ -41,6 +42,7 @@
import org.apache.paimon.table.source.Split;
import org.apache.paimon.table.source.SplitGenerator;
import org.apache.paimon.types.RowKind;
+import org.apache.paimon.utils.BranchManager;
import org.apache.paimon.utils.Preconditions;
import java.io.IOException;
@@ -114,7 +116,22 @@ protected BiConsumer nonPartitionFilterConsumer() {
public InnerTableRead newRead() {
AppendOnlyFileStoreRead read = store().newRead();
return new AbstractDataTableRead(read, schema()) {
+ @Override
+ public void projection(int[][] projection) {
+ read.withProjection(projection);
+ }
+
+ @Override
+ public RecordReader reader(Split split) throws IOException {
+ return read.createReader((DataSplit) split);
+ }
+ };
+ }
+ @Override
+ public InnerTableRead newRead(String branch) {
+ AppendOnlyFileStoreRead read = store().newRead(branch);
+ return new AbstractDataTableRead(read, schema()) {
@Override
public void projection(int[][] projection) {
read.withProjection(projection);
@@ -129,15 +146,27 @@ public RecordReader reader(Split split) throws IOException {
@Override
public TableWriteImpl newWrite(String commitUser) {
- return newWrite(commitUser, null);
+ return newWrite(commitUser, null, BranchManager.DEFAULT_MAIN_BRANCH);
+ }
+
+ @Override
+ public InnerTableWrite newWrite(String commitUser, String branch) {
+ return newWrite(commitUser, null, branch);
}
@Override
public TableWriteImpl newWrite(
String commitUser, ManifestCacheFilter manifestFilter) {
+ return newWrite(commitUser, manifestFilter, BranchManager.DEFAULT_MAIN_BRANCH);
+ }
+
+ @Override
+ public TableWriteImpl newWrite(
+ String commitUser, ManifestCacheFilter manifestFilter, String branchName) {
// if this table is unaware-bucket table, we skip compaction and restored files searching
AppendOnlyFileStoreWrite writer =
- store().newWrite(commitUser, manifestFilter).withBucketMode(bucketMode());
+ store().newWrite(commitUser, manifestFilter, branchName)
+ .withBucketMode(bucketMode());
return new TableWriteImpl<>(
writer,
createRowKeyExtractor(),
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTable.java
index 2134d97cac98d..ab253695bdf90 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTable.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/FileStoreTable.java
@@ -88,6 +88,9 @@ default Optional comment() {
/** Sometimes we have to change some Immutable options to implement features. */
FileStoreTable internalCopyWithoutCheck(Map dynamicOptions);
+ /** TODO: this method is weird, old options will overwrite new options. */
+ FileStoreTable copyWithLatestSchema(String branch);
+
/** TODO: this method is weird, old options will overwrite new options. */
FileStoreTable copyWithLatestSchema();
@@ -96,6 +99,9 @@ default Optional comment() {
TableWriteImpl> newWrite(String commitUser, ManifestCacheFilter manifestFilter);
+ TableWriteImpl> newWrite(
+ String commitUser, ManifestCacheFilter manifestFilter, String branchName);
+
@Override
TableCommitImpl newCommit(String commitUser);
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..72f69704b53c7 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
@@ -26,6 +26,7 @@
import org.apache.paimon.options.Options;
import org.apache.paimon.schema.SchemaManager;
import org.apache.paimon.schema.TableSchema;
+import org.apache.paimon.utils.BranchManager;
import java.io.IOException;
import java.io.UncheckedIOException;
@@ -36,13 +37,17 @@
public class FileStoreTableFactory {
public static FileStoreTable create(CatalogContext context) {
+ return create(context, BranchManager.DEFAULT_MAIN_BRANCH);
+ }
+
+ public static FileStoreTable create(CatalogContext context, String branch) {
FileIO fileIO;
try {
fileIO = FileIO.get(CoreOptions.path(context.options()), context);
} catch (IOException e) {
throw new UncheckedIOException(e);
}
- return create(fileIO, context.options());
+ return create(fileIO, context.options(), branch);
}
public static FileStoreTable create(FileIO fileIO, Path path) {
@@ -51,11 +56,11 @@ public static FileStoreTable create(FileIO fileIO, Path path) {
return create(fileIO, options);
}
- public static FileStoreTable create(FileIO fileIO, Options options) {
+ public static FileStoreTable create(FileIO fileIO, Options options, String branch) {
Path tablePath = CoreOptions.path(options);
TableSchema tableSchema =
new SchemaManager(fileIO, tablePath)
- .latest()
+ .latest(branch)
.orElseThrow(
() ->
new IllegalArgumentException(
@@ -70,6 +75,10 @@ public static FileStoreTable create(FileIO fileIO, Options options) {
new CatalogEnvironment(Lock.emptyFactory(), null, null));
}
+ public static FileStoreTable create(FileIO fileIO, Options options) {
+ return create(fileIO, options, BranchManager.DEFAULT_MAIN_BRANCH);
+ }
+
public static FileStoreTable create(FileIO fileIO, Path tablePath, TableSchema tableSchema) {
return create(
fileIO,
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/InnerTable.java b/paimon-core/src/main/java/org/apache/paimon/table/InnerTable.java
index 456834ebcf8aa..477887305e648 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/InnerTable.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/InnerTable.java
@@ -35,14 +35,30 @@ public interface InnerTable extends Table {
InnerTableScan newScan();
+ default InnerTableScan newScan(String branch) {
+ return newScan();
+ }
+
InnerStreamTableScan newStreamScan();
+ default InnerStreamTableScan newStreamScan(String branch) {
+ return newStreamScan();
+ }
+
InnerTableRead newRead();
+ default InnerTableRead newRead(String branch) {
+ return newRead();
+ }
+
InnerTableWrite newWrite(String commitUser);
+ InnerTableWrite newWrite(String commitUser, String branch);
+
InnerTableCommit newCommit(String commitUser);
+ InnerTableCommit newCommit(String commitUser, String branch);
+
@Override
default ReadBuilder newReadBuilder() {
return new ReadBuilderImpl(this);
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java b/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java
index 7b30fb832ef07..06559638ae26e 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/PrimaryKeyFileStoreTable.java
@@ -52,6 +52,7 @@
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;
/** {@link FileStoreTable} for primary key table. */
class PrimaryKeyFileStoreTable extends AbstractFileStoreTable {
@@ -153,6 +154,29 @@ protected BiConsumer nonPartitionFilterConsumer() {
};
}
+ @Override
+ public InnerTableRead newRead(String branch) {
+ return new KeyValueTableRead(store().newRead(branch), schema()) {
+
+ @Override
+ public void projection(int[][] projection) {
+ read.withValueProjection(projection);
+ }
+
+ @Override
+ protected RecordReader.RecordIterator rowDataRecordIteratorFromKv(
+ RecordReader.RecordIterator kvRecordIterator) {
+ return new ValueContentRowDataRecordIterator(kvRecordIterator);
+ }
+
+ @Override
+ public InnerTableRead forceKeepDelete() {
+ read.forceKeepDelete();
+ return this;
+ }
+ };
+ }
+
@Override
public InnerTableRead newRead() {
return new KeyValueTableRead(store().newRead(), schema()) {
@@ -178,18 +202,24 @@ public InnerTableRead forceKeepDelete() {
@Override
public TableWriteImpl newWrite(String commitUser) {
- return newWrite(commitUser, null);
+ return newWrite(commitUser, null, DEFAULT_MAIN_BRANCH);
}
@Override
public TableWriteImpl newWrite(
String commitUser, ManifestCacheFilter manifestFilter) {
+ return newWrite(commitUser, manifestFilter, DEFAULT_MAIN_BRANCH);
+ }
+
+ @Override
+ public TableWriteImpl newWrite(
+ String commitUser, ManifestCacheFilter manifestFilter, String branchName) {
TableSchema schema = schema();
CoreOptions options = store().options();
RowKindGenerator rowKindGenerator = RowKindGenerator.create(schema, options);
KeyValue kv = new KeyValue();
return new TableWriteImpl<>(
- store().newWrite(commitUser, manifestFilter),
+ store().newWrite(commitUser, manifestFilter, branchName),
createRowKeyExtractor(),
record -> {
InternalRow row = record.row();
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/ReadonlyTable.java b/paimon-core/src/main/java/org/apache/paimon/table/ReadonlyTable.java
index f0d52b641015e..0e6a497b6a975 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/ReadonlyTable.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/ReadonlyTable.java
@@ -77,6 +77,14 @@ default InnerTableWrite newWrite(String commitUser) {
this.getClass().getSimpleName()));
}
+ @Override
+ default InnerTableWrite newWrite(String commitUser, String branch) {
+ throw new UnsupportedOperationException(
+ String.format(
+ "Readonly Table %s does not support newWrite with branch.",
+ this.getClass().getSimpleName()));
+ }
+
@Override
default InnerTableCommit newCommit(String commitUser) {
throw new UnsupportedOperationException(
@@ -85,6 +93,14 @@ default InnerTableCommit newCommit(String commitUser) {
this.getClass().getSimpleName()));
}
+ @Override
+ default InnerTableCommit newCommit(String commitUser, String branch) {
+ throw new UnsupportedOperationException(
+ String.format(
+ "Readonly Table %s does not support newCommit with branch.",
+ this.getClass().getSimpleName()));
+ }
+
@Override
default InnerStreamTableScan newStreamScan() {
throw new UnsupportedOperationException(
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/sink/BatchWriteBuilder.java b/paimon-core/src/main/java/org/apache/paimon/table/sink/BatchWriteBuilder.java
index ee0091a7bff98..5832a0d8feca9 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/sink/BatchWriteBuilder.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/sink/BatchWriteBuilder.java
@@ -61,6 +61,8 @@ default BatchWriteBuilder withOverwrite() {
return this;
}
+ BatchWriteBuilder toBranch(String branch);
+
/** Overwrite writing, same as the 'INSERT OVERWRITE T PARTITION (...)' semantics of SQL. */
BatchWriteBuilder withOverwrite(@Nullable Map staticPartition);
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/sink/BatchWriteBuilderImpl.java b/paimon-core/src/main/java/org/apache/paimon/table/sink/BatchWriteBuilderImpl.java
index 87693ba0a7b86..756296cb2e094 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/sink/BatchWriteBuilderImpl.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/sink/BatchWriteBuilderImpl.java
@@ -20,6 +20,7 @@
import org.apache.paimon.table.InnerTable;
import org.apache.paimon.types.RowType;
+import org.apache.paimon.utils.BranchManager;
import javax.annotation.Nullable;
@@ -33,6 +34,7 @@ public class BatchWriteBuilderImpl implements BatchWriteBuilder {
private final InnerTable table;
private final String commitUser = UUID.randomUUID().toString();
+ private String branch = BranchManager.DEFAULT_MAIN_BRANCH;
private Map staticPartition;
@@ -56,16 +58,23 @@ public BatchWriteBuilder withOverwrite(@Nullable Map staticParti
return this;
}
+ @Override
+ public BatchWriteBuilder toBranch(String branch) {
+ this.branch = branch;
+ return this;
+ }
+
@Override
public BatchTableWrite newWrite() {
- return table.newWrite(commitUser)
+ return table.newWrite(commitUser, branch)
.withIgnorePreviousFiles(staticPartition != null)
.withExecutionMode(false);
}
@Override
public BatchTableCommit newCommit() {
- InnerTableCommit commit = table.newCommit(commitUser).withOverwrite(staticPartition);
+ InnerTableCommit commit =
+ table.newCommit(commitUser, branch).withOverwrite(staticPartition);
commit.ignoreEmptyCommit(true);
return commit;
}
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/sink/StreamWriteBuilder.java b/paimon-core/src/main/java/org/apache/paimon/table/sink/StreamWriteBuilder.java
index 7f272e0751265..3236aaa52ff01 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/sink/StreamWriteBuilder.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/sink/StreamWriteBuilder.java
@@ -51,6 +51,8 @@ public interface StreamWriteBuilder extends WriteBuilder {
*/
StreamWriteBuilder withCommitUser(String commitUser);
+ StreamWriteBuilder toBranch(String branch);
+
/** Create a {@link TableWrite} to write {@link InternalRow}s. */
@Override
StreamTableWrite newWrite();
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/sink/StreamWriteBuilderImpl.java b/paimon-core/src/main/java/org/apache/paimon/table/sink/StreamWriteBuilderImpl.java
index 402b0beb2eab9..d78bcedd49c0f 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/sink/StreamWriteBuilderImpl.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/sink/StreamWriteBuilderImpl.java
@@ -20,6 +20,7 @@
import org.apache.paimon.table.InnerTable;
import org.apache.paimon.types.RowType;
+import org.apache.paimon.utils.BranchManager;
import java.util.UUID;
@@ -31,6 +32,7 @@ public class StreamWriteBuilderImpl implements StreamWriteBuilder {
private final InnerTable table;
private String commitUser = UUID.randomUUID().toString();
+ private String branch = BranchManager.DEFAULT_MAIN_BRANCH;
public StreamWriteBuilderImpl(InnerTable table) {
this.table = table;
@@ -57,13 +59,19 @@ public StreamWriteBuilder withCommitUser(String commitUser) {
return this;
}
+ @Override
+ public StreamWriteBuilder toBranch(String branch) {
+ this.branch = branch;
+ return this;
+ }
+
@Override
public StreamTableWrite newWrite() {
- return table.newWrite(commitUser);
+ return table.newWrite(commitUser, branch);
}
@Override
public StreamTableCommit newCommit() {
- return table.newCommit(commitUser).ignoreEmptyCommit(false);
+ return table.newCommit(commitUser, branch).ignoreEmptyCommit(false);
}
}
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractInnerTableScan.java b/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractInnerTableScan.java
index b597ec70bdbf2..7fba9a699d4f7 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractInnerTableScan.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/source/AbstractInnerTableScan.java
@@ -59,10 +59,13 @@ public abstract class AbstractInnerTableScan implements InnerTableScan {
private final CoreOptions options;
protected final SnapshotReader snapshotReader;
+ protected final String branch;
- protected AbstractInnerTableScan(CoreOptions options, SnapshotReader snapshotReader) {
+ protected AbstractInnerTableScan(
+ CoreOptions options, SnapshotReader snapshotReader, String branch) {
this.options = options;
this.snapshotReader = snapshotReader;
+ this.branch = branch;
}
@VisibleForTesting
@@ -106,10 +109,10 @@ protected StartingScanner createStartingScanner(boolean isStreaming) {
case COMPACT_BUCKET_TABLE:
checkArgument(
isStreaming, "Set 'streaming-compact' in batch mode. This is unexpected.");
- return new ContinuousCompactorStartingScanner(snapshotManager);
+ return new ContinuousCompactorStartingScanner(snapshotManager).withBranch(branch);
case COMPACT_APPEND_NO_BUCKET:
case FILE_MONITOR:
- return new FullStartingScanner(snapshotManager);
+ return new FullStartingScanner(snapshotManager).withBranch(branch);
}
// read from consumer id
@@ -119,18 +122,19 @@ protected StartingScanner createStartingScanner(boolean isStreaming) {
Optional consumer = consumerManager.consumer(consumerId);
if (consumer.isPresent()) {
return new ContinuousFromSnapshotStartingScanner(
- snapshotManager, consumer.get().nextSnapshot());
+ snapshotManager, consumer.get().nextSnapshot())
+ .withBranch(branch);
}
}
CoreOptions.StartupMode startupMode = options.startupMode();
switch (startupMode) {
case LATEST_FULL:
- return new FullStartingScanner(snapshotManager);
+ return new FullStartingScanner(snapshotManager).withBranch(branch);
case LATEST:
return isStreaming
- ? new ContinuousLatestStartingScanner(snapshotManager)
- : new FullStartingScanner(snapshotManager);
+ ? new ContinuousLatestStartingScanner(snapshotManager).withBranch(branch)
+ : new FullStartingScanner(snapshotManager).withBranch(branch);
case COMPACTED_FULL:
if (options.changelogProducer() == ChangelogProducer.FULL_COMPACTION
|| options.toConfiguration().contains(FULL_COMPACTION_DELTA_COMMITS)) {
@@ -138,36 +142,45 @@ protected StartingScanner createStartingScanner(boolean isStreaming) {
options.toConfiguration()
.getOptional(FULL_COMPACTION_DELTA_COMMITS)
.orElse(1);
- return new FullCompactedStartingScanner(snapshotManager, deltaCommits);
+ return new FullCompactedStartingScanner(snapshotManager, deltaCommits)
+ .withBranch(branch);
} else {
- return new CompactedStartingScanner(snapshotManager);
+ return new CompactedStartingScanner(snapshotManager).withBranch(branch);
}
case FROM_TIMESTAMP:
Long startupMillis = options.scanTimestampMills();
return isStreaming
? new ContinuousFromTimestampStartingScanner(snapshotManager, startupMillis)
- : new StaticFromTimestampStartingScanner(snapshotManager, startupMillis);
+ .withBranch(branch)
+ : new StaticFromTimestampStartingScanner(snapshotManager, startupMillis)
+ .withBranch(branch);
case FROM_FILE_CREATION_TIME:
Long fileCreationTimeMills = options.scanFileCreationTimeMills();
- return new FileCreationTimeStartingScanner(snapshotManager, fileCreationTimeMills);
+ return new FileCreationTimeStartingScanner(snapshotManager, fileCreationTimeMills)
+ .withBranch(branch);
case FROM_SNAPSHOT:
if (options.scanSnapshotId() != null) {
return isStreaming
? new ContinuousFromSnapshotStartingScanner(
- snapshotManager, options.scanSnapshotId())
+ snapshotManager, options.scanSnapshotId())
+ .withBranch(branch)
: new StaticFromSnapshotStartingScanner(
- snapshotManager, options.scanSnapshotId());
+ snapshotManager, options.scanSnapshotId())
+ .withBranch(branch);
} else {
checkArgument(!isStreaming, "Cannot scan from tag in streaming mode.");
return new StaticFromTagStartingScanner(
- snapshotManager, options().scanTagName());
+ snapshotManager, options().scanTagName())
+ .withBranch(branch);
}
case FROM_SNAPSHOT_FULL:
return isStreaming
? new ContinuousFromSnapshotFullStartingScanner(
- snapshotManager, options.scanSnapshotId())
+ snapshotManager, options.scanSnapshotId())
+ .withBranch(branch)
: new StaticFromSnapshotStartingScanner(
- snapshotManager, options.scanSnapshotId());
+ snapshotManager, options.scanSnapshotId())
+ .withBranch(branch);
case INCREMENTAL:
checkArgument(!isStreaming, "Cannot read incremental in streaming mode.");
Pair incrementalBetween = options.incrementalBetween();
@@ -188,22 +201,25 @@ protected StartingScanner createStartingScanner(boolean isStreaming) {
if (options.toMap().get(CoreOptions.INCREMENTAL_BETWEEN.key()) != null) {
try {
return new IncrementalStartingScanner(
- snapshotManager,
- Long.parseLong(incrementalBetween.getLeft()),
- Long.parseLong(incrementalBetween.getRight()),
- scanMode);
+ snapshotManager,
+ Long.parseLong(incrementalBetween.getLeft()),
+ Long.parseLong(incrementalBetween.getRight()),
+ scanMode)
+ .withBranch(branch);
} catch (NumberFormatException e) {
return new IncrementalTagStartingScanner(
- snapshotManager,
- incrementalBetween.getLeft(),
- incrementalBetween.getRight());
+ snapshotManager,
+ incrementalBetween.getLeft(),
+ incrementalBetween.getRight())
+ .withBranch(branch);
}
} else {
return new IncrementalTimeStampStartingScanner(
- snapshotManager,
- Long.parseLong(incrementalBetween.getLeft()),
- Long.parseLong(incrementalBetween.getRight()),
- scanMode);
+ snapshotManager,
+ Long.parseLong(incrementalBetween.getLeft()),
+ Long.parseLong(incrementalBetween.getRight()),
+ scanMode)
+ .withBranch(branch);
}
default:
throw new UnsupportedOperationException(
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/InnerStreamTableScanImpl.java b/paimon-core/src/main/java/org/apache/paimon/table/source/InnerStreamTableScanImpl.java
index 5fc8866fd6c04..92a8ded5b549b 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/source/InnerStreamTableScanImpl.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/source/InnerStreamTableScanImpl.java
@@ -36,6 +36,7 @@
import org.apache.paimon.table.source.snapshot.StartingScanner;
import org.apache.paimon.table.source.snapshot.StartingScanner.ScannedResult;
import org.apache.paimon.table.source.snapshot.StaticFromSnapshotStartingScanner;
+import org.apache.paimon.utils.BranchManager;
import org.apache.paimon.utils.Filter;
import org.apache.paimon.utils.SnapshotManager;
@@ -65,6 +66,7 @@ public class InnerStreamTableScanImpl extends AbstractInnerTableScan
private boolean isFullPhaseEnd = false;
@Nullable private Long currentWatermark;
@Nullable private Long nextSnapshotId;
+ private String branch;
public InnerStreamTableScanImpl(
CoreOptions options,
@@ -72,11 +74,28 @@ public InnerStreamTableScanImpl(
SnapshotManager snapshotManager,
boolean supportStreamingReadOverwrite,
DefaultValueAssigner defaultValueAssigner) {
- super(options, snapshotReader);
+ this(
+ options,
+ snapshotReader,
+ snapshotManager,
+ supportStreamingReadOverwrite,
+ defaultValueAssigner,
+ BranchManager.DEFAULT_MAIN_BRANCH);
+ }
+
+ public InnerStreamTableScanImpl(
+ CoreOptions options,
+ SnapshotReader snapshotReader,
+ SnapshotManager snapshotManager,
+ boolean supportStreamingReadOverwrite,
+ DefaultValueAssigner defaultValueAssigner,
+ String branch) {
+ super(options, snapshotReader, branch);
this.options = options;
this.snapshotManager = snapshotManager;
this.supportStreamingReadOverwrite = supportStreamingReadOverwrite;
this.defaultValueAssigner = defaultValueAssigner;
+ this.branch = branch;
}
@Override
@@ -146,14 +165,15 @@ private Plan tryFirstPlan() {
nextSnapshotId = currentSnapshotId + 1;
}
isFullPhaseEnd =
- boundedChecker.shouldEndInput(snapshotManager.snapshot(currentSnapshotId));
+ boundedChecker.shouldEndInput(
+ snapshotManager.snapshot(branch, currentSnapshotId));
return scannedResult.plan();
} else if (result instanceof StartingScanner.NextSnapshot) {
nextSnapshotId = ((StartingScanner.NextSnapshot) result).nextSnapshotId();
isFullPhaseEnd =
- snapshotManager.snapshotExists(nextSnapshotId - 1)
+ snapshotManager.snapshotExists(branch, nextSnapshotId - 1)
&& boundedChecker.shouldEndInput(
- snapshotManager.snapshot(nextSnapshotId - 1));
+ snapshotManager.snapshot(branch, nextSnapshotId - 1));
}
return SnapshotNotExistPlan.INSTANCE;
}
@@ -164,8 +184,8 @@ private Plan nextPlan() {
throw new EndOfScanException();
}
- if (!snapshotManager.snapshotExists(nextSnapshotId)) {
- Long earliestSnapshotId = snapshotManager.earliestSnapshotId();
+ if (!snapshotManager.snapshotExists(branch, nextSnapshotId)) {
+ Long earliestSnapshotId = snapshotManager.earliestSnapshotId(branch);
if (earliestSnapshotId != null && earliestSnapshotId > nextSnapshotId) {
throw new OutOfRangeException(
String.format(
@@ -180,7 +200,7 @@ private Plan nextPlan() {
return SnapshotNotExistPlan.INSTANCE;
}
- Snapshot snapshot = snapshotManager.snapshot(nextSnapshotId);
+ Snapshot snapshot = snapshotManager.snapshot(branch, nextSnapshotId);
if (boundedChecker.shouldEndInput(snapshot)) {
throw new EndOfScanException();
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/InnerTableScan.java b/paimon-core/src/main/java/org/apache/paimon/table/source/InnerTableScan.java
index 613b2efc26efb..66964fb779b9a 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/source/InnerTableScan.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/source/InnerTableScan.java
@@ -29,6 +29,10 @@ public interface InnerTableScan extends TableScan {
InnerTableScan withFilter(Predicate predicate);
+ default InnerTableScan withBranch(String branch) {
+ return this;
+ }
+
default InnerTableScan withLimit(int limit) {
return this;
}
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/InnerTableScanImpl.java b/paimon-core/src/main/java/org/apache/paimon/table/source/InnerTableScanImpl.java
index 375ef7e0ac6c2..f89b6ad37106b 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/source/InnerTableScanImpl.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/source/InnerTableScanImpl.java
@@ -24,6 +24,7 @@
import org.apache.paimon.table.source.snapshot.SnapshotReader;
import org.apache.paimon.table.source.snapshot.StartingScanner;
import org.apache.paimon.table.source.snapshot.StartingScanner.ScannedResult;
+import org.apache.paimon.utils.BranchManager;
import java.util.ArrayList;
import java.util.List;
@@ -42,7 +43,15 @@ public InnerTableScanImpl(
CoreOptions options,
SnapshotReader snapshotReader,
DefaultValueAssigner defaultValueAssigner) {
- super(options, snapshotReader);
+ this(options, snapshotReader, defaultValueAssigner, BranchManager.DEFAULT_MAIN_BRANCH);
+ }
+
+ public InnerTableScanImpl(
+ CoreOptions options,
+ SnapshotReader snapshotReader,
+ DefaultValueAssigner defaultValueAssigner,
+ String branch) {
+ super(options, snapshotReader, branch);
this.hasNext = true;
this.defaultValueAssigner = defaultValueAssigner;
if (options.deletionVectorsEnabled()) {
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/ReadBuilder.java b/paimon-core/src/main/java/org/apache/paimon/table/source/ReadBuilder.java
index fc7e41569867e..07b22f7b1ed1a 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/source/ReadBuilder.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/source/ReadBuilder.java
@@ -122,6 +122,9 @@ default ReadBuilder withProjection(int[] projection) {
/** the row number pushed down. */
ReadBuilder withLimit(int limit);
+ /** which branch to read from. */
+ ReadBuilder fromBranch(String branch);
+
/** Create a {@link TableScan} to perform batch planning. */
TableScan newScan();
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/ReadBuilderImpl.java b/paimon-core/src/main/java/org/apache/paimon/table/source/ReadBuilderImpl.java
index 1ceb614ec4576..685eb4211c39b 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/source/ReadBuilderImpl.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/source/ReadBuilderImpl.java
@@ -22,6 +22,7 @@
import org.apache.paimon.predicate.PredicateBuilder;
import org.apache.paimon.table.InnerTable;
import org.apache.paimon.types.RowType;
+import org.apache.paimon.utils.BranchManager;
import org.apache.paimon.utils.Projection;
import org.apache.paimon.utils.TypeUtils;
@@ -43,6 +44,9 @@ public class ReadBuilderImpl implements ReadBuilder {
private Map partitionSpec;
+ // default read main branch.
+ private String branch = BranchManager.DEFAULT_MAIN_BRANCH;
+
public ReadBuilderImpl(InnerTable table) {
this.table = table;
}
@@ -88,10 +92,16 @@ public ReadBuilder withLimit(int limit) {
return this;
}
+ @Override
+ public ReadBuilder fromBranch(String branch) {
+ this.branch = branch;
+ return this;
+ }
+
@Override
public TableScan newScan() {
InnerTableScan tableScan =
- table.newScan().withFilter(filter).withPartitionFilter(partitionSpec);
+ table.newScan(branch).withFilter(filter).withPartitionFilter(partitionSpec);
if (limit != null) {
tableScan.withLimit(limit);
}
@@ -100,12 +110,12 @@ public TableScan newScan() {
@Override
public StreamTableScan newStreamScan() {
- return (StreamTableScan) table.newStreamScan().withFilter(filter);
+ return (StreamTableScan) table.newStreamScan(branch).withFilter(filter);
}
@Override
public TableRead newRead() {
- InnerTableRead read = table.newRead().withFilter(filter);
+ InnerTableRead read = table.newRead(branch).withFilter(filter);
if (projection != null) {
read.withProjection(projection);
}
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/AbstractStartingScanner.java b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/AbstractStartingScanner.java
index d711781d7d62b..ae74e3e59926f 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/AbstractStartingScanner.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/AbstractStartingScanner.java
@@ -19,6 +19,7 @@
package org.apache.paimon.table.source.snapshot;
import org.apache.paimon.table.source.ScanMode;
+import org.apache.paimon.utils.BranchManager;
import org.apache.paimon.utils.SnapshotManager;
/** The abstract class for StartingScanner. */
@@ -28,6 +29,8 @@ public abstract class AbstractStartingScanner implements StartingScanner {
protected Long startingSnapshotId = null;
+ protected String branch = BranchManager.DEFAULT_MAIN_BRANCH;
+
AbstractStartingScanner(SnapshotManager snapshotManager) {
this.snapshotManager = snapshotManager;
}
@@ -36,6 +39,11 @@ protected ScanMode startingScanMode() {
return ScanMode.DELTA;
}
+ public StartingScanner withBranch(String branch) {
+ this.branch = branch;
+ return this;
+ }
+
@Override
public StartingContext startingContext() {
if (startingSnapshotId == null) {
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/CompactedStartingScanner.java b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/CompactedStartingScanner.java
index a1595462769d5..e9c0a69951a1f 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/CompactedStartingScanner.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/CompactedStartingScanner.java
@@ -47,7 +47,7 @@ public ScanMode startingScanMode() {
public Result scan(SnapshotReader snapshotReader) {
Long startingSnapshotId = pick();
if (startingSnapshotId == null) {
- startingSnapshotId = snapshotManager.latestSnapshotId();
+ startingSnapshotId = snapshotManager.latestSnapshotId(branch);
if (startingSnapshotId == null) {
LOG.debug("There is currently no snapshot. Wait for the snapshot generation.");
return new NoSnapshot();
@@ -64,6 +64,7 @@ public Result scan(SnapshotReader snapshotReader) {
@Nullable
protected Long pick() {
- return snapshotManager.pickOrLatest(s -> s.commitKind() == Snapshot.CommitKind.COMPACT);
+ return snapshotManager.pickOrLatest(
+ branch, s -> s.commitKind() == Snapshot.CommitKind.COMPACT);
}
}
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/ContinuousCompactorStartingScanner.java b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/ContinuousCompactorStartingScanner.java
index 5f24366ca1cf6..8ee02d713503e 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/ContinuousCompactorStartingScanner.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/ContinuousCompactorStartingScanner.java
@@ -37,15 +37,15 @@ public ContinuousCompactorStartingScanner(SnapshotManager snapshotManager) {
@Override
public Result scan(SnapshotReader snapshotReader) {
- Long latestSnapshotId = snapshotManager.latestSnapshotId();
- Long earliestSnapshotId = snapshotManager.earliestSnapshotId();
+ Long latestSnapshotId = snapshotManager.latestSnapshotId(branch);
+ Long earliestSnapshotId = snapshotManager.earliestSnapshotId(branch);
if (latestSnapshotId == null || earliestSnapshotId == null) {
LOG.debug("There is currently no snapshot. Wait for the snapshot generation.");
return new NoSnapshot();
}
for (long id = latestSnapshotId; id >= earliestSnapshotId; id--) {
- Snapshot snapshot = snapshotManager.snapshot(id);
+ Snapshot snapshot = snapshotManager.snapshot(branch, id);
if (snapshot.commitKind() == Snapshot.CommitKind.COMPACT) {
LOG.debug("Found latest compact snapshot {}, reading from the next snapshot.", id);
return new NextSnapshot(id + 1);
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/ContinuousFromSnapshotFullStartingScanner.java b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/ContinuousFromSnapshotFullStartingScanner.java
index 29bd0b7cdc26e..78ca7c6d9647d 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/ContinuousFromSnapshotFullStartingScanner.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/ContinuousFromSnapshotFullStartingScanner.java
@@ -41,7 +41,7 @@ public ScanMode startingScanMode() {
@Override
public Result scan(SnapshotReader snapshotReader) {
- Long earliestSnapshotId = snapshotManager.earliestSnapshotId();
+ Long earliestSnapshotId = snapshotManager.earliestSnapshotId(branch);
if (earliestSnapshotId == null) {
return new NoSnapshot();
}
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/ContinuousFromSnapshotStartingScanner.java b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/ContinuousFromSnapshotStartingScanner.java
index c966de58c46f2..b1953e0b59bde 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/ContinuousFromSnapshotStartingScanner.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/ContinuousFromSnapshotStartingScanner.java
@@ -34,7 +34,7 @@ public ContinuousFromSnapshotStartingScanner(SnapshotManager snapshotManager, lo
@Override
public Result scan(SnapshotReader snapshotReader) {
- Long earliestSnapshotId = snapshotManager.earliestSnapshotId();
+ Long earliestSnapshotId = snapshotManager.earliestSnapshotId(branch);
if (earliestSnapshotId == null) {
return new NoSnapshot();
}
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/ContinuousFromTimestampStartingScanner.java b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/ContinuousFromTimestampStartingScanner.java
index 6113773ffe5a2..a9b6c9aa9daac 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/ContinuousFromTimestampStartingScanner.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/ContinuousFromTimestampStartingScanner.java
@@ -53,7 +53,7 @@ public StartingContext startingContext() {
@Override
public Result scan(SnapshotReader snapshotReader) {
- Long startingSnapshotId = snapshotManager.earlierThanTimeMills(startupMillis);
+ Long startingSnapshotId = snapshotManager.earlierThanTimeMills(branch, startupMillis);
if (startingSnapshotId == null) {
LOG.debug("There is currently no snapshot. Waiting for snapshot generation.");
return new NoSnapshot();
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/ContinuousLatestStartingScanner.java b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/ContinuousLatestStartingScanner.java
index 96c3e700f4eee..be64f3212fb13 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/ContinuousLatestStartingScanner.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/ContinuousLatestStartingScanner.java
@@ -41,7 +41,7 @@ public ContinuousLatestStartingScanner(SnapshotManager snapshotManager) {
@Override
public Result scan(SnapshotReader snapshotReader) {
- Long latestSnapshotId = snapshotManager.latestSnapshotId();
+ Long latestSnapshotId = snapshotManager.latestSnapshotId(branch);
if (latestSnapshotId == null) {
LOG.debug("There is currently no snapshot. Wait for the snapshot generation.");
return new NoSnapshot();
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/FileCreationTimeStartingScanner.java b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/FileCreationTimeStartingScanner.java
index a3a87bef23a50..4b2d0d24a5cc4 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/FileCreationTimeStartingScanner.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/FileCreationTimeStartingScanner.java
@@ -48,7 +48,7 @@ public ScanMode startingScanMode() {
@Override
public Result scan(SnapshotReader snapshotReader) {
- Long startingSnapshotId = snapshotManager.latestSnapshotId();
+ Long startingSnapshotId = snapshotManager.latestSnapshotId(branch);
if (startingSnapshotId == null) {
LOG.debug("There is currently no snapshot. Waiting for snapshot generation.");
return new NoSnapshot();
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/FullCompactedStartingScanner.java b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/FullCompactedStartingScanner.java
index 5015d56c7221c..b4065d657b640 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/FullCompactedStartingScanner.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/FullCompactedStartingScanner.java
@@ -65,7 +65,7 @@ public ScanMode startingScanMode() {
public Result scan(SnapshotReader snapshotReader) {
Long startingSnapshotId = pick();
if (startingSnapshotId == null) {
- startingSnapshotId = snapshotManager.latestSnapshotId();
+ startingSnapshotId = snapshotManager.latestSnapshotId(branch);
if (startingSnapshotId == null) {
LOG.debug("There is currently no snapshot. Wait for the snapshot generation.");
return new NoSnapshot();
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/FullStartingScanner.java b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/FullStartingScanner.java
index c177be7eafb71..ac19e7a0ff562 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/FullStartingScanner.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/FullStartingScanner.java
@@ -42,7 +42,7 @@ public ScanMode startingScanMode() {
@Override
public Result scan(SnapshotReader snapshotReader) {
- Long startingSnapshotId = snapshotManager.latestSnapshotId();
+ Long startingSnapshotId = snapshotManager.latestSnapshotId(branch);
if (startingSnapshotId == null) {
LOG.debug("There is currently no snapshot. Waiting for snapshot generation.");
return new NoSnapshot();
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/IncrementalStartingScanner.java b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/IncrementalStartingScanner.java
index 49ab3a87e7643..9489159faa882 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/IncrementalStartingScanner.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/IncrementalStartingScanner.java
@@ -53,7 +53,7 @@ public IncrementalStartingScanner(
public Result scan(SnapshotReader reader) {
Map, List> grouped = new HashMap<>();
for (long i = startingSnapshotId + 1; i < endingSnapshotId + 1; i++) {
- List splits = readSplits(reader, snapshotManager.snapshot(i));
+ List splits = readSplits(reader, snapshotManager.snapshot(branch, i));
for (DataSplit split : splits) {
grouped.computeIfAbsent(
Pair.of(split.partition(), split.bucket()), k -> new ArrayList<>())
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/IncrementalTagStartingScanner.java b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/IncrementalTagStartingScanner.java
index 2cdf5bff9d26e..cd1c78e22040e 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/IncrementalTagStartingScanner.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/IncrementalTagStartingScanner.java
@@ -45,8 +45,8 @@ public IncrementalTagStartingScanner(
public Result scan(SnapshotReader reader) {
TagManager tagManager =
new TagManager(snapshotManager.fileIO(), snapshotManager.tablePath());
- Snapshot tag1 = tagManager.taggedSnapshot(start);
- Snapshot tag2 = tagManager.taggedSnapshot(end);
+ Snapshot tag1 = tagManager.taggedSnapshot(branch, start);
+ Snapshot tag2 = tagManager.taggedSnapshot(branch, end);
if (tag2.id() <= tag1.id()) {
throw new IllegalArgumentException(
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/IncrementalTimeStampStartingScanner.java b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/IncrementalTimeStampStartingScanner.java
index 9fa59fdd77d4f..3e0df08adb2de 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/IncrementalTimeStampStartingScanner.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/IncrementalTimeStampStartingScanner.java
@@ -46,19 +46,22 @@ public IncrementalTimeStampStartingScanner(
@Override
public Result scan(SnapshotReader reader) {
- Snapshot earliestSnapshot = snapshotManager.snapshot(snapshotManager.earliestSnapshotId());
- Snapshot latestSnapshot = snapshotManager.latestSnapshot();
+ Snapshot earliestSnapshot =
+ snapshotManager.snapshot(branch, snapshotManager.earliestSnapshotId(branch));
+ Snapshot latestSnapshot = snapshotManager.latestSnapshot(branch);
if (startTimestamp > latestSnapshot.timeMillis()
|| endTimestamp < earliestSnapshot.timeMillis()) {
return new NoSnapshot();
}
Long startSnapshotId =
(startingSnapshotId == null) ? earliestSnapshot.id() - 1 : startingSnapshotId;
- Snapshot endSnapshot = snapshotManager.earlierOrEqualTimeMills(endTimestamp);
+ Snapshot endSnapshot = snapshotManager.earlierOrEqualTimeMills(branch, endTimestamp);
Long endSnapshotId = (endSnapshot == null) ? latestSnapshot.id() : endSnapshot.id();
IncrementalStartingScanner incrementalStartingScanner =
- new IncrementalStartingScanner(
- snapshotManager, startSnapshotId, endSnapshotId, scanMode);
+ (IncrementalStartingScanner)
+ new IncrementalStartingScanner(
+ snapshotManager, startSnapshotId, endSnapshotId, scanMode)
+ .withBranch(branch);
return incrementalStartingScanner.scan(reader);
}
}
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/StaticFromSnapshotStartingScanner.java b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/StaticFromSnapshotStartingScanner.java
index 97144e8f84d24..3983925d7e4d0 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/StaticFromSnapshotStartingScanner.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/StaticFromSnapshotStartingScanner.java
@@ -41,7 +41,7 @@ public ScanMode startingScanMode() {
@Override
public Result scan(SnapshotReader snapshotReader) {
if (snapshotManager.earliestSnapshotId() == null
- || startingSnapshotId < snapshotManager.earliestSnapshotId()) {
+ || startingSnapshotId < snapshotManager.earliestSnapshotId(branch)) {
return new NoSnapshot();
}
return StartingScanner.fromPlan(
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/StaticFromTagStartingScanner.java b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/StaticFromTagStartingScanner.java
index 3850f41a8a03e..645a9444f8960 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/StaticFromTagStartingScanner.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/StaticFromTagStartingScanner.java
@@ -43,7 +43,7 @@ public ScanMode startingScanMode() {
public Result scan(SnapshotReader snapshotReader) {
TagManager tagManager =
new TagManager(snapshotManager.fileIO(), snapshotManager.tablePath());
- Snapshot snapshot = tagManager.taggedSnapshot(tagName);
+ Snapshot snapshot = tagManager.taggedSnapshot(branch, tagName);
return StartingScanner.fromPlan(
snapshotReader.withMode(ScanMode.ALL).withSnapshot(snapshot).read());
}
diff --git a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/StaticFromTimestampStartingScanner.java b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/StaticFromTimestampStartingScanner.java
index 1c039c0c7b8d9..e917095e9a481 100644
--- a/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/StaticFromTimestampStartingScanner.java
+++ b/paimon-core/src/main/java/org/apache/paimon/table/source/snapshot/StaticFromTimestampStartingScanner.java
@@ -42,14 +42,14 @@ public class StaticFromTimestampStartingScanner extends AbstractStartingScanner
public StaticFromTimestampStartingScanner(SnapshotManager snapshotManager, long startupMillis) {
super(snapshotManager);
this.startupMillis = startupMillis;
- Snapshot snapshot = timeTravelToTimestamp(snapshotManager, startupMillis);
- if (snapshot != null) {
- this.startingSnapshotId = snapshot.id();
- }
}
@Override
public Result scan(SnapshotReader snapshotReader) {
+ Snapshot snapshot = timeTravelToTimestamp(snapshotManager, startupMillis, branch);
+ if (snapshot != null) {
+ this.startingSnapshotId = snapshot.id();
+ }
if (startingSnapshotId == null) {
LOG.debug(
"There is currently no snapshot earlier than or equal to timestamp[{}]",
@@ -64,4 +64,10 @@ public Result scan(SnapshotReader snapshotReader) {
public static Snapshot timeTravelToTimestamp(SnapshotManager snapshotManager, long timestamp) {
return snapshotManager.earlierOrEqualTimeMills(timestamp);
}
+
+ @Nullable
+ public static Snapshot timeTravelToTimestamp(
+ SnapshotManager snapshotManager, long timestamp, String branch) {
+ return snapshotManager.earlierOrEqualTimeMills(branch, timestamp);
+ }
}
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..bbd28cb93230d 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
@@ -78,29 +78,24 @@ public Path snapshotDirectory() {
return new Path(tablePath + "/snapshot");
}
- 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 snapshotDirectory(String branchName) {
+ return branchName.equals(DEFAULT_MAIN_BRANCH)
+ ? snapshotDirectory()
+ : 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 snapshotPath(long snapshotId) {
+ return new Path(tablePath + "/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);
- }
-
- public Path snapshotDirByBranch(String branchName) {
- return branchName.equals(DEFAULT_MAIN_BRANCH)
- ? snapshotDirectory()
- : branchSnapshotDirectory(branchName);
+ : new Path(
+ getBranchPath(tablePath, branchName)
+ + "/snapshot/"
+ + SNAPSHOT_PREFIX
+ + snapshotId);
}
public Snapshot snapshot(long snapshotId) {
@@ -108,12 +103,16 @@ public Snapshot snapshot(long snapshotId) {
}
public Snapshot snapshot(String branchName, long snapshotId) {
- Path snapshotPath = snapshotPathByBranch(branchName, snapshotId);
+ Path snapshotPath = snapshotPath(branchName, snapshotId);
return Snapshot.fromPath(fileIO, snapshotPath);
}
public boolean snapshotExists(long snapshotId) {
- Path path = snapshotPath(snapshotId);
+ return snapshotExists(DEFAULT_MAIN_BRANCH, snapshotId);
+ }
+
+ public boolean snapshotExists(String branchName, long snapshotId) {
+ Path path = snapshotPath(branchName, snapshotId);
try {
return fileIO.exists(path);
} catch (IOException e) {
@@ -149,6 +148,11 @@ public boolean snapshotExists(long snapshotId) {
return snapshotId == null ? null : snapshot(snapshotId);
}
+ public @Nullable Snapshot earliestSnapshot(String branchName) {
+ Long snapshotId = earliestSnapshotId(branchName);
+ return snapshotId == null ? null : snapshot(snapshotId);
+ }
+
public @Nullable Long earliestSnapshotId() {
return earliestSnapshotId(DEFAULT_MAIN_BRANCH);
}
@@ -162,15 +166,19 @@ public boolean snapshotExists(long snapshotId) {
}
public @Nullable Long pickOrLatest(Predicate predicate) {
- Long latestId = latestSnapshotId();
- Long earliestId = earliestSnapshotId();
+ return pickOrLatest(DEFAULT_MAIN_BRANCH, predicate);
+ }
+
+ public @Nullable Long pickOrLatest(String branchName, Predicate predicate) {
+ Long latestId = latestSnapshotId(branchName);
+ Long earliestId = earliestSnapshotId(branchName);
if (latestId == null || earliestId == null) {
return null;
}
for (long snapshotId = latestId; snapshotId >= earliestId; snapshotId--) {
if (snapshotExists(snapshotId)) {
- Snapshot snapshot = snapshot(snapshotId);
+ Snapshot snapshot = snapshot(branchName, snapshotId);
if (predicate.test(snapshot)) {
return snapshot.id();
}
@@ -180,13 +188,17 @@ public boolean snapshotExists(long snapshotId) {
return latestId;
}
+ public @Nullable Long earlierThanTimeMills(long timestampMills) {
+ return earlierThanTimeMills(DEFAULT_MAIN_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 branchName, long timestampMills) {
+ Long earliest = earliestSnapshotId(branchName);
+ Long latest = latestSnapshotId(branchName);
if (earliest == null || latest == null) {
return null;
@@ -207,13 +219,17 @@ public boolean snapshotExists(long snapshotId) {
return earliest;
}
+ public @Nullable Snapshot earlierOrEqualTimeMills(long timestampMills) {
+ return earlierOrEqualTimeMills(DEFAULT_MAIN_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 branchName, long timestampMills) {
+ Long earliest = earliestSnapshotId(branchName);
+ Long latest = latestSnapshotId(branchName);
if (earliest == null || latest == null) {
return null;
}
@@ -243,6 +259,10 @@ public long snapshotCount() throws IOException {
return listVersionedFiles(fileIO, snapshotDirectory(), SNAPSHOT_PREFIX).count();
}
+ 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)
.map(this::snapshot)
@@ -299,18 +319,22 @@ private Predicate nonSnapshotFileFilter() {
}
public Optional latestSnapshotOfUser(String user) {
- Long latestId = latestSnapshotId();
+ return latestSnapshotOfUser(DEFAULT_MAIN_BRANCH, user);
+ }
+
+ public Optional latestSnapshotOfUser(String branchName, String user) {
+ Long latestId = latestSnapshotId(branchName);
if (latestId == null) {
return Optional.empty();
}
long earliestId =
Preconditions.checkNotNull(
- earliestSnapshotId(),
+ earliestSnapshotId(branchName),
"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(branchName, id);
if (user.equals(snapshot.commitUser())) {
return Optional.of(snapshot);
}
@@ -318,13 +342,18 @@ 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, DEFAULT_MAIN_BRANCH);
+ }
+
+ /** Find the snapshot of the specified identifiers written by the specified user. */
+ public List findSnapshotsForIdentifiers(
+ @Nonnull String user, List identifiers, String branchName) {
if (identifiers.isEmpty()) {
return Collections.emptyList();
}
- Long latestId = latestSnapshotId();
+ Long latestId = latestSnapshotId(branchName);
if (latestId == null) {
return Collections.emptyList();
}
@@ -338,7 +367,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(branchName, id);
if (user.equals(snapshot.commitUser())) {
if (remainingIdentifiers.remove(snapshot.commitIdentifier())) {
matchedSnapshots.add(snapshot);
@@ -394,7 +423,7 @@ public Snapshot traversalSnapshotsFromLatestSafely(Filter checker) {
}
private @Nullable Long findLatest(String branchName) throws IOException {
- Path snapshotDir = snapshotDirByBranch(branchName);
+ Path snapshotDir = snapshotDirectory(branchName);
if (!fileIO.exists(snapshotDir)) {
return null;
}
@@ -412,7 +441,7 @@ public Snapshot traversalSnapshotsFromLatestSafely(Filter checker) {
}
private @Nullable Long findEarliest(String branchName) throws IOException {
- Path snapshotDir = snapshotDirByBranch(branchName);
+ Path snapshotDir = snapshotDirectory(branchName);
if (!fileIO.exists(snapshotDir)) {
return null;
}
@@ -431,7 +460,7 @@ public Long readHint(String fileName) {
}
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) {
@@ -451,7 +480,7 @@ public Long readHint(String fileName, String branchName) {
private Long findByListFiles(BinaryOperator reducer, String branchName)
throws IOException {
- Path snapshotDir = snapshotDirByBranch(branchName);
+ Path snapshotDir = snapshotDirectory(branchName);
return listVersionedFiles(fileIO, snapshotDir, SNAPSHOT_PREFIX)
.reduce(reducer)
.orElse(null);
@@ -475,7 +504,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..4a789c49af5c5 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;
@@ -64,22 +65,36 @@ public Path tagDirectory() {
return new Path(tablePath + "/tag");
}
+ /** Return the root Directory of tags. */
+ public Path tagDirectory(String branchName) {
+ return branchName.equals(DEFAULT_MAIN_BRANCH)
+ ? tagDirectory()
+ : new Path(getBranchPath(tablePath, branchName) + "/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 branchName, String tagName) {
+ return branchName.equals(DEFAULT_MAIN_BRANCH)
+ ? tagPath(tagName)
+ : new Path(getBranchPath(tablePath, branchName) + "/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, DEFAULT_MAIN_BRANCH);
+ }
+
+ /** Create a tag from given snapshot and save it in the storage. */
+ public void createTag(
+ Snapshot snapshot, String tagName, List callbacks, String branchName) {
checkArgument(!StringUtils.isBlank(tagName), "Tag name '%s' is blank.", tagName);
- checkArgument(!tagExists(tagName), "Tag name '%s' already exists.", tagName);
+ checkArgument(!tagExists(branchName, tagName), "Tag name '%s' already exists.", tagName);
- Path newTagPath = tagPath(tagName);
+ Path newTagPath = tagPath(branchName, tagName);
try {
fileIO.writeFileUtf8(newTagPath, snapshot.toJson());
} catch (IOException e) {
@@ -102,8 +117,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 branchName) {
+ Snapshot taggedSnapshot = taggedSnapshot(branchName, tagNames.get(0));
List taggedSnapshots;
// skip file deletion if snapshot exists
@@ -112,29 +130,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(branchName);
+ tagNames.forEach(tagName -> fileIO.deleteQuietly(tagPath(branchName, tagName)));
}
- doClean(taggedSnapshot, taggedSnapshots, snapshotManager, tagDeletion);
+ doClean(taggedSnapshot, taggedSnapshots, snapshotManager, tagDeletion, branchName);
}
public void deleteTag(
String tagName, TagDeletion tagDeletion, SnapshotManager snapshotManager) {
+ deleteTag(tagName, tagDeletion, snapshotManager, DEFAULT_MAIN_BRANCH);
+ }
+
+ public void deleteTag(
+ String tagName,
+ TagDeletion tagDeletion,
+ SnapshotManager snapshotManager,
+ String branchName) {
checkArgument(!StringUtils.isBlank(tagName), "Tag name '%s' is blank.", tagName);
- checkArgument(tagExists(tagName), "Tag '%s' doesn't exist.", tagName);
+ checkArgument(tagExists(branchName, tagName), "Tag '%s' doesn't exist.", tagName);
- Snapshot taggedSnapshot = taggedSnapshot(tagName);
+ Snapshot taggedSnapshot = taggedSnapshot(branchName, tagName);
List taggedSnapshots;
// skip file deletion if snapshot exists
if (snapshotManager.snapshotExists(taggedSnapshot.id())) {
- fileIO.deleteQuietly(tagPath(tagName));
+ fileIO.deleteQuietly(tagPath(branchName, 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(branchName, tagName));
// skip data file clean if more than 1 tags are created based on this snapshot
if (tags.get(taggedSnapshot).size() > 1) {
@@ -143,14 +169,15 @@ public void deleteTag(
taggedSnapshots = new ArrayList<>(tags.keySet());
}
- doClean(taggedSnapshot, taggedSnapshots, snapshotManager, tagDeletion);
+ doClean(taggedSnapshot, taggedSnapshots, snapshotManager, tagDeletion, branchName);
}
private void doClean(
Snapshot taggedSnapshot,
List taggedSnapshots,
SnapshotManager snapshotManager,
- TagDeletion tagDeletion) {
+ TagDeletion tagDeletion,
+ String branchName) {
// 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 +188,7 @@ private void doClean(
skippedSnapshots.add(taggedSnapshots.get(index - 1));
}
// the nearest right neighbor
- Snapshot right = snapshotManager.earliestSnapshot();
+ Snapshot right = snapshotManager.earliestSnapshot(branchName);
if (index + 1 < taggedSnapshots.size()) {
Snapshot rightTag = taggedSnapshots.get(index + 1);
right = right.id() < rightTag.id() ? right : rightTag;
@@ -191,9 +218,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 branchName, String tagName) {
+ Path path = tagPath(branchName, tagName);
try {
return fileIO.exists(path);
} catch (IOException e) {
@@ -204,10 +231,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 branchName, String tagName) {
+ checkArgument(tagExists(branchName, tagName), "Tag '%s' doesn't exist.", tagName);
+ return Snapshot.fromPath(fileIO, tagPath(branchName, tagName));
+ }
+
+ public long tagCount(String branchName) {
+ try {
+ return listVersionedFileStatus(fileIO, tagDirectory(branchName), TAG_PREFIX).count();
+ } catch (IOException e) {
+ throw new RuntimeException(e);
+ }
}
public long tagCount() {
@@ -223,11 +268,25 @@ public List taggedSnapshots() {
return new ArrayList<>(tags().keySet());
}
+ /** Get all tagged snapshots sorted by snapshot id. */
+ public List taggedSnapshots(String branchName) {
+ return new ArrayList<>(tags(branchName).keySet());
+ }
+
+ /** Get all tagged snapshots with names sorted by snapshot id. */
+ public SortedMap> tags(String branchName) {
+ return tags(branchName, 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 +298,12 @@ public SortedMap> tags() {
* name.
* @throws RuntimeException if an IOException occurs during retrieval of snapshots.
*/
- public SortedMap> tags(Predicate filter) {
+ public SortedMap> tags(String branchName, Predicate filter) {
TreeMap> tags =
new TreeMap<>(Comparator.comparingLong(Snapshot::id));
try {
List paths =
- listVersionedFileStatus(fileIO, tagDirectory(), TAG_PREFIX)
+ listVersionedFileStatus(fileIO, tagDirectory(branchName), TAG_PREFIX)
.map(FileStatus::getPath)
.collect(Collectors.toList());
@@ -269,11 +328,15 @@ public SortedMap> tags(Predicate filter) {
}
public List sortTagsOfOneSnapshot(List tagNames) {
+ return sortTagsOfOneSnapshot(DEFAULT_MAIN_BRANCH, tagNames);
+ }
+
+ public List sortTagsOfOneSnapshot(String branchName, List tagNames) {
return tagNames.stream()
.map(
name -> {
try {
- return fileIO.getFileStatus(tagPath(name));
+ return fileIO.getFileStatus(tagPath(branchName, 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 d9cc7f87866ac..0762f3d1a35d2 100644
--- a/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java
+++ b/paimon-core/src/test/java/org/apache/paimon/TestFileStore.java
@@ -51,10 +51,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;
@@ -96,6 +98,8 @@ public class TestFileStore extends KeyValueFileStore {
private long commitIdentifier;
+ private String branch;
+
private TestFileStore(
String root,
CoreOptions options,
@@ -103,7 +107,8 @@ private TestFileStore(
RowType keyType,
RowType valueType,
KeyValueFieldsExtractor keyValueFieldsExtractor,
- MergeFunctionFactory mfFactory) {
+ MergeFunctionFactory mfFactory,
+ String branch) {
super(
FileIOFinder.find(new Path(root)),
new SchemaManager(FileIOFinder.find(new Path(root)), options.path()),
@@ -125,14 +130,15 @@ private TestFileStore(
this.commitUser = UUID.randomUUID().toString();
this.commitIdentifier = 0L;
+ this.branch = branch;
}
public AbstractFileStoreWrite newWrite() {
- return super.newWrite(commitUser);
+ return super.newWrite(commitUser, branch);
}
public FileStoreCommitImpl newCommit() {
- return super.newCommit(commitUser);
+ return super.newCommit(commitUser, branch);
}
public ExpireSnapshots newExpire(int numRetainedMin, int numRetainedMax, long millisRetained) {
@@ -285,7 +291,7 @@ public List commitDataImpl(
.write(kv);
}
- FileStoreCommit commit = newCommit(commitUser);
+ FileStoreCommit commit = newCommit(commitUser, branch);
ManifestCommittable committable =
new ManifestCommittable(
identifier == null ? commitIdentifier++ : identifier, watermark);
@@ -306,12 +312,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;
}
@@ -332,13 +338,13 @@ public List commitDataImpl(
List snapshots = new ArrayList<>();
for (long id = snapshotIdBeforeCommit + 1; id <= snapshotIdAfterCommit; id++) {
- snapshots.add(snapshotManager.snapshot(id));
+ snapshots.add(snapshotManager.snapshot(branch, id));
}
return snapshots;
}
public List readKvsFromSnapshot(long snapshotId) throws Exception {
- List entries = newScan().withSnapshot(snapshotId).plan().files();
+ List entries = newScan(branch).withSnapshot(snapshotId).plan().files();
return readKvsFromManifestEntries(entries, false);
}
@@ -380,7 +386,7 @@ public List readKvsFromManifestEntries(
}
List kvs = new ArrayList<>();
- FileStoreRead read = newRead();
+ FileStoreRead read = newRead(branch);
for (Map.Entry>> entryWithPartition :
filesPerPartitionAndBucket.entrySet()) {
for (Map.Entry> entryWithBucket :
@@ -451,18 +457,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);
@@ -481,7 +487,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();
@@ -565,6 +573,7 @@ public static class Builder {
private final MergeFunctionFactory mfFactory;
private CoreOptions.ChangelogProducer changelogProducer;
+ private final String branch;
public Builder(
String format,
@@ -575,6 +584,28 @@ public Builder(
RowType valueType,
KeyValueFieldsExtractor keyValueFieldsExtractor,
MergeFunctionFactory mfFactory) {
+ this(
+ format,
+ root,
+ numBuckets,
+ partitionType,
+ keyType,
+ valueType,
+ keyValueFieldsExtractor,
+ mfFactory,
+ BranchManager.DEFAULT_MAIN_BRANCH);
+ }
+
+ public Builder(
+ String format,
+ String root,
+ int numBuckets,
+ RowType partitionType,
+ RowType keyType,
+ RowType valueType,
+ KeyValueFieldsExtractor keyValueFieldsExtractor,
+ MergeFunctionFactory mfFactory,
+ String branch) {
this.format = format;
this.root = root;
this.numBuckets = numBuckets;
@@ -585,6 +616,7 @@ public Builder(
this.mfFactory = mfFactory;
this.changelogProducer = CoreOptions.ChangelogProducer.NONE;
+ this.branch = StringUtils.isEmpty(branch) ? BranchManager.DEFAULT_MAIN_BRANCH : branch;
}
public Builder changelogProducer(CoreOptions.ChangelogProducer changelogProducer) {
@@ -620,7 +652,8 @@ public TestFileStore build() {
keyType,
valueType,
keyValueFieldsExtractor,
- mfFactory);
+ mfFactory,
+ branch);
}
}
}
diff --git a/paimon-core/src/test/java/org/apache/paimon/operation/AppendOnlyFileStoreWriteTest.java b/paimon-core/src/test/java/org/apache/paimon/operation/AppendOnlyFileStoreWriteTest.java
deleted file mode 100644
index 9281a7b35a9ce..0000000000000
--- a/paimon-core/src/test/java/org/apache/paimon/operation/AppendOnlyFileStoreWriteTest.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.paimon.operation;
-
-import org.apache.paimon.append.AppendOnlyWriter;
-import org.apache.paimon.catalog.Catalog;
-import org.apache.paimon.catalog.FileSystemCatalog;
-import org.apache.paimon.catalog.Identifier;
-import org.apache.paimon.data.BinaryRow;
-import org.apache.paimon.data.BinaryRowWriter;
-import org.apache.paimon.data.GenericRow;
-import org.apache.paimon.data.InternalRow;
-import org.apache.paimon.disk.ExternalBuffer;
-import org.apache.paimon.fs.Path;
-import org.apache.paimon.fs.local.LocalFileIO;
-import org.apache.paimon.io.DataFileMeta;
-import org.apache.paimon.schema.Schema;
-import org.apache.paimon.table.FileStoreTable;
-import org.apache.paimon.table.sink.CommitMessage;
-import org.apache.paimon.table.sink.CommitMessageImpl;
-import org.apache.paimon.types.DataTypes;
-
-import org.assertj.core.api.Assertions;
-import org.junit.jupiter.api.Test;
-import org.junit.jupiter.api.io.TempDir;
-
-import java.util.List;
-import java.util.Map;
-
-/** Tests for {@link AppendOnlyFileStoreWrite}. */
-public class AppendOnlyFileStoreWriteTest {
-
- @TempDir java.nio.file.Path tempDir;
-
- @Test
- public void testWritesInBatch() throws Exception {
- FileStoreTable table = createFileStoreTable();
-
- AppendOnlyFileStoreWrite write = (AppendOnlyFileStoreWrite) table.store().newWrite("ss");
- write.withExecutionMode(false);
-
- write.write(partition(0), 0, GenericRow.of(0, 0, 0));
- write.write(partition(1), 1, GenericRow.of(1, 1, 0));
- write.write(partition(2), 2, GenericRow.of(2, 2, 0));
- write.write(partition(3), 3, GenericRow.of(3, 3, 0));
- write.write(partition(4), 4, GenericRow.of(4, 4, 0));
-
- for (Map> bucketWriters :
- write.writers().values()) {
- for (AbstractFileStoreWrite.WriterContainer writerContainer :
- bucketWriters.values()) {
- Assertions.assertThat(((AppendOnlyWriter) writerContainer.writer).getWriteBuffer())
- .isEqualTo(null);
- }
- }
-
- write.write(partition(5), 5, GenericRow.of(5, 5, 0));
- for (Map> bucketWriters :
- write.writers().values()) {
- for (AbstractFileStoreWrite.WriterContainer writerContainer :
- bucketWriters.values()) {
- Assertions.assertThat(((AppendOnlyWriter) writerContainer.writer).getWriteBuffer())
- .isInstanceOf(ExternalBuffer.class);
- }
- }
-
- write.write(partition(6), 6, GenericRow.of(6, 6, 0));
- write.write(partition(0), 0, GenericRow.of(0, 0, 0));
- write.write(partition(1), 1, GenericRow.of(1, 1, 0));
- write.write(partition(2), 2, GenericRow.of(2, 2, 0));
- write.write(partition(3), 3, GenericRow.of(3, 3, 0));
- List commit = write.prepareCommit(true, Long.MAX_VALUE);
-
- Assertions.assertThat(commit.size()).isEqualTo(7);
-
- long records =
- commit.stream()
- .map(s -> (CommitMessageImpl) s)
- .mapToLong(
- s ->
- s.newFilesIncrement().newFiles().stream()
- .mapToLong(DataFileMeta::rowCount)
- .sum())
- .sum();
- Assertions.assertThat(records).isEqualTo(11);
- }
-
- protected FileStoreTable createFileStoreTable() throws Exception {
- Catalog catalog = new FileSystemCatalog(LocalFileIO.create(), new Path(tempDir.toString()));
- Schema schema =
- Schema.newBuilder()
- .column("f0", DataTypes.INT())
- .column("f1", DataTypes.INT())
- .column("f2", DataTypes.INT())
- .partitionKeys("f0")
- .option("bucket", "100")
- .build();
- Identifier identifier = Identifier.create("default", "test");
- catalog.createDatabase("default", false);
- catalog.createTable(identifier, schema, false);
- return (FileStoreTable) catalog.getTable(identifier);
- }
-
- private BinaryRow partition(int i) {
- BinaryRow binaryRow = new BinaryRow(1);
- BinaryRowWriter writer = new BinaryRowWriter(binaryRow);
- writer.writeInt(0, i);
- writer.complete();
- return binaryRow;
- }
-}
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..2bbe1b2590ab9
--- /dev/null
+++ b/paimon-core/src/test/java/org/apache/paimon/operation/KeyValueFileStoreReadBranchTest.java
@@ -0,0 +1,357 @@
+/*
+ * 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(branch);
+ Long snapshotId = store.snapshotManager().latestSnapshotId(branch);
+ Map> filesGroupedByPartition =
+ scan.withSnapshot(snapshotId).plan().files().stream()
+ .collect(Collectors.groupingBy(ManifestEntry::partition));
+ KeyValueFileStoreRead read = store.newRead(branch);
+ 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);
+ 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),
+ branch);
+ 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 KeyValueFileStoreReadBranchTest.TestValueCountMergeFunction();
+ }
+ }
+ }
+}
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..f9399868550b5
--- /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()));
+ schemaManager.createTable(
+ new Schema(
+ TestKeyValueGenerator.DEFAULT_ROW_TYPE.getFields(),
+ TestKeyValueGenerator.DEFAULT_PART_TYPE.getFieldNames(),
+ TestKeyValueGenerator.getPrimaryKeys(
+ TestKeyValueGenerator.GeneratorMode.MULTI_PARTITIONED),
+ Collections.emptyMap(),
+ null),
+ branch);
+ }
+
+ @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(branch);
+ 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(branch);
+ 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(branch);
+ scan.withSnapshot(snapshot.id());
+ List files = scan.plan().files();
+
+ scan = store.newScan(branch);
+ 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(branch);
+ scan.withSnapshot(snapshot.id());
+ List files = scan.plan().files();
+
+ scan = store.newScan(branch);
+ 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(branch);
+ 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(branch);
+ 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(branch, wantedSnapshotId);
+ List wantedManifests = wantedSnapshot.dataManifests(manifestList);
+
+ FileStoreScan scan = store.newScan(branch);
+ 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..e66131f1a4247 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
@@ -532,6 +532,11 @@ public TableSchema commitChanges(List changes) {
public TableSchema schema(long id) {
return checkNotNull(tableSchemas.get(id));
}
+
+ @Override
+ public TableSchema schema(String branchName, long id) {
+ return checkNotNull(tableSchemas.get(id));
+ }
}
protected List toSplits(List dataSplits) {
diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java
index ae9301362de9c..39be317cb22e2 100644
--- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java
+++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/action/cdc/SyncTableActionBase.java
@@ -29,6 +29,7 @@
import org.apache.paimon.flink.sink.cdc.RichCdcMultiplexRecordEventParser;
import org.apache.paimon.schema.Schema;
import org.apache.paimon.table.FileStoreTable;
+import org.apache.paimon.utils.BranchManager;
import org.apache.flink.api.common.functions.FlatMapFunction;
import org.apache.flink.streaming.api.datastream.DataStream;
@@ -181,6 +182,12 @@ protected void buildSink(
if (sinkParallelism != null) {
sinkBuilder.withParallelism(Integer.parseInt(sinkParallelism));
}
+ String branch = tableConfig.get(FlinkConnectorOptions.BRANCH.key());
+ if (branch != null) {
+ sinkBuilder.toBranch(branch);
+ } else {
+ sinkBuilder.toBranch(BranchManager.DEFAULT_MAIN_BRANCH);
+ }
sinkBuilder.build();
}
diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicBucketSink.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicBucketSink.java
index 574ff685f3fa4..7431f7d2aa4dc 100644
--- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicBucketSink.java
+++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicBucketSink.java
@@ -44,6 +44,6 @@ protected KeyAndBucketExtractor createExtractor(TableSchema schema) {
@Override
protected OneInputStreamOperator, Committable> createWriteOperator(
StoreSinkWrite.Provider writeProvider, String commitUser) {
- return new CdcDynamicBucketWriteOperator(table, writeProvider, commitUser);
+ return new CdcDynamicBucketWriteOperator(table, writeProvider, commitUser).toBranch(branch);
}
}
diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicBucketWriteOperator.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicBucketWriteOperator.java
index b2fbdc3e93eef..3fa71c1077dd6 100644
--- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicBucketWriteOperator.java
+++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcDynamicBucketWriteOperator.java
@@ -69,7 +69,7 @@ public void processElement(StreamRecord> element) thr
Optional optionalConverted = toGenericRow(record.f0, table.schema().fields());
if (!optionalConverted.isPresent()) {
while (true) {
- table = table.copyWithLatestSchema();
+ table = table.copyWithLatestSchema(branch);
optionalConverted = toGenericRow(record.f0, table.schema().fields());
if (optionalConverted.isPresent()) {
break;
diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcFixedBucketSink.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcFixedBucketSink.java
index 59bdb192beea7..1106201c3db13 100644
--- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcFixedBucketSink.java
+++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcFixedBucketSink.java
@@ -41,6 +41,6 @@ public CdcFixedBucketSink(FileStoreTable table) {
@Override
protected OneInputStreamOperator createWriteOperator(
StoreSinkWrite.Provider writeProvider, String commitUser) {
- return new CdcRecordStoreWriteOperator(table, writeProvider, commitUser);
+ return new CdcRecordStoreWriteOperator(table, writeProvider, commitUser).toBranch(branch);
}
}
diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperator.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperator.java
index af544def01a02..84f1708acd301 100644
--- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperator.java
+++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperator.java
@@ -149,7 +149,7 @@ public void processElement(StreamRecord element) throws Exce
if (!optionalConverted.isPresent()) {
FileStoreTable latestTable = table;
while (true) {
- latestTable = latestTable.copyWithLatestSchema();
+ latestTable = latestTable.copyWithLatestSchema(branch);
tables.put(tableId, latestTable);
optionalConverted = toGenericRow(record.record(), latestTable.schema().fields());
if (optionalConverted.isPresent()) {
diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperator.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperator.java
index dd0aa2e5622c2..62631903154dd 100644
--- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperator.java
+++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperator.java
@@ -61,7 +61,7 @@ public CdcRecordStoreWriteOperator(
@Override
public void initializeState(StateInitializationContext context) throws Exception {
- table = table.copyWithLatestSchema();
+ table = table.copyWithLatestSchema(branch);
super.initializeState(context);
}
@@ -76,7 +76,7 @@ public void processElement(StreamRecord element) throws Exception {
Optional optionalConverted = toGenericRow(record, table.schema().fields());
if (!optionalConverted.isPresent()) {
while (true) {
- table = table.copyWithLatestSchema();
+ table = table.copyWithLatestSchema(branch);
optionalConverted = toGenericRow(record, table.schema().fields());
if (optionalConverted.isPresent()) {
break;
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..9ccd8d0baceed 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
@@ -26,6 +26,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;
@@ -52,6 +53,8 @@ public class CdcSinkBuilder {
@Nullable private Integer parallelism;
+ private String branch = BranchManager.DEFAULT_MAIN_BRANCH;
+
public CdcSinkBuilder withInput(DataStream input) {
this.input = input;
return this;
@@ -82,6 +85,11 @@ public CdcSinkBuilder withCatalogLoader(Catalog.Loader catalogLoader) {
return this;
}
+ public CdcSinkBuilder toBranch(String branch) {
+ this.branch = branch;
+ return this;
+ }
+
public DataStreamSink> build() {
Preconditions.checkNotNull(input, "Input DataStream can not be null.");
Preconditions.checkNotNull(parserFactory, "Event ParserFactory can not be null.");
@@ -95,7 +103,6 @@ public DataStreamSink> build() {
}
FileStoreTable dataTable = (FileStoreTable) table;
-
SingleOutputStreamOperator parsed =
input.forward()
.process(new CdcParsingProcessFunction<>(parserFactory))
@@ -108,7 +115,8 @@ public DataStreamSink> build() {
new UpdatedDataFieldsProcessFunction(
new SchemaManager(dataTable.fileIO(), dataTable.location()),
identifier,
- catalogLoader));
+ catalogLoader,
+ branch));
schemaChangeProcessFunction.getTransformation().setParallelism(1);
schemaChangeProcessFunction.getTransformation().setMaxParallelism(1);
@@ -117,7 +125,9 @@ public DataStreamSink> build() {
case FIXED:
return buildForFixedBucket(parsed);
case DYNAMIC:
- return new CdcDynamicBucketSink((FileStoreTable) table).build(parsed, parallelism);
+ return ((CdcDynamicBucketSink)
+ (new CdcDynamicBucketSink((FileStoreTable) table).toBranch(branch)))
+ .build(parsed, parallelism);
case UNAWARE:
return buildForUnawareBucket(parsed);
default:
@@ -129,11 +139,11 @@ private DataStreamSink> buildForFixedBucket(DataStream parsed) {
FileStoreTable dataTable = (FileStoreTable) table;
DataStream partitioned =
partition(parsed, new CdcRecordChannelComputer(dataTable.schema()), parallelism);
- return new CdcFixedBucketSink(dataTable).sinkFrom(partitioned);
+ return new CdcFixedBucketSink(dataTable).toBranch(branch).sinkFrom(partitioned);
}
private DataStreamSink> buildForUnawareBucket(DataStream parsed) {
FileStoreTable dataTable = (FileStoreTable) table;
- return new CdcUnawareBucketSink(dataTable, parallelism).sinkFrom(parsed);
+ return new CdcUnawareBucketSink(dataTable, parallelism).toBranch(branch).sinkFrom(parsed);
}
}
diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcUnawareBucketSink.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcUnawareBucketSink.java
index 5cd3d6085f5a7..f34d2926d3ab7 100644
--- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcUnawareBucketSink.java
+++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/CdcUnawareBucketSink.java
@@ -35,6 +35,6 @@ public CdcUnawareBucketSink(FileStoreTable table, Integer parallelism) {
@Override
protected OneInputStreamOperator createWriteOperator(
StoreSinkWrite.Provider writeProvider, String commitUser) {
- return new CdcUnawareBucketWriteOperator(table, writeProvider, commitUser);
+ return new CdcUnawareBucketWriteOperator(table, writeProvider, commitUser).toBranch(branch);
}
}
diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java
index 99d933d89c1b6..1779ca9eb4047 100644
--- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java
+++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/FlinkCdcMultiTableSink.java
@@ -36,6 +36,7 @@
import org.apache.paimon.manifest.WrappedManifestCommittable;
import org.apache.paimon.options.MemorySize;
import org.apache.paimon.options.Options;
+import org.apache.paimon.utils.BranchManager;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.DataStreamSink;
@@ -66,13 +67,24 @@ public class FlinkCdcMultiTableSink implements Serializable {
private final double commitCpuCores;
@Nullable private final MemorySize commitHeapMemory;
+ private final String branch;
+
public FlinkCdcMultiTableSink(
Catalog.Loader catalogLoader,
double commitCpuCores,
@Nullable MemorySize commitHeapMemory) {
+ this(catalogLoader, commitCpuCores, commitHeapMemory, BranchManager.DEFAULT_MAIN_BRANCH);
+ }
+
+ public FlinkCdcMultiTableSink(
+ Catalog.Loader catalogLoader,
+ double commitCpuCores,
+ @Nullable MemorySize commitHeapMemory,
+ String branch) {
this.catalogLoader = catalogLoader;
this.commitCpuCores = commitCpuCores;
this.commitHeapMemory = commitHeapMemory;
+ this.branch = branch;
}
private StoreSinkWrite.WithWriteBufferProvider createWriteProvider() {
@@ -87,7 +99,8 @@ private StoreSinkWrite.WithWriteBufferProvider createWriteProvider() {
false,
true,
memoryPoolFactory,
- metricGroup);
+ metricGroup,
+ branch);
}
public DataStreamSink> sinkFrom(DataStream input) {
@@ -143,7 +156,8 @@ public DataStreamSink> sinkFrom(
protected OneInputStreamOperator createWriteOperator(
StoreSinkWrite.WithWriteBufferProvider writeProvider, String commitUser) {
return new CdcRecordStoreMultiWriteOperator(
- catalogLoader, writeProvider, commitUser, new Options());
+ catalogLoader, writeProvider, commitUser, new Options())
+ .toBranch(branch);
}
// Table committers are dynamically created at runtime
@@ -153,7 +167,8 @@ protected OneInputStreamOperator crea
// commit new files list even if they're empty.
// Otherwise we can't tell if the commit is successful after
// a restart.
- return (user, metricGroup) -> new StoreMultiCommitter(catalogLoader, user, metricGroup);
+ return (user, metricGroup) ->
+ new StoreMultiCommitter(catalogLoader, user, metricGroup, branch);
}
protected CommittableStateManager createCommittableStateManager() {
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..8a0df92965249 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
@@ -76,6 +76,8 @@ public class FlinkCdcSyncDatabaseSinkBuilder {
private String database;
private MultiTablesSinkMode mode;
+ private String branch;
+
public FlinkCdcSyncDatabaseSinkBuilder withInput(DataStream input) {
this.input = input;
return this;
@@ -99,7 +101,7 @@ public FlinkCdcSyncDatabaseSinkBuilder withTableOptions(Map o
public FlinkCdcSyncDatabaseSinkBuilder withTableOptions(Options options) {
this.parallelism = options.get(FlinkConnectorOptions.SINK_PARALLELISM);
this.committerCpu = options.get(FlinkConnectorOptions.SINK_COMMITTER_CPU);
- this.committerMemory = options.get(FlinkConnectorOptions.SINK_COMMITTER_MEMORY);
+ this.branch = options.get(FlinkConnectorOptions.BRANCH);
return this;
}
@@ -150,7 +152,7 @@ private void buildCombinedCdcSink() {
parsed,
CdcDynamicTableParsingProcessFunction.DYNAMIC_SCHEMA_CHANGE_OUTPUT_TAG)
.keyBy(t -> t.f0)
- .process(new MultiTableUpdatedDataFieldsProcessFunction(catalogLoader))
+ .process(new MultiTableUpdatedDataFieldsProcessFunction(catalogLoader, branch))
.name("Schema Evolution");
DataStream partitioned =
@@ -160,18 +162,18 @@ private void buildCombinedCdcSink() {
parallelism);
FlinkCdcMultiTableSink sink =
- new FlinkCdcMultiTableSink(catalogLoader, committerCpu, committerMemory);
+ new FlinkCdcMultiTableSink(catalogLoader, committerCpu, committerMemory, branch);
sink.sinkFrom(partitioned);
}
private void buildForFixedBucket(FileStoreTable table, DataStream