vinothchandar commented on code in PR #12342:
URL: https://github.com/apache/hudi/pull/12342#discussion_r1864690841
##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/helpers/IncrSourceHelper.java:
##########
@@ -107,25 +112,30 @@ public static QueryInfo
generateQueryInfo(JavaSparkContext jssc, String srcBaseP
.setConf(HadoopFSUtils.getStorageConfWithCopy(jssc.hadoopConfiguration()))
.setBasePath(srcBasePath).setLoadActiveTimelineOnLoad(true).build();
+ // TODO(yihua): handle transition time in CheckpointV1
HoodieTimeline completedCommitTimeline =
srcMetaClient.getCommitsAndCompactionTimeline().filterCompletedInstants();
final HoodieTimeline activeCommitTimeline =
handleHollowCommitIfNeeded(completedCommitTimeline, srcMetaClient,
handlingMode);
Function<HoodieInstant, String> timestampForLastInstant = instant ->
handlingMode == HollowCommitHandling.USE_TRANSITION_TIME
? instant.getCompletionTime() : instant.requestedTime();
- String beginInstantTime = beginInstant.orElseGet(() -> {
+
+ Option<Checkpoint> translatedCheckpoint = beginInstant.isPresent()
Review Comment:
this is for the code 1.0/tableVersion=6 path.. where it can continue to
write old key? IIUC this is orthogonal to using transition time etc..
##########
hudi-common/src/main/java/org/apache/hudi/common/table/checkpoint/CheckpointUtils.java:
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.common.table.checkpoint;
+
+import org.apache.hudi.common.model.HoodieCommitMetadata;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.HoodieTableVersion;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.exception.HoodieException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static
org.apache.hudi.common.table.checkpoint.CheckpointV1.STREAMER_CHECKPOINT_KEY_V1;
+import static
org.apache.hudi.common.table.checkpoint.CheckpointV1.STREAMER_CHECKPOINT_RESET_KEY_V1;
+import static
org.apache.hudi.common.table.checkpoint.CheckpointV2.STREAMER_CHECKPOINT_KEY_V2;
+import static
org.apache.hudi.common.table.checkpoint.CheckpointV2.STREAMER_CHECKPOINT_RESET_KEY_V2;
+
+public class CheckpointUtils {
+ private static final Logger LOG =
LoggerFactory.getLogger(CheckpointUtils.class);
+
+ public static Checkpoint getCheckpoint(HoodieCommitMetadata commitMetadata) {
+ if
(!StringUtils.isNullOrEmpty(commitMetadata.getMetadata(STREAMER_CHECKPOINT_KEY_V2))
+ ||
!StringUtils.isNullOrEmpty(commitMetadata.getMetadata(STREAMER_CHECKPOINT_RESET_KEY_V2)))
{
+ return new CheckpointV2(commitMetadata);
+ }
+ if
(!StringUtils.isNullOrEmpty(commitMetadata.getMetadata(STREAMER_CHECKPOINT_KEY_V1))
+ ||
!StringUtils.isNullOrEmpty(commitMetadata.getMetadata(STREAMER_CHECKPOINT_RESET_KEY_V1)))
{
+ return new CheckpointV1(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();
+ }
+
+ // TODO(yihua): for checkpoint translation, handle cases where the
checkpoint is not exactly the
+ // instant or completion time
+ public static CheckpointV2 convertToCheckpointV2ForCommitTime(
+ Checkpoint checkpoint, HoodieTableMetaClient metaClient) {
+ if (checkpoint instanceof CheckpointV2) {
+ return (CheckpointV2) checkpoint;
+ }
+ if (checkpoint instanceof CheckpointV1) {
Review Comment:
this is what we expect to hit, when 1.x code runs for first time..
--
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]