Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[clone]refactor the clone action as we introduced external path #4844

Merged
merged 11 commits into from
Feb 12, 2025
Merged
Changes from 5 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
22 changes: 22 additions & 0 deletions paimon-core/src/main/java/org/apache/paimon/io/DataFileMeta.java
Original file line number Diff line number Diff line change
@@ -484,6 +484,28 @@ public DataFileMeta copy(List<String> newExtraFiles) {
externalPath);
}

public DataFileMeta newExternalPath(String newExternalPath) {
return new DataFileMeta(
fileName,
fileSize,
rowCount,
minKey,
maxKey,
keyStats,
valueStats,
minSequenceNumber,
maxSequenceNumber,
schemaId,
level,
extraFiles,
creationTime,
deleteRowCount,
embeddedIndex,
fileSource,
valueStatsCols,
newExternalPath);
}

public DataFileMeta copy(byte[] newEmbeddedIndex) {
return new DataFileMeta(
fileName,
Original file line number Diff line number Diff line change
@@ -116,7 +116,12 @@ public RollingFileWriter<ManifestEntry, ManifestFileMeta> createRollingWriter()
suggestedFileSize);
}

private class ManifestEntryWriter extends SingleFileWriter<ManifestEntry, ManifestFileMeta> {
public ManifestEntryWriter createManifestEntryWriter(Path manifestPath) {
return new ManifestEntryWriter(writerFactory, manifestPath, compression);
}

/** Writer for {@link ManifestEntry}. */
public class ManifestEntryWriter extends SingleFileWriter<ManifestEntry, ManifestFileMeta> {

private final SimpleStatsCollector partitionStatsCollector;
private final SimpleStatsConverter partitionStatsSerializer;
Original file line number Diff line number Diff line change
@@ -21,7 +21,10 @@
import org.apache.paimon.flink.clone.CloneFileInfo;
import org.apache.paimon.flink.clone.CloneSourceBuilder;
import org.apache.paimon.flink.clone.CopyFileOperator;
import org.apache.paimon.flink.clone.PickFilesForCloneOperator;
import org.apache.paimon.flink.clone.CopyManifestFileOperator;
import org.apache.paimon.flink.clone.PickDataFileForCloneOperator;
import org.apache.paimon.flink.clone.PickManifestFileForCloneOperator;
import org.apache.paimon.flink.clone.PickSchemaFilesForCloneOperator;
import org.apache.paimon.flink.clone.SnapshotHintChannelComputer;
import org.apache.paimon.flink.clone.SnapshotHintOperator;
import org.apache.paimon.flink.sink.FlinkStreamPartitioner;
@@ -105,20 +108,55 @@ private void buildCloneFlinkJob(StreamExecutionEnvironment env) throws Exception
targetTableName)
.build();

SingleOutputStreamOperator<CloneFileInfo> pickFilesForClone =
SingleOutputStreamOperator<CloneFileInfo> pickSchemaFilesForClone =
cloneSource
.transform(
"Pick Files",
"Pick Schema Files",
TypeInformation.of(CloneFileInfo.class),
new PickFilesForCloneOperator(
new PickSchemaFilesForCloneOperator(
sourceCatalogConfig, targetCatalogConfig))
.forceNonParallel();
.setParallelism(1);

SingleOutputStreamOperator<CloneFileInfo> copySchemaFile =
pickSchemaFilesForClone
.transform(
"Copy Schema Files",
TypeInformation.of(CloneFileInfo.class),
new CopyFileOperator(sourceCatalogConfig, targetCatalogConfig))
.setParallelism(1);

SingleOutputStreamOperator<CloneFileInfo> pickManifestFile =
copySchemaFile
.transform(
"Pick Manifest Files",
TypeInformation.of(CloneFileInfo.class),
new PickManifestFileForCloneOperator(
sourceCatalogConfig, targetCatalogConfig))
.setParallelism(1);

SingleOutputStreamOperator<CloneFileInfo> copyManifestFile =
pickManifestFile
.transform(
"Copy Manifest Files",
TypeInformation.of(CloneFileInfo.class),
new CopyManifestFileOperator(
sourceCatalogConfig, targetCatalogConfig))
.setParallelism(parallelism);

SingleOutputStreamOperator<CloneFileInfo> pickDataFile =
copyManifestFile
.transform(
"Pick Data Files",
TypeInformation.of(CloneFileInfo.class),
new PickDataFileForCloneOperator(
sourceCatalogConfig, targetCatalogConfig))
.setParallelism(1);

SingleOutputStreamOperator<CloneFileInfo> copyFiles =
pickFilesForClone
.rebalance()
pickDataFile
.keyBy(CloneFileInfo::getSourceIdentifier) // key by source identifier
.transform(
"Copy Files",
"Copy Data Files",
TypeInformation.of(CloneFileInfo.class),
new CopyFileOperator(sourceCatalogConfig, targetCatalogConfig))
.setParallelism(parallelism);
Original file line number Diff line number Diff line change
@@ -18,29 +18,35 @@

package org.apache.paimon.flink.clone;

import javax.annotation.Nullable;

/** The information of copy file. */
public class CloneFileInfo {

private final String sourceFilePath;
private final String filePathExcludeTableRoot;
@Nullable private final String sourceFilePath;
@Nullable private final String filePathExcludeTableRoot;
private final String sourceIdentifier;
private final String targetIdentifier;
private final long snapshotId;

public CloneFileInfo(
String sourceFilePath,
String filePathExcludeTableRoot,
@Nullable String sourceFilePath,
@Nullable String filePathExcludeTableRoot,
String sourceIdentifier,
String targetIdentifier) {
String targetIdentifier,
long snapshotId) {
this.sourceFilePath = sourceFilePath;
this.filePathExcludeTableRoot = filePathExcludeTableRoot;
this.sourceIdentifier = sourceIdentifier;
this.targetIdentifier = targetIdentifier;
this.snapshotId = snapshotId;
}

@Nullable
public String getSourceFilePath() {
return sourceFilePath;
}

@Nullable
public String getFilePathExcludeTableRoot() {
return filePathExcludeTableRoot;
}
@@ -53,10 +59,19 @@ public String getTargetIdentifier() {
return targetIdentifier;
}

public long getSnapshotId() {
return snapshotId;
}

@Override
public String toString() {
return String.format(
"{ sourceFilePath: %s, filePathExcludeTableRoot: %s, sourceIdentifier: %s, targetIdentifier: %s }",
sourceFilePath, filePathExcludeTableRoot, sourceIdentifier, targetIdentifier);
"{ sourceFilePath: %s, filePathExcludeTableRoot: %s, sourceIdentifier: %s, targetIdentifier: %s, "
+ "snapshotId: %d}",
sourceFilePath,
filePathExcludeTableRoot,
sourceIdentifier,
targetIdentifier,
snapshotId);
}
}
Loading