yihua commented on code in PR #9755:
URL: https://github.com/apache/hudi/pull/9755#discussion_r1331923138


##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala:
##########
@@ -1724,6 +1724,54 @@ class TestCOWDataSource extends 
HoodieSparkClientTestBase with ScalaAssertionSup
     val metadata = TimelineUtils.getCommitMetadata(latestCommit.get(), 
timeline)
     metadata.getOperationType.equals(WriteOperationType.UPSERT)
   }
+
+  @ParameterizedTest
+  @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", 
"SPARK"))
+  def testInsertOverwriteCluster(recordType: HoodieRecordType): Unit = {
+    val (writeOpts, _) = getWriterReaderOpts(recordType)
+
+    // Insert Operation
+    val records = recordsToStrings(dataGen.generateInserts("000", 100)).toList
+    val inputDF = spark.read.json(spark.sparkContext.parallelize(records, 2))
+
+    val optsWithCluster = Map(
+      INLINE_CLUSTERING_ENABLE.key() -> "true",
+      "hoodie.clustering.inline.max.commits" -> "2",
+      "hoodie.clustering.plan.strategy.sort.columns" -> "_row_key",
+      "hoodie.insert.shuffle.parallelism" -> "4",
+      "hoodie.upsert.shuffle.parallelism" -> "4",
+      DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
+      DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
+      HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
+    ) ++ writeOpts
+    inputDF.write.format("hudi")
+      .options(optsWithCluster)
+      .option(DataSourceWriteOptions.OPERATION.key, 
DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
+      .mode(SaveMode.Overwrite)
+      .save(basePath)
+
+    for (i <- 1 until 6) {
+      val records = recordsToStrings(dataGen.generateInsertsForPartition("00" 
+ i, 10, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)).toList
+      val inputDF = spark.read.json(spark.sparkContext.parallelize(records, 2))
+      inputDF.write.format("hudi")
+        .options(optsWithCluster)
+        .option(DataSourceWriteOptions.OPERATION.key, 
DataSourceWriteOptions.INSERT_OVERWRITE_OPERATION_OPT_VAL)
+        .mode(SaveMode.Append)
+        .save(basePath)
+    }
+
+    val metaClient = HoodieTableMetaClient.builder()
+      .setBasePath(basePath)
+      .setConf(hadoopConf)
+      .build()
+    val timeline = metaClient.getActiveTimeline
+    val instants = 
timeline.getAllCommitsTimeline.filterCompletedInstants.getInstants
+    assertEquals(9, instants.size)

Review Comment:
   nit: add assertion on the number of replacecommits which should be 8?



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala:
##########
@@ -1724,6 +1724,54 @@ class TestCOWDataSource extends 
HoodieSparkClientTestBase with ScalaAssertionSup
     val metadata = TimelineUtils.getCommitMetadata(latestCommit.get(), 
timeline)
     metadata.getOperationType.equals(WriteOperationType.UPSERT)
   }
+
+  @ParameterizedTest
+  @EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", 
"SPARK"))
+  def testInsertOverwriteCluster(recordType: HoodieRecordType): Unit = {
+    val (writeOpts, _) = getWriterReaderOpts(recordType)
+
+    // Insert Operation
+    val records = recordsToStrings(dataGen.generateInserts("000", 100)).toList
+    val inputDF = spark.read.json(spark.sparkContext.parallelize(records, 2))
+
+    val optsWithCluster = Map(
+      INLINE_CLUSTERING_ENABLE.key() -> "true",
+      "hoodie.clustering.inline.max.commits" -> "2",
+      "hoodie.clustering.plan.strategy.sort.columns" -> "_row_key",
+      "hoodie.insert.shuffle.parallelism" -> "4",
+      "hoodie.upsert.shuffle.parallelism" -> "4",
+      DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
+      DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
+      HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
+    ) ++ writeOpts
+    inputDF.write.format("hudi")
+      .options(optsWithCluster)
+      .option(DataSourceWriteOptions.OPERATION.key, 
DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
+      .mode(SaveMode.Overwrite)
+      .save(basePath)
+
+    for (i <- 1 until 6) {
+      val records = recordsToStrings(dataGen.generateInsertsForPartition("00" 
+ i, 10, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)).toList
+      val inputDF = spark.read.json(spark.sparkContext.parallelize(records, 2))
+      inputDF.write.format("hudi")
+        .options(optsWithCluster)
+        .option(DataSourceWriteOptions.OPERATION.key, 
DataSourceWriteOptions.INSERT_OVERWRITE_OPERATION_OPT_VAL)
+        .mode(SaveMode.Append)
+        .save(basePath)
+    }
+
+    val metaClient = HoodieTableMetaClient.builder()
+      .setBasePath(basePath)
+      .setConf(hadoopConf)
+      .build()
+    val timeline = metaClient.getActiveTimeline
+    val instants = 
timeline.getAllCommitsTimeline.filterCompletedInstants.getInstants
+    assertEquals(9, instants.size)
+    val compactionInstants = instants.filter(i => {

Review Comment:
   nit: `compactionInstants` -> `clusteringInstants`



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