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


##########
hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableInsertUpdateDelete.java:
##########
@@ -108,58 +108,63 @@ private static Stream<Arguments> 
testSimpleInsertAndUpdate() {
   @ParameterizedTest
   @MethodSource
   public void testSimpleInsertAndUpdate(HoodieFileFormat fileFormat, boolean 
populateMetaFields) throws Exception {
-    Properties properties = populateMetaFields ? new Properties() : 
getPropertiesForKeyGen();
-    properties.setProperty(HoodieTableConfig.BASE_FILE_FORMAT.key(), 
fileFormat.toString());
-    HoodieTableMetaClient metaClient = 
getHoodieMetaClient(HoodieTableType.MERGE_ON_READ, properties);
-
-    HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(true);
-    addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
-    HoodieWriteConfig cfg = cfgBuilder.build();
-    try (SparkRDDWriteClient client = getHoodieWriteClient(cfg)) {
-
-      HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator();
-      /*
-       * Write 1 (only inserts)
-       */
-      String newCommitTime = "001";
-      client.startCommitWithTime(newCommitTime);
-
-      List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, 200);
-      Stream<HoodieBaseFile> dataFiles = insertRecordsToMORTable(metaClient, 
records, client, cfg, newCommitTime);
-      assertTrue(dataFiles.findAny().isPresent(), "should list the base files 
we wrote in the delta commit");
-
-      /*
-       * Write 2 (updates)
-       */
-      newCommitTime = "004";
-      client.startCommitWithTime(newCommitTime);
-      records = dataGen.generateUpdates(newCommitTime, 100);
-      updateRecordsInMORTable(metaClient, records, client, cfg, newCommitTime, 
false);
-
-      String compactionCommitTime = 
client.scheduleCompaction(Option.empty()).get().toString();
-      client.compact(compactionCommitTime);
-
-      HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context(), 
metaClient);
-      hoodieTable.getHoodieView().sync();
-      List<StoragePathInfo> allFiles = listAllBaseFilesInPath(hoodieTable);
-      HoodieTableFileSystemView tableView =
-          getHoodieTableFileSystemView(metaClient, 
hoodieTable.getCompletedCommitsTimeline(), allFiles);
-      Stream<HoodieBaseFile> dataFilesToRead = tableView.getLatestBaseFiles();
-      assertTrue(dataFilesToRead.findAny().isPresent());
-
-      // verify that there is a commit
-      metaClient = HoodieTableMetaClient.reload(metaClient);
-      HoodieTimeline timeline = 
metaClient.getCommitTimeline().filterCompletedInstants();
-      assertEquals(1, timeline.findInstantsAfter("000", 
Integer.MAX_VALUE).countInstants(),
-          "Expecting a single commit.");
-      String latestCompactionCommitTime = 
timeline.lastInstant().get().requestedTime();
-      assertTrue(compareTimestamps("000", LESSER_THAN, 
latestCompactionCommitTime));
-
-      if (cfg.populateMetaFields()) {
-        assertEquals(200, 
HoodieClientTestUtils.countRecordsOptionallySince(jsc(), basePath(), 
sqlContext(), timeline, Option.of("000")),
-            "Must contain 200 records");
-      } else {
-        assertEquals(200, 
HoodieClientTestUtils.countRecordsOptionallySince(jsc(), basePath(), 
sqlContext(), timeline, Option.empty()));
+    if (fileFormat != HoodieFileFormat.HFILE) {

Review Comment:
   hey @lokeshj1703 : can you re-enable and see what issue we are running into. 
   when I worked on this, compaction in hfile was failing bcoz, we support 
hfile only for metadata table and the log files are also expected to be in 
hfile format. But in this test compaction was failing since log files were in 
avro or something. 
   I don't exactly remember. but something of that nature. 
   



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