From b71143dc260ec3ff519d93498bd9514bd588c32d Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Thu, 6 Jan 2022 17:47:41 +0530 Subject: [PATCH 01/21] [HUDI-3173] Add INDEX action type and corresponding commit metadata Fix timeline tests Make MetadataPartitionType#all generic Add instant info to plan and address review comments [HUDI-3175] Add index planner and executor Fix enum constant Rebase and resolve conflicts Fix some failing tests in CI Add indexer job utility --- .../apache/hudi/cli/commands/SparkMain.java | 20 +- .../hudi/client/BaseHoodieWriteClient.java | 18 +- .../apache/hudi/config/HoodieWriteConfig.java | 4 + .../HoodieBackedTableMetadataWriter.java | 107 ++++++++- .../metadata/HoodieTableMetadataWriter.java | 6 +- .../org/apache/hudi/table/HoodieTable.java | 25 +- .../action/index/RunIndexActionExecutor.java | 185 +++++++++++++++ .../index/ScheduleIndexActionExecutor.java | 83 +++++++ .../FlinkHoodieBackedTableMetadataWriter.java | 5 + .../table/HoodieFlinkCopyOnWriteTable.java | 12 + .../table/HoodieJavaCopyOnWriteTable.java | 14 ++ .../SparkHoodieBackedTableMetadataWriter.java | 9 + .../table/HoodieSparkCopyOnWriteTable.java | 14 ++ hudi-common/pom.xml | 3 + .../main/avro/HoodieArchivedMetaEntry.avsc | 8 + .../main/avro/HoodieIndexCommitMetadata.avsc | 51 ++++ .../main/avro/HoodieIndexPartitionInfo.avsc | 48 ++++ .../src/main/avro/HoodieIndexPlan.avsc | 43 ++++ .../common/config/HoodieMetadataConfig.java | 12 + .../hudi/common/model/WriteOperationType.java | 4 + .../table/timeline/HoodieActiveTimeline.java | 67 +++++- .../timeline/HoodieArchivedTimeline.java | 7 +- .../table/timeline/HoodieDefaultTimeline.java | 17 +- .../common/table/timeline/HoodieInstant.java | 4 + .../common/table/timeline/HoodieTimeline.java | 36 ++- .../table/timeline/TimelineMetadataUtils.java | 18 ++ .../hudi/metadata/BaseTableMetadata.java | 10 +- .../timeline/TestHoodieActiveTimeline.java | 15 +- .../hudi/utilities/HoodieClusteringJob.java | 52 ++--- .../apache/hudi/utilities/HoodieIndexer.java | 219 ++++++++++++++++++ .../apache/hudi/utilities/UtilHelpers.java | 42 ++-- .../functional/TestHoodieDeltaStreamer.java | 13 +- 32 files changed, 1081 insertions(+), 90 deletions(-) create mode 100644 hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java create mode 100644 hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java create mode 100644 hudi-common/src/main/avro/HoodieIndexCommitMetadata.avsc create mode 100644 hudi-common/src/main/avro/HoodieIndexPartitionInfo.avsc create mode 100644 hudi-common/src/main/avro/HoodieIndexPlan.avsc create mode 100644 hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java index 0de1a1adfe0be..06216d2b5ad08 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java @@ -18,7 +18,6 @@ package org.apache.hudi.cli.commands; -import org.apache.hadoop.fs.Path; import org.apache.hudi.DataSourceWriteOptions; import org.apache.hudi.cli.DeDupeType; import org.apache.hudi.cli.DedupeSparkJob; @@ -52,9 +51,10 @@ import org.apache.hudi.utilities.HoodieCompactionAdminTool; import org.apache.hudi.utilities.HoodieCompactionAdminTool.Operation; import org.apache.hudi.utilities.HoodieCompactor; -import org.apache.hudi.utilities.UtilHelpers; import org.apache.hudi.utilities.deltastreamer.BootstrapExecutor; import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer; + +import org.apache.hadoop.fs.Path; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SQLContext; @@ -65,6 +65,12 @@ import java.util.List; import java.util.Locale; +import static org.apache.hudi.utilities.UtilHelpers.EXECUTE; +import static org.apache.hudi.utilities.UtilHelpers.SCHEDULE; +import static org.apache.hudi.utilities.UtilHelpers.SCHEDULE_AND_EXECUTE; +import static org.apache.hudi.utilities.UtilHelpers.buildProperties; +import static org.apache.hudi.utilities.UtilHelpers.readConfig; + /** * This class deals with initializing spark context based on command entered to hudi-cli. */ @@ -192,7 +198,7 @@ public static void main(String[] args) throws Exception { configs.addAll(Arrays.asList(args).subList(9, args.length)); } returnCode = cluster(jsc, args[3], args[4], args[5], Integer.parseInt(args[6]), args[2], - Integer.parseInt(args[7]), HoodieClusteringJob.EXECUTE, propsFilePath, configs); + Integer.parseInt(args[7]), EXECUTE, propsFilePath, configs); break; case CLUSTERING_SCHEDULE_AND_EXECUTE: assert (args.length >= 8); @@ -205,7 +211,7 @@ public static void main(String[] args) throws Exception { configs.addAll(Arrays.asList(args).subList(8, args.length)); } returnCode = cluster(jsc, args[3], args[4], null, Integer.parseInt(args[5]), args[2], - Integer.parseInt(args[6]), HoodieClusteringJob.SCHEDULE_AND_EXECUTE, propsFilePath, configs); + Integer.parseInt(args[6]), SCHEDULE_AND_EXECUTE, propsFilePath, configs); break; case CLUSTERING_SCHEDULE: assert (args.length >= 7); @@ -218,7 +224,7 @@ public static void main(String[] args) throws Exception { configs.addAll(Arrays.asList(args).subList(7, args.length)); } returnCode = cluster(jsc, args[3], args[4], args[5], 1, args[2], - 0, HoodieClusteringJob.SCHEDULE, propsFilePath, configs); + 0, SCHEDULE, propsFilePath, configs); break; case CLEAN: assert (args.length >= 5); @@ -411,8 +417,8 @@ private static int doBootstrap(JavaSparkContext jsc, String tableName, String ta String bootstrapIndexClass, String selectorClass, String keyGenerator, String fullBootstrapInputProvider, String payloadClassName, String enableHiveSync, String propsFilePath, List configs) throws IOException { - TypedProperties properties = propsFilePath == null ? UtilHelpers.buildProperties(configs) - : UtilHelpers.readConfig(jsc.hadoopConfiguration(), new Path(propsFilePath), configs).getProps(true); + TypedProperties properties = propsFilePath == null ? buildProperties(configs) + : readConfig(jsc.hadoopConfiguration(), new Path(propsFilePath), configs).getProps(true); properties.setProperty(HoodieBootstrapConfig.BASE_PATH.key(), sourcePath); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java index 5eb8e270a1436..9a4099f4f7c5d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java @@ -24,6 +24,8 @@ import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.avro.model.HoodieIndexCommitMetadata; +import org.apache.hudi.avro.model.HoodieIndexPlan; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRestorePlan; import org.apache.hudi.avro.model.HoodieRollbackMetadata; @@ -400,7 +402,6 @@ protected void rollbackFailedBootstrap() { public abstract O bulkInsert(I records, final String instantTime, Option userDefinedBulkInsertPartitioner); - /** * Loads the given HoodieRecords, as inserts into the table. This is suitable for doing big bulk loads into a Hoodie * table for the very first time (e.g: converting an existing table to Hoodie). The input records should contain no @@ -925,6 +926,21 @@ public boolean scheduleCompactionAtInstant(String instantTime, Option scheduleIndexing(List partitions) { + String instantTime = HoodieActiveTimeline.createNewInstantTime(); + return scheduleIndexingAtInstant(partitions, instantTime) ? Option.of(instantTime) : Option.empty(); + } + + private boolean scheduleIndexingAtInstant(List partitionsToIndex, String instantTime) throws HoodieIOException { + Option indexPlan = createTable(config, hadoopConf, config.isMetadataTableEnabled()) + .scheduleIndex(context, instantTime, partitionsToIndex); + return indexPlan.isPresent(); + } + + public Option index(String indexInstantTime) { + return createTable(config, hadoopConf, config.isMetadataTableEnabled()).index(context, indexInstantTime); + } + /** * Performs Compaction for the workload stored in instant-time. * diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 776c9066767c9..6c52a01b311bc 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -1892,6 +1892,10 @@ public boolean isMetadataAsyncClean() { return getBoolean(HoodieMetadataConfig.ASYNC_CLEAN_ENABLE); } + public boolean isMetadataAsyncIndex() { + return getBooleanOrDefault(HoodieMetadataConfig.ASYNC_INDEX_ENABLE); + } + public int getMetadataMaxCommitsToKeep() { return getInt(HoodieMetadataConfig.MAX_COMMITS_TO_KEEP); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index ce167f7c2c750..e07601c01af9d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -19,6 +19,8 @@ package org.apache.hudi.metadata; import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieIndexPartitionInfo; +import org.apache.hudi.avro.model.HoodieIndexPlan; import org.apache.hudi.avro.model.HoodieInstantInfo; import org.apache.hudi.avro.model.HoodieMetadataRecord; import org.apache.hudi.avro.model.HoodieRestoreMetadata; @@ -50,8 +52,10 @@ import org.apache.hudi.common.table.marker.MarkerType; 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.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; +import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; @@ -62,6 +66,7 @@ import org.apache.hudi.config.metrics.HoodieMetricsGraphiteConfig; import org.apache.hudi.config.metrics.HoodieMetricsJmxConfig; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieIndexException; import org.apache.hudi.exception.HoodieMetadataException; import org.apache.avro.specific.SpecificRecordBase; @@ -80,6 +85,7 @@ import java.util.HashMap; import java.util.LinkedList; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Properties; import java.util.concurrent.atomic.AtomicLong; @@ -336,6 +342,8 @@ protected abstract void initialize(HoodieEngineCo Option actionMetadata, Option inflightInstantTimestamp); + protected abstract void scheduleIndex(List partitions); + public void initTableMetadata() { try { if (this.metadata != null) { @@ -392,6 +400,12 @@ protected void initializeIfNeeded(HoodieTableMeta } if (!exists) { + if (metadataWriteConfig.isMetadataAsyncIndex()) { + // with async metadata indexing enabled, there can be inflight writers + // TODO: schedule indexing only for enabled partition types + scheduleIndex(MetadataPartitionType.allPaths()); + return; + } // Initialize for the first time by listing partitions and files directly from the file system if (initializeFromFilesystem(dataMetaClient, inflightInstantTimestamp)) { metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.INITIALIZE_STR, timer.endTimer())); @@ -553,7 +567,7 @@ private List listAllPartitions(HoodieTableMetaClient datasetMetaC // In each round we will list a section of directories int numDirsToList = Math.min(fileListingParallelism, pathsToList.size()); // List all directories in parallel - List processedDirectories = engineContext.map(pathsToList.subList(0, numDirsToList), path -> { + List processedDirectories = engineContext.map(pathsToList.subList(0, numDirsToList), path -> { FileSystem fs = path.getFileSystem(conf.get()); String relativeDirPath = FSUtils.getRelativePartitionPath(new Path(datasetBasePath), path); return new DirectoryInfo(relativeDirPath, fs.listStatus(path)); @@ -624,8 +638,14 @@ private void initializeFileGroups(HoodieTableMetaClient dataMetaClient, Metadata LOG.info(String.format("Creating %d file groups for partition %s with base fileId %s at instant time %s", fileGroupCount, metadataPartition.getPartitionPath(), metadataPartition.getFileIdPrefix(), instantTime)); + HoodieTableFileSystemView fsView = HoodieTableMetadataUtil.getFileSystemView(metadataMetaClient); + List fileSlices = HoodieTableMetadataUtil.getPartitionLatestFileSlices(metadataMetaClient, Option.ofNullable(fsView), metadataPartition.getPartitionPath()); for (int i = 0; i < fileGroupCount; ++i) { final String fileGroupFileId = String.format("%s%04d", metadataPartition.getFileIdPrefix(), i); + // if a writer or async indexer had already initialized the filegroup then continue + if (!fileSlices.isEmpty() && fileSlices.stream().anyMatch(fileSlice -> fileGroupFileId.equals(fileSlice.getFileGroupId().getFileId()))) { + continue; + } try { HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder() .onParentPath(FSUtils.getPartitionPath(metadataWriteConfig.getBasePath(), metadataPartition.getPartitionPath())) @@ -663,20 +683,99 @@ private interface ConvertMetadataFunction { /** * Processes commit metadata from data table and commits to metadata table. + * * @param instantTime instant time of interest. * @param convertMetadataFunction converter function to convert the respective metadata to List of HoodieRecords to be written to metadata table. * @param type of commit metadata. * @param canTriggerTableService true if table services can be triggered. false otherwise. */ private void processAndCommit(String instantTime, ConvertMetadataFunction convertMetadataFunction, boolean canTriggerTableService) { - if (enabled && metadata != null) { - Map> partitionRecordsMap = convertMetadataFunction.convertMetadata(); - commit(instantTime, partitionRecordsMap, canTriggerTableService); + List partitionsToUpdate = getMetadataPartitionsToUpdate(); + partitionsToUpdate.forEach(p -> { + if (enabled && metadata != null) { + try { + initializeFileGroups(dataMetaClient, MetadataPartitionType.valueOf(p.toUpperCase(Locale.ROOT)), instantTime, 1); + } catch (IOException e) { + throw new HoodieIndexException(String.format("Unable to initialize file groups for metadata partition: %s, instant: %s", p, instantTime)); + } + Map> partitionRecordsMap = convertMetadataFunction.convertMetadata(); + commit(instantTime, partitionRecordsMap, canTriggerTableService); + } + }); + } + + private List getMetadataPartitionsToUpdate() { + // find last (pending or) completed index instant and get partitions (to be) written + Option lastIndexingInstant = dataMetaClient.getActiveTimeline() + .getTimelineOfActions(CollectionUtils.createImmutableSet(HoodieTimeline.INDEX_ACTION)).lastInstant(); + if (lastIndexingInstant.isPresent()) { + try { + // TODO: handle inflight instant, if it is inflight then read from requested file. + HoodieIndexPlan indexPlan = TimelineMetadataUtils.deserializeIndexPlan( + dataMetaClient.getActiveTimeline().readIndexPlanAsBytes(lastIndexingInstant.get()).get()); + return indexPlan.getIndexPartitionInfos().stream().map(HoodieIndexPartitionInfo::getMetadataPartitionPath).collect(Collectors.toList()); + } catch (IOException e) { + LOG.warn("Could not read index plan. Falling back to FileSystem.exists() check."); + return getExistingMetadataPartitions(); + } } + // TODO: return only enabled partitions + return MetadataPartitionType.allPaths(); + } + + private List getExistingMetadataPartitions() { + return MetadataPartitionType.allPaths().stream() + .filter(p -> { + try { + // TODO: avoid fs.exists() check + return metadataMetaClient.getFs().exists(FSUtils.getPartitionPath(metadataWriteConfig.getBasePath(), p)); + } catch (IOException e) { + return false; + } + }) + .collect(Collectors.toList()); + } + + @Override + public void index(HoodieEngineContext engineContext, List indexPartitionInfos) { + indexPartitionInfos.forEach(indexPartitionInfo -> { + String indexUptoInstantTime = indexPartitionInfo.getIndexUptoInstant(); + String relativePartitionPath = indexPartitionInfo.getMetadataPartitionPath(); + LOG.info(String.format("Creating a new metadata index for partition '%s' under path %s upto instant %s", + relativePartitionPath, metadataWriteConfig.getBasePath(), indexUptoInstantTime)); + try { + HoodieTableMetaClient.withPropertyBuilder() + .setTableType(HoodieTableType.MERGE_ON_READ) + .setTableName(tableName) + .setArchiveLogFolder(ARCHIVELOG_FOLDER.defaultValue()) + .setPayloadClassName(HoodieMetadataPayload.class.getName()) + .setBaseFileFormat(HoodieFileFormat.HFILE.toString()) + .setRecordKeyFields(RECORD_KEY_FIELD_NAME) + .setPopulateMetaFields(dataWriteConfig.getMetadataConfig().populateMetaFields()) + .setKeyGeneratorClassProp(HoodieTableMetadataKeyGenerator.class.getCanonicalName()) + .initTable(hadoopConf.get(), metadataWriteConfig.getBasePath()); + initTableMetadata(); + initializeFileGroups(dataMetaClient, MetadataPartitionType.valueOf(relativePartitionPath.toUpperCase(Locale.ROOT)), indexUptoInstantTime, 1); + } catch (IOException e) { + throw new HoodieIndexException(String.format("Unable to initialize file groups for metadata partition: %s, indexUptoInstant: %s", + relativePartitionPath, indexUptoInstantTime)); + } + + // List all partitions in the basePath of the containing dataset + LOG.info("Initializing metadata table by using file listings in " + dataWriteConfig.getBasePath()); + engineContext.setJobStatus(this.getClass().getSimpleName(), "MetadataIndex: initializing metadata table by listing files and partitions"); + List dirInfoList = listAllPartitions(dataMetaClient); + + // During bootstrap, the list of files to be committed can be huge. So creating a HoodieCommitMetadata out of these + // large number of files and calling the existing update(HoodieCommitMetadata) function does not scale well. + // Hence, we have a special commit just for the bootstrap scenario. + initialCommit(indexUptoInstantTime); + }); } /** * Update from {@code HoodieCommitMetadata}. + * * @param commitMetadata {@code HoodieCommitMetadata} * @param instantTime Timestamp at which the commit was performed * @param isTableServiceAction {@code true} if commit metadata is pertaining to a table service. {@code false} otherwise. diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java index 4f5ac027c91eb..f9ce081b1d887 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java @@ -19,17 +19,22 @@ package org.apache.hudi.metadata; import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieIndexPartitionInfo; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; +import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCommitMetadata; import java.io.Serializable; +import java.util.List; /** * Interface that supports updating metadata for a given table, as actions complete. */ public interface HoodieTableMetadataWriter extends Serializable, AutoCloseable { + void index(HoodieEngineContext engineContext, List indexPartitionInfos); + /** * Update the metadata table due to a COMMIT operation. * @param commitMetadata commit metadata of the operation of interest. @@ -59,5 +64,4 @@ public interface HoodieTableMetadataWriter extends Serializable, AutoCloseable { * @param instantTime instant time of the commit. */ void update(HoodieRollbackMetadata rollbackMetadata, String instantTime); - } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java index 62a4f089a45b5..d3cf118db3178 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java @@ -23,6 +23,8 @@ import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.avro.model.HoodieIndexCommitMetadata; +import org.apache.hudi.avro.model.HoodieIndexPlan; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRestorePlan; import org.apache.hudi.avro.model.HoodieRollbackMetadata; @@ -66,6 +68,7 @@ import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.metadata.HoodieTableMetadataWriter; +import org.apache.hudi.metadata.MetadataPartitionType; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; import org.apache.hudi.table.marker.WriteMarkers; @@ -447,7 +450,6 @@ public abstract Option scheduleClustering(HoodieEngineCont */ public abstract void rollbackBootstrap(HoodieEngineContext context, String instantTime); - /** * Schedule cleaning for the instant time. * @@ -497,6 +499,25 @@ public abstract HoodieRollbackMetadata rollback(HoodieEngineContext context, boolean deleteInstants, boolean skipLocking); + /** + * Schedules Indexing for the table to the given instant. + * + * @param context HoodieEngineContext + * @param indexInstantTime Instant time for scheduling index action. + * @param partitionsToIndex List of {@link MetadataPartitionType#partitionPath()} that should be indexed. + * @return HoodieIndexPlan containing metadata partitions and instant upto which they should be indexed. + */ + public abstract Option scheduleIndex(HoodieEngineContext context, String indexInstantTime, List partitionsToIndex); + + /** + * Execute requested index action. + * + * @param context HoodieEngineContext + * @param indexInstantTime Instant time for which index action was scheduled. + * @return HoodieIndexCommitMetadata containing write stats for each metadata partition. + */ + public abstract Option index(HoodieEngineContext context, String indexInstantTime); + /** * Create a savepoint at the specified instant, so that the table can be restored * to this point-in-timeline later if needed. @@ -764,7 +785,7 @@ public HoodieEngineContext getContext() { * Get Table metadata writer. * * @param triggeringInstantTimestamp - The instant that is triggering this metadata write - * @return instance of {@link HoodieTableMetadataWriter + * @return instance of {@link HoodieTableMetadataWriter} */ public final Option getMetadataWriter(String triggeringInstantTimestamp) { return getMetadataWriter(triggeringInstantTimestamp, Option.empty()); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java new file mode 100644 index 0000000000000..43b958df0d84a --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java @@ -0,0 +1,185 @@ +/* + * 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.table.action.index; + +import org.apache.hudi.avro.model.HoodieCleanMetadata; +import org.apache.hudi.avro.model.HoodieIndexCommitMetadata; +import org.apache.hudi.avro.model.HoodieIndexPartitionInfo; +import org.apache.hudi.avro.model.HoodieIndexPlan; +import org.apache.hudi.avro.model.HoodieRestoreMetadata; +import org.apache.hudi.avro.model.HoodieRollbackMetadata; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieRecordPayload; +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.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.util.CleanerUtils; +import org.apache.hudi.common.util.HoodieTimer; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIndexException; +import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.metadata.HoodieTableMetadataWriter; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.BaseActionExecutor; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class RunIndexActionExecutor extends BaseActionExecutor> { + + private static final Logger LOG = LogManager.getLogger(RunIndexActionExecutor.class); + private static final Integer INDEX_COMMIT_METADATA_VERSION_1 = 1; + private static final Integer LATEST_INDEX_COMMIT_METADATA_VERSION = INDEX_COMMIT_METADATA_VERSION_1; + private static final int MAX_CONCURRENT_INDEXING = 1; + + public RunIndexActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, String instantTime) { + super(context, config, table, instantTime); + } + + @Override + public Option execute() { + HoodieTimer indexTimer = new HoodieTimer(); + indexTimer.startTimer(); + + HoodieInstant indexInstant = table.getActiveTimeline() + .filterPendingIndexTimeline() + .filter(instant -> instant.getTimestamp().equals(instantTime)) + .lastInstant() + .orElseThrow(() -> new HoodieIndexException(String.format("No pending index instant found: %s", instantTime))); + ValidationUtils.checkArgument(HoodieInstant.State.INFLIGHT.equals(indexInstant.getState()), + String.format("Index instant %s already inflight", instantTime)); + try { + // read HoodieIndexPlan assuming indexInstant is requested + // TODO: handle inflight instant, if it is inflight then throw error. + HoodieIndexPlan indexPlan = TimelineMetadataUtils.deserializeIndexPlan(table.getActiveTimeline().readIndexPlanAsBytes(indexInstant).get()); + List indexPartitionInfos = indexPlan.getIndexPartitionInfos(); + if (indexPartitionInfos == null || indexPartitionInfos.isEmpty()) { + throw new HoodieIndexException(String.format("No partitions to index for instant: %s", instantTime)); + } + // transition requested indexInstant to inflight + table.getActiveTimeline().transitionIndexRequestedToInflight(indexInstant, Option.empty()); + // start indexing for each partition + HoodieTableMetadataWriter metadataWriter = table.getMetadataWriter(instantTime) + .orElseThrow(() -> new HoodieIndexException(String.format("Could not get metadata writer to run index action for instant: %s", instantTime))); + metadataWriter.index(context, indexPartitionInfos); + // get all completed instants since the plan completed + // assumption is that all metadata partitions had same instant upto which they were scheduled to be indexed + String indexUptoInstant = indexPartitionInfos.get(0).getIndexUptoInstant(); + Stream remainingInstantsToIndex = table.getActiveTimeline().getWriteTimeline().getReverseOrderedInstants() + .filter(instant -> instant.isCompleted() && HoodieActiveTimeline.GREATER_THAN.test(instant.getTimestamp(), indexUptoInstant)); + // reconcile with metadata table timeline + String metadataBasePath = HoodieTableMetadata.getMetadataTableBasePath(table.getMetaClient().getBasePath()); + HoodieTableMetaClient metadataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(metadataBasePath).build(); + Set metadataCompletedTimeline = metadataMetaClient.getActiveTimeline() + .getCommitsTimeline().filterCompletedInstants().getInstants().collect(Collectors.toSet()); + List finalRemainingInstantsToIndex = remainingInstantsToIndex.map( + instant -> new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instant.getTimestamp()) + ).filter(instant -> !metadataCompletedTimeline.contains(instant)).collect(Collectors.toList()); + + // index all remaining instants with a timeout + ExecutorService executorService = Executors.newFixedThreadPool(MAX_CONCURRENT_INDEXING); + Future postRequestIndexingTaskFuture = executorService.submit(new PostRequestIndexingTask(metadataWriter, finalRemainingInstantsToIndex)); + try { + // TODO: configure timeout + postRequestIndexingTaskFuture.get(60, TimeUnit.SECONDS); + } catch (TimeoutException | InterruptedException | ExecutionException e) { + postRequestIndexingTaskFuture.cancel(true); + } finally { + executorService.shutdownNow(); + } + Option lastMetadataInstant = metadataMetaClient.reloadActiveTimeline().getCommitsTimeline().filterCompletedInstants().lastInstant(); + if (lastMetadataInstant.isPresent() && indexUptoInstant.equals(lastMetadataInstant.get().getTimestamp())) { + return Option.of(HoodieIndexCommitMetadata.newBuilder() + .setVersion(LATEST_INDEX_COMMIT_METADATA_VERSION).setIndexPartitionInfos(indexPartitionInfos).build()); + } + List finalIndexPartitionInfos = indexPartitionInfos.stream() + .map(info -> new HoodieIndexPartitionInfo( + info.getVersion(), + info.getMetadataPartitionPath(), + lastMetadataInstant.get().getTimestamp())).collect(Collectors.toList()); + return Option.of(HoodieIndexCommitMetadata.newBuilder() + .setVersion(LATEST_INDEX_COMMIT_METADATA_VERSION).setIndexPartitionInfos(finalIndexPartitionInfos).build()); + } catch (IOException e) { + throw new HoodieIndexException(String.format("Unable to index instant: %s", indexInstant)); + } + } + + class PostRequestIndexingTask implements Runnable { + + private final HoodieTableMetadataWriter metadataWriter; + private final List instantsToIndex; + + PostRequestIndexingTask(HoodieTableMetadataWriter metadataWriter, List instantsToIndex) { + this.metadataWriter = metadataWriter; + this.instantsToIndex = instantsToIndex; + } + + @Override + public void run() { + while (!Thread.interrupted()) { + for (HoodieInstant instant : instantsToIndex) { + try { + switch (instant.getAction()) { + case HoodieTimeline.COMMIT_ACTION: + case HoodieTimeline.DELTA_COMMIT_ACTION: + case HoodieTimeline.REPLACE_COMMIT_ACTION: + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(table.getActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class); + metadataWriter.update(commitMetadata, instant.getTimestamp(), false); + break; + case HoodieTimeline.CLEAN_ACTION: + HoodieCleanMetadata cleanMetadata = CleanerUtils.getCleanerMetadata(table.getMetaClient(), instant); + metadataWriter.update(cleanMetadata, instant.getTimestamp()); + break; + case HoodieTimeline.RESTORE_ACTION: + HoodieRestoreMetadata restoreMetadata = TimelineMetadataUtils.deserializeHoodieRestoreMetadata(table.getActiveTimeline().getInstantDetails(instant).get()); + metadataWriter.update(restoreMetadata, instant.getTimestamp()); + break; + case HoodieTimeline.ROLLBACK_ACTION: + HoodieRollbackMetadata rollbackMetadata = TimelineMetadataUtils.deserializeHoodieRollbackMetadata(table.getActiveTimeline().getInstantDetails(instant).get()); + metadataWriter.update(rollbackMetadata, instant.getTimestamp()); + break; + default: + throw new IllegalStateException("Unexpected value: " + instant.getAction()); + } + } catch (IOException e) { + e.printStackTrace(); + } + } + } + } + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java new file mode 100644 index 0000000000000..d0418b4c7f9d1 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java @@ -0,0 +1,83 @@ +/* + * 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.table.action.index; + +import org.apache.hudi.avro.model.HoodieIndexPartitionInfo; +import org.apache.hudi.avro.model.HoodieIndexPlan; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieRecordPayload; +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.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.exception.HoodieIndexException; +import org.apache.hudi.metadata.MetadataPartitionType; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.BaseActionExecutor; + +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; + +public class ScheduleIndexActionExecutor extends BaseActionExecutor> { + + private static final Logger LOG = LogManager.getLogger(ScheduleIndexActionExecutor.class); + private static final Integer INDEX_PLAN_VERSION_1 = 1; + private static final Integer LATEST_INDEX_PLAN_VERSION = INDEX_PLAN_VERSION_1; + + private final List partitionsToIndex; + + public ScheduleIndexActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, String instantTime, List partitionsToIndex) { + super(context, config, table, instantTime); + this.partitionsToIndex = partitionsToIndex; + } + + @Override + public Option execute() { + // validate partitionsToIndex + if (!MetadataPartitionType.allPaths().containsAll(partitionsToIndex)) { + throw new HoodieIndexException("Not all partitions are valid: " + partitionsToIndex); + } + // get last completed instant + Option indexUptoInstant = table.getActiveTimeline().filterCompletedInstants().lastInstant(); + if (indexUptoInstant.isPresent()) { + final HoodieInstant indexInstant = HoodieTimeline.getIndexRequestedInstant(instantTime); + // for each partitionToIndex add that time to the plan + List indexPartitionInfos = partitionsToIndex.stream() + .map(p -> new HoodieIndexPartitionInfo(LATEST_INDEX_PLAN_VERSION, p, indexUptoInstant.get().getTimestamp())) + .collect(Collectors.toList()); + HoodieIndexPlan indexPlan = new HoodieIndexPlan(LATEST_INDEX_PLAN_VERSION, indexPartitionInfos); + try { + table.getActiveTimeline().saveToPendingIndexCommit(indexInstant, TimelineMetadataUtils.serializeIndexPlan(indexPlan)); + } catch (IOException e) { + LOG.error("Error while saving index requested file", e); + throw new HoodieIOException(e.getMessage(), e); + } + table.getMetaClient().reloadActiveTimeline(); + return Option.of(indexPlan); + } + return Option.empty(); + } +} diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java index 9351ccf178075..85417ad3d9c5e 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java @@ -99,6 +99,11 @@ protected void initialize(HoodieEngineContext eng } } + @Override + protected void scheduleIndex(List partitions) { + throw new UnsupportedOperationException("Metadata indexing not supported for Flink table yet."); + } + @Override protected void commit(String instantTime, Map> partitionRecordsMap, boolean canTriggerTableService) { diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java index 14937d6fee244..6a4807297735b 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java @@ -22,6 +22,8 @@ import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.avro.model.HoodieIndexCommitMetadata; +import org.apache.hudi.avro.model.HoodieIndexPlan; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRestorePlan; import org.apache.hudi.avro.model.HoodieRollbackMetadata; @@ -339,6 +341,16 @@ public HoodieRollbackMetadata rollback(HoodieEngineContext context, String rollb return new CopyOnWriteRollbackActionExecutor(context, config, this, rollbackInstantTime, commitInstant, deleteInstants, skipLocking).execute(); } + @Override + public Option scheduleIndex(HoodieEngineContext context, String indexInstantTime, List partitionsToIndex) { + throw new HoodieNotSupportedException("Indexing is not supported for a Flink table yet."); + } + + @Override + public Option index(HoodieEngineContext context, String indexInstantTime) { + throw new HoodieNotSupportedException("Indexing is not supported for a Flink table yet."); + } + @Override public HoodieSavepointMetadata savepoint(HoodieEngineContext context, String instantToSavepoint, String user, String comment) { throw new HoodieNotSupportedException("Savepoint is not supported yet"); diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java index 06c23049d974d..a07481c210963 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java @@ -22,6 +22,8 @@ import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.avro.model.HoodieIndexCommitMetadata; +import org.apache.hudi.avro.model.HoodieIndexPlan; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRestorePlan; import org.apache.hudi.avro.model.HoodieRollbackMetadata; @@ -61,6 +63,8 @@ import org.apache.hudi.table.action.commit.JavaMergeHelper; import org.apache.hudi.table.action.commit.JavaUpsertCommitActionExecutor; import org.apache.hudi.table.action.commit.JavaUpsertPreppedCommitActionExecutor; +import org.apache.hudi.table.action.index.RunIndexActionExecutor; +import org.apache.hudi.table.action.index.ScheduleIndexActionExecutor; import org.apache.hudi.table.action.restore.CopyOnWriteRestoreActionExecutor; import org.apache.hudi.table.action.rollback.BaseRollbackPlanActionExecutor; import org.apache.hudi.table.action.rollback.CopyOnWriteRollbackActionExecutor; @@ -241,6 +245,16 @@ public HoodieRollbackMetadata rollback(HoodieEngineContext context, context, config, this, rollbackInstantTime, commitInstant, deleteInstants, skipLocking).execute(); } + @Override + public Option scheduleIndex(HoodieEngineContext context, String indexInstantTime, List partitionsToIndex) { + return new ScheduleIndexActionExecutor<>(context, config, this, indexInstantTime, partitionsToIndex).execute(); + } + + @Override + public Option index(HoodieEngineContext context, String indexInstantTime) { + return new RunIndexActionExecutor<>(context, config, this, indexInstantTime).execute(); + } + @Override public HoodieSavepointMetadata savepoint(HoodieEngineContext context, String instantToSavepoint, diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java index d0173f984a2f0..d27995bf0a271 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java @@ -121,6 +121,15 @@ protected void initialize(HoodieEngineContext eng } } + @Override + protected void scheduleIndex(List partitions) { + ValidationUtils.checkState(metadataMetaClient != null, "Metadata table is not fully initialized yet."); + ValidationUtils.checkState(enabled, "Metadata table cannot be indexed as it is not enabled."); + try (SparkRDDWriteClient writeClient = new SparkRDDWriteClient(engineContext, metadataWriteConfig, true)) { + writeClient.scheduleIndexing(partitions); + } + } + @Override protected void commit(String instantTime, Map> partitionRecordsMap, boolean canTriggerTableService) { ValidationUtils.checkState(metadataMetaClient != null, "Metadata table is not fully initialized yet."); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java index 8f5211212253a..5c1541a4bf6d5 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java @@ -24,6 +24,8 @@ import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.avro.model.HoodieIndexCommitMetadata; +import org.apache.hudi.avro.model.HoodieIndexPlan; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRestorePlan; import org.apache.hudi.avro.model.HoodieRollbackMetadata; @@ -73,6 +75,8 @@ import org.apache.hudi.table.action.commit.SparkInsertPreppedCommitActionExecutor; import org.apache.hudi.table.action.commit.SparkUpsertCommitActionExecutor; import org.apache.hudi.table.action.commit.SparkUpsertPreppedCommitActionExecutor; +import org.apache.hudi.table.action.index.RunIndexActionExecutor; +import org.apache.hudi.table.action.index.ScheduleIndexActionExecutor; import org.apache.hudi.table.action.restore.CopyOnWriteRestoreActionExecutor; import org.apache.hudi.table.action.rollback.BaseRollbackPlanActionExecutor; import org.apache.hudi.table.action.rollback.CopyOnWriteRollbackActionExecutor; @@ -346,6 +350,16 @@ public HoodieRollbackMetadata rollback(HoodieEngineContext context, String rollb deleteInstants, skipLocking).execute(); } + @Override + public Option scheduleIndex(HoodieEngineContext context, String indexInstantTime, List partitionsToIndex) { + return new ScheduleIndexActionExecutor<>(context, config, this, indexInstantTime, partitionsToIndex).execute(); + } + + @Override + public Option index(HoodieEngineContext context, String indexInstantTime) { + return new RunIndexActionExecutor<>(context, config, this, indexInstantTime).execute(); + } + @Override public HoodieSavepointMetadata savepoint(HoodieEngineContext context, String instantToSavepoint, String user, String comment) { return new SavepointActionExecutor<>(context, config, this, instantToSavepoint, user, comment).execute(); diff --git a/hudi-common/pom.xml b/hudi-common/pom.xml index a7de212e53a80..f60573875cf43 100644 --- a/hudi-common/pom.xml +++ b/hudi-common/pom.xml @@ -90,6 +90,9 @@ ${basedir}/src/main/avro/HoodieClusteringPlan.avsc ${basedir}/src/main/avro/HoodieRequestedReplaceMetadata.avsc ${basedir}/src/main/avro/HoodieMetadata.avsc + ${basedir}/src/main/avro/HoodieIndexPartitionInfo.avsc + ${basedir}/src/main/avro/HoodieIndexPlan.avsc + ${basedir}/src/main/avro/HoodieIndexCommitMetadata.avsc ${basedir}/src/main/avro/HoodieArchivedMetaEntry.avsc diff --git a/hudi-common/src/main/avro/HoodieArchivedMetaEntry.avsc b/hudi-common/src/main/avro/HoodieArchivedMetaEntry.avsc index c052147f718ea..81bcaf745e5b8 100644 --- a/hudi-common/src/main/avro/HoodieArchivedMetaEntry.avsc +++ b/hudi-common/src/main/avro/HoodieArchivedMetaEntry.avsc @@ -120,6 +120,14 @@ "HoodieCommitMetadata" ], "default": null + }, + { + "name":"hoodieIndexCommitMetadata", + "type":[ + "null", + "HoodieIndexCommitMetadata" + ], + "default": null } ] } diff --git a/hudi-common/src/main/avro/HoodieIndexCommitMetadata.avsc b/hudi-common/src/main/avro/HoodieIndexCommitMetadata.avsc new file mode 100644 index 0000000000000..07e6793dc49cd --- /dev/null +++ b/hudi-common/src/main/avro/HoodieIndexCommitMetadata.avsc @@ -0,0 +1,51 @@ +/* + * 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. + */ +{ + "namespace": "org.apache.hudi.avro.model", + "type": "record", + "name": "HoodieIndexCommitMetadata", + "fields": [ + { + "name": "version", + "type": [ + "int", + "null" + ], + "default": 1 + }, + { + "name": "operationType", + "type": [ + "null", + "string" + ], + "default": null + }, + { + "name": "indexPartitionInfos", + "type": [ + "null", + { + "type": "array", + "items": "HoodieIndexPartitionInfo" + } + ], + "default": null + } + ] +} diff --git a/hudi-common/src/main/avro/HoodieIndexPartitionInfo.avsc b/hudi-common/src/main/avro/HoodieIndexPartitionInfo.avsc new file mode 100644 index 0000000000000..52ed1e96aa3b3 --- /dev/null +++ b/hudi-common/src/main/avro/HoodieIndexPartitionInfo.avsc @@ -0,0 +1,48 @@ +/* + * 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. + */ +{ + "namespace": "org.apache.hudi.avro.model", + "type": "record", + "name": "HoodieIndexPartitionInfo", + "fields": [ + { + "name": "version", + "type": [ + "int", + "null" + ], + "default": 1 + }, + { + "name": "metadataPartitionPath", + "type": [ + "null", + "string" + ], + "default": null + }, + { + "name": "indexUptoInstant", + "type": [ + "null", + "string" + ], + "default": null + } + ] +} diff --git a/hudi-common/src/main/avro/HoodieIndexPlan.avsc b/hudi-common/src/main/avro/HoodieIndexPlan.avsc new file mode 100644 index 0000000000000..9fb7ec311e34a --- /dev/null +++ b/hudi-common/src/main/avro/HoodieIndexPlan.avsc @@ -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. + */ +{ + "namespace": "org.apache.hudi.avro.model", + "type": "record", + "name": "HoodieIndexPlan", + "fields": [ + { + "name": "version", + "type": [ + "int", + "null" + ], + "default": 1 + }, + { + "name": "indexPartitionInfos", + "type": [ + "null", + { + "type": "array", + "items": "HoodieIndexPartitionInfo" + } + ], + "default": null + } + ] +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java index 7d964f3582d69..f3f5c0ae511ac 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java @@ -71,6 +71,13 @@ public final class HoodieMetadataConfig extends HoodieConfig { .sinceVersion("0.7.0") .withDocumentation("Enable asynchronous cleaning for metadata table"); + // Async index + public static final ConfigProperty ASYNC_INDEX_ENABLE = ConfigProperty + .key(METADATA_PREFIX + ".index.async") + .defaultValue(false) + .sinceVersion("0.11.0") + .withDocumentation("Enable asynchronous indexing of metadata table."); + // Maximum delta commits before compaction occurs public static final ConfigProperty COMPACT_NUM_DELTA_COMMITS = ConfigProperty .key(METADATA_PREFIX + ".compact.max.delta.commits") @@ -320,6 +327,11 @@ public Builder withAsyncClean(boolean asyncClean) { return this; } + public Builder withAsyncIndex(boolean asyncIndex) { + metadataConfig.setValue(ASYNC_INDEX_ENABLE, String.valueOf(asyncIndex)); + return this; + } + public Builder withMaxNumDeltaCommitsBeforeCompaction(int maxNumDeltaCommitsBeforeCompaction) { metadataConfig.setValue(COMPACT_NUM_DELTA_COMMITS, String.valueOf(maxNumDeltaCommitsBeforeCompaction)); return this; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java b/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java index b5a3cc002366e..a75640547e2bd 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java @@ -48,6 +48,8 @@ public enum WriteOperationType { INSERT_OVERWRITE_TABLE("insert_overwrite_table"), // compact COMPACT("compact"), + + INDEX("index"), // used for old version UNKNOWN("unknown"); @@ -86,6 +88,8 @@ public static WriteOperationType fromValue(String value) { return CLUSTER; case "compact": return COMPACT; + case "index": + return INDEX; case "unknown": return UNKNOWN; default: diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java index 36dd5368d4a63..4a614d0de741a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java @@ -73,7 +73,8 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline { INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION, REQUESTED_RESTORE_EXTENSION, INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION, ROLLBACK_EXTENSION, REQUESTED_ROLLBACK_EXTENSION, INFLIGHT_ROLLBACK_EXTENSION, - REQUESTED_REPLACE_COMMIT_EXTENSION, INFLIGHT_REPLACE_COMMIT_EXTENSION, REPLACE_COMMIT_EXTENSION)); + REQUESTED_REPLACE_COMMIT_EXTENSION, INFLIGHT_REPLACE_COMMIT_EXTENSION, REPLACE_COMMIT_EXTENSION, + REQUESTED_INDEX_COMMIT_EXTENSION, INFLIGHT_INDEX_COMMIT_EXTENSION, INDEX_COMMIT_EXTENSION)); private static final Logger LOG = LogManager.getLogger(HoodieActiveTimeline.class); protected HoodieTableMetaClient metaClient; @@ -99,7 +100,6 @@ public static String createNewInstantTime() { return HoodieInstantTimeGenerator.createNewInstantTime(0); } - /** * Returns next instant time that adds N milliseconds to current time. * Ensures each instant time is atleast 1 second apart since we create instant times at second granularity @@ -334,6 +334,10 @@ public Option readCompactionPlanAsBytes(HoodieInstant instant) { } } + public Option readIndexPlanAsBytes(HoodieInstant instant) { + return readDataFromPath(new Path(metaClient.getMetaPath(), instant.getFileName())); + } + /** * Revert compaction State from inflight to requested. * @@ -647,6 +651,65 @@ public void saveToRestoreRequested(HoodieInstant instant, Option content createFileInMetaPath(instant.getFileName(), content, false); } + /** + * Transition index instant state from requested to inflight. + * + * @param requestedInstant Inflight Instant + * @return inflight instant + */ + public HoodieInstant transitionIndexRequestedToInflight(HoodieInstant requestedInstant, Option data) { + ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.INDEX_ACTION), + String.format("%s is not equal to %s action", requestedInstant.getAction(), INDEX_ACTION)); + ValidationUtils.checkArgument(requestedInstant.isRequested(), + String.format("Instant %s not in requested state", requestedInstant.getTimestamp())); + HoodieInstant inflightInstant = new HoodieInstant(State.INFLIGHT, INDEX_ACTION, requestedInstant.getTimestamp()); + transitionState(requestedInstant, inflightInstant, data); + return inflightInstant; + } + + /** + * Transition index instant state from inflight to completed. + * @param inflightInstant Inflight Instant + * @return completed instant + */ + public HoodieInstant transitionIndexInflightToComplete(HoodieInstant inflightInstant, Option data) { + ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.INDEX_ACTION), + String.format("%s is not equal to %s action", inflightInstant.getAction(), INDEX_ACTION)); + ValidationUtils.checkArgument(inflightInstant.isInflight(), + String.format("Instant %s not inflight", inflightInstant.getTimestamp())); + HoodieInstant commitInstant = new HoodieInstant(State.COMPLETED, INDEX_ACTION, inflightInstant.getTimestamp()); + transitionState(inflightInstant, commitInstant, data); + return commitInstant; + } + + /** + * Revert index instant state from inflight to requested. + * @param inflightInstant Inflight Instant + * @return requested instant + */ + public HoodieInstant revertIndexInflightToRequested(HoodieInstant inflightInstant) { + ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.INDEX_ACTION), + String.format("%s is not equal to %s action", inflightInstant.getAction(), INDEX_ACTION)); + ValidationUtils.checkArgument(inflightInstant.isInflight(), + String.format("Instant %s not inflight", inflightInstant.getTimestamp())); + HoodieInstant requestedInstant = new HoodieInstant(State.REQUESTED, INDEX_ACTION, inflightInstant.getTimestamp()); + if (metaClient.getTimelineLayoutVersion().isNullVersion()) { + transitionState(inflightInstant, requestedInstant, Option.empty()); + } else { + deleteInflight(inflightInstant); + } + return requestedInstant; + } + + /** + * Save content for inflight/requested index instant. + */ + public void saveToPendingIndexCommit(HoodieInstant instant, Option content) { + ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.INDEX_ACTION), + String.format("%s is not equal to %s action", instant.getAction(), INDEX_ACTION)); + createFileInMetaPath(instant.getFileName(), content, false); + } + private void createFileInMetaPath(String filename, Option content, boolean allowOverwrite) { Path fullPath = new Path(metaClient.getMetaPath(), filename); if (allowOverwrite || metaClient.getTimelineLayoutVersion().isNullVersion()) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java index a966fee1d20e9..c38fdb3b52db5 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java @@ -118,7 +118,8 @@ record -> HoodieInstant.State.COMPLETED.toString().equals(record.get(ACTION_STAT * * @deprecated */ - public HoodieArchivedTimeline() {} + public HoodieArchivedTimeline() { + } /** * This method is only used when this object is deserialized in a spark executor. @@ -207,6 +208,8 @@ private Option getMetadataKey(String action) { return Option.of("hoodieCompactionPlan"); case HoodieTimeline.REPLACE_COMMIT_ACTION: return Option.of("hoodieReplaceCommitMetadata"); + case HoodieTimeline.INDEX_ACTION: + return Option.of("hoodieIndexCommitMetadata"); default: LOG.error(String.format("Unknown action in metadata (%s)", action)); return Option.empty(); @@ -360,7 +363,7 @@ private int getArchivedFileSuffix(FileStatus f) { @Override public HoodieDefaultTimeline getWriteTimeline() { // filter in-memory instants - Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, REPLACE_COMMIT_ACTION); + Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, REPLACE_COMMIT_ACTION, INDEX_ACTION); return new HoodieDefaultTimeline(getInstants().filter(i -> readCommits.containsKey(i.getTimestamp())) .filter(s -> validActions.contains(s.getAction())), details); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java index 2cf111e91c812..02c5c49984c8c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java @@ -75,7 +75,8 @@ public void setInstants(List instants) { * * @deprecated */ - public HoodieDefaultTimeline() {} + public HoodieDefaultTimeline() { + } @Override public HoodieTimeline filterInflights() { @@ -108,7 +109,7 @@ public HoodieTimeline filterCompletedAndCompactionInstants() { @Override public HoodieDefaultTimeline getWriteTimeline() { - Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, REPLACE_COMMIT_ACTION); + Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, REPLACE_COMMIT_ACTION, INDEX_ACTION); return new HoodieDefaultTimeline(instants.stream().filter(s -> validActions.contains(s.getAction())), details); } @@ -181,6 +182,16 @@ public HoodieTimeline filter(Predicate filter) { return new HoodieDefaultTimeline(instants.stream().filter(filter), details); } + @Override + public HoodieTimeline filterPendingIndexTimeline() { + return new HoodieDefaultTimeline(instants.stream().filter(s -> s.getAction().equals(INDEX_ACTION) && !s.isCompleted()), details); + } + + @Override + public HoodieTimeline filterCompletedIndexTimeline() { + return new HoodieDefaultTimeline(instants.stream().filter(s -> s.getAction().equals(INDEX_ACTION)).filter(HoodieInstant::isCompleted), details); + } + /** * Get all instants (commits, delta commits) that produce new data, in the active timeline. */ @@ -194,7 +205,7 @@ public HoodieTimeline getCommitsTimeline() { */ public HoodieTimeline getAllCommitsTimeline() { return getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, - CLEAN_ACTION, COMPACTION_ACTION, SAVEPOINT_ACTION, ROLLBACK_ACTION, REPLACE_COMMIT_ACTION)); + CLEAN_ACTION, COMPACTION_ACTION, SAVEPOINT_ACTION, ROLLBACK_ACTION, REPLACE_COMMIT_ACTION, INDEX_ACTION)); } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java index 9cd0883126495..9c0d03c7b41d0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java @@ -172,6 +172,10 @@ public String getFileName() { return isInflight() ? HoodieTimeline.makeInflightReplaceFileName(timestamp) : isRequested() ? HoodieTimeline.makeRequestedReplaceFileName(timestamp) : HoodieTimeline.makeReplaceFileName(timestamp); + } else if (HoodieTimeline.INDEX_ACTION.equals(action)) { + return isInflight() ? HoodieTimeline.makeInflightIndexFileName(timestamp) + : isRequested() ? HoodieTimeline.makeRequestedIndexFileName(timestamp) + : HoodieTimeline.makeIndexCommitFileName(timestamp); } throw new IllegalArgumentException("Cannot get file name for unknown action " + action); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java index 25b9c2ec6f2e4..9b5de8ace1e16 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java @@ -55,10 +55,11 @@ public interface HoodieTimeline extends Serializable { String COMPACTION_ACTION = "compaction"; String REQUESTED_EXTENSION = ".requested"; String RESTORE_ACTION = "restore"; + String INDEX_ACTION = "index"; String[] VALID_ACTIONS_IN_TIMELINE = {COMMIT_ACTION, DELTA_COMMIT_ACTION, CLEAN_ACTION, SAVEPOINT_ACTION, RESTORE_ACTION, ROLLBACK_ACTION, - COMPACTION_ACTION, REPLACE_COMMIT_ACTION}; + COMPACTION_ACTION, REPLACE_COMMIT_ACTION, INDEX_ACTION}; String COMMIT_EXTENSION = "." + COMMIT_ACTION; String DELTA_COMMIT_EXTENSION = "." + DELTA_COMMIT_ACTION; @@ -84,6 +85,9 @@ public interface HoodieTimeline extends Serializable { String INFLIGHT_REPLACE_COMMIT_EXTENSION = "." + REPLACE_COMMIT_ACTION + INFLIGHT_EXTENSION; String REQUESTED_REPLACE_COMMIT_EXTENSION = "." + REPLACE_COMMIT_ACTION + REQUESTED_EXTENSION; String REPLACE_COMMIT_EXTENSION = "." + REPLACE_COMMIT_ACTION; + String INFLIGHT_INDEX_COMMIT_EXTENSION = "." + INDEX_ACTION + INFLIGHT_EXTENSION; + String REQUESTED_INDEX_COMMIT_EXTENSION = "." + INDEX_ACTION + REQUESTED_EXTENSION; + String INDEX_COMMIT_EXTENSION = "." + INDEX_ACTION; String INVALID_INSTANT_TS = "0"; @@ -198,6 +202,16 @@ public interface HoodieTimeline extends Serializable { */ HoodieTimeline filter(Predicate filter); + /** + * Filter this timeline to just include requested and inflight index instants. + */ + HoodieTimeline filterPendingIndexTimeline(); + + /** + * Filter this timeline to just include completed index instants. + */ + HoodieTimeline filterCompletedIndexTimeline(); + /** * If the timeline has any instants. * @@ -341,6 +355,14 @@ static HoodieInstant getRollbackRequestedInstant(HoodieInstant instant) { return instant.isRequested() ? instant : HoodieTimeline.getRequestedInstant(instant); } + static HoodieInstant getIndexRequestedInstant(final String timestamp) { + return new HoodieInstant(State.REQUESTED, INDEX_ACTION, timestamp); + } + + static HoodieInstant getIndexInflightInstant(final String timestamp) { + return new HoodieInstant(State.INFLIGHT, INDEX_ACTION, timestamp); + } + /** * Returns the inflight instant corresponding to the instant being passed. Takes care of changes in action names * between inflight and completed instants (compaction <=> commit). @@ -454,4 +476,16 @@ static String makeFileNameAsComplete(String fileName) { static String makeFileNameAsInflight(String fileName) { return StringUtils.join(fileName, HoodieTimeline.INFLIGHT_EXTENSION); } + + static String makeIndexCommitFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.INDEX_COMMIT_EXTENSION); + } + + static String makeInflightIndexFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.INFLIGHT_INDEX_COMMIT_EXTENSION); + } + + static String makeRequestedIndexFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.REQUESTED_INDEX_COMMIT_EXTENSION); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineMetadataUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineMetadataUtils.java index 70a23f1b4c0fb..b50846b8780bf 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineMetadataUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineMetadataUtils.java @@ -22,6 +22,8 @@ import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.avro.model.HoodieIndexCommitMetadata; +import org.apache.hudi.avro.model.HoodieIndexPlan; import org.apache.hudi.avro.model.HoodieInstantInfo; import org.apache.hudi.avro.model.HoodieReplaceCommitMetadata; import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; @@ -137,6 +139,14 @@ public static Option serializeRequestedReplaceMetadata(HoodieRequestedRe return serializeAvroMetadata(clusteringPlan, HoodieRequestedReplaceMetadata.class); } + public static Option serializeIndexPlan(HoodieIndexPlan indexPlan) throws IOException { + return serializeAvroMetadata(indexPlan, HoodieIndexPlan.class); + } + + public static Option serializeIndexCommitMetadata(HoodieIndexCommitMetadata indexCommitMetadata) throws IOException { + return serializeAvroMetadata(indexCommitMetadata, HoodieIndexCommitMetadata.class); + } + public static Option serializeAvroMetadata(T metadata, Class clazz) throws IOException { DatumWriter datumWriter = new SpecificDatumWriter<>(clazz); @@ -180,6 +190,14 @@ public static HoodieReplaceCommitMetadata deserializeHoodieReplaceMetadata(byte[ return deserializeAvroMetadata(bytes, HoodieReplaceCommitMetadata.class); } + public static HoodieIndexPlan deserializeIndexPlan(byte[] bytes) throws IOException { + return deserializeAvroMetadata(bytes, HoodieIndexPlan.class); + } + + public static HoodieIndexCommitMetadata deserializeIndexCommitMetadata(byte[] bytes) throws IOException { + return deserializeAvroMetadata(bytes, HoodieIndexCommitMetadata.class); + } + public static T deserializeAvroMetadata(byte[] bytes, Class clazz) throws IOException { DatumReader reader = new SpecificDatumReader<>(clazz); diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java index 2dce66e700479..73b5dcb89f3af 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java @@ -284,7 +284,7 @@ protected List fetchAllPartitionPaths() { List partitions = Collections.emptyList(); if (hoodieRecord.isPresent()) { - mayBeHandleSpuriousDeletes(hoodieRecord, "\"all partitions\""); + handleSpuriousDeletes(hoodieRecord, "\"all partitions\""); partitions = hoodieRecord.get().getData().getFilenames(); // Partition-less tables have a single empty partition if (partitions.contains(NON_PARTITIONED_NAME)) { @@ -315,7 +315,7 @@ FileStatus[] fetchAllFilesInPartition(Path partitionPath) throws IOException { FileStatus[] statuses = {}; if (hoodieRecord.isPresent()) { - mayBeHandleSpuriousDeletes(hoodieRecord, partitionName); + handleSpuriousDeletes(hoodieRecord, partitionName); statuses = hoodieRecord.get().getData().getFileStatuses(hadoopConf.get(), partitionPath); } @@ -350,7 +350,7 @@ Map fetchAllFilesInPartitionPaths(List partitionPath for (Pair>> entry: partitionsFileStatus) { if (entry.getValue().isPresent()) { - mayBeHandleSpuriousDeletes(entry.getValue(), entry.getKey()); + handleSpuriousDeletes(entry.getValue(), entry.getKey()); result.put(partitionInfo.get(entry.getKey()).toString(), entry.getValue().get().getData().getFileStatuses(hadoopConf.get(), partitionInfo.get(entry.getKey()))); } } @@ -360,11 +360,11 @@ Map fetchAllFilesInPartitionPaths(List partitionPath } /** - * Maybe handle spurious deletes. Depending on config, throw an exception or log a warn msg. + * Handle spurious deletes. Depending on config, throw an exception or log a warn msg. * @param hoodieRecord instance of {@link HoodieRecord} of interest. * @param partitionName partition name of interest. */ - private void mayBeHandleSpuriousDeletes(Option> hoodieRecord, String partitionName) { + private void handleSpuriousDeletes(Option> hoodieRecord, String partitionName) { if (!hoodieRecord.get().getData().getDeletions().isEmpty()) { if (metadataConfig.ignoreSpuriousDeletes()) { LOG.warn("Metadata record for " + partitionName + " encountered some files to be deleted which was not added before. " diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java b/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java index 576cfd7cb0f3f..543a712e635ae 100755 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java @@ -218,20 +218,19 @@ public void testTimelineGetOperations() { // Test that various types of getXXX operations from HoodieActiveTimeline // return the correct set of Instant - checkTimeline.accept(timeline.getCommitsTimeline(), - CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION)); - checkTimeline.accept(timeline.getWriteTimeline(), - CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.COMPACTION_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION)); + checkTimeline.accept(timeline.getCommitsTimeline(), CollectionUtils.createSet( + HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION)); + checkTimeline.accept(timeline.getWriteTimeline(), CollectionUtils.createSet( + HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.COMPACTION_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION, HoodieTimeline.INDEX_ACTION)); checkTimeline.accept(timeline.getCommitTimeline(), CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION)); checkTimeline.accept(timeline.getDeltaCommitTimeline(), Collections.singleton(HoodieTimeline.DELTA_COMMIT_ACTION)); checkTimeline.accept(timeline.getCleanerTimeline(), Collections.singleton(HoodieTimeline.CLEAN_ACTION)); checkTimeline.accept(timeline.getRollbackTimeline(), Collections.singleton(HoodieTimeline.ROLLBACK_ACTION)); checkTimeline.accept(timeline.getRestoreTimeline(), Collections.singleton(HoodieTimeline.RESTORE_ACTION)); checkTimeline.accept(timeline.getSavePointTimeline(), Collections.singleton(HoodieTimeline.SAVEPOINT_ACTION)); - checkTimeline.accept(timeline.getAllCommitsTimeline(), - CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, - HoodieTimeline.CLEAN_ACTION, HoodieTimeline.COMPACTION_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION, - HoodieTimeline.SAVEPOINT_ACTION, HoodieTimeline.ROLLBACK_ACTION)); + checkTimeline.accept(timeline.getAllCommitsTimeline(), CollectionUtils.createSet( + HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.CLEAN_ACTION, HoodieTimeline.COMPACTION_ACTION, + HoodieTimeline.REPLACE_COMMIT_ACTION, HoodieTimeline.SAVEPOINT_ACTION, HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.INDEX_ACTION)); // Get some random Instants Random rand = new Random(); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java index 26639628eab1b..f6905f92d9440 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java @@ -20,24 +20,19 @@ import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.common.config.TypedProperties; -import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.exception.HoodieClusteringException; -import org.apache.hudi.exception.HoodieException; 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.log4j.LogManager; import org.apache.log4j.Logger; @@ -49,15 +44,15 @@ import java.util.Date; import java.util.List; +import static org.apache.hudi.utilities.UtilHelpers.EXECUTE; +import static org.apache.hudi.utilities.UtilHelpers.SCHEDULE; +import static org.apache.hudi.utilities.UtilHelpers.SCHEDULE_AND_EXECUTE; + public class HoodieClusteringJob { - public static final String EXECUTE = "execute"; - public static final String SCHEDULE = "schedule"; - public static final String SCHEDULE_AND_EXECUTE = "scheduleandexecute"; private static final Logger LOG = LogManager.getLogger(HoodieClusteringJob.class); private final Config cfg; - private transient FileSystem fs; - private TypedProperties props; + private final TypedProperties props; private final JavaSparkContext jsc; private final HoodieTableMetaClient metaClient; @@ -83,34 +78,34 @@ public static class Config implements Serializable { @Parameter(names = {"--instant-time", "-it"}, description = "Clustering Instant time, only used when set --mode execute. " + "If the instant time is not provided with --mode execute, " + "the earliest scheduled clustering instant time is used by default. " - + "When set \"--mode scheduleAndExecute\" this instant-time will be ignored.", required = false) + + "When set \"--mode scheduleAndExecute\" this instant-time will be ignored.") public String clusteringInstantTime = null; - @Parameter(names = {"--parallelism", "-pl"}, description = "Parallelism for hoodie insert", required = false) + @Parameter(names = {"--parallelism", "-pl"}, description = "Parallelism for hoodie insert") public int parallelism = 1; - @Parameter(names = {"--spark-master", "-ms"}, description = "Spark master", required = false) + @Parameter(names = {"--spark-master", "-ms"}, description = "Spark master") public String sparkMaster = null; @Parameter(names = {"--spark-memory", "-sm"}, description = "spark memory to use", required = true) public String sparkMemory = null; - @Parameter(names = {"--retry", "-rt"}, description = "number of retries", required = false) + @Parameter(names = {"--retry", "-rt"}, description = "number of retries") public int retry = 0; @Parameter(names = {"--schedule", "-sc"}, description = "Schedule clustering @desperate soon please use \"--mode schedule\" instead") public Boolean runSchedule = false; @Parameter(names = {"--retry-last-failed-clustering-job", "-rc"}, description = "Take effect when using --mode/-m scheduleAndExecute. Set true means " - + "check, rollback and execute last failed clustering plan instead of planing a new clustering job directly.", required = false) + + "check, rollback and execute last failed clustering plan instead of planing a new clustering job directly.") public Boolean retryLastFailedClusteringJob = false; @Parameter(names = {"--mode", "-m"}, description = "Set job mode: Set \"schedule\" means make a cluster plan; " - + "Set \"execute\" means execute a cluster plan at given instant which means --instant-time is needed here; " - + "Set \"scheduleAndExecute\" means make a cluster plan first and execute that plan immediately", required = false) + + "Set \"execute\" means execute a cluster plan at given instant which means --instant-time is needed here; " + + "Set \"scheduleAndExecute\" means make a cluster plan first and execute that plan immediately") public String runningMode = null; @Parameter(names = {"--help", "-h"}, help = true) public Boolean help = false; @Parameter(names = {"--job-max-processing-time-ms", "-jt"}, description = "Take effect when using --mode/-m scheduleAndExecute and --retry-last-failed-clustering-job/-rc true. " - + "If maxProcessingTimeMs passed but clustering job is still unfinished, hoodie would consider this job as failed and relaunch.", required = false) + + "If maxProcessingTimeMs passed but clustering job is still unfinished, hoodie would consider this job as failed and relaunch.") public long maxProcessingTimeMs = 0; @Parameter(names = {"--props"}, description = "path to properties file on localfs or dfs, with configurations for " @@ -119,7 +114,7 @@ public static class Config implements Serializable { @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) + splitter = IdentitySplitter.class) public List configs = new ArrayList<>(); } @@ -155,10 +150,9 @@ private static void validateRunningMode(Config cfg) { } public int cluster(int retry) { - this.fs = FSUtils.getFs(cfg.basePath, jsc.hadoopConfiguration()); // need to do validate in case that users call cluster() directly without setting cfg.runningMode validateRunningMode(cfg); - int ret = UtilHelpers.retry(retry, () -> { + return UtilHelpers.retry(retry, () -> { switch (cfg.runningMode.toLowerCase()) { case SCHEDULE: { LOG.info("Running Mode: [" + SCHEDULE + "]; Do schedule"); @@ -183,20 +177,10 @@ public int cluster(int retry) { } } }, "Cluster failed"); - return ret; - } - - private String getSchemaFromLatestInstant() throws Exception { - TableSchemaResolver schemaResolver = new TableSchemaResolver(metaClient); - if (metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().countInstants() == 0) { - throw new HoodieException("Cannot run clustering without any completed commits"); - } - Schema schema = schemaResolver.getTableAvroSchema(false); - return schema.toString(); } private int doCluster(JavaSparkContext jsc) throws Exception { - String schemaStr = getSchemaFromLatestInstant(); + String schemaStr = UtilHelpers.getSchemaFromLatestInstant(metaClient); try (SparkRDDWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props)) { if (StringUtils.isNullOrEmpty(cfg.clusteringInstantTime)) { // Instant time is not specified @@ -224,7 +208,7 @@ public Option doSchedule() throws Exception { } private Option doSchedule(JavaSparkContext jsc) throws Exception { - String schemaStr = getSchemaFromLatestInstant(); + String schemaStr = UtilHelpers.getSchemaFromLatestInstant(metaClient); try (SparkRDDWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props)) { return doSchedule(client); } @@ -240,7 +224,7 @@ private Option doSchedule(SparkRDDWriteClient clien private int doScheduleAndCluster(JavaSparkContext jsc) throws Exception { LOG.info("Step 1: Do schedule"); - String schemaStr = getSchemaFromLatestInstant(); + String schemaStr = UtilHelpers.getSchemaFromLatestInstant(metaClient); try (SparkRDDWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props)) { Option instantTime = Option.empty(); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java new file mode 100644 index 0000000000000..9cae6b80c819b --- /dev/null +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java @@ -0,0 +1,219 @@ +/* + * 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.HoodieIndexCommitMetadata; +import org.apache.hudi.avro.model.HoodieIndexPartitionInfo; +import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.exception.HoodieIndexException; + +import com.beust.jcommander.JCommander; +import com.beust.jcommander.Parameter; +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaSparkContext; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.hudi.common.table.timeline.HoodieInstant.State.INFLIGHT; +import static org.apache.hudi.utilities.UtilHelpers.EXECUTE; +import static org.apache.hudi.utilities.UtilHelpers.SCHEDULE; +import static org.apache.hudi.utilities.UtilHelpers.SCHEDULE_AND_EXECUTE; + +public class HoodieIndexer { + + private static final Logger LOG = LogManager.getLogger(HoodieIndexer.class); + + private final HoodieIndexer.Config cfg; + private TypedProperties props; + private final JavaSparkContext jsc; + private final HoodieTableMetaClient metaClient; + + public HoodieIndexer(JavaSparkContext jsc, HoodieIndexer.Config cfg) { + this.cfg = cfg; + this.jsc = jsc; + this.props = StringUtils.isNullOrEmpty(cfg.propsFilePath) + ? UtilHelpers.buildProperties(cfg.configs) + : readConfigFromFileSystem(jsc, cfg); + this.metaClient = UtilHelpers.createMetaClient(jsc, cfg.basePath, true); + } + + private TypedProperties readConfigFromFileSystem(JavaSparkContext jsc, HoodieIndexer.Config cfg) { + return UtilHelpers.readConfig(jsc.hadoopConfiguration(), new Path(cfg.propsFilePath), cfg.configs) + .getProps(true); + } + + public static class Config implements Serializable { + @Parameter(names = {"--base-path", "-sp"}, description = "Base path for the table", required = true) + public String basePath = null; + @Parameter(names = {"--table-name", "-tn"}, description = "Table name", required = true) + public String tableName = null; + @Parameter(names = {"--instant-time", "-it"}, description = "Indexing Instant time") + public String indexInstantTime = null; + @Parameter(names = {"--parallelism", "-pl"}, description = "Parallelism for hoodie insert", required = true) + public int parallelism = 1; + @Parameter(names = {"--spark-master", "-ms"}, description = "Spark master") + public String sparkMaster = null; + @Parameter(names = {"--spark-memory", "-sm"}, description = "spark memory to use", required = true) + public String sparkMemory = null; + @Parameter(names = {"--retry", "-rt"}, description = "number of retries") + public int retry = 0; + @Parameter(names = {"--schedule", "-sc"}, description = "Schedule indexing") + public Boolean runSchedule = false; + @Parameter(names = {"--strategy", "-st"}, description = "Comma-separated index types to be built, e.g. BLOOM,FILES,COLSTATS") + public String indexTypes = null; + @Parameter(names = {"--mode", "-m"}, description = "Set job mode: Set \"schedule\" to generate an indexing plan; " + + "Set \"execute\" to execute the indexing plan at the given instant, which means --instant-time is required here; " + + "Set \"scheduleAndExecute\" to generate an indexing plan first and execute that plan immediately") + public String runningMode = null; + @Parameter(names = {"--help", "-h"}, help = true) + public Boolean help = false; + + @Parameter(names = {"--props"}, description = "path to properties file on localfs or dfs, with configurations for " + + "hoodie client for compacting") + public String propsFilePath = null; + + @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 configs = new ArrayList<>(); + } + + public static void main(String[] args) { + final HoodieIndexer.Config cfg = new HoodieIndexer.Config(); + JCommander cmd = new JCommander(cfg, null, args); + + if (cfg.help || args.length == 0) { + cmd.usage(); + System.exit(1); + } + + final JavaSparkContext jsc = UtilHelpers.buildSparkContext("indexing-" + cfg.tableName, cfg.sparkMaster, cfg.sparkMemory); + HoodieIndexer indexer = new HoodieIndexer(jsc, cfg); + int result = indexer.start(cfg.retry); + String resultMsg = String.format("Indexing with basePath: %s, tableName: %s, runningMode: %s", + cfg.basePath, cfg.tableName, cfg.runningMode); + if (result == -1) { + LOG.error(resultMsg + " failed"); + } else { + LOG.info(resultMsg + " success"); + } + jsc.stop(); + } + + private int start(int retry) { + return UtilHelpers.retry(retry, () -> { + switch (cfg.runningMode.toLowerCase()) { + case SCHEDULE: { + LOG.info("Running Mode: [" + SCHEDULE + "]; Do schedule"); + Option instantTime = scheduleIndexing(jsc); + int result = instantTime.isPresent() ? 0 : -1; + if (result == 0) { + LOG.info("The schedule instant time is " + instantTime.get()); + } + return result; + } + case SCHEDULE_AND_EXECUTE: { + LOG.info("Running Mode: [" + SCHEDULE_AND_EXECUTE + "]"); + return scheduleAndRunIndexing(jsc); + } + case EXECUTE: { + LOG.info("Running Mode: [" + EXECUTE + "];"); + return runIndexing(jsc); + } + default: { + LOG.info("Unsupported running mode [" + cfg.runningMode + "], quit the job directly"); + return -1; + } + } + }, "Indexer failed"); + } + + private Option scheduleIndexing(JavaSparkContext jsc) throws Exception { + String schemaStr = UtilHelpers.getSchemaFromLatestInstant(metaClient); + try (SparkRDDWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props)) { + return doSchedule(client); + } + } + + private Option doSchedule(SparkRDDWriteClient client) { + List partitionsToIndex = Arrays.asList(cfg.indexTypes.split(",")); + Option indexingInstant = client.scheduleIndexing(partitionsToIndex); + if (!indexingInstant.isPresent()) { + LOG.error("Scheduling of index action did not return any instant."); + } + return indexingInstant; + } + + private int runIndexing(JavaSparkContext jsc) throws Exception { + String schemaStr = UtilHelpers.getSchemaFromLatestInstant(metaClient); + try (SparkRDDWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props)) { + if (StringUtils.isNullOrEmpty(cfg.indexInstantTime)) { + // Instant time is not specified + // Find the earliest scheduled indexing instant for execution + Option earliestPendingIndexInstant = metaClient.getActiveTimeline() + .filterPendingIndexTimeline() + .filter(i -> !(i.isCompleted() || INFLIGHT.equals(i.getState()))) + .firstInstant(); + if (earliestPendingIndexInstant.isPresent()) { + cfg.indexInstantTime = earliestPendingIndexInstant.get().getTimestamp(); + LOG.info("Found the earliest scheduled indexing instant which will be executed: " + + cfg.indexInstantTime); + } else { + throw new HoodieIndexException("There is no scheduled indexing in the table."); + } + } + return handleError(client.index(cfg.indexInstantTime)); + } + } + + private int scheduleAndRunIndexing(JavaSparkContext jsc) throws Exception { + String schemaStr = UtilHelpers.getSchemaFromLatestInstant(metaClient); + try (SparkRDDWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props)) { + Option indexingInstantTime = doSchedule(client); + if (indexingInstantTime.isPresent()) { + return handleError(client.index(indexingInstantTime.get())); + } else { + return -1; + } + } + } + + private int handleError(Option commitMetadata) { + if (!commitMetadata.isPresent()) { + LOG.error("Indexing failed as no commit metadata present."); + return -1; + } + List indexPartitionInfos = commitMetadata.get().getIndexPartitionInfos(); + LOG.info(String.format("Indexing complete for partitions: %s", + indexPartitionInfos.stream().map(HoodieIndexPartitionInfo::getMetadataPartitionPath).collect(Collectors.toList()))); + return 0; + } +} diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java index 4dc0604ddbf21..92e123b7c88b0 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/UtilHelpers.java @@ -104,21 +104,26 @@ * Bunch of helper methods. */ public class UtilHelpers { + + public static final String EXECUTE = "execute"; + public static final String SCHEDULE = "schedule"; + public static final String SCHEDULE_AND_EXECUTE = "scheduleandexecute"; + private static final Logger LOG = LogManager.getLogger(UtilHelpers.class); public static Source createSource(String sourceClass, TypedProperties cfg, JavaSparkContext jssc, - SparkSession sparkSession, SchemaProvider schemaProvider, - HoodieDeltaStreamerMetrics metrics) throws IOException { + SparkSession sparkSession, SchemaProvider schemaProvider, + HoodieDeltaStreamerMetrics metrics) throws IOException { try { try { return (Source) ReflectionUtils.loadClass(sourceClass, - new Class[]{TypedProperties.class, JavaSparkContext.class, + new Class[] {TypedProperties.class, JavaSparkContext.class, SparkSession.class, SchemaProvider.class, HoodieDeltaStreamerMetrics.class}, cfg, jssc, sparkSession, schemaProvider, metrics); } catch (HoodieException e) { return (Source) ReflectionUtils.loadClass(sourceClass, - new Class[]{TypedProperties.class, JavaSparkContext.class, + new Class[] {TypedProperties.class, JavaSparkContext.class, SparkSession.class, SchemaProvider.class}, cfg, jssc, sparkSession, schemaProvider); } @@ -238,7 +243,7 @@ public static void validateAndAddProperties(String[] configs, SparkLauncher spar /** * Parse Schema from file. * - * @param fs File System + * @param fs File System * @param schemaFile Schema File */ public static String parseSchema(FileSystem fs, String schemaFile) throws Exception { @@ -300,13 +305,13 @@ public static JavaSparkContext buildSparkContext(String appName, String sparkMas /** * Build Hoodie write client. * - * @param jsc Java Spark Context - * @param basePath Base Path - * @param schemaStr Schema + * @param jsc Java Spark Context + * @param basePath Base Path + * @param schemaStr Schema * @param parallelism Parallelism */ public static SparkRDDWriteClient createHoodieClient(JavaSparkContext jsc, String basePath, String schemaStr, - int parallelism, Option compactionStrategyClass, TypedProperties properties) { + int parallelism, Option compactionStrategyClass, TypedProperties properties) { HoodieCompactionConfig compactionConfig = compactionStrategyClass .map(strategy -> HoodieCompactionConfig.newBuilder().withInlineCompaction(false) .withCompactionStrategy(ReflectionUtils.loadClass(strategy)).build()) @@ -466,8 +471,7 @@ public static SchemaProviderWithPostProcessor wrapSchemaProviderWithPostProcesso Option.ofNullable(createSchemaPostProcessor(schemaPostProcessorClass, cfg, jssc))); } - public static SchemaProvider createRowBasedSchemaProvider(StructType structType, - TypedProperties cfg, JavaSparkContext jssc) { + public static SchemaProvider createRowBasedSchemaProvider(StructType structType, TypedProperties cfg, JavaSparkContext jssc) { SchemaProvider rowSchemaProvider = new RowBasedSchemaProvider(structType); return wrapSchemaProviderWithPostProcessor(rowSchemaProvider, cfg, jssc, null); } @@ -476,13 +480,13 @@ public static SchemaProvider createRowBasedSchemaProvider(StructType structType, * Create latest schema provider for Target schema. * * @param structType spark data type of incoming batch. - * @param jssc instance of {@link JavaSparkContext}. - * @param fs instance of {@link FileSystem}. - * @param basePath base path of the table. + * @param jssc instance of {@link JavaSparkContext}. + * @param fs instance of {@link FileSystem}. + * @param basePath base path of the table. * @return the schema provider where target schema refers to latest schema(either incoming schema or table schema). */ public static SchemaProvider createLatestSchemaProvider(StructType structType, - JavaSparkContext jssc, FileSystem fs, String basePath) { + JavaSparkContext jssc, FileSystem fs, String basePath) { SchemaProvider rowSchemaProvider = new RowBasedSchemaProvider(structType); Schema writeSchema = rowSchemaProvider.getTargetSchema(); Schema latestTableSchema = writeSchema; @@ -540,4 +544,12 @@ public static int retry(int maxRetryCount, CheckedSupplier supplier, St return ret; } + public static String getSchemaFromLatestInstant(HoodieTableMetaClient metaClient) throws Exception { + TableSchemaResolver schemaResolver = new TableSchemaResolver(metaClient); + if (metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().countInstants() == 0) { + throw new HoodieException("Cannot run clustering without any completed commits"); + } + Schema schema = schemaResolver.getTableAvroSchema(false); + return schema.toString(); + } } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java index 62b862e4b86a5..066e81849056b 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java @@ -129,6 +129,9 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.utilities.UtilHelpers.EXECUTE; +import static org.apache.hudi.utilities.UtilHelpers.SCHEDULE; +import static org.apache.hudi.utilities.UtilHelpers.SCHEDULE_AND_EXECUTE; import static org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.CHECKPOINT_KEY; import static org.apache.hudi.utilities.schema.RowBasedSchemaProvider.HOODIE_RECORD_NAMESPACE; import static org.apache.hudi.utilities.schema.RowBasedSchemaProvider.HOODIE_RECORD_STRUCT_NAME; @@ -1131,28 +1134,28 @@ public void testHoodieAsyncClusteringJobWithScheduleAndExecute(String runningMod LOG.info("Cluster success"); } else { LOG.warn("Import failed"); - if (!runningMode.toLowerCase().equals(HoodieClusteringJob.EXECUTE)) { + if (!runningMode.toLowerCase().equals(EXECUTE)) { return false; } } } catch (Exception e) { LOG.warn("ScheduleAndExecute clustering failed", e); exception = e; - if (!runningMode.equalsIgnoreCase(HoodieClusteringJob.EXECUTE)) { + if (!runningMode.equalsIgnoreCase(EXECUTE)) { return false; } } switch (runningMode.toLowerCase()) { - case HoodieClusteringJob.SCHEDULE_AND_EXECUTE: { + case SCHEDULE_AND_EXECUTE: { TestHelpers.assertAtLeastNReplaceCommits(2, tableBasePath, dfs); return true; } - case HoodieClusteringJob.SCHEDULE: { + case SCHEDULE: { TestHelpers.assertAtLeastNReplaceRequests(2, tableBasePath, dfs); TestHelpers.assertNoReplaceCommits(tableBasePath, dfs); return true; } - case HoodieClusteringJob.EXECUTE: { + case EXECUTE: { TestHelpers.assertNoReplaceCommits(tableBasePath, dfs); return true; } From 3d0b5f0a11aef78ca858fcab55b942a952b084d8 Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Sat, 12 Mar 2022 16:44:21 +0530 Subject: [PATCH 02/21] Take lock and initialize filegroup while scheduling --- .../hudi/client/BaseHoodieWriteClient.java | 9 +++--- .../HoodieBackedTableMetadataWriter.java | 8 +++-- .../metadata/HoodieTableMetadataWriter.java | 4 +++ .../org/apache/hudi/table/HoodieTable.java | 4 +-- .../index/ScheduleIndexActionExecutor.java | 32 +++++++++++++++++-- .../FlinkHoodieBackedTableMetadataWriter.java | 2 +- .../table/HoodieFlinkCopyOnWriteTable.java | 3 +- .../table/HoodieJavaCopyOnWriteTable.java | 3 +- .../SparkHoodieBackedTableMetadataWriter.java | 4 +-- .../table/HoodieSparkCopyOnWriteTable.java | 3 +- .../apache/hudi/utilities/HoodieIndexer.java | 5 ++- 11 files changed, 58 insertions(+), 19 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java index 9a4099f4f7c5d..7c98154406a25 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java @@ -69,6 +69,7 @@ import org.apache.hudi.exception.HoodieSavepointException; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.metadata.HoodieTableMetadataWriter; +import org.apache.hudi.metadata.MetadataPartitionType; import org.apache.hudi.metrics.HoodieMetrics; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.table.HoodieTable; @@ -926,14 +927,14 @@ public boolean scheduleCompactionAtInstant(String instantTime, Option scheduleIndexing(List partitions) { + public Option scheduleIndexing(List partitionTypes) { String instantTime = HoodieActiveTimeline.createNewInstantTime(); - return scheduleIndexingAtInstant(partitions, instantTime) ? Option.of(instantTime) : Option.empty(); + return scheduleIndexingAtInstant(partitionTypes, instantTime) ? Option.of(instantTime) : Option.empty(); } - private boolean scheduleIndexingAtInstant(List partitionsToIndex, String instantTime) throws HoodieIOException { + private boolean scheduleIndexingAtInstant(List partitionTypes, String instantTime) throws HoodieIOException { Option indexPlan = createTable(config, hadoopConf, config.isMetadataTableEnabled()) - .scheduleIndex(context, instantTime, partitionsToIndex); + .scheduleIndex(context, instantTime, partitionTypes); return indexPlan.isPresent(); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index e07601c01af9d..191fa63c6a0cb 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -342,7 +342,7 @@ protected abstract void initialize(HoodieEngineCo Option actionMetadata, Option inflightInstantTimestamp); - protected abstract void scheduleIndex(List partitions); + protected abstract void scheduleIndex(List partitionTypes); public void initTableMetadata() { try { @@ -403,7 +403,8 @@ protected void initializeIfNeeded(HoodieTableMeta if (metadataWriteConfig.isMetadataAsyncIndex()) { // with async metadata indexing enabled, there can be inflight writers // TODO: schedule indexing only for enabled partition types - scheduleIndex(MetadataPartitionType.allPaths()); + MetadataRecordsGenerationParams indexParams = getRecordsGenerationParams(); + scheduleIndex(indexParams.getEnabledPartitionTypes()); return; } // Initialize for the first time by listing partitions and files directly from the file system @@ -626,7 +627,7 @@ private void initializeEnabledFileGroups(HoodieTableMetaClient dataMetaClient, S * File groups will be named as : * record-index-bucket-0000, .... -> ..., record-index-bucket-0009 */ - private void initializeFileGroups(HoodieTableMetaClient dataMetaClient, MetadataPartitionType metadataPartition, String instantTime, + public void initializeFileGroups(HoodieTableMetaClient dataMetaClient, MetadataPartitionType metadataPartition, String instantTime, int fileGroupCount) throws IOException { final HashMap blockHeader = new HashMap<>(); @@ -755,6 +756,7 @@ public void index(HoodieEngineContext engineContext, List indexPartitionInfos); + void initializeFileGroups(HoodieTableMetaClient dataMetaClient, MetadataPartitionType metadataPartition, String instantTime, int fileGroupCount) throws IOException; + /** * Update the metadata table due to a COMMIT operation. * @param commitMetadata commit metadata of the operation of interest. diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java index d3cf118db3178..2831ca8d6e43d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java @@ -504,10 +504,10 @@ public abstract HoodieRollbackMetadata rollback(HoodieEngineContext context, * * @param context HoodieEngineContext * @param indexInstantTime Instant time for scheduling index action. - * @param partitionsToIndex List of {@link MetadataPartitionType#partitionPath()} that should be indexed. + * @param partitionsToIndex List of {@link MetadataPartitionType} that should be indexed. * @return HoodieIndexPlan containing metadata partitions and instant upto which they should be indexed. */ - public abstract Option scheduleIndex(HoodieEngineContext context, String indexInstantTime, List partitionsToIndex); + public abstract Option scheduleIndex(HoodieEngineContext context, String indexInstantTime, List partitionsToIndex); /** * Execute requested index action. diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java index d0418b4c7f9d1..571177e527542 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java @@ -21,6 +21,7 @@ import org.apache.hudi.avro.model.HoodieIndexPartitionInfo; import org.apache.hudi.avro.model.HoodieIndexPlan; +import org.apache.hudi.client.transaction.TransactionManager; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.timeline.HoodieInstant; @@ -30,6 +31,7 @@ import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieIndexException; +import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.metadata.MetadataPartitionType; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.BaseActionExecutor; @@ -47,11 +49,17 @@ public class ScheduleIndexActionExecutor private static final Integer INDEX_PLAN_VERSION_1 = 1; private static final Integer LATEST_INDEX_PLAN_VERSION = INDEX_PLAN_VERSION_1; - private final List partitionsToIndex; + private final List partitionsToIndex; + private final TransactionManager txnManager; - public ScheduleIndexActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, String instantTime, List partitionsToIndex) { + public ScheduleIndexActionExecutor(HoodieEngineContext context, + HoodieWriteConfig config, + HoodieTable table, + String instantTime, + List partitionsToIndex) { super(context, config, table, instantTime); this.partitionsToIndex = partitionsToIndex; + this.txnManager = new TransactionManager(config, table.getMetaClient().getFs()); } @Override @@ -66,7 +74,7 @@ public Option execute() { final HoodieInstant indexInstant = HoodieTimeline.getIndexRequestedInstant(instantTime); // for each partitionToIndex add that time to the plan List indexPartitionInfos = partitionsToIndex.stream() - .map(p -> new HoodieIndexPartitionInfo(LATEST_INDEX_PLAN_VERSION, p, indexUptoInstant.get().getTimestamp())) + .map(p -> new HoodieIndexPartitionInfo(LATEST_INDEX_PLAN_VERSION, p.getPartitionPath(), indexUptoInstant.get().getTimestamp())) .collect(Collectors.toList()); HoodieIndexPlan indexPlan = new HoodieIndexPlan(LATEST_INDEX_PLAN_VERSION, indexPartitionInfos); try { @@ -76,6 +84,24 @@ public Option execute() { throw new HoodieIOException(e.getMessage(), e); } table.getMetaClient().reloadActiveTimeline(); + + // start initializing filegroups + // 1. get metadata writer + HoodieTableMetadataWriter metadataWriter = table.getMetadataWriter(instantTime) + .orElseThrow(() -> new HoodieIndexException(String.format("Could not get metadata writer to run index action for instant: %s", instantTime))); + // 2. take a lock --> begin tx (data table) + try { + this.txnManager.beginTransaction(Option.of(indexInstant), Option.empty()); + // 3. initialize filegroups as per plan for the enabled partition types + for (MetadataPartitionType partitionType : partitionsToIndex) { + metadataWriter.initializeFileGroups(table.getMetaClient(), partitionType, indexInstant.getTimestamp(), 1); + } + } catch (IOException e) { + LOG.error("Could not initialize file groups"); + throw new HoodieIOException(e.getMessage(), e); + } finally { + this.txnManager.endTransaction(Option.of(indexInstant)); + } return Option.of(indexPlan); } return Option.empty(); diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java index 85417ad3d9c5e..5b2ef8cf65667 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java @@ -100,7 +100,7 @@ protected void initialize(HoodieEngineContext eng } @Override - protected void scheduleIndex(List partitions) { + protected void scheduleIndex(List partitionsTypes) { throw new UnsupportedOperationException("Metadata indexing not supported for Flink table yet."); } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java index 6a4807297735b..4683ed7169c2b 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java @@ -51,6 +51,7 @@ import org.apache.hudi.io.HoodieWriteHandle; import org.apache.hudi.keygen.BaseKeyGenerator; import org.apache.hudi.keygen.factory.HoodieAvroKeyGeneratorFactory; +import org.apache.hudi.metadata.MetadataPartitionType; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; import org.apache.hudi.table.action.clean.CleanActionExecutor; @@ -342,7 +343,7 @@ public HoodieRollbackMetadata rollback(HoodieEngineContext context, String rollb } @Override - public Option scheduleIndex(HoodieEngineContext context, String indexInstantTime, List partitionsToIndex) { + public Option scheduleIndex(HoodieEngineContext context, String indexInstantTime, List partitionsToIndex) { throw new HoodieNotSupportedException("Indexing is not supported for a Flink table yet."); } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java index a07481c210963..b0746a833c67e 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java @@ -47,6 +47,7 @@ import org.apache.hudi.io.HoodieCreateHandle; import org.apache.hudi.io.HoodieMergeHandle; import org.apache.hudi.io.HoodieSortedMergeHandle; +import org.apache.hudi.metadata.MetadataPartitionType; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; import org.apache.hudi.table.action.clean.CleanActionExecutor; @@ -246,7 +247,7 @@ public HoodieRollbackMetadata rollback(HoodieEngineContext context, } @Override - public Option scheduleIndex(HoodieEngineContext context, String indexInstantTime, List partitionsToIndex) { + public Option scheduleIndex(HoodieEngineContext context, String indexInstantTime, List partitionsToIndex) { return new ScheduleIndexActionExecutor<>(context, config, this, indexInstantTime, partitionsToIndex).execute(); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java index d27995bf0a271..6e279bb5b201a 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java @@ -122,11 +122,11 @@ protected void initialize(HoodieEngineContext eng } @Override - protected void scheduleIndex(List partitions) { + protected void scheduleIndex(List partitionTypes) { ValidationUtils.checkState(metadataMetaClient != null, "Metadata table is not fully initialized yet."); ValidationUtils.checkState(enabled, "Metadata table cannot be indexed as it is not enabled."); try (SparkRDDWriteClient writeClient = new SparkRDDWriteClient(engineContext, metadataWriteConfig, true)) { - writeClient.scheduleIndexing(partitions); + writeClient.scheduleIndexing(partitionTypes); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java index 5c1541a4bf6d5..24855da88ab3d 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java @@ -57,6 +57,7 @@ import org.apache.hudi.io.HoodieSortedMergeHandle; import org.apache.hudi.keygen.BaseKeyGenerator; import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory; +import org.apache.hudi.metadata.MetadataPartitionType; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata; import org.apache.hudi.table.action.bootstrap.SparkBootstrapCommitActionExecutor; @@ -351,7 +352,7 @@ public HoodieRollbackMetadata rollback(HoodieEngineContext context, String rollb } @Override - public Option scheduleIndex(HoodieEngineContext context, String indexInstantTime, List partitionsToIndex) { + public Option scheduleIndex(HoodieEngineContext context, String indexInstantTime, List partitionsToIndex) { return new ScheduleIndexActionExecutor<>(context, config, this, indexInstantTime, partitionsToIndex).execute(); } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java index 9cae6b80c819b..c99a9d145cff8 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java @@ -28,6 +28,7 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.exception.HoodieIndexException; +import org.apache.hudi.metadata.MetadataPartitionType; import com.beust.jcommander.JCommander; import com.beust.jcommander.Parameter; @@ -165,7 +166,9 @@ private Option scheduleIndexing(JavaSparkContext jsc) throws Exception { private Option doSchedule(SparkRDDWriteClient client) { List partitionsToIndex = Arrays.asList(cfg.indexTypes.split(",")); - Option indexingInstant = client.scheduleIndexing(partitionsToIndex); + List partitionTypes = partitionsToIndex.stream() + .map(MetadataPartitionType::valueOf).collect(Collectors.toList()); + Option indexingInstant = client.scheduleIndexing(partitionTypes); if (!indexingInstant.isPresent()) { LOG.error("Scheduling of index action did not return any instant."); } From 93d8b179b6bda60ed7c5a309b972cc9ba15949d9 Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Mon, 14 Mar 2022 18:23:24 +0530 Subject: [PATCH 03/21] Support indexing subset of columns --- .../org/apache/hudi/config/HoodieWriteConfig.java | 8 ++++++-- .../apache/hudi/index/bloom/HoodieBloomIndex.java | 2 +- .../org/apache/hudi/io/HoodieAppendHandle.java | 10 ++++++---- .../metadata/HoodieBackedTableMetadataWriter.java | 6 ++++-- .../action/index/RunIndexActionExecutor.java | 4 ++++ .../action/index/ScheduleIndexActionExecutor.java | 8 ++++++++ .../hudi/common/config/HoodieMetadataConfig.java | 15 +++++++++++++++ .../metadata/MetadataRecordsGenerationParams.java | 8 +++++++- 8 files changed, 51 insertions(+), 10 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 6c52a01b311bc..7ab750f636cdb 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -1507,8 +1507,12 @@ public boolean isMetadataBloomFilterIndexEnabled() { return isMetadataTableEnabled() && getMetadataConfig().isBloomFilterIndexEnabled(); } - public boolean isMetadataIndexColumnStatsForAllColumnsEnabled() { - return isMetadataTableEnabled() && getMetadataConfig().isMetadataColumnStatsIndexForAllColumnsEnabled(); + public boolean isMetadataColumnStatsIndexEnabled() { + return isMetadataTableEnabled() && getMetadataConfig().isColumnStatsIndexEnabled(); + } + + public String getColumnsEnabledForColumnStatsIndex() { + return getMetadataConfig().getColumnsEnabledForColumnStatsIndex(); } public int getColumnStatsIndexParallelism() { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java index d3e73c058cc56..a923a4c0714dc 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bloom/HoodieBloomIndex.java @@ -120,7 +120,7 @@ private HoodiePairData lookupIndex( // Step 2: Load all involved files as pairs List> fileInfoList; if (config.getBloomIndexPruneByRanges()) { - fileInfoList = (config.getMetadataConfig().isColumnStatsIndexEnabled() + fileInfoList = (config.isMetadataColumnStatsIndexEnabled() ? loadColumnRangesFromMetaIndex(affectedPartitionPathList, context, hoodieTable) : loadColumnRangesFromFiles(affectedPartitionPathList, context, hoodieTable)); } else { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java index f8081019683e7..fcd1c7fcbc2d1 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java @@ -71,6 +71,7 @@ import java.util.Properties; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.accumulateColumnRanges; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.aggregateColumnStats; @@ -343,16 +344,17 @@ private void processAppendResult(AppendResult result, List record updateWriteStatus(stat, result); } - if (config.isMetadataIndexColumnStatsForAllColumnsEnabled()) { + if (config.isMetadataColumnStatsIndexEnabled()) { + List columnsToIndex = Stream.of(config.getColumnsEnabledForColumnStatsIndex().split(",")).map(String::trim).collect(Collectors.toList()); Map> columnRangeMap = stat.getRecordsStats().isPresent() ? stat.getRecordsStats().get().getStats() : new HashMap<>(); final String filePath = stat.getPath(); // initialize map of column name to map of stats name to stats value Map> columnToStats = new HashMap<>(); - writeSchemaWithMetaFields.getFields().forEach(field -> columnToStats.putIfAbsent(field.name(), new HashMap<>())); + columnsToIndex.forEach(column -> columnToStats.putIfAbsent(column, new HashMap<>())); // collect stats for columns at once per record and keep iterating through every record to eventually find col stats for all fields. - recordList.forEach(record -> aggregateColumnStats(record, writeSchemaWithMetaFields, columnToStats, config.isConsistentLogicalTimestampEnabled())); - writeSchemaWithMetaFields.getFields().forEach(field -> accumulateColumnRanges(field, filePath, columnRangeMap, columnToStats)); + recordList.forEach(record -> aggregateColumnStats(record, columnsToIndex, columnToStats, config.isConsistentLogicalTimestampEnabled())); + columnsToIndex.forEach(column -> accumulateColumnRanges(column, filePath, columnRangeMap, columnToStats)); stat.setRecordsStats(new HoodieDeltaWriteStat.RecordsStats<>(columnRangeMap)); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index 191fa63c6a0cb..a24659098ee81 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -90,6 +90,7 @@ import java.util.Properties; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.apache.hudi.common.table.HoodieTableConfig.ARCHIVELOG_FOLDER; import static org.apache.hudi.metadata.HoodieTableMetadata.METADATA_TABLE_NAME_SUFFIX; @@ -670,8 +671,9 @@ private MetadataRecordsGenerationParams getRecordsGenerationParams() { return new MetadataRecordsGenerationParams( dataMetaClient, enabledPartitionTypes, dataWriteConfig.getBloomFilterType(), dataWriteConfig.getBloomIndexParallelism(), - dataWriteConfig.isMetadataIndexColumnStatsForAllColumnsEnabled(), - dataWriteConfig.getColumnStatsIndexParallelism()); + dataWriteConfig.isMetadataColumnStatsIndexEnabled(), + dataWriteConfig.getColumnStatsIndexParallelism(), + Stream.of(dataWriteConfig.getColumnsEnabledForColumnStatsIndex().split(",")).map(String::trim).collect(Collectors.toList())); } /** diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java index 43b958df0d84a..c08b55c312568 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java @@ -59,6 +59,10 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +/** + * Reads the index plan and executes the plan. + * It also reconciles updates on data timeline while indexing was in progress. + */ public class RunIndexActionExecutor extends BaseActionExecutor> { private static final Logger LOG = LogManager.getLogger(RunIndexActionExecutor.class); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java index 571177e527542..d18fd80b4cd1e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java @@ -43,6 +43,14 @@ import java.util.List; import java.util.stream.Collectors; +/** + * Schedules INDEX action. + *
  • + * 1. Fetch last completed instant on data timeline. + * 2. Write the index plan to the .index.requested. + * 3. Initialize filegroups for the enabled partition types within a transaction. + *
  • + */ public class ScheduleIndexActionExecutor extends BaseActionExecutor> { private static final Logger LOG = LogManager.getLogger(ScheduleIndexActionExecutor.class); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java index f3f5c0ae511ac..68c27ad2815e9 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java @@ -182,6 +182,12 @@ public final class HoodieMetadataConfig extends HoodieConfig { .sinceVersion("0.11.0") .withDocumentation("Parallelism to use, when generating column stats index."); + public static final ConfigProperty COLUMN_STATS_INDEX_FOR_COLUMNS = ConfigProperty + .key(METADATA_PREFIX + ".index.column.stats.for.columns") + .defaultValue("") + .sinceVersion("0.11.0") + .withDocumentation("Comma-separated list of columns for which column stats index will be built."); + public static final ConfigProperty POPULATE_META_FIELDS = ConfigProperty .key(METADATA_PREFIX + ".populate.meta.fields") .defaultValue(false) @@ -228,6 +234,10 @@ public boolean isMetadataColumnStatsIndexForAllColumnsEnabled() { return getBooleanOrDefault(ENABLE_METADATA_INDEX_COLUMN_STATS_FOR_ALL_COLUMNS); } + public String getColumnsEnabledForColumnStatsIndex() { + return getString(COLUMN_STATS_INDEX_FOR_COLUMNS); + } + public int getBloomFilterIndexFileGroupCount() { return getIntOrDefault(METADATA_INDEX_BLOOM_FILTER_FILE_GROUP_COUNT); } @@ -312,6 +322,11 @@ public Builder withMetadataIndexForAllColumns(boolean enable) { return this; } + public Builder withColumnStatsIndexForColumns(String columns) { + metadataConfig.setValue(COLUMN_STATS_INDEX_FOR_COLUMNS, columns); + return this; + } + public Builder enableMetrics(boolean enableMetrics) { metadataConfig.setValue(METRICS_ENABLE, String.valueOf(enableMetrics)); return this; diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataRecordsGenerationParams.java b/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataRecordsGenerationParams.java index 21d5b173b2338..773e314f636bd 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataRecordsGenerationParams.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataRecordsGenerationParams.java @@ -35,15 +35,17 @@ public class MetadataRecordsGenerationParams implements Serializable { private final int bloomIndexParallelism; private final boolean isAllColumnStatsIndexEnabled; private final int columnStatsIndexParallelism; + private final List columnsToIndex; MetadataRecordsGenerationParams(HoodieTableMetaClient dataMetaClient, List enabledPartitionTypes, String bloomFilterType, int bloomIndexParallelism, - boolean isAllColumnStatsIndexEnabled, int columnStatsIndexParallelism) { + boolean isAllColumnStatsIndexEnabled, int columnStatsIndexParallelism, List columnsToIndex) { this.dataMetaClient = dataMetaClient; this.enabledPartitionTypes = enabledPartitionTypes; this.bloomFilterType = bloomFilterType; this.bloomIndexParallelism = bloomIndexParallelism; this.isAllColumnStatsIndexEnabled = isAllColumnStatsIndexEnabled; this.columnStatsIndexParallelism = columnStatsIndexParallelism; + this.columnsToIndex = columnsToIndex; } public HoodieTableMetaClient getDataMetaClient() { @@ -69,4 +71,8 @@ public int getBloomIndexParallelism() { public int getColumnStatsIndexParallelism() { return columnStatsIndexParallelism; } + + public List getColumnsToIndex() { + return columnsToIndex; + } } From 03600acfb22ce05b3b59a3e4741346274513990b Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Mon, 14 Mar 2022 20:48:25 +0530 Subject: [PATCH 04/21] [HUDI-3368] Add support for bloom index for secondary keys --- .../org/apache/hudi/config/HoodieWriteConfig.java | 4 ++++ .../metadata/HoodieBackedTableMetadataWriter.java | 3 ++- .../org/apache/hudi/common/bloom/BloomFilter.java | 11 +++++++++++ .../bloom/HoodieDynamicBoundedBloomFilter.java | 10 ++++++++++ .../hudi/common/bloom/SimpleBloomFilter.java | 14 ++++++++++++-- .../hudi/common/config/HoodieMetadataConfig.java | 15 +++++++++++++++ .../hudi/metadata/HoodieTableMetadataUtil.java | 2 ++ .../metadata/MetadataRecordsGenerationParams.java | 8 +++++++- 8 files changed, 63 insertions(+), 4 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 7ab750f636cdb..a806bd754d710 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -1515,6 +1515,10 @@ public String getColumnsEnabledForColumnStatsIndex() { return getMetadataConfig().getColumnsEnabledForColumnStatsIndex(); } + public String getColumnsEnabledForBloomFilterIndex() { + return getMetadataConfig().getColumnsEnabledForBloomFilterIndex(); + } + public int getColumnStatsIndexParallelism() { return metadataConfig.getColumnStatsIndexParallelism(); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index a24659098ee81..98e8e518d6cf6 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -673,7 +673,8 @@ private MetadataRecordsGenerationParams getRecordsGenerationParams() { dataWriteConfig.getBloomIndexParallelism(), dataWriteConfig.isMetadataColumnStatsIndexEnabled(), dataWriteConfig.getColumnStatsIndexParallelism(), - Stream.of(dataWriteConfig.getColumnsEnabledForColumnStatsIndex().split(",")).map(String::trim).collect(Collectors.toList())); + Stream.of(dataWriteConfig.getColumnsEnabledForColumnStatsIndex().split(",")).map(String::trim).collect(Collectors.toList()), + Stream.of(dataWriteConfig.getColumnsEnabledForBloomFilterIndex().split(",")).map(String::trim).collect(Collectors.toList())); } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/common/bloom/BloomFilter.java b/hudi-common/src/main/java/org/apache/hudi/common/bloom/BloomFilter.java index 7997da159b7f9..60e20e45c210f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/bloom/BloomFilter.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/bloom/BloomFilter.java @@ -18,6 +18,10 @@ package org.apache.hudi.common.bloom; +import javax.annotation.Nonnull; + +import java.util.List; + /** * A Bloom filter interface. */ @@ -30,6 +34,13 @@ public interface BloomFilter { */ void add(String key); + /** + * Add secondary key to the {@link BloomFilter}. + * + * @param keys list of secondary keys to add to the {@link BloomFilter} + */ + void add(@Nonnull List keys); + /** * Tests for key membership. * diff --git a/hudi-common/src/main/java/org/apache/hudi/common/bloom/HoodieDynamicBoundedBloomFilter.java b/hudi-common/src/main/java/org/apache/hudi/common/bloom/HoodieDynamicBoundedBloomFilter.java index d4bc287c551c2..d8850eac10556 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/bloom/HoodieDynamicBoundedBloomFilter.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/bloom/HoodieDynamicBoundedBloomFilter.java @@ -23,12 +23,15 @@ import org.apache.hadoop.util.bloom.Key; +import javax.annotation.Nonnull; + import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; import java.nio.charset.StandardCharsets; +import java.util.List; /** * Hoodie's dynamic bloom bounded bloom filter. This is based largely on Hadoop's DynamicBloomFilter, but with a bound @@ -81,6 +84,13 @@ public void add(String key) { internalDynamicBloomFilter.add(new Key(key.getBytes(StandardCharsets.UTF_8))); } + @Override + public void add(@Nonnull List keys) { + for (String key: keys) { + add(key); + } + } + @Override public boolean mightContain(String key) { return internalDynamicBloomFilter.membershipTest(new Key(key.getBytes(StandardCharsets.UTF_8))); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/bloom/SimpleBloomFilter.java b/hudi-common/src/main/java/org/apache/hudi/common/bloom/SimpleBloomFilter.java index b0278319fce46..7eafa295f7e97 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/bloom/SimpleBloomFilter.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/bloom/SimpleBloomFilter.java @@ -23,6 +23,8 @@ import org.apache.hadoop.util.bloom.Key; +import javax.annotation.Nonnull; + import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.DataInput; @@ -33,6 +35,7 @@ import java.io.ObjectInputStream; import java.io.ObjectOutputStream; import java.nio.charset.StandardCharsets; +import java.util.List; /** * A Simple Bloom filter implementation built on top of {@link org.apache.hadoop.util.bloom.BloomFilter}. @@ -78,15 +81,22 @@ public SimpleBloomFilter(String serString) { @Override public void add(String key) { if (key == null) { - throw new NullPointerException("Key cannot by null"); + throw new NullPointerException("Key cannot be null"); } filter.add(new Key(key.getBytes(StandardCharsets.UTF_8))); } + @Override + public void add(@Nonnull List keys) { + for (String key: keys) { + add(key); + } + } + @Override public boolean mightContain(String key) { if (key == null) { - throw new NullPointerException("Key cannot by null"); + throw new NullPointerException("Key cannot be null"); } return filter.membershipTest(new Key(key.getBytes(StandardCharsets.UTF_8))); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java index 68c27ad2815e9..c67f7a8e7d0ef 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java @@ -188,6 +188,12 @@ public final class HoodieMetadataConfig extends HoodieConfig { .sinceVersion("0.11.0") .withDocumentation("Comma-separated list of columns for which column stats index will be built."); + public static final ConfigProperty BLOOM_FILTER_INDEX_FOR_COLUMNS = ConfigProperty + .key(METADATA_PREFIX + ".index.bloom.filter.for.columns") + .defaultValue("") + .sinceVersion("0.11.0") + .withDocumentation("Comma-separated list of columns for which bloom filter index will be built."); + public static final ConfigProperty POPULATE_META_FIELDS = ConfigProperty .key(METADATA_PREFIX + ".populate.meta.fields") .defaultValue(false) @@ -238,6 +244,10 @@ public String getColumnsEnabledForColumnStatsIndex() { return getString(COLUMN_STATS_INDEX_FOR_COLUMNS); } + public String getColumnsEnabledForBloomFilterIndex() { + return getString(BLOOM_FILTER_INDEX_FOR_COLUMNS); + } + public int getBloomFilterIndexFileGroupCount() { return getIntOrDefault(METADATA_INDEX_BLOOM_FILTER_FILE_GROUP_COUNT); } @@ -327,6 +337,11 @@ public Builder withColumnStatsIndexForColumns(String columns) { return this; } + public Builder withBloomFilterIndexForColumns(String columns) { + metadataConfig.setValue(BLOOM_FILTER_INDEX_FOR_COLUMNS, columns); + return this; + } + public Builder enableMetrics(boolean enableMetrics) { metadataConfig.setValue(METRICS_ENABLE, String.valueOf(enableMetrics)); return this; diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java index 9e1db9eba2362..7bd5ccc9cb824 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java @@ -266,6 +266,7 @@ public static HoodieData convertMetadataToBloomFilterRecords( LOG.error("Failed to read bloom filter for " + writeFilePath); return Collections.emptyListIterator(); } + fileBloomFilter.add(recordsGenerationParams.getBloomSecondaryKeys()); ByteBuffer bloomByteBuffer = ByteBuffer.wrap(fileBloomFilter.serializeToString().getBytes()); HoodieRecord record = HoodieMetadataPayload.createBloomFilterMetadataRecord( partition, fileName, instantTime, recordsGenerationParams.getBloomFilterType(), bloomByteBuffer, false); @@ -710,6 +711,7 @@ public static HoodieData convertFilesToBloomFilterRecords(HoodieEn LOG.error("Failed to read bloom filter for " + appendedFilePath); return Stream.empty(); } + fileBloomFilter.add(recordsGenerationParams.getBloomSecondaryKeys()); ByteBuffer bloomByteBuffer = ByteBuffer.wrap(fileBloomFilter.serializeToString().getBytes()); HoodieRecord record = HoodieMetadataPayload.createBloomFilterMetadataRecord( partition, appendedFile, instantTime, recordsGenerationParams.getBloomFilterType(), bloomByteBuffer, false); diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataRecordsGenerationParams.java b/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataRecordsGenerationParams.java index 773e314f636bd..d28be75586b82 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataRecordsGenerationParams.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/MetadataRecordsGenerationParams.java @@ -36,9 +36,10 @@ public class MetadataRecordsGenerationParams implements Serializable { private final boolean isAllColumnStatsIndexEnabled; private final int columnStatsIndexParallelism; private final List columnsToIndex; + private final List bloomSecondaryKeys; MetadataRecordsGenerationParams(HoodieTableMetaClient dataMetaClient, List enabledPartitionTypes, String bloomFilterType, int bloomIndexParallelism, - boolean isAllColumnStatsIndexEnabled, int columnStatsIndexParallelism, List columnsToIndex) { + boolean isAllColumnStatsIndexEnabled, int columnStatsIndexParallelism, List columnsToIndex, List bloomSecondaryKeys) { this.dataMetaClient = dataMetaClient; this.enabledPartitionTypes = enabledPartitionTypes; this.bloomFilterType = bloomFilterType; @@ -46,6 +47,7 @@ public class MetadataRecordsGenerationParams implements Serializable { this.isAllColumnStatsIndexEnabled = isAllColumnStatsIndexEnabled; this.columnStatsIndexParallelism = columnStatsIndexParallelism; this.columnsToIndex = columnsToIndex; + this.bloomSecondaryKeys = bloomSecondaryKeys; } public HoodieTableMetaClient getDataMetaClient() { @@ -75,4 +77,8 @@ public int getColumnStatsIndexParallelism() { public List getColumnsToIndex() { return columnsToIndex; } + + public List getBloomSecondaryKeys() { + return bloomSecondaryKeys; + } } From e9c528deee4ab724065691b506c0ff4fb75a46db Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Mon, 14 Mar 2022 21:56:14 +0530 Subject: [PATCH 05/21] [HUDI-3382] Add support for drop index in metadata writer --- .../hudi/metadata/HoodieBackedTableMetadataWriter.java | 8 ++++++++ .../apache/hudi/metadata/HoodieTableMetadataWriter.java | 2 ++ 2 files changed, 10 insertions(+) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index 98e8e518d6cf6..8fcb6b84ce30e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -667,6 +667,14 @@ public void initializeFileGroups(HoodieTableMetaClient dataMetaClient, MetadataP } } + public void dropIndex(List indexesToDrop) throws IOException { + // TODO: update table config and do it in a transaction + for (MetadataPartitionType partitionType : indexesToDrop) { + LOG.warn("Deleting Metadata Table partitions: " + partitionType.getPartitionPath()); + dataMetaClient.getFs().delete(new Path(metadataWriteConfig.getBasePath(), partitionType.getPartitionPath()), true); + } + } + private MetadataRecordsGenerationParams getRecordsGenerationParams() { return new MetadataRecordsGenerationParams( dataMetaClient, enabledPartitionTypes, dataWriteConfig.getBloomFilterType(), diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java index 2a1975f142977..3730582f3f148 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java @@ -39,6 +39,8 @@ public interface HoodieTableMetadataWriter extends Serializable, AutoCloseable { void initializeFileGroups(HoodieTableMetaClient dataMetaClient, MetadataPartitionType metadataPartition, String instantTime, int fileGroupCount) throws IOException; + void dropIndex(List indexesToDrop) throws IOException; + /** * Update the metadata table due to a COMMIT operation. * @param commitMetadata commit metadata of the operation of interest. From ab0b369fccab29b085ce120d9d20f294fab4e0e2 Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Thu, 17 Mar 2022 19:27:59 +0530 Subject: [PATCH 06/21] Handle upgrade downgrade and consider archival timeline --- .../apache/hudi/config/HoodieWriteConfig.java | 4 + .../HoodieBackedTableMetadataWriter.java | 49 ++--- .../metadata/HoodieTableMetadataWriter.java | 15 ++ .../action/index/RunIndexActionExecutor.java | 168 ++++++++++++------ .../index/ScheduleIndexActionExecutor.java | 28 ++- .../upgrade/FourToThreeDowngradeHandler.java | 6 + .../upgrade/ThreeToFourUpgradeHandler.java | 12 +- .../common/config/HoodieMetadataConfig.java | 16 ++ .../hudi/common/table/HoodieTableConfig.java | 20 +++ .../timeline/HoodieArchivedTimeline.java | 2 +- .../table/timeline/HoodieDefaultTimeline.java | 22 ++- .../common/table/timeline/HoodieTimeline.java | 9 + .../metadata/HoodieTableMetadataUtil.java | 16 ++ 13 files changed, 281 insertions(+), 86 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index a806bd754d710..e5fb1dc610cbf 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -1519,6 +1519,10 @@ public String getColumnsEnabledForBloomFilterIndex() { return getMetadataConfig().getColumnsEnabledForBloomFilterIndex(); } + public int getIndexingCheckTimeout() { + return getMetadataConfig().getIndexingCheckTimeout(); + } + public int getColumnStatsIndexParallelism() { return metadataConfig.getColumnStatsIndexParallelism(); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index 8fcb6b84ce30e..4c7afead84361 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -750,40 +750,41 @@ private List getExistingMetadataPartitions() { @Override public void index(HoodieEngineContext engineContext, List indexPartitionInfos) { + if (indexPartitionInfos.isEmpty()) { + LOG.warn("No partition to index in the plan"); + return; + } + String indexUptoInstantTime = indexPartitionInfos.get(0).getIndexUptoInstant(); indexPartitionInfos.forEach(indexPartitionInfo -> { - String indexUptoInstantTime = indexPartitionInfo.getIndexUptoInstant(); String relativePartitionPath = indexPartitionInfo.getMetadataPartitionPath(); LOG.info(String.format("Creating a new metadata index for partition '%s' under path %s upto instant %s", relativePartitionPath, metadataWriteConfig.getBasePath(), indexUptoInstantTime)); try { - HoodieTableMetaClient.withPropertyBuilder() - .setTableType(HoodieTableType.MERGE_ON_READ) - .setTableName(tableName) - .setArchiveLogFolder(ARCHIVELOG_FOLDER.defaultValue()) - .setPayloadClassName(HoodieMetadataPayload.class.getName()) - .setBaseFileFormat(HoodieFileFormat.HFILE.toString()) - .setRecordKeyFields(RECORD_KEY_FIELD_NAME) - .setPopulateMetaFields(dataWriteConfig.getMetadataConfig().populateMetaFields()) - .setKeyGeneratorClassProp(HoodieTableMetadataKeyGenerator.class.getCanonicalName()) - .initTable(hadoopConf.get(), metadataWriteConfig.getBasePath()); - initTableMetadata(); - // this part now moves to scheduling - initializeFileGroups(dataMetaClient, MetadataPartitionType.valueOf(relativePartitionPath.toUpperCase(Locale.ROOT)), indexUptoInstantTime, 1); + // filegroup should have already been initialized while scheduling index for this partition + if (!dataMetaClient.getFs().exists(new Path(metadataWriteConfig.getBasePath(), relativePartitionPath))) { + throw new HoodieIndexException(String.format("File group not initialized for metadata partition: %s, indexUptoInstant: %s. Looks like index scheduling failed!", + relativePartitionPath, indexUptoInstantTime)); + } } catch (IOException e) { - throw new HoodieIndexException(String.format("Unable to initialize file groups for metadata partition: %s, indexUptoInstant: %s", + throw new HoodieIndexException(String.format("Unable to check whether file group is initialized for metadata partition: %s, indexUptoInstant: %s", relativePartitionPath, indexUptoInstantTime)); } - // List all partitions in the basePath of the containing dataset - LOG.info("Initializing metadata table by using file listings in " + dataWriteConfig.getBasePath()); - engineContext.setJobStatus(this.getClass().getSimpleName(), "MetadataIndex: initializing metadata table by listing files and partitions"); - List dirInfoList = listAllPartitions(dataMetaClient); - - // During bootstrap, the list of files to be committed can be huge. So creating a HoodieCommitMetadata out of these - // large number of files and calling the existing update(HoodieCommitMetadata) function does not scale well. - // Hence, we have a special commit just for the bootstrap scenario. - initialCommit(indexUptoInstantTime); + // return early and populate enabledPartitionTypses correctly (check in initialCommit) + MetadataPartitionType partitionType = MetadataPartitionType.valueOf(relativePartitionPath); + if (!enabledPartitionTypes.contains(partitionType)) { + throw new HoodieIndexException(String.format("Indexing for metadata partition: %s is not enabled", partitionType)); + } }); + // before initial commit update table config + dataMetaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_INDEX_INFLIGHT.key(), indexPartitionInfos.stream() + .map(HoodieIndexPartitionInfo::getMetadataPartitionPath).collect(Collectors.joining(","))); + HoodieTableConfig.update(dataMetaClient.getFs(), new Path(dataMetaClient.getMetaPath()), dataMetaClient.getTableConfig().getProps()); + initialCommit(indexUptoInstantTime); + dataMetaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_INDEX_INFLIGHT.key(), ""); + dataMetaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_INDEX_COMPLETED.key(), indexPartitionInfos.stream() + .map(HoodieIndexPartitionInfo::getMetadataPartitionPath).collect(Collectors.joining(","))); + HoodieTableConfig.update(dataMetaClient.getFs(), new Path(dataMetaClient.getMetaPath()), dataMetaClient.getTableConfig().getProps()); } /** diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java index 3730582f3f148..4bfe7047b8f23 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java @@ -35,8 +35,23 @@ */ public interface HoodieTableMetadataWriter extends Serializable, AutoCloseable { + /** + * Execute the index action for the given partitions. + * + * @param engineContext + * @param indexPartitionInfos - partitions to index + */ void index(HoodieEngineContext engineContext, List indexPartitionInfos); + /** + * Initialize file groups while scheduling index action. + * + * @param dataMetaClient - meta client for the data table + * @param metadataPartition - metadata partition for which file groups needs to be initialized + * @param instantTime - instant time of the index action + * @param fileGroupCount - number of file groups to be initialized + * @throws IOException + */ void initializeFileGroups(HoodieTableMetaClient dataMetaClient, MetadataPartitionType metadataPartition, String instantTime, int fileGroupCount) throws IOException; void dropIndex(List indexesToDrop) throws IOException; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java index c08b55c312568..899abd53bde74 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java @@ -25,18 +25,18 @@ import org.apache.hudi.avro.model.HoodieIndexPlan; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; +import org.apache.hudi.client.transaction.TransactionManager; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieRecordPayload; 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.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.util.CleanerUtils; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIndexException; import org.apache.hudi.metadata.HoodieTableMetadata; @@ -57,7 +57,11 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; -import java.util.stream.Stream; + +import static org.apache.hudi.common.model.WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL; +import static org.apache.hudi.common.table.timeline.HoodieInstant.State.COMPLETED; +import static org.apache.hudi.common.table.timeline.HoodieInstant.State.REQUESTED; +import static org.apache.hudi.config.HoodieWriteConfig.WRITE_CONCURRENCY_MODE; /** * Reads the index plan and executes the plan. @@ -70,8 +74,16 @@ public class RunIndexActionExecutor exte private static final Integer LATEST_INDEX_COMMIT_METADATA_VERSION = INDEX_COMMIT_METADATA_VERSION_1; private static final int MAX_CONCURRENT_INDEXING = 1; + // we use this to update the latest instant in data timeline that has been indexed in metadata table + // this needs to be volatile as it can be updated in the IndexingCheckTask spawned by this executor + // assumption is that only one indexer can execute at a time + private volatile String currentIndexedInstant; + + private final TransactionManager txnManager; + public RunIndexActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, String instantTime) { super(context, config, table, instantTime); + this.txnManager = new TransactionManager(config, table.getMetaClient().getFs()); } @Override @@ -79,16 +91,19 @@ public Option execute() { HoodieTimer indexTimer = new HoodieTimer(); indexTimer.startTimer(); + // ensure lock provider configured + if (!config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl() || StringUtils.isNullOrEmpty(config.getLockProviderClass())) { + throw new HoodieIndexException(String.format("Need to set %s as %s and configure lock provider class", + WRITE_CONCURRENCY_MODE.key(), OPTIMISTIC_CONCURRENCY_CONTROL.name())); + } + HoodieInstant indexInstant = table.getActiveTimeline() .filterPendingIndexTimeline() - .filter(instant -> instant.getTimestamp().equals(instantTime)) + .filter(instant -> instant.getTimestamp().equals(instantTime) && REQUESTED.equals(instant.getState())) .lastInstant() - .orElseThrow(() -> new HoodieIndexException(String.format("No pending index instant found: %s", instantTime))); - ValidationUtils.checkArgument(HoodieInstant.State.INFLIGHT.equals(indexInstant.getState()), - String.format("Index instant %s already inflight", instantTime)); + .orElseThrow(() -> new HoodieIndexException(String.format("No requested index instant found: %s", instantTime))); try { - // read HoodieIndexPlan assuming indexInstant is requested - // TODO: handle inflight instant, if it is inflight then throw error. + // read HoodieIndexPlan HoodieIndexPlan indexPlan = TimelineMetadataUtils.deserializeIndexPlan(table.getActiveTimeline().readIndexPlanAsBytes(indexInstant).get()); List indexPartitionInfos = indexPlan.getIndexPartitionInfos(); if (indexPartitionInfos == null || indexPartitionInfos.isEmpty()) { @@ -100,41 +115,38 @@ public Option execute() { HoodieTableMetadataWriter metadataWriter = table.getMetadataWriter(instantTime) .orElseThrow(() -> new HoodieIndexException(String.format("Could not get metadata writer to run index action for instant: %s", instantTime))); metadataWriter.index(context, indexPartitionInfos); - // get all completed instants since the plan completed + + // get all instants since the plan completed (both from active timeline and archived timeline) // assumption is that all metadata partitions had same instant upto which they were scheduled to be indexed + table.getMetaClient().reloadActiveTimeline(); String indexUptoInstant = indexPartitionInfos.get(0).getIndexUptoInstant(); - Stream remainingInstantsToIndex = table.getActiveTimeline().getWriteTimeline().getReverseOrderedInstants() - .filter(instant -> instant.isCompleted() && HoodieActiveTimeline.GREATER_THAN.test(instant.getTimestamp(), indexUptoInstant)); + List instantsToIndex = getRemainingArchivedAndActiveInstantsSince(indexUptoInstant, table.getMetaClient()); + // reconcile with metadata table timeline String metadataBasePath = HoodieTableMetadata.getMetadataTableBasePath(table.getMetaClient().getBasePath()); HoodieTableMetaClient metadataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(metadataBasePath).build(); - Set metadataCompletedTimeline = metadataMetaClient.getActiveTimeline() - .getCommitsTimeline().filterCompletedInstants().getInstants().collect(Collectors.toSet()); - List finalRemainingInstantsToIndex = remainingInstantsToIndex.map( - instant -> new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instant.getTimestamp()) - ).filter(instant -> !metadataCompletedTimeline.contains(instant)).collect(Collectors.toList()); + Set metadataCompletedTimestamps = getCompletedArchivedAndActiveInstantsAfter(indexUptoInstant, metadataMetaClient).stream() + .map(HoodieInstant::getTimestamp).collect(Collectors.toSet()); // index all remaining instants with a timeout + currentIndexedInstant = indexUptoInstant; ExecutorService executorService = Executors.newFixedThreadPool(MAX_CONCURRENT_INDEXING); - Future postRequestIndexingTaskFuture = executorService.submit(new PostRequestIndexingTask(metadataWriter, finalRemainingInstantsToIndex)); + Future postRequestIndexingTaskFuture = executorService.submit( + new IndexingCheckTask(metadataWriter, instantsToIndex, metadataCompletedTimestamps, table.getMetaClient())); try { - // TODO: configure timeout - postRequestIndexingTaskFuture.get(60, TimeUnit.SECONDS); + postRequestIndexingTaskFuture.get(config.getIndexingCheckTimeout(), TimeUnit.SECONDS); } catch (TimeoutException | InterruptedException | ExecutionException e) { postRequestIndexingTaskFuture.cancel(true); } finally { executorService.shutdownNow(); } - Option lastMetadataInstant = metadataMetaClient.reloadActiveTimeline().getCommitsTimeline().filterCompletedInstants().lastInstant(); - if (lastMetadataInstant.isPresent() && indexUptoInstant.equals(lastMetadataInstant.get().getTimestamp())) { - return Option.of(HoodieIndexCommitMetadata.newBuilder() - .setVersion(LATEST_INDEX_COMMIT_METADATA_VERSION).setIndexPartitionInfos(indexPartitionInfos).build()); - } + // build index commit metadata and return List finalIndexPartitionInfos = indexPartitionInfos.stream() .map(info -> new HoodieIndexPartitionInfo( info.getVersion(), info.getMetadataPartitionPath(), - lastMetadataInstant.get().getTimestamp())).collect(Collectors.toList()); + currentIndexedInstant)) + .collect(Collectors.toList()); return Option.of(HoodieIndexCommitMetadata.newBuilder() .setVersion(LATEST_INDEX_COMMIT_METADATA_VERSION).setIndexPartitionInfos(finalIndexPartitionInfos).build()); } catch (IOException e) { @@ -142,45 +154,97 @@ public Option execute() { } } - class PostRequestIndexingTask implements Runnable { + private static List getRemainingArchivedAndActiveInstantsSince(String instant, HoodieTableMetaClient metaClient) { + List remainingInstantsToIndex = metaClient.getArchivedTimeline() + .getWriteTimeline() + .findInstantsAfter(instant) + .getInstants().collect(Collectors.toList()); + remainingInstantsToIndex.addAll(metaClient.getActiveTimeline().getWriteTimeline().findInstantsAfter(instant).getInstants().collect(Collectors.toList())); + return remainingInstantsToIndex; + } + + private static List getCompletedArchivedAndActiveInstantsAfter(String instant, HoodieTableMetaClient metaClient) { + List completedInstants = metaClient.getArchivedTimeline() + .filterCompletedInstants() + .findInstantsAfter(instant) + .getInstants().collect(Collectors.toList()); + completedInstants.addAll(metaClient.getActiveTimeline().filterCompletedInstants().findInstantsAfter(instant).getInstants().collect(Collectors.toList())); + return completedInstants; + } + + /** + * Indexing check runs for instants that completed after the base instant (in the index plan). + * It will check if these later instants have logged updates to metadata table or not. + * If not, then it will do the update. If a later instant is inflight, it will wait until it is completed or the task times out. + */ + class IndexingCheckTask implements Runnable { private final HoodieTableMetadataWriter metadataWriter; private final List instantsToIndex; + private final Set metadataCompletedInstants; + private final HoodieTableMetaClient metaClient; - PostRequestIndexingTask(HoodieTableMetadataWriter metadataWriter, List instantsToIndex) { + IndexingCheckTask(HoodieTableMetadataWriter metadataWriter, + List instantsToIndex, + Set metadataCompletedInstants, + HoodieTableMetaClient metaClient) { this.metadataWriter = metadataWriter; this.instantsToIndex = instantsToIndex; + this.metadataCompletedInstants = metadataCompletedInstants; + this.metaClient = metaClient; } @Override public void run() { while (!Thread.interrupted()) { for (HoodieInstant instant : instantsToIndex) { - try { - switch (instant.getAction()) { - case HoodieTimeline.COMMIT_ACTION: - case HoodieTimeline.DELTA_COMMIT_ACTION: - case HoodieTimeline.REPLACE_COMMIT_ACTION: - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(table.getActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class); - metadataWriter.update(commitMetadata, instant.getTimestamp(), false); - break; - case HoodieTimeline.CLEAN_ACTION: - HoodieCleanMetadata cleanMetadata = CleanerUtils.getCleanerMetadata(table.getMetaClient(), instant); - metadataWriter.update(cleanMetadata, instant.getTimestamp()); - break; - case HoodieTimeline.RESTORE_ACTION: - HoodieRestoreMetadata restoreMetadata = TimelineMetadataUtils.deserializeHoodieRestoreMetadata(table.getActiveTimeline().getInstantDetails(instant).get()); - metadataWriter.update(restoreMetadata, instant.getTimestamp()); - break; - case HoodieTimeline.ROLLBACK_ACTION: - HoodieRollbackMetadata rollbackMetadata = TimelineMetadataUtils.deserializeHoodieRollbackMetadata(table.getActiveTimeline().getInstantDetails(instant).get()); - metadataWriter.update(rollbackMetadata, instant.getTimestamp()); - break; - default: - throw new IllegalStateException("Unexpected value: " + instant.getAction()); + // metadata index already updated for this instant + if (metadataCompletedInstants.contains(instant.getTimestamp())) { + currentIndexedInstant = instant.getTimestamp(); + continue; + } + while (!instant.isCompleted()) { + // reload timeline and fetch instant details again wait until timeout + String instantTime = instant.getTimestamp(); + Option currentInstant = metaClient.reloadActiveTimeline() + .filterCompletedInstants().filter(i -> i.getTimestamp().equals(instantTime)).firstInstant(); + instant = currentInstant.orElse(instant); + } + // update metadata for this completed instant + if (COMPLETED.equals(instant.getState())) { + try { + // we need take a lock here as inflight writer could also try to update the timeline + txnManager.beginTransaction(Option.of(instant), Option.empty()); + switch (instant.getAction()) { + case HoodieTimeline.COMMIT_ACTION: + case HoodieTimeline.DELTA_COMMIT_ACTION: + case HoodieTimeline.REPLACE_COMMIT_ACTION: + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( + table.getActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class); + metadataWriter.update(commitMetadata, instant.getTimestamp(), false); + break; + case HoodieTimeline.CLEAN_ACTION: + HoodieCleanMetadata cleanMetadata = CleanerUtils.getCleanerMetadata(table.getMetaClient(), instant); + metadataWriter.update(cleanMetadata, instant.getTimestamp()); + break; + case HoodieTimeline.RESTORE_ACTION: + HoodieRestoreMetadata restoreMetadata = TimelineMetadataUtils.deserializeHoodieRestoreMetadata( + table.getActiveTimeline().getInstantDetails(instant).get()); + metadataWriter.update(restoreMetadata, instant.getTimestamp()); + break; + case HoodieTimeline.ROLLBACK_ACTION: + HoodieRollbackMetadata rollbackMetadata = TimelineMetadataUtils.deserializeHoodieRollbackMetadata( + table.getActiveTimeline().getInstantDetails(instant).get()); + metadataWriter.update(rollbackMetadata, instant.getTimestamp()); + break; + default: + throw new IllegalStateException("Unexpected value: " + instant.getAction()); + } + } catch (IOException e) { + LOG.error("Could not update metadata partition for instant: " + instant); + } finally { + txnManager.endTransaction(Option.of(instant)); } - } catch (IOException e) { - e.printStackTrace(); } } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java index d18fd80b4cd1e..6948ffeb5368a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java @@ -28,6 +28,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieIndexException; @@ -40,15 +41,19 @@ import org.apache.log4j.Logger; import java.io.IOException; +import java.util.EnumSet; import java.util.List; import java.util.stream.Collectors; +import static org.apache.hudi.common.model.WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL; +import static org.apache.hudi.config.HoodieWriteConfig.WRITE_CONCURRENCY_MODE; + /** * Schedules INDEX action. *
  • - * 1. Fetch last completed instant on data timeline. - * 2. Write the index plan to the .index.requested. - * 3. Initialize filegroups for the enabled partition types within a transaction. + * 1. Fetch last completed instant on data timeline. + * 2. Write the index plan to the .index.requested. + * 3. Initialize filegroups for the enabled partition types within a transaction. *
  • */ public class ScheduleIndexActionExecutor extends BaseActionExecutor> { @@ -73,11 +78,20 @@ public ScheduleIndexActionExecutor(HoodieEngineContext context, @Override public Option execute() { // validate partitionsToIndex - if (!MetadataPartitionType.allPaths().containsAll(partitionsToIndex)) { + if (!EnumSet.allOf(MetadataPartitionType.class).containsAll(partitionsToIndex)) { throw new HoodieIndexException("Not all partitions are valid: " + partitionsToIndex); } + // ensure lock provider configured + if (!config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl() || StringUtils.isNullOrEmpty(config.getLockProviderClass())) { + throw new HoodieIndexException(String.format("Need to set %s as %s and configure lock provider class", + WRITE_CONCURRENCY_MODE.key(), OPTIMISTIC_CONCURRENCY_CONTROL.name())); + } + // make sure that it is idempotent, check with previously pending index operations. + String[] indexesInflight = table.getMetaClient().getTableConfig().getInflightMetadataIndexes().split(","); + String[] indexesCompleted = table.getMetaClient().getTableConfig().getCompletedMetadataIndexes().split(","); + // get last completed instant - Option indexUptoInstant = table.getActiveTimeline().filterCompletedInstants().lastInstant(); + Option indexUptoInstant = table.getActiveTimeline().getContiguousCompletedWriteTimeline().lastInstant(); if (indexUptoInstant.isPresent()) { final HoodieInstant indexInstant = HoodieTimeline.getIndexRequestedInstant(instantTime); // for each partitionToIndex add that time to the plan @@ -96,13 +110,13 @@ public Option execute() { // start initializing filegroups // 1. get metadata writer HoodieTableMetadataWriter metadataWriter = table.getMetadataWriter(instantTime) - .orElseThrow(() -> new HoodieIndexException(String.format("Could not get metadata writer to run index action for instant: %s", instantTime))); + .orElseThrow(() -> new HoodieIndexException(String.format("Could not get metadata writer to initialize filegroups for indexing for instant: %s", instantTime))); // 2. take a lock --> begin tx (data table) try { this.txnManager.beginTransaction(Option.of(indexInstant), Option.empty()); // 3. initialize filegroups as per plan for the enabled partition types for (MetadataPartitionType partitionType : partitionsToIndex) { - metadataWriter.initializeFileGroups(table.getMetaClient(), partitionType, indexInstant.getTimestamp(), 1); + metadataWriter.initializeFileGroups(table.getMetaClient(), partitionType, indexInstant.getTimestamp(), partitionType.getFileGroupCount()); } } catch (IOException e) { LOG.error("Could not initialize file groups"); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/FourToThreeDowngradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/FourToThreeDowngradeHandler.java index 17dc01d0213e7..86a594af17c5e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/FourToThreeDowngradeHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/FourToThreeDowngradeHandler.java @@ -22,6 +22,7 @@ import org.apache.hudi.common.config.ConfigProperty; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; import java.util.Collections; import java.util.Map; @@ -33,6 +34,11 @@ public class FourToThreeDowngradeHandler implements DowngradeHandler { @Override public Map downgrade(HoodieWriteConfig config, HoodieEngineContext context, String instantTime, SupportsUpgradeDowngrade upgradeDowngradeHelper) { + if (config.isMetadataTableEnabled()) { + // Metadata Table in version 4 has a schema that is not forward compatible. + // Hence, it is safe to delete the metadata table, which will be re-initialized in subsequent commit. + HoodieTableMetadataUtil.deleteMetadataTable(config.getBasePath(), context); + } return Collections.emptyMap(); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ThreeToFourUpgradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ThreeToFourUpgradeHandler.java index 72e96bb4103bc..c5847bdd06c67 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ThreeToFourUpgradeHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ThreeToFourUpgradeHandler.java @@ -23,10 +23,15 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.metadata.MetadataPartitionType; import java.util.Hashtable; import java.util.Map; +import static org.apache.hudi.common.table.HoodieTableConfig.TABLE_CHECKSUM; +import static org.apache.hudi.common.table.HoodieTableConfig.TABLE_METADATA_INDEX_COMPLETED; +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.metadataPartitionExists; + /** * UpgradeHandler to assist in upgrading {@link org.apache.hudi.table.HoodieTable} from version 3 to 4. */ @@ -35,7 +40,12 @@ public class ThreeToFourUpgradeHandler implements UpgradeHandler { @Override public Map upgrade(HoodieWriteConfig config, HoodieEngineContext context, String instantTime, SupportsUpgradeDowngrade upgradeDowngradeHelper) { Map tablePropsToAdd = new Hashtable<>(); - tablePropsToAdd.put(HoodieTableConfig.TABLE_CHECKSUM, String.valueOf(HoodieTableConfig.generateChecksum(config.getProps()))); + tablePropsToAdd.put(TABLE_CHECKSUM, String.valueOf(HoodieTableConfig.generateChecksum(config.getProps()))); + // if metadata is enabled and files partition exist then update TABLE_METADATA_INDEX_COMPLETED + // schema for the files partition is same between the two versions + if (config.isMetadataTableEnabled() && metadataPartitionExists(config.getBasePath(), context, MetadataPartitionType.FILES)) { + tablePropsToAdd.put(TABLE_METADATA_INDEX_COMPLETED, MetadataPartitionType.FILES.getPartitionPath()); + } return tablePropsToAdd; } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java index c67f7a8e7d0ef..1488ee1e19a53 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java @@ -194,6 +194,13 @@ public final class HoodieMetadataConfig extends HoodieConfig { .sinceVersion("0.11.0") .withDocumentation("Comma-separated list of columns for which bloom filter index will be built."); + public static final ConfigProperty METADATA_INDEX_CHECK_TIMEOUT_SECONDS = ConfigProperty + .key(METADATA_PREFIX + ".index.check.timeout.seconds") + .defaultValue(300) + .sinceVersion("0.11.0") + .withDocumentation("After the async indexer has finished indexing upto the base instant, it will reconcile with commits that happened after the base instant. " + + "This check could take finite amount of time depending on number of commits, so it needs to be bounded by a timeout which can configured with this key."); + public static final ConfigProperty POPULATE_META_FIELDS = ConfigProperty .key(METADATA_PREFIX + ".populate.meta.fields") .defaultValue(false) @@ -260,6 +267,10 @@ public int getColumnStatsIndexParallelism() { return getIntOrDefault(COLUMN_STATS_INDEX_PARALLELISM); } + public int getIndexingCheckTimeout() { + return getIntOrDefault(METADATA_INDEX_CHECK_TIMEOUT_SECONDS); + } + public boolean enableMetrics() { return getBoolean(METRICS_ENABLE); } @@ -342,6 +353,11 @@ public Builder withBloomFilterIndexForColumns(String columns) { return this; } + public Builder withIndexingCheckTimeout(int timeoutInSeconds) { + metadataConfig.setValue(METADATA_INDEX_CHECK_TIMEOUT_SECONDS, String.valueOf(timeoutInSeconds)); + return this; + } + public Builder enableMetrics(boolean enableMetrics) { metadataConfig.setValue(METRICS_ENABLE, String.valueOf(enableMetrics)); return this; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java index 6b8103db3a663..ba6ab0e9be79b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java @@ -208,6 +208,18 @@ public class HoodieTableConfig extends HoodieConfig { .sinceVersion("0.11.0") .withDocumentation("Table checksum is used to guard against partial writes in HDFS. It is added as the last entry in hoodie.properties and then used to validate while reading table config."); + public static final ConfigProperty TABLE_METADATA_INDEX_INFLIGHT = ConfigProperty + .key("hoodie.table.metadata.index.inflight") + .noDefaultValue() + .sinceVersion("0.11.0") + .withDocumentation("Comma-separated list of metadata partitions whose indexing is in progress."); + + public static final ConfigProperty TABLE_METADATA_INDEX_COMPLETED = ConfigProperty + .key("hoodie.table.metadata.index.completed") + .noDefaultValue() + .sinceVersion("0.11.0") + .withDocumentation("Comma-separated list of metadata partitions whose indexing is complete."); + private static final String TABLE_CHECKSUM_FORMAT = "%s.%s"; // . public HoodieTableConfig(FileSystem fs, String metaPath, String payloadClassName) { @@ -585,6 +597,14 @@ private Long getTableChecksum() { return getLong(TABLE_CHECKSUM); } + public String getInflightMetadataIndexes() { + return getStringOrDefault(TABLE_METADATA_INDEX_INFLIGHT, ""); + } + + public String getCompletedMetadataIndexes() { + return getStringOrDefault(TABLE_METADATA_INDEX_COMPLETED, ""); + } + public Map propsMap() { return props.entrySet().stream() .collect(Collectors.toMap(e -> String.valueOf(e.getKey()), e -> String.valueOf(e.getValue()))); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java index c38fdb3b52db5..0cb3f56470692 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java @@ -363,7 +363,7 @@ private int getArchivedFileSuffix(FileStatus f) { @Override public HoodieDefaultTimeline getWriteTimeline() { // filter in-memory instants - Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, REPLACE_COMMIT_ACTION, INDEX_ACTION); + Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, REPLACE_COMMIT_ACTION); return new HoodieDefaultTimeline(getInstants().filter(i -> readCommits.containsKey(i.getTimestamp())) .filter(s -> validActions.contains(s.getAction())), details); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java index 02c5c49984c8c..01210082b41f0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java @@ -27,6 +27,7 @@ import java.io.Serializable; import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; +import java.util.ArrayList; import java.util.List; import java.util.Set; import java.util.function.Function; @@ -109,10 +110,29 @@ public HoodieTimeline filterCompletedAndCompactionInstants() { @Override public HoodieDefaultTimeline getWriteTimeline() { - Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, REPLACE_COMMIT_ACTION, INDEX_ACTION); + Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, REPLACE_COMMIT_ACTION); return new HoodieDefaultTimeline(instants.stream().filter(s -> validActions.contains(s.getAction())), details); } + @Override + public HoodieDefaultTimeline getContiguousCompletedWriteTimeline() { + List contiguousCompletedInstants = new ArrayList<>(); + String prevTime = ""; + for (HoodieInstant instant : instants) { + if (prevTime.equals(instant.getTimestamp())) { + continue; + } + if (!instant.isCompleted()) { + break; + } + contiguousCompletedInstants.add(instant); + prevTime = instant.getTimestamp(); + } + + Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, REPLACE_COMMIT_ACTION); + return new HoodieDefaultTimeline(contiguousCompletedInstants.stream().filter(s -> validActions.contains(s.getAction())), details); + } + @Override public HoodieTimeline getCompletedReplaceTimeline() { return new HoodieDefaultTimeline( diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java index 9b5de8ace1e16..6e17e604d82a8 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java @@ -143,6 +143,15 @@ public interface HoodieTimeline extends Serializable { */ HoodieTimeline getWriteTimeline(); + /** + * Timeline to just include commits (commit/deltacommit), compaction and replace actions that are completed and contiguous. + * For example, if timeline is [C0.completed, C1.completed, C2.completed, C3.inflight, C4.completed]. + * Then, a timeline of [C0.completed, C1.completed, C2.completed] will be returned. + * + * @return + */ + HoodieTimeline getContiguousCompletedWriteTimeline(); + /** * Timeline to just include replace instants that have valid (commit/deltacommit) actions. * diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java index 7bd5ccc9cb824..5f1e73ba584da 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java @@ -123,6 +123,22 @@ public static void deleteMetadataTable(String basePath, HoodieEngineContext cont } } + /** + * Check if the given metadata partition exists. + * + * @param basePath base path of the dataset + * @param context instance of {@link HoodieEngineContext}. + */ + public static boolean metadataPartitionExists(String basePath, HoodieEngineContext context, MetadataPartitionType partitionType) { + final String metadataTablePath = HoodieTableMetadata.getMetadataTableBasePath(basePath); + FileSystem fs = FSUtils.getFs(metadataTablePath, context.getHadoopConf().get()); + try { + return fs.exists(new Path(metadataTablePath, partitionType.getPartitionPath())); + } catch (Exception e) { + throw new HoodieIOException(String.format("Failed to check metadata partition %s exists.", partitionType.getPartitionPath())); + } + } + /** * Convert commit action to metadata records for the enabled partition types. * From b4d41004b085e31cac721f269c54c43c75852e9e Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Mon, 21 Mar 2022 14:57:38 +0530 Subject: [PATCH 07/21] Add drop index action to utility and avoid fs.exists check Resolve minor rebase conflict --- .../hudi/client/BaseHoodieWriteClient.java | 18 +++++ .../apache/hudi/io/HoodieAppendHandle.java | 4 +- .../HoodieBackedTableMetadataWriter.java | 65 ++++++++----------- .../metadata/HoodieTableMetadataWriter.java | 6 ++ .../apache/hudi/utilities/HoodieIndexer.java | 42 ++++++++---- 5 files changed, 84 insertions(+), 51 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java index 7c98154406a25..c84021daca23f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java @@ -942,6 +942,24 @@ public Option index(String indexInstantTime) { return createTable(config, hadoopConf, config.isMetadataTableEnabled()).index(context, indexInstantTime); } + public void dropIndex(List partitionTypes) { + HoodieTable table = createTable(config, hadoopConf); + String dropInstant = HoodieActiveTimeline.createNewInstantTime(); + this.txnManager.beginTransaction(); + try { + context.setJobStatus(this.getClass().getSimpleName(), "Dropping partitions from metadata table"); + table.getMetadataWriter(dropInstant).ifPresent(w -> { + try { + ((HoodieTableMetadataWriter) w).dropIndex(partitionTypes); + } catch (IOException e) { + LOG.error("Failed to drop metadata index. ", e); + } + }); + } finally { + this.txnManager.endTransaction(); + } + } + /** * Performs Compaction for the workload stored in instant-time. * diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java index fcd1c7fcbc2d1..840a77667fbb1 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java @@ -353,8 +353,8 @@ private void processAppendResult(AppendResult result, List record Map> columnToStats = new HashMap<>(); columnsToIndex.forEach(column -> columnToStats.putIfAbsent(column, new HashMap<>())); // collect stats for columns at once per record and keep iterating through every record to eventually find col stats for all fields. - recordList.forEach(record -> aggregateColumnStats(record, columnsToIndex, columnToStats, config.isConsistentLogicalTimestampEnabled())); - columnsToIndex.forEach(column -> accumulateColumnRanges(column, filePath, columnRangeMap, columnToStats)); + recordList.forEach(record -> aggregateColumnStats(record, writeSchemaWithMetaFields, columnToStats, config.isConsistentLogicalTimestampEnabled())); + writeSchemaWithMetaFields.getFields().forEach(field -> accumulateColumnRanges(field, filePath, columnRangeMap, columnToStats)); stat.setRecordsStats(new HoodieDeltaWriteStat.RecordsStats<>(columnRangeMap)); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index 4c7afead84361..8f2452dd47fb6 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -20,7 +20,6 @@ import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieIndexPartitionInfo; -import org.apache.hudi.avro.model.HoodieIndexPlan; import org.apache.hudi.avro.model.HoodieInstantInfo; import org.apache.hudi.avro.model.HoodieMetadataRecord; import org.apache.hudi.avro.model.HoodieRestoreMetadata; @@ -52,10 +51,8 @@ import org.apache.hudi.common.table.marker.MarkerType; 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.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; -import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; @@ -88,6 +85,7 @@ import java.util.Locale; import java.util.Map; import java.util.Properties; +import java.util.Set; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -120,7 +118,6 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta protected boolean enabled; protected SerializableConfiguration hadoopConf; protected final transient HoodieEngineContext engineContext; - // TODO: HUDI-3258 Support secondary key via multiple partitions within a single type protected final List enabledPartitionTypes; /** @@ -403,7 +400,6 @@ protected void initializeIfNeeded(HoodieTableMeta if (!exists) { if (metadataWriteConfig.isMetadataAsyncIndex()) { // with async metadata indexing enabled, there can be inflight writers - // TODO: schedule indexing only for enabled partition types MetadataRecordsGenerationParams indexParams = getRecordsGenerationParams(); scheduleIndex(indexParams.getEnabledPartitionTypes()); return; @@ -668,11 +664,21 @@ public void initializeFileGroups(HoodieTableMetaClient dataMetaClient, MetadataP } public void dropIndex(List indexesToDrop) throws IOException { - // TODO: update table config and do it in a transaction + Set completedIndexes = Stream.of(dataMetaClient.getTableConfig().getCompletedMetadataIndexes().split(",")).collect(Collectors.toSet()); + Set inflightIndexes = Stream.of(dataMetaClient.getTableConfig().getInflightMetadataIndexes().split(",")).collect(Collectors.toSet()); for (MetadataPartitionType partitionType : indexesToDrop) { - LOG.warn("Deleting Metadata Table partitions: " + partitionType.getPartitionPath()); - dataMetaClient.getFs().delete(new Path(metadataWriteConfig.getBasePath(), partitionType.getPartitionPath()), true); + String partitionPath = partitionType.getPartitionPath(); + if (inflightIndexes.contains(partitionPath)) { + LOG.error("Metadata indexing in progress: " + partitionPath); + return; + } + LOG.warn("Deleting Metadata Table partitions: " + partitionPath); + dataMetaClient.getFs().delete(new Path(metadataWriteConfig.getBasePath(), partitionPath), true); + completedIndexes.remove(partitionPath); } + // update table config + dataMetaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_INDEX_COMPLETED.key(), String.join(",", completedIndexes)); + HoodieTableConfig.update(dataMetaClient.getFs(), new Path(dataMetaClient.getMetaPath()), dataMetaClient.getTableConfig().getProps()); } private MetadataRecordsGenerationParams getRecordsGenerationParams() { @@ -702,7 +708,10 @@ private interface ConvertMetadataFunction { * @param canTriggerTableService true if table services can be triggered. false otherwise. */ private void processAndCommit(String instantTime, ConvertMetadataFunction convertMetadataFunction, boolean canTriggerTableService) { - List partitionsToUpdate = getMetadataPartitionsToUpdate(); + if (!dataWriteConfig.isMetadataTableEnabled()) { + return; + } + Set partitionsToUpdate = getMetadataPartitionsToUpdate(); partitionsToUpdate.forEach(p -> { if (enabled && metadata != null) { try { @@ -716,36 +725,16 @@ private void processAndCommit(String instantTime, ConvertMetadataFunction co }); } - private List getMetadataPartitionsToUpdate() { - // find last (pending or) completed index instant and get partitions (to be) written - Option lastIndexingInstant = dataMetaClient.getActiveTimeline() - .getTimelineOfActions(CollectionUtils.createImmutableSet(HoodieTimeline.INDEX_ACTION)).lastInstant(); - if (lastIndexingInstant.isPresent()) { - try { - // TODO: handle inflight instant, if it is inflight then read from requested file. - HoodieIndexPlan indexPlan = TimelineMetadataUtils.deserializeIndexPlan( - dataMetaClient.getActiveTimeline().readIndexPlanAsBytes(lastIndexingInstant.get()).get()); - return indexPlan.getIndexPartitionInfos().stream().map(HoodieIndexPartitionInfo::getMetadataPartitionPath).collect(Collectors.toList()); - } catch (IOException e) { - LOG.warn("Could not read index plan. Falling back to FileSystem.exists() check."); - return getExistingMetadataPartitions(); - } + private Set getMetadataPartitionsToUpdate() { + // fetch partitions to update from table config + Set partitionsToUpdate = Stream.of(dataMetaClient.getTableConfig().getCompletedMetadataIndexes().split(",")).collect(Collectors.toSet()); + partitionsToUpdate.addAll(Stream.of(dataMetaClient.getTableConfig().getInflightMetadataIndexes().split(",")).collect(Collectors.toSet())); + if (!partitionsToUpdate.isEmpty()) { + return partitionsToUpdate; } - // TODO: return only enabled partitions - return MetadataPartitionType.allPaths(); - } - - private List getExistingMetadataPartitions() { - return MetadataPartitionType.allPaths().stream() - .filter(p -> { - try { - // TODO: avoid fs.exists() check - return metadataMetaClient.getFs().exists(FSUtils.getPartitionPath(metadataWriteConfig.getBasePath(), p)); - } catch (IOException e) { - return false; - } - }) - .collect(Collectors.toList()); + // fallback to update files partition only if table config returned no partitions + partitionsToUpdate.add(MetadataPartitionType.FILES.getPartitionPath()); + return partitionsToUpdate; } @Override diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java index 4bfe7047b8f23..6e066880746ea 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java @@ -54,6 +54,12 @@ public interface HoodieTableMetadataWriter extends Serializable, AutoCloseable { */ void initializeFileGroups(HoodieTableMetaClient dataMetaClient, MetadataPartitionType metadataPartition, String instantTime, int fileGroupCount) throws IOException; + /** + * Drop the given metadata indexes. + * + * @param indexesToDrop + * @throws IOException + */ void dropIndex(List indexesToDrop) throws IOException; /** diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java index c99a9d145cff8..fb674f14ff0bf 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java @@ -48,9 +48,13 @@ import static org.apache.hudi.utilities.UtilHelpers.SCHEDULE; import static org.apache.hudi.utilities.UtilHelpers.SCHEDULE_AND_EXECUTE; +/** + * A tool to run metadata indexing asynchronously. + */ public class HoodieIndexer { private static final Logger LOG = LogManager.getLogger(HoodieIndexer.class); + private static final String DROP_INDEX = "dropindex"; private final HoodieIndexer.Config cfg; private TypedProperties props; @@ -86,19 +90,17 @@ public static class Config implements Serializable { public String sparkMemory = null; @Parameter(names = {"--retry", "-rt"}, description = "number of retries") public int retry = 0; - @Parameter(names = {"--schedule", "-sc"}, description = "Schedule indexing") - public Boolean runSchedule = false; - @Parameter(names = {"--strategy", "-st"}, description = "Comma-separated index types to be built, e.g. BLOOM,FILES,COLSTATS") + @Parameter(names = {"--index-types", "-it"}, description = "Comma-separated index types to be built, e.g. BLOOM_FILTERS,COLUMN_STATS", required = true) public String indexTypes = null; @Parameter(names = {"--mode", "-m"}, description = "Set job mode: Set \"schedule\" to generate an indexing plan; " + "Set \"execute\" to execute the indexing plan at the given instant, which means --instant-time is required here; " - + "Set \"scheduleAndExecute\" to generate an indexing plan first and execute that plan immediately") + + "Set \"scheduleandExecute\" to generate an indexing plan first and execute that plan immediately;" + + "Set \"dropindex\" to drop the index types specified in --index-types;") public String runningMode = null; @Parameter(names = {"--help", "-h"}, help = true) public Boolean help = false; - @Parameter(names = {"--props"}, description = "path to properties file on localfs or dfs, with configurations for " - + "hoodie client for compacting") + @Parameter(names = {"--props"}, description = "path to properties file on localfs or dfs, with configurations for hoodie client for indexing") public String propsFilePath = null; @Parameter(names = {"--hoodie-conf"}, description = "Any configuration that can be set in the properties file " @@ -149,6 +151,10 @@ private int start(int retry) { LOG.info("Running Mode: [" + EXECUTE + "];"); return runIndexing(jsc); } + case DROP_INDEX: { + LOG.info("Running Mode: [" + DROP_INDEX + "];"); + return dropIndex(jsc); + } default: { LOG.info("Unsupported running mode [" + cfg.runningMode + "], quit the job directly"); return -1; @@ -193,7 +199,7 @@ private int runIndexing(JavaSparkContext jsc) throws Exception { throw new HoodieIndexException("There is no scheduled indexing in the table."); } } - return handleError(client.index(cfg.indexInstantTime)); + return handleResponse(client.index(cfg.indexInstantTime)) ? 0 : 1; } } @@ -202,21 +208,35 @@ private int scheduleAndRunIndexing(JavaSparkContext jsc) throws Exception { try (SparkRDDWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props)) { Option indexingInstantTime = doSchedule(client); if (indexingInstantTime.isPresent()) { - return handleError(client.index(indexingInstantTime.get())); + return handleResponse(client.index(indexingInstantTime.get())) ? 0 : 1; } else { return -1; } } } - private int handleError(Option commitMetadata) { + private int dropIndex(JavaSparkContext jsc) throws Exception { + List partitionsToDrop = Arrays.asList(cfg.indexTypes.split(",")); + List partitionTypes = partitionsToDrop.stream() + .map(MetadataPartitionType::valueOf).collect(Collectors.toList()); + String schemaStr = UtilHelpers.getSchemaFromLatestInstant(metaClient); + try (SparkRDDWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props)) { + client.dropIndex(partitionTypes); + return 0; + } catch (Exception e) { + LOG.error("Failed to drop index. ", e); + return -1; + } + } + + private boolean handleResponse(Option commitMetadata) { if (!commitMetadata.isPresent()) { LOG.error("Indexing failed as no commit metadata present."); - return -1; + return false; } List indexPartitionInfos = commitMetadata.get().getIndexPartitionInfos(); LOG.info(String.format("Indexing complete for partitions: %s", indexPartitionInfos.stream().map(HoodieIndexPartitionInfo::getMetadataPartitionPath).collect(Collectors.toList()))); - return 0; + return true; } } From 3b85bb0ff61cda00b022de78c06d015d3b9a3a1c Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Thu, 24 Mar 2022 17:10:54 +0530 Subject: [PATCH 08/21] Minor fix for empty partition path --- .../org/apache/hudi/io/HoodieAppendHandle.java | 3 ++- .../HoodieBackedTableMetadataWriter.java | 18 ++++++++++++------ .../index/ScheduleIndexActionExecutor.java | 15 ++++++++++++--- 3 files changed, 26 insertions(+), 10 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java index 840a77667fbb1..890c9ace134bd 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java @@ -345,7 +345,8 @@ private void processAppendResult(AppendResult result, List record } if (config.isMetadataColumnStatsIndexEnabled()) { - List columnsToIndex = Stream.of(config.getColumnsEnabledForColumnStatsIndex().split(",")).map(String::trim).collect(Collectors.toList()); + List columnsToIndex = Stream.of(config.getColumnsEnabledForColumnStatsIndex().split(",")) + .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList()); Map> columnRangeMap = stat.getRecordsStats().isPresent() ? stat.getRecordsStats().get().getStats() : new HashMap<>(); final String filePath = stat.getPath(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index 8f2452dd47fb6..992c455fdb51d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -664,8 +664,10 @@ public void initializeFileGroups(HoodieTableMetaClient dataMetaClient, MetadataP } public void dropIndex(List indexesToDrop) throws IOException { - Set completedIndexes = Stream.of(dataMetaClient.getTableConfig().getCompletedMetadataIndexes().split(",")).collect(Collectors.toSet()); - Set inflightIndexes = Stream.of(dataMetaClient.getTableConfig().getInflightMetadataIndexes().split(",")).collect(Collectors.toSet()); + Set completedIndexes = Stream.of(dataMetaClient.getTableConfig().getCompletedMetadataIndexes().split(",")) + .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet()); + Set inflightIndexes = Stream.of(dataMetaClient.getTableConfig().getInflightMetadataIndexes().split(",")) + .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet()); for (MetadataPartitionType partitionType : indexesToDrop) { String partitionPath = partitionType.getPartitionPath(); if (inflightIndexes.contains(partitionPath)) { @@ -687,8 +689,10 @@ private MetadataRecordsGenerationParams getRecordsGenerationParams() { dataWriteConfig.getBloomIndexParallelism(), dataWriteConfig.isMetadataColumnStatsIndexEnabled(), dataWriteConfig.getColumnStatsIndexParallelism(), - Stream.of(dataWriteConfig.getColumnsEnabledForColumnStatsIndex().split(",")).map(String::trim).collect(Collectors.toList()), - Stream.of(dataWriteConfig.getColumnsEnabledForBloomFilterIndex().split(",")).map(String::trim).collect(Collectors.toList())); + Stream.of(dataWriteConfig.getColumnsEnabledForColumnStatsIndex().split(",")) + .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList()), + Stream.of(dataWriteConfig.getColumnsEnabledForBloomFilterIndex().split(",")) + .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList())); } /** @@ -727,8 +731,10 @@ private void processAndCommit(String instantTime, ConvertMetadataFunction co private Set getMetadataPartitionsToUpdate() { // fetch partitions to update from table config - Set partitionsToUpdate = Stream.of(dataMetaClient.getTableConfig().getCompletedMetadataIndexes().split(",")).collect(Collectors.toSet()); - partitionsToUpdate.addAll(Stream.of(dataMetaClient.getTableConfig().getInflightMetadataIndexes().split(",")).collect(Collectors.toSet())); + Set partitionsToUpdate = Stream.of(dataMetaClient.getTableConfig().getCompletedMetadataIndexes().split(",")) + .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet()); + partitionsToUpdate.addAll(Stream.of(dataMetaClient.getTableConfig().getInflightMetadataIndexes().split(",")) + .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet())); if (!partitionsToUpdate.isEmpty()) { return partitionsToUpdate; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java index 6948ffeb5368a..e70dc26bb4a70 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java @@ -43,7 +43,9 @@ import java.io.IOException; import java.util.EnumSet; import java.util.List; +import java.util.Set; import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.apache.hudi.common.model.WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL; import static org.apache.hudi.config.HoodieWriteConfig.WRITE_CONCURRENCY_MODE; @@ -87,9 +89,16 @@ public Option execute() { WRITE_CONCURRENCY_MODE.key(), OPTIMISTIC_CONCURRENCY_CONTROL.name())); } // make sure that it is idempotent, check with previously pending index operations. - String[] indexesInflight = table.getMetaClient().getTableConfig().getInflightMetadataIndexes().split(","); - String[] indexesCompleted = table.getMetaClient().getTableConfig().getCompletedMetadataIndexes().split(","); - + Set indexesInflightOrCompleted = Stream.of(table.getMetaClient().getTableConfig().getInflightMetadataIndexes().split(",")) + .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet()); + indexesInflightOrCompleted.addAll(Stream.of(table.getMetaClient().getTableConfig().getInflightMetadataIndexes().split(",")) + .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet())); + Set requestedPartitions = partitionsToIndex.stream().map(MetadataPartitionType::getPartitionPath).collect(Collectors.toSet()); + requestedPartitions.retainAll(indexesInflightOrCompleted); + if (!requestedPartitions.isEmpty()) { + LOG.error("Following partitions already exist or inflight: " + requestedPartitions); + return Option.empty(); + } // get last completed instant Option indexUptoInstant = table.getActiveTimeline().getContiguousCompletedWriteTimeline().lastInstant(); if (indexUptoInstant.isPresent()) { From 3e37433e72957326fc0b8d1c7cd92d85298e120a Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Thu, 24 Mar 2022 19:21:16 +0530 Subject: [PATCH 09/21] Minor fix for no columns configured --- .../apache/hudi/io/HoodieAppendHandle.java | 22 ++++++++++++++----- .../metadata/HoodieTableMetadataUtil.java | 6 ++--- 2 files changed, 20 insertions(+), 8 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java index 890c9ace134bd..f4aa7349817a3 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java @@ -50,12 +50,14 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.SizeEstimator; +import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieAppendException; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieUpsertException; import org.apache.hudi.table.HoodieTable; +import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; import org.apache.hadoop.fs.Path; @@ -69,6 +71,7 @@ import java.util.List; import java.util.Map; import java.util.Properties; +import java.util.Set; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -345,17 +348,26 @@ private void processAppendResult(AppendResult result, List record } if (config.isMetadataColumnStatsIndexEnabled()) { - List columnsToIndex = Stream.of(config.getColumnsEnabledForColumnStatsIndex().split(",")) - .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList()); + final List fieldsToIndex; + if (!StringUtils.isNullOrEmpty(config.getColumnsEnabledForColumnStatsIndex())) { + Set columnsToIndex = Stream.of(config.getColumnsEnabledForColumnStatsIndex().split(",")) + .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet()); + fieldsToIndex = writeSchemaWithMetaFields.getFields().stream() + .filter(field -> columnsToIndex.contains(field.name())).collect(Collectors.toList()); + } else { + // if column stats index is enabled but columns not configured then we assume that all columns should be indexed + fieldsToIndex = writeSchemaWithMetaFields.getFields(); + } + Map> columnRangeMap = stat.getRecordsStats().isPresent() ? stat.getRecordsStats().get().getStats() : new HashMap<>(); final String filePath = stat.getPath(); // initialize map of column name to map of stats name to stats value Map> columnToStats = new HashMap<>(); - columnsToIndex.forEach(column -> columnToStats.putIfAbsent(column, new HashMap<>())); + fieldsToIndex.forEach(field -> columnToStats.putIfAbsent(field.name(), new HashMap<>())); // collect stats for columns at once per record and keep iterating through every record to eventually find col stats for all fields. - recordList.forEach(record -> aggregateColumnStats(record, writeSchemaWithMetaFields, columnToStats, config.isConsistentLogicalTimestampEnabled())); - writeSchemaWithMetaFields.getFields().forEach(field -> accumulateColumnRanges(field, filePath, columnRangeMap, columnToStats)); + recordList.forEach(record -> aggregateColumnStats(record, fieldsToIndex, columnToStats, config.isConsistentLogicalTimestampEnabled())); + fieldsToIndex.forEach(field -> accumulateColumnRanges(field, filePath, columnRangeMap, columnToStats)); stat.setRecordsStats(new HoodieDeltaWriteStat.RecordsStats<>(columnRangeMap)); } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java index 5f1e73ba584da..9946fe3dfb6f7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java @@ -1073,18 +1073,18 @@ public static void accumulateColumnRanges(Schema.Field field, String filePath, * Aggregates column stats for each field. * * @param record - current record - * @param schema - write schema + * @param fields - fields for which stats will be aggregated * @param columnToStats - map of column to map of each stat and its value which gets updates in this method * @param consistentLogicalTimestampEnabled - flag to deal with logical timestamp type when getting column value */ - public static void aggregateColumnStats(IndexedRecord record, Schema schema, + public static void aggregateColumnStats(IndexedRecord record, List fields, Map> columnToStats, boolean consistentLogicalTimestampEnabled) { if (!(record instanceof GenericRecord)) { throw new HoodieIOException("Record is not a generic type to get column range metadata!"); } - schema.getFields().forEach(field -> { + fields.forEach(field -> { Map columnStats = columnToStats.getOrDefault(field.name(), new HashMap<>()); final String fieldVal = getNestedFieldValAsString((GenericRecord) record, field.name(), true, consistentLogicalTimestampEnabled); // update stats From 808934d1ad29ef04337cc0e3a203dbc55c01f62f Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Thu, 24 Mar 2022 20:05:33 +0530 Subject: [PATCH 10/21] Fix active timeline test --- .../hudi/common/table/timeline/TestHoodieActiveTimeline.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java b/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java index 543a712e635ae..0b591adcb8707 100755 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java @@ -221,7 +221,7 @@ public void testTimelineGetOperations() { checkTimeline.accept(timeline.getCommitsTimeline(), CollectionUtils.createSet( HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION)); checkTimeline.accept(timeline.getWriteTimeline(), CollectionUtils.createSet( - HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.COMPACTION_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION, HoodieTimeline.INDEX_ACTION)); + HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.COMPACTION_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION)); checkTimeline.accept(timeline.getCommitTimeline(), CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION)); checkTimeline.accept(timeline.getDeltaCommitTimeline(), Collections.singleton(HoodieTimeline.DELTA_COMMIT_ACTION)); checkTimeline.accept(timeline.getCleanerTimeline(), Collections.singleton(HoodieTimeline.CLEAN_ACTION)); From 2b4871b4dc0c4ee04d0bf68225f633b48cbbd4f0 Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Fri, 25 Mar 2022 07:57:06 +0530 Subject: [PATCH 11/21] Take lock before writing index completed to data timeline --- .../metadata/HoodieBackedTableMetadataWriter.java | 9 ++------- .../table/action/index/RunIndexActionExecutor.java | 13 +++++++++++-- .../org/apache/hudi/utilities/HoodieIndexer.java | 9 ++++++--- 3 files changed, 19 insertions(+), 12 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index 992c455fdb51d..0f2a246ac2ac5 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -718,11 +718,6 @@ private void processAndCommit(String instantTime, ConvertMetadataFunction co Set partitionsToUpdate = getMetadataPartitionsToUpdate(); partitionsToUpdate.forEach(p -> { if (enabled && metadata != null) { - try { - initializeFileGroups(dataMetaClient, MetadataPartitionType.valueOf(p.toUpperCase(Locale.ROOT)), instantTime, 1); - } catch (IOException e) { - throw new HoodieIndexException(String.format("Unable to initialize file groups for metadata partition: %s, instant: %s", p, instantTime)); - } Map> partitionRecordsMap = convertMetadataFunction.convertMetadata(); commit(instantTime, partitionRecordsMap, canTriggerTableService); } @@ -765,8 +760,8 @@ public void index(HoodieEngineContext engineContext, List execute() { info.getMetadataPartitionPath(), currentIndexedInstant)) .collect(Collectors.toList()); - return Option.of(HoodieIndexCommitMetadata.newBuilder() - .setVersion(LATEST_INDEX_COMMIT_METADATA_VERSION).setIndexPartitionInfos(finalIndexPartitionInfos).build()); + HoodieIndexCommitMetadata indexCommitMetadata = HoodieIndexCommitMetadata.newBuilder() + .setVersion(LATEST_INDEX_COMMIT_METADATA_VERSION).setIndexPartitionInfos(finalIndexPartitionInfos).build(); + try { + txnManager.beginTransaction(); + table.getActiveTimeline().saveAsComplete( + new HoodieInstant(true, HoodieTimeline.RESTORE_ACTION, indexInstant.getTimestamp()), + TimelineMetadataUtils.serializeIndexCommitMetadata(indexCommitMetadata)); + } finally { + txnManager.endTransaction(); + } + return Option.of(indexCommitMetadata); } catch (IOException e) { throw new HoodieIndexException(String.format("Unable to index instant: %s", indexInstant)); } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java index fb674f14ff0bf..fbdb57daf3eb8 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java @@ -41,6 +41,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Locale; import java.util.stream.Collectors; import static org.apache.hudi.common.table.timeline.HoodieInstant.State.INFLIGHT; @@ -90,7 +91,7 @@ public static class Config implements Serializable { public String sparkMemory = null; @Parameter(names = {"--retry", "-rt"}, description = "number of retries") public int retry = 0; - @Parameter(names = {"--index-types", "-it"}, description = "Comma-separated index types to be built, e.g. BLOOM_FILTERS,COLUMN_STATS", required = true) + @Parameter(names = {"--index-types", "-ixt"}, description = "Comma-separated index types to be built, e.g. BLOOM_FILTERS,COLUMN_STATS", required = true) public String indexTypes = null; @Parameter(names = {"--mode", "-m"}, description = "Set job mode: Set \"schedule\" to generate an indexing plan; " + "Set \"execute\" to execute the indexing plan at the given instant, which means --instant-time is required here; " @@ -173,7 +174,8 @@ private Option scheduleIndexing(JavaSparkContext jsc) throws Exception { private Option doSchedule(SparkRDDWriteClient client) { List partitionsToIndex = Arrays.asList(cfg.indexTypes.split(",")); List partitionTypes = partitionsToIndex.stream() - .map(MetadataPartitionType::valueOf).collect(Collectors.toList()); + .map(p -> MetadataPartitionType.valueOf(p.toUpperCase(Locale.ROOT))) + .collect(Collectors.toList()); Option indexingInstant = client.scheduleIndexing(partitionTypes); if (!indexingInstant.isPresent()) { LOG.error("Scheduling of index action did not return any instant."); @@ -218,7 +220,8 @@ private int scheduleAndRunIndexing(JavaSparkContext jsc) throws Exception { private int dropIndex(JavaSparkContext jsc) throws Exception { List partitionsToDrop = Arrays.asList(cfg.indexTypes.split(",")); List partitionTypes = partitionsToDrop.stream() - .map(MetadataPartitionType::valueOf).collect(Collectors.toList()); + .map(p -> MetadataPartitionType.valueOf(p.toUpperCase(Locale.ROOT))) + .collect(Collectors.toList()); String schemaStr = UtilHelpers.getSchemaFromLatestInstant(metaClient); try (SparkRDDWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props)) { client.dropIndex(partitionTypes); From 80fee236111c0df99eb9212805f9695c8c454479 Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Fri, 25 Mar 2022 17:09:12 +0530 Subject: [PATCH 12/21] Add test for indexer with continuous deltastreamer checkstyle fix --- .../HoodieBackedTableMetadataWriter.java | 15 ++++- .../metadata/HoodieTableMetadataWriter.java | 25 ++++---- .../action/index/RunIndexActionExecutor.java | 5 +- .../index/ScheduleIndexActionExecutor.java | 4 +- .../table/timeline/HoodieDefaultTimeline.java | 4 +- .../table/view/HoodieTableFileSystemView.java | 17 ++++- .../metadata/HoodieTableMetadataUtil.java | 22 ++++++- .../apache/hudi/utilities/HoodieIndexer.java | 33 +++++++++- .../HoodieDeltaStreamerTestBase.java | 1 + .../functional/TestHoodieDeltaStreamer.java | 64 +++++++++++++++++++ .../delta-streamer-config/indexer.properties | 24 +++++++ 11 files changed, 191 insertions(+), 23 deletions(-) create mode 100644 hudi-utilities/src/test/resources/delta-streamer-config/indexer.properties diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index 0f2a246ac2ac5..43bada531dcc3 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -614,6 +614,12 @@ private void initializeEnabledFileGroups(HoodieTableMetaClient dataMetaClient, S } } + public void scheduleIndex(HoodieTableMetaClient dataMetaClient, List metadataPartitions, String instantTime) throws IOException { + for (MetadataPartitionType partitionType : metadataPartitions) { + initializeFileGroups(dataMetaClient, partitionType, instantTime, partitionType.getFileGroupCount()); + } + } + /** * Initialize file groups for a partition. For file listing, we just have one file group. * @@ -770,6 +776,7 @@ public void index(HoodieEngineContext engineContext, List prepRecords(Map fileSlices = HoodieTableMetadataUtil.getPartitionLatestFileSlices(metadataMetaClient, Option.ofNullable(fsView), partitionName); + if (fileSlices.isEmpty()) { + // scheduling of INDEX only initializes the file group and not add commit + // so if there are no committed file slices, look for inflight slices + fileSlices = HoodieTableMetadataUtil.getPartitionLatestFileSlicesIncludingInflight(metadataMetaClient, Option.ofNullable(fsView), partitionName); + } ValidationUtils.checkArgument(fileSlices.size() == fileGroupCount, String.format("Invalid number of file groups for partition:%s, found=%d, required=%d", partitionName, fileSlices.size(), fileGroupCount)); + List finalFileSlices = fileSlices; HoodieData rddSinglePartitionRecords = records.map(r -> { - FileSlice slice = fileSlices.get(HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex(r.getRecordKey(), + FileSlice slice = finalFileSlices.get(HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex(r.getRecordKey(), fileGroupCount)); r.setCurrentLocation(new HoodieRecordLocation(slice.getBaseInstantTime(), slice.getFileId())); return r; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java index 6e066880746ea..4741726b077d0 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java @@ -46,13 +46,12 @@ public interface HoodieTableMetadataWriter extends Serializable, AutoCloseable { /** * Initialize file groups while scheduling index action. * - * @param dataMetaClient - meta client for the data table - * @param metadataPartition - metadata partition for which file groups needs to be initialized - * @param instantTime - instant time of the index action - * @param fileGroupCount - number of file groups to be initialized + * @param dataMetaClient - meta client for the data table + * @param metadataPartitions - metadata partitions for which file groups needs to be initialized + * @param instantTime - instant time of the index action * @throws IOException */ - void initializeFileGroups(HoodieTableMetaClient dataMetaClient, MetadataPartitionType metadataPartition, String instantTime, int fileGroupCount) throws IOException; + void scheduleIndex(HoodieTableMetaClient dataMetaClient, List metadataPartitions, String instantTime) throws IOException; /** * Drop the given metadata indexes. @@ -64,31 +63,35 @@ public interface HoodieTableMetadataWriter extends Serializable, AutoCloseable { /** * Update the metadata table due to a COMMIT operation. - * @param commitMetadata commit metadata of the operation of interest. - * @param instantTime instant time of the commit. + * + * @param commitMetadata commit metadata of the operation of interest. + * @param instantTime instant time of the commit. * @param isTableServiceAction true if caller is a table service. false otherwise. Only regular write operations can trigger metadata table services and this argument - * will assist in this. + * will assist in this. */ void update(HoodieCommitMetadata commitMetadata, String instantTime, boolean isTableServiceAction); /** * Update the metadata table due to a CLEAN operation. + * * @param cleanMetadata clean metadata of the operation of interest. - * @param instantTime instant time of the commit. + * @param instantTime instant time of the commit. */ void update(HoodieCleanMetadata cleanMetadata, String instantTime); /** * Update the metadata table due to a RESTORE operation. + * * @param restoreMetadata restore metadata of the operation of interest. - * @param instantTime instant time of the commit. + * @param instantTime instant time of the commit. */ void update(HoodieRestoreMetadata restoreMetadata, String instantTime); /** * Update the metadata table due to a ROLLBACK operation. + * * @param rollbackMetadata rollback metadata of the operation of interest. - * @param instantTime instant time of the commit. + * @param instantTime instant time of the commit. */ void update(HoodieRollbackMetadata rollbackMetadata, String instantTime); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java index eaea87c519885..2e034a533609f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java @@ -61,6 +61,7 @@ import static org.apache.hudi.common.model.WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL; import static org.apache.hudi.common.table.timeline.HoodieInstant.State.COMPLETED; import static org.apache.hudi.common.table.timeline.HoodieInstant.State.REQUESTED; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.INDEX_ACTION; import static org.apache.hudi.config.HoodieWriteConfig.WRITE_CONCURRENCY_MODE; /** @@ -140,7 +141,7 @@ public Option execute() { } finally { executorService.shutdownNow(); } - // build index commit metadata and return + // save index commit metadata and return List finalIndexPartitionInfos = indexPartitionInfos.stream() .map(info -> new HoodieIndexPartitionInfo( info.getVersion(), @@ -152,7 +153,7 @@ public Option execute() { try { txnManager.beginTransaction(); table.getActiveTimeline().saveAsComplete( - new HoodieInstant(true, HoodieTimeline.RESTORE_ACTION, indexInstant.getTimestamp()), + new HoodieInstant(true, INDEX_ACTION, indexInstant.getTimestamp()), TimelineMetadataUtils.serializeIndexCommitMetadata(indexCommitMetadata)); } finally { txnManager.endTransaction(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java index e70dc26bb4a70..5afc027e75289 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java @@ -124,9 +124,7 @@ public Option execute() { try { this.txnManager.beginTransaction(Option.of(indexInstant), Option.empty()); // 3. initialize filegroups as per plan for the enabled partition types - for (MetadataPartitionType partitionType : partitionsToIndex) { - metadataWriter.initializeFileGroups(table.getMetaClient(), partitionType, indexInstant.getTimestamp(), partitionType.getFileGroupCount()); - } + metadataWriter.scheduleIndex(table.getMetaClient(), partitionsToIndex, indexInstant.getTimestamp()); } catch (IOException e) { LOG.error("Could not initialize file groups"); throw new HoodieIOException(e.getMessage(), e); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java index 01210082b41f0..ef0cf113f7ae4 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java @@ -209,7 +209,7 @@ public HoodieTimeline filterPendingIndexTimeline() { @Override public HoodieTimeline filterCompletedIndexTimeline() { - return new HoodieDefaultTimeline(instants.stream().filter(s -> s.getAction().equals(INDEX_ACTION)).filter(HoodieInstant::isCompleted), details); + return new HoodieDefaultTimeline(instants.stream().filter(s -> s.getAction().equals(INDEX_ACTION) && s.isCompleted()), details); } /** @@ -220,7 +220,7 @@ public HoodieTimeline getCommitsTimeline() { } /** - * Get all instants (commits, delta commits, compaction, clean, savepoint, rollback) that result in actions, + * Get all instants (commits, delta commits, compaction, clean, savepoint, rollback, replace commits, index) that result in actions, * in the active timeline. */ public HoodieTimeline getAllCommitsTimeline() { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java index 299dbab10c368..9dac36081384b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java @@ -18,9 +18,9 @@ package org.apache.hudi.common.table.view; -import org.apache.hadoop.fs.FileStatus; import org.apache.hudi.common.model.BootstrapBaseFileMapping; import org.apache.hudi.common.model.CompactionOperation; +import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -29,6 +29,8 @@ 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.hadoop.fs.FileStatus; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -358,6 +360,19 @@ protected Option getReplaceInstant(final HoodieFileGroupId fileGr return Option.ofNullable(fgIdToReplaceInstants.get(fileGroupId)); } + /** + * Get the latest file slices for a given partition including the inflight ones. + * + * @param partitionPath + * @return Stream of latest {@link FileSlice} in the partition path. + */ + public Stream fetchLatestFileSlicesIncludingInflight(String partitionPath) { + return fetchAllStoredFileGroups(partitionPath) + .map(HoodieFileGroup::getLatestFileSlicesIncludingInflight) + .filter(Option::isPresent) + .map(Option::get); + } + @Override public void close() { super.close(); diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java index 9946fe3dfb6f7..76b23ca244bc2 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java @@ -896,6 +896,24 @@ private static List getPartitionFileSlices(HoodieTableMetaClient meta return fileSliceStream.sorted(Comparator.comparing(FileSlice::getFileId)).collect(Collectors.toList()); } + /** + * Get the latest file slices for a given partition including the inflight ones. + * + * @param metaClient - instance of {@link HoodieTableMetaClient} + * @param fileSystemView - hoodie table file system view, which will be fetched from meta client if not already present + * @param partition - name of the partition whose file groups are to be loaded + * @return + */ + public static List getPartitionLatestFileSlicesIncludingInflight(HoodieTableMetaClient metaClient, + Option fileSystemView, + String partition) { + HoodieTableFileSystemView fsView = fileSystemView.orElse(getFileSystemView(metaClient)); + Stream fileSliceStream = fsView.fetchLatestFileSlicesIncludingInflight(partition); + return fileSliceStream + .sorted(Comparator.comparing(FileSlice::getFileId)) + .collect(Collectors.toList()); + } + public static HoodieData convertMetadataToColumnStatsRecords(HoodieCommitMetadata commitMetadata, HoodieEngineContext engineContext, MetadataRecordsGenerationParams recordsGenerationParams) { @@ -911,8 +929,8 @@ public static HoodieData convertMetadataToColumnStatsRecords(Hoodi Option.ofNullable(commitMetadata.getMetadata(HoodieCommitMetadata.SCHEMA_KEY)) .flatMap(writerSchemaStr -> isNullOrEmpty(writerSchemaStr) - ? Option.empty() - : Option.of(new Schema.Parser().parse(writerSchemaStr))); + ? Option.empty() + : Option.of(new Schema.Parser().parse(writerSchemaStr))); HoodieTableMetaClient dataTableMetaClient = recordsGenerationParams.getDataMetaClient(); HoodieTableConfig tableConfig = dataTableMetaClient.getTableConfig(); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java index fbdb57daf3eb8..c3a500ba38b06 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java @@ -36,6 +36,7 @@ import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaSparkContext; +import org.jetbrains.annotations.TestOnly; import java.io.Serializable; import java.util.ArrayList; @@ -51,6 +52,27 @@ /** * A tool to run metadata indexing asynchronously. + *

    + * Example command (assuming indexer.properties contains related index configs, see {@link org.apache.hudi.common.config.HoodieMetadataConfig} for configs): + *

    + * spark-submit \ + * --class org.apache.hudi.utilities.HoodieIndexer \ + * /path/to/hudi/packaging/hudi-utilities-bundle/target/hudi-utilities-bundle_2.11-0.11.0-SNAPSHOT.jar \ + * --props /path/to/indexer.properties \ + * --mode scheduleAndExecute \ + * --base-path /tmp/hudi_trips_cow \ + * --table-name hudi_trips_cow \ + * --index-types COLUMN_STATS \ + * --parallelism 1 \ + * --spark-memory 1g + *

    + * A sample indexer.properties file: + *

    + * hoodie.metadata.index.async=true + * hoodie.metadata.index.column.stats.enable=true + * hoodie.metadata.index.check.timeout.seconds=60 + * hoodie.write.concurrency.mode=optimistic_concurrency_control + * hoodie.write.lock.provider=org.apache.hudi.client.transaction.lock.ZookeeperBasedLockProvider */ public class HoodieIndexer { @@ -132,7 +154,7 @@ public static void main(String[] args) { jsc.stop(); } - private int start(int retry) { + public int start(int retry) { return UtilHelpers.retry(retry, () -> { switch (cfg.runningMode.toLowerCase()) { case SCHEDULE: { @@ -164,6 +186,11 @@ private int start(int retry) { }, "Indexer failed"); } + @TestOnly + public Option doSchedule() throws Exception { + return this.scheduleIndexing(jsc); + } + private Option scheduleIndexing(JavaSparkContext jsc) throws Exception { String schemaStr = UtilHelpers.getSchemaFromLatestInstant(metaClient); try (SparkRDDWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props)) { @@ -176,6 +203,10 @@ private Option doSchedule(SparkRDDWriteClient clien List partitionTypes = partitionsToIndex.stream() .map(p -> MetadataPartitionType.valueOf(p.toUpperCase(Locale.ROOT))) .collect(Collectors.toList()); + if (cfg.indexInstantTime != null) { + client.scheduleClusteringAtInstant(cfg.indexInstantTime, Option.empty()); + return Option.of(cfg.indexInstantTime); + } Option indexingInstant = client.scheduleIndexing(partitionTypes); if (!indexingInstant.isPresent()) { LOG.error("Scheduling of index action did not return any instant."); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/HoodieDeltaStreamerTestBase.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/HoodieDeltaStreamerTestBase.java index 304c25b1d4ac3..a9de85ce5ac9e 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/HoodieDeltaStreamerTestBase.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/HoodieDeltaStreamerTestBase.java @@ -129,6 +129,7 @@ protected static void prepareInitialConfigs(FileSystem dfs, String dfsBasePath, UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/uber_config.properties", dfs, dfsBasePath + "/config/uber_config.properties"); UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/short_trip_uber_config.properties", dfs, dfsBasePath + "/config/short_trip_uber_config.properties"); UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/clusteringjob.properties", dfs, dfsBasePath + "/clusteringjob.properties"); + UtilitiesTestBase.Helpers.copyToDFS("delta-streamer-config/indexer.properties", dfs, dfsBasePath + "/indexer.properties"); writeCommonPropsToFile(dfs, dfsBasePath); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java index 066e81849056b..e76567812958a 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java @@ -57,6 +57,7 @@ import org.apache.hudi.keygen.SimpleKeyGenerator; import org.apache.hudi.utilities.DummySchemaProvider; import org.apache.hudi.utilities.HoodieClusteringJob; +import org.apache.hudi.utilities.HoodieIndexer; import org.apache.hudi.utilities.deltastreamer.DeltaSync; import org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer; import org.apache.hudi.utilities.schema.FilebasedSchemaProvider; @@ -400,6 +401,22 @@ static void assertAtLeastNReplaceCommits(int minExpected, String tablePath, File assertTrue(minExpected <= numDeltaCommits, "Got=" + numDeltaCommits + ", exp >=" + minExpected); } + static void assertPendingIndexCommit(String tablePath, FileSystem fs) { + HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tablePath).setLoadActiveTimelineOnLoad(true).build(); + HoodieTimeline timeline = meta.getActiveTimeline().getAllCommitsTimeline().filterPendingIndexTimeline(); + LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants().collect(Collectors.toList())); + int numIndexCommits = (int) timeline.getInstants().count(); + assertEquals(1, numIndexCommits, "Got=" + numIndexCommits + ", exp=1"); + } + + static void assertCompletedIndexCommit(String tablePath, FileSystem fs) { + HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tablePath).setLoadActiveTimelineOnLoad(true).build(); + HoodieTimeline timeline = meta.getActiveTimeline().getAllCommitsTimeline().filterCompletedIndexTimeline(); + LOG.info("Timeline Instants=" + meta.getActiveTimeline().getInstants().collect(Collectors.toList())); + int numIndexCommits = (int) timeline.getInstants().count(); + assertEquals(1, numIndexCommits, "Got=" + numIndexCommits + ", exp=1"); + } + static void assertNoReplaceCommits(String tablePath, FileSystem fs) { HoodieTableMetaClient meta = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(tablePath).setLoadActiveTimelineOnLoad(true).build(); HoodieTimeline timeline = meta.getActiveTimeline().getCompletedReplaceTimeline(); @@ -964,6 +981,53 @@ private HoodieClusteringJob.Config buildHoodieClusteringUtilConfig(String basePa return config; } + private HoodieIndexer.Config buildIndexerConfig(String basePath, + String tableName, + String indexInstantTime, + String runningMode, + String indexTypes) { + HoodieIndexer.Config config = new HoodieIndexer.Config(); + config.basePath = basePath; + config.tableName = tableName; + config.indexInstantTime = indexInstantTime; + config.propsFilePath = dfsBasePath + "/indexer.properties"; + config.runningMode = runningMode; + config.indexTypes = indexTypes; + return config; + } + + @Test + public void testHoodieIndexer() throws Exception { + String tableBasePath = dfsBasePath + "/asyncindexer"; + HoodieDeltaStreamer ds = initialHoodieDeltaStreamer(tableBasePath, 1000, "false"); + + deltaStreamerTestRunner(ds, (r) -> { + TestHelpers.assertAtLeastNCommits(2, tableBasePath, dfs); + + Option scheduleIndexInstantTime = Option.empty(); + try { + HoodieIndexer scheduleIndexingJob = new HoodieIndexer(jsc, + buildIndexerConfig(tableBasePath, ds.getConfig().targetTableName, null, SCHEDULE, "COLUMN_STATS")); + scheduleIndexInstantTime = scheduleIndexingJob.doSchedule(); + } catch (Exception e) { + LOG.info("Schedule clustering failed", e); + return false; + } + if (scheduleIndexInstantTime.isPresent()) { + TestHelpers.assertPendingIndexCommit(tableBasePath, dfs); + LOG.info("Schedule clustering success, now cluster with instant time " + scheduleIndexInstantTime.get()); + HoodieIndexer runIndexingJob = new HoodieIndexer(jsc, + buildIndexerConfig(tableBasePath, ds.getConfig().targetTableName, scheduleIndexInstantTime.get(), EXECUTE, "COLUMN_STATS")); + runIndexingJob.start(0); + LOG.info("Cluster success"); + } else { + LOG.warn("Schedule clustering failed"); + } + TestHelpers.assertCompletedIndexCommit(tableBasePath, dfs); + return true; + }); + } + @Disabled("HUDI-3710 to fix the ConcurrentModificationException") @ParameterizedTest @ValueSource(booleans = {true, false}) diff --git a/hudi-utilities/src/test/resources/delta-streamer-config/indexer.properties b/hudi-utilities/src/test/resources/delta-streamer-config/indexer.properties new file mode 100644 index 0000000000000..08a1e0f109ddb --- /dev/null +++ b/hudi-utilities/src/test/resources/delta-streamer-config/indexer.properties @@ -0,0 +1,24 @@ +# +# 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. +# + +hoodie.metadata.index.async=true +hoodie.metadata.index.column.stats.enable=true +hoodie.metadata.index.check.timeout.seconds=60 +hoodie.write.concurrency.mode=optimistic_concurrency_control +hoodie.write.lock.provider=org.apache.hudi.client.transaction.lock.InProcessLockProvider \ No newline at end of file From 6d6178c749a6d0a120d311feae30fa05ecb40b42 Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Mon, 28 Mar 2022 16:47:32 +0530 Subject: [PATCH 13/21] Address feedback from second review --- .../hudi/client/BaseHoodieWriteClient.java | 21 ++- .../apache/hudi/config/HoodieWriteConfig.java | 4 +- .../HoodieBackedTableMetadataWriter.java | 73 +++++---- .../metadata/HoodieTableMetadataWriter.java | 2 +- .../action/index/RunIndexActionExecutor.java | 146 +++++++++++------- .../index/ScheduleIndexActionExecutor.java | 44 +++--- .../table/HoodieFlinkCopyOnWriteTable.java | 4 +- .../main/avro/HoodieIndexCommitMetadata.avsc | 3 + .../apache/hudi/common/bloom/BloomFilter.java | 11 -- .../HoodieDynamicBoundedBloomFilter.java | 10 -- .../hudi/common/bloom/SimpleBloomFilter.java | 10 -- .../common/config/HoodieMetadataConfig.java | 6 +- .../hudi/common/table/HoodieTableConfig.java | 6 +- .../table/timeline/HoodieDefaultTimeline.java | 22 +-- .../metadata/HoodieTableMetadataUtil.java | 2 - .../timeline/TestHoodieActiveTimeline.java | 40 +++++ .../apache/hudi/utilities/HoodieIndexer.java | 48 ++++-- .../hudi/utilities/TestHoodieIndexer.java | 122 +++++++++++++++ .../functional/TestHoodieDeltaStreamer.java | 10 +- .../delta-streamer-config/indexer.properties | 1 + 20 files changed, 386 insertions(+), 199 deletions(-) create mode 100644 hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieIndexer.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java index c84021daca23f..a7aa74da25cbd 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java @@ -64,6 +64,7 @@ import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.exception.HoodieIndexException; import org.apache.hudi.exception.HoodieRestoreException; import org.apache.hudi.exception.HoodieRollbackException; import org.apache.hudi.exception.HoodieSavepointException; @@ -927,6 +928,13 @@ public boolean scheduleCompactionAtInstant(String instantTime, Option scheduleIndexing(List partitionTypes) { String instantTime = HoodieActiveTimeline.createNewInstantTime(); return scheduleIndexingAtInstant(partitionTypes, instantTime) ? Option.of(instantTime) : Option.empty(); @@ -938,10 +946,21 @@ private boolean scheduleIndexingAtInstant(List partitionT return indexPlan.isPresent(); } + /** + * Runs INDEX action to build out the metadata partitions as planned for the given instant time. + * + * @param indexInstantTime - instant time for the requested INDEX action + * @return {@link Option} after successful indexing. + */ public Option index(String indexInstantTime) { return createTable(config, hadoopConf, config.isMetadataTableEnabled()).index(context, indexInstantTime); } + /** + * Drops the index and removes the metadata partitions. + * + * @param partitionTypes - list of {@link MetadataPartitionType} which needs to be indexed + */ public void dropIndex(List partitionTypes) { HoodieTable table = createTable(config, hadoopConf); String dropInstant = HoodieActiveTimeline.createNewInstantTime(); @@ -952,7 +971,7 @@ public void dropIndex(List partitionTypes) { try { ((HoodieTableMetadataWriter) w).dropIndex(partitionTypes); } catch (IOException e) { - LOG.error("Failed to drop metadata index. ", e); + throw new HoodieIndexException("Failed to drop metadata index. ", e); } }); } finally { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index e5fb1dc610cbf..ee5f85fe35a57 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -1519,8 +1519,8 @@ public String getColumnsEnabledForBloomFilterIndex() { return getMetadataConfig().getColumnsEnabledForBloomFilterIndex(); } - public int getIndexingCheckTimeout() { - return getMetadataConfig().getIndexingCheckTimeout(); + public int getIndexingCheckTimeoutSeconds() { + return getMetadataConfig().getIndexingCheckTimeoutSeconds(); } public int getColumnStatsIndexParallelism() { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index 43bada531dcc3..f27aac391973c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -91,6 +91,7 @@ import java.util.stream.Stream; import static org.apache.hudi.common.table.HoodieTableConfig.ARCHIVELOG_FOLDER; +import static org.apache.hudi.common.util.StringUtils.EMPTY_STRING; import static org.apache.hudi.metadata.HoodieTableMetadata.METADATA_TABLE_NAME_SUFFIX; import static org.apache.hudi.metadata.HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP; @@ -369,6 +370,18 @@ protected void initializeIfNeeded(HoodieTableMeta Option inflightInstantTimestamp) throws IOException { HoodieTimer timer = new HoodieTimer().startTimer(); + boolean exists = metadataExists(dataMetaClient, actionMetadata); + + if (!exists) { + // Initialize for the first time by listing partitions and files directly from the file system + if (initializeFromFilesystem(dataMetaClient, inflightInstantTimestamp)) { + metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.INITIALIZE_STR, timer.endTimer())); + } + } + } + + private boolean metadataExists(HoodieTableMetaClient dataMetaClient, + Option actionMetadata) throws IOException { boolean exists = dataMetaClient.getFs().exists(new Path(metadataWriteConfig.getBasePath(), HoodieTableMetaClient.METAFOLDER_NAME)); boolean reInitialize = false; @@ -397,18 +410,7 @@ protected void initializeIfNeeded(HoodieTableMeta exists = false; } - if (!exists) { - if (metadataWriteConfig.isMetadataAsyncIndex()) { - // with async metadata indexing enabled, there can be inflight writers - MetadataRecordsGenerationParams indexParams = getRecordsGenerationParams(); - scheduleIndex(indexParams.getEnabledPartitionTypes()); - return; - } - // Initialize for the first time by listing partitions and files directly from the file system - if (initializeFromFilesystem(dataMetaClient, inflightInstantTimestamp)) { - metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.INITIALIZE_STR, timer.endTimer())); - } - } + return exists; } /** @@ -463,7 +465,7 @@ private boolean isCommitRevertedByInFlightAction( final String INSTANT_ACTION = (actionMetadata.get() instanceof HoodieRollbackMetadata ? HoodieTimeline.ROLLBACK_ACTION - : (actionMetadata.get() instanceof HoodieRestoreMetadata ? HoodieTimeline.RESTORE_ACTION : "")); + : (actionMetadata.get() instanceof HoodieRestoreMetadata ? HoodieTimeline.RESTORE_ACTION : EMPTY_STRING)); List affectedInstantTimestamps; switch (INSTANT_ACTION) { @@ -630,9 +632,8 @@ public void scheduleIndex(HoodieTableMetaClient dataMetaClient, List ..., record-index-bucket-0009 */ - public void initializeFileGroups(HoodieTableMetaClient dataMetaClient, MetadataPartitionType metadataPartition, String instantTime, + private void initializeFileGroups(HoodieTableMetaClient dataMetaClient, MetadataPartitionType metadataPartition, String instantTime, int fileGroupCount) throws IOException { - final HashMap blockHeader = new HashMap<>(); blockHeader.put(HeaderMetadataType.INSTANT_TIME, instantTime); // Archival of data table has a dependency on compaction(base files) in metadata table. @@ -642,14 +643,8 @@ public void initializeFileGroups(HoodieTableMetaClient dataMetaClient, MetadataP LOG.info(String.format("Creating %d file groups for partition %s with base fileId %s at instant time %s", fileGroupCount, metadataPartition.getPartitionPath(), metadataPartition.getFileIdPrefix(), instantTime)); - HoodieTableFileSystemView fsView = HoodieTableMetadataUtil.getFileSystemView(metadataMetaClient); - List fileSlices = HoodieTableMetadataUtil.getPartitionLatestFileSlices(metadataMetaClient, Option.ofNullable(fsView), metadataPartition.getPartitionPath()); for (int i = 0; i < fileGroupCount; ++i) { final String fileGroupFileId = String.format("%s%04d", metadataPartition.getFileIdPrefix(), i); - // if a writer or async indexer had already initialized the filegroup then continue - if (!fileSlices.isEmpty() && fileSlices.stream().anyMatch(fileSlice -> fileGroupFileId.equals(fileSlice.getFileGroupId().getFileId()))) { - continue; - } try { HoodieLogFormat.Writer writer = HoodieLogFormat.newWriterBuilder() .onParentPath(FSUtils.getPartitionPath(metadataWriteConfig.getBasePath(), metadataPartition.getPartitionPath())) @@ -674,19 +669,21 @@ public void dropIndex(List indexesToDrop) throws IOExcept .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet()); Set inflightIndexes = Stream.of(dataMetaClient.getTableConfig().getInflightMetadataIndexes().split(",")) .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet()); + for (MetadataPartitionType partitionType : indexesToDrop) { String partitionPath = partitionType.getPartitionPath(); + // first update table config if (inflightIndexes.contains(partitionPath)) { - LOG.error("Metadata indexing in progress: " + partitionPath); - return; + inflightIndexes.remove(partitionPath); + dataMetaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_INDEX_INFLIGHT.key(), String.join(",", inflightIndexes)); + } else if (completedIndexes.contains(partitionPath)) { + completedIndexes.remove(partitionPath); + dataMetaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_INDEX_COMPLETED.key(), String.join(",", completedIndexes)); } + HoodieTableConfig.update(dataMetaClient.getFs(), new Path(dataMetaClient.getMetaPath()), dataMetaClient.getTableConfig().getProps()); LOG.warn("Deleting Metadata Table partitions: " + partitionPath); dataMetaClient.getFs().delete(new Path(metadataWriteConfig.getBasePath(), partitionPath), true); - completedIndexes.remove(partitionPath); } - // update table config - dataMetaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_INDEX_COMPLETED.key(), String.join(",", completedIndexes)); - HoodieTableConfig.update(dataMetaClient.getFs(), new Path(dataMetaClient.getMetaPath()), dataMetaClient.getTableConfig().getProps()); } private MetadataRecordsGenerationParams getRecordsGenerationParams() { @@ -722,10 +719,13 @@ private void processAndCommit(String instantTime, ConvertMetadataFunction co return; } Set partitionsToUpdate = getMetadataPartitionsToUpdate(); + Set inflightIndexes = Stream.of(dataMetaClient.getTableConfig().getInflightMetadataIndexes().split(",")) + .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet()); partitionsToUpdate.forEach(p -> { if (enabled && metadata != null) { Map> partitionRecordsMap = convertMetadataFunction.convertMetadata(); - commit(instantTime, partitionRecordsMap, canTriggerTableService); + // if indexing is inflight then don't trigger table service + commit(instantTime, partitionRecordsMap, !inflightIndexes.contains(p) && canTriggerTableService); } }); } @@ -745,7 +745,7 @@ private Set getMetadataPartitionsToUpdate() { } @Override - public void index(HoodieEngineContext engineContext, List indexPartitionInfos) { + public void buildIndex(HoodieEngineContext engineContext, List indexPartitionInfos) { if (indexPartitionInfos.isEmpty()) { LOG.warn("No partition to index in the plan"); return; @@ -756,7 +756,7 @@ public void index(HoodieEngineContext engineContext, List inflightIndexes = Stream.of(dataMetaClient.getTableConfig().getInflightMetadataIndexes().split(",")) + .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet()); + inflightIndexes.addAll(indexPartitionInfos.stream().map(HoodieIndexPartitionInfo::getMetadataPartitionPath).collect(Collectors.toSet())); + dataMetaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_INDEX_INFLIGHT.key(), String.join(",", inflightIndexes)); HoodieTableConfig.update(dataMetaClient.getFs(), new Path(dataMetaClient.getMetaPath()), dataMetaClient.getTableConfig().getProps()); - // check here for enabled partition types whether filegroups initialized or not initialCommit(indexUptoInstantTime); - dataMetaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_INDEX_INFLIGHT.key(), ""); - dataMetaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_INDEX_COMPLETED.key(), indexPartitionInfos.stream() - .map(HoodieIndexPartitionInfo::getMetadataPartitionPath).collect(Collectors.joining(","))); - HoodieTableConfig.update(dataMetaClient.getFs(), new Path(dataMetaClient.getMetaPath()), dataMetaClient.getTableConfig().getProps()); } /** diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java index 4741726b077d0..b83f5448cd491 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java @@ -41,7 +41,7 @@ public interface HoodieTableMetadataWriter extends Serializable, AutoCloseable { * @param engineContext * @param indexPartitionInfos - partitions to index */ - void index(HoodieEngineContext engineContext, List indexPartitionInfos); + void buildIndex(HoodieEngineContext engineContext, List indexPartitionInfos); /** * Initialize file groups while scheduling index action. diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java index 2e034a533609f..df59d69731b05 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java @@ -29,6 +29,7 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -44,19 +45,19 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.BaseActionExecutor; +import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import java.io.IOException; import java.util.List; import java.util.Set; -import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.apache.hudi.common.model.WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL; import static org.apache.hudi.common.table.timeline.HoodieInstant.State.COMPLETED; @@ -115,13 +116,16 @@ public Option execute() { // start indexing for each partition HoodieTableMetadataWriter metadataWriter = table.getMetadataWriter(instantTime) .orElseThrow(() -> new HoodieIndexException(String.format("Could not get metadata writer to run index action for instant: %s", instantTime))); - metadataWriter.index(context, indexPartitionInfos); + // this will only build index upto base instant as generated by the plan, we will be doing catchup later + LOG.info("Starting Index Building"); + metadataWriter.buildIndex(context, indexPartitionInfos); // get all instants since the plan completed (both from active timeline and archived timeline) // assumption is that all metadata partitions had same instant upto which they were scheduled to be indexed table.getMetaClient().reloadActiveTimeline(); String indexUptoInstant = indexPartitionInfos.get(0).getIndexUptoInstant(); List instantsToIndex = getRemainingArchivedAndActiveInstantsSince(indexUptoInstant, table.getMetaClient()); + LOG.info("Total remaining instants to index: " + instantsToIndex.size()); // reconcile with metadata table timeline String metadataBasePath = HoodieTableMetadata.getMetadataTableBasePath(table.getMetaClient().getBasePath()); @@ -129,19 +133,21 @@ public Option execute() { Set metadataCompletedTimestamps = getCompletedArchivedAndActiveInstantsAfter(indexUptoInstant, metadataMetaClient).stream() .map(HoodieInstant::getTimestamp).collect(Collectors.toSet()); - // index all remaining instants with a timeout + // index catchup for all remaining instants with a timeout currentIndexedInstant = indexUptoInstant; ExecutorService executorService = Executors.newFixedThreadPool(MAX_CONCURRENT_INDEXING); - Future postRequestIndexingTaskFuture = executorService.submit( - new IndexingCheckTask(metadataWriter, instantsToIndex, metadataCompletedTimestamps, table.getMetaClient())); + Future indexingCatchupTaskFuture = executorService.submit( + new IndexingCatchupTask(metadataWriter, instantsToIndex, metadataCompletedTimestamps, table.getMetaClient())); try { - postRequestIndexingTaskFuture.get(config.getIndexingCheckTimeout(), TimeUnit.SECONDS); - } catch (TimeoutException | InterruptedException | ExecutionException e) { - postRequestIndexingTaskFuture.cancel(true); + LOG.info("Starting index catchup task"); + indexingCatchupTaskFuture.get(config.getIndexingCheckTimeoutSeconds(), TimeUnit.SECONDS); + } catch (Exception e) { + indexingCatchupTaskFuture.cancel(true); + throw new HoodieIndexException(String.format("Index catchup failed. Current indexed instant = %s. Aborting!", currentIndexedInstant), e); } finally { executorService.shutdownNow(); } - // save index commit metadata and return + // save index commit metadata and update table config List finalIndexPartitionInfos = indexPartitionInfos.stream() .map(info -> new HoodieIndexPartitionInfo( info.getVersion(), @@ -151,7 +157,9 @@ public Option execute() { HoodieIndexCommitMetadata indexCommitMetadata = HoodieIndexCommitMetadata.newBuilder() .setVersion(LATEST_INDEX_COMMIT_METADATA_VERSION).setIndexPartitionInfos(finalIndexPartitionInfos).build(); try { + // update the table config and timeline in a lock as there could be another indexer running txnManager.beginTransaction(); + updateTableConfig(table.getMetaClient(), finalIndexPartitionInfos); table.getActiveTimeline().saveAsComplete( new HoodieInstant(true, INDEX_ACTION, indexInstant.getTimestamp()), TimelineMetadataUtils.serializeIndexCommitMetadata(indexCommitMetadata)); @@ -178,26 +186,41 @@ private static List getCompletedArchivedAndActiveInstantsAfter(St .filterCompletedInstants() .findInstantsAfter(instant) .getInstants().collect(Collectors.toList()); - completedInstants.addAll(metaClient.getActiveTimeline().filterCompletedInstants().findInstantsAfter(instant).getInstants().collect(Collectors.toList())); + completedInstants.addAll(metaClient.reloadActiveTimeline().filterCompletedInstants().findInstantsAfter(instant).getInstants().collect(Collectors.toList())); return completedInstants; } + private void updateTableConfig(HoodieTableMetaClient metaClient, List indexPartitionInfos) { + // remove from inflight and update completed indexes + Set inflightIndexes = Stream.of(metaClient.getTableConfig().getInflightMetadataIndexes().split(",")) + .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet()); + Set completedIndexes = Stream.of(metaClient.getTableConfig().getCompletedMetadataIndexes().split(",")) + .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet()); + Set indexesRequested = indexPartitionInfos.stream().map(HoodieIndexPartitionInfo::getMetadataPartitionPath).collect(Collectors.toSet()); + inflightIndexes.removeAll(indexesRequested); + completedIndexes.addAll(indexesRequested); + // update table config + metaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_INDEX_INFLIGHT.key(), String.join(",", inflightIndexes)); + metaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_INDEX_COMPLETED.key(), String.join(",", completedIndexes)); + HoodieTableConfig.update(metaClient.getFs(), new Path(metaClient.getMetaPath()), metaClient.getTableConfig().getProps()); + } + /** * Indexing check runs for instants that completed after the base instant (in the index plan). * It will check if these later instants have logged updates to metadata table or not. * If not, then it will do the update. If a later instant is inflight, it will wait until it is completed or the task times out. */ - class IndexingCheckTask implements Runnable { + class IndexingCatchupTask implements Runnable { private final HoodieTableMetadataWriter metadataWriter; private final List instantsToIndex; private final Set metadataCompletedInstants; private final HoodieTableMetaClient metaClient; - IndexingCheckTask(HoodieTableMetadataWriter metadataWriter, - List instantsToIndex, - Set metadataCompletedInstants, - HoodieTableMetaClient metaClient) { + IndexingCatchupTask(HoodieTableMetadataWriter metadataWriter, + List instantsToIndex, + Set metadataCompletedInstants, + HoodieTableMetaClient metaClient) { this.metadataWriter = metadataWriter; this.instantsToIndex = instantsToIndex; this.metadataCompletedInstants = metadataCompletedInstants; @@ -206,55 +229,62 @@ class IndexingCheckTask implements Runnable { @Override public void run() { - while (!Thread.interrupted()) { - for (HoodieInstant instant : instantsToIndex) { - // metadata index already updated for this instant - if (metadataCompletedInstants.contains(instant.getTimestamp())) { - currentIndexedInstant = instant.getTimestamp(); - continue; - } - while (!instant.isCompleted()) { + for (HoodieInstant instant : instantsToIndex) { + // metadata index already updated for this instant + if (metadataCompletedInstants.contains(instant.getTimestamp())) { + currentIndexedInstant = instant.getTimestamp(); + continue; + } + while (!instant.isCompleted()) { + try { + LOG.info("instant not completed, reloading timeline " + instant); // reload timeline and fetch instant details again wait until timeout String instantTime = instant.getTimestamp(); Option currentInstant = metaClient.reloadActiveTimeline() .filterCompletedInstants().filter(i -> i.getTimestamp().equals(instantTime)).firstInstant(); instant = currentInstant.orElse(instant); + // so that timeline is not reloaded very frequently + Thread.sleep(5000); + } catch (InterruptedException e) { + throw new HoodieIndexException(String.format("Thread interrupted while running indexing check for instant: %s", instant), e); } - // update metadata for this completed instant - if (COMPLETED.equals(instant.getState())) { - try { - // we need take a lock here as inflight writer could also try to update the timeline - txnManager.beginTransaction(Option.of(instant), Option.empty()); - switch (instant.getAction()) { - case HoodieTimeline.COMMIT_ACTION: - case HoodieTimeline.DELTA_COMMIT_ACTION: - case HoodieTimeline.REPLACE_COMMIT_ACTION: - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( - table.getActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class); - metadataWriter.update(commitMetadata, instant.getTimestamp(), false); - break; - case HoodieTimeline.CLEAN_ACTION: - HoodieCleanMetadata cleanMetadata = CleanerUtils.getCleanerMetadata(table.getMetaClient(), instant); - metadataWriter.update(cleanMetadata, instant.getTimestamp()); - break; - case HoodieTimeline.RESTORE_ACTION: - HoodieRestoreMetadata restoreMetadata = TimelineMetadataUtils.deserializeHoodieRestoreMetadata( - table.getActiveTimeline().getInstantDetails(instant).get()); - metadataWriter.update(restoreMetadata, instant.getTimestamp()); - break; - case HoodieTimeline.ROLLBACK_ACTION: - HoodieRollbackMetadata rollbackMetadata = TimelineMetadataUtils.deserializeHoodieRollbackMetadata( - table.getActiveTimeline().getInstantDetails(instant).get()); - metadataWriter.update(rollbackMetadata, instant.getTimestamp()); - break; - default: - throw new IllegalStateException("Unexpected value: " + instant.getAction()); - } - } catch (IOException e) { - LOG.error("Could not update metadata partition for instant: " + instant); - } finally { - txnManager.endTransaction(Option.of(instant)); + } + // update metadata for this completed instant + if (COMPLETED.equals(instant.getState())) { + try { + // we need take a lock here as inflight writer could also try to update the timeline + txnManager.beginTransaction(Option.of(instant), Option.empty()); + LOG.info("Updating metadata table for instant: " + instant); + switch (instant.getAction()) { + case HoodieTimeline.COMMIT_ACTION: + case HoodieTimeline.DELTA_COMMIT_ACTION: + case HoodieTimeline.REPLACE_COMMIT_ACTION: + HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( + table.getActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class); + // do not trigger any table service as partition is not fully built out yet + metadataWriter.update(commitMetadata, instant.getTimestamp(), false); + break; + case HoodieTimeline.CLEAN_ACTION: + HoodieCleanMetadata cleanMetadata = CleanerUtils.getCleanerMetadata(table.getMetaClient(), instant); + metadataWriter.update(cleanMetadata, instant.getTimestamp()); + break; + case HoodieTimeline.RESTORE_ACTION: + HoodieRestoreMetadata restoreMetadata = TimelineMetadataUtils.deserializeHoodieRestoreMetadata( + table.getActiveTimeline().getInstantDetails(instant).get()); + metadataWriter.update(restoreMetadata, instant.getTimestamp()); + break; + case HoodieTimeline.ROLLBACK_ACTION: + HoodieRollbackMetadata rollbackMetadata = TimelineMetadataUtils.deserializeHoodieRollbackMetadata( + table.getActiveTimeline().getInstantDetails(instant).get()); + metadataWriter.update(rollbackMetadata, instant.getTimestamp()); + break; + default: + throw new IllegalStateException("Unexpected value: " + instant.getAction()); } + } catch (IOException e) { + throw new HoodieIndexException(String.format("Could not update metadata partition for instant: %s", instant), e); + } finally { + txnManager.endTransaction(Option.of(instant)); } } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java index 5afc027e75289..f95d21d619e5d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java @@ -55,7 +55,7 @@ *

  • * 1. Fetch last completed instant on data timeline. * 2. Write the index plan to the .index.requested. - * 3. Initialize filegroups for the enabled partition types within a transaction. + * 3. Initialize file groups for the enabled partition types within a transaction. *
  • */ public class ScheduleIndexActionExecutor extends BaseActionExecutor> { @@ -91,23 +91,39 @@ public Option execute() { // make sure that it is idempotent, check with previously pending index operations. Set indexesInflightOrCompleted = Stream.of(table.getMetaClient().getTableConfig().getInflightMetadataIndexes().split(",")) .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet()); - indexesInflightOrCompleted.addAll(Stream.of(table.getMetaClient().getTableConfig().getInflightMetadataIndexes().split(",")) + indexesInflightOrCompleted.addAll(Stream.of(table.getMetaClient().getTableConfig().getCompletedMetadataIndexes().split(",")) .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet())); Set requestedPartitions = partitionsToIndex.stream().map(MetadataPartitionType::getPartitionPath).collect(Collectors.toSet()); - requestedPartitions.retainAll(indexesInflightOrCompleted); + requestedPartitions.removeAll(indexesInflightOrCompleted); if (!requestedPartitions.isEmpty()) { - LOG.error("Following partitions already exist or inflight: " + requestedPartitions); - return Option.empty(); + LOG.warn(String.format("Following partitions already exist or inflight: %s. Going to index only these partitions: %s", + indexesInflightOrCompleted, requestedPartitions)); } + List finalPartitionsToIndex = partitionsToIndex.stream() + .filter(p -> requestedPartitions.contains(p.getPartitionPath())).collect(Collectors.toList()); // get last completed instant Option indexUptoInstant = table.getActiveTimeline().getContiguousCompletedWriteTimeline().lastInstant(); if (indexUptoInstant.isPresent()) { final HoodieInstant indexInstant = HoodieTimeline.getIndexRequestedInstant(instantTime); + // start initializing file groups + // in case FILES partition itself was not initialized before (i.e. metadata was never enabled), this will initialize synchronously + HoodieTableMetadataWriter metadataWriter = table.getMetadataWriter(instantTime) + .orElseThrow(() -> new HoodieIndexException(String.format("Could not get metadata writer to initialize filegroups for indexing for instant: %s", instantTime))); + try { + this.txnManager.beginTransaction(Option.of(indexInstant), Option.empty()); + metadataWriter.scheduleIndex(table.getMetaClient(), finalPartitionsToIndex, indexInstant.getTimestamp()); + } catch (IOException e) { + LOG.error("Could not initialize file groups", e); + throw new HoodieIOException(e.getMessage(), e); + } finally { + this.txnManager.endTransaction(Option.of(indexInstant)); + } // for each partitionToIndex add that time to the plan - List indexPartitionInfos = partitionsToIndex.stream() + List indexPartitionInfos = finalPartitionsToIndex.stream() .map(p -> new HoodieIndexPartitionInfo(LATEST_INDEX_PLAN_VERSION, p.getPartitionPath(), indexUptoInstant.get().getTimestamp())) .collect(Collectors.toList()); HoodieIndexPlan indexPlan = new HoodieIndexPlan(LATEST_INDEX_PLAN_VERSION, indexPartitionInfos); + // update data timeline with requested instant try { table.getActiveTimeline().saveToPendingIndexCommit(indexInstant, TimelineMetadataUtils.serializeIndexPlan(indexPlan)); } catch (IOException e) { @@ -115,22 +131,6 @@ public Option execute() { throw new HoodieIOException(e.getMessage(), e); } table.getMetaClient().reloadActiveTimeline(); - - // start initializing filegroups - // 1. get metadata writer - HoodieTableMetadataWriter metadataWriter = table.getMetadataWriter(instantTime) - .orElseThrow(() -> new HoodieIndexException(String.format("Could not get metadata writer to initialize filegroups for indexing for instant: %s", instantTime))); - // 2. take a lock --> begin tx (data table) - try { - this.txnManager.beginTransaction(Option.of(indexInstant), Option.empty()); - // 3. initialize filegroups as per plan for the enabled partition types - metadataWriter.scheduleIndex(table.getMetaClient(), partitionsToIndex, indexInstant.getTimestamp()); - } catch (IOException e) { - LOG.error("Could not initialize file groups"); - throw new HoodieIOException(e.getMessage(), e); - } finally { - this.txnManager.endTransaction(Option.of(indexInstant)); - } return Option.of(indexPlan); } return Option.empty(); diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java index 4683ed7169c2b..5e25945bb8915 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java @@ -344,12 +344,12 @@ public HoodieRollbackMetadata rollback(HoodieEngineContext context, String rollb @Override public Option scheduleIndex(HoodieEngineContext context, String indexInstantTime, List partitionsToIndex) { - throw new HoodieNotSupportedException("Indexing is not supported for a Flink table yet."); + throw new HoodieNotSupportedException("Metadata indexing is not supported for a Flink table yet."); } @Override public Option index(HoodieEngineContext context, String indexInstantTime) { - throw new HoodieNotSupportedException("Indexing is not supported for a Flink table yet."); + throw new HoodieNotSupportedException("Metadata indexing is not supported for a Flink table yet."); } @Override diff --git a/hudi-common/src/main/avro/HoodieIndexCommitMetadata.avsc b/hudi-common/src/main/avro/HoodieIndexCommitMetadata.avsc index 07e6793dc49cd..098a8c88e7328 100644 --- a/hudi-common/src/main/avro/HoodieIndexCommitMetadata.avsc +++ b/hudi-common/src/main/avro/HoodieIndexCommitMetadata.avsc @@ -22,6 +22,7 @@ "fields": [ { "name": "version", + "doc": "This field replaces the field filesToBeDeletedPerPartition", "type": [ "int", "null" @@ -30,6 +31,7 @@ }, { "name": "operationType", + "doc": "This field replaces the field filesToBeDeletedPerPartition", "type": [ "null", "string" @@ -38,6 +40,7 @@ }, { "name": "indexPartitionInfos", + "doc": "This field contains the info for each partition that got indexed", "type": [ "null", { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/bloom/BloomFilter.java b/hudi-common/src/main/java/org/apache/hudi/common/bloom/BloomFilter.java index 60e20e45c210f..7997da159b7f9 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/bloom/BloomFilter.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/bloom/BloomFilter.java @@ -18,10 +18,6 @@ package org.apache.hudi.common.bloom; -import javax.annotation.Nonnull; - -import java.util.List; - /** * A Bloom filter interface. */ @@ -34,13 +30,6 @@ public interface BloomFilter { */ void add(String key); - /** - * Add secondary key to the {@link BloomFilter}. - * - * @param keys list of secondary keys to add to the {@link BloomFilter} - */ - void add(@Nonnull List keys); - /** * Tests for key membership. * diff --git a/hudi-common/src/main/java/org/apache/hudi/common/bloom/HoodieDynamicBoundedBloomFilter.java b/hudi-common/src/main/java/org/apache/hudi/common/bloom/HoodieDynamicBoundedBloomFilter.java index d8850eac10556..d4bc287c551c2 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/bloom/HoodieDynamicBoundedBloomFilter.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/bloom/HoodieDynamicBoundedBloomFilter.java @@ -23,15 +23,12 @@ import org.apache.hadoop.util.bloom.Key; -import javax.annotation.Nonnull; - import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; import java.nio.charset.StandardCharsets; -import java.util.List; /** * Hoodie's dynamic bloom bounded bloom filter. This is based largely on Hadoop's DynamicBloomFilter, but with a bound @@ -84,13 +81,6 @@ public void add(String key) { internalDynamicBloomFilter.add(new Key(key.getBytes(StandardCharsets.UTF_8))); } - @Override - public void add(@Nonnull List keys) { - for (String key: keys) { - add(key); - } - } - @Override public boolean mightContain(String key) { return internalDynamicBloomFilter.membershipTest(new Key(key.getBytes(StandardCharsets.UTF_8))); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/bloom/SimpleBloomFilter.java b/hudi-common/src/main/java/org/apache/hudi/common/bloom/SimpleBloomFilter.java index 7eafa295f7e97..2403ffd995750 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/bloom/SimpleBloomFilter.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/bloom/SimpleBloomFilter.java @@ -23,8 +23,6 @@ import org.apache.hadoop.util.bloom.Key; -import javax.annotation.Nonnull; - import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.DataInput; @@ -35,7 +33,6 @@ import java.io.ObjectInputStream; import java.io.ObjectOutputStream; import java.nio.charset.StandardCharsets; -import java.util.List; /** * A Simple Bloom filter implementation built on top of {@link org.apache.hadoop.util.bloom.BloomFilter}. @@ -86,13 +83,6 @@ public void add(String key) { filter.add(new Key(key.getBytes(StandardCharsets.UTF_8))); } - @Override - public void add(@Nonnull List keys) { - for (String key: keys) { - add(key); - } - } - @Override public boolean mightContain(String key) { if (key == null) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java index 1488ee1e19a53..e5cd6d76ecc0c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java @@ -186,13 +186,13 @@ public final class HoodieMetadataConfig extends HoodieConfig { .key(METADATA_PREFIX + ".index.column.stats.for.columns") .defaultValue("") .sinceVersion("0.11.0") - .withDocumentation("Comma-separated list of columns for which column stats index will be built."); + .withDocumentation("Comma-separated list of columns for which column stats index will be built. If not set, all columns will be indexed"); public static final ConfigProperty BLOOM_FILTER_INDEX_FOR_COLUMNS = ConfigProperty .key(METADATA_PREFIX + ".index.bloom.filter.for.columns") .defaultValue("") .sinceVersion("0.11.0") - .withDocumentation("Comma-separated list of columns for which bloom filter index will be built."); + .withDocumentation("Comma-separated list of columns for which bloom filter index will be built. If not set, only record key will be indexed."); public static final ConfigProperty METADATA_INDEX_CHECK_TIMEOUT_SECONDS = ConfigProperty .key(METADATA_PREFIX + ".index.check.timeout.seconds") @@ -267,7 +267,7 @@ public int getColumnStatsIndexParallelism() { return getIntOrDefault(COLUMN_STATS_INDEX_PARALLELISM); } - public int getIndexingCheckTimeout() { + public int getIndexingCheckTimeoutSeconds() { return getIntOrDefault(METADATA_INDEX_CHECK_TIMEOUT_SECONDS); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java index ba6ab0e9be79b..aeff72114291b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java @@ -209,13 +209,13 @@ public class HoodieTableConfig extends HoodieConfig { .withDocumentation("Table checksum is used to guard against partial writes in HDFS. It is added as the last entry in hoodie.properties and then used to validate while reading table config."); public static final ConfigProperty TABLE_METADATA_INDEX_INFLIGHT = ConfigProperty - .key("hoodie.table.metadata.index.inflight") + .key("hoodie.table.metadata.indexes.inflight") .noDefaultValue() .sinceVersion("0.11.0") .withDocumentation("Comma-separated list of metadata partitions whose indexing is in progress."); public static final ConfigProperty TABLE_METADATA_INDEX_COMPLETED = ConfigProperty - .key("hoodie.table.metadata.index.completed") + .key("hoodie.table.metadata.indexes.completed") .noDefaultValue() .sinceVersion("0.11.0") .withDocumentation("Comma-separated list of metadata partitions whose indexing is complete."); @@ -601,6 +601,8 @@ public String getInflightMetadataIndexes() { return getStringOrDefault(TABLE_METADATA_INDEX_INFLIGHT, ""); } + // TODO getInflightAndCompletedMetadataIndexes + public String getCompletedMetadataIndexes() { return getStringOrDefault(TABLE_METADATA_INDEX_COMPLETED, ""); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java index ef0cf113f7ae4..7f2fea5ba1ab7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java @@ -27,7 +27,6 @@ import java.io.Serializable; import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; -import java.util.ArrayList; import java.util.List; import java.util.Set; import java.util.function.Function; @@ -115,22 +114,13 @@ public HoodieDefaultTimeline getWriteTimeline() { } @Override - public HoodieDefaultTimeline getContiguousCompletedWriteTimeline() { - List contiguousCompletedInstants = new ArrayList<>(); - String prevTime = ""; - for (HoodieInstant instant : instants) { - if (prevTime.equals(instant.getTimestamp())) { - continue; - } - if (!instant.isCompleted()) { - break; - } - contiguousCompletedInstants.add(instant); - prevTime = instant.getTimestamp(); + public HoodieTimeline getContiguousCompletedWriteTimeline() { + Option earliestPending = getWriteTimeline().filterInflightsAndRequested().firstInstant(); + if (earliestPending.isPresent()) { + return getWriteTimeline().filterCompletedInstants() + .filter(instant -> HoodieTimeline.compareTimestamps(instant.getTimestamp(), LESSER_THAN, earliestPending.get().getTimestamp())); } - - Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, REPLACE_COMMIT_ACTION); - return new HoodieDefaultTimeline(contiguousCompletedInstants.stream().filter(s -> validActions.contains(s.getAction())), details); + return getWriteTimeline().filterCompletedInstants(); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java index 76b23ca244bc2..94d3e272d7bf3 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java @@ -282,7 +282,6 @@ public static HoodieData convertMetadataToBloomFilterRecords( LOG.error("Failed to read bloom filter for " + writeFilePath); return Collections.emptyListIterator(); } - fileBloomFilter.add(recordsGenerationParams.getBloomSecondaryKeys()); ByteBuffer bloomByteBuffer = ByteBuffer.wrap(fileBloomFilter.serializeToString().getBytes()); HoodieRecord record = HoodieMetadataPayload.createBloomFilterMetadataRecord( partition, fileName, instantTime, recordsGenerationParams.getBloomFilterType(), bloomByteBuffer, false); @@ -727,7 +726,6 @@ public static HoodieData convertFilesToBloomFilterRecords(HoodieEn LOG.error("Failed to read bloom filter for " + appendedFilePath); return Stream.empty(); } - fileBloomFilter.add(recordsGenerationParams.getBloomSecondaryKeys()); ByteBuffer bloomByteBuffer = ByteBuffer.wrap(fileBloomFilter.serializeToString().getBytes()); HoodieRecord record = HoodieMetadataPayload.createBloomFilterMetadataRecord( partition, appendedFile, instantTime, recordsGenerationParams.getBloomFilterType(), bloomByteBuffer, false); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java b/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java index 0b591adcb8707..1dd1eedfbde59 100755 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java @@ -199,6 +199,46 @@ public void testTimelineOperations() { assertTrue(activeCommitTimeline.isBeforeTimelineStarts("00")); } + @Test + public void testGetContiguousCompletedWriteTimeline() { + // a mock timeline with holes + timeline = new MockHoodieTimeline(Stream.of("01", "03", "05", "07", "13", "15", "17"), + Stream.of("09", "11", "19")); + assertTrue(timeline.getContiguousCompletedWriteTimeline().lastInstant().isPresent()); + assertEquals("07", timeline.getContiguousCompletedWriteTimeline().lastInstant().get().getTimestamp()); + + // add some instants where two are inflight and one of them (instant8 below) is not part of write timeline + HoodieInstant instant1 = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "1"); + HoodieInstant instant2 = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "2"); + HoodieInstant instant3 = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "3"); + HoodieInstant instant4 = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "4"); + HoodieInstant instant5 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "5"); + HoodieInstant instant6 = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "6"); + HoodieInstant instant7 = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "7"); + HoodieInstant instant8 = new HoodieInstant(true, HoodieTimeline.RESTORE_ACTION, "8"); + + timeline = new HoodieActiveTimeline(metaClient); + timeline.createNewInstant(instant1); + timeline.createNewInstant(instant2); + timeline.createNewInstant(instant3); + timeline.createNewInstant(instant4); + timeline.createNewInstant(instant5); + timeline.createNewInstant(instant6); + timeline.createNewInstant(instant7); + timeline.createNewInstant(instant8); + timeline.setInstants(Stream.of(instant1, instant2, instant3, instant4, instant5, instant6, instant7, instant8).collect(Collectors.toList())); + + assertTrue(timeline.getContiguousCompletedWriteTimeline().lastInstant().isPresent()); + assertEquals(instant4.getTimestamp(), timeline.getContiguousCompletedWriteTimeline().lastInstant().get().getTimestamp()); + // transition both inflight instants to complete + timeline.saveAsComplete(new HoodieInstant(true, instant5.getAction(), instant5.getTimestamp()), Option.empty()); + timeline.saveAsComplete(new HoodieInstant(true, instant8.getAction(), instant8.getTimestamp()), Option.empty()); + timeline = timeline.reload(); + // instant8 in not considered in write timeline, so last completed instant in timeline should be instant7 + assertTrue(timeline.getContiguousCompletedWriteTimeline().lastInstant().isPresent()); + assertEquals(instant7.getTimestamp(), timeline.getContiguousCompletedWriteTimeline().lastInstant().get().getTimestamp()); + } + @Test public void testTimelineGetOperations() { List allInstants = getAllInstants(); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java index c3a500ba38b06..9316e154ac923 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java @@ -21,12 +21,12 @@ import org.apache.hudi.avro.model.HoodieIndexCommitMetadata; import org.apache.hudi.avro.model.HoodieIndexPartitionInfo; import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.exception.HoodieIndexException; import org.apache.hudi.metadata.MetadataPartitionType; @@ -43,9 +43,11 @@ import java.util.Arrays; import java.util.List; import java.util.Locale; +import java.util.Set; import java.util.stream.Collectors; import static org.apache.hudi.common.table.timeline.HoodieInstant.State.INFLIGHT; +import static org.apache.hudi.common.util.StringUtils.isNullOrEmpty; import static org.apache.hudi.utilities.UtilHelpers.EXECUTE; import static org.apache.hudi.utilities.UtilHelpers.SCHEDULE; import static org.apache.hudi.utilities.UtilHelpers.SCHEDULE_AND_EXECUTE; @@ -87,7 +89,7 @@ public class HoodieIndexer { public HoodieIndexer(JavaSparkContext jsc, HoodieIndexer.Config cfg) { this.cfg = cfg; this.jsc = jsc; - this.props = StringUtils.isNullOrEmpty(cfg.propsFilePath) + this.props = isNullOrEmpty(cfg.propsFilePath) ? UtilHelpers.buildProperties(cfg.configs) : readConfigFromFileSystem(jsc, cfg); this.metaClient = UtilHelpers.createMetaClient(jsc, cfg.basePath, true); @@ -155,6 +157,12 @@ public static void main(String[] args) { } public int start(int retry) { + // indexing should be done only if metadata is enabled + if (!props.getBoolean(HoodieMetadataConfig.ENABLE.key())) { + LOG.error(String.format("Metadata is not enabled. Please set %s to true.", HoodieMetadataConfig.ENABLE.key())); + return -1; + } + return UtilHelpers.retry(retry, () -> { switch (cfg.runningMode.toLowerCase()) { case SCHEDULE: { @@ -199,14 +207,7 @@ private Option scheduleIndexing(JavaSparkContext jsc) throws Exception { } private Option doSchedule(SparkRDDWriteClient client) { - List partitionsToIndex = Arrays.asList(cfg.indexTypes.split(",")); - List partitionTypes = partitionsToIndex.stream() - .map(p -> MetadataPartitionType.valueOf(p.toUpperCase(Locale.ROOT))) - .collect(Collectors.toList()); - if (cfg.indexInstantTime != null) { - client.scheduleClusteringAtInstant(cfg.indexInstantTime, Option.empty()); - return Option.of(cfg.indexInstantTime); - } + List partitionTypes = getRequestedPartitionTypes(cfg.indexTypes); Option indexingInstant = client.scheduleIndexing(partitionTypes); if (!indexingInstant.isPresent()) { LOG.error("Scheduling of index action did not return any instant."); @@ -217,7 +218,7 @@ private Option doSchedule(SparkRDDWriteClient clien private int runIndexing(JavaSparkContext jsc) throws Exception { String schemaStr = UtilHelpers.getSchemaFromLatestInstant(metaClient); try (SparkRDDWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props)) { - if (StringUtils.isNullOrEmpty(cfg.indexInstantTime)) { + if (isNullOrEmpty(cfg.indexInstantTime)) { // Instant time is not specified // Find the earliest scheduled indexing instant for execution Option earliestPendingIndexInstant = metaClient.getActiveTimeline() @@ -249,10 +250,7 @@ private int scheduleAndRunIndexing(JavaSparkContext jsc) throws Exception { } private int dropIndex(JavaSparkContext jsc) throws Exception { - List partitionsToDrop = Arrays.asList(cfg.indexTypes.split(",")); - List partitionTypes = partitionsToDrop.stream() - .map(p -> MetadataPartitionType.valueOf(p.toUpperCase(Locale.ROOT))) - .collect(Collectors.toList()); + List partitionTypes = getRequestedPartitionTypes(cfg.indexTypes); String schemaStr = UtilHelpers.getSchemaFromLatestInstant(metaClient); try (SparkRDDWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props)) { client.dropIndex(partitionTypes); @@ -271,6 +269,24 @@ private boolean handleResponse(Option commitMetadata) List indexPartitionInfos = commitMetadata.get().getIndexPartitionInfos(); LOG.info(String.format("Indexing complete for partitions: %s", indexPartitionInfos.stream().map(HoodieIndexPartitionInfo::getMetadataPartitionPath).collect(Collectors.toList()))); - return true; + return isIndexBuiltForAllRequestedTypes(indexPartitionInfos); + } + + boolean isIndexBuiltForAllRequestedTypes(List indexPartitionInfos) { + Set indexedPartitions = indexPartitionInfos.stream() + .map(HoodieIndexPartitionInfo::getMetadataPartitionPath).collect(Collectors.toSet()); + Set requestedPartitions = getRequestedPartitionTypes(cfg.indexTypes).stream() + .map(MetadataPartitionType::getPartitionPath).collect(Collectors.toSet()); + requestedPartitions.removeAll(indexedPartitions); + return requestedPartitions.isEmpty(); + } + + List getRequestedPartitionTypes(String indexTypes) { + List requestedIndexTypes = Arrays.asList(indexTypes.split(",")); + return requestedIndexTypes.stream() + .map(p -> MetadataPartitionType.valueOf(p.toUpperCase(Locale.ROOT))) + // FILES partition is initialized synchronously while getting metadata writer + .filter(p -> !MetadataPartitionType.FILES.equals(p)) + .collect(Collectors.toList()); } } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieIndexer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieIndexer.java new file mode 100644 index 0000000000000..9ce8eef313de3 --- /dev/null +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieIndexer.java @@ -0,0 +1,122 @@ +/* + * 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.HoodieIndexCommitMetadata; +import org.apache.hudi.avro.model.HoodieIndexPartitionInfo; +import org.apache.hudi.client.HoodieReadClient; +import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.testutils.HoodieCommonTestHarness; +import org.apache.hudi.common.testutils.HoodieTestUtils; +import org.apache.hudi.metadata.MetadataPartitionType; +import org.apache.hudi.testutils.providers.SparkProvider; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SparkSession; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; + +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestHoodieIndexer extends HoodieCommonTestHarness implements SparkProvider { + + private static transient SparkSession spark; + private static transient SQLContext sqlContext; + private static transient JavaSparkContext jsc; + private static transient HoodieSparkEngineContext context; + + @BeforeEach + public void init() throws IOException { + boolean initialized = spark != null; + if (!initialized) { + SparkConf sparkConf = conf(); + SparkRDDWriteClient.registerClasses(sparkConf); + HoodieReadClient.addHoodieSupport(sparkConf); + spark = SparkSession.builder().config(sparkConf).getOrCreate(); + sqlContext = spark.sqlContext(); + jsc = new JavaSparkContext(spark.sparkContext()); + context = new HoodieSparkEngineContext(jsc); + } + initPath(); + metaClient = HoodieTestUtils.init(basePath, getTableType()); + } + + @Test + public void testGetRequestedPartitionTypes() { + HoodieIndexer.Config config = new HoodieIndexer.Config(); + config.basePath = basePath; + config.tableName = "indexer_test"; + config.indexTypes = "FILES,BLOOM_FILTERS,COLUMN_STATS"; + HoodieIndexer indexer = new HoodieIndexer(jsc, config); + List partitionTypes = indexer.getRequestedPartitionTypes(config.indexTypes); + assertFalse(partitionTypes.contains(MetadataPartitionType.FILES)); + assertTrue(partitionTypes.contains(MetadataPartitionType.BLOOM_FILTERS)); + assertTrue(partitionTypes.contains(MetadataPartitionType.COLUMN_STATS)); + } + + @Test + public void testIsIndexBuiltForAllRequestedTypes() { + HoodieIndexer.Config config = new HoodieIndexer.Config(); + config.basePath = basePath; + config.tableName = "indexer_test"; + config.indexTypes = "BLOOM_FILTERS,COLUMN_STATS"; + HoodieIndexer indexer = new HoodieIndexer(jsc, config); + HoodieIndexCommitMetadata commitMetadata = HoodieIndexCommitMetadata.newBuilder() + .setIndexPartitionInfos(Arrays.asList(new HoodieIndexPartitionInfo( + 1, + MetadataPartitionType.COLUMN_STATS.getPartitionPath(), + "0000"))) + .build(); + assertFalse(indexer.isIndexBuiltForAllRequestedTypes(commitMetadata.getIndexPartitionInfos())); + + config.indexTypes = "COLUMN_STATS"; + indexer = new HoodieIndexer(jsc, config); + assertTrue(indexer.isIndexBuiltForAllRequestedTypes(commitMetadata.getIndexPartitionInfos())); + } + + @Override + public HoodieEngineContext context() { + return context; + } + + @Override + public SparkSession spark() { + return spark; + } + + @Override + public SQLContext sqlContext() { + return sqlContext; + } + + @Override + public JavaSparkContext jsc() { + return jsc; + } +} diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java index e76567812958a..2db72cbd4102e 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java @@ -1010,20 +1010,20 @@ public void testHoodieIndexer() throws Exception { buildIndexerConfig(tableBasePath, ds.getConfig().targetTableName, null, SCHEDULE, "COLUMN_STATS")); scheduleIndexInstantTime = scheduleIndexingJob.doSchedule(); } catch (Exception e) { - LOG.info("Schedule clustering failed", e); + LOG.info("Schedule indexing failed", e); return false; } if (scheduleIndexInstantTime.isPresent()) { TestHelpers.assertPendingIndexCommit(tableBasePath, dfs); - LOG.info("Schedule clustering success, now cluster with instant time " + scheduleIndexInstantTime.get()); + LOG.info("Schedule indexing success, now build index with instant time " + scheduleIndexInstantTime.get()); HoodieIndexer runIndexingJob = new HoodieIndexer(jsc, buildIndexerConfig(tableBasePath, ds.getConfig().targetTableName, scheduleIndexInstantTime.get(), EXECUTE, "COLUMN_STATS")); runIndexingJob.start(0); - LOG.info("Cluster success"); + LOG.info("Metadata indexing success"); + TestHelpers.assertCompletedIndexCommit(tableBasePath, dfs); } else { - LOG.warn("Schedule clustering failed"); + LOG.warn("Metadata indexing failed"); } - TestHelpers.assertCompletedIndexCommit(tableBasePath, dfs); return true; }); } diff --git a/hudi-utilities/src/test/resources/delta-streamer-config/indexer.properties b/hudi-utilities/src/test/resources/delta-streamer-config/indexer.properties index 08a1e0f109ddb..aa42f8fd27057 100644 --- a/hudi-utilities/src/test/resources/delta-streamer-config/indexer.properties +++ b/hudi-utilities/src/test/resources/delta-streamer-config/indexer.properties @@ -17,6 +17,7 @@ # under the License. # +hoodie.metadata.enable=true hoodie.metadata.index.async=true hoodie.metadata.index.column.stats.enable=true hoodie.metadata.index.check.timeout.seconds=60 From a8ab116726030f2d0e6298c67b8415b8e461326e Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Tue, 29 Mar 2022 18:53:48 +0530 Subject: [PATCH 14/21] Cleanup and fix one bug in index catchup --- .../hudi/client/BaseHoodieWriteClient.java | 6 +----- .../HoodieBackedTableMetadataWriter.java | 2 -- .../action/index/RunIndexActionExecutor.java | 19 ++++++++++++++----- .../FlinkHoodieBackedTableMetadataWriter.java | 5 ----- .../SparkHoodieBackedTableMetadataWriter.java | 9 --------- 5 files changed, 15 insertions(+), 26 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java index a7aa74da25cbd..adc0445cd357a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java @@ -937,13 +937,9 @@ public boolean scheduleCompactionAtInstant(String instantTime, Option scheduleIndexing(List partitionTypes) { String instantTime = HoodieActiveTimeline.createNewInstantTime(); - return scheduleIndexingAtInstant(partitionTypes, instantTime) ? Option.of(instantTime) : Option.empty(); - } - - private boolean scheduleIndexingAtInstant(List partitionTypes, String instantTime) throws HoodieIOException { Option indexPlan = createTable(config, hadoopConf, config.isMetadataTableEnabled()) .scheduleIndex(context, instantTime, partitionTypes); - return indexPlan.isPresent(); + return indexPlan.isPresent() ? Option.of(instantTime) : Option.empty(); } /** diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index f27aac391973c..b1a8bdf4ce971 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -341,8 +341,6 @@ protected abstract void initialize(HoodieEngineCo Option actionMetadata, Option inflightInstantTimestamp); - protected abstract void scheduleIndex(List partitionTypes); - public void initTableMetadata() { try { if (this.metadata != null) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java index df59d69731b05..1f2c78296bbc5 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java @@ -137,7 +137,7 @@ public Option execute() { currentIndexedInstant = indexUptoInstant; ExecutorService executorService = Executors.newFixedThreadPool(MAX_CONCURRENT_INDEXING); Future indexingCatchupTaskFuture = executorService.submit( - new IndexingCatchupTask(metadataWriter, instantsToIndex, metadataCompletedTimestamps, table.getMetaClient())); + new IndexingCatchupTask(metadataWriter, instantsToIndex, metadataCompletedTimestamps, table.getMetaClient(), metadataMetaClient)); try { LOG.info("Starting index catchup task"); indexingCatchupTaskFuture.get(config.getIndexingCheckTimeoutSeconds(), TimeUnit.SECONDS); @@ -216,28 +216,31 @@ class IndexingCatchupTask implements Runnable { private final List instantsToIndex; private final Set metadataCompletedInstants; private final HoodieTableMetaClient metaClient; + private final HoodieTableMetaClient metadataMetaClient; IndexingCatchupTask(HoodieTableMetadataWriter metadataWriter, List instantsToIndex, Set metadataCompletedInstants, - HoodieTableMetaClient metaClient) { + HoodieTableMetaClient metaClient, + HoodieTableMetaClient metadataMetaClient) { this.metadataWriter = metadataWriter; this.instantsToIndex = instantsToIndex; this.metadataCompletedInstants = metadataCompletedInstants; this.metaClient = metaClient; + this.metadataMetaClient = metadataMetaClient; } @Override public void run() { for (HoodieInstant instant : instantsToIndex) { // metadata index already updated for this instant - if (metadataCompletedInstants.contains(instant.getTimestamp())) { + if (!metadataCompletedInstants.isEmpty() && metadataCompletedInstants.contains(instant.getTimestamp())) { currentIndexedInstant = instant.getTimestamp(); continue; } while (!instant.isCompleted()) { try { - LOG.info("instant not completed, reloading timeline " + instant); + LOG.warn("instant not completed, reloading timeline " + instant); // reload timeline and fetch instant details again wait until timeout String instantTime = instant.getTimestamp(); Option currentInstant = metaClient.reloadActiveTimeline() @@ -249,8 +252,14 @@ public void run() { throw new HoodieIndexException(String.format("Thread interrupted while running indexing check for instant: %s", instant), e); } } - // update metadata for this completed instant + // if instant completed, ensure that there was metadata commit, else update metadata for this completed instant if (COMPLETED.equals(instant.getState())) { + String instantTime = instant.getTimestamp(); + Option metadataInstant = metadataMetaClient.reloadActiveTimeline() + .filterCompletedInstants().filter(i -> i.getTimestamp().equals(instantTime)).firstInstant(); + if (metadataInstant.isPresent()) { + continue; + } try { // we need take a lock here as inflight writer could also try to update the timeline txnManager.beginTransaction(Option.of(instant), Option.empty()); diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java index 5b2ef8cf65667..9351ccf178075 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java @@ -99,11 +99,6 @@ protected void initialize(HoodieEngineContext eng } } - @Override - protected void scheduleIndex(List partitionsTypes) { - throw new UnsupportedOperationException("Metadata indexing not supported for Flink table yet."); - } - @Override protected void commit(String instantTime, Map> partitionRecordsMap, boolean canTriggerTableService) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java index 6e279bb5b201a..d0173f984a2f0 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/metadata/SparkHoodieBackedTableMetadataWriter.java @@ -121,15 +121,6 @@ protected void initialize(HoodieEngineContext eng } } - @Override - protected void scheduleIndex(List partitionTypes) { - ValidationUtils.checkState(metadataMetaClient != null, "Metadata table is not fully initialized yet."); - ValidationUtils.checkState(enabled, "Metadata table cannot be indexed as it is not enabled."); - try (SparkRDDWriteClient writeClient = new SparkRDDWriteClient(engineContext, metadataWriteConfig, true)) { - writeClient.scheduleIndexing(partitionTypes); - } - } - @Override protected void commit(String instantTime, Map> partitionRecordsMap, boolean canTriggerTableService) { ValidationUtils.checkState(metadataMetaClient != null, "Metadata table is not fully initialized yet."); From d25a8fb0ad02d123b7918726357f7e1d223f6230 Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Tue, 29 Mar 2022 21:23:18 +0530 Subject: [PATCH 15/21] Update table configs for files partition --- .../HoodieBackedTableMetadataWriter.java | 72 ++++++++++++------- 1 file changed, 47 insertions(+), 25 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index b1a8bdf4ce971..0dedb5c3e8fc9 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -523,15 +523,33 @@ private boolean initializeFromFilesystem(HoodieTableMetaClient dataMetaClient, initializeMetaClient(dataWriteConfig.getMetadataConfig().populateMetaFields()); initTableMetadata(); - initializeEnabledFileGroups(dataMetaClient, createInstantTime); + // if async metadata indexing is enabled, + // then only initialize files partition as other partitions will be built using HoodieIndexer + List enabledPartitionTypes = new ArrayList<>(); + if (dataWriteConfig.isMetadataAsyncIndex()) { + enabledPartitionTypes.add(MetadataPartitionType.FILES); + } else { + // all enabled ones should be initialized + enabledPartitionTypes = this.enabledPartitionTypes; + } + initializeEnabledFileGroups(dataMetaClient, createInstantTime, enabledPartitionTypes); // During cold startup, the list of files to be committed can be huge. So creating a HoodieCommitMetadata out // of these large number of files and calling the existing update(HoodieCommitMetadata) function does not scale // well. Hence, we have a special commit just for the initialization scenario. - initialCommit(createInstantTime); + initialCommit(createInstantTime, enabledPartitionTypes); + updateCompletedIndexesInTableConfig(enabledPartitionTypes); return true; } + private void updateCompletedIndexesInTableConfig(List partitionTypes) { + Set completedIndexes = Stream.of(dataMetaClient.getTableConfig().getCompletedMetadataIndexes().split(",")) + .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet()); + completedIndexes.addAll(partitionTypes.stream().map(MetadataPartitionType::getPartitionPath).collect(Collectors.toList())); + dataMetaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_INDEX_COMPLETED.key(), String.join(",", completedIndexes)); + HoodieTableConfig.update(dataMetaClient.getFs(), new Path(dataMetaClient.getMetaPath()), dataMetaClient.getTableConfig().getProps()); + } + private HoodieTableMetaClient initializeMetaClient(boolean populatMetaFields) throws IOException { return HoodieTableMetaClient.withPropertyBuilder() .setTableType(HoodieTableType.MERGE_ON_READ) @@ -607,8 +625,8 @@ private List listAllPartitions(HoodieTableMetaClient datasetMetaC * @param createInstantTime - Metadata table create instant time * @throws IOException */ - private void initializeEnabledFileGroups(HoodieTableMetaClient dataMetaClient, String createInstantTime) throws IOException { - for (MetadataPartitionType enabledPartitionType : this.enabledPartitionTypes) { + private void initializeEnabledFileGroups(HoodieTableMetaClient dataMetaClient, String createInstantTime, List partitionTypes) throws IOException { + for (MetadataPartitionType enabledPartitionType : partitionTypes) { initializeFileGroups(dataMetaClient, enabledPartitionType, createInstantTime, enabledPartitionType.getFileGroupCount()); } @@ -749,6 +767,7 @@ public void buildIndex(HoodieEngineContext engineContext, List partitionTypes = new ArrayList<>(); indexPartitionInfos.forEach(indexPartitionInfo -> { String relativePartitionPath = indexPartitionInfo.getMetadataPartitionPath(); LOG.info(String.format("Creating a new metadata index for partition '%s' under path %s upto instant %s", @@ -769,6 +788,7 @@ public void buildIndex(HoodieEngineContext engineContext, List inflightIndexes = Stream.of(dataMetaClient.getTableConfig().getInflightMetadataIndexes().split(",")) @@ -776,7 +796,7 @@ public void buildIndex(HoodieEngineContext engineContext, List partitionTypes) { // List all partitions in the basePath of the containing dataset LOG.info("Initializing metadata table by using file listings in " + dataWriteConfig.getBasePath()); engineContext.setJobStatus(this.getClass().getSimpleName(), "Initializing metadata table by listing files and partitions"); @@ -988,32 +1008,34 @@ private void initialCommit(String createInstantTime) { return; } - HoodieData filesPartitionRecords = engineContext.parallelize(Arrays.asList(allPartitionRecord), 1); - if (!partitionInfoList.isEmpty()) { - HoodieData fileListRecords = engineContext.parallelize(partitionInfoList, partitionInfoList.size()).map(partitionInfo -> { - Map fileNameToSizeMap = partitionInfo.getFileNameToSizeMap(); - // filter for files that are part of the completed commits - Map validFileNameToSizeMap = fileNameToSizeMap.entrySet().stream().filter(fileSizePair -> { - String commitTime = FSUtils.getCommitTime(fileSizePair.getKey()); - return HoodieTimeline.compareTimestamps(commitTime, HoodieTimeline.LESSER_THAN_OR_EQUALS, createInstantTime); - }).collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - - // Record which saves files within a partition - return HoodieMetadataPayload.createPartitionFilesRecord( - HoodieTableMetadataUtil.getPartition(partitionInfo.getRelativePath()), Option.of(validFileNameToSizeMap), Option.empty()); - }); - filesPartitionRecords = filesPartitionRecords.union(fileListRecords); + if (partitionTypes.contains(MetadataPartitionType.FILES)) { + HoodieData filesPartitionRecords = engineContext.parallelize(Arrays.asList(allPartitionRecord), 1); + if (!partitionInfoList.isEmpty()) { + HoodieData fileListRecords = engineContext.parallelize(partitionInfoList, partitionInfoList.size()).map(partitionInfo -> { + Map fileNameToSizeMap = partitionInfo.getFileNameToSizeMap(); + // filter for files that are part of the completed commits + Map validFileNameToSizeMap = fileNameToSizeMap.entrySet().stream().filter(fileSizePair -> { + String commitTime = FSUtils.getCommitTime(fileSizePair.getKey()); + return HoodieTimeline.compareTimestamps(commitTime, HoodieTimeline.LESSER_THAN_OR_EQUALS, createInstantTime); + }).collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + // Record which saves files within a partition + return HoodieMetadataPayload.createPartitionFilesRecord( + HoodieTableMetadataUtil.getPartition(partitionInfo.getRelativePath()), Option.of(validFileNameToSizeMap), Option.empty()); + }); + filesPartitionRecords = filesPartitionRecords.union(fileListRecords); + } + ValidationUtils.checkState(filesPartitionRecords.count() == (partitions.size() + 1)); + partitionToRecordsMap.put(MetadataPartitionType.FILES, filesPartitionRecords); } - ValidationUtils.checkState(filesPartitionRecords.count() == (partitions.size() + 1)); - partitionToRecordsMap.put(MetadataPartitionType.FILES, filesPartitionRecords); - if (enabledPartitionTypes.contains(MetadataPartitionType.BLOOM_FILTERS)) { + if (partitionTypes.contains(MetadataPartitionType.BLOOM_FILTERS)) { final HoodieData recordsRDD = HoodieTableMetadataUtil.convertFilesToBloomFilterRecords( engineContext, Collections.emptyMap(), partitionToFilesMap, getRecordsGenerationParams(), createInstantTime); partitionToRecordsMap.put(MetadataPartitionType.BLOOM_FILTERS, recordsRDD); } - if (enabledPartitionTypes.contains(MetadataPartitionType.COLUMN_STATS)) { + if (partitionTypes.contains(MetadataPartitionType.COLUMN_STATS)) { final HoodieData recordsRDD = HoodieTableMetadataUtil.convertFilesToColumnStatsRecords( engineContext, Collections.emptyMap(), partitionToFilesMap, getRecordsGenerationParams()); partitionToRecordsMap.put(MetadataPartitionType.COLUMN_STATS, recordsRDD); From 010de76ddd6c0201db746a13a5b04fc5e94125d4 Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Tue, 29 Mar 2022 22:51:04 +0530 Subject: [PATCH 16/21] Fix processAndCommit to consider partitions from table config --- .../HoodieBackedTableMetadataWriter.java | 25 +++++++++++-------- 1 file changed, 14 insertions(+), 11 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index 0dedb5c3e8fc9..3e21791ddb315 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -550,7 +550,7 @@ private void updateCompletedIndexesInTableConfig(List par HoodieTableConfig.update(dataMetaClient.getFs(), new Path(dataMetaClient.getMetaPath()), dataMetaClient.getTableConfig().getProps()); } - private HoodieTableMetaClient initializeMetaClient(boolean populatMetaFields) throws IOException { + private HoodieTableMetaClient initializeMetaClient(boolean populateMetaFields) throws IOException { return HoodieTableMetaClient.withPropertyBuilder() .setTableType(HoodieTableType.MERGE_ON_READ) .setTableName(tableName) @@ -558,7 +558,7 @@ private HoodieTableMetaClient initializeMetaClient(boolean populatMetaFields) th .setPayloadClassName(HoodieMetadataPayload.class.getName()) .setBaseFileFormat(HoodieFileFormat.HFILE.toString()) .setRecordKeyFields(RECORD_KEY_FIELD_NAME) - .setPopulateMetaFields(populatMetaFields) + .setPopulateMetaFields(populateMetaFields) .setKeyGeneratorClassProp(HoodieTableMetadataKeyGenerator.class.getCanonicalName()) .initTable(hadoopConf.get(), metadataWriteConfig.getBasePath()); } @@ -737,26 +737,29 @@ private void processAndCommit(String instantTime, ConvertMetadataFunction co Set partitionsToUpdate = getMetadataPartitionsToUpdate(); Set inflightIndexes = Stream.of(dataMetaClient.getTableConfig().getInflightMetadataIndexes().split(",")) .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet()); - partitionsToUpdate.forEach(p -> { - if (enabled && metadata != null) { - Map> partitionRecordsMap = convertMetadataFunction.convertMetadata(); - // if indexing is inflight then don't trigger table service - commit(instantTime, partitionRecordsMap, !inflightIndexes.contains(p) && canTriggerTableService); - } - }); + // if indexing is inflight then do not trigger table service + boolean doNotTriggerTableService = partitionsToUpdate.stream().anyMatch(inflightIndexes::contains); + + if (enabled && metadata != null) { + // convert metadata and filter only the entries whose partition path are in partitionsToUpdate + Map> partitionRecordsMap = convertMetadataFunction.convertMetadata().entrySet().stream() + .filter(entry -> partitionsToUpdate.contains(entry.getKey().getPartitionPath())).collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + commit(instantTime, partitionRecordsMap, !doNotTriggerTableService && canTriggerTableService); + } } private Set getMetadataPartitionsToUpdate() { // fetch partitions to update from table config Set partitionsToUpdate = Stream.of(dataMetaClient.getTableConfig().getCompletedMetadataIndexes().split(",")) .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet()); + // add inflight indexes as well because the file groups have already been initialized, so writers can log updates partitionsToUpdate.addAll(Stream.of(dataMetaClient.getTableConfig().getInflightMetadataIndexes().split(",")) .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet())); if (!partitionsToUpdate.isEmpty()) { return partitionsToUpdate; } - // fallback to update files partition only if table config returned no partitions - partitionsToUpdate.add(MetadataPartitionType.FILES.getPartitionPath()); + // fallback to all enabled partitions if table config returned no partitions + partitionsToUpdate.addAll(getEnabledPartitionTypes().stream().map(MetadataPartitionType::getPartitionPath).collect(Collectors.toList())); return partitionsToUpdate; } From a3ee4cd75320e578235cea4490ed7470bb721ea5 Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Wed, 30 Mar 2022 21:50:49 +0530 Subject: [PATCH 17/21] Tidying up, renames, refactoring --- .../hudi/client/BaseHoodieWriteClient.java | 4 +- .../HoodieBackedTableMetadataWriter.java | 99 ++++++++------- .../metadata/HoodieTableMetadataWriter.java | 16 +-- .../org/apache/hudi/table/HoodieTable.java | 2 +- .../action/index/RunIndexActionExecutor.java | 116 ++++++++++-------- .../index/ScheduleIndexActionExecutor.java | 46 +++---- .../upgrade/ThreeToFourUpgradeHandler.java | 4 +- .../table/HoodieFlinkCopyOnWriteTable.java | 2 +- .../table/HoodieJavaCopyOnWriteTable.java | 2 +- .../table/HoodieSparkCopyOnWriteTable.java | 2 +- .../common/config/HoodieMetadataConfig.java | 14 +-- .../hudi/common/table/HoodieTableConfig.java | 25 ++-- .../table/timeline/HoodieActiveTimeline.java | 2 +- .../common/table/timeline/HoodieTimeline.java | 2 +- .../apache/hudi/common/util/StringUtils.java | 34 ++++- .../metadata/HoodieTableMetadataUtil.java | 9 ++ .../hudi/common/util/TestStringUtils.java | 22 ++++ 17 files changed, 241 insertions(+), 160 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java index adc0445cd357a..1ebda4beafbfb 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java @@ -938,7 +938,7 @@ public boolean scheduleCompactionAtInstant(String instantTime, Option scheduleIndexing(List partitionTypes) { String instantTime = HoodieActiveTimeline.createNewInstantTime(); Option indexPlan = createTable(config, hadoopConf, config.isMetadataTableEnabled()) - .scheduleIndex(context, instantTime, partitionTypes); + .scheduleIndexing(context, instantTime, partitionTypes); return indexPlan.isPresent() ? Option.of(instantTime) : Option.empty(); } @@ -965,7 +965,7 @@ public void dropIndex(List partitionTypes) { context.setJobStatus(this.getClass().getSimpleName(), "Dropping partitions from metadata table"); table.getMetadataWriter(dropInstant).ifPresent(w -> { try { - ((HoodieTableMetadataWriter) w).dropIndex(partitionTypes); + ((HoodieTableMetadataWriter) w).dropMetadataPartitions(partitionTypes); } catch (IOException e) { throw new HoodieIndexException("Failed to drop metadata index. ", e); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index 3e21791ddb315..a1479c9815ddc 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -55,6 +55,7 @@ import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieCompactionConfig; @@ -88,12 +89,13 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; -import java.util.stream.Stream; import static org.apache.hudi.common.table.HoodieTableConfig.ARCHIVELOG_FOLDER; import static org.apache.hudi.common.util.StringUtils.EMPTY_STRING; import static org.apache.hudi.metadata.HoodieTableMetadata.METADATA_TABLE_NAME_SUFFIX; import static org.apache.hudi.metadata.HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP; +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getCompletedMetadataPartitions; +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getInflightMetadataPartitions; /** * Writer implementation backed by an internal hudi table. Partition and file listing are saved within an internal MOR table @@ -368,7 +370,7 @@ protected void initializeIfNeeded(HoodieTableMeta Option inflightInstantTimestamp) throws IOException { HoodieTimer timer = new HoodieTimer().startTimer(); - boolean exists = metadataExists(dataMetaClient, actionMetadata); + boolean exists = metadataTableExists(dataMetaClient, actionMetadata); if (!exists) { // Initialize for the first time by listing partitions and files directly from the file system @@ -378,8 +380,8 @@ protected void initializeIfNeeded(HoodieTableMeta } } - private boolean metadataExists(HoodieTableMetaClient dataMetaClient, - Option actionMetadata) throws IOException { + private boolean metadataTableExists(HoodieTableMetaClient dataMetaClient, + Option actionMetadata) throws IOException { boolean exists = dataMetaClient.getFs().exists(new Path(metadataWriteConfig.getBasePath(), HoodieTableMetaClient.METAFOLDER_NAME)); boolean reInitialize = false; @@ -538,15 +540,14 @@ private boolean initializeFromFilesystem(HoodieTableMetaClient dataMetaClient, // of these large number of files and calling the existing update(HoodieCommitMetadata) function does not scale // well. Hence, we have a special commit just for the initialization scenario. initialCommit(createInstantTime, enabledPartitionTypes); - updateCompletedIndexesInTableConfig(enabledPartitionTypes); + updateInitializedPartitionsInTableConfig(enabledPartitionTypes); return true; } - private void updateCompletedIndexesInTableConfig(List partitionTypes) { - Set completedIndexes = Stream.of(dataMetaClient.getTableConfig().getCompletedMetadataIndexes().split(",")) - .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet()); - completedIndexes.addAll(partitionTypes.stream().map(MetadataPartitionType::getPartitionPath).collect(Collectors.toList())); - dataMetaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_INDEX_COMPLETED.key(), String.join(",", completedIndexes)); + private void updateInitializedPartitionsInTableConfig(List partitionTypes) { + Set completedIndexes = getCompletedMetadataPartitions(dataMetaClient.getTableConfig()); + completedIndexes.addAll(partitionTypes.stream().map(MetadataPartitionType::getPartitionPath).collect(Collectors.toSet())); + dataMetaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_PARTITIONS.key(), String.join(",", completedIndexes)); HoodieTableConfig.update(dataMetaClient.getFs(), new Path(dataMetaClient.getMetaPath()), dataMetaClient.getTableConfig().getProps()); } @@ -632,7 +633,7 @@ private void initializeEnabledFileGroups(HoodieTableMetaClient dataMetaClient, S } } - public void scheduleIndex(HoodieTableMetaClient dataMetaClient, List metadataPartitions, String instantTime) throws IOException { + public void initializeMetadataPartitions(HoodieTableMetaClient dataMetaClient, List metadataPartitions, String instantTime) throws IOException { for (MetadataPartitionType partitionType : metadataPartitions) { initializeFileGroups(dataMetaClient, partitionType, instantTime, partitionType.getFileGroupCount()); } @@ -680,21 +681,19 @@ private void initializeFileGroups(HoodieTableMetaClient dataMetaClient, Metadata } } - public void dropIndex(List indexesToDrop) throws IOException { - Set completedIndexes = Stream.of(dataMetaClient.getTableConfig().getCompletedMetadataIndexes().split(",")) - .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet()); - Set inflightIndexes = Stream.of(dataMetaClient.getTableConfig().getInflightMetadataIndexes().split(",")) - .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet()); + public void dropMetadataPartitions(List metadataPartitions) throws IOException { + Set completedIndexes = getCompletedMetadataPartitions(dataMetaClient.getTableConfig()); + Set inflightIndexes = getInflightMetadataPartitions(dataMetaClient.getTableConfig()); - for (MetadataPartitionType partitionType : indexesToDrop) { + for (MetadataPartitionType partitionType : metadataPartitions) { String partitionPath = partitionType.getPartitionPath(); // first update table config if (inflightIndexes.contains(partitionPath)) { inflightIndexes.remove(partitionPath); - dataMetaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_INDEX_INFLIGHT.key(), String.join(",", inflightIndexes)); + dataMetaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_PARTITIONS_INFLIGHT.key(), String.join(",", inflightIndexes)); } else if (completedIndexes.contains(partitionPath)) { completedIndexes.remove(partitionPath); - dataMetaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_INDEX_COMPLETED.key(), String.join(",", completedIndexes)); + dataMetaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_PARTITIONS.key(), String.join(",", completedIndexes)); } HoodieTableConfig.update(dataMetaClient.getFs(), new Path(dataMetaClient.getMetaPath()), dataMetaClient.getTableConfig().getProps()); LOG.warn("Deleting Metadata Table partitions: " + partitionPath); @@ -708,10 +707,8 @@ private MetadataRecordsGenerationParams getRecordsGenerationParams() { dataWriteConfig.getBloomIndexParallelism(), dataWriteConfig.isMetadataColumnStatsIndexEnabled(), dataWriteConfig.getColumnStatsIndexParallelism(), - Stream.of(dataWriteConfig.getColumnsEnabledForColumnStatsIndex().split(",")) - .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList()), - Stream.of(dataWriteConfig.getColumnsEnabledForBloomFilterIndex().split(",")) - .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList())); + StringUtils.toList(dataWriteConfig.getColumnsEnabledForColumnStatsIndex()), + StringUtils.toList(dataWriteConfig.getColumnsEnabledForBloomFilterIndex())); } /** @@ -735,8 +732,7 @@ private void processAndCommit(String instantTime, ConvertMetadataFunction co return; } Set partitionsToUpdate = getMetadataPartitionsToUpdate(); - Set inflightIndexes = Stream.of(dataMetaClient.getTableConfig().getInflightMetadataIndexes().split(",")) - .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet()); + Set inflightIndexes = getInflightMetadataPartitions(dataMetaClient.getTableConfig()); // if indexing is inflight then do not trigger table service boolean doNotTriggerTableService = partitionsToUpdate.stream().anyMatch(inflightIndexes::contains); @@ -750,21 +746,18 @@ private void processAndCommit(String instantTime, ConvertMetadataFunction co private Set getMetadataPartitionsToUpdate() { // fetch partitions to update from table config - Set partitionsToUpdate = Stream.of(dataMetaClient.getTableConfig().getCompletedMetadataIndexes().split(",")) - .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet()); + Set partitionsToUpdate = getCompletedMetadataPartitions(dataMetaClient.getTableConfig()); // add inflight indexes as well because the file groups have already been initialized, so writers can log updates - partitionsToUpdate.addAll(Stream.of(dataMetaClient.getTableConfig().getInflightMetadataIndexes().split(",")) - .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet())); + partitionsToUpdate.addAll(getInflightMetadataPartitions(dataMetaClient.getTableConfig())); if (!partitionsToUpdate.isEmpty()) { return partitionsToUpdate; } // fallback to all enabled partitions if table config returned no partitions - partitionsToUpdate.addAll(getEnabledPartitionTypes().stream().map(MetadataPartitionType::getPartitionPath).collect(Collectors.toList())); - return partitionsToUpdate; + return getEnabledPartitionTypes().stream().map(MetadataPartitionType::getPartitionPath).collect(Collectors.toSet()); } @Override - public void buildIndex(HoodieEngineContext engineContext, List indexPartitionInfos) { + public void buildMetadataPartitions(HoodieEngineContext engineContext, List indexPartitionInfos) { if (indexPartitionInfos.isEmpty()) { LOG.warn("No partition to index in the plan"); return; @@ -794,10 +787,9 @@ public void buildIndex(HoodieEngineContext engineContext, List inflightIndexes = Stream.of(dataMetaClient.getTableConfig().getInflightMetadataIndexes().split(",")) - .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet()); + Set inflightIndexes = getInflightMetadataPartitions(dataMetaClient.getTableConfig()); inflightIndexes.addAll(indexPartitionInfos.stream().map(HoodieIndexPartitionInfo::getMetadataPartitionPath).collect(Collectors.toSet())); - dataMetaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_INDEX_INFLIGHT.key(), String.join(",", inflightIndexes)); + dataMetaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_PARTITIONS_INFLIGHT.key(), String.join(",", inflightIndexes)); HoodieTableConfig.update(dataMetaClient.getFs(), new Path(dataMetaClient.getMetaPath()), dataMetaClient.getTableConfig().getProps()); initialCommit(indexUptoInstantTime, partitionTypes); } @@ -1012,22 +1004,7 @@ private void initialCommit(String createInstantTime, List } if (partitionTypes.contains(MetadataPartitionType.FILES)) { - HoodieData filesPartitionRecords = engineContext.parallelize(Arrays.asList(allPartitionRecord), 1); - if (!partitionInfoList.isEmpty()) { - HoodieData fileListRecords = engineContext.parallelize(partitionInfoList, partitionInfoList.size()).map(partitionInfo -> { - Map fileNameToSizeMap = partitionInfo.getFileNameToSizeMap(); - // filter for files that are part of the completed commits - Map validFileNameToSizeMap = fileNameToSizeMap.entrySet().stream().filter(fileSizePair -> { - String commitTime = FSUtils.getCommitTime(fileSizePair.getKey()); - return HoodieTimeline.compareTimestamps(commitTime, HoodieTimeline.LESSER_THAN_OR_EQUALS, createInstantTime); - }).collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - - // Record which saves files within a partition - return HoodieMetadataPayload.createPartitionFilesRecord( - HoodieTableMetadataUtil.getPartition(partitionInfo.getRelativePath()), Option.of(validFileNameToSizeMap), Option.empty()); - }); - filesPartitionRecords = filesPartitionRecords.union(fileListRecords); - } + HoodieData filesPartitionRecords = getFilesPartitionRecords(createInstantTime, partitionInfoList, allPartitionRecord); ValidationUtils.checkState(filesPartitionRecords.count() == (partitions.size() + 1)); partitionToRecordsMap.put(MetadataPartitionType.FILES, filesPartitionRecords); } @@ -1048,6 +1025,26 @@ private void initialCommit(String createInstantTime, List commit(createInstantTime, partitionToRecordsMap, false); } + private HoodieData getFilesPartitionRecords(String createInstantTime, List partitionInfoList, HoodieRecord allPartitionRecord) { + HoodieData filesPartitionRecords = engineContext.parallelize(Arrays.asList(allPartitionRecord), 1); + if (!partitionInfoList.isEmpty()) { + HoodieData fileListRecords = engineContext.parallelize(partitionInfoList, partitionInfoList.size()).map(partitionInfo -> { + Map fileNameToSizeMap = partitionInfo.getFileNameToSizeMap(); + // filter for files that are part of the completed commits + Map validFileNameToSizeMap = fileNameToSizeMap.entrySet().stream().filter(fileSizePair -> { + String commitTime = FSUtils.getCommitTime(fileSizePair.getKey()); + return HoodieTimeline.compareTimestamps(commitTime, HoodieTimeline.LESSER_THAN_OR_EQUALS, createInstantTime); + }).collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + // Record which saves files within a partition + return HoodieMetadataPayload.createPartitionFilesRecord( + HoodieTableMetadataUtil.getPartition(partitionInfo.getRelativePath()), Option.of(validFileNameToSizeMap), Option.empty()); + }); + filesPartitionRecords = filesPartitionRecords.union(fileListRecords); + } + return filesPartitionRecords; + } + /** * A class which represents a directory and the files and directories inside it. *

    diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java index b83f5448cd491..777c785e263db 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java @@ -36,30 +36,30 @@ public interface HoodieTableMetadataWriter extends Serializable, AutoCloseable { /** - * Execute the index action for the given partitions. + * Builds the given metadata partitions to create index. * * @param engineContext - * @param indexPartitionInfos - partitions to index + * @param indexPartitionInfos - information about partitions to build such as partition type and base instant time */ - void buildIndex(HoodieEngineContext engineContext, List indexPartitionInfos); + void buildMetadataPartitions(HoodieEngineContext engineContext, List indexPartitionInfos); /** - * Initialize file groups while scheduling index action. + * Initialize file groups for the given metadata partitions when indexing is requested. * * @param dataMetaClient - meta client for the data table * @param metadataPartitions - metadata partitions for which file groups needs to be initialized * @param instantTime - instant time of the index action * @throws IOException */ - void scheduleIndex(HoodieTableMetaClient dataMetaClient, List metadataPartitions, String instantTime) throws IOException; + void initializeMetadataPartitions(HoodieTableMetaClient dataMetaClient, List metadataPartitions, String instantTime) throws IOException; /** - * Drop the given metadata indexes. + * Drop the given metadata partitions. * - * @param indexesToDrop + * @param metadataPartitions * @throws IOException */ - void dropIndex(List indexesToDrop) throws IOException; + void dropMetadataPartitions(List metadataPartitions) throws IOException; /** * Update the metadata table due to a COMMIT operation. diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java index 2831ca8d6e43d..6ee3e0c93d11d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/HoodieTable.java @@ -507,7 +507,7 @@ public abstract HoodieRollbackMetadata rollback(HoodieEngineContext context, * @param partitionsToIndex List of {@link MetadataPartitionType} that should be indexed. * @return HoodieIndexPlan containing metadata partitions and instant upto which they should be indexed. */ - public abstract Option scheduleIndex(HoodieEngineContext context, String indexInstantTime, List partitionsToIndex); + public abstract Option scheduleIndexing(HoodieEngineContext context, String indexInstantTime, List partitionsToIndex); /** * Execute requested index action. diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java index 1f2c78296bbc5..d689312b7c018 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java @@ -40,7 +40,6 @@ import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIndexException; -import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.BaseActionExecutor; @@ -57,13 +56,15 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; -import java.util.stream.Stream; import static org.apache.hudi.common.model.WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL; import static org.apache.hudi.common.table.timeline.HoodieInstant.State.COMPLETED; import static org.apache.hudi.common.table.timeline.HoodieInstant.State.REQUESTED; import static org.apache.hudi.common.table.timeline.HoodieTimeline.INDEX_ACTION; import static org.apache.hudi.config.HoodieWriteConfig.WRITE_CONCURRENCY_MODE; +import static org.apache.hudi.metadata.HoodieTableMetadata.getMetadataTableBasePath; +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getCompletedMetadataPartitions; +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getInflightMetadataPartitions; /** * Reads the index plan and executes the plan. @@ -75,6 +76,7 @@ public class RunIndexActionExecutor exte private static final Integer INDEX_COMMIT_METADATA_VERSION_1 = 1; private static final Integer LATEST_INDEX_COMMIT_METADATA_VERSION = INDEX_COMMIT_METADATA_VERSION_1; private static final int MAX_CONCURRENT_INDEXING = 1; + private static final int TIMELINE_RELOAD_INTERVAL_MILLIS = 5000; // we use this to update the latest instant in data timeline that has been indexed in metadata table // this needs to be volatile as it can be updated in the IndexingCheckTask spawned by this executor @@ -93,17 +95,7 @@ public Option execute() { HoodieTimer indexTimer = new HoodieTimer(); indexTimer.startTimer(); - // ensure lock provider configured - if (!config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl() || StringUtils.isNullOrEmpty(config.getLockProviderClass())) { - throw new HoodieIndexException(String.format("Need to set %s as %s and configure lock provider class", - WRITE_CONCURRENCY_MODE.key(), OPTIMISTIC_CONCURRENCY_CONTROL.name())); - } - - HoodieInstant indexInstant = table.getActiveTimeline() - .filterPendingIndexTimeline() - .filter(instant -> instant.getTimestamp().equals(instantTime) && REQUESTED.equals(instant.getState())) - .lastInstant() - .orElseThrow(() -> new HoodieIndexException(String.format("No requested index instant found: %s", instantTime))); + HoodieInstant indexInstant = validateAndGetIndexInstant(); try { // read HoodieIndexPlan HoodieIndexPlan indexPlan = TimelineMetadataUtils.deserializeIndexPlan(table.getActiveTimeline().readIndexPlanAsBytes(indexInstant).get()); @@ -117,36 +109,25 @@ public Option execute() { HoodieTableMetadataWriter metadataWriter = table.getMetadataWriter(instantTime) .orElseThrow(() -> new HoodieIndexException(String.format("Could not get metadata writer to run index action for instant: %s", instantTime))); // this will only build index upto base instant as generated by the plan, we will be doing catchup later - LOG.info("Starting Index Building"); - metadataWriter.buildIndex(context, indexPartitionInfos); + String indexUptoInstant = indexPartitionInfos.get(0).getIndexUptoInstant(); + LOG.info("Starting Index Building with base instant: " + indexUptoInstant); + metadataWriter.buildMetadataPartitions(context, indexPartitionInfos); // get all instants since the plan completed (both from active timeline and archived timeline) // assumption is that all metadata partitions had same instant upto which they were scheduled to be indexed table.getMetaClient().reloadActiveTimeline(); - String indexUptoInstant = indexPartitionInfos.get(0).getIndexUptoInstant(); List instantsToIndex = getRemainingArchivedAndActiveInstantsSince(indexUptoInstant, table.getMetaClient()); LOG.info("Total remaining instants to index: " + instantsToIndex.size()); // reconcile with metadata table timeline - String metadataBasePath = HoodieTableMetadata.getMetadataTableBasePath(table.getMetaClient().getBasePath()); + String metadataBasePath = getMetadataTableBasePath(table.getMetaClient().getBasePath()); HoodieTableMetaClient metadataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(metadataBasePath).build(); Set metadataCompletedTimestamps = getCompletedArchivedAndActiveInstantsAfter(indexUptoInstant, metadataMetaClient).stream() .map(HoodieInstant::getTimestamp).collect(Collectors.toSet()); // index catchup for all remaining instants with a timeout currentIndexedInstant = indexUptoInstant; - ExecutorService executorService = Executors.newFixedThreadPool(MAX_CONCURRENT_INDEXING); - Future indexingCatchupTaskFuture = executorService.submit( - new IndexingCatchupTask(metadataWriter, instantsToIndex, metadataCompletedTimestamps, table.getMetaClient(), metadataMetaClient)); - try { - LOG.info("Starting index catchup task"); - indexingCatchupTaskFuture.get(config.getIndexingCheckTimeoutSeconds(), TimeUnit.SECONDS); - } catch (Exception e) { - indexingCatchupTaskFuture.cancel(true); - throw new HoodieIndexException(String.format("Index catchup failed. Current indexed instant = %s. Aborting!", currentIndexedInstant), e); - } finally { - executorService.shutdownNow(); - } + catchupWithInflightWriters(metadataWriter, instantsToIndex, metadataMetaClient, metadataCompletedTimestamps); // save index commit metadata and update table config List finalIndexPartitionInfos = indexPartitionInfos.stream() .map(info -> new HoodieIndexPartitionInfo( @@ -156,22 +137,60 @@ public Option execute() { .collect(Collectors.toList()); HoodieIndexCommitMetadata indexCommitMetadata = HoodieIndexCommitMetadata.newBuilder() .setVersion(LATEST_INDEX_COMMIT_METADATA_VERSION).setIndexPartitionInfos(finalIndexPartitionInfos).build(); - try { - // update the table config and timeline in a lock as there could be another indexer running - txnManager.beginTransaction(); - updateTableConfig(table.getMetaClient(), finalIndexPartitionInfos); - table.getActiveTimeline().saveAsComplete( - new HoodieInstant(true, INDEX_ACTION, indexInstant.getTimestamp()), - TimelineMetadataUtils.serializeIndexCommitMetadata(indexCommitMetadata)); - } finally { - txnManager.endTransaction(); - } + updateTableConfigAndTimeline(indexInstant, finalIndexPartitionInfos, indexCommitMetadata); return Option.of(indexCommitMetadata); } catch (IOException e) { throw new HoodieIndexException(String.format("Unable to index instant: %s", indexInstant)); } } + private HoodieInstant validateAndGetIndexInstant() { + // ensure lock provider configured + if (!config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl() || StringUtils.isNullOrEmpty(config.getLockProviderClass())) { + throw new HoodieIndexException(String.format("Need to set %s as %s and configure lock provider class", + WRITE_CONCURRENCY_MODE.key(), OPTIMISTIC_CONCURRENCY_CONTROL.name())); + } + + HoodieInstant indexInstant = table.getActiveTimeline() + .filterPendingIndexTimeline() + .filter(instant -> instant.getTimestamp().equals(instantTime) && REQUESTED.equals(instant.getState())) + .lastInstant() + .orElseThrow(() -> new HoodieIndexException(String.format("No requested index instant found: %s", instantTime))); + return indexInstant; + } + + private void updateTableConfigAndTimeline(HoodieInstant indexInstant, + List finalIndexPartitionInfos, + HoodieIndexCommitMetadata indexCommitMetadata) throws IOException { + try { + // update the table config and timeline in a lock as there could be another indexer running + txnManager.beginTransaction(); + updateMetadataPartitionsTableConfig(table.getMetaClient(), + finalIndexPartitionInfos.stream().map(HoodieIndexPartitionInfo::getMetadataPartitionPath).collect(Collectors.toSet())); + table.getActiveTimeline().saveAsComplete( + new HoodieInstant(true, INDEX_ACTION, indexInstant.getTimestamp()), + TimelineMetadataUtils.serializeIndexCommitMetadata(indexCommitMetadata)); + } finally { + txnManager.endTransaction(); + } + } + + private void catchupWithInflightWriters(HoodieTableMetadataWriter metadataWriter, List instantsToIndex, + HoodieTableMetaClient metadataMetaClient, Set metadataCompletedTimestamps) { + ExecutorService executorService = Executors.newFixedThreadPool(MAX_CONCURRENT_INDEXING); + Future indexingCatchupTaskFuture = executorService.submit( + new IndexingCatchupTask(metadataWriter, instantsToIndex, metadataCompletedTimestamps, table.getMetaClient(), metadataMetaClient)); + try { + LOG.info("Starting index catchup task"); + indexingCatchupTaskFuture.get(config.getIndexingCheckTimeoutSeconds(), TimeUnit.SECONDS); + } catch (Exception e) { + indexingCatchupTaskFuture.cancel(true); + throw new HoodieIndexException(String.format("Index catchup failed. Current indexed instant = %s. Aborting!", currentIndexedInstant), e); + } finally { + executorService.shutdownNow(); + } + } + private static List getRemainingArchivedAndActiveInstantsSince(String instant, HoodieTableMetaClient metaClient) { List remainingInstantsToIndex = metaClient.getArchivedTimeline() .getWriteTimeline() @@ -190,18 +209,15 @@ private static List getCompletedArchivedAndActiveInstantsAfter(St return completedInstants; } - private void updateTableConfig(HoodieTableMetaClient metaClient, List indexPartitionInfos) { + private void updateMetadataPartitionsTableConfig(HoodieTableMetaClient metaClient, Set metadataPartitions) { // remove from inflight and update completed indexes - Set inflightIndexes = Stream.of(metaClient.getTableConfig().getInflightMetadataIndexes().split(",")) - .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet()); - Set completedIndexes = Stream.of(metaClient.getTableConfig().getCompletedMetadataIndexes().split(",")) - .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet()); - Set indexesRequested = indexPartitionInfos.stream().map(HoodieIndexPartitionInfo::getMetadataPartitionPath).collect(Collectors.toSet()); - inflightIndexes.removeAll(indexesRequested); - completedIndexes.addAll(indexesRequested); + Set inflightPartitions = getInflightMetadataPartitions(metaClient.getTableConfig()); + Set completedPartitions = getCompletedMetadataPartitions(metaClient.getTableConfig()); + inflightPartitions.removeAll(metadataPartitions); + completedPartitions.addAll(metadataPartitions); // update table config - metaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_INDEX_INFLIGHT.key(), String.join(",", inflightIndexes)); - metaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_INDEX_COMPLETED.key(), String.join(",", completedIndexes)); + metaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_PARTITIONS_INFLIGHT.key(), String.join(",", inflightPartitions)); + metaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_PARTITIONS.key(), String.join(",", completedPartitions)); HoodieTableConfig.update(metaClient.getFs(), new Path(metaClient.getMetaPath()), metaClient.getTableConfig().getProps()); } @@ -247,7 +263,7 @@ public void run() { .filterCompletedInstants().filter(i -> i.getTimestamp().equals(instantTime)).firstInstant(); instant = currentInstant.orElse(instant); // so that timeline is not reloaded very frequently - Thread.sleep(5000); + Thread.sleep(TIMELINE_RELOAD_INTERVAL_MILLIS); } catch (InterruptedException e) { throw new HoodieIndexException(String.format("Thread interrupted while running indexing check for instant: %s", instant), e); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java index f95d21d619e5d..5057bbb006131 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java @@ -24,6 +24,7 @@ import org.apache.hudi.client.transaction.TransactionManager; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; @@ -45,10 +46,11 @@ import java.util.List; import java.util.Set; import java.util.stream.Collectors; -import java.util.stream.Stream; import static org.apache.hudi.common.model.WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL; import static org.apache.hudi.config.HoodieWriteConfig.WRITE_CONCURRENCY_MODE; +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getCompletedMetadataPartitions; +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getInflightMetadataPartitions; /** * Schedules INDEX action. @@ -64,42 +66,33 @@ public class ScheduleIndexActionExecutor private static final Integer INDEX_PLAN_VERSION_1 = 1; private static final Integer LATEST_INDEX_PLAN_VERSION = INDEX_PLAN_VERSION_1; - private final List partitionsToIndex; + private final List partitionIndexTypes; private final TransactionManager txnManager; public ScheduleIndexActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, String instantTime, - List partitionsToIndex) { + List partitionIndexTypes) { super(context, config, table, instantTime); - this.partitionsToIndex = partitionsToIndex; + this.partitionIndexTypes = partitionIndexTypes; this.txnManager = new TransactionManager(config, table.getMetaClient().getFs()); } @Override public Option execute() { - // validate partitionsToIndex - if (!EnumSet.allOf(MetadataPartitionType.class).containsAll(partitionsToIndex)) { - throw new HoodieIndexException("Not all partitions are valid: " + partitionsToIndex); - } - // ensure lock provider configured - if (!config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl() || StringUtils.isNullOrEmpty(config.getLockProviderClass())) { - throw new HoodieIndexException(String.format("Need to set %s as %s and configure lock provider class", - WRITE_CONCURRENCY_MODE.key(), OPTIMISTIC_CONCURRENCY_CONTROL.name())); - } + validateBeforeScheduling(); // make sure that it is idempotent, check with previously pending index operations. - Set indexesInflightOrCompleted = Stream.of(table.getMetaClient().getTableConfig().getInflightMetadataIndexes().split(",")) - .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet()); - indexesInflightOrCompleted.addAll(Stream.of(table.getMetaClient().getTableConfig().getCompletedMetadataIndexes().split(",")) - .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet())); - Set requestedPartitions = partitionsToIndex.stream().map(MetadataPartitionType::getPartitionPath).collect(Collectors.toSet()); + HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig(); + Set indexesInflightOrCompleted = getInflightMetadataPartitions(tableConfig); + indexesInflightOrCompleted.addAll(getCompletedMetadataPartitions(tableConfig)); + Set requestedPartitions = partitionIndexTypes.stream().map(MetadataPartitionType::getPartitionPath).collect(Collectors.toSet()); requestedPartitions.removeAll(indexesInflightOrCompleted); if (!requestedPartitions.isEmpty()) { LOG.warn(String.format("Following partitions already exist or inflight: %s. Going to index only these partitions: %s", indexesInflightOrCompleted, requestedPartitions)); } - List finalPartitionsToIndex = partitionsToIndex.stream() + List finalPartitionsToIndex = partitionIndexTypes.stream() .filter(p -> requestedPartitions.contains(p.getPartitionPath())).collect(Collectors.toList()); // get last completed instant Option indexUptoInstant = table.getActiveTimeline().getContiguousCompletedWriteTimeline().lastInstant(); @@ -111,7 +104,7 @@ public Option execute() { .orElseThrow(() -> new HoodieIndexException(String.format("Could not get metadata writer to initialize filegroups for indexing for instant: %s", instantTime))); try { this.txnManager.beginTransaction(Option.of(indexInstant), Option.empty()); - metadataWriter.scheduleIndex(table.getMetaClient(), finalPartitionsToIndex, indexInstant.getTimestamp()); + metadataWriter.initializeMetadataPartitions(table.getMetaClient(), finalPartitionsToIndex, indexInstant.getTimestamp()); } catch (IOException e) { LOG.error("Could not initialize file groups", e); throw new HoodieIOException(e.getMessage(), e); @@ -125,7 +118,7 @@ public Option execute() { HoodieIndexPlan indexPlan = new HoodieIndexPlan(LATEST_INDEX_PLAN_VERSION, indexPartitionInfos); // update data timeline with requested instant try { - table.getActiveTimeline().saveToPendingIndexCommit(indexInstant, TimelineMetadataUtils.serializeIndexPlan(indexPlan)); + table.getActiveTimeline().saveToPendingIndexAction(indexInstant, TimelineMetadataUtils.serializeIndexPlan(indexPlan)); } catch (IOException e) { LOG.error("Error while saving index requested file", e); throw new HoodieIOException(e.getMessage(), e); @@ -135,4 +128,15 @@ public Option execute() { } return Option.empty(); } + + private void validateBeforeScheduling() { + if (!EnumSet.allOf(MetadataPartitionType.class).containsAll(partitionIndexTypes)) { + throw new HoodieIndexException("Not all index types are valid: " + partitionIndexTypes); + } + // ensure lock provider configured + if (!config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl() || StringUtils.isNullOrEmpty(config.getLockProviderClass())) { + throw new HoodieIndexException(String.format("Need to set %s as %s and configure lock provider class", + WRITE_CONCURRENCY_MODE.key(), OPTIMISTIC_CONCURRENCY_CONTROL.name())); + } + } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ThreeToFourUpgradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ThreeToFourUpgradeHandler.java index c5847bdd06c67..4da675ea82004 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ThreeToFourUpgradeHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/ThreeToFourUpgradeHandler.java @@ -29,7 +29,7 @@ import java.util.Map; import static org.apache.hudi.common.table.HoodieTableConfig.TABLE_CHECKSUM; -import static org.apache.hudi.common.table.HoodieTableConfig.TABLE_METADATA_INDEX_COMPLETED; +import static org.apache.hudi.common.table.HoodieTableConfig.TABLE_METADATA_PARTITIONS; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.metadataPartitionExists; /** @@ -44,7 +44,7 @@ public Map upgrade(HoodieWriteConfig config, HoodieEngin // if metadata is enabled and files partition exist then update TABLE_METADATA_INDEX_COMPLETED // schema for the files partition is same between the two versions if (config.isMetadataTableEnabled() && metadataPartitionExists(config.getBasePath(), context, MetadataPartitionType.FILES)) { - tablePropsToAdd.put(TABLE_METADATA_INDEX_COMPLETED, MetadataPartitionType.FILES.getPartitionPath()); + tablePropsToAdd.put(TABLE_METADATA_PARTITIONS, MetadataPartitionType.FILES.getPartitionPath()); } return tablePropsToAdd; } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java index 5e25945bb8915..bb4678bc5cab6 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkCopyOnWriteTable.java @@ -343,7 +343,7 @@ public HoodieRollbackMetadata rollback(HoodieEngineContext context, String rollb } @Override - public Option scheduleIndex(HoodieEngineContext context, String indexInstantTime, List partitionsToIndex) { + public Option scheduleIndexing(HoodieEngineContext context, String indexInstantTime, List partitionsToIndex) { throw new HoodieNotSupportedException("Metadata indexing is not supported for a Flink table yet."); } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java index b0746a833c67e..ff3d29b0b9bb3 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaCopyOnWriteTable.java @@ -247,7 +247,7 @@ public HoodieRollbackMetadata rollback(HoodieEngineContext context, } @Override - public Option scheduleIndex(HoodieEngineContext context, String indexInstantTime, List partitionsToIndex) { + public Option scheduleIndexing(HoodieEngineContext context, String indexInstantTime, List partitionsToIndex) { return new ScheduleIndexActionExecutor<>(context, config, this, indexInstantTime, partitionsToIndex).execute(); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java index 24855da88ab3d..1d9ef7add9eb2 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java @@ -352,7 +352,7 @@ public HoodieRollbackMetadata rollback(HoodieEngineContext context, String rollb } @Override - public Option scheduleIndex(HoodieEngineContext context, String indexInstantTime, List partitionsToIndex) { + public Option scheduleIndexing(HoodieEngineContext context, String indexInstantTime, List partitionsToIndex) { return new ScheduleIndexActionExecutor<>(context, config, this, indexInstantTime, partitionsToIndex).execute(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java index e5cd6d76ecc0c..9dbdf6d64d9b6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieMetadataConfig.java @@ -183,23 +183,23 @@ public final class HoodieMetadataConfig extends HoodieConfig { .withDocumentation("Parallelism to use, when generating column stats index."); public static final ConfigProperty COLUMN_STATS_INDEX_FOR_COLUMNS = ConfigProperty - .key(METADATA_PREFIX + ".index.column.stats.for.columns") - .defaultValue("") + .key(METADATA_PREFIX + ".index.column.stats.column.list") + .noDefaultValue() .sinceVersion("0.11.0") .withDocumentation("Comma-separated list of columns for which column stats index will be built. If not set, all columns will be indexed"); public static final ConfigProperty BLOOM_FILTER_INDEX_FOR_COLUMNS = ConfigProperty - .key(METADATA_PREFIX + ".index.bloom.filter.for.columns") - .defaultValue("") + .key(METADATA_PREFIX + ".index.bloom.filter.column.list") + .noDefaultValue() .sinceVersion("0.11.0") .withDocumentation("Comma-separated list of columns for which bloom filter index will be built. If not set, only record key will be indexed."); public static final ConfigProperty METADATA_INDEX_CHECK_TIMEOUT_SECONDS = ConfigProperty .key(METADATA_PREFIX + ".index.check.timeout.seconds") - .defaultValue(300) + .defaultValue(900) .sinceVersion("0.11.0") - .withDocumentation("After the async indexer has finished indexing upto the base instant, it will reconcile with commits that happened after the base instant. " - + "This check could take finite amount of time depending on number of commits, so it needs to be bounded by a timeout which can configured with this key."); + .withDocumentation("After the async indexer has finished indexing upto the base instant, it will ensure that all inflight writers " + + "reliably write index updates as well. If this timeout expires, then the indexer will abort itself safely."); public static final ConfigProperty POPULATE_META_FIELDS = ConfigProperty .key(METADATA_PREFIX + ".populate.meta.fields") diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java index aeff72114291b..923ee274556f6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java @@ -36,6 +36,7 @@ import org.apache.hudi.common.util.BinaryUtil; import org.apache.hudi.common.util.FileIOUtils; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; @@ -208,17 +209,19 @@ public class HoodieTableConfig extends HoodieConfig { .sinceVersion("0.11.0") .withDocumentation("Table checksum is used to guard against partial writes in HDFS. It is added as the last entry in hoodie.properties and then used to validate while reading table config."); - public static final ConfigProperty TABLE_METADATA_INDEX_INFLIGHT = ConfigProperty - .key("hoodie.table.metadata.indexes.inflight") + public static final ConfigProperty TABLE_METADATA_PARTITIONS_INFLIGHT = ConfigProperty + .key("hoodie.table.metadata.partitions.inflight") .noDefaultValue() .sinceVersion("0.11.0") - .withDocumentation("Comma-separated list of metadata partitions whose indexing is in progress."); + .withDocumentation("Comma-separated list of metadata partitions whose building is in progress. " + + "These partitions are not yet ready for use by the readers."); - public static final ConfigProperty TABLE_METADATA_INDEX_COMPLETED = ConfigProperty - .key("hoodie.table.metadata.indexes.completed") + public static final ConfigProperty TABLE_METADATA_PARTITIONS = ConfigProperty + .key("hoodie.table.metadata.partitions") .noDefaultValue() .sinceVersion("0.11.0") - .withDocumentation("Comma-separated list of metadata partitions whose indexing is complete."); + .withDocumentation("Comma-separated list of metadata partitions that have been completely built and in-sync with data table. " + + "These partitions are ready for use by the readers"); private static final String TABLE_CHECKSUM_FORMAT = "%s.%s"; // . @@ -597,14 +600,12 @@ private Long getTableChecksum() { return getLong(TABLE_CHECKSUM); } - public String getInflightMetadataIndexes() { - return getStringOrDefault(TABLE_METADATA_INDEX_INFLIGHT, ""); + public String getMetadataPartitionsInflight() { + return getStringOrDefault(TABLE_METADATA_PARTITIONS_INFLIGHT, StringUtils.EMPTY_STRING); } - // TODO getInflightAndCompletedMetadataIndexes - - public String getCompletedMetadataIndexes() { - return getStringOrDefault(TABLE_METADATA_INDEX_COMPLETED, ""); + public String getMetadataPartitions() { + return getStringOrDefault(TABLE_METADATA_PARTITIONS, StringUtils.EMPTY_STRING); } public Map propsMap() { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java index 4a614d0de741a..f96dd71aae985 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java @@ -704,7 +704,7 @@ public HoodieInstant revertIndexInflightToRequested(HoodieInstant inflightInstan /** * Save content for inflight/requested index instant. */ - public void saveToPendingIndexCommit(HoodieInstant instant, Option content) { + public void saveToPendingIndexAction(HoodieInstant instant, Option content) { ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.INDEX_ACTION), String.format("%s is not equal to %s action", instant.getAction(), INDEX_ACTION)); createFileInMetaPath(instant.getFileName(), content, false); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java index 6e17e604d82a8..a6a64467bab88 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java @@ -55,7 +55,7 @@ public interface HoodieTimeline extends Serializable { String COMPACTION_ACTION = "compaction"; String REQUESTED_EXTENSION = ".requested"; String RESTORE_ACTION = "restore"; - String INDEX_ACTION = "index"; + String INDEX_ACTION = "indexing"; String[] VALID_ACTIONS_IN_TIMELINE = {COMMIT_ACTION, DELTA_COMMIT_ACTION, CLEAN_ACTION, SAVEPOINT_ACTION, RESTORE_ACTION, ROLLBACK_ACTION, diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java index 326bf05277f0b..4a092aa6f3400 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java @@ -20,12 +20,24 @@ import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + /** * Simple utility for operations on strings. */ public class StringUtils { public static final String EMPTY_STRING = ""; + private static final Function> STRING_TO_SET = (str) -> Stream.of(str.split(",")) + .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toSet()); + private static final Function> STRING_TO_LIST = (str) -> Stream.of(str.split(",")) + .map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList()); /** *

    @@ -46,7 +58,7 @@ public class StringUtils { * */ public static String join(final String... elements) { - return join(elements, ""); + return join(elements, EMPTY_STRING); } public static String joinUsingDelim(String delim, final String... elements) { @@ -100,4 +112,24 @@ public static String objToString(@Nullable Object obj) { private static boolean stringIsNullOrEmpty(@Nullable String string) { return string == null || string.isEmpty(); } + + /** + * Converts the input string, delimited by comma, to a set of strings. + * + * @param input + * @return + */ + public static Set toSet(@Nullable String input) { + return isNullOrEmpty(input) ? new HashSet<>() : STRING_TO_SET.apply(input); + } + + /** + * Converts the input string, delimited by comma, to a list of strings. + * + * @param input + * @return + */ + public static List toList(@Nullable String input) { + return isNullOrEmpty(input) ? new ArrayList<>() : STRING_TO_LIST.apply(input); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java index 94d3e272d7bf3..c964ad14d7733 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java @@ -76,6 +76,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Set; import java.util.function.BiFunction; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -1141,4 +1142,12 @@ private static Option tryResolveSchemaForTable(HoodieTableMetaClient dat throw new HoodieException("Failed to get latest columns for " + dataTableMetaClient.getBasePath(), e); } } + + public static Set getInflightMetadataPartitions(HoodieTableConfig tableConfig) { + return StringUtils.toSet(tableConfig.getMetadataPartitionsInflight()); + } + + public static Set getCompletedMetadataPartitions(HoodieTableConfig tableConfig) { + return StringUtils.toSet(tableConfig.getMetadataPartitions()); + } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestStringUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestStringUtils.java index b402996fa78ae..83a84a3cc9cc5 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/TestStringUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestStringUtils.java @@ -20,6 +20,12 @@ import org.junit.jupiter.api.Test; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNull; @@ -61,4 +67,20 @@ public void testStringNullOrEmpty() { assertNotEquals(null, StringUtils.isNullOrEmpty("this is not empty")); assertTrue(StringUtils.isNullOrEmpty("")); } + + @Test + public void testStringToSet() { + assertEquals(new HashSet<>(), StringUtils.toSet(null)); + assertEquals(new HashSet<>(), StringUtils.toSet("")); + Set expected = new HashSet<>(Arrays.asList("a", "b", "c")); + assertEquals(expected, StringUtils.toSet("a,b, c")); + } + + @Test + public void testStringToList() { + assertEquals(new ArrayList<>(), StringUtils.toList(null)); + assertEquals(new ArrayList<>(), StringUtils.toList("")); + List expected = Arrays.asList("a", "b", "c"); + assertEquals(expected, StringUtils.toList("a,b, c")); + } } From 514c0512fe56f0d3c945f724e1ae149f91ca5b65 Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Thu, 31 Mar 2022 01:18:56 +0530 Subject: [PATCH 18/21] - Handle corner cases related to partial failures. - Abort gracefully after deleting partition and instant. - Handle other actions in timeline to consider before catching up --- .../action/index/RunIndexActionExecutor.java | 107 ++++++++++++++---- .../index/ScheduleIndexActionExecutor.java | 67 ++++++----- .../table/timeline/HoodieActiveTimeline.java | 2 +- .../metadata/HoodieTableMetadataUtil.java | 17 +++ 4 files changed, 140 insertions(+), 53 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java index d689312b7c018..4779dd1f81a6f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java @@ -35,12 +35,14 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.util.CleanerUtils; +import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieIndexException; import org.apache.hudi.metadata.HoodieTableMetadataWriter; +import org.apache.hudi.metadata.MetadataPartitionType; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.BaseActionExecutor; @@ -50,6 +52,7 @@ import java.io.IOException; import java.util.List; +import java.util.Locale; import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -59,12 +62,19 @@ import static org.apache.hudi.common.model.WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL; import static org.apache.hudi.common.table.timeline.HoodieInstant.State.COMPLETED; +import static org.apache.hudi.common.table.timeline.HoodieInstant.State.INFLIGHT; import static org.apache.hudi.common.table.timeline.HoodieInstant.State.REQUESTED; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.CLEAN_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.GREATER_THAN_OR_EQUALS; import static org.apache.hudi.common.table.timeline.HoodieTimeline.INDEX_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.RESTORE_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.ROLLBACK_ACTION; import static org.apache.hudi.config.HoodieWriteConfig.WRITE_CONCURRENCY_MODE; import static org.apache.hudi.metadata.HoodieTableMetadata.getMetadataTableBasePath; +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.deleteMetadataPartition; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getCompletedMetadataPartitions; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getInflightMetadataPartitions; +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.metadataPartitionExists; /** * Reads the index plan and executes the plan. @@ -96,13 +106,29 @@ public Option execute() { indexTimer.startTimer(); HoodieInstant indexInstant = validateAndGetIndexInstant(); + // read HoodieIndexPlan + HoodieIndexPlan indexPlan; + try { + indexPlan = TimelineMetadataUtils.deserializeIndexPlan(table.getActiveTimeline().readIndexPlanAsBytes(indexInstant).get()); + } catch (IOException e) { + throw new HoodieIndexException("Failed to read the index plan for instant: " + indexInstant); + } + List indexPartitionInfos = indexPlan.getIndexPartitionInfos(); try { - // read HoodieIndexPlan - HoodieIndexPlan indexPlan = TimelineMetadataUtils.deserializeIndexPlan(table.getActiveTimeline().readIndexPlanAsBytes(indexInstant).get()); - List indexPartitionInfos = indexPlan.getIndexPartitionInfos(); if (indexPartitionInfos == null || indexPartitionInfos.isEmpty()) { throw new HoodieIndexException(String.format("No partitions to index for instant: %s", instantTime)); } + // ensure the metadata partitions for the requested indexes are not already available (or inflight) + HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig(); + Set indexesInflightOrCompleted = getInflightMetadataPartitions(tableConfig); + indexesInflightOrCompleted.addAll(getCompletedMetadataPartitions(tableConfig)); + Set requestedPartitions = indexPartitionInfos.stream() + .map(HoodieIndexPartitionInfo::getMetadataPartitionPath).collect(Collectors.toSet()); + requestedPartitions.retainAll(indexesInflightOrCompleted); + if (!requestedPartitions.isEmpty()) { + throw new HoodieIndexException(String.format("Following partitions already exist or inflight: %s", requestedPartitions)); + } + // transition requested indexInstant to inflight table.getActiveTimeline().transitionIndexRequestedToInflight(indexInstant, Option.empty()); // start indexing for each partition @@ -113,11 +139,9 @@ public Option execute() { LOG.info("Starting Index Building with base instant: " + indexUptoInstant); metadataWriter.buildMetadataPartitions(context, indexPartitionInfos); - // get all instants since the plan completed (both from active timeline and archived timeline) - // assumption is that all metadata partitions had same instant upto which they were scheduled to be indexed - table.getMetaClient().reloadActiveTimeline(); - List instantsToIndex = getRemainingArchivedAndActiveInstantsSince(indexUptoInstant, table.getMetaClient()); - LOG.info("Total remaining instants to index: " + instantsToIndex.size()); + // get remaining instants to catchup + List instantsToCatchup = getInstantsToCatchup(indexUptoInstant); + LOG.info("Total remaining instants to index: " + instantsToCatchup.size()); // reconcile with metadata table timeline String metadataBasePath = getMetadataTableBasePath(table.getMetaClient().getBasePath()); @@ -127,7 +151,7 @@ public Option execute() { // index catchup for all remaining instants with a timeout currentIndexedInstant = indexUptoInstant; - catchupWithInflightWriters(metadataWriter, instantsToIndex, metadataMetaClient, metadataCompletedTimestamps); + catchupWithInflightWriters(metadataWriter, instantsToCatchup, metadataMetaClient, metadataCompletedTimestamps); // save index commit metadata and update table config List finalIndexPartitionInfos = indexPartitionInfos.stream() .map(info -> new HoodieIndexPartitionInfo( @@ -140,10 +164,43 @@ public Option execute() { updateTableConfigAndTimeline(indexInstant, finalIndexPartitionInfos, indexCommitMetadata); return Option.of(indexCommitMetadata); } catch (IOException e) { + // abort gracefully + abort(indexInstant, indexPartitionInfos.stream().map(HoodieIndexPartitionInfo::getMetadataPartitionPath).collect(Collectors.toSet())); throw new HoodieIndexException(String.format("Unable to index instant: %s", indexInstant)); } } + private void abort(HoodieInstant indexInstant, Set requestedPartitions) { + // delete metadata partition + requestedPartitions.forEach(partition -> { + MetadataPartitionType partitionType = MetadataPartitionType.valueOf(partition.toUpperCase(Locale.ROOT)); + if (metadataPartitionExists(table.getMetaClient().getBasePath(), context, partitionType)) { + deleteMetadataPartition(table.getMetaClient().getBasePath(), context, partitionType); + } + }); + // delete inflight instant + table.getMetaClient().reloadActiveTimeline().deleteInstantFileIfExists(new HoodieInstant(INFLIGHT, indexInstant.getAction(), indexInstant.getTimestamp())); + } + + private List getInstantsToCatchup(String indexUptoInstant) { + // since only write timeline was considered while scheduling index, which gives us the indexUpto instant + // here we consider other valid actions to pick catchupStart instant + Set validActions = CollectionUtils.createSet(CLEAN_ACTION, RESTORE_ACTION, ROLLBACK_ACTION); + HoodieInstant catchupStartInstant = table.getMetaClient().reloadActiveTimeline() + .getTimelineOfActions(validActions) + .filterInflightsAndRequested() + .findInstantsBefore(indexUptoInstant) + .firstInstant().orElseGet(() -> null); + // get all instants since the plan completed (both from active timeline and archived timeline) + List instantsToIndex; + if (catchupStartInstant != null) { + instantsToIndex = getRemainingArchivedAndActiveInstantsSince(catchupStartInstant.getTimestamp(), table.getMetaClient()); + } else { + instantsToIndex = getRemainingArchivedAndActiveInstantsSince(indexUptoInstant, table.getMetaClient()); + } + return instantsToIndex; + } + private HoodieInstant validateAndGetIndexInstant() { // ensure lock provider configured if (!config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl() || StringUtils.isNullOrEmpty(config.getLockProviderClass())) { @@ -151,12 +208,11 @@ private HoodieInstant validateAndGetIndexInstant() { WRITE_CONCURRENCY_MODE.key(), OPTIMISTIC_CONCURRENCY_CONTROL.name())); } - HoodieInstant indexInstant = table.getActiveTimeline() + return table.getActiveTimeline() .filterPendingIndexTimeline() .filter(instant -> instant.getTimestamp().equals(instantTime) && REQUESTED.equals(instant.getState())) .lastInstant() .orElseThrow(() -> new HoodieIndexException(String.format("No requested index instant found: %s", instantTime))); - return indexInstant; } private void updateTableConfigAndTimeline(HoodieInstant indexInstant, @@ -192,20 +248,22 @@ private void catchupWithInflightWriters(HoodieTableMetadataWriter metadataWriter } private static List getRemainingArchivedAndActiveInstantsSince(String instant, HoodieTableMetaClient metaClient) { - List remainingInstantsToIndex = metaClient.getArchivedTimeline() - .getWriteTimeline() - .findInstantsAfter(instant) - .getInstants().collect(Collectors.toList()); - remainingInstantsToIndex.addAll(metaClient.getActiveTimeline().getWriteTimeline().findInstantsAfter(instant).getInstants().collect(Collectors.toList())); + List remainingInstantsToIndex = metaClient.getArchivedTimeline().getInstants() + .filter(i -> HoodieTimeline.compareTimestamps(i.getTimestamp(), GREATER_THAN_OR_EQUALS, instant)) + .filter(i -> !INDEX_ACTION.equals(i.getAction())) + .collect(Collectors.toList()); + remainingInstantsToIndex.addAll(metaClient.getActiveTimeline().findInstantsAfter(instant).getInstants() + .filter(i -> HoodieTimeline.compareTimestamps(i.getTimestamp(), GREATER_THAN_OR_EQUALS, instant)) + .filter(i -> !INDEX_ACTION.equals(i.getAction())) + .collect(Collectors.toList())); return remainingInstantsToIndex; } private static List getCompletedArchivedAndActiveInstantsAfter(String instant, HoodieTableMetaClient metaClient) { - List completedInstants = metaClient.getArchivedTimeline() - .filterCompletedInstants() - .findInstantsAfter(instant) - .getInstants().collect(Collectors.toList()); - completedInstants.addAll(metaClient.reloadActiveTimeline().filterCompletedInstants().findInstantsAfter(instant).getInstants().collect(Collectors.toList())); + List completedInstants = metaClient.getArchivedTimeline().filterCompletedInstants().findInstantsAfter(instant) + .getInstants().filter(i -> !INDEX_ACTION.equals(i.getAction())).collect(Collectors.toList()); + completedInstants.addAll(metaClient.reloadActiveTimeline().filterCompletedInstants().findInstantsAfter(instant) + .getInstants().filter(i -> !INDEX_ACTION.equals(i.getAction())).collect(Collectors.toList())); return completedInstants; } @@ -274,6 +332,7 @@ public void run() { Option metadataInstant = metadataMetaClient.reloadActiveTimeline() .filterCompletedInstants().filter(i -> i.getTimestamp().equals(instantTime)).firstInstant(); if (metadataInstant.isPresent()) { + currentIndexedInstant = instantTime; continue; } try { @@ -289,16 +348,16 @@ public void run() { // do not trigger any table service as partition is not fully built out yet metadataWriter.update(commitMetadata, instant.getTimestamp(), false); break; - case HoodieTimeline.CLEAN_ACTION: + case CLEAN_ACTION: HoodieCleanMetadata cleanMetadata = CleanerUtils.getCleanerMetadata(table.getMetaClient(), instant); metadataWriter.update(cleanMetadata, instant.getTimestamp()); break; - case HoodieTimeline.RESTORE_ACTION: + case RESTORE_ACTION: HoodieRestoreMetadata restoreMetadata = TimelineMetadataUtils.deserializeHoodieRestoreMetadata( table.getActiveTimeline().getInstantDetails(instant).get()); metadataWriter.update(restoreMetadata, instant.getTimestamp()); break; - case HoodieTimeline.ROLLBACK_ACTION: + case ROLLBACK_ACTION: HoodieRollbackMetadata rollbackMetadata = TimelineMetadataUtils.deserializeHoodieRollbackMetadata( table.getActiveTimeline().getInstantDetails(instant).get()); metadataWriter.update(rollbackMetadata, instant.getTimestamp()); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java index 5057bbb006131..9b7758e5296bf 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java @@ -49,8 +49,10 @@ import static org.apache.hudi.common.model.WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL; import static org.apache.hudi.config.HoodieWriteConfig.WRITE_CONCURRENCY_MODE; +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.deleteMetadataPartition; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getCompletedMetadataPartitions; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getInflightMetadataPartitions; +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.metadataPartitionExists; /** * Schedules INDEX action. @@ -94,38 +96,36 @@ public Option execute() { } List finalPartitionsToIndex = partitionIndexTypes.stream() .filter(p -> requestedPartitions.contains(p.getPartitionPath())).collect(Collectors.toList()); - // get last completed instant - Option indexUptoInstant = table.getActiveTimeline().getContiguousCompletedWriteTimeline().lastInstant(); - if (indexUptoInstant.isPresent()) { - final HoodieInstant indexInstant = HoodieTimeline.getIndexRequestedInstant(instantTime); - // start initializing file groups - // in case FILES partition itself was not initialized before (i.e. metadata was never enabled), this will initialize synchronously - HoodieTableMetadataWriter metadataWriter = table.getMetadataWriter(instantTime) - .orElseThrow(() -> new HoodieIndexException(String.format("Could not get metadata writer to initialize filegroups for indexing for instant: %s", instantTime))); - try { - this.txnManager.beginTransaction(Option.of(indexInstant), Option.empty()); + final HoodieInstant indexInstant = HoodieTimeline.getIndexRequestedInstant(instantTime); + try { + this.txnManager.beginTransaction(Option.of(indexInstant), Option.empty()); + // get last completed instant + Option indexUptoInstant = table.getActiveTimeline().getContiguousCompletedWriteTimeline().lastInstant(); + if (indexUptoInstant.isPresent()) { + // start initializing file groups + // in case FILES partition itself was not initialized before (i.e. metadata was never enabled), this will initialize synchronously + HoodieTableMetadataWriter metadataWriter = table.getMetadataWriter(instantTime) + .orElseThrow(() -> new HoodieIndexException(String.format("Could not get metadata writer to initialize filegroups for indexing for instant: %s", instantTime))); metadataWriter.initializeMetadataPartitions(table.getMetaClient(), finalPartitionsToIndex, indexInstant.getTimestamp()); - } catch (IOException e) { - LOG.error("Could not initialize file groups", e); - throw new HoodieIOException(e.getMessage(), e); - } finally { - this.txnManager.endTransaction(Option.of(indexInstant)); - } - // for each partitionToIndex add that time to the plan - List indexPartitionInfos = finalPartitionsToIndex.stream() - .map(p -> new HoodieIndexPartitionInfo(LATEST_INDEX_PLAN_VERSION, p.getPartitionPath(), indexUptoInstant.get().getTimestamp())) - .collect(Collectors.toList()); - HoodieIndexPlan indexPlan = new HoodieIndexPlan(LATEST_INDEX_PLAN_VERSION, indexPartitionInfos); - // update data timeline with requested instant - try { + + // for each partitionToIndex add that time to the plan + List indexPartitionInfos = finalPartitionsToIndex.stream() + .map(p -> new HoodieIndexPartitionInfo(LATEST_INDEX_PLAN_VERSION, p.getPartitionPath(), indexUptoInstant.get().getTimestamp())) + .collect(Collectors.toList()); + HoodieIndexPlan indexPlan = new HoodieIndexPlan(LATEST_INDEX_PLAN_VERSION, indexPartitionInfos); + // update data timeline with requested instant table.getActiveTimeline().saveToPendingIndexAction(indexInstant, TimelineMetadataUtils.serializeIndexPlan(indexPlan)); - } catch (IOException e) { - LOG.error("Error while saving index requested file", e); - throw new HoodieIOException(e.getMessage(), e); + return Option.of(indexPlan); } - table.getMetaClient().reloadActiveTimeline(); - return Option.of(indexPlan); + } catch (IOException e) { + LOG.error("Could not initialize file groups", e); + // abort gracefully + abort(indexInstant); + throw new HoodieIOException(e.getMessage(), e); + } finally { + this.txnManager.endTransaction(Option.of(indexInstant)); } + return Option.empty(); } @@ -139,4 +139,15 @@ private void validateBeforeScheduling() { WRITE_CONCURRENCY_MODE.key(), OPTIMISTIC_CONCURRENCY_CONTROL.name())); } } + + private void abort(HoodieInstant indexInstant) { + // delete metadata partition + partitionIndexTypes.forEach(partitionType -> { + if (metadataPartitionExists(table.getMetaClient().getBasePath(), context, partitionType)) { + deleteMetadataPartition(table.getMetaClient().getBasePath(), context, partitionType); + } + }); + // delete requested instant + table.getMetaClient().reloadActiveTimeline().deleteInstantFileIfExists(indexInstant); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java index f96dd71aae985..9093a218d0b2f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java @@ -220,7 +220,7 @@ public void deleteCompactionRequested(HoodieInstant instant) { deleteInstantFile(instant); } - private void deleteInstantFileIfExists(HoodieInstant instant) { + public void deleteInstantFileIfExists(HoodieInstant instant) { LOG.info("Deleting instant " + instant); Path inFlightCommitFilePath = new Path(metaClient.getMetaPath(), instant.getFileName()); try { diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java index c964ad14d7733..c9d81cee5dc5e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java @@ -124,6 +124,23 @@ public static void deleteMetadataTable(String basePath, HoodieEngineContext cont } } + /** + * Deletes the metadata partition from the file system. + * + * @param basePath - base path of the dataset + * @param context - instance of {@link HoodieEngineContext} + * @param partitionType - {@link MetadataPartitionType} of the partition to delete + */ + public static void deleteMetadataPartition(String basePath, HoodieEngineContext context, MetadataPartitionType partitionType) { + final String metadataTablePath = HoodieTableMetadata.getMetadataTableBasePath(basePath); + FileSystem fs = FSUtils.getFs(metadataTablePath, context.getHadoopConf().get()); + try { + fs.delete(new Path(metadataTablePath, partitionType.getPartitionPath()), true); + } catch (Exception e) { + throw new HoodieMetadataException(String.format("Failed to remove metadata partition %s from path %s", partitionType, metadataTablePath), e); + } + } + /** * Check if the given metadata partition exists. * From 18b9acd3320e68ee6688ea4eec693676350a9e15 Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Thu, 31 Mar 2022 01:39:50 +0530 Subject: [PATCH 19/21] Check for existing indexes in HoodieIndexer --- .../apache/hudi/utilities/HoodieIndexer.java | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java index 9316e154ac923..2741e2b98a667 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java @@ -46,8 +46,9 @@ import java.util.Set; import java.util.stream.Collectors; -import static org.apache.hudi.common.table.timeline.HoodieInstant.State.INFLIGHT; import static org.apache.hudi.common.util.StringUtils.isNullOrEmpty; +import static org.apache.hudi.common.util.ValidationUtils.checkArgument; +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getCompletedMetadataPartitions; import static org.apache.hudi.utilities.UtilHelpers.EXECUTE; import static org.apache.hudi.utilities.UtilHelpers.SCHEDULE; import static org.apache.hudi.utilities.UtilHelpers.SCHEDULE_AND_EXECUTE; @@ -208,6 +209,10 @@ private Option scheduleIndexing(JavaSparkContext jsc) throws Exception { private Option doSchedule(SparkRDDWriteClient client) { List partitionTypes = getRequestedPartitionTypes(cfg.indexTypes); + checkArgument(partitionTypes.size() == 1, "Currently, only one index type can be scheduled at a time."); + if (indexExists(partitionTypes)) { + return Option.empty(); + } Option indexingInstant = client.scheduleIndexing(partitionTypes); if (!indexingInstant.isPresent()) { LOG.error("Scheduling of index action did not return any instant."); @@ -215,6 +220,17 @@ private Option doSchedule(SparkRDDWriteClient clien return indexingInstant; } + private boolean indexExists(List partitionTypes) { + Set indexedMetadataPartitions = getCompletedMetadataPartitions(metaClient.getTableConfig()); + Set requestedIndexPartitionPaths = partitionTypes.stream().map(MetadataPartitionType::getPartitionPath).collect(Collectors.toSet()); + requestedIndexPartitionPaths.retainAll(indexedMetadataPartitions); + if (!requestedIndexPartitionPaths.isEmpty()) { + LOG.error("Following indexes already built: " + requestedIndexPartitionPaths); + return true; + } + return false; + } + private int runIndexing(JavaSparkContext jsc) throws Exception { String schemaStr = UtilHelpers.getSchemaFromLatestInstant(metaClient); try (SparkRDDWriteClient client = UtilHelpers.createHoodieClient(jsc, cfg.basePath, schemaStr, cfg.parallelism, Option.empty(), props)) { @@ -223,7 +239,6 @@ private int runIndexing(JavaSparkContext jsc) throws Exception { // Find the earliest scheduled indexing instant for execution Option earliestPendingIndexInstant = metaClient.getActiveTimeline() .filterPendingIndexTimeline() - .filter(i -> !(i.isCompleted() || INFLIGHT.equals(i.getState()))) .firstInstant(); if (earliestPendingIndexInstant.isPresent()) { cfg.indexInstantTime = earliestPendingIndexInstant.get().getTimestamp(); From fc9ac46f36a4df8d9d590845b9848d48af1f7cae Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Thu, 31 Mar 2022 03:45:01 +0530 Subject: [PATCH 20/21] Rename index_action and other nits --- .../HoodieBackedTableMetadataWriter.java | 6 ++--- .../action/index/RunIndexActionExecutor.java | 23 ++++++++++++------- .../table/timeline/HoodieActiveTimeline.java | 22 +++++++++--------- .../timeline/HoodieArchivedTimeline.java | 2 +- .../table/timeline/HoodieDefaultTimeline.java | 6 ++--- .../common/table/timeline/HoodieInstant.java | 2 +- .../common/table/timeline/HoodieTimeline.java | 14 +++++------ .../timeline/TestHoodieActiveTimeline.java | 2 +- 8 files changed, 42 insertions(+), 35 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java index a1479c9815ddc..abed2f78baf32 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java @@ -545,9 +545,9 @@ private boolean initializeFromFilesystem(HoodieTableMetaClient dataMetaClient, } private void updateInitializedPartitionsInTableConfig(List partitionTypes) { - Set completedIndexes = getCompletedMetadataPartitions(dataMetaClient.getTableConfig()); - completedIndexes.addAll(partitionTypes.stream().map(MetadataPartitionType::getPartitionPath).collect(Collectors.toSet())); - dataMetaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_PARTITIONS.key(), String.join(",", completedIndexes)); + Set completedPartitions = getCompletedMetadataPartitions(dataMetaClient.getTableConfig()); + completedPartitions.addAll(partitionTypes.stream().map(MetadataPartitionType::getPartitionPath).collect(Collectors.toSet())); + dataMetaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_PARTITIONS.key(), String.join(",", completedPartitions)); HoodieTableConfig.update(dataMetaClient.getFs(), new Path(dataMetaClient.getMetaPath()), dataMetaClient.getTableConfig().getProps()); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java index 4779dd1f81a6f..8d15d9275d820 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java @@ -62,11 +62,10 @@ import static org.apache.hudi.common.model.WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL; import static org.apache.hudi.common.table.timeline.HoodieInstant.State.COMPLETED; -import static org.apache.hudi.common.table.timeline.HoodieInstant.State.INFLIGHT; import static org.apache.hudi.common.table.timeline.HoodieInstant.State.REQUESTED; import static org.apache.hudi.common.table.timeline.HoodieTimeline.CLEAN_ACTION; import static org.apache.hudi.common.table.timeline.HoodieTimeline.GREATER_THAN_OR_EQUALS; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.INDEX_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.INDEXING_ACTION; import static org.apache.hudi.common.table.timeline.HoodieTimeline.RESTORE_ACTION; import static org.apache.hudi.common.table.timeline.HoodieTimeline.ROLLBACK_ACTION; import static org.apache.hudi.config.HoodieWriteConfig.WRITE_CONCURRENCY_MODE; @@ -171,15 +170,23 @@ public Option execute() { } private void abort(HoodieInstant indexInstant, Set requestedPartitions) { + Set inflightPartitions = getInflightMetadataPartitions(table.getMetaClient().getTableConfig()); + Set completedPartitions = getCompletedMetadataPartitions(table.getMetaClient().getTableConfig()); // delete metadata partition requestedPartitions.forEach(partition -> { MetadataPartitionType partitionType = MetadataPartitionType.valueOf(partition.toUpperCase(Locale.ROOT)); if (metadataPartitionExists(table.getMetaClient().getBasePath(), context, partitionType)) { deleteMetadataPartition(table.getMetaClient().getBasePath(), context, partitionType); } + inflightPartitions.remove(partition); + completedPartitions.remove(partition); }); + // update table config + table.getMetaClient().getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_PARTITIONS_INFLIGHT.key(), String.join(",", inflightPartitions)); + table.getMetaClient().getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_PARTITIONS.key(), String.join(",", completedPartitions)); + HoodieTableConfig.update(table.getMetaClient().getFs(), new Path(table.getMetaClient().getMetaPath()), table.getMetaClient().getTableConfig().getProps()); // delete inflight instant - table.getMetaClient().reloadActiveTimeline().deleteInstantFileIfExists(new HoodieInstant(INFLIGHT, indexInstant.getAction(), indexInstant.getTimestamp())); + table.getMetaClient().reloadActiveTimeline().deleteInstantFileIfExists(HoodieTimeline.getIndexInflightInstant(indexInstant.getTimestamp())); } private List getInstantsToCatchup(String indexUptoInstant) { @@ -224,7 +231,7 @@ private void updateTableConfigAndTimeline(HoodieInstant indexInstant, updateMetadataPartitionsTableConfig(table.getMetaClient(), finalIndexPartitionInfos.stream().map(HoodieIndexPartitionInfo::getMetadataPartitionPath).collect(Collectors.toSet())); table.getActiveTimeline().saveAsComplete( - new HoodieInstant(true, INDEX_ACTION, indexInstant.getTimestamp()), + new HoodieInstant(true, INDEXING_ACTION, indexInstant.getTimestamp()), TimelineMetadataUtils.serializeIndexCommitMetadata(indexCommitMetadata)); } finally { txnManager.endTransaction(); @@ -250,20 +257,20 @@ private void catchupWithInflightWriters(HoodieTableMetadataWriter metadataWriter private static List getRemainingArchivedAndActiveInstantsSince(String instant, HoodieTableMetaClient metaClient) { List remainingInstantsToIndex = metaClient.getArchivedTimeline().getInstants() .filter(i -> HoodieTimeline.compareTimestamps(i.getTimestamp(), GREATER_THAN_OR_EQUALS, instant)) - .filter(i -> !INDEX_ACTION.equals(i.getAction())) + .filter(i -> !INDEXING_ACTION.equals(i.getAction())) .collect(Collectors.toList()); remainingInstantsToIndex.addAll(metaClient.getActiveTimeline().findInstantsAfter(instant).getInstants() .filter(i -> HoodieTimeline.compareTimestamps(i.getTimestamp(), GREATER_THAN_OR_EQUALS, instant)) - .filter(i -> !INDEX_ACTION.equals(i.getAction())) + .filter(i -> !INDEXING_ACTION.equals(i.getAction())) .collect(Collectors.toList())); return remainingInstantsToIndex; } private static List getCompletedArchivedAndActiveInstantsAfter(String instant, HoodieTableMetaClient metaClient) { List completedInstants = metaClient.getArchivedTimeline().filterCompletedInstants().findInstantsAfter(instant) - .getInstants().filter(i -> !INDEX_ACTION.equals(i.getAction())).collect(Collectors.toList()); + .getInstants().filter(i -> !INDEXING_ACTION.equals(i.getAction())).collect(Collectors.toList()); completedInstants.addAll(metaClient.reloadActiveTimeline().filterCompletedInstants().findInstantsAfter(instant) - .getInstants().filter(i -> !INDEX_ACTION.equals(i.getAction())).collect(Collectors.toList())); + .getInstants().filter(i -> !INDEXING_ACTION.equals(i.getAction())).collect(Collectors.toList())); return completedInstants; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java index 9093a218d0b2f..b22c0a111f1a7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java @@ -658,11 +658,11 @@ public void saveToRestoreRequested(HoodieInstant instant, Option content * @return inflight instant */ public HoodieInstant transitionIndexRequestedToInflight(HoodieInstant requestedInstant, Option data) { - ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.INDEX_ACTION), - String.format("%s is not equal to %s action", requestedInstant.getAction(), INDEX_ACTION)); + ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.INDEXING_ACTION), + String.format("%s is not equal to %s action", requestedInstant.getAction(), INDEXING_ACTION)); ValidationUtils.checkArgument(requestedInstant.isRequested(), String.format("Instant %s not in requested state", requestedInstant.getTimestamp())); - HoodieInstant inflightInstant = new HoodieInstant(State.INFLIGHT, INDEX_ACTION, requestedInstant.getTimestamp()); + HoodieInstant inflightInstant = new HoodieInstant(State.INFLIGHT, INDEXING_ACTION, requestedInstant.getTimestamp()); transitionState(requestedInstant, inflightInstant, data); return inflightInstant; } @@ -673,11 +673,11 @@ public HoodieInstant transitionIndexRequestedToInflight(HoodieInstant requestedI * @return completed instant */ public HoodieInstant transitionIndexInflightToComplete(HoodieInstant inflightInstant, Option data) { - ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.INDEX_ACTION), - String.format("%s is not equal to %s action", inflightInstant.getAction(), INDEX_ACTION)); + ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.INDEXING_ACTION), + String.format("%s is not equal to %s action", inflightInstant.getAction(), INDEXING_ACTION)); ValidationUtils.checkArgument(inflightInstant.isInflight(), String.format("Instant %s not inflight", inflightInstant.getTimestamp())); - HoodieInstant commitInstant = new HoodieInstant(State.COMPLETED, INDEX_ACTION, inflightInstant.getTimestamp()); + HoodieInstant commitInstant = new HoodieInstant(State.COMPLETED, INDEXING_ACTION, inflightInstant.getTimestamp()); transitionState(inflightInstant, commitInstant, data); return commitInstant; } @@ -688,11 +688,11 @@ public HoodieInstant transitionIndexInflightToComplete(HoodieInstant inflightIns * @return requested instant */ public HoodieInstant revertIndexInflightToRequested(HoodieInstant inflightInstant) { - ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.INDEX_ACTION), - String.format("%s is not equal to %s action", inflightInstant.getAction(), INDEX_ACTION)); + ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.INDEXING_ACTION), + String.format("%s is not equal to %s action", inflightInstant.getAction(), INDEXING_ACTION)); ValidationUtils.checkArgument(inflightInstant.isInflight(), String.format("Instant %s not inflight", inflightInstant.getTimestamp())); - HoodieInstant requestedInstant = new HoodieInstant(State.REQUESTED, INDEX_ACTION, inflightInstant.getTimestamp()); + HoodieInstant requestedInstant = new HoodieInstant(State.REQUESTED, INDEXING_ACTION, inflightInstant.getTimestamp()); if (metaClient.getTimelineLayoutVersion().isNullVersion()) { transitionState(inflightInstant, requestedInstant, Option.empty()); } else { @@ -705,8 +705,8 @@ public HoodieInstant revertIndexInflightToRequested(HoodieInstant inflightInstan * Save content for inflight/requested index instant. */ public void saveToPendingIndexAction(HoodieInstant instant, Option content) { - ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.INDEX_ACTION), - String.format("%s is not equal to %s action", instant.getAction(), INDEX_ACTION)); + ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.INDEXING_ACTION), + String.format("%s is not equal to %s action", instant.getAction(), INDEXING_ACTION)); createFileInMetaPath(instant.getFileName(), content, false); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java index 0cb3f56470692..21c7c4db21754 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java @@ -208,7 +208,7 @@ private Option getMetadataKey(String action) { return Option.of("hoodieCompactionPlan"); case HoodieTimeline.REPLACE_COMMIT_ACTION: return Option.of("hoodieReplaceCommitMetadata"); - case HoodieTimeline.INDEX_ACTION: + case HoodieTimeline.INDEXING_ACTION: return Option.of("hoodieIndexCommitMetadata"); default: LOG.error(String.format("Unknown action in metadata (%s)", action)); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java index 7f2fea5ba1ab7..ac1dd007d0527 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java @@ -194,12 +194,12 @@ public HoodieTimeline filter(Predicate filter) { @Override public HoodieTimeline filterPendingIndexTimeline() { - return new HoodieDefaultTimeline(instants.stream().filter(s -> s.getAction().equals(INDEX_ACTION) && !s.isCompleted()), details); + return new HoodieDefaultTimeline(instants.stream().filter(s -> s.getAction().equals(INDEXING_ACTION) && !s.isCompleted()), details); } @Override public HoodieTimeline filterCompletedIndexTimeline() { - return new HoodieDefaultTimeline(instants.stream().filter(s -> s.getAction().equals(INDEX_ACTION) && s.isCompleted()), details); + return new HoodieDefaultTimeline(instants.stream().filter(s -> s.getAction().equals(INDEXING_ACTION) && s.isCompleted()), details); } /** @@ -215,7 +215,7 @@ public HoodieTimeline getCommitsTimeline() { */ public HoodieTimeline getAllCommitsTimeline() { return getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, - CLEAN_ACTION, COMPACTION_ACTION, SAVEPOINT_ACTION, ROLLBACK_ACTION, REPLACE_COMMIT_ACTION, INDEX_ACTION)); + CLEAN_ACTION, COMPACTION_ACTION, SAVEPOINT_ACTION, ROLLBACK_ACTION, REPLACE_COMMIT_ACTION, INDEXING_ACTION)); } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java index 9c0d03c7b41d0..52de898208b38 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java @@ -172,7 +172,7 @@ public String getFileName() { return isInflight() ? HoodieTimeline.makeInflightReplaceFileName(timestamp) : isRequested() ? HoodieTimeline.makeRequestedReplaceFileName(timestamp) : HoodieTimeline.makeReplaceFileName(timestamp); - } else if (HoodieTimeline.INDEX_ACTION.equals(action)) { + } else if (HoodieTimeline.INDEXING_ACTION.equals(action)) { return isInflight() ? HoodieTimeline.makeInflightIndexFileName(timestamp) : isRequested() ? HoodieTimeline.makeRequestedIndexFileName(timestamp) : HoodieTimeline.makeIndexCommitFileName(timestamp); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java index a6a64467bab88..666cc7d638da7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java @@ -55,11 +55,11 @@ public interface HoodieTimeline extends Serializable { String COMPACTION_ACTION = "compaction"; String REQUESTED_EXTENSION = ".requested"; String RESTORE_ACTION = "restore"; - String INDEX_ACTION = "indexing"; + String INDEXING_ACTION = "indexing"; String[] VALID_ACTIONS_IN_TIMELINE = {COMMIT_ACTION, DELTA_COMMIT_ACTION, CLEAN_ACTION, SAVEPOINT_ACTION, RESTORE_ACTION, ROLLBACK_ACTION, - COMPACTION_ACTION, REPLACE_COMMIT_ACTION, INDEX_ACTION}; + COMPACTION_ACTION, REPLACE_COMMIT_ACTION, INDEXING_ACTION}; String COMMIT_EXTENSION = "." + COMMIT_ACTION; String DELTA_COMMIT_EXTENSION = "." + DELTA_COMMIT_ACTION; @@ -85,9 +85,9 @@ public interface HoodieTimeline extends Serializable { String INFLIGHT_REPLACE_COMMIT_EXTENSION = "." + REPLACE_COMMIT_ACTION + INFLIGHT_EXTENSION; String REQUESTED_REPLACE_COMMIT_EXTENSION = "." + REPLACE_COMMIT_ACTION + REQUESTED_EXTENSION; String REPLACE_COMMIT_EXTENSION = "." + REPLACE_COMMIT_ACTION; - String INFLIGHT_INDEX_COMMIT_EXTENSION = "." + INDEX_ACTION + INFLIGHT_EXTENSION; - String REQUESTED_INDEX_COMMIT_EXTENSION = "." + INDEX_ACTION + REQUESTED_EXTENSION; - String INDEX_COMMIT_EXTENSION = "." + INDEX_ACTION; + String INFLIGHT_INDEX_COMMIT_EXTENSION = "." + INDEXING_ACTION + INFLIGHT_EXTENSION; + String REQUESTED_INDEX_COMMIT_EXTENSION = "." + INDEXING_ACTION + REQUESTED_EXTENSION; + String INDEX_COMMIT_EXTENSION = "." + INDEXING_ACTION; String INVALID_INSTANT_TS = "0"; @@ -365,11 +365,11 @@ static HoodieInstant getRollbackRequestedInstant(HoodieInstant instant) { } static HoodieInstant getIndexRequestedInstant(final String timestamp) { - return new HoodieInstant(State.REQUESTED, INDEX_ACTION, timestamp); + return new HoodieInstant(State.REQUESTED, INDEXING_ACTION, timestamp); } static HoodieInstant getIndexInflightInstant(final String timestamp) { - return new HoodieInstant(State.INFLIGHT, INDEX_ACTION, timestamp); + return new HoodieInstant(State.INFLIGHT, INDEXING_ACTION, timestamp); } /** diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java b/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java index 1dd1eedfbde59..9ff17cdbd2688 100755 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java @@ -270,7 +270,7 @@ public void testTimelineGetOperations() { checkTimeline.accept(timeline.getSavePointTimeline(), Collections.singleton(HoodieTimeline.SAVEPOINT_ACTION)); checkTimeline.accept(timeline.getAllCommitsTimeline(), CollectionUtils.createSet( HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.CLEAN_ACTION, HoodieTimeline.COMPACTION_ACTION, - HoodieTimeline.REPLACE_COMMIT_ACTION, HoodieTimeline.SAVEPOINT_ACTION, HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.INDEX_ACTION)); + HoodieTimeline.REPLACE_COMMIT_ACTION, HoodieTimeline.SAVEPOINT_ACTION, HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.INDEXING_ACTION)); // Get some random Instants Random rand = new Random(); From 01120c1b4a0dacaec5f3b968ac421f5faa0bc1b9 Mon Sep 17 00:00:00 2001 From: Sagar Sumit Date: Thu, 31 Mar 2022 22:30:00 +0530 Subject: [PATCH 21/21] Address some minors from last pass --- .../action/index/RunIndexActionExecutor.java | 48 +++++++++++-------- .../index/ScheduleIndexActionExecutor.java | 11 ++--- .../metadata/HoodieTableMetadataUtil.java | 6 +++ 3 files changed, 38 insertions(+), 27 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java index 8d15d9275d820..8c86a298f8a4b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java @@ -61,6 +61,8 @@ import java.util.stream.Collectors; import static org.apache.hudi.common.model.WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL; +import static org.apache.hudi.common.table.HoodieTableConfig.TABLE_METADATA_PARTITIONS; +import static org.apache.hudi.common.table.HoodieTableConfig.TABLE_METADATA_PARTITIONS_INFLIGHT; import static org.apache.hudi.common.table.timeline.HoodieInstant.State.COMPLETED; import static org.apache.hudi.common.table.timeline.HoodieInstant.State.REQUESTED; import static org.apache.hudi.common.table.timeline.HoodieTimeline.CLEAN_ACTION; @@ -72,6 +74,7 @@ import static org.apache.hudi.metadata.HoodieTableMetadata.getMetadataTableBasePath; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.deleteMetadataPartition; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getCompletedMetadataPartitions; +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getInflightAndCompletedMetadataPartitions; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getInflightMetadataPartitions; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.metadataPartitionExists; @@ -90,7 +93,7 @@ public class RunIndexActionExecutor exte // we use this to update the latest instant in data timeline that has been indexed in metadata table // this needs to be volatile as it can be updated in the IndexingCheckTask spawned by this executor // assumption is that only one indexer can execute at a time - private volatile String currentIndexedInstant; + private volatile String currentCaughtupInstant; private final TransactionManager txnManager; @@ -118,9 +121,7 @@ public Option execute() { throw new HoodieIndexException(String.format("No partitions to index for instant: %s", instantTime)); } // ensure the metadata partitions for the requested indexes are not already available (or inflight) - HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig(); - Set indexesInflightOrCompleted = getInflightMetadataPartitions(tableConfig); - indexesInflightOrCompleted.addAll(getCompletedMetadataPartitions(tableConfig)); + Set indexesInflightOrCompleted = getInflightAndCompletedMetadataPartitions(table.getMetaClient().getTableConfig()); Set requestedPartitions = indexPartitionInfos.stream() .map(HoodieIndexPartitionInfo::getMetadataPartitionPath).collect(Collectors.toSet()); requestedPartitions.retainAll(indexesInflightOrCompleted); @@ -149,14 +150,14 @@ public Option execute() { .map(HoodieInstant::getTimestamp).collect(Collectors.toSet()); // index catchup for all remaining instants with a timeout - currentIndexedInstant = indexUptoInstant; + currentCaughtupInstant = indexUptoInstant; catchupWithInflightWriters(metadataWriter, instantsToCatchup, metadataMetaClient, metadataCompletedTimestamps); // save index commit metadata and update table config List finalIndexPartitionInfos = indexPartitionInfos.stream() .map(info -> new HoodieIndexPartitionInfo( info.getVersion(), info.getMetadataPartitionPath(), - currentIndexedInstant)) + currentCaughtupInstant)) .collect(Collectors.toList()); HoodieIndexCommitMetadata indexCommitMetadata = HoodieIndexCommitMetadata.newBuilder() .setVersion(LATEST_INDEX_COMMIT_METADATA_VERSION).setIndexPartitionInfos(finalIndexPartitionInfos).build(); @@ -172,19 +173,23 @@ public Option execute() { private void abort(HoodieInstant indexInstant, Set requestedPartitions) { Set inflightPartitions = getInflightMetadataPartitions(table.getMetaClient().getTableConfig()); Set completedPartitions = getCompletedMetadataPartitions(table.getMetaClient().getTableConfig()); + // update table config + requestedPartitions.forEach(partition -> { + inflightPartitions.remove(partition); + completedPartitions.remove(partition); + }); + table.getMetaClient().getTableConfig().setValue(TABLE_METADATA_PARTITIONS_INFLIGHT.key(), String.join(",", inflightPartitions)); + table.getMetaClient().getTableConfig().setValue(TABLE_METADATA_PARTITIONS.key(), String.join(",", completedPartitions)); + HoodieTableConfig.update(table.getMetaClient().getFs(), new Path(table.getMetaClient().getMetaPath()), table.getMetaClient().getTableConfig().getProps()); + // delete metadata partition requestedPartitions.forEach(partition -> { MetadataPartitionType partitionType = MetadataPartitionType.valueOf(partition.toUpperCase(Locale.ROOT)); if (metadataPartitionExists(table.getMetaClient().getBasePath(), context, partitionType)) { deleteMetadataPartition(table.getMetaClient().getBasePath(), context, partitionType); } - inflightPartitions.remove(partition); - completedPartitions.remove(partition); }); - // update table config - table.getMetaClient().getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_PARTITIONS_INFLIGHT.key(), String.join(",", inflightPartitions)); - table.getMetaClient().getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_PARTITIONS.key(), String.join(",", completedPartitions)); - HoodieTableConfig.update(table.getMetaClient().getFs(), new Path(table.getMetaClient().getMetaPath()), table.getMetaClient().getTableConfig().getProps()); + // delete inflight instant table.getMetaClient().reloadActiveTimeline().deleteInstantFileIfExists(HoodieTimeline.getIndexInflightInstant(indexInstant.getTimestamp())); } @@ -193,15 +198,15 @@ private List getInstantsToCatchup(String indexUptoInstant) { // since only write timeline was considered while scheduling index, which gives us the indexUpto instant // here we consider other valid actions to pick catchupStart instant Set validActions = CollectionUtils.createSet(CLEAN_ACTION, RESTORE_ACTION, ROLLBACK_ACTION); - HoodieInstant catchupStartInstant = table.getMetaClient().reloadActiveTimeline() + Option catchupStartInstant = table.getMetaClient().reloadActiveTimeline() .getTimelineOfActions(validActions) .filterInflightsAndRequested() .findInstantsBefore(indexUptoInstant) - .firstInstant().orElseGet(() -> null); + .firstInstant(); // get all instants since the plan completed (both from active timeline and archived timeline) List instantsToIndex; - if (catchupStartInstant != null) { - instantsToIndex = getRemainingArchivedAndActiveInstantsSince(catchupStartInstant.getTimestamp(), table.getMetaClient()); + if (catchupStartInstant.isPresent()) { + instantsToIndex = getRemainingArchivedAndActiveInstantsSince(catchupStartInstant.get().getTimestamp(), table.getMetaClient()); } else { instantsToIndex = getRemainingArchivedAndActiveInstantsSince(indexUptoInstant, table.getMetaClient()); } @@ -248,7 +253,7 @@ private void catchupWithInflightWriters(HoodieTableMetadataWriter metadataWriter indexingCatchupTaskFuture.get(config.getIndexingCheckTimeoutSeconds(), TimeUnit.SECONDS); } catch (Exception e) { indexingCatchupTaskFuture.cancel(true); - throw new HoodieIndexException(String.format("Index catchup failed. Current indexed instant = %s. Aborting!", currentIndexedInstant), e); + throw new HoodieIndexException(String.format("Index catchup failed. Current indexed instant = %s. Aborting!", currentCaughtupInstant), e); } finally { executorService.shutdownNow(); } @@ -281,8 +286,8 @@ private void updateMetadataPartitionsTableConfig(HoodieTableMetaClient metaClien inflightPartitions.removeAll(metadataPartitions); completedPartitions.addAll(metadataPartitions); // update table config - metaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_PARTITIONS_INFLIGHT.key(), String.join(",", inflightPartitions)); - metaClient.getTableConfig().setValue(HoodieTableConfig.TABLE_METADATA_PARTITIONS.key(), String.join(",", completedPartitions)); + metaClient.getTableConfig().setValue(TABLE_METADATA_PARTITIONS_INFLIGHT.key(), String.join(",", inflightPartitions)); + metaClient.getTableConfig().setValue(TABLE_METADATA_PARTITIONS.key(), String.join(",", completedPartitions)); HoodieTableConfig.update(metaClient.getFs(), new Path(metaClient.getMetaPath()), metaClient.getTableConfig().getProps()); } @@ -316,7 +321,7 @@ public void run() { for (HoodieInstant instant : instantsToIndex) { // metadata index already updated for this instant if (!metadataCompletedInstants.isEmpty() && metadataCompletedInstants.contains(instant.getTimestamp())) { - currentIndexedInstant = instant.getTimestamp(); + currentCaughtupInstant = instant.getTimestamp(); continue; } while (!instant.isCompleted()) { @@ -339,7 +344,7 @@ public void run() { Option metadataInstant = metadataMetaClient.reloadActiveTimeline() .filterCompletedInstants().filter(i -> i.getTimestamp().equals(instantTime)).firstInstant(); if (metadataInstant.isPresent()) { - currentIndexedInstant = instantTime; + currentCaughtupInstant = instantTime; continue; } try { @@ -347,6 +352,7 @@ public void run() { txnManager.beginTransaction(Option.of(instant), Option.empty()); LOG.info("Updating metadata table for instant: " + instant); switch (instant.getAction()) { + // TODO: see if this can be moved to metadata writer itself case HoodieTimeline.COMMIT_ACTION: case HoodieTimeline.DELTA_COMMIT_ACTION: case HoodieTimeline.REPLACE_COMMIT_ACTION: diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java index 9b7758e5296bf..5afebee8a1d7f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java @@ -24,7 +24,6 @@ import org.apache.hudi.client.transaction.TransactionManager; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; @@ -50,8 +49,7 @@ import static org.apache.hudi.common.model.WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL; import static org.apache.hudi.config.HoodieWriteConfig.WRITE_CONCURRENCY_MODE; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.deleteMetadataPartition; -import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getCompletedMetadataPartitions; -import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getInflightMetadataPartitions; +import static org.apache.hudi.metadata.HoodieTableMetadataUtil.getInflightAndCompletedMetadataPartitions; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.metadataPartitionExists; /** @@ -85,14 +83,15 @@ public ScheduleIndexActionExecutor(HoodieEngineContext context, public Option execute() { validateBeforeScheduling(); // make sure that it is idempotent, check with previously pending index operations. - HoodieTableConfig tableConfig = table.getMetaClient().getTableConfig(); - Set indexesInflightOrCompleted = getInflightMetadataPartitions(tableConfig); - indexesInflightOrCompleted.addAll(getCompletedMetadataPartitions(tableConfig)); + Set indexesInflightOrCompleted = getInflightAndCompletedMetadataPartitions(table.getMetaClient().getTableConfig()); Set requestedPartitions = partitionIndexTypes.stream().map(MetadataPartitionType::getPartitionPath).collect(Collectors.toSet()); requestedPartitions.removeAll(indexesInflightOrCompleted); if (!requestedPartitions.isEmpty()) { LOG.warn(String.format("Following partitions already exist or inflight: %s. Going to index only these partitions: %s", indexesInflightOrCompleted, requestedPartitions)); + } else { + LOG.error("All requested index types are inflight or completed: " + partitionIndexTypes); + return Option.empty(); } List finalPartitionsToIndex = partitionIndexTypes.stream() .filter(p -> requestedPartitions.contains(p.getPartitionPath())).collect(Collectors.toList()); diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java index c9d81cee5dc5e..621d1cc99bb64 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java @@ -1167,4 +1167,10 @@ public static Set getInflightMetadataPartitions(HoodieTableConfig tableC public static Set getCompletedMetadataPartitions(HoodieTableConfig tableConfig) { return StringUtils.toSet(tableConfig.getMetadataPartitions()); } + + public static Set getInflightAndCompletedMetadataPartitions(HoodieTableConfig tableConfig) { + Set inflightAndCompletedPartitions = getInflightMetadataPartitions(tableConfig); + inflightAndCompletedPartitions.addAll(getCompletedMetadataPartitions(tableConfig)); + return inflightAndCompletedPartitions; + } }