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 @@ -33,7 +33,6 @@
import com.facebook.presto.parquet.RichColumnDescriptor;
import com.facebook.presto.parquet.cache.MetadataReader;
import com.facebook.presto.parquet.predicate.Predicate;
import com.facebook.presto.parquet.reader.ColumnIndexFilterUtils;
import com.facebook.presto.parquet.reader.ParquetReader;
import com.facebook.presto.spi.ColumnHandle;
import com.facebook.presto.spi.ConnectorPageSource;
Expand All @@ -53,6 +52,9 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.security.AccessControlException;
import org.apache.parquet.column.ColumnDescriptor;
import org.apache.parquet.crypto.DecryptionPropertiesFactory;
import org.apache.parquet.crypto.FileDecryptionProperties;
import org.apache.parquet.crypto.InternalFileDecryptor;
import org.apache.parquet.hadoop.metadata.BlockMetaData;
import org.apache.parquet.hadoop.metadata.FileMetaData;
import org.apache.parquet.hadoop.metadata.ParquetMetadata;
Expand Down Expand Up @@ -95,8 +97,10 @@
import static com.facebook.presto.parquet.ParquetTypeUtils.getSubfieldType;
import static com.facebook.presto.parquet.ParquetTypeUtils.lookupColumnByName;
import static com.facebook.presto.parquet.ParquetTypeUtils.nestedColumnPath;
import static com.facebook.presto.parquet.cache.MetadataReader.findFirstNonHiddenColumnId;
import static com.facebook.presto.parquet.predicate.PredicateUtils.buildPredicate;
import static com.facebook.presto.parquet.predicate.PredicateUtils.predicateMatches;
import static com.facebook.presto.parquet.reader.ColumnIndexFilterUtils.getColumnIndexStore;
import static com.facebook.presto.spi.StandardErrorCode.PERMISSION_DENIED;
import static com.google.common.base.Preconditions.checkArgument;
import static com.google.common.base.Strings.nullToEmpty;
Expand Down Expand Up @@ -219,9 +223,13 @@ private static ConnectorPageSource createParquetPageSource(
ParquetDataSource dataSource = null;
try {
FSDataInputStream inputStream = hdfsEnvironment.getFileSystem(user, path, configuration).open(path);
dataSource = buildHdfsParquetDataSource(inputStream, path, stats);
ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, fileSize).getParquetMetadata();

// Lambda expression below requires final variable, so we define a new variable parquetDataSource.
final ParquetDataSource parquetDataSource = buildHdfsParquetDataSource(inputStream, path, stats);
dataSource = parquetDataSource;
DecryptionPropertiesFactory cryptoFactory = DecryptionPropertiesFactory.loadFactory(configuration);
FileDecryptionProperties fileDecryptionProperties = (cryptoFactory == null) ? null : cryptoFactory.getFileDecryptionProperties(configuration, path);
Optional<InternalFileDecryptor> fileDecryptor = (fileDecryptionProperties == null) ? Optional.empty() : Optional.of(new InternalFileDecryptor(fileDecryptionProperties));
ParquetMetadata parquetMetadata = hdfsEnvironment.doAs(user, () -> MetadataReader.readFooter(parquetDataSource, fileSize, fileDecryptor).getParquetMetadata());
FileMetaData fileMetaData = parquetMetadata.getFileMetaData();
MessageType fileSchema = fileMetaData.getSchema();

Expand All @@ -237,9 +245,12 @@ private static ConnectorPageSource createParquetPageSource(

ImmutableList.Builder<BlockMetaData> footerBlocks = ImmutableList.builder();
for (BlockMetaData block : parquetMetadata.getBlocks()) {
long firstDataPage = block.getColumns().get(0).getFirstDataPageOffset();
if (firstDataPage >= start && firstDataPage < start + length) {
footerBlocks.add(block);
Optional<Integer> firstIndex = findFirstNonHiddenColumnId(block);
if (firstIndex.isPresent()) {
long firstDataPage = block.getColumns().get(firstIndex.get()).getFirstDataPageOffset();
if (firstDataPage >= start && firstDataPage < start + length) {
footerBlocks.add(block);
}
}
}

Expand All @@ -250,7 +261,7 @@ private static ConnectorPageSource createParquetPageSource(
ImmutableList.Builder<BlockMetaData> blocks = ImmutableList.builder();
List<ColumnIndexStore> blockIndexStores = new ArrayList<>();
for (BlockMetaData block : footerBlocks.build()) {
Optional<ColumnIndexStore> columnIndexStore = ColumnIndexFilterUtils.getColumnIndexStore(parquetPredicate, finalDataSource, block, descriptorsByPath, columnIndexFilterEnabled);
Optional<ColumnIndexStore> columnIndexStore = getColumnIndexStore(parquetPredicate, finalDataSource, block, descriptorsByPath, columnIndexFilterEnabled);
if (predicateMatches(parquetPredicate, block, finalDataSource, descriptorsByPath, parquetTupleDomain, columnIndexStore, columnIndexFilterEnabled)) {
blocks.add(block);
blockIndexStores.add(columnIndexStore.orElse(null));
Expand All @@ -268,7 +279,8 @@ private static ConnectorPageSource createParquetPageSource(
verificationEnabled,
parquetPredicate,
blockIndexStores,
columnIndexFilterEnabled);
columnIndexFilterEnabled,
fileDecryptor);

ImmutableList.Builder<String> namesBuilder = ImmutableList.builder();
ImmutableList.Builder<Type> typesBuilder = ImmutableList.builder();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,9 @@
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.security.AccessControlException;
import org.apache.parquet.column.ColumnDescriptor;
import org.apache.parquet.crypto.DecryptionPropertiesFactory;
import org.apache.parquet.crypto.FileDecryptionProperties;
import org.apache.parquet.crypto.InternalFileDecryptor;
import org.apache.parquet.hadoop.metadata.BlockMetaData;
import org.apache.parquet.hadoop.metadata.FileMetaData;
import org.apache.parquet.hadoop.metadata.ParquetMetadata;
Expand Down Expand Up @@ -114,6 +117,7 @@
import static com.facebook.presto.parquet.ParquetTypeUtils.getSubfieldType;
import static com.facebook.presto.parquet.ParquetTypeUtils.lookupColumnByName;
import static com.facebook.presto.parquet.ParquetTypeUtils.nestedColumnPath;
import static com.facebook.presto.parquet.cache.MetadataReader.findFirstNonHiddenColumnId;
import static com.facebook.presto.parquet.predicate.PredicateUtils.buildPredicate;
import static com.facebook.presto.parquet.predicate.PredicateUtils.predicateMatches;
import static com.facebook.presto.spi.StandardErrorCode.PERMISSION_DENIED;
Expand All @@ -122,6 +126,7 @@
import static java.lang.String.format;
import static java.util.Objects.requireNonNull;
import static org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector.Category.PRIMITIVE;
import static org.apache.parquet.crypto.DecryptionPropertiesFactory.loadFactory;
import static org.apache.parquet.io.ColumnIOConverter.constructField;
import static org.apache.parquet.io.ColumnIOConverter.findNestedColumnIO;

Expand Down Expand Up @@ -238,12 +243,18 @@ public static ConnectorPageSource createParquetPageSource(
ParquetDataSource dataSource = null;
try {
FSDataInputStream inputStream = hdfsEnvironment.getFileSystem(user, path, configuration).openFile(path, hiveFileContext);
dataSource = buildHdfsParquetDataSource(inputStream, path, stats);
ParquetMetadata parquetMetadata = parquetMetadataSource.getParquetMetadata(
dataSource,
// Lambda expression below requires final variable, so we define a new variable parquetDataSource.
final ParquetDataSource parquetDataSource = buildHdfsParquetDataSource(inputStream, path, stats);
dataSource = parquetDataSource;
DecryptionPropertiesFactory cryptoFactory = loadFactory(configuration);
FileDecryptionProperties fileDecryptionProperties = (cryptoFactory == null) ? null : cryptoFactory.getFileDecryptionProperties(configuration, path);
Optional<InternalFileDecryptor> fileDecryptor = (fileDecryptionProperties == null) ? Optional.empty() : Optional.of(new InternalFileDecryptor(fileDecryptionProperties));
ParquetMetadata parquetMetadata = hdfsEnvironment.doAs(user, () -> parquetMetadataSource.getParquetMetadata(
parquetDataSource,
fileSize,
hiveFileContext.isCacheable(),
hiveFileContext.getModificationTime()).getParquetMetadata();
hiveFileContext.getModificationTime(),
fileDecryptor).getParquetMetadata());

if (!columns.isEmpty() && columns.stream().allMatch(hiveColumnHandle -> hiveColumnHandle.getColumnType() == AGGREGATED)) {
return new AggregatedParquetPageSource(columns, parquetMetadata, typeManager, functionResolution);
Expand All @@ -264,9 +275,12 @@ public static ConnectorPageSource createParquetPageSource(

ImmutableList.Builder<BlockMetaData> footerBlocks = ImmutableList.builder();
for (BlockMetaData block : parquetMetadata.getBlocks()) {
long firstDataPage = block.getColumns().get(0).getFirstDataPageOffset();
if (firstDataPage >= start && firstDataPage < start + length) {
footerBlocks.add(block);
Optional<Integer> firstIndex = findFirstNonHiddenColumnId(block);
if (firstIndex.isPresent()) {
long firstDataPage = block.getColumns().get(firstIndex.get()).getFirstDataPageOffset();
if (firstDataPage >= start && firstDataPage < start + length) {
footerBlocks.add(block);
}
}
}

Expand Down Expand Up @@ -308,7 +322,8 @@ public static ConnectorPageSource createParquetPageSource(
verificationEnabled,
parquetPredicate,
blockIndexStores,
columnIndexFilterEnabled);
columnIndexFilterEnabled,
fileDecryptor);

ImmutableList.Builder<String> namesBuilder = ImmutableList.builder();
ImmutableList.Builder<Type> typesBuilder = ImmutableList.builder();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -267,7 +267,7 @@ public void setup()
PageFunctionCompiler pageFunctionCompiler = new PageFunctionCompiler(metadata, 0);
pageProcessor = new ExpressionCompiler(metadata, pageFunctionCompiler).compilePageProcessor(testSession.getSqlFunctionProperties(), filterConjunction(), projections).get();

parquetMetadata = MetadataReader.readFooter(new FileParquetDataSource(parquetFile), parquetFile.length()).getParquetMetadata();
parquetMetadata = MetadataReader.readFooter(new FileParquetDataSource(parquetFile), parquetFile.length(), Optional.empty()).getParquetMetadata();
}

@TearDown
Expand All @@ -290,7 +290,7 @@ ParquetPageSource createParquetPageSource()
fields.add(ColumnIOConverter.constructField(getTypeFromTypeSignature(), messageColumnIO.getChild(i)));
}

ParquetReader parquetReader = new ParquetReader(messageColumnIO, parquetMetadata.getBlocks(), Optional.empty(), dataSource, newSimpleAggregatedMemoryContext(), new DataSize(16, MEGABYTE), batchReadEnabled, enableVerification, null, null, false);
ParquetReader parquetReader = new ParquetReader(messageColumnIO, parquetMetadata.getBlocks(), Optional.empty(), dataSource, newSimpleAggregatedMemoryContext(), new DataSize(16, MEGABYTE), batchReadEnabled, enableVerification, null, null, false, Optional.empty());
return new ParquetPageSource(parquetReader, Collections.nCopies(channelCount, type), fields, columnNames, new RuntimeStats());
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@
import com.facebook.presto.parquet.RichColumnDescriptor;
import com.facebook.presto.parquet.cache.MetadataReader;
import com.facebook.presto.parquet.predicate.Predicate;
import com.facebook.presto.parquet.reader.ColumnIndexFilterUtils;
import com.facebook.presto.parquet.reader.ParquetReader;
import com.facebook.presto.spi.ConnectorPageSource;
import com.facebook.presto.spi.PrestoException;
Expand All @@ -42,6 +41,9 @@
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.parquet.column.ColumnDescriptor;
import org.apache.parquet.crypto.DecryptionPropertiesFactory;
import org.apache.parquet.crypto.FileDecryptionProperties;
import org.apache.parquet.crypto.InternalFileDecryptor;
import org.apache.parquet.hadoop.metadata.BlockMetaData;
import org.apache.parquet.hadoop.metadata.FileMetaData;
import org.apache.parquet.hadoop.metadata.ParquetMetadata;
Expand All @@ -65,8 +67,10 @@
import static com.facebook.presto.parquet.ParquetTypeUtils.getColumnIO;
import static com.facebook.presto.parquet.ParquetTypeUtils.getDescriptors;
import static com.facebook.presto.parquet.ParquetTypeUtils.getParquetTypeByName;
import static com.facebook.presto.parquet.cache.MetadataReader.findFirstNonHiddenColumnId;
import static com.facebook.presto.parquet.predicate.PredicateUtils.buildPredicate;
import static com.facebook.presto.parquet.predicate.PredicateUtils.predicateMatches;
import static com.facebook.presto.parquet.reader.ColumnIndexFilterUtils.getColumnIndexStore;
import static com.google.common.collect.ImmutableList.toImmutableList;
import static java.lang.String.format;
import static java.util.stream.Collectors.toList;
Expand Down Expand Up @@ -110,8 +114,15 @@ public static ConnectorPageSource createParquetPageSource(
modificationTime,
false);
FSDataInputStream inputStream = filesystem.openFile(path, hiveFileContext);
dataSource = buildHdfsParquetDataSource(inputStream, path, fileFormatDataSourceStats);
ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, fileSize).getParquetMetadata();
// Lambda expression below requires final variable, so we define a new variable parquetDataSource.
final ParquetDataSource parquetDataSource = buildHdfsParquetDataSource(inputStream, path, fileFormatDataSourceStats);
dataSource = parquetDataSource;
DecryptionPropertiesFactory cryptoFactory = DecryptionPropertiesFactory.loadFactory(configuration);
FileDecryptionProperties fileDecryptionProperties = (cryptoFactory == null) ?
null : cryptoFactory.getFileDecryptionProperties(configuration, path);
Optional<InternalFileDecryptor> fileDecryptor = (fileDecryptionProperties == null) ?
Optional.empty() : Optional.of(new InternalFileDecryptor(fileDecryptionProperties));
ParquetMetadata parquetMetadata = hdfsEnvironment.doAs(user, () -> MetadataReader.readFooter(parquetDataSource, fileSize, fileDecryptor).getParquetMetadata());
FileMetaData fileMetaData = parquetMetadata.getFileMetaData();
MessageType fileSchema = fileMetaData.getSchema();
List<Type> parquetFields = regularColumns.stream()
Expand All @@ -127,12 +138,15 @@ public static ConnectorPageSource createParquetPageSource(
List<BlockMetaData> blocks = new ArrayList<>();
List<ColumnIndexStore> blockIndexStores = new ArrayList<>();
for (BlockMetaData block : parquetMetadata.getBlocks()) {
long firstDataPage = block.getColumns().get(0).getFirstDataPageOffset();
Optional<ColumnIndexStore> columnIndexStore = ColumnIndexFilterUtils.getColumnIndexStore(parquetPredicate, finalDataSource, block, descriptorsByPath, columnIndexFilterEnabled);
if ((firstDataPage >= start) && (firstDataPage < (start + length)) &&
predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parquetTupleDomain, columnIndexStore, columnIndexFilterEnabled)) {
blocks.add(block);
blockIndexStores.add(columnIndexStore.orElse(null));
Optional<Integer> firstIndex = findFirstNonHiddenColumnId(block);
if (firstIndex.isPresent()) {
long firstDataPage = block.getColumns().get(firstIndex.get()).getFirstDataPageOffset();
Optional<ColumnIndexStore> columnIndexStore = getColumnIndexStore(parquetPredicate, finalDataSource, block, descriptorsByPath, columnIndexFilterEnabled);
if ((firstDataPage >= start) && (firstDataPage < (start + length)) &&
predicateMatches(parquetPredicate, block, dataSource, descriptorsByPath, parquetTupleDomain, columnIndexStore, columnIndexFilterEnabled)) {
blocks.add(block);
blockIndexStores.add(columnIndexStore.orElse(null));
}
}
}

Expand All @@ -148,7 +162,8 @@ public static ConnectorPageSource createParquetPageSource(
verificationEnabled,
parquetPredicate,
blockIndexStores,
columnIndexFilterEnabled);
columnIndexFilterEnabled,
fileDecryptor);

ImmutableList.Builder<String> namesBuilder = ImmutableList.builder();
ImmutableList.Builder<com.facebook.presto.common.type.Type> prestoTypes = ImmutableList.builder();
Expand Down
Loading