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


##########
hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncClient.java:
##########
@@ -894,7 +894,7 @@ public void close() {
   @Override
   public void updateLastCommitTimeSynced(String tableName) {
     HoodieTimeline activeTimeline = getActiveTimeline();
-    Option<String> lastCommitSynced = 
activeTimeline.lastInstant().map(HoodieInstant::getTimestamp);
+    Option<String> lastCommitSynced = 
activeTimeline.lastInstant().map(HoodieInstant::getRequestTime);

Review Comment:
   let's just call this `HoodieInstant::requestedTime` (the `ed` and drop `get` 
prefix) 
   
    In general, I think renaming HoodieInstant to HoodieAction might be better. 
HoodieAction -> has type , state, time. Today `HoodieInstant` is kind of used 
like `HoodieAction`. We can punt this from this PR.



##########
hudi-aws/src/test/java/org/apache/hudi/aws/testutils/GlueTestUtil.java:
##########
@@ -112,7 +112,7 @@ public static void createHoodieTable() throws IOException {
 
     String instantTime = "101";
     HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(false);
-    createMetaFile(basePath, HoodieTimeline.makeCommitFileName(instantTime), 
commitMetadata);
+    createMetaFile(basePath, new 
InstantFileNameFactoryV2().makeCommitFileName(instantTime), commitMetadata);

Review Comment:
   this will become `new DefaultInstantFileNameFactory()`



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelation.scala:
##########
@@ -153,10 +153,10 @@ class IncrementalRelation(val sqlContext: SQLContext,
       }.toMap
 
       for (commit <- commitsToReturn) {
-        val metadata: HoodieCommitMetadata = 
HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit)
-          .get, classOf[HoodieCommitMetadata])
+        val metadata: HoodieCommitMetadata = 
metaClient.getTimelineLayout.getCommitMetadataSerDe.deserialize(commit,

Review Comment:
   we do this kind of special casing only for HoodieCommitMetadata?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java:
##########
@@ -617,7 +631,9 @@ protected void runTableServicesInline(HoodieTable table, 
HoodieCommitMetadata me
   public Option<String> scheduleTableService(String instantTime, 
Option<Map<String, String>> extraMetadata,
                                              TableServiceType 
tableServiceType) {
     // A lock is required to guard against race conditions between an ongoing 
writer and scheduling a table service.
-    final Option<HoodieInstant> inflightInstant = Option.of(new 
HoodieInstant(HoodieInstant.State.REQUESTED,
+    HoodieTable<?, ?, ?, ?> table = createTable(config, storageConf, true);

Review Comment:
   lets avoid this extra `createTable()` call. we need to create the table from 
within the lock, so the state is ready without any other changes. 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java:
##########
@@ -217,7 +223,8 @@ protected HoodieWriteMetadata<O> logCompact(String 
logCompactionInstantTime, boo
     }
 
     HoodieTimeline pendingLogCompactionTimeline = 
table.getActiveTimeline().filterPendingLogCompactionTimeline();
-    HoodieInstant inflightInstant = 
HoodieTimeline.getLogCompactionInflightInstant(logCompactionInstantTime);
+    InstantFactory instantFactory = 
table.getMetaClient().getTimelineLayout().getInstantFactory();

Review Comment:
   Just aligning on use of factory pattern still. Are `InstantFactory` and 
`InstantFileNameFactory` factory pattern classes or just classes that can also 
be named `InstantGenerator` and `InstantFileNameGenerator` - given we just hang 
these off timeline layout and dynamically choose the right "factory". 
   
   Typically in a factory pattern, there is a single factory class, that 
dynamically decides what object to create?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/BaseHoodieCompactionPlanGenerator.java:
##########
@@ -81,7 +81,7 @@ public HoodieCompactionPlan generateCompactionPlan(String 
compactionInstant) thr
     // TODO : check if maxMemory is not greater than JVM or executor memory
     // TODO - rollback any compactions in flight
     HoodieTableMetaClient metaClient = hoodieTable.getMetaClient();
-    CompletionTimeQueryView completionTimeQueryView = new 
CompletionTimeQueryView(metaClient);
+    CompletionTimeQueryView completionTimeQueryView = 
metaClient.getTimelineLayout().getTimelineFactory().createCompletionTimeQueryView(metaClient);

Review Comment:
   I am wondering if hanging `completionTimeQueryView` off TimelineFactory is 
the right thing. `TimelineFactory` should have 1 job - create timelines?



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/AbstractHoodieBaseTimeline.java:
##########
@@ -104,133 +120,121 @@ public void appendInstants(List<HoodieInstant> 
newInstants) {
    *
    * @deprecated
    */
-  public HoodieDefaultTimeline() {
+  public AbstractHoodieBaseTimeline() {

Review Comment:
   can we remove this or still needed for serialization



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/InstantComparator.java:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.Comparator;
+
+public interface InstantComparator extends Serializable {
+
+  /**
+   * Returns Comparator that only uses action for ordering taking into account 
equivalent actions.
+   * @return
+   */
+  Comparator<HoodieInstant> getActionOnlyComparator();
+
+  /**
+   * Returns comparator that orders primarily based on timestamp and secondary 
ordering based on action and state.
+   * @return
+   */
+  Comparator<HoodieInstant> getRequestTimePrimaryOrderingComparator();

Review Comment:
   prior comments to rename these



##########
hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/HoodieStreamer.java:
##########
@@ -792,6 +793,10 @@ private void reInitDeltaSync() throws IOException {
     @Override
     protected Pair<CompletableFuture, ExecutorService> startService() {
       ExecutorService executor = Executors.newFixedThreadPool(1);
+      HoodieTableMetaClient meta = HoodieTableMetaClient.builder()

Review Comment:
   MetaClient is expensive.. so having one just for `InstantFactory` feels like 
a stretch. we need to fix this (anything like this)



##########
hudi-platform-service/hudi-metaserver/hudi-metaserver-client/src/main/java/org/apache/hudi/metaserver/client/HoodieMetaserverClientImp.java:
##########
@@ -56,6 +58,7 @@ public class HoodieMetaserverClientImp implements 
HoodieMetaserverClient {
   private boolean isLocal;
   private ThriftHoodieMetaserver.Iface client;
   private TTransport transport;
+  private InstantFactory instantFactory = new InstantFactoryV2();

Review Comment:
   similarly - can we have a `DefaultInstantFactory` instead of hard coding V2?



##########
hudi-aws/src/test/java/org/apache/hudi/aws/testutils/GlueTestUtil.java:
##########
@@ -23,7 +23,7 @@
 import org.apache.hudi.common.model.HoodieCommitMetadata;
 import org.apache.hudi.common.model.HoodieTableType;
 import org.apache.hudi.common.table.HoodieTableMetaClient;
-import org.apache.hudi.common.table.timeline.HoodieTimeline;
+import 
org.apache.hudi.common.table.timeline.versioning.v2.InstantFileNameFactoryV2;

Review Comment:
   can there there a `DefaultInstantFileNameFactory` that can either call V1 or 
V2.. and have that be used in classes this far outside? This way if there is a 
`V3`, then we need not chase these usage down again



##########
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/CommitsCompareProcedure.scala:
##########
@@ -54,15 +53,15 @@ class CommitsCompareProcedure() extends BaseProcedure with 
ProcedureBuilder {
     val sourceTimeline = 
source.getActiveTimeline.getCommitsTimeline.filterCompletedInstants
     val targetTimeline = 
target.getActiveTimeline.getCommitsTimeline.filterCompletedInstants
     val targetLatestCommit =
-      if (targetTimeline.getInstants.iterator().hasNext) 
targetTimeline.lastInstant.get.getTimestamp else "0"
+      if (targetTimeline.getInstants.iterator().hasNext) 
targetTimeline.lastInstant.get.getRequestTime else "0"
     val sourceLatestCommit =
-      if (sourceTimeline.getInstants.iterator().hasNext) 
sourceTimeline.lastInstant.get.getTimestamp else "0"
+      if (sourceTimeline.getInstants.iterator().hasNext) 
sourceTimeline.lastInstant.get.getRequestTime else "0"
 
-    if (sourceLatestCommit != null && 
HoodieTimeline.compareTimestamps(targetLatestCommit, 
HoodieTimeline.GREATER_THAN, sourceLatestCommit)) { // source is behind the 
target
-      val commitsToCatchup = 
targetTimeline.findInstantsAfter(sourceLatestCommit, 
Integer.MAX_VALUE).getInstants.iterator().asScala.map(instant => 
instant.getTimestamp).toList.asJava
+    if (sourceLatestCommit != null && compareTimestamps(targetLatestCommit, 
InstantComparatorUtils.GREATER_THAN, sourceLatestCommit)) { // source is behind 
the target

Review Comment:
   I'd like for key things like `GREATER_THAN` to be in some standard class, 
not `xxxUtils` classes



##########
hudi-cli/src/main/java/org/apache/hudi/cli/commands/SavepointsCommand.java:
##########
@@ -141,7 +142,8 @@ public String deleteSavepoint(
     if (completedInstants.empty()) {
       throw new HoodieException("There are no completed savepoint to run 
delete");
     }
-    HoodieInstant savePoint = new HoodieInstant(false, 
HoodieTimeline.SAVEPOINT_ACTION, instantTime);
+    InstantFactory instantFactory = 
metaClient.getTimelineLayout().getInstantFactory();
+    HoodieInstant savePoint = 
instantFactory.createNewInstant(HoodieInstant.State.COMPLETED, 
HoodieTimeline.SAVEPOINT_ACTION, instantTime);

Review Comment:
   a lot of these changes are due to these raw calls to `createNewInstant` 
right. and we are using these `InstantFactory` classes everywhere to adapt 
behavior. 
   
   Instead would adding a `createNewInstant()` to the new Timeline abstraction 
inside `metaClient` be better encapsulation. 
   
   i.e this line can be `metaClient.getTimeline.createNewInstant(COMPLETED, 
SAVEPOINT_ACTION, time)` .. and within that we get the layoutVersion and create 
factory objects accordingly?
   



##########
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowArchivedCommitsProcedure.scala:
##########
@@ -135,24 +134,25 @@ class ShowArchivedCommitsProcedure(includeExtraMetadata: 
Boolean) extends BasePr
     rows.stream().limit(limit).toArray().map(r => r.asInstanceOf[Row]).toList
   }
 
-  private def getSortCommits(timeline: HoodieDefaultTimeline): 
(util.ArrayList[Row], util.ArrayList[HoodieInstant]) = {
+  private def getSortCommits(timeline: HoodieTimeline): (util.ArrayList[Row], 
util.ArrayList[HoodieInstant]) = {
     val rows = new util.ArrayList[Row]
     // timeline can be read from multiple files. So sort is needed instead of 
reversing the collection
     val commits: util.List[HoodieInstant] = 
timeline.getCommitsTimeline.filterCompletedInstants
       .getInstants.toArray().map(instant => 
instant.asInstanceOf[HoodieInstant]).toList.asJava
     val newCommits = new util.ArrayList[HoodieInstant](commits)
-    Collections.sort(newCommits, 
HoodieInstant.INSTANT_TIME_COMPARATOR.reversed)
+    val layout = TimelineLayout.getLayout(timeline.getTimelineLayoutVersion)
+    Collections.sort(newCommits, 
layout.getInstantComparator.getRequestTimePrimaryOrderingComparator.reversed)

Review Comment:
   rename: `getRequestTimePrimaryOrderingComparator` -> 
`requestedTimeOrderedComparator` - does this work? 



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlCommonUtils.scala:
##########
@@ -259,7 +258,7 @@ object HoodieSqlCommonUtils extends SparkAdapterSupport {
       validateInstant(queryInstant)
       queryInstant
     } else if (instantLength == 10) { // for yyyy-MM-dd
-      
HoodieActiveTimeline.formatDate(defaultDateFormat.get().parse(queryInstant))
+      
ActiveTimelineUtils.formatDate(defaultDateFormat.get().parse(queryInstant))

Review Comment:
   consolidate into just `TimelineUtils`? there 3 util classes now.



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java:
##########
@@ -1436,7 +1445,7 @@ protected void tryUpgrade(HoodieTableMetaClient 
metaClient, Option<String> insta
     UpgradeDowngrade upgradeDowngrade =
         new UpgradeDowngrade(metaClient, config, context, 
upgradeDowngradeHelper);
 
-    if 
(upgradeDowngrade.needsUpgradeOrDowngrade(HoodieTableVersion.current())) {
+    if (upgradeDowngrade.needsUpgradeOrDowngrade(config.getWriteVersion())) {

Review Comment:
   nice. 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java:
##########
@@ -816,7 +832,7 @@ protected void archive(HoodieTable table) {
       // Reload table timeline to reflect the latest commits,
       // there are some table services (for e.g, the cleaning) that executed 
right before the archiving.
       table.getMetaClient().reloadActiveTimeline();
-      HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(config, 
table);
+      HoodieTimelineArchiver archiver = new 
TimelineArchiverFactory<>().getTimelineArchiver(table.getMetaClient().getTimelineLayoutVersion(),
 config, table);

Review Comment:
   In general, can we have static `.create()` method for these factories. i.e 
`TimelineArchiverFactory.createTimelineArchiver()` without needing a `new 
TimelineArchiverFactory`. is n't that more typical



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/AbstractHoodieBaseTimeline.java:
##########
@@ -520,27 +492,27 @@ public List<HoodieInstant> getInstants() {
 
   @Override
   public Stream<HoodieInstant> getReverseOrderedInstants() {
-    return getInstantsAsStream().sorted(INSTANT_TIME_COMPARATOR.reversed());
+    return 
getInstantsAsStream().sorted(instantComparator.getRequestTimePrimaryOrderingComparator().reversed());
   }
 
   @Override
   public Option<String> getLatestCompletionTime() {
     return Option.fromJavaOptional(getInstantsAsStream().filter(s -> 
s.getCompletionTime() != null)
-        .max(HoodieInstant.COMPLETION_TIME_COMPARATOR)
+        .max(instantComparator.getCompletionTimePrimaryOrderingComparator())
         .map(HoodieInstant::getCompletionTime));
   }
 
   @Override
   public Stream<HoodieInstant> getInstantsOrderedByCompletionTime() {
     return getInstantsAsStream().filter(s -> s.getCompletionTime() != null)
-        .sorted(HoodieInstant.COMPLETION_TIME_COMPARATOR);
+        
.sorted(instantComparator.getCompletionTimePrimaryOrderingComparator());

Review Comment:
   rename: completionTimeOrderedComparator



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/versioning/v2/TimelineArchiverV2.java:
##########
@@ -0,0 +1,369 @@
+/*
+ * 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.timeline.versioning.v2;
+
+import org.apache.hudi.client.timeline.HoodieTimelineArchiver;
+import org.apache.hudi.client.transaction.TransactionManager;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.model.HoodieAvroPayload;
+import org.apache.hudi.common.model.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.ActiveAction;
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
+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.table.timeline.versioning.v2.ActiveTimelineV2;
+import org.apache.hudi.common.table.timeline.versioning.v2.InstantComparatorV2;
+import org.apache.hudi.common.util.ClusteringUtils;
+import org.apache.hudi.common.util.CollectionUtils;
+import org.apache.hudi.common.util.CompactionUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.exception.HoodieLockException;
+import org.apache.hudi.metadata.HoodieTableMetadata;
+import org.apache.hudi.metrics.HoodieMetrics;
+import org.apache.hudi.table.HoodieTable;
+import org.apache.hudi.table.action.compact.CompactionTriggerStrategy;
+import org.apache.hudi.table.marker.WriteMarkers;
+import org.apache.hudi.table.marker.WriteMarkersFactory;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import static 
org.apache.hudi.client.utils.ArchivalUtils.getMinAndMaxInstantsToKeep;
+import static 
org.apache.hudi.common.table.timeline.InstantComparatorUtils.LESSER_THAN;
+import static 
org.apache.hudi.common.table.timeline.InstantComparatorUtils.compareTimestamps;
+
+/**
+ * Archiver to bound the growth of files under .hoodie meta path.
+ */
+public class TimelineArchiverV2<T extends HoodieAvroPayload, I, K, O> 
implements HoodieTimelineArchiver<T, I, K, O> {

Review Comment:
   are tests working on just V2 archiver.. and not V1? just confirming my 
understanding.



##########
hudi-common/src/main/java/org/apache/hudi/common/table/read/IncrementalQueryAnalyzer.java:
##########
@@ -200,6 +201,8 @@ public QueryContext analyze() {
           startCompletionTime.isEmpty() ? lastInstant : instants.get(0);
       String endInstant = endCompletionTime.isEmpty() ? null : lastInstant;
       return QueryContext.create(startInstant, endInstant, instants, 
archivedInstants, activeInstants, filteredTimeline, archivedReadTimeline);
+    } catch (Exception ex) {
+      throw new HoodieException(ex);

Review Comment:
   add error message



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieClient.java:
##########
@@ -198,14 +196,14 @@ protected HoodieTableMetaClient createMetaClient(boolean 
loadActiveTimelineOnLoa
    * @param milliseconds Milliseconds to add to current time while generating 
the new instant time.
    */
   public String createNewInstantTime(long milliseconds) {
-    return HoodieActiveTimeline.createNewInstantTime(true, timeGenerator, 
milliseconds);
+    return ActiveTimelineUtils.createNewInstantTime(true, timeGenerator, 
milliseconds);

Review Comment:
   while we are at it, lets call `createNewInstantTime` -> 
`generateInstantTime` 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieWriteClient.java:
##########
@@ -729,20 +734,21 @@ public void restoreToSavepoint(String savepointTime) {
         Option<HoodieInstant> oldestMdtCompaction = 
mdtMetaClient.getCommitTimeline().filterCompletedInstants().firstInstant();
         boolean deleteMDT = false;
         if (oldestMdtCompaction.isPresent()) {
-          if (HoodieTimeline.LESSER_THAN_OR_EQUALS.test(savepointTime, 
oldestMdtCompaction.get().getTimestamp())) {
+          if (LESSER_THAN_OR_EQUALS.test(savepointTime, 
oldestMdtCompaction.get().getRequestTime())) {
             LOG.warn(String.format("Deleting MDT during restore to %s as the 
savepoint is older than oldest compaction %s on MDT",
-                savepointTime, oldestMdtCompaction.get().getTimestamp()));
+                savepointTime, oldestMdtCompaction.get().getRequestTime()));
             deleteMDT = true;
           }
         }
 
         // The instant required to sync rollback to MDT has been archived and 
the mdt syncing will be failed
         // So that we need to delete the whole MDT here.
         if (!deleteMDT) {
-          HoodieInstant syncedInstant = new HoodieInstant(false, 
HoodieTimeline.DELTA_COMMIT_ACTION, savepointTime);
-          if 
(mdtMetaClient.getCommitsTimeline().isBeforeTimelineStarts(syncedInstant.getTimestamp()))
 {
+          InstantFactory instantFactory = 
mdtMetaClient.getTimelineLayout().getInstantFactory();
+          HoodieInstant syncedInstant = 
instantFactory.createNewInstant(HoodieInstant.State.COMPLETED, 
HoodieTimeline.DELTA_COMMIT_ACTION, savepointTime);

Review Comment:
   I guess - we cannot fully eliminate uses of `createNewInstant` since there 
are probably tons of these usages to compare/check against timeline.



##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCommitMetadata.java:
##########
@@ -495,20 +493,6 @@ public int hashCode() {
     return result;
   }
 
-  public static <T> T fromBytes(byte[] bytes, Class<T> clazz) throws 
IOException {

Review Comment:
   assume this is moved over to the commitMetadataSerDe



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/AbstractHoodieBaseTimeline.java:
##########
@@ -53,9 +54,9 @@
  *
  * @see HoodieTimeline
  */
-public class HoodieDefaultTimeline implements HoodieTimeline {
+public abstract class AbstractHoodieBaseTimeline implements HoodieTimeline {

Review Comment:
   rename : `BaseHoodieTimeline`



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/ActiveTimelineUtils.java:
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.model.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.storage.HoodieStorage;
+import org.apache.hudi.storage.StoragePath;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.text.ParseException;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.Set;
+
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION;
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.COMPACTION_ACTION;
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.DELTA_COMMIT_ACTION;
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.LOG_COMPACTION_ACTION;
+
+public class ActiveTimelineUtils {

Review Comment:
   are these methods unit tested



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/InstantComparatorUtils.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.util.StringUtils;
+
+import java.util.Objects;
+import java.util.function.BiPredicate;
+
+/**
+ * Helper methods to compare instants.
+ **/
+public class InstantComparatorUtils {
+
+  public static final BiPredicate<String, String> EQUALS = (commit1, commit2) 
-> commit1.compareTo(commit2) == 0;

Review Comment:
   lets just leave these as static members on the interface itself? 



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/InstantComparator.java:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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 java.io.Serializable;
+import java.util.Comparator;
+
+public interface InstantComparator extends Serializable {
+
+  /**
+   * Returns Comparator that only uses action for ordering taking into account 
equivalent actions.
+   * @return
+   */
+  Comparator<HoodieInstant> getActionOnlyComparator();

Review Comment:
   we can drop the `get` prefix IMO



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/CommitMetadataSerDe.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.model.HoodieCommitMetadata;
+import org.apache.hudi.common.util.Option;
+
+import java.io.IOException;
+import java.io.Serializable;
+
+/**
+ * Interface for serializing and deserializing commit metadata.
+ */
+public interface CommitMetadataSerDe extends Serializable {

Review Comment:
   Should we bring all instant metadata SerDe into an interface instead of just 
commit metadata? 
   
   even if not in this PR - should be a good follow on?



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/AbstractHoodieBaseTimeline.java:
##########
@@ -633,34 +581,13 @@ private Set<String> getOrCreateInstantSet() {
     if (this.instantTimeSet == null) {
       synchronized (this) {
         if (this.instantTimeSet == null) {
-          this.instantTimeSet = 
this.instants.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toSet());
+          this.instantTimeSet = 
this.instants.stream().map(HoodieInstant::getRequestTime).collect(Collectors.toSet());
         }
       }
     }
     return this.instantTimeSet;
   }
 
-  private Set<String> getOrCreatePendingClusteringInstantSet() {

Review Comment:
   I guess, you have just moved these into the BaseXXXV1/V2 classes, while 
copying the older impl back?



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/ActiveTimelineUtils.java:
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.model.HoodieTableType;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.storage.HoodieStorage;
+import org.apache.hudi.storage.StoragePath;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.text.ParseException;
+import java.util.Date;
+import java.util.HashSet;
+import java.util.Set;
+
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION;
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.COMPACTION_ACTION;
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.DELTA_COMMIT_ACTION;
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.LOG_COMPACTION_ACTION;
+
+public class ActiveTimelineUtils {
+  private static final Logger LOG = 
LoggerFactory.getLogger(ActiveTimelineUtils.class);
+
+  public static final Set<String> NOT_PARSABLE_TIMESTAMPS = new 
HashSet<String>(3) {
+    {
+      add(HoodieTimeline.INIT_INSTANT_TS);
+      add(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS);
+      add(HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS);
+    }
+  };
+
+  /**
+   * Parse the timestamp of an Instant and return a {@code Date}.
+   * Throw ParseException if timestamp is not valid format as
+   *  {@link 
org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator#SECS_INSTANT_TIMESTAMP_FORMAT}.
+   *
+   * @param timestamp a timestamp String which follow pattern as
+   *  {@link 
org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator#SECS_INSTANT_TIMESTAMP_FORMAT}.
+   * @return Date of instant timestamp
+   */
+  public static Date parseDateFromInstantTime(String timestamp) throws 
ParseException {
+    return HoodieInstantTimeGenerator.parseDateFromInstantTime(timestamp);
+  }
+
+  /**
+   * The same parsing method as above, but this method will mute 
ParseException.
+   * If the given timestamp is invalid, returns {@code Option.empty}.
+   * Or a corresponding Date value if these timestamp strings are provided
+   *  {@link 
org.apache.hudi.common.table.timeline.HoodieTimeline#INIT_INSTANT_TS},
+   *  {@link 
org.apache.hudi.common.table.timeline.HoodieTimeline#METADATA_BOOTSTRAP_INSTANT_TS},
+   *  {@link 
org.apache.hudi.common.table.timeline.HoodieTimeline#FULL_BOOTSTRAP_INSTANT_TS}.
+   * This method is useful when parsing timestamp for metrics
+   *
+   * @param timestamp a timestamp String which follow pattern as
+   *  {@link 
org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator#SECS_INSTANT_TIMESTAMP_FORMAT}.
+   * @return {@code Option<Date>} of instant timestamp, {@code Option.empty} 
if invalid timestamp
+   */
+  public static Option<Date> parseDateFromInstantTimeSafely(String timestamp) {
+    Option<Date> parsedDate;
+    try {
+      parsedDate = 
Option.of(HoodieInstantTimeGenerator.parseDateFromInstantTime(timestamp));
+    } catch (ParseException e) {
+      if (NOT_PARSABLE_TIMESTAMPS.contains(timestamp)) {
+        parsedDate = Option.of(new Date(Integer.parseInt(timestamp)));
+      } else {
+        LOG.warn("Failed to parse timestamp {}: {}", timestamp, 
e.getMessage());
+        parsedDate = Option.empty();
+      }
+    }
+    return parsedDate;
+  }
+
+  /**
+   * Format the Date to a String representing the timestamp of a Hoodie 
Instant.
+   */
+  public static String formatDate(Date timestamp) {
+    return HoodieInstantTimeGenerator.formatDate(timestamp);
+  }
+
+  /**
+   * Returns next instant time in the correct format.
+   * Ensures each instant time is at least 1 millisecond apart since we create 
instant times at millisecond granularity.
+   *
+   * @param shouldLock whether the lock should be enabled to get the instant 
time.
+   * @param timeGenerator TimeGenerator used to generate the instant time.
+   */
+  public static String createNewInstantTime(boolean shouldLock, TimeGenerator 
timeGenerator) {
+    return createNewInstantTime(shouldLock, timeGenerator, 0L);
+  }
+
+  /**
+   * Returns next instant time in the correct format.
+   * Ensures each instant time is at least 1 millisecond apart since we create 
instant times at millisecond granularity.
+   *
+   * @param shouldLock whether the lock should be enabled to get the instant 
time.
+   * @param timeGenerator TimeGenerator used to generate the instant time.
+   * @param milliseconds Milliseconds to add to current time while generating 
the new instant time
+   */
+  public static String createNewInstantTime(boolean shouldLock, TimeGenerator 
timeGenerator, long milliseconds) {
+    return HoodieInstantTimeGenerator.createNewInstantTime(shouldLock, 
timeGenerator, milliseconds);
+  }
+
+  /**
+   * Delete Instant file from storage
+   * @param storage Hoodie Storage.
+   * @param metaPath Path.
+   * @param instant instant to delete.
+   * @param factory Factory to generate file name.
+   */
+  public static void deleteInstantFile(HoodieStorage storage, StoragePath 
metaPath, HoodieInstant instant, InstantFileNameFactory factory) {
+    String filePath = factory.getFileName(instant);
+    try {
+      storage.deleteFile(new StoragePath(metaPath, filePath));
+    } catch (IOException e) {
+      throw new HoodieIOException("Could not delete instant file" + filePath, 
e);
+    }
+  }
+
+  /**
+   * Returns the inflight instant corresponding to the instant being passed. 
Takes care of changes in action names
+   * between inflight and completed instants (compaction <=> commit) and 
(logcompaction <==> deltacommit).
+   * @param instant Hoodie Instant
+   * @param metaClient Hoodie metaClient to fetch tableType and fileSystem.
+   * @return Inflight Hoodie Instant
+   */
+  public static HoodieInstant getInflightInstant(final HoodieInstant instant, 
final HoodieTableMetaClient metaClient) {

Review Comment:
   is this new code or helper borrowed back from 0.x?



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineLayout.java:
##########
@@ -45,33 +58,133 @@ public static TimelineLayout 
getLayout(TimelineLayoutVersion version) {
 
   public abstract Stream<HoodieInstant> 
filterHoodieInstants(Stream<HoodieInstant> instantStream);
 
+  public abstract InstantFactory getInstantFactory();
+
+  public abstract InstantFileNameFactory getInstantFileNameFactory();
+
+  public abstract TimelineFactory getTimelineFactory();
+
+  public abstract InstantComparator getInstantComparator();
+
+  public abstract InstantFileNameParser getInstantFileNameParser();
+
+  public abstract CommitMetadataSerDe getCommitMetadataSerDe();
+
   /**
    * Table Layout where state transitions are managed by renaming files.
    */
   private static class TimelineLayoutV0 extends TimelineLayout {
 
+    private final InstantFactory instantFactory = new InstantFactoryV1();
+    private final InstantFileNameFactory instantFileNameFactory = new 
InstantFileNameFactoryV1();
+    private final TimelineFactory timelineFactory = new 
TimelineV1Factory(this);
+    private final InstantComparator instantComparator = new 
InstantComparatorV1();
+    private final InstantFileNameParser fileNameParser = new 
InstantFileNameParserV2();
+
     @Override
     public Stream<HoodieInstant> filterHoodieInstants(Stream<HoodieInstant> 
instantStream) {
       return instantStream;
     }
+
+    @Override
+    public InstantFactory getInstantFactory() {
+      return instantFactory;
+    }
+
+    @Override
+    public InstantFileNameFactory getInstantFileNameFactory() {
+      return instantFileNameFactory;
+    }
+
+    @Override
+    public TimelineFactory getTimelineFactory() {
+      return timelineFactory;
+    }
+
+    @Override
+    public InstantComparator getInstantComparator() {
+      return instantComparator;
+    }
+
+    @Override
+    public InstantFileNameParser getInstantFileNameParser() {
+      return fileNameParser;
+    }
+
+    @Override
+    public CommitMetadataSerDe getCommitMetadataSerDe() {
+      return new CommitMetadataSerDeV1();
+    }
+  }
+
+  private static Stream<HoodieInstant> 
filterHoodieInstantsByLatestState(Stream<HoodieInstant> instantStream,

Review Comment:
   new method? tested?



##########
hudi-common/src/main/java/org/apache/hudi/common/util/InternalSchemaCache.java:
##########
@@ -194,7 +199,10 @@ public static InternalSchema 
getInternalSchemaByVersionId(long versionId, String
         } catch (IOException e) {
           throw e;
         }
-        HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes(data, 
HoodieCommitMetadata.class);
+        short repl = 0;
+        HoodieCommitMetadata metadata = 
layout.getCommitMetadataSerDe().deserialize(layout.getInstantFactory().createNewInstant(
+            new StoragePathInfo(candidateCommitFile, -1, false, repl, 0L, 0L)),

Review Comment:
   anyway to avoid the `repl`



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/common/InstantComparatorHelper.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.common;
+
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+
+import java.io.Serializable;
+import java.util.Comparator;
+import java.util.Map;
+
+/**
+ * Comparators for HoodieInstant that are also serializable.
+ * java.util.Comparators are not serializable.
+ */
+public class InstantComparatorHelper {
+
+  public static class ActionComparator implements Serializable, 
Comparator<HoodieInstant> {
+    private final Map<String, String> comparableActions;
+
+    public ActionComparator(Map<String, String> comparableActions) {
+      this.comparableActions = comparableActions;
+    }
+
+    @Override
+    public int compare(HoodieInstant instant1, HoodieInstant instant2) {
+      String action1 = getComparableAction(instant1.getAction());
+      String action2 = getComparableAction(instant2.getAction());
+      return action1.compareTo(action2);
+    }
+
+    private String getComparableAction(String action) {
+      return comparableActions.getOrDefault(action, action);
+    }
+
+  }
+
+  public static class TimestampBasedComparator implements Serializable, 
Comparator<HoodieInstant> {

Review Comment:
   rename



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/common/InstantComparatorHelper.java:
##########
@@ -0,0 +1,89 @@
+/*
+ * 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.common;
+
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+
+import java.io.Serializable;
+import java.util.Comparator;
+import java.util.Map;
+
+/**
+ * Comparators for HoodieInstant that are also serializable.
+ * java.util.Comparators are not serializable.
+ */
+public class InstantComparatorHelper {

Review Comment:
   rename: InstantCompartors (this is not a helper class per se. but a 
collection of classes)



##########
hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark3ParquetSchemaEvolutionUtils.scala:
##########
@@ -57,9 +57,11 @@ class Spark3ParquetSchemaEvolutionUtils(sharedConf: 
Configuration,
   private lazy val tablePath: String = 
sharedConf.get(SparkInternalSchemaConverter.HOODIE_TABLE_PATH)
   private lazy val fileSchema: InternalSchema = if (shouldUseInternalSchema) {
     val commitInstantTime = FSUtils.getCommitTime(filePath.getName).toLong;
+    //TODO: HARDCODED TIMELINE OBJECT
     val validCommits = 
sharedConf.get(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST)
     InternalSchemaCache.getInternalSchemaByVersionId(commitInstantTime, 
tablePath,
-      new HoodieHadoopStorage(tablePath, sharedConf), if (validCommits == 
null) "" else validCommits)
+      new HoodieHadoopStorage(tablePath, sharedConf), if (validCommits == 
null) "" else validCommits,

Review Comment:
   any way to make the cache not aware of timeline layout etc? 



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/InstantComparatorUtils.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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.util.StringUtils;
+
+import java.util.Objects;
+import java.util.function.BiPredicate;
+
+/**
+ * Helper methods to compare instants.
+ **/
+public class InstantComparatorUtils {
+
+  public static final BiPredicate<String, String> EQUALS = (commit1, commit2) 
-> commit1.compareTo(commit2) == 0;

Review Comment:
   alternatively - call this `InstantComparison`



##########
hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineFactory.java:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.Option;
+
+import java.io.Serializable;
+import java.util.function.Function;
+import java.util.stream.Stream;
+
+public abstract class TimelineFactory implements Serializable {
+
+  public abstract HoodieTimeline createDefaultTimeline(Stream<HoodieInstant> 
instants,
+                                                       Function<HoodieInstant, 
Option<byte[]>> details);
+
+  public abstract HoodieActiveTimeline createActiveTimeline();
+
+  public abstract HoodieArchivedTimeline 
createArchivedTimeline(HoodieTableMetaClient metaClient);
+
+  public abstract HoodieArchivedTimeline 
createArchivedTimeline(HoodieTableMetaClient metaClient, String startTs);
+
+  public abstract ArchivedTimelineLoader createArchivedTimelineLoader();
+
+  public abstract HoodieActiveTimeline 
createActiveTimeline(HoodieTableMetaClient metaClient);
+
+  public abstract HoodieActiveTimeline 
createActiveTimeline(HoodieTableMetaClient metaClient, boolean 
applyLayoutFilter);
+
+  public abstract CompletionTimeQueryView 
createCompletionTimeQueryView(HoodieTableMetaClient metaClient);

Review Comment:
   does not fit that well here? or we just rename this differently.. and not a 
"factory of timelines"



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