diff --git a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/HdfsParquetDataSource.java b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/HdfsParquetDataSource.java index 04be3dd10922b..a789e9516db12 100644 --- a/presto-hive/src/main/java/com/facebook/presto/hive/parquet/HdfsParquetDataSource.java +++ b/presto-hive/src/main/java/com/facebook/presto/hive/parquet/HdfsParquetDataSource.java @@ -45,6 +45,12 @@ public HdfsParquetDataSource(Path path, long size, FSDataInputStream inputStream this.stats = stats; } + @Override + public String getPath() + { + return name; + } + @Override public final long getReadBytes() { diff --git a/presto-parquet/src/main/java/com/facebook/presto/parquet/ParquetDataSource.java b/presto-parquet/src/main/java/com/facebook/presto/parquet/ParquetDataSource.java index 43458a6661997..92f3d12cff169 100644 --- a/presto-parquet/src/main/java/com/facebook/presto/parquet/ParquetDataSource.java +++ b/presto-parquet/src/main/java/com/facebook/presto/parquet/ParquetDataSource.java @@ -19,6 +19,8 @@ public interface ParquetDataSource extends Closeable { + String getPath(); + long getReadBytes(); long getSize(); diff --git a/presto-parquet/src/main/java/com/facebook/presto/parquet/reader/ParquetReader.java b/presto-parquet/src/main/java/com/facebook/presto/parquet/reader/ParquetReader.java index 570bd37bd2a98..641bc2d831d69 100644 --- a/presto-parquet/src/main/java/com/facebook/presto/parquet/reader/ParquetReader.java +++ b/presto-parquet/src/main/java/com/facebook/presto/parquet/reader/ParquetReader.java @@ -28,6 +28,7 @@ import com.facebook.presto.spi.type.MapType; import com.facebook.presto.spi.type.Type; import com.facebook.presto.spi.type.TypeSignatureParameter; +import com.google.common.base.Joiner; import it.unimi.dsi.fastutil.booleans.BooleanArrayList; import it.unimi.dsi.fastutil.booleans.BooleanList; import it.unimi.dsi.fastutil.ints.IntArrayList; @@ -53,6 +54,8 @@ import static com.google.common.base.Preconditions.checkArgument; import static java.lang.Math.min; import static java.lang.Math.toIntExact; +import static java.lang.String.format; +import static java.util.Locale.ENGLISH; import static java.util.Objects.requireNonNull; public class ParquetReader @@ -206,7 +209,18 @@ private ColumnChunk readPrimitive(PrimitiveField field) ParquetColumnChunk columnChunk = new ParquetColumnChunk(descriptor, buffer, 0); columnReader.setPageReader(columnChunk.readAllPages()); } - return columnReader.readPrimitive(field); + try { + return columnReader.readPrimitive(field); + } + catch (UnsupportedOperationException e) { + throw new ParquetCorruptionException(format( + "There is a mismatch between parquet file schema and partition schema. " + + "The column %s in file %s is declared as type %s but parquet file declared column type as %s.", + Joiner.on(".").join(columnDescriptor.getPath()).toLowerCase(ENGLISH), + dataSource.getPath(), + field.getType(), + columnDescriptor.getType())); + } } private byte[] allocateBlock(int length)