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


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/IndexStats.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.client;
+
+import org.apache.hudi.common.model.HoodieRecordDelegate;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Class to hold all index stats required to generate Metadata records for all 
enabled partitions.
+ * Supported stats are record level index stats and secondary index stats.
+ */
+public class IndexStats implements Serializable {
+  private static final long serialVersionUID = 1L;
+
+  private final List<HoodieRecordDelegate> writtenRecordDelegates = new 
ArrayList<>();
+  private final Map<String, List<SecondaryIndexStats>> secondaryIndexStats = 
new HashMap<>();
+
+  void addHoodieRecordDelegate(HoodieRecordDelegate hoodieRecordDelegate) {
+    this.writtenRecordDelegates.add(hoodieRecordDelegate);
+  }
+
+  public List<HoodieRecordDelegate> getWrittenRecordDelegates() {
+    return writtenRecordDelegates;
+  }
+
+  public void addSecondaryIndexStats(String secondaryIndexPartitionPath, 
String recordKey, String secondaryIndexValue, boolean isDeleted) {
+    secondaryIndexStats.computeIfAbsent(secondaryIndexPartitionPath, k -> new 
ArrayList<>())
+        .add(new SecondaryIndexStats(recordKey, secondaryIndexValue, 
isDeleted));
+  }
+
+  public Map<String, List<SecondaryIndexStats>> getSecondaryIndexStats() {
+    return secondaryIndexStats;
+  }
+
+  void clear() {
+    this.writtenRecordDelegates.clear();

Review Comment:
   can you clear secondaryIndexStats as well 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/SecondaryIndexStats.java:
##########
@@ -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.
+ */
+
+package org.apache.hudi.client;
+
+/**
+ * Class is used to hold secondary index metadata updates. These updates are 
generated from
+ * various write handles during write.
+ */
+public class SecondaryIndexStats {
+
+  private String recordKey;

Review Comment:
   final 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java:
##########
@@ -173,6 +178,23 @@ record = record.prependMetaFields(schema, 
writeSchemaWithMetaFields, new Metadat
     }
   }
 
+  private void trackMetadataIndexStats(HoodieRecord record) {
+    if (!config.isSecondaryIndexEnabled() || secondaryIndexDefns.isEmpty() || 
!config.isMetadataStreamingWritesEnabled(hoodieTable.getMetaClient().getTableConfig().getTableVersion()))
 {

Review Comment:
   can you move this condition to HoodieWriteHandle and use it all sub class. 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java:
##########
@@ -546,12 +568,70 @@ public List<WriteStatus> close() {
         status.getStat().setFileSizeInBytes(logFileSize);
       }
 
+      // generate Secondary index stats if streaming is enabled.
+      if (generateStatsForStreamingMetadataWrites) {
+        
trackMetadataIndexStatsForStreamingMetadataWrites(fileSliceOpt.or(getFileSlice()),
 statuses.stream().map(status -> 
status.getStat().getPath()).collect(Collectors.toList()),
+            statuses.get(statuses.size() - 1));

Review Comment:
   can you add java docs on why we update just the last WriteStatus when we 
have more than 1 WriteStatus



##########
hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/io/TestMergeHandle.java:
##########
@@ -81,4 +85,57 @@ public void testMergeHandleRLIStats() throws IOException {
     assertEquals(records.size(), writeStatus.getStat().getNumWrites());
     assertEquals(10, writeStatus.getStat().getNumUpdateWrites());
   }
+
+  @Test
+  public void testMergeHandleSecondaryIndexStats() throws Exception {
+    // init config and table
+    HoodieWriteConfig config = getConfigBuilder(basePath)
+        .withPopulateMetaFields(false)
+        
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder().withRemoteServerPort(timelineServicePort).build())
+        .withMetadataConfig(HoodieMetadataConfig.newBuilder()
+            .enable(true)
+            .withEnableRecordIndex(true)
+            .withStreamingWriteEnabled(true)
+            .withSecondaryIndexEnabled(true)
+            .withSecondaryIndexName("sec-rider")
+            .withSecondaryIndexForColumn("rider")
+            .build())
+        
.withKeyGenerator(KeyGeneratorForDataGeneratorRecords.class.getCanonicalName())
+        .build();
+    HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) 
HoodieSparkTable.create(config, new HoodieLocalEngineContext(storageConf), 
metaClient);
+    HoodieTableMetadataWriter metadataWriter = 
SparkMetadataWriterFactory.create(storageConf, config, context, 
table.getMetaClient().getTableConfig());
+    metadataWriter.close();
+
+    // one round per partition
+    String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0];
+    // init some args
+    String fileId = UUID.randomUUID().toString();
+    String instantTime = "000";
+
+    // Create a parquet file
+    config.setSchema(TRIP_EXAMPLE_SCHEMA);
+    metaClient = HoodieTableMetaClient.reload(metaClient);
+    table = (HoodieSparkCopyOnWriteTable) 
HoodieSparkCopyOnWriteTable.create(config, context, metaClient);
+    HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new 
String[] {partitionPath});
+
+    Pair<WriteStatus, List<HoodieRecord>> statusListPair = 
createParquetFile(config, table, partitionPath, fileId, instantTime, 
dataGenerator, false);
+    WriteStatus writeStatus = statusListPair.getLeft();
+    List<HoodieRecord> records = statusListPair.getRight();
+    assertEquals(records.size(), writeStatus.getTotalRecords());
+    assertEquals(0, writeStatus.getTotalErrorRecords());
+
+    instantTime = "001";
+    List<HoodieRecord> updates = 
dataGenerator.generateUniqueUpdates(instantTime, 10);
+    HoodieMergeHandle mergeHandle = new HoodieMergeHandle(config, instantTime, 
table, updates.iterator(), partitionPath, fileId, new 
LocalTaskContextSupplier(),
+        new HoodieBaseFile(writeStatus.getStat().getPath()), Option.of(new 
KeyGeneratorForDataGeneratorRecords(config.getProps())));
+    HoodieMergeHelper.newInstance().runMerge(table, mergeHandle);
+    writeStatus = mergeHandle.writeStatus;
+    // verify stats after merge
+    assertEquals(records.size(), writeStatus.getStat().getNumWrites());
+    assertEquals(10, writeStatus.getStat().getNumUpdateWrites());
+    // verify secondary index stats
+    assertEquals(1, 
writeStatus.getIndexStats().getSecondaryIndexStats().size());
+    // 10 si records for old secondary keys and 10 for new secondary keys
+    assertEquals(20, 
writeStatus.getIndexStats().getSecondaryIndexStats().values().stream().findFirst().get().size());

Review Comment:
   lets ensure there is a mix of updates and deletes as well. 
   



##########
hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/io/TestMergeHandle.java:
##########
@@ -81,4 +85,57 @@ public void testMergeHandleRLIStats() throws IOException {
     assertEquals(records.size(), writeStatus.getStat().getNumWrites());
     assertEquals(10, writeStatus.getStat().getNumUpdateWrites());
   }
+
+  @Test
+  public void testMergeHandleSecondaryIndexStats() throws Exception {
+    // init config and table
+    HoodieWriteConfig config = getConfigBuilder(basePath)
+        .withPopulateMetaFields(false)
+        
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder().withRemoteServerPort(timelineServicePort).build())
+        .withMetadataConfig(HoodieMetadataConfig.newBuilder()
+            .enable(true)
+            .withEnableRecordIndex(true)
+            .withStreamingWriteEnabled(true)
+            .withSecondaryIndexEnabled(true)
+            .withSecondaryIndexName("sec-rider")
+            .withSecondaryIndexForColumn("rider")
+            .build())
+        
.withKeyGenerator(KeyGeneratorForDataGeneratorRecords.class.getCanonicalName())
+        .build();
+    HoodieSparkCopyOnWriteTable table = (HoodieSparkCopyOnWriteTable) 
HoodieSparkTable.create(config, new HoodieLocalEngineContext(storageConf), 
metaClient);
+    HoodieTableMetadataWriter metadataWriter = 
SparkMetadataWriterFactory.create(storageConf, config, context, 
table.getMetaClient().getTableConfig());
+    metadataWriter.close();
+
+    // one round per partition
+    String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0];
+    // init some args
+    String fileId = UUID.randomUUID().toString();
+    String instantTime = "000";
+
+    // Create a parquet file
+    config.setSchema(TRIP_EXAMPLE_SCHEMA);
+    metaClient = HoodieTableMetaClient.reload(metaClient);
+    table = (HoodieSparkCopyOnWriteTable) 
HoodieSparkCopyOnWriteTable.create(config, context, metaClient);
+    HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new 
String[] {partitionPath});
+
+    Pair<WriteStatus, List<HoodieRecord>> statusListPair = 
createParquetFile(config, table, partitionPath, fileId, instantTime, 
dataGenerator, false);
+    WriteStatus writeStatus = statusListPair.getLeft();
+    List<HoodieRecord> records = statusListPair.getRight();
+    assertEquals(records.size(), writeStatus.getTotalRecords());
+    assertEquals(0, writeStatus.getTotalErrorRecords());
+
+    instantTime = "001";
+    List<HoodieRecord> updates = 
dataGenerator.generateUniqueUpdates(instantTime, 10);
+    HoodieMergeHandle mergeHandle = new HoodieMergeHandle(config, instantTime, 
table, updates.iterator(), partitionPath, fileId, new 
LocalTaskContextSupplier(),
+        new HoodieBaseFile(writeStatus.getStat().getPath()), Option.of(new 
KeyGeneratorForDataGeneratorRecords(config.getProps())));
+    HoodieMergeHelper.newInstance().runMerge(table, mergeHandle);
+    writeStatus = mergeHandle.writeStatus;
+    // verify stats after merge
+    assertEquals(records.size(), writeStatus.getStat().getNumWrites());
+    assertEquals(10, writeStatus.getStat().getNumUpdateWrites());
+    // verify secondary index stats
+    assertEquals(1, 
writeStatus.getIndexStats().getSecondaryIndexStats().size());
+    // 10 si records for old secondary keys and 10 for new secondary keys
+    assertEquals(20, 
writeStatus.getIndexStats().getSecondaryIndexStats().values().stream().findFirst().get().size());

