From fe05997129abb8813b462f78ace9096de9a3f2a5 Mon Sep 17 00:00:00 2001 From: "Chen, Junjie" Date: Sun, 8 Mar 2020 16:43:53 +0800 Subject: [PATCH 01/20] Support namemapping resolution for parquet --- .../ParquetDictionaryRowGroupFilter.java | 34 +++++++++++++-- .../parquet/ParquetMetricsRowGroupFilter.java | 31 ++++++++++++-- .../iceberg/parquet/ParquetSchemaUtil.java | 29 +++++++++++++ .../org/apache/iceberg/parquet/ReadConf.java | 24 ++++++----- .../spark/source/TestSparkTableUtil.java | 42 +++++++++++++++++++ 5 files changed, 144 insertions(+), 16 deletions(-) diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java index 6dc1f6de47a3..70d97d89e684 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java @@ -36,6 +36,7 @@ import org.apache.iceberg.expressions.ExpressionVisitors.BoundExpressionVisitor; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.types.Types.StructType; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.column.Dictionary; @@ -47,8 +48,11 @@ import org.apache.parquet.schema.PrimitiveType; public class ParquetDictionaryRowGroupFilter { - private final Expression expr; + private Expression expr; + private Schema schema; private transient ThreadLocal visitors = null; + private NameMapping nameMapping = null; + private boolean caseSensitive; private EvalVisitor visitor() { if (visitors == null) { @@ -61,9 +65,15 @@ public ParquetDictionaryRowGroupFilter(Schema schema, Expression unbound) { this(schema, unbound, true); } + public ParquetDictionaryRowGroupFilter withNameMapping(NameMapping newNameMapping) { + this.nameMapping = newNameMapping; + return this; + } + public ParquetDictionaryRowGroupFilter(Schema schema, Expression unbound, boolean caseSensitive) { - StructType struct = schema.asStruct(); - this.expr = Binder.bind(struct, Expressions.rewriteNot(unbound), caseSensitive); + this.schema = schema; + this.expr = unbound; + this.caseSensitive = caseSensitive; } /** @@ -75,6 +85,16 @@ public ParquetDictionaryRowGroupFilter(Schema schema, Expression unbound, boolea */ public boolean shouldRead(MessageType fileSchema, BlockMetaData rowGroup, DictionaryPageReadStore dictionaries) { + StructType struct; + + if (nameMapping != null) { + MessageType project = ParquetSchemaUtil.pruneColumnsByName(fileSchema, schema); + struct = ParquetSchemaUtil.convert(project).asStruct(); + } else { + struct = schema.asStruct(); + } + + this.expr = Binder.bind(struct, Expressions.rewriteNot(expr), caseSensitive); return visitor().eval(fileSchema, rowGroup, dictionaries); } @@ -104,6 +124,10 @@ private boolean eval(MessageType fileSchema, BlockMetaData rowGroup, int id = colType.getId().intValue(); cols.put(id, desc); conversions.put(id, ParquetConversions.converterFromParquet(colType)); + } else { + int id = nameMapping.find(colType.getName()).id(); + cols.put(id, desc); + conversions.put(id, ParquetConversions.converterFromParquet(colType)); } } @@ -113,6 +137,10 @@ private boolean eval(MessageType fileSchema, BlockMetaData rowGroup, int id = colType.getId().intValue(); isFallback.put(id, ParquetUtil.hasNonDictionaryPages(meta)); mayContainNulls.put(id, mayContainNull(meta)); + } else { + int id = nameMapping.find(colType.getName()).id(); + isFallback.put(id, ParquetUtil.hasNonDictionaryPages(meta)); + mayContainNulls.put(id, mayContainNull(meta)); } } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java index 2d072222cfbc..3e3f823c038e 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java @@ -35,6 +35,7 @@ import org.apache.iceberg.expressions.ExpressionVisitors.BoundExpressionVisitor; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types.StructType; @@ -48,8 +49,10 @@ public class ParquetMetricsRowGroupFilter { private final Schema schema; - private final Expression expr; + private Expression expr; private transient ThreadLocal visitors = null; + private NameMapping nameMapping = null; + private boolean caseSensitive; private MetricsEvalVisitor visitor() { if (visitors == null) { @@ -64,8 +67,14 @@ public ParquetMetricsRowGroupFilter(Schema schema, Expression unbound) { public ParquetMetricsRowGroupFilter(Schema schema, Expression unbound, boolean caseSensitive) { this.schema = schema; - StructType struct = schema.asStruct(); - this.expr = Binder.bind(struct, Expressions.rewriteNot(unbound), caseSensitive); + this.expr = unbound; + this.caseSensitive = caseSensitive; + } + + + public ParquetMetricsRowGroupFilter withNameMapping(NameMapping newNameMapping) { + this.nameMapping = newNameMapping; + return this; } /** @@ -76,6 +85,17 @@ public ParquetMetricsRowGroupFilter(Schema schema, Expression unbound, boolean c * @return false if the file cannot contain rows that match the expression, true otherwise. */ public boolean shouldRead(MessageType fileSchema, BlockMetaData rowGroup) { + StructType struct; + + if (nameMapping != null) { + MessageType project = ParquetSchemaUtil.pruneColumnsByName(fileSchema, schema); + struct = ParquetSchemaUtil.convert(project).asStruct(); + } else { + struct = schema.asStruct(); + } + + this.expr = Binder.bind(struct, Expressions.rewriteNot(expr), caseSensitive); + return visitor().eval(fileSchema, rowGroup); } @@ -102,6 +122,11 @@ private boolean eval(MessageType fileSchema, BlockMetaData rowGroup) { stats.put(id, col.getStatistics()); valueCounts.put(id, col.getValueCount()); conversions.put(id, ParquetConversions.converterFromParquet(colType)); + } else { + int id = nameMapping.find(colType.getName()).id(); + stats.put(id, col.getStatistics()); + valueCounts.put(id, col.getValueCount()); + conversions.put(id, ParquetConversions.converterFromParquet(colType)); } } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java index 421020235653..baa2dd686727 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java @@ -82,6 +82,35 @@ public static MessageType pruneColumnsFallback(MessageType fileSchema, Schema ex return builder.named(fileSchema.getName()); } + + /** + * Prunes columns from a Parquet file schema that was written without field ids. + * The order of columns in the resulting Parquet schema matches the Parquet file. + * + * @param fileSchema schema from a Parquet file that does not have field ids. + * @param expectedSchema expected schema + * @return a parquet schema pruned using the expected schema + */ + public static MessageType pruneColumnsByName(MessageType fileSchema, Schema expectedSchema) { + Set selectedNames = Sets.newHashSet(); + + for (Types.NestedField field : expectedSchema.columns()) { + selectedNames.add(field.name()); + } + + MessageTypeBuilder builder = org.apache.parquet.schema.Types.buildMessage(); + + int ordinal = 1; + for (Type type : fileSchema.getFields()) { + if (selectedNames.contains(type.getName())) { + builder.addField(type.withId(ordinal)); + } + ordinal += 1; + } + + return builder.named(fileSchema.getName()); + } + public static boolean hasIds(MessageType fileSchema) { try { // Try to convert the type to Iceberg. If an ID assignment is needed, return false. diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java index 51a19d4a2b9c..707fa1630d8e 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java @@ -32,6 +32,8 @@ import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.mapping.MappingUtil; +import org.apache.iceberg.mapping.NameMapping; import org.apache.parquet.ParquetReadOptions; import org.apache.parquet.hadoop.ParquetFileReader; import org.apache.parquet.hadoop.metadata.BlockMetaData; @@ -72,29 +74,31 @@ class ReadConf { this.options = options; this.reader = newReader(file, options); MessageType fileSchema = reader.getFileMetaData().getSchema(); - boolean hasIds = ParquetSchemaUtil.hasIds(fileSchema); - MessageType typeWithIds = hasIds ? fileSchema : ParquetSchemaUtil.addFallbackIds(fileSchema); this.projection = hasIds ? - ParquetSchemaUtil.pruneColumns(fileSchema, expectedSchema) : - ParquetSchemaUtil.pruneColumnsFallback(fileSchema, expectedSchema); + ParquetSchemaUtil.pruneColumns(fileSchema, expectedSchema) : + ParquetSchemaUtil.pruneColumnsByName(fileSchema, expectedSchema); + this.rowGroups = reader.getRowGroups(); this.shouldSkip = new boolean[rowGroups.size()]; ParquetMetricsRowGroupFilter statsFilter = null; ParquetDictionaryRowGroupFilter dictFilter = null; if (filter != null) { - statsFilter = new ParquetMetricsRowGroupFilter(expectedSchema, filter, caseSensitive); - dictFilter = new ParquetDictionaryRowGroupFilter(expectedSchema, filter, caseSensitive); + NameMapping nameMapping = MappingUtil.create(ParquetSchemaUtil.convert(fileSchema)); + statsFilter = new ParquetMetricsRowGroupFilter(expectedSchema, filter, caseSensitive) + .withNameMapping(nameMapping); + dictFilter = new ParquetDictionaryRowGroupFilter(expectedSchema, filter, caseSensitive) + .withNameMapping(nameMapping); } long computedTotalValues = 0L; for (int i = 0; i < shouldSkip.length; i += 1) { BlockMetaData rowGroup = rowGroups.get(i); boolean shouldRead = filter == null || ( - statsFilter.shouldRead(typeWithIds, rowGroup) && - dictFilter.shouldRead(typeWithIds, rowGroup, reader.getDictionaryReader(rowGroup))); + statsFilter.shouldRead(fileSchema, rowGroup) && + dictFilter.shouldRead(fileSchema, rowGroup, reader.getDictionaryReader(rowGroup))); this.shouldSkip[i] = !shouldRead; if (shouldRead) { computedTotalValues += rowGroup.getRowCount(); @@ -103,12 +107,12 @@ class ReadConf { this.totalValues = computedTotalValues; if (readerFunc != null) { - this.model = (ParquetValueReader) readerFunc.apply(typeWithIds); + this.model = (ParquetValueReader) readerFunc.apply(fileSchema); this.vectorizedModel = null; this.columnChunkMetaDataForRowGroups = null; } else { this.model = null; - this.vectorizedModel = (VectorizedReader) batchedReaderFunc.apply(typeWithIds); + this.vectorizedModel = (VectorizedReader) batchedReaderFunc.apply(fileSchema); this.columnChunkMetaDataForRowGroups = getColumnChunkMetadataForRowGroups(); } diff --git a/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTableUtil.java b/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTableUtil.java index 12cc12468993..5635a92f9a8b 100644 --- a/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTableUtil.java +++ b/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTableUtil.java @@ -24,15 +24,18 @@ import java.io.File; import java.io.IOException; import java.util.List; +import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.hadoop.HadoopTables; import org.apache.iceberg.hive.HiveTableBaseTest; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.SparkTableUtil; import org.apache.iceberg.spark.SparkTableUtil.SparkPartition; +import org.apache.iceberg.types.Types; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SQLContext; @@ -48,6 +51,8 @@ import org.junit.rules.TemporaryFolder; import scala.collection.Seq; +import static org.apache.iceberg.types.Types.NestedField.optional; + public class TestSparkTableUtil extends HiveTableBaseTest { private static final Configuration CONF = HiveTableBaseTest.hiveConf; private static final String tableName = "hive_table"; @@ -200,4 +205,41 @@ public void testImportAsHiveTable() throws Exception { long count2 = spark.read().format("iceberg").load(DB_NAME + ".test_partitioned_table").count(); Assert.assertEquals("three values ", 3, count2); } + + @Test + public void testImportWithIncompatibleSchema() throws Exception { + spark.table(qualifiedTableName).write().mode("overwrite").format("parquet") + .saveAsTable("original_table"); + + // The field is different so that it will project with name mapping + Schema filteredSchema = new Schema( + optional(1, "data", Types.StringType.get()) + ); + + TableIdentifier source = new TableIdentifier("original_table"); + Table table = catalog.createTable( + org.apache.iceberg.catalog.TableIdentifier.of(DB_NAME, "target_table"), + filteredSchema, + SparkSchemaUtil.specForTable(spark, "original_table")); + File stagingDir = temp.newFolder("staging-dir"); + SparkTableUtil.importSparkTable(spark, source, table, stagingDir.toString()); + + // The filter invoke the metric/dictionary row group filter in which it project schema + // with name mapping again to match the metric read from footer. + List actual = spark.read().format("iceberg").load(DB_NAME + ".target_table") + .select("data") + .sort("data") + .filter("data<'c'") + .collectAsList() + .stream() + .map(r -> r.getString(0)) + .collect(Collectors.toList()); + + List expected = Lists.newArrayList( + new SimpleRecord(2, "a"), + new SimpleRecord(1, "b") + ); + + Assert.assertEquals(expected.stream().map(SimpleRecord::getData).collect(Collectors.toList()), actual); + } } From b2f6b96409421d19e1e800ce7e71a641ccb9a412 Mon Sep 17 00:00:00 2001 From: "Chen, Junjie" Date: Wed, 11 Mar 2020 00:55:38 +0800 Subject: [PATCH 02/20] create name mapping from expected schema --- .../org/apache/iceberg/parquet/Parquet.java | 13 +++-- .../ParquetDictionaryRowGroupFilter.java | 15 ++++-- .../parquet/ParquetMetricsRowGroupFilter.java | 9 +++- .../iceberg/parquet/ParquetReadSupport.java | 3 +- .../apache/iceberg/parquet/ParquetReader.java | 7 ++- .../iceberg/parquet/ParquetSchemaUtil.java | 52 ++----------------- .../org/apache/iceberg/parquet/ReadConf.java | 13 +++-- .../parquet/VectorizedParquetReader.java | 8 ++- 8 files changed, 53 insertions(+), 67 deletions(-) diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java index 689ddb1f8405..452e0d0f8528 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java @@ -41,6 +41,7 @@ import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.mapping.NameMapping; import org.apache.parquet.HadoopReadOptions; import org.apache.parquet.ParquetReadOptions; import org.apache.parquet.avro.AvroReadSupport; @@ -312,6 +313,7 @@ public static class ReadBuilder { private boolean callInit = false; private boolean reuseContainers = false; private int maxRecordsPerBatch = 10000; + private NameMapping nameMapping = null; private ReadBuilder(InputFile file) { this.file = file; @@ -393,6 +395,11 @@ public ReadBuilder recordsPerBatch(int numRowsPerBatch) { return this; } + public ReadBuilder withNameMapping(NameMapping newNameMapping) { + this.nameMapping = newNameMapping; + return this; + } + @SuppressWarnings({"unchecked", "checkstyle:CyclomaticComplexity"}) public CloseableIterable build() { if (readerFunc != null || batchedReaderFunc != null) { @@ -419,11 +426,11 @@ public CloseableIterable build() { ParquetReadOptions options = optionsBuilder.build(); if (batchedReaderFunc != null) { - return new VectorizedParquetReader(file, schema, options, batchedReaderFunc, filter, reuseContainers, - caseSensitive, maxRecordsPerBatch); + return new VectorizedParquetReader(file, schema, options, batchedReaderFunc, nameMapping, filter, + reuseContainers, caseSensitive, maxRecordsPerBatch); } else { return new org.apache.iceberg.parquet.ParquetReader<>( - file, schema, options, readerFunc, filter, reuseContainers, caseSensitive); + file, schema, options, readerFunc, nameMapping, filter, reuseContainers, caseSensitive); } } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java index 70d97d89e684..46637554e9bc 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java @@ -36,6 +36,7 @@ import org.apache.iceberg.expressions.ExpressionVisitors.BoundExpressionVisitor; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.mapping.MappedField; import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.types.Types.StructType; import org.apache.parquet.column.ColumnDescriptor; @@ -88,7 +89,7 @@ public boolean shouldRead(MessageType fileSchema, BlockMetaData rowGroup, StructType struct; if (nameMapping != null) { - MessageType project = ParquetSchemaUtil.pruneColumnsByName(fileSchema, schema); + MessageType project = ParquetSchemaUtil.pruneColumnsByName(fileSchema, schema, nameMapping); struct = ParquetSchemaUtil.convert(project).asStruct(); } else { struct = schema.asStruct(); @@ -125,7 +126,11 @@ private boolean eval(MessageType fileSchema, BlockMetaData rowGroup, cols.put(id, desc); conversions.put(id, ParquetConversions.converterFromParquet(colType)); } else { - int id = nameMapping.find(colType.getName()).id(); + MappedField field = nameMapping.find(colType.getName()); + if (field == null) { + continue; + } + int id = field.id(); cols.put(id, desc); conversions.put(id, ParquetConversions.converterFromParquet(colType)); } @@ -138,7 +143,11 @@ private boolean eval(MessageType fileSchema, BlockMetaData rowGroup, isFallback.put(id, ParquetUtil.hasNonDictionaryPages(meta)); mayContainNulls.put(id, mayContainNull(meta)); } else { - int id = nameMapping.find(colType.getName()).id(); + MappedField field = nameMapping.find(colType.getName()); + if (field == null) { + continue; + } + int id = field.id(); isFallback.put(id, ParquetUtil.hasNonDictionaryPages(meta)); mayContainNulls.put(id, mayContainNull(meta)); } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java index 3e3f823c038e..c236115f532c 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java @@ -35,6 +35,7 @@ import org.apache.iceberg.expressions.ExpressionVisitors.BoundExpressionVisitor; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.mapping.MappedField; import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Type; @@ -88,7 +89,7 @@ public boolean shouldRead(MessageType fileSchema, BlockMetaData rowGroup) { StructType struct; if (nameMapping != null) { - MessageType project = ParquetSchemaUtil.pruneColumnsByName(fileSchema, schema); + MessageType project = ParquetSchemaUtil.pruneColumnsByName(fileSchema, schema, nameMapping); struct = ParquetSchemaUtil.convert(project).asStruct(); } else { struct = schema.asStruct(); @@ -123,7 +124,11 @@ private boolean eval(MessageType fileSchema, BlockMetaData rowGroup) { valueCounts.put(id, col.getValueCount()); conversions.put(id, ParquetConversions.converterFromParquet(colType)); } else { - int id = nameMapping.find(colType.getName()).id(); + MappedField field = nameMapping.find(colType.getName()); + if (field == null) { + continue; + } + int id = field.id(); stats.put(id, col.getStatistics()); valueCounts.put(id, col.getValueCount()); conversions.put(id, ParquetConversions.converterFromParquet(colType)); diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java index 9588676cce20..19f5faff2e81 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java @@ -26,6 +26,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.Schema; import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.mapping.MappingUtil; import org.apache.parquet.avro.AvroReadSupport; import org.apache.parquet.hadoop.api.InitContext; import org.apache.parquet.hadoop.api.ReadSupport; @@ -57,7 +58,7 @@ public ReadContext init(Configuration configuration, Map keyValu MessageType projection = ParquetSchemaUtil.hasIds(fileSchema) ? ParquetSchemaUtil.pruneColumns(fileSchema, expectedSchema) : - ParquetSchemaUtil.pruneColumnsFallback(fileSchema, expectedSchema); + ParquetSchemaUtil.pruneColumnsByName(fileSchema, expectedSchema, MappingUtil.create(expectedSchema)); // override some known backward-compatibility options configuration.set("parquet.strict.typing", "false"); diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java index 9ba36b33724a..51a65f000289 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java @@ -30,6 +30,7 @@ import org.apache.iceberg.io.CloseableGroup; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.mapping.NameMapping; import org.apache.parquet.ParquetReadOptions; import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.hadoop.ParquetFileReader; @@ -43,9 +44,10 @@ public class ParquetReader extends CloseableGroup implements CloseableIterabl private final Expression filter; private final boolean reuseContainers; private final boolean caseSensitive; + private final NameMapping nameMapping; public ParquetReader(InputFile input, Schema expectedSchema, ParquetReadOptions options, - Function> readerFunc, + Function> readerFunc, NameMapping nameMapping, Expression filter, boolean reuseContainers, boolean caseSensitive) { this.input = input; this.expectedSchema = expectedSchema; @@ -55,6 +57,7 @@ public ParquetReader(InputFile input, Schema expectedSchema, ParquetReadOptions this.filter = filter == Expressions.alwaysTrue() ? null : filter; this.reuseContainers = reuseContainers; this.caseSensitive = caseSensitive; + this.nameMapping = nameMapping; } private ReadConf conf = null; @@ -62,7 +65,7 @@ public ParquetReader(InputFile input, Schema expectedSchema, ParquetReadOptions private ReadConf init() { if (conf == null) { ReadConf readConf = new ReadConf<>( - input, options, expectedSchema, filter, readerFunc, null, reuseContainers, caseSensitive, null); + input, options, expectedSchema, filter, readerFunc, null, nameMapping, reuseContainers, caseSensitive, null); this.conf = readConf.copy(); return readConf; } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java index baa2dd686727..c6b09e7afb28 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java @@ -22,6 +22,7 @@ import com.google.common.collect.Sets; import java.util.Set; import org.apache.iceberg.Schema; +import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; import org.apache.parquet.schema.MessageType; @@ -52,46 +53,13 @@ public static MessageType pruneColumns(MessageType fileSchema, Schema expectedSc /** * Prunes columns from a Parquet file schema that was written without field ids. - *

- * Files that were written without field ids are read assuming that schema evolution preserved - * column order. Deleting columns was not allowed. - *

* The order of columns in the resulting Parquet schema matches the Parquet file. * * @param fileSchema schema from a Parquet file that does not have field ids. * @param expectedSchema expected schema * @return a parquet schema pruned using the expected schema */ - public static MessageType pruneColumnsFallback(MessageType fileSchema, Schema expectedSchema) { - Set selectedIds = Sets.newHashSet(); - - for (Types.NestedField field : expectedSchema.columns()) { - selectedIds.add(field.fieldId()); - } - - MessageTypeBuilder builder = org.apache.parquet.schema.Types.buildMessage(); - - int ordinal = 1; - for (Type type : fileSchema.getFields()) { - if (selectedIds.contains(ordinal)) { - builder.addField(type.withId(ordinal)); - } - ordinal += 1; - } - - return builder.named(fileSchema.getName()); - } - - - /** - * Prunes columns from a Parquet file schema that was written without field ids. - * The order of columns in the resulting Parquet schema matches the Parquet file. - * - * @param fileSchema schema from a Parquet file that does not have field ids. - * @param expectedSchema expected schema - * @return a parquet schema pruned using the expected schema - */ - public static MessageType pruneColumnsByName(MessageType fileSchema, Schema expectedSchema) { + public static MessageType pruneColumnsByName(MessageType fileSchema, Schema expectedSchema, NameMapping nameMapping) { Set selectedNames = Sets.newHashSet(); for (Types.NestedField field : expectedSchema.columns()) { @@ -100,12 +68,10 @@ public static MessageType pruneColumnsByName(MessageType fileSchema, Schema expe MessageTypeBuilder builder = org.apache.parquet.schema.Types.buildMessage(); - int ordinal = 1; for (Type type : fileSchema.getFields()) { if (selectedNames.contains(type.getName())) { - builder.addField(type.withId(ordinal)); + builder.addField(type.withId(nameMapping.find(type.getName()).id())); } - ordinal += 1; } return builder.named(fileSchema.getName()); @@ -129,16 +95,4 @@ protected int nextId() { return false; } } - - public static MessageType addFallbackIds(MessageType fileSchema) { - MessageTypeBuilder builder = org.apache.parquet.schema.Types.buildMessage(); - - int ordinal = 1; // ids are assigned starting at 1 - for (Type type : fileSchema.getFields()) { - builder.addField(type.withId(ordinal)); - ordinal += 1; - } - - return builder.named(fileSchema.getName()); - } } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java index 707fa1630d8e..96bc1e2c24f4 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java @@ -61,6 +61,8 @@ class ReadConf { private final boolean reuseContainers; @Nullable private final Integer batchSize; + @Nullable + private final NameMapping nameMapping; // List of column chunk metadata for each row group private final List> columnChunkMetaDataForRowGroups; @@ -68,17 +70,18 @@ class ReadConf { @SuppressWarnings("unchecked") ReadConf(InputFile file, ParquetReadOptions options, Schema expectedSchema, Expression filter, Function> readerFunc, Function> batchedReaderFunc, boolean reuseContainers, + VectorizedReader> batchedReaderFunc, NameMapping nameMapping, boolean reuseContainers, boolean caseSensitive, Integer bSize) { this.file = file; this.options = options; this.reader = newReader(file, options); MessageType fileSchema = reader.getFileMetaData().getSchema(); boolean hasIds = ParquetSchemaUtil.hasIds(fileSchema); + this.nameMapping = nameMapping == null ? MappingUtil.create(expectedSchema) : nameMapping; this.projection = hasIds ? ParquetSchemaUtil.pruneColumns(fileSchema, expectedSchema) : - ParquetSchemaUtil.pruneColumnsByName(fileSchema, expectedSchema); + ParquetSchemaUtil.pruneColumnsByName(fileSchema, expectedSchema, this.nameMapping); this.rowGroups = reader.getRowGroups(); this.shouldSkip = new boolean[rowGroups.size()]; @@ -86,11 +89,10 @@ class ReadConf { ParquetMetricsRowGroupFilter statsFilter = null; ParquetDictionaryRowGroupFilter dictFilter = null; if (filter != null) { - NameMapping nameMapping = MappingUtil.create(ParquetSchemaUtil.convert(fileSchema)); statsFilter = new ParquetMetricsRowGroupFilter(expectedSchema, filter, caseSensitive) - .withNameMapping(nameMapping); + .withNameMapping(this.nameMapping); dictFilter = new ParquetDictionaryRowGroupFilter(expectedSchema, filter, caseSensitive) - .withNameMapping(nameMapping); + .withNameMapping(this.nameMapping); } long computedTotalValues = 0L; @@ -133,6 +135,7 @@ private ReadConf(ReadConf toCopy) { this.batchSize = toCopy.batchSize; this.vectorizedModel = toCopy.vectorizedModel; this.columnChunkMetaDataForRowGroups = toCopy.columnChunkMetaDataForRowGroups; + this.nameMapping = toCopy.nameMapping; } ParquetFileReader reader() { diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java b/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java index 077464b07599..19d1a2122b84 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java @@ -33,6 +33,7 @@ import org.apache.iceberg.io.CloseableGroup; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.mapping.NameMapping; import org.apache.parquet.ParquetReadOptions; import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.hadoop.ParquetFileReader; @@ -49,10 +50,11 @@ public class VectorizedParquetReader extends CloseableGroup implements Closea private boolean reuseContainers; private final boolean caseSensitive; private final int batchSize; + private final NameMapping nameMapping; public VectorizedParquetReader( InputFile input, Schema expectedSchema, ParquetReadOptions options, - Function> readerFunc, + Function> readerFunc, NameMapping nameMapping, Expression filter, boolean reuseContainers, boolean caseSensitive, int maxRecordsPerBatch) { this.input = input; this.expectedSchema = expectedSchema; @@ -63,6 +65,7 @@ public VectorizedParquetReader( this.reuseContainers = reuseContainers; this.caseSensitive = caseSensitive; this.batchSize = maxRecordsPerBatch; + this.nameMapping = nameMapping; } private ReadConf conf = null; @@ -70,7 +73,8 @@ public VectorizedParquetReader( private ReadConf init() { if (conf == null) { ReadConf readConf = new ReadConf( - input, options, expectedSchema, filter, null, batchReaderFunc, reuseContainers, caseSensitive, batchSize); + input, options, expectedSchema, filter, null, batchReaderFunc, nameMapping, + reuseContainers, caseSensitive, batchSize); this.conf = readConf.copy(); return readConf; } From 502b1b935a902a9fa208d5e2443fe011fb51eff5 Mon Sep 17 00:00:00 2001 From: Junjie Chen Date: Tue, 24 Mar 2020 01:15:41 +0800 Subject: [PATCH 03/20] Address comments --- .../java/org/apache/iceberg/avro/Avro.java | 2 +- .../iceberg/avro/TestAvroNameMapping.java | 2 +- .../iceberg/parquet/MessageTypeToType.java | 2 +- .../iceberg/parquet/ParquetSchemaUtil.java | 18 +++++++++++------- .../org/apache/iceberg/parquet/ReadConf.java | 5 ----- 5 files changed, 14 insertions(+), 15 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/avro/Avro.java b/core/src/main/java/org/apache/iceberg/avro/Avro.java index 40c43cbe5148..1a586b631dcb 100644 --- a/core/src/main/java/org/apache/iceberg/avro/Avro.java +++ b/core/src/main/java/org/apache/iceberg/avro/Avro.java @@ -235,7 +235,7 @@ public ReadBuilder rename(String fullName, String newName) { return this; } - public ReadBuilder nameMapping(NameMapping newNameMapping) { + public ReadBuilder withNameMapping(NameMapping newNameMapping) { this.nameMapping = newNameMapping; return this; } diff --git a/core/src/test/java/org/apache/iceberg/avro/TestAvroNameMapping.java b/core/src/test/java/org/apache/iceberg/avro/TestAvroNameMapping.java index b8e1d419e423..fe95cc4bc392 100644 --- a/core/src/test/java/org/apache/iceberg/avro/TestAvroNameMapping.java +++ b/core/src/test/java/org/apache/iceberg/avro/TestAvroNameMapping.java @@ -315,7 +315,7 @@ private Record writeAndRead(Schema writeSchema, Iterable records = Avro.read(Files.localInput(file)) .project(readSchema) - .nameMapping(nameMapping) + .withNameMapping(nameMapping) .build(); return Iterables.getOnlyElement(records); diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/MessageTypeToType.java b/parquet/src/main/java/org/apache/iceberg/parquet/MessageTypeToType.java index 993dfc77ef2b..88a42cc2067b 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/MessageTypeToType.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/MessageTypeToType.java @@ -235,7 +235,7 @@ protected int nextId() { return current; } - private int getId(org.apache.parquet.schema.Type type) { + protected int getId(org.apache.parquet.schema.Type type) { org.apache.parquet.schema.Type.ID id = type.getId(); if (id != null) { return id.intValue(); diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java index c6b09e7afb28..2bb69ec8126e 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java @@ -82,17 +82,21 @@ public static boolean hasIds(MessageType fileSchema) { // Try to convert the type to Iceberg. If an ID assignment is needed, return false. ParquetTypeVisitor.visit(fileSchema, new MessageTypeToType(fileSchema) { @Override - protected int nextId() { - throw new IllegalStateException("Needed to assign ID"); + protected int getId(org.apache.parquet.schema.Type type) { + org.apache.parquet.schema.Type.ID id = type.getId(); + if (id != null) { + throw new IllegalStateException("at least one ID exists"); + } else { + return nextId(); + } } }); - // no assignment was needed - return true; - - } catch (IllegalStateException e) { - // at least one field was missing an id. + // all IDs are assigned from nextId() return false; + } catch (IllegalStateException e) { + // at least one field exists. + return true; } } } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java index 96bc1e2c24f4..6cae7ab326f5 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java @@ -27,7 +27,6 @@ import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; -import javax.annotation.Nullable; import org.apache.iceberg.Schema; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.expressions.Expression; @@ -51,17 +50,13 @@ class ReadConf { private final InputFile file; private final ParquetReadOptions options; private final MessageType projection; - @Nullable private final ParquetValueReader model; - @Nullable private final VectorizedReader vectorizedModel; private final List rowGroups; private final boolean[] shouldSkip; private final long totalValues; private final boolean reuseContainers; - @Nullable private final Integer batchSize; - @Nullable private final NameMapping nameMapping; // List of column chunk metadata for each row group From 1fd7d605b54a637d76a2d0bd86cf67914f68e93f Mon Sep 17 00:00:00 2001 From: "Chen, Junjie" Date: Wed, 8 Apr 2020 18:12:48 +0800 Subject: [PATCH 04/20] add ID fallback way back --- .../org/apache/iceberg/parquet/Parquet.java | 11 +++- .../ParquetDictionaryRowGroupFilter.java | 27 ++------ .../parquet/ParquetMetricsRowGroupFilter.java | 15 +---- .../iceberg/parquet/ParquetReadSupport.java | 19 ++++-- .../iceberg/parquet/ParquetSchemaUtil.java | 65 ++++++++++++++++++- .../org/apache/iceberg/parquet/ReadConf.java | 33 ++++++---- .../iceberg/spark/source/RowDataReader.java | 4 ++ 7 files changed, 117 insertions(+), 57 deletions(-) diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java index 452e0d0f8528..c4037cb4a600 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java @@ -41,6 +41,7 @@ import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.mapping.MappingUtil; import org.apache.iceberg.mapping.NameMapping; import org.apache.parquet.HadoopReadOptions; import org.apache.parquet.ParquetReadOptions; @@ -436,7 +437,7 @@ public CloseableIterable build() { ParquetReadBuilder builder = new ParquetReadBuilder<>(ParquetIO.file(file)); - builder.project(schema); + builder.project(schema).withNameMapping(MappingUtil.create(schema)); if (readSupport != null) { builder.readSupport((ReadSupport) readSupport); @@ -490,6 +491,7 @@ private static class ParquetReadBuilder extends ParquetReader.Builder { private Schema schema = null; private ReadSupport readSupport = null; private boolean callInit = false; + private NameMapping nameMapping = null; private ParquetReadBuilder(org.apache.parquet.io.InputFile file) { super(file); @@ -500,6 +502,11 @@ public ParquetReadBuilder project(Schema newSchema) { return this; } + public ParquetReadBuilder withNameMapping(NameMapping newNameMapping) { + this.nameMapping = newNameMapping; + return this; + } + public ParquetReadBuilder readSupport(ReadSupport newReadSupport) { this.readSupport = newReadSupport; return this; @@ -512,7 +519,7 @@ public ParquetReadBuilder callInit() { @Override protected ReadSupport getReadSupport() { - return new ParquetReadSupport<>(schema, readSupport, callInit); + return new ParquetReadSupport<>(schema, readSupport, callInit, nameMapping); } } } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java index 46637554e9bc..b834b8c950c8 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java @@ -36,7 +36,6 @@ import org.apache.iceberg.expressions.ExpressionVisitors.BoundExpressionVisitor; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.expressions.Literal; -import org.apache.iceberg.mapping.MappedField; import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.types.Types.StructType; import org.apache.parquet.column.ColumnDescriptor; @@ -121,16 +120,9 @@ private boolean eval(MessageType fileSchema, BlockMetaData rowGroup, for (ColumnDescriptor desc : fileSchema.getColumns()) { PrimitiveType colType = fileSchema.getType(desc.getPath()).asPrimitiveType(); - if (colType.getId() != null) { - int id = colType.getId().intValue(); - cols.put(id, desc); - conversions.put(id, ParquetConversions.converterFromParquet(colType)); - } else { - MappedField field = nameMapping.find(colType.getName()); - if (field == null) { - continue; - } - int id = field.id(); + Integer id = ParquetSchemaUtil.getFieldId(nameMapping, colType); + + if (id != null) { cols.put(id, desc); conversions.put(id, ParquetConversions.converterFromParquet(colType)); } @@ -138,16 +130,9 @@ private boolean eval(MessageType fileSchema, BlockMetaData rowGroup, for (ColumnChunkMetaData meta : rowGroup.getColumns()) { PrimitiveType colType = fileSchema.getType(meta.getPath().toArray()).asPrimitiveType(); - if (colType.getId() != null) { - int id = colType.getId().intValue(); - isFallback.put(id, ParquetUtil.hasNonDictionaryPages(meta)); - mayContainNulls.put(id, mayContainNull(meta)); - } else { - MappedField field = nameMapping.find(colType.getName()); - if (field == null) { - continue; - } - int id = field.id(); + Integer id = ParquetSchemaUtil.getFieldId(nameMapping, colType); + + if (id != null) { isFallback.put(id, ParquetUtil.hasNonDictionaryPages(meta)); mayContainNulls.put(id, mayContainNull(meta)); } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java index c236115f532c..9c142df616f0 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java @@ -35,7 +35,6 @@ import org.apache.iceberg.expressions.ExpressionVisitors.BoundExpressionVisitor; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.expressions.Literal; -import org.apache.iceberg.mapping.MappedField; import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Type; @@ -118,17 +117,9 @@ private boolean eval(MessageType fileSchema, BlockMetaData rowGroup) { this.conversions = Maps.newHashMap(); for (ColumnChunkMetaData col : rowGroup.getColumns()) { PrimitiveType colType = fileSchema.getType(col.getPath().toArray()).asPrimitiveType(); - if (colType.getId() != null) { - int id = colType.getId().intValue(); - stats.put(id, col.getStatistics()); - valueCounts.put(id, col.getValueCount()); - conversions.put(id, ParquetConversions.converterFromParquet(colType)); - } else { - MappedField field = nameMapping.find(colType.getName()); - if (field == null) { - continue; - } - int id = field.id(); + Integer id = ParquetSchemaUtil.getFieldId(nameMapping, colType); + + if (id != null) { stats.put(id, col.getStatistics()); valueCounts.put(id, col.getValueCount()); conversions.put(id, ParquetConversions.converterFromParquet(colType)); diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java index 19f5faff2e81..872d7bbd5b9e 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java @@ -26,7 +26,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.Schema; import org.apache.iceberg.avro.AvroSchemaUtil; -import org.apache.iceberg.mapping.MappingUtil; +import org.apache.iceberg.mapping.NameMapping; import org.apache.parquet.avro.AvroReadSupport; import org.apache.parquet.hadoop.api.InitContext; import org.apache.parquet.hadoop.api.ReadSupport; @@ -42,11 +42,13 @@ class ParquetReadSupport extends ReadSupport { private final Schema expectedSchema; private final ReadSupport wrapped; private final boolean callInit; + private final NameMapping nameMapping; - ParquetReadSupport(Schema expectedSchema, ReadSupport readSupport, boolean callInit) { + ParquetReadSupport(Schema expectedSchema, ReadSupport readSupport, boolean callInit, NameMapping nameMapping) { this.expectedSchema = expectedSchema; this.wrapped = readSupport; this.callInit = callInit; + this.nameMapping = nameMapping; } @Override @@ -56,9 +58,16 @@ public ReadContext init(Configuration configuration, Map keyValu // matching to the file's columns by full path, so this must select columns by using the path // in the file's schema. - MessageType projection = ParquetSchemaUtil.hasIds(fileSchema) ? - ParquetSchemaUtil.pruneColumns(fileSchema, expectedSchema) : - ParquetSchemaUtil.pruneColumnsByName(fileSchema, expectedSchema, MappingUtil.create(expectedSchema)); + MessageType projection; + if (ParquetSchemaUtil.hasIds(fileSchema)) { + projection = ParquetSchemaUtil.pruneColumns(fileSchema, expectedSchema); + } else { + if (nameMapping != null) { + projection = ParquetSchemaUtil.pruneColumnsByName(fileSchema, expectedSchema, nameMapping); + } else { + projection = ParquetSchemaUtil.pruneColumnsFallback(fileSchema, expectedSchema); + } + } // override some known backward-compatibility options configuration.set("parquet.strict.typing", "false"); diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java index 2bb69ec8126e..90ea8c13fa79 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java @@ -22,10 +22,12 @@ import com.google.common.collect.Sets; import java.util.Set; import org.apache.iceberg.Schema; +import org.apache.iceberg.mapping.MappedField; import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; import org.apache.parquet.schema.Type; import org.apache.parquet.schema.Types.MessageTypeBuilder; @@ -51,6 +53,38 @@ public static MessageType pruneColumns(MessageType fileSchema, Schema expectedSc return (MessageType) ParquetTypeVisitor.visit(fileSchema, new PruneColumns(selectedIds)); } + /** + * Prunes columns from a Parquet file schema that was written without field ids. + *

+ * Files that were written without field ids are read assuming that schema evolution preserved + * column order. Deleting columns was not allowed. + *

+ * The order of columns in the resulting Parquet schema matches the Parquet file. + * + * @param fileSchema schema from a Parquet file that does not have field ids. + * @param expectedSchema expected schema + * @return a parquet schema pruned using the expected schema + */ + public static MessageType pruneColumnsFallback(MessageType fileSchema, Schema expectedSchema) { + Set selectedIds = Sets.newHashSet(); + + for (Types.NestedField field : expectedSchema.columns()) { + selectedIds.add(field.fieldId()); + } + + MessageTypeBuilder builder = org.apache.parquet.schema.Types.buildMessage(); + + int ordinal = 1; + for (Type type : fileSchema.getFields()) { + if (selectedIds.contains(ordinal)) { + builder.addField(type.withId(ordinal)); + } + ordinal += 1; + } + + return builder.named(fileSchema.getName()); + } + /** * Prunes columns from a Parquet file schema that was written without field ids. * The order of columns in the resulting Parquet schema matches the Parquet file. @@ -79,7 +113,7 @@ public static MessageType pruneColumnsByName(MessageType fileSchema, Schema expe public static boolean hasIds(MessageType fileSchema) { try { - // Try to convert the type to Iceberg. If an ID assignment is needed, return false. + // Try to convert the type to Iceberg. If all ID assignment is from nextId(), return false. ParquetTypeVisitor.visit(fileSchema, new MessageTypeToType(fileSchema) { @Override protected int getId(org.apache.parquet.schema.Type type) { @@ -92,11 +126,36 @@ protected int getId(org.apache.parquet.schema.Type type) { } }); - // all IDs are assigned from nextId() + // All IDs are assigned from nextId() return false; } catch (IllegalStateException e) { - // at least one field exists. + // At least one field exists. return true; } } + + public static MessageType addFallbackIds(MessageType fileSchema) { + MessageTypeBuilder builder = org.apache.parquet.schema.Types.buildMessage(); + + int ordinal = 1; // ids are assigned starting at 1 + for (Type type : fileSchema.getFields()) { + builder.addField(type.withId(ordinal)); + ordinal += 1; + } + + return builder.named(fileSchema.getName()); + } + + public static Integer getFieldId(NameMapping nameMapping, PrimitiveType colType) { + if (nameMapping != null) { + MappedField field = nameMapping.find(colType.getName()); + if (field == null) { + return null; + } + return field.id(); + } else if (colType.getId() != null) { + return colType.getId().intValue(); + } + return null; + } } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java index 6cae7ab326f5..d326619c6e2d 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java @@ -31,7 +31,6 @@ import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.io.InputFile; -import org.apache.iceberg.mapping.MappingUtil; import org.apache.iceberg.mapping.NameMapping; import org.apache.parquet.ParquetReadOptions; import org.apache.parquet.hadoop.ParquetFileReader; @@ -57,7 +56,6 @@ class ReadConf { private final long totalValues; private final boolean reuseContainers; private final Integer batchSize; - private final NameMapping nameMapping; // List of column chunk metadata for each row group private final List> columnChunkMetaDataForRowGroups; @@ -72,11 +70,16 @@ class ReadConf { this.reader = newReader(file, options); MessageType fileSchema = reader.getFileMetaData().getSchema(); boolean hasIds = ParquetSchemaUtil.hasIds(fileSchema); - this.nameMapping = nameMapping == null ? MappingUtil.create(expectedSchema) : nameMapping; - - this.projection = hasIds ? - ParquetSchemaUtil.pruneColumns(fileSchema, expectedSchema) : - ParquetSchemaUtil.pruneColumnsByName(fileSchema, expectedSchema, this.nameMapping); + MessageType typeWithIds = hasIds ? fileSchema : ParquetSchemaUtil.addFallbackIds(fileSchema); + if (!hasIds) { + if (nameMapping != null) { + this.projection = ParquetSchemaUtil.pruneColumnsByName(fileSchema, expectedSchema, nameMapping); + } else { + this.projection = ParquetSchemaUtil.pruneColumnsFallback(fileSchema, expectedSchema); + } + } else { + this.projection = ParquetSchemaUtil.pruneColumns(fileSchema, expectedSchema); + } this.rowGroups = reader.getRowGroups(); this.shouldSkip = new boolean[rowGroups.size()]; @@ -84,18 +87,21 @@ class ReadConf { ParquetMetricsRowGroupFilter statsFilter = null; ParquetDictionaryRowGroupFilter dictFilter = null; if (filter != null) { - statsFilter = new ParquetMetricsRowGroupFilter(expectedSchema, filter, caseSensitive) - .withNameMapping(this.nameMapping); - dictFilter = new ParquetDictionaryRowGroupFilter(expectedSchema, filter, caseSensitive) - .withNameMapping(this.nameMapping); + statsFilter = new ParquetMetricsRowGroupFilter(expectedSchema, filter, caseSensitive); + dictFilter = new ParquetDictionaryRowGroupFilter(expectedSchema, filter, caseSensitive); + + if (nameMapping != null) { + statsFilter = statsFilter.withNameMapping(nameMapping); + dictFilter = dictFilter.withNameMapping(nameMapping); + } } long computedTotalValues = 0L; for (int i = 0; i < shouldSkip.length; i += 1) { BlockMetaData rowGroup = rowGroups.get(i); boolean shouldRead = filter == null || ( - statsFilter.shouldRead(fileSchema, rowGroup) && - dictFilter.shouldRead(fileSchema, rowGroup, reader.getDictionaryReader(rowGroup))); + statsFilter.shouldRead(typeWithIds, rowGroup) && + dictFilter.shouldRead(typeWithIds, rowGroup, reader.getDictionaryReader(rowGroup))); this.shouldSkip[i] = !shouldRead; if (shouldRead) { computedTotalValues += rowGroup.getRowCount(); @@ -130,7 +136,6 @@ private ReadConf(ReadConf toCopy) { this.batchSize = toCopy.batchSize; this.vectorizedModel = toCopy.vectorizedModel; this.columnChunkMetaDataForRowGroups = toCopy.columnChunkMetaDataForRowGroups; - this.nameMapping = toCopy.nameMapping; } ParquetFileReader reader() { diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java b/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java index 8e240a1ce565..2356191dc1bd 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java @@ -45,6 +45,8 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.mapping.MappingUtil; +import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.orc.ORC; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.spark.SparkSchemaUtil; @@ -174,8 +176,10 @@ private CloseableIterable newParquetIterable( FileScanTask task, Schema readSchema, Map idToConstant) { + NameMapping nameMapping = MappingUtil.create(readSchema); return Parquet.read(location) .project(readSchema) + .withNameMapping(nameMapping) .split(task.start(), task.length()) .createReaderFunc(fileSchema -> SparkParquetReaders.buildReader(readSchema, fileSchema, idToConstant)) .filter(task.residual()) From d851ad520a3358b1156b057dfb41f9d3f39b2bb8 Mon Sep 17 00:00:00 2001 From: Junjie Chen Date: Thu, 9 Apr 2020 20:15:39 +0800 Subject: [PATCH 05/20] Add HasIds visitor to detect schema ID --- .../iceberg/parquet/MessageTypeToType.java | 4 +- .../iceberg/parquet/ParquetSchemaUtil.java | 63 +++++++++++++------ 2 files changed, 45 insertions(+), 22 deletions(-) diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/MessageTypeToType.java b/parquet/src/main/java/org/apache/iceberg/parquet/MessageTypeToType.java index 88a42cc2067b..f1cce22ee428 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/MessageTypeToType.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/MessageTypeToType.java @@ -229,13 +229,13 @@ private void addAlias(String name, int fieldId) { aliasToId.put(DOT.join(path(name)), fieldId); } - protected int nextId() { + private int nextId() { int current = nextId; nextId += 1; return current; } - protected int getId(org.apache.parquet.schema.Type type) { + private int getId(org.apache.parquet.schema.Type type) { org.apache.parquet.schema.Type.ID id = type.getId(); if (id != null) { return id.intValue(); diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java index 90ea8c13fa79..466f3d1498f4 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java @@ -20,12 +20,14 @@ package org.apache.iceberg.parquet; import com.google.common.collect.Sets; +import java.util.List; import java.util.Set; import org.apache.iceberg.Schema; import org.apache.iceberg.mapping.MappedField; import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; +import org.apache.parquet.schema.GroupType; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; import org.apache.parquet.schema.Type; @@ -112,26 +114,7 @@ public static MessageType pruneColumnsByName(MessageType fileSchema, Schema expe } public static boolean hasIds(MessageType fileSchema) { - try { - // Try to convert the type to Iceberg. If all ID assignment is from nextId(), return false. - ParquetTypeVisitor.visit(fileSchema, new MessageTypeToType(fileSchema) { - @Override - protected int getId(org.apache.parquet.schema.Type type) { - org.apache.parquet.schema.Type.ID id = type.getId(); - if (id != null) { - throw new IllegalStateException("at least one ID exists"); - } else { - return nextId(); - } - } - }); - - // All IDs are assigned from nextId() - return false; - } catch (IllegalStateException e) { - // At least one field exists. - return true; - } + return ParquetTypeVisitor.visit(fileSchema, new HasIds()); } public static MessageType addFallbackIds(MessageType fileSchema) { @@ -158,4 +141,44 @@ public static Integer getFieldId(NameMapping nameMapping, PrimitiveType colType) } return null; } + + public static class HasIds extends ParquetTypeVisitor { + @Override + public Boolean message(MessageType message, List fields) { + return struct(message, fields); + } + + @Override + public Boolean struct(GroupType struct, List hasIds) { + for (Boolean hasId : hasIds) { + if (hasId) { + return true; + } + } + return struct.getId() != null; + } + + @Override + public Boolean list(GroupType array, Boolean hasId) { + if (hasId) { + return true; + } else { + return array.getId() != null; + } + } + + @Override + public Boolean map(GroupType map, Boolean keyHasId, Boolean valueHasId) { + if (keyHasId || valueHasId) { + return true; + } else { + return map.getId() != null; + } + } + + @Override + public Boolean primitive(PrimitiveType primitive) { + return primitive.getId() != null; + } + } } From 19809366c8913658675df8dfca7314bf6b9d3456 Mon Sep 17 00:00:00 2001 From: Junjie Chen Date: Thu, 9 Apr 2020 21:14:08 +0800 Subject: [PATCH 06/20] fix coding style --- parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java | 2 +- .../org/apache/iceberg/parquet/VectorizedParquetReader.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java index c4037cb4a600..eba5c78122e4 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java @@ -428,7 +428,7 @@ public CloseableIterable build() { if (batchedReaderFunc != null) { return new VectorizedParquetReader(file, schema, options, batchedReaderFunc, nameMapping, filter, - reuseContainers, caseSensitive, maxRecordsPerBatch); + reuseContainers, caseSensitive, maxRecordsPerBatch); } else { return new org.apache.iceberg.parquet.ParquetReader<>( file, schema, options, readerFunc, nameMapping, filter, reuseContainers, caseSensitive); diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java b/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java index 19d1a2122b84..4e0b5af61b8a 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java @@ -73,8 +73,8 @@ public VectorizedParquetReader( private ReadConf init() { if (conf == null) { ReadConf readConf = new ReadConf( - input, options, expectedSchema, filter, null, batchReaderFunc, nameMapping, - reuseContainers, caseSensitive, batchSize); + input, options, expectedSchema, filter, null, batchReaderFunc, nameMapping, reuseContainers, + caseSensitive, batchSize); this.conf = readConf.copy(); return readConf; } From 7db31f5eeb6f304b669841814cd4b507bd72f1d6 Mon Sep 17 00:00:00 2001 From: jimmyjchen Date: Fri, 10 Apr 2020 11:09:17 +0800 Subject: [PATCH 07/20] use typeWithIds --- .../ParquetDictionaryRowGroupFilter.java | 40 ++++++++----------- .../parquet/ParquetMetricsRowGroupFilter.java | 39 +++++++----------- .../iceberg/parquet/ParquetSchemaUtil.java | 18 ++++----- .../org/apache/iceberg/parquet/ReadConf.java | 14 +++---- 4 files changed, 45 insertions(+), 66 deletions(-) diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java index b834b8c950c8..69f843b358ef 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java @@ -36,7 +36,6 @@ import org.apache.iceberg.expressions.ExpressionVisitors.BoundExpressionVisitor; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.expressions.Literal; -import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.types.Types.StructType; import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.column.Dictionary; @@ -46,12 +45,15 @@ import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class ParquetDictionaryRowGroupFilter { + private static final Logger LOG = LoggerFactory.getLogger(ParquetDictionaryRowGroupFilter.class); + private Expression expr; private Schema schema; private transient ThreadLocal visitors = null; - private NameMapping nameMapping = null; private boolean caseSensitive; private EvalVisitor visitor() { @@ -65,11 +67,6 @@ public ParquetDictionaryRowGroupFilter(Schema schema, Expression unbound) { this(schema, unbound, true); } - public ParquetDictionaryRowGroupFilter withNameMapping(NameMapping newNameMapping) { - this.nameMapping = newNameMapping; - return this; - } - public ParquetDictionaryRowGroupFilter(Schema schema, Expression unbound, boolean caseSensitive) { this.schema = schema; this.expr = unbound; @@ -85,14 +82,7 @@ public ParquetDictionaryRowGroupFilter(Schema schema, Expression unbound, boolea */ public boolean shouldRead(MessageType fileSchema, BlockMetaData rowGroup, DictionaryPageReadStore dictionaries) { - StructType struct; - - if (nameMapping != null) { - MessageType project = ParquetSchemaUtil.pruneColumnsByName(fileSchema, schema, nameMapping); - struct = ParquetSchemaUtil.convert(project).asStruct(); - } else { - struct = schema.asStruct(); - } + StructType struct = schema.asStruct(); this.expr = Binder.bind(struct, Expressions.rewriteNot(expr), caseSensitive); return visitor().eval(fileSchema, rowGroup, dictionaries); @@ -120,21 +110,23 @@ private boolean eval(MessageType fileSchema, BlockMetaData rowGroup, for (ColumnDescriptor desc : fileSchema.getColumns()) { PrimitiveType colType = fileSchema.getType(desc.getPath()).asPrimitiveType(); - Integer id = ParquetSchemaUtil.getFieldId(nameMapping, colType); - - if (id != null) { + if (colType.getId() != null) { + int id = colType.getId().intValue(); cols.put(id, desc); conversions.put(id, ParquetConversions.converterFromParquet(colType)); } } for (ColumnChunkMetaData meta : rowGroup.getColumns()) { - PrimitiveType colType = fileSchema.getType(meta.getPath().toArray()).asPrimitiveType(); - Integer id = ParquetSchemaUtil.getFieldId(nameMapping, colType); - - if (id != null) { - isFallback.put(id, ParquetUtil.hasNonDictionaryPages(meta)); - mayContainNulls.put(id, mayContainNull(meta)); + try { + PrimitiveType colType = fileSchema.getType(meta.getPath().toArray()).asPrimitiveType(); + if (colType.getId() != null) { + int id = colType.getId().intValue(); + isFallback.put(id, ParquetUtil.hasNonDictionaryPages(meta)); + mayContainNulls.put(id, mayContainNull(meta)); + } + } catch (org.apache.parquet.io.InvalidRecordException e) { + LOG.warn("Column {} not found in given schema.", meta.getPath().toDotString(), e); } } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java index 9c142df616f0..ef0f80926e7a 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java @@ -35,7 +35,6 @@ import org.apache.iceberg.expressions.ExpressionVisitors.BoundExpressionVisitor; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.expressions.Literal; -import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types.StructType; @@ -46,12 +45,15 @@ import org.apache.parquet.io.api.Binary; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class ParquetMetricsRowGroupFilter { + private static final Logger LOG = LoggerFactory.getLogger(ParquetMetricsRowGroupFilter.class); + private final Schema schema; private Expression expr; private transient ThreadLocal visitors = null; - private NameMapping nameMapping = null; private boolean caseSensitive; private MetricsEvalVisitor visitor() { @@ -71,12 +73,6 @@ public ParquetMetricsRowGroupFilter(Schema schema, Expression unbound, boolean c this.caseSensitive = caseSensitive; } - - public ParquetMetricsRowGroupFilter withNameMapping(NameMapping newNameMapping) { - this.nameMapping = newNameMapping; - return this; - } - /** * Test whether the file may contain records that match the expression. * @@ -85,15 +81,7 @@ public ParquetMetricsRowGroupFilter withNameMapping(NameMapping newNameMapping) * @return false if the file cannot contain rows that match the expression, true otherwise. */ public boolean shouldRead(MessageType fileSchema, BlockMetaData rowGroup) { - StructType struct; - - if (nameMapping != null) { - MessageType project = ParquetSchemaUtil.pruneColumnsByName(fileSchema, schema, nameMapping); - struct = ParquetSchemaUtil.convert(project).asStruct(); - } else { - struct = schema.asStruct(); - } - + StructType struct = schema.asStruct(); this.expr = Binder.bind(struct, Expressions.rewriteNot(expr), caseSensitive); return visitor().eval(fileSchema, rowGroup); @@ -116,13 +104,16 @@ private boolean eval(MessageType fileSchema, BlockMetaData rowGroup) { this.valueCounts = Maps.newHashMap(); this.conversions = Maps.newHashMap(); for (ColumnChunkMetaData col : rowGroup.getColumns()) { - PrimitiveType colType = fileSchema.getType(col.getPath().toArray()).asPrimitiveType(); - Integer id = ParquetSchemaUtil.getFieldId(nameMapping, colType); - - if (id != null) { - stats.put(id, col.getStatistics()); - valueCounts.put(id, col.getValueCount()); - conversions.put(id, ParquetConversions.converterFromParquet(colType)); + try { + PrimitiveType colType = fileSchema.getType(col.getPath().toArray()).asPrimitiveType(); + if (colType.getId() != null) { + int id = colType.getId().intValue(); + stats.put(id, col.getStatistics()); + valueCounts.put(id, col.getValueCount()); + conversions.put(id, ParquetConversions.converterFromParquet(colType)); + } + } catch (org.apache.parquet.io.InvalidRecordException e) { + LOG.warn("Column {} not found in given schema.", col.getPath().toDotString(), e); } } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java index 466f3d1498f4..5bc20316e510 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java @@ -23,7 +23,6 @@ import java.util.List; import java.util.Set; import org.apache.iceberg.Schema; -import org.apache.iceberg.mapping.MappedField; import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; @@ -129,17 +128,16 @@ public static MessageType addFallbackIds(MessageType fileSchema) { return builder.named(fileSchema.getName()); } - public static Integer getFieldId(NameMapping nameMapping, PrimitiveType colType) { - if (nameMapping != null) { - MappedField field = nameMapping.find(colType.getName()); - if (field == null) { - return null; + public static MessageType addFallbackIds(MessageType fileSchema, NameMapping nameMapping) { + MessageTypeBuilder builder = org.apache.parquet.schema.Types.buildMessage(); + + for (Type type : fileSchema.getFields()) { + if (nameMapping.find(type.getName()) != null) { + builder.addField(type.withId(nameMapping.find(type.getName()).id())); } - return field.id(); - } else if (colType.getId() != null) { - return colType.getId().intValue(); } - return null; + + return builder.named(fileSchema.getName()); } public static class HasIds extends ParquetTypeVisitor { diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java index d326619c6e2d..75d46df2d365 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java @@ -70,15 +70,18 @@ class ReadConf { this.reader = newReader(file, options); MessageType fileSchema = reader.getFileMetaData().getSchema(); boolean hasIds = ParquetSchemaUtil.hasIds(fileSchema); - MessageType typeWithIds = hasIds ? fileSchema : ParquetSchemaUtil.addFallbackIds(fileSchema); + MessageType typeWithIds; if (!hasIds) { if (nameMapping != null) { this.projection = ParquetSchemaUtil.pruneColumnsByName(fileSchema, expectedSchema, nameMapping); + typeWithIds = ParquetSchemaUtil.addFallbackIds(fileSchema, nameMapping); } else { this.projection = ParquetSchemaUtil.pruneColumnsFallback(fileSchema, expectedSchema); + typeWithIds = ParquetSchemaUtil.addFallbackIds(fileSchema); } } else { this.projection = ParquetSchemaUtil.pruneColumns(fileSchema, expectedSchema); + typeWithIds = fileSchema; } this.rowGroups = reader.getRowGroups(); @@ -89,11 +92,6 @@ class ReadConf { if (filter != null) { statsFilter = new ParquetMetricsRowGroupFilter(expectedSchema, filter, caseSensitive); dictFilter = new ParquetDictionaryRowGroupFilter(expectedSchema, filter, caseSensitive); - - if (nameMapping != null) { - statsFilter = statsFilter.withNameMapping(nameMapping); - dictFilter = dictFilter.withNameMapping(nameMapping); - } } long computedTotalValues = 0L; @@ -110,12 +108,12 @@ class ReadConf { this.totalValues = computedTotalValues; if (readerFunc != null) { - this.model = (ParquetValueReader) readerFunc.apply(fileSchema); + this.model = (ParquetValueReader) readerFunc.apply(typeWithIds); this.vectorizedModel = null; this.columnChunkMetaDataForRowGroups = null; } else { this.model = null; - this.vectorizedModel = (VectorizedReader) batchedReaderFunc.apply(fileSchema); + this.vectorizedModel = (VectorizedReader) batchedReaderFunc.apply(typeWithIds); this.columnChunkMetaDataForRowGroups = getColumnChunkMetadataForRowGroups(); } From c3145244890a4984cff80a98ba7e6ad823091df1 Mon Sep 17 00:00:00 2001 From: jimmyjchen Date: Sat, 11 Apr 2020 21:37:44 +0800 Subject: [PATCH 08/20] Use ParquetTypeVisitor to assign IDs, also add a unit test --- .../ParquetDictionaryRowGroupFilter.java | 12 +- .../parquet/ParquetMetricsRowGroupFilter.java | 10 +- .../iceberg/parquet/ParquetReadSupport.java | 3 +- .../iceberg/parquet/ParquetSchemaUtil.java | 143 ++++++++++++------ .../org/apache/iceberg/parquet/ReadConf.java | 16 +- .../parquet/TestParquetSchemaUtil.java | 95 ++++++++++++ 6 files changed, 206 insertions(+), 73 deletions(-) create mode 100644 parquet/src/test/java/org/apache/iceberg/parquet/TestParquetSchemaUtil.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java index 69f843b358ef..35e518d12877 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java @@ -51,10 +51,8 @@ public class ParquetDictionaryRowGroupFilter { private static final Logger LOG = LoggerFactory.getLogger(ParquetDictionaryRowGroupFilter.class); - private Expression expr; - private Schema schema; + private final Expression expr; private transient ThreadLocal visitors = null; - private boolean caseSensitive; private EvalVisitor visitor() { if (visitors == null) { @@ -68,9 +66,8 @@ public ParquetDictionaryRowGroupFilter(Schema schema, Expression unbound) { } public ParquetDictionaryRowGroupFilter(Schema schema, Expression unbound, boolean caseSensitive) { - this.schema = schema; - this.expr = unbound; - this.caseSensitive = caseSensitive; + StructType struct = schema.asStruct(); + this.expr = Binder.bind(struct, Expressions.rewriteNot(unbound), caseSensitive); } /** @@ -82,9 +79,6 @@ public ParquetDictionaryRowGroupFilter(Schema schema, Expression unbound, boolea */ public boolean shouldRead(MessageType fileSchema, BlockMetaData rowGroup, DictionaryPageReadStore dictionaries) { - StructType struct = schema.asStruct(); - - this.expr = Binder.bind(struct, Expressions.rewriteNot(expr), caseSensitive); return visitor().eval(fileSchema, rowGroup, dictionaries); } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java index ef0f80926e7a..8579f4371fc6 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java @@ -52,9 +52,8 @@ public class ParquetMetricsRowGroupFilter { private static final Logger LOG = LoggerFactory.getLogger(ParquetMetricsRowGroupFilter.class); private final Schema schema; - private Expression expr; + private final Expression expr; private transient ThreadLocal visitors = null; - private boolean caseSensitive; private MetricsEvalVisitor visitor() { if (visitors == null) { @@ -69,8 +68,8 @@ public ParquetMetricsRowGroupFilter(Schema schema, Expression unbound) { public ParquetMetricsRowGroupFilter(Schema schema, Expression unbound, boolean caseSensitive) { this.schema = schema; - this.expr = unbound; - this.caseSensitive = caseSensitive; + StructType struct = schema.asStruct(); + this.expr = Binder.bind(struct, Expressions.rewriteNot(unbound), caseSensitive); } /** @@ -81,9 +80,6 @@ public ParquetMetricsRowGroupFilter(Schema schema, Expression unbound, boolean c * @return false if the file cannot contain rows that match the expression, true otherwise. */ public boolean shouldRead(MessageType fileSchema, BlockMetaData rowGroup) { - StructType struct = schema.asStruct(); - this.expr = Binder.bind(struct, Expressions.rewriteNot(expr), caseSensitive); - return visitor().eval(fileSchema, rowGroup); } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java index 872d7bbd5b9e..b1469fe669a0 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java @@ -63,7 +63,8 @@ public ReadContext init(Configuration configuration, Map keyValu projection = ParquetSchemaUtil.pruneColumns(fileSchema, expectedSchema); } else { if (nameMapping != null) { - projection = ParquetSchemaUtil.pruneColumnsByName(fileSchema, expectedSchema, nameMapping); + MessageType typeWithIds = ParquetSchemaUtil.addFallbackIds(fileSchema, nameMapping); + projection = ParquetSchemaUtil.pruneColumns(typeWithIds, expectedSchema); } else { projection = ParquetSchemaUtil.pruneColumnsFallback(fileSchema, expectedSchema); } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java index 5bc20316e510..d37658faf22a 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java @@ -20,12 +20,17 @@ package org.apache.iceberg.parquet; import com.google.common.collect.Sets; +import java.util.Iterator; import java.util.List; +import java.util.Objects; import java.util.Set; +import java.util.stream.Collectors; import org.apache.iceberg.Schema; +import org.apache.iceberg.mapping.MappedField; import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; +import org.apache.parquet.Preconditions; import org.apache.parquet.schema.GroupType; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; @@ -86,58 +91,24 @@ public static MessageType pruneColumnsFallback(MessageType fileSchema, Schema ex return builder.named(fileSchema.getName()); } - /** - * Prunes columns from a Parquet file schema that was written without field ids. - * The order of columns in the resulting Parquet schema matches the Parquet file. - * - * @param fileSchema schema from a Parquet file that does not have field ids. - * @param expectedSchema expected schema - * @return a parquet schema pruned using the expected schema - */ - public static MessageType pruneColumnsByName(MessageType fileSchema, Schema expectedSchema, NameMapping nameMapping) { - Set selectedNames = Sets.newHashSet(); - - for (Types.NestedField field : expectedSchema.columns()) { - selectedNames.add(field.name()); - } - - MessageTypeBuilder builder = org.apache.parquet.schema.Types.buildMessage(); - - for (Type type : fileSchema.getFields()) { - if (selectedNames.contains(type.getName())) { - builder.addField(type.withId(nameMapping.find(type.getName()).id())); - } - } - - return builder.named(fileSchema.getName()); - } - public static boolean hasIds(MessageType fileSchema) { - return ParquetTypeVisitor.visit(fileSchema, new HasIds()); - } - - public static MessageType addFallbackIds(MessageType fileSchema) { - MessageTypeBuilder builder = org.apache.parquet.schema.Types.buildMessage(); - - int ordinal = 1; // ids are assigned starting at 1 - for (Type type : fileSchema.getFields()) { - builder.addField(type.withId(ordinal)); - ordinal += 1; - } - - return builder.named(fileSchema.getName()); + return ParquetTypeVisitor.visit(fileSchema, new HasIds(), true); } public static MessageType addFallbackIds(MessageType fileSchema, NameMapping nameMapping) { - MessageTypeBuilder builder = org.apache.parquet.schema.Types.buildMessage(); + if (nameMapping == null) { + MessageTypeBuilder builder = org.apache.parquet.schema.Types.buildMessage(); - for (Type type : fileSchema.getFields()) { - if (nameMapping.find(type.getName()) != null) { - builder.addField(type.withId(nameMapping.find(type.getName()).id())); + int ordinal = 1; // ids are assigned starting at 1 + for (Type type : fileSchema.getFields()) { + builder.addField(type.withId(ordinal)); + ordinal += 1; } - } - return builder.named(fileSchema.getName()); + return builder.named(fileSchema.getName()); + } else { + return (MessageType) ParquetTypeVisitor.visit(fileSchema, new AssignIdsByNameMapping(nameMapping), true); + } } public static class HasIds extends ParquetTypeVisitor { @@ -179,4 +150,86 @@ public Boolean primitive(PrimitiveType primitive) { return primitive.getId() != null; } } + + public static class AssignIdsByNameMapping extends ParquetTypeVisitor { + private final NameMapping nameMapping; + + public AssignIdsByNameMapping(NameMapping nameMapping) { + this.nameMapping = nameMapping; + } + + private String[] currentPath() { + String[] path = new String[fieldNames.size()]; + if (!fieldNames.isEmpty()) { + Iterator iter = fieldNames.descendingIterator(); + for (int i = 0; iter.hasNext(); i += 1) { + path[i] = iter.next(); + } + } + + return path; + } + + @Override + public Type message(MessageType message, List fields) { + MessageTypeBuilder builder = org.apache.parquet.schema.Types.buildMessage(); + fields.stream().filter(Objects::nonNull).forEach(builder::addField); + + return builder.named(message.getName()); + } + + @Override + public Type struct(GroupType struct, List types) { + MappedField field = nameMapping.find(currentPath()); + if (field == null) { + return null; + } + List actualTypes = types.stream().filter(Objects::nonNull).collect(Collectors.toList()); + + return struct.withNewFields(actualTypes).withId(field.id()); + } + + @Override + public Type list(GroupType list, Type elementType) { + MappedField field = nameMapping.find(currentPath()); + if (field == null) { + return null; + } + + Preconditions.checkArgument(elementType != null, + "List type must have element field"); + + return org.apache.parquet.schema.Types.list(list.getRepetition()) + .element(elementType) + .id(field.id()) + .named(list.getName()); + } + + @Override + public Type map(GroupType map, Type keyType, Type valueType) { + MappedField field = nameMapping.find(currentPath()); + if (field == null) { + return null; + } + + Preconditions.checkArgument(keyType != null && valueType != null, + "Map type must have both key field and value field"); + + return org.apache.parquet.schema.Types.map(map.getRepetition()) + .key(keyType) + .value(valueType) + .id(field.id()) + .named(map.getName()); + } + + @Override + public Type primitive(PrimitiveType primitive) { + MappedField field = nameMapping.find(currentPath()); + if (field == null) { + return null; + } else { + return primitive.withId(field.id()); + } + } + } } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java index 75d46df2d365..04914e1a8bf7 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java @@ -70,18 +70,12 @@ class ReadConf { this.reader = newReader(file, options); MessageType fileSchema = reader.getFileMetaData().getSchema(); boolean hasIds = ParquetSchemaUtil.hasIds(fileSchema); - MessageType typeWithIds; - if (!hasIds) { - if (nameMapping != null) { - this.projection = ParquetSchemaUtil.pruneColumnsByName(fileSchema, expectedSchema, nameMapping); - typeWithIds = ParquetSchemaUtil.addFallbackIds(fileSchema, nameMapping); - } else { - this.projection = ParquetSchemaUtil.pruneColumnsFallback(fileSchema, expectedSchema); - typeWithIds = ParquetSchemaUtil.addFallbackIds(fileSchema); - } + MessageType typeWithIds = hasIds ? fileSchema : ParquetSchemaUtil.addFallbackIds(fileSchema, nameMapping); + + if (nameMapping == null && !hasIds) { + this.projection = ParquetSchemaUtil.pruneColumnsFallback(fileSchema, expectedSchema); } else { - this.projection = ParquetSchemaUtil.pruneColumns(fileSchema, expectedSchema); - typeWithIds = fileSchema; + this.projection = ParquetSchemaUtil.pruneColumns(typeWithIds, expectedSchema); } this.rowGroups = reader.getRowGroups(); diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetSchemaUtil.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetSchemaUtil.java new file mode 100644 index 000000000000..49fbd16cb464 --- /dev/null +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetSchemaUtil.java @@ -0,0 +1,95 @@ +/* + * 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.iceberg.parquet; + +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.iceberg.Schema; +import org.apache.iceberg.mapping.MappingUtil; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; +import org.apache.parquet.schema.MessageType; +import org.junit.Assert; +import org.junit.Test; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; + +public class TestParquetSchemaUtil { + private static final Types.StructType SUPPORTED_PRIMITIVES = Types.StructType.of( + required(100, "id", Types.LongType.get()), + optional(101, "data", Types.StringType.get()), + required(102, "b", Types.BooleanType.get()), + optional(103, "i", Types.IntegerType.get()), + required(104, "l", Types.LongType.get()), + optional(105, "f", Types.FloatType.get()), + required(106, "d", Types.DoubleType.get()), + optional(107, "date", Types.DateType.get()), + required(108, "ts", Types.TimestampType.withZone()), + required(110, "s", Types.StringType.get()), + required(112, "fixed", Types.FixedType.ofLength(7)), + optional(113, "bytes", Types.BinaryType.get()), + required(114, "dec_9_0", Types.DecimalType.of(9, 0)), + required(115, "dec_11_2", Types.DecimalType.of(11, 2)), + required(116, "dec_38_10", Types.DecimalType.of(38, 10)) // spark's maximum precision + ); + + @Test + public void testAssignIdsByNameMapping() { + Types.StructType structType = Types.StructType.of( + required(0, "id", Types.LongType.get()), + optional(1, "list_of_maps", + Types.ListType.ofOptional(2, Types.MapType.ofOptional(3, 4, + Types.StringType.get(), + SUPPORTED_PRIMITIVES))), + optional(5, "map_of_lists", + Types.MapType.ofOptional(6, 7, + Types.StringType.get(), + Types.ListType.ofOptional(8, SUPPORTED_PRIMITIVES))), + required(9, "list_of_lists", + Types.ListType.ofOptional(10, Types.ListType.ofOptional(11, SUPPORTED_PRIMITIVES))), + required(12, "map_of_maps", + Types.MapType.ofOptional(13, 14, + Types.StringType.get(), + Types.MapType.ofOptional(15, 16, + Types.StringType.get(), + SUPPORTED_PRIMITIVES))), + required(17, "list_of_struct_of_nested_types", Types.ListType.ofOptional(19, Types.StructType.of( + Types.NestedField.required(20, "m1", Types.MapType.ofOptional(21, 22, + Types.StringType.get(), + SUPPORTED_PRIMITIVES)), + Types.NestedField.optional(23, "l1", Types.ListType.ofRequired(24, SUPPORTED_PRIMITIVES)), + Types.NestedField.required(25, "l2", Types.ListType.ofRequired(26, SUPPORTED_PRIMITIVES)), + Types.NestedField.optional(27, "m2", Types.MapType.ofOptional(28, 29, + Types.StringType.get(), + SUPPORTED_PRIMITIVES)) + ))) + ); + + Schema schema = new Schema(TypeUtil.assignFreshIds(structType, new AtomicInteger(0)::incrementAndGet) + .asStructType().fields()); + NameMapping nameMapping = MappingUtil.create(schema); + MessageType messageType = ParquetSchemaUtil.convert(schema, "complex_schema"); + MessageType typeWithIdsFromNameMapping = ParquetSchemaUtil.addFallbackIds(messageType, nameMapping); + Schema newSchema = ParquetSchemaUtil.convert(typeWithIdsFromNameMapping); + + Assert.assertEquals(schema.toString(), newSchema.toString()); + } +} From fd4c7d0444040510aad01aecd04364f8b0829f69 Mon Sep 17 00:00:00 2001 From: Junjie Chen Date: Tue, 5 May 2020 21:28:22 +0800 Subject: [PATCH 09/20] Address comments and rebase to master --- .../parquet/AssignIdsByNameMapping.java | 102 +++++++++++++++ .../iceberg/parquet/ParquetReadSupport.java | 11 +- .../iceberg/parquet/ParquetSchemaUtil.java | 122 ++---------------- .../org/apache/iceberg/parquet/ReadConf.java | 14 +- .../parquet/TestParquetSchemaUtil.java | 4 +- 5 files changed, 132 insertions(+), 121 deletions(-) create mode 100644 parquet/src/main/java/org/apache/iceberg/parquet/AssignIdsByNameMapping.java diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/AssignIdsByNameMapping.java b/parquet/src/main/java/org/apache/iceberg/parquet/AssignIdsByNameMapping.java new file mode 100644 index 000000000000..1a50841a4b41 --- /dev/null +++ b/parquet/src/main/java/org/apache/iceberg/parquet/AssignIdsByNameMapping.java @@ -0,0 +1,102 @@ +/* + * 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.iceberg.parquet; + +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; +import org.apache.iceberg.mapping.MappedField; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.parquet.Preconditions; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; +import org.apache.parquet.schema.Types; + +class AssignIdsByNameMapping extends ParquetTypeVisitor { + private final NameMapping nameMapping; + + AssignIdsByNameMapping(NameMapping nameMapping) { + this.nameMapping = nameMapping; + } + + @Override + public Type message(MessageType message, List fields) { + Types.MessageTypeBuilder builder = org.apache.parquet.schema.Types.buildMessage(); + fields.stream().filter(Objects::nonNull).forEach(builder::addField); + + return builder.named(message.getName()); + } + + @Override + public Type struct(GroupType struct, List types) { + MappedField field = nameMapping.find(currentPath()); + if (field == null) { + return null; + } + List actualTypes = types.stream().filter(Objects::nonNull).collect(Collectors.toList()); + + return struct.withNewFields(actualTypes).withId(field.id()); + } + + @Override + public Type list(GroupType list, Type elementType) { + MappedField field = nameMapping.find(currentPath()); + if (field == null) { + return null; + } + + Preconditions.checkArgument(elementType != null, + "List type must have element field"); + + return org.apache.parquet.schema.Types.list(list.getRepetition()) + .element(elementType) + .id(field.id()) + .named(list.getName()); + } + + @Override + public Type map(GroupType map, Type keyType, Type valueType) { + MappedField field = nameMapping.find(currentPath()); + if (field == null) { + return null; + } + + Preconditions.checkArgument(keyType != null && valueType != null, + "Map type must have both key field and value field"); + + return org.apache.parquet.schema.Types.map(map.getRepetition()) + .key(keyType) + .value(valueType) + .id(field.id()) + .named(map.getName()); + } + + @Override + public Type primitive(PrimitiveType primitive) { + MappedField field = nameMapping.find(currentPath()); + if (field == null) { + return null; + } else { + return primitive.withId(field.id()); + } + } +} diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java index b1469fe669a0..3bda012602a2 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java @@ -59,15 +59,14 @@ public ReadContext init(Configuration configuration, Map keyValu // in the file's schema. MessageType projection; + if (ParquetSchemaUtil.hasIds(fileSchema)) { projection = ParquetSchemaUtil.pruneColumns(fileSchema, expectedSchema); + } else if (nameMapping != null) { + MessageType typeWithIds = ParquetSchemaUtil.applyNameMapping(fileSchema, nameMapping); + projection = ParquetSchemaUtil.pruneColumns(typeWithIds, expectedSchema); } else { - if (nameMapping != null) { - MessageType typeWithIds = ParquetSchemaUtil.addFallbackIds(fileSchema, nameMapping); - projection = ParquetSchemaUtil.pruneColumns(typeWithIds, expectedSchema); - } else { - projection = ParquetSchemaUtil.pruneColumnsFallback(fileSchema, expectedSchema); - } + projection = ParquetSchemaUtil.pruneColumnsFallback(fileSchema, expectedSchema); } // override some known backward-compatibility options diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java index d37658faf22a..1c4b2934c414 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java @@ -20,17 +20,12 @@ package org.apache.iceberg.parquet; import com.google.common.collect.Sets; -import java.util.Iterator; import java.util.List; -import java.util.Objects; import java.util.Set; -import java.util.stream.Collectors; import org.apache.iceberg.Schema; -import org.apache.iceberg.mapping.MappedField; import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; -import org.apache.parquet.Preconditions; import org.apache.parquet.schema.GroupType; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; @@ -95,20 +90,20 @@ public static boolean hasIds(MessageType fileSchema) { return ParquetTypeVisitor.visit(fileSchema, new HasIds(), true); } - public static MessageType addFallbackIds(MessageType fileSchema, NameMapping nameMapping) { - if (nameMapping == null) { - MessageTypeBuilder builder = org.apache.parquet.schema.Types.buildMessage(); - - int ordinal = 1; // ids are assigned starting at 1 - for (Type type : fileSchema.getFields()) { - builder.addField(type.withId(ordinal)); - ordinal += 1; - } + public static MessageType addFallbackIds(MessageType fileSchema) { + MessageTypeBuilder builder = org.apache.parquet.schema.Types.buildMessage(); - return builder.named(fileSchema.getName()); - } else { - return (MessageType) ParquetTypeVisitor.visit(fileSchema, new AssignIdsByNameMapping(nameMapping), true); + int ordinal = 1; // ids are assigned starting at 1 + for (Type type : fileSchema.getFields()) { + builder.addField(type.withId(ordinal)); + ordinal += 1; } + + return builder.named(fileSchema.getName()); + } + + public static MessageType applyNameMapping(MessageType fileSchema, NameMapping nameMapping) { + return (MessageType) ParquetTypeVisitor.visit(fileSchema, new AssignIdsByNameMapping(nameMapping), true); } public static class HasIds extends ParquetTypeVisitor { @@ -129,20 +124,12 @@ public Boolean struct(GroupType struct, List hasIds) { @Override public Boolean list(GroupType array, Boolean hasId) { - if (hasId) { - return true; - } else { - return array.getId() != null; - } + return hasId || array.getId() != null; } @Override public Boolean map(GroupType map, Boolean keyHasId, Boolean valueHasId) { - if (keyHasId || valueHasId) { - return true; - } else { - return map.getId() != null; - } + return keyHasId || valueHasId || map.getId() != null; } @Override @@ -151,85 +138,4 @@ public Boolean primitive(PrimitiveType primitive) { } } - public static class AssignIdsByNameMapping extends ParquetTypeVisitor { - private final NameMapping nameMapping; - - public AssignIdsByNameMapping(NameMapping nameMapping) { - this.nameMapping = nameMapping; - } - - private String[] currentPath() { - String[] path = new String[fieldNames.size()]; - if (!fieldNames.isEmpty()) { - Iterator iter = fieldNames.descendingIterator(); - for (int i = 0; iter.hasNext(); i += 1) { - path[i] = iter.next(); - } - } - - return path; - } - - @Override - public Type message(MessageType message, List fields) { - MessageTypeBuilder builder = org.apache.parquet.schema.Types.buildMessage(); - fields.stream().filter(Objects::nonNull).forEach(builder::addField); - - return builder.named(message.getName()); - } - - @Override - public Type struct(GroupType struct, List types) { - MappedField field = nameMapping.find(currentPath()); - if (field == null) { - return null; - } - List actualTypes = types.stream().filter(Objects::nonNull).collect(Collectors.toList()); - - return struct.withNewFields(actualTypes).withId(field.id()); - } - - @Override - public Type list(GroupType list, Type elementType) { - MappedField field = nameMapping.find(currentPath()); - if (field == null) { - return null; - } - - Preconditions.checkArgument(elementType != null, - "List type must have element field"); - - return org.apache.parquet.schema.Types.list(list.getRepetition()) - .element(elementType) - .id(field.id()) - .named(list.getName()); - } - - @Override - public Type map(GroupType map, Type keyType, Type valueType) { - MappedField field = nameMapping.find(currentPath()); - if (field == null) { - return null; - } - - Preconditions.checkArgument(keyType != null && valueType != null, - "Map type must have both key field and value field"); - - return org.apache.parquet.schema.Types.map(map.getRepetition()) - .key(keyType) - .value(valueType) - .id(field.id()) - .named(map.getName()); - } - - @Override - public Type primitive(PrimitiveType primitive) { - MappedField field = nameMapping.find(currentPath()); - if (field == null) { - return null; - } else { - return primitive.withId(field.id()); - } - } - } } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java index 04914e1a8bf7..b0ae13149ce1 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java @@ -69,13 +69,17 @@ class ReadConf { this.options = options; this.reader = newReader(file, options); MessageType fileSchema = reader.getFileMetaData().getSchema(); - boolean hasIds = ParquetSchemaUtil.hasIds(fileSchema); - MessageType typeWithIds = hasIds ? fileSchema : ParquetSchemaUtil.addFallbackIds(fileSchema, nameMapping); - if (nameMapping == null && !hasIds) { - this.projection = ParquetSchemaUtil.pruneColumnsFallback(fileSchema, expectedSchema); - } else { + MessageType typeWithIds; + if (ParquetSchemaUtil.hasIds(fileSchema)) { + typeWithIds = fileSchema; + this.projection = ParquetSchemaUtil.pruneColumns(fileSchema, expectedSchema); + } else if (nameMapping != null) { + typeWithIds = ParquetSchemaUtil.applyNameMapping(fileSchema, nameMapping); this.projection = ParquetSchemaUtil.pruneColumns(typeWithIds, expectedSchema); + } else { + typeWithIds = ParquetSchemaUtil.addFallbackIds(fileSchema); + this.projection = ParquetSchemaUtil.pruneColumnsFallback(fileSchema, expectedSchema); } this.rowGroups = reader.getRowGroups(); diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetSchemaUtil.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetSchemaUtil.java index 49fbd16cb464..f61ca5ebff5c 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetSchemaUtil.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetSchemaUtil.java @@ -87,9 +87,9 @@ public void testAssignIdsByNameMapping() { .asStructType().fields()); NameMapping nameMapping = MappingUtil.create(schema); MessageType messageType = ParquetSchemaUtil.convert(schema, "complex_schema"); - MessageType typeWithIdsFromNameMapping = ParquetSchemaUtil.addFallbackIds(messageType, nameMapping); + MessageType typeWithIdsFromNameMapping = ParquetSchemaUtil.applyNameMapping(messageType, nameMapping); Schema newSchema = ParquetSchemaUtil.convert(typeWithIdsFromNameMapping); - Assert.assertEquals(schema.toString(), newSchema.toString()); + Assert.assertEquals(schema.asStruct(), newSchema.asStruct()); } } From 3323ae016246fa2353732b8340c44dca445ec97b Mon Sep 17 00:00:00 2001 From: Junjie Chen Date: Fri, 8 May 2020 16:42:01 +0800 Subject: [PATCH 10/20] address comments --- ...NameMapping.java => ApplyNameMapping.java} | 4 ++-- .../org/apache/iceberg/parquet/Parquet.java | 13 ++++++++--- .../iceberg/parquet/ParquetReadSupport.java | 1 - .../apache/iceberg/parquet/ParquetReader.java | 8 +++++-- .../iceberg/parquet/ParquetSchemaUtil.java | 2 +- .../org/apache/iceberg/parquet/ReadConf.java | 15 +++++++++---- .../parquet/VectorizedParquetReader.java | 8 ++++--- .../iceberg/spark/source/RowDataReader.java | 5 +---- .../spark/source/TestSparkTableUtil.java | 22 +++++++++---------- 9 files changed, 47 insertions(+), 31 deletions(-) rename parquet/src/main/java/org/apache/iceberg/parquet/{AssignIdsByNameMapping.java => ApplyNameMapping.java} (96%) diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/AssignIdsByNameMapping.java b/parquet/src/main/java/org/apache/iceberg/parquet/ApplyNameMapping.java similarity index 96% rename from parquet/src/main/java/org/apache/iceberg/parquet/AssignIdsByNameMapping.java rename to parquet/src/main/java/org/apache/iceberg/parquet/ApplyNameMapping.java index 1a50841a4b41..c77b65242c9e 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/AssignIdsByNameMapping.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ApplyNameMapping.java @@ -31,10 +31,10 @@ import org.apache.parquet.schema.Type; import org.apache.parquet.schema.Types; -class AssignIdsByNameMapping extends ParquetTypeVisitor { +class ApplyNameMapping extends ParquetTypeVisitor { private final NameMapping nameMapping; - AssignIdsByNameMapping(NameMapping nameMapping) { + ApplyNameMapping(NameMapping nameMapping) { this.nameMapping = nameMapping; } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java index eba5c78122e4..6151ed25735c 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java @@ -315,6 +315,7 @@ public static class ReadBuilder { private boolean reuseContainers = false; private int maxRecordsPerBatch = 10000; private NameMapping nameMapping = null; + private boolean applyNameMapping = false; private ReadBuilder(InputFile file) { this.file = file; @@ -401,6 +402,11 @@ public ReadBuilder withNameMapping(NameMapping newNameMapping) { return this; } + public ReadBuilder applyNameMapping() { + this.applyNameMapping = true; + return this; + } + @SuppressWarnings({"unchecked", "checkstyle:CyclomaticComplexity"}) public CloseableIterable build() { if (readerFunc != null || batchedReaderFunc != null) { @@ -427,11 +433,12 @@ public CloseableIterable build() { ParquetReadOptions options = optionsBuilder.build(); if (batchedReaderFunc != null) { - return new VectorizedParquetReader(file, schema, options, batchedReaderFunc, nameMapping, filter, - reuseContainers, caseSensitive, maxRecordsPerBatch); + return new VectorizedParquetReader(file, schema, options, batchedReaderFunc, nameMapping, + applyNameMapping, filter, reuseContainers, caseSensitive, maxRecordsPerBatch); } else { return new org.apache.iceberg.parquet.ParquetReader<>( - file, schema, options, readerFunc, nameMapping, filter, reuseContainers, caseSensitive); + file, schema, options, readerFunc, nameMapping, applyNameMapping, filter, reuseContainers, + caseSensitive); } } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java index 3bda012602a2..12009ab0b36f 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java @@ -59,7 +59,6 @@ public ReadContext init(Configuration configuration, Map keyValu // in the file's schema. MessageType projection; - if (ParquetSchemaUtil.hasIds(fileSchema)) { projection = ParquetSchemaUtil.pruneColumns(fileSchema, expectedSchema); } else if (nameMapping != null) { diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java index 51a65f000289..83426442a641 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java @@ -45,10 +45,12 @@ public class ParquetReader extends CloseableGroup implements CloseableIterabl private final boolean reuseContainers; private final boolean caseSensitive; private final NameMapping nameMapping; + private final boolean applyNameMapping; public ParquetReader(InputFile input, Schema expectedSchema, ParquetReadOptions options, Function> readerFunc, NameMapping nameMapping, - Expression filter, boolean reuseContainers, boolean caseSensitive) { + boolean applyNameMapping, Expression filter, boolean reuseContainers, + boolean caseSensitive) { this.input = input; this.expectedSchema = expectedSchema; this.options = options; @@ -58,6 +60,7 @@ public ParquetReader(InputFile input, Schema expectedSchema, ParquetReadOptions this.reuseContainers = reuseContainers; this.caseSensitive = caseSensitive; this.nameMapping = nameMapping; + this.applyNameMapping = applyNameMapping; } private ReadConf conf = null; @@ -65,7 +68,8 @@ public ParquetReader(InputFile input, Schema expectedSchema, ParquetReadOptions private ReadConf init() { if (conf == null) { ReadConf readConf = new ReadConf<>( - input, options, expectedSchema, filter, readerFunc, null, nameMapping, reuseContainers, caseSensitive, null); + input, options, expectedSchema, filter, readerFunc, null, nameMapping, + applyNameMapping, reuseContainers, caseSensitive, null); this.conf = readConf.copy(); return readConf; } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java index 1c4b2934c414..41a60cf794fa 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java @@ -103,7 +103,7 @@ public static MessageType addFallbackIds(MessageType fileSchema) { } public static MessageType applyNameMapping(MessageType fileSchema, NameMapping nameMapping) { - return (MessageType) ParquetTypeVisitor.visit(fileSchema, new AssignIdsByNameMapping(nameMapping), true); + return (MessageType) ParquetTypeVisitor.visit(fileSchema, new ApplyNameMapping(nameMapping), true); } public static class HasIds extends ParquetTypeVisitor { diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java index b0ae13149ce1..3a217f1152da 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java @@ -31,6 +31,7 @@ import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.mapping.MappingUtil; import org.apache.iceberg.mapping.NameMapping; import org.apache.parquet.ParquetReadOptions; import org.apache.parquet.hadoop.ParquetFileReader; @@ -63,8 +64,8 @@ class ReadConf { @SuppressWarnings("unchecked") ReadConf(InputFile file, ParquetReadOptions options, Schema expectedSchema, Expression filter, Function> readerFunc, Function> batchedReaderFunc, NameMapping nameMapping, boolean reuseContainers, - boolean caseSensitive, Integer bSize) { + VectorizedReader> batchedReaderFunc, NameMapping nameMapping, boolean applyNameMapping, + boolean reuseContainers, boolean caseSensitive, Integer bSize) { this.file = file; this.options = options; this.reader = newReader(file, options); @@ -74,8 +75,14 @@ class ReadConf { if (ParquetSchemaUtil.hasIds(fileSchema)) { typeWithIds = fileSchema; this.projection = ParquetSchemaUtil.pruneColumns(fileSchema, expectedSchema); - } else if (nameMapping != null) { - typeWithIds = ParquetSchemaUtil.applyNameMapping(fileSchema, nameMapping); + } else if (applyNameMapping) { + NameMapping actualNameMapping; + if (nameMapping == null) { + actualNameMapping = MappingUtil.create(expectedSchema); + } else { + actualNameMapping = nameMapping; + } + typeWithIds = ParquetSchemaUtil.applyNameMapping(fileSchema, actualNameMapping); this.projection = ParquetSchemaUtil.pruneColumns(typeWithIds, expectedSchema); } else { typeWithIds = ParquetSchemaUtil.addFallbackIds(fileSchema); diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java b/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java index 4e0b5af61b8a..b3fcb2f0dc7a 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java @@ -51,10 +51,11 @@ public class VectorizedParquetReader extends CloseableGroup implements Closea private final boolean caseSensitive; private final int batchSize; private final NameMapping nameMapping; + private final boolean applyNameMapping; public VectorizedParquetReader( InputFile input, Schema expectedSchema, ParquetReadOptions options, - Function> readerFunc, NameMapping nameMapping, + Function> readerFunc, NameMapping nameMapping, boolean applyNameMapping, Expression filter, boolean reuseContainers, boolean caseSensitive, int maxRecordsPerBatch) { this.input = input; this.expectedSchema = expectedSchema; @@ -66,6 +67,7 @@ public VectorizedParquetReader( this.caseSensitive = caseSensitive; this.batchSize = maxRecordsPerBatch; this.nameMapping = nameMapping; + this.applyNameMapping = applyNameMapping; } private ReadConf conf = null; @@ -73,8 +75,8 @@ public VectorizedParquetReader( private ReadConf init() { if (conf == null) { ReadConf readConf = new ReadConf( - input, options, expectedSchema, filter, null, batchReaderFunc, nameMapping, reuseContainers, - caseSensitive, batchSize); + input, options, expectedSchema, filter, null, batchReaderFunc, nameMapping, applyNameMapping, + reuseContainers, caseSensitive, batchSize); this.conf = readConf.copy(); return readConf; } diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java b/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java index 2356191dc1bd..ac3472d6be52 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java @@ -45,8 +45,6 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.InputFile; -import org.apache.iceberg.mapping.MappingUtil; -import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.orc.ORC; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.spark.SparkSchemaUtil; @@ -176,10 +174,9 @@ private CloseableIterable newParquetIterable( FileScanTask task, Schema readSchema, Map idToConstant) { - NameMapping nameMapping = MappingUtil.create(readSchema); return Parquet.read(location) .project(readSchema) - .withNameMapping(nameMapping) + .applyNameMapping() .split(task.start(), task.length()) .createReaderFunc(fileSchema -> SparkParquetReaders.buildReader(readSchema, fileSchema, idToConstant)) .filter(task.residual()) diff --git a/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTableUtil.java b/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTableUtil.java index 5635a92f9a8b..e08a0120193e 100644 --- a/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTableUtil.java +++ b/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTableUtil.java @@ -207,13 +207,13 @@ public void testImportAsHiveTable() throws Exception { } @Test - public void testImportWithIncompatibleSchema() throws Exception { + public void testImportWithNameMapping() throws Exception { spark.table(qualifiedTableName).write().mode("overwrite").format("parquet") .saveAsTable("original_table"); // The field is different so that it will project with name mapping Schema filteredSchema = new Schema( - optional(1, "data", Types.StringType.get()) + optional(1, "data", Types.StringType.get()) ); TableIdentifier source = new TableIdentifier("original_table"); @@ -227,17 +227,17 @@ public void testImportWithIncompatibleSchema() throws Exception { // The filter invoke the metric/dictionary row group filter in which it project schema // with name mapping again to match the metric read from footer. List actual = spark.read().format("iceberg").load(DB_NAME + ".target_table") - .select("data") - .sort("data") - .filter("data<'c'") - .collectAsList() - .stream() - .map(r -> r.getString(0)) - .collect(Collectors.toList()); + .select("data") + .sort("data") + .filter("data<'c'") + .collectAsList() + .stream() + .map(r -> r.getString(0)) + .collect(Collectors.toList()); List expected = Lists.newArrayList( - new SimpleRecord(2, "a"), - new SimpleRecord(1, "b") + new SimpleRecord(2, "a"), + new SimpleRecord(1, "b") ); Assert.assertEquals(expected.stream().map(SimpleRecord::getData).collect(Collectors.toList()), actual); From d718765d849f91a5910f16ba7cfb72d454026eda Mon Sep 17 00:00:00 2001 From: Junjie Chen Date: Fri, 8 May 2020 17:06:35 +0800 Subject: [PATCH 11/20] update ApplyNameMapping implementation --- .../iceberg/parquet/ApplyNameMapping.java | 34 ++++++------------- 1 file changed, 11 insertions(+), 23 deletions(-) diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ApplyNameMapping.java b/parquet/src/main/java/org/apache/iceberg/parquet/ApplyNameMapping.java index c77b65242c9e..4fd979abaea8 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ApplyNameMapping.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ApplyNameMapping.java @@ -49,54 +49,42 @@ public Type message(MessageType message, List fields) { @Override public Type struct(GroupType struct, List types) { MappedField field = nameMapping.find(currentPath()); - if (field == null) { - return null; - } List actualTypes = types.stream().filter(Objects::nonNull).collect(Collectors.toList()); + Type structType = struct.withNewFields(actualTypes); - return struct.withNewFields(actualTypes).withId(field.id()); + return field == null ? structType : structType.withId(field.id()); } @Override public Type list(GroupType list, Type elementType) { - MappedField field = nameMapping.find(currentPath()); - if (field == null) { - return null; - } - Preconditions.checkArgument(elementType != null, "List type must have element field"); - return org.apache.parquet.schema.Types.list(list.getRepetition()) + MappedField field = nameMapping.find(currentPath()); + Type listType = org.apache.parquet.schema.Types.list(list.getRepetition()) .element(elementType) - .id(field.id()) .named(list.getName()); + + return field == null ? listType : listType.withId(field.id()); } @Override public Type map(GroupType map, Type keyType, Type valueType) { - MappedField field = nameMapping.find(currentPath()); - if (field == null) { - return null; - } - Preconditions.checkArgument(keyType != null && valueType != null, "Map type must have both key field and value field"); - return org.apache.parquet.schema.Types.map(map.getRepetition()) + MappedField field = nameMapping.find(currentPath()); + Type mapType = org.apache.parquet.schema.Types.map(map.getRepetition()) .key(keyType) .value(valueType) - .id(field.id()) .named(map.getName()); + + return field == null ? mapType : mapType.withId(field.id()); } @Override public Type primitive(PrimitiveType primitive) { MappedField field = nameMapping.find(currentPath()); - if (field == null) { - return null; - } else { - return primitive.withId(field.id()); - } + return field == null ? null : primitive.withId(field.id()); } } From a12f430b02bda993c19c3b9f20cdd4d8a6ff0d37 Mon Sep 17 00:00:00 2001 From: Junjie Chen Date: Thu, 14 May 2020 10:16:42 +0800 Subject: [PATCH 12/20] update ApplyNameMapping to use latest ParquetTypeVisito --- .../apache/iceberg/parquet/ApplyNameMapping.java | 16 ++++++++++++++++ .../iceberg/parquet/ParquetSchemaUtil.java | 4 ++-- 2 files changed, 18 insertions(+), 2 deletions(-) diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ApplyNameMapping.java b/parquet/src/main/java/org/apache/iceberg/parquet/ApplyNameMapping.java index 4fd979abaea8..17732e8930c2 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ApplyNameMapping.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ApplyNameMapping.java @@ -87,4 +87,20 @@ public Type primitive(PrimitiveType primitive) { MappedField field = nameMapping.find(currentPath()); return field == null ? null : primitive.withId(field.id()); } + + @Override + public void beforeRepeatedElement(Type element) { + } + + @Override + public void afterRepeatedElement(Type element) { + } + + @Override + public void beforeRepeatedKeyValue(Type keyValue) { + } + + @Override + public void afterRepeatedKeyValue(Type keyValue) { + } } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java index 41a60cf794fa..4d72a683dcc0 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java @@ -87,7 +87,7 @@ public static MessageType pruneColumnsFallback(MessageType fileSchema, Schema ex } public static boolean hasIds(MessageType fileSchema) { - return ParquetTypeVisitor.visit(fileSchema, new HasIds(), true); + return ParquetTypeVisitor.visit(fileSchema, new HasIds()); } public static MessageType addFallbackIds(MessageType fileSchema) { @@ -103,7 +103,7 @@ public static MessageType addFallbackIds(MessageType fileSchema) { } public static MessageType applyNameMapping(MessageType fileSchema, NameMapping nameMapping) { - return (MessageType) ParquetTypeVisitor.visit(fileSchema, new ApplyNameMapping(nameMapping), true); + return (MessageType) ParquetTypeVisitor.visit(fileSchema, new ApplyNameMapping(nameMapping)); } public static class HasIds extends ParquetTypeVisitor { From 7d2a1ddbaafe7e82dac8d355c6cb90e446f2e8bb Mon Sep 17 00:00:00 2001 From: Junjie Chen Date: Fri, 22 May 2020 14:00:10 +0800 Subject: [PATCH 13/20] address comments --- .../src/main/java/org/apache/iceberg/parquet/Parquet.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java index 6151ed25735c..6ca56dbc455c 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java @@ -41,7 +41,6 @@ import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; -import org.apache.iceberg.mapping.MappingUtil; import org.apache.iceberg.mapping.NameMapping; import org.apache.parquet.HadoopReadOptions; import org.apache.parquet.ParquetReadOptions; @@ -444,7 +443,7 @@ public CloseableIterable build() { ParquetReadBuilder builder = new ParquetReadBuilder<>(ParquetIO.file(file)); - builder.project(schema).withNameMapping(MappingUtil.create(schema)); + builder.project(schema); if (readSupport != null) { builder.readSupport((ReadSupport) readSupport); @@ -490,6 +489,10 @@ public CloseableIterable build() { builder.withFileRange(start, start + length); } + if (nameMapping != null) { + builder.withNameMapping(nameMapping); + } + return new ParquetIterable<>(builder); } } From 0c8b3da646578e9ccd5a2da33fa0d88f8b141d14 Mon Sep 17 00:00:00 2001 From: Junjie Chen Date: Fri, 22 May 2020 14:00:33 +0800 Subject: [PATCH 14/20] revert ApplyNameMapping to ignore field without ID --- .../java/org/apache/iceberg/parquet/ApplyNameMapping.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ApplyNameMapping.java b/parquet/src/main/java/org/apache/iceberg/parquet/ApplyNameMapping.java index 17732e8930c2..f0ce677753ee 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ApplyNameMapping.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ApplyNameMapping.java @@ -52,7 +52,7 @@ public Type struct(GroupType struct, List types) { List actualTypes = types.stream().filter(Objects::nonNull).collect(Collectors.toList()); Type structType = struct.withNewFields(actualTypes); - return field == null ? structType : structType.withId(field.id()); + return field == null ? null : structType.withId(field.id()); } @Override @@ -65,7 +65,7 @@ public Type list(GroupType list, Type elementType) { .element(elementType) .named(list.getName()); - return field == null ? listType : listType.withId(field.id()); + return field == null ? null : listType.withId(field.id()); } @Override @@ -79,7 +79,7 @@ public Type map(GroupType map, Type keyType, Type valueType) { .value(valueType) .named(map.getName()); - return field == null ? mapType : mapType.withId(field.id()); + return field == null ? null : mapType.withId(field.id()); } @Override From 4baa9003ad428ee33e83e7b607018cdc044ff306 Mon Sep 17 00:00:00 2001 From: Junjie Chen Date: Sat, 23 May 2020 13:53:08 +0800 Subject: [PATCH 15/20] get name mapping from table property --- .../org/apache/iceberg/parquet/Parquet.java | 13 +++--------- .../apache/iceberg/parquet/ParquetReader.java | 9 +++----- .../org/apache/iceberg/parquet/ReadConf.java | 15 ++++--------- .../parquet/VectorizedParquetReader.java | 12 +++++------ .../apache/iceberg/spark/source/Reader.java | 21 ++++++++++++++++--- .../iceberg/spark/source/RowDataReader.java | 15 +++++++------ .../spark/source/TestSparkTableUtil.java | 11 ++++++++++ 7 files changed, 53 insertions(+), 43 deletions(-) diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java index 6ca56dbc455c..646a82449c8c 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java @@ -314,7 +314,6 @@ public static class ReadBuilder { private boolean reuseContainers = false; private int maxRecordsPerBatch = 10000; private NameMapping nameMapping = null; - private boolean applyNameMapping = false; private ReadBuilder(InputFile file) { this.file = file; @@ -401,11 +400,6 @@ public ReadBuilder withNameMapping(NameMapping newNameMapping) { return this; } - public ReadBuilder applyNameMapping() { - this.applyNameMapping = true; - return this; - } - @SuppressWarnings({"unchecked", "checkstyle:CyclomaticComplexity"}) public CloseableIterable build() { if (readerFunc != null || batchedReaderFunc != null) { @@ -432,12 +426,11 @@ public CloseableIterable build() { ParquetReadOptions options = optionsBuilder.build(); if (batchedReaderFunc != null) { - return new VectorizedParquetReader(file, schema, options, batchedReaderFunc, nameMapping, - applyNameMapping, filter, reuseContainers, caseSensitive, maxRecordsPerBatch); + return new VectorizedParquetReader(file, schema, options, batchedReaderFunc, nameMapping, filter, + reuseContainers, caseSensitive, maxRecordsPerBatch); } else { return new org.apache.iceberg.parquet.ParquetReader<>( - file, schema, options, readerFunc, nameMapping, applyNameMapping, filter, reuseContainers, - caseSensitive); + file, schema, options, readerFunc, nameMapping, filter, reuseContainers, caseSensitive); } } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java index 83426442a641..5a7445b0a703 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReader.java @@ -45,12 +45,10 @@ public class ParquetReader extends CloseableGroup implements CloseableIterabl private final boolean reuseContainers; private final boolean caseSensitive; private final NameMapping nameMapping; - private final boolean applyNameMapping; public ParquetReader(InputFile input, Schema expectedSchema, ParquetReadOptions options, Function> readerFunc, NameMapping nameMapping, - boolean applyNameMapping, Expression filter, boolean reuseContainers, - boolean caseSensitive) { + Expression filter, boolean reuseContainers, boolean caseSensitive) { this.input = input; this.expectedSchema = expectedSchema; this.options = options; @@ -60,7 +58,6 @@ public ParquetReader(InputFile input, Schema expectedSchema, ParquetReadOptions this.reuseContainers = reuseContainers; this.caseSensitive = caseSensitive; this.nameMapping = nameMapping; - this.applyNameMapping = applyNameMapping; } private ReadConf conf = null; @@ -68,8 +65,8 @@ public ParquetReader(InputFile input, Schema expectedSchema, ParquetReadOptions private ReadConf init() { if (conf == null) { ReadConf readConf = new ReadConf<>( - input, options, expectedSchema, filter, readerFunc, null, nameMapping, - applyNameMapping, reuseContainers, caseSensitive, null); + input, options, expectedSchema, filter, readerFunc, null, nameMapping, reuseContainers, + caseSensitive, null); this.conf = readConf.copy(); return readConf; } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java index 3a217f1152da..b0ae13149ce1 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ReadConf.java @@ -31,7 +31,6 @@ import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.io.InputFile; -import org.apache.iceberg.mapping.MappingUtil; import org.apache.iceberg.mapping.NameMapping; import org.apache.parquet.ParquetReadOptions; import org.apache.parquet.hadoop.ParquetFileReader; @@ -64,8 +63,8 @@ class ReadConf { @SuppressWarnings("unchecked") ReadConf(InputFile file, ParquetReadOptions options, Schema expectedSchema, Expression filter, Function> readerFunc, Function> batchedReaderFunc, NameMapping nameMapping, boolean applyNameMapping, - boolean reuseContainers, boolean caseSensitive, Integer bSize) { + VectorizedReader> batchedReaderFunc, NameMapping nameMapping, boolean reuseContainers, + boolean caseSensitive, Integer bSize) { this.file = file; this.options = options; this.reader = newReader(file, options); @@ -75,14 +74,8 @@ class ReadConf { if (ParquetSchemaUtil.hasIds(fileSchema)) { typeWithIds = fileSchema; this.projection = ParquetSchemaUtil.pruneColumns(fileSchema, expectedSchema); - } else if (applyNameMapping) { - NameMapping actualNameMapping; - if (nameMapping == null) { - actualNameMapping = MappingUtil.create(expectedSchema); - } else { - actualNameMapping = nameMapping; - } - typeWithIds = ParquetSchemaUtil.applyNameMapping(fileSchema, actualNameMapping); + } else if (nameMapping != null) { + typeWithIds = ParquetSchemaUtil.applyNameMapping(fileSchema, nameMapping); this.projection = ParquetSchemaUtil.pruneColumns(typeWithIds, expectedSchema); } else { typeWithIds = ParquetSchemaUtil.addFallbackIds(fileSchema); diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java b/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java index b3fcb2f0dc7a..c886aa0058b4 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java @@ -51,12 +51,11 @@ public class VectorizedParquetReader extends CloseableGroup implements Closea private final boolean caseSensitive; private final int batchSize; private final NameMapping nameMapping; - private final boolean applyNameMapping; public VectorizedParquetReader( - InputFile input, Schema expectedSchema, ParquetReadOptions options, - Function> readerFunc, NameMapping nameMapping, boolean applyNameMapping, - Expression filter, boolean reuseContainers, boolean caseSensitive, int maxRecordsPerBatch) { + InputFile input, Schema expectedSchema, ParquetReadOptions options, Function> readerFunc, NameMapping nameMapping, Expression filter, boolean reuseContainers, + boolean caseSensitive, int maxRecordsPerBatch) { this.input = input; this.expectedSchema = expectedSchema; this.options = options; @@ -67,7 +66,6 @@ public VectorizedParquetReader( this.caseSensitive = caseSensitive; this.batchSize = maxRecordsPerBatch; this.nameMapping = nameMapping; - this.applyNameMapping = applyNameMapping; } private ReadConf conf = null; @@ -75,8 +73,8 @@ public VectorizedParquetReader( private ReadConf init() { if (conf == null) { ReadConf readConf = new ReadConf( - input, options, expectedSchema, filter, null, batchReaderFunc, nameMapping, applyNameMapping, - reuseContainers, caseSensitive, batchSize); + input, options, expectedSchema, filter, null, batchReaderFunc, nameMapping, reuseContainers, + caseSensitive, batchSize); this.conf = readConf.copy(); return readConf; } diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java b/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java index 51c0572f0b46..04fa98aeae16 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java @@ -44,6 +44,8 @@ import org.apache.iceberg.hadoop.Util; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.mapping.NameMappingParser; import org.apache.iceberg.spark.SparkFilters; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.spark.broadcast.Broadcast; @@ -64,6 +66,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.iceberg.TableProperties.DEFAULT_NAME_MAPPING; + class Reader implements DataSourceReader, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics { private static final Logger LOG = LoggerFactory.getLogger(Reader.class); @@ -177,11 +181,12 @@ public StructType readSchema() { public List> planInputPartitions() { String tableSchemaString = SchemaParser.toJson(table.schema()); String expectedSchemaString = SchemaParser.toJson(lazySchema()); + String nameMappingString = table.properties().get(DEFAULT_NAME_MAPPING); List> readTasks = Lists.newArrayList(); for (CombinedScanTask task : tasks()) { readTasks.add( - new ReadTask(task, tableSchemaString, expectedSchemaString, io, encryptionManager, + new ReadTask(task, tableSchemaString, expectedSchemaString, nameMappingString, io, encryptionManager, caseSensitive, localityPreferred)); } @@ -306,6 +311,7 @@ private static class ReadTask implements InputPartition, Serializab private final CombinedScanTask task; private final String tableSchemaString; private final String expectedSchemaString; + private final String nameMappingString; private final Broadcast io; private final Broadcast encryptionManager; private final boolean caseSensitive; @@ -313,10 +319,11 @@ private static class ReadTask implements InputPartition, Serializab private transient Schema tableSchema = null; private transient Schema expectedSchema = null; + private transient NameMapping nameMapping = null; private transient String[] preferredLocations; private ReadTask(CombinedScanTask task, String tableSchemaString, String expectedSchemaString, - Broadcast io, Broadcast encryptionManager, + String nameMappingString, Broadcast io, Broadcast encryptionManager, boolean caseSensitive, boolean localityPreferred) { this.task = task; this.tableSchemaString = tableSchemaString; @@ -326,11 +333,12 @@ private ReadTask(CombinedScanTask task, String tableSchemaString, String expecte this.caseSensitive = caseSensitive; this.localityPreferred = localityPreferred; this.preferredLocations = getPreferredLocations(); + this.nameMappingString = nameMappingString; } @Override public InputPartitionReader createPartitionReader() { - return new RowDataReader(task, lazyTableSchema(), lazyExpectedSchema(), io.value(), + return new RowDataReader(task, lazyTableSchema(), lazyExpectedSchema(), lazyNameMapping(), io.value(), encryptionManager.value(), caseSensitive); } @@ -353,6 +361,13 @@ private Schema lazyExpectedSchema() { return expectedSchema; } + private NameMapping lazyNameMapping() { + if (nameMapping == null && nameMappingString != null) { + this.nameMapping = NameMappingParser.fromJson(nameMappingString); + } + return nameMapping; + } + private String[] getPreferredLocations() { if (!localityPreferred) { return new String[0]; diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java b/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java index ac3472d6be52..34a16438fbe7 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java @@ -45,6 +45,7 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.orc.ORC; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.spark.SparkSchemaUtil; @@ -76,14 +77,16 @@ class RowDataReader extends BaseDataReader { private final Schema tableSchema; private final Schema expectedSchema; + private final NameMapping nameMapping; private final boolean caseSensitive; RowDataReader( - CombinedScanTask task, Schema tableSchema, Schema expectedSchema, FileIO fileIo, + CombinedScanTask task, Schema tableSchema, Schema expectedSchema, NameMapping nameMapping, FileIO fileIo, EncryptionManager encryptionManager, boolean caseSensitive) { super(task, fileIo, encryptionManager); this.tableSchema = tableSchema; this.expectedSchema = expectedSchema; + this.nameMapping = nameMapping; this.caseSensitive = caseSensitive; } @@ -174,14 +177,14 @@ private CloseableIterable newParquetIterable( FileScanTask task, Schema readSchema, Map idToConstant) { - return Parquet.read(location) - .project(readSchema) - .applyNameMapping() + Parquet.ReadBuilder builder = Parquet.read(location) .split(task.start(), task.length()) + .project(readSchema) .createReaderFunc(fileSchema -> SparkParquetReaders.buildReader(readSchema, fileSchema, idToConstant)) .filter(task.residual()) - .caseSensitive(caseSensitive) - .build(); + .caseSensitive(caseSensitive); + + return nameMapping != null ? builder.withNameMapping(nameMapping).build() : builder.build(); } private CloseableIterable newOrcIterable( diff --git a/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTableUtil.java b/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTableUtil.java index e08a0120193e..db4bde6f7b92 100644 --- a/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTableUtil.java +++ b/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTableUtil.java @@ -32,6 +32,9 @@ import org.apache.iceberg.Table; import org.apache.iceberg.hadoop.HadoopTables; import org.apache.iceberg.hive.HiveTableBaseTest; +import org.apache.iceberg.mapping.MappingUtil; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.mapping.NameMappingParser; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.SparkTableUtil; import org.apache.iceberg.spark.SparkTableUtil.SparkPartition; @@ -51,6 +54,7 @@ import org.junit.rules.TemporaryFolder; import scala.collection.Seq; +import static org.apache.iceberg.TableProperties.DEFAULT_NAME_MAPPING; import static org.apache.iceberg.types.Types.NestedField.optional; public class TestSparkTableUtil extends HiveTableBaseTest { @@ -216,14 +220,21 @@ public void testImportWithNameMapping() throws Exception { optional(1, "data", Types.StringType.get()) ); + NameMapping nameMapping = MappingUtil.create(filteredSchema); + + TableIdentifier source = new TableIdentifier("original_table"); Table table = catalog.createTable( org.apache.iceberg.catalog.TableIdentifier.of(DB_NAME, "target_table"), filteredSchema, SparkSchemaUtil.specForTable(spark, "original_table")); + + table.updateProperties().set(DEFAULT_NAME_MAPPING, NameMappingParser.toJson(nameMapping)).commit(); + File stagingDir = temp.newFolder("staging-dir"); SparkTableUtil.importSparkTable(spark, source, table, stagingDir.toString()); + // The filter invoke the metric/dictionary row group filter in which it project schema // with name mapping again to match the metric read from footer. List actual = spark.read().format("iceberg").load(DB_NAME + ".target_table") From 9529f0df1be9970c6dc4c14c0637d0ed2395634e Mon Sep 17 00:00:00 2001 From: Junjie Chen Date: Fri, 29 May 2020 11:28:11 +0800 Subject: [PATCH 16/20] allow types without IDs --- .../iceberg/parquet/ApplyNameMapping.java | 8 +- .../org/apache/iceberg/parquet/Parquet.java | 2 +- .../ParquetDictionaryRowGroupFilter.java | 14 ++-- .../parquet/ParquetMetricsRowGroupFilter.java | 16 ++-- .../iceberg/parquet/ParquetReadSupport.java | 2 +- .../iceberg/parquet/ParquetSchemaUtil.java | 2 +- .../apache/iceberg/parquet/PruneColumns.java | 75 +++++++++++-------- .../spark/data/SparkParquetReaders.java | 8 +- .../apache/iceberg/spark/source/Reader.java | 4 +- 9 files changed, 67 insertions(+), 64 deletions(-) diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ApplyNameMapping.java b/parquet/src/main/java/org/apache/iceberg/parquet/ApplyNameMapping.java index f0ce677753ee..efddcfc30cb3 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ApplyNameMapping.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ApplyNameMapping.java @@ -52,7 +52,7 @@ public Type struct(GroupType struct, List types) { List actualTypes = types.stream().filter(Objects::nonNull).collect(Collectors.toList()); Type structType = struct.withNewFields(actualTypes); - return field == null ? null : structType.withId(field.id()); + return field == null ? structType : structType.withId(field.id()); } @Override @@ -65,7 +65,7 @@ public Type list(GroupType list, Type elementType) { .element(elementType) .named(list.getName()); - return field == null ? null : listType.withId(field.id()); + return field == null ? listType : listType.withId(field.id()); } @Override @@ -79,13 +79,13 @@ public Type map(GroupType map, Type keyType, Type valueType) { .value(valueType) .named(map.getName()); - return field == null ? null : mapType.withId(field.id()); + return field == null ? mapType : mapType.withId(field.id()); } @Override public Type primitive(PrimitiveType primitive) { MappedField field = nameMapping.find(currentPath()); - return field == null ? null : primitive.withId(field.id()); + return field == null ? primitive : primitive.withId(field.id()); } @Override diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java index e3723a20ca2a..fa6c80da20a3 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java @@ -38,10 +38,10 @@ import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.mapping.NameMapping; import org.apache.parquet.HadoopReadOptions; import org.apache.parquet.ParquetReadOptions; import org.apache.parquet.avro.AvroReadSupport; diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java index 93b44554c161..07654c3ff13b 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java @@ -112,15 +112,11 @@ private boolean eval(MessageType fileSchema, BlockMetaData rowGroup, } for (ColumnChunkMetaData meta : rowGroup.getColumns()) { - try { - PrimitiveType colType = fileSchema.getType(meta.getPath().toArray()).asPrimitiveType(); - if (colType.getId() != null) { - int id = colType.getId().intValue(); - isFallback.put(id, ParquetUtil.hasNonDictionaryPages(meta)); - mayContainNulls.put(id, mayContainNull(meta)); - } - } catch (org.apache.parquet.io.InvalidRecordException e) { - LOG.warn("Column {} not found in given schema.", meta.getPath().toDotString(), e); + PrimitiveType colType = fileSchema.getType(meta.getPath().toArray()).asPrimitiveType(); + if (colType.getId() != null) { + int id = colType.getId().intValue(); + isFallback.put(id, ParquetUtil.hasNonDictionaryPages(meta)); + mayContainNulls.put(id, mayContainNull(meta)); } } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java index 60199c49109e..c1a646f8c41a 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java @@ -100,16 +100,12 @@ private boolean eval(MessageType fileSchema, BlockMetaData rowGroup) { this.valueCounts = Maps.newHashMap(); this.conversions = Maps.newHashMap(); for (ColumnChunkMetaData col : rowGroup.getColumns()) { - try { - PrimitiveType colType = fileSchema.getType(col.getPath().toArray()).asPrimitiveType(); - if (colType.getId() != null) { - int id = colType.getId().intValue(); - stats.put(id, col.getStatistics()); - valueCounts.put(id, col.getValueCount()); - conversions.put(id, ParquetConversions.converterFromParquet(colType)); - } - } catch (org.apache.parquet.io.InvalidRecordException e) { - LOG.warn("Column {} not found in given schema.", col.getPath().toDotString(), e); + PrimitiveType colType = fileSchema.getType(col.getPath().toArray()).asPrimitiveType(); + if (colType.getId() != null) { + int id = colType.getId().intValue(); + stats.put(id, col.getStatistics()); + valueCounts.put(id, col.getValueCount()); + conversions.put(id, ParquetConversions.converterFromParquet(colType)); } } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java index c0246abc494e..645b3fa0d1fb 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetReadSupport.java @@ -24,9 +24,9 @@ import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.Schema; import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.mapping.NameMapping; import org.apache.parquet.avro.AvroReadSupport; import org.apache.parquet.hadoop.api.InitContext; import org.apache.parquet.hadoop.api.ReadSupport; diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java index 6c9bfd467c67..2460096aae95 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java @@ -22,8 +22,8 @@ import java.util.List; import java.util.Set; import org.apache.iceberg.Schema; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; import org.apache.parquet.schema.GroupType; diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java b/parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java index bd5db4016e92..680c64559b70 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java @@ -21,7 +21,6 @@ import java.util.List; import java.util.Set; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.parquet.schema.GroupType; import org.apache.parquet.schema.MessageType; @@ -45,13 +44,16 @@ public Type message(MessageType message, List fields) { for (int i = 0; i < fields.size(); i += 1) { Type originalField = message.getType(i); Type field = fields.get(i); - if (selectedIds.contains(getId(originalField))) { - builder.addField(originalField); - fieldCount += 1; - } else if (field != null) { - builder.addField(field); - fieldCount += 1; - hasChange = true; + Integer fieldId = getId(originalField); + if (fieldId != null) { + if (selectedIds.contains(fieldId)) { + builder.addField(originalField); + fieldCount += 1; + } else if (field != null) { + builder.addField(field); + fieldCount += 1; + hasChange = true; + } } } @@ -71,11 +73,14 @@ public Type struct(GroupType struct, List fields) { for (int i = 0; i < fields.size(); i += 1) { Type originalField = struct.getType(i); Type field = fields.get(i); - if (selectedIds.contains(getId(originalField))) { - filteredFields.add(originalField); - } else if (field != null) { - filteredFields.add(originalField); - hasChange = true; + Integer fieldId = getId(originalField); + if (fieldId != null) { + if (selectedIds.contains(fieldId)) { + filteredFields.add(originalField); + } else if (field != null) { + filteredFields.add(originalField); + hasChange = true; + } } } @@ -94,19 +99,22 @@ public Type struct(GroupType struct, List fields) { public Type list(GroupType list, Type element) { GroupType repeated = list.getType(0).asGroupType(); Type originalElement = repeated.getType(0); - int elementId = getId(originalElement); - - if (selectedIds.contains(elementId)) { - return list; - } else if (element != null) { - if (element != originalElement) { - // the element type was projected - return Types.list(list.getRepetition()) - .element(element) - .id(getId(list)) - .named(list.getName()); + Integer elementId = getId(originalElement); + + if (elementId != null) { + if (selectedIds.contains(elementId)) { + return list; + } else if (element != null) { + if (element != originalElement) { + Integer listId = getId(list); + // the element type was projected + Type listType = Types.list(list.getRepetition()) + .element(element) + .named(list.getName()); + return listId == null ? listType : listType.withId(listId); + } + return list; } - return list; } return null; @@ -118,18 +126,20 @@ public Type map(GroupType map, Type key, Type value) { Type originalKey = repeated.getType(0); Type originalValue = repeated.getType(1); - int keyId = getId(originalKey); - int valueId = getId(originalValue); + Integer keyId = getId(originalKey); + Integer valueId = getId(originalValue); - if (selectedIds.contains(keyId) || selectedIds.contains(valueId)) { + if ((keyId != null && selectedIds.contains(keyId)) || (valueId != null && selectedIds.contains(valueId))) { return map; } else if (value != null) { + Integer mapId = getId(map); if (value != originalValue) { - return Types.map(map.getRepetition()) + Type mapType = Types.map(map.getRepetition()) .key(originalKey) .value(value) - .id(getId(map)) .named(map.getName()); + + return mapId == null ? mapType : mapType.withId(mapId); } return map; } @@ -142,8 +152,7 @@ public Type primitive(PrimitiveType primitive) { return null; } - private int getId(Type type) { - Preconditions.checkNotNull(type.getId(), "Missing id for type: %s", type); - return type.getId().intValue(); + private Integer getId(Type type) { + return type.getId() == null ? null : type.getId().intValue(); } } diff --git a/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java b/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java index 2baf59ea6f08..51ddc9432bc3 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java +++ b/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetReaders.java @@ -142,9 +142,11 @@ public ParquetValueReader struct(Types.StructType expected, GroupType struct, for (int i = 0; i < fields.size(); i += 1) { Type fieldType = fields.get(i); int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1; - int id = fieldType.getId().intValue(); - readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i))); - typesById.put(id, fieldType); + if (fieldType.getId() != null) { + int id = fieldType.getId().intValue(); + readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i))); + typesById.put(id, fieldType); + } } List expectedFields = expected != null ? diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java b/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java index 7610ecf680a3..4b1e9bc9c16f 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java @@ -43,10 +43,10 @@ import org.apache.iceberg.hadoop.Util; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.mapping.NameMappingParser; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.SparkFilters; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.spark.broadcast.Broadcast; From 526e79becfe9fe83d6065f9be1569957f8d49cc7 Mon Sep 17 00:00:00 2001 From: Junjie Chen Date: Fri, 29 May 2020 14:37:19 +0800 Subject: [PATCH 17/20] remove unrelated changes --- .../iceberg/parquet/ParquetDictionaryRowGroupFilter.java | 3 --- .../apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java | 3 --- 2 files changed, 6 deletions(-) diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java index 07654c3ff13b..09738be01269 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java @@ -45,11 +45,8 @@ import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class ParquetDictionaryRowGroupFilter { - private static final Logger LOG = LoggerFactory.getLogger(ParquetDictionaryRowGroupFilter.class); private final Expression expr; private transient ThreadLocal visitors = null; diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java index c1a646f8c41a..ac5df881bbce 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java @@ -45,11 +45,8 @@ import org.apache.parquet.io.api.Binary; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class ParquetMetricsRowGroupFilter { - private static final Logger LOG = LoggerFactory.getLogger(ParquetMetricsRowGroupFilter.class); private final Schema schema; private final Expression expr; From 18a803f9bd033e4c5fada44394ee7ebfb4468216 Mon Sep 17 00:00:00 2001 From: Junjie Chen Date: Tue, 16 Jun 2020 10:50:07 +0800 Subject: [PATCH 18/20] resolve merge conflicts --- .../iceberg/spark/source/BatchDataReader.java | 13 ++++-- .../apache/iceberg/spark/source/Reader.java | 44 ++++++------------- .../iceberg/spark/source/RowDataReader.java | 9 ++-- .../iceberg/spark/source/RowDataRewriter.java | 7 ++- 4 files changed, 34 insertions(+), 39 deletions(-) diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java b/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java index eeb3ad559858..16b065ba83f4 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java @@ -29,6 +29,7 @@ import org.apache.iceberg.io.CloseableIterator; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.mapping.NameMappingParser; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.spark.data.vectorized.VectorizedSparkParquetReaders; @@ -36,14 +37,16 @@ class BatchDataReader extends BaseDataReader { private final Schema expectedSchema; + private final String nameMapping; private final boolean caseSensitive; private final int batchSize; BatchDataReader( - CombinedScanTask task, Schema expectedSchema, FileIO fileIo, + CombinedScanTask task, Schema expectedSchema, String nameMapping, FileIO fileIo, EncryptionManager encryptionManager, boolean caseSensitive, int size) { super(task, fileIo, encryptionManager); this.expectedSchema = expectedSchema; + this.nameMapping = nameMapping; this.caseSensitive = caseSensitive; this.batchSize = size; } @@ -54,7 +57,7 @@ CloseableIterator open(FileScanTask task) { InputFile location = getInputFile(task); Preconditions.checkNotNull(location, "Could not find InputFile associated with FileScanTask"); if (task.file().format() == FileFormat.PARQUET) { - iter = Parquet.read(location) + Parquet.ReadBuilder builder = Parquet.read(location) .project(expectedSchema) .split(task.start(), task.length()) .createBatchedReaderFunc(fileSchema -> VectorizedSparkParquetReaders.buildReader(expectedSchema, @@ -65,8 +68,10 @@ CloseableIterator open(FileScanTask task) { // Spark eagerly consumes the batches. So the underlying memory allocated could be reused // without worrying about subsequent reads clobbering over each other. This improves // read performance as every batch read doesn't have to pay the cost of allocating memory. - .reuseContainers() - .build(); + .reuseContainers(); + + iter = nameMapping != null ? + builder.withNameMapping(NameMappingParser.fromJson(nameMapping)).build() : builder.build(); } else { throw new UnsupportedOperationException( "Format: " + task.file().format() + " not supported for batched reads"); diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java b/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java index df74b301f07b..a9faf0021fb4 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/Reader.java @@ -45,7 +45,6 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.mapping.NameMapping; -import org.apache.iceberg.mapping.NameMappingParser; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -74,8 +73,6 @@ import static org.apache.iceberg.TableProperties.DEFAULT_NAME_MAPPING; -class Reader implements DataSourceReader, SupportsPushDownFilters, SupportsPushDownRequiredColumns, - SupportsReportStatistics { class Reader implements DataSourceReader, SupportsScanColumnarBatch, SupportsPushDownFilters, SupportsPushDownRequiredColumns, SupportsReportStatistics { private static final Logger LOG = LoggerFactory.getLogger(Reader.class); @@ -208,12 +205,13 @@ public List> planBatchInputPartitions() { Preconditions.checkState(batchSize > 0, "Invalid batch size"); String tableSchemaString = SchemaParser.toJson(table.schema()); String expectedSchemaString = SchemaParser.toJson(lazySchema()); + String nameMappingString = table.properties().get(DEFAULT_NAME_MAPPING); List> readTasks = Lists.newArrayList(); for (CombinedScanTask task : tasks()) { readTasks.add(new ReadTask<>( - task, tableSchemaString, expectedSchemaString, io, encryptionManager, caseSensitive, localityPreferred, - new BatchReaderFactory(batchSize))); + task, tableSchemaString, expectedSchemaString, nameMappingString, io, encryptionManager, caseSensitive, + localityPreferred, new BatchReaderFactory(batchSize))); } LOG.info("Batching input partitions with {} tasks.", readTasks.size()); @@ -231,12 +229,9 @@ public List> planInputPartitions() { List> readTasks = Lists.newArrayList(); for (CombinedScanTask task : tasks()) { - readTasks.add( - new ReadTask(task, tableSchemaString, expectedSchemaString, nameMappingString, io, encryptionManager, - caseSensitive, localityPreferred)); readTasks.add(new ReadTask<>( - task, tableSchemaString, expectedSchemaString, io, encryptionManager, caseSensitive, localityPreferred, - InternalRowReaderFactory.INSTANCE)); + task, tableSchemaString, expectedSchemaString, nameMappingString, io, encryptionManager, caseSensitive, + localityPreferred, InternalRowReaderFactory.INSTANCE)); } return readTasks; @@ -406,8 +401,6 @@ private static class ReadTask implements Serializable, InputPartition { private ReadTask(CombinedScanTask task, String tableSchemaString, String expectedSchemaString, String nameMappingString, Broadcast io, Broadcast encryptionManager, - boolean caseSensitive, boolean localityPreferred) { - Broadcast io, Broadcast encryptionManager, boolean caseSensitive, boolean localityPreferred, ReaderFactory readerFactory) { this.task = task; this.tableSchemaString = tableSchemaString; @@ -422,11 +415,8 @@ private ReadTask(CombinedScanTask task, String tableSchemaString, String expecte } @Override - public InputPartitionReader createPartitionReader() { - return new RowDataReader(task, lazyTableSchema(), lazyExpectedSchema(), lazyNameMapping(), io.value(), - encryptionManager.value(), caseSensitive); public InputPartitionReader createPartitionReader() { - return readerFactory.create(task, lazyTableSchema(), lazyExpectedSchema(), io.value(), + return readerFactory.create(task, lazyTableSchema(), lazyExpectedSchema(), nameMappingString, io.value(), encryptionManager.value(), caseSensitive); } @@ -449,13 +439,6 @@ private Schema lazyExpectedSchema() { return expectedSchema; } - private NameMapping lazyNameMapping() { - if (nameMapping == null && nameMappingString != null) { - this.nameMapping = NameMappingParser.fromJson(nameMappingString); - } - return nameMapping; - } - private String[] getPreferredLocations() { if (!localityPreferred) { return new String[0]; @@ -467,7 +450,8 @@ private String[] getPreferredLocations() { } private interface ReaderFactory extends Serializable { - InputPartitionReader create(CombinedScanTask task, Schema tableSchema, Schema expectedSchema, FileIO io, + InputPartitionReader create(CombinedScanTask task, Schema tableSchema, Schema expectedSchema, + String nameMapping, FileIO io, EncryptionManager encryptionManager, boolean caseSensitive); } @@ -479,9 +463,9 @@ private InternalRowReaderFactory() { @Override public InputPartitionReader create(CombinedScanTask task, Schema tableSchema, Schema expectedSchema, - FileIO io, EncryptionManager encryptionManager, - boolean caseSensitive) { - return new RowDataReader(task, tableSchema, expectedSchema, io, encryptionManager, caseSensitive); + String nameMapping, FileIO io, + EncryptionManager encryptionManager, boolean caseSensitive) { + return new RowDataReader(task, tableSchema, expectedSchema, nameMapping, io, encryptionManager, caseSensitive); } } @@ -494,9 +478,9 @@ private static class BatchReaderFactory implements ReaderFactory @Override public InputPartitionReader create(CombinedScanTask task, Schema tableSchema, Schema expectedSchema, - FileIO io, EncryptionManager encryptionManager, - boolean caseSensitive) { - return new BatchDataReader(task, expectedSchema, io, encryptionManager, caseSensitive, batchSize); + String nameMapping, FileIO io, + EncryptionManager encryptionManager, boolean caseSensitive) { + return new BatchDataReader(task, expectedSchema, nameMapping, io, encryptionManager, caseSensitive, batchSize); } } diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java b/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java index 8587a485559f..04d0504a7431 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java @@ -39,7 +39,7 @@ import org.apache.iceberg.io.CloseableIterator; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.InputFile; -import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.mapping.NameMappingParser; import org.apache.iceberg.orc.ORC; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -72,11 +72,11 @@ class RowDataReader extends BaseDataReader { private final Schema tableSchema; private final Schema expectedSchema; - private final NameMapping nameMapping; + private final String nameMapping; private final boolean caseSensitive; RowDataReader( - CombinedScanTask task, Schema tableSchema, Schema expectedSchema, NameMapping nameMapping, FileIO fileIo, + CombinedScanTask task, Schema tableSchema, Schema expectedSchema, String nameMapping, FileIO fileIo, EncryptionManager encryptionManager, boolean caseSensitive) { super(task, fileIo, encryptionManager); this.tableSchema = tableSchema; @@ -161,7 +161,8 @@ private CloseableIterable newParquetIterable( .filter(task.residual()) .caseSensitive(caseSensitive); - return nameMapping != null ? builder.withNameMapping(nameMapping).build() : builder.build(); + return nameMapping != null ? + builder.withNameMapping(NameMappingParser.fromJson(nameMapping)).build() : builder.build(); } private CloseableIterable newOrcIterable( diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/RowDataRewriter.java b/spark/src/main/java/org/apache/iceberg/spark/source/RowDataRewriter.java index 0a568740caf5..384a95bf631c 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/RowDataRewriter.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/RowDataRewriter.java @@ -42,6 +42,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import static org.apache.iceberg.TableProperties.DEFAULT_NAME_MAPPING; + public class RowDataRewriter implements Serializable { private static final Logger LOG = LoggerFactory.getLogger(RowDataRewriter.class); @@ -49,6 +51,7 @@ public class RowDataRewriter implements Serializable { private final Broadcast fileIO; private final Broadcast encryptionManager; private final String tableSchema; + private final String nameMapping; private final Writer.WriterFactory writerFactory; private final boolean caseSensitive; @@ -60,6 +63,7 @@ public RowDataRewriter(Table table, PartitionSpec spec, boolean caseSensitive, this.caseSensitive = caseSensitive; this.tableSchema = SchemaParser.toJson(table.schema()); + this.nameMapping = table.properties().get(DEFAULT_NAME_MAPPING); String formatString = table.properties().getOrDefault( TableProperties.DEFAULT_FILE_FORMAT, TableProperties.DEFAULT_FILE_FORMAT_DEFAULT); @@ -80,7 +84,8 @@ private Writer.TaskCommit rewriteDataForTask(CombinedScanTask task) throws Excep TaskContext context = TaskContext.get(); RowDataReader dataReader = new RowDataReader(task, SchemaParser.fromJson(tableSchema), - SchemaParser.fromJson(tableSchema), fileIO.value(), encryptionManager.value(), caseSensitive); + SchemaParser.fromJson(tableSchema), nameMapping, fileIO.value(), + encryptionManager.value(), caseSensitive); int partitionId = context.partitionId(); long taskId = context.taskAttemptId(); From 2f47631f65b54c0c798ae8c0d3af7ceee003b9f5 Mon Sep 17 00:00:00 2001 From: Junjie Chen Date: Wed, 17 Jun 2020 11:18:05 +0800 Subject: [PATCH 19/20] address comments --- .../iceberg/parquet/ApplyNameMapping.java | 4 ++ .../iceberg/parquet/MessageTypeToType.java | 2 +- .../ParquetDictionaryRowGroupFilter.java | 1 - .../parquet/ParquetMetricsRowGroupFilter.java | 1 - .../apache/iceberg/parquet/PruneColumns.java | 52 ++++++++----------- .../parquet/VectorizedParquetReader.java | 6 +-- .../VectorizedSparkParquetReaders.java | 4 ++ .../iceberg/spark/source/BatchDataReader.java | 7 ++- .../iceberg/spark/source/RowDataReader.java | 7 ++- .../spark/source/TestSparkTableUtil.java | 48 ++++++++++++++++- 10 files changed, 92 insertions(+), 40 deletions(-) diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ApplyNameMapping.java b/parquet/src/main/java/org/apache/iceberg/parquet/ApplyNameMapping.java index efddcfc30cb3..85d612216723 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ApplyNameMapping.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ApplyNameMapping.java @@ -90,17 +90,21 @@ public Type primitive(PrimitiveType primitive) { @Override public void beforeRepeatedElement(Type element) { + // do not add the repeated element's name } @Override public void afterRepeatedElement(Type element) { + // do not remove the repeated element's name } @Override public void beforeRepeatedKeyValue(Type keyValue) { + // do not add the repeated element's name } @Override public void afterRepeatedKeyValue(Type keyValue) { + // do not remove the repeated element's name } } diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/MessageTypeToType.java b/parquet/src/main/java/org/apache/iceberg/parquet/MessageTypeToType.java index be523c61cc6f..43c2cd2b70d2 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/MessageTypeToType.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/MessageTypeToType.java @@ -229,7 +229,7 @@ private void addAlias(String name, int fieldId) { aliasToId.put(DOT.join(path(name)), fieldId); } - private int nextId() { + protected int nextId() { int current = nextId; nextId += 1; return current; diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java index 09738be01269..351fb69512d3 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetDictionaryRowGroupFilter.java @@ -47,7 +47,6 @@ import org.apache.parquet.schema.PrimitiveType; public class ParquetDictionaryRowGroupFilter { - private final Expression expr; private transient ThreadLocal visitors = null; diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java index ac5df881bbce..3fdf905cb9a9 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetMetricsRowGroupFilter.java @@ -47,7 +47,6 @@ import org.apache.parquet.schema.PrimitiveType; public class ParquetMetricsRowGroupFilter { - private final Schema schema; private final Expression expr; private transient ThreadLocal visitors = null; diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java b/parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java index 680c64559b70..d05db4625776 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java @@ -45,15 +45,13 @@ public Type message(MessageType message, List fields) { Type originalField = message.getType(i); Type field = fields.get(i); Integer fieldId = getId(originalField); - if (fieldId != null) { - if (selectedIds.contains(fieldId)) { - builder.addField(originalField); - fieldCount += 1; - } else if (field != null) { - builder.addField(field); - fieldCount += 1; - hasChange = true; - } + if (fieldId != null && selectedIds.contains(fieldId)) { + builder.addField(originalField); + fieldCount += 1; + } else if (field != null) { + builder.addField(field); + fieldCount += 1; + hasChange = true; } } @@ -74,13 +72,11 @@ public Type struct(GroupType struct, List fields) { Type originalField = struct.getType(i); Type field = fields.get(i); Integer fieldId = getId(originalField); - if (fieldId != null) { - if (selectedIds.contains(fieldId)) { - filteredFields.add(originalField); - } else if (field != null) { - filteredFields.add(originalField); - hasChange = true; - } + if (fieldId != null && selectedIds.contains(fieldId)) { + filteredFields.add(originalField); + } else if (field != null) { + filteredFields.add(originalField); + hasChange = true; } } @@ -101,20 +97,18 @@ public Type list(GroupType list, Type element) { Type originalElement = repeated.getType(0); Integer elementId = getId(originalElement); - if (elementId != null) { - if (selectedIds.contains(elementId)) { - return list; - } else if (element != null) { - if (element != originalElement) { - Integer listId = getId(list); - // the element type was projected - Type listType = Types.list(list.getRepetition()) - .element(element) - .named(list.getName()); - return listId == null ? listType : listType.withId(listId); - } - return list; + if (elementId != null && selectedIds.contains(elementId)) { + return list; + } else if (element != null) { + if (element != originalElement) { + Integer listId = getId(list); + // the element type was projected + Type listType = Types.list(list.getRepetition()) + .element(element) + .named(list.getName()); + return listId == null ? listType : listType.withId(listId); } + return list; } return null; diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java b/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java index 20c992575143..481012cb8bbe 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java @@ -52,9 +52,9 @@ public class VectorizedParquetReader extends CloseableGroup implements Closea private final NameMapping nameMapping; public VectorizedParquetReader( - InputFile input, Schema expectedSchema, ParquetReadOptions options, Function> readerFunc, NameMapping nameMapping, Expression filter, boolean reuseContainers, - boolean caseSensitive, int maxRecordsPerBatch) { + InputFile input, Schema expectedSchema, ParquetReadOptions options, + Function> readerFunc, NameMapping nameMapping, Expression filter, + boolean reuseContainers, boolean caseSensitive, int maxRecordsPerBatch) { this.input = input; this.expectedSchema = expectedSchema; this.options = options; diff --git a/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java b/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java index 01cbe6f286ad..3eb55ebaf22a 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java +++ b/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/VectorizedSparkParquetReaders.java @@ -78,6 +78,7 @@ public VectorizedReader message( List fields = groupType.getFields(); IntStream.range(0, fields.size()) + .filter(pos -> fields.get(pos).getId() != null) .forEach(pos -> readersById.put(fields.get(pos).getId().intValue(), fieldReaders.get(pos))); List icebergFields = expected != null ? @@ -114,6 +115,9 @@ public VectorizedReader primitive( PrimitiveType primitive) { // Create arrow vector for this field + if (primitive.getId() == null) { + return null; + } int parquetFieldId = primitive.getId().intValue(); ColumnDescriptor desc = parquetSchema.getColumnDescription(currentPath()); // Nested types not yet supported for vectorized reads diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java b/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java index 16b065ba83f4..f784b638d376 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/BatchDataReader.java @@ -70,8 +70,11 @@ CloseableIterator open(FileScanTask task) { // read performance as every batch read doesn't have to pay the cost of allocating memory. .reuseContainers(); - iter = nameMapping != null ? - builder.withNameMapping(NameMappingParser.fromJson(nameMapping)).build() : builder.build(); + if (nameMapping != null) { + builder.withNameMapping(NameMappingParser.fromJson(nameMapping)); + } + + iter = builder.build(); } else { throw new UnsupportedOperationException( "Format: " + task.file().format() + " not supported for batched reads"); diff --git a/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java b/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java index 04d0504a7431..fb0b43dac2a8 100644 --- a/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java +++ b/spark/src/main/java/org/apache/iceberg/spark/source/RowDataReader.java @@ -161,8 +161,11 @@ private CloseableIterable newParquetIterable( .filter(task.residual()) .caseSensitive(caseSensitive); - return nameMapping != null ? - builder.withNameMapping(NameMappingParser.fromJson(nameMapping)).build() : builder.build(); + if (nameMapping != null) { + builder.withNameMapping(NameMappingParser.fromJson(nameMapping)); + } + + return builder.build(); } private CloseableIterable newOrcIterable( diff --git a/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTableUtil.java b/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTableUtil.java index 4dcfb2868efe..4caa4a15f7f8 100644 --- a/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTableUtil.java +++ b/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTableUtil.java @@ -55,6 +55,7 @@ import scala.collection.Seq; import static org.apache.iceberg.TableProperties.DEFAULT_NAME_MAPPING; +import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED; import static org.apache.iceberg.types.Types.NestedField.optional; public class TestSparkTableUtil extends HiveTableBaseTest { @@ -222,7 +223,6 @@ public void testImportWithNameMapping() throws Exception { NameMapping nameMapping = MappingUtil.create(filteredSchema); - TableIdentifier source = new TableIdentifier("original_table"); Table table = catalog.createTable( org.apache.iceberg.catalog.TableIdentifier.of(DB_NAME, "target_table"), @@ -235,6 +235,52 @@ public void testImportWithNameMapping() throws Exception { SparkTableUtil.importSparkTable(spark, source, table, stagingDir.toString()); + // The filter invoke the metric/dictionary row group filter in which it project schema + // with name mapping again to match the metric read from footer. + List actual = spark.read().format("iceberg").load(DB_NAME + ".target_table") + .select("data") + .sort("data") + .filter("data<'c'") + .collectAsList() + .stream() + .map(r -> r.getString(0)) + .collect(Collectors.toList()); + + List expected = Lists.newArrayList( + new SimpleRecord(2, "a"), + new SimpleRecord(1, "b") + ); + + Assert.assertEquals(expected.stream().map(SimpleRecord::getData).collect(Collectors.toList()), actual); + } + + @Test + public void testImportWithNameMappingForVectorizedParquetReader() throws Exception { + spark.table(qualifiedTableName).write().mode("overwrite").format("parquet") + .saveAsTable("original_table"); + + // The field is different so that it will project with name mapping + Schema filteredSchema = new Schema( + optional(1, "data", Types.StringType.get()) + ); + + NameMapping nameMapping = MappingUtil.create(filteredSchema); + + TableIdentifier source = new TableIdentifier("original_table"); + Table table = catalog.createTable( + org.apache.iceberg.catalog.TableIdentifier.of(DB_NAME, "target_table"), + filteredSchema, + SparkSchemaUtil.specForTable(spark, "original_table")); + + table.updateProperties() + .set(DEFAULT_NAME_MAPPING, NameMappingParser.toJson(nameMapping)) + .set(PARQUET_VECTORIZATION_ENABLED, "true") + .commit(); + + File stagingDir = temp.newFolder("staging-dir"); + SparkTableUtil.importSparkTable(spark, source, table, stagingDir.toString()); + + // The filter invoke the metric/dictionary row group filter in which it project schema // with name mapping again to match the metric read from footer. List actual = spark.read().format("iceberg").load(DB_NAME + ".target_table") From 3ca14c7de19fb6b51e9a691306f4de64e67c86b8 Mon Sep 17 00:00:00 2001 From: Junjie Chen Date: Wed, 17 Jun 2020 13:08:55 +0800 Subject: [PATCH 20/20] fix style --- .../apache/iceberg/spark/source/TestSparkTableUtil.java | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTableUtil.java b/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTableUtil.java index 4caa4a15f7f8..37f57d483364 100644 --- a/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTableUtil.java +++ b/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkTableUtil.java @@ -234,7 +234,6 @@ public void testImportWithNameMapping() throws Exception { File stagingDir = temp.newFolder("staging-dir"); SparkTableUtil.importSparkTable(spark, source, table, stagingDir.toString()); - // The filter invoke the metric/dictionary row group filter in which it project schema // with name mapping again to match the metric read from footer. List actual = spark.read().format("iceberg").load(DB_NAME + ".target_table") @@ -268,7 +267,7 @@ public void testImportWithNameMappingForVectorizedParquetReader() throws Excepti TableIdentifier source = new TableIdentifier("original_table"); Table table = catalog.createTable( - org.apache.iceberg.catalog.TableIdentifier.of(DB_NAME, "target_table"), + org.apache.iceberg.catalog.TableIdentifier.of(DB_NAME, "target_table_for_vectorization"), filteredSchema, SparkSchemaUtil.specForTable(spark, "original_table")); @@ -280,10 +279,10 @@ public void testImportWithNameMappingForVectorizedParquetReader() throws Excepti File stagingDir = temp.newFolder("staging-dir"); SparkTableUtil.importSparkTable(spark, source, table, stagingDir.toString()); - // The filter invoke the metric/dictionary row group filter in which it project schema // with name mapping again to match the metric read from footer. - List actual = spark.read().format("iceberg").load(DB_NAME + ".target_table") + List actual = spark.read().format("iceberg") + .load(DB_NAME + ".target_table_for_vectorization") .select("data") .sort("data") .filter("data<'c'")