codope commented on a change in pull request #4693:
URL: https://github.com/apache/hudi/pull/4693#discussion_r838165685



##########
File path: 
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.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;
+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;
+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;
+
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+
+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;
+
+/**
+ * Schedules INDEX action.
+ * <li>
+ * 1. Fetch last completed instant on data timeline.
+ * 2. Write the index plan to the <instant>.index.requested.
+ * 3. Initialize file groups for the enabled partition types within a 
transaction.
+ * </li>
+ */
+public class ScheduleIndexActionExecutor<T extends HoodieRecordPayload, I, K, 
O> extends BaseActionExecutor<T, I, K, O, Option<HoodieIndexPlan>> {
+
+  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<MetadataPartitionType> partitionsToIndex;
+  private final TransactionManager txnManager;
+
+  public ScheduleIndexActionExecutor(HoodieEngineContext context,
+                                     HoodieWriteConfig config,
+                                     HoodieTable<T, I, K, O> table,
+                                     String instantTime,
+                                     List<MetadataPartitionType> 
partitionsToIndex) {
+    super(context, config, table, instantTime);
+    this.partitionsToIndex = partitionsToIndex;
+    this.txnManager = new TransactionManager(config, 
table.getMetaClient().getFs());
+  }
+
+  @Override
+  public Option<HoodieIndexPlan> 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()));
+    }
+    // make sure that it is idempotent, check with previously pending index 
operations.
+    Set<String> 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<String> requestedPartitions = 
partitionsToIndex.stream().map(MetadataPartitionType::getPartitionPath).collect(Collectors.toSet());
+    requestedPartitions.removeAll(indexesInflightOrCompleted);
+    if (!requestedPartitions.isEmpty()) {

Review comment:
       > how do we handle the scenario where we fail after updating the 
tableConfig/hoodie.props, but before writing the requested indexing to the 
timeline.
   
   Case 1: scheduling index
   No table config is updated while scehduling. What could happen here is for 
e.g. column_stats partition was initialized and executor failed just before 
writing the requested indexing to the timeline. When scheduling is 
re-triggered, it will begin with a new instant and redo the whole thing. So, 
this isn't truly idempotent. I need to add a check to ignore if initialization 
was complete.
   
   Case 2: building index
   "completed" partitions table config gets updated only after index catchup 
and partition is fully built out. If the table config gets updated but executor 
failed just before writing the completed indexing to the timeline, the regular 
writers will see that the partition is availbale for updates. There will be an 
inflight indexing instant in the timeline forever. When indexer is 
re-triggered, it will fail again because it will that there is already an 
inflight indexing instant with the same timestamp.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to