nsivabalan commented on code in PR #8758:
URL: https://github.com/apache/hudi/pull/8758#discussion_r1213863164


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java:
##########
@@ -351,6 +347,7 @@ public void run() {
                 .filterCompletedInstants().filter(i -> 
i.getTimestamp().equals(instantTime)).firstInstant();
             instant = currentInstant.orElse(instant);
             // so that timeline is not reloaded very frequently
+            // TODO: this does not handle the case that the commit has indeed 
failed. Maybe use HB detection here.

Review Comment:
   can we file follow up tickets for all these please?



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java:
##########
@@ -338,14 +338,20 @@ protected void initMetadataTable(Option<String> 
instantTime) {
    * @param inFlightInstantTimestamp - The in-flight action responsible for 
the metadata table initialization
    */
   private void initializeMetadataTable(Option<String> 
inFlightInstantTimestamp) {
-    if (config.isMetadataTableEnabled()) {
-      HoodieTableMetadataWriter writer = 
SparkHoodieBackedTableMetadataWriter.create(context.getHadoopConf().get(), 
config,
-          context, Option.empty(), inFlightInstantTimestamp);
-      try {
-        writer.close();
-      } catch (Exception e) {
-        throw new HoodieException("Failed to instantiate Metadata table ", e);
+    if (!config.isMetadataTableEnabled()) {
+      LOG.error("================================111111111111111111");

Review Comment:
   lets fix these unintended changes



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkMetadataTableRecordIndex.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.index;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.data.HoodiePairData;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.model.HoodieAvroRecord;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordGlobalLocation;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.config.HoodieIndexConfig;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.data.HoodieJavaPairRDD;
+import org.apache.hudi.data.HoodieJavaRDD;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.exception.TableNotFoundException;
+import org.apache.hudi.metadata.HoodieTableMetadata;
+import org.apache.hudi.metadata.HoodieTableMetadataUtil;
+import org.apache.hudi.metadata.MetadataPartitionType;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.function.PairFlatMapFunction;
+import org.apache.spark.sql.execution.PartitionIdPassthrough;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Tuple2;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.GREATER_THAN;
+
+/**
+ * Hoodie Index implementation backed by the record index present in the 
Metadata Table.
+ */
+public class SparkMetadataTableRecordIndex extends HoodieIndex<Object, Object> 
{
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SparkMetadataTableRecordIndex.class);
+  // The index to fallback upon when record index is not initialized yet.
+  // This should be a global index like record index so that the behavior of 
tagging across partitions is not changed.
+  private static final HoodieIndex.IndexType FALLBACK_INDEX_TYPE = 
IndexType.GLOBAL_SIMPLE;
+
+  public SparkMetadataTableRecordIndex(HoodieWriteConfig config) {
+    super(config);
+  }
+
+  @Override
+  public <R> HoodieData<HoodieRecord<R>> 
tagLocation(HoodieData<HoodieRecord<R>> records, HoodieEngineContext context, 
HoodieTable hoodieTable) throws HoodieIndexException {
+    int fileGroupSize;
+    try {
+      
ValidationUtils.checkState(hoodieTable.getMetaClient().getTableConfig().isMetadataPartitionEnabled(MetadataPartitionType.RECORD_INDEX));
+      fileGroupSize = 
HoodieTableMetadataUtil.getPartitionLatestMergedFileSlices(hoodieTable.getMetaClient(),
 (HoodieTableFileSystemView) hoodieTable.getFileSystemView(),
+          MetadataPartitionType.RECORD_INDEX.getPartitionPath()).size();
+      ValidationUtils.checkState(fileGroupSize > 0, "Record index should have 
at least one file group");
+    } catch (TableNotFoundException | IllegalStateException e) {
+      // This means that record index has not been initialized.
+      LOG.warn(String.format("Record index not initialized so falling back to 
%s for tagging records", FALLBACK_INDEX_TYPE.name()));
+
+      // Fallback to another index so that tagLocation is still accurate and 
there are no duplicates.
+      HoodieWriteConfig otherConfig = 
HoodieWriteConfig.newBuilder().withProperties(config.getProps())
+          
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(FALLBACK_INDEX_TYPE).build()).build();
+      HoodieIndex fallbackIndex = 
SparkHoodieIndexFactory.createIndex(otherConfig);
+
+      // Fallback index needs to be a global index like record index
+      ValidationUtils.checkArgument(fallbackIndex.isGlobal(), "Fallback index 
needs to be a global index like record index");
+
+      return fallbackIndex.tagLocation(records, context, hoodieTable);
+    }
+
+    // final variable required for lamda functions below
+    final int numFileGroups = fileGroupSize;
+
+    // Partition the record keys to lookup such that each partition looks up 
one record index shard
+    JavaRDD<String> partitionedKeyRDD = HoodieJavaRDD.getJavaRDD(records)
+        .map(HoodieRecord::getRecordKey)
+        .keyBy(k -> HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex(k, 
numFileGroups))
+        .partitionBy(new PartitionIdPassthrough(numFileGroups))
+        .map(t -> t._2);
+    ValidationUtils.checkState(partitionedKeyRDD.getNumPartitions() <= 
numFileGroups);
+
+    // Lookup the keys in the record index
+    HoodiePairData<String, HoodieRecordGlobalLocation> keyToLocationPairRDD =
+        HoodieJavaPairRDD.of(partitionedKeyRDD.mapPartitionsToPair(new 
RecordIndexFileGroupLookupFunction(hoodieTable)));
+
+    // Tag the incoming records, as inserts or updates, by joining with 
existing record keys
+    HoodieData<HoodieRecord<R>> taggedRecords = 
tagLocationBackToRecords(keyToLocationPairRDD, records);
+
+    // The number of partitions in the taggedRecords is expected to the 
maximum of the partitions in
+    // keyToLocationPairRDD and records RDD.
+
+    return taggedRecords;
+  }
+
+  @Override
+  public HoodieData<WriteStatus> updateLocation(HoodieData<WriteStatus> 
writeStatuses, HoodieEngineContext context,
+      HoodieTable hoodieTable) {
+    // This is a no-op as metadata record index updates are automatically 
maintained within the metadata table.
+    return writeStatuses;
+  }
+
+  @Override
+  public boolean rollbackCommit(String instantTime) {
+    // Only those deltacommits which have a valid completed commit on the 
dataset are read. Since, the instantTime
+    // is being rolled back on the dataset, we will not load the records from 
the deltacommit and it is virtually
+    // rolled back. In other words, there is no need to rollback any 
deltacommit here except if the deltacommit
+    // was compacted and a new basefile has been created.
+    try {
+      HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
+          
.setBasePath(HoodieTableMetadata.getMetadataTableBasePath(config.getBasePath()))
+          .setConf(new Configuration()).build();
+      HoodieTimeline commitTimeline = 
metaClient.getCommitTimeline().filterCompletedInstants();
+      if (commitTimeline.empty()) {
+        // No compaction yet so no need to check for deltacommits due to the 
logic above
+        return true;
+      }
+
+      if (HoodieTimeline.compareTimestamps(instantTime, GREATER_THAN, 
commitTimeline.lastInstant().get().getTimestamp())) {
+        // After the last compaction so no rollback required as per logic above
+        return true;
+      }
+      LOG.warn("Cannot rollback instant " + instantTime + " because the 
corresponding deltacommit has been compacted "
+          + " in " + commitTimeline.lastInstant().get().getTimestamp());
+      return false;
+    } catch (TableNotFoundException e) {
+      // Metadata table is not setup.  Nothing to rollback.  Exit gracefully.
+      LOG.warn("Cannot rollback instant " + instantTime + " as metadata table 
is not found");
+      return true;
+    }
+  }
+
+  @Override
+  public boolean isGlobal() {
+    return true;
+  }
+
+  @Override
+  public boolean canIndexLogFiles() {
+    return true;
+  }
+
+  @Override
+  public boolean isImplicitWithStorage() {
+    return false;
+  }
+
+  private <R> HoodieData<HoodieRecord<R>> tagLocationBackToRecords(
+      HoodiePairData<String, HoodieRecordGlobalLocation> keyFilenamePair,
+      HoodieData<HoodieRecord<R>> records) {
+    HoodiePairData<String, HoodieRecord<R>> keyRecordPairs =
+        records.mapToPair(record -> new ImmutablePair<>(record.getRecordKey(), 
record));
+    // Here as the records might have more data than keyFilenamePairs (some 
row keys' not found in record index),
+    // we will do left outer join.
+    return keyRecordPairs.leftOuterJoin(keyFilenamePair).values()
+        .map(v -> {
+          HoodieRecord<R> record = v.getLeft();
+          Option<HoodieRecordGlobalLocation> location = 
Option.ofNullable(v.getRight().orElse(null));
+          if (!location.isPresent()) {
+            // No location found.
+            return record;
+          }
+          // Ensure the partitionPath is also set correctly in the key
+          if 
(!record.getPartitionPath().equals(location.get().getPartitionPath())) {
+            record = new HoodieAvroRecord(new HoodieKey(record.getRecordKey(), 
location.get().getPartitionPath()), (HoodieRecordPayload) record.getData());
+          }
+
+          // Perform the tagging. Not using HoodieIndexUtils.getTaggedRecord 
to prevent an additional copy which is not necessary for this index.
+          record.unseal();
+          record.setCurrentLocation(location.get());
+          record.seal();
+          return record;
+        });
+  }
+
+  /**
+   * Function that lookups a list of keys in a single shard of the record index
+   */
+  private static class RecordIndexFileGroupLookupFunction implements 
PairFlatMapFunction<Iterator<String>, String, HoodieRecordGlobalLocation> {
+    private final HoodieTable hoodieTable;
+
+    public RecordIndexFileGroupLookupFunction(HoodieTable hoodieTable) {
+      this.hoodieTable = hoodieTable;
+    }
+
+    @Override
+    public Iterator<Tuple2<String, HoodieRecordGlobalLocation>> 
call(Iterator<String> recordKeyIterator) {
+      List<String> keysToLookup = new ArrayList<>();
+      recordKeyIterator.forEachRemaining(keysToLookup::add);
+
+      // recordIndexInfo object only contains records that are present in 
record_index.
+      Map<String, HoodieRecordGlobalLocation> recordIndexInfo = 
hoodieTable.getMetadataTable().readRecordIndex(keysToLookup);
+
+      HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
+      HoodieTimeline commitsTimeline = 
metaClient.getCommitsTimeline().filterCompletedInstants();
+      return recordIndexInfo.entrySet().stream()
+          .filter(e -> HoodieIndexUtils.checkIfValidCommit(commitsTimeline, 
e.getValue().getInstantTime()))

Review Comment:
   I feel this might be adding some overhead to the latency. Is there any 
optimization we can do here? 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java:
##########
@@ -374,11 +371,15 @@ public void run() {
               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);
+                // Indexes may require WriteStatus which cannot be read from 
the HoodieCommitMetadata. So if the original commit has not

Review Comment:
   if RLI is not enabled, we don't need to be strict here. 



##########
hudi-client/hudi-client-common/src/test/java/org/apache/hudi/common/testutils/HoodieMetadataTestTable.java:
##########
@@ -77,7 +79,8 @@ public HoodieCommitMetadata doWriteOperation(String 
commitTime, WriteOperationTy
     HoodieCommitMetadata commitMetadata = super.doWriteOperation(commitTime, 
operationType, newPartitionsToAdd,
         partitionToFilesNameLengthMap, bootstrap, createInflightCommit);
     if (writer != null && !createInflightCommit) {
-      writer.update(commitMetadata, commitTime, false);
+      writer.performTableServices(Option.of(commitTime));
+      writer.update(commitMetadata, 
HoodieListData.eager(Collections.EMPTY_LIST), commitTime);

Review Comment:
   we should not set empty list. Lets try to set some valid entries. or atleast 
empty javaRdd. 
   If we wish to expand any of existing tests to enable RLI, then fixing this 
is very much required. 
   If we are adding new tests specifically for RLI, may be we can punt this for 
now



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataWriteUtils.java:
##########
@@ -170,6 +171,17 @@ public static HoodieWriteConfig createMetadataWriteConfig(
           throw new HoodieMetadataException("Unsupported Metrics Reporter type 
" + writeConfig.getMetricsReporterType());
       }
     }
-    return builder.build();
+
+    HoodieWriteConfig metadataWriteConfig = builder.build();
+    // Inline compaction and auto clean is required as we do not expose this 
table outside
+    ValidationUtils.checkArgument(!metadataWriteConfig.isAutoClean(), 
"Cleaning is controlled internally for Metadata table.");

Review Comment:
   we have fixed some of these configs to be hard coded within code. so, we 
don't really need to validate here. can you check those and remove the 
validations as applicable. if its not configurable by end user, we don't really 
need to validate 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RunIndexActionExecutor.java:
##########
@@ -374,11 +371,15 @@ public void run() {
               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);
+                // Indexes may require WriteStatus which cannot be read from 
the HoodieCommitMetadata. So if the original commit has not
+                // written to the MDT then we cannot sync that commit here
+                // TODO: maybe the above check for timeout should be based on 
HB rather than busy loop forever.
+                throw new HoodieIndexException(String.format("Cannot sync 
instant to MDT: %s", instant));
+                //HoodieCommitMetadata commitMetadata = 
HoodieCommitMetadata.fromBytes(

Review Comment:
   can we fix the commented out code properly. and file follow up tickets w/ 
good amount of context and details



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java:
##########
@@ -100,15 +99,6 @@ public Option<HoodieIndexPlan> execute() {
       // get last completed instant
       Option<HoodieInstant> indexUptoInstant = 
table.getActiveTimeline().getContiguousCompletedWriteTimeline().lastInstant();
       if (indexUptoInstant.isPresent()) {
-        // start initializing file groups

Review Comment:
   after this patch, whats the expected behavior in this case. 
   ie. someone tries to enable col stats using async indexer where metadata 
itself wasn't present for the data table of interest? 



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkMetadataTableRecordIndex.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.index;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.data.HoodiePairData;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.model.HoodieAvroRecord;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordGlobalLocation;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.config.HoodieIndexConfig;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.data.HoodieJavaPairRDD;
+import org.apache.hudi.data.HoodieJavaRDD;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.exception.TableNotFoundException;
+import org.apache.hudi.metadata.HoodieTableMetadata;
+import org.apache.hudi.metadata.HoodieTableMetadataUtil;
+import org.apache.hudi.metadata.MetadataPartitionType;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.function.PairFlatMapFunction;
+import org.apache.spark.sql.execution.PartitionIdPassthrough;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Tuple2;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.GREATER_THAN;
+
+/**
+ * Hoodie Index implementation backed by the record index present in the 
Metadata Table.
+ */
+public class SparkMetadataTableRecordIndex extends HoodieIndex<Object, Object> 
{
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SparkMetadataTableRecordIndex.class);
+  // The index to fallback upon when record index is not initialized yet.
+  // This should be a global index like record index so that the behavior of 
tagging across partitions is not changed.
+  private static final HoodieIndex.IndexType FALLBACK_INDEX_TYPE = 
IndexType.GLOBAL_SIMPLE;
+
+  public SparkMetadataTableRecordIndex(HoodieWriteConfig config) {
+    super(config);
+  }
+
+  @Override
+  public <R> HoodieData<HoodieRecord<R>> 
tagLocation(HoodieData<HoodieRecord<R>> records, HoodieEngineContext context, 
HoodieTable hoodieTable) throws HoodieIndexException {
+    int fileGroupSize;
+    try {
+      
ValidationUtils.checkState(hoodieTable.getMetaClient().getTableConfig().isMetadataPartitionEnabled(MetadataPartitionType.RECORD_INDEX));
+      fileGroupSize = 
HoodieTableMetadataUtil.getPartitionLatestMergedFileSlices(hoodieTable.getMetaClient(),
 (HoodieTableFileSystemView) hoodieTable.getFileSystemView(),
+          MetadataPartitionType.RECORD_INDEX.getPartitionPath()).size();
+      ValidationUtils.checkState(fileGroupSize > 0, "Record index should have 
at least one file group");
+    } catch (TableNotFoundException | IllegalStateException e) {
+      // This means that record index has not been initialized.
+      LOG.warn(String.format("Record index not initialized so falling back to 
%s for tagging records", FALLBACK_INDEX_TYPE.name()));
+
+      // Fallback to another index so that tagLocation is still accurate and 
there are no duplicates.
+      HoodieWriteConfig otherConfig = 
HoodieWriteConfig.newBuilder().withProperties(config.getProps())
+          
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(FALLBACK_INDEX_TYPE).build()).build();
+      HoodieIndex fallbackIndex = 
SparkHoodieIndexFactory.createIndex(otherConfig);
+
+      // Fallback index needs to be a global index like record index
+      ValidationUtils.checkArgument(fallbackIndex.isGlobal(), "Fallback index 
needs to be a global index like record index");
+
+      return fallbackIndex.tagLocation(records, context, hoodieTable);
+    }
+
+    // final variable required for lamda functions below
+    final int numFileGroups = fileGroupSize;
+
+    // Partition the record keys to lookup such that each partition looks up 
one record index shard
+    JavaRDD<String> partitionedKeyRDD = HoodieJavaRDD.getJavaRDD(records)
+        .map(HoodieRecord::getRecordKey)
+        .keyBy(k -> HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex(k, 
numFileGroups))
+        .partitionBy(new PartitionIdPassthrough(numFileGroups))
+        .map(t -> t._2);
+    ValidationUtils.checkState(partitionedKeyRDD.getNumPartitions() <= 
numFileGroups);
+
+    // Lookup the keys in the record index
+    HoodiePairData<String, HoodieRecordGlobalLocation> keyToLocationPairRDD =
+        HoodieJavaPairRDD.of(partitionedKeyRDD.mapPartitionsToPair(new 
RecordIndexFileGroupLookupFunction(hoodieTable)));
+
+    // Tag the incoming records, as inserts or updates, by joining with 
existing record keys
+    HoodieData<HoodieRecord<R>> taggedRecords = 
tagLocationBackToRecords(keyToLocationPairRDD, records);
+
+    // The number of partitions in the taggedRecords is expected to the 
maximum of the partitions in
+    // keyToLocationPairRDD and records RDD.
+
+    return taggedRecords;
+  }
+
+  @Override
+  public HoodieData<WriteStatus> updateLocation(HoodieData<WriteStatus> 
writeStatuses, HoodieEngineContext context,
+      HoodieTable hoodieTable) {
+    // This is a no-op as metadata record index updates are automatically 
maintained within the metadata table.
+    return writeStatuses;
+  }
+
+  @Override
+  public boolean rollbackCommit(String instantTime) {
+    // Only those deltacommits which have a valid completed commit on the 
dataset are read. Since, the instantTime
+    // is being rolled back on the dataset, we will not load the records from 
the deltacommit and it is virtually
+    // rolled back. In other words, there is no need to rollback any 
deltacommit here except if the deltacommit
+    // was compacted and a new basefile has been created.
+    try {
+      HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()

Review Comment:
   mdtMetaClient or metadataMetaClient



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkMetadataTableRecordIndex.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.index;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.data.HoodiePairData;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.model.HoodieAvroRecord;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordGlobalLocation;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.config.HoodieIndexConfig;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.data.HoodieJavaPairRDD;
+import org.apache.hudi.data.HoodieJavaRDD;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.exception.TableNotFoundException;
+import org.apache.hudi.metadata.HoodieTableMetadata;
+import org.apache.hudi.metadata.HoodieTableMetadataUtil;
+import org.apache.hudi.metadata.MetadataPartitionType;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.function.PairFlatMapFunction;
+import org.apache.spark.sql.execution.PartitionIdPassthrough;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Tuple2;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.GREATER_THAN;
+
+/**
+ * Hoodie Index implementation backed by the record index present in the 
Metadata Table.
+ */
+public class SparkMetadataTableRecordIndex extends HoodieIndex<Object, Object> 
{
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SparkMetadataTableRecordIndex.class);
+  // The index to fallback upon when record index is not initialized yet.
+  // This should be a global index like record index so that the behavior of 
tagging across partitions is not changed.
+  private static final HoodieIndex.IndexType FALLBACK_INDEX_TYPE = 
IndexType.GLOBAL_SIMPLE;
+
+  public SparkMetadataTableRecordIndex(HoodieWriteConfig config) {
+    super(config);
+  }
+
+  @Override
+  public <R> HoodieData<HoodieRecord<R>> 
tagLocation(HoodieData<HoodieRecord<R>> records, HoodieEngineContext context, 
HoodieTable hoodieTable) throws HoodieIndexException {
+    int fileGroupSize;
+    try {
+      
ValidationUtils.checkState(hoodieTable.getMetaClient().getTableConfig().isMetadataPartitionEnabled(MetadataPartitionType.RECORD_INDEX));
+      fileGroupSize = 
HoodieTableMetadataUtil.getPartitionLatestMergedFileSlices(hoodieTable.getMetaClient(),
 (HoodieTableFileSystemView) hoodieTable.getFileSystemView(),
+          MetadataPartitionType.RECORD_INDEX.getPartitionPath()).size();
+      ValidationUtils.checkState(fileGroupSize > 0, "Record index should have 
at least one file group");
+    } catch (TableNotFoundException | IllegalStateException e) {
+      // This means that record index has not been initialized.
+      LOG.warn(String.format("Record index not initialized so falling back to 
%s for tagging records", FALLBACK_INDEX_TYPE.name()));
+
+      // Fallback to another index so that tagLocation is still accurate and 
there are no duplicates.
+      HoodieWriteConfig otherConfig = 
HoodieWriteConfig.newBuilder().withProperties(config.getProps())
+          
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(FALLBACK_INDEX_TYPE).build()).build();
+      HoodieIndex fallbackIndex = 
SparkHoodieIndexFactory.createIndex(otherConfig);
+
+      // Fallback index needs to be a global index like record index
+      ValidationUtils.checkArgument(fallbackIndex.isGlobal(), "Fallback index 
needs to be a global index like record index");
+
+      return fallbackIndex.tagLocation(records, context, hoodieTable);
+    }
+
+    // final variable required for lamda functions below
+    final int numFileGroups = fileGroupSize;
+
+    // Partition the record keys to lookup such that each partition looks up 
one record index shard
+    JavaRDD<String> partitionedKeyRDD = HoodieJavaRDD.getJavaRDD(records)
+        .map(HoodieRecord::getRecordKey)
+        .keyBy(k -> HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex(k, 
numFileGroups))
+        .partitionBy(new PartitionIdPassthrough(numFileGroups))
+        .map(t -> t._2);
+    ValidationUtils.checkState(partitionedKeyRDD.getNumPartitions() <= 
numFileGroups);
+
+    // Lookup the keys in the record index
+    HoodiePairData<String, HoodieRecordGlobalLocation> keyToLocationPairRDD =
+        HoodieJavaPairRDD.of(partitionedKeyRDD.mapPartitionsToPair(new 
RecordIndexFileGroupLookupFunction(hoodieTable)));
+
+    // Tag the incoming records, as inserts or updates, by joining with 
existing record keys
+    HoodieData<HoodieRecord<R>> taggedRecords = 
tagLocationBackToRecords(keyToLocationPairRDD, records);
+
+    // The number of partitions in the taggedRecords is expected to the 
maximum of the partitions in
+    // keyToLocationPairRDD and records RDD.
+
+    return taggedRecords;
+  }
+
+  @Override
+  public HoodieData<WriteStatus> updateLocation(HoodieData<WriteStatus> 
writeStatuses, HoodieEngineContext context,
+      HoodieTable hoodieTable) {
+    // This is a no-op as metadata record index updates are automatically 
maintained within the metadata table.
+    return writeStatuses;
+  }
+
+  @Override
+  public boolean rollbackCommit(String instantTime) {
+    // Only those deltacommits which have a valid completed commit on the 
dataset are read. Since, the instantTime
+    // is being rolled back on the dataset, we will not load the records from 
the deltacommit and it is virtually
+    // rolled back. In other words, there is no need to rollback any 
deltacommit here except if the deltacommit
+    // was compacted and a new basefile has been created.
+    try {
+      HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
+          
.setBasePath(HoodieTableMetadata.getMetadataTableBasePath(config.getBasePath()))
+          .setConf(new Configuration()).build();
+      HoodieTimeline commitTimeline = 
metaClient.getCommitTimeline().filterCompletedInstants();
+      if (commitTimeline.empty()) {
+        // No compaction yet so no need to check for deltacommits due to the 
logic above
+        return true;
+      }
+
+      if (HoodieTimeline.compareTimestamps(instantTime, GREATER_THAN, 
commitTimeline.lastInstant().get().getTimestamp())) {
+        // After the last compaction so no rollback required as per logic above
+        return true;
+      }
+      LOG.warn("Cannot rollback instant " + instantTime + " because the 
corresponding deltacommit has been compacted "
+          + " in " + commitTimeline.lastInstant().get().getTimestamp());
+      return false;
+    } catch (TableNotFoundException e) {
+      // Metadata table is not setup.  Nothing to rollback.  Exit gracefully.
+      LOG.warn("Cannot rollback instant " + instantTime + " as metadata table 
is not found");
+      return true;
+    }
+  }
+
+  @Override
+  public boolean isGlobal() {
+    return true;
+  }
+
+  @Override
+  public boolean canIndexLogFiles() {
+    return true;
+  }
+
+  @Override
+  public boolean isImplicitWithStorage() {
+    return false;
+  }
+
+  private <R> HoodieData<HoodieRecord<R>> tagLocationBackToRecords(
+      HoodiePairData<String, HoodieRecordGlobalLocation> keyFilenamePair,
+      HoodieData<HoodieRecord<R>> records) {
+    HoodiePairData<String, HoodieRecord<R>> keyRecordPairs =
+        records.mapToPair(record -> new ImmutablePair<>(record.getRecordKey(), 
record));
+    // Here as the records might have more data than keyFilenamePairs (some 
row keys' not found in record index),
+    // we will do left outer join.
+    return keyRecordPairs.leftOuterJoin(keyFilenamePair).values()
+        .map(v -> {
+          HoodieRecord<R> record = v.getLeft();
+          Option<HoodieRecordGlobalLocation> location = 
Option.ofNullable(v.getRight().orElse(null));
+          if (!location.isPresent()) {
+            // No location found.
+            return record;
+          }
+          // Ensure the partitionPath is also set correctly in the key
+          if 
(!record.getPartitionPath().equals(location.get().getPartitionPath())) {

Review Comment:
   lets create a follow up ticket. this works for update partition path = 
false. 
   but we also need to fix update partition path = true. 
   btw, we might need to create a new config for this as well. we have one 
config global bloom, one for global simple. and we might need one for RLI as 
well 



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/SparkMetadataTableRecordIndex.java:
##########
@@ -0,0 +1,227 @@
+/*
+ * 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.index;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.data.HoodieData;
+import org.apache.hudi.common.data.HoodiePairData;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.model.HoodieAvroRecord;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecordGlobalLocation;
+import org.apache.hudi.common.model.HoodieRecordPayload;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.table.view.HoodieTableFileSystemView;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.common.util.collection.ImmutablePair;
+import org.apache.hudi.config.HoodieIndexConfig;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.data.HoodieJavaPairRDD;
+import org.apache.hudi.data.HoodieJavaRDD;
+import org.apache.hudi.exception.HoodieIndexException;
+import org.apache.hudi.exception.TableNotFoundException;
+import org.apache.hudi.metadata.HoodieTableMetadata;
+import org.apache.hudi.metadata.HoodieTableMetadataUtil;
+import org.apache.hudi.metadata.MetadataPartitionType;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.function.PairFlatMapFunction;
+import org.apache.spark.sql.execution.PartitionIdPassthrough;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Tuple2;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.GREATER_THAN;
+
+/**
+ * Hoodie Index implementation backed by the record index present in the 
Metadata Table.
+ */
+public class SparkMetadataTableRecordIndex extends HoodieIndex<Object, Object> 
{
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SparkMetadataTableRecordIndex.class);
+  // The index to fallback upon when record index is not initialized yet.
+  // This should be a global index like record index so that the behavior of 
tagging across partitions is not changed.
+  private static final HoodieIndex.IndexType FALLBACK_INDEX_TYPE = 
IndexType.GLOBAL_SIMPLE;
+
+  public SparkMetadataTableRecordIndex(HoodieWriteConfig config) {
+    super(config);
+  }
+
+  @Override
+  public <R> HoodieData<HoodieRecord<R>> 
tagLocation(HoodieData<HoodieRecord<R>> records, HoodieEngineContext context, 
HoodieTable hoodieTable) throws HoodieIndexException {
+    int fileGroupSize;
+    try {
+      
ValidationUtils.checkState(hoodieTable.getMetaClient().getTableConfig().isMetadataPartitionEnabled(MetadataPartitionType.RECORD_INDEX));
+      fileGroupSize = 
HoodieTableMetadataUtil.getPartitionLatestMergedFileSlices(hoodieTable.getMetaClient(),
 (HoodieTableFileSystemView) hoodieTable.getFileSystemView(),
+          MetadataPartitionType.RECORD_INDEX.getPartitionPath()).size();
+      ValidationUtils.checkState(fileGroupSize > 0, "Record index should have 
at least one file group");
+    } catch (TableNotFoundException | IllegalStateException e) {
+      // This means that record index has not been initialized.
+      LOG.warn(String.format("Record index not initialized so falling back to 
%s for tagging records", FALLBACK_INDEX_TYPE.name()));
+
+      // Fallback to another index so that tagLocation is still accurate and 
there are no duplicates.
+      HoodieWriteConfig otherConfig = 
HoodieWriteConfig.newBuilder().withProperties(config.getProps())
+          
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(FALLBACK_INDEX_TYPE).build()).build();
+      HoodieIndex fallbackIndex = 
SparkHoodieIndexFactory.createIndex(otherConfig);
+
+      // Fallback index needs to be a global index like record index
+      ValidationUtils.checkArgument(fallbackIndex.isGlobal(), "Fallback index 
needs to be a global index like record index");
+
+      return fallbackIndex.tagLocation(records, context, hoodieTable);
+    }
+
+    // final variable required for lamda functions below
+    final int numFileGroups = fileGroupSize;
+
+    // Partition the record keys to lookup such that each partition looks up 
one record index shard
+    JavaRDD<String> partitionedKeyRDD = HoodieJavaRDD.getJavaRDD(records)
+        .map(HoodieRecord::getRecordKey)
+        .keyBy(k -> HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex(k, 
numFileGroups))
+        .partitionBy(new PartitionIdPassthrough(numFileGroups))
+        .map(t -> t._2);
+    ValidationUtils.checkState(partitionedKeyRDD.getNumPartitions() <= 
numFileGroups);
+
+    // Lookup the keys in the record index
+    HoodiePairData<String, HoodieRecordGlobalLocation> keyToLocationPairRDD =
+        HoodieJavaPairRDD.of(partitionedKeyRDD.mapPartitionsToPair(new 
RecordIndexFileGroupLookupFunction(hoodieTable)));
+
+    // Tag the incoming records, as inserts or updates, by joining with 
existing record keys
+    HoodieData<HoodieRecord<R>> taggedRecords = 
tagLocationBackToRecords(keyToLocationPairRDD, records);
+
+    // The number of partitions in the taggedRecords is expected to the 
maximum of the partitions in
+    // keyToLocationPairRDD and records RDD.
+
+    return taggedRecords;
+  }
+
+  @Override
+  public HoodieData<WriteStatus> updateLocation(HoodieData<WriteStatus> 
writeStatuses, HoodieEngineContext context,
+      HoodieTable hoodieTable) {
+    // This is a no-op as metadata record index updates are automatically 
maintained within the metadata table.
+    return writeStatuses;
+  }
+
+  @Override
+  public boolean rollbackCommit(String instantTime) {
+    // Only those deltacommits which have a valid completed commit on the 
dataset are read. Since, the instantTime
+    // is being rolled back on the dataset, we will not load the records from 
the deltacommit and it is virtually
+    // rolled back. In other words, there is no need to rollback any 
deltacommit here except if the deltacommit
+    // was compacted and a new basefile has been created.
+    try {
+      HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
+          
.setBasePath(HoodieTableMetadata.getMetadataTableBasePath(config.getBasePath()))
+          .setConf(new Configuration()).build();
+      HoodieTimeline commitTimeline = 
metaClient.getCommitTimeline().filterCompletedInstants();
+      if (commitTimeline.empty()) {
+        // No compaction yet so no need to check for deltacommits due to the 
logic above
+        return true;
+      }
+
+      if (HoodieTimeline.compareTimestamps(instantTime, GREATER_THAN, 
commitTimeline.lastInstant().get().getTimestamp())) {
+        // After the last compaction so no rollback required as per logic above
+        return true;
+      }
+      LOG.warn("Cannot rollback instant " + instantTime + " because the 
corresponding deltacommit has been compacted "
+          + " in " + commitTimeline.lastInstant().get().getTimestamp());
+      return false;
+    } catch (TableNotFoundException e) {
+      // Metadata table is not setup.  Nothing to rollback.  Exit gracefully.
+      LOG.warn("Cannot rollback instant " + instantTime + " as metadata table 
is not found");
+      return true;
+    }
+  }
+
+  @Override
+  public boolean isGlobal() {
+    return true;
+  }
+
+  @Override
+  public boolean canIndexLogFiles() {
+    return true;
+  }
+
+  @Override
+  public boolean isImplicitWithStorage() {
+    return false;
+  }
+
+  private <R> HoodieData<HoodieRecord<R>> tagLocationBackToRecords(
+      HoodiePairData<String, HoodieRecordGlobalLocation> keyFilenamePair,
+      HoodieData<HoodieRecord<R>> records) {
+    HoodiePairData<String, HoodieRecord<R>> keyRecordPairs =
+        records.mapToPair(record -> new ImmutablePair<>(record.getRecordKey(), 
record));
+    // Here as the records might have more data than keyFilenamePairs (some 
row keys' not found in record index),
+    // we will do left outer join.
+    return keyRecordPairs.leftOuterJoin(keyFilenamePair).values()
+        .map(v -> {
+          HoodieRecord<R> record = v.getLeft();
+          Option<HoodieRecordGlobalLocation> location = 
Option.ofNullable(v.getRight().orElse(null));
+          if (!location.isPresent()) {
+            // No location found.
+            return record;
+          }
+          // Ensure the partitionPath is also set correctly in the key
+          if 
(!record.getPartitionPath().equals(location.get().getPartitionPath())) {
+            record = new HoodieAvroRecord(new HoodieKey(record.getRecordKey(), 
location.get().getPartitionPath()), (HoodieRecordPayload) record.getData());

Review Comment:
   Does this work of the record type is Spark?  
   may be we can invoke this method in HoodieIndexUtils
   ```
     public static <R> HoodieRecord<R> getTaggedRecord(HoodieRecord<R> 
inputRecord, Option<HoodieRecordLocation> location) {
   ```



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java:
##########
@@ -338,14 +338,20 @@ protected void initMetadataTable(Option<String> 
instantTime) {
    * @param inFlightInstantTimestamp - The in-flight action responsible for 
the metadata table initialization
    */
   private void initializeMetadataTable(Option<String> 
inFlightInstantTimestamp) {
-    if (config.isMetadataTableEnabled()) {
-      HoodieTableMetadataWriter writer = 
SparkHoodieBackedTableMetadataWriter.create(context.getHadoopConf().get(), 
config,
-          context, Option.empty(), inFlightInstantTimestamp);
-      try {
-        writer.close();
-      } catch (Exception e) {
-        throw new HoodieException("Failed to instantiate Metadata table ", e);
+    if (!config.isMetadataTableEnabled()) {
+      LOG.error("================================111111111111111111");
+      return;
+    }
+
+    try (HoodieTableMetadataWriter writer = 
SparkHoodieBackedTableMetadataWriter.create(context.getHadoopConf().get(), 
config,
+        context, Option.empty(), inFlightInstantTimestamp)) {
+      if (writer.isInitialized()) {
+        writer.performTableServices(inFlightInstantTimestamp);
+      } else {
+        throw new HoodieException((".................2222222222"));

Review Comment:
   same here



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataWriter.java:
##########
@@ -99,7 +90,25 @@ public interface HoodieTableMetadataWriter extends 
Serializable, AutoCloseable {
    * Deletes the given metadata partitions. This path reuses DELETE_PARTITION 
operation.
    *
    * @param instantTime - instant time when replacecommit corresponding to the 
drop will be recorded in the metadata timeline
-   * @param partitions - list of {@link MetadataPartitionType} to drop
+   * @param partitions  - list of {@link MetadataPartitionType} to drop
    */
   void deletePartitions(String instantTime, List<MetadataPartitionType> 
partitions);
+
+  /**
+   * It returns write client for metadata table.
+   */
+  BaseHoodieWriteClient getWriteClient();

Review Comment:
   +1



-- 
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


Reply via email to