Return-Path: X-Original-To: apmail-parquet-commits-archive@minotaur.apache.org Delivered-To: apmail-parquet-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id BAF0D106C7 for ; Wed, 4 Mar 2015 20:14:11 +0000 (UTC) Received: (qmail 11934 invoked by uid 500); 4 Mar 2015 20:13:37 -0000 Delivered-To: apmail-parquet-commits-archive@parquet.apache.org Received: (qmail 11912 invoked by uid 500); 4 Mar 2015 20:13:37 -0000 Mailing-List: contact commits-help@parquet.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@parquet.incubator.apache.org Delivered-To: mailing list commits@parquet.incubator.apache.org Received: (qmail 11903 invoked by uid 99); 4 Mar 2015 20:13:37 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 04 Mar 2015 20:13:37 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED,T_RP_MATCHES_RCVD X-Spam-Check-By: apache.org Received: from [140.211.11.3] (HELO mail.apache.org) (140.211.11.3) by apache.org (qpsmtpd/0.29) with SMTP; Wed, 04 Mar 2015 20:13:36 +0000 Received: (qmail 8542 invoked by uid 99); 4 Mar 2015 20:12:01 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 04 Mar 2015 20:12:01 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id CB4FFE0AC5; Wed, 4 Mar 2015 20:12:00 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: blue@apache.org To: commits@parquet.incubator.apache.org Message-Id: <923c6278121d4d429c3fc37beb10b1fa@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: incubator-parquet-mr git commit: PARQUET-191: Fix map Type to Avro Schema conversion. Date: Wed, 4 Mar 2015 20:12:00 +0000 (UTC) X-Virus-Checked: Checked by ClamAV on apache.org Repository: incubator-parquet-mr Updated Branches: refs/heads/master 4f87e0f48 -> f1b54876a PARQUET-191: Fix map Type to Avro Schema conversion. Author: Ryan Blue Closes #126 from rdblue/PARQUET-191-fix-map-value-conversion and squashes the following commits: 33f6bbc [Ryan Blue] PARQUET-191: Fix map Type to Avro Schema conversion. Project: http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/commit/f1b54876 Tree: http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/tree/f1b54876 Diff: http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/diff/f1b54876 Branch: refs/heads/master Commit: f1b54876ab8893a5d9c0e3d7c1a9c884e683dc8a Parents: 4f87e0f Author: Ryan Blue Authored: Wed Mar 4 12:11:50 2015 -0800 Committer: Ryan Blue Committed: Wed Mar 4 12:11:50 2015 -0800 ---------------------------------------------------------------------- .../avro/AvroIndexedRecordConverter.java | 4 +-- .../java/parquet/avro/AvroSchemaConverter.java | 19 ++++++++---- .../parquet/avro/TestAvroSchemaConverter.java | 31 +++++++++++++++++--- 3 files changed, 42 insertions(+), 12 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/f1b54876/parquet-avro/src/main/java/parquet/avro/AvroIndexedRecordConverter.java ---------------------------------------------------------------------- diff --git a/parquet-avro/src/main/java/parquet/avro/AvroIndexedRecordConverter.java b/parquet-avro/src/main/java/parquet/avro/AvroIndexedRecordConverter.java index 48dbd3c..85804ad 100644 --- a/parquet-avro/src/main/java/parquet/avro/AvroIndexedRecordConverter.java +++ b/parquet-avro/src/main/java/parquet/avro/AvroIndexedRecordConverter.java @@ -563,8 +563,8 @@ class AvroIndexedRecordConverter extends GroupConverter }; Type valueType = parquetSchema.asGroupType().getType(0).asGroupType().getType(1); - Schema valueSchema = avroSchema.getValueType(); - valueConverter = newConverter(valueSchema, valueType, model, new ParentValueContainer() { + Schema nonNullValueSchema = AvroSchemaConverter.getNonNull(avroSchema.getValueType()); + valueConverter = newConverter(nonNullValueSchema, valueType, model, new ParentValueContainer() { @Override @SuppressWarnings("unchecked") void add(Object value) { http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/f1b54876/parquet-avro/src/main/java/parquet/avro/AvroSchemaConverter.java ---------------------------------------------------------------------- diff --git a/parquet-avro/src/main/java/parquet/avro/AvroSchemaConverter.java b/parquet-avro/src/main/java/parquet/avro/AvroSchemaConverter.java index 6aec668..986776c 100644 --- a/parquet-avro/src/main/java/parquet/avro/AvroSchemaConverter.java +++ b/parquet-avro/src/main/java/parquet/avro/AvroSchemaConverter.java @@ -189,11 +189,7 @@ public class AvroSchemaConverter { if (parquetType.isRepetition(Type.Repetition.REPEATED)) { throw new UnsupportedOperationException("REPEATED not supported outside LIST or MAP. Type: " + parquetType); } else if (parquetType.isRepetition(Type.Repetition.OPTIONAL)) { - List types = new ArrayList(); - types.add(Schema.create(Schema.Type.NULL)); - types.add(fieldSchema); - Schema optionalFieldSchema = Schema.createUnion(types); - fields.add(new Schema.Field(parquetType.getName(), optionalFieldSchema, null, + fields.add(new Schema.Field(parquetType.getName(), optional(fieldSchema), null, NullNode.getInstance())); } else { // REQUIRED fields.add(new Schema.Field(parquetType.getName(), fieldSchema, null, null)); @@ -281,7 +277,11 @@ public class AvroSchemaConverter { + keyType); } Type valueType = mapKeyValType.getType(1); - return Schema.createMap(convertField(valueType)); + if (valueType.isRepetition(Type.Repetition.OPTIONAL)) { + return Schema.createMap(optional(convertField(valueType))); + } else { + return Schema.createMap(convertField(valueType)); + } case ENUM: return Schema.create(Schema.Type.STRING); case MAP_KEY_VALUE: @@ -297,4 +297,11 @@ public class AvroSchemaConverter { } } } + + private static Schema optional(Schema original) { + // null is first in the union because Parquet's default is always null + return Schema.createUnion(Arrays.asList( + Schema.create(Schema.Type.NULL), + original)); + } } http://git-wip-us.apache.org/repos/asf/incubator-parquet-mr/blob/f1b54876/parquet-avro/src/test/java/parquet/avro/TestAvroSchemaConverter.java ---------------------------------------------------------------------- diff --git a/parquet-avro/src/test/java/parquet/avro/TestAvroSchemaConverter.java b/parquet-avro/src/test/java/parquet/avro/TestAvroSchemaConverter.java index 575a7ac..efdc9ed 100644 --- a/parquet-avro/src/test/java/parquet/avro/TestAvroSchemaConverter.java +++ b/parquet-avro/src/test/java/parquet/avro/TestAvroSchemaConverter.java @@ -148,6 +148,7 @@ public class TestAvroSchemaConverter { public void testAllTypesParquetToAvro() throws Exception { Schema schema = new Schema.Parser().parse( Resources.getResource("allFromParquet.avsc").openStream()); + // Cannot use round-trip assertion because enum is lost testParquetToAvroConversion(schema, ALL_PARQUET_SCHEMA); } @@ -173,7 +174,7 @@ public class TestAvroSchemaConverter { schema.setFields(Arrays.asList( new Schema.Field("myint", optionalInt, null, NullNode.getInstance()) )); - testAvroToParquetConversion( + testRoundTripConversion( schema, "message record1 {\n" + " optional int32 myint;\n" + @@ -181,6 +182,25 @@ public class TestAvroSchemaConverter { } @Test + public void testOptionalMapValue() throws Exception { + Schema schema = Schema.createRecord("record1", null, null, false); + Schema optionalIntMap = Schema.createMap(optional(Schema.create(Schema.Type.INT))); + schema.setFields(Arrays.asList( + new Schema.Field("myintmap", optionalIntMap, null, null) + )); + testRoundTripConversion( + schema, + "message record1 {\n" + + " required group myintmap (MAP) {\n" + + " repeated group map (MAP_KEY_VALUE) {\n" + + " required binary key (UTF8);\n" + + " optional int32 value;\n" + + " }\n" + + " }\n" + + "}\n"); + } + + @Test public void testUnionOfTwoTypes() throws Exception { Schema schema = Schema.createRecord("record2", null, null, false); Schema multipleTypes = Schema.createUnion(Arrays.asList(Schema.create(Schema.Type @@ -190,7 +210,8 @@ public class TestAvroSchemaConverter { schema.setFields(Arrays.asList( new Schema.Field("myunion", multipleTypes, null, NullNode.getInstance()))); - // Avro union is modelled using optional data members of thw different types; + // Avro union is modelled using optional data members of the different + // types. This does not translate back into an Avro union testAvroToParquetConversion( schema, "message record2 {\n" + @@ -216,6 +237,7 @@ public class TestAvroSchemaConverter { )); System.err.println("Avro schema: " + schema.toString(true)); + // Cannot use round-trip assertion because InnerRecord optional is removed testAvroToParquetConversion(schema, "message HasArray {\n" + " required group myarray (LIST) {\n" + " repeated group array {\n" + @@ -227,7 +249,8 @@ public class TestAvroSchemaConverter { } public static Schema optional(Schema original) { - return Schema.createUnion(Lists.newArrayList(original, - Schema.create(Schema.Type.NULL))); + return Schema.createUnion(Lists.newArrayList( + Schema.create(Schema.Type.NULL), + original)); } }