/**
* 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.tajo.storage.thirdparty.parquet;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.parquet.Preconditions;
import org.apache.parquet.filter.UnboundRecordFilter;
import org.apache.parquet.filter2.compat.FilterCompat;
import org.apache.parquet.filter2.compat.FilterCompat.Filter;
import org.apache.parquet.filter2.compat.RowGroupFilter;
import org.apache.parquet.hadoop.Footer;
import org.apache.parquet.hadoop.ParquetFileReader;
import org.apache.parquet.hadoop.api.ReadSupport;
import org.apache.parquet.hadoop.metadata.BlockMetaData;
import org.apache.parquet.hadoop.util.HiddenFileFilter;
import org.apache.parquet.schema.MessageType;
import java.io.Closeable;
import java.io.IOException;
import java.util.Arrays;
import java.util.Iterator;
import java.util.List;
import static org.apache.parquet.Preconditions.checkNotNull;
/**
* This class is borrowed from parquet-mr-1.8.1, but it is modified in order to fix the PARQUET-363 and progress.
*
* Read records from a Parquet file.
*/
public class ParquetReader<T> implements Closeable {
private final ReadSupport<T> readSupport;
private final Configuration conf;
private final Iterator<Footer> footersIterator;
private final FilterCompat.Filter filter;
private InternalParquetRecordReader<T> reader;
private long totalRowCount;
/**
* @param file the file to read
* @param readSupport to materialize records
* @throws IOException
* @deprecated use {@link #builder(ReadSupport, Path)}
*/
@Deprecated
public ParquetReader(Path file, ReadSupport<T> readSupport) throws IOException {
this(new Configuration(), file, readSupport, FilterCompat.NOOP);
}
/**
* @param conf the configuration
* @param file the file to read
* @param readSupport to materialize records
* @throws IOException
* @deprecated use {@link #builder(ReadSupport, Path)}
*/
@Deprecated
public ParquetReader(Configuration conf, Path file, ReadSupport<T> readSupport) throws IOException {
this(conf, file, readSupport, FilterCompat.NOOP);
}
/**
* @param file the file to read
* @param readSupport to materialize records
* @param unboundRecordFilter the filter to use to filter records
* @throws IOException
* @deprecated use {@link #builder(ReadSupport, Path)}
*/
@Deprecated
public ParquetReader(Path file, ReadSupport<T> readSupport, UnboundRecordFilter unboundRecordFilter) throws IOException {
this(new Configuration(), file, readSupport, FilterCompat.get(unboundRecordFilter));
}
/**
* @param conf the configuration
* @param file the file to read
* @param readSupport to materialize records
* @param unboundRecordFilter the filter to use to filter records
* @throws IOException
* @deprecated use {@link #builder(ReadSupport, Path)}
*/
@Deprecated
public ParquetReader(Configuration conf, Path file, ReadSupport<T> readSupport, UnboundRecordFilter unboundRecordFilter) throws IOException {
this(conf, file, readSupport, FilterCompat.get(unboundRecordFilter));
}
private ParquetReader(Configuration conf,
Path file,
ReadSupport<T> readSupport,
Filter filter) throws IOException {
this.readSupport = readSupport;
this.filter = checkNotNull(filter, "filter");
this.conf = conf;
FileSystem fs = file.getFileSystem(conf);
List<FileStatus> statuses = Arrays.asList(fs.listStatus(file, HiddenFileFilter.INSTANCE));
List<Footer> footers = ParquetFileReader.readAllFootersInParallelUsingSummaryFiles(conf, statuses, false);
this.footersIterator = footers.iterator();
for (Footer footer : footers) {
for(BlockMetaData block : footer.getParquetMetadata().getBlocks()) {
totalRowCount += block.getRowCount();
}
}
}
/**
* @return the next record or null if finished
* @throws IOException
*/
public T read() throws IOException {
try {
if (reader != null && reader.nextKeyValue()) {
return reader.getCurrentValue();
} else {
initReader();
return reader == null ? null : read();
}
} catch (InterruptedException e) {
throw new IOException(e);
}
}
private void initReader() throws IOException {
if (reader != null) {
reader.close();
reader = null;
}
if (footersIterator.hasNext()) {
Footer footer = footersIterator.next();
List<BlockMetaData> blocks = footer.getParquetMetadata().getBlocks();
MessageType fileSchema = footer.getParquetMetadata().getFileMetaData().getSchema();
List<BlockMetaData> filteredBlocks = RowGroupFilter.filterRowGroups(
filter, blocks, fileSchema);
reader = new InternalParquetRecordReader<T>(readSupport, filter);
reader.initialize(footer.getParquetMetadata().getFileMetaData(),
footer.getFile(), filteredBlocks, conf);
}
}
@Override
public void close() throws IOException {
if (reader != null) {
reader.close();
}
}
public float getProgress() {
if (reader != null) {
return reader.getProgress();
} else {
if (!footersIterator.hasNext()) {
return 1.0f;
} else {
return 0.0f;
}
}
}
public long getTotalRowCount() {
return totalRowCount;
}
public static <T> Builder<T> builder(ReadSupport<T> readSupport, Path path) {
return new Builder<T>(readSupport, path);
}
public static class Builder<T> {
private final ReadSupport<T> readSupport;
private final Path file;
private Filter filter;
protected Configuration conf;
private Builder(ReadSupport<T> readSupport, Path path) {
this.readSupport = checkNotNull(readSupport, "readSupport");
this.file = checkNotNull(path, "path");
this.conf = new Configuration();
this.filter = FilterCompat.NOOP;
}
protected Builder(Path path) {
this.readSupport = null;
this.file = checkNotNull(path, "path");
this.conf = new Configuration();
this.filter = FilterCompat.NOOP;
}
public Builder<T> withConf(Configuration conf) {
this.conf = checkNotNull(conf, "conf");
return this;
}
public Builder<T> withFilter(Filter filter) {
this.filter = checkNotNull(filter, "filter");
return this;
}
protected ReadSupport<T> getReadSupport() {
// if readSupport is null, the protected constructor must have been used
Preconditions.checkArgument(readSupport != null,
"[BUG] Classes that extend Builder should override getReadSupport()");
return readSupport;
}
public ParquetReader<T> build() throws IOException {
return new ParquetReader<T>(conf, file, getReadSupport(), filter);
}
}
}