jotarada commented on code in PR #10024: URL: https://github.com/apache/iceberg/pull/10024#discussion_r1537942005
########## spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/BaseCopyTableSparkAction.java: ########## @@ -0,0 +1,871 @@ +/* + * 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.iceberg.spark.actions; + +import java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.ManifestEntry; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.ManifestLists; +import org.apache.iceberg.ManifestReader; +import org.apache.iceberg.ManifestWriter; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StaticTableOperations; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableMetadata.MetadataLogEntry; +import org.apache.iceberg.TableMetadataParser; +import org.apache.iceberg.TableMetadataUtil; +import org.apache.iceberg.actions.BaseCopyTableActionResult; +import org.apache.iceberg.actions.CopyTable; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.avro.DataReader; +import org.apache.iceberg.data.avro.DataWriter; +import org.apache.iceberg.data.orc.GenericOrcReader; +import org.apache.iceberg.data.orc.GenericOrcWriter; +import org.apache.iceberg.data.parquet.GenericParquetReaders; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.deletes.PositionDeleteWriter; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.DeleteSchemaUtil; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.spark.JobGroupInfo; +import org.apache.spark.api.java.function.MapPartitionsFunction; +import org.apache.spark.broadcast.Broadcast; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoder; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.functions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class BaseCopyTableSparkAction extends BaseSparkAction<CopyTable> implements CopyTable { + + private static final Logger LOG = LoggerFactory.getLogger(BaseCopyTableSparkAction.class); + private static final String DATA_FILE_LIST_DIR = "data-file-list-to-move"; + private static final String METADATA_FILE_LIST_DIR = "metadata-file-list-to-move"; + + private final Table table; + private final Set<String> metadataFilesToMove = Collections.synchronizedSet(Sets.newHashSet()); + private final Set<String> manifestFilePaths = Collections.synchronizedSet(Sets.newHashSet()); + private final Set<ManifestFile> manifestFilesToRewrite = + Collections.synchronizedSet(Sets.newHashSet()); + private String dataFileListPath = null; + private String metadataFileListPath = null; + private final boolean enabledPME; + + private String sourcePrefix = ""; + private String targetPrefix = ""; + private String startVersion = ""; + private String endVersion = ""; + private String stagingDir = ""; + private Table targetTable = null; + + private Table startStaticTable = null; + private Table endStaticTable = null; + + public BaseCopyTableSparkAction(SparkSession spark, Table table) { + super(spark); + this.table = table; + enabledPME = table.properties().containsKey("parquet.encryption.footer.key"); + } + + @Override + public CopyTable rewriteLocationPrefix(String sPrefix, String tPrefix) { + Preconditions.checkArgument( + sPrefix != null && !sPrefix.isEmpty(), "Source prefix('%s') cannot be empty.", sPrefix); + this.sourcePrefix = sPrefix; + + if (tPrefix != null) { + this.targetPrefix = tPrefix; + } + return this; + } + + @Override + public CopyTable lastCopiedVersion(String sVersion) { + Preconditions.checkArgument( + sVersion != null && !sVersion.trim().isEmpty(), + "Last copied version('%s') cannot be empty.", + sVersion); + this.startVersion = sVersion; + return this; + } + + @Override + public CopyTable endVersion(String eVersion) { + Preconditions.checkArgument( + eVersion != null && !eVersion.trim().isEmpty(), + "End version('%s') cannot be empty.", + eVersion); + this.endVersion = eVersion; + return this; + } + + @Override + public CopyTable stagingLocation(String stagingLocation) { + Preconditions.checkArgument( + stagingLocation != null && !stagingLocation.isEmpty(), + "Staging location('%s') cannot be empty.", + stagingLocation); + this.stagingDir = stagingLocation; + return this; + } + + @Override + public CopyTable targetTable(Table tgtTable) { + this.targetTable = tgtTable; + return this; + } + + @Override + protected CopyTable self() { + return this; + } + + @Override + public Result execute() { + validateInputs(); + JobGroupInfo info = newJobGroupInfo("COPY-TABLE", jobDesc()); + return withJobGroupInfo(info, this::doExecute); + } + + private Result doExecute() { + rebuildMetaData(); + return new BaseCopyTableActionResult( + stagingDir, dataFileListPath, metadataFileListPath, fileName(endVersion)); + } + + private void validateInputs() { + Preconditions.checkArgument( + sourcePrefix != null && !sourcePrefix.isEmpty(), + "Source prefix('%s') cannot be empty.", + sourcePrefix); + + validateAndSetEndVersion(); + + endStaticTable = newStaticTable(TableMetadataParser.read(table.io(), endVersion), table.io()); + + TableMetadata tableMetadata = ((HasTableOperations) endStaticTable).operations().current(); + Preconditions.checkArgument( + tableMetadata.formatVersion() == 2, "Support Iceberg format version 2 only."); + + validateAndSetStartVersion(tableMetadata); + + if (fileExist(startVersion)) { + startStaticTable = + newStaticTable(TableMetadataParser.read(table.io(), startVersion), table.io()); + } + + if (stagingDir.isEmpty()) { + String stagingDirName = "copy-table-staging-" + UUID.randomUUID(); + stagingDir = combinePaths(table.location(), stagingDirName); + } + + if (!stagingDir.endsWith("/")) { + stagingDir = stagingDir + "/"; + } + } + + private void validateAndSetEndVersion() { + if (endVersion.isEmpty()) { + endVersion = currentMetadataPath(table); + } else { + TableMetadata tableMetadata = ((HasTableOperations) table).operations().current(); + if (versionInFilePath(tableMetadata.metadataFileLocation(), endVersion)) { + endVersion = tableMetadata.metadataFileLocation(); + } + for (MetadataLogEntry metadataLogEntry : tableMetadata.previousFiles()) { + if (versionInFilePath(metadataLogEntry.file(), endVersion)) { + endVersion = metadataLogEntry.file(); + break; + } + } + + Preconditions.checkArgument( + fileExist(endVersion), + "Cannot find the end version('%s') in the current version " + "files", + endVersion); + } + } + + private void validateAndSetStartVersion(TableMetadata tableMetadata) { + if (startVersion.isEmpty()) { + if (targetTable == null) { + LOG.warn("No input of the start version. Will do a full copy."); + } else { + String tgtTableCurrentVersion = fileName(currentMetadataPath(targetTable)); + + for (MetadataLogEntry metadataLogEntry : tableMetadata.previousFiles()) { + if (metadataLogEntry.file().endsWith(tgtTableCurrentVersion)) { + startVersion = metadataLogEntry.file(); + break; + } + } + + if (fileNotExist(startVersion)) { + throw new IllegalArgumentException( + "Cannot find the current version of target table in the source table. " + + "Please make sure the target table is a subset of source table."); + } + } + } else { + for (MetadataLogEntry metadataLogEntry : tableMetadata.previousFiles()) { + if (versionInFilePath(metadataLogEntry.file(), startVersion)) { + startVersion = metadataLogEntry.file(); + break; + } + } + + Preconditions.checkArgument( + fileExist(startVersion), "Start version('%s') is NOT valid.", startVersion); + + if (targetTable != null + && !fileName(startVersion).equals(fileName(currentMetadataPath(targetTable)))) { + throw new IllegalArgumentException( + "The start version isn't the current version of the target table. " + + "Please make sure the target table is a subset of source table."); + } + } + } + + private boolean versionInFilePath(String path, String version) { + return fileName(path).equals(version); + } + + private String jobDesc() { + if (startVersion.isEmpty()) { + return String.format( + "Replacing path prefixes '%s' with '%s' in the metadata files of table %s," + + "up to version '%s'.", + sourcePrefix, targetPrefix, table.name(), endVersion); + } else { + return String.format( + "Replacing path prefixes '%s' with '%s' in the metadata files of table %s," + + "from version '%s' to '%s'.", + sourcePrefix, targetPrefix, table.name(), startVersion, endVersion); + } + } + + /** + * Here are steps: 1. rebuild version files 2. rebuild manifest list files 3. rebuild manifest + * files 4. get all data files need to move + */ + private void rebuildMetaData() { + TableMetadata tableMetadata = ((HasTableOperations) endStaticTable).operations().current(); + + // rebuild version files + Set<Long> allSnapshotIds = rewriteVersionFiles(tableMetadata); + + Set<Long> diffSnapshotIds = getDiffSnapshotIds(allSnapshotIds); + + // get all manifest file paths need to rewrite + List<String> manifestFilePathToMove = manifestFilesToMove(diffSnapshotIds); + manifestFilePaths.addAll(manifestFilePathToMove); + + // rebuild manifest-list files + Set<Snapshot> validSnapshots = + Sets.difference(snapshotSet(endVersion), snapshotSet(startVersion)); + validSnapshots.forEach(snapshot -> rewriteManifestList(snapshot, tableMetadata)); + + // rebuild manifest files + List<ManifestFile> newManifests = rewriteManifests(tableMetadata); + newManifests.forEach(manifestFile -> addToRebuiltFiles(manifestFile.path())); + saveMetadataFileList(); + + Dataset<Row> dataFiles = getDiffDataFiles(diffSnapshotIds); + saveDataFileList(dataFiles); + } + + private void saveMetadataFileList() { + List<String> fileList = Lists.newArrayList(); + fileList.addAll(metadataFilesToMove); + Dataset<String> metadataFileList = spark().createDataset(fileList, Encoders.STRING()); + metadataFileListPath = stagingDir + METADATA_FILE_LIST_DIR; + metadataFileList + .repartition(1) + .write() + .mode(SaveMode.Overwrite) + .format("text") + .save(metadataFileListPath); + } + + private void saveDataFileList(Dataset<Row> dataFiles) { + dataFileListPath = stagingDir + DATA_FILE_LIST_DIR; + + try { + dataFiles + .repartition(1) + .write() + .mode(SaveMode.Overwrite) + .format("text") + .save(dataFileListPath); + } catch (Exception e) { + throw new UnsupportedOperationException( + "Failed to build the data files dataframe, the end version you are " + + "trying to copy may contain invalid snapshots, please use the younger version which doesn't have invalid " + + "snapshots", + e); + } + } + + private Set<Long> getDiffSnapshotIds(Set<Long> allSnapshotIds) { + Set<Long> snapshotIdsInStartVersion = Sets.newHashSet(); + if (startStaticTable != null) { + startStaticTable + .snapshots() + .forEach(snapshot -> snapshotIdsInStartVersion.add(snapshot.snapshotId())); + } + return Sets.difference(allSnapshotIds, snapshotIdsInStartVersion); + } + + private Set<Long> rewriteVersionFiles(TableMetadata metadata) { + Set<Long> allSnapshotIds = Sets.newHashSet(); + + String stagingPath = combinePaths(stagingDir, relativize(endVersion, sourcePrefix)); + metadata.snapshots().forEach(snapshot -> allSnapshotIds.add(snapshot.snapshotId())); + rewriteVersionFile(metadata, stagingPath); + + List<MetadataLogEntry> versions = metadata.previousFiles(); + for (int i = versions.size() - 1; i >= 0; i--) { Review Comment: Could be rewritten as ```List<MetadataLogEntry> versions = Lists.reverse(metadata.previousFiles()); for (MetadataLogEntry version: versions) { if (version.file().equals(startVersion)) { break; } }``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org