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 parsed) { DataStream partitioned = partition(parsed, new CdcRecordChannelComputer(table.schema()), parallelism); - new CdcFixedBucketSink(table).sinkFrom(partitioned); + new CdcFixedBucketSink(table).toBranch(branch).sinkFrom(partitioned); } private void buildForUnawareBucket(FileStoreTable table, DataStream parsed) { - new CdcUnawareBucketSink(table, parallelism).sinkFrom(parsed); + new CdcUnawareBucketSink(table, parallelism).toBranch(branch).sinkFrom(parsed); } private void buildDividedCdcSink() { @@ -192,7 +194,8 @@ private void buildDividedCdcSink() { new UpdatedDataFieldsProcessFunction( new SchemaManager(table.fileIO(), table.location()), Identifier.create(database, table.name()), - catalogLoader)); + catalogLoader, + branch)); schemaChangeProcessFunction.getTransformation().setParallelism(1); schemaChangeProcessFunction.getTransformation().setMaxParallelism(1); @@ -208,7 +211,8 @@ private void buildDividedCdcSink() { buildForFixedBucket(table, parsedForTable); break; case DYNAMIC: - new CdcDynamicBucketSink(table).build(parsedForTable, parallelism); + ((CdcDynamicBucketSink) new CdcDynamicBucketSink(table).toBranch(branch)) + .build(parsedForTable, parallelism); break; case UNAWARE: buildForUnawareBucket(table, parsedForTable); diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/MultiTableUpdatedDataFieldsProcessFunction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/MultiTableUpdatedDataFieldsProcessFunction.java index 0ad412e47d349..e46d9ac777181 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/MultiTableUpdatedDataFieldsProcessFunction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/MultiTableUpdatedDataFieldsProcessFunction.java @@ -51,8 +51,8 @@ public class MultiTableUpdatedDataFieldsProcessFunction private final Map schemaManagers = new HashMap<>(); - public MultiTableUpdatedDataFieldsProcessFunction(Catalog.Loader catalogLoader) { - super(catalogLoader); + public MultiTableUpdatedDataFieldsProcessFunction(Catalog.Loader catalogLoader, String branch) { + super(catalogLoader, branch); } @Override diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunction.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunction.java index 4a33eb1b7ec94..b8fe24afbe227 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunction.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunction.java @@ -44,8 +44,11 @@ public class UpdatedDataFieldsProcessFunction private final Identifier identifier; public UpdatedDataFieldsProcessFunction( - SchemaManager schemaManager, Identifier identifier, Catalog.Loader catalogLoader) { - super(catalogLoader); + SchemaManager schemaManager, + Identifier identifier, + Catalog.Loader catalogLoader, + String branch) { + super(catalogLoader, branch); this.schemaManager = schemaManager; this.identifier = identifier; } diff --git a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java index 3c1d6e15b3695..e2a916a1477ba 100644 --- a/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java +++ b/paimon-flink/paimon-flink-cdc/src/main/java/org/apache/paimon/flink/sink/cdc/UpdatedDataFieldsProcessFunctionBase.java @@ -28,7 +28,9 @@ import org.apache.paimon.types.DataTypeChecks; import org.apache.paimon.types.DataTypeRoot; import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.BranchManager; import org.apache.paimon.utils.Preconditions; +import org.apache.paimon.utils.StringUtils; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.functions.ProcessFunction; @@ -64,8 +66,11 @@ public abstract class UpdatedDataFieldsProcessFunctionBase extends Process private static final List DECIMAL_TYPES = Arrays.asList(DataTypeRoot.DECIMAL); - protected UpdatedDataFieldsProcessFunctionBase(Catalog.Loader catalogLoader) { + private final String branch; + + protected UpdatedDataFieldsProcessFunctionBase(Catalog.Loader catalogLoader, String branch) { this.catalogLoader = catalogLoader; + this.branch = StringUtils.isEmpty(branch) ? BranchManager.DEFAULT_MAIN_BRANCH : branch; } @Override @@ -96,7 +101,7 @@ protected void applySchemaChange( (SchemaChange.UpdateColumnType) schemaChange; TableSchema schema = schemaManager - .latest() + .latest(branch) .orElseThrow( () -> new RuntimeException( @@ -180,7 +185,7 @@ public static ConvertAction canConvert(DataType oldType, DataType newType) { protected List extractSchemaChanges( SchemaManager schemaManager, List updatedDataFields) { - RowType oldRowType = schemaManager.latest().get().logicalRowType(); + RowType oldRowType = schemaManager.latest(branch).get().logicalRowType(); Map oldFields = new HashMap<>(); for (DataField oldField : oldRowType.getFields()) { oldFields.put(oldField.name(), oldField); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java index 7a621030217a6..bcd90583bea94 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/CdcActionITCaseBase.java @@ -18,6 +18,7 @@ package org.apache.paimon.flink.action.cdc; +import org.apache.paimon.flink.FlinkConnectorOptions; import org.apache.paimon.flink.action.ActionBase; import org.apache.paimon.flink.action.ActionITCaseBase; import org.apache.paimon.flink.action.cdc.kafka.KafkaSyncDatabaseActionFactory; @@ -54,6 +55,7 @@ import java.util.Map; import java.util.concurrent.ThreadLocalRandom; +import static org.apache.paimon.options.CatalogOptions.BRANCH; import static org.assertj.core.api.Assertions.assertThat; /** CDC IT case base. */ @@ -130,7 +132,7 @@ protected void waitForResult( break; } } - table = table.copyWithLatestSchema(); + table = table.copyWithLatestSchema(branch); Thread.sleep(1000); } @@ -138,7 +140,7 @@ protected void waitForResult( List sortedExpected = new ArrayList<>(expected); Collections.sort(sortedExpected); while (true) { - ReadBuilder readBuilder = table.newReadBuilder(); + ReadBuilder readBuilder = table.newReadBuilder().fromBranch(branch); TableScan.Plan plan = readBuilder.newScan().plan(); List result = getResult( @@ -154,6 +156,16 @@ protected void waitForResult( } } + protected Map getCatalogOptions(Map catalogOptions) { + catalogOptions.put(BRANCH.key(), branch); + return catalogOptions; + } + + protected Map getTableConfig(Map tableConfig) { + tableConfig.put(FlinkConnectorOptions.BRANCH.key(), branch); + return tableConfig; + } + protected Map getBasicTableConfig() { Map config = new HashMap<>(); ThreadLocalRandom random = ThreadLocalRandom.current(); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncDatabaseToBranchActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncDatabaseToBranchActionITCase.java new file mode 100644 index 0000000000000..53246624744f2 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncDatabaseToBranchActionITCase.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.action.cdc.kafka; + +import org.junit.jupiter.api.BeforeEach; + +import java.io.IOException; + +/** IT cases for {@link KafkaSyncDatabaseAction}. */ +public class KafkaDebeziumSyncDatabaseToBranchActionITCase + extends KafkaDebeziumSyncDatabaseActionITCase { + @BeforeEach + public void before() throws IOException { + branch = "testKafkaDebeziumSyncDatabaseBranch"; + super.before(); + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncTableToBranchActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncTableToBranchActionITCase.java new file mode 100644 index 0000000000000..77860bbb37e8b --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaDebeziumSyncTableToBranchActionITCase.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.action.cdc.kafka; + +import org.junit.jupiter.api.BeforeEach; + +import java.io.IOException; + +/** IT cases for {@link KafkaSyncTableAction}. */ +public class KafkaDebeziumSyncTableToBranchActionITCase extends KafkaDebeziumSyncTableActionITCase { + + @BeforeEach + public void before() throws IOException { + branch = "testKafkaDebeziumSyncTableBranch"; + super.before(); + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java index cdad175ea15fa..cf474ea60f318 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncDatabaseActionITCase.java @@ -28,6 +28,7 @@ import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.UUID; @@ -69,7 +70,8 @@ protected void testSchemaEvolutionMultiTopic(String format) throws Exception { kafkaConfig.put(TOPIC.key(), String.join(";", topics)); KafkaSyncDatabaseAction action = syncDatabaseActionBuilder(kafkaConfig) - .withTableConfig(getBasicTableConfig()) + .withTableConfig(getTableConfig(getBasicTableConfig())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) .build(); runActionWithDefaultEnv(action); @@ -103,7 +105,8 @@ protected void testSchemaEvolutionOneTopic(String format) throws Exception { kafkaConfig.put(TOPIC.key(), String.join(";", topics)); KafkaSyncDatabaseAction action = syncDatabaseActionBuilder(kafkaConfig) - .withTableConfig(getBasicTableConfig()) + .withTableConfig(getTableConfig(getBasicTableConfig())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) .build(); runActionWithDefaultEnv(action); @@ -112,6 +115,7 @@ protected void testSchemaEvolutionOneTopic(String format) throws Exception { private void testSchemaEvolutionImpl( List topics, boolean writeOne, int fileCount, String format) throws Exception { + waitingTables("t1", "t2"); FileStoreTable table1 = getFileStoreTable("t1"); @@ -212,7 +216,11 @@ protected void testTopicIsEmpty(String format) { Map kafkaConfig = getBasicKafkaConfig(); kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); - KafkaSyncDatabaseAction action = syncDatabaseActionBuilder(kafkaConfig).build(); + KafkaSyncDatabaseAction action = + syncDatabaseActionBuilder(kafkaConfig) + .withTableConfig(getTableConfig(new HashMap<>())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) + .build(); assertThatThrownBy(action::run) .satisfies( @@ -267,7 +275,8 @@ protected void testTableAffixMultiTopic(String format) throws Exception { syncDatabaseActionBuilder(kafkaConfig) .withTablePrefix("test_prefix_") .withTableSuffix("_test_suffix") - .withTableConfig(getBasicTableConfig()) + .withTableConfig(getTableConfig(getBasicTableConfig())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) // test including check with affix .includingTables(ThreadLocalRandom.current().nextBoolean() ? "t1|t2" : ".*") .build(); @@ -321,7 +330,8 @@ protected void testTableAffixOneTopic(String format) throws Exception { syncDatabaseActionBuilder(kafkaConfig) .withTablePrefix("test_prefix_") .withTableSuffix("_test_suffix") - .withTableConfig(getBasicTableConfig()) + .withTableConfig(getTableConfig(getBasicTableConfig())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) // test including check with affix .includingTables(ThreadLocalRandom.current().nextBoolean() ? "t1|t2" : ".*") .build(); @@ -490,7 +500,8 @@ private void includingAndExcludingTablesImpl( syncDatabaseActionBuilder(kafkaConfig) .includingTables(includingTables) .excludingTables(excludingTables) - .withTableConfig(getBasicTableConfig()) + .withTableConfig(getTableConfig(getBasicTableConfig())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) .build(); runActionWithDefaultEnv(action); @@ -516,12 +527,13 @@ protected void testCaseInsensitive(String format) throws Exception { kafkaConfig.put(VALUE_FORMAT.key(), format + "-json"); kafkaConfig.put(TOPIC.key(), topic); + Map catalogConfig = new HashMap<>(); + catalogConfig.put(FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false"); + KafkaSyncDatabaseAction action = syncDatabaseActionBuilder(kafkaConfig) - .withTableConfig(getBasicTableConfig()) - .withCatalogConfig( - Collections.singletonMap( - FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false")) + .withTableConfig(getTableConfig(getBasicTableConfig())) + .withCatalogConfig(getCatalogOptions(catalogConfig)) .build(); runActionWithDefaultEnv(action); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java index f2f8fc246682d..e6e6310b5130f 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/kafka/KafkaSyncTableActionITCase.java @@ -33,6 +33,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -74,7 +75,8 @@ protected void runSingleTableSchemaEvolution(String sourceDir, String format) th KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) .withPrimaryKeys("id") - .withTableConfig(getBasicTableConfig()) + .withTableConfig(getTableConfig(getBasicTableConfig())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) .build(); runActionWithDefaultEnv(action); @@ -177,7 +179,8 @@ public void testNotSupportFormat(String format) throws Exception { KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) .withPrimaryKeys("id") - .withTableConfig(getBasicTableConfig()) + .withTableConfig(getTableConfig(getBasicTableConfig())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) .build(); assertThatThrownBy(action::run) @@ -216,7 +219,8 @@ protected void testAssertSchemaCompatible(String format) throws Exception { KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) .withPrimaryKeys("id") - .withTableConfig(getBasicTableConfig()) + .withTableConfig(getTableConfig(getBasicTableConfig())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) .build(); assertThatThrownBy(action::run) @@ -248,7 +252,8 @@ protected void testStarUpOptionSpecific(String format) throws Exception { KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) .withPrimaryKeys("id") - .withTableConfig(getBasicTableConfig()) + .withTableConfig(getTableConfig(getBasicTableConfig())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) .build(); runActionWithDefaultEnv(action); @@ -289,7 +294,8 @@ protected void testStarUpOptionLatest(String format) throws Exception { KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) .withPrimaryKeys("id") - .withTableConfig(getBasicTableConfig()) + .withTableConfig(getTableConfig(getBasicTableConfig())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) .build(); runActionWithDefaultEnv(action); @@ -344,7 +350,8 @@ public void testStarUpOptionTimestamp(String format) throws Exception { KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) .withPrimaryKeys("id") - .withTableConfig(getBasicTableConfig()) + .withTableConfig(getTableConfig(getBasicTableConfig())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) .build(); runActionWithDefaultEnv(action); @@ -396,7 +403,8 @@ public void testStarUpOptionEarliest(String format) throws Exception { KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) .withPrimaryKeys("id") - .withTableConfig(getBasicTableConfig()) + .withTableConfig(getTableConfig(getBasicTableConfig())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) .build(); runActionWithDefaultEnv(action); @@ -450,7 +458,8 @@ public void testStarUpOptionGroup(String format) throws Exception { KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) .withPrimaryKeys("id") - .withTableConfig(getBasicTableConfig()) + .withTableConfig(getTableConfig(getBasicTableConfig())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) .build(); runActionWithDefaultEnv(action); @@ -506,7 +515,8 @@ public void testComputedColumn(String format) throws Exception { .withPartitionKeys("_year") .withPrimaryKeys("_id", "_year") .withComputedColumnArgs("_year=year(_date)") - .withTableConfig(getBasicTableConfig()) + .withTableConfig(getTableConfig(getBasicTableConfig())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) .build(); runActionWithDefaultEnv(action); @@ -542,7 +552,8 @@ protected void testCDCOperations(String format) throws Exception { KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) .withPrimaryKeys("id") - .withTableConfig(getBasicTableConfig()) + .withTableConfig(getTableConfig(getBasicTableConfig())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) .build(); runActionWithDefaultEnv(action); @@ -648,14 +659,17 @@ public void testWaterMarkSyncTable(String format) throws Exception { config.put("scan.watermark.alignment.update-interval", "1 s"); KafkaSyncTableAction action = - syncTableActionBuilder(kafkaConfig).withTableConfig(config).build(); + syncTableActionBuilder(kafkaConfig) + .withTableConfig(getTableConfig(config)) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) + .build(); runActionWithDefaultEnv(action); FileStoreTable table = (FileStoreTable) catalog.getTable(new Identifier(database, tableName)); while (true) { - if (table.snapshotManager().snapshotCount() > 0 - && table.snapshotManager().latestSnapshot().watermark() + if (table.snapshotManager().snapshotCount(branch) > 0 + && table.snapshotManager().latestSnapshot(branch).watermark() != -9223372036854775808L) { return; } @@ -680,7 +694,8 @@ public void testSchemaIncludeRecord(String format) throws Exception { KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) .withPrimaryKeys("id") - .withTableConfig(getBasicTableConfig()) + .withTableConfig(getTableConfig(getBasicTableConfig())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) .build(); runActionWithDefaultEnv(action); @@ -717,7 +732,8 @@ public void testAllTypesWithSchemaImpl(String format) throws Exception { syncTableActionBuilder(kafkaConfig) .withPartitionKeys("pt") .withPrimaryKeys("pt", "_id") - .withTableConfig(getBasicTableConfig()) + .withTableConfig(getTableConfig(getBasicTableConfig())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) .build(); runActionWithDefaultEnv(action); @@ -969,7 +985,8 @@ protected void testTableFiledValNull(String format) throws Exception { KafkaSyncTableAction action = syncTableActionBuilder(kafkaConfig) .withPrimaryKeys("id") - .withTableConfig(getBasicTableConfig()) + .withTableConfig(getTableConfig(getBasicTableConfig())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) .build(); runActionWithDefaultEnv(action); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlActionITCaseBase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlActionITCaseBase.java index 010041268c8a5..38bcec75e4b93 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlActionITCaseBase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlActionITCaseBase.java @@ -68,6 +68,9 @@ protected static void start() { } protected Statement getStatement() throws SQLException { + System.out.println(MYSQL_CONTAINER.getJdbcUrl()); + System.out.println(MYSQL_CONTAINER.getUsername()); + System.out.println(MYSQL_CONTAINER.getPassword()); Connection conn = DriverManager.getConnection( MYSQL_CONTAINER.getJdbcUrl(), diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseActionITCase.java index 84169ffe02b76..a00abd48369f1 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseActionITCase.java @@ -79,7 +79,8 @@ public void testSchemaEvolution() throws Exception { MySqlSyncDatabaseAction action = syncDatabaseActionBuilder(mySqlConfig) - .withTableConfig(getBasicTableConfig()) + .withTableConfig(getTableConfig(getBasicTableConfig())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) .build(); runActionWithDefaultEnv(action); @@ -207,7 +208,11 @@ public void testSpecifiedMySqlTable() { mySqlConfig.put("database-name", "paimon_sync_database"); mySqlConfig.put("table-name", "my_table"); - MySqlSyncDatabaseAction action = syncDatabaseActionBuilder(mySqlConfig).build(); + MySqlSyncDatabaseAction action = + syncDatabaseActionBuilder(mySqlConfig) + .withTableConfig(getTableConfig(new HashMap<>())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) + .build(); assertThatThrownBy(action::run) .isInstanceOf(IllegalArgumentException.class) @@ -222,7 +227,11 @@ public void testInvalidDatabase() { Map mySqlConfig = getBasicMySqlConfig(); mySqlConfig.put("database-name", "invalid"); - MySqlSyncDatabaseAction action = syncDatabaseActionBuilder(mySqlConfig).build(); + MySqlSyncDatabaseAction action = + syncDatabaseActionBuilder(mySqlConfig) + .withTableConfig(getTableConfig(new HashMap<>())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) + .build(); assertThatThrownBy(action::run) .isInstanceOf(IllegalArgumentException.class) @@ -249,7 +258,8 @@ public void testIgnoreIncompatibleTables() throws Exception { MySqlSyncDatabaseAction action = syncDatabaseActionBuilder(mySqlConfig) - .withTableConfig(getBasicTableConfig()) + .withTableConfig(getTableConfig(getBasicTableConfig())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) .ignoreIncompatible(true) .build(); runActionWithDefaultEnv(action); @@ -296,7 +306,8 @@ public void testTableAffix() throws Exception { MySqlSyncDatabaseAction action = syncDatabaseActionBuilder(mySqlConfig) - .withTableConfig(getBasicTableConfig()) + .withTableConfig(getTableConfig(getBasicTableConfig())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) .withTablePrefix("test_prefix_") .withTableSuffix("_test_suffix") // test including check with affix @@ -453,7 +464,8 @@ private void includingAndExcludingTablesImpl( MySqlSyncDatabaseAction action = syncDatabaseActionBuilder(mySqlConfig) - .withTableConfig(getBasicTableConfig()) + .withTableConfig(getTableConfig(getBasicTableConfig())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) .includingTables(includingTables) .excludingTables(excludingTables) .build(); @@ -470,12 +482,13 @@ public void testIgnoreCase() throws Exception { Map mySqlConfig = getBasicMySqlConfig(); mySqlConfig.put("database-name", "paimon_ignore_CASE"); + Map catalogConfig = new HashMap<>(); + catalogConfig.put(FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false"); + MySqlSyncDatabaseAction action = syncDatabaseActionBuilder(mySqlConfig) - .withCatalogConfig( - Collections.singletonMap( - FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false")) - .withTableConfig(getBasicTableConfig()) + .withTableConfig(getTableConfig(getBasicTableConfig())) + .withCatalogConfig(getCatalogOptions(catalogConfig)) .build(); runActionWithDefaultEnv(action); @@ -569,6 +582,8 @@ public void testAddIgnoredTable() throws Exception { MySqlSyncDatabaseAction action = syncDatabaseActionBuilder(mySqlConfig) .withTableConfig(getBasicTableConfig()) + .withTableConfig(getTableConfig(getBasicTableConfig())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) .includingTables("t.+") .excludingTables(".*a$") .withMode(COMBINED.configString()) @@ -846,16 +861,15 @@ private JobClient buildSyncDatabaseActionWithNewlyAddedTables( mySqlConfig.put("database-name", databaseName); mySqlConfig.put("scan.incremental.snapshot.chunk.size", "1"); - Map catalogConfig = - testSchemaChange - ? Collections.singletonMap( - CatalogOptions.METASTORE.key(), "test-alter-table") - : Collections.emptyMap(); + Map catalogConfig = new HashMap<>(); + if (testSchemaChange) { + catalogConfig.put(CatalogOptions.METASTORE.key(), "test-alter-table"); + } MySqlSyncDatabaseAction action = syncDatabaseActionBuilder(mySqlConfig) - .withCatalogConfig(catalogConfig) - .withTableConfig(getBasicTableConfig()) + .withTableConfig(getTableConfig(getBasicTableConfig())) + .withCatalogConfig(getCatalogOptions(catalogConfig)) .includingTables("t.+") .withMode(COMBINED.configString()) .build(); @@ -895,7 +909,8 @@ public void testSyncManyTableWithLimitedMemory() throws Exception { MySqlSyncDatabaseAction action = syncDatabaseActionBuilder(mySqlConfig) - .withTableConfig(tableConfig) + .withTableConfig(getTableConfig(tableConfig)) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) .withMode(COMBINED.configString()) .build(); runActionWithDefaultEnv(action); @@ -942,7 +957,8 @@ public void testSyncMultipleShards() throws Exception { MultiTablesSinkMode mode = ThreadLocalRandom.current().nextBoolean() ? DIVIDED : COMBINED; MySqlSyncDatabaseAction action = syncDatabaseActionBuilder(mySqlConfig) - .withTableConfig(getBasicTableConfig()) + .withTableConfig(getTableConfig(getBasicTableConfig())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) .withMode(mode.configString()) .build(); runActionWithDefaultEnv(action); @@ -1050,7 +1066,8 @@ public void testSyncMultipleShardsWithoutMerging() throws Exception { MultiTablesSinkMode mode = ThreadLocalRandom.current().nextBoolean() ? DIVIDED : COMBINED; MySqlSyncDatabaseAction action = syncDatabaseActionBuilder(mySqlConfig) - .withTableConfig(getBasicTableConfig()) + .withTableConfig(getTableConfig(getBasicTableConfig())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) .mergeShards(false) .withMode(mode.configString()) .build(); @@ -1165,6 +1182,8 @@ public void testMonitoredAndExcludedTablesWithMering() throws Exception { MySqlSyncDatabaseAction action = syncDatabaseActionBuilder(mySqlConfig) + .withTableConfig(getTableConfig(new HashMap<>())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) .ignoreIncompatible(true) .withMode(COMBINED.configString()) .build(); @@ -1202,7 +1221,8 @@ public void testNewlyAddedTablesOptionsChange() throws Exception { MySqlSyncDatabaseAction action1 = syncDatabaseActionBuilder(mySqlConfig) - .withTableConfig(tableConfig) + .withTableConfig(getTableConfig(tableConfig)) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) .withMode(COMBINED.configString()) .build(); @@ -1228,7 +1248,10 @@ public void testNewlyAddedTablesOptionsChange() throws Exception { } MySqlSyncDatabaseAction action2 = - syncDatabaseActionBuilder(mySqlConfig).withTableConfig(tableConfig).build(); + syncDatabaseActionBuilder(mySqlConfig) + .withTableConfig(getTableConfig(tableConfig)) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) + .build(); runActionWithDefaultEnv(action2); waitingTables("t2"); @@ -1238,15 +1261,20 @@ public void testNewlyAddedTablesOptionsChange() throws Exception { @Test public void testCatalogAndTableConfig() { + Map catalogConfig = new HashMap<>(); + catalogConfig.put("catalog-key", "catalog-value"); + + Map tableConfig = new HashMap<>(); + tableConfig.put("table-key", "table-value"); + MySqlSyncDatabaseAction action = syncDatabaseActionBuilder(getBasicMySqlConfig()) - .withCatalogConfig(Collections.singletonMap("catalog-key", "catalog-value")) - .withTableConfig(Collections.singletonMap("table-key", "table-value")) + .withTableConfig(getTableConfig(tableConfig)) + .withCatalogConfig(getCatalogOptions(catalogConfig)) .build(); assertThat(action.catalogConfig()).containsEntry("catalog-key", "catalog-value"); - assertThat(action.tableConfig()) - .containsExactlyEntriesOf(Collections.singletonMap("table-key", "table-value")); + assertThat(action.tableConfig()).containsExactlyEntriesOf(getTableConfig(tableConfig)); } @Test @@ -1258,7 +1286,8 @@ public void testMetadataColumns() throws Exception { MultiTablesSinkMode mode = ThreadLocalRandom.current().nextBoolean() ? DIVIDED : COMBINED; MySqlSyncDatabaseAction action = syncDatabaseActionBuilder(mySqlConfig) - .withTableConfig(getBasicTableConfig()) + .withTableConfig(getTableConfig(getBasicTableConfig())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) .withMode(mode.configString()) .withMetadataColumn(Arrays.asList("table_name", "database_name")) .build(); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseActionToBranchITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseActionToBranchITCase.java new file mode 100644 index 0000000000000..e6cf68455bb10 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncDatabaseActionToBranchITCase.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.action.cdc.mysql; + +import org.junit.jupiter.api.BeforeEach; + +import java.io.IOException; + +/** IT cases for {@link MySqlSyncDatabaseAction}. */ +public class MySqlSyncDatabaseActionToBranchITCase extends MySqlSyncDatabaseActionITCase { + + @BeforeEach + @Override + public void before() throws IOException { + this.branch = "testMySqlSyncDatabaseActionBranch"; + super.before(); + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java index 6cd35eec83284..95cec9e0bf871 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionITCase.java @@ -77,12 +77,13 @@ public void testSchemaEvolution() throws Exception { mySqlConfig.put("database-name", DATABASE_NAME); mySqlConfig.put("table-name", "schema_evolution_\\d+"); + Map catalogConfig = getBasicMySqlConfig(); + catalogConfig.put(CatalogOptions.METASTORE.key(), "test-alter-table"); + MySqlSyncTableAction action = syncTableActionBuilder(mySqlConfig) - .withCatalogConfig( - Collections.singletonMap( - CatalogOptions.METASTORE.key(), "test-alter-table")) - .withTableConfig(getBasicTableConfig()) + .withCatalogConfig(getCatalogOptions(catalogConfig)) + .withTableConfig(getTableConfig(getBasicTableConfig())) .withPartitionKeys("pt") .withPrimaryKeys("pt", "_id") .build(); @@ -256,7 +257,11 @@ public void testMultipleSchemaEvolutions() throws Exception { mySqlConfig.put("database-name", DATABASE_NAME); mySqlConfig.put("table-name", "schema_evolution_multiple"); - MySqlSyncTableAction action = syncTableActionBuilder(mySqlConfig).build(); + MySqlSyncTableAction action = + syncTableActionBuilder(mySqlConfig) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) + .withTableConfig(getTableConfig(new HashMap<>())) + .build(); runActionWithDefaultEnv(action); checkTableSchema( @@ -321,7 +326,7 @@ private void testSchemaEvolutionMultipleImpl(Statement statement) throws Excepti } @Test - @Timeout(90) + @Timeout(180) public void testAllTypes() throws Exception { // the first round checks for table creation // the second round checks for running the action on an existing table @@ -330,13 +335,15 @@ public void testAllTypes() throws Exception { } } - private void testAllTypesOnce() throws Exception { + protected void testAllTypesOnce() throws Exception { Map mySqlConfig = getBasicMySqlConfig(); mySqlConfig.put("database-name", DATABASE_NAME); mySqlConfig.put("table-name", "all_types_table"); MySqlSyncTableAction action = syncTableActionBuilder(mySqlConfig) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) + .withTableConfig(getTableConfig(new HashMap<>())) .withPartitionKeys("pt") .withPrimaryKeys("pt", "_id") .build(); @@ -616,7 +623,7 @@ private void testAllTypesImpl(Statement statement) throws Exception { } finally { statement.executeUpdate("ALTER TABLE all_types_table DROP COLUMN v"); SchemaManager schemaManager = new SchemaManager(table.fileIO(), table.location()); - schemaManager.commitChanges(SchemaChange.dropColumn("v")); + schemaManager.commitChanges(branch, SchemaChange.dropColumn("v")); } } @@ -626,7 +633,11 @@ public void testIncompatibleMySqlTable() { mySqlConfig.put("database-name", DATABASE_NAME); mySqlConfig.put("table-name", "incompatible_field_\\d+"); - MySqlSyncTableAction action = syncTableActionBuilder(mySqlConfig).build(); + MySqlSyncTableAction action = + syncTableActionBuilder(mySqlConfig) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) + .withTableConfig(getTableConfig(new HashMap<>())) + .build(); assertThatThrownBy(action::run) .satisfies( @@ -652,7 +663,11 @@ public void testIncompatiblePaimonTable() throws Exception { new HashMap<>()); MySqlSyncTableAction action = - syncTableActionBuilder(mySqlConfig).withPrimaryKeys("a").build(); + syncTableActionBuilder(mySqlConfig) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) + .withTableConfig(getTableConfig(new HashMap<>())) + .withPrimaryKeys("a") + .build(); assertThatThrownBy(action::run) .satisfies( @@ -668,7 +683,11 @@ public void testInvalidPrimaryKey() { mySqlConfig.put("table-name", "schema_evolution_\\d+"); MySqlSyncTableAction action = - syncTableActionBuilder(mySqlConfig).withPrimaryKeys("pk").build(); + syncTableActionBuilder(mySqlConfig) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) + .withTableConfig(getTableConfig(new HashMap<>())) + .withPrimaryKeys("pk") + .build(); assertThatThrownBy(action::run) .satisfies( @@ -683,7 +702,11 @@ public void testNoPrimaryKey() { mySqlConfig.put("database-name", DATABASE_NAME); mySqlConfig.put("table-name", "incompatible_pk_\\d+"); - MySqlSyncTableAction action = syncTableActionBuilder(mySqlConfig).build(); + MySqlSyncTableAction action = + syncTableActionBuilder(mySqlConfig) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) + .withTableConfig(getTableConfig(new HashMap<>())) + .build(); assertThatThrownBy(action::run) .satisfies( @@ -695,7 +718,7 @@ public void testNoPrimaryKey() { } @Test - @Timeout(60) + @Timeout(240) public void testComputedColumn() throws Exception { // the first round checks for table creation // the second round checks for running the action on an existing table @@ -738,6 +761,8 @@ private void innerTestComputedColumn(boolean executeMysql) throws Exception { MySqlSyncTableAction action = syncTableActionBuilder(mySqlConfig) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) + .withTableConfig(getTableConfig(new HashMap<>())) .withPartitionKeys("_year_date") .withPrimaryKeys("pk", "_year_date") .withComputedColumnArgs(computedColumnDefs) @@ -1010,6 +1035,8 @@ public void testSyncShards() throws Exception { MySqlSyncTableAction action = syncTableActionBuilder(mySqlConfig) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) + .withTableConfig(getTableConfig(new HashMap<>())) .withPartitionKeys("pt") .withPrimaryKeys("pk", "pt") .withComputedColumnArgs("pt=substring(_date,5)") @@ -1061,7 +1088,8 @@ public void testOptionsChange() throws Exception { .withPartitionKeys("pt") .withPrimaryKeys("pk", "pt") .withComputedColumnArgs("pt=substring(_date,5)") - .withTableConfig(tableConfig) + .withTableConfig(getTableConfig(tableConfig)) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) .build(); JobClient jobClient = runActionWithDefaultEnv(action1); try (Statement statement = getStatement()) { @@ -1089,7 +1117,8 @@ public void testOptionsChange() throws Exception { .withPartitionKeys("pt") .withPrimaryKeys("pk", "pt") .withComputedColumnArgs("pt=substring(_date,5)") - .withTableConfig(tableConfig) + .withTableConfig(getTableConfig(tableConfig)) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) .build(); runActionWithDefaultEnv(action2); @@ -1111,6 +1140,8 @@ public void testMetadataColumns() throws Exception { MySqlSyncTableAction action = syncTableActionBuilder(mySqlConfig) + .withTableConfig(getTableConfig(new HashMap<>())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) .withPrimaryKeys("pk") .withMetadataColumns("table_name", "database_name", "op_ts") .build(); @@ -1148,15 +1179,20 @@ public void testMetadataColumns() throws Exception { @Test public void testCatalogAndTableConfig() { + Map catalogOptions = new HashMap<>(); + catalogOptions.put("catalog-key", "catalog-value"); + + Map tableConfig = new HashMap<>(); + tableConfig.put("table-key", "table-value"); + MySqlSyncTableAction action = syncTableActionBuilder(getBasicMySqlConfig()) - .withCatalogConfig(Collections.singletonMap("catalog-key", "catalog-value")) - .withTableConfig(Collections.singletonMap("table-key", "table-value")) + .withTableConfig(getTableConfig(tableConfig)) + .withCatalogConfig(getCatalogOptions(catalogOptions)) .build(); assertThat(action.catalogConfig()).containsEntry("catalog-key", "catalog-value"); - assertThat(action.tableConfig()) - .containsExactlyEntriesOf(Collections.singletonMap("table-key", "table-value")); + assertThat(action.tableConfig()).containsExactlyEntriesOf(getTableConfig(tableConfig)); } private FileStoreTable getFileStoreTable() throws Exception { @@ -1173,7 +1209,11 @@ public void testDefaultCheckpointInterval() throws Exception { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setRestartStrategy(RestartStrategies.noRestart()); - MySqlSyncTableAction action = syncTableActionBuilder(mySqlConfig).build(); + MySqlSyncTableAction action = + syncTableActionBuilder(mySqlConfig) + .withTableConfig(getTableConfig(new HashMap<>())) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) + .build(); action.withStreamExecutionEnvironment(env); Thread thread = @@ -1204,11 +1244,13 @@ public void testComputedColumnWithCaseInsensitive() throws Exception { mySqlConfig.put("database-name", "computed_column_with_case_insensitive"); mySqlConfig.put("table-name", "t"); + Map catalogConfig = new HashMap<>(); + catalogConfig.put(FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false"); + MySqlSyncTableAction action = syncTableActionBuilder(mySqlConfig) - .withCatalogConfig( - Collections.singletonMap( - FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false")) + .withCatalogConfig(getCatalogOptions(catalogConfig)) + .withTableConfig(getTableConfig(new HashMap<>())) .withComputedColumnArgs("SUBSTRING=substring(UPPERCASE_STRING,2)") .build(); runActionWithDefaultEnv(action); @@ -1240,11 +1282,13 @@ public void testSpecifyKeysWithCaseInsensitive() throws Exception { mySqlConfig.put("database-name", "specify_key_with_case_insensitive"); mySqlConfig.put("table-name", "t"); + Map catalogConfig = new HashMap<>(); + catalogConfig.put(FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false"); + MySqlSyncTableAction action = syncTableActionBuilder(mySqlConfig) - .withCatalogConfig( - Collections.singletonMap( - FileSystemCatalogOptions.CASE_SENSITIVE.key(), "false")) + .withCatalogConfig(getCatalogOptions(catalogConfig)) + .withTableConfig(getTableConfig(new HashMap<>())) .withPrimaryKeys("ID1", "PART") .withPartitionKeys("PART") .build(); @@ -1268,9 +1312,13 @@ public void testInvalidAlterBucket() throws Exception { mySqlConfig.put("database-name", "invalid_alter_bucket"); mySqlConfig.put("table-name", "t"); + Map tableConfig = new HashMap<>(); + tableConfig.put(BUCKET.key(), "2"); + MySqlSyncTableAction action = syncTableActionBuilder(mySqlConfig) - .withTableConfig(Collections.singletonMap(BUCKET.key(), "2")) + .withTableConfig(getTableConfig(tableConfig)) + .withCatalogConfig(getCatalogOptions(new HashMap<>())) .build(); assertThatCode(action::build).doesNotThrowAnyException(); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionToBranchITCase.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionToBranchITCase.java new file mode 100644 index 0000000000000..7567ec7ea6402 --- /dev/null +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/action/cdc/mysql/MySqlSyncTableActionToBranchITCase.java @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.action.cdc.mysql; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; + +import java.io.IOException; + +/** IT cases for {@link MySqlSyncTableAction}. */ +public class MySqlSyncTableActionToBranchITCase extends MySqlSyncTableActionITCase { + @BeforeEach + public void before() throws IOException { + super.branch = "testMySqlSyncTableActionBranch"; + super.before(); + } + + @Test + @Timeout(120) + public void testAllTypes() throws Exception { + // Waiting to continue verification + testAllTypesOnce(); + } +} diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperatorTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperatorTest.java index 08d5fa72c2e6d..951a6bc8bc9fb 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperatorTest.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreMultiWriteOperatorTest.java @@ -39,6 +39,7 @@ import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowKind; import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.BranchManager; import org.apache.paimon.utils.CommonTestUtils; import org.apache.paimon.utils.TraceableFileIO; @@ -722,7 +723,8 @@ public void testUsingTheSameCompactExecutor() throws Exception { false, true, memoryPoolFactory, - metricGroup), + metricGroup, + BranchManager.DEFAULT_MAIN_BRANCH), commitUser, Options.fromMap(new HashMap<>())); TypeSerializer inputSerializer = new JavaSerializer<>(); diff --git a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperatorTest.java b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperatorTest.java index 9af7eabdaaadb..419de68fa5171 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperatorTest.java +++ b/paimon-flink/paimon-flink-cdc/src/test/java/org/apache/paimon/flink/sink/cdc/CdcRecordStoreWriteOperatorTest.java @@ -36,6 +36,7 @@ import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowKind; import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.BranchManager; import org.apache.paimon.utils.TraceableFileIO; import org.apache.flink.api.common.ExecutionConfig; @@ -266,7 +267,8 @@ private OneInputStreamOperatorTestHarness createTestHarn false, true, memoryPool, - metricGroup), + metricGroup, + BranchManager.DEFAULT_MAIN_BRANCH), commitUser); TypeSerializer inputSerializer = new JavaSerializer<>(); TypeSerializer outputSerializer = diff --git a/paimon-flink/paimon-flink-cdc/src/test/resources/mysql/sync_database_setup.sql b/paimon-flink/paimon-flink-cdc/src/test/resources/mysql/sync_database_setup.sql index bd70146be48cc..82b8170f23545 100644 --- a/paimon-flink/paimon-flink-cdc/src/test/resources/mysql/sync_database_setup.sql +++ b/paimon-flink/paimon-flink-cdc/src/test/resources/mysql/sync_database_setup.sql @@ -369,6 +369,56 @@ CREATE TABLE t3 ( v1 VARCHAR(10) ); + +-- ################################################################################ +-- testSyncMultipleShardsWithBranch +-- ################################################################################ + +CREATE DATABASE database_branch_shard_1; +USE database_branch_shard_1; + +CREATE TABLE t1 ( + k INT, + v1 VARCHAR(10), + PRIMARY KEY (k) +); + +CREATE TABLE t2 ( + k BIGINT, + v1 DOUBLE, + PRIMARY KEY (k) +); + +CREATE TABLE t3 ( + k INT, + v1 VARCHAR(10), + PRIMARY KEY (k) +); + +CREATE DATABASE database_branch_shard_2; +USE database_branch_shard_2; + +-- test schema merging +CREATE TABLE t1 ( + k INT, + v1 VARCHAR(20), + v2 BIGINT, + PRIMARY KEY (k) +); + +-- test schema evolution +CREATE TABLE t2 ( + k BIGINT, + v1 DOUBLE, + PRIMARY KEY (k) +); + +-- test some shard doesn't have primary key +CREATE TABLE t3 ( + k INT, + v1 VARCHAR(10) +); + -- ################################################################################ -- testSyncMultipleShardsWithoutMerging -- ################################################################################ diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/AbstractFlinkTableFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/AbstractFlinkTableFactory.java index 782a9804e505f..4f01b25a21510 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/AbstractFlinkTableFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/AbstractFlinkTableFactory.java @@ -41,6 +41,7 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.FileStoreTableFactory; import org.apache.paimon.table.Table; +import org.apache.paimon.utils.BranchManager; import org.apache.paimon.utils.Preconditions; import org.apache.flink.api.common.RuntimeExecutionMode; @@ -248,13 +249,18 @@ static Table buildPaimonTable(DynamicTableFactory.Context context) { newOptions.putAll(origin.getOptions()); newOptions.putAll(dynamicOptions); + String branch = dynamicOptions.get(FlinkConnectorOptions.BRANCH); + if (branch == null) { + branch = BranchManager.DEFAULT_MAIN_BRANCH; + } + // notice that the Paimon table schema must be the same with the Flink's if (origin instanceof DataCatalogTable) { FileStoreTable fileStoreTable = (FileStoreTable) ((DataCatalogTable) origin).table(); table = fileStoreTable.copyWithoutTimeTravel(newOptions); } else { table = - FileStoreTableFactory.create(createCatalogContext(context)) + FileStoreTableFactory.create(createCatalogContext(context), branch) .copyWithoutTimeTravel(newOptions); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkConnectorOptions.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkConnectorOptions.java index 94371cb60f156..d485be1a9b6e9 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkConnectorOptions.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkConnectorOptions.java @@ -30,6 +30,7 @@ import org.apache.paimon.options.description.Description; import org.apache.paimon.options.description.InlineElement; import org.apache.paimon.options.description.TextElement; +import org.apache.paimon.utils.BranchManager; import java.lang.reflect.Field; import java.time.Duration; @@ -48,6 +49,12 @@ public class FlinkConnectorOptions { public static final String TABLE_DYNAMIC_OPTION_PREFIX = "paimon"; + public static final ConfigOption BRANCH = + key("branch") + .stringType() + .defaultValue(BranchManager.DEFAULT_MAIN_BRANCH) + .withDescription("Specify branch name."); + @ExcludeFromDocumentation("Confused without log system") public static final ConfigOption LOG_SYSTEM = ConfigOptions.key("log.system") diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkTableFactory.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkTableFactory.java index 4c9f8ff9b55de..f4bbdb5ee8a90 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkTableFactory.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/FlinkTableFactory.java @@ -23,6 +23,7 @@ import org.apache.paimon.fs.Path; import org.apache.paimon.options.Options; import org.apache.paimon.schema.SchemaManager; +import org.apache.paimon.utils.BranchManager; import org.apache.flink.table.catalog.ResolvedCatalogTable; import org.apache.flink.table.connector.sink.DynamicTableSink; @@ -58,11 +59,15 @@ private void createTableIfNeeded(Context context) { if (options.get(AUTO_CREATE)) { try { Path tablePath = CoreOptions.path(table.getOptions()); + String branch = + options.getString( + FlinkConnectorOptions.BRANCH.key(), + BranchManager.DEFAULT_MAIN_BRANCH); SchemaManager schemaManager = new SchemaManager( FileIO.get(tablePath, createCatalogContext(context)), tablePath); - if (!schemaManager.latest().isPresent()) { - schemaManager.createTable(FlinkCatalog.fromCatalogTable(table)); + if (!schemaManager.latest(branch).isPresent()) { + schemaManager.createTable(FlinkCatalog.fromCatalogTable(table), branch); } } catch (Exception e) { throw new RuntimeException(e); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendOnlyTableCompactionWorkerOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendOnlyTableCompactionWorkerOperator.java index 0860ccb3ac8be..9a88af4db316b 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendOnlyTableCompactionWorkerOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AppendOnlyTableCompactionWorkerOperator.java @@ -74,7 +74,7 @@ Iterable> result() { @Override public void open() throws Exception { LOG.debug("Opened a append-only table compaction worker."); - this.write = (AppendOnlyFileStoreWrite) table.store().newWrite(commitUser); + this.write = (AppendOnlyFileStoreWrite) table.store().newWrite(commitUser, branch); this.result = new LinkedList<>(); } @@ -149,7 +149,7 @@ void shutdown() throws Exception { return; } - try (TableCommitImpl tableCommit = table.newCommit(commitUser)) { + try (TableCommitImpl tableCommit = table.newCommit(commitUser, branch)) { tableCommit.abort(messages); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperator.java index da3425e9b0d35..03db8b6ab5543 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/AutoTagForSavepointCommitterOperator.java @@ -21,6 +21,7 @@ import org.apache.paimon.Snapshot; import org.apache.paimon.operation.TagDeletion; import org.apache.paimon.table.sink.TagCallback; +import org.apache.paimon.utils.BranchManager; import org.apache.paimon.utils.SerializableSupplier; import org.apache.paimon.utils.SnapshotManager; import org.apache.paimon.utils.TagManager; @@ -86,6 +87,8 @@ public class AutoTagForSavepointCommitterOperator private transient ListState identifiersForTagsState; + private transient String branchName; + public AutoTagForSavepointCommitterOperator( CommitterOperator commitOperator, SerializableSupplier snapshotManagerFactory, @@ -98,6 +101,23 @@ public AutoTagForSavepointCommitterOperator( this.tagDeletionFactory = tagDeletionFactory; this.callbacksSupplier = callbacksSupplier; this.identifiersForTags = new TreeSet<>(); + this.branchName = BranchManager.DEFAULT_MAIN_BRANCH; + } + + public AutoTagForSavepointCommitterOperator( + CommitterOperator commitOperator, + SerializableSupplier snapshotManagerFactory, + SerializableSupplier tagManagerFactory, + SerializableSupplier tagDeletionFactory, + SerializableSupplier> callbacksSupplier, + String branchName) { + this.commitOperator = commitOperator; + this.tagManagerFactory = tagManagerFactory; + this.snapshotManagerFactory = snapshotManagerFactory; + this.tagDeletionFactory = tagDeletionFactory; + this.callbacksSupplier = callbacksSupplier; + this.identifiersForTags = new TreeSet<>(); + this.branchName = branchName; } @Override @@ -155,19 +175,19 @@ public void notifyCheckpointAborted(long checkpointId) throws Exception { commitOperator.notifyCheckpointAborted(checkpointId); identifiersForTags.remove(checkpointId); String tagName = SAVEPOINT_TAG_PREFIX + checkpointId; - if (tagManager.tagExists(tagName)) { - tagManager.deleteTag(tagName, tagDeletion, snapshotManager); + if (tagManager.tagExists(branchName, tagName)) { + tagManager.deleteTag(tagName, tagDeletion, snapshotManager, branchName); } } private void createTagForIdentifiers(List identifiers) { List snapshotForTags = snapshotManager.findSnapshotsForIdentifiers( - commitOperator.getCommitUser(), identifiers); + commitOperator.getCommitUser(), identifiers, branchName); for (Snapshot snapshot : snapshotForTags) { String tagName = SAVEPOINT_TAG_PREFIX + snapshot.commitIdentifier(); - if (!tagManager.tagExists(tagName)) { - tagManager.createTag(snapshot, tagName, callbacks); + if (!tagManager.tagExists(branchName, tagName)) { + tagManager.createTag(snapshot, tagName, callbacks, branchName); } } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperator.java index d65ab74140fe5..b1ea26421d31e 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/BatchWriteGeneratorTagOperator.java @@ -21,6 +21,7 @@ import org.apache.paimon.Snapshot; import org.apache.paimon.operation.TagDeletion; import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.utils.BranchManager; import org.apache.paimon.utils.SnapshotManager; import org.apache.paimon.utils.TagManager; @@ -65,10 +66,22 @@ public class BatchWriteGeneratorTagOperator protected final FileStoreTable table; + protected final String branchName; + public BatchWriteGeneratorTagOperator( CommitterOperator commitOperator, FileStoreTable table) { this.table = table; this.commitOperator = commitOperator; + this.branchName = BranchManager.DEFAULT_MAIN_BRANCH; + } + + public BatchWriteGeneratorTagOperator( + CommitterOperator commitOperator, + FileStoreTable table, + String branchName) { + this.table = table; + this.commitOperator = commitOperator; + this.branchName = branchName; } @Override @@ -100,7 +113,7 @@ public void notifyCheckpointAborted(long checkpointId) throws Exception { private void createTag() { SnapshotManager snapshotManager = table.snapshotManager(); - Snapshot snapshot = snapshotManager.latestSnapshot(); + Snapshot snapshot = snapshotManager.latestSnapshot(branchName); if (snapshot == null) { return; } @@ -113,11 +126,11 @@ private void createTag() { + localDateTime.format(DateTimeFormatter.ofPattern("yyyy-MM-dd")); try { // If the tag already exists, delete the tag - if (tagManager.tagExists(tagName)) { - tagManager.deleteTag(tagName, tagDeletion, snapshotManager); + if (tagManager.tagExists(branchName, tagName)) { + tagManager.deleteTag(tagName, tagDeletion, snapshotManager, branchName); } // Create a new tag - tagManager.createTag(snapshot, tagName, table.store().createTagCallbacks()); + tagManager.createTag(snapshot, tagName, table.store().createTagCallbacks(), branchName); // Expire the tag expireTag(); } catch (Exception e) { @@ -136,18 +149,20 @@ private void expireTag() { } TagManager tagManager = table.tagManager(); TagDeletion tagDeletion = table.store().newTagDeletion(); - long tagCount = tagManager.tagCount(); + long tagCount = tagManager.tagCount(branchName); while (tagCount > tagNumRetainedMax) { - for (List tagNames : tagManager.tags().values()) { + for (List tagNames : tagManager.tags(branchName).values()) { if (tagCount - tagNames.size() >= tagNumRetainedMax) { tagManager.deleteAllTagsOfOneSnapshot( - tagNames, tagDeletion, snapshotManager); + tagNames, tagDeletion, snapshotManager, branchName); tagCount = tagCount - tagNames.size(); } else { - List sortedTagNames = tagManager.sortTagsOfOneSnapshot(tagNames); + List sortedTagNames = + tagManager.sortTagsOfOneSnapshot(branchName, tagNames); for (String toBeDeleted : sortedTagNames) { - tagManager.deleteTag(toBeDeleted, tagDeletion, snapshotManager); + tagManager.deleteTag( + toBeDeleted, tagDeletion, snapshotManager, branchName); tagCount--; if (tagCount == tagNumRetainedMax) { break; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactorSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactorSink.java index 8e08824567ad3..d349d5ae73896 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactorSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/CompactorSink.java @@ -36,13 +36,14 @@ public CompactorSink(FileStoreTable table) { @Override protected OneInputStreamOperator createWriteOperator( StoreSinkWrite.Provider writeProvider, String commitUser) { - return new StoreCompactOperator(table, writeProvider, commitUser); + return new StoreCompactOperator(table, writeProvider, commitUser).toBranch(branch); } @Override protected Committer.Factory createCommitterFactory( boolean streamingCheckpointEnabled) { - return (user, metricGroup) -> new StoreCommitter(table.newCommit(user), metricGroup); + return (user, metricGroup) -> + new StoreCommitter(table.newCommit(user, branch), metricGroup); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FixedBucketSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FixedBucketSink.java index 613bf369b0524..51da4d93e7d84 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FixedBucketSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FixedBucketSink.java @@ -45,6 +45,7 @@ public FixedBucketSink( @Override protected OneInputStreamOperator createWriteOperator( StoreSinkWrite.Provider writeProvider, String commitUser) { - return new RowDataStoreWriteOperator(table, logSinkFunction, writeProvider, commitUser); + return new RowDataStoreWriteOperator(table, logSinkFunction, writeProvider, commitUser) + .toBranch(branch); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java index 545bd7f070729..26104baefbce8 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSink.java @@ -25,6 +25,7 @@ import org.apache.paimon.options.MemorySize; import org.apache.paimon.options.Options; import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.utils.BranchManager; import org.apache.paimon.utils.Preconditions; import org.apache.paimon.utils.SerializableRunnable; @@ -76,12 +77,18 @@ public abstract class FlinkSink implements Serializable { protected final FileStoreTable table; private final boolean ignorePreviousFiles; + protected String branch = BranchManager.DEFAULT_MAIN_BRANCH; public FlinkSink(FileStoreTable table, boolean ignorePreviousFiles) { this.table = table; this.ignorePreviousFiles = ignorePreviousFiles; } + public FlinkSink toBranch(String branch) { + this.branch = branch; + return this; + } + private StoreSinkWrite.Provider createWriteProvider( CheckpointConfig checkpointConfig, boolean isStreaming, boolean hasSinkMaterializer) { SerializableRunnable assertNoSinkMaterializer = @@ -128,7 +135,8 @@ private StoreSinkWrite.Provider createWriteProvider( finalDeltaCommits, isStreaming, memoryPool, - metricGroup); + metricGroup, + branch); }; } } @@ -144,7 +152,8 @@ private StoreSinkWrite.Provider createWriteProvider( waitCompaction, isStreaming, memoryPool, - metricGroup); + metricGroup, + branch); }; } @@ -238,6 +247,7 @@ protected DataStreamSink doCommit(DataStream written, String com commitUser, createCommitterFactory(streamingCheckpointEnabled), createCommittableStateManager()); + if (Options.fromMap(table.options()).get(SINK_AUTO_TAG_FOR_SAVEPOINT)) { committerOperator = new AutoTagForSavepointCommitterOperator<>( @@ -245,14 +255,16 @@ protected DataStreamSink doCommit(DataStream written, String com table::snapshotManager, table::tagManager, () -> table.store().newTagDeletion(), - () -> table.store().createTagCallbacks()); + () -> table.store().createTagCallbacks(), + branch); } if (conf.get(ExecutionOptions.RUNTIME_MODE) == RuntimeExecutionMode.BATCH && table.coreOptions().tagCreationMode() == TagCreationMode.BATCH) { committerOperator = new BatchWriteGeneratorTagOperator<>( (CommitterOperator) committerOperator, - table); + table, + branch); } SingleOutputStreamOperator committed = written.transform( diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSinkBuilder.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSinkBuilder.java index b5de897b92f83..824bfe40c99da 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSinkBuilder.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSinkBuilder.java @@ -22,6 +22,7 @@ import org.apache.paimon.flink.sink.index.GlobalDynamicBucketSink; import org.apache.paimon.table.BucketMode; import org.apache.paimon.table.FileStoreTable; +import org.apache.paimon.utils.BranchManager; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; @@ -38,13 +39,13 @@ public class FlinkSinkBuilder { private final FileStoreTable table; - private DataStream input; @Nullable private Map overwritePartition; @Nullable private LogSinkFunction logSinkFunction; @Nullable private Integer parallelism; private boolean boundedInput = false; private boolean compactSink = false; + private String branch = BranchManager.DEFAULT_MAIN_BRANCH; public FlinkSinkBuilder(FileStoreTable table) { this.table = table; @@ -89,6 +90,11 @@ public FlinkSinkBuilder forCompact(boolean compactSink) { return this; } + public FlinkSinkBuilder toBranch(String branch) { + this.branch = branch; + return this; + } + public DataStreamSink build() { DataStream input = MapToInternalRow.map(this.input, table.rowType()); if (table.coreOptions().localMergeEnabled() && table.schema().primaryKeys().size() > 0) { @@ -121,11 +127,18 @@ private DataStreamSink buildDynamicBucketSink( checkArgument(logSinkFunction == null, "Dynamic bucket mode can not work with log system."); return compactSink && !globalIndex // todo support global index sort compact - ? new DynamicBucketCompactSink(table, overwritePartition).build(input, parallelism) + ? ((DynamicBucketCompactSink) + new DynamicBucketCompactSink(table, overwritePartition) + .toBranch(branch)) + .build(input, parallelism) : globalIndex - ? new GlobalDynamicBucketSink(table, overwritePartition) + ? ((GlobalDynamicBucketSink) + new GlobalDynamicBucketSink(table, overwritePartition) + .toBranch(branch)) .build(input, parallelism) - : new RowDynamicBucketSink(table, overwritePartition) + : ((RowDynamicBucketSink) + new RowDynamicBucketSink(table, overwritePartition) + .toBranch(branch)) .build(input, parallelism); } @@ -135,7 +148,10 @@ private DataStreamSink buildForFixedBucket(DataStream input) { input, new RowDataChannelComputer(table.schema(), logSinkFunction != null), parallelism); - FixedBucketSink sink = new FixedBucketSink(table, overwritePartition, logSinkFunction); + FixedBucketSink sink = + (FixedBucketSink) + new FixedBucketSink(table, overwritePartition, logSinkFunction) + .toBranch(branch); return sink.sinkFrom(partitioned); } @@ -145,6 +161,7 @@ private DataStreamSink buildUnawareBucketSink(DataStream input) "Unaware bucket mode only works with append-only table for now."); return new RowUnawareBucketSink( table, overwritePartition, logSinkFunction, parallelism, boundedInput) + .toBranch(branch) .sinkFrom(input); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkTableSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkTableSink.java index 50bc45b752f8d..d2d191a95db60 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkTableSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkTableSink.java @@ -18,11 +18,14 @@ package org.apache.paimon.flink.sink; +import org.apache.paimon.flink.FlinkConnectorOptions; import org.apache.paimon.flink.log.LogStoreTableFactory; import org.apache.paimon.operation.FileStoreCommit; +import org.apache.paimon.options.Options; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; import org.apache.paimon.table.sink.BatchWriteBuilder; +import org.apache.paimon.utils.BranchManager; import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.connector.sink.abilities.SupportsTruncate; @@ -47,7 +50,14 @@ public FlinkTableSink( @Override public void executeTruncation() { FileStoreCommit commit = - ((FileStoreTable) table).store().newCommit(UUID.randomUUID().toString()); + ((FileStoreTable) table) + .store() + .newCommit( + UUID.randomUUID().toString(), + Options.fromMap(table.options()) + .getString( + FlinkConnectorOptions.BRANCH.key(), + BranchManager.DEFAULT_MAIN_BRANCH)); long identifier = BatchWriteBuilder.COMMIT_IDENTIFIER; commit.purgeTable(identifier); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkWriteSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkWriteSink.java index b812c04912b12..85ebd5947856e 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkWriteSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkWriteSink.java @@ -48,7 +48,7 @@ protected Committer.Factory createCommitterFac // a restart. return (user, metricGroup) -> new StoreCommitter( - table.newCommit(user) + table.newCommit(user, branch) .withOverwrite(overwritePartition) .ignoreEmptyCommit(!streamingCheckpointEnabled), metricGroup); diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/GlobalFullCompactionSinkWrite.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/GlobalFullCompactionSinkWrite.java index 3fb6fe6ae7751..a8adba607bf40 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/GlobalFullCompactionSinkWrite.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/GlobalFullCompactionSinkWrite.java @@ -75,7 +75,8 @@ public GlobalFullCompactionSinkWrite( int deltaCommits, boolean isStreaming, @Nullable MemorySegmentPool memoryPool, - MetricGroup metricGroup) { + MetricGroup metricGroup, + String branchName) { super( table, commitUser, @@ -85,7 +86,8 @@ public GlobalFullCompactionSinkWrite( waitCompaction, isStreaming, memoryPool, - metricGroup); + metricGroup, + branchName); this.deltaCommits = deltaCommits; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java index 766fb762a3446..2ad8fb069e8c3 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/MultiTablesStoreCompactOperator.java @@ -267,7 +267,8 @@ private StoreSinkWrite.Provider createWriteProvider( finalDeltaCommits, isStreaming, memoryPool, - metricGroup); + metricGroup, + branch); } } @@ -281,6 +282,7 @@ private StoreSinkWrite.Provider createWriteProvider( waitCompaction, isStreaming, memoryPool, - metricGroup); + metricGroup, + branch); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PrepareCommitOperator.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PrepareCommitOperator.java index 7dbab7f2dcdee..96fa6b3261b17 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PrepareCommitOperator.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/PrepareCommitOperator.java @@ -22,6 +22,7 @@ import org.apache.paimon.flink.memory.MemorySegmentAllocator; import org.apache.paimon.memory.MemorySegmentPool; import org.apache.paimon.options.Options; +import org.apache.paimon.utils.BranchManager; import org.apache.flink.runtime.memory.MemoryManager; import org.apache.flink.streaming.api.graph.StreamConfig; @@ -49,12 +50,18 @@ public abstract class PrepareCommitOperator extends AbstractStreamOpera @Nullable private transient MemorySegmentAllocator memoryAllocator; private final Options options; private boolean endOfInput = false; + protected String branch = BranchManager.DEFAULT_MAIN_BRANCH; public PrepareCommitOperator(Options options) { this.options = options; setChainingStrategy(ChainingStrategy.ALWAYS); } + public PrepareCommitOperator toBranch(String branch) { + this.branch = branch; + return this; + } + @Override public void setup( StreamTask containingTask, diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDynamicBucketSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDynamicBucketSink.java index bf6c70f0aa295..d72f64f4ad611 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDynamicBucketSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowDynamicBucketSink.java @@ -62,6 +62,6 @@ protected ChannelComputer> channelComputer2() { @Override protected OneInputStreamOperator, Committable> createWriteOperator( StoreSinkWrite.Provider writeProvider, String commitUser) { - return new DynamicBucketRowWriteOperator(table, writeProvider, commitUser); + return new DynamicBucketRowWriteOperator(table, writeProvider, commitUser).toBranch(branch); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowUnawareBucketSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowUnawareBucketSink.java index d7b8c76cfc40b..c0745c7a0a9bd 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowUnawareBucketSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/RowUnawareBucketSink.java @@ -40,6 +40,7 @@ public RowUnawareBucketSink( @Override protected OneInputStreamOperator createWriteOperator( StoreSinkWrite.Provider writeProvider, String commitUser) { - return new RowDataStoreWriteOperator(table, logSinkFunction, writeProvider, commitUser); + return new RowDataStoreWriteOperator(table, logSinkFunction, writeProvider, commitUser) + .toBranch(branch); } } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreMultiCommitter.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreMultiCommitter.java index a2d3f16ba0215..54ae4eee5520b 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreMultiCommitter.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreMultiCommitter.java @@ -25,6 +25,7 @@ import org.apache.paimon.manifest.WrappedManifestCommittable; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.sink.CommitMessage; +import org.apache.paimon.utils.BranchManager; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.metrics.groups.OperatorMetricGroup; @@ -60,11 +61,21 @@ public class StoreMultiCommitter // compact job needs set "write-only" of a table to false private final boolean isCompactJob; + private String branch; + public StoreMultiCommitter( Catalog.Loader catalogLoader, String commitUser, @Nullable OperatorMetricGroup flinkMetricGroup) { - this(catalogLoader, commitUser, flinkMetricGroup, false); + this(catalogLoader, commitUser, flinkMetricGroup, BranchManager.DEFAULT_MAIN_BRANCH); + } + + public StoreMultiCommitter( + Catalog.Loader catalogLoader, + String commitUser, + @Nullable OperatorMetricGroup flinkMetricGroup, + String branch) { + this(catalogLoader, commitUser, flinkMetricGroup, false, branch); } public StoreMultiCommitter( @@ -72,11 +83,26 @@ public StoreMultiCommitter( String commitUser, @Nullable OperatorMetricGroup flinkMetricGroup, boolean isCompactJob) { + this( + catalogLoader, + commitUser, + flinkMetricGroup, + isCompactJob, + BranchManager.DEFAULT_MAIN_BRANCH); + } + + public StoreMultiCommitter( + Catalog.Loader catalogLoader, + String commitUser, + @Nullable OperatorMetricGroup flinkMetricGroup, + boolean isCompactJob, + String branch) { this.catalog = catalogLoader.load(); this.commitUser = commitUser; this.flinkMetricGroup = flinkMetricGroup; this.tableCommitters = new HashMap<>(); this.isCompactJob = isCompactJob; + this.branch = branch; } @Override @@ -206,7 +232,7 @@ private StoreCommitter getStoreCommitter(Identifier tableId) { } committer = new StoreCommitter( - table.newCommit(commitUser).ignoreEmptyCommit(isCompactJob), + table.newCommit(commitUser, branch).ignoreEmptyCommit(isCompactJob), flinkMetricGroup); tableCommitters.put(tableId, committer); } diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreSinkWriteImpl.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreSinkWriteImpl.java index c70f6038ec962..cde51320d0aa3 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreSinkWriteImpl.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/StoreSinkWriteImpl.java @@ -32,6 +32,8 @@ import org.apache.paimon.table.sink.CommitMessage; import org.apache.paimon.table.sink.SinkRecord; import org.apache.paimon.table.sink.TableWriteImpl; +import org.apache.paimon.utils.BranchManager; +import org.apache.paimon.utils.StringUtils; import org.apache.flink.metrics.MetricGroup; import org.apache.flink.runtime.io.disk.iomanager.IOManager; @@ -65,6 +67,8 @@ public class StoreSinkWriteImpl implements StoreSinkWrite { @Nullable private final MetricGroup metricGroup; + private final String branchName; + public StoreSinkWriteImpl( FileStoreTable table, String commitUser, @@ -74,7 +78,8 @@ public StoreSinkWriteImpl( boolean waitCompaction, boolean isStreamingMode, @Nullable MemorySegmentPool memoryPool, - @Nullable MetricGroup metricGroup) { + @Nullable MetricGroup metricGroup, + String branchName) { this( table, commitUser, @@ -85,7 +90,8 @@ public StoreSinkWriteImpl( isStreamingMode, memoryPool, null, - metricGroup); + metricGroup, + branchName); } public StoreSinkWriteImpl( @@ -97,7 +103,8 @@ public StoreSinkWriteImpl( boolean waitCompaction, boolean isStreamingMode, MemoryPoolFactory memoryPoolFactory, - @Nullable MetricGroup metricGroup) { + @Nullable MetricGroup metricGroup, + String branchName) { this( table, commitUser, @@ -108,7 +115,8 @@ public StoreSinkWriteImpl( isStreamingMode, null, memoryPoolFactory, - metricGroup); + metricGroup, + branchName); } private StoreSinkWriteImpl( @@ -121,7 +129,8 @@ private StoreSinkWriteImpl( boolean isStreamingMode, @Nullable MemorySegmentPool memoryPool, @Nullable MemoryPoolFactory memoryPoolFactory, - @Nullable MetricGroup metricGroup) { + @Nullable MetricGroup metricGroup, + String branchName) { this.commitUser = commitUser; this.state = state; this.paimonIOManager = new IOManagerImpl(ioManager.getSpillingDirectoriesPaths()); @@ -131,6 +140,8 @@ private StoreSinkWriteImpl( this.memoryPool = memoryPool; this.memoryPoolFactory = memoryPoolFactory; this.metricGroup = metricGroup; + this.branchName = + StringUtils.isBlank(branchName) ? BranchManager.DEFAULT_MAIN_BRANCH : branchName; this.write = newTableWrite(table); } @@ -143,7 +154,8 @@ private TableWriteImpl newTableWrite(FileStoreTable table) { table.newWrite( commitUser, (part, bucket) -> - state.stateValueFilter().filter(table.name(), part, bucket)) + state.stateValueFilter().filter(table.name(), part, bucket), + branchName) .withIOManager(paimonIOManager) .withIgnorePreviousFiles(ignorePreviousFiles) .withExecutionMode(isStreamingMode) diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/UnawareBucketCompactionSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/UnawareBucketCompactionSink.java index cf825cec7ba41..ca192782439fc 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/UnawareBucketCompactionSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/UnawareBucketCompactionSink.java @@ -44,13 +44,13 @@ public static DataStreamSink sink( @Override protected OneInputStreamOperator createWriteOperator( StoreSinkWrite.Provider writeProvider, String commitUser) { - return new AppendOnlyTableCompactionWorkerOperator(table, commitUser); + return new AppendOnlyTableCompactionWorkerOperator(table, commitUser).toBranch(branch); } @Override protected Committer.Factory createCommitterFactory( boolean streamingCheckpointEnabled) { - return (s, metricGroup) -> new StoreCommitter(table.newCommit(s), metricGroup); + return (s, metricGroup) -> new StoreCommitter(table.newCommit(s, branch), metricGroup); } @Override diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/GlobalDynamicBucketSink.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/GlobalDynamicBucketSink.java index ccab48f0aac98..efc651556beb7 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/GlobalDynamicBucketSink.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/index/GlobalDynamicBucketSink.java @@ -64,7 +64,7 @@ public GlobalDynamicBucketSink( @Override protected OneInputStreamOperator, Committable> createWriteOperator( StoreSinkWrite.Provider writeProvider, String commitUser) { - return new DynamicBucketRowWriteOperator(table, writeProvider, commitUser); + return new DynamicBucketRowWriteOperator(table, writeProvider, commitUser).toBranch(branch); } public DataStreamSink build(DataStream input, @Nullable Integer parallelism) { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/DataTableSource.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/DataTableSource.java index fd29dd0482d6f..31237cbca6927 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/DataTableSource.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/DataTableSource.java @@ -243,8 +243,9 @@ private DataStream configureSource( parallelism, options.get(FlinkConnectorOptions.INFER_SCAN_MAX_PARALLELISM)); } + String branch = options.get(FlinkConnectorOptions.BRANCH); - return sourceBuilder.withParallelism(parallelism).withEnv(env).build(); + return sourceBuilder.withParallelism(parallelism).fromBranch(branch).withEnv(env).build(); } private void scanSplitsForInference() { diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java index d8878f360c6fd..25bd4fca722dc 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/source/FlinkSourceBuilder.java @@ -33,6 +33,7 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; import org.apache.paimon.table.source.ReadBuilder; +import org.apache.paimon.utils.StringUtils; import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -80,6 +81,7 @@ public class FlinkSourceBuilder { @Nullable private Long limit; @Nullable private WatermarkStrategy watermarkStrategy; @Nullable private DynamicPartitionFilteringInfo dynamicPartitionFilteringInfo; + private String branch; public FlinkSourceBuilder(ObjectIdentifier tableIdentifier, Table table) { this.tableIdentifier = tableIdentifier; @@ -128,6 +130,11 @@ public FlinkSourceBuilder withWatermarkStrategy( return this; } + public FlinkSourceBuilder fromBranch(String branch) { + this.branch = branch; + return this; + } + public FlinkSourceBuilder withDynamicPartitionFilteringFields( List dynamicPartitionFilteringFields) { if (dynamicPartitionFilteringFields != null && !dynamicPartitionFilteringFields.isEmpty()) { @@ -147,9 +154,14 @@ public FlinkSourceBuilder withDynamicPartitionFilteringFields( private ReadBuilder createReadBuilder() { ReadBuilder readBuilder = table.newReadBuilder().withProjection(projectedFields).withFilter(predicate); + if (limit != null) { readBuilder.withLimit(limit.intValue()); } + + if (!StringUtils.isBlank(branch)) { + readBuilder.fromBranch(branch); + } return readBuilder; } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreITCase.java index b7c739e96e68d..32b4bd30fe9c7 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreITCase.java @@ -33,6 +33,7 @@ import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.FileStoreTableFactory; import org.apache.paimon.utils.BlockingIterator; +import org.apache.paimon.utils.BranchManager; import org.apache.paimon.utils.FailingFileIO; import org.apache.flink.api.common.RuntimeExecutionMode; @@ -60,6 +61,7 @@ import org.apache.flink.types.Row; import org.apache.flink.types.RowKind; import org.apache.flink.util.CloseableIterator; +import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -122,11 +124,18 @@ public class FileStoreITCase extends AbstractTestBase { private final StreamExecutionEnvironment env; + protected static String branch; + public FileStoreITCase(boolean isBatch) { this.isBatch = isBatch; this.env = isBatch ? buildBatchEnv() : buildStreamEnv(); } + @BeforeAll + public static void before() { + branch = BranchManager.DEFAULT_MAIN_BRANCH; + } + @Parameters(name = "isBatch-{0}") public static List getVarSeg() { return Arrays.asList(true, false); @@ -141,12 +150,19 @@ public void testPartitioned() throws Exception { FileStoreTable table = buildFileStoreTable(new int[] {1}, new int[] {1, 2}); // write - new FlinkSinkBuilder(table).withInput(buildTestSource(env, isBatch)).build(); + new FlinkSinkBuilder(table) + .withInput(buildTestSource(env, isBatch)) + .toBranch(branch) + .build(); env.execute(); // read List results = - executeAndCollect(new FlinkSourceBuilder(IDENTIFIER, table).withEnv(env).build()); + executeAndCollect( + new FlinkSourceBuilder(IDENTIFIER, table) + .withEnv(env) + .fromBranch(branch) + .build()); // assert Row[] expected = @@ -161,12 +177,19 @@ public void testNonPartitioned() throws Exception { FileStoreTable table = buildFileStoreTable(new int[0], new int[] {2}); // write - new FlinkSinkBuilder(table).withInput(buildTestSource(env, isBatch)).build(); + new FlinkSinkBuilder(table) + .withInput(buildTestSource(env, isBatch)) + .toBranch(branch) + .build(); env.execute(); // read List results = - executeAndCollect(new FlinkSourceBuilder(IDENTIFIER, table).withEnv(env).build()); + executeAndCollect( + new FlinkSourceBuilder(IDENTIFIER, table) + .withEnv(env) + .fromBranch(branch) + .build()); // assert Row[] expected = new Row[] {Row.of(5, "p2", 1), Row.of(0, "p1", 2), Row.of(3, "p2", 5)}; @@ -180,7 +203,10 @@ public void testOverwrite() throws Exception { FileStoreTable table = buildFileStoreTable(new int[] {1}, new int[] {1, 2}); // write - new FlinkSinkBuilder(table).withInput(buildTestSource(env, isBatch)).build(); + new FlinkSinkBuilder(table) + .withInput(buildTestSource(env, isBatch)) + .toBranch(branch) + .build(); env.execute(); // overwrite p2 @@ -194,12 +220,17 @@ public void testOverwrite() throws Exception { new FlinkSinkBuilder(table) .withInput(partialData) .withOverwritePartition(overwrite) + .toBranch(branch) .build(); env.execute(); // read List results = - executeAndCollect(new FlinkSourceBuilder(IDENTIFIER, table).withEnv(env).build()); + executeAndCollect( + new FlinkSourceBuilder(IDENTIFIER, table) + .withEnv(env) + .fromBranch(branch) + .build()); Row[] expected = new Row[] {Row.of(9, "p2", 5), Row.of(5, "p1", 1), Row.of(0, "p1", 2)}; assertThat(results).containsExactlyInAnyOrder(expected); @@ -213,11 +244,17 @@ public void testOverwrite() throws Exception { new FlinkSinkBuilder(table) .withInput(partialData) .withOverwritePartition(new HashMap<>()) + .toBranch(branch) .build(); env.execute(); // read - results = executeAndCollect(new FlinkSourceBuilder(IDENTIFIER, table).withEnv(env).build()); + results = + executeAndCollect( + new FlinkSourceBuilder(IDENTIFIER, table) + .withEnv(env) + .fromBranch(branch) + .build()); expected = new Row[] {Row.of(19, "p2", 6), Row.of(5, "p1", 1), Row.of(0, "p1", 2)}; assertThat(results).containsExactlyInAnyOrder(expected); @@ -233,11 +270,17 @@ public void testOverwrite() throws Exception { CoreOptions.DYNAMIC_PARTITION_OVERWRITE.key(), "false"))) .withInput(partialData) .withOverwritePartition(new HashMap<>()) + .toBranch(branch) .build(); env.execute(); // read - results = executeAndCollect(new FlinkSourceBuilder(IDENTIFIER, table).withEnv(env).build()); + results = + executeAndCollect( + new FlinkSourceBuilder(IDENTIFIER, table) + .withEnv(env) + .fromBranch(branch) + .build()); expected = new Row[] {Row.of(20, "p2", 3)}; assertThat(results).containsExactlyInAnyOrder(expected); } @@ -247,12 +290,19 @@ public void testPartitionedNonKey() throws Exception { FileStoreTable table = buildFileStoreTable(new int[] {1}, new int[0]); // write - new FlinkSinkBuilder(table).withInput(buildTestSource(env, isBatch)).build(); + new FlinkSinkBuilder(table) + .withInput(buildTestSource(env, isBatch)) + .toBranch(branch) + .build(); env.execute(); // read List results = - executeAndCollect(new FlinkSourceBuilder(IDENTIFIER, table).withEnv(env).build()); + executeAndCollect( + new FlinkSourceBuilder(IDENTIFIER, table) + .withEnv(env) + .fromBranch(branch) + .build()); // assert // in streaming mode, expect origin data X 2 (FiniteTestSource) @@ -276,7 +326,10 @@ public void testNonKeyedProjection() throws Exception { private void testProjection(FileStoreTable table) throws Exception { // write - new FlinkSinkBuilder(table).withInput(buildTestSource(env, isBatch)).build(); + new FlinkSinkBuilder(table) + .withInput(buildTestSource(env, isBatch)) + .toBranch(branch) + .build(); env.execute(); // read @@ -292,6 +345,7 @@ private void testProjection(FileStoreTable table) throws Exception { new FlinkSourceBuilder(IDENTIFIER, table) .withProjection(projection.toNestedIndexes()) .withEnv(env) + .fromBranch(branch) .build(), converter); @@ -332,6 +386,7 @@ public void testContinuousBounded() throws Exception { new FlinkSourceBuilder(IDENTIFIER, table) .withContinuousMode(true) .withEnv(env) + .fromBranch(branch) .build(); Transformation transformation = source.getTransformation(); assertThat(transformation).isInstanceOf(SourceTransformation.class); @@ -347,6 +402,7 @@ private void innerTestContinuous(FileStoreTable table) throws Exception { new FlinkSourceBuilder(IDENTIFIER, table) .withContinuousMode(true) .withEnv(env) + .fromBranch(branch) .build() .executeAndCollect(), CONVERTER::toExternal); @@ -383,7 +439,7 @@ private void sinkAndValidate( } DataStreamSource source = env.addSource(new FiniteTestSource<>(src, true), InternalTypeInfo.of(TABLE_TYPE)); - new FlinkSinkBuilder(table).withInput(source).build(); + new FlinkSinkBuilder(table).withInput(source).toBranch(branch).build(); env.execute(); assertThat(iterator.collect(expected.length)).containsExactlyInAnyOrder(expected); } @@ -429,8 +485,8 @@ public static FileStoreTable buildFileStoreTable( ""); return retryArtificialException( () -> { - new SchemaManager(LocalFileIO.create(), tablePath).createTable(schema); - return FileStoreTableFactory.create(LocalFileIO.create(), options); + new SchemaManager(LocalFileIO.create(), tablePath).createTable(schema, branch); + return FileStoreTableFactory.create(LocalFileIO.create(), options, branch); }); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreWithBranchITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreWithBranchITCase.java new file mode 100644 index 0000000000000..8a2374b3c367b --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/FileStoreWithBranchITCase.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink; + +import org.apache.paimon.flink.sink.FixedBucketSink; +import org.apache.paimon.flink.source.ContinuousFileStoreSource; +import org.apache.paimon.flink.source.StaticFileStoreSource; + +import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.extension.ExtendWith; + +/** + * ITCase for {@link StaticFileStoreSource}, {@link ContinuousFileStoreSource} and {@link + * FixedBucketSink}. + */ +@ExtendWith(ParameterizedTestExtension.class) +public class FileStoreWithBranchITCase extends FileStoreITCase { + public FileStoreWithBranchITCase(boolean isBatch) { + super(isBatch); + } + + @BeforeAll + public static void before() { + branch = "testBranch"; + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/ActionITCaseBase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/ActionITCaseBase.java index 80cf2cc767119..54827158f96b2 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/ActionITCaseBase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/action/ActionITCaseBase.java @@ -27,6 +27,7 @@ import org.apache.paimon.data.InternalRow; import org.apache.paimon.flink.util.AbstractTestBase; import org.apache.paimon.fs.Path; +import org.apache.paimon.options.Options; import org.apache.paimon.reader.RecordReader; import org.apache.paimon.schema.Schema; import org.apache.paimon.table.FileStoreTable; @@ -35,6 +36,7 @@ import org.apache.paimon.table.source.Split; import org.apache.paimon.table.source.TableRead; import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.BranchManager; import org.apache.flink.api.common.RuntimeExecutionMode; import org.apache.flink.api.common.restartstrategy.RestartStrategies; @@ -57,6 +59,9 @@ import java.util.UUID; import java.util.concurrent.ThreadLocalRandom; +import static org.apache.paimon.options.CatalogOptions.BRANCH; +import static org.apache.paimon.options.CatalogOptions.WAREHOUSE; + /** {@link Action} test base. */ public abstract class ActionITCaseBase extends AbstractTestBase { @@ -67,7 +72,8 @@ public abstract class ActionITCaseBase extends AbstractTestBase { protected StreamTableWrite write; protected StreamTableCommit commit; protected Catalog catalog; - private long incrementalIdentifier; + protected long incrementalIdentifier; + protected String branch = BranchManager.DEFAULT_MAIN_BRANCH; @BeforeEach public void before() throws IOException { @@ -76,7 +82,12 @@ public void before() throws IOException { tableName = "test_table_" + UUID.randomUUID(); commitUser = UUID.randomUUID().toString(); incrementalIdentifier = 0; - catalog = CatalogFactory.createCatalog(CatalogContext.create(new Path(warehouse))); + Map options = new HashMap<>(); + options.put(WAREHOUSE.key(), new Path(warehouse).toUri().toString()); + if (!branch.equals(BranchManager.DEFAULT_MAIN_BRANCH)) { + options.put(BRANCH.key(), branch); + } + catalog = CatalogFactory.createCatalog(CatalogContext.create(Options.fromMap(options))); } @AfterEach diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactorSinkITCase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactorSinkITCase.java index 68c3c42b0a502..3402b7dae335c 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactorSinkITCase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/CompactorSinkITCase.java @@ -45,6 +45,7 @@ import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.BranchManager; import org.apache.paimon.utils.SnapshotManager; import org.apache.flink.api.common.ExecutionConfig; @@ -255,7 +256,8 @@ protected StoreCompactOperator createCompactOperator(FileStoreTable table) { false, false, memoryPool, - metricGroup), + metricGroup, + BranchManager.DEFAULT_MAIN_BRANCH), "test"); } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/FlinkSinkTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/FlinkSinkTest.java index 23b983b062380..9047d80e669d5 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/FlinkSinkTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/FlinkSinkTest.java @@ -36,6 +36,7 @@ import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.BranchManager; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.RuntimeExecutionMode; @@ -176,7 +177,8 @@ protected RowDataStoreWriteOperator createWriteOperator(FileStoreTable table) { false, true, memoryPool, - metricGroup), + metricGroup, + BranchManager.DEFAULT_MAIN_BRANCH), "test"); } @@ -193,7 +195,8 @@ protected DynamicBucketRowWriteOperator createDynamicBucketWriteOperator(FileSto false, true, memoryPool, - metricGroup), + metricGroup, + BranchManager.DEFAULT_MAIN_BRANCH), "test"); } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterOperatorTestBase.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterOperatorTestBase.java index 1fc6e8e615f91..204a1c44cbe00 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterOperatorTestBase.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sink/WriterOperatorTestBase.java @@ -35,6 +35,7 @@ import org.apache.paimon.types.DataType; import org.apache.paimon.types.DataTypes; import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.BranchManager; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.TypeSerializer; @@ -139,7 +140,8 @@ private static RowDataStoreWriteOperator getRowDataStoreWriteOperator( false, true, memoryPool, - metricGroup); + metricGroup, + BranchManager.DEFAULT_MAIN_BRANCH); RowDataStoreWriteOperator operator = new RowDataStoreWriteOperator(fileStoreTable, null, provider, "test"); return operator; diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceReaderFromBranchTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceReaderFromBranchTest.java new file mode 100644 index 0000000000000..076813587e59e --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceReaderFromBranchTest.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.source; + +import org.junit.jupiter.api.BeforeEach; + +import java.util.UUID; + +/** Unit tests for the {@link FileStoreSourceReader}. */ +public class FileStoreSourceReaderFromBranchTest extends FileStoreSourceReaderTest { + + @BeforeEach + public void beforeEach() throws Exception { + branch = "testBranch-" + UUID.randomUUID(); + super.beforeEach(); + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceReaderTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceReaderTest.java index 882763cf74da2..64d3da75626d4 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceReaderTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceReaderTest.java @@ -28,6 +28,7 @@ import org.apache.paimon.types.DataField; import org.apache.paimon.types.IntType; import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.BranchManager; import org.apache.flink.api.connector.source.SourceEvent; import org.apache.flink.connector.testutils.source.reader.TestingReaderContext; @@ -57,6 +58,8 @@ public class FileStoreSourceReaderTest { @TempDir protected java.nio.file.Path tempDir; + protected String branch = BranchManager.DEFAULT_MAIN_BRANCH; + @BeforeEach public void beforeEach() throws Exception { SchemaManager schemaManager = @@ -72,7 +75,8 @@ public void beforeEach() throws Exception { Collections.singletonList("default"), Arrays.asList("k", "default"), Collections.emptyMap(), - null)); + null), + branch); } @Test @@ -138,7 +142,7 @@ public void testReaderOnSplitFinished() throws Exception { protected FileStoreSourceReader createReader(TestingReaderContext context) { return new FileStoreSourceReader( context, - new TestChangelogDataReadWrite(tempDir.toString()).createReadWithKey(), + new TestChangelogDataReadWrite(tempDir.toString()).createReadWithKey(branch), new FileStoreSourceReaderMetrics(new DummyMetricGroup()), IOManager.create(tempDir.toString()), null); diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceSplitReaderFromBranchTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceSplitReaderFromBranchTest.java new file mode 100644 index 0000000000000..92ac692df101f --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceSplitReaderFromBranchTest.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.source; + +import org.junit.jupiter.api.BeforeEach; + +import java.util.UUID; + +/** Test for {@link FileStoreSourceSplitReader}. */ +public class FileStoreSourceSplitReaderFromBranchTest extends FileStoreSourceSplitReaderTest { + + @BeforeEach + public void beforeEach() throws Exception { + branch = "testBranch-" + UUID.randomUUID(); + super.beforeEach(); + } +} diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceSplitReaderTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceSplitReaderTest.java index 67b3c3e603f84..8c7b9b586fca1 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceSplitReaderTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/FileStoreSourceSplitReaderTest.java @@ -28,13 +28,13 @@ import org.apache.paimon.schema.Schema; import org.apache.paimon.schema.SchemaManager; import org.apache.paimon.table.source.TableRead; +import org.apache.paimon.utils.BranchManager; import org.apache.paimon.utils.RecordWriter; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; -import org.apache.flink.connector.file.src.reader.BulkFormat; import org.apache.flink.connector.file.src.reader.BulkFormat.RecordIterator; import org.apache.flink.connector.file.src.util.RecordAndPosition; import org.apache.flink.table.data.RowData; @@ -67,6 +67,8 @@ public class FileStoreSourceSplitReaderTest { @TempDir java.nio.file.Path tempDir; + protected String branch = BranchManager.DEFAULT_MAIN_BRANCH; + @BeforeEach public void beforeEach() throws Exception { SchemaManager schemaManager = @@ -84,7 +86,8 @@ public void beforeEach() throws Exception { Collections.singletonList("default"), Arrays.asList("k", "default"), Collections.emptyMap(), - null)); + null), + branch); } @Test @@ -106,10 +109,10 @@ private FileStoreSourceSplitReader createReader(TableRead tableRead, @Nullable L private void innerTestOnce(int skip) throws Exception { TestChangelogDataReadWrite rw = new TestChangelogDataReadWrite(tempDir.toString()); - FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(), null); + FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(branch), null); List> input = kvs(); - List files = rw.writeFiles(row(1), 0, input); + List files = rw.writeFiles(row(1), 0, input, branch); assignSplit(reader, newSourceSplit("id1", row(1), 0, files, skip)); @@ -132,10 +135,10 @@ private void innerTestOnce(int skip) throws Exception { @Test public void testPrimaryKeyWithDelete() throws Exception { TestChangelogDataReadWrite rw = new TestChangelogDataReadWrite(tempDir.toString()); - FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(), null); + FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(branch), null); List> input = kvs(); - RecordWriter writer = rw.createMergeTreeWriter(row(1), 0); + RecordWriter writer = rw.createMergeTreeWriter(row(1), 0, branch); for (Tuple2 tuple2 : input) { writer.write( new KeyValue() @@ -154,7 +157,7 @@ public void testPrimaryKeyWithDelete() throws Exception { writer.close(); assignSplit(reader, newSourceSplit("id1", row(1), 0, files, true)); - RecordsWithSplitIds> records = reader.fetch(); + RecordsWithSplitIds> records = reader.fetch(); List> expected = input.stream() @@ -174,18 +177,18 @@ public void testPrimaryKeyWithDelete() throws Exception { @Test public void testMultipleBatchInSplit() throws Exception { TestChangelogDataReadWrite rw = new TestChangelogDataReadWrite(tempDir.toString()); - FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(), null); + FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(branch), null); List> input1 = kvs(); - List files = rw.writeFiles(row(1), 0, input1); + List files = rw.writeFiles(row(1), 0, input1, branch); List> input2 = kvs(6); - List files2 = rw.writeFiles(row(1), 0, input2); + List files2 = rw.writeFiles(row(1), 0, input2, branch); files.addAll(files2); assignSplit(reader, newSourceSplit("id1", row(1), 0, files)); - RecordsWithSplitIds> records = reader.fetch(); + RecordsWithSplitIds> records = reader.fetch(); assertRecords( records, null, @@ -210,14 +213,14 @@ public void testMultipleBatchInSplit() throws Exception { @Test public void testRestore() throws Exception { TestChangelogDataReadWrite rw = new TestChangelogDataReadWrite(tempDir.toString()); - FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(), null); + FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(branch), null); List> input = kvs(); - List files = rw.writeFiles(row(1), 0, input); + List files = rw.writeFiles(row(1), 0, input, branch); assignSplit(reader, newSourceSplit("id1", row(1), 0, files, 3)); - RecordsWithSplitIds> records = reader.fetch(); + RecordsWithSplitIds> records = reader.fetch(); assertRecords( records, null, @@ -236,18 +239,18 @@ public void testRestore() throws Exception { @Test public void testRestoreMultipleBatchInSplit() throws Exception { TestChangelogDataReadWrite rw = new TestChangelogDataReadWrite(tempDir.toString()); - FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(), null); + FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(branch), null); List> input1 = kvs(); - List files = rw.writeFiles(row(1), 0, input1); + List files = rw.writeFiles(row(1), 0, input1, branch); List> input2 = kvs(6); - List files2 = rw.writeFiles(row(1), 0, input2); + List files2 = rw.writeFiles(row(1), 0, input2, branch); files.addAll(files2); assignSplit(reader, newSourceSplit("id1", row(1), 0, files, 7)); - RecordsWithSplitIds> records = reader.fetch(); + RecordsWithSplitIds> records = reader.fetch(); assertRecords( records, null, @@ -267,17 +270,17 @@ public void testRestoreMultipleBatchInSplit() throws Exception { @Test public void testMultipleSplits() throws Exception { TestChangelogDataReadWrite rw = new TestChangelogDataReadWrite(tempDir.toString()); - FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(), null); + FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(branch), null); List> input1 = kvs(); - List files1 = rw.writeFiles(row(1), 0, input1); + List files1 = rw.writeFiles(row(1), 0, input1, branch); assignSplit(reader, newSourceSplit("id1", row(1), 0, files1)); List> input2 = kvs(); - List files2 = rw.writeFiles(row(2), 1, input2); + List files2 = rw.writeFiles(row(2), 1, input2, branch); assignSplit(reader, newSourceSplit("id2", row(2), 1, files2)); - RecordsWithSplitIds> records = reader.fetch(); + RecordsWithSplitIds> records = reader.fetch(); assertRecords( records, null, @@ -305,7 +308,7 @@ public void testMultipleSplits() throws Exception { @Test public void testNoSplit() throws Exception { TestChangelogDataReadWrite rw = new TestChangelogDataReadWrite(tempDir.toString()); - FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(), null); + FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(branch), null); assertThatThrownBy(reader::fetch).hasMessageContaining("no split remaining"); reader.close(); } @@ -313,14 +316,14 @@ public void testNoSplit() throws Exception { @Test public void testLimit() throws Exception { TestChangelogDataReadWrite rw = new TestChangelogDataReadWrite(tempDir.toString()); - FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(), 2L); + FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(branch), 2L); List> input = kvs(); - List files = rw.writeFiles(row(1), 0, input); + List files = rw.writeFiles(row(1), 0, input, branch); assignSplit(reader, newSourceSplit("id1", row(1), 0, files, 0)); - RecordsWithSplitIds> records = reader.fetch(); + RecordsWithSplitIds> records = reader.fetch(); List> expected = input.stream() @@ -346,19 +349,19 @@ public void testLimit() throws Exception { @Test public void testPauseOrResumeSplits() throws Exception { TestChangelogDataReadWrite rw = new TestChangelogDataReadWrite(tempDir.toString()); - FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(), null); + FileStoreSourceSplitReader reader = createReader(rw.createReadWithKey(branch), null); List> input1 = kvs(); - List files = rw.writeFiles(row(1), 0, input1); + List files = rw.writeFiles(row(1), 0, input1, branch); List> input2 = kvs(6); - List files2 = rw.writeFiles(row(1), 0, input2); + List files2 = rw.writeFiles(row(1), 0, input2, branch); files.addAll(files2); FileStoreSourceSplit split1 = newSourceSplit("id1", row(1), 0, files); assignSplit(reader, split1); - RecordsWithSplitIds> records = reader.fetch(); + RecordsWithSplitIds> records = reader.fetch(); assertRecords( records, null, @@ -373,7 +376,7 @@ public void testPauseOrResumeSplits() throws Exception { // assign next split List> input3 = kvs(12); - List files3 = rw.writeFiles(row(1), 0, input3); + List files3 = rw.writeFiles(row(1), 0, input3, branch); FileStoreSourceSplit split2 = newSourceSplit("id2", row(1), 0, files3); assignSplit(reader, split2); @@ -408,7 +411,7 @@ public void testPauseOrResumeSplits() throws Exception { } private void assertRecords( - RecordsWithSplitIds> records, + RecordsWithSplitIds> records, String finishedSplit, String nextSplit, long startRecordSkipCount, diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java index 7ffa948094544..2cdba3d51bd16 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/TestChangelogDataReadWrite.java @@ -47,6 +47,7 @@ import org.apache.paimon.types.IntType; import org.apache.paimon.types.RowKind; import org.apache.paimon.types.RowType; +import org.apache.paimon.utils.BranchManager; import org.apache.paimon.utils.FileStorePathFactory; import org.apache.paimon.utils.RecordWriter; import org.apache.paimon.utils.SnapshotManager; @@ -110,14 +111,20 @@ public TestChangelogDataReadWrite(String root) { } public TableRead createReadWithKey() { - return createRead(ValueContentRowDataRecordIterator::new); + return createRead( + ValueContentRowDataRecordIterator::new, BranchManager.DEFAULT_MAIN_BRANCH); + } + + public TableRead createReadWithKey(String branch) { + return createRead(ValueContentRowDataRecordIterator::new, branch); } private TableRead createRead( Function< RecordReader.RecordIterator, RecordReader.RecordIterator> - rowDataIteratorCreator) { + rowDataIteratorCreator, + String branch) { SchemaManager schemaManager = new SchemaManager(LocalFileIO.create(), tablePath); long schemaId = 0; CoreOptions options = new CoreOptions(new HashMap<>()); @@ -139,7 +146,9 @@ private TableRead createRead( ignore -> avro, pathFactory, EXTRACTOR, - options)); + options, + branch), + branch); return new KeyValueTableRead(read, null) { @Override @@ -157,7 +166,13 @@ protected RecordReader.RecordIterator rowDataRecordIteratorFromKv( public List writeFiles( BinaryRow partition, int bucket, List> kvs) throws Exception { - RecordWriter writer = createMergeTreeWriter(partition, bucket); + return writeFiles(partition, bucket, kvs, BranchManager.DEFAULT_MAIN_BRANCH); + } + + public List writeFiles( + BinaryRow partition, int bucket, List> kvs, String branch) + throws Exception { + RecordWriter writer = createMergeTreeWriter(partition, bucket, branch); for (Tuple2 tuple2 : kvs) { writer.write( new KeyValue() @@ -172,6 +187,11 @@ public List writeFiles( } public RecordWriter createMergeTreeWriter(BinaryRow partition, int bucket) { + return createMergeTreeWriter(partition, bucket, BranchManager.DEFAULT_MAIN_BRANCH); + } + + public RecordWriter createMergeTreeWriter( + BinaryRow partition, int bucket, String branch) { CoreOptions options = new CoreOptions(Collections.singletonMap(CoreOptions.FILE_FORMAT.key(), "avro")); @@ -197,7 +217,8 @@ public RecordWriter createMergeTreeWriter(BinaryRow partition, int buc null, options, EXTRACTOR, - tablePath.getName()) + tablePath.getName(), + branch) .createWriterContainer(partition, bucket, true) .writer; ((MemoryOwner) writer) diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/align/AlignedSourceReaderWithBranchTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/align/AlignedSourceReaderWithBranchTest.java new file mode 100644 index 0000000000000..15bc68241c472 --- /dev/null +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/source/align/AlignedSourceReaderWithBranchTest.java @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.paimon.flink.source.align; + +import org.apache.paimon.disk.IOManager; +import org.apache.paimon.flink.source.FileStoreSourceReader; +import org.apache.paimon.flink.source.FileStoreSourceReaderTest; +import org.apache.paimon.flink.source.TestChangelogDataReadWrite; +import org.apache.paimon.flink.source.metrics.FileStoreSourceReaderMetrics; + +import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue; +import org.apache.flink.connector.testutils.source.reader.TestingReaderContext; +import org.apache.flink.connector.testutils.source.reader.TestingReaderOutput; +import org.apache.flink.table.data.RowData; +import org.junit.Ignore; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Optional; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Unit tests for the {@link AlignedSourceReader}. */ +public class AlignedSourceReaderWithBranchTest extends FileStoreSourceReaderTest { + + @Override + @Test + public void testAddMultipleSplits() throws Exception { + final TestingReaderContext context = new TestingReaderContext(); + final AlignedSourceReader reader = (AlignedSourceReader) createReader(context); + + reader.start(); + assertThat(context.getNumSplitRequests()).isEqualTo(1); + + reader.addSplits(Arrays.asList(createTestFileSplit("id1"), createTestFileSplit("id2"))); + TestingReaderOutput output = new TestingReaderOutput<>(); + while (reader.getNumberOfCurrentlyAssignedSplits() > 0) { + reader.pollNext(output); + Thread.sleep(10); + } + // splits are only requested when a checkpoint is ready to be triggered + assertThat(context.getNumSplitRequests()).isEqualTo(1); + + // prepare to trigger checkpoint + reader.handleSourceEvents(new CheckpointEvent(1L)); + assertThat(reader.shouldTriggerCheckpoint()).isEqualTo(Optional.of(1L)); + assertThat(context.getNumSplitRequests()).isEqualTo(2); + } + + @Override + @Ignore + public void testReaderOnSplitFinished() throws Exception { + // ignore + } + + @Override + protected FileStoreSourceReader createReader(TestingReaderContext context) { + return new AlignedSourceReader( + context, + new TestChangelogDataReadWrite(tempDir.toString()).createReadWithKey(branch), + new FileStoreSourceReaderMetrics(new DummyMetricGroup()), + IOManager.create(tempDir.toString()), + null, + new FutureCompletingBlockingQueue<>(2)); + } +} diff --git a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java index 589e920370e02..3b48a844c24b1 100644 --- a/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java +++ b/paimon-hive/paimon-hive-catalog/src/main/java/org/apache/paimon/hive/HiveCatalog.java @@ -330,7 +330,7 @@ public TableSchema getDataTableSchema(Identifier identifier) throws TableNotExis } Path tableLocation = getDataTableLocation(identifier); return new SchemaManager(fileIO, tableLocation) - .latest() + .latest(branchName) .orElseThrow( () -> new RuntimeException("There is no paimon table in " + tableLocation)); } @@ -373,7 +373,7 @@ protected void createTableImpl(Identifier identifier, Schema schema) { // if changes on Hive fails there is no harm to perform the same changes to files again TableSchema tableSchema; try { - tableSchema = schemaManager(identifier).createTable(schema); + tableSchema = schemaManager(identifier).createTable(schema, branchName); } catch (Exception e) { throw new RuntimeException( "Failed to commit changes of table " @@ -440,7 +440,7 @@ protected void alterTableImpl(Identifier identifier, List changes) final SchemaManager schemaManager = schemaManager(identifier); // first commit changes to underlying files - TableSchema schema = schemaManager.commitChanges(changes); + TableSchema schema = schemaManager.commitChanges(branchName, changes); try { // sync to hive hms @@ -450,7 +450,7 @@ protected void alterTableImpl(Identifier identifier, List changes) client.alter_table( identifier.getDatabaseName(), identifier.getObjectName(), table, true); } catch (Exception te) { - schemaManager.deleteSchema(schema.id()); + schemaManager.deleteSchema(branchName, schema.id()); throw new RuntimeException(te); } }