yihua commented on code in PR #12781:
URL: https://github.com/apache/hudi/pull/12781#discussion_r1980247803


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConcurrentSchemaEvolutionTableSchemaGetter.java:
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.transaction;
+
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.HoodieSchemaNotFoundException;
+import org.apache.hudi.common.model.HoodieCommitMetadata;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.TimelineLayout;
+import org.apache.hudi.common.util.ClusteringUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.VisibleForTesting;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.internal.schema.HoodieSchemaException;
+import org.apache.hudi.util.Lazy;
+
+import org.apache.avro.JsonProperties;
+import org.apache.avro.Schema;
+import org.apache.avro.Schema.Field;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Supplier;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.avro.AvroSchemaUtils.appendFieldsToSchema;
+import static org.apache.hudi.avro.AvroSchemaUtils.containsFieldInSchema;
+import static org.apache.hudi.avro.AvroSchemaUtils.createNullableSchema;
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION;
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.DELTA_COMMIT_ACTION;
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION;
+import static 
org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN_OR_EQUALS;
+import static 
org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps;
+
+/**
+ * Helper class to read table schema. ONLY USE IT FOR 
SimpleConcurrentFileWritesConflictResolutionStrategy.
+ */
+class ConcurrentSchemaEvolutionTableSchemaGetter {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ConcurrentSchemaEvolutionTableSchemaGetter.class);
+
+  protected final HoodieTableMetaClient metaClient;
+
+  private final Lazy<ConcurrentHashMap<HoodieInstant, Schema>> 
tableSchemaCache;
+
+  private Option<HoodieInstant> latestCommitWithValidSchema = Option.empty();
+
+  @VisibleForTesting
+  public ConcurrentHashMap<HoodieInstant, Schema> getTableSchemaCache() {
+    return tableSchemaCache.get();
+  }
+
+  public ConcurrentSchemaEvolutionTableSchemaGetter(HoodieTableMetaClient 
metaClient) {
+    this.metaClient = metaClient;
+    // Unbounded sized map. Should replace with some caching library.
+    this.tableSchemaCache = Lazy.lazily(ConcurrentHashMap::new);
+  }
+
+  /**
+   * Handles partition column logic for a given schema.
+   *
+   * @param schema the input schema to process
+   * @return the processed schema with partition columns handled appropriately
+   */
+  private Schema handlePartitionColumnsIfNeeded(Schema schema) {
+    if (metaClient.getTableConfig().shouldDropPartitionColumns()) {
+      return metaClient.getTableConfig().getPartitionFields()
+          .map(partitionFields -> appendPartitionColumns(schema, 
Option.ofNullable(partitionFields)))
+          .or(() -> Option.of(schema))
+          .get();
+    }
+    return schema;
+  }
+
+  public Option<Schema> getTableAvroSchemaIfPresent(boolean 
includeMetadataFields, Option<HoodieInstant> instant) {
+    return getTableAvroSchemaFromTimelineWithCache(instant) // Get table 
schema from schema evolution timeline.
+        .or(this::getTableCreateSchemaWithoutMetaField) // Fall back: read 
create schema from table config.
+        .map(tableSchema -> includeMetadataFields ? 
HoodieAvroUtils.addMetadataFields(tableSchema, false) : 
HoodieAvroUtils.removeMetadataFields(tableSchema))
+        .map(this::handlePartitionColumnsIfNeeded);
+  }
+
+  private Option<Schema> getTableCreateSchemaWithoutMetaField() {
+    return metaClient.getTableConfig().getTableCreateSchema();
+  }
+
+  private void setCachedLatestCommitWithValidSchema(Option<HoodieInstant> 
instantOption) {
+    latestCommitWithValidSchema = instantOption;
+  }
+
+  private Option<HoodieInstant> getCachedLatestCommitWithValidSchema() {
+    return latestCommitWithValidSchema;
+  }
+
+  @VisibleForTesting
+  Option<Schema> getTableAvroSchemaFromTimelineWithCache(Option<HoodieInstant> 
instantTime) {
+    return 
getTableAvroSchemaFromTimelineWithCache(computeSchemaEvolutionTimelineInReverseOrder(),
 instantTime);
+  }
+
+  Option<Schema> getTableAvroSchemaFromTimelineWithCache(Stream<HoodieInstant> 
reversedTimelineStream, Option<HoodieInstant> instantTime) {
+    // If instantTime is empty it means read the latest one. In that case, get 
the cached instant if there is one.
+    boolean fetchFromLastValidCommit = instantTime.isEmpty();
+    Option<HoodieInstant> targetInstant = 
instantTime.or(getCachedLatestCommitWithValidSchema());
+    Schema cachedTableSchema = null;
+
+    // Try cache first if there is a target instant to fetch for.
+    if (!targetInstant.isEmpty()) {
+      cachedTableSchema = 
tableSchemaCache.get().getOrDefault(targetInstant.get(), null);
+    }
+
+    // Cache miss on either latestCommitWithValidSchema or 
commitMetadataCache. Compute the result.
+    if (cachedTableSchema == null) {
+      Option<Pair<HoodieInstant, Schema>> instantWithSchema = 
getLastCommitMetadataWithValidSchemaFromTimeline(reversedTimelineStream, 
targetInstant);
+      if (instantWithSchema.isPresent()) {
+        targetInstant = Option.of(instantWithSchema.get().getLeft());
+        cachedTableSchema = instantWithSchema.get().getRight();
+      }
+    }
+
+    // After computation, update the cache for the instant and commit metadata.
+    if (fetchFromLastValidCommit) {
+      setCachedLatestCommitWithValidSchema(targetInstant);
+    }
+    if (cachedTableSchema != null) {
+      // We save cache for 2 cases
+      // - input specifies a specific instant, we update the cache for that 
instant, which is instantTime.
+      // - input is empty implying fetch the latest schema, update the cache 
for the
+      //   latest valid commit which is targetInstant
+      if (instantTime.isPresent()) {
+        tableSchemaCache.get().putIfAbsent(instantTime.get(), 
cachedTableSchema);
+      }
+      if (targetInstant.isPresent()) {
+        tableSchemaCache.get().putIfAbsent(targetInstant.get(), 
cachedTableSchema);
+      }
+    }
+
+    // Finally process the computation results and return.
+    if (cachedTableSchema == null) {
+      return Option.empty();
+    }
+
+    return Option.of(cachedTableSchema);
+  }
+
+  @VisibleForTesting
+  Option<Pair<HoodieInstant, Schema>> 
getLastCommitMetadataWithValidSchemaFromTimeline(Stream<HoodieInstant> 
reversedTimelineStream, Option<HoodieInstant> instant) {
+    // To find the table schema given an instant time, need to walk backwards 
from the latest instant in
+    // the timeline finding a completed instant containing a valid schema.
+    ConcurrentHashMap<HoodieInstant, Schema> tableSchemaAtInstant = new 
ConcurrentHashMap<>();
+    Option<HoodieInstant> instantWithTableSchema = 
Option.fromJavaOptional(reversedTimelineStream
+        // If a completion time is specified, find the first eligible instant 
in the schema evolution timeline.
+        // Should switch to completion time based.
+        .filter(s -> instant.isEmpty() || compareTimestamps(s.requestedTime(), 
LESSER_THAN_OR_EQUALS, instant.get().requestedTime()))

Review Comment:
   is completion time here?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConcurrentSchemaEvolutionTableSchemaGetter.java:
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.transaction;
+
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.HoodieSchemaNotFoundException;
+import org.apache.hudi.common.model.HoodieCommitMetadata;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.TimelineLayout;
+import org.apache.hudi.common.util.ClusteringUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.VisibleForTesting;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.internal.schema.HoodieSchemaException;
+import org.apache.hudi.util.Lazy;
+
+import org.apache.avro.JsonProperties;
+import org.apache.avro.Schema;
+import org.apache.avro.Schema.Field;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Supplier;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.avro.AvroSchemaUtils.appendFieldsToSchema;
+import static org.apache.hudi.avro.AvroSchemaUtils.containsFieldInSchema;
+import static org.apache.hudi.avro.AvroSchemaUtils.createNullableSchema;
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION;
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.DELTA_COMMIT_ACTION;
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION;
+import static 
org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN_OR_EQUALS;
+import static 
org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps;
+
+/**
+ * Helper class to read table schema. ONLY USE IT FOR 
SimpleConcurrentFileWritesConflictResolutionStrategy.
+ */
+class ConcurrentSchemaEvolutionTableSchemaGetter {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ConcurrentSchemaEvolutionTableSchemaGetter.class);
+
+  protected final HoodieTableMetaClient metaClient;
+
+  private final Lazy<ConcurrentHashMap<HoodieInstant, Schema>> 
tableSchemaCache;
+
+  private Option<HoodieInstant> latestCommitWithValidSchema = Option.empty();
+
+  @VisibleForTesting
+  public ConcurrentHashMap<HoodieInstant, Schema> getTableSchemaCache() {
+    return tableSchemaCache.get();
+  }
+
+  public ConcurrentSchemaEvolutionTableSchemaGetter(HoodieTableMetaClient 
metaClient) {
+    this.metaClient = metaClient;
+    // Unbounded sized map. Should replace with some caching library.
+    this.tableSchemaCache = Lazy.lazily(ConcurrentHashMap::new);
+  }
+
+  /**
+   * Handles partition column logic for a given schema.
+   *
+   * @param schema the input schema to process
+   * @return the processed schema with partition columns handled appropriately
+   */
+  private Schema handlePartitionColumnsIfNeeded(Schema schema) {
+    if (metaClient.getTableConfig().shouldDropPartitionColumns()) {
+      return metaClient.getTableConfig().getPartitionFields()
+          .map(partitionFields -> appendPartitionColumns(schema, 
Option.ofNullable(partitionFields)))
+          .or(() -> Option.of(schema))
+          .get();
+    }
+    return schema;
+  }
+
+  public Option<Schema> getTableAvroSchemaIfPresent(boolean 
includeMetadataFields, Option<HoodieInstant> instant) {
+    return getTableAvroSchemaFromTimelineWithCache(instant) // Get table 
schema from schema evolution timeline.
+        .or(this::getTableCreateSchemaWithoutMetaField) // Fall back: read 
create schema from table config.
+        .map(tableSchema -> includeMetadataFields ? 
HoodieAvroUtils.addMetadataFields(tableSchema, false) : 
HoodieAvroUtils.removeMetadataFields(tableSchema))
+        .map(this::handlePartitionColumnsIfNeeded);
+  }
+
+  private Option<Schema> getTableCreateSchemaWithoutMetaField() {
+    return metaClient.getTableConfig().getTableCreateSchema();
+  }
+
+  private void setCachedLatestCommitWithValidSchema(Option<HoodieInstant> 
instantOption) {
+    latestCommitWithValidSchema = instantOption;
+  }
+
+  private Option<HoodieInstant> getCachedLatestCommitWithValidSchema() {
+    return latestCommitWithValidSchema;
+  }
+
+  @VisibleForTesting
+  Option<Schema> getTableAvroSchemaFromTimelineWithCache(Option<HoodieInstant> 
instantTime) {
+    return 
getTableAvroSchemaFromTimelineWithCache(computeSchemaEvolutionTimelineInReverseOrder(),
 instantTime);
+  }
+
+  Option<Schema> getTableAvroSchemaFromTimelineWithCache(Stream<HoodieInstant> 
reversedTimelineStream, Option<HoodieInstant> instantTime) {
+    // If instantTime is empty it means read the latest one. In that case, get 
the cached instant if there is one.
+    boolean fetchFromLastValidCommit = instantTime.isEmpty();
+    Option<HoodieInstant> targetInstant = 
instantTime.or(getCachedLatestCommitWithValidSchema());
+    Schema cachedTableSchema = null;
+
+    // Try cache first if there is a target instant to fetch for.
+    if (!targetInstant.isEmpty()) {
+      cachedTableSchema = 
tableSchemaCache.get().getOrDefault(targetInstant.get(), null);
+    }
+
+    // Cache miss on either latestCommitWithValidSchema or 
commitMetadataCache. Compute the result.
+    if (cachedTableSchema == null) {
+      Option<Pair<HoodieInstant, Schema>> instantWithSchema = 
getLastCommitMetadataWithValidSchemaFromTimeline(reversedTimelineStream, 
targetInstant);
+      if (instantWithSchema.isPresent()) {
+        targetInstant = Option.of(instantWithSchema.get().getLeft());
+        cachedTableSchema = instantWithSchema.get().getRight();
+      }
+    }
+
+    // After computation, update the cache for the instant and commit metadata.
+    if (fetchFromLastValidCommit) {
+      setCachedLatestCommitWithValidSchema(targetInstant);
+    }
+    if (cachedTableSchema != null) {
+      // We save cache for 2 cases
+      // - input specifies a specific instant, we update the cache for that 
instant, which is instantTime.
+      // - input is empty implying fetch the latest schema, update the cache 
for the
+      //   latest valid commit which is targetInstant
+      if (instantTime.isPresent()) {
+        tableSchemaCache.get().putIfAbsent(instantTime.get(), 
cachedTableSchema);
+      }
+      if (targetInstant.isPresent()) {
+        tableSchemaCache.get().putIfAbsent(targetInstant.get(), 
cachedTableSchema);
+      }
+    }
+
+    // Finally process the computation results and return.
+    if (cachedTableSchema == null) {
+      return Option.empty();
+    }
+
+    return Option.of(cachedTableSchema);
+  }
+
+  @VisibleForTesting
+  Option<Pair<HoodieInstant, Schema>> 
getLastCommitMetadataWithValidSchemaFromTimeline(Stream<HoodieInstant> 
reversedTimelineStream, Option<HoodieInstant> instant) {
+    // To find the table schema given an instant time, need to walk backwards 
from the latest instant in
+    // the timeline finding a completed instant containing a valid schema.
+    ConcurrentHashMap<HoodieInstant, Schema> tableSchemaAtInstant = new 
ConcurrentHashMap<>();
+    Option<HoodieInstant> instantWithTableSchema = 
Option.fromJavaOptional(reversedTimelineStream
+        // If a completion time is specified, find the first eligible instant 
in the schema evolution timeline.
+        // Should switch to completion time based.
+        .filter(s -> instant.isEmpty() || compareTimestamps(s.requestedTime(), 
LESSER_THAN_OR_EQUALS, instant.get().requestedTime()))
+        // Make sure the commit metadata has a valid schema inside. Same 
caching the result for expensive operation.
+        .filter(s -> {
+          try {
+            // If we processed the instant before, do not parse the commit 
metadata again.
+            if (tableSchemaCache.get().containsKey(s)) {
+              tableSchemaAtInstant.putIfAbsent(s, 
tableSchemaCache.get().get(s));
+              return true;
+            }
+            HoodieCommitMetadata metadata = 
metaClient.getActiveTimeline().readCommitMetadata(s);
+            String schemaStr = 
metadata.getMetadata(HoodieCommitMetadata.SCHEMA_KEY);
+            boolean validSchemaStr = !StringUtils.isNullOrEmpty(schemaStr);

Review Comment:
   ```suggestion
               boolean isValidSchemaStr = !StringUtils.isNullOrEmpty(schemaStr);
   ```



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConcurrentSchemaEvolutionTableSchemaGetter.java:
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.transaction;
+
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.HoodieSchemaNotFoundException;
+import org.apache.hudi.common.model.HoodieCommitMetadata;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.TimelineLayout;
+import org.apache.hudi.common.util.ClusteringUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.VisibleForTesting;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.internal.schema.HoodieSchemaException;
+import org.apache.hudi.util.Lazy;
+
+import org.apache.avro.JsonProperties;
+import org.apache.avro.Schema;
+import org.apache.avro.Schema.Field;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Supplier;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.avro.AvroSchemaUtils.appendFieldsToSchema;
+import static org.apache.hudi.avro.AvroSchemaUtils.containsFieldInSchema;
+import static org.apache.hudi.avro.AvroSchemaUtils.createNullableSchema;
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION;
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.DELTA_COMMIT_ACTION;
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION;
+import static 
org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN_OR_EQUALS;
+import static 
org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps;
+
+/**
+ * Helper class to read table schema. ONLY USE IT FOR 
SimpleConcurrentFileWritesConflictResolutionStrategy.
+ */
+class ConcurrentSchemaEvolutionTableSchemaGetter {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ConcurrentSchemaEvolutionTableSchemaGetter.class);
+
+  protected final HoodieTableMetaClient metaClient;
+
+  private final Lazy<ConcurrentHashMap<HoodieInstant, Schema>> 
tableSchemaCache;
+
+  private Option<HoodieInstant> latestCommitWithValidSchema = Option.empty();
+
+  @VisibleForTesting
+  public ConcurrentHashMap<HoodieInstant, Schema> getTableSchemaCache() {
+    return tableSchemaCache.get();
+  }
+
+  public ConcurrentSchemaEvolutionTableSchemaGetter(HoodieTableMetaClient 
metaClient) {
+    this.metaClient = metaClient;
+    // Unbounded sized map. Should replace with some caching library.
+    this.tableSchemaCache = Lazy.lazily(ConcurrentHashMap::new);
+  }
+
+  /**
+   * Handles partition column logic for a given schema.
+   *
+   * @param schema the input schema to process
+   * @return the processed schema with partition columns handled appropriately
+   */
+  private Schema handlePartitionColumnsIfNeeded(Schema schema) {
+    if (metaClient.getTableConfig().shouldDropPartitionColumns()) {
+      return metaClient.getTableConfig().getPartitionFields()
+          .map(partitionFields -> appendPartitionColumns(schema, 
Option.ofNullable(partitionFields)))
+          .or(() -> Option.of(schema))
+          .get();
+    }
+    return schema;
+  }
+
+  public Option<Schema> getTableAvroSchemaIfPresent(boolean 
includeMetadataFields, Option<HoodieInstant> instant) {
+    return getTableAvroSchemaFromTimelineWithCache(instant) // Get table 
schema from schema evolution timeline.
+        .or(this::getTableCreateSchemaWithoutMetaField) // Fall back: read 
create schema from table config.
+        .map(tableSchema -> includeMetadataFields ? 
HoodieAvroUtils.addMetadataFields(tableSchema, false) : 
HoodieAvroUtils.removeMetadataFields(tableSchema))
+        .map(this::handlePartitionColumnsIfNeeded);
+  }
+
+  private Option<Schema> getTableCreateSchemaWithoutMetaField() {
+    return metaClient.getTableConfig().getTableCreateSchema();
+  }
+
+  private void setCachedLatestCommitWithValidSchema(Option<HoodieInstant> 
instantOption) {
+    latestCommitWithValidSchema = instantOption;
+  }
+
+  private Option<HoodieInstant> getCachedLatestCommitWithValidSchema() {
+    return latestCommitWithValidSchema;
+  }
+
+  @VisibleForTesting
+  Option<Schema> getTableAvroSchemaFromTimelineWithCache(Option<HoodieInstant> 
instantTime) {
+    return 
getTableAvroSchemaFromTimelineWithCache(computeSchemaEvolutionTimelineInReverseOrder(),
 instantTime);
+  }
+
+  Option<Schema> getTableAvroSchemaFromTimelineWithCache(Stream<HoodieInstant> 
reversedTimelineStream, Option<HoodieInstant> instantTime) {

Review Comment:
   This method is a bit complex, having a few `if` checks. Wondering if it can 
be simplified.



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConcurrentSchemaEvolutionTableSchemaGetter.java:
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.transaction;
+
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.common.HoodieSchemaNotFoundException;
+import org.apache.hudi.common.model.HoodieCommitMetadata;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.table.timeline.TimelineLayout;
+import org.apache.hudi.common.util.ClusteringUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.VisibleForTesting;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.internal.schema.HoodieSchemaException;
+import org.apache.hudi.util.Lazy;
+
+import org.apache.avro.JsonProperties;
+import org.apache.avro.Schema;
+import org.apache.avro.Schema.Field;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.function.Supplier;
+import java.util.stream.Stream;
+
+import static org.apache.hudi.avro.AvroSchemaUtils.appendFieldsToSchema;
+import static org.apache.hudi.avro.AvroSchemaUtils.containsFieldInSchema;
+import static org.apache.hudi.avro.AvroSchemaUtils.createNullableSchema;
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION;
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.DELTA_COMMIT_ACTION;
+import static 
org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION;
+import static 
org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN_OR_EQUALS;
+import static 
org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps;
+
+/**
+ * Helper class to read table schema. ONLY USE IT FOR 
SimpleConcurrentFileWritesConflictResolutionStrategy.
+ */
+class ConcurrentSchemaEvolutionTableSchemaGetter {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(ConcurrentSchemaEvolutionTableSchemaGetter.class);
+
+  protected final HoodieTableMetaClient metaClient;
+
+  private final Lazy<ConcurrentHashMap<HoodieInstant, Schema>> 
tableSchemaCache;
+
+  private Option<HoodieInstant> latestCommitWithValidSchema = Option.empty();
+
+  @VisibleForTesting
+  public ConcurrentHashMap<HoodieInstant, Schema> getTableSchemaCache() {
+    return tableSchemaCache.get();
+  }
+
+  public ConcurrentSchemaEvolutionTableSchemaGetter(HoodieTableMetaClient 
metaClient) {
+    this.metaClient = metaClient;
+    // Unbounded sized map. Should replace with some caching library.
+    this.tableSchemaCache = Lazy.lazily(ConcurrentHashMap::new);
+  }
+
+  /**
+   * Handles partition column logic for a given schema.
+   *
+   * @param schema the input schema to process
+   * @return the processed schema with partition columns handled appropriately
+   */
+  private Schema handlePartitionColumnsIfNeeded(Schema schema) {
+    if (metaClient.getTableConfig().shouldDropPartitionColumns()) {
+      return metaClient.getTableConfig().getPartitionFields()
+          .map(partitionFields -> appendPartitionColumns(schema, 
Option.ofNullable(partitionFields)))
+          .or(() -> Option.of(schema))
+          .get();
+    }
+    return schema;
+  }
+
+  public Option<Schema> getTableAvroSchemaIfPresent(boolean 
includeMetadataFields, Option<HoodieInstant> instant) {
+    return getTableAvroSchemaFromTimelineWithCache(instant) // Get table 
schema from schema evolution timeline.
+        .or(this::getTableCreateSchemaWithoutMetaField) // Fall back: read 
create schema from table config.
+        .map(tableSchema -> includeMetadataFields ? 
HoodieAvroUtils.addMetadataFields(tableSchema, false) : 
HoodieAvroUtils.removeMetadataFields(tableSchema))
+        .map(this::handlePartitionColumnsIfNeeded);
+  }
+
+  private Option<Schema> getTableCreateSchemaWithoutMetaField() {
+    return metaClient.getTableConfig().getTableCreateSchema();
+  }
+
+  private void setCachedLatestCommitWithValidSchema(Option<HoodieInstant> 
instantOption) {
+    latestCommitWithValidSchema = instantOption;
+  }
+
+  private Option<HoodieInstant> getCachedLatestCommitWithValidSchema() {
+    return latestCommitWithValidSchema;
+  }
+
+  @VisibleForTesting
+  Option<Schema> getTableAvroSchemaFromTimelineWithCache(Option<HoodieInstant> 
instantTime) {
+    return 
getTableAvroSchemaFromTimelineWithCache(computeSchemaEvolutionTimelineInReverseOrder(),
 instantTime);
+  }
+
+  Option<Schema> getTableAvroSchemaFromTimelineWithCache(Stream<HoodieInstant> 
reversedTimelineStream, Option<HoodieInstant> instantTime) {
+    // If instantTime is empty it means read the latest one. In that case, get 
the cached instant if there is one.
+    boolean fetchFromLastValidCommit = instantTime.isEmpty();
+    Option<HoodieInstant> targetInstant = 
instantTime.or(getCachedLatestCommitWithValidSchema());
+    Schema cachedTableSchema = null;
+
+    // Try cache first if there is a target instant to fetch for.
+    if (!targetInstant.isEmpty()) {
+      cachedTableSchema = 
tableSchemaCache.get().getOrDefault(targetInstant.get(), null);
+    }
+
+    // Cache miss on either latestCommitWithValidSchema or 
commitMetadataCache. Compute the result.
+    if (cachedTableSchema == null) {
+      Option<Pair<HoodieInstant, Schema>> instantWithSchema = 
getLastCommitMetadataWithValidSchemaFromTimeline(reversedTimelineStream, 
targetInstant);
+      if (instantWithSchema.isPresent()) {
+        targetInstant = Option.of(instantWithSchema.get().getLeft());
+        cachedTableSchema = instantWithSchema.get().getRight();
+      }
+    }
+
+    // After computation, update the cache for the instant and commit metadata.
+    if (fetchFromLastValidCommit) {
+      setCachedLatestCommitWithValidSchema(targetInstant);
+    }
+    if (cachedTableSchema != null) {
+      // We save cache for 2 cases
+      // - input specifies a specific instant, we update the cache for that 
instant, which is instantTime.
+      // - input is empty implying fetch the latest schema, update the cache 
for the
+      //   latest valid commit which is targetInstant
+      if (instantTime.isPresent()) {
+        tableSchemaCache.get().putIfAbsent(instantTime.get(), 
cachedTableSchema);
+      }
+      if (targetInstant.isPresent()) {
+        tableSchemaCache.get().putIfAbsent(targetInstant.get(), 
cachedTableSchema);
+      }
+    }
+
+    // Finally process the computation results and return.
+    if (cachedTableSchema == null) {
+      return Option.empty();
+    }
+
+    return Option.of(cachedTableSchema);
+  }
+
+  @VisibleForTesting
+  Option<Pair<HoodieInstant, Schema>> 
getLastCommitMetadataWithValidSchemaFromTimeline(Stream<HoodieInstant> 
reversedTimelineStream, Option<HoodieInstant> instant) {
+    // To find the table schema given an instant time, need to walk backwards 
from the latest instant in
+    // the timeline finding a completed instant containing a valid schema.
+    ConcurrentHashMap<HoodieInstant, Schema> tableSchemaAtInstant = new 
ConcurrentHashMap<>();
+    Option<HoodieInstant> instantWithTableSchema = 
Option.fromJavaOptional(reversedTimelineStream
+        // If a completion time is specified, find the first eligible instant 
in the schema evolution timeline.
+        // Should switch to completion time based.
+        .filter(s -> instant.isEmpty() || compareTimestamps(s.requestedTime(), 
LESSER_THAN_OR_EQUALS, instant.get().requestedTime()))

Review Comment:
   Does the instant stream passed in contain completed instants only?



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