danny0405 commented on code in PR #12967: URL: https://github.com/apache/hudi/pull/12967#discussion_r2002764366
########## hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/v2/RowDataLogWriteHandle.java: ########## @@ -0,0 +1,366 @@ +/* + * 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.v2; + +import org.apache.hudi.avro.AvroSchemaCache; +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.config.HoodieStorageConfig; +import org.apache.hudi.common.engine.TaskContextSupplier; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.DeleteRecord; +import org.apache.hudi.common.model.HoodieColumnRangeMetadata; +import org.apache.hudi.common.model.HoodieDeltaWriteStat; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieOperation; +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.model.IOType; +import org.apache.hudi.common.model.MetadataValues; +import org.apache.hudi.common.table.log.AppendResult; +import org.apache.hudi.common.table.log.HoodieLogFormat; +import org.apache.hudi.common.table.log.block.HoodieDeleteBlock; +import org.apache.hudi.common.table.log.block.HoodieLogBlock; +import org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieAppendException; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.io.HoodieWriteHandle; +import org.apache.hudi.io.MiniBatchHandle; +import org.apache.hudi.io.log.block.HoodieFlinkParquetDataBlock; +import org.apache.hudi.io.storage.ColumnRangeMetadataProvider; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.storage.StorageConfiguration; +import org.apache.hudi.storage.StoragePath; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.util.Lazy; + +import org.apache.avro.Schema; +import org.apache.flink.util.Preconditions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; + +import static org.apache.hudi.common.config.HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME; +import static org.apache.hudi.common.config.HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION; +import static org.apache.hudi.common.config.HoodieStorageConfig.PARQUET_DICTIONARY_ENABLED; +import static org.apache.hudi.common.model.HoodieRecordLocation.INVALID_POSITION; + +/** + * A write handle that supports creating a log file and writing records based on record Iterator. + * The differences from {@code FlinkAppendHandle} are: + * + * <p> 1. {@code RowDataLogHandle} does not convert RowData into Avro record before writing. + * <p> 2. {@code RowDataLogHandle} writes Parquet data block by default. + * <p> 3. {@code RowDataLogHandle} does not buffer data internally, instead, it employs + * record iterator to write data blocks, thereby enhancing memory efficiency. + * + * <p>The back-up writer may roll over to a new log file if there already exists a log file for the + * given file group and instant. + */ +public class RowDataLogWriteHandle<T, I, K, O> extends HoodieWriteHandle<T, I, K, O> implements MiniBatchHandle { + private static final Logger LOG = LoggerFactory.getLogger(RowDataLogWriteHandle.class); + + private HoodieLogFormat.Writer writer; + // Header metadata for a log block + protected final Map<HoodieLogBlock.HeaderMetadataType, String> header = new HashMap<>(); + protected final Schema logWriteSchema; + // Total number of records written during appending + protected long recordsWritten = 0; + // Total number of records deleted during appending + protected long recordsDeleted = 0; + // Total number of records updated during appending + protected long updatedRecordsWritten = 0; + // Total number of new records inserted into the delta file + protected long insertRecordsWritten = 0; + private boolean isClosed = false; + + private static final AtomicLong RECORD_COUNTER = new AtomicLong(1); + + public RowDataLogWriteHandle( + HoodieWriteConfig config, + String instantTime, + HoodieTable<T, I, K, O> hoodieTable, + String fileId, + String partitionPath, + TaskContextSupplier taskContextSupplier) { + super(config, instantTime, partitionPath, fileId, hoodieTable, taskContextSupplier); + initWriteConf(storage.getConf(), config); + this.writer = createLogWriter(this.instantTime, null); + this.logWriteSchema = AvroSchemaCache.intern( + HoodieAvroUtils.addMetadataFields(writeSchema, config.populateMetaFields(), config.allowOperationMetadataField())); + } + + /** + * Append data and delete blocks into log file. + */ + public WriteStatus appendRowData(Iterator<HoodieRecord> records) { + initPartitionMeta(); + initWriteStatus(); + try { + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, instantTime); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, logWriteSchema.toString()); + List<HoodieLogBlock> blocks = new ArrayList<>(2); + List<Pair<DeleteRecord, Long>> recordsToDelete = new ArrayList<>(); + // todo do not need list if there is no delete records, HUDI-9195 + List<HoodieRecord> recordList = new ArrayList<>(); + + boolean needPrependMetaFields = config.populateMetaFields() || config.allowOperationMetadataField(); + while (records.hasNext()) { + HoodieRecord record = records.next(); + if (HoodieOperation.isDelete(record.getOperation()) && !config.allowOperationMetadataField()) { + DeleteRecord deleteRecord = DeleteRecord.create(record.getKey(), record.getOrderingValue(writeSchema, config.getProps())); + recordsToDelete.add(Pair.of(deleteRecord, INVALID_POSITION)); + } else { + record = needPrependMetaFields + ? record.prependMetaFields(writeSchema, logWriteSchema, populateMetadataFields(record), config.getProps()) + : record; + recordList.add(record); + } + } + + // add data block + HoodieLogBlock dataBlock = null; + if (!recordList.isEmpty()) { + String keyField = config.populateMetaFields() + ? HoodieRecord.RECORD_KEY_METADATA_FIELD + : hoodieTable.getMetaClient().getTableConfig().getRecordKeyFieldProp(); + dataBlock = genDataBlock(config, getLogDataBlockFormat(), recordList, keyField, header); + blocks.add(dataBlock); + } + + // add delete block + if (!recordsToDelete.isEmpty()) { + blocks.add(new HoodieDeleteBlock(recordsToDelete, header)); + } + + if (!blocks.isEmpty()) { + AppendResult appendResult = writer.appendBlocks(blocks); + processAppendResult(appendResult, dataBlock); + } + } catch (Exception e) { + throw new HoodieAppendException("Failed while appending records to " + writer.getLogFile().getPath(), e); + } + return writeStatus; + } + + public void initWriteStatus() { + HoodieDeltaWriteStat deltaWriteStat = new HoodieDeltaWriteStat(); + deltaWriteStat.setPartitionPath(partitionPath); + deltaWriteStat.setFileId(fileId); + this.writeStatus.setFileId(fileId); + this.writeStatus.setPartitionPath(partitionPath); + this.writeStatus.setStat(deltaWriteStat); + } + + /** + * Save hoodie partition meta in the partition path. + */ + protected void initPartitionMeta() { + HoodiePartitionMetadata partitionMetadata = new HoodiePartitionMetadata(storage, instantTime, + new StoragePath(config.getBasePath()), + FSUtils.constructAbsolutePath(config.getBasePath(), partitionPath), + hoodieTable.getPartitionMetafileFormat()); + partitionMetadata.trySave(); + } + + private void initWriteConf(StorageConfiguration<?> storageConf, HoodieWriteConfig writeConfig) { + storageConf.set( + HoodieStorageConfig.PARQUET_WRITE_UTC_TIMEZONE.key(), + writeConfig.getString(HoodieStorageConfig.PARQUET_WRITE_UTC_TIMEZONE.key())); + } + + private MetadataValues populateMetadataFields(HoodieRecord<T> hoodieRecord) { + MetadataValues metadataValues = new MetadataValues(); + if (config.populateMetaFields()) { + String seqId = + HoodieRecord.generateSequenceId(instantTime, getPartitionId(), RECORD_COUNTER.getAndIncrement()); + metadataValues.setFileName(fileId); + metadataValues.setPartitionPath(partitionPath); + metadataValues.setRecordKey(hoodieRecord.getRecordKey()); + metadataValues.setCommitTime(instantTime); + metadataValues.setCommitSeqno(seqId); + } + if (config.allowOperationMetadataField()) { + metadataValues.setOperation(hoodieRecord.getOperation().getName()); + } + return metadataValues; + } + + private void processAppendResult(AppendResult result, HoodieLogBlock dataBlock) { + HoodieDeltaWriteStat stat = (HoodieDeltaWriteStat) this.writeStatus.getStat(); + + Preconditions.checkArgument(stat.getPath() == null, "Only one append are expected for " + this.getClass().getSimpleName()); + // first time writing to this log block. + updateWriteStatus(stat, result); + + // for parquet data block, we can get column stats from parquet footer directly. + if (config.isMetadataColumnStatsIndexEnabled()) { + Set<String> columnsToIndexSet = new HashSet<>(HoodieTableMetadataUtil + .getColumnsToIndex(hoodieTable.getMetaClient().getTableConfig(), + config.getMetadataConfig(), Lazy.eagerly(Option.of(writeSchemaWithMetaFields)), + Option.of(HoodieRecord.HoodieRecordType.FLINK)).keySet()); + + Map<String, HoodieColumnRangeMetadata<Comparable>> columnRangeMetadata; + if (dataBlock == null) { + // only delete block exists + columnRangeMetadata = new HashMap<>(); + for (String col: columnsToIndexSet) { + columnRangeMetadata.put(col, HoodieColumnRangeMetadata.create( + stat.getPath(), col, null, null, 0L, 0L, 0L, 0L)); + } + } else { + ValidationUtils.checkArgument(dataBlock instanceof ColumnRangeMetadataProvider, + "Log block for Flink ingestion should always be an instance of ColumnRangeMetadataProvider for collecting column stats efficiently."); + columnRangeMetadata = + ((ColumnRangeMetadataProvider) dataBlock).getColumnRangeMeta().entrySet().stream() + .filter(e -> columnsToIndexSet.contains(e.getKey())) + .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().copy(stat.getPath()))); Review Comment: Why we must copy the file path? -- 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]
