-
Notifications
You must be signed in to change notification settings - Fork 1.5k
PARQUET-367: "parquet-cat -j" doesn't show all records. #281
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
sircodesalotOfTheRound
wants to merge
1
commit into
apache:master
from
sircodesalotOfTheRound:fix-parquet-cat
Closed
Changes from all commits
Commits
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
132 changes: 132 additions & 0 deletions
132
parquet-tools/src/main/java/org/apache/parquet/tools/json/JsonRecordFormatter.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,132 @@ | ||
| /* | ||
| * 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.tools.json; | ||
|
|
||
| import org.apache.parquet.schema.GroupType; | ||
| import org.apache.parquet.schema.MessageType; | ||
| import org.apache.parquet.schema.Type; | ||
| import org.apache.parquet.tools.read.SimpleRecord; | ||
| import org.codehaus.jackson.map.ObjectMapper; | ||
|
|
||
| import java.io.IOException; | ||
| import java.util.*; | ||
|
|
||
| public abstract class JsonRecordFormatter<T> { | ||
| private static final int SINGLE_VALUE = 0; | ||
|
|
||
| public static class JsonPrimitiveWriter extends JsonRecordFormatter<Object> { | ||
|
|
||
| public JsonPrimitiveWriter(Type primitiveType) { | ||
| super(primitiveType); | ||
| } | ||
|
|
||
| @Override | ||
| protected Object formatResults(List<Object> listOfValues) { | ||
| if (super.typeInfo.getRepetition() == Type.Repetition.REPEATED) { | ||
| return listOfValues; | ||
| } else { | ||
| return listOfValues.get(SINGLE_VALUE); | ||
| } | ||
| } | ||
| } | ||
|
|
||
| public static class JsonGroupFormatter extends JsonRecordFormatter<SimpleRecord> { | ||
| private final Map<String, JsonRecordFormatter> formatters; | ||
|
|
||
| public JsonGroupFormatter(GroupType schema) { | ||
| super(schema); | ||
|
|
||
| this.formatters = buildWriters(schema); | ||
| } | ||
|
|
||
| private Map<String, JsonRecordFormatter> buildWriters(GroupType groupSchema) { | ||
| Map<String, JsonRecordFormatter> writers = new LinkedHashMap<String, JsonRecordFormatter>(); | ||
| for (Type type : groupSchema.getFields()) { | ||
| if (type.isPrimitive()) { | ||
| writers.put(type.getName(), new JsonPrimitiveWriter(type)); | ||
| } else { | ||
| writers.put(type.getName(), new JsonGroupFormatter((GroupType) type)); | ||
| } | ||
| } | ||
|
|
||
| return writers; | ||
| } | ||
|
|
||
| private Object add(SimpleRecord record) { | ||
| return formatEntries(collateEntries(record)); | ||
| } | ||
|
|
||
| private Map<String, List<Object>> collateEntries(SimpleRecord record) { | ||
| Map<String, List<Object>> collatedEntries = new LinkedHashMap<String, List<Object>>(); | ||
| for (SimpleRecord.NameValue value : record.getValues()) { | ||
| if (collatedEntries.containsKey(value.getName())) { | ||
| collatedEntries.get(value.getName()).add(value.getValue()); | ||
| } else { | ||
| List<Object> newResultListForKey = new ArrayList<Object>(); | ||
| newResultListForKey.add(value.getValue()); | ||
| collatedEntries.put(value.getName(), newResultListForKey); | ||
| } | ||
| } | ||
|
|
||
| return collatedEntries; | ||
| } | ||
|
|
||
| private Object formatEntries(Map<String, List<Object>> entries) { | ||
| Map<String, Object> results = new LinkedHashMap<String, Object>(); | ||
| for (Map.Entry<String, List<Object>> entry : entries.entrySet()) { | ||
| JsonRecordFormatter formatter = formatters.get(entry.getKey()); | ||
| results.put(entry.getKey(), formatter.formatResults(entry.getValue())); | ||
| } | ||
|
|
||
| return results; | ||
| } | ||
|
|
||
| @Override | ||
| protected Object formatResults(List<SimpleRecord> values) { | ||
| if (super.typeInfo.getRepetition() == Type.Repetition.REPEATED) { | ||
| List<Object> results = new ArrayList<Object>(); | ||
| for (SimpleRecord object : values) { | ||
| results.add(add(object)); | ||
| } | ||
|
|
||
| return results; | ||
| } else { | ||
| return add(values.get(SINGLE_VALUE)); | ||
| } | ||
| } | ||
|
|
||
| public String formatRecord(SimpleRecord value) throws IOException { | ||
| ObjectMapper mapper = new ObjectMapper(); | ||
| return mapper.writeValueAsString(add(value)); | ||
| } | ||
| } | ||
|
|
||
| protected final Type typeInfo; | ||
|
|
||
| protected JsonRecordFormatter(Type type) { | ||
| this.typeInfo = type; | ||
| } | ||
|
|
||
| protected abstract Object formatResults(List<T> values); | ||
|
|
||
| public static JsonGroupFormatter fromSchema(MessageType messageType) { | ||
| return new JsonGroupFormatter(messageType); | ||
| } | ||
| } |
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.
Could you explain how this was missing records? Was it a bug in the prettyPrintJson method?
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.
The primary issue is that
SimpleObjectdoesn't understand the concept of an array. It simulates an array by adding multiple items to the same key. The problem with that approach is that the Json representation then depends on the amount of data written. A key with only one value would appear as a singleton-object, while items with multiple entries appended to the same key would appear as an array. This would result in an inconsistent json encoding.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.
Reading you comment again I think I misunderstood your question. Basically, the issue is with
prettyPrintJson:It's the same basic issue I outlined in my previous comment (but I was addressing the design choice rather than why it was failing in the first place - which was your actual question I think). Please see the comment above.
Thanks