From d7e4b04940be9aac68a166c62c0eee2455f2d8f9 Mon Sep 17 00:00:00 2001 From: Rebecca Schlussel Date: Fri, 8 Mar 2024 13:52:16 -0500 Subject: [PATCH 1/2] Fix parquet exception handling accidentally used the orc exception handling during refactoring, as the Parquet file never got added --- .../ParquetAggregatedPageSourceFactory.java | 2 +- .../parquet/ParquetPageSourceFactory.java | 2 +- .../ParquetPageSourceFactoryUtils.java | 63 +++++++++++++++++++ 3 files changed, 65 insertions(+), 2 deletions(-) create mode 100644 presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSourceFactoryUtils.java diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetAggregatedPageSourceFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetAggregatedPageSourceFactory.java index 7144141e289c4..956bcd077474d 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetAggregatedPageSourceFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetAggregatedPageSourceFactory.java @@ -42,9 +42,9 @@ import java.util.Set; import static com.facebook.presto.hive.HiveCommonSessionProperties.getReadNullMaskedParquetEncryptedValue; -import static com.facebook.presto.hive.orc.OrcPageSourceFactoryUtils.mapToPrestoException; import static com.facebook.presto.hive.parquet.HdfsParquetDataSource.buildHdfsParquetDataSource; import static com.facebook.presto.hive.parquet.ParquetPageSourceFactory.createDecryptor; +import static com.facebook.presto.hive.parquet.ParquetPageSourceFactoryUtils.mapToPrestoException; import static java.util.Objects.requireNonNull; public class ParquetAggregatedPageSourceFactory diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSourceFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSourceFactory.java index efbca4fe040f6..86cfc4c7f05d4 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSourceFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSourceFactory.java @@ -103,8 +103,8 @@ import static com.facebook.presto.hive.HiveCommonSessionProperties.isUseParquetColumnNames; import static com.facebook.presto.hive.HiveErrorCode.HIVE_PARTITION_SCHEMA_MISMATCH; import static com.facebook.presto.hive.HiveSessionProperties.columnIndexFilterEnabled; -import static com.facebook.presto.hive.orc.OrcPageSourceFactoryUtils.mapToPrestoException; import static com.facebook.presto.hive.parquet.HdfsParquetDataSource.buildHdfsParquetDataSource; +import static com.facebook.presto.hive.parquet.ParquetPageSourceFactoryUtils.mapToPrestoException; import static com.facebook.presto.memory.context.AggregatedMemoryContext.newSimpleAggregatedMemoryContext; import static com.facebook.presto.parquet.ParquetTypeUtils.columnPathFromSubfield; import static com.facebook.presto.parquet.ParquetTypeUtils.getColumnIO; diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSourceFactoryUtils.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSourceFactoryUtils.java new file mode 100644 index 0000000000000..7f53af37afcca --- /dev/null +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetPageSourceFactoryUtils.java @@ -0,0 +1,63 @@ +/* + * Licensed 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 com.facebook.presto.hive.parquet; + +import com.facebook.presto.hive.HiveFileSplit; +import com.facebook.presto.parquet.ParquetCorruptionException; +import com.facebook.presto.spi.PrestoException; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.security.AccessControlException; +import org.apache.parquet.crypto.HiddenColumnException; + +import java.io.FileNotFoundException; + +import static com.facebook.presto.hive.HiveErrorCode.HIVE_BAD_DATA; +import static com.facebook.presto.hive.HiveErrorCode.HIVE_CANNOT_OPEN_SPLIT; +import static com.facebook.presto.hive.HiveErrorCode.HIVE_MISSING_DATA; +import static com.facebook.presto.spi.StandardErrorCode.PERMISSION_DENIED; +import static com.google.common.base.Strings.nullToEmpty; +import static java.lang.String.format; + +public class ParquetPageSourceFactoryUtils +{ + private ParquetPageSourceFactoryUtils() {} + + public static PrestoException mapToPrestoException(Exception e, Path path, HiveFileSplit fileSplit) + { + if (e instanceof PrestoException) { + throw (PrestoException) e; + } + if (e instanceof ParquetCorruptionException) { + throw new PrestoException(HIVE_BAD_DATA, e); + } + if (e instanceof AccessControlException) { + throw new PrestoException(PERMISSION_DENIED, e.getMessage(), e); + } + if (nullToEmpty(e.getMessage()).trim().equals("Filesystem closed") || + e instanceof FileNotFoundException) { + throw new PrestoException(HIVE_CANNOT_OPEN_SPLIT, e); + } + String message = format("Error opening Hive split %s (offset=%s, length=%s): %s", path, fileSplit.getStart(), fileSplit.getLength(), e.getMessage()); + if (e.getClass().getSimpleName().equals("BlockMissingException")) { + throw new PrestoException(HIVE_MISSING_DATA, message, e); + } + if (e instanceof HiddenColumnException) { + message = format("User does not have access to encryption key for encrypted column = %s. If returning 'null' for encrypted " + + "columns is acceptable to your query, please add 'set session hive.read_null_masked_parquet_encrypted_value_enabled=true' before your query", ((HiddenColumnException) e).getColumn()); + throw new PrestoException(PERMISSION_DENIED, message, e); + } + throw new PrestoException(HIVE_CANNOT_OPEN_SPLIT, message, e); + } +} From dfd39e44b61eaffa281f7520081fb36d7d6ac211 Mon Sep 17 00:00:00 2001 From: Rebecca Schlussel Date: Fri, 8 Mar 2024 13:54:10 -0500 Subject: [PATCH 2/2] Fix resource leak in aggregatePageSourceFactories The OrcDataSource wasn't being closed unless there was an error. --- .../presto/hive/orc/OrcAggregatedPageSourceFactory.java | 4 +++- .../hive/parquet/ParquetAggregatedPageSourceFactory.java | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcAggregatedPageSourceFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcAggregatedPageSourceFactory.java index f0a0d3df12944..108b7a1501c93 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcAggregatedPageSourceFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/orc/OrcAggregatedPageSourceFactory.java @@ -198,12 +198,14 @@ public static ConnectorPageSource createOrcPageSource( return new AggregatedOrcPageSource(physicalColumns, reader.getFooter(), typeManager, functionResolution); } catch (Exception e) { + throw mapToPrestoException(e, path, fileSplit); + } + finally { try { orcDataSource.close(); } catch (IOException ignored) { } - throw mapToPrestoException(e, path, fileSplit); } } } diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetAggregatedPageSourceFactory.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetAggregatedPageSourceFactory.java index 956bcd077474d..28c90035ce534 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetAggregatedPageSourceFactory.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/ParquetAggregatedPageSourceFactory.java @@ -137,6 +137,9 @@ public static ConnectorPageSource createParquetPageSource( return new AggregatedParquetPageSource(columns, parquetMetadata, typeManager, functionResolution); } catch (Exception e) { + throw mapToPrestoException(e, path, fileSplit); + } + finally { try { if (dataSource != null) { dataSource.close(); @@ -144,7 +147,6 @@ public static ConnectorPageSource createParquetPageSource( } catch (IOException ignored) { } - throw mapToPrestoException(e, path, fileSplit); } } }