nsivabalan commented on code in PR #13411: URL: https://github.com/apache/hudi/pull/13411#discussion_r2151229813
########## hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/FileGroupReaderBasedAppendHandle.java: ########## @@ -0,0 +1,120 @@ +/* + * 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.client.WriteStatus; +import org.apache.hudi.common.config.HoodieMemoryConfig; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.engine.HoodieReaderContext; +import org.apache.hudi.common.engine.TaskContextSupplier; +import org.apache.hudi.common.model.CompactionOperation; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.table.read.HoodieFileGroupReader; +import org.apache.hudi.common.table.read.HoodieReadStats; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIOException; +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.table.HoodieTable; +import org.apache.hudi.table.action.compact.strategy.CompactionStrategy; + +import javax.annotation.concurrent.NotThreadSafe; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; + +import static org.apache.hudi.common.config.HoodieReaderConfig.MERGE_USE_RECORD_POSITIONS; + +/** + * A base append handle implementation based on the {@link HoodieFileGroupReader}. + * <p> + * This append-handle is used for log-compaction, which passes a file slice from the + * compaction operation of a single file group to a file group reader, get an iterator of + * the records, and writes the records to a new log file. + */ +@NotThreadSafe +public class FileGroupReaderBasedAppendHandle<T, I, K, O> extends HoodieAppendHandle<T, I, K, O> { + private final HoodieReaderContext<T> readerContext; + private final FileSlice fileSlice; + private final CompactionOperation operation; + private HoodieReadStats readStats; + + public FileGroupReaderBasedAppendHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable, + FileSlice fileSlice, CompactionOperation operation, TaskContextSupplier taskContextSupplier, + HoodieReaderContext<T> readerContext) { + super(config, instantTime, hoodieTable, operation.getPartitionPath(), operation.getFileId(), taskContextSupplier); + this.operation = operation; + this.readerContext = readerContext; + this.fileSlice = fileSlice; + } + + @Override + public void doAppend() { + boolean usePosition = config.getBooleanOrDefault(MERGE_USE_RECORD_POSITIONS); + Option<InternalSchema> internalSchemaOption = SerDeHelper.fromJson(config.getInternalSchema()); + TypedProperties props = new TypedProperties(); + hoodieTable.getMetaClient().getTableConfig().getProps().forEach(props::putIfAbsent); Review Comment: not a blocker. if we see this pattern repeating in many places where we wanted to construct a new TypedProperties from HoodieWriteconfig and tableConfig, we can make a common utility method. ########## hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecord.java: ########## @@ -168,6 +182,8 @@ public boolean isDelete(Schema recordSchema, Properties props) throws IOExceptio } if (this.data instanceof BaseAvroPayload) { return ((BaseAvroPayload) this.data).isDeleted(recordSchema, props); + } else if (this.data instanceof HoodieMetadataPayload) { Review Comment: I feel we should not leak the payload into to this class. as of now, we only use `BaseAvroPayload` and no other payloads in here. I checked the impl of ``` public boolean isDeleted() { return isDeletedRecord; } ``` in HoodieMetadataPayload. and wondering why can't we just impl the same method. ``` public boolean isDeleted(Schema schema, Properties props) { return isDeletedRecord; } ``` and it should work right? ########## hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/FileGroupReaderBasedMergeHandle.java: ########## @@ -173,6 +216,7 @@ public List<WriteStatus> close() { writeStatus.getStat().setTotalLogBlocks(readStats.getTotalLogBlocks()); writeStatus.getStat().setTotalCorruptLogBlock(readStats.getTotalCorruptLogBlock()); writeStatus.getStat().setTotalRollbackBlocks(readStats.getTotalRollbackBlocks()); + writeStatus.getStat().setTotalLogSizeCompacted(operation.getMetrics().get(CompactionStrategy.TOTAL_LOG_FILE_SIZE).longValue()); Review Comment: where are we setting the totalUpsertTime w/n RuntimeStats? ########## hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/FileGroupReaderBasedMergeHandle.java: ########## @@ -139,23 +137,68 @@ private void init(CompactionOperation operation, String partitionPath, Option<Ho // Create the writer for writing the new version file fileWriter = HoodieFileWriterFactory.getFileWriter(instantTime, newFilePath, hoodieTable.getStorage(), - config, writeSchemaWithMetaFields, taskContextSupplier, getRecordType()); + config, writeSchemaWithMetaFields, taskContextSupplier, recordType); } catch (IOException io) { - LOG.error("Error in update task at commit " + instantTime, io); + LOG.error("Error in update task at commit {}", instantTime, io); writeStatus.setGlobalError(io); throw new HoodieUpsertException("Failed to initialize HoodieUpdateHandle for FileId: " + fileId + " on commit " + instantTime + " on path " + hoodieTable.getMetaClient().getBasePath(), io); } } - protected abstract HoodieRecord.HoodieRecordType getRecordType(); - /** * Reads the file slice of a compaction operation using a file group reader, * by getting an iterator of the records; then writes the records to a new base file * using parquet writer. */ - protected abstract void write(); + public void write() { Review Comment: minor. lets fix the java docs for this method. it talks about parquet writer. but in case of metadata table, its not parquet. ########## hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/FileGroupReaderBasedMergeHandle.java: ########## @@ -20,81 +20,80 @@ package org.apache.hudi.io; import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.config.HoodieMemoryConfig; +import org.apache.hudi.common.config.TypedProperties; 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.HoodieLogFile; import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.read.HoodieFileGroupReader; import org.apache.hudi.common.table.read.HoodieReadStats; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.ClosableIterator; import org.apache.hudi.config.HoodieWriteConfig; 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.HoodieFileWriterFactory; -import org.apache.hudi.keygen.BaseKeyGenerator; import org.apache.hudi.storage.StoragePath; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.compact.strategy.CompactionStrategy; 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.List; -import java.util.stream.Collectors; +import java.util.Map; + +import static org.apache.hudi.common.config.HoodieReaderConfig.MERGE_USE_RECORD_POSITIONS; +import static org.apache.hudi.common.model.HoodieFileFormat.HFILE; /** - * A base merge handle implementation based on the {@link HoodieFileGroupReader}. + * A merge handle implementation based on the {@link HoodieFileGroupReader}. * <p> * This merge handle is used for compaction, which passes a file slice from the * compaction operation of a single file group to a file group reader, get an iterator of * the records, and writes the records to a new base file. */ -public abstract class BaseFileGroupReaderBasedMergeHandle<T, I, K, O> extends HoodieMergeHandle<T, I, K, O> { - private static final Logger LOG = LoggerFactory.getLogger(BaseFileGroupReaderBasedMergeHandle.class); - - protected HoodieReaderContext<T> readerContext; - protected FileSlice fileSlice; - protected HoodieReadStats readStats; - - public BaseFileGroupReaderBasedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable, - CompactionOperation operation, TaskContextSupplier taskContextSupplier, - Option<BaseKeyGenerator> keyGeneratorOpt, - HoodieReaderContext<T> readerContext) { +@NotThreadSafe +public class FileGroupReaderBasedMergeHandle<T, I, K, O> extends HoodieMergeHandle<T, I, K, O> { + private static final Logger LOG = LoggerFactory.getLogger(FileGroupReaderBasedMergeHandle.class); + + private final HoodieReaderContext<T> readerContext; + private final FileSlice fileSlice; + private final CompactionOperation operation; + private final String maxInstantTime; + private HoodieReadStats readStats; + private final HoodieRecord.HoodieRecordType recordType; + + public FileGroupReaderBasedMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable, + FileSlice fileSlice, CompactionOperation operation, TaskContextSupplier taskContextSupplier, + HoodieReaderContext<T> readerContext, String maxInstantTime, + HoodieRecord.HoodieRecordType enginRecordType) { super(config, instantTime, operation.getPartitionPath(), operation.getFileId(), hoodieTable, taskContextSupplier); + this.maxInstantTime = maxInstantTime; this.keyToNewRecords = Collections.emptyMap(); 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); + this.fileSlice = fileSlice; + this.operation = operation; this.preserveMetadata = true; - init(operation, this.partitionPath, baseFileOpt); - validateAndSetAndKeyGenProps(keyGeneratorOpt, config.populateMetaFields()); - } - - private void validateAndSetAndKeyGenProps(Option<BaseKeyGenerator> keyGeneratorOpt, boolean populateMetaFields) { - ValidationUtils.checkArgument(populateMetaFields == !keyGeneratorOpt.isPresent()); - this.keyGeneratorOpt = keyGeneratorOpt; + // If the table is a metadata table or the base file is an HFile, we use AVRO record type, otherwise we use the engine record type. + this.recordType = hoodieTable.isMetadataTable() || HFILE.getFileExtension().equals(hoodieTable.getBaseFileExtension()) ? HoodieRecord.HoodieRecordType.AVRO : enginRecordType; Review Comment: instead of fixing this here, shouldn't we fix HoodieCompactor.getEngineRecordType() to return the right value. we might have to rename the method though if we fix it -- 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]
