@Override public ParquetValueReader<?> message(StructType expected, MessageType message, List<ParquetValueReader<?>> fieldReaders) { // the top level matches by ID, but the remaining IDs are missing return super.struct(expected, message, fieldReaders); }
@Override public ParquetValueWriter<?> message(MessageType message, List<ParquetValueWriter<?>> fieldWriters) { return struct(message.asGroupType(), fieldWriters); }
@SuppressWarnings("unchecked") public static <T> ParquetValueWriter<T> buildWriter(MessageType type) { return (ParquetValueWriter<T>) ParquetTypeVisitor.visit(type, new WriteBuilder(type)); }
@Override public ParquetValueReader<?> primitive(com.netflix.iceberg.types.Type.PrimitiveType expected, PrimitiveType primitive) { ColumnDescriptor desc = type.getColumnDescription(currentPath()); return new UnboxedReader<>(desc); case DATE: return new DateReader(desc); case TIMESTAMP_MICROS: TimestampType tsMicrosType = (TimestampType) expected; if (tsMicrosType.shouldAdjustToUTC()) { return new TimestamptzReader(desc); } else { return new TimestampReader(desc); return new TimestamptzMillisReader(desc); } else { return new TimestampMillisReader(desc); return new FixedReader(desc); case BINARY: return new BytesReader(desc);
@Override public ParquetValueWriter<?> primitive(PrimitiveType primitive) { ColumnDescriptor desc = type.getColumnDescription(currentPath()); return unboxed(desc); case DATE: return new DateWriter(desc); case TIME_MICROS: return new TimeWriter(desc); case TIMESTAMP_MICROS: return new TimestamptzWriter(desc); case DECIMAL: DecimalMetadata decimal = primitive.getDecimalMetadata(); return new FixedWriter(desc); case BINARY: return byteBuffers(desc);
protected Record writeAndRead(String desc, Schema writeSchema, Schema readSchema, Record record) throws IOException { File file = temp.newFile(desc + ".parquet"); file.delete(); try (FileAppender<Record> appender = Parquet.write(Files.localOutput(file)) .schema(writeSchema) .createWriterFunc(GenericParquetWriter::buildWriter) .build()) { appender.add(record); } Iterable<Record> records = Parquet.read(Files.localInput(file)) .project(readSchema) .createReaderFunc(fileSchema -> GenericParquetReaders.buildReader(readSchema, fileSchema)) .build(); return Iterables.getOnlyElement(records); } }
@Override public ParquetValueWriter<?> list(GroupType array, ParquetValueWriter<?> elementWriter) { GroupType repeated = array.getFields().get(0).asGroupType(); String[] repeatedPath = currentPath(); int repeatedD = type.getMaxDefinitionLevel(repeatedPath); int repeatedR = type.getMaxRepetitionLevel(repeatedPath); org.apache.parquet.schema.Type elementType = repeated.getType(0); int elementD = type.getMaxDefinitionLevel(path(elementType.getName())); return collections(repeatedD, repeatedR, option(elementType, elementD, elementWriter)); }
@Override public ParquetValueReader<?> list(Types.ListType expectedList, GroupType array, ParquetValueReader<?> elementReader) { GroupType repeated = array.getFields().get(0).asGroupType(); String[] repeatedPath = currentPath(); int repeatedD = type.getMaxDefinitionLevel(repeatedPath)-1; int repeatedR = type.getMaxRepetitionLevel(repeatedPath)-1; Type elementType = repeated.getType(0); int elementD = type.getMaxDefinitionLevel(path(elementType.getName()))-1; return new ListReader<>(repeatedD, repeatedR, option(elementType, elementD, elementReader)); }
@Override public ParquetValueWriter<?> struct(GroupType struct, List<ParquetValueWriter<?>> fieldWriters) { List<Type> fields = struct.getFields(); List<ParquetValueWriter<?>> writers = Lists.newArrayListWithExpectedSize(fieldWriters.size()); for (int i = 0; i < fields.size(); i += 1) { Type fieldType = struct.getType(i); int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())); writers.add(option(fieldType, fieldD, fieldWriters.get(i))); } return new RecordWriter(writers); }
@Override public ParquetValueReader<?> struct(StructType expected, GroupType struct, List<ParquetValueReader<?>> fieldReaders) { // the expected struct is ignored because nested fields are never found when the List<ParquetValueReader<?>> newFields = Lists.newArrayListWithExpectedSize( fieldReaders.size()); List<Type> types = Lists.newArrayListWithExpectedSize(fieldReaders.size()); List<Type> fields = struct.getFields(); for (int i = 0; i < fields.size(); i += 1) { Type fieldType = fields.get(i); int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName()))-1; newFields.add(option(fieldType, fieldD, fieldReaders.get(i))); types.add(fieldType); } return new RecordReader(types, newFields, expected); } }
@SuppressWarnings("unchecked") public static ParquetValueReader<GenericRecord> buildReader(Schema expectedSchema, MessageType fileSchema) { if (hasIds(fileSchema)) { return (ParquetValueReader<GenericRecord>) TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema, new ReadBuilder(fileSchema)); } else { return (ParquetValueReader<GenericRecord>) TypeWithSchemaVisitor.visit(expectedSchema.asStruct(), fileSchema, new FallbackReadBuilder(fileSchema)); } }
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, option(fieldType, fieldD, fieldReaders.get(i))); return new RecordReader(types, reorderedFields, expected);
protected void writeAndValidate(Schema schema) throws IOException { List<Record> expected = RandomGenericData.generate(schema, 100, 0L); File testFile = temp.newFile(); Assert.assertTrue("Delete should succeed", testFile.delete()); try (FileAppender<Record> appender = Parquet.write(Files.localOutput(testFile)) .schema(schema) .createWriterFunc(GenericParquetWriter::buildWriter) .build()) { appender.addAll(expected); } List<Record> rows; try (CloseableIterable<Record> reader = Parquet.read(Files.localInput(testFile)) .project(schema) .createReaderFunc(fileSchema -> GenericParquetReaders.buildReader(schema, fileSchema)) .build()) { rows = Lists.newArrayList(reader); } for (int i = 0; i < expected.size(); i += 1) { DataTestHelpers.assertEquals(schema.asStruct(), expected.get(i), rows.get(i)); } } }
@Override public ParquetValueWriter<?> map(GroupType map, ParquetValueWriter<?> keyWriter, ParquetValueWriter<?> valueWriter) { GroupType repeatedKeyValue = map.getFields().get(0).asGroupType(); String[] repeatedPath = currentPath(); int repeatedD = type.getMaxDefinitionLevel(repeatedPath); int repeatedR = type.getMaxRepetitionLevel(repeatedPath); org.apache.parquet.schema.Type keyType = repeatedKeyValue.getType(0); int keyD = type.getMaxDefinitionLevel(path(keyType.getName())); org.apache.parquet.schema.Type valueType = repeatedKeyValue.getType(1); int valueD = type.getMaxDefinitionLevel(path(valueType.getName())); return maps(repeatedD, repeatedR, option(keyType, keyD, keyWriter), option(valueType, valueD, valueWriter)); }
@Override public ParquetValueReader<?> map(Types.MapType expectedMap, GroupType map, ParquetValueReader<?> keyReader, ParquetValueReader<?> valueReader) { GroupType repeatedKeyValue = map.getFields().get(0).asGroupType(); String[] repeatedPath = currentPath(); int repeatedD = type.getMaxDefinitionLevel(repeatedPath)-1; int repeatedR = type.getMaxRepetitionLevel(repeatedPath)-1; Type keyType = repeatedKeyValue.getType(0); int keyD = type.getMaxDefinitionLevel(path(keyType.getName()))-1; Type valueType = repeatedKeyValue.getType(1); int valueD = type.getMaxDefinitionLevel(path(valueType.getName()))-1; return new MapReader<>(repeatedD, repeatedR, option(keyType, keyD, keyReader), option(valueType, valueD, valueReader)); }
@Override public ParquetValueReader<?> message(StructType expected, MessageType message, List<ParquetValueReader<?>> fieldReaders) { return struct(expected, message.asGroupType(), fieldReaders); }
private CloseableIterable<Record> open(FileScanTask task) { InputFile input = ops.io().newInputFile(task.file().path().toString()); // TODO: join to partition data from the manifest file switch (task.file().format()) { case AVRO: Avro.ReadBuilder avro = Avro.read(input) .project(projection) .createReaderFunc(DataReader::create) .split(task.start(), task.length()); if (reuseContainers) { avro.reuseContainers(); } return avro.build(); case PARQUET: Parquet.ReadBuilder parquet = Parquet.read(input) .project(projection) .createReaderFunc(fileSchema -> buildReader(projection, fileSchema)) .split(task.start(), task.length()); if (reuseContainers) { parquet.reuseContainers(); } return parquet.build(); default: throw new UnsupportedOperationException(String.format("Cannot read %s file: %s", task.file().format().name(), task.file().path())); } }