Review Comment:
   we should def validate the contents here. 
   



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieCreateHandle.java:
##########
@@ -62,7 +65,7 @@ public class HoodieCreateHandle<T, I, K, O> extends 
HoodieWriteHandle<T, I, K, O
   protected long recordsDeleted = 0;
   private Map<String, HoodieRecord<T>> recordMap;
   private boolean useWriterSchema = false;
-  private final boolean preserveMetadata;
+  private boolean preserveMetadata;

Review Comment:
   why reverting this ? 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/IndexStats.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.client;
+
+import org.apache.hudi.common.model.HoodieRecordDelegate;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Class to hold all index stats required to generate Metadata records for all 
enabled partitions.
+ * Supported stats are record level index stats and secondary index stats.
+ */
+public class IndexStats implements Serializable {
+  private static final long serialVersionUID = 1L;
+
+  private final List<HoodieRecordDelegate> writtenRecordDelegates = new 
ArrayList<>();
+  private final Map<String, List<SecondaryIndexStats>> secondaryIndexStats = 
new HashMap<>();
+
+  void addHoodieRecordDelegate(HoodieRecordDelegate hoodieRecordDelegate) {
+    this.writtenRecordDelegates.add(hoodieRecordDelegate);
+  }
+
+  public List<HoodieRecordDelegate> getWrittenRecordDelegates() {
+    return writtenRecordDelegates;
+  }
+
+  public void addSecondaryIndexStats(String secondaryIndexPartitionPath, 
String recordKey, String secondaryIndexValue, boolean isDeleted) {
+    secondaryIndexStats.computeIfAbsent(secondaryIndexPartitionPath, k -> new 
ArrayList<>())

Review Comment:
   can you add an explicit method named initializeSecondaryIndexStats. 
   and within constructor of HoodieWriteHandle, we can call this method when we 
know sec index is enabled and if there are any sec index defns found. 



##########
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/BaseTestHandle.java:
##########
@@ -55,13 +55,15 @@ public void tearDown() throws Exception {
   }
 
   Pair<WriteStatus, List<HoodieRecord>> createParquetFile(HoodieWriteConfig 
config, HoodieTable table, String partitionPath,
-                                                          String fileId, 
String instantTime, HoodieTestDataGenerator dataGenerator) {
+                                                          String fileId, 
String instantTime, HoodieTestDataGenerator dataGenerator,
+                                                          boolean 
preserveMetadata) {
     List<HoodieRecord> records = dataGenerator.generateInserts(instantTime, 
100);
     Map<String, HoodieRecord> recordMap = new HashMap<>();
     for (int i = 0; i < records.size(); i++) {
       recordMap.put(String.valueOf(i), records.get(i));
     }
     HoodieCreateHandle handle = new HoodieCreateHandle(config, instantTime, 
table, partitionPath, fileId, recordMap, new LocalTaskContextSupplier());
+    handle.setPreserveMetadata(preserveMetadata);

Review Comment:
   why can't we call the other overloaded constructor in L65 which takes in the 
`preserveMetadata` arg. 
   why do we need this setter just in tests? 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java:
##########
@@ -194,8 +210,7 @@ private Option<FileSlice> 
populateWriteStatAndFetchFileSlice(HoodieRecord record
       prevCommit = instantTime;
       if (hoodieTable.getMetaClient().getTableConfig().isCDCEnabled()) {
         // the cdc reader needs the base file metadata to have deterministic 
update sequence.
-        TableFileSystemView.SliceView rtView = hoodieTable.getSliceView();
-        fileSlice = rtView.getLatestFileSlice(partitionPath, fileId);
+        fileSlice = getFileSlice();

Review Comment:
   if CDC is not enabled and for table version 8, where are we populating the 
`fileSlice` ? 
   I could not locate it.



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadataWriter.java:
##########
@@ -149,7 +152,8 @@ public abstract class HoodieBackedTableMetadataWriter<I, O> 
implements HoodieTab
   private static final String RECORD_KEY_FIELD_NAME = 
HoodieMetadataPayload.KEY_FIELD_NAME;
 
   // tracks the list of MDT partitions which can write to metadata table in a 
streaming manner.
-  private static final List<MetadataPartitionType> 
STREAMING_WRITES_SUPPORTED_PARTITIONS = Arrays.asList(RECORD_INDEX);
+  private static final List<MetadataPartitionType> 
STREAMING_WRITES_SUPPORTED_PARTITIONS = Arrays.asList(RECORD_INDEX, 
SECONDARY_INDEX);
+  private static final List<MetadataPartitionType> 
STREAMING_WRITES_SUPPORTED_PARTITION_PREFIXES = Arrays.asList(SECONDARY_INDEX);

Review Comment:
   Note to Reviewer: 
   Certain partitions in MDT are one on one mapped to the 
MetadataPartitionType. but its not always the case. 
   For eg, in case of secondary index, we generally do `startsWtith` and not 
really exact match, since we could have N no of sec indexes built. 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java:
##########
@@ -166,6 +175,13 @@ public HoodieAppendHandle(HoodieWriteConfig config, String 
instantTime, HoodieTa
     this.baseFileInstantTimeOfPositions = shouldWriteRecordPositions
         ? getBaseFileInstantTimeOfPositions()
         : Option.empty();
+    if (!hoodieTable.isMetadataTable() && config.isSecondaryIndexEnabled() && 
isStreamingWriteToMetadataEnabled) {

Review Comment:
   can you help me understand what does `populateMetaFields` have something to 
do w/ streaming writes stats generation for SI



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java:
##########
@@ -452,6 +458,51 @@ protected HoodieRecord<T> updateFileName(HoodieRecord<T> 
record, Schema schema,
     return record.prependMetaFields(schema, targetSchema, metadataValues, 
prop);
   }
 
+  private void trackMetadataIndexStats(Option<HoodieKey> hoodieKeyOpt, 
Option<HoodieRecord> combinedRecordOpt, Option<HoodieRecord<T>> oldRecordOpt, 
boolean isDelete) {
+    if (!config.isSecondaryIndexEnabled() || secondaryIndexDefns.isEmpty() || 
!config.isMetadataStreamingWritesEnabled(hoodieTable.getMetaClient().getTableConfig().getTableVersion()))
 {

Review Comment:
   based on earlier feedback, we should be able to access a protected method 
from HoodieWriteHandle for this



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieAppendHandle.java:
##########
@@ -133,6 +139,9 @@ public class HoodieAppendHandle<T, I, K, O> extends 
HoodieWriteHandle<T, I, K, O
   private boolean useWriterSchema = false;
 
   private final Properties recordProperties = new Properties();
+  private boolean generateStatsForStreamingMetadataWrites;

Review Comment:
   we can move this boolean to HoodieWriteHandle as well. and also the L 179.



##########
hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/io/TestCreateHandle.java:
##########
@@ -65,20 +68,65 @@ public void testCreateHandleRLIStats(boolean 
populateMetaFields) {
 
     config.setSchema(TRIP_EXAMPLE_SCHEMA);
     HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new 
String[] {partitionPath});
-    Pair<WriteStatus, List<HoodieRecord>> statusListPair = 
createParquetFile(config, table, partitionPath, fileId, instantTime, 
dataGenerator);
+    Pair<WriteStatus, List<HoodieRecord>> statusListPair = 
createParquetFile(config, table, partitionPath, fileId, instantTime, 
dataGenerator, true);
     WriteStatus writeStatus = statusListPair.getLeft();
     List<HoodieRecord> records = statusListPair.getRight();
 
     assertEquals(records.size(), writeStatus.getTotalRecords());
     assertEquals(0, writeStatus.getTotalErrorRecords());
     // validate write status has all record delegates
     if (populateMetaFields) {
-      assertEquals(records.size(), 
writeStatus.getWrittenRecordDelegates().size());
-      for (HoodieRecordDelegate recordDelegate : 
writeStatus.getWrittenRecordDelegates()) {
+      assertEquals(records.size(), 
writeStatus.getIndexStats().getWrittenRecordDelegates().size());
+      for (HoodieRecordDelegate recordDelegate : 
writeStatus.getIndexStats().getWrittenRecordDelegates()) {
         assertTrue(recordDelegate.getNewLocation().isPresent());
         assertEquals(fileId, 
recordDelegate.getNewLocation().get().getFileId());
         assertEquals(instantTime, 
recordDelegate.getNewLocation().get().getInstantTime());
       }
     }
   }
+
+  @ParameterizedTest
+  @ValueSource(booleans = { true, false })
+  public void testCreateHandleSecondaryIndexStats(boolean populateMetaFields) 
throws Exception {
+    // init config and table
+    HoodieWriteConfig config = getConfigBuilder(basePath)
+        
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder().withRemoteServerPort(timelineServicePort).build())
+        .withPopulateMetaFields(populateMetaFields)
+        .withMetadataConfig(HoodieMetadataConfig.newBuilder()
+            .enable(true)
+            .withEnableRecordIndex(true)
+            .withStreamingWriteEnabled(true)
+            .withSecondaryIndexEnabled(true)
+            .withSecondaryIndexName("sec-rider")
+            .withSecondaryIndexForColumn("rider")
+            .build())
+        .build();
+
+    HoodieTable table = HoodieSparkTable.create(config, context, metaClient);
+    HoodieTableMetadataWriter metadataWriter = 
SparkMetadataWriterFactory.create(storageConf, config, context, 
table.getMetaClient().getTableConfig());
+    metadataWriter.close();
+
+    // one round per partition
+    String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0];
+
+    // init some args
+    String fileId = UUID.randomUUID().toString();
+    String instantTime = "000";
+
+    config.setSchema(TRIP_EXAMPLE_SCHEMA);
+    metaClient = HoodieTableMetaClient.reload(metaClient);
+    table = HoodieSparkTable.create(config, context, metaClient);
+    HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new 
String[] {partitionPath});
+    Pair<WriteStatus, List<HoodieRecord>> statusListPair = 
createParquetFile(config, table, partitionPath, fileId, instantTime, 
dataGenerator, true);
+    WriteStatus writeStatus = statusListPair.getLeft();
+    List<HoodieRecord> records = statusListPair.getRight();
+
+    assertEquals(records.size(), writeStatus.getTotalRecords());
+    assertEquals(0, writeStatus.getTotalErrorRecords());
+    // validate write status has all record delegates
+    if (populateMetaFields) {
+      assertEquals(1, 
writeStatus.getIndexStats().getSecondaryIndexStats().size());
+      assertEquals(100, 
writeStatus.getIndexStats().getSecondaryIndexStats().values().stream().findFirst().get().size());

Review Comment:
   same here.



##########
hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/io/TestAppendHandle.java:
##########
@@ -85,12 +88,69 @@ public void testAppendHandleRLIStats(boolean 
populateMetaFields) {
     assertEquals(0, writeStatus.getTotalErrorRecords());
     // validate write status has all record delegates
     if (populateMetaFields) {
-      assertEquals(records.size(), 
writeStatus.getWrittenRecordDelegates().size());
-      for (HoodieRecordDelegate recordDelegate : 
writeStatus.getWrittenRecordDelegates()) {
+      assertEquals(records.size(), 
writeStatus.getIndexStats().getWrittenRecordDelegates().size());
+      for (HoodieRecordDelegate recordDelegate : 
writeStatus.getIndexStats().getWrittenRecordDelegates()) {
         assertTrue(recordDelegate.getNewLocation().isPresent());
         assertEquals(fileId, 
recordDelegate.getNewLocation().get().getFileId());
         assertEquals(instantTime, 
recordDelegate.getNewLocation().get().getInstantTime());
       }
     }
   }
+
+  @ParameterizedTest
+  @ValueSource(booleans = { true, false })
+  public void testAppendHandleSecondaryIndexStats(boolean populateMetaFields) 
throws Exception {
+    // init config and table
+    HoodieWriteConfig config = getConfigBuilder(basePath)
+        
.withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder().withRemoteServerPort(timelineServicePort).build())
+        .withPopulateMetaFields(populateMetaFields)
+        .withMetadataConfig(HoodieMetadataConfig.newBuilder()
+            .enable(true)
+            .withEnableRecordIndex(true)
+            .withStreamingWriteEnabled(true)
+            .withSecondaryIndexEnabled(true)
+            .withSecondaryIndexName("sec-rider")
+            .withSecondaryIndexForColumn("rider")
+            .build())
+        .build();
+
+    HoodieTable table = HoodieSparkTable.create(config, context, metaClient);
+    HoodieTableMetadataWriter metadataWriter = 
SparkMetadataWriterFactory.create(storageConf, config, context, 
table.getMetaClient().getTableConfig());
+    metadataWriter.close();
+
+    // one round per partition
+    String partitionPath = HoodieTestDataGenerator.DEFAULT_PARTITION_PATHS[0];
+    // init some args
+    String fileId = UUID.randomUUID().toString();
+    String instantTime = "000";
+
+    config.setSchema(TRIP_EXAMPLE_SCHEMA);
+    HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new 
String[] {partitionPath});
+    // create parquet file
+    createParquetFile(config, table, partitionPath, fileId, instantTime, 
dataGenerator, true);
+    // generate update records
+    instantTime = "001";
+    List<HoodieRecord> records = 
dataGenerator.generateUniqueUpdates(instantTime, 50);
+    metaClient = HoodieTableMetaClient.reload(metaClient);
+    table = HoodieSparkTable.create(config, context, metaClient);
+    HoodieAppendHandle handle = new HoodieAppendHandle(config, instantTime, 
table, partitionPath, fileId, records.iterator(), new 
LocalTaskContextSupplier());
+    Map<String, HoodieRecord> recordMap = new HashMap<>();
+    for (int i = 0; i < records.size(); i++) {
+      recordMap.put(String.valueOf(i), records.get(i));
+    }
+    // write the update records
+    handle.write(recordMap);
+    WriteStatus writeStatus = handle.writeStatus;
+    handle.close();
+
+    assertEquals(records.size(), writeStatus.getTotalRecords());
+    assertEquals(0, writeStatus.getTotalErrorRecords());
+    // validate write status has all record delegates
+    if (populateMetaFields) {
+      assertEquals(1, 
writeStatus.getIndexStats().getSecondaryIndexStats().size());
+      // Since the MDT is not populated during the create, the updates would 
be considered as new records by the Append handle
+      // Therefore only secondary index records for the 50 updates would 
appear here
+      assertEquals(50, 
writeStatus.getIndexStats().getSecondaryIndexStats().values().stream().findFirst().get().size());

Review Comment:
   can we validate the content as well



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