You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@parquet.apache.org by ga...@apache.org on 2020/10/22 14:54:39 UTC
[parquet-mr] branch master updated: PARQUET-1917: Don't write
values for oneOf fields that aren't set (#820)
This is an automated email from the ASF dual-hosted git repository.
gabor pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/parquet-mr.git
The following commit(s) were added to refs/heads/master by this push:
new 14b7a8b PARQUET-1917: Don't write values for oneOf fields that aren't set (#820)
14b7a8b is described below
commit 14b7a8be8ac98c7f96fc3fd2d5c21cbddf736b64
Author: Aaron Niskode-Dossett <do...@gmail.com>
AuthorDate: Thu Oct 22 09:54:29 2020 -0500
PARQUET-1917: Don't write values for oneOf fields that aren't set (#820)
---
.../apache/parquet/proto/ProtoWriteSupport.java | 6 +++
.../parquet/proto/ProtoWriteSupportTest.java | 63 ++++++++++++++++++++++
.../src/test/resources/TestProto3.proto | 9 ++++
3 files changed, 78 insertions(+)
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 2322667..b0a0e50 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
@@ -337,6 +337,12 @@ public class ProtoWriteSupport<T extends MessageOrBuilder> extends WriteSupport<
List<FieldDescriptor> fieldDescriptors = messageDescriptor.getFields();
for (FieldDescriptor fieldDescriptor : fieldDescriptors) {
FieldDescriptor.Type type = fieldDescriptor.getType();
+
+ //For a field in a oneOf that isn't set don't write anything
+ if (fieldDescriptor.getContainingOneof() != null && !pb.hasField(fieldDescriptor)) {
+ continue;
+ }
+
if (!fieldDescriptor.isRepeated() && FieldDescriptor.Type.MESSAGE.equals(type) && !pb.hasField(fieldDescriptor)) {
continue;
}
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 2a7b7f5..36140d3 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
@@ -20,7 +20,9 @@ package org.apache.parquet.proto;
import com.google.protobuf.Message;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
import org.junit.Test;
+import static org.junit.Assert.*;
import org.mockito.InOrder;
import org.mockito.Mockito;
import org.apache.parquet.io.api.Binary;
@@ -28,6 +30,9 @@ import org.apache.parquet.io.api.RecordConsumer;
import org.apache.parquet.proto.test.TestProto3;
import org.apache.parquet.proto.test.TestProtobuf;
+import java.io.IOException;
+import java.util.List;
+
public class ProtoWriteSupportTest {
private <T extends Message> ProtoWriteSupport<T> createReadConsumerInstance(Class<T> cls, RecordConsumer readConsumerMock) {
@@ -913,4 +918,62 @@ public class ProtoWriteSupportTest {
instance.write(msg.build());
}
+
+ @Test
+ public void testMessageOneOf() {
+ RecordConsumer readConsumerMock = Mockito.mock(RecordConsumer.class);
+ ProtoWriteSupport<TestProto3.OneOfTestMessage> spyWriter = createReadConsumerInstance(TestProto3.OneOfTestMessage.class, readConsumerMock);
+ final int theInt = 99;
+
+ TestProto3.OneOfTestMessage.Builder msg = TestProto3.OneOfTestMessage.newBuilder();
+ msg.setSecond(theInt);
+ spyWriter.write(msg.build());
+
+ InOrder inOrder = Mockito.inOrder(readConsumerMock);
+
+ inOrder.verify(readConsumerMock).startMessage();
+ inOrder.verify(readConsumerMock).startField("second", 1);
+ inOrder.verify(readConsumerMock).addInteger(theInt);
+ inOrder.verify(readConsumerMock).endField("second", 1);
+ inOrder.verify(readConsumerMock).endMessage();
+ Mockito.verifyNoMoreInteractions(readConsumerMock);
+ }
+
+ /**
+ * Ensure that a message with a oneOf gets written out correctly and can be
+ * read back as expected.
+ */
+ @Test
+ public void testMessageOneOfRoundTrip() throws IOException {
+
+ TestProto3.OneOfTestMessage.Builder msgBuilder = TestProto3.OneOfTestMessage.newBuilder();
+ msgBuilder.setSecond(99);
+ TestProto3.OneOfTestMessage theMessage = msgBuilder.build();
+
+ TestProto3.OneOfTestMessage.Builder msgBuilder2 = TestProto3.OneOfTestMessage.newBuilder();
+ TestProto3.OneOfTestMessage theMessageNothingSet = msgBuilder2.build();
+
+ TestProto3.OneOfTestMessage.Builder msgBuilder3 = TestProto3.OneOfTestMessage.newBuilder();
+ msgBuilder3.setFirst(42);
+ TestProto3.OneOfTestMessage theMessageFirstSet = msgBuilder3.build();
+
+ //Write them out and read them back
+ Path tmpFilePath = TestUtils.writeMessages(theMessage, theMessageNothingSet, theMessageFirstSet);
+ List<TestProto3.OneOfTestMessage> gotBack = TestUtils.readMessages(tmpFilePath, TestProto3.OneOfTestMessage.class);
+
+ //First message
+ TestProto3.OneOfTestMessage gotBackFirst = gotBack.get(0);
+ assertEquals(gotBackFirst.getSecond(), 99);
+ assertEquals(gotBackFirst.getTheOneofCase(), TestProto3.OneOfTestMessage.TheOneofCase.SECOND);
+
+ //Second message with nothing set
+ TestProto3.OneOfTestMessage gotBackSecond = gotBack.get(1);
+ assertEquals(gotBackSecond.getTheOneofCase(), TestProto3.OneOfTestMessage.TheOneofCase.THEONEOF_NOT_SET);
+
+ //Third message with opposite field set
+ TestProto3.OneOfTestMessage gotBackThird = gotBack.get(2);
+ assertEquals(gotBackThird.getFirst(), 42);
+ assertEquals(gotBackThird.getTheOneofCase(), TestProto3.OneOfTestMessage.TheOneofCase.FIRST);
+
+ }
}
diff --git a/parquet-protobuf/src/test/resources/TestProto3.proto b/parquet-protobuf/src/test/resources/TestProto3.proto
index 5729ef5..fb4da1b 100644
--- a/parquet-protobuf/src/test/resources/TestProto3.proto
+++ b/parquet-protobuf/src/test/resources/TestProto3.proto
@@ -107,6 +107,15 @@ message IOFormatMessage {
//end protocol buffers for ProtoInputOutputFormatTest
+//begin protocol buffers for ProtoWriteSupport
+message OneOfTestMessage {
+ oneof the_oneof {
+ int32 first = 1;
+ int32 second = 2;
+ }
+}
+
+//end protocol buffers for ProtoWriteSupport
message InnerMessage {
string one = 1;