nsivabalan commented on a change in pull request #4693: URL: https://github.com/apache/hudi/pull/4693#discussion_r835780780
########## File path: hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java ########## @@ -0,0 +1,263 @@ +/* + * 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.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.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.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; + +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 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; + +/** + * Reads the index plan and executes the plan. + * It also reconciles updates on data timeline while indexing was in progress. + */ +public class RunIndexActionExecutor<T extends HoodieRecordPayload, I, K, O> extends BaseActionExecutor<T, I, K, O, Option<HoodieIndexCommitMetadata>> { + + 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; + + // 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<T, I, K, O> table, String instantTime) { + super(context, config, table, instantTime); + this.txnManager = new TransactionManager(config, table.getMetaClient().getFs()); + } + + @Override + public Option<HoodieIndexCommitMetadata> 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))); + try { + // read HoodieIndexPlan + HoodieIndexPlan indexPlan = TimelineMetadataUtils.deserializeIndexPlan(table.getActiveTimeline().readIndexPlanAsBytes(indexInstant).get()); + List<HoodieIndexPartitionInfo> 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 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<HoodieInstant> 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<String> 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 IndexingCheckTask(metadataWriter, instantsToIndex, metadataCompletedTimestamps, table.getMetaClient())); + try { + postRequestIndexingTaskFuture.get(config.getIndexingCheckTimeout(), TimeUnit.SECONDS); + } catch (TimeoutException | InterruptedException | ExecutionException e) { + postRequestIndexingTaskFuture.cancel(true); + } finally { + executorService.shutdownNow(); + } + // save index commit metadata and return + List<HoodieIndexPartitionInfo> finalIndexPartitionInfos = indexPartitionInfos.stream() Review comment: we should be updating the completed list of partitions in tableConfig at the end. Thats what guards the readers to use the fully built out partitions. I see from current path, we update it before doing catch up. Lets take it towards the end when we are fully sure that certain partition is fully built and ready for readers to start cosuming. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@hudi.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org