-
Notifications
You must be signed in to change notification settings - Fork 1.4k
/
ProtoMessageConverter.java
469 lines (366 loc) · 14.2 KB
/
ProtoMessageConverter.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
/*
* 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.proto;
import com.google.protobuf.ByteString;
import com.google.protobuf.Descriptors;
import com.google.protobuf.Message;
import com.twitter.elephantbird.util.Protobufs;
import org.apache.parquet.column.Dictionary;
import org.apache.parquet.io.InvalidRecordException;
import org.apache.parquet.io.ParquetDecodingException;
import org.apache.parquet.io.api.Binary;
import org.apache.parquet.io.api.Converter;
import org.apache.parquet.io.api.GroupConverter;
import org.apache.parquet.io.api.PrimitiveConverter;
import org.apache.parquet.schema.GroupType;
import org.apache.parquet.schema.IncompatibleSchemaModificationException;
import org.apache.parquet.schema.OriginalType;
import org.apache.parquet.schema.Type;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import static com.google.protobuf.Descriptors.FieldDescriptor.JavaType;
/**
* Converts Protocol Buffer message (both top level and inner) to parquet.
* This is internal class, use {@link ProtoRecordConverter}.
*/
class ProtoMessageConverter extends GroupConverter {
private final Converter[] converters;
private final ParentValueContainer parent;
private final Message.Builder myBuilder;
// used in record converter
ProtoMessageConverter(ParentValueContainer pvc, Class<? extends Message> protoClass, GroupType parquetSchema) {
this(pvc, Protobufs.getMessageBuilder(protoClass), parquetSchema);
}
// For usage in message arrays
ProtoMessageConverter(ParentValueContainer pvc, Message.Builder builder, GroupType parquetSchema) {
int schemaSize = parquetSchema.getFieldCount();
converters = new Converter[schemaSize];
this.parent = pvc;
int parquetFieldIndex = 1;
if (pvc == null) {
throw new IllegalStateException("Missing parent value container");
}
myBuilder = builder;
Descriptors.Descriptor protoDescriptor = builder.getDescriptorForType();
for (Type parquetField : parquetSchema.getFields()) {
Descriptors.FieldDescriptor protoField = protoDescriptor.findFieldByName(parquetField.getName());
if (protoField == null) {
String description = "Scheme mismatch \n\"" + parquetField + "\"" +
"\n proto descriptor:\n" + protoDescriptor.toProto();
throw new IncompatibleSchemaModificationException("Cant find \"" + parquetField.getName() + "\" " + description);
}
converters[parquetFieldIndex - 1] = newMessageConverter(myBuilder, protoField, parquetField);
parquetFieldIndex++;
}
}
@Override
public Converter getConverter(int fieldIndex) {
return converters[fieldIndex];
}
@Override
public void start() {
}
@Override
public void end() {
parent.add(myBuilder.build());
myBuilder.clear();
}
private Converter newMessageConverter(final Message.Builder parentBuilder, final Descriptors.FieldDescriptor fieldDescriptor, Type parquetType) {
boolean isRepeated = fieldDescriptor.isRepeated();
ParentValueContainer parent;
if (isRepeated) {
parent = new ParentValueContainer() {
@Override
public void add(Object value) {
parentBuilder.addRepeatedField(fieldDescriptor, value);
}
};
} else {
parent = new ParentValueContainer() {
@Override
public void add(Object value) {
parentBuilder.setField(fieldDescriptor, value);
}
};
}
if (OriginalType.LIST == parquetType.getOriginalType()) {
return new ListConverter(parentBuilder, fieldDescriptor, parquetType);
}
if (OriginalType.MAP == parquetType.getOriginalType()) {
return new MapConverter(parentBuilder, fieldDescriptor, parquetType);
}
return newScalarConverter(parent, parentBuilder, fieldDescriptor, parquetType);
}
private Converter newScalarConverter(ParentValueContainer pvc, Message.Builder parentBuilder, Descriptors.FieldDescriptor fieldDescriptor, Type parquetType) {
JavaType javaType = fieldDescriptor.getJavaType();
switch (javaType) {
case STRING: return new ProtoStringConverter(pvc);
case FLOAT: return new ProtoFloatConverter(pvc);
case DOUBLE: return new ProtoDoubleConverter(pvc);
case BOOLEAN: return new ProtoBooleanConverter(pvc);
case BYTE_STRING: return new ProtoBinaryConverter(pvc);
case ENUM: return new ProtoEnumConverter(pvc, fieldDescriptor);
case INT: return new ProtoIntConverter(pvc);
case LONG: return new ProtoLongConverter(pvc);
case MESSAGE: {
Message.Builder subBuilder = parentBuilder.newBuilderForField(fieldDescriptor);
return new ProtoMessageConverter(pvc, subBuilder, parquetType.asGroupType());
}
}
throw new UnsupportedOperationException(String.format("Cannot convert type: %s" +
" (Parquet type: %s) ", javaType, parquetType));
}
public Message.Builder getBuilder() {
return myBuilder;
}
static abstract class ParentValueContainer {
/**
* Adds the value to the parent.
*/
public abstract void add(Object value);
}
final class ProtoEnumConverter extends PrimitiveConverter {
private final Descriptors.FieldDescriptor fieldType;
private final Map<Binary, Descriptors.EnumValueDescriptor> enumLookup;
private Descriptors.EnumValueDescriptor[] dict;
private final ParentValueContainer parent;
public ProtoEnumConverter(ParentValueContainer parent, Descriptors.FieldDescriptor fieldType) {
this.parent = parent;
this.fieldType = fieldType;
this.enumLookup = makeLookupStructure(fieldType);
}
/**
* Fills lookup structure for translating between parquet enum values and Protocol buffer enum values.
* */
private Map<Binary, Descriptors.EnumValueDescriptor> makeLookupStructure(Descriptors.FieldDescriptor enumFieldType) {
Descriptors.EnumDescriptor enumType = enumFieldType.getEnumType();
Map<Binary, Descriptors.EnumValueDescriptor> lookupStructure = new HashMap<Binary, Descriptors.EnumValueDescriptor>();
List<Descriptors.EnumValueDescriptor> enumValues = enumType.getValues();
for (Descriptors.EnumValueDescriptor value : enumValues) {
String name = value.getName();
lookupStructure.put(Binary.fromString(name), enumType.findValueByName(name));
}
return lookupStructure;
}
/**
* Translates given parquet enum value to protocol buffer enum value.
* @throws org.apache.parquet.io.InvalidRecordException is there is no corresponding value.
* */
private Descriptors.EnumValueDescriptor translateEnumValue(Binary binaryValue) {
Descriptors.EnumValueDescriptor protoValue = enumLookup.get(binaryValue);
if (protoValue == null) {
Set<Binary> knownValues = enumLookup.keySet();
String msg = "Illegal enum value \"" + binaryValue + "\""
+ " in protocol buffer \"" + fieldType.getFullName() + "\""
+ " legal values are: \"" + knownValues + "\"";
throw new InvalidRecordException(msg);
}
return protoValue;
}
@Override
final public void addBinary(Binary binaryValue) {
Descriptors.EnumValueDescriptor protoValue = translateEnumValue(binaryValue);
parent.add(protoValue);
}
@Override
public void addValueFromDictionary(int dictionaryId) {
parent.add(dict[dictionaryId]);
}
@Override
public boolean hasDictionarySupport() {
return true;
}
@Override
public void setDictionary(Dictionary dictionary) {
dict = new Descriptors.EnumValueDescriptor[dictionary.getMaxId() + 1];
for (int i = 0; i <= dictionary.getMaxId(); i++) {
Binary binaryValue = dictionary.decodeToBinary(i);
dict[i] = translateEnumValue(binaryValue);
}
}
}
final class ProtoBinaryConverter extends PrimitiveConverter {
final ParentValueContainer parent;
public ProtoBinaryConverter(ParentValueContainer parent) {
this.parent = parent;
}
@Override
public void addBinary(Binary binary) {
ByteString byteString = ByteString.copyFrom(binary.toByteBuffer());
parent.add(byteString);
}
}
final class ProtoBooleanConverter extends PrimitiveConverter {
final ParentValueContainer parent;
public ProtoBooleanConverter(ParentValueContainer parent) {
this.parent = parent;
}
@Override
final public void addBoolean(boolean value) {
parent.add(value);
}
}
final class ProtoDoubleConverter extends PrimitiveConverter {
final ParentValueContainer parent;
public ProtoDoubleConverter(ParentValueContainer parent) {
this.parent = parent;
}
@Override
public void addDouble(double value) {
parent.add(value);
}
}
final class ProtoFloatConverter extends PrimitiveConverter {
final ParentValueContainer parent;
public ProtoFloatConverter(ParentValueContainer parent) {
this.parent = parent;
}
@Override
public void addFloat(float value) {
parent.add(value);
}
}
final class ProtoIntConverter extends PrimitiveConverter {
final ParentValueContainer parent;
public ProtoIntConverter(ParentValueContainer parent) {
this.parent = parent;
}
@Override
public void addInt(int value) {
parent.add(value);
}
}
final class ProtoLongConverter extends PrimitiveConverter {
final ParentValueContainer parent;
public ProtoLongConverter(ParentValueContainer parent) {
this.parent = parent;
}
@Override
public void addLong(long value) {
parent.add(value);
}
}
final class ProtoStringConverter extends PrimitiveConverter {
final ParentValueContainer parent;
public ProtoStringConverter(ParentValueContainer parent) {
this.parent = parent;
}
@Override
public void addBinary(Binary binary) {
String str = binary.toStringUsingUTF8();
parent.add(str);
}
}
/**
* This class unwraps the additional LIST wrapper and makes it possible to read the underlying data and then convert
* it to protobuf.
* <p>
* Consider the following protobuf schema:
* message SimpleList {
* repeated int64 first_array = 1;
* }
* <p>
* A LIST wrapper is created in parquet for the above mentioned protobuf schema:
* message SimpleList {
* optional group first_array (LIST) = 1 {
* repeated group list {
* optional int32 element;
* }
* }
* }
* <p>
* The LIST wrappers are used by 3rd party tools, such as Hive, to read parquet arrays. The wrapper contains
* a repeated group named 'list', itself containing only one field called 'element' of the type of the repeated
* object (can be a primitive as in this example or a group in case of a repeated message in protobuf).
*/
final class ListConverter extends GroupConverter {
private final Converter converter;
public ListConverter(Message.Builder parentBuilder, Descriptors.FieldDescriptor fieldDescriptor, Type parquetType) {
OriginalType originalType = parquetType.getOriginalType();
if (originalType != OriginalType.LIST || parquetType.isPrimitive()) {
throw new ParquetDecodingException("Expected LIST wrapper. Found: " + originalType + " instead.");
}
GroupType rootWrapperType = parquetType.asGroupType();
if (!rootWrapperType.containsField("list") || rootWrapperType.getType("list").isPrimitive()) {
throw new ParquetDecodingException("Expected repeated 'list' group inside LIST wrapperr but got: " + rootWrapperType);
}
GroupType listType = rootWrapperType.getType("list").asGroupType();
if (!listType.containsField("element")) {
throw new ParquetDecodingException("Expected 'element' inside repeated list group but got: " + listType);
}
Type elementType = listType.getType("element");
converter = newMessageConverter(parentBuilder, fieldDescriptor, elementType);
}
@Override
public Converter getConverter(int fieldIndex) {
if (fieldIndex > 0) {
throw new ParquetDecodingException("Unexpected multiple fields in the LIST wrapper");
}
return new GroupConverter() {
@Override
public Converter getConverter(int fieldIndex) {
return converter;
}
@Override
public void start() {
}
@Override
public void end() {
}
};
}
@Override
public void start() {
}
@Override
public void end() {
}
}
final class MapConverter extends GroupConverter {
private final Converter converter;
public MapConverter(Message.Builder parentBuilder, Descriptors.FieldDescriptor fieldDescriptor, Type parquetType) {
OriginalType originalType = parquetType.getOriginalType();
if (originalType != OriginalType.MAP) {
throw new ParquetDecodingException("Expected MAP wrapper. Found: " + originalType + " instead.");
}
Type parquetSchema;
if (parquetType.asGroupType().containsField("key_value")){
parquetSchema = parquetType.asGroupType().getType("key_value");
} else {
throw new ParquetDecodingException("Expected map but got: " + parquetType);
}
converter = newMessageConverter(parentBuilder, fieldDescriptor, parquetSchema);
}
@Override
public Converter getConverter(int fieldIndex) {
if (fieldIndex > 0) {
throw new ParquetDecodingException("Unexpected multiple fields in the MAP wrapper");
}
return converter;
}
@Override
public void start() {
}
@Override
public void end() {
}
}
}