Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,9 @@
package org.apache.hudi.io.storage;

import java.io.IOException;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.Set;

import org.apache.avro.Schema;
Expand All @@ -30,14 +32,17 @@
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.util.BaseFileUtils;
import org.apache.hudi.common.util.ParquetReaderIterator;

import org.apache.parquet.avro.AvroParquetReader;
import org.apache.parquet.avro.AvroReadSupport;
import org.apache.parquet.hadoop.ParquetReader;

public class HoodieParquetReader<R extends IndexedRecord> implements HoodieFileReader<R> {

private final Path path;
private final Configuration conf;
private final BaseFileUtils parquetUtils;
private List<ParquetReaderIterator> readerIterators = new ArrayList<>();
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@nsivabalan please make it final


public HoodieParquetReader(Configuration configuration, Path path) {
this.conf = configuration;
Expand All @@ -64,7 +69,9 @@ public Set<String> filterRowKeys(Set<String> candidateRowKeys) {
public Iterator<R> getRecordIterator(Schema schema) throws IOException {
AvroReadSupport.setAvroReadSchema(conf, schema);
ParquetReader<R> reader = AvroParquetReader.<R>builder(path).withConf(conf).build();
return new ParquetReaderIterator<>(reader);
ParquetReaderIterator parquetReaderIterator = new ParquetReaderIterator<>(reader);
readerIterators.add(parquetReaderIterator);
return parquetReaderIterator;
}

@Override
Expand All @@ -74,6 +81,7 @@ public Schema getSchema() {

@Override
public void close() {
readerIterators.forEach(entry -> entry.close());
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ import org.apache.hudi.common.util.StringUtils
import org.apache.hudi.common.util.ValidationUtils.checkState
import org.apache.hudi.io.storage.HoodieHFileReader
import org.apache.hudi.metadata.HoodieTableMetadata
import org.apache.spark.TaskContext
import org.apache.spark.execution.datasources.HoodieInMemoryFileIndex
import org.apache.spark.internal.Logging
import org.apache.spark.rdd.RDD
Expand All @@ -47,6 +48,7 @@ import org.apache.spark.sql.sources.{BaseRelation, Filter, PrunedFilteredScan}
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.{Row, SQLContext, SparkSession}

import java.io.Closeable
import scala.collection.JavaConverters._
import scala.util.Try

Expand Down Expand Up @@ -333,7 +335,13 @@ object HoodieBaseRelation {
partitionedFile => {
val extension = FSUtils.getFileExtension(partitionedFile.filePath)
if (HoodieFileFormat.PARQUET.getFileExtension.equals(extension)) {
parquetReader.apply(partitionedFile)
val iter = parquetReader.apply(partitionedFile)
if (iter.isInstanceOf[Closeable]) {
// register a callback to close parquetReader which will be executed on task completion.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Did you mean AutoClosable ?

// when tasks finished, this method will be called, and release resources.
Option(TaskContext.get()).foreach(_.addTaskCompletionListener[Unit](_ => iter.asInstanceOf[Closeable].close()))
}
iter
} else if (HoodieFileFormat.HFILE.getFileExtension.equals(extension)) {
hfileReader.apply(partitionedFile)
} else {
Expand Down