Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Fix bugs related to partial aggregation pushdown refactoring #22131

Merged
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 @@ -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);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Copy link
Contributor

Choose a reason for hiding this comment

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

good catch

import static java.util.Objects.requireNonNull;

public class ParquetAggregatedPageSourceFactory
Expand Down Expand Up @@ -137,14 +137,16 @@ 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();
}
}
catch (IOException ignored) {
}
throw mapToPrestoException(e, path, fileSplit);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down
Original file line number Diff line number Diff line change
@@ -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)
Copy link
Contributor Author

Choose a reason for hiding this comment

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

this was copied wholesale from the code that had been in ParquetPageSourceFactory. No changes here.

Copy link
Collaborator

Choose a reason for hiding this comment

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

We can probably generalize common exceptions and keep them in a all purpose PageSourceFactoryUtils

Copy link
Contributor Author

Choose a reason for hiding this comment

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

To clarify this is the common area. This code was copied from what had previously been in ParquetPageSourceFactory. I deleted it from there in my last PR to move it to this common utility class, but accidentally didn't include this file when I pushed the changes in my last PR and didn't notice because I also had the wrong import in the code.

{
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);
}
}
Loading