vinothchandar commented on code in PR #11923:
URL: https://github.com/apache/hudi/pull/11923#discussion_r1831374973


##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/CompletionTimeQueryViewV1.java:
##########
@@ -0,0 +1,239 @@
+/*
+ * 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.versioning.v1;
+
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.log.InstantRange;
+import org.apache.hudi.common.table.timeline.CompletionTimeQueryView;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator;
+import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import org.apache.hudi.common.table.timeline.InstantComparatorUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.VisibleForTesting;
+
+import java.io.Serializable;
+import java.time.Instant;
+import java.util.Date;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.function.Function;
+
+import static 
org.apache.hudi.common.table.timeline.InstantComparatorUtils.GREATER_THAN_OR_EQUALS;
+import static 
org.apache.hudi.common.table.timeline.InstantComparatorUtils.LESSER_THAN;
+
+public class CompletionTimeQueryViewV1 implements CompletionTimeQueryView, 
Serializable {
+  private static final long serialVersionUID = 1L;
+
+  private static final long MILLI_SECONDS_IN_THREE_DAYS = 3 * 24 * 3600 * 1000;
+
+  private static final long MILLI_SECONDS_IN_ONE_DAY = 24 * 3600 * 1000;
+
+  private final HoodieTableMetaClient metaClient;
+
+  /**
+   * Mapping from instant start time -> completion time.
+   * Should be thread-safe data structure.
+   */
+  private final ConcurrentMap<String, String> beginToCompletionInstantTimeMap;
+
+  /**
+   * The cursor instant time to eagerly load from, by default load last N days 
of completed instants.
+   * It can grow dynamically with lazy loading. e.g. assuming an initial 
cursor instant as t10,
+   * a completion query for t5 would trigger lazy loading with this cursor 
instant updated to t5.
+   * This sliding window model amortizes redundant loading from different 
queries.
+   */
+  private volatile String cursorInstant;
+
+  /**
+   * The first write instant on the active timeline, used for query 
optimization.
+   */
+  private final String firstNonSavepointCommit;
+
+  /**
+   * The constructor.
+   *
+   * @param metaClient   The table meta client.
+   */
+  public CompletionTimeQueryViewV1(HoodieTableMetaClient metaClient) {
+    this(metaClient, HoodieInstantTimeGenerator.formatDate(new 
Date(Instant.now().minusMillis(MILLI_SECONDS_IN_THREE_DAYS).toEpochMilli())));
+  }
+
+  /**
+   * The constructor.
+   *
+   * @param metaClient   The table meta client.
+   * @param eagerLoadInstant The earliest instant time to eagerly load from, 
by default load last N days of completed instants.
+   */
+  public CompletionTimeQueryViewV1(HoodieTableMetaClient metaClient, String 
eagerLoadInstant) {
+    this.metaClient = metaClient;
+    this.beginToCompletionInstantTimeMap = new ConcurrentHashMap<>();
+    this.cursorInstant = InstantComparatorUtils.minInstant(eagerLoadInstant, 
metaClient.getActiveTimeline().firstInstant().map(HoodieInstant::getRequestTime).orElse(""));
+    // Note: use getWriteTimeline() to keep sync with the fs view 
visibleCommitsAndCompactionTimeline, see 
AbstractTableFileSystemView.refreshTimeline.
+    this.firstNonSavepointCommit = 
metaClient.getActiveTimeline().getWriteTimeline().getFirstNonSavepointCommit().map(HoodieInstant::getRequestTime).orElse("");
+    load();
+  }
+
+  @Override
+  public boolean isCompleted(String beginInstantTime) {
+    // archival does not proceed beyond the first savepoint, so any instant 
before that is completed.
+    return this.beginToCompletionInstantTimeMap.containsKey(beginInstantTime) 
|| isArchived(beginInstantTime);
+  }
+
+  @Override
+  public boolean isArchived(String instantTime) {
+    return InstantComparatorUtils.compareTimestamps(instantTime, LESSER_THAN, 
this.firstNonSavepointCommit);
+  }
+
+  @Override
+  public boolean isCompletedBefore(String baseInstant, String instantTime) {
+    Option<String> completionTimeOpt = getCompletionTime(baseInstant, 
instantTime);
+    if (completionTimeOpt.isPresent()) {
+      return InstantComparatorUtils.compareTimestamps(completionTimeOpt.get(), 
LESSER_THAN, baseInstant);
+    }
+    return false;
+  }
+
+  @Override
+  public boolean isSlicedAfterOrOn(String baseInstant, String instantTime) {
+    Option<String> completionTimeOpt = getCompletionTime(baseInstant, 
instantTime);
+    if (completionTimeOpt.isPresent()) {
+      return InstantComparatorUtils.compareTimestamps(completionTimeOpt.get(), 
GREATER_THAN_OR_EQUALS, baseInstant);
+    }
+    return true;
+  }
+
+  @Override
+  public Option<String> getCompletionTime(String baseInstant, String 
instantTime) {
+    Option<String> completionTimeOpt = getCompletionTime(instantTime);
+    if (completionTimeOpt.isPresent()) {
+      String completionTime = completionTimeOpt.get();
+      if (completionTime.length() != baseInstant.length()) {
+        // ==============================================================
+        // LEGACY CODE
+        // ==============================================================
+        // Fixes the completion time to reflect the completion sequence 
correctly
+        // if the file slice base instant time is not in datetime format.
+        // For example, many test cases just use integer string as the instant 
time.
+        // CAUTION: this fix only works for OCC(Optimistic Concurrency 
Control).
+        // for NB-CC(Non-blocking Concurrency Control), the file slicing may 
be incorrect.
+        return Option.of(instantTime);
+      }
+    }
+    return completionTimeOpt;
+  }
+
+  @Override
+  public Option<String> getCompletionTime(String beginTime) {
+    String completionTime = 
this.beginToCompletionInstantTimeMap.get(beginTime);
+    if (completionTime != null) {
+      return Option.of(completionTime);
+    }
+
+    // ***This is the key change between V1 and V2 completion time 
query-view***

Review Comment:
   querying v1 archived timeline will be slow and a non-starter for any 
queries.. Thats the reason. 



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