Skip to content

Commit 67207ef

Browse files
PARQUET-367: "parquet-cat -j" doesn't show all records.
Added JsonRecordFormatter which formats SimpleRecords into an structure that can be used with ObjectMapper to create a valid json structure. Unit test included.
1 parent c381968 commit 67207ef

File tree

4 files changed

+372
-1
lines changed

4 files changed

+372
-1
lines changed

parquet-tools/src/main/java/org/apache/parquet/tools/command/CatCommand.java

Lines changed: 8 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -24,12 +24,16 @@
2424
import org.apache.commons.cli.Option;
2525
import org.apache.commons.cli.OptionBuilder;
2626
import org.apache.commons.cli.Options;
27+
import org.apache.hadoop.conf.Configuration;
2728
import org.apache.hadoop.fs.Path;
2829

30+
import org.apache.parquet.hadoop.ParquetFileReader;
2931
import org.apache.parquet.hadoop.ParquetReader;
32+
import org.apache.parquet.hadoop.metadata.ParquetMetadata;
3033
import org.apache.parquet.tools.Main;
3134
import org.apache.parquet.tools.read.SimpleReadSupport;
3235
import org.apache.parquet.tools.read.SimpleRecord;
36+
import org.apache.parquet.tools.json.JsonRecordFormatter;
3337

3438
public class CatCommand extends ArgsOnlyCommand {
3539
public static final String[] USAGE = new String[] {
@@ -71,9 +75,12 @@ public void execute(CommandLine options) throws Exception {
7175
try {
7276
PrintWriter writer = new PrintWriter(Main.out, true);
7377
reader = ParquetReader.builder(new SimpleReadSupport(), new Path(input)).build();
78+
ParquetMetadata metadata = ParquetFileReader.readFooter(new Configuration(), new Path(input));
79+
JsonRecordFormatter.JsonGroupFormatter formatter = JsonRecordFormatter.fromSchema(metadata.getFileMetaData().getSchema());
80+
7481
for (SimpleRecord value = reader.read(); value != null; value = reader.read()) {
7582
if (options.hasOption('j')) {
76-
value.prettyPrintJson(writer);
83+
writer.write(formatter.formatRecord(value));
7784
} else {
7885
value.prettyPrint(writer);
7986
}
Lines changed: 132 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,132 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing,
13+
* software distributed under the License is distributed on an
14+
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15+
* KIND, either express or implied. See the License for the
16+
* specific language governing permissions and limitations
17+
* under the License.
18+
*/
19+
20+
package org.apache.parquet.tools.json;
21+
22+
import org.apache.parquet.schema.GroupType;
23+
import org.apache.parquet.schema.MessageType;
24+
import org.apache.parquet.schema.Type;
25+
import org.apache.parquet.tools.read.SimpleRecord;
26+
import org.codehaus.jackson.map.ObjectMapper;
27+
28+
import java.io.IOException;
29+
import java.util.*;
30+
31+
public abstract class JsonRecordFormatter<T> {
32+
private static final int SINGLE_VALUE = 0;
33+
34+
public static class JsonPrimitiveWriter extends JsonRecordFormatter<Object> {
35+
36+
public JsonPrimitiveWriter(Type primitiveType) {
37+
super(primitiveType);
38+
}
39+
40+
@Override
41+
protected Object formatResults(List<Object> listOfValues) {
42+
if (super.typeInfo.getRepetition() == Type.Repetition.REPEATED) {
43+
return listOfValues;
44+
} else {
45+
return listOfValues.get(SINGLE_VALUE);
46+
}
47+
}
48+
}
49+
50+
public static class JsonGroupFormatter extends JsonRecordFormatter<SimpleRecord> {
51+
private final Map<String, JsonRecordFormatter> formatters;
52+
53+
public JsonGroupFormatter(GroupType schema) {
54+
super(schema);
55+
56+
this.formatters = buildWriters(schema);
57+
}
58+
59+
private Map<String, JsonRecordFormatter> buildWriters(GroupType groupSchema) {
60+
Map<String, JsonRecordFormatter> writers = new LinkedHashMap<String, JsonRecordFormatter>();
61+
for (Type type : groupSchema.getFields()) {
62+
if (type.isPrimitive()) {
63+
writers.put(type.getName(), new JsonPrimitiveWriter(type));
64+
} else {
65+
writers.put(type.getName(), new JsonGroupFormatter((GroupType) type));
66+
}
67+
}
68+
69+
return writers;
70+
}
71+
72+
private Object add(SimpleRecord record) {
73+
return formatEntries(collateEntries(record));
74+
}
75+
76+
private Map<String, List<Object>> collateEntries(SimpleRecord record) {
77+
Map<String, List<Object>> collatedEntries = new LinkedHashMap<String, List<Object>>();
78+
for (SimpleRecord.NameValue value : record.getValues()) {
79+
if (collatedEntries.containsKey(value.getName())) {
80+
collatedEntries.get(value.getName()).add(value.getValue());
81+
} else {
82+
List<Object> newResultListForKey = new ArrayList<Object>();
83+
newResultListForKey.add(value.getValue());
84+
collatedEntries.put(value.getName(), newResultListForKey);
85+
}
86+
}
87+
88+
return collatedEntries;
89+
}
90+
91+
private Object formatEntries(Map<String, List<Object>> entries) {
92+
Map<String, Object> results = new LinkedHashMap<String, Object>();
93+
for (Map.Entry<String, List<Object>> entry : entries.entrySet()) {
94+
JsonRecordFormatter formatter = formatters.get(entry.getKey());
95+
results.put(entry.getKey(), formatter.formatResults(entry.getValue()));
96+
}
97+
98+
return results;
99+
}
100+
101+
@Override
102+
protected Object formatResults(List<SimpleRecord> values) {
103+
if (super.typeInfo.getRepetition() == Type.Repetition.REPEATED) {
104+
List<Object> results = new ArrayList<Object>();
105+
for (SimpleRecord object : values) {
106+
results.add(add(object));
107+
}
108+
109+
return results;
110+
} else {
111+
return add(values.get(SINGLE_VALUE));
112+
}
113+
}
114+
115+
public String formatRecord(SimpleRecord value) throws IOException {
116+
ObjectMapper mapper = new ObjectMapper();
117+
return mapper.writeValueAsString(add(value));
118+
}
119+
}
120+
121+
protected final Type typeInfo;
122+
123+
protected JsonRecordFormatter(Type type) {
124+
this.typeInfo = type;
125+
}
126+
127+
protected abstract Object formatResults(List<T> values);
128+
129+
public static JsonGroupFormatter fromSchema(MessageType messageType) {
130+
return new JsonGroupFormatter(messageType);
131+
}
132+
}

parquet-tools/src/main/java/org/apache/parquet/tools/read/SimpleRecord.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -111,6 +111,7 @@ protected Object toJsonObject() {
111111
for (NameValue value : values) {
112112
result.put(value.getName(), toJsonValue(value.getValue()));
113113
}
114+
114115
return result;
115116
}
116117

0 commit comments

Comments
 (0)