-
Notifications
You must be signed in to change notification settings - Fork 1.5k
PARQUET-212: Implement LIST read compatibility rules in Thrift #300
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
Closed
Changes from all commits
Commits
Show all changes
8 commits
Select commit
Hold shift + click to select a range
4e148dc
PARQUET-212: Add DirectWriterTest base class.
rdblue 0bf2b45
PARQUET-212: Read non-thrift files if a Thrift class is supplied.
rdblue 3d1e92f
PARQUET-212: Update thrift reads for LIST compatibility rules.
rdblue b87eb65
PARQUET-212: Add property to ignore nulls in lists.
rdblue b5f207f
PARQUET-212: Add Configuration to the ThriftRecordConverter ctor.
rdblue 5d3b094
PARQUET-212: Fix list handling with projection.
rdblue 356fdb7
PARQUET-212: Rename isElementType => isListElementType.
rdblue ac7c405
PARQUET-212: Add tests for list of list cases from PARQUET-364.
rdblue File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
102 changes: 102 additions & 0 deletions
102
parquet-hadoop/src/test/java/org/apache/parquet/DirectWriterTest.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -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.parquet; | ||
|
|
||
| import java.io.File; | ||
| import java.io.IOException; | ||
| import java.util.HashMap; | ||
| import java.util.Map; | ||
| import java.util.UUID; | ||
| import org.apache.hadoop.conf.Configuration; | ||
| import org.apache.hadoop.fs.Path; | ||
| import org.junit.Rule; | ||
| import org.junit.rules.TemporaryFolder; | ||
| import org.apache.parquet.hadoop.ParquetWriter; | ||
| import org.apache.parquet.hadoop.api.WriteSupport; | ||
| import org.apache.parquet.io.api.RecordConsumer; | ||
| import org.apache.parquet.schema.MessageType; | ||
| import org.apache.parquet.schema.MessageTypeParser; | ||
|
|
||
| public class DirectWriterTest { | ||
|
|
||
| @Rule | ||
| public final TemporaryFolder tempDir = new TemporaryFolder(); | ||
|
|
||
| protected interface DirectWriter { | ||
| public void write(RecordConsumer consumer); | ||
| } | ||
|
|
||
| protected Path writeDirect(String type, DirectWriter writer) throws IOException { | ||
| return writeDirect(MessageTypeParser.parseMessageType(type), writer); | ||
| } | ||
|
|
||
| protected Path writeDirect(String type, DirectWriter writer, | ||
| Map<String, String> metadata) throws IOException { | ||
| return writeDirect(MessageTypeParser.parseMessageType(type), writer, metadata); | ||
| } | ||
|
|
||
| protected Path writeDirect(MessageType type, DirectWriter writer) throws IOException { | ||
| return writeDirect(type, writer, new HashMap<String, String>()); | ||
| } | ||
|
|
||
| protected Path writeDirect(MessageType type, DirectWriter writer, | ||
| Map<String, String> metadata) throws IOException { | ||
| File temp = tempDir.newFile(UUID.randomUUID().toString()); | ||
| temp.deleteOnExit(); | ||
| temp.delete(); | ||
|
|
||
| Path path = new Path(temp.getPath()); | ||
|
|
||
| ParquetWriter<Void> parquetWriter = new ParquetWriter<Void>( | ||
| path, new DirectWriteSupport(type, writer, metadata)); | ||
| parquetWriter.write(null); | ||
| parquetWriter.close(); | ||
|
|
||
| return path; | ||
| } | ||
|
|
||
| protected static class DirectWriteSupport extends WriteSupport<Void> { | ||
| private RecordConsumer recordConsumer; | ||
| private final MessageType type; | ||
| private final DirectWriter writer; | ||
| private final Map<String, String> metadata; | ||
|
|
||
| protected DirectWriteSupport(MessageType type, DirectWriter writer, | ||
| Map<String, String> metadata) { | ||
| this.type = type; | ||
| this.writer = writer; | ||
| this.metadata = metadata; | ||
| } | ||
|
|
||
| @Override | ||
| public WriteContext init(Configuration configuration) { | ||
| return new WriteContext(type, metadata); | ||
| } | ||
|
|
||
| @Override | ||
| public void prepareForWrite(RecordConsumer recordConsumer) { | ||
| this.recordConsumer = recordConsumer; | ||
| } | ||
|
|
||
| @Override | ||
| public void write(Void record) { | ||
| writer.write(recordConsumer); | ||
| } | ||
| } | ||
| } |
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't quite get the purpose of this change. Is this mainly for more robust support for schema evolution? Take the following Avro schema and Parquet schema as an example:
According to the condition of this
ifbranch, we should interpretstruct_arrayas aLISTof AvroStruct, with allf1field set asnull. However,f1defined inStructisn't optional and can't be null.Did I miss something here?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
My last comment is a little bit ambiguous. In the example I made, the Avro record
Structis theelementSchema, while the Parquet repeated grouparrayis therepeatedTypein the code.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I guess what is missing here is that we should check all the missing fields are indeed optional, unless this constraint has already been guaranteed somewhere else (then let's document it in the comment).
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This is for the case where only one field of a nested record is projected. Avro has a projection schema and an expected schema so they can differ. Previously, the logic to match the requested schema (elementSchema) with the file's structure checked whether the name of the single field matched the name of the single field of the element schema. But that left out the case where the two do match (as in your example) but the requested schema has an optional field that should be defaulted.
Whether the other field is optional doesn't matter at this point in the code. Avro fills in default values later and throws an exception if something is missing and has no default at that point.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Oh I see, this totally makes sense. Thanks for the explanation!