diff --git a/parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoSchemaConverter.java b/parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoSchemaConverter.java index 8c9685a9b0..3f6ed6bfc4 100644 --- a/parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoSchemaConverter.java +++ b/parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoSchemaConverter.java @@ -86,21 +86,21 @@ private Builder>, GroupBuilder> addF Type.Repetition repetition = getRepetition(descriptor); JavaType javaType = descriptor.getJavaType(); switch (javaType) { - case BOOLEAN : return builder.primitive(BOOLEAN, repetition); - case INT : return builder.primitive(INT32, repetition); - case LONG : return builder.primitive(INT64, repetition); - case FLOAT : return builder.primitive(FLOAT, repetition); - case DOUBLE: return builder.primitive(DOUBLE, repetition); - case BYTE_STRING: return builder.primitive(BINARY, repetition); - case STRING: return builder.primitive(BINARY, repetition).as(UTF8); - case MESSAGE: { - GroupBuilder> group = builder.group(repetition); - convertFields(group, descriptor.getMessageType().getFields()); - return group; - } - case ENUM: return builder.primitive(BINARY, repetition).as(ENUM); - default: - throw new UnsupportedOperationException("Cannot convert Protocol Buffer: unknown type " + javaType); + case BOOLEAN: return builder.primitive(BOOLEAN, repetition); + case INT: return builder.primitive(INT32, repetition); + case LONG: return builder.primitive(INT64, repetition); + case FLOAT: return builder.primitive(FLOAT, repetition); + case DOUBLE: return builder.primitive(DOUBLE, repetition); + case BYTE_STRING: return builder.primitive(BINARY, repetition); + case STRING: return builder.primitive(BINARY, repetition).as(UTF8); + case MESSAGE: { + GroupBuilder> group = builder.group(repetition); + convertFields(group, descriptor.getMessageType().getFields()); + return group; + } + case ENUM: return builder.primitive(BINARY, repetition).as(ENUM); + default: + throw new UnsupportedOperationException("Cannot convert Protocol Buffer: unknown type " + javaType); } } diff --git a/parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoWriteSupport.java b/parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoWriteSupport.java index d7f7a53512..cef2b93219 100644 --- a/parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoWriteSupport.java +++ b/parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoWriteSupport.java @@ -156,7 +156,6 @@ class MessageWriter extends FieldWriter { List fields = descriptor.getFields(); fieldWriters = (FieldWriter[]) Array.newInstance(FieldWriter.class, fields.size()); - int i = 0; for (Descriptors.FieldDescriptor fieldDescriptor: fields) { String name = fieldDescriptor.getName(); Type type = schema.getType(name); @@ -169,8 +168,7 @@ class MessageWriter extends FieldWriter { writer.setFieldName(name); writer.setIndex(schema.getFieldIndex(name)); - fieldWriters[i] = writer; - i++; + fieldWriters[fieldDescriptor.getIndex()] = writer; } } @@ -220,6 +218,13 @@ private void writeAllFields(MessageOrBuilder pb) { for (Map.Entry entry : changedPbFields.entrySet()) { Descriptors.FieldDescriptor fieldDescriptor = entry.getKey(); + + if(fieldDescriptor.isExtension()) { + // Field index of an extension field might overlap with a base field. + throw new UnsupportedOperationException( + "Cannot convert Protobuf message with extension field(s)"); + } + int fieldIndex = fieldDescriptor.getIndex(); fieldWriters[fieldIndex].writeField(entry.getValue()); } @@ -276,7 +281,7 @@ final void writeRawValue(Object value) { } class IntWriter extends FieldWriter { - @Override + @Override final void writeRawValue(Object value) { recordConsumer.addInteger((Integer) value); } diff --git a/parquet-protobuf/src/test/java/org/apache/parquet/proto/ProtoWriteSupportTest.java b/parquet-protobuf/src/test/java/org/apache/parquet/proto/ProtoWriteSupportTest.java index 73f77345b4..3a273c995f 100644 --- a/parquet-protobuf/src/test/java/org/apache/parquet/proto/ProtoWriteSupportTest.java +++ b/parquet-protobuf/src/test/java/org/apache/parquet/proto/ProtoWriteSupportTest.java @@ -165,4 +165,19 @@ public void testOptionalInnerMessage() throws Exception { inOrder.verify(readConsumerMock).endMessage(); Mockito.verifyNoMoreInteractions(readConsumerMock); } + + @Test(expected = UnsupportedOperationException.class) + public void testMessageWithExtensions() throws Exception { + RecordConsumer readConsumerMock = Mockito.mock(RecordConsumer.class); + ProtoWriteSupport instance = createReadConsumerInstance(TestProtobuf.Vehicle.class, readConsumerMock); + + TestProtobuf.Vehicle.Builder msg = TestProtobuf.Vehicle.newBuilder(); + msg.setHorsePower(300); + // Currently there's no support for extension fields. This test tests that the extension field + // will cause an exception. + msg.setExtension(TestProtobuf.Airplane.wingSpan, 50); + + instance.write(msg.build()); + } + } diff --git a/parquet-protobuf/src/test/resources/TestProtobuf.proto b/parquet-protobuf/src/test/resources/TestProtobuf.proto index afa0f63833..d7cdf03a91 100644 --- a/parquet-protobuf/src/test/resources/TestProtobuf.proto +++ b/parquet-protobuf/src/test/resources/TestProtobuf.proto @@ -137,3 +137,14 @@ message SecondCustomClassMessage { } //please place your unit test Protocol Buffer definitions here. + +message Vehicle { + optional int32 horsePower = 1; + extensions 100 to 199; +} + +message Airplane { + extend Vehicle { + optional int32 wingSpan = 101; + } +}