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


##########
hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java:
##########
@@ -1692,6 +1701,114 @@ public void 
testIngestionCommitInPresenceOfCompletedClusteringCommit() throws Ex
     assertThrows(HoodieWriteConflictException.class, () -> 
client.commit(inflightCommit, ingestionResult));
   }
 
+  @ParameterizedTest
+  @MethodSource("conflictResolutionStrategyParams")
+  public void testParallelInsertOverwriteOperations(ConflictResolutionStrategy 
strategy) throws Exception {
+    Properties properties = new Properties();
+    HoodieLockConfig lockConfig = HoodieLockConfig.newBuilder()
+        .withLockProvider(InProcessLockProvider.class)
+        .withConflictResolutionStrategy(strategy)
+        .build();
+    HoodieWriteConfig insertWriteConfig = getConfigBuilder()
+        .withLockConfig(lockConfig)
+        
.withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL)
+        .withProperties(properties)
+        .build();
+    SparkRDDWriteClient client = getHoodieWriteClient(insertWriteConfig);
+    // Create base commit.
+    String firstCommit = WriteClientTestUtils.createNewInstantTime();
+    insertFirstBatch(insertWriteConfig, client, firstCommit, "000", 10,
+        SparkRDDWriteClient::insert, false, false, 10,
+        insertWriteConfig.populateMetaFields(), INSTANT_GENERATOR, false);
+    assertEquals(firstCommit, metaClient.reloadActiveTimeline()
+            .filterCompletedInstants().lastInstant().get().requestedTime());
+
+    // Do insert-overwrite operation on the existing partitions, without 
committing the data.
+    String secondCommit = WriteClientTestUtils.createNewInstantTime();
+    WriteClientTestUtils.startCommitWithTime(client, secondCommit, 
REPLACE_COMMIT_ACTION);
+    List<HoodieRecord> records1 = dataGen.generateInserts(secondCommit, 10);
+    JavaRDD<HoodieRecord> writeRecords1 = jsc.parallelize(records1, 1);
+    HoodieWriteResult result1 = client.insertOverwrite(writeRecords1, 
secondCommit);
+    assertEquals(secondCommit, metaClient.reloadActiveTimeline()
+        .filterInflightsAndRequested().lastInstant().get().requestedTime());
+
+    // Create second writer and do another insert-overwrite operation on the 
existing partitions,
+    // without committing the data.
+    SparkRDDWriteClient client2 = new SparkRDDWriteClient(context, 
insertWriteConfig);
+    String thirdCommit = WriteClientTestUtils.createNewInstantTime();
+    WriteClientTestUtils.startCommitWithTime(client2, thirdCommit, 
REPLACE_COMMIT_ACTION);
+    List<HoodieRecord> records2 = dataGen.generateInserts(thirdCommit, 10);
+    JavaRDD<HoodieRecord> writeRecords2 = jsc.parallelize(records2, 1);
+    HoodieWriteResult result2 = client2.insertOverwrite(writeRecords2, 
thirdCommit);
+    assertEquals(thirdCommit, metaClient.reloadActiveTimeline()
+        .filterInflightsAndRequested().lastInstant().get().requestedTime());
+
+    // Complete first insert-overwrite operation.
+    client.commit(secondCommit, result1.getWriteStatuses(),
+        Option.empty(), REPLACE_COMMIT_ACTION, 
result1.getPartitionToReplaceFileIds());
+
+    // Now try completing the second insert-overwrite operation, this should 
throw an error since it is replacing
+    // the same files as the previous insert overwrite operation.
+    assertThrows(HoodieWriteConflictException.class, () ->
+        client2.commit(thirdCommit, result2.getWriteStatuses(), Option.empty(),
+            REPLACE_COMMIT_ACTION, result2.getPartitionToReplaceFileIds()));
+  }
+
+  @ParameterizedTest
+  @MethodSource("conflictResolutionStrategyParams")
+  public void 
testParallelClusteringAndUpdateOperation(ConflictResolutionStrategy strategy) 
throws Exception {
+    Properties properties = new Properties();
+    HoodieLockConfig lockConfig = HoodieLockConfig.newBuilder()
+        .withLockProvider(InProcessLockProvider.class)
+        .withConflictResolutionStrategy(strategy)
+        .build();
+    HoodieClusteringConfig clusteringConfig = createClusteringBuilder(false, 1)
+        
.withClusteringUpdatesStrategy(SparkAllowUpdateStrategy.class.getName())
+        .withAsyncClusteringMaxCommits(1)
+        .build();
+    HoodieWriteConfig insertWriteConfig = getConfigBuilder()
+        .withLockConfig(lockConfig)
+        .withClusteringConfig(clusteringConfig)
+        
.withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL)
+        .withProperties(properties)
+        .build();
+    SparkRDDWriteClient client = getHoodieWriteClient(insertWriteConfig);
+    // Create base commit.
+    String firstCommit = WriteClientTestUtils.createNewInstantTime();
+    insertFirstBatch(insertWriteConfig, client, firstCommit, "000", 200,
+        SparkRDDWriteClient::insert, false, false, 10,
+        insertWriteConfig.populateMetaFields(), INSTANT_GENERATOR, false);
+    assertEquals(firstCommit, metaClient.reloadActiveTimeline()
+            .filterCompletedInstants().lastInstant().get().requestedTime());
+
+    // Create clustering writer
+    SparkRDDWriteClient clusteringClient = 
getHoodieWriteClient(insertWriteConfig);
+    Option<String> clusteringInstant = 
clusteringClient.scheduleClustering(Option.empty());
+    assertTrue(clusteringInstant.isPresent());
+    assertEquals(clusteringInstant.get(), metaClient.reloadActiveTimeline()
+            
.filterPendingReplaceOrClusteringTimeline().lastInstant().get().requestedTime());
+
+    String secondInstant = WriteClientTestUtils.createNewInstantTime();
+    if (strategy instanceof PreferWriterConflictResolutionStrategy) {
+      updateBatch(insertWriteConfig, client, secondInstant, firstCommit,
+          Option.of(Arrays.asList()), "000", 100,
+          SparkRDDWriteClient::upsert, false, false, 100,
+          200, 3, insertWriteConfig.populateMetaFields(), INSTANT_GENERATOR);
+      HoodieClusteringException exception = 
assertThrows(HoodieClusteringException.class, () ->
+          clusteringClient.cluster(clusteringInstant.get()));
+      assertTrue(exception.getCause() instanceof HoodieWriteConflictException);
+      HoodieActiveTimeline timeline = metaClient.reloadActiveTimeline();
+      assertEquals(clusteringInstant.get(), 
timeline.filterPendingReplaceOrClusteringTimeline().lastInstant().get().requestedTime());
+      assertEquals(secondInstant, 
timeline.filterCompletedInstants().lastInstant().get().requestedTime());
+    } else {
+      assertThrows(HoodieWriteConflictException.class, () ->
+          updateBatch(insertWriteConfig, client, secondInstant, firstCommit,
+              Option.of(Arrays.asList()), "000", 100,
+              SparkRDDWriteClient::upsert, false, false, 100,
+              200, 3, insertWriteConfig.populateMetaFields(), 
INSTANT_GENERATOR));

Review Comment:
   Fixed



##########
hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnCopyOnWriteStorage.java:
##########
@@ -1692,6 +1700,103 @@ public void 
testIngestionCommitInPresenceOfCompletedClusteringCommit() throws Ex
     assertThrows(HoodieWriteConflictException.class, () -> 
client.commit(inflightCommit, ingestionResult));
   }
 
