codope commented on code in PR #9624: URL: https://github.com/apache/hudi/pull/9624#discussion_r1342691263
########## hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/util/CloseableInternalRowIterator.scala: ########## @@ -0,0 +1,79 @@ +/* + * 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.util + +import org.apache.hudi.common.util.collection.ClosableIterator +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.vectorized.ColumnarBatch + +/** + * A [[ClosableIterator]] returning [[InternalRow]] by iterating through the entries returned + * by a Spark reader. + * + * @param iterator the input iterator that can either contain [[InternalRow]] (non-vectorized) + * or [[ColumnarBatch]] (vectorized), as returned by the Spark reader. + */ +class CloseableInternalRowIterator(iterator: Iterator[_]) extends ClosableIterator[InternalRow] { + private var entryTypeKnown = false + private var isColumnarBatch = false + private var nextBatch: ColumnarBatch = _ + private var seqInBatch: Int = -1 + + override def close(): Unit = { + // No op + } + + override def hasNext: Boolean = { + seqInBatch >= 0 || iterator.hasNext + } + + override def next: InternalRow = { + if (!entryTypeKnown) { + // First entry + val nextVal = iterator.next + seqInBatch = 0 + nextVal match { + case _: ColumnarBatch => + isColumnarBatch = true + nextBatch = nextVal.asInstanceOf[ColumnarBatch] + val result = nextBatch.getRow(seqInBatch) + seqInBatch += 1 + if (seqInBatch >= nextBatch.numRows()) { + seqInBatch = -1 + } + result + case _ => + seqInBatch = -1 + nextVal.asInstanceOf[InternalRow] + } Review Comment: set `entryTypeKnown=true`? I think this may not be needed. `isColumnarBatch` is sufficient for the functionality. ########## hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/util/CloseableInternalRowIterator.scala: ########## @@ -0,0 +1,79 @@ +/* + * 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.util + +import org.apache.hudi.common.util.collection.ClosableIterator +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.vectorized.ColumnarBatch + +/** + * A [[ClosableIterator]] returning [[InternalRow]] by iterating through the entries returned + * by a Spark reader. + * + * @param iterator the input iterator that can either contain [[InternalRow]] (non-vectorized) + * or [[ColumnarBatch]] (vectorized), as returned by the Spark reader. + */ +class CloseableInternalRowIterator(iterator: Iterator[_]) extends ClosableIterator[InternalRow] { + private var entryTypeKnown = false + private var isColumnarBatch = false + private var nextBatch: ColumnarBatch = _ + private var seqInBatch: Int = -1 + + override def close(): Unit = { + // No op + } + + override def hasNext: Boolean = { + seqInBatch >= 0 || iterator.hasNext + } + + override def next: InternalRow = { + if (!entryTypeKnown) { + // First entry + val nextVal = iterator.next + seqInBatch = 0 + nextVal match { Review Comment: minor optimization: we can handle the detection of the entry type (ColumnarBatch or InternalRow) in `hasNext` and remove the unnecessary type casting in the `next` method by ensuring that we only call next when we are sure that the next element is of the expected type. ########## hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java: ########## @@ -177,11 +178,24 @@ protected <T> ClosableIterator<HoodieRecord<T>> deserializeRecords(byte[] conten FileSystem fs = FSUtils.getFs(pathForReader.toString(), hadoopConf); // Read the content try (HoodieAvroHFileReader reader = new HoodieAvroHFileReader(hadoopConf, pathForReader, new CacheConfig(hadoopConf), - fs, content, Option.of(getSchemaFromHeader()))) { + fs, content, Option.of(getSchemaFromHeader()))) { return unsafeCast(reader.getRecordIterator(readerSchema)); } } + @Override + protected <T> ClosableIterator<T> deserializeRecords(HoodieReaderContext<T> readerContext, byte[] content) throws IOException { + checkState(readerSchema != null, "Reader's schema has to be non-null"); + + Configuration hadoopConf = FSUtils.buildInlineConf(getBlockContentLocation().get().getHadoopConf()); Review Comment: Is this required per block deserialization? ########## hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala: ########## @@ -0,0 +1,64 @@ +/* + * 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 + +import org.apache.avro.Schema +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hudi.common.engine.HoodieReaderContext +import org.apache.hudi.common.util.collection.ClosableIterator +import org.apache.hudi.util.CloseableInternalRowIterator +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.{HoodieInternalRowUtils, SparkSession} + +/** + * Implementation of {@link HoodieReaderContext} to read {@link InternalRow}s with + * {@link ParquetFileFormat} on Spark. + * + * This uses Spark parquet reader to read parquet data files or parquet log blocks. + * + * @param sparkSession {@link SparkSession} instance. + * @param parquetFileFormat {@link ParquetFileFormat} instance for parquet file format in Spark. + * @param hadoopConf Hadoop configuration. + */ +class SparkFileFormatInternalRowReaderContext(sparkSession: SparkSession, + parquetFileFormat: ParquetFileFormat, + hadoopConf: Configuration) extends BaseSparkInternalRowReaderContext { + lazy val sparkAdapter = SparkAdapterSupport.sparkAdapter + + override def getFileRecordIterator(filePath: Path, + start: Long, + length: Long, + dataSchema: Schema, + requiredSchema: Schema, + conf: Configuration): ClosableIterator[InternalRow] = { + val fileInfo = sparkAdapter.getSparkPartitionedFileUtils.createPartitionedFile( + InternalRow.empty, filePath, 0, length) Review Comment: maybe pass `start` instead of `0`? ########## hudi-common/src/main/java/org/apache/hudi/common/engine/HoodieReaderContext.java: ########## @@ -0,0 +1,168 @@ +/* + * 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.common.engine; + +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.ClosableIterator; + +import org.apache.avro.Schema; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import java.util.HashMap; +import java.util.Map; + +/** + * An abstract reader context class for {@code HoodieFileGroupReader} to use, containing APIs for + * engine-specific implementation on reading data files, getting field values from a record, + * transforming a record, etc. + * <p> + * For each query engine, this class should be extended and plugged into {@code HoodieFileGroupReader} + * to realize the file group reading. + * + * @param <T> The type of engine-specific record representation, e.g.,{@code InternalRow} in Spark + * and {@code RowData} in Flink. + */ +public abstract class HoodieReaderContext<T> { Review Comment: Why not make it an interface? ########## hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/util/CloseableInternalRowIterator.scala: ########## @@ -0,0 +1,79 @@ +/* + * 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.util + +import org.apache.hudi.common.util.collection.ClosableIterator +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.vectorized.ColumnarBatch + +/** + * A [[ClosableIterator]] returning [[InternalRow]] by iterating through the entries returned + * by a Spark reader. + * + * @param iterator the input iterator that can either contain [[InternalRow]] (non-vectorized) + * or [[ColumnarBatch]] (vectorized), as returned by the Spark reader. + */ +class CloseableInternalRowIterator(iterator: Iterator[_]) extends ClosableIterator[InternalRow] { + private var entryTypeKnown = false + private var isColumnarBatch = false + private var nextBatch: ColumnarBatch = _ + private var seqInBatch: Int = -1 + + override def close(): Unit = { + // No op Review Comment: close `nextBatch` here? ########## hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/util/CloseableInternalRowIterator.scala: ########## @@ -0,0 +1,79 @@ +/* + * 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.util + +import org.apache.hudi.common.util.collection.ClosableIterator +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.vectorized.ColumnarBatch + +/** + * A [[ClosableIterator]] returning [[InternalRow]] by iterating through the entries returned + * by a Spark reader. + * + * @param iterator the input iterator that can either contain [[InternalRow]] (non-vectorized) + * or [[ColumnarBatch]] (vectorized), as returned by the Spark reader. + */ +class CloseableInternalRowIterator(iterator: Iterator[_]) extends ClosableIterator[InternalRow] { + private var entryTypeKnown = false + private var isColumnarBatch = false + private var nextBatch: ColumnarBatch = _ + private var seqInBatch: Int = -1 + + override def close(): Unit = { + // No op + } + + override def hasNext: Boolean = { + seqInBatch >= 0 || iterator.hasNext + } + + override def next: InternalRow = { Review Comment: Should we validate something as below? ``` if (!hasNext) { throw new NoSuchElementException("No more elements to iterate") } ``` ########## hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieParquetDataBlock.java: ########## @@ -162,8 +163,33 @@ protected <T> ClosableIterator<HoodieRecord<T>> readRecordsFromBlockPayload(Hood return iterator; } + @Override + protected <T> ClosableIterator<T> readRecordsFromBlockPayload(HoodieReaderContext<T> readerContext) throws IOException { + HoodieLogBlockContentLocation blockContentLoc = getBlockContentLocation().get(); + + // NOTE: It's important to extend Hadoop configuration here to make sure configuration + // is appropriately carried over + Configuration inlineConf = FSUtils.buildInlineConf(blockContentLoc.getHadoopConf()); Review Comment: But the problem is that it will do `new Configuration` per block even though `hadoopConf` is at file reader level. Can we avoid new-ing configuration per block? ########## hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkFileFormatInternalRowReaderContext.scala: ########## @@ -0,0 +1,64 @@ +/* + * 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 + +import org.apache.avro.Schema +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hudi.common.engine.HoodieReaderContext +import org.apache.hudi.common.util.collection.ClosableIterator +import org.apache.hudi.util.CloseableInternalRowIterator +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.{HoodieInternalRowUtils, SparkSession} + +/** + * Implementation of {@link HoodieReaderContext} to read {@link InternalRow}s with + * {@link ParquetFileFormat} on Spark. + * + * This uses Spark parquet reader to read parquet data files or parquet log blocks. + * + * @param sparkSession {@link SparkSession} instance. + * @param parquetFileFormat {@link ParquetFileFormat} instance for parquet file format in Spark. + * @param hadoopConf Hadoop configuration. + */ +class SparkFileFormatInternalRowReaderContext(sparkSession: SparkSession, + parquetFileFormat: ParquetFileFormat, + hadoopConf: Configuration) extends BaseSparkInternalRowReaderContext { + lazy val sparkAdapter = SparkAdapterSupport.sparkAdapter Review Comment: nit: typically we follow the convention of adding the type annotation fir lazy vals. ########## hudi-common/src/main/java/org/apache/hudi/common/table/read/FileGroupReaderState.java: ########## @@ -0,0 +1,36 @@ +/* + * 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.common.table.read; + +import org.apache.hudi.common.config.TypedProperties; + +import org.apache.avro.Schema; + +/** + * A class holding the state that is needed by {@code HoodieFileGroupReader}, + * e.g., schema, merging strategy, etc. + */ +public class FileGroupReaderState { Review Comment: don't need getters? ########## hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java: ########## @@ -0,0 +1,233 @@ +/* + * 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.common.table.read; + +import org.apache.hudi.common.config.HoodieMemoryConfig; +import org.apache.hudi.common.config.HoodieReaderConfig; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.engine.HoodieReaderContext; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.model.HoodieTableQueryType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.log.HoodieMergedLogRecordReader; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.ClosableIterator; +import org.apache.hudi.common.util.collection.EmptyIterator; +import org.apache.hudi.common.util.collection.Pair; + +import org.apache.avro.Schema; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +import static org.apache.hudi.common.fs.FSUtils.getRelativePartitionPath; +import static org.apache.hudi.common.table.HoodieTableConfig.RECORD_MERGER_STRATEGY; +import static org.apache.hudi.common.util.ConfigUtils.getBooleanWithAltKeys; +import static org.apache.hudi.common.util.ConfigUtils.getIntWithAltKeys; +import static org.apache.hudi.common.util.ConfigUtils.getStringWithAltKeys; + +/** + * A file group reader that iterates through the records in a single file group. + * <p> + * This should be used by the every engine integration, by plugging in a + * {@link HoodieReaderContext<T>} implementation. + * + * @param <T> The type of engine-specific record representation, e.g.,{@code InternalRow} + * in Spark and {@code RowData} in Flink. + */ +public final class HoodieFileGroupReader<T> implements Closeable { + private final HoodieReaderContext<T> readerContext; + private final Option<HoodieBaseFile> baseFilePath; + private final Option<List<String>> logFilePathList; + private final Configuration hadoopConf; + private final TypedProperties props; + // Byte offset to start reading from the base file + private final long start; + // Length of bytes to read from the base file + private final long length; + // Key to record and metadata mapping from log files + private final Map<String, Pair<Option<T>, Map<String, Object>>> logFileRecordMapping = new HashMap<>(); + private final FileGroupReaderState readerState = new FileGroupReaderState(); + private ClosableIterator<T> baseFileIterator; + // This is only initialized and used after all records from the base file are iterated + private Iterator<Pair<Option<T>, Map<String, Object>>> logRecordIterator; + private HoodieRecordMerger recordMerger; + + T nextRecord; + + public HoodieFileGroupReader(HoodieReaderContext<T> readerContext, + HoodieTableMetaClient metaClient, + String fileGroupId, + TypedProperties props, + HoodieTimeline timeline, + HoodieTableQueryType queryType, + Option<String> instantTime, + Option<String> startInstantTime) { + // This constructor is a placeholder now to allow automatically fetching the correct list of + // base and log files for a file group. + // Derive base and log files and call the corresponding constructor. + this.readerContext = readerContext; + this.hadoopConf = metaClient.getHadoopConf(); + this.baseFilePath = Option.empty(); + this.logFilePathList = Option.empty(); + this.props = props; + this.start = 0; + this.length = Long.MAX_VALUE; + this.baseFileIterator = new EmptyIterator<>(); + } + + public HoodieFileGroupReader(HoodieReaderContext<T> readerContext, + Configuration hadoopConf, + String tablePath, + String latestCommitTime, + Option<HoodieBaseFile> baseFilePath, + Option<List<String>> logFilePathList, + Schema avroSchema, + TypedProperties props, + long start, + long length) { + this.readerContext = readerContext; + this.hadoopConf = hadoopConf; + this.baseFilePath = baseFilePath; + this.logFilePathList = logFilePathList; + this.props = props; + this.start = start; + this.length = length; + this.recordMerger = readerContext.getRecordMerger( + getStringWithAltKeys(props, RECORD_MERGER_STRATEGY, RECORD_MERGER_STRATEGY.defaultValue())); + this.readerState.tablePath = tablePath; + this.readerState.latestCommitTime = latestCommitTime; + this.readerState.baseFileAvroSchema = avroSchema; + this.readerState.logRecordAvroSchema = avroSchema; + this.readerState.mergeProps.putAll(props); + } + + /** + * Initialize internal iterators on the base and log files. + */ + public void initRecordIterators() { + this.baseFileIterator = baseFilePath.isPresent() + ? readerContext.getFileRecordIterator( + baseFilePath.get().getHadoopPath(), 0, baseFilePath.get().getFileLen(), + readerState.baseFileAvroSchema, readerState.baseFileAvroSchema, hadoopConf) + : new EmptyIterator<>(); + scanLogFiles(); + } + + /** + * @return {@code true} if the next record exists; {@code false} otherwise. + * @throws IOException on reader error. + */ + public boolean hasNext() throws IOException { + while (baseFileIterator.hasNext()) { + T baseRecord = baseFileIterator.next(); + String recordKey = readerContext.getRecordKey(baseRecord, readerState.baseFileAvroSchema); + Pair<Option<T>, Map<String, Object>> logRecordInfo = logFileRecordMapping.remove(recordKey); + Option<T> resultRecord = logRecordInfo != null + ? merge(Option.of(baseRecord), Collections.emptyMap(), logRecordInfo.getLeft(), logRecordInfo.getRight()) + : merge(Option.empty(), Collections.emptyMap(), Option.of(baseRecord), Collections.emptyMap()); + if (resultRecord.isPresent()) { + nextRecord = readerContext.seal(resultRecord.get()); + return true; + } + } + + if (logRecordIterator == null) { + logRecordIterator = logFileRecordMapping.values().iterator(); + } + + while (logRecordIterator.hasNext()) { + Pair<Option<T>, Map<String, Object>> nextRecordInfo = logRecordIterator.next(); + Option<T> resultRecord = merge(Option.empty(), Collections.emptyMap(), + nextRecordInfo.getLeft(), nextRecordInfo.getRight()); + if (resultRecord.isPresent()) { + nextRecord = readerContext.seal(resultRecord.get()); + return true; + } + } + + return false; + } + + /** + * @return The next record after calling {@link #hasNext}. + */ + public T next() { + T result = nextRecord; + nextRecord = null; + return result; + } + + private void scanLogFiles() { + if (logFilePathList.isPresent()) { + FileSystem fs = readerContext.getFs(logFilePathList.get().get(0), hadoopConf); + HoodieMergedLogRecordReader<T> logRecordReader = HoodieMergedLogRecordReader.newBuilder() + .withHoodieReaderContext(readerContext) + .withFileSystem(fs) Review Comment: let's directly set `readerContext.getFs(logFilePathList.get().get(0), hadoopConf)`. then we can remove `FileSystem` import. ########## hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java: ########## @@ -0,0 +1,233 @@ +/* + * 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.common.table.read; + +import org.apache.hudi.common.config.HoodieMemoryConfig; +import org.apache.hudi.common.config.HoodieReaderConfig; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.engine.HoodieReaderContext; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.model.HoodieTableQueryType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.log.HoodieMergedLogRecordReader; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.ClosableIterator; +import org.apache.hudi.common.util.collection.EmptyIterator; +import org.apache.hudi.common.util.collection.Pair; + +import org.apache.avro.Schema; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +import static org.apache.hudi.common.fs.FSUtils.getRelativePartitionPath; +import static org.apache.hudi.common.table.HoodieTableConfig.RECORD_MERGER_STRATEGY; +import static org.apache.hudi.common.util.ConfigUtils.getBooleanWithAltKeys; +import static org.apache.hudi.common.util.ConfigUtils.getIntWithAltKeys; +import static org.apache.hudi.common.util.ConfigUtils.getStringWithAltKeys; + +/** + * A file group reader that iterates through the records in a single file group. + * <p> + * This should be used by the every engine integration, by plugging in a + * {@link HoodieReaderContext<T>} implementation. + * + * @param <T> The type of engine-specific record representation, e.g.,{@code InternalRow} + * in Spark and {@code RowData} in Flink. + */ +public final class HoodieFileGroupReader<T> implements Closeable { + private final HoodieReaderContext<T> readerContext; + private final Option<HoodieBaseFile> baseFilePath; + private final Option<List<String>> logFilePathList; + private final Configuration hadoopConf; + private final TypedProperties props; + // Byte offset to start reading from the base file + private final long start; + // Length of bytes to read from the base file + private final long length; + // Key to record and metadata mapping from log files + private final Map<String, Pair<Option<T>, Map<String, Object>>> logFileRecordMapping = new HashMap<>(); + private final FileGroupReaderState readerState = new FileGroupReaderState(); + private ClosableIterator<T> baseFileIterator; + // This is only initialized and used after all records from the base file are iterated + private Iterator<Pair<Option<T>, Map<String, Object>>> logRecordIterator; + private HoodieRecordMerger recordMerger; + + T nextRecord; + + public HoodieFileGroupReader(HoodieReaderContext<T> readerContext, + HoodieTableMetaClient metaClient, + String fileGroupId, + TypedProperties props, + HoodieTimeline timeline, + HoodieTableQueryType queryType, + Option<String> instantTime, + Option<String> startInstantTime) { + // This constructor is a placeholder now to allow automatically fetching the correct list of + // base and log files for a file group. + // Derive base and log files and call the corresponding constructor. + this.readerContext = readerContext; + this.hadoopConf = metaClient.getHadoopConf(); + this.baseFilePath = Option.empty(); + this.logFilePathList = Option.empty(); + this.props = props; + this.start = 0; + this.length = Long.MAX_VALUE; + this.baseFileIterator = new EmptyIterator<>(); + } + + public HoodieFileGroupReader(HoodieReaderContext<T> readerContext, + Configuration hadoopConf, + String tablePath, + String latestCommitTime, + Option<HoodieBaseFile> baseFilePath, + Option<List<String>> logFilePathList, + Schema avroSchema, + TypedProperties props, + long start, + long length) { + this.readerContext = readerContext; + this.hadoopConf = hadoopConf; + this.baseFilePath = baseFilePath; + this.logFilePathList = logFilePathList; + this.props = props; + this.start = start; + this.length = length; + this.recordMerger = readerContext.getRecordMerger( + getStringWithAltKeys(props, RECORD_MERGER_STRATEGY, RECORD_MERGER_STRATEGY.defaultValue())); + this.readerState.tablePath = tablePath; + this.readerState.latestCommitTime = latestCommitTime; + this.readerState.baseFileAvroSchema = avroSchema; + this.readerState.logRecordAvroSchema = avroSchema; + this.readerState.mergeProps.putAll(props); + } + + /** + * Initialize internal iterators on the base and log files. + */ + public void initRecordIterators() { + this.baseFileIterator = baseFilePath.isPresent() + ? readerContext.getFileRecordIterator( + baseFilePath.get().getHadoopPath(), 0, baseFilePath.get().getFileLen(), + readerState.baseFileAvroSchema, readerState.baseFileAvroSchema, hadoopConf) + : new EmptyIterator<>(); + scanLogFiles(); + } + + /** + * @return {@code true} if the next record exists; {@code false} otherwise. + * @throws IOException on reader error. + */ + public boolean hasNext() throws IOException { + while (baseFileIterator.hasNext()) { + T baseRecord = baseFileIterator.next(); + String recordKey = readerContext.getRecordKey(baseRecord, readerState.baseFileAvroSchema); + Pair<Option<T>, Map<String, Object>> logRecordInfo = logFileRecordMapping.remove(recordKey); + Option<T> resultRecord = logRecordInfo != null + ? merge(Option.of(baseRecord), Collections.emptyMap(), logRecordInfo.getLeft(), logRecordInfo.getRight()) + : merge(Option.empty(), Collections.emptyMap(), Option.of(baseRecord), Collections.emptyMap()); + if (resultRecord.isPresent()) { + nextRecord = readerContext.seal(resultRecord.get()); + return true; + } + } + + if (logRecordIterator == null) { + logRecordIterator = logFileRecordMapping.values().iterator(); + } + + while (logRecordIterator.hasNext()) { + Pair<Option<T>, Map<String, Object>> nextRecordInfo = logRecordIterator.next(); + Option<T> resultRecord = merge(Option.empty(), Collections.emptyMap(), + nextRecordInfo.getLeft(), nextRecordInfo.getRight()); + if (resultRecord.isPresent()) { + nextRecord = readerContext.seal(resultRecord.get()); + return true; + } + } + + return false; + } + + /** + * @return The next record after calling {@link #hasNext}. + */ + public T next() { + T result = nextRecord; + nextRecord = null; + return result; + } + + private void scanLogFiles() { + if (logFilePathList.isPresent()) { + FileSystem fs = readerContext.getFs(logFilePathList.get().get(0), hadoopConf); + HoodieMergedLogRecordReader<T> logRecordReader = HoodieMergedLogRecordReader.newBuilder() Review Comment: didn't see an option to enable optimized log blocks scan. If log compaction is enabled, it would be nice every engine can skip some processing. ########## hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java: ########## @@ -0,0 +1,233 @@ +/* + * 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.common.table.read; + +import org.apache.hudi.common.config.HoodieMemoryConfig; +import org.apache.hudi.common.config.HoodieReaderConfig; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.engine.HoodieReaderContext; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.model.HoodieTableQueryType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.log.HoodieMergedLogRecordReader; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.ClosableIterator; +import org.apache.hudi.common.util.collection.EmptyIterator; +import org.apache.hudi.common.util.collection.Pair; + +import org.apache.avro.Schema; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +import static org.apache.hudi.common.fs.FSUtils.getRelativePartitionPath; +import static org.apache.hudi.common.table.HoodieTableConfig.RECORD_MERGER_STRATEGY; +import static org.apache.hudi.common.util.ConfigUtils.getBooleanWithAltKeys; +import static org.apache.hudi.common.util.ConfigUtils.getIntWithAltKeys; +import static org.apache.hudi.common.util.ConfigUtils.getStringWithAltKeys; + +/** + * A file group reader that iterates through the records in a single file group. + * <p> + * This should be used by the every engine integration, by plugging in a + * {@link HoodieReaderContext<T>} implementation. + * + * @param <T> The type of engine-specific record representation, e.g.,{@code InternalRow} + * in Spark and {@code RowData} in Flink. + */ +public final class HoodieFileGroupReader<T> implements Closeable { + private final HoodieReaderContext<T> readerContext; + private final Option<HoodieBaseFile> baseFilePath; + private final Option<List<String>> logFilePathList; + private final Configuration hadoopConf; + private final TypedProperties props; + // Byte offset to start reading from the base file + private final long start; + // Length of bytes to read from the base file + private final long length; + // Key to record and metadata mapping from log files + private final Map<String, Pair<Option<T>, Map<String, Object>>> logFileRecordMapping = new HashMap<>(); + private final FileGroupReaderState readerState = new FileGroupReaderState(); + private ClosableIterator<T> baseFileIterator; + // This is only initialized and used after all records from the base file are iterated + private Iterator<Pair<Option<T>, Map<String, Object>>> logRecordIterator; + private HoodieRecordMerger recordMerger; + + T nextRecord; + + public HoodieFileGroupReader(HoodieReaderContext<T> readerContext, + HoodieTableMetaClient metaClient, + String fileGroupId, + TypedProperties props, + HoodieTimeline timeline, + HoodieTableQueryType queryType, + Option<String> instantTime, + Option<String> startInstantTime) { + // This constructor is a placeholder now to allow automatically fetching the correct list of + // base and log files for a file group. + // Derive base and log files and call the corresponding constructor. + this.readerContext = readerContext; + this.hadoopConf = metaClient.getHadoopConf(); + this.baseFilePath = Option.empty(); + this.logFilePathList = Option.empty(); + this.props = props; + this.start = 0; + this.length = Long.MAX_VALUE; + this.baseFileIterator = new EmptyIterator<>(); + } + + public HoodieFileGroupReader(HoodieReaderContext<T> readerContext, + Configuration hadoopConf, + String tablePath, + String latestCommitTime, + Option<HoodieBaseFile> baseFilePath, + Option<List<String>> logFilePathList, + Schema avroSchema, Review Comment: I am assuming this will be the projected schema? Also, i think schema will be required in the other contructor too right? ########## hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReader.java: ########## @@ -0,0 +1,233 @@ +/* + * 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.common.table.read; + +import org.apache.hudi.common.config.HoodieMemoryConfig; +import org.apache.hudi.common.config.HoodieReaderConfig; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.engine.HoodieReaderContext; +import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordMerger; +import org.apache.hudi.common.model.HoodieTableQueryType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.log.HoodieMergedLogRecordReader; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.ClosableIterator; +import org.apache.hudi.common.util.collection.EmptyIterator; +import org.apache.hudi.common.util.collection.Pair; + +import org.apache.avro.Schema; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +import static org.apache.hudi.common.fs.FSUtils.getRelativePartitionPath; +import static org.apache.hudi.common.table.HoodieTableConfig.RECORD_MERGER_STRATEGY; +import static org.apache.hudi.common.util.ConfigUtils.getBooleanWithAltKeys; +import static org.apache.hudi.common.util.ConfigUtils.getIntWithAltKeys; +import static org.apache.hudi.common.util.ConfigUtils.getStringWithAltKeys; + +/** + * A file group reader that iterates through the records in a single file group. + * <p> + * This should be used by the every engine integration, by plugging in a + * {@link HoodieReaderContext<T>} implementation. + * + * @param <T> The type of engine-specific record representation, e.g.,{@code InternalRow} + * in Spark and {@code RowData} in Flink. + */ +public final class HoodieFileGroupReader<T> implements Closeable { + private final HoodieReaderContext<T> readerContext; + private final Option<HoodieBaseFile> baseFilePath; + private final Option<List<String>> logFilePathList; + private final Configuration hadoopConf; + private final TypedProperties props; + // Byte offset to start reading from the base file + private final long start; + // Length of bytes to read from the base file + private final long length; + // Key to record and metadata mapping from log files + private final Map<String, Pair<Option<T>, Map<String, Object>>> logFileRecordMapping = new HashMap<>(); + private final FileGroupReaderState readerState = new FileGroupReaderState(); + private ClosableIterator<T> baseFileIterator; + // This is only initialized and used after all records from the base file are iterated + private Iterator<Pair<Option<T>, Map<String, Object>>> logRecordIterator; + private HoodieRecordMerger recordMerger; + + T nextRecord; + + public HoodieFileGroupReader(HoodieReaderContext<T> readerContext, + HoodieTableMetaClient metaClient, + String fileGroupId, + TypedProperties props, + HoodieTimeline timeline, + HoodieTableQueryType queryType, + Option<String> instantTime, + Option<String> startInstantTime) { + // This constructor is a placeholder now to allow automatically fetching the correct list of + // base and log files for a file group. + // Derive base and log files and call the corresponding constructor. + this.readerContext = readerContext; + this.hadoopConf = metaClient.getHadoopConf(); + this.baseFilePath = Option.empty(); + this.logFilePathList = Option.empty(); + this.props = props; + this.start = 0; + this.length = Long.MAX_VALUE; + this.baseFileIterator = new EmptyIterator<>(); + } + + public HoodieFileGroupReader(HoodieReaderContext<T> readerContext, + Configuration hadoopConf, + String tablePath, + String latestCommitTime, + Option<HoodieBaseFile> baseFilePath, + Option<List<String>> logFilePathList, + Schema avroSchema, + TypedProperties props, + long start, + long length) { + this.readerContext = readerContext; + this.hadoopConf = hadoopConf; + this.baseFilePath = baseFilePath; + this.logFilePathList = logFilePathList; + this.props = props; + this.start = start; + this.length = length; + this.recordMerger = readerContext.getRecordMerger( + getStringWithAltKeys(props, RECORD_MERGER_STRATEGY, RECORD_MERGER_STRATEGY.defaultValue())); + this.readerState.tablePath = tablePath; + this.readerState.latestCommitTime = latestCommitTime; + this.readerState.baseFileAvroSchema = avroSchema; + this.readerState.logRecordAvroSchema = avroSchema; + this.readerState.mergeProps.putAll(props); + } + + /** + * Initialize internal iterators on the base and log files. + */ + public void initRecordIterators() { + this.baseFileIterator = baseFilePath.isPresent() + ? readerContext.getFileRecordIterator( + baseFilePath.get().getHadoopPath(), 0, baseFilePath.get().getFileLen(), + readerState.baseFileAvroSchema, readerState.baseFileAvroSchema, hadoopConf) + : new EmptyIterator<>(); + scanLogFiles(); + } + + /** + * @return {@code true} if the next record exists; {@code false} otherwise. + * @throws IOException on reader error. + */ + public boolean hasNext() throws IOException { + while (baseFileIterator.hasNext()) { + T baseRecord = baseFileIterator.next(); + String recordKey = readerContext.getRecordKey(baseRecord, readerState.baseFileAvroSchema); + Pair<Option<T>, Map<String, Object>> logRecordInfo = logFileRecordMapping.remove(recordKey); + Option<T> resultRecord = logRecordInfo != null + ? merge(Option.of(baseRecord), Collections.emptyMap(), logRecordInfo.getLeft(), logRecordInfo.getRight()) + : merge(Option.empty(), Collections.emptyMap(), Option.of(baseRecord), Collections.emptyMap()); + if (resultRecord.isPresent()) { + nextRecord = readerContext.seal(resultRecord.get()); + return true; + } + } + + if (logRecordIterator == null) { + logRecordIterator = logFileRecordMapping.values().iterator(); + } + + while (logRecordIterator.hasNext()) { + Pair<Option<T>, Map<String, Object>> nextRecordInfo = logRecordIterator.next(); + Option<T> resultRecord = merge(Option.empty(), Collections.emptyMap(), + nextRecordInfo.getLeft(), nextRecordInfo.getRight()); + if (resultRecord.isPresent()) { + nextRecord = readerContext.seal(resultRecord.get()); + return true; + } + } + + return false; + } + + /** + * @return The next record after calling {@link #hasNext}. + */ + public T next() { + T result = nextRecord; + nextRecord = null; + return result; + } + + private void scanLogFiles() { + if (logFilePathList.isPresent()) { + FileSystem fs = readerContext.getFs(logFilePathList.get().get(0), hadoopConf); + HoodieMergedLogRecordReader<T> logRecordReader = HoodieMergedLogRecordReader.newBuilder() + .withHoodieReaderContext(readerContext) + .withFileSystem(fs) + .withBasePath(readerState.tablePath) + .withLogFilePaths(logFilePathList.get()) + .withLatestInstantTime(readerState.latestCommitTime) + .withReaderSchema(readerState.logRecordAvroSchema) + .withReadBlocksLazily(getBooleanWithAltKeys(props, HoodieReaderConfig.COMPACTION_LAZY_BLOCK_READ_ENABLE)) + .withReverseReader(false) + .withBufferSize(getIntWithAltKeys(props, HoodieMemoryConfig.MAX_DFS_STREAM_BUFFER_SIZE)) + .withPartition(getRelativePartitionPath( + new Path(readerState.tablePath), new Path(logFilePathList.get().get(0)).getParent() Review Comment: can we parameterize the partition path itself? then we can get rid of `Path` usage. -- 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]
