the-other-tim-brown commented on code in PR #13842:
URL: https://github.com/apache/hudi/pull/13842#discussion_r2323926550


##########
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestSavepointRestoreCopyOnWriteWithTestFormat.java:
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.functional;
+
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.client.WriteClientTestUtils;
+import org.apache.hudi.common.config.HoodieMetadataConfig;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.metadata.HoodieTableMetadata;
+import org.apache.hudi.table.HoodieSparkTable;
+import org.apache.hudi.testutils.HoodieClientTestBase;
+
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import java.util.Objects;
+
+import static 
org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR;
+import static 
org.apache.hudi.metadata.HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Test cases for COPY_ON_WRITE table savepoint restore.
+ */
+@Tag("functional")
+public class TestSavepointRestoreCopyOnWriteWithTestFormat extends 
HoodieClientTestBase {
+
+  /**
+   * Actions: C1, C2, savepoint C2, C3, C4, restore.
+   * Should go back to C2,
+   * C3 and C4 should be cleaned up.
+   */
+  @Test
+  void testBasicRollback() throws Exception {
+    TypedProperties props = new TypedProperties();
+    props.setProperty(HoodieTableConfig.TABLE_FORMAT.key(), "test-format");
+    initMetaClient(getTableType(), props);
+    HoodieWriteConfig hoodieWriteConfig = 
getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY)
+        .withRollbackUsingMarkers(true)
+        .build();
+    try (SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig)) 
{
+      String savepointCommit = null;
+      String prevInstant = HoodieTimeline.INIT_INSTANT_TS;
+      final int numRecords = 10;
+      for (int i = 1; i <= 4; i++) {
+        String newCommitTime = WriteClientTestUtils.createNewInstantTime();
+        // Write 4 inserts with the 2nd commit been rolled back
+        insertBatch(hoodieWriteConfig, client, newCommitTime, prevInstant, 
numRecords, SparkRDDWriteClient::insert,
+            false, true, numRecords, numRecords * i, 1, Option.empty(), 
INSTANT_GENERATOR);
+        prevInstant = newCommitTime;
+        if (i == 2) {
+          // trigger savepoint
+          savepointCommit = newCommitTime;
+          client.savepoint("user1", "Savepoint for 2nd commit");
+        }
+      }
+      assertRowNumberEqualsTo(40);
+      // restore
+      client.restoreToSavepoint(Objects.requireNonNull(savepointCommit, 
"restore commit should not be null"));
+      assertRowNumberEqualsTo(20);

Review Comment:
   Are there any validations that the test-format is used?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/versioning/v2/TimelineArchiverV2.java:
##########
@@ -214,7 +215,9 @@ private List<HoodieInstant> getCommitInstantsToArchive() 
throws IOException {
 
     // 4. If metadata table is enabled, do not archive instants which are more 
recent than the last compaction on the
     // metadata table.
-    if (config.isMetadataTableEnabled() && 
table.getMetaClient().getTableConfig().isMetadataTableAvailable()) {
+    if (config.isMetadataTableEnabled()

Review Comment:
   Any way we can test this to avoid regressions in the future?



##########
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestSavepointRestoreCopyOnWriteWithTestFormat.java:
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.functional;
+
+import org.apache.hudi.client.SparkRDDWriteClient;
+import org.apache.hudi.client.WriteClientTestUtils;
+import org.apache.hudi.common.config.HoodieMetadataConfig;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.metadata.HoodieTableMetadata;
+import org.apache.hudi.table.HoodieSparkTable;
+import org.apache.hudi.testutils.HoodieClientTestBase;
+
+import org.junit.jupiter.api.Tag;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import java.util.Objects;
+
+import static 
org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR;
+import static 
org.apache.hudi.metadata.HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * Test cases for COPY_ON_WRITE table savepoint restore.
+ */
+@Tag("functional")
+public class TestSavepointRestoreCopyOnWriteWithTestFormat extends 
HoodieClientTestBase {
+
+  /**
+   * Actions: C1, C2, savepoint C2, C3, C4, restore.
+   * Should go back to C2,
+   * C3 and C4 should be cleaned up.
+   */
+  @Test
+  void testBasicRollback() throws Exception {
+    TypedProperties props = new TypedProperties();
+    props.setProperty(HoodieTableConfig.TABLE_FORMAT.key(), "test-format");
+    initMetaClient(getTableType(), props);
+    HoodieWriteConfig hoodieWriteConfig = 
getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY)
+        .withRollbackUsingMarkers(true)
+        .build();
+    try (SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig)) 
{
+      String savepointCommit = null;
+      String prevInstant = HoodieTimeline.INIT_INSTANT_TS;
+      final int numRecords = 10;
+      for (int i = 1; i <= 4; i++) {
+        String newCommitTime = WriteClientTestUtils.createNewInstantTime();
+        // Write 4 inserts with the 2nd commit been rolled back
+        insertBatch(hoodieWriteConfig, client, newCommitTime, prevInstant, 
numRecords, SparkRDDWriteClient::insert,
+            false, true, numRecords, numRecords * i, 1, Option.empty(), 
INSTANT_GENERATOR);
+        prevInstant = newCommitTime;
+        if (i == 2) {
+          // trigger savepoint
+          savepointCommit = newCommitTime;
+          client.savepoint("user1", "Savepoint for 2nd commit");
+        }
+      }
+      assertRowNumberEqualsTo(40);
+      // restore
+      client.restoreToSavepoint(Objects.requireNonNull(savepointCommit, 
"restore commit should not be null"));
+      assertRowNumberEqualsTo(20);
+    }
+  }
+
+  @Test
+  void testRollbackBeyondLastMDTCompaction() throws Exception {
+    TypedProperties props = new TypedProperties();
+    props.setProperty(HoodieTableConfig.TABLE_FORMAT.key(), "test-format");
+    initMetaClient(getTableType(), props);
+    HoodieWriteConfig hoodieWriteConfig = 
getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY)
+        .withRollbackUsingMarkers(true)
+        .withMetadataConfig(HoodieMetadataConfig.newBuilder()
+            .withMaxNumDeltaCommitsBeforeCompaction(4)
+            .build())
+        .build();
+    try (SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig)) 
{
+      String savepointCommit = null;
+      String prevInstant = HoodieTimeline.INIT_INSTANT_TS;
+      final int numRecords = 10;
+      final int iterations = 5;
+      for (int i = 1; i <= 5; i++) {
+        String newCommitTime = WriteClientTestUtils.createNewInstantTime();
+        // Write 4 inserts with the 2nd commit been rolled back
+        insertBatch(hoodieWriteConfig, client, newCommitTime, prevInstant, 
numRecords, SparkRDDWriteClient::insert,
+            false, true, numRecords, numRecords * i, 1, Option.empty(), 
INSTANT_GENERATOR);
+        prevInstant = newCommitTime;
+        if (i == 2) {
+          // trigger savepoint
+          savepointCommit = newCommitTime;
+          client.savepoint("user1", "Savepoint for 2nd commit");
+        }
+      }
+      assertRowNumberEqualsTo(iterations * numRecords);
+      // restore will be forced to rebuild the metadata table
+      client.restoreToSavepoint(Objects.requireNonNull(savepointCommit, 
"restore commit should not be null"));
+      assertRowNumberEqualsTo(20);
+      // check if the metadata table is rebuilt
+      String metadataTableBasePath = 
HoodieTableMetadata.getMetadataTableBasePath(hoodieWriteConfig.getBasePath());
+      HoodieTableMetaClient metadataMetaClient = 
HoodieTableMetaClient.builder()
+          .setBasePath(metadataTableBasePath)
+          .setConf(storageConf)
+          .setLoadActiveTimelineOnLoad(true)
+          .build();
+      assertEquals(1, metadataMetaClient.getCommitsTimeline().filter(instant 
-> !instant.requestedTime().startsWith(SOLO_COMMIT_TIMESTAMP)).countInstants());
+    }
+  }
+
+  /**
+   * The restore should roll back all the pending instants that are beyond the 
savepoint.
+   *
+   * <p>Actions: C1, C2, savepoint C2, C3, C4 inflight, restore.
+   * Should go back to C2,
+   * C3, C4 should be cleaned up.
+   */
+  @Test
+  void testCleaningPendingInstants() throws Exception {
+    TypedProperties props = new TypedProperties();
+    props.setProperty(HoodieTableConfig.TABLE_FORMAT.key(), "test-format");
+    initMetaClient(getTableType(), props);
+    HoodieWriteConfig hoodieWriteConfig = 
getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY)
+        .withRollbackUsingMarkers(true)
+        .build();
+    try (SparkRDDWriteClient client = getHoodieWriteClient(hoodieWriteConfig)) 
{
+      String savepointCommit = null;
+      String prevInstant = HoodieTimeline.INIT_INSTANT_TS;
+      final int numRecords = 10;
+      for (int i = 1; i <= 3; i++) {
+        String newCommitTime = WriteClientTestUtils.createNewInstantTime();
+        // Write 4 inserts with the 2nd commit been rolled back
+        insertBatch(hoodieWriteConfig, client, newCommitTime, prevInstant, 
numRecords, SparkRDDWriteClient::insert,
+            false, true, numRecords, numRecords * i, 1, Option.empty(), 
INSTANT_GENERATOR);
+        prevInstant = newCommitTime;
+        if (i == 2) {
+          // trigger savepoint
+          savepointCommit = newCommitTime;
+          client.savepoint("user1", "Savepoint for 2nd commit");
+        }
+      }
+      assertRowNumberEqualsTo(30);
+      // write another pending instant
+      insertBatchWithoutCommit(numRecords);

Review Comment:
   can we assert this commit is no longer present?



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