nsivabalan commented on code in PR #14260: URL: https://github.com/apache/hudi/pull/14260#discussion_r2558800866
########## hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedMetadataSyncMetadataWriter.java: ########## @@ -0,0 +1,198 @@ +/* + * 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.hudi.metadata; + +import org.apache.hudi.common.config.HoodieCommonConfig; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; +import org.apache.hudi.common.table.view.FileSystemViewStorageType; +import org.apache.hudi.common.table.view.SpillableMapBasedFileSystemView; +import org.apache.hudi.common.util.ExternalFilePathUtil; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; + +import org.apache.hadoop.conf.Configuration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class SparkHoodieBackedMetadataSyncMetadataWriter extends SparkHoodieBackedTableMetadataWriter { Review Comment: java docs ########## hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java: ########## @@ -909,11 +909,20 @@ && isIndexingCommit(instant.getTimestamp())) * @return true if rollback was triggered. false otherwise. */ protected Boolean rollbackFailedWrites() { + return rollbackFailedWrites(false); Review Comment: should we not remove these changes? ########## hudi-common/src/main/java/org/apache/hudi/common/model/HoodieBaseFile.java: ########## @@ -62,40 +66,61 @@ public HoodieBaseFile(String filePath) { public HoodieBaseFile(String filePath, BaseFile bootstrapBaseFile) { super(filePath); this.bootstrapBaseFile = Option.ofNullable(bootstrapBaseFile); - String[] fileIdAndCommitTime = getFileIdAndCommitTimeFromFileName(getFileName()); - this.fileId = fileIdAndCommitTime[0]; - this.commitTime = fileIdAndCommitTime[1]; + String[] fileInfo = getInfoFromFileName(getFileName()); + this.fileId = fileInfo[0]; + this.commitTime = fileInfo[1]; + this.numPartitionLevels = StringUtils.isNullOrEmpty(fileInfo[3]) + ? Option.empty() : Option.ofNullable(parseSafe(fileInfo[3])); } public HoodieBaseFile(String filePath, String fileId, String commitTime, BaseFile bootstrapBaseFile) { super(filePath); this.bootstrapBaseFile = Option.ofNullable(bootstrapBaseFile); this.fileId = fileId; this.commitTime = commitTime; + this.numPartitionLevels = Option.empty(); } private HoodieBaseFile(FileStatus fileStatus, String[] fileIdAndCommitTime, BaseFile bootstrapBaseFile) { - this(fileStatus, fileIdAndCommitTime[0], fileIdAndCommitTime[1], bootstrapBaseFile); + this(fileStatus, fileIdAndCommitTime[0], fileIdAndCommitTime[1], fileIdAndCommitTime[2], fileIdAndCommitTime[3], bootstrapBaseFile); } public HoodieBaseFile(FileStatus fileStatus, String fileId, String commitTime, BaseFile bootstrapBaseFile) { - super(maybeHandleExternallyGeneratedFileName(fileStatus, fileId)); + this(fileStatus, fileId, commitTime, fileId, bootstrapBaseFile); + } + + public HoodieBaseFile(FileStatus fileStatus, String fileId, String commitTime, String originalFileName, BaseFile bootstrapBaseFile) { + this(fileStatus, fileId, commitTime, originalFileName, "", bootstrapBaseFile); Review Comment: StringUtils.EMPTY_STRING ########## hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java: ########## @@ -246,6 +246,34 @@ public static String getRelativePartitionPath(Path basePath, Path fullPartitionP : fullPartitionPathStr.substring(partitionStartIndex + basePath.getName().length() + 1); } + public static String getRelativePartitionPath(Path fullPartitionPath, int numPartitionPathLevels) { Review Comment: do we have UTs for this ########## hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java: ########## @@ -813,6 +813,8 @@ public static class PropertyBuilder { private String metadataPartitions; private String inflightMetadataPartitions; private String secondaryIndexesMetadata; + private Boolean allowBasePathOverridesWithMetadata; + private Integer numPartitionPathLevels; Review Comment: are these supposed to be cleaned up? ########## hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java: ########## @@ -179,33 +191,36 @@ protected List<HoodieFileGroup> buildFileGroups(FileStatus[] statuses, HoodieTim protected List<HoodieFileGroup> buildFileGroups(Stream<HoodieBaseFile> baseFileStream, Stream<HoodieLogFile> logFileStream, HoodieTimeline timeline, boolean addPendingCompactionFileSlice) { - Map<Pair<String, String>, List<HoodieBaseFile>> baseFiles = + Map<Triple<String, String, Option<String>>, List<HoodieBaseFile>> baseFiles = Review Comment: Note toe self. review this code block. ########## hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java: ########## @@ -99,9 +100,16 @@ protected Option<HoodieTableMetadataWriter> getMetadataWriter( // Create the metadata table writer. First time after the upgrade this creation might trigger // metadata table bootstrapping. Bootstrapping process could fail and checking the table // existence after the creation is needed. - HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create( - context.getHadoopConf().get(), config, failedWritesCleaningPolicy, context, - Option.of(triggeringInstantTimestamp)); + HoodieTableMetadataWriter metadataWriter; + if (config.shouldEnableBootstrapMetadataSync()) { + metadataWriter = SparkHoodieBackedMetadataSyncMetadataWriter.create( Review Comment: why do we need this change? we are directly instantiating `SparkHoodieBackedMetadataSyncMetadataWriter` within `HoodieMetadataSync` right. So, why we need these changes. ########## hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataSync.java: ########## @@ -0,0 +1,545 @@ +/* + * 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.hudi.utilities; + +import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieCleanPartitionMetadata; +import org.apache.hudi.client.HoodieTimelineArchiver; +import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.client.transaction.TransactionManager; +import org.apache.hudi.client.transaction.lock.InProcessLockProvider; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.TableSchemaResolver; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.util.ExternalFilePathUtil; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieLockConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.TableNotFoundException; +import org.apache.hudi.metadata.HoodieBackedTableMetadata; +import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.metadata.HoodieTableMetadataWriter; +import org.apache.hudi.metadata.MetadataPartitionType; +import org.apache.hudi.metadata.SparkHoodieBackedMetadataSyncMetadataWriter; +import org.apache.hudi.table.HoodieSparkTable; + +import com.beust.jcommander.JCommander; +import com.beust.jcommander.Parameter; +import org.apache.avro.Schema; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.Serializable; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Properties; +import java.util.TreeMap; +import java.util.stream.Collectors; + +import static org.apache.hudi.index.HoodieIndex.IndexType.INMEMORY; +import static org.apache.hudi.utilities.MetadataSyncUtils.getHoodieCommitMetadata; +import static org.apache.hudi.utilities.MetadataSyncUtils.getInstantsToSyncAndLastSyncCheckpoint; +import static org.apache.hudi.utilities.MetadataSyncUtils.getPendingWriteInstants; +import static org.apache.hudi.utilities.MetadataSyncUtils.getTableSyncExtraMetadata; + +/** + * Handles metadata table synchronization for a Hudi table. + * <p> + * This class initializes Spark context, loads configuration properties, + * and provides utilities required to perform metadata sync between a + * source Hudi table and a target Hudi table. + * </p> + */ +public class HoodieMetadataSync implements Serializable { + + private static final Logger LOG = LoggerFactory.getLogger(HoodieMetadataSync.class); + // Spark context + private transient JavaSparkContext jsc; + // config + private Config cfg; + // Properties with source, hoodie client, key generator etc. + private TypedProperties props; + + public HoodieMetadataSync(JavaSparkContext jsc, Config cfg) { + this.jsc = jsc; + this.cfg = cfg; + this.props = StringUtils.isNullOrEmpty(cfg.propsFilePath) + ? UtilHelpers.buildProperties(cfg.configs) + : readConfigFromFileSystem(jsc, cfg); + } + + /** + * Reads config from the file system. + * + * @param jsc {@link JavaSparkContext} instance. + * @param cfg {@link TableSizeStats.Config} instance. + * @return the {@link TypedProperties} instance. + */ + private TypedProperties readConfigFromFileSystem(JavaSparkContext jsc, Config cfg) { + return UtilHelpers.readConfig(jsc.hadoopConfiguration(), new Path(cfg.propsFilePath), cfg.configs) + .getProps(true); + } + + public static class Config implements Serializable { + @Parameter(names = {"--source-base-path", "-sbp"}, description = "Source Base path for the table", required = true) + public String sourceBasePath = null; + + @Parameter(names = {"--target-base-path", "-tbp"}, description = "Target Base path for the table", required = true) + public String targetBasePath = null; + + @Parameter(names = {"--target-table-name", "-ttn"}, description = "Target table name", required = true) + public String targetTableName = null; + + @Parameter(names = {"--commit-to-sync", "-cts"}, description = "Commit of interest to sync", required = false) + public String commitToSync = null; + + @Parameter(names = {"--spark-master", "-ms"}, description = "Spark master", required = false) + public String sparkMaster = null; + + @Parameter(names = {"--spark-memory", "-sm"}, description = "spark memory to use", required = false) + public String sparkMemory = "1g"; + + @Parameter(names = {"--hoodie-conf"}, description = "Any configuration that can be set in the properties file " + + "(using the CLI parameter \"--props\") can also be passed command line using this parameter. This can be repeated", + splitter = IdentitySplitter.class) + public List<String> configs = new ArrayList<>(); + + @Parameter(names = {"--boostrap"}, description = "boostraps metadata table", + splitter = IdentitySplitter.class) + public Boolean boostrap = false; + + @Parameter(names = {"--restore"}, description = "restore target table", + splitter = IdentitySplitter.class) + public Boolean doRestore = false; + + @Parameter(names = {"--commit-to-restore", "-ctr"}, description = "Commit to restore to", required = false) + public String commitToRestore = null; + + @Parameter(names = {"--perform-table-maintenance"}, description = "performs table maintenance", + splitter = IdentitySplitter.class) + public Boolean performTableMaintenance = false; + + @Parameter(names = {"--props"}, description = "path to properties file on localfs or dfs, with configurations for " + + "hoodie client for clustering") + public String propsFilePath = null; + + @Parameter(names = {"--help", "-h"}, help = true) + public Boolean help = false; + + @Override + public String toString() { + return "TableSizeStats {\n" + + " --source-base-path " + sourceBasePath + ", \n" + + " --target-base-path " + targetBasePath + ", \n" + + " --commit-to-sync " + commitToSync + ", \n" + + " --hoodie-conf " + configs + + "\n}"; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof Config)) { + return false; + } + Config config = (Config) o; + return sourceBasePath.equals(config.sourceBasePath) && targetBasePath.equals(config.targetBasePath) && Objects.equals(commitToSync, config.commitToSync) + && Objects.equals(sparkMaster, config.sparkMaster) && Objects.equals(sparkMemory, config.sparkMemory); + } + + @Override + public int hashCode() { + return Objects.hash(sourceBasePath, targetBasePath, commitToSync, sparkMaster, sparkMemory); + } + } + + public static void main(String[] args) { + final Config cfg = new Config(); + JCommander cmd = new JCommander(cfg, null, args); + + if (cfg.help || args.length == 0) { + cmd.usage(); + System.exit(1); + } + + SparkConf sparkConf = UtilHelpers.buildSparkConf("HoodieMetadataSync", cfg.sparkMaster); + sparkConf.set("spark.executor.memory", cfg.sparkMemory); + JavaSparkContext jsc = new JavaSparkContext(sparkConf); + + try { + HoodieMetadataSync hoodieMetadataSync = new HoodieMetadataSync(jsc, cfg); + hoodieMetadataSync.run(); + } catch (TableNotFoundException e) { + LOG.warn(String.format("The Hudi data table is not found: [%s].", cfg.sourceBasePath), e); + } catch (Throwable throwable) { + LOG.error("Failed to get table size stats for " + cfg, throwable); + } finally { + jsc.stop(); + } + } + + public void run() throws Exception { + Path sourceTablePath = new Path(cfg.targetBasePath); + FileSystem fs = sourceTablePath.getFileSystem(jsc.hadoopConfiguration()); + HoodieTableMetaClient sourceTableMetaClient = HoodieTableMetaClient.builder().setBasePath(cfg.sourceBasePath).setConf(jsc.hadoopConfiguration()) + .build(); + + Path targetTablePath = new Path(cfg.targetBasePath); + this.props = StringUtils.isNullOrEmpty(cfg.propsFilePath) + ? UtilHelpers.buildProperties(cfg.configs) + : readConfigFromFileSystem(jsc, cfg); + boolean targetTableExists = fs.exists(targetTablePath); + if (!targetTableExists) { + LOG.info("Initializing target table for first time", cfg.targetBasePath); + TypedProperties props = sourceTableMetaClient.getTableConfig().getProps(); + props.remove("hoodie.table.metadata.partitions"); + HoodieTableMetaClient.withPropertyBuilder() + .fromProperties(props) + .setTableName(cfg.targetTableName) + .initTable(jsc.hadoopConfiguration(), cfg.targetBasePath); + } + HoodieTableMetaClient targetTableMetaClient = HoodieTableMetaClient.builder().setBasePath(cfg.targetBasePath) + .setConf(jsc.hadoopConfiguration()).build(); + Schema schema = new TableSchemaResolver(sourceTableMetaClient).getTableAvroSchema(false); + runMetadataSync(sourceTableMetaClient, targetTableMetaClient, schema); + LOG.info("Completed syncing {} to target table", cfg.commitToSync); + } + + private void runMetadataSync(HoodieTableMetaClient sourceTableMetaClient, HoodieTableMetaClient targetTableMetaClient, Schema schema) throws Exception { + HoodieWriteConfig writeConfig = getWriteConfig(schema, targetTableMetaClient, cfg.sourceBasePath, cfg.boostrap); + HoodieSparkEngineContext hoodieSparkEngineContext = new HoodieSparkEngineContext(jsc); + TransactionManager txnManager = new TransactionManager(writeConfig, FSUtils.getFs(writeConfig.getBasePath(), hoodieSparkEngineContext.getHadoopConf().get())); + + HoodieSparkTable sparkTable = HoodieSparkTable.create(writeConfig, hoodieSparkEngineContext, targetTableMetaClient); + try (SparkRDDWriteClient writeClient = new SparkRDDWriteClient(hoodieSparkEngineContext, writeConfig)) { + txnManager.beginTransaction(Option.empty(), Option.empty()); + if (cfg.boostrap) { + runBootstrapSync(sparkTable, sourceTableMetaClient, targetTableMetaClient, writeClient, schema); + } else { + Pair<TreeMap<HoodieInstant, Boolean>, Option<String>> instantsToSyncAndLastSyncCheckpointPair = + getInstantsToSyncAndLastSyncCheckpoint(cfg.sourceBasePath, targetTableMetaClient, sourceTableMetaClient); + + TreeMap<HoodieInstant, Boolean> instantsStatusMap = instantsToSyncAndLastSyncCheckpointPair.getLeft(); + Option<String> lastSyncCheckpoint = instantsToSyncAndLastSyncCheckpointPair.getRight(); + + for (Map.Entry<HoodieInstant, Boolean> entry : instantsStatusMap.entrySet()) { + HoodieInstant instant = entry.getKey(); + boolean isCompleted = entry.getValue(); + if (!isCompleted) { + // if instant is pending state, skip it + continue; + } + + Option<byte[]> commitMetadataInBytes = Option.empty(); + SyncMetadata syncMetadata = getTableSyncExtraMetadata(targetTableMetaClient.reloadActiveTimeline().getWriteTimeline().filterCompletedInstants().lastInstant(), + targetTableMetaClient, cfg.sourceBasePath, instant.getTimestamp(), instantsStatusMap, lastSyncCheckpoint, instant); + + if (!getPendingWriteInstants(targetTableMetaClient.reloadActiveTimeline(), Option.empty()).isEmpty()) { + // rollback failing writes + writeClient.rollbackFailedWrites(true); + } + + String commitTime = writeClient.startCommit(instant.getAction(), targetTableMetaClient); + targetTableMetaClient + .reloadActiveTimeline() + .transitionRequestedToInflight( + instant.getAction(), + commitTime); + HoodieTableMetadataWriter hoodieTableMetadataWriter = Review Comment: can we close the `hoodieTableMetadataWriter` at the end of this method ########## hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataSync.java: ########## @@ -0,0 +1,545 @@ +/* + * 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.hudi.utilities; + +import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieCleanPartitionMetadata; +import org.apache.hudi.client.HoodieTimelineArchiver; +import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.client.transaction.TransactionManager; +import org.apache.hudi.client.transaction.lock.InProcessLockProvider; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.TableSchemaResolver; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.util.ExternalFilePathUtil; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieLockConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.TableNotFoundException; +import org.apache.hudi.metadata.HoodieBackedTableMetadata; +import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.metadata.HoodieTableMetadataWriter; +import org.apache.hudi.metadata.MetadataPartitionType; +import org.apache.hudi.metadata.SparkHoodieBackedMetadataSyncMetadataWriter; +import org.apache.hudi.table.HoodieSparkTable; + +import com.beust.jcommander.JCommander; +import com.beust.jcommander.Parameter; +import org.apache.avro.Schema; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.Serializable; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Properties; +import java.util.TreeMap; +import java.util.stream.Collectors; + +import static org.apache.hudi.index.HoodieIndex.IndexType.INMEMORY; +import static org.apache.hudi.utilities.MetadataSyncUtils.getHoodieCommitMetadata; +import static org.apache.hudi.utilities.MetadataSyncUtils.getInstantsToSyncAndLastSyncCheckpoint; +import static org.apache.hudi.utilities.MetadataSyncUtils.getPendingWriteInstants; +import static org.apache.hudi.utilities.MetadataSyncUtils.getTableSyncExtraMetadata; + +/** + * Handles metadata table synchronization for a Hudi table. + * <p> + * This class initializes Spark context, loads configuration properties, + * and provides utilities required to perform metadata sync between a + * source Hudi table and a target Hudi table. + * </p> + */ +public class HoodieMetadataSync implements Serializable { + + private static final Logger LOG = LoggerFactory.getLogger(HoodieMetadataSync.class); + // Spark context + private transient JavaSparkContext jsc; + // config + private Config cfg; + // Properties with source, hoodie client, key generator etc. + private TypedProperties props; + + public HoodieMetadataSync(JavaSparkContext jsc, Config cfg) { + this.jsc = jsc; + this.cfg = cfg; + this.props = StringUtils.isNullOrEmpty(cfg.propsFilePath) + ? UtilHelpers.buildProperties(cfg.configs) + : readConfigFromFileSystem(jsc, cfg); + } + + /** + * Reads config from the file system. + * + * @param jsc {@link JavaSparkContext} instance. + * @param cfg {@link TableSizeStats.Config} instance. + * @return the {@link TypedProperties} instance. + */ + private TypedProperties readConfigFromFileSystem(JavaSparkContext jsc, Config cfg) { + return UtilHelpers.readConfig(jsc.hadoopConfiguration(), new Path(cfg.propsFilePath), cfg.configs) + .getProps(true); + } + + public static class Config implements Serializable { + @Parameter(names = {"--source-base-path", "-sbp"}, description = "Source Base path for the table", required = true) + public String sourceBasePath = null; + + @Parameter(names = {"--target-base-path", "-tbp"}, description = "Target Base path for the table", required = true) + public String targetBasePath = null; + + @Parameter(names = {"--target-table-name", "-ttn"}, description = "Target table name", required = true) + public String targetTableName = null; + + @Parameter(names = {"--commit-to-sync", "-cts"}, description = "Commit of interest to sync", required = false) + public String commitToSync = null; + + @Parameter(names = {"--spark-master", "-ms"}, description = "Spark master", required = false) + public String sparkMaster = null; + + @Parameter(names = {"--spark-memory", "-sm"}, description = "spark memory to use", required = false) + public String sparkMemory = "1g"; + + @Parameter(names = {"--hoodie-conf"}, description = "Any configuration that can be set in the properties file " + + "(using the CLI parameter \"--props\") can also be passed command line using this parameter. This can be repeated", + splitter = IdentitySplitter.class) + public List<String> configs = new ArrayList<>(); + + @Parameter(names = {"--boostrap"}, description = "boostraps metadata table", + splitter = IdentitySplitter.class) + public Boolean boostrap = false; + + @Parameter(names = {"--restore"}, description = "restore target table", + splitter = IdentitySplitter.class) + public Boolean doRestore = false; + + @Parameter(names = {"--commit-to-restore", "-ctr"}, description = "Commit to restore to", required = false) + public String commitToRestore = null; + + @Parameter(names = {"--perform-table-maintenance"}, description = "performs table maintenance", + splitter = IdentitySplitter.class) + public Boolean performTableMaintenance = false; + + @Parameter(names = {"--props"}, description = "path to properties file on localfs or dfs, with configurations for " + + "hoodie client for clustering") + public String propsFilePath = null; + + @Parameter(names = {"--help", "-h"}, help = true) + public Boolean help = false; + + @Override + public String toString() { + return "TableSizeStats {\n" + + " --source-base-path " + sourceBasePath + ", \n" + + " --target-base-path " + targetBasePath + ", \n" + + " --commit-to-sync " + commitToSync + ", \n" + + " --hoodie-conf " + configs + + "\n}"; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof Config)) { + return false; + } + Config config = (Config) o; + return sourceBasePath.equals(config.sourceBasePath) && targetBasePath.equals(config.targetBasePath) && Objects.equals(commitToSync, config.commitToSync) + && Objects.equals(sparkMaster, config.sparkMaster) && Objects.equals(sparkMemory, config.sparkMemory); + } + + @Override + public int hashCode() { + return Objects.hash(sourceBasePath, targetBasePath, commitToSync, sparkMaster, sparkMemory); + } + } + + public static void main(String[] args) { + final Config cfg = new Config(); + JCommander cmd = new JCommander(cfg, null, args); + + if (cfg.help || args.length == 0) { + cmd.usage(); + System.exit(1); + } + + SparkConf sparkConf = UtilHelpers.buildSparkConf("HoodieMetadataSync", cfg.sparkMaster); + sparkConf.set("spark.executor.memory", cfg.sparkMemory); + JavaSparkContext jsc = new JavaSparkContext(sparkConf); + + try { + HoodieMetadataSync hoodieMetadataSync = new HoodieMetadataSync(jsc, cfg); + hoodieMetadataSync.run(); + } catch (TableNotFoundException e) { + LOG.warn(String.format("The Hudi data table is not found: [%s].", cfg.sourceBasePath), e); + } catch (Throwable throwable) { + LOG.error("Failed to get table size stats for " + cfg, throwable); + } finally { + jsc.stop(); + } + } + + public void run() throws Exception { + Path sourceTablePath = new Path(cfg.targetBasePath); + FileSystem fs = sourceTablePath.getFileSystem(jsc.hadoopConfiguration()); + HoodieTableMetaClient sourceTableMetaClient = HoodieTableMetaClient.builder().setBasePath(cfg.sourceBasePath).setConf(jsc.hadoopConfiguration()) + .build(); + + Path targetTablePath = new Path(cfg.targetBasePath); + this.props = StringUtils.isNullOrEmpty(cfg.propsFilePath) + ? UtilHelpers.buildProperties(cfg.configs) + : readConfigFromFileSystem(jsc, cfg); + boolean targetTableExists = fs.exists(targetTablePath); + if (!targetTableExists) { + LOG.info("Initializing target table for first time", cfg.targetBasePath); + TypedProperties props = sourceTableMetaClient.getTableConfig().getProps(); + props.remove("hoodie.table.metadata.partitions"); + HoodieTableMetaClient.withPropertyBuilder() + .fromProperties(props) + .setTableName(cfg.targetTableName) + .initTable(jsc.hadoopConfiguration(), cfg.targetBasePath); + } + HoodieTableMetaClient targetTableMetaClient = HoodieTableMetaClient.builder().setBasePath(cfg.targetBasePath) + .setConf(jsc.hadoopConfiguration()).build(); + Schema schema = new TableSchemaResolver(sourceTableMetaClient).getTableAvroSchema(false); + runMetadataSync(sourceTableMetaClient, targetTableMetaClient, schema); + LOG.info("Completed syncing {} to target table", cfg.commitToSync); + } + + private void runMetadataSync(HoodieTableMetaClient sourceTableMetaClient, HoodieTableMetaClient targetTableMetaClient, Schema schema) throws Exception { + HoodieWriteConfig writeConfig = getWriteConfig(schema, targetTableMetaClient, cfg.sourceBasePath, cfg.boostrap); + HoodieSparkEngineContext hoodieSparkEngineContext = new HoodieSparkEngineContext(jsc); + TransactionManager txnManager = new TransactionManager(writeConfig, FSUtils.getFs(writeConfig.getBasePath(), hoodieSparkEngineContext.getHadoopConf().get())); + + HoodieSparkTable sparkTable = HoodieSparkTable.create(writeConfig, hoodieSparkEngineContext, targetTableMetaClient); + try (SparkRDDWriteClient writeClient = new SparkRDDWriteClient(hoodieSparkEngineContext, writeConfig)) { + txnManager.beginTransaction(Option.empty(), Option.empty()); + if (cfg.boostrap) { + runBootstrapSync(sparkTable, sourceTableMetaClient, targetTableMetaClient, writeClient, schema); + } else { + Pair<TreeMap<HoodieInstant, Boolean>, Option<String>> instantsToSyncAndLastSyncCheckpointPair = + getInstantsToSyncAndLastSyncCheckpoint(cfg.sourceBasePath, targetTableMetaClient, sourceTableMetaClient); + + TreeMap<HoodieInstant, Boolean> instantsStatusMap = instantsToSyncAndLastSyncCheckpointPair.getLeft(); + Option<String> lastSyncCheckpoint = instantsToSyncAndLastSyncCheckpointPair.getRight(); + + for (Map.Entry<HoodieInstant, Boolean> entry : instantsStatusMap.entrySet()) { + HoodieInstant instant = entry.getKey(); + boolean isCompleted = entry.getValue(); + if (!isCompleted) { + // if instant is pending state, skip it + continue; + } + + Option<byte[]> commitMetadataInBytes = Option.empty(); + SyncMetadata syncMetadata = getTableSyncExtraMetadata(targetTableMetaClient.reloadActiveTimeline().getWriteTimeline().filterCompletedInstants().lastInstant(), + targetTableMetaClient, cfg.sourceBasePath, instant.getTimestamp(), instantsStatusMap, lastSyncCheckpoint, instant); + + if (!getPendingWriteInstants(targetTableMetaClient.reloadActiveTimeline(), Option.empty()).isEmpty()) { + // rollback failing writes + writeClient.rollbackFailedWrites(true); + } + + String commitTime = writeClient.startCommit(instant.getAction(), targetTableMetaClient); + targetTableMetaClient + .reloadActiveTimeline() + .transitionRequestedToInflight( + instant.getAction(), + commitTime); + HoodieTableMetadataWriter hoodieTableMetadataWriter = + (HoodieTableMetadataWriter) sparkTable.getMetadataWriter(commitTime).get(); + // perform table services if required on metadata table + hoodieTableMetadataWriter.performTableServices(Option.of(commitTime)); + switch (instant.getAction()) { + case HoodieTimeline.COMMIT_ACTION: + HoodieCommitMetadata sourceCommitMetadata = getHoodieCommitMetadata(instant.getTimestamp(), sourceTableMetaClient); + HoodieCommitMetadata tgtCommitMetadata = buildHoodieCommitMetadata(sourceCommitMetadata, commitTime); + hoodieTableMetadataWriter.update(tgtCommitMetadata, hoodieSparkEngineContext.emptyHoodieData(), commitTime); + + // add metadata sync checkpoint info + tgtCommitMetadata.addMetadata(SyncMetadata.TABLE_SYNC_METADATA, syncMetadata.toJson()); + commitMetadataInBytes = Option.of(tgtCommitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)); + break; + case HoodieTimeline.REPLACE_COMMIT_ACTION: + + HoodieReplaceCommitMetadata srcReplaceCommitMetadata = HoodieReplaceCommitMetadata.fromBytes( + sourceTableMetaClient.getCommitsTimeline().getInstantDetails(instant).get(), HoodieReplaceCommitMetadata.class); + HoodieReplaceCommitMetadata tgtReplaceCommitMetadata = buildReplaceCommitMetadata(srcReplaceCommitMetadata, commitTime); + hoodieTableMetadataWriter.update(tgtReplaceCommitMetadata, hoodieSparkEngineContext.emptyHoodieData(), commitTime); + + // add metadata sync checkpoint info + tgtReplaceCommitMetadata.addMetadata(SyncMetadata.TABLE_SYNC_METADATA, syncMetadata.toJson()); + commitMetadataInBytes = Option.of(tgtReplaceCommitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)); + break; + case HoodieTimeline.CLEAN_ACTION: + HoodieCleanMetadata srcCleanMetadata = TimelineMetadataUtils.deserializeHoodieCleanMetadata( + sourceTableMetaClient.getCommitsTimeline().getInstantDetails(instant).get()); + HoodieCleanMetadata tgtCleanMetadata = reconstructHoodieCleanCommitMetadata(srcCleanMetadata, + writeConfig, hoodieSparkEngineContext, targetTableMetaClient); + hoodieTableMetadataWriter.update(tgtCleanMetadata, commitTime); + + commitMetadataInBytes = TimelineMetadataUtils.serializeCleanMetadata(tgtCleanMetadata); + break; + default: + break; + } + + targetTableMetaClient + .reloadActiveTimeline() + .saveAsComplete(new HoodieInstant(true, instant.getAction(), commitTime), commitMetadataInBytes); + + if (cfg.performTableMaintenance) { + runArchiver(sparkTable, writeClient.getConfig(), hoodieSparkEngineContext); + } + + if (!lastSyncCheckpoint.isPresent() || instant.getTimestamp().compareTo(lastSyncCheckpoint.get()) > 0) { + lastSyncCheckpoint = Option.of(instant.getTimestamp()); Review Comment: we should move L 265 until here to another method. and if possible, lets try to add private methods based on "Single Responsibility Principle" rule, and so we can keep main method lean ########## hudi-common/src/main/java/org/apache/hudi/common/model/HoodieBaseFile.java: ########## @@ -62,40 +66,61 @@ public HoodieBaseFile(String filePath) { public HoodieBaseFile(String filePath, BaseFile bootstrapBaseFile) { super(filePath); this.bootstrapBaseFile = Option.ofNullable(bootstrapBaseFile); - String[] fileIdAndCommitTime = getFileIdAndCommitTimeFromFileName(getFileName()); - this.fileId = fileIdAndCommitTime[0]; - this.commitTime = fileIdAndCommitTime[1]; + String[] fileInfo = getInfoFromFileName(getFileName()); + this.fileId = fileInfo[0]; + this.commitTime = fileInfo[1]; + this.numPartitionLevels = StringUtils.isNullOrEmpty(fileInfo[3]) + ? Option.empty() : Option.ofNullable(parseSafe(fileInfo[3])); } public HoodieBaseFile(String filePath, String fileId, String commitTime, BaseFile bootstrapBaseFile) { super(filePath); this.bootstrapBaseFile = Option.ofNullable(bootstrapBaseFile); this.fileId = fileId; this.commitTime = commitTime; + this.numPartitionLevels = Option.empty(); } private HoodieBaseFile(FileStatus fileStatus, String[] fileIdAndCommitTime, BaseFile bootstrapBaseFile) { - this(fileStatus, fileIdAndCommitTime[0], fileIdAndCommitTime[1], bootstrapBaseFile); + this(fileStatus, fileIdAndCommitTime[0], fileIdAndCommitTime[1], fileIdAndCommitTime[2], fileIdAndCommitTime[3], bootstrapBaseFile); Review Comment: lets rename 2nd arg as fileInfo instead of "fileIdAndCommitTime" ########## hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataFileSystemView.java: ########## @@ -74,17 +74,10 @@ protected List<String> getAllPartitionPaths() throws IOException { @Override protected Map<Pair<String, Path>, FileStatus[]> listPartitions(List<Pair<String, Path>> partitionPathList) throws IOException { - Map<String, Pair<String, Path>> absoluteToPairMap = partitionPathList.stream() Review Comment: note to self: review this code block ########## hudi-common/src/main/java/org/apache/hudi/common/model/HoodieBaseFile.java: ########## @@ -115,27 +140,43 @@ private static String[] handleHudiGeneratedFile(String fileName) { } // case where there is no '.' in file name (no file suffix like .parquet) values[1] = fileName.substring(lastUnderscoreIndex + 1); + values[2] = fileName; return values; } private static String[] handleExternallyGeneratedFile(String fileName) { - String[] values = new String[2]; - // file name has format <originalFileName>_<commitTime>_hudiext and originalFileName is used as fileId + String[] values = new String[4]; int lastUnderscore = fileName.lastIndexOf(UNDERSCORE); int secondToLastUnderscore = fileName.lastIndexOf(UNDERSCORE, lastUnderscore - 1); - values[0] = fileName.substring(0, secondToLastUnderscore); - values[1] = fileName.substring(secondToLastUnderscore + 1, lastUnderscore); + if (fileName.substring(secondToLastUnderscore + 1, lastUnderscore).equals(UNIFIED_VIEW_SUFFIX)) { + // This external is generated by Unified View Table + // file name has format <originalFileName>_<commitTime>_<numPartitionLevels>_uv_hudiext and originalFileName is used as fileId + // originalFileName has format <fileId>_suffix + int thirdUnderscoreFromLast = fileName.lastIndexOf(UNDERSCORE, secondToLastUnderscore - 1); + int fourthUnderscoreFromLast = fileName.lastIndexOf(UNDERSCORE, thirdUnderscoreFromLast - 1); + String numPartitionLevels = fileName.substring(thirdUnderscoreFromLast + 1, secondToLastUnderscore); + int firstUnderscore = fileName.indexOf(UNDERSCORE); + values[0] = fileName.substring(0, firstUnderscore); // file id + values[1] = fileName.substring(fourthUnderscoreFromLast + 1, thirdUnderscoreFromLast); // commit time + values[2] = fileName.substring(0, fourthUnderscoreFromLast); + values[3] = numPartitionLevels; + } else { + // file name has format <originalFileName>_<commitTime>_hudiext and originalFileName is used as fileId + values[0] = fileName.substring(0, secondToLastUnderscore); + values[1] = fileName.substring(secondToLastUnderscore + 1, lastUnderscore); + values[2] = fileName.substring(0, secondToLastUnderscore); + } return values; } /** * If the file was created externally, the original file path will have a '_[commitTime]_hudiext' suffix when stored in the metadata table. That suffix needs to be removed from the FileStatus so * that the actual file can be found and read. * @param fileStatus an input file status that may require updating - * @param fileId the fileId for the file + * @param fileName the fileId for the file Review Comment: fix comments ########## hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java: ########## @@ -967,6 +969,16 @@ public PropertyBuilder setSecondaryIndexesMetadata(String secondaryIndexesMetada return this; } + public PropertyBuilder setAllowBasePathOverrides(boolean allowBasePathOverridesWithMetadata) { + this.allowBasePathOverridesWithMetadata = allowBasePathOverridesWithMetadata; + return this; + } + + public PropertyBuilder setNumPartitionPathLevels(int numPartitionPathLevels) { Review Comment: same comment as above ########## hudi-common/src/main/java/org/apache/hudi/common/model/HoodieBaseFile.java: ########## @@ -35,8 +36,10 @@ public class HoodieBaseFile extends BaseFile { private static final long serialVersionUID = 1L; private static final char UNDERSCORE = '_'; private static final char DOT = '.'; + private static final String UNIFIED_VIEW_SUFFIX = "uv"; private final String fileId; private final String commitTime; + private final Option<Integer> numPartitionLevels; Review Comment: lets add comments as to why we need this ########## hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java: ########## @@ -246,6 +246,20 @@ public static String getRelativePartitionPath(Path basePath, Path fullPartitionP : fullPartitionPathStr.substring(partitionStartIndex + basePath.getName().length() + 1); } + public static String getRelativePartitionPath(Path fullPartitionPath, int numPartitionPathLevels) { Review Comment: gotcha ########## hudi-utilities/src/main/java/org/apache/hudi/utilities/MetadataSyncUtils.java: ########## @@ -0,0 +1,77 @@ +package org.apache.hudi.utilities; + +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieException; + +import java.io.IOException; +import java.time.Instant; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +public class MetadataSyncUtils { + + public static List<String> getPendingInstants( + HoodieActiveTimeline activeTimeline, + HoodieInstant latestCommit) { + List<HoodieInstant> pendingHoodieInstants = + activeTimeline + .filterInflightsAndRequested() + .findInstantsBefore(latestCommit.getTimestamp()) + .getInstants(); + return pendingHoodieInstants.stream() + .map(HoodieInstant::getTimestamp) + .collect(Collectors.toList()); + } + + public static SyncMetadata getTableSyncExtraMetadata(Option<HoodieInstant> targetTableLastInstant, HoodieTableMetaClient targetTableMetaClient, String sourceIdentifier, + String sourceInstantSynced, List<String> pendingInstantsToSync) { + return targetTableLastInstant.map(instant -> { + SyncMetadata lastSyncMetadata = null; + try { + lastSyncMetadata = getTableSyncMetadataFromCommitMetadata(instant, targetTableMetaClient); + } catch (IOException e) { + throw new HoodieException("Failed to get sync metadata"); + } + + TableCheckpointInfo checkpointInfo = TableCheckpointInfo.of(sourceInstantSynced, pendingInstantsToSync, sourceIdentifier); + List<TableCheckpointInfo> updatedCheckpointInfos = lastSyncMetadata.getTableCheckpointInfos().stream() + .filter(metadata -> !metadata.getSourceIdentifier().equals(sourceIdentifier)).collect(Collectors.toList()); + updatedCheckpointInfos.add(checkpointInfo); + return SyncMetadata.of(Instant.now(), updatedCheckpointInfos); + }).orElseGet(() -> { + List<TableCheckpointInfo> checkpointInfos = Collections.singletonList(TableCheckpointInfo.of(sourceInstantSynced, pendingInstantsToSync, sourceIdentifier)); + return SyncMetadata.of(Instant.now(), checkpointInfos); + }); + } + + public static SyncMetadata getTableSyncMetadataFromCommitMetadata(HoodieInstant instant, HoodieTableMetaClient metaClient) throws IOException { + HoodieCommitMetadata commitMetadata = getHoodieCommitMetadata(instant.getTimestamp(), metaClient); + Option<String> tableSyncMetadataJson = Option.ofNullable(commitMetadata.getMetadata(SyncMetadata.TABLE_SYNC_METADATA)); + if (!tableSyncMetadataJson.isPresent()) { + // if table sync metadata is not present, sync all commits from source table + throw new HoodieException("Table sync metadata is missing in the target table commit metadata"); Review Comment: sure ########## hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataSync.java: ########## @@ -0,0 +1,545 @@ +/* + * 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.hudi.utilities; + +import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieCleanPartitionMetadata; +import org.apache.hudi.client.HoodieTimelineArchiver; +import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.client.transaction.TransactionManager; +import org.apache.hudi.client.transaction.lock.InProcessLockProvider; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.TableSchemaResolver; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.util.ExternalFilePathUtil; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieLockConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.TableNotFoundException; +import org.apache.hudi.metadata.HoodieBackedTableMetadata; +import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.metadata.HoodieTableMetadataWriter; +import org.apache.hudi.metadata.MetadataPartitionType; +import org.apache.hudi.metadata.SparkHoodieBackedMetadataSyncMetadataWriter; +import org.apache.hudi.table.HoodieSparkTable; + +import com.beust.jcommander.JCommander; +import com.beust.jcommander.Parameter; +import org.apache.avro.Schema; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.Serializable; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Properties; +import java.util.TreeMap; +import java.util.stream.Collectors; + +import static org.apache.hudi.index.HoodieIndex.IndexType.INMEMORY; +import static org.apache.hudi.utilities.MetadataSyncUtils.getHoodieCommitMetadata; +import static org.apache.hudi.utilities.MetadataSyncUtils.getInstantsToSyncAndLastSyncCheckpoint; +import static org.apache.hudi.utilities.MetadataSyncUtils.getPendingWriteInstants; +import static org.apache.hudi.utilities.MetadataSyncUtils.getTableSyncExtraMetadata; + +/** + * Handles metadata table synchronization for a Hudi table. Review Comment: lets add details about unified view etc. ########## hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedMetadataSyncMetadataWriter.java: ########## @@ -0,0 +1,198 @@ +/* + * 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.hudi.metadata; + +import org.apache.hudi.common.config.HoodieCommonConfig; +import org.apache.hudi.common.data.HoodieData; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; +import org.apache.hudi.common.table.view.FileSystemViewStorageType; +import org.apache.hudi.common.table.view.SpillableMapBasedFileSystemView; +import org.apache.hudi.common.util.ExternalFilePathUtil; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; + +import org.apache.hadoop.conf.Configuration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class SparkHoodieBackedMetadataSyncMetadataWriter extends SparkHoodieBackedTableMetadataWriter { Review Comment: do we have tests for this class directly? ########## hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieMetadataSync.java: ########## @@ -0,0 +1,545 @@ +/* + * 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.hudi.utilities; + +import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieCleanPartitionMetadata; +import org.apache.hudi.client.HoodieTimelineArchiver; +import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.client.transaction.TransactionManager; +import org.apache.hudi.client.transaction.lock.InProcessLockProvider; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.TableSchemaResolver; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.util.ExternalFilePathUtil; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieLockConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.TableNotFoundException; +import org.apache.hudi.metadata.HoodieBackedTableMetadata; +import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.metadata.HoodieTableMetadataWriter; +import org.apache.hudi.metadata.MetadataPartitionType; +import org.apache.hudi.metadata.SparkHoodieBackedMetadataSyncMetadataWriter; +import org.apache.hudi.table.HoodieSparkTable; + +import com.beust.jcommander.JCommander; +import com.beust.jcommander.Parameter; +import org.apache.avro.Schema; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.Serializable; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Properties; +import java.util.TreeMap; +import java.util.stream.Collectors; + +import static org.apache.hudi.index.HoodieIndex.IndexType.INMEMORY; +import static org.apache.hudi.utilities.MetadataSyncUtils.getHoodieCommitMetadata; +import static org.apache.hudi.utilities.MetadataSyncUtils.getInstantsToSyncAndLastSyncCheckpoint; +import static org.apache.hudi.utilities.MetadataSyncUtils.getPendingWriteInstants; +import static org.apache.hudi.utilities.MetadataSyncUtils.getTableSyncExtraMetadata; + +/** + * Handles metadata table synchronization for a Hudi table. + * <p> + * This class initializes Spark context, loads configuration properties, + * and provides utilities required to perform metadata sync between a + * source Hudi table and a target Hudi table. + * </p> + */ +public class HoodieMetadataSync implements Serializable { + + private static final Logger LOG = LoggerFactory.getLogger(HoodieMetadataSync.class); + // Spark context + private transient JavaSparkContext jsc; + // config + private Config cfg; + // Properties with source, hoodie client, key generator etc. + private TypedProperties props; + + public HoodieMetadataSync(JavaSparkContext jsc, Config cfg) { + this.jsc = jsc; + this.cfg = cfg; + this.props = StringUtils.isNullOrEmpty(cfg.propsFilePath) + ? UtilHelpers.buildProperties(cfg.configs) + : readConfigFromFileSystem(jsc, cfg); + } + + /** + * Reads config from the file system. + * + * @param jsc {@link JavaSparkContext} instance. + * @param cfg {@link TableSizeStats.Config} instance. + * @return the {@link TypedProperties} instance. + */ + private TypedProperties readConfigFromFileSystem(JavaSparkContext jsc, Config cfg) { + return UtilHelpers.readConfig(jsc.hadoopConfiguration(), new Path(cfg.propsFilePath), cfg.configs) + .getProps(true); + } + + public static class Config implements Serializable { + @Parameter(names = {"--source-base-path", "-sbp"}, description = "Source Base path for the table", required = true) + public String sourceBasePath = null; + + @Parameter(names = {"--target-base-path", "-tbp"}, description = "Target Base path for the table", required = true) + public String targetBasePath = null; + + @Parameter(names = {"--target-table-name", "-ttn"}, description = "Target table name", required = true) + public String targetTableName = null; + + @Parameter(names = {"--commit-to-sync", "-cts"}, description = "Commit of interest to sync", required = false) + public String commitToSync = null; + + @Parameter(names = {"--spark-master", "-ms"}, description = "Spark master", required = false) + public String sparkMaster = null; + + @Parameter(names = {"--spark-memory", "-sm"}, description = "spark memory to use", required = false) + public String sparkMemory = "1g"; + + @Parameter(names = {"--hoodie-conf"}, description = "Any configuration that can be set in the properties file " + + "(using the CLI parameter \"--props\") can also be passed command line using this parameter. This can be repeated", + splitter = IdentitySplitter.class) + public List<String> configs = new ArrayList<>(); + + @Parameter(names = {"--boostrap"}, description = "boostraps metadata table", + splitter = IdentitySplitter.class) + public Boolean boostrap = false; + + @Parameter(names = {"--restore"}, description = "restore target table", + splitter = IdentitySplitter.class) + public Boolean doRestore = false; + + @Parameter(names = {"--commit-to-restore", "-ctr"}, description = "Commit to restore to", required = false) + public String commitToRestore = null; + + @Parameter(names = {"--perform-table-maintenance"}, description = "performs table maintenance", + splitter = IdentitySplitter.class) + public Boolean performTableMaintenance = false; + + @Parameter(names = {"--props"}, description = "path to properties file on localfs or dfs, with configurations for " + + "hoodie client for clustering") + public String propsFilePath = null; + + @Parameter(names = {"--help", "-h"}, help = true) + public Boolean help = false; + + @Override + public String toString() { + return "TableSizeStats {\n" + + " --source-base-path " + sourceBasePath + ", \n" + + " --target-base-path " + targetBasePath + ", \n" + + " --commit-to-sync " + commitToSync + ", \n" + + " --hoodie-conf " + configs + + "\n}"; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (!(o instanceof Config)) { + return false; + } + Config config = (Config) o; + return sourceBasePath.equals(config.sourceBasePath) && targetBasePath.equals(config.targetBasePath) && Objects.equals(commitToSync, config.commitToSync) + && Objects.equals(sparkMaster, config.sparkMaster) && Objects.equals(sparkMemory, config.sparkMemory); + } + + @Override + public int hashCode() { + return Objects.hash(sourceBasePath, targetBasePath, commitToSync, sparkMaster, sparkMemory); + } + } + + public static void main(String[] args) { + final Config cfg = new Config(); + JCommander cmd = new JCommander(cfg, null, args); + + if (cfg.help || args.length == 0) { + cmd.usage(); + System.exit(1); + } + + SparkConf sparkConf = UtilHelpers.buildSparkConf("HoodieMetadataSync", cfg.sparkMaster); + sparkConf.set("spark.executor.memory", cfg.sparkMemory); + JavaSparkContext jsc = new JavaSparkContext(sparkConf); + + try { + HoodieMetadataSync hoodieMetadataSync = new HoodieMetadataSync(jsc, cfg); + hoodieMetadataSync.run(); + } catch (TableNotFoundException e) { + LOG.warn(String.format("The Hudi data table is not found: [%s].", cfg.sourceBasePath), e); + } catch (Throwable throwable) { + LOG.error("Failed to get table size stats for " + cfg, throwable); + } finally { + jsc.stop(); + } + } + + public void run() throws Exception { + Path sourceTablePath = new Path(cfg.targetBasePath); + FileSystem fs = sourceTablePath.getFileSystem(jsc.hadoopConfiguration()); + HoodieTableMetaClient sourceTableMetaClient = HoodieTableMetaClient.builder().setBasePath(cfg.sourceBasePath).setConf(jsc.hadoopConfiguration()) + .build(); + + Path targetTablePath = new Path(cfg.targetBasePath); + this.props = StringUtils.isNullOrEmpty(cfg.propsFilePath) + ? UtilHelpers.buildProperties(cfg.configs) + : readConfigFromFileSystem(jsc, cfg); + boolean targetTableExists = fs.exists(targetTablePath); + if (!targetTableExists) { + LOG.info("Initializing target table for first time", cfg.targetBasePath); + TypedProperties props = sourceTableMetaClient.getTableConfig().getProps(); + props.remove("hoodie.table.metadata.partitions"); + HoodieTableMetaClient.withPropertyBuilder() + .fromProperties(props) + .setTableName(cfg.targetTableName) + .initTable(jsc.hadoopConfiguration(), cfg.targetBasePath); + } + HoodieTableMetaClient targetTableMetaClient = HoodieTableMetaClient.builder().setBasePath(cfg.targetBasePath) + .setConf(jsc.hadoopConfiguration()).build(); + Schema schema = new TableSchemaResolver(sourceTableMetaClient).getTableAvroSchema(false); + runMetadataSync(sourceTableMetaClient, targetTableMetaClient, schema); + LOG.info("Completed syncing {} to target table", cfg.commitToSync); + } + + private void runMetadataSync(HoodieTableMetaClient sourceTableMetaClient, HoodieTableMetaClient targetTableMetaClient, Schema schema) throws Exception { + HoodieWriteConfig writeConfig = getWriteConfig(schema, targetTableMetaClient, cfg.sourceBasePath, cfg.boostrap); + HoodieSparkEngineContext hoodieSparkEngineContext = new HoodieSparkEngineContext(jsc); + TransactionManager txnManager = new TransactionManager(writeConfig, FSUtils.getFs(writeConfig.getBasePath(), hoodieSparkEngineContext.getHadoopConf().get())); + + HoodieSparkTable sparkTable = HoodieSparkTable.create(writeConfig, hoodieSparkEngineContext, targetTableMetaClient); + try (SparkRDDWriteClient writeClient = new SparkRDDWriteClient(hoodieSparkEngineContext, writeConfig)) { + txnManager.beginTransaction(Option.empty(), Option.empty()); + if (cfg.boostrap) { + runBootstrapSync(sparkTable, sourceTableMetaClient, targetTableMetaClient, writeClient, schema); + } else { + Pair<TreeMap<HoodieInstant, Boolean>, Option<String>> instantsToSyncAndLastSyncCheckpointPair = + getInstantsToSyncAndLastSyncCheckpoint(cfg.sourceBasePath, targetTableMetaClient, sourceTableMetaClient); + + TreeMap<HoodieInstant, Boolean> instantsStatusMap = instantsToSyncAndLastSyncCheckpointPair.getLeft(); + Option<String> lastSyncCheckpoint = instantsToSyncAndLastSyncCheckpointPair.getRight(); + + for (Map.Entry<HoodieInstant, Boolean> entry : instantsStatusMap.entrySet()) { + HoodieInstant instant = entry.getKey(); + boolean isCompleted = entry.getValue(); + if (!isCompleted) { + // if instant is pending state, skip it + continue; + } + + Option<byte[]> commitMetadataInBytes = Option.empty(); + SyncMetadata syncMetadata = getTableSyncExtraMetadata(targetTableMetaClient.reloadActiveTimeline().getWriteTimeline().filterCompletedInstants().lastInstant(), + targetTableMetaClient, cfg.sourceBasePath, instant.getTimestamp(), instantsStatusMap, lastSyncCheckpoint, instant); + + if (!getPendingWriteInstants(targetTableMetaClient.reloadActiveTimeline(), Option.empty()).isEmpty()) { + // rollback failing writes + writeClient.rollbackFailedWrites(true); + } + + String commitTime = writeClient.startCommit(instant.getAction(), targetTableMetaClient); + targetTableMetaClient + .reloadActiveTimeline() + .transitionRequestedToInflight( + instant.getAction(), + commitTime); + HoodieTableMetadataWriter hoodieTableMetadataWriter = + (HoodieTableMetadataWriter) sparkTable.getMetadataWriter(commitTime).get(); + // perform table services if required on metadata table + hoodieTableMetadataWriter.performTableServices(Option.of(commitTime)); + switch (instant.getAction()) { + case HoodieTimeline.COMMIT_ACTION: + HoodieCommitMetadata sourceCommitMetadata = getHoodieCommitMetadata(instant.getTimestamp(), sourceTableMetaClient); + HoodieCommitMetadata tgtCommitMetadata = buildHoodieCommitMetadata(sourceCommitMetadata, commitTime); + hoodieTableMetadataWriter.update(tgtCommitMetadata, hoodieSparkEngineContext.emptyHoodieData(), commitTime); + + // add metadata sync checkpoint info + tgtCommitMetadata.addMetadata(SyncMetadata.TABLE_SYNC_METADATA, syncMetadata.toJson()); + commitMetadataInBytes = Option.of(tgtCommitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)); + break; + case HoodieTimeline.REPLACE_COMMIT_ACTION: + + HoodieReplaceCommitMetadata srcReplaceCommitMetadata = HoodieReplaceCommitMetadata.fromBytes( + sourceTableMetaClient.getCommitsTimeline().getInstantDetails(instant).get(), HoodieReplaceCommitMetadata.class); + HoodieReplaceCommitMetadata tgtReplaceCommitMetadata = buildReplaceCommitMetadata(srcReplaceCommitMetadata, commitTime); + hoodieTableMetadataWriter.update(tgtReplaceCommitMetadata, hoodieSparkEngineContext.emptyHoodieData(), commitTime); + + // add metadata sync checkpoint info + tgtReplaceCommitMetadata.addMetadata(SyncMetadata.TABLE_SYNC_METADATA, syncMetadata.toJson()); + commitMetadataInBytes = Option.of(tgtReplaceCommitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8)); + break; + case HoodieTimeline.CLEAN_ACTION: + HoodieCleanMetadata srcCleanMetadata = TimelineMetadataUtils.deserializeHoodieCleanMetadata( + sourceTableMetaClient.getCommitsTimeline().getInstantDetails(instant).get()); + HoodieCleanMetadata tgtCleanMetadata = reconstructHoodieCleanCommitMetadata(srcCleanMetadata, + writeConfig, hoodieSparkEngineContext, targetTableMetaClient); + hoodieTableMetadataWriter.update(tgtCleanMetadata, commitTime); + + commitMetadataInBytes = TimelineMetadataUtils.serializeCleanMetadata(tgtCleanMetadata); + break; + default: + break; + } + + targetTableMetaClient + .reloadActiveTimeline() + .saveAsComplete(new HoodieInstant(true, instant.getAction(), commitTime), commitMetadataInBytes); + + if (cfg.performTableMaintenance) { + runArchiver(sparkTable, writeClient.getConfig(), hoodieSparkEngineContext); + } + + if (!lastSyncCheckpoint.isPresent() || instant.getTimestamp().compareTo(lastSyncCheckpoint.get()) > 0) { + lastSyncCheckpoint = Option.of(instant.getTimestamp()); + } + } + } + } finally { + txnManager.endTransaction(Option.empty()); + } + } + + private void runBootstrapSync(HoodieSparkTable sparkTable, HoodieTableMetaClient sourceTableMetaClient, + HoodieTableMetaClient targetTableMetaClient, SparkRDDWriteClient writeClient, Schema schema) throws Exception { + Option<HoodieInstant> sourceLastInstant = sourceTableMetaClient.getActiveTimeline().getWriteTimeline().filterCompletedInstants().lastInstant(); + if (!sourceLastInstant.isPresent()) { + return; + } + String commitTime = writeClient.startCommit(HoodieTimeline.REPLACE_COMMIT_ACTION, targetTableMetaClient); // single writer. will rollback any pending commits from previous round. + targetTableMetaClient + .reloadActiveTimeline() + .transitionRequestedToInflight( + HoodieTimeline.REPLACE_COMMIT_ACTION, + commitTime); + + SparkHoodieBackedMetadataSyncMetadataWriter metadataWriter = Review Comment: can we close the writer instance -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
