|
| 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.util; |
| 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 | + Map<String, List<Object>> collateEntries = collateEntries(record); |
| 74 | + return formatEntries(collateEntries); |
| 75 | + } |
| 76 | + |
| 77 | + private Map<String, List<Object>> collateEntries(SimpleRecord record) { |
| 78 | + Map<String, List<Object>> collatedEntries = new LinkedHashMap<String, List<Object>>(); |
| 79 | + for (SimpleRecord.NameValue value : record.getValues()) { |
| 80 | + if (collatedEntries.containsKey(value.getName())) { |
| 81 | + collatedEntries.get(value.getName()).add(value.getValue()); |
| 82 | + } else { |
| 83 | + List<Object> newResultListForKey = new ArrayList<Object>(); |
| 84 | + newResultListForKey.add(value.getValue()); |
| 85 | + collatedEntries.put(value.getName(), newResultListForKey); |
| 86 | + } |
| 87 | + } |
| 88 | + |
| 89 | + return collatedEntries; |
| 90 | + } |
| 91 | + |
| 92 | + private Object formatEntries(Map<String, List<Object>> entries) { |
| 93 | + Map<String, Object> results = new LinkedHashMap<String, Object>(); |
| 94 | + for (Map.Entry<String, List<Object>> entry : entries.entrySet()) { |
| 95 | + JsonRecordFormatter formatter = formatters.get(entry.getKey()); |
| 96 | + results.put(entry.getKey(), formatter.formatResults(entry.getValue())); |
| 97 | + } |
| 98 | + |
| 99 | + return results; |
| 100 | + } |
| 101 | + |
| 102 | + @Override |
| 103 | + protected Object formatResults(List<SimpleRecord> values) { |
| 104 | + if (super.typeInfo.getRepetition() == Type.Repetition.REPEATED) { |
| 105 | + List<Object> results = new ArrayList<Object>(); |
| 106 | + for (SimpleRecord object : values) { |
| 107 | + results.add(add(object)); |
| 108 | + } |
| 109 | + |
| 110 | + return results; |
| 111 | + } else { |
| 112 | + return add(values.get(SINGLE_VALUE)); |
| 113 | + } |
| 114 | + } |
| 115 | + |
| 116 | + public String formatRecord(SimpleRecord value) throws IOException { |
| 117 | + ObjectMapper mapper = new ObjectMapper(); |
| 118 | + return mapper.writeValueAsString(add(value)); |
| 119 | + } |
| 120 | + } |
| 121 | + |
| 122 | + protected final Type typeInfo; |
| 123 | + |
| 124 | + protected JsonRecordFormatter(Type type) { |
| 125 | + this.typeInfo = type; |
| 126 | + } |
| 127 | + |
| 128 | + protected abstract Object formatResults(List<T> values); |
| 129 | + |
| 130 | + public static JsonGroupFormatter fromSchema(MessageType messageType) { |
| 131 | + return new JsonGroupFormatter(messageType); |
| 132 | + } |
| 133 | +} |
0 commit comments