yihua commented on code in PR #12781: URL: https://github.com/apache/hudi/pull/12781#discussion_r1980124747
########## hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/SimpleSchemaConflictResolutionStrategy.java: ########## @@ -0,0 +1,178 @@ +/* + * 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.avro.Schema; +import org.apache.hudi.avro.AvroSchemaComparatorForSchemaEvolution; +import org.apache.hudi.common.table.timeline.HoodieInstant; +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.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.table.HoodieTable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.stream.Stream; + +import static org.apache.hudi.avro.HoodieAvroUtils.isSchemaNull; +import static org.apache.hudi.client.transaction.SchemaConflictResolutionStrategy.throwConcurrentSchemaEvolutionException; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMPACTION_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; + +/** + * The implementation of SchemaConflictResolutionStrategy that detects incompatible + * schema evolution from multiple writers + */ +public class SimpleSchemaConflictResolutionStrategy implements SchemaConflictResolutionStrategy { + private static final Logger LOG = LoggerFactory.getLogger(SimpleSchemaConflictResolutionStrategy.class); + + @Override + public Option<Schema> resolveConcurrentSchemaEvolution( + HoodieTable table, + HoodieWriteConfig config, + Option<HoodieInstant> lastCompletedTxnOwnerInstant, + Option<HoodieInstant> currTxnOwnerInstant) { + + // If this is compaction table service, skip schema evolution check as it does not evolve schema. + if (!currTxnOwnerInstant.isPresent() + || currTxnOwnerInstant.get().getAction().equals(COMPACTION_ACTION) + || (currTxnOwnerInstant.get().getAction().equals(REPLACE_COMMIT_ACTION) + && ClusteringUtils.isClusteringInstant(table.getMetaClient().getActiveTimeline(), currTxnOwnerInstant.get(), table.getMetaClient().getInstantGenerator()))) { + return Option.empty(); + } + + // Guard against unrecognized cases where writers do not come with a writer schema. + if (StringUtils.isNullOrEmpty(config.getWriteSchema())) { + LOG.warn(StringUtils.join("Writer config does not come with a valid writer schema. Writer config: ", + config.toString(), ". Owner instant: ", currTxnOwnerInstant.get().toString())); + return Option.empty(); + } + + Schema writerSchemaOfTxn = new Schema.Parser().parse(config.getWriteSchema()); + // If a writer does not come with a meaningful schema, skip the schema resolution. + if (isSchemaNull(writerSchemaOfTxn)) { + return getTableSchemaAtInstant(new ConcurrentSchemaEvolutionTableSchemaGetter(table.getMetaClient()), currTxnOwnerInstant.get()); + } + + // Fast path: We can tell there is no schema conflict by just comparing the instants without involving table/writer schema comparison. + ConcurrentSchemaEvolutionTableSchemaGetter schemaResolver = new ConcurrentSchemaEvolutionTableSchemaGetter(table.getMetaClient()); + + // schema and writer schema. + HoodieInstant lastCompletedInstantAtTxnStart = lastCompletedTxnOwnerInstant.isPresent() + ? getInstantInTimelineImmediatelyPriorToTimestamp( + lastCompletedTxnOwnerInstant.get().requestedTime(), schemaResolver.computeSchemaEvolutionTimelineInReverseOrder()).orElse(null) Review Comment: Should this use completion time now? ```suggestion lastCompletedTxnOwnerInstant.get().getCompletionTime(), schemaResolver.computeSchemaEvolutionTimelineInReverseOrder()).orElse(null) ``` -- 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]
