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 92d8b624d9..173fa7799b 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 @@ public Optional visit(LogicalTypeAnnotation.ListLogicalTypeAnnotation public Optional 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 5544dc6887..f40b202905 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 void testMapIntMessageClassSchemaCompliant() throws Exception { 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 result = readUsingMR.read(outputPath); @@ -303,7 +303,7 @@ public void testRepeatedInnerMessageClassSchemaCompliant() throws Exception { 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 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 8905968b01..1171a1566d 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.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 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 { - 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 55f9237ec5..90bb3fd756 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.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 WriteUsingMR() { } public WriteUsingMR(Configuration conf) { - this.conf = new Configuration(); + this.conf = conf; } public Configuration getConfiguration() {