+  @ParameterizedTest
+  @MethodSource("conflictResolutionStrategyParams")
+  public void testParallelInsertOverwriteOperations(ConflictResolutionStrategy 
strategy) throws Exception {
+    Properties properties = new Properties();
+    properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + 
"/.hoodie/.locks");
+    HoodieLockConfig lockConfig = HoodieLockConfig.newBuilder()
+        .withLockProvider(FileSystemBasedLockProviderTestClass.class)
+        .withConflictResolutionStrategy(strategy)
+        .build();
+    HoodieWriteConfig insertWriteConfig = getConfigBuilder()
+        .withLockConfig(lockConfig)
+        
.withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL)
+        .withProperties(properties)
+        .build();
+    SparkRDDWriteClient client = getHoodieWriteClient(insertWriteConfig);
+    // Create base commit.
+    String firstCommit = WriteClientTestUtils.createNewInstantTime();
+    insertFirstBatch(insertWriteConfig, client, firstCommit, "000", 10,
+        SparkRDDWriteClient::insert, false, false, 10,
+        insertWriteConfig.populateMetaFields(), INSTANT_GENERATOR, false);
+
+    // Do insert-overwrite operation on the existing partitions, without 
committing the data.
+    String secondCommit = WriteClientTestUtils.createNewInstantTime();
+    WriteClientTestUtils.startCommitWithTime(client, secondCommit, 
REPLACE_COMMIT_ACTION);
+    List<HoodieRecord> records1 = dataGen.generateInserts(secondCommit, 10);
+    JavaRDD<HoodieRecord> writeRecords1 = jsc.parallelize(records1, 1);
+    HoodieWriteResult result1 = client.insertOverwrite(writeRecords1, 
secondCommit);
+
+    // Create second writer and do another insert-overwrite operation on the 
existing partitions,
+    // without committing the data.
+    SparkRDDWriteClient client2 = new SparkRDDWriteClient(context, 
insertWriteConfig);
+    String thirdCommit = WriteClientTestUtils.createNewInstantTime();
+    WriteClientTestUtils.startCommitWithTime(client2, thirdCommit, 
REPLACE_COMMIT_ACTION);
+    List<HoodieRecord> records2 = dataGen.generateInserts(thirdCommit, 10);
+    JavaRDD<HoodieRecord> writeRecords2 = jsc.parallelize(records2, 1);
+    HoodieWriteResult result2 = client2.insertOverwrite(writeRecords2, 
thirdCommit);
+
+    // Complete first insert-overwrite operation.
+    client.commit(secondCommit, result1.getWriteStatuses(),

Review Comment:
   Fixed



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