danny0405 commented on code in PR #7627:
URL: https://github.com/apache/hudi/pull/7627#discussion_r1168233922
##########
hudi-common/src/main/java/org/apache/hudi/common/config/HoodieCommonConfig.java:
##########
@@ -67,6 +67,13 @@ public class HoodieCommonConfig extends HoodieConfig {
.defaultValue(true)
.withDocumentation("Turn on compression for BITCASK disk map used by the
External Spillable Map");
+ public static final ConfigProperty<Boolean>
INCREMENTAL_FETCH_INSTANT_BY_STATE_TRANSITION_TIME = ConfigProperty
+ .key("hoodie.incremental.fetch.instant.by.state.transition.time")
+ .defaultValue(false)
+ .sinceVersion("0.13.0")
Review Comment:
need to fix the since version to 0.14.0
##########
hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieInstant.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * 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.timeline;
+
+import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
+import org.apache.hudi.common.util.Option;
+import org.junit.jupiter.api.Test;
+
+import java.io.IOException;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.common.testutils.Assertions.assertStreamEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class TestHoodieInstant extends HoodieCommonTestHarness {
+
+ @Test
+ public void testExtractTimestamp() {
+ String fileName = "20230104152218702.inflight";
+ assertEquals("20230104152218702",
HoodieInstant.extractTimestamp(fileName));
+
+ fileName = "20230104152218702.commit.request";
+ assertEquals("20230104152218702",
HoodieInstant.extractTimestamp(fileName));
+ }
+
+ @Test
+ public void testGetTimelineFileExtension() {
+ String fileName = "20230104152218702.inflight";
+ assertEquals(".inflight",
HoodieInstant.getTimelineFileExtension(fileName));
+
+ fileName = "20230104152218702.commit.request";
+ assertEquals(".commit.request",
HoodieInstant.getTimelineFileExtension(fileName));
+ }
+
+ @Test
+ public void testCreateHoodieInstantByFileStatus() throws IOException {
+ try {
+ initMetaClient();
+ HoodieInstant instantRequested =
+ new HoodieInstant(HoodieInstant.State.REQUESTED,
HoodieTimeline.COMMIT_ACTION, "001");
+ HoodieInstant instantCommitted =
+ new HoodieInstant(HoodieInstant.State.COMPLETED,
HoodieTimeline.COMMIT_ACTION, "001");
+ HoodieActiveTimeline timeline = metaClient.getActiveTimeline();
+ timeline.createNewInstant(instantRequested);
+ timeline.transitionRequestedToInflight(instantRequested, Option.empty());
+ timeline.saveAsComplete(
+ new HoodieInstant(true, instantRequested.getAction(),
instantRequested.getTimestamp()),
+ Option.empty());
+ metaClient.reloadActiveTimeline();
Review Comment:
We also need some tests for incremental data source.
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/MergeOnReadIncrementalRelation.scala:
##########
@@ -133,8 +133,16 @@ trait HoodieIncrementalRelationTrait extends
HoodieBaseRelation {
// Validate this Incremental implementation is properly configured
validate()
+ private val useStateTransitionTime =
optParams.get(DataSourceReadOptions.INCREMENTAL_FETCH_INSTANT_BY_STATE_TRANSITION_TIME.key)
+ .map(_.toBoolean)
+
.getOrElse(DataSourceReadOptions.INCREMENTAL_FETCH_INSTANT_BY_STATE_TRANSITION_TIME.defaultValue)
+
protected def startTimestamp: String =
optParams(DataSourceReadOptions.BEGIN_INSTANTTIME.key)
- protected def endTimestamp: String =
optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME.key,
super.timeline.lastInstant().get.getTimestamp)
+ protected def endTimestamp: String = if (useStateTransitionTime) {
+ optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME.key,
super.timeline.lastInstant().get.getStateTransitionTime)
+ } else {
Review Comment:
We need some clarification for the semantics change of the two options when
`useStateTransitionTime` is set as true.We should add some clarification for
the new sementics of `BEGIN_INSTANTTIME` and `END_INSTANTTIME`.
##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelation.scala:
##########
@@ -82,9 +87,17 @@ class IncrementalRelation(val sqlContext: SQLContext,
private val lastInstant = commitTimeline.lastInstant().get()
- private val commitsTimelineToReturn = commitTimeline.findInstantsInRange(
- optParams(DataSourceReadOptions.BEGIN_INSTANTTIME.key),
- optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME.key(),
lastInstant.getTimestamp))
+ private val commitsTimelineToReturn = {
+ if (useStateTransitionTime) {
Review Comment:
Guess this could find out the out of order commits like compaction or
clustering, but the assumption for `BEGIN_INSTANTTIME` and `END_INSTANTTIME`
should be different, once `useStateTransitionTime` is true, user actually
should uses the real transition time instead of the instant time to filter the
instants.
--
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]