parquet-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ziva...@apache.org
Subject [parquet-mr] branch master updated: PARQUET-1478: Can't read spec compliant, 3-level lists via parquet-proto (#578)
Date Mon, 07 Jan 2019 15:46:12 GMT
This is an automated email from the ASF dual-hosted git repository.

zivanfi 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 58edbd7  PARQUET-1478: Can't read spec compliant, 3-level lists via parquet-proto
(#578)
58edbd7 is described below

commit 58edbd78ca93b360daee791376e4b1bcf01c5f61
Author: nandorKollar <nandorKollar@users.noreply.github.com>
AuthorDate: Mon Jan 7 16:46:06 2019 +0100

    PARQUET-1478: Can't read spec compliant, 3-level lists via parquet-proto (#578)
---
 .../apache/parquet/proto/ProtoMessageConverter.java   |  2 +-
 .../parquet/proto/ProtoInputOutputFormatTest.java     |  4 ++--
 .../org/apache/parquet/proto/utils/ReadUsingMR.java   | 19 +++++++++++++------
 .../org/apache/parquet/proto/utils/WriteUsingMR.java  |  9 ++++-----
 4 files changed, 20 insertions(+), 14 deletions(-)

diff --git a/parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoMessageConverter.java
b/parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoMessageConverter.java
index 92d8b62..173fa77 100644
--- a/parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoMessageConverter.java
+++ b/parquet-protobuf/src/main/java/org/apache/parquet/proto/ProtoMessageConverter.java
@@ -145,7 +145,7 @@ class ProtoMessageConverter extends GroupConverter {
       public Optional<Converter> visit(LogicalTypeAnnotation.MapLogicalTypeAnnotation
mapLogicalType) {
         return of(new MapConverter(parentBuilder, fieldDescriptor, parquetType));
       }
-    }).orElse(newScalarConverter(parent, parentBuilder, fieldDescriptor, parquetType));
+    }).orElseGet(() -> newScalarConverter(parent, parentBuilder, fieldDescriptor, parquetType));
   }
 
   private Converter newScalarConverter(ParentValueContainer pvc, Message.Builder parentBuilder,
Descriptors.FieldDescriptor fieldDescriptor, Type parquetType) {
diff --git a/parquet-protobuf/src/test/java/org/apache/parquet/proto/ProtoInputOutputFormatTest.java
b/parquet-protobuf/src/test/java/org/apache/parquet/proto/ProtoInputOutputFormatTest.java
index 5544dc6..f40b202 100644
--- a/parquet-protobuf/src/test/java/org/apache/parquet/proto/ProtoInputOutputFormatTest.java
+++ b/parquet-protobuf/src/test/java/org/apache/parquet/proto/ProtoInputOutputFormatTest.java
@@ -262,7 +262,7 @@ public class ProtoInputOutputFormatTest {
     ProtoWriteSupport.setWriteSpecsCompliant(conf, true);
 
     Path outputPath = new WriteUsingMR(conf).write(msgEmpty, msgNonEmpty);
-    ReadUsingMR readUsingMR = new ReadUsingMR();
+    ReadUsingMR readUsingMR = new ReadUsingMR(conf);
     String customClass = TestProtobuf.MapIntMessage.class.getName();
     ProtoReadSupport.setProtobufClass(readUsingMR.getConfiguration(), customClass);
     List<Message> result = readUsingMR.read(outputPath);
@@ -303,7 +303,7 @@ public class ProtoInputOutputFormatTest {
     ProtoWriteSupport.setWriteSpecsCompliant(conf, true);
 
     Path outputPath = new WriteUsingMR(conf).write(msgEmpty, msgNonEmpty);
-    ReadUsingMR readUsingMR = new ReadUsingMR();
+    ReadUsingMR readUsingMR = new ReadUsingMR(conf);
     String customClass = TestProtobuf.RepeatedInnerMessage.class.getName();
     ProtoReadSupport.setProtobufClass(readUsingMR.getConfiguration(), customClass);
     List<Message> result = readUsingMR.read(outputPath);
diff --git a/parquet-protobuf/src/test/java/org/apache/parquet/proto/utils/ReadUsingMR.java
b/parquet-protobuf/src/test/java/org/apache/parquet/proto/utils/ReadUsingMR.java
index 8905968..1171a15 100644
--- a/parquet-protobuf/src/test/java/org/apache/parquet/proto/utils/ReadUsingMR.java
+++ b/parquet-protobuf/src/test/java/org/apache/parquet/proto/utils/ReadUsingMR.java
@@ -1,4 +1,4 @@
-/* 
+/*
  * 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
@@ -6,9 +6,9 @@
  * 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
@@ -28,7 +28,6 @@ import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
 import org.apache.parquet.proto.ProtoParquetInputFormat;
 
-import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
@@ -41,19 +40,27 @@ public class ReadUsingMR {
 
   private static List<Message> outputMessages;
 
-  Configuration conf = new Configuration();
+  Configuration conf;
   private String projection;
 
   public void setRequestedProjection(String projection) {
     this.projection = projection;
   }
 
+  public ReadUsingMR() {
+    this(new Configuration());
+  }
+
+  public ReadUsingMR(Configuration conf) {
+    this.conf = conf;
+  }
+
   public Configuration getConfiguration() {
     return conf;
   }
 
   public static class ReadingMapper extends Mapper<Void, MessageOrBuilder, LongWritable,
Message> {
-    protected void map(Void key, MessageOrBuilder value, Context context) throws IOException,
InterruptedException {
+    protected void map(Void key, MessageOrBuilder value, Context context) {
       Message clone = ((Message.Builder) value).build();
       outputMessages.add(clone);
     }
diff --git a/parquet-protobuf/src/test/java/org/apache/parquet/proto/utils/WriteUsingMR.java
b/parquet-protobuf/src/test/java/org/apache/parquet/proto/utils/WriteUsingMR.java
index 55f9237..90bb3fd 100644
--- a/parquet-protobuf/src/test/java/org/apache/parquet/proto/utils/WriteUsingMR.java
+++ b/parquet-protobuf/src/test/java/org/apache/parquet/proto/utils/WriteUsingMR.java
@@ -1,4 +1,4 @@
-/* 
+/*
  * 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
@@ -6,9 +6,9 @@
  * 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
@@ -33,7 +33,6 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
@@ -55,7 +54,7 @@ public class WriteUsingMR {
   }
 
   public WriteUsingMR(Configuration conf) {
-    this.conf = new Configuration();
+    this.conf = conf;
   }
 
   public Configuration getConfiguration() {


Mime
View raw message