|
22 | 22 | import java.util.List;
|
23 | 23 | import java.util.Map;
|
24 | 24 |
|
| 25 | +import org.apache.parquet.example.data.Group; |
| 26 | +import org.apache.parquet.example.data.simple.BinaryValue; |
| 27 | +import org.apache.parquet.example.data.simple.BooleanValue; |
| 28 | +import org.apache.parquet.example.data.simple.DoubleValue; |
| 29 | +import org.apache.parquet.example.data.simple.FloatValue; |
| 30 | +import org.apache.parquet.example.data.simple.IntegerValue; |
| 31 | +import org.apache.parquet.example.data.simple.LongValue; |
| 32 | +import org.apache.parquet.io.api.Binary; |
| 33 | +import org.apache.parquet.schema.GroupType; |
| 34 | +import org.apache.parquet.schema.LogicalTypeAnnotation; |
| 35 | +import org.apache.parquet.schema.MessageType; |
| 36 | +import org.apache.parquet.schema.PrimitiveType; |
| 37 | +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName; |
| 38 | +import org.apache.parquet.schema.Type; |
| 39 | +import org.apache.parquet.schema.Types; |
| 40 | + |
25 | 41 | import com.google.gson.JsonArray;
|
26 | 42 | import com.google.gson.JsonElement;
|
27 | 43 | import com.google.gson.JsonObject;
|
28 | 44 |
|
29 |
| -import parquet.example.data.Group; |
30 |
| -import parquet.example.data.simple.BinaryValue; |
31 |
| -import parquet.example.data.simple.BooleanValue; |
32 |
| -import parquet.example.data.simple.DoubleValue; |
33 |
| -import parquet.example.data.simple.FloatValue; |
34 |
| -import parquet.example.data.simple.IntegerValue; |
35 |
| -import parquet.example.data.simple.LongValue; |
36 |
| -import parquet.io.api.Binary; |
37 |
| -import parquet.schema.GroupType; |
38 |
| -import parquet.schema.MessageType; |
39 |
| -import parquet.schema.PrimitiveType; |
40 |
| -import parquet.schema.PrimitiveType.PrimitiveTypeName; |
41 |
| -import parquet.schema.Type; |
42 |
| -import parquet.schema.Types; |
43 |
| - |
44 | 45 | import org.apache.gobblin.converter.parquet.JsonSchema.*;
|
45 | 46 |
|
| 47 | +import static org.apache.gobblin.converter.parquet.JsonElementConversionFactory.RecordConverter.RecordType.CHILD; |
46 | 48 | import static org.apache.gobblin.converter.parquet.JsonSchema.*;
|
47 | 49 | import static org.apache.gobblin.converter.parquet.JsonSchema.InputType.STRING;
|
48 |
| -import static org.apache.gobblin.converter.parquet.JsonElementConversionFactory.RecordConverter.RecordType.CHILD; |
49 |
| -import static parquet.schema.OriginalType.UTF8; |
50 |
| -import static parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY; |
51 |
| -import static parquet.schema.PrimitiveType.PrimitiveTypeName.INT32; |
52 |
| -import static parquet.schema.PrimitiveType.PrimitiveTypeName.INT64; |
53 |
| -import static parquet.schema.Type.Repetition.OPTIONAL; |
54 |
| -import static parquet.schema.Type.Repetition.REPEATED; |
55 |
| -import static parquet.schema.Type.Repetition.REQUIRED; |
| 50 | +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY; |
| 51 | +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32; |
| 52 | +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64; |
| 53 | +import static org.apache.parquet.schema.Type.Repetition.OPTIONAL; |
| 54 | +import static org.apache.parquet.schema.Type.Repetition.REPEATED; |
| 55 | +import static org.apache.parquet.schema.Type.Repetition.REQUIRED; |
56 | 56 |
|
57 | 57 |
|
58 | 58 | /**
|
@@ -293,13 +293,13 @@ BinaryValue convertField(JsonElement value) {
|
293 | 293 | protected Type buildSchema() {
|
294 | 294 | String columnName = this.jsonSchema.getColumnName();
|
295 | 295 | if (this.repeated) {
|
296 |
| - return Types.repeated(BINARY).as(UTF8).named(columnName); |
| 296 | + return Types.repeated(BINARY).as(LogicalTypeAnnotation.StringLogicalTypeAnnotation.stringType()).named(columnName); |
297 | 297 | }
|
298 | 298 | switch (optionalOrRequired(this.jsonSchema)) {
|
299 | 299 | case OPTIONAL:
|
300 |
| - return Types.optional(BINARY).as(UTF8).named(columnName); |
| 300 | + return Types.optional(BINARY).as(LogicalTypeAnnotation.StringLogicalTypeAnnotation.stringType()).named(columnName); |
301 | 301 | case REQUIRED:
|
302 |
| - return Types.required(BINARY).as(UTF8).named(columnName); |
| 302 | + return Types.required(BINARY).as(LogicalTypeAnnotation.StringLogicalTypeAnnotation.stringType()).named(columnName); |
303 | 303 | default:
|
304 | 304 | throw new RuntimeException("Unsupported Repetition type");
|
305 | 305 | }
|
|
0 commit comments