nsivabalan commented on code in PR #12688:
URL: https://github.com/apache/hudi/pull/12688#discussion_r1929983138
##########
hudi-common/src/main/java/org/apache/hudi/common/table/checkpoint/CheckpointUtils.java:
##########
@@ -24,19 +24,29 @@
import org.apache.hudi.common.table.HoodieTableVersion;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.table.timeline.TimelineUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.exception.HoodieException;
+import java.util.Arrays;
+import java.util.HashSet;
import java.util.Objects;
+import java.util.Set;
import static
org.apache.hudi.common.table.checkpoint.StreamerCheckpointV1.STREAMER_CHECKPOINT_KEY_V1;
import static
org.apache.hudi.common.table.checkpoint.StreamerCheckpointV1.STREAMER_CHECKPOINT_RESET_KEY_V1;
import static
org.apache.hudi.common.table.checkpoint.StreamerCheckpointV2.STREAMER_CHECKPOINT_KEY_V2;
import static
org.apache.hudi.common.table.checkpoint.StreamerCheckpointV2.STREAMER_CHECKPOINT_RESET_KEY_V2;
+import static
org.apache.hudi.common.table.timeline.TimelineUtils.HollowCommitHandling.USE_TRANSITION_TIME;
public class CheckpointUtils {
+ public static final Set<String> DATASOURCES_MUST_USE_CKP_V1 = new
HashSet<>(Arrays.asList(
Review Comment:
can we name this "DATASOURCES_NOT_SUPPORTED_WITH_CKPT_V2"
##########
hudi-common/src/test/java/org/apache/hudi/common/table/checkpoint/TestCheckpointUtils.java:
##########
@@ -168,7 +173,39 @@ public void testConvertCheckpointWithInitTimestamp() {
assertEquals(HoodieTimeline.INIT_INSTANT_TS,
translated.getCheckpointKey());
checkpoint = new StreamerCheckpointV2(instantTime);
- translated =
CheckpointUtils.convertToCheckpointV2ForCommitTime(checkpoint, metaClient);
+ translated =
CheckpointUtils.convertToCheckpointV2ForCommitTime(checkpoint, metaClient,
BLOCK);
assertEquals(HoodieTimeline.INIT_INSTANT_TS,
translated.getCheckpointKey());
}
+
+ @Test
+ public void testConvertCheckpointWithUseTransitionTime() {
+ String instantTime = "20231127010101";
+ String completionTime = "20231127020102";
+
+ // Mock active timeline
+ HoodieInstant instant = new HoodieInstant(HoodieInstant.State.COMPLETED,
"commit", instantTime, completionTime,
InstantComparatorV1.REQUESTED_TIME_BASED_COMPARATOR);
+ when(activeTimeline.getInstantsAsStream()).thenReturn(Stream.of(instant));
+
+ Checkpoint checkpoint = new StreamerCheckpointV1(completionTime);
+ StreamerCheckpointV2 translatedCheckpoint =
CheckpointUtils.convertToCheckpointV2ForCommitTime(checkpoint, metaClient,
USE_TRANSITION_TIME);
+
+ assertEquals(completionTime, translatedCheckpoint.getCheckpointKey());
+ }
+
+ @ParameterizedTest
+ @CsvSource({
+ // version, sourceClassName, expectedResult
+ // Version >= 8 with allowed sources should return true
+ "8, org.apache.hudi.utilities.sources.TestSource, true",
+ "9, org.apache.hudi.utilities.sources.AnotherSource, true",
+ // Version < 8 should return false regardless of source
+ "7, org.apache.hudi.utilities.sources.TestSource, false",
+ "6, org.apache.hudi.utilities.sources.AnotherSource, false",
+ // Disallowed sources should return false even with version >= 8
+ "8, org.apache.hudi.utilities.sources.S3EventsHoodieIncrSource, false",
+ "8, org.apache.hudi.utilities.sources.GcsEventsHoodieIncrSource, false"
Review Comment:
can we also add MockS3EventsHoodieIncrSource
##########
hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/MockS3EventsHoodieIncrSource.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.utilities.sources;
+
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.table.checkpoint.Checkpoint;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.utilities.ingestion.HoodieIngestionMetrics;
+import org.apache.hudi.utilities.schema.SchemaProvider;
+import org.apache.hudi.utilities.sources.helpers.CloudDataFetcher;
+import org.apache.hudi.utilities.sources.helpers.QueryRunner;
+import org.apache.hudi.utilities.streamer.DefaultStreamContext;
+import org.apache.hudi.utilities.streamer.StreamContext;
+
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+
+/**
+ * A mock implementation of S3EventsHoodieIncrSource used for testing
StreamSync functionality.
+ * This class simulates different checkpoint and data fetch scenarios to test
the checkpoint handling
+ * and data ingestion behavior of the StreamSync class.
+ */
+public class MockS3EventsHoodieIncrSource extends S3EventsHoodieIncrSource {
Review Comment:
can we write a similar one for GCS ?
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/streaming/HoodieStreamSourceV1.scala:
##########
@@ -188,7 +188,7 @@ class HoodieStreamSourceV1(sqlContext: SQLContext,
private def translateCheckpoint(commitTime: String): String = {
if (writeTableVersion.greaterThanOrEquals(HoodieTableVersion.EIGHT)) {
CheckpointUtils.convertToCheckpointV2ForCommitTime(
- new StreamerCheckpointV1(commitTime), metaClient).getCheckpointKey
+ new StreamerCheckpointV1(commitTime), metaClient,
hollowCommitHandling).getCheckpointKey
Review Comment:
same here. naming -> `hollowCommitHandlingMode`
##########
hudi-common/src/main/java/org/apache/hudi/common/table/checkpoint/CheckpointUtils.java:
##########
@@ -49,14 +59,15 @@ public static Checkpoint getCheckpoint(HoodieCommitMetadata
commitMetadata) {
throw new HoodieException("Checkpoint is not found in the commit metadata:
" + commitMetadata.getExtraMetadata());
}
- public static boolean targetCheckpointV2(int writeTableVersion) {
- return writeTableVersion >= HoodieTableVersion.EIGHT.versionCode();
+ public static boolean targetCheckpointV2(int writeTableVersion, String
sourceClassName) {
+ return writeTableVersion >= HoodieTableVersion.EIGHT.versionCode()
+ && !DATASOURCES_MUST_USE_CKP_V1.contains(sourceClassName);
}
// TODO(yihua): for checkpoint translation, handle cases where the
checkpoint is not exactly the
// instant or completion time
public static StreamerCheckpointV2 convertToCheckpointV2ForCommitTime(
- Checkpoint checkpoint, HoodieTableMetaClient metaClient) {
+ Checkpoint checkpoint, HoodieTableMetaClient metaClient,
TimelineUtils.HollowCommitHandling handlingMode) {
Review Comment:
minor.
`hollowCommitHandlingMode`
##########
hudi-utilities/src/test/java/org/apache/hudi/utilities/streamer/TestStreamSync.java:
##########
@@ -149,32 +155,6 @@ void testFetchNextBatchFromSource(Boolean useRowWriter,
Boolean hasTransformer,
HoodieErrorTableConfig.ERROR_ENABLE_VALIDATE_TARGET_SCHEMA.defaultValue());
}
- @ParameterizedTest
- @MethodSource("getCheckpointToResumeCases")
- void testGetCheckpointToResume(HoodieStreamer.Config cfg,
HoodieCommitMetadata commitMetadata, Option<String> expectedResumeCheckpoint)
throws IOException {
Review Comment:
why removed this? did you move this elsewhere?
##########
hudi-common/src/main/java/org/apache/hudi/common/table/checkpoint/CheckpointUtils.java:
##########
@@ -49,14 +59,15 @@ public static Checkpoint getCheckpoint(HoodieCommitMetadata
commitMetadata) {
throw new HoodieException("Checkpoint is not found in the commit metadata:
" + commitMetadata.getExtraMetadata());
}
- public static boolean targetCheckpointV2(int writeTableVersion) {
- return writeTableVersion >= HoodieTableVersion.EIGHT.versionCode();
+ public static boolean targetCheckpointV2(int writeTableVersion, String
sourceClassName) {
Review Comment:
lets rename this to
```
shouldTargetCheckpointV2
```
--
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]