yihua commented on code in PR #12390: URL: https://github.com/apache/hudi/pull/12390#discussion_r1866827932
########## hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/HoodieSparkMergeHandleV2.java: ########## @@ -0,0 +1,410 @@ +/* + * 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.io; + +import org.apache.hudi.AvroConversionUtils; +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.engine.HoodieReaderContext; +import org.apache.hudi.common.engine.TaskContextSupplier; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.CompactionOperation; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodiePartitionMetadata; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieSparkRecord; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats; +import org.apache.hudi.common.model.IOType; +import org.apache.hudi.common.model.MetadataValues; +import org.apache.hudi.common.table.read.HoodieFileGroupReader; +import org.apache.hudi.common.table.read.HoodieFileGroupReader.HoodieFileGroupReaderIterator; +import org.apache.hudi.common.table.read.HoodieFileGroupReaderStats; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieCorruptedDataException; +import org.apache.hudi.exception.HoodieUpsertException; +import org.apache.hudi.internal.schema.InternalSchema; +import org.apache.hudi.internal.schema.utils.SerDeHelper; +import org.apache.hudi.io.storage.HoodieFileReader; +import org.apache.hudi.io.storage.HoodieFileWriter; +import org.apache.hudi.io.storage.HoodieFileWriterFactory; +import org.apache.hudi.io.storage.HoodieIOFactory; +import org.apache.hudi.keygen.BaseKeyGenerator; +import org.apache.hudi.storage.StoragePath; +import org.apache.hudi.table.HoodieTable; + +import org.apache.avro.Schema; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.types.StructType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.concurrent.NotThreadSafe; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Properties; +import java.util.Set; +import java.util.stream.Collectors; + +import static org.apache.hudi.common.config.HoodieReaderConfig.MERGE_USE_RECORD_POSITIONS; + +@SuppressWarnings("Duplicates") +/** + * Handle to merge incoming records to those in storage. + * <p> + * Simplified Logic: + * For every existing record + * Check if there is a new record coming in. If yes, merge two records and write to file + * else write the record as is + * For all pending records from incoming batch, write to file. + * + * Illustration with simple data. + * Incoming data: + * rec1_2, rec4_2, rec5_1, rec6_1 + * Existing data: + * rec1_1, rec2_1, rec3_1, rec4_1 + * + * For every existing record, merge w/ incoming if required and write to storage. + * => rec1_1 and rec1_2 is merged to write rec1_2 to storage + * => rec2_1 is written as is + * => rec3_1 is written as is + * => rec4_2 and rec4_1 is merged to write rec4_2 to storage + * Write all pending records from incoming set to storage + * => rec5_1 and rec6_1 + * + * Final snapshot in storage + * rec1_2, rec2_1, rec3_1, rec4_2, rec5_1, rec6_1 + * + * </p> + */ +@NotThreadSafe +public class HoodieSparkMergeHandleV2<T, I, K, O> extends HoodieWriteHandle<T, I, K, O> { + + private static final Logger LOG = LoggerFactory.getLogger(HoodieMergeHandle.class); + + protected Set<String> writtenRecordKeys; + protected HoodieReaderContext readerContext; + protected HoodieFileWriter fileWriter; + protected boolean preserveMetadata = false; + + protected StoragePath newFilePath; + protected StoragePath oldFilePath; + protected long recordsWritten = 0; + // TODO(yihua): audit delete stats because file group reader may not return deletes + protected long recordsDeleted = 0; + protected long updatedRecordsWritten = 0; + protected long insertRecordsWritten = 0; + protected Option<BaseKeyGenerator> keyGeneratorOpt; + protected FileSlice fileSlice; + private HoodieBaseFile baseFileToMerge; + + protected Option<String[]> partitionFields = Option.empty(); + protected Object[] partitionValues = new Object[0]; + + /** + * Called by compactor code path using the file group reader. + */ + public HoodieSparkMergeHandleV2(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable, + String partitionPath, String fileId, + CompactionOperation operation, TaskContextSupplier taskContextSupplier, Option<BaseKeyGenerator> keyGeneratorOpt, + HoodieReaderContext readerContext) { + super(config, instantTime, partitionPath, fileId, hoodieTable, taskContextSupplier); + this.readerContext = readerContext; + Option<HoodieBaseFile> baseFileOpt = + operation.getBaseFile(config.getBasePath(), operation.getPartitionPath()); + List<HoodieLogFile> logFiles = operation.getDeltaFileNames().stream().map(p -> + new HoodieLogFile(new StoragePath(FSUtils.constructAbsolutePath( + config.getBasePath(), operation.getPartitionPath()), p))) + .collect(Collectors.toList()); + this.fileSlice = new FileSlice( + operation.getFileGroupId(), + operation.getBaseInstantTime(), + baseFileOpt.isPresent() ? baseFileOpt.get() : null, + logFiles); + ValidationUtils.checkArgument(baseFileOpt.isPresent(), Review Comment: While I was working on the PoC, this is a reminder for myself to implement compaction for log files only (no base file), which should go through `HoodieSparkMergeHandleV2`. The old code path uses `HoodieCreateHandle` to handle log only merging (i.e., `HoodieCompactionHandler#handleInsert`). I've added the compaction support based on the file group reader for log files only (no base file) and removed the check. -- 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]
