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 2589cc8 PARQUET-1879: MapKeyValue is not a valid Logical Type (#798)
2589cc8 is described below
commit 2589cc821d2d470be1e79b86f511eb1f5fee4e5c
Author: Matthew McMahon <[email protected]>
AuthorDate: Mon Jul 6 17:32:37 2020 +1000
PARQUET-1879: MapKeyValue is not a valid Logical Type (#798)
* Writing UNKNOWN logical type into the schema, causes a breakage
when parsing the file with Apache Arrow
* Instead use the default, of falling back to null when that
backwards-compatibility only logical type is present, but still
write the original type
---
.../parquet/avro/TestAvroSchemaConverter.java | 10 +-
.../org/apache/parquet/avro/TestReadWrite.java | 4 +-
.../apache/parquet/schema/ConversionPatterns.java | 5 +-
.../main/java/org/apache/parquet/schema/Types.java | 8 +-
.../apache/parquet/schema/TestTypeBuilders.java | 56 +++++------
.../format/converter/ParquetMetadataConverter.java | 7 +-
.../converter/TestParquetMetadataConverter.java | 105 +++++++++++++++++++++
.../ql/io/parquet/TestHiveSchemaConverter.java | 10 +-
.../org/apache/parquet/pig/PigSchemaConverter.java | 3 +-
.../apache/parquet/pig/TestPigSchemaConverter.java | 20 ++--
.../parquet/pig/TestTupleRecordConsumer.java | 10 +-
.../thrift/TestThriftToParquetFileWriter.java | 22 ++---
.../parquet/thrift/TestParquetWriteProtocol.java | 30 +++---
.../parquet/thrift/TestThriftSchemaConverter.java | 8 +-
.../TestThriftSchemaConverterProjectUnion.java | 12 +--
15 files changed, 207 insertions(+), 103 deletions(-)
diff --git
a/parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java
b/parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java
index 786477b..6f87acf 100644
---
a/parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java
+++
b/parquet-avro/src/test/java/org/apache/parquet/avro/TestAvroSchemaConverter.java
@@ -202,13 +202,13 @@ public class TestAvroSchemaConverter {
" }\n" +
" }\n" +
" required group mymap (MAP) {\n" +
- " repeated group map (MAP_KEY_VALUE) {\n" +
+ " repeated group key_value (MAP_KEY_VALUE) {\n" +
" required binary key (UTF8);\n" +
" required int32 value;\n" +
" }\n" +
" }\n" +
" required group myemptymap (MAP) {\n" +
- " repeated group map (MAP_KEY_VALUE) {\n" +
+ " repeated group key_value (MAP_KEY_VALUE) {\n" +
" required binary key (UTF8);\n" +
" required int32 value;\n" +
" }\n" +
@@ -249,13 +249,13 @@ public class TestAvroSchemaConverter {
" repeated int32 array;\n" +
" }\n" +
" required group mymap (MAP) {\n" +
- " repeated group map (MAP_KEY_VALUE) {\n" +
+ " repeated group key_value (MAP_KEY_VALUE) {\n" +
" required binary key (UTF8);\n" +
" required int32 value;\n" +
" }\n" +
" }\n" +
" required group myemptymap (MAP) {\n" +
- " repeated group map (MAP_KEY_VALUE) {\n" +
+ " repeated group key_value (MAP_KEY_VALUE) {\n" +
" required binary key (UTF8);\n" +
" required int32 value;\n" +
" }\n" +
@@ -320,7 +320,7 @@ public class TestAvroSchemaConverter {
schema,
"message record1 {\n" +
" required group myintmap (MAP) {\n" +
- " repeated group map (MAP_KEY_VALUE) {\n" +
+ " repeated group key_value (MAP_KEY_VALUE) {\n" +
" required binary key (UTF8);\n" +
" optional int32 value;\n" +
" }\n" +
diff --git
a/parquet-avro/src/test/java/org/apache/parquet/avro/TestReadWrite.java
b/parquet-avro/src/test/java/org/apache/parquet/avro/TestReadWrite.java
index 1e726c3..6bdf61e 100644
--- a/parquet-avro/src/test/java/org/apache/parquet/avro/TestReadWrite.java
+++ b/parquet-avro/src/test/java/org/apache/parquet/avro/TestReadWrite.java
@@ -539,7 +539,7 @@ public class TestReadWrite {
recordConsumer.startField("mymap", index);
recordConsumer.startGroup();
- recordConsumer.startField("map", 0);
+ recordConsumer.startField("key_value", 0);
recordConsumer.startGroup();
Map<String, Integer> mymap = (Map<String, Integer>)
record.get("mymap");
recordConsumer.startField("key", 0);
@@ -553,7 +553,7 @@ public class TestReadWrite {
}
recordConsumer.endField("value", 1);
recordConsumer.endGroup();
- recordConsumer.endField("map", 0);
+ recordConsumer.endField("key_value", 0);
recordConsumer.endGroup();
recordConsumer.endField("mymap", index++);
diff --git
a/parquet-column/src/main/java/org/apache/parquet/schema/ConversionPatterns.java
b/parquet-column/src/main/java/org/apache/parquet/schema/ConversionPatterns.java
index a530db1..8ae66f0 100644
---
a/parquet-column/src/main/java/org/apache/parquet/schema/ConversionPatterns.java
+++
b/parquet-column/src/main/java/org/apache/parquet/schema/ConversionPatterns.java
@@ -30,6 +30,7 @@ import static
org.apache.parquet.schema.LogicalTypeAnnotation.stringType;
*/
public abstract class ConversionPatterns {
+ static final String MAP_REPEATED_NAME = "key_value";
private static final String ELEMENT_NAME = "element";
/**
@@ -49,7 +50,7 @@ public abstract class ConversionPatterns {
}
public static GroupType mapType(Repetition repetition, String alias, Type
keyType, Type valueType) {
- return mapType(repetition, alias, "map", keyType, valueType);
+ return mapType(repetition, alias, MAP_REPEATED_NAME, keyType, valueType);
}
public static GroupType stringKeyMapType(Repetition repetition, String
alias, String mapAlias, Type valueType) {
@@ -57,7 +58,7 @@ public abstract class ConversionPatterns {
}
public static GroupType stringKeyMapType(Repetition repetition, String
alias, Type valueType) {
- return stringKeyMapType(repetition, alias, "map", valueType);
+ return stringKeyMapType(repetition, alias, MAP_REPEATED_NAME, valueType);
}
public static GroupType mapType(Repetition repetition, String alias, String
mapAlias, Type keyType, Type valueType) {
diff --git a/parquet-column/src/main/java/org/apache/parquet/schema/Types.java
b/parquet-column/src/main/java/org/apache/parquet/schema/Types.java
index 05db655..570cd06 100644
--- a/parquet-column/src/main/java/org/apache/parquet/schema/Types.java
+++ b/parquet-column/src/main/java/org/apache/parquet/schema/Types.java
@@ -31,6 +31,8 @@ import org.apache.parquet.schema.Type.ID;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import static org.apache.parquet.schema.LogicalTypeAnnotation.mapType;
+
/**
* This class provides fluent builders that produce Parquet schema Types.
* <p>
@@ -1179,18 +1181,18 @@ public class Types {
keyType = STRING_KEY;
}
- GroupBuilder<GroupType> builder =
buildGroup(repetition).as(OriginalType.MAP);
+ GroupBuilder<GroupType> builder = buildGroup(repetition).as(mapType());
if (id != null) {
builder.id(id.intValue());
}
if (valueType != null) {
return builder
- .repeatedGroup().addFields(keyType, valueType).named("map")
+ .repeatedGroup().addFields(keyType,
valueType).named(ConversionPatterns.MAP_REPEATED_NAME)
.named(name);
} else {
return builder
- .repeatedGroup().addFields(keyType).named("map")
+
.repeatedGroup().addFields(keyType).named(ConversionPatterns.MAP_REPEATED_NAME)
.named(name);
}
}
diff --git
a/parquet-column/src/test/java/org/apache/parquet/schema/TestTypeBuilders.java
b/parquet-column/src/test/java/org/apache/parquet/schema/TestTypeBuilders.java
index a881635..8d8be8e 100644
---
a/parquet-column/src/test/java/org/apache/parquet/schema/TestTypeBuilders.java
+++
b/parquet-column/src/test/java/org/apache/parquet/schema/TestTypeBuilders.java
@@ -486,7 +486,7 @@ public class TestTypeBuilders {
typeList.add(new PrimitiveType(REQUIRED, INT64, "key"));
typeList.add(new PrimitiveType(REQUIRED, INT64, "value"));
GroupType expected = new GroupType(REQUIRED, "myMap", OriginalType.MAP,
new GroupType(REPEATED,
- "map",
+ "key_value",
typeList));
GroupType actual = Types.requiredMap()
.key(INT64)
@@ -501,7 +501,7 @@ public class TestTypeBuilders {
typeList.add(new PrimitiveType(REQUIRED, INT64, "key"));
typeList.add(new PrimitiveType(REQUIRED, INT64, "value"));
GroupType expected = new GroupType(OPTIONAL, "myMap", OriginalType.MAP,
new GroupType(REPEATED,
- "map",
+ "key_value",
typeList));
GroupType actual = Types.optionalMap()
.key(INT64)
@@ -515,7 +515,7 @@ public class TestTypeBuilders {
List<Type> typeList = new ArrayList<>();
typeList.add(new PrimitiveType(REQUIRED, INT64, "key"));
typeList.add(new PrimitiveType(REQUIRED, INT64, "value"));
- GroupType map = new GroupType(REQUIRED, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "map",
+ GroupType map = new GroupType(REQUIRED, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "key_value",
typeList));
MessageType expected = new MessageType("mapParent", map);
GroupType actual = Types.buildMessage().requiredMap()
@@ -530,7 +530,7 @@ public class TestTypeBuilders {
List<Type> typeList = new ArrayList<>();
typeList.add(new PrimitiveType(REQUIRED, INT64, "key"));
typeList.add(new PrimitiveType(OPTIONAL, INT64, "value"));
- GroupType map = new GroupType(REQUIRED, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "map",
+ GroupType map = new GroupType(REQUIRED, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "key_value",
typeList));
MessageType expected = new MessageType("mapParent", map);
GroupType actual = Types.buildMessage().requiredMap()
@@ -554,7 +554,7 @@ public class TestTypeBuilders {
valueFields.add(new PrimitiveType(OPTIONAL, INT32, "two"));
typeList.add(new GroupType(OPTIONAL, "value", valueFields));
- GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "map",
+ GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "key_value",
typeList));
GroupType actual = Types.optionalMap()
@@ -582,7 +582,7 @@ public class TestTypeBuilders {
valueFields.add(new PrimitiveType(OPTIONAL, INT32, "two"));
typeList.add(new GroupType(REQUIRED, "value", valueFields));
- GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "map",
+ GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "key_value",
typeList));
MessageType expected = new MessageType("mapParent", map);
@@ -611,7 +611,7 @@ public class TestTypeBuilders {
typeList.add(new PrimitiveType(OPTIONAL, DOUBLE, "value"));
- GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "map",
+ GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "key_value",
typeList));
MessageType expected = new MessageType("mapParent", map);
@@ -637,7 +637,7 @@ public class TestTypeBuilders {
typeList.add(new PrimitiveType(REQUIRED, DOUBLE, "value"));
- GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "map",
+ GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "key_value",
typeList));
MessageType expected = new MessageType("mapParent", map);
@@ -666,7 +666,7 @@ public class TestTypeBuilders {
valueFields.add(new PrimitiveType(OPTIONAL, INT32, "two"));
typeList.add(new GroupType(OPTIONAL, "value", valueFields));
- GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "map",
+ GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "key_value",
typeList));
MessageType expected = new MessageType("mapParent", map);
@@ -693,7 +693,7 @@ public class TestTypeBuilders {
valueFields.add(new PrimitiveType(OPTIONAL, INT32, "two"));
typeList.add(new GroupType(REQUIRED, "value", valueFields));
- GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "map",
+ GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "key_value",
typeList));
MessageType expected = new MessageType("mapParent", map);
@@ -727,7 +727,7 @@ public class TestTypeBuilders {
valueFields.add(new PrimitiveType(OPTIONAL, INT32, "two"));
typeList.add(new GroupType(OPTIONAL, "value", valueFields));
- GroupType map = new GroupType(REQUIRED, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "map",
+ GroupType map = new GroupType(REQUIRED, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "key_value",
typeList));
MessageType expected = new MessageType("mapParent", map);
@@ -761,7 +761,7 @@ public class TestTypeBuilders {
"list",
new PrimitiveType(OPTIONAL, INT64, "element"))));
- GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "map",
+ GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "key_value",
typeList));
MessageType expected = new MessageType("mapParent", map);
@@ -786,7 +786,7 @@ public class TestTypeBuilders {
"list",
new PrimitiveType(OPTIONAL, INT64, "element"))));
- GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "map",
+ GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "key_value",
typeList));
MessageType expected = new MessageType("mapParent", map);
@@ -811,9 +811,9 @@ public class TestTypeBuilders {
typeList.add(new PrimitiveType(REQUIRED, INT64, "key"));
typeList.add(new GroupType(REQUIRED, "value", OriginalType.MAP,
- new GroupType(REPEATED, "map", innerMapTypeList)));
+ new GroupType(REPEATED, "key_value", innerMapTypeList)));
- GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "map",
+ GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "key_value",
typeList));
MessageType expected = new MessageType("mapParent", map);
@@ -839,9 +839,9 @@ public class TestTypeBuilders {
typeList.add(new PrimitiveType(REQUIRED, INT64, "key"));
typeList.add(new GroupType(OPTIONAL, "value", OriginalType.MAP,
- new GroupType(REPEATED, "map", innerMapTypeList)));
+ new GroupType(REPEATED, "key_value", innerMapTypeList)));
- GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "map",
+ GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "key_value",
typeList));
MessageType expected = new MessageType("mapParent", map);
@@ -869,7 +869,7 @@ public class TestTypeBuilders {
"list",
new PrimitiveType(OPTIONAL, INT64, "element"))));
- GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "map",
+ GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "key_value",
typeList));
MessageType expected = new MessageType("mapParent", map);
@@ -898,7 +898,7 @@ public class TestTypeBuilders {
"list",
new PrimitiveType(OPTIONAL, INT64, "element"))));
- GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "map",
+ GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "key_value",
typeList));
MessageType expected = new MessageType("mapParent", map);
@@ -928,9 +928,9 @@ public class TestTypeBuilders {
"first"
)));
typeList.add(new GroupType(REQUIRED, "value", OriginalType.MAP,
- new GroupType(REPEATED, "map", innerMapTypeList)));
+ new GroupType(REPEATED, "key_value", innerMapTypeList)));
- GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "map",
+ GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "key_value",
typeList));
MessageType expected = new MessageType("mapParent", map);
@@ -961,9 +961,9 @@ public class TestTypeBuilders {
"first"
)));
typeList.add(new GroupType(OPTIONAL, "value", OriginalType.MAP,
- new GroupType(REPEATED, "map", innerMapTypeList)));
+ new GroupType(REPEATED, "key_value", innerMapTypeList)));
- GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "map",
+ GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "key_value",
typeList));
MessageType expected = new MessageType("mapParent", map);
@@ -986,7 +986,7 @@ public class TestTypeBuilders {
List<Type> typeList = new ArrayList<>();
typeList.add(new PrimitiveType(REQUIRED, INT64, "key"));
- GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "map",
+ GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "key_value",
typeList));
MessageType expected = new MessageType("mapParent", map);
@@ -1004,7 +1004,7 @@ public class TestTypeBuilders {
List<Type> typeList = new ArrayList<>();
typeList.add(new PrimitiveType(REQUIRED, BINARY, "key",
OriginalType.UTF8));
- GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "map",
+ GroupType map = new GroupType(OPTIONAL, "myMap", OriginalType.MAP, new
GroupType(REPEATED, "key_value",
typeList));
MessageType expected = new MessageType("mapParent", map);
@@ -1022,7 +1022,7 @@ public class TestTypeBuilders {
Type valueType = Types.required(BOOLEAN).named("value");
GroupType map = new GroupType(REQUIRED, "myMap", OriginalType.MAP,
- new GroupType(REPEATED, "map", new Type[] {
+ new GroupType(REPEATED, "key_value", new Type[] {
keyType,
valueType
}));
@@ -1204,7 +1204,7 @@ public class TestTypeBuilders {
List<Type> fields = new ArrayList<>();
fields.add(new GroupType(REQUIRED, "element", OriginalType.MAP,
new GroupType(REPEATED,
- "map",
+ "key_value",
innerFields)));
GroupType expected = new GroupType(OPTIONAL, "myList", OriginalType.LIST,
new GroupType(REPEATED,
@@ -1230,7 +1230,7 @@ public class TestTypeBuilders {
List<Type> fields = new ArrayList<>();
fields.add(new GroupType(OPTIONAL, "element", OriginalType.MAP,
new GroupType(REPEATED,
- "map",
+ "key_value",
innerFields)));
GroupType expected = new GroupType(OPTIONAL, "myList", OriginalType.LIST,
new GroupType(REPEATED,
diff --git
a/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java
b/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java
index e2a93eb..5d8e67b 100644
---
a/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java
+++
b/parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java
@@ -282,7 +282,7 @@ public class ParquetMetadataConverter {
}
LogicalType convertToLogicalType(LogicalTypeAnnotation
logicalTypeAnnotation) {
- return logicalTypeAnnotation.accept(LOGICAL_TYPE_ANNOTATION_VISITOR).get();
+ return
logicalTypeAnnotation.accept(LOGICAL_TYPE_ANNOTATION_VISITOR).orElse(null);
}
ConvertedType convertToConvertedType(LogicalTypeAnnotation
logicalTypeAnnotation) {
@@ -464,11 +464,6 @@ public class ParquetMetadataConverter {
public Optional<LogicalType>
visit(LogicalTypeAnnotation.IntervalLogicalTypeAnnotation intervalLogicalType) {
return of(LogicalType.UNKNOWN(new NullType()));
}
-
- @Override
- public Optional<LogicalType>
visit(LogicalTypeAnnotation.MapKeyValueTypeAnnotation mapKeyValueLogicalType) {
- return of(LogicalType.UNKNOWN(new NullType()));
- }
}
private void addRowGroup(ParquetMetadata parquetMetadata, List<RowGroup>
rowGroups, BlockMetaData block) {
diff --git
a/parquet-hadoop/src/test/java/org/apache/parquet/format/converter/TestParquetMetadataConverter.java
b/parquet-hadoop/src/test/java/org/apache/parquet/format/converter/TestParquetMetadataConverter.java
index c8f631d..aee2dc6 100644
---
a/parquet-hadoop/src/test/java/org/apache/parquet/format/converter/TestParquetMetadataConverter.java
+++
b/parquet-hadoop/src/test/java/org/apache/parquet/format/converter/TestParquetMetadataConverter.java
@@ -38,6 +38,7 @@ import static
org.apache.parquet.schema.LogicalTypeAnnotation.uuidType;
import static org.apache.parquet.schema.MessageTypeParser.parseMessageType;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertNull;
import static org.junit.Assert.assertSame;
import static org.junit.Assert.assertTrue;
@@ -68,6 +69,7 @@ import java.util.TreeSet;
import com.google.common.collect.Sets;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.UTF8;
import org.apache.parquet.FixedBinaryTestUtils;
import org.apache.parquet.Version;
@@ -83,9 +85,17 @@ import org.apache.parquet.column.statistics.FloatStatistics;
import org.apache.parquet.column.statistics.IntStatistics;
import org.apache.parquet.column.statistics.LongStatistics;
import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.example.data.Group;
+import org.apache.parquet.example.data.simple.SimpleGroup;
import org.apache.parquet.format.DecimalType;
import org.apache.parquet.format.LogicalType;
+import org.apache.parquet.format.MapType;
+import org.apache.parquet.format.StringType;
import org.apache.parquet.format.Util;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.ParquetWriter;
+import org.apache.parquet.hadoop.example.ExampleParquetWriter;
+import org.apache.parquet.hadoop.example.GroupReadSupport;
import org.apache.parquet.hadoop.metadata.BlockMetaData;
import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
import org.apache.parquet.hadoop.metadata.ColumnPath;
@@ -100,6 +110,7 @@ import org.apache.parquet.io.api.Binary;
import org.apache.parquet.schema.PrimitiveType;
import org.apache.parquet.schema.LogicalTypeAnnotation;
import org.junit.Assert;
+import org.junit.Rule;
import org.junit.Test;
import org.apache.parquet.example.Paper;
import org.apache.parquet.format.ColumnChunk;
@@ -120,6 +131,7 @@ import org.apache.parquet.schema.Type.Repetition;
import org.apache.parquet.schema.Types;
import com.google.common.collect.Lists;
+import org.junit.rules.TemporaryFolder;
public class TestParquetMetadataConverter {
private static SecureRandom random = new SecureRandom();
@@ -128,6 +140,9 @@ public class TestParquetMetadataConverter {
private static final String NUMBER = "0123456789";
private static final String DATA_FOR_RANDOM_STRING = CHAR_LOWER + CHAR_UPPER
+ NUMBER;
+ @Rule
+ public TemporaryFolder temporaryFolder = new TemporaryFolder();
+
@Test
public void testPageHeader() throws IOException {
ByteArrayOutputStream out = new ByteArrayOutputStream();
@@ -1261,4 +1276,94 @@ public class TestParquetMetadataConverter {
ParquetMetadataConverter.fromParquetColumnIndex(Types.required(PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY)
.length(12).as(OriginalType.INTERVAL).named("test_interval"),
parquetColumnIndex));
}
+
+ @Test
+ public void testMapLogicalType() {
+ ParquetMetadataConverter parquetMetadataConverter = new
ParquetMetadataConverter();
+ MessageType expected = Types.buildMessage()
+ .requiredGroup().as(mapType())
+
.repeatedGroup().as(LogicalTypeAnnotation.MapKeyValueTypeAnnotation.getInstance())
+ .required(PrimitiveTypeName.BINARY).as(stringType()).named("key")
+ .required(PrimitiveTypeName.INT32).named("value")
+ .named("key_value")
+ .named("testMap")
+ .named("Message");
+
+ List<SchemaElement> parquetSchema =
parquetMetadataConverter.toParquetSchema(expected);
+ assertEquals(5, parquetSchema.size());
+ assertEquals(new SchemaElement("Message").setNum_children(1),
parquetSchema.get(0));
+ assertEquals(new
SchemaElement("testMap").setRepetition_type(FieldRepetitionType.REQUIRED).setNum_children(1).setConverted_type(ConvertedType.MAP).setLogicalType(LogicalType.MAP(new
MapType())), parquetSchema.get(1));
+ // PARQUET-1879 ensure that LogicalType is not written (null) but
ConvertedType is MAP_KEY_VALUE for backwards-compatibility
+ assertEquals(new
SchemaElement("key_value").setRepetition_type(FieldRepetitionType.REPEATED).setNum_children(2).setConverted_type(ConvertedType.MAP_KEY_VALUE).setLogicalType(null),
parquetSchema.get(2));
+ assertEquals(new
SchemaElement("key").setType(Type.BYTE_ARRAY).setRepetition_type(FieldRepetitionType.REQUIRED).setConverted_type(ConvertedType.UTF8).setLogicalType(LogicalType.STRING(new
StringType())), parquetSchema.get(3));
+ assertEquals(new
SchemaElement("value").setType(Type.INT32).setRepetition_type(FieldRepetitionType.REQUIRED).setConverted_type(null).setLogicalType(null),
parquetSchema.get(4));
+
+ MessageType schema =
parquetMetadataConverter.fromParquetSchema(parquetSchema, null);
+ assertEquals(expected, schema);
+ }
+
+ @Test
+ public void testMapLogicalTypeReadWrite() throws Exception {
+ MessageType messageType = Types.buildMessage()
+ .requiredGroup().as(mapType())
+
.repeatedGroup().as(LogicalTypeAnnotation.MapKeyValueTypeAnnotation.getInstance())
+ .required(PrimitiveTypeName.BINARY).as(stringType()).named("key")
+ .required(PrimitiveTypeName.INT64).named("value")
+ .named("key_value")
+ .named("testMap")
+ .named("example");
+
+ verifyMapMessageType(messageType, "key_value");
+ }
+
+ @Test
+ public void testMapConvertedTypeReadWrite() throws Exception {
+ List<SchemaElement> oldConvertedTypeSchemaElements = new ArrayList<>();
+ oldConvertedTypeSchemaElements.add(new
SchemaElement("example").setNum_children(1));
+ oldConvertedTypeSchemaElements.add(new
SchemaElement("testMap").setRepetition_type(FieldRepetitionType.REQUIRED).setNum_children(1).setConverted_type(ConvertedType.MAP).setLogicalType(null));
+ oldConvertedTypeSchemaElements.add(new
SchemaElement("map").setRepetition_type(FieldRepetitionType.REPEATED).setNum_children(2).setConverted_type(ConvertedType.MAP_KEY_VALUE).setLogicalType(null));
+ oldConvertedTypeSchemaElements.add(new
SchemaElement("key").setType(Type.BYTE_ARRAY).setRepetition_type(FieldRepetitionType.REQUIRED).setConverted_type(ConvertedType.UTF8).setLogicalType(null));
+ oldConvertedTypeSchemaElements.add(new
SchemaElement("value").setType(Type.INT64).setRepetition_type(FieldRepetitionType.REQUIRED).setConverted_type(null).setLogicalType(null));
+
+ ParquetMetadataConverter parquetMetadataConverter = new
ParquetMetadataConverter();
+ MessageType messageType =
parquetMetadataConverter.fromParquetSchema(oldConvertedTypeSchemaElements,
null);
+
+ verifyMapMessageType(messageType, "map");
+ }
+
+ private void verifyMapMessageType(final MessageType messageType, final
String keyValueName) throws IOException {
+ Path file = new
Path(temporaryFolder.newFolder("verifyMapMessageType").getPath(), keyValueName
+ ".parquet");
+
+ try (ParquetWriter<Group> writer =
+ ExampleParquetWriter
+ .builder(file)
+ .withType(messageType)
+ .build()) {
+ final Group group = new SimpleGroup(messageType);
+ final Group mapGroup = group.addGroup("testMap");
+
+ for (int index = 0; index < 5; index++) {
+ final Group keyValueGroup = mapGroup.addGroup(keyValueName);
+ keyValueGroup.add("key", Binary.fromString("key" + index));
+ keyValueGroup.add("value", 100L + index);
+ }
+
+ writer.write(group);
+ }
+
+ try (ParquetReader<Group> reader = ParquetReader.builder(new
GroupReadSupport(), file).build()) {
+ Group group = reader.read();
+
+ assertNotNull(group);
+
+ Group testMap = group.getGroup("testMap", 0);
+ assertNotNull(testMap);
+ assertEquals(5, testMap.getFieldRepetitionCount(keyValueName));
+
+ for (int index = 0; index < 5; index++) {
+ assertEquals("key" + index, testMap.getGroup(keyValueName,
index).getString("key", 0));
+ assertEquals(100L + index, testMap.getGroup(keyValueName,
index).getLong("value", 0));
+ }
+ }
+ }
}
diff --git
a/parquet-hive/parquet-hive-storage-handler/src/test/java/org/apache/hadoop/hive/ql/io/parquet/TestHiveSchemaConverter.java
b/parquet-hive/parquet-hive-storage-handler/src/test/java/org/apache/hadoop/hive/ql/io/parquet/TestHiveSchemaConverter.java
index ad10426..859fe70 100644
---
a/parquet-hive/parquet-hive-storage-handler/src/test/java/org/apache/hadoop/hive/ql/io/parquet/TestHiveSchemaConverter.java
+++
b/parquet-hive/parquet-hive-storage-handler/src/test/java/org/apache/hadoop/hive/ql/io/parquet/TestHiveSchemaConverter.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
@@ -111,7 +111,7 @@ public class TestHiveSchemaConverter {
"map<string,string>",
"message hive_schema {\n"
+ " optional group mapCol (MAP) {\n"
- + " repeated group map (MAP_KEY_VALUE) {\n"
+ + " repeated group key_value (MAP_KEY_VALUE) {\n"
+ " required binary key;\n"
+ " optional binary value;\n"
+ " }\n"
@@ -136,7 +136,7 @@ public class TestHiveSchemaConverter {
assertEquals(1, topLevel.asGroupType().getFieldCount());
org.apache.parquet.schema.Type secondLevel =
topLevel.asGroupType().getFields().get(0);
//there is one repeated field for mapCol, the field name is "map" and its
original Type is MAP_KEY_VALUE;
- assertEquals("map", secondLevel.getName());
+ assertEquals("key_value", secondLevel.getName());
assertEquals(OriginalType.MAP_KEY_VALUE, secondLevel.getOriginalType());
assertEquals(Repetition.REPEATED, secondLevel.getRepetition());
}
diff --git
a/parquet-pig/src/main/java/org/apache/parquet/pig/PigSchemaConverter.java
b/parquet-pig/src/main/java/org/apache/parquet/pig/PigSchemaConverter.java
index 1935661..99058bc 100644
--- a/parquet-pig/src/main/java/org/apache/parquet/pig/PigSchemaConverter.java
+++ b/parquet-pig/src/main/java/org/apache/parquet/pig/PigSchemaConverter.java
@@ -62,6 +62,7 @@ import static
org.apache.parquet.schema.LogicalTypeAnnotation.stringType;
*/
public class PigSchemaConverter {
private static final Logger LOG =
LoggerFactory.getLogger(PigSchemaConverter.class);
+ private static final String MAP_REPEATED_NAME = "key_value";
static final String ARRAY_VALUE_NAME = "value";
private ColumnAccess columnAccess;
@@ -459,7 +460,7 @@ public class PigSchemaConverter {
throw new SchemaConversionException("Invalid map schema, cannot infer
innerschema: ", fe);
}
Type convertedValue = convertWithName(innerField, "value");
- return ConversionPatterns.stringKeyMapType(Repetition.OPTIONAL, alias,
name(innerField.alias, "map"),
+ return ConversionPatterns.stringKeyMapType(Repetition.OPTIONAL, alias,
name(innerField.alias, MAP_REPEATED_NAME),
convertedValue);
}
diff --git
a/parquet-pig/src/test/java/org/apache/parquet/pig/TestPigSchemaConverter.java
b/parquet-pig/src/test/java/org/apache/parquet/pig/TestPigSchemaConverter.java
index 64d5961..f0d8199 100644
---
a/parquet-pig/src/test/java/org/apache/parquet/pig/TestPigSchemaConverter.java
+++
b/parquet-pig/src/test/java/org/apache/parquet/pig/TestPigSchemaConverter.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
@@ -143,7 +143,7 @@ public class TestPigSchemaConverter {
"message pig_schema {\n" +
" optional binary a (UTF8);\n" +
" optional group b (MAP) {\n" +
- " repeated group map (MAP_KEY_VALUE) {\n" +
+ " repeated group key_value (MAP_KEY_VALUE) {\n" +
" required binary key (UTF8);\n" +
" optional group value {\n" +
" optional binary c (UTF8);\n" +
@@ -159,7 +159,7 @@ public class TestPigSchemaConverter {
testConversion("a:map[int]",
"message pig_schema {\n" +
" optional group a (MAP) {\n" +
- " repeated group map (MAP_KEY_VALUE) {\n" +
+ " repeated group key_value (MAP_KEY_VALUE) {\n" +
" required binary key (UTF8);\n" +
" optional int32 value;" +
" }\n" +
@@ -172,10 +172,10 @@ public class TestPigSchemaConverter {
testConversion("a:map[map[int]]",
"message pig_schema {\n" +
" optional group a (MAP) {\n" +
- " repeated group map (MAP_KEY_VALUE) {\n" +
+ " repeated group key_value (MAP_KEY_VALUE) {\n" +
" required binary key (UTF8);\n" +
" optional group value (MAP) {\n" +
- " repeated group map (MAP_KEY_VALUE) {\n" +
+ " repeated group key_value (MAP_KEY_VALUE) {\n" +
" required binary key (UTF8);\n" +
" optional int32 value;\n" +
" }\n" +
@@ -190,7 +190,7 @@ public class TestPigSchemaConverter {
testConversion("a:map[bag{(a:int)}]",
"message pig_schema {\n" +
" optional group a (MAP) {\n" +
- " repeated group map (MAP_KEY_VALUE) {\n" +
+ " repeated group key_value (MAP_KEY_VALUE) {\n" +
" required binary key (UTF8);\n" +
" optional group value (LIST) {\n" +
" repeated group bag {\n" +
@@ -228,7 +228,7 @@ public class TestPigSchemaConverter {
"message pig_schema {\n" +
" optional binary a;\n" +
" optional group b (MAP) {\n" +
- " repeated group map (MAP_KEY_VALUE) {\n" +
+ " repeated group key_value (MAP_KEY_VALUE) {\n" +
" required binary key;\n" +
" optional group value {\n" +
" optional fixed_len_byte_array(5) c;\n" +
@@ -246,7 +246,7 @@ public class TestPigSchemaConverter {
"message spark_schema {\n" +
" optional binary a;\n" +
" optional group b (MAP) {\n" +
- " repeated group map {\n" +
+ " repeated group key_value {\n" +
" required binary key;\n" +
" optional group value {\n" +
" optional fixed_len_byte_array(5) c;\n" +
diff --git
a/parquet-pig/src/test/java/org/apache/parquet/pig/TestTupleRecordConsumer.java
b/parquet-pig/src/test/java/org/apache/parquet/pig/TestTupleRecordConsumer.java
index ef048f2..ff4bd87 100644
---
a/parquet-pig/src/test/java/org/apache/parquet/pig/TestTupleRecordConsumer.java
+++
b/parquet-pig/src/test/java/org/apache/parquet/pig/TestTupleRecordConsumer.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
@@ -93,8 +93,8 @@ public class TestTupleRecordConsumer {
String pigSchemaString = "a: [(b: chararray)]";
SimpleGroup g = new SimpleGroup(getMessageType(pigSchemaString));
Group map = g.addGroup("a");
- map.addGroup("map").append("key", "foo").addGroup("value").append("b",
"foo");
- map.addGroup("map").append("key", "bar").addGroup("value").append("b",
"bar");
+ map.addGroup("key_value").append("key",
"foo").addGroup("value").append("b", "foo");
+ map.addGroup("key_value").append("key",
"bar").addGroup("value").append("b", "bar");
testFromGroups(pigSchemaString, Arrays.<Group>asList(g));
}
diff --git
a/parquet-thrift/src/test/java/org/apache/parquet/hadoop/thrift/TestThriftToParquetFileWriter.java
b/parquet-thrift/src/test/java/org/apache/parquet/hadoop/thrift/TestThriftToParquetFileWriter.java
index 66b804c..b8ff23c 100644
---
a/parquet-thrift/src/test/java/org/apache/parquet/hadoop/thrift/TestThriftToParquetFileWriter.java
+++
b/parquet-thrift/src/test/java/org/apache/parquet/hadoop/thrift/TestThriftToParquetFileWriter.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
@@ -210,9 +210,9 @@ public class TestThriftToParquetFileWriter {
assertEquals(listMap.names.size(),
g.getGroup("names", 0).getFieldRepetitionCount("names_tuple"));
assertEquals(listMap.names.get(0).size(),
- g.getGroup("names", 0).getGroup("names_tuple",
0).getFieldRepetitionCount("map"));
+ g.getGroup("names", 0).getGroup("names_tuple",
0).getFieldRepetitionCount("key_value"));
assertEquals(listMap.names.get(1).size(),
- g.getGroup("names", 0).getGroup("names_tuple",
1).getFieldRepetitionCount("map"));
+ g.getGroup("names", 0).getGroup("names_tuple",
1).getFieldRepetitionCount("key_value"));
}
}
@@ -228,9 +228,9 @@ public class TestThriftToParquetFileWriter {
Group g = null;
while((g = reader.read()) != null) {
assertEquals("key",
- g.getGroup("names", 0).getGroup("map",0).getBinary("key",
0).toStringUsingUTF8());
+ g.getGroup("names", 0).getGroup("key_value",0).getBinary("key",
0).toStringUsingUTF8());
assertEquals(map.get("key").size(),
- g.getGroup("names", 0).getGroup("map",0).getGroup("value",
0).getFieldRepetitionCount(0));
+ g.getGroup("names", 0).getGroup("key_value",0).getGroup("value",
0).getFieldRepetitionCount(0));
}
}
@@ -246,13 +246,13 @@ public class TestThriftToParquetFileWriter {
Group g = null;
while((g = reader.read()) != null) {
assertEquals("key1",
- g.getGroup("names", 0).getGroup("map",0).getGroup("key",
0).getBinary("key_tuple", 0).toStringUsingUTF8());
+ g.getGroup("names", 0).getGroup("key_value",0).getGroup("key",
0).getBinary("key_tuple", 0).toStringUsingUTF8());
assertEquals("key2",
- g.getGroup("names", 0).getGroup("map",0).getGroup("key",
0).getBinary("key_tuple", 1).toStringUsingUTF8());
+ g.getGroup("names", 0).getGroup("key_value",0).getGroup("key",
0).getBinary("key_tuple", 1).toStringUsingUTF8());
assertEquals("val1",
- g.getGroup("names", 0).getGroup("map",0).getGroup("value",
0).getBinary("value_tuple", 0).toStringUsingUTF8());
+ g.getGroup("names", 0).getGroup("key_value",0).getGroup("value",
0).getBinary("value_tuple", 0).toStringUsingUTF8());
assertEquals("val2",
- g.getGroup("names", 0).getGroup("map",0).getGroup("value",
0).getBinary("value_tuple", 1).toStringUsingUTF8());
+ g.getGroup("names", 0).getGroup("key_value",0).getGroup("value",
0).getBinary("value_tuple", 1).toStringUsingUTF8());
}
}
diff --git
a/parquet-thrift/src/test/java/org/apache/parquet/thrift/TestParquetWriteProtocol.java
b/parquet-thrift/src/test/java/org/apache/parquet/thrift/TestParquetWriteProtocol.java
index df60766..f9702c0 100644
---
a/parquet-thrift/src/test/java/org/apache/parquet/thrift/TestParquetWriteProtocol.java
+++
b/parquet-thrift/src/test/java/org/apache/parquet/thrift/TestParquetWriteProtocol.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
@@ -78,7 +78,7 @@ public class TestParquetWriteProtocol {
"endField(name, 0)",
"startField(names, 1)",
"startGroup()",
- "startField(map, 0)",
+ "startField(key_value, 0)",
"startGroup()",
"startField(key, 0)",
"addBinary(foo)",
@@ -95,7 +95,7 @@ public class TestParquetWriteProtocol {
"addBinary(bar2)",
"endField(value, 1)",
"endGroup()",
- "endField(map, 0)",
+ "endField(key_value, 0)",
"endGroup()",
"endField(names, 1)",
"endMessage()"
@@ -107,7 +107,7 @@ public class TestParquetWriteProtocol {
"endField(name, 0)",
"startField(names, 1)",
"startGroup()",
- "startField(map, 0)",
+ "startField(key_value, 0)",
"startGroup()",
"startField(key, 0)",
"addBinary(foo2)",
@@ -124,7 +124,7 @@ public class TestParquetWriteProtocol {
"addBinary(bar)",
"endField(value, 1)",
"endGroup()",
- "endField(map, 0)",
+ "endField(key_value, 0)",
"endGroup()",
"endField(names, 1)",
"endMessage()"
@@ -163,7 +163,7 @@ public class TestParquetWriteProtocol {
"startGroup()",
"startField(names_tuple, 0)", // map: optional field
"startGroup()",
- "startField(map, 0)", // repeated field
+ "startField(key_value, 0)", // repeated field
"startGroup()",
"startField(key, 0)", // key
"addBinary(foo)",
@@ -172,7 +172,7 @@ public class TestParquetWriteProtocol {
"addBinary(bar)",
"endField(value, 1)",
"endGroup()",
- "endField(map, 0)",
+ "endField(key_value, 0)",
"endGroup()",
"endField(names_tuple, 0)",
"endGroup()",
@@ -198,7 +198,7 @@ public class TestParquetWriteProtocol {
"startGroup()",
"startField(names_tuple, 0)", // map: optional field
"startGroup()",
- "startField(map, 0)", // repeated field
+ "startField(key_value, 0)", // repeated field
"startGroup()",
"startField(key, 0)", // key
"addBinary(foo)",
@@ -207,7 +207,7 @@ public class TestParquetWriteProtocol {
"addBinary(bar)",
"endField(value, 1)",
"endGroup()",
- "endField(map, 0)",
+ "endField(key_value, 0)",
"endGroup()",
"endField(names_tuple, 0)",
"endGroup()",
@@ -278,7 +278,7 @@ public class TestParquetWriteProtocol {
"endField(name, 0)",
"startField(names, 1)",
"startGroup()",
- "startField(map, 0)",
+ "startField(key_value, 0)",
"startGroup()",
"startField(key, 0)",
"addBinary(foo)",
@@ -302,12 +302,12 @@ public class TestParquetWriteProtocol {
"endGroup()",
"endField(value, 1)",
"endGroup()",
- "endField(map, 0)",
+ "endField(key_value, 0)",
"endGroup()",
"endField(names, 1)",
"startField(name_to_id, 2)",
"startGroup()",
- "startField(map, 0)",
+ "startField(key_value, 0)",
"startGroup()",
"startField(key, 0)",
"addBinary(bar)",
@@ -316,7 +316,7 @@ public class TestParquetWriteProtocol {
"addInt(10)",
"endField(value, 1)",
"endGroup()",
- "endField(map, 0)",
+ "endField(key_value, 0)",
"endGroup()",
"endField(name_to_id, 2)",
"endMessage()"
diff --git
a/parquet-thrift/src/test/java/org/apache/parquet/thrift/TestThriftSchemaConverter.java
b/parquet-thrift/src/test/java/org/apache/parquet/thrift/TestThriftSchemaConverter.java
index 5a7b209..41c8c29 100644
---
a/parquet-thrift/src/test/java/org/apache/parquet/thrift/TestThriftSchemaConverter.java
+++
b/parquet-thrift/src/test/java/org/apache/parquet/thrift/TestThriftSchemaConverter.java
@@ -152,7 +152,7 @@ public class TestThriftSchemaConverter {
shouldGetProjectedSchema("name;names/key*;names/value/**",
"name;names.key*;names.value", "message ParquetSchema {\n" +
" optional binary name (UTF8) = 1;\n" +
" optional group names (MAP) = 2 {\n" +
- " repeated group map (MAP_KEY_VALUE) {\n" +
+ " repeated group key_value (MAP_KEY_VALUE) {\n" +
" required binary key (UTF8);\n" +
" optional group value {\n" +
" optional group name = 1 {\n" +
@@ -160,7 +160,7 @@ public class TestThriftSchemaConverter {
" optional binary last_name (UTF8) = 2;\n" +
" }\n" +
" optional group phones (MAP) = 2 {\n" +
- " repeated group map (MAP_KEY_VALUE) {\n" +
+ " repeated group key_value (MAP_KEY_VALUE) {\n" +
" required binary key (ENUM);\n" +
" optional binary value (UTF8);\n" +
" }\n" +
@@ -174,7 +174,7 @@ public class TestThriftSchemaConverter {
shouldGetProjectedSchema("name;names/key;names/value/name/*",
"name;names.key;names.value.name","message ParquetSchema {\n" +
" optional binary name (UTF8) = 1;\n" +
" optional group names (MAP) = 2 {\n" +
- " repeated group map (MAP_KEY_VALUE) {\n" +
+ " repeated group key_value (MAP_KEY_VALUE) {\n" +
" required binary key (UTF8);\n" +
" optional group value {\n" +
" optional group name = 1 {\n" +
@@ -192,7 +192,7 @@ public class TestThriftSchemaConverter {
shouldGetProjectedSchema("name;names/key", "name;names.key", "message
ParquetSchema {\n" +
" optional binary name (UTF8) = 1;\n" +
" optional group names (MAP) = 2 {\n" +
- " repeated group map (MAP_KEY_VALUE) {\n" +
+ " repeated group key_value (MAP_KEY_VALUE) {\n" +
" required binary key (UTF8);\n" +
" optional group value {\n" +
" optional group name = 1 {\n" +
diff --git
a/parquet-thrift/src/test/java/org/apache/parquet/thrift/TestThriftSchemaConverterProjectUnion.java
b/parquet-thrift/src/test/java/org/apache/parquet/thrift/TestThriftSchemaConverterProjectUnion.java
index 611a1a9..e37934a 100644
---
a/parquet-thrift/src/test/java/org/apache/parquet/thrift/TestThriftSchemaConverterProjectUnion.java
+++
b/parquet-thrift/src/test/java/org/apache/parquet/thrift/TestThriftSchemaConverterProjectUnion.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
@@ -308,7 +308,7 @@ public class TestThriftSchemaConverterProjectUnion {
"optMapWithUnionKey.key",
"message ParquetSchema {\n" +
" optional group optMapWithUnionKey (MAP) = 1 {\n" +
- " repeated group map (MAP_KEY_VALUE) {\n" +
+ " repeated group key_value (MAP_KEY_VALUE) {\n" +
" required group key {\n" +
" optional group structV3 = 1 {\n" +
" required binary name (UTF8) = 1;\n" +
@@ -341,7 +341,7 @@ public class TestThriftSchemaConverterProjectUnion {
"optMapWithUnionKey.{key,value.gender}",
"message ParquetSchema {\n" +
" optional group optMapWithUnionKey (MAP) = 1 {\n" +
- " repeated group map (MAP_KEY_VALUE) {\n" +
+ " repeated group key_value (MAP_KEY_VALUE) {\n" +
" required group key {\n" +
" optional group structV3 = 1 {\n" +
" required binary name (UTF8) = 1;\n" +
@@ -377,7 +377,7 @@ public class TestThriftSchemaConverterProjectUnion {
"optMapWithUnionValue.{key,value.structV4.addedStruct.gender}",
"message ParquetSchema {\n" +
" optional group optMapWithUnionValue (MAP) = 1 {\n" +
- " repeated group map (MAP_KEY_VALUE) {\n" +
+ " repeated group key_value (MAP_KEY_VALUE) {\n" +
" required group key {\n" +
" required binary name (UTF8) = 1;\n" +
" optional binary age (UTF8) = 2;\n" +