From 8edc893e60e584a03369b6ac71800232bc13f7d6 Mon Sep 17 00:00:00 2001 From: lukasnalezenec Date: Wed, 23 Oct 2013 12:16:27 -0700 Subject: [PATCH 01/31] Initial commit --- .gitignore | 1 + README.md | 4 ++++ 2 files changed, 5 insertions(+) create mode 100644 .gitignore create mode 100644 README.md diff --git a/.gitignore b/.gitignore new file mode 100644 index 0000000000..2f7896d1d1 --- /dev/null +++ b/.gitignore @@ -0,0 +1 @@ +target/ diff --git a/README.md b/README.md new file mode 100644 index 0000000000..7b57d5f376 --- /dev/null +++ b/README.md @@ -0,0 +1,4 @@ +parquet-protobuf +================ + +protobuffer support for Parquet columnar format From 5c46f055890eb86652e9b573c686190769599c29 Mon Sep 17 00:00:00 2001 From: Lukas Date: Wed, 23 Oct 2013 21:48:16 +0200 Subject: [PATCH 02/31] initial commit --- pom.xml | 173 ++++++++++++++++ .../proto/ProtoParquetInputFormat.java | 35 ++++ .../proto/ProtoParquetOutputFormat.java | 37 ++++ .../parquet/proto/ProtoParquetReader.java | 40 ++++ .../parquet/proto/ProtoParquetWriter.java | 78 ++++++++ .../java/parquet/proto/ProtoReadSupport.java | 70 +++++++ .../proto/ProtoRecordMaterializer.java | 41 ++++ .../parquet/proto/ProtoSchemaConverter.java | 131 +++++++++++++ .../java/parquet/proto/ProtoWriteSupport.java | 185 ++++++++++++++++++ .../proto/ProtobufferRecordConverter.java | 68 +++++++ .../converters/ParentValueContainer.java | 10 + .../proto/converters/ProtoArrayConverter.java | 28 +++ .../converters/ProtoBinaryConverter.java | 18 ++ .../converters/ProtoBooleanConverter.java | 18 ++ .../converters/ProtoDoubleConverter.java | 17 ++ .../proto/converters/ProtoEnumConverter.java | 48 +++++ .../proto/converters/ProtoFloatConverter.java | 17 ++ .../proto/converters/ProtoIntConverter.java | 17 ++ .../proto/converters/ProtoLongConverter.java | 17 ++ .../converters/ProtoMessageConverter.java | 170 ++++++++++++++++ .../converters/ProtobufStringConverter.java | 20 ++ src/main/java/parquet/proto/package-info.java | 89 +++++++++ src/main/java/parquet/proto/todo.txt | 93 +++++++++ .../java/parquet/proto/BugHuntingTest.java | 54 +++++ src/test/java/parquet/proto/ProtoTest.java | 154 +++++++++++++++ src/test/java/parquet/proto/TestSandbox.java | 43 ++++ src/test/java/parquet/proto/TestUtils.java | 159 +++++++++++++++ src/test/resources/TestProtobuf.proto | 127 ++++++++++++ 28 files changed, 1957 insertions(+) create mode 100644 pom.xml create mode 100644 src/main/java/parquet/proto/ProtoParquetInputFormat.java create mode 100644 src/main/java/parquet/proto/ProtoParquetOutputFormat.java create mode 100644 src/main/java/parquet/proto/ProtoParquetReader.java create mode 100644 src/main/java/parquet/proto/ProtoParquetWriter.java create mode 100644 src/main/java/parquet/proto/ProtoReadSupport.java create mode 100644 src/main/java/parquet/proto/ProtoRecordMaterializer.java create mode 100644 src/main/java/parquet/proto/ProtoSchemaConverter.java create mode 100644 src/main/java/parquet/proto/ProtoWriteSupport.java create mode 100644 src/main/java/parquet/proto/ProtobufferRecordConverter.java create mode 100644 src/main/java/parquet/proto/converters/ParentValueContainer.java create mode 100644 src/main/java/parquet/proto/converters/ProtoArrayConverter.java create mode 100644 src/main/java/parquet/proto/converters/ProtoBinaryConverter.java create mode 100644 src/main/java/parquet/proto/converters/ProtoBooleanConverter.java create mode 100644 src/main/java/parquet/proto/converters/ProtoDoubleConverter.java create mode 100644 src/main/java/parquet/proto/converters/ProtoEnumConverter.java create mode 100644 src/main/java/parquet/proto/converters/ProtoFloatConverter.java create mode 100644 src/main/java/parquet/proto/converters/ProtoIntConverter.java create mode 100644 src/main/java/parquet/proto/converters/ProtoLongConverter.java create mode 100644 src/main/java/parquet/proto/converters/ProtoMessageConverter.java create mode 100644 src/main/java/parquet/proto/converters/ProtobufStringConverter.java create mode 100644 src/main/java/parquet/proto/package-info.java create mode 100644 src/main/java/parquet/proto/todo.txt create mode 100644 src/test/java/parquet/proto/BugHuntingTest.java create mode 100644 src/test/java/parquet/proto/ProtoTest.java create mode 100644 src/test/java/parquet/proto/TestSandbox.java create mode 100644 src/test/java/parquet/proto/TestUtils.java create mode 100644 src/test/resources/TestProtobuf.proto diff --git a/pom.xml b/pom.xml new file mode 100644 index 0000000000..cb01c689d3 --- /dev/null +++ b/pom.xml @@ -0,0 +1,173 @@ + + + com.twitter + parquet + ../pom.xml + 1.2.3-SNAPSHOT + + + 4.0.0 + + parquet-protobuf + jar + + + 3.0.8 + + + + Parquet Protobuf + https://github.com/lukasnalezenec/parquet-protobuf.git + + + + com.google.protobuf + protobuf-java + 2.4.1 + + + com.twitter + parquet-column + ${project.version} + + + com.twitter + parquet-hadoop + ${project.version} + + + com.twitter + parquet-format + 1.0.0 + + + org.apache.hadoop + hadoop-client + ${hadoop.version} + + + com.google.guava + guava + 11.0 + test + + + com.twitter + parquet-column + ${project.version} + test-jar + test + + + + com.twitter.elephantbird + elephant-bird-core + 3.0.8 + + + + + + lukasnalezenec + Lukas Nalezenec + + + + + + + org.apache.maven.plugins + maven-jar-plugin + ${maven-jar-plugin.version} + + + + ${buildNumber} + + + + + + + test-jar + + + + + + org.apache.maven.plugins + maven-shade-plugin + + + package + + shade + + + + + org.codehaus.jackson:jackson-mapper-asl + org.codehaus.jackson:jackson-core-asl + + + + + org.codehaus.jackson + parquet.org.codehaus.jackson + + + + + + + + + org.codehaus.mojo + build-helper-maven-plugin + 1.8 + + + add-test-sources + generate-test-sources + + add-test-source + + + + ${project.build.directory}/generated-test-sources + + + + + + + + maven-antrun-plugin + + + generate-sources + generate-test-sources + + + + + + + + + + + src/main/java + target/generated-sources/java + + + run + + + + + + + + + diff --git a/src/main/java/parquet/proto/ProtoParquetInputFormat.java b/src/main/java/parquet/proto/ProtoParquetInputFormat.java new file mode 100644 index 0000000000..4415e00eb1 --- /dev/null +++ b/src/main/java/parquet/proto/ProtoParquetInputFormat.java @@ -0,0 +1,35 @@ +/** + * Copyright 2012 Twitter, Inc. + * + * Licensed 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 KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package parquet.proto; + +import com.google.protobuf.MessageOrBuilder; +import org.apache.hadoop.mapreduce.Job; +import parquet.hadoop.ParquetInputFormat; +import parquet.hadoop.util.ContextUtil; + +/** + * A Hadoop {@link org.apache.hadoop.mapreduce.InputFormat} for Parquet files. + */ +public class ProtoParquetInputFormat extends ParquetInputFormat { + public ProtoParquetInputFormat() { + super(ProtoReadSupport.class); + } + + public static void setRequestedProjection(Job job, String requestedProjection) { + ProtoReadSupport.setRequestedProjection(ContextUtil.getConfiguration(job), requestedProjection); + } + +} diff --git a/src/main/java/parquet/proto/ProtoParquetOutputFormat.java b/src/main/java/parquet/proto/ProtoParquetOutputFormat.java new file mode 100644 index 0000000000..8039223d20 --- /dev/null +++ b/src/main/java/parquet/proto/ProtoParquetOutputFormat.java @@ -0,0 +1,37 @@ +/** + * Copyright 2012 Twitter, Inc. + * + * Licensed 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 KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package parquet.proto; + +import com.google.protobuf.Message; +import com.google.protobuf.MessageOrBuilder; +import org.apache.hadoop.mapreduce.Job; +import parquet.hadoop.ParquetOutputFormat; +import parquet.hadoop.util.ContextUtil; + +/** + * A Hadoop {@link org.apache.hadoop.mapreduce.OutputFormat} for Parquet files. + */ +public class ProtoParquetOutputFormat extends ParquetOutputFormat { + + public static void setSchema(Job job, Class protoClass) { + ProtoWriteSupport.setSchema(ContextUtil.getConfiguration(job), protoClass); + } + + public ProtoParquetOutputFormat() { + super(new ProtoWriteSupport()); + } + +} diff --git a/src/main/java/parquet/proto/ProtoParquetReader.java b/src/main/java/parquet/proto/ProtoParquetReader.java new file mode 100644 index 0000000000..fe4609adac --- /dev/null +++ b/src/main/java/parquet/proto/ProtoParquetReader.java @@ -0,0 +1,40 @@ +/** + * Copyright 2012 Twitter, Inc. + * + * Licensed 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 KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package parquet.proto; + +import com.google.protobuf.MessageOrBuilder; +import org.apache.hadoop.fs.Path; +import parquet.filter.UnboundRecordFilter; +import parquet.hadoop.ParquetReader; +import parquet.hadoop.api.ReadSupport; + +import java.io.IOException; + +/** + * Read Avro records from a Parquet file. + */ +public class ProtoParquetReader extends ParquetReader { + + public ProtoParquetReader(Path file) throws IOException { + super(file, (ReadSupport) new ProtoReadSupport()); + } + + public ProtoParquetReader(Path file, UnboundRecordFilter recordFilter) throws IOException { + super(file, (ReadSupport) new ProtoReadSupport(), recordFilter); + } + + //TODO here should be option to override pb from file +} diff --git a/src/main/java/parquet/proto/ProtoParquetWriter.java b/src/main/java/parquet/proto/ProtoParquetWriter.java new file mode 100644 index 0000000000..c0f1b28473 --- /dev/null +++ b/src/main/java/parquet/proto/ProtoParquetWriter.java @@ -0,0 +1,78 @@ +/** + * Copyright 2012 Twitter, Inc. + * + * Licensed 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 KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package parquet.proto; + +import com.google.protobuf.Message; +import com.google.protobuf.MessageOrBuilder; +import org.apache.hadoop.fs.Path; +import parquet.hadoop.ParquetWriter; +import parquet.hadoop.api.WriteSupport; +import parquet.hadoop.metadata.CompressionCodecName; + +import java.io.IOException; + +/** + * Write Protobuffer records to a Parquet file. + */ +public class ProtoParquetWriter extends ParquetWriter { + + /** + * Create a new {@link ProtoParquetWriter}. + * + * @param file + * @param compressionCodecName + * @param blockSize + * @param pageSize + * @throws IOException + */ + public ProtoParquetWriter(Path file, Class protoMessage, + CompressionCodecName compressionCodecName, int blockSize, + int pageSize) throws IOException { + super(file, (WriteSupport) new ProtoWriteSupport(protoMessage), + compressionCodecName, blockSize, pageSize); + } + + /** + * Create a new {@link ProtoParquetWriter}. + * + * @param file The file name to write to. + * @param compressionCodecName Compression code to use, or CompressionCodecName.UNCOMPRESSED + * @param blockSize HDFS block size + * @param pageSize See parquet write up. Blocks are subdivided into pages for alignment and other purposes. + * @param enableDictionary Whether to use a dictionary to compress columns. + * @throws IOException + */ + public ProtoParquetWriter(Path file, Class protoMessage, + CompressionCodecName compressionCodecName, int blockSize, + int pageSize, boolean enableDictionary) throws IOException { + super(file, (WriteSupport) + new ProtoWriteSupport(protoMessage), + compressionCodecName, blockSize, pageSize, enableDictionary, false); + } + + /** + * Create a new {@link ProtoParquetWriter}. The default block size is 50 MB.The default + * page size is 1 MB. Default compression is no compression. (Inherited from {@link ParquetWriter}) + * + * @param file The file name to write to. + * @throws IOException + */ + public ProtoParquetWriter(Path file, Class protoMessage) throws IOException { + this(file, protoMessage, CompressionCodecName.UNCOMPRESSED, + DEFAULT_BLOCK_SIZE, DEFAULT_PAGE_SIZE); + } + +} diff --git a/src/main/java/parquet/proto/ProtoReadSupport.java b/src/main/java/parquet/proto/ProtoReadSupport.java new file mode 100644 index 0000000000..22608bbbcb --- /dev/null +++ b/src/main/java/parquet/proto/ProtoReadSupport.java @@ -0,0 +1,70 @@ +/** + * Copyright 2012 Twitter, Inc. + * + * Licensed 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 KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package parquet.proto; + +import com.google.protobuf.Message; +import com.twitter.elephantbird.util.Protobufs; +import org.apache.hadoop.conf.Configuration; +import parquet.hadoop.api.ReadSupport; +import parquet.io.api.RecordMaterializer; +import parquet.schema.MessageType; + +import java.util.Map; + + +/** + * @author Lukas Nalezenec + */ +public class ProtoReadSupport extends ReadSupport { + + public static final String PB_REQUESTED_PROJECTION = "parquet.proto.projection"; + + public static final String PB_CLASS = "parquet.proto.class"; + public static final String PB_DESCRIPTOR = "parquet.proto.descriptor"; + + public static void setRequestedProjection(Configuration configuration, String requestedProjection) { + configuration.set(PB_REQUESTED_PROJECTION, requestedProjection); + } + + @Override + public ReadContext init(Configuration configuration, Map keyValueMetaData, MessageType fileSchema) { + String requestedProjectionString = configuration.get(PB_REQUESTED_PROJECTION); + if (requestedProjectionString != null) { + // TODO +// Schema avroRequestedProjection = new Schema.Parser().parse(requestedProjectionString); +// MessageType requestedProjection = new ProtoSchemaConverter().convert(avroRequestedProjection); +// fileSchema.checkContains(requestedProjection); +// return new ReadContext(requestedProjection); + + throw new RuntimeException("Not implemented yet"); + } else { + return new ReadContext(fileSchema); + } + } + + @Override + public RecordMaterializer prepareForRead(Configuration configuration, Map keyValueMetaData, MessageType fileSchema, ReadContext readContext) { + String strProtoClass = keyValueMetaData.get(PB_CLASS); + + if (strProtoClass == null) { + throw new RuntimeException("Needs parameter " + PB_CLASS + " with protobufer class"); + } + + return new ProtoRecordMaterializer(readContext.getRequestedSchema(), Protobufs.getProtobufClass(strProtoClass)); + } + + +} diff --git a/src/main/java/parquet/proto/ProtoRecordMaterializer.java b/src/main/java/parquet/proto/ProtoRecordMaterializer.java new file mode 100644 index 0000000000..f88b829429 --- /dev/null +++ b/src/main/java/parquet/proto/ProtoRecordMaterializer.java @@ -0,0 +1,41 @@ +/** + * Copyright 2012 Twitter, Inc. + * + * Licensed 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 KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package parquet.proto; + +import com.google.protobuf.Message; +import com.google.protobuf.MessageOrBuilder; +import parquet.io.api.GroupConverter; +import parquet.io.api.RecordMaterializer; +import parquet.schema.MessageType; + +class ProtoRecordMaterializer extends RecordMaterializer { + + private final ProtobufferRecordConverter root; + + public ProtoRecordMaterializer(MessageType requestedSchema, Class protobufClass) { + this.root = new ProtobufferRecordConverter(protobufClass, requestedSchema); + } + + @Override + public T getCurrentRecord() { + return root.getCurrentRecord(); + } + + @Override + public GroupConverter getRootConverter() { + return root; + } +} diff --git a/src/main/java/parquet/proto/ProtoSchemaConverter.java b/src/main/java/parquet/proto/ProtoSchemaConverter.java new file mode 100644 index 0000000000..304d520d60 --- /dev/null +++ b/src/main/java/parquet/proto/ProtoSchemaConverter.java @@ -0,0 +1,131 @@ +/** + * Copyright 2012 Twitter, Inc. + * + * Licensed 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 KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package parquet.proto; + +import com.google.protobuf.Descriptors; +import com.google.protobuf.Message; +import com.twitter.elephantbird.util.Protobufs; +import parquet.schema.*; + +import java.util.ArrayList; +import java.util.List; + +import static parquet.schema.OriginalType.ENUM; +import static parquet.schema.OriginalType.UTF8; +import static parquet.schema.PrimitiveType.PrimitiveTypeName.*; + +/** + *

+ * Converts an Protobuffer Descriptor into a Parquet schema. + * + * @author Lukas Nalezenec + */ +public class ProtoSchemaConverter { + + public MessageType convert(Class protobufClass) { + Descriptors.Descriptor descriptor = Protobufs.getMessageDescriptor(protobufClass); + //TODO co vlastne dela vraci metoda getFullName ? + MessageType messageType = new MessageType(descriptor.getFullName(), convertFields(descriptor.getFields())); + + System.out.println("Convertor info:\n " + descriptor.toProto() + " was converted to \n" + messageType); + return messageType; + } + + /* Iterates over list of fields. **/ + private List convertFields(List fieldDescriptors) { + List types = new ArrayList(); + + // todo tahle metoda je dulezita protoze urcuje poradi fieldu ve schematu. + // (ale na poradi by nemelo zalezet protoze parujese podle jmen + + for (Descriptors.FieldDescriptor fieldDescriptor : fieldDescriptors) { + + String fieldName = fieldDescriptor.getName(); + Type.Repetition repetition = getRepetition(fieldDescriptor); + + Type type; + if (fieldDescriptor.isRepeated()) { + //TODO proc jsou tady dve repetice, proc dole neni konstanta ? + //Tohle je zajimave mist, tady muzou byt chyby ze spatneho prevodu + + + //tady by mohl byt spatne ten descriptor + + Type nestedType = convertScalarField(fieldName + "_tuple", fieldDescriptor, Type.Repetition.REPEATED); + type = ConversionPatterns.listType(Type.Repetition.OPTIONAL, fieldName, nestedType); + //throw new RuntimeException("Tady je chyba, v promenne repetition musi byt neco jineho - asi repetition nadrazeneho fieldu"); + } else { + type = convertScalarField(fieldName, fieldDescriptor, repetition); + } + + types.add(type); + } + return types; + } + + private Type.Repetition getRepetition(Descriptors.FieldDescriptor descriptor) { + Type.Repetition repetition; + if (descriptor.isRequired()) { + repetition = Type.Repetition.REQUIRED; + } else if (descriptor.isRepeated()) { + repetition = Type.Repetition.REPEATED; + } else { + repetition = Type.Repetition.OPTIONAL; + } + return repetition; + } + + private Type convertScalarField(String fieldName, Descriptors.FieldDescriptor descriptor, Type.Repetition repetition) { + Descriptors.FieldDescriptor.Type type = descriptor.getType(); + if (type.equals(Descriptors.FieldDescriptor.Type.BOOL)) { + return primitive(fieldName, BOOLEAN, repetition); + } else if (type.equals(Descriptors.FieldDescriptor.Type.INT32)) { + return primitive(fieldName, INT32, repetition); + } else if (type.equals(Descriptors.FieldDescriptor.Type.INT64)) { + return primitive(fieldName, INT64, repetition); + } else if (type.equals(Descriptors.FieldDescriptor.Type.FLOAT)) { + return primitive(fieldName, FLOAT, repetition); + } else if (type.equals(Descriptors.FieldDescriptor.Type.DOUBLE)) { + return primitive(fieldName, DOUBLE, repetition); + } else if (type.equals(Descriptors.FieldDescriptor.Type.BYTES)) { + return primitive(fieldName, BINARY, repetition); + } else if (type.equals(Descriptors.FieldDescriptor.Type.STRING)) { + return primitive(fieldName, BINARY, repetition, UTF8); + } else if (type.equals(Descriptors.FieldDescriptor.Type.MESSAGE)) { + Descriptors.Descriptor messageDescriptor = descriptor.getMessageType(); + List fields = convertFields(messageDescriptor.getFields()); + return new GroupType(repetition, fieldName, fields); + } else if (type.equals(Descriptors.FieldDescriptor.Type.ENUM)) { + return primitive(fieldName, BINARY, repetition, ENUM); + } + + throw new UnsupportedOperationException("Cannot convert Protobuffer type " + type); + } + + /** + * Makes primitive type with additional information. Used for String and Binary types + */ + private Type primitive(String name, PrimitiveType.PrimitiveTypeName primitive, + Type.Repetition repetition, OriginalType originalType) { + return new PrimitiveType(repetition, primitive, name, originalType); + } + + private PrimitiveType primitive(String name, PrimitiveType.PrimitiveTypeName + primitive, Type.Repetition repetition) { + return new PrimitiveType(repetition, primitive, name, null); + } + +} \ No newline at end of file diff --git a/src/main/java/parquet/proto/ProtoWriteSupport.java b/src/main/java/parquet/proto/ProtoWriteSupport.java new file mode 100644 index 0000000000..94e159aef9 --- /dev/null +++ b/src/main/java/parquet/proto/ProtoWriteSupport.java @@ -0,0 +1,185 @@ +/** + * Copyright 2012 Twitter, Inc. + * + * Licensed 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 KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package parquet.proto; + +import com.google.protobuf.DescriptorProtos; +import com.google.protobuf.Descriptors; +import com.google.protobuf.Message; +import com.google.protobuf.MessageOrBuilder; +import com.twitter.elephantbird.util.Protobufs; +import org.apache.hadoop.conf.Configuration; +import parquet.hadoop.api.WriteSupport; +import parquet.io.api.Binary; +import parquet.io.api.RecordConsumer; +import parquet.schema.GroupType; +import parquet.schema.MessageType; +import parquet.schema.Type; + +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + + +public class ProtoWriteSupport extends WriteSupport { + + private RecordConsumer recordConsumer; + private MessageType rootSchema; + private static final String PB_CLASS_WRITE = "parquet.proto.writeClass"; + private Class protoMessage; + + public ProtoWriteSupport() { + } + + public ProtoWriteSupport(Class protobufClass) { + this.protoMessage = protobufClass; + rootSchema = new ProtoSchemaConverter().convert(protobufClass); + } + + @Override + public WriteContext init(Configuration configuration) { + Map extraMetaData = new HashMap(); + extraMetaData.put(ProtoReadSupport.PB_CLASS, protoMessage.getName()); + extraMetaData.put(ProtoReadSupport.PB_DESCRIPTOR, serializeDescriptor(protoMessage)); + + // TODO add also pig descriptor + // see Thrift code ThriftWriteSupport + return new WriteContext(rootSchema, extraMetaData); + } + + private String serializeDescriptor(Class protoClass) { + Descriptors.Descriptor descriptor = Protobufs.getMessageDescriptor(protoClass); + DescriptorProtos.DescriptorProto asProto = descriptor.toProto(); + return asProto.toString(); + } + + + public static void setSchema(Configuration configuration, Class protoClass) { + configuration.setClass(PB_CLASS_WRITE, protoClass, Message.class); + } + + @Override + public void prepareForWrite(RecordConsumer recordConsumer) { + this.recordConsumer = recordConsumer; + } + + @Override + public void write(T record) { + recordConsumer.startMessage(); + writeRecordFields(rootSchema, record); + recordConsumer.endMessage(); + } + + private void writeMessage(GroupType schema, T message) { + recordConsumer.startGroup(); + writeRecordFields(schema, message); + recordConsumer.endGroup(); + } + + private void writeRecordFields(GroupType parquetSchema, T record) { + List fields = parquetSchema.getFields(); + + Map pbFields = record.getAllFields(); + + for (Map.Entry entry : pbFields.entrySet()) { + Descriptors.FieldDescriptor fieldDescriptor = entry.getKey(); + int protoIndex = fieldDescriptor.getIndex(); + //TODO tohle je preci blbost, tady musi byt mapovani + // UPDATE: blbost to neni ale mel bych si to pojistit proti zmenam + Object value = entry.getValue(); + + Type fieldType = fields.get(protoIndex); + + String parName = fieldType.getName();//TODO remove me + String proName = fieldDescriptor.getName(); + if (!parName.equals(proName )) throw new RuntimeException("Field mismatch"); + + + if (value != null) { + + if (fieldDescriptor.isRepeated()) { + int parquetIndex = parquetSchema.getFieldIndex(fieldDescriptor.getName()); + recordConsumer.startField(fieldType.getName(), parquetIndex); + Type subType = parquetSchema.asGroupType().getType(0); + writeArray(fieldType.asGroupType(), fieldDescriptor, (List) value); + recordConsumer.endField(fieldType.getName(), parquetIndex); + } else { + int parquetIndex = parquetSchema.getFieldIndex(fieldDescriptor.getName()); + recordConsumer.startField(fieldType.getName(), parquetIndex); + String parquetName = fieldType.getName(); // TODO remove + String protoName = fieldDescriptor.getName(); + if (parquetName != protoName) throw new RuntimeException("Name mismatch " + parquetName + " != " + protoName); + writeScalarValue(fieldType, fieldDescriptor, value); + recordConsumer.endField(fieldType.getName(), parquetIndex); + } + + } else if (fieldType.isRepetition(Type.Repetition.REQUIRED)) { + throw new RuntimeException("Null-value for required field: " + fieldDescriptor.getName()); + } + } + } + + private void writeArray(GroupType schema, Descriptors.FieldDescriptor fieldDescriptor, + List array) { + if (!schema.getName().equals(fieldDescriptor.getName())) throw new RuntimeException("Mismatch");//TODO remove me + + recordConsumer.startGroup(); + if (array.iterator().hasNext()) { + String arrayType = schema.getName(); + recordConsumer.startField(arrayType, 0); + for (T elt : array) { + writeScalarValue((schema.getType(0)), fieldDescriptor, elt); // patch + } + recordConsumer.endField(arrayType, 0); + } + recordConsumer.endGroup(); + } + + + private void writeScalarValue(Type type, Descriptors.FieldDescriptor fieldDescriptor, Object value) { + + Descriptors.FieldDescriptor.JavaType javaType = fieldDescriptor.getJavaType(); + + if (javaType.equals(Descriptors.FieldDescriptor.JavaType.BOOLEAN)) { + recordConsumer.addBoolean((Boolean) value); + } else if (javaType.equals(Descriptors.FieldDescriptor.JavaType.INT)) { + recordConsumer.addInteger(((Number) value).intValue()); + } else if (javaType.equals(Descriptors.FieldDescriptor.JavaType.LONG)) { + recordConsumer.addLong(((Number) value).longValue()); + } else if (javaType.equals(Descriptors.FieldDescriptor.JavaType.FLOAT)) { + recordConsumer.addFloat(((Number) value).floatValue()); + } else if (javaType.equals(Descriptors.FieldDescriptor.JavaType.DOUBLE)) { + recordConsumer.addDouble(((Number) value).doubleValue()); + } else if (javaType.equals(Descriptors.FieldDescriptor.JavaType.BYTE_STRING)) { + recordConsumer.addBinary(Binary.fromByteBuffer((ByteBuffer) value)); + } else if (javaType.equals(Descriptors.FieldDescriptor.JavaType.STRING)) { + recordConsumer.addBinary(stringToBinary(value)); + } else if (javaType.equals(Descriptors.FieldDescriptor.JavaType.MESSAGE)) { + writeMessage(type.asGroupType(), (T) value);// patched + } else if (javaType.equals(Descriptors.FieldDescriptor.JavaType.ENUM)) { + Descriptors.EnumValueDescriptor enumDescriptor = (Descriptors.EnumValueDescriptor) value; + recordConsumer.addBinary(Binary.fromString(enumDescriptor.getName())); + } else { + String msg = "Cannot write " + value + " with descriptor " + fieldDescriptor + " and type " + javaType; + throw new RuntimeException(msg); + } + } + + private Binary stringToBinary(Object value) { + return Binary.fromString(value.toString()); + } + +} diff --git a/src/main/java/parquet/proto/ProtobufferRecordConverter.java b/src/main/java/parquet/proto/ProtobufferRecordConverter.java new file mode 100644 index 0000000000..a952e721bc --- /dev/null +++ b/src/main/java/parquet/proto/ProtobufferRecordConverter.java @@ -0,0 +1,68 @@ +/** + * Copyright 2012 Twitter, Inc. + * + * Licensed 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 KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package parquet.proto; + + +import com.google.protobuf.Message; +import com.google.protobuf.MessageOrBuilder; +import parquet.proto.converters.ParentValueContainer; +import parquet.schema.MessageType; + +/** + * Converts root protobuffer message. + * + * @author Lukas Nalezenec + */ +class ProtobufferRecordConverter extends parquet.proto.converters.ProtoMessageConverter { + + final Message.Builder reusedBuilder; + boolean buildBefore; + + /** We dont need to write message value at top level. */ + private static class SkipParentValueContainer extends ParentValueContainer { + @Override + public void add(Object a) { + throw new RuntimeException("Should never happen"); + } + } + + + public ProtobufferRecordConverter(Class protoclass, MessageType parquetSchema) { + super(new SkipParentValueContainer(), protoclass, parquetSchema); + reusedBuilder = getBuilder(); + } + + + @Override + public void start() { + reusedBuilder.clear(); + super.start(); + } + + @Override + public void end() { + // do nothing, dont call ParentValueContainer. + } + + T getCurrentRecord() { + if (buildBefore) { + return (T) this.reusedBuilder.build(); + } else { + return (T) this.reusedBuilder; + } + } + +} diff --git a/src/main/java/parquet/proto/converters/ParentValueContainer.java b/src/main/java/parquet/proto/converters/ParentValueContainer.java new file mode 100644 index 0000000000..4531cf60d7 --- /dev/null +++ b/src/main/java/parquet/proto/converters/ParentValueContainer.java @@ -0,0 +1,10 @@ +package parquet.proto.converters; + +public abstract class ParentValueContainer { + + /** + * Adds the value to the parent. + */ + public abstract void add(Object value); + +} \ No newline at end of file diff --git a/src/main/java/parquet/proto/converters/ProtoArrayConverter.java b/src/main/java/parquet/proto/converters/ProtoArrayConverter.java new file mode 100644 index 0000000000..a9ef770322 --- /dev/null +++ b/src/main/java/parquet/proto/converters/ProtoArrayConverter.java @@ -0,0 +1,28 @@ +package parquet.proto.converters; + +import parquet.io.api.Converter; +import parquet.io.api.GroupConverter; + +public class ProtoArrayConverter extends GroupConverter { + + private final Converter converter; + + public ProtoArrayConverter(Converter innerConverter) { + converter = innerConverter; + } + + @Override + public Converter getConverter(int i) { + return converter; + } + + @Override + public void start() { + + } + + @Override + public void end() { + + } +} diff --git a/src/main/java/parquet/proto/converters/ProtoBinaryConverter.java b/src/main/java/parquet/proto/converters/ProtoBinaryConverter.java new file mode 100644 index 0000000000..90dd5cba27 --- /dev/null +++ b/src/main/java/parquet/proto/converters/ProtoBinaryConverter.java @@ -0,0 +1,18 @@ +package parquet.proto.converters; + +import parquet.io.api.Binary; +import parquet.io.api.PrimitiveConverter; + +public final class ProtoBinaryConverter extends PrimitiveConverter { + + final ParentValueContainer parent; + + public ProtoBinaryConverter(ParentValueContainer parent) { + this.parent = parent; + } + + @Override + public void addBinary(Binary value) { + parent.add(value); + } +} diff --git a/src/main/java/parquet/proto/converters/ProtoBooleanConverter.java b/src/main/java/parquet/proto/converters/ProtoBooleanConverter.java new file mode 100644 index 0000000000..58974f0a2c --- /dev/null +++ b/src/main/java/parquet/proto/converters/ProtoBooleanConverter.java @@ -0,0 +1,18 @@ +package parquet.proto.converters; + +import parquet.io.api.PrimitiveConverter; + +public final class ProtoBooleanConverter extends PrimitiveConverter { + + final ParentValueContainer parent; + + public ProtoBooleanConverter(ParentValueContainer parent) { + this.parent = parent; + } + + @Override + final public void addBoolean(boolean value) { + parent.add(value ? Boolean.TRUE : Boolean.FALSE); + } + +} diff --git a/src/main/java/parquet/proto/converters/ProtoDoubleConverter.java b/src/main/java/parquet/proto/converters/ProtoDoubleConverter.java new file mode 100644 index 0000000000..9dfd995b95 --- /dev/null +++ b/src/main/java/parquet/proto/converters/ProtoDoubleConverter.java @@ -0,0 +1,17 @@ +package parquet.proto.converters; + +import parquet.io.api.PrimitiveConverter; + +public final class ProtoDoubleConverter extends PrimitiveConverter { + + final ParentValueContainer parent; + + public ProtoDoubleConverter(ParentValueContainer parent) { + this.parent = parent; + } + + @Override + public void addDouble(double value) { + parent.add(value); + } +} diff --git a/src/main/java/parquet/proto/converters/ProtoEnumConverter.java b/src/main/java/parquet/proto/converters/ProtoEnumConverter.java new file mode 100644 index 0000000000..1a848f6fe5 --- /dev/null +++ b/src/main/java/parquet/proto/converters/ProtoEnumConverter.java @@ -0,0 +1,48 @@ +package parquet.proto.converters; + +import com.google.protobuf.Descriptors; +import parquet.io.api.Binary; +import parquet.io.api.PrimitiveConverter; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public final class ProtoEnumConverter extends PrimitiveConverter { + + private final Descriptors.FieldDescriptor fieldType; + private final Map enumLookup; + private final ParentValueContainer parent; + + public ProtoEnumConverter(ParentValueContainer parent, Descriptors.FieldDescriptor fieldType) { + this.parent = parent; + this.fieldType = fieldType; + this.enumLookup = makeLookupStructure(fieldType); + } + + Map makeLookupStructure(Descriptors.FieldDescriptor enumFieldType) { + Descriptors.EnumDescriptor enumType = enumFieldType.getEnumType(); + Map lookupStructure = new HashMap(); + + List enumValues = enumType.getValues(); + + for (Descriptors.EnumValueDescriptor value : enumValues) { + String name = value.getName(); + lookupStructure.put(Binary.fromString(name), enumType.findValueByName(name)); + } + + return lookupStructure; + } + + @Override + final public void addBinary(Binary value) { + Descriptors.EnumValueDescriptor protoValue = enumLookup.get(value); + + if (protoValue == null) { + throw new RuntimeException("Illegal enum value " + value + " in protoBuffer " + fieldType.getFullName()); + } + + parent.add(protoValue); + } + +} diff --git a/src/main/java/parquet/proto/converters/ProtoFloatConverter.java b/src/main/java/parquet/proto/converters/ProtoFloatConverter.java new file mode 100644 index 0000000000..3b3157ee97 --- /dev/null +++ b/src/main/java/parquet/proto/converters/ProtoFloatConverter.java @@ -0,0 +1,17 @@ +package parquet.proto.converters; + +import parquet.io.api.PrimitiveConverter; + +public final class ProtoFloatConverter extends PrimitiveConverter { + + final ParentValueContainer parent; + + public ProtoFloatConverter(ParentValueContainer parent) { + this.parent = parent; + } + + @Override + public void addFloat(float value) { + parent.add(value); + } +} diff --git a/src/main/java/parquet/proto/converters/ProtoIntConverter.java b/src/main/java/parquet/proto/converters/ProtoIntConverter.java new file mode 100644 index 0000000000..934391494e --- /dev/null +++ b/src/main/java/parquet/proto/converters/ProtoIntConverter.java @@ -0,0 +1,17 @@ +package parquet.proto.converters; + +import parquet.io.api.PrimitiveConverter; + +public final class ProtoIntConverter extends PrimitiveConverter { + + final ParentValueContainer parent; + + public ProtoIntConverter(ParentValueContainer parent) { + this.parent = parent; + } + + @Override + public void addInt(int value) { + parent.add(value); + } +} diff --git a/src/main/java/parquet/proto/converters/ProtoLongConverter.java b/src/main/java/parquet/proto/converters/ProtoLongConverter.java new file mode 100644 index 0000000000..f90c9ee538 --- /dev/null +++ b/src/main/java/parquet/proto/converters/ProtoLongConverter.java @@ -0,0 +1,17 @@ +package parquet.proto.converters; + +import parquet.io.api.PrimitiveConverter; + +public final class ProtoLongConverter extends PrimitiveConverter { + + final ParentValueContainer parent; + + public ProtoLongConverter(ParentValueContainer parent) { + this.parent = parent; + } + + @Override + public void addLong(long value) { + parent.add(value); + } +} diff --git a/src/main/java/parquet/proto/converters/ProtoMessageConverter.java b/src/main/java/parquet/proto/converters/ProtoMessageConverter.java new file mode 100644 index 0000000000..5ad52dfc5a --- /dev/null +++ b/src/main/java/parquet/proto/converters/ProtoMessageConverter.java @@ -0,0 +1,170 @@ +package parquet.proto.converters; + +import com.google.protobuf.Descriptors; +import com.google.protobuf.Message; +import com.twitter.elephantbird.util.Protobufs; +import parquet.io.api.Converter; +import parquet.io.api.GroupConverter; +import parquet.schema.GroupType; +import parquet.schema.Type; + +/** + * @author Lukas Nalezenec + */ +public class ProtoMessageConverter extends GroupConverter { + + private final Converter[] converters; + private final ParentValueContainer parent; + private final Message.Builder myBuilder; + + // used in record converter + public ProtoMessageConverter(ParentValueContainer pvc, Class protoClass, GroupType parquetSchema) { + this(pvc, Protobufs.getMessageBuilder(protoClass), parquetSchema); + } + + /** + * For usage in MSG arrays + * buidler je tam protoze predtim tam byl descriptor ale nejde jednoduse vytvorit builder z descriptoru + */ + public ProtoMessageConverter(ParentValueContainer pvc, Message.Builder builder, GroupType parquetSchema) { + + int schemaSize = parquetSchema.getFieldCount(); + converters = new Converter[schemaSize]; + + this.parent = pvc; + int parquetFieldIndex = 1; + + if (pvc == null) { + throw new IllegalStateException("Missing parent value container"); + } + + myBuilder = builder; + + Descriptors.Descriptor protoDescriptor = builder.getDescriptorForType(); + + for (Type parquetField : parquetSchema.getFields()) { // ziskavat jeden field bych IMHO mel o uroven vis + Descriptors.FieldDescriptor protoField = protoDescriptor.findFieldByName(parquetField.getName()); + + if (parquetField.isRepetition(Type.Repetition.REPEATED)) { + GroupType groupType = parquetField.asGroupType(); + if (groupType.getFieldCount() != 1) throw new RuntimeException("One field expected but found " + groupType); + + // TODO find this hack in avro and Thrift + parquetField = groupType.getType(0); + protoField = protoDescriptor.findFieldByName(parquetField.getName()); + } + + if (protoField == null) { + throw new RuntimeException("Cant find " + parquetField.getName() + " Scheme mismatch \n\"" + parquetField + "\"\n proto descriptor:\n" + protoDescriptor.toProto()); + } + + converters[parquetFieldIndex - 1] = newMessageConverter(myBuilder, protoField, parquetField); + + parquetFieldIndex++; + } + } + + + @Override + public Converter getConverter(int fieldIndex) { + return converters[fieldIndex]; + } + + @Override + public void start() { + + } + + @Override + public void end() { + parent.add(myBuilder.build()); + //todo myBuilder.clear(); + // TODO should be overriden in MessageRecordConverter + } + + private Converter newMessageConverter(final Message.Builder parentBuilder, final Descriptors.FieldDescriptor fieldDescriptor, Type parquetType) { + + boolean isRepeated = fieldDescriptor.isRepeated(); + + ParentValueContainer parent; + + if (isRepeated) { + parent = new ParentValueContainer() { + @Override + public void add(Object value) { + parentBuilder.addRepeatedField(fieldDescriptor, value); + } + }; + } else { + parent = new ParentValueContainer() { + @Override + public void add(Object value) { + parentBuilder.setField(fieldDescriptor, value); + } + }; + } + + if (isRepeated) { + parquetType = parquetType.asGroupType().getType(0); + } + + Converter innerConverter = newScalarConverter(parent, parentBuilder, fieldDescriptor, parquetType); + + if (isRepeated) { + return new ProtoArrayConverter(innerConverter); + } else { + return innerConverter; + } + } + + + private static Converter newScalarConverter(ParentValueContainer pvc, Message.Builder parentBuilder, Descriptors.FieldDescriptor fieldDescriptor, Type parquetType) { + + Descriptors.FieldDescriptor.JavaType javaType = fieldDescriptor.getJavaType(); + + boolean isMessage = javaType.equals(Descriptors.FieldDescriptor.JavaType.MESSAGE); + + if (isMessage) { + + if (!fieldDescriptor.getContainingType().equals(parentBuilder.getDescriptorForType())) { + throw new RuntimeException(fieldDescriptor.getFullName() + " is not inside " + parentBuilder.getDescriptorForType().getFullName()); + } + + GroupType parquetSubType = parquetType.asGroupType();//.getType(fieldDescriptor.getName()).asGroupType(); + Message.Builder subBuilder; + if (fieldDescriptor.isRepeated()) { + subBuilder = parentBuilder.newBuilderForField(fieldDescriptor); + //subBuilder = parentBuilder. getFieldBuilder(fieldDescriptor); + } else { + subBuilder = parentBuilder.newBuilderForField(fieldDescriptor); + } + + return new ProtoMessageConverter(pvc, subBuilder, parquetSubType); + } else { + if (javaType.equals(Descriptors.FieldDescriptor.JavaType.STRING)) { + return new ProtobufStringConverter(pvc); + } else if (javaType.equals(Descriptors.FieldDescriptor.JavaType.FLOAT)) { + return new ProtoFloatConverter(pvc); + } else if (javaType.equals(Descriptors.FieldDescriptor.JavaType.DOUBLE)) { + return new ProtoDoubleConverter(pvc); + } else if (javaType.equals(Descriptors.FieldDescriptor.JavaType.BOOLEAN)) { + return new ProtoBooleanConverter(pvc); + } else if (javaType.equals(Descriptors.FieldDescriptor.JavaType.BYTE_STRING)) { + return new ProtoBinaryConverter(pvc); + } else if (javaType.equals(Descriptors.FieldDescriptor.JavaType.ENUM)) { + return new ProtoEnumConverter(pvc, fieldDescriptor); + } else if (javaType.equals(Descriptors.FieldDescriptor.JavaType.INT)) { + return new ProtoIntConverter(pvc); + } else if (javaType.equals(Descriptors.FieldDescriptor.JavaType.LONG)) { + return new ProtoLongConverter(pvc); + } + } + + throw new UnsupportedOperationException(String.format("Cannot convert type: %s" + + " (Parquet type: %s) ", javaType, parquetType)); + } + + public Message.Builder getBuilder() { + return myBuilder; + } +} diff --git a/src/main/java/parquet/proto/converters/ProtobufStringConverter.java b/src/main/java/parquet/proto/converters/ProtobufStringConverter.java new file mode 100644 index 0000000000..564a3d8ada --- /dev/null +++ b/src/main/java/parquet/proto/converters/ProtobufStringConverter.java @@ -0,0 +1,20 @@ +package parquet.proto.converters; + +import parquet.io.api.Binary; +import parquet.io.api.PrimitiveConverter; + +public final class ProtobufStringConverter extends PrimitiveConverter { + + final ParentValueContainer parent; + + public ProtobufStringConverter(ParentValueContainer parent) { + this.parent = parent; + } + + + @Override + public void addBinary(Binary value) { + parent.add(value.toStringUsingUTF8()); + } + +} diff --git a/src/main/java/parquet/proto/package-info.java b/src/main/java/parquet/proto/package-info.java new file mode 100644 index 0000000000..60e02d55c5 --- /dev/null +++ b/src/main/java/parquet/proto/package-info.java @@ -0,0 +1,89 @@ +/** + * Copyright 2012 Twitter, Inc. + * + * Licensed 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 KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +/** + * + *

+ * Provides classes to store Avro data in Parquet files. Avro schemas are converted to + * parquet schemas as follows. Only record schemas are converted, + * other top-level schema types are not converted and attempting to do so will result + * in an error. Avro types are converted to Parquet types using the mapping shown here: + *

+ * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + *
Avro typeParquet type
nullno type (the field is not encoded in Parquet), unless a null union
booleanboolean
intint32
longint64
floatfloat
doubledouble
bytesbinary
stringbinary (with original type UTF8)
recordgroup containing nested fields
enumbinary (with original type ENUM)
arraygroup (with original type LIST) containing one repeated group field
mapgroup (with original type MAP) containing one repeated group + * field (with original type MAP_KEY_VALUE) of (key, value)
fixedfixed_len_byte_array
unionan optional type, in the case of a null union, otherwise not supported
+ */ +package parquet.proto; diff --git a/src/main/java/parquet/proto/todo.txt b/src/main/java/parquet/proto/todo.txt new file mode 100644 index 0000000000..f454f55ef8 --- /dev/null +++ b/src/main/java/parquet/proto/todo.txt @@ -0,0 +1,93 @@ + +// ALL TESTS GREEN + +!!! Asi musim zapisovat protobuffer zpatky v MessageConverter.end() + + +soucasna chyba: mozna tam mam jeden repeated level navic: +message TestProtobuf.RepeatedInnerMessage { + repeated group internal (LIST) { + repeated group internal_tuple { + optional int32 int; + } + } +} + + + +protobuffery muzou byt rekurzivni. Pokud rekurzi detekuju, tak bych mel treba treti level delat lazy. +! jak zpracovavat repeated fieldy ? Je to vlastne Avro pole. +OPTIONAL: Udelat lepsi varovani pro required fieldy. + + + +!!! ja bych mel identifikovat fieldy podle cisel, ne podle jmen. Cisla jsou k tomu urcena. +Podivat se na Thrift, tam to bude lepsi nez v Avru. +Nejspis se jedna o tridu ThriftSchemaConverter - metoda toThriftField +return new ThriftField(name, field.getId(), requirement, type); + +TODO: spravit unit testy. + +Otestovat psani msg +Testovat zpravy o vice fieldech. +zkontrolovat zapis poli - jestli se tam dobre volaji zacatky a konce + + +subProtobuffer je asi nutny vzdycky ulozit zpatky. + +ThriftSchemaConvertVisitor - tam se konvertuj typy v thriftu. + + +Jak se pri cteni mapuji jmena fieldu: +1. Avro. Projde schema ulozene v souboru a namatchuje se podle jmena fieldu +2. Thrift zapisuje si extra informace. Ma v hlavicce zapsany svuj descriptor ( a tridu) + matchovani probiha v konstruktoru StructConverter v ThriftRecordConverter + !!! a je to take podle jmena !!! ;)))) + +Ja konvertuju v ProtoWriteSupport, Avro ma to same. +Ja pouzivam protobufferove indexy, to je spatne. +Avro dostane fieldy tak serazene ze muze pouzit jejich index v poli +V Thrift se o to staraa ParquetWriteProtocol +Thrift pouziva mapovani thrift id -> parquet typ -> parquet id. Tak to chci udelat ja. +Me nestaci udelat jen protokol na zpravy a mozna pole protoze pro me bude vyhodne delat datove konverze primitivnich typu. + +Ja bych si mohl tak udelat slozeni trid. Bude se konstruovat pomoci ProtobufRecordSerializeru + +Co se pri zapisu musi delat: Zpravy musi byt uzavrene v group +uzavirat fieldy s indexem +spravne uzavreni pole. S polem by mohl byt problem. +konverze fieldu + +jak to udelat: initWrite +writery fieldu budou asi hloupe. Budou jen konvertovat. +Nebudu mit s polema problem ze tam musi byt pomocna trida ParentValue ? +Imho ne. +Jak mam pri zapisovani prochazet strom ? Zalezi na tom ? + + +mapovani je v ParquetWriteProtocol::StructWriteProtocol + +Odpoved je mozna v GroupType: + + public GroupType(Repetition repetition, String name, OriginalType originalType, List fields) { + super(name, repetition, originalType); + this.fields = fields; + this.indexByName = new HashMap(); + for (int i = 0; i < fields.size(); i++) { + indexByName.put(fields.get(i).getName(), i); + } + +To by znamenalo ze ty intexy jsou rekurzivni, platne jen v ramci jedne zpravy + + +public RecordMaterializer prepareForRead(Configuration configuration, + Map keyValueMetaData, MessageType fileSchema, + parquet.hadoop.api.ReadSupport.ReadContext readContext) { + ThriftMetaData thriftMetaData = ThriftMetaData.fromExtraMetaData(keyValueMetaData); + try { + + + } +ToDo: Najit propojeni na Grouptype v Avru + + diff --git a/src/test/java/parquet/proto/BugHuntingTest.java b/src/test/java/parquet/proto/BugHuntingTest.java new file mode 100644 index 0000000000..4d3ef34e63 --- /dev/null +++ b/src/test/java/parquet/proto/BugHuntingTest.java @@ -0,0 +1,54 @@ +package parquet.proto; + +import org.apache.hadoop.util.IndexedSortable; +import org.junit.Test; +import parquet.protobuf.test.TestProtobuf; + +import java.util.List; + +import static org.junit.Assert.assertEquals; + +public class BugHuntingTest { + + @Test + public void testIndexes() throws Exception { + TestProtobuf.Indexes.Builder indexes = TestProtobuf.Indexes.newBuilder(); + indexes.setField1("field1"); + indexes.setField2("field2"); + indexes.setField4("field4"); + indexes.setField5("field5"); + + List results; + results = TestUtils.writeAndRead((TestProtobuf.IndexesOrBuilder) indexes); + + TestProtobuf.IndexesOrBuilder result = results.get(0); + + assertEquals("field1", result.getField1()); + assertEquals("field2", result.getField2()); + assertEquals("field4", result.getField4()); + assertEquals("field5", result.getField5()); + } + + @Test + public void testIntArray() throws Exception { + TestProtobuf.IntArrayMessage.Builder a = TestProtobuf.IntArrayMessage.newBuilder(); + a.addIntArray(123).addIntArray(23).setSecondField("SecondField"); + + List result = TestUtils.writeAndRead((TestProtobuf.IntArrayMessageOrBuilder) a); + + List intArrayList = result.get(0).getIntArrayList(); + assertEquals(123, (int) intArrayList.get(0)); + assertEquals(23, (int) intArrayList.get(1)); + } + + @Test + public void testMessageArray() throws Exception { + TestProtobuf.MessageArrayMessage.Builder a = TestProtobuf.MessageArrayMessage .newBuilder(); + a.addMsgArrayBuilder().setInnerValue("test").setSecondField("SecondFieldInner"); + a.setSecondField("SecondFieldOuter"); + List result = TestUtils.writeAndRead((TestProtobuf.MessageArrayMessageOrBuilder) a); + + assertEquals("test", result.get(0).getMsgArrayList().get(0).getInnerValue()); + } + } + diff --git a/src/test/java/parquet/proto/ProtoTest.java b/src/test/java/parquet/proto/ProtoTest.java new file mode 100644 index 0000000000..bafa7b585a --- /dev/null +++ b/src/test/java/parquet/proto/ProtoTest.java @@ -0,0 +1,154 @@ +package parquet.proto; + +import com.google.protobuf.MessageOrBuilder; +import org.apache.hadoop.fs.Path; +import org.junit.Test; +import parquet.protobuf.test.TestProtobuf; + +import java.util.List; + +import static junit.framework.Assert.assertEquals; +import static junit.framework.Assert.fail; +import static parquet.proto.TestUtils.*; +import static parquet.protobuf.test.TestProtobuf.*; + +public class ProtoTest { + + + @Test + public void testModificationAfterWrite() throws Exception { + IntMessage.Builder record = IntMessage.newBuilder().setInt(776); + + Path file = tempDirectory(); + + ProtoParquetWriter writer = + new ProtoParquetWriter(file, IntMessage.class); + + writer.write(record); + record.setInt(666); + writer.close(); + + IntMessageOrBuilder nextRecord = (IntMessageOrBuilder) readMessages(file).get(0); + + assertEquals(776, nextRecord.getInt()); + } + + @Test + public void testWriteIntMessage() throws Exception { + IntMessage record = IntMessage.newBuilder().setInt(776).build(); + + IntMessageOrBuilder nextRecord = writeAndRead(record).get(0); + + assertEquals(776, nextRecord.getInt()); + } + + @Test + public void testWriteEnumMessage() throws Exception { + TestProtobuf.EnumMessage.Builder record = TestProtobuf.EnumMessage.newBuilder().setEnum(TestProtobuf.EnumMessage.Example.EnumValueA); + + TestProtobuf.EnumMessageOrBuilder nextRecord = writeAndRead(record).get(0); + + assertEquals(TestProtobuf.EnumMessage.Example.EnumValueA, nextRecord.getEnum()); + } + + + /** + * Tests that i can write both builder and message. + * */ + @Test + public void testWriteBuilder() throws Exception { + AllInOneMessage.Builder msg1 = AllInOneMessage.newBuilder(); + AllInOneSubMessage.Builder subMsg1 = msg1.getOptionalMsgBuilder(); + + AllInOneMessage.Builder msg2 = AllInOneMessage.newBuilder(); + AllInOneSubMessage.Builder subMsg2 = msg1.getOptionalMsgBuilder(); + + msg1.setRequiredString(" : Pulp fiction"); + subMsg1.setOptionalString("Jan Sverak: Tmavomodry svet"); + + msg2.setRequiredString(" : Snatch"); + subMsg2.setOptionalString(":Das Experiment"); + subMsg2.addRepeatedInt(2005); + + testData(msg1, msg2); + } + + + + + @Test + /** + * If i write two messages, both writes are independent. + * */ + public void testTwoMessages() throws Exception { + IntMessageOrBuilder record1 = IntMessage.newBuilder().setInt(776); + IntMessageOrBuilder record2 = IntMessage.newBuilder().setInt(1223); + + List result = writeAndRead(record1, record2); + + assertEquals(776, result.get(0).getInt()); + assertEquals(1223, result.get(1).getInt()); + assertEquals(2, result.size()); + } + + @Test + public void testMessageArray() throws Exception { + TestProtobuf.RepeatedInnerMessage.Builder record1 = TestProtobuf.RepeatedInnerMessage.newBuilder(); + + record1.addInternalBuilder().setInt(11); + record1.addInternalBuilder().setInt(22); + + + List result = writeAndRead((TestProtobuf.RepeatedInnerMessageOrBuilder) record1); + IntMessageOrBuilder resultInternal1 = result.get(0).getInternalOrBuilder(0); + IntMessageOrBuilder resultInternal2 = result.get(0).getInternalOrBuilder(1); + + assertEquals(11, resultInternal1.getInt()); + assertEquals(22, resultInternal2.getInt()); + } + + + @Test + public void testArray() throws Exception { + fail("Not implemented"); + } + + @Test + public void testInnerMessage() throws Exception { + fail("Not implemented"); + } + + @Test + public void testGroup() throws Exception { + fail("Not implemented"); + } + + @Test + public void testMessage() throws Exception { + + + TestProtobuf.WebPage.Builder record1 = TestProtobuf.WebPage.newBuilder(); + record1.setUrl("http://goout.cz"); + TestProtobuf.InternalMessage.Builder internalMsg = record1.getInternalMsgBuilder(); + internalMsg.setI(7878).setStr("someString"); + + List result = writeAndRead((TestProtobuf.WebPageOrBuilder) record1); + TestProtobuf.InternalMessage resultInternal = result.get(0).getInternalMsg(); + + assertEquals(7878, resultInternal.getI()); + assertEquals("someString", resultInternal.getStr()); + } + + @Test + public void testString() throws Exception { + TestProtobuf.StringMessageOrBuilder record1 = StringMessage.newBuilder().setName("as").setDescription("Des2"); + TestProtobuf.StringMessageOrBuilder record2 = StringMessage.newBuilder().setName("bbas").setDescription("Des1"); + + List x = writeAndRead(record1, record2); + + assertEquals("as", x.get(0).getName()); + assertEquals("Des2", x.get(0).getDescription()); + assertEquals("bbas", x.get(1).getName()); + assertEquals("Des1", x.get(1).getDescription()); + } +} diff --git a/src/test/java/parquet/proto/TestSandbox.java b/src/test/java/parquet/proto/TestSandbox.java new file mode 100644 index 0000000000..d6c9ef820c --- /dev/null +++ b/src/test/java/parquet/proto/TestSandbox.java @@ -0,0 +1,43 @@ +package parquet.proto; + +import com.google.protobuf.Descriptors; +import com.google.protobuf.ProtocolMessageEnum; +import org.junit.Test; +import parquet.protobuf.test.TestProtobuf; + +public class TestSandbox { + + @Test + public void testSandbox() throws Exception { + TestProtobuf.EnumMessage.Builder b = TestProtobuf.EnumMessage.newBuilder(); + b.setEnum(TestProtobuf.EnumMessage.Example.EnumValueA); + } + + @Test + public void XXA() throws Exception { + TestProtobuf.EnumMessage.Builder b = TestProtobuf.EnumMessage.newBuilder(); + b.setEnum(TestProtobuf.EnumMessage.Example.EnumValueA); + + + ProtocolMessageEnum[] enumValues = TestProtobuf.EnumMessage.Example.values(); + for (ProtocolMessageEnum value : enumValues) { + int index = value.getNumber(); + String name = value.getValueDescriptor().getName(); + System.out.println(name + " " + index); + } + } + + @Test + public void testName() throws Exception { + TestProtobuf.EnumMessage.Example x = TestProtobuf.EnumMessage.Example.EnumValueB; + Descriptors.EnumValueDescriptor valueDescriptor; + valueDescriptor = x.getValueDescriptor(); + + Descriptors.FieldDescriptor fieldDescriptor = TestProtobuf.EnumMessage.getDescriptor().findFieldByName("enum"); + TestProtobuf.EnumMessage.Builder builder = TestProtobuf.EnumMessage.newBuilder(); + builder.setField(fieldDescriptor, valueDescriptor); + + + System.out.println(builder.getEnum()); + } +} diff --git a/src/test/java/parquet/proto/TestUtils.java b/src/test/java/parquet/proto/TestUtils.java new file mode 100644 index 0000000000..d03f408fc4 --- /dev/null +++ b/src/test/java/parquet/proto/TestUtils.java @@ -0,0 +1,159 @@ +package parquet.proto; + +import com.google.protobuf.Message; +import com.google.protobuf.MessageOrBuilder; +import org.apache.hadoop.fs.Path; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.fail; + +public class TestUtils { + + public static Path tempDirectory() throws IOException { + File tmp = File.createTempFile("ParquetProtobuf_unitTest", ".tmp"); + tmp.deleteOnExit(); + tmp.delete(); + return new Path(tmp.getPath()); + } + + public static T writeAndReadSingle(T records) throws IOException { + return writeAndRead(records).get(0); + } + + public static List writeAndRead(T... records) throws IOException { + Class cls = inferRecordsClass(records); + + Path file = writeMessages(cls, records); + + return readMessages(file); + } + + private static Class inferRecordsClass(T[] records) { + Class cls = null; + + for (T record : records) { + Class recordClass; + if (record instanceof Message.Builder) { + recordClass = ((Message.Builder) record).build().getClass(); + } else if (record instanceof Message) { + recordClass = ((Message) record).getClass(); + } else { + throw new RuntimeException("Illegal class " + record); + } + + if (cls == null) { + cls = recordClass; + } else if (!cls.equals(recordClass)) { + throw new RuntimeException("Wrong class " + cls + ", expected protobuffer."); + } + } + return cls; + } + + public static void testData(MessageOrBuilder... messages) throws IOException { + + checkSameBuilderInstance(messages); + + List input = cloneList(messages); + + List output = writeAndRead(messages); + + assertEquals("The protobuffers are not same:\n", asMessages(input), asMessages(output)); + } + + private static List cloneList(MessageOrBuilder[] messages) { + List result = new ArrayList(); + + for (MessageOrBuilder mob: messages) { + result.add(asMessage(mob)); + } + + return result; + } + + public static List asMessages(List mobs) { + List result = new ArrayList(); + for (MessageOrBuilder messageOrBuilder : mobs) { + result.add(asMessage(messageOrBuilder)); + } + + return result; + } + /** Given message or builder returns same data as messagee*/ + public static Message asMessage(MessageOrBuilder mob) { + Message message; + if (mob instanceof Message.Builder) { + message = ((Message.Builder) mob).build(); + } else { + message = (Message) mob; + } + return message; + } + + /** Fails if some instance of builder is two times in list.*/ + private static void checkSameBuilderInstance(MessageOrBuilder[] messages) { + for (int i = 0; i < messages.length; i++) { + MessageOrBuilder firstMessage = messages[i]; + boolean isBuilder = firstMessage instanceof Message.Builder; + + if (isBuilder) { + for (int j = 0; j < messages.length; j++) { + MessageOrBuilder secondMessage = messages[j]; + + if (i != j) { + boolean isSame = secondMessage == firstMessage; + if (isSame) { + fail("Data contains two references to same instance." + secondMessage); + } + } + } + } + } + } + + /** Reads messages from given file. The file could/should be created by method writeMessages */ + public static List readMessages(Path file) throws IOException { + ProtoParquetReader reader = new ProtoParquetReader(file); + + List result = new ArrayList(); + boolean hasNext = true; + while (hasNext) { + T item = reader.read(); + if (item == null) { + hasNext = false; + } else { + assertNotNull(item); + // It makes sense to return message but production code wont work with messages + result.add((T) asMessage(item).toBuilder()); + } + } + reader.close(); + return result; + } + + public static Path writeMessages(MessageOrBuilder... records) throws IOException { + return writeMessages(inferRecordsClass(records), records); + } + + public static Path writeMessages(Class cls, MessageOrBuilder... records) throws IOException { + Path file = tempDirectory(); + + ProtoParquetWriter writer = + new ProtoParquetWriter(file, cls); + + for (MessageOrBuilder record : records) { + writer.write(record); + } + + writer.close(); + + return file; + } + +} diff --git a/src/test/resources/TestProtobuf.proto b/src/test/resources/TestProtobuf.proto new file mode 100644 index 0000000000..b13e86415e --- /dev/null +++ b/src/test/resources/TestProtobuf.proto @@ -0,0 +1,127 @@ +package TestProtobuf; + +option java_package = "parquet.protobuf.test"; + +// original dremel paper structures: Original paper used groups, not internal +// messages but groups were depricated. + +message Document { + required int64 DocId = 1; + optional Links links = 32; + repeated group Name = 24 { + repeated Language name = 4; + optional string url = 5; + } +} + +message Language { + required string code = 12; + optional string Country = 14; +} + +message Links { + repeated int64 Backward = 1; + repeated int64 Forward = 2; +} + + +message AllInOneMessage { + required string requiredString = 1; + optional string optionalString = 2; + repeated string repeatedString = 3; + + optional int32 optionalInt = 5; + repeated int32 repeatedInt = 6; + + optional float optionalFloat = 8; + repeated float repeatedFloat = 9; + + optional AllInOneSubMessage optionalMsg = 11; + repeated AllInOneSubMessage repeatedMsg = 12; +} + +message AllInOneSubMessage { + optional string optionalString = 20; + repeated string repeatedString = 30; + + optional int32 optionalInt = 50; + repeated int32 repeatedInt = 60; + + optional float optionalFloat = 80; + repeated float repeatedFloat = 90; +} + + +// Old: +message Link { + required string url = 1; + optional float weight = 2; + optional bool internal = 3; +} + +message InternalMessage { + optional string str = 1; + optional int32 i = 2; +} + +message WebPage { + required string url = 7; + //repeated Link forwardLinks = 23; + optional InternalMessage internalMsg = 55; +} + +message RecursiveMessage { + optional RecursiveMessage rec = 1; +} + +message IntMessage { + optional int32 int = 155; +} + +message RepeatedInnerMessage { + repeated IntMessage internal = 13; +} + +message EnumMessage { + enum Example { + EnumValueA = 435; + EnumValueB = 21; + EnumValueC = 44; + } + optional Example enum = 1; +} + +message StringMessage { + optional string name = 1; + optional string description = 2; +} + +//Bug hunting +message Indexes{ + optional string field1 = 1; + optional string field2 = 2; + optional string field4 = 4; + optional string field5 = 50; +} + +//Bug hunting +message IntArrayMessage { + repeated int32 intArray= 1; + optional string secondField = 2; +} + +message InnerArrayMessage{ + optional string innerValue = 1; + optional string secondField = 2; +} + +message MessageArrayMessage { + repeated InnerArrayMessage msgArray= 1; + optional string secondField = 2; +} + + + + + + From dd536a421b53f78550ed0c33a7927b813d391677 Mon Sep 17 00:00:00 2001 From: lukasnalezenec Date: Wed, 23 Oct 2013 21:56:02 +0200 Subject: [PATCH 03/31] Delete todo.txt --- src/main/java/parquet/proto/todo.txt | 93 ---------------------------- 1 file changed, 93 deletions(-) delete mode 100644 src/main/java/parquet/proto/todo.txt diff --git a/src/main/java/parquet/proto/todo.txt b/src/main/java/parquet/proto/todo.txt deleted file mode 100644 index f454f55ef8..0000000000 --- a/src/main/java/parquet/proto/todo.txt +++ /dev/null @@ -1,93 +0,0 @@ - -// ALL TESTS GREEN - -!!! Asi musim zapisovat protobuffer zpatky v MessageConverter.end() - - -soucasna chyba: mozna tam mam jeden repeated level navic: -message TestProtobuf.RepeatedInnerMessage { - repeated group internal (LIST) { - repeated group internal_tuple { - optional int32 int; - } - } -} - - - -protobuffery muzou byt rekurzivni. Pokud rekurzi detekuju, tak bych mel treba treti level delat lazy. -! jak zpracovavat repeated fieldy ? Je to vlastne Avro pole. -OPTIONAL: Udelat lepsi varovani pro required fieldy. - - - -!!! ja bych mel identifikovat fieldy podle cisel, ne podle jmen. Cisla jsou k tomu urcena. -Podivat se na Thrift, tam to bude lepsi nez v Avru. -Nejspis se jedna o tridu ThriftSchemaConverter - metoda toThriftField -return new ThriftField(name, field.getId(), requirement, type); - -TODO: spravit unit testy. - -Otestovat psani msg -Testovat zpravy o vice fieldech. -zkontrolovat zapis poli - jestli se tam dobre volaji zacatky a konce - - -subProtobuffer je asi nutny vzdycky ulozit zpatky. - -ThriftSchemaConvertVisitor - tam se konvertuj typy v thriftu. - - -Jak se pri cteni mapuji jmena fieldu: -1. Avro. Projde schema ulozene v souboru a namatchuje se podle jmena fieldu -2. Thrift zapisuje si extra informace. Ma v hlavicce zapsany svuj descriptor ( a tridu) - matchovani probiha v konstruktoru StructConverter v ThriftRecordConverter - !!! a je to take podle jmena !!! ;)))) - -Ja konvertuju v ProtoWriteSupport, Avro ma to same. -Ja pouzivam protobufferove indexy, to je spatne. -Avro dostane fieldy tak serazene ze muze pouzit jejich index v poli -V Thrift se o to staraa ParquetWriteProtocol -Thrift pouziva mapovani thrift id -> parquet typ -> parquet id. Tak to chci udelat ja. -Me nestaci udelat jen protokol na zpravy a mozna pole protoze pro me bude vyhodne delat datove konverze primitivnich typu. - -Ja bych si mohl tak udelat slozeni trid. Bude se konstruovat pomoci ProtobufRecordSerializeru - -Co se pri zapisu musi delat: Zpravy musi byt uzavrene v group -uzavirat fieldy s indexem -spravne uzavreni pole. S polem by mohl byt problem. -konverze fieldu - -jak to udelat: initWrite -writery fieldu budou asi hloupe. Budou jen konvertovat. -Nebudu mit s polema problem ze tam musi byt pomocna trida ParentValue ? -Imho ne. -Jak mam pri zapisovani prochazet strom ? Zalezi na tom ? - - -mapovani je v ParquetWriteProtocol::StructWriteProtocol - -Odpoved je mozna v GroupType: - - public GroupType(Repetition repetition, String name, OriginalType originalType, List fields) { - super(name, repetition, originalType); - this.fields = fields; - this.indexByName = new HashMap(); - for (int i = 0; i < fields.size(); i++) { - indexByName.put(fields.get(i).getName(), i); - } - -To by znamenalo ze ty intexy jsou rekurzivni, platne jen v ramci jedne zpravy - - -public RecordMaterializer prepareForRead(Configuration configuration, - Map keyValueMetaData, MessageType fileSchema, - parquet.hadoop.api.ReadSupport.ReadContext readContext) { - ThriftMetaData thriftMetaData = ThriftMetaData.fromExtraMetaData(keyValueMetaData); - try { - - - } -ToDo: Najit propojeni na Grouptype v Avru - - From e337bd2f9c54b58245e3e050372a32c555b2417f Mon Sep 17 00:00:00 2001 From: Lukas Date: Sun, 27 Oct 2013 21:24:46 +0100 Subject: [PATCH 04/31] Protobuf conversion over Java types --- .../parquet/proto/ProtoSchemaConverter.java | 23 ++++++++++--------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/src/main/java/parquet/proto/ProtoSchemaConverter.java b/src/main/java/parquet/proto/ProtoSchemaConverter.java index 304d520d60..86fe123860 100644 --- a/src/main/java/parquet/proto/ProtoSchemaConverter.java +++ b/src/main/java/parquet/proto/ProtoSchemaConverter.java @@ -89,30 +89,31 @@ private Type.Repetition getRepetition(Descriptors.FieldDescriptor descriptor) { } private Type convertScalarField(String fieldName, Descriptors.FieldDescriptor descriptor, Type.Repetition repetition) { - Descriptors.FieldDescriptor.Type type = descriptor.getType(); - if (type.equals(Descriptors.FieldDescriptor.Type.BOOL)) { + Descriptors.FieldDescriptor.JavaType javaType = descriptor.getJavaType(); + + if (javaType.equals(Descriptors.FieldDescriptor.JavaType.BOOLEAN)) { return primitive(fieldName, BOOLEAN, repetition); - } else if (type.equals(Descriptors.FieldDescriptor.Type.INT32)) { + } else if (javaType.equals(Descriptors.FieldDescriptor.JavaType.INT)) { return primitive(fieldName, INT32, repetition); - } else if (type.equals(Descriptors.FieldDescriptor.Type.INT64)) { + } else if (javaType.equals(Descriptors.FieldDescriptor.JavaType.LONG)) { return primitive(fieldName, INT64, repetition); - } else if (type.equals(Descriptors.FieldDescriptor.Type.FLOAT)) { + } else if (javaType.equals(Descriptors.FieldDescriptor.JavaType.FLOAT)) { return primitive(fieldName, FLOAT, repetition); - } else if (type.equals(Descriptors.FieldDescriptor.Type.DOUBLE)) { + } else if (javaType.equals(Descriptors.FieldDescriptor.JavaType.DOUBLE)) { return primitive(fieldName, DOUBLE, repetition); - } else if (type.equals(Descriptors.FieldDescriptor.Type.BYTES)) { + } else if (javaType.equals(Descriptors.FieldDescriptor.JavaType.BYTE_STRING)) { return primitive(fieldName, BINARY, repetition); - } else if (type.equals(Descriptors.FieldDescriptor.Type.STRING)) { + } else if (javaType.equals(Descriptors.FieldDescriptor.JavaType.STRING)) { return primitive(fieldName, BINARY, repetition, UTF8); - } else if (type.equals(Descriptors.FieldDescriptor.Type.MESSAGE)) { + } else if (javaType.equals(Descriptors.FieldDescriptor.JavaType.MESSAGE)) { Descriptors.Descriptor messageDescriptor = descriptor.getMessageType(); List fields = convertFields(messageDescriptor.getFields()); return new GroupType(repetition, fieldName, fields); - } else if (type.equals(Descriptors.FieldDescriptor.Type.ENUM)) { + } else if (javaType.equals(Descriptors.FieldDescriptor.JavaType.ENUM)) { return primitive(fieldName, BINARY, repetition, ENUM); } - throw new UnsupportedOperationException("Cannot convert Protobuffer type " + type); + throw new UnsupportedOperationException("Cannot convert Protobuffer: type " + javaType + " fieldName " + fieldName); } /** From 65ca5ed89db94a8217e8300b3222abd2918f1848 Mon Sep 17 00:00:00 2001 From: Lukas Date: Fri, 1 Nov 2013 16:28:27 +0100 Subject: [PATCH 05/31] Copyrights in converters --- .../proto/converters/ParentValueContainer.java | 16 ++++++++++++++++ .../proto/converters/ProtoArrayConverter.java | 16 ++++++++++++++++ .../proto/converters/ProtoBinaryConverter.java | 16 ++++++++++++++++ .../proto/converters/ProtoBooleanConverter.java | 16 ++++++++++++++++ .../proto/converters/ProtoDoubleConverter.java | 16 ++++++++++++++++ .../proto/converters/ProtoEnumConverter.java | 16 ++++++++++++++++ .../proto/converters/ProtoFloatConverter.java | 16 ++++++++++++++++ .../proto/converters/ProtoIntConverter.java | 16 ++++++++++++++++ .../proto/converters/ProtoLongConverter.java | 16 ++++++++++++++++ .../proto/converters/ProtoMessageConverter.java | 16 ++++++++++++++++ .../converters/ProtobufStringConverter.java | 16 ++++++++++++++++ 11 files changed, 176 insertions(+) diff --git a/src/main/java/parquet/proto/converters/ParentValueContainer.java b/src/main/java/parquet/proto/converters/ParentValueContainer.java index 4531cf60d7..33a0f4d431 100644 --- a/src/main/java/parquet/proto/converters/ParentValueContainer.java +++ b/src/main/java/parquet/proto/converters/ParentValueContainer.java @@ -1,3 +1,19 @@ +/** + * Copyright 2013 Lukas Nalezenec. + * + * Licensed 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 KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package parquet.proto.converters; public abstract class ParentValueContainer { diff --git a/src/main/java/parquet/proto/converters/ProtoArrayConverter.java b/src/main/java/parquet/proto/converters/ProtoArrayConverter.java index a9ef770322..e74d62fc41 100644 --- a/src/main/java/parquet/proto/converters/ProtoArrayConverter.java +++ b/src/main/java/parquet/proto/converters/ProtoArrayConverter.java @@ -1,3 +1,19 @@ +/** + * Copyright 2013 Lukas Nalezenec. + * + * Licensed 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 KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package parquet.proto.converters; import parquet.io.api.Converter; diff --git a/src/main/java/parquet/proto/converters/ProtoBinaryConverter.java b/src/main/java/parquet/proto/converters/ProtoBinaryConverter.java index 90dd5cba27..3149c412be 100644 --- a/src/main/java/parquet/proto/converters/ProtoBinaryConverter.java +++ b/src/main/java/parquet/proto/converters/ProtoBinaryConverter.java @@ -1,3 +1,19 @@ +/** + * Copyright 2013 Lukas Nalezenec. + * + * Licensed 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 KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package parquet.proto.converters; import parquet.io.api.Binary; diff --git a/src/main/java/parquet/proto/converters/ProtoBooleanConverter.java b/src/main/java/parquet/proto/converters/ProtoBooleanConverter.java index 58974f0a2c..b3cba6266f 100644 --- a/src/main/java/parquet/proto/converters/ProtoBooleanConverter.java +++ b/src/main/java/parquet/proto/converters/ProtoBooleanConverter.java @@ -1,3 +1,19 @@ +/** + * Copyright 2013 Lukas Nalezenec. + * + * Licensed 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 KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package parquet.proto.converters; import parquet.io.api.PrimitiveConverter; diff --git a/src/main/java/parquet/proto/converters/ProtoDoubleConverter.java b/src/main/java/parquet/proto/converters/ProtoDoubleConverter.java index 9dfd995b95..6921cc1f8a 100644 --- a/src/main/java/parquet/proto/converters/ProtoDoubleConverter.java +++ b/src/main/java/parquet/proto/converters/ProtoDoubleConverter.java @@ -1,3 +1,19 @@ +/** + * Copyright 2013 Lukas Nalezenec. + * + * Licensed 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 KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package parquet.proto.converters; import parquet.io.api.PrimitiveConverter; diff --git a/src/main/java/parquet/proto/converters/ProtoEnumConverter.java b/src/main/java/parquet/proto/converters/ProtoEnumConverter.java index 1a848f6fe5..0403429ded 100644 --- a/src/main/java/parquet/proto/converters/ProtoEnumConverter.java +++ b/src/main/java/parquet/proto/converters/ProtoEnumConverter.java @@ -1,3 +1,19 @@ +/** + * Copyright 2013 Lukas Nalezenec. + * + * Licensed 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 KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package parquet.proto.converters; import com.google.protobuf.Descriptors; diff --git a/src/main/java/parquet/proto/converters/ProtoFloatConverter.java b/src/main/java/parquet/proto/converters/ProtoFloatConverter.java index 3b3157ee97..2bb9fde6f6 100644 --- a/src/main/java/parquet/proto/converters/ProtoFloatConverter.java +++ b/src/main/java/parquet/proto/converters/ProtoFloatConverter.java @@ -1,3 +1,19 @@ +/** + * Copyright 2013 Lukas Nalezenec. + * + * Licensed 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 KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package parquet.proto.converters; import parquet.io.api.PrimitiveConverter; diff --git a/src/main/java/parquet/proto/converters/ProtoIntConverter.java b/src/main/java/parquet/proto/converters/ProtoIntConverter.java index 934391494e..04ae624327 100644 --- a/src/main/java/parquet/proto/converters/ProtoIntConverter.java +++ b/src/main/java/parquet/proto/converters/ProtoIntConverter.java @@ -1,3 +1,19 @@ +/** + * Copyright 2013 Lukas Nalezenec. + * + * Licensed 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 KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package parquet.proto.converters; import parquet.io.api.PrimitiveConverter; diff --git a/src/main/java/parquet/proto/converters/ProtoLongConverter.java b/src/main/java/parquet/proto/converters/ProtoLongConverter.java index f90c9ee538..ae76f08aa2 100644 --- a/src/main/java/parquet/proto/converters/ProtoLongConverter.java +++ b/src/main/java/parquet/proto/converters/ProtoLongConverter.java @@ -1,3 +1,19 @@ +/** + * Copyright 2013 Lukas Nalezenec. + * + * Licensed 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 KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package parquet.proto.converters; import parquet.io.api.PrimitiveConverter; diff --git a/src/main/java/parquet/proto/converters/ProtoMessageConverter.java b/src/main/java/parquet/proto/converters/ProtoMessageConverter.java index 5ad52dfc5a..490919d8e3 100644 --- a/src/main/java/parquet/proto/converters/ProtoMessageConverter.java +++ b/src/main/java/parquet/proto/converters/ProtoMessageConverter.java @@ -1,3 +1,19 @@ +/** + * Copyright 2013 Lukas Nalezenec. + * + * Licensed 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 KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package parquet.proto.converters; import com.google.protobuf.Descriptors; diff --git a/src/main/java/parquet/proto/converters/ProtobufStringConverter.java b/src/main/java/parquet/proto/converters/ProtobufStringConverter.java index 564a3d8ada..48cf08deab 100644 --- a/src/main/java/parquet/proto/converters/ProtobufStringConverter.java +++ b/src/main/java/parquet/proto/converters/ProtobufStringConverter.java @@ -1,3 +1,19 @@ +/** + * Copyright 2013 Lukas Nalezenec. + * + * Licensed 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 KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package parquet.proto.converters; import parquet.io.api.Binary; From a7de264ff2fa559ca5d5e6b8a33e3a5d2590952f Mon Sep 17 00:00:00 2001 From: Lukas Date: Sun, 3 Nov 2013 00:53:40 +0100 Subject: [PATCH 06/31] Specification of written protobuffer class in output format --- .../proto/ProtoParquetOutputFormat.java | 23 ++++++++++++++++--- .../java/parquet/proto/ProtoWriteSupport.java | 15 ++++++++++-- 2 files changed, 33 insertions(+), 5 deletions(-) diff --git a/src/main/java/parquet/proto/ProtoParquetOutputFormat.java b/src/main/java/parquet/proto/ProtoParquetOutputFormat.java index 8039223d20..e2757f8beb 100644 --- a/src/main/java/parquet/proto/ProtoParquetOutputFormat.java +++ b/src/main/java/parquet/proto/ProtoParquetOutputFormat.java @@ -1,5 +1,5 @@ /** - * Copyright 2012 Twitter, Inc. + * Copyright 2013 Lukas Nalezenec * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -22,14 +22,31 @@ import parquet.hadoop.util.ContextUtil; /** - * A Hadoop {@link org.apache.hadoop.mapreduce.OutputFormat} for Parquet files. + * A Hadoop {@link org.apache.hadoop.mapreduce.OutputFormat} for Protobuffer Parquet files. + * + * Usage: + * + *
+ * {@code
+ * final Job job = new Job(conf, "Parquet writing job");
+ * job.setOutputFormatClass(ProtoParquetOutputFormat.class);
+ * ProtoParquetOutputFormat.setOutputPath(job, parquetPath);
+ * ProtoParquetOutputFormat.setProtobufferClass(job, YourProtobuffer.class);
+ * }
+ * 
+ * + * @author Lukas Nalezenec */ public class ProtoParquetOutputFormat extends ParquetOutputFormat { - public static void setSchema(Job job, Class protoClass) { + public static void setProtobufferClass(Job job, Class protoClass) { ProtoWriteSupport.setSchema(ContextUtil.getConfiguration(job), protoClass); } + public ProtoParquetOutputFormat(Class msg) { + super(new ProtoWriteSupport(msg)); + } + public ProtoParquetOutputFormat() { super(new ProtoWriteSupport()); } diff --git a/src/main/java/parquet/proto/ProtoWriteSupport.java b/src/main/java/parquet/proto/ProtoWriteSupport.java index 94e159aef9..e2c297e71f 100644 --- a/src/main/java/parquet/proto/ProtoWriteSupport.java +++ b/src/main/java/parquet/proto/ProtoWriteSupport.java @@ -1,5 +1,5 @@ /** - * Copyright 2012 Twitter, Inc. + * Copyright 2013 Lukas Nalezenec * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -46,11 +46,22 @@ public ProtoWriteSupport() { public ProtoWriteSupport(Class protobufClass) { this.protoMessage = protobufClass; - rootSchema = new ProtoSchemaConverter().convert(protobufClass); + rootSchema = new ProtoSchemaConverter().convert(protoMessage); } @Override public WriteContext init(Configuration configuration) { + if (protoMessage == null) { + Class pbClass = configuration.getClass(PB_CLASS_WRITE, null, Message.class); + if (pbClass != null) { + rootSchema = new ProtoSchemaConverter().convert(protoMessage); + } else { + String msg = "Protobuffer class not specified."; + String hint = " Please use method ProtoParquetOutputFormat.setProtobufferClass(...) or other similar method."; + throw new RuntimeException(msg + hint); + } + } + Map extraMetaData = new HashMap(); extraMetaData.put(ProtoReadSupport.PB_CLASS, protoMessage.getName()); extraMetaData.put(ProtoReadSupport.PB_DESCRIPTOR, serializeDescriptor(protoMessage)); From 2e78704e82f62822f1183131340cbd59c7b7831f Mon Sep 17 00:00:00 2001 From: Lukas Date: Sun, 3 Nov 2013 16:36:59 +0100 Subject: [PATCH 07/31] Code cleanup --- .../java/parquet/proto/ProtoWriteSupport.java | 19 ++++--------------- 1 file changed, 4 insertions(+), 15 deletions(-) diff --git a/src/main/java/parquet/proto/ProtoWriteSupport.java b/src/main/java/parquet/proto/ProtoWriteSupport.java index e2c297e71f..a01cc03fac 100644 --- a/src/main/java/parquet/proto/ProtoWriteSupport.java +++ b/src/main/java/parquet/proto/ProtoWriteSupport.java @@ -106,33 +106,22 @@ private void writeRecordFields(GroupType parquetSchema, T record) { Map pbFields = record.getAllFields(); for (Map.Entry entry : pbFields.entrySet()) { + Descriptors.FieldDescriptor fieldDescriptor = entry.getKey(); int protoIndex = fieldDescriptor.getIndex(); - //TODO tohle je preci blbost, tady musi byt mapovani - // UPDATE: blbost to neni ale mel bych si to pojistit proti zmenam - Object value = entry.getValue(); - Type fieldType = fields.get(protoIndex); - String parName = fieldType.getName();//TODO remove me - String proName = fieldDescriptor.getName(); - if (!parName.equals(proName )) throw new RuntimeException("Field mismatch"); - + Object value = entry.getValue(); if (value != null) { + int parquetIndex = parquetSchema.getFieldIndex(fieldDescriptor.getName()); + if (fieldDescriptor.isRepeated()) { - int parquetIndex = parquetSchema.getFieldIndex(fieldDescriptor.getName()); recordConsumer.startField(fieldType.getName(), parquetIndex); - Type subType = parquetSchema.asGroupType().getType(0); writeArray(fieldType.asGroupType(), fieldDescriptor, (List) value); recordConsumer.endField(fieldType.getName(), parquetIndex); - } else { - int parquetIndex = parquetSchema.getFieldIndex(fieldDescriptor.getName()); recordConsumer.startField(fieldType.getName(), parquetIndex); - String parquetName = fieldType.getName(); // TODO remove - String protoName = fieldDescriptor.getName(); - if (parquetName != protoName) throw new RuntimeException("Name mismatch " + parquetName + " != " + protoName); writeScalarValue(fieldType, fieldDescriptor, value); recordConsumer.endField(fieldType.getName(), parquetIndex); } From 090a2a41e5e7ef8c409fbfc8f872707d3e1907c4 Mon Sep 17 00:00:00 2001 From: Lukas Date: Sun, 3 Nov 2013 16:51:10 +0100 Subject: [PATCH 08/31] Code Cleanup --- .../java/parquet/proto/ProtoSchemaConverter.java | 14 ++------------ 1 file changed, 2 insertions(+), 12 deletions(-) diff --git a/src/main/java/parquet/proto/ProtoSchemaConverter.java b/src/main/java/parquet/proto/ProtoSchemaConverter.java index 86fe123860..61cc47a151 100644 --- a/src/main/java/parquet/proto/ProtoSchemaConverter.java +++ b/src/main/java/parquet/proto/ProtoSchemaConverter.java @@ -29,7 +29,7 @@ /** *

- * Converts an Protobuffer Descriptor into a Parquet schema. + * Converts a Protobuffer Descriptor into a Parquet schema. * * @author Lukas Nalezenec */ @@ -37,7 +37,7 @@ public class ProtoSchemaConverter { public MessageType convert(Class protobufClass) { Descriptors.Descriptor descriptor = Protobufs.getMessageDescriptor(protobufClass); - //TODO co vlastne dela vraci metoda getFullName ? + MessageType messageType = new MessageType(descriptor.getFullName(), convertFields(descriptor.getFields())); System.out.println("Convertor info:\n " + descriptor.toProto() + " was converted to \n" + messageType); @@ -48,9 +48,6 @@ public MessageType convert(Class protobufClass) { private List convertFields(List fieldDescriptors) { List types = new ArrayList(); - // todo tahle metoda je dulezita protoze urcuje poradi fieldu ve schematu. - // (ale na poradi by nemelo zalezet protoze parujese podle jmen - for (Descriptors.FieldDescriptor fieldDescriptor : fieldDescriptors) { String fieldName = fieldDescriptor.getName(); @@ -58,15 +55,8 @@ private List convertFields(List fieldDescript Type type; if (fieldDescriptor.isRepeated()) { - //TODO proc jsou tady dve repetice, proc dole neni konstanta ? - //Tohle je zajimave mist, tady muzou byt chyby ze spatneho prevodu - - - //tady by mohl byt spatne ten descriptor - Type nestedType = convertScalarField(fieldName + "_tuple", fieldDescriptor, Type.Repetition.REPEATED); type = ConversionPatterns.listType(Type.Repetition.OPTIONAL, fieldName, nestedType); - //throw new RuntimeException("Tady je chyba, v promenne repetition musi byt neco jineho - asi repetition nadrazeneho fieldu"); } else { type = convertScalarField(fieldName, fieldDescriptor, repetition); } From 1bec97fa188a450a52289a4836b71edfa607e017 Mon Sep 17 00:00:00 2001 From: Lukas Date: Sun, 3 Nov 2013 17:06:03 +0100 Subject: [PATCH 09/31] Projections in read support --- src/main/java/parquet/proto/ProtoReadSupport.java | 13 ++++--------- 1 file changed, 4 insertions(+), 9 deletions(-) diff --git a/src/main/java/parquet/proto/ProtoReadSupport.java b/src/main/java/parquet/proto/ProtoReadSupport.java index 22608bbbcb..6957e19c77 100644 --- a/src/main/java/parquet/proto/ProtoReadSupport.java +++ b/src/main/java/parquet/proto/ProtoReadSupport.java @@ -42,14 +42,9 @@ public static void setRequestedProjection(Configuration configuration, String re @Override public ReadContext init(Configuration configuration, Map keyValueMetaData, MessageType fileSchema) { String requestedProjectionString = configuration.get(PB_REQUESTED_PROJECTION); - if (requestedProjectionString != null) { - // TODO -// Schema avroRequestedProjection = new Schema.Parser().parse(requestedProjectionString); -// MessageType requestedProjection = new ProtoSchemaConverter().convert(avroRequestedProjection); -// fileSchema.checkContains(requestedProjection); -// return new ReadContext(requestedProjection); - - throw new RuntimeException("Not implemented yet"); + if (requestedProjectionString != null && !requestedProjectionString.trim().isEmpty()) { + MessageType requestedProjection = getSchemaForRead(fileSchema, requestedProjectionString); + return new ReadContext(requestedProjection); } else { return new ReadContext(fileSchema); } @@ -60,7 +55,7 @@ public RecordMaterializer prepareForRead(Configuration configuration, Map Date: Sun, 3 Nov 2013 21:59:02 +0100 Subject: [PATCH 10/31] artifact version changed to 1.2.5, unused dependencies removed. --- pom.xml | 33 ++------------------------------- 1 file changed, 2 insertions(+), 31 deletions(-) diff --git a/pom.xml b/pom.xml index cb01c689d3..2ecc656e61 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ com.twitter parquet ../pom.xml - 1.2.3-SNAPSHOT + 1.2.5-SNAPSHOT 4.0.0 @@ -25,44 +25,15 @@ protobuf-java 2.4.1 - - com.twitter - parquet-column - ${project.version} - com.twitter parquet-hadoop ${project.version} - - com.twitter - parquet-format - 1.0.0 - - - org.apache.hadoop - hadoop-client - ${hadoop.version} - - - com.google.guava - guava - 11.0 - test - - - com.twitter - parquet-column - ${project.version} - test-jar - test - - com.twitter.elephantbird elephant-bird-core - 3.0.8 + ${elephant-bird.version} From 402e96dfb58077cdf3e6bf2e496bdca6bad11743 Mon Sep 17 00:00:00 2001 From: Lukas Date: Sat, 9 Nov 2013 22:43:56 +0100 Subject: [PATCH 11/31] Wrong merge --- src/main/java/parquet/proto/ProtoWriteSupport.java | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/java/parquet/proto/ProtoWriteSupport.java b/src/main/java/parquet/proto/ProtoWriteSupport.java index a01cc03fac..7fd3f0f35b 100644 --- a/src/main/java/parquet/proto/ProtoWriteSupport.java +++ b/src/main/java/parquet/proto/ProtoWriteSupport.java @@ -121,6 +121,7 @@ private void writeRecordFields(GroupType parquetSchema, T record) { recordConsumer.startField(fieldType.getName(), parquetIndex); writeArray(fieldType.asGroupType(), fieldDescriptor, (List) value); recordConsumer.endField(fieldType.getName(), parquetIndex); + } else { recordConsumer.startField(fieldType.getName(), parquetIndex); writeScalarValue(fieldType, fieldDescriptor, value); recordConsumer.endField(fieldType.getName(), parquetIndex); From e2d819c2ae6ab5573606e32f325c0743c6ec0c2f Mon Sep 17 00:00:00 2001 From: Lukas Date: Thu, 12 Dec 2013 16:59:33 +0100 Subject: [PATCH 12/31] Loading correct pbClass to ProtoSchemaConverter --- src/main/java/parquet/proto/ProtoWriteSupport.java | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/main/java/parquet/proto/ProtoWriteSupport.java b/src/main/java/parquet/proto/ProtoWriteSupport.java index 7fd3f0f35b..605335dbb2 100644 --- a/src/main/java/parquet/proto/ProtoWriteSupport.java +++ b/src/main/java/parquet/proto/ProtoWriteSupport.java @@ -38,7 +38,7 @@ public class ProtoWriteSupport extends WriteSupport< private RecordConsumer recordConsumer; private MessageType rootSchema; - private static final String PB_CLASS_WRITE = "parquet.proto.writeClass"; + static final String PB_CLASS_WRITE = "parquet.proto.writeClass"; private Class protoMessage; public ProtoWriteSupport() { @@ -51,10 +51,13 @@ public ProtoWriteSupport(Class protobufClass) { @Override public WriteContext init(Configuration configuration) { - if (protoMessage == null) { + + // if no protobuffer descriptor was given in constructor, load descriptor from configuration (set with setProtobufferClass) + if (protoMessage == null) { Class pbClass = configuration.getClass(PB_CLASS_WRITE, null, Message.class); if (pbClass != null) { - rootSchema = new ProtoSchemaConverter().convert(protoMessage); + protoMessage = pbClass; + rootSchema = new ProtoSchemaConverter().convert(pbClass); } else { String msg = "Protobuffer class not specified."; String hint = " Please use method ProtoParquetOutputFormat.setProtobufferClass(...) or other similar method."; From 08a204ddb8cfda6b41309af911c5a4511fd426df Mon Sep 17 00:00:00 2001 From: Lukas Date: Thu, 12 Dec 2013 17:00:34 +0100 Subject: [PATCH 13/31] Depricated init override removed --- src/main/java/parquet/proto/ProtoReadSupport.java | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/src/main/java/parquet/proto/ProtoReadSupport.java b/src/main/java/parquet/proto/ProtoReadSupport.java index 6957e19c77..edf06480bf 100644 --- a/src/main/java/parquet/proto/ProtoReadSupport.java +++ b/src/main/java/parquet/proto/ProtoReadSupport.java @@ -18,6 +18,7 @@ import com.google.protobuf.Message; import com.twitter.elephantbird.util.Protobufs; import org.apache.hadoop.conf.Configuration; +import parquet.hadoop.api.InitContext; import parquet.hadoop.api.ReadSupport; import parquet.io.api.RecordMaterializer; import parquet.schema.MessageType; @@ -40,13 +41,13 @@ public static void setRequestedProjection(Configuration configuration, String re } @Override - public ReadContext init(Configuration configuration, Map keyValueMetaData, MessageType fileSchema) { - String requestedProjectionString = configuration.get(PB_REQUESTED_PROJECTION); + public ReadContext init(InitContext context) { + String requestedProjectionString = context.getConfiguration().get(PB_REQUESTED_PROJECTION); if (requestedProjectionString != null && !requestedProjectionString.trim().isEmpty()) { - MessageType requestedProjection = getSchemaForRead(fileSchema, requestedProjectionString); + MessageType requestedProjection = getSchemaForRead(context.getFileSchema(), requestedProjectionString); return new ReadContext(requestedProjection); } else { - return new ReadContext(fileSchema); + return new ReadContext(context.getFileSchema()); } } From 83f06463cd68732d282c16bffc0a32e21555d066 Mon Sep 17 00:00:00 2001 From: Lukas Date: Thu, 12 Dec 2013 17:05:34 +0100 Subject: [PATCH 14/31] pom.xml version 1.2.10-SNAPSHOT --- pom.xml | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 2ecc656e61..a2332e5f18 100644 --- a/pom.xml +++ b/pom.xml @@ -3,7 +3,7 @@ com.twitter parquet ../pom.xml - 1.2.5-SNAPSHOT + 1.2.10-SNAPSHOT 4.0.0 @@ -35,6 +35,13 @@ elephant-bird-core ${elephant-bird.version} + + + org.apache.hadoop + hadoop-client + ${hadoop.version} + + From 051725362f48c5b6490a15ae5241967a81b523f9 Mon Sep 17 00:00:00 2001 From: Lukas Date: Thu, 12 Dec 2013 17:09:03 +0100 Subject: [PATCH 15/31] TestUtils refactoring --- src/test/java/parquet/proto/ProtoTest.java | 2 +- src/test/java/parquet/proto/TestUtils.java | 14 ++++++++++---- 2 files changed, 11 insertions(+), 5 deletions(-) diff --git a/src/test/java/parquet/proto/ProtoTest.java b/src/test/java/parquet/proto/ProtoTest.java index bafa7b585a..0fa1964f55 100644 --- a/src/test/java/parquet/proto/ProtoTest.java +++ b/src/test/java/parquet/proto/ProtoTest.java @@ -19,7 +19,7 @@ public class ProtoTest { public void testModificationAfterWrite() throws Exception { IntMessage.Builder record = IntMessage.newBuilder().setInt(776); - Path file = tempDirectory(); + Path file = someTemporaryFilePath(); ProtoParquetWriter writer = new ProtoParquetWriter(file, IntMessage.class); diff --git a/src/test/java/parquet/proto/TestUtils.java b/src/test/java/parquet/proto/TestUtils.java index d03f408fc4..874c0a614f 100644 --- a/src/test/java/parquet/proto/TestUtils.java +++ b/src/test/java/parquet/proto/TestUtils.java @@ -15,7 +15,7 @@ public class TestUtils { - public static Path tempDirectory() throws IOException { + public static Path someTemporaryFilePath() throws IOException { File tmp = File.createTempFile("ParquetProtobuf_unitTest", ".tmp"); tmp.deleteOnExit(); tmp.delete(); @@ -34,10 +34,10 @@ public static List writeAndRead(T... records) th return readMessages(file); } - private static Class inferRecordsClass(T[] records) { + private static Class inferRecordsClass(MessageOrBuilder[] records) { Class cls = null; - for (T record : records) { + for (MessageOrBuilder record : records) { Class recordClass; if (record instanceof Message.Builder) { recordClass = ((Message.Builder) record).build().getClass(); @@ -56,6 +56,9 @@ private static Class inferRecord return cls; } + /** + * Writes messages to file, reads messages from file and checks if everything is OK. + * */ public static void testData(MessageOrBuilder... messages) throws IOException { checkSameBuilderInstance(messages); @@ -137,12 +140,15 @@ public static List readMessages(Path file) throw return result; } + /** + * Writes messages to temporary file and returns its path. + * */ public static Path writeMessages(MessageOrBuilder... records) throws IOException { return writeMessages(inferRecordsClass(records), records); } public static Path writeMessages(Class cls, MessageOrBuilder... records) throws IOException { - Path file = tempDirectory(); + Path file = someTemporaryFilePath(); ProtoParquetWriter writer = new ProtoParquetWriter(file, cls); From c590038425156af6a99f5424d8ccecd4526f071f Mon Sep 17 00:00:00 2001 From: Lukas Date: Thu, 12 Dec 2013 17:13:16 +0100 Subject: [PATCH 16/31] Obsolete test removed --- .../java/parquet/proto/BugHuntingTest.java | 54 ------ src/test/java/parquet/proto/ProtoTest.java | 154 ------------------ src/test/java/parquet/proto/TestSandbox.java | 43 ----- src/test/resources/TestProtobuf.proto | 101 +----------- 4 files changed, 1 insertion(+), 351 deletions(-) delete mode 100644 src/test/java/parquet/proto/BugHuntingTest.java delete mode 100644 src/test/java/parquet/proto/ProtoTest.java delete mode 100644 src/test/java/parquet/proto/TestSandbox.java diff --git a/src/test/java/parquet/proto/BugHuntingTest.java b/src/test/java/parquet/proto/BugHuntingTest.java deleted file mode 100644 index 4d3ef34e63..0000000000 --- a/src/test/java/parquet/proto/BugHuntingTest.java +++ /dev/null @@ -1,54 +0,0 @@ -package parquet.proto; - -import org.apache.hadoop.util.IndexedSortable; -import org.junit.Test; -import parquet.protobuf.test.TestProtobuf; - -import java.util.List; - -import static org.junit.Assert.assertEquals; - -public class BugHuntingTest { - - @Test - public void testIndexes() throws Exception { - TestProtobuf.Indexes.Builder indexes = TestProtobuf.Indexes.newBuilder(); - indexes.setField1("field1"); - indexes.setField2("field2"); - indexes.setField4("field4"); - indexes.setField5("field5"); - - List results; - results = TestUtils.writeAndRead((TestProtobuf.IndexesOrBuilder) indexes); - - TestProtobuf.IndexesOrBuilder result = results.get(0); - - assertEquals("field1", result.getField1()); - assertEquals("field2", result.getField2()); - assertEquals("field4", result.getField4()); - assertEquals("field5", result.getField5()); - } - - @Test - public void testIntArray() throws Exception { - TestProtobuf.IntArrayMessage.Builder a = TestProtobuf.IntArrayMessage.newBuilder(); - a.addIntArray(123).addIntArray(23).setSecondField("SecondField"); - - List result = TestUtils.writeAndRead((TestProtobuf.IntArrayMessageOrBuilder) a); - - List intArrayList = result.get(0).getIntArrayList(); - assertEquals(123, (int) intArrayList.get(0)); - assertEquals(23, (int) intArrayList.get(1)); - } - - @Test - public void testMessageArray() throws Exception { - TestProtobuf.MessageArrayMessage.Builder a = TestProtobuf.MessageArrayMessage .newBuilder(); - a.addMsgArrayBuilder().setInnerValue("test").setSecondField("SecondFieldInner"); - a.setSecondField("SecondFieldOuter"); - List result = TestUtils.writeAndRead((TestProtobuf.MessageArrayMessageOrBuilder) a); - - assertEquals("test", result.get(0).getMsgArrayList().get(0).getInnerValue()); - } - } - diff --git a/src/test/java/parquet/proto/ProtoTest.java b/src/test/java/parquet/proto/ProtoTest.java deleted file mode 100644 index 0fa1964f55..0000000000 --- a/src/test/java/parquet/proto/ProtoTest.java +++ /dev/null @@ -1,154 +0,0 @@ -package parquet.proto; - -import com.google.protobuf.MessageOrBuilder; -import org.apache.hadoop.fs.Path; -import org.junit.Test; -import parquet.protobuf.test.TestProtobuf; - -import java.util.List; - -import static junit.framework.Assert.assertEquals; -import static junit.framework.Assert.fail; -import static parquet.proto.TestUtils.*; -import static parquet.protobuf.test.TestProtobuf.*; - -public class ProtoTest { - - - @Test - public void testModificationAfterWrite() throws Exception { - IntMessage.Builder record = IntMessage.newBuilder().setInt(776); - - Path file = someTemporaryFilePath(); - - ProtoParquetWriter writer = - new ProtoParquetWriter(file, IntMessage.class); - - writer.write(record); - record.setInt(666); - writer.close(); - - IntMessageOrBuilder nextRecord = (IntMessageOrBuilder) readMessages(file).get(0); - - assertEquals(776, nextRecord.getInt()); - } - - @Test - public void testWriteIntMessage() throws Exception { - IntMessage record = IntMessage.newBuilder().setInt(776).build(); - - IntMessageOrBuilder nextRecord = writeAndRead(record).get(0); - - assertEquals(776, nextRecord.getInt()); - } - - @Test - public void testWriteEnumMessage() throws Exception { - TestProtobuf.EnumMessage.Builder record = TestProtobuf.EnumMessage.newBuilder().setEnum(TestProtobuf.EnumMessage.Example.EnumValueA); - - TestProtobuf.EnumMessageOrBuilder nextRecord = writeAndRead(record).get(0); - - assertEquals(TestProtobuf.EnumMessage.Example.EnumValueA, nextRecord.getEnum()); - } - - - /** - * Tests that i can write both builder and message. - * */ - @Test - public void testWriteBuilder() throws Exception { - AllInOneMessage.Builder msg1 = AllInOneMessage.newBuilder(); - AllInOneSubMessage.Builder subMsg1 = msg1.getOptionalMsgBuilder(); - - AllInOneMessage.Builder msg2 = AllInOneMessage.newBuilder(); - AllInOneSubMessage.Builder subMsg2 = msg1.getOptionalMsgBuilder(); - - msg1.setRequiredString(" : Pulp fiction"); - subMsg1.setOptionalString("Jan Sverak: Tmavomodry svet"); - - msg2.setRequiredString(" : Snatch"); - subMsg2.setOptionalString(":Das Experiment"); - subMsg2.addRepeatedInt(2005); - - testData(msg1, msg2); - } - - - - - @Test - /** - * If i write two messages, both writes are independent. - * */ - public void testTwoMessages() throws Exception { - IntMessageOrBuilder record1 = IntMessage.newBuilder().setInt(776); - IntMessageOrBuilder record2 = IntMessage.newBuilder().setInt(1223); - - List result = writeAndRead(record1, record2); - - assertEquals(776, result.get(0).getInt()); - assertEquals(1223, result.get(1).getInt()); - assertEquals(2, result.size()); - } - - @Test - public void testMessageArray() throws Exception { - TestProtobuf.RepeatedInnerMessage.Builder record1 = TestProtobuf.RepeatedInnerMessage.newBuilder(); - - record1.addInternalBuilder().setInt(11); - record1.addInternalBuilder().setInt(22); - - - List result = writeAndRead((TestProtobuf.RepeatedInnerMessageOrBuilder) record1); - IntMessageOrBuilder resultInternal1 = result.get(0).getInternalOrBuilder(0); - IntMessageOrBuilder resultInternal2 = result.get(0).getInternalOrBuilder(1); - - assertEquals(11, resultInternal1.getInt()); - assertEquals(22, resultInternal2.getInt()); - } - - - @Test - public void testArray() throws Exception { - fail("Not implemented"); - } - - @Test - public void testInnerMessage() throws Exception { - fail("Not implemented"); - } - - @Test - public void testGroup() throws Exception { - fail("Not implemented"); - } - - @Test - public void testMessage() throws Exception { - - - TestProtobuf.WebPage.Builder record1 = TestProtobuf.WebPage.newBuilder(); - record1.setUrl("http://goout.cz"); - TestProtobuf.InternalMessage.Builder internalMsg = record1.getInternalMsgBuilder(); - internalMsg.setI(7878).setStr("someString"); - - List result = writeAndRead((TestProtobuf.WebPageOrBuilder) record1); - TestProtobuf.InternalMessage resultInternal = result.get(0).getInternalMsg(); - - assertEquals(7878, resultInternal.getI()); - assertEquals("someString", resultInternal.getStr()); - } - - @Test - public void testString() throws Exception { - TestProtobuf.StringMessageOrBuilder record1 = StringMessage.newBuilder().setName("as").setDescription("Des2"); - TestProtobuf.StringMessageOrBuilder record2 = StringMessage.newBuilder().setName("bbas").setDescription("Des1"); - - List x = writeAndRead(record1, record2); - - assertEquals("as", x.get(0).getName()); - assertEquals("Des2", x.get(0).getDescription()); - assertEquals("bbas", x.get(1).getName()); - assertEquals("Des1", x.get(1).getDescription()); - } -} diff --git a/src/test/java/parquet/proto/TestSandbox.java b/src/test/java/parquet/proto/TestSandbox.java deleted file mode 100644 index d6c9ef820c..0000000000 --- a/src/test/java/parquet/proto/TestSandbox.java +++ /dev/null @@ -1,43 +0,0 @@ -package parquet.proto; - -import com.google.protobuf.Descriptors; -import com.google.protobuf.ProtocolMessageEnum; -import org.junit.Test; -import parquet.protobuf.test.TestProtobuf; - -public class TestSandbox { - - @Test - public void testSandbox() throws Exception { - TestProtobuf.EnumMessage.Builder b = TestProtobuf.EnumMessage.newBuilder(); - b.setEnum(TestProtobuf.EnumMessage.Example.EnumValueA); - } - - @Test - public void XXA() throws Exception { - TestProtobuf.EnumMessage.Builder b = TestProtobuf.EnumMessage.newBuilder(); - b.setEnum(TestProtobuf.EnumMessage.Example.EnumValueA); - - - ProtocolMessageEnum[] enumValues = TestProtobuf.EnumMessage.Example.values(); - for (ProtocolMessageEnum value : enumValues) { - int index = value.getNumber(); - String name = value.getValueDescriptor().getName(); - System.out.println(name + " " + index); - } - } - - @Test - public void testName() throws Exception { - TestProtobuf.EnumMessage.Example x = TestProtobuf.EnumMessage.Example.EnumValueB; - Descriptors.EnumValueDescriptor valueDescriptor; - valueDescriptor = x.getValueDescriptor(); - - Descriptors.FieldDescriptor fieldDescriptor = TestProtobuf.EnumMessage.getDescriptor().findFieldByName("enum"); - TestProtobuf.EnumMessage.Builder builder = TestProtobuf.EnumMessage.newBuilder(); - builder.setField(fieldDescriptor, valueDescriptor); - - - System.out.println(builder.getEnum()); - } -} diff --git a/src/test/resources/TestProtobuf.proto b/src/test/resources/TestProtobuf.proto index b13e86415e..5f7b1d5f13 100644 --- a/src/test/resources/TestProtobuf.proto +++ b/src/test/resources/TestProtobuf.proto @@ -24,104 +24,5 @@ message Links { repeated int64 Forward = 2; } - -message AllInOneMessage { - required string requiredString = 1; - optional string optionalString = 2; - repeated string repeatedString = 3; - - optional int32 optionalInt = 5; - repeated int32 repeatedInt = 6; - - optional float optionalFloat = 8; - repeated float repeatedFloat = 9; - - optional AllInOneSubMessage optionalMsg = 11; - repeated AllInOneSubMessage repeatedMsg = 12; -} - -message AllInOneSubMessage { - optional string optionalString = 20; - repeated string repeatedString = 30; - - optional int32 optionalInt = 50; - repeated int32 repeatedInt = 60; - - optional float optionalFloat = 80; - repeated float repeatedFloat = 90; -} - - -// Old: -message Link { - required string url = 1; - optional float weight = 2; - optional bool internal = 3; -} - -message InternalMessage { - optional string str = 1; - optional int32 i = 2; -} - -message WebPage { - required string url = 7; - //repeated Link forwardLinks = 23; - optional InternalMessage internalMsg = 55; -} - -message RecursiveMessage { - optional RecursiveMessage rec = 1; -} - -message IntMessage { - optional int32 int = 155; -} - -message RepeatedInnerMessage { - repeated IntMessage internal = 13; -} - -message EnumMessage { - enum Example { - EnumValueA = 435; - EnumValueB = 21; - EnumValueC = 44; - } - optional Example enum = 1; -} - -message StringMessage { - optional string name = 1; - optional string description = 2; -} - -//Bug hunting -message Indexes{ - optional string field1 = 1; - optional string field2 = 2; - optional string field4 = 4; - optional string field5 = 50; -} - -//Bug hunting -message IntArrayMessage { - repeated int32 intArray= 1; - optional string secondField = 2; -} - -message InnerArrayMessage{ - optional string innerValue = 1; - optional string secondField = 2; -} - -message MessageArrayMessage { - repeated InnerArrayMessage msgArray= 1; - optional string secondField = 2; -} - - - - - +//please place your unit test protobuffer definitions here. From 978e396663297338a186bac466f5ee7319943c6e Mon Sep 17 00:00:00 2001 From: Lukas Date: Fri, 20 Dec 2013 12:28:27 +0100 Subject: [PATCH 17/31] ProtoSchemaConverterUnitTest --- .../parquet/proto/ProtoSchemaConverter.java | 22 +++- .../proto/ProtoSchemaConverterTest.java | 101 ++++++++++++++++++ src/test/resources/TestProtobuf.proto | 41 +++++++ 3 files changed, 160 insertions(+), 4 deletions(-) create mode 100644 src/test/java/parquet/proto/ProtoSchemaConverterTest.java diff --git a/src/main/java/parquet/proto/ProtoSchemaConverter.java b/src/main/java/parquet/proto/ProtoSchemaConverter.java index 61cc47a151..c409bb9185 100644 --- a/src/main/java/parquet/proto/ProtoSchemaConverter.java +++ b/src/main/java/parquet/proto/ProtoSchemaConverter.java @@ -18,14 +18,25 @@ import com.google.protobuf.Descriptors; import com.google.protobuf.Message; import com.twitter.elephantbird.util.Protobufs; -import parquet.schema.*; +import parquet.Log; +import parquet.schema.ConversionPatterns; +import parquet.schema.GroupType; +import parquet.schema.MessageType; +import parquet.schema.OriginalType; +import parquet.schema.PrimitiveType; +import parquet.schema.Type; import java.util.ArrayList; import java.util.List; import static parquet.schema.OriginalType.ENUM; import static parquet.schema.OriginalType.UTF8; -import static parquet.schema.PrimitiveType.PrimitiveTypeName.*; +import static parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY; +import static parquet.schema.PrimitiveType.PrimitiveTypeName.BOOLEAN; +import static parquet.schema.PrimitiveType.PrimitiveTypeName.DOUBLE; +import static parquet.schema.PrimitiveType.PrimitiveTypeName.FLOAT; +import static parquet.schema.PrimitiveType.PrimitiveTypeName.INT32; +import static parquet.schema.PrimitiveType.PrimitiveTypeName.INT64; /** *

@@ -35,12 +46,15 @@ */ public class ProtoSchemaConverter { + private static final Log LOG = Log.getLog(ProtoSchemaConverter.class); + public MessageType convert(Class protobufClass) { + LOG.debug("Converting protobuffer class \"" + protobufClass + "\" to parquet schema."); Descriptors.Descriptor descriptor = Protobufs.getMessageDescriptor(protobufClass); MessageType messageType = new MessageType(descriptor.getFullName(), convertFields(descriptor.getFields())); - System.out.println("Convertor info:\n " + descriptor.toProto() + " was converted to \n" + messageType); + LOG.debug("Convertor info:\n " + descriptor.toProto() + " was converted to \n" + messageType); return messageType; } @@ -103,7 +117,7 @@ private Type convertScalarField(String fieldName, Descriptors.FieldDescriptor de return primitive(fieldName, BINARY, repetition, ENUM); } - throw new UnsupportedOperationException("Cannot convert Protobuffer: type " + javaType + " fieldName " + fieldName); + throw new UnsupportedOperationException("Cannot convert Protobuffer: unknown type " + javaType + " fieldName " + fieldName); } /** diff --git a/src/test/java/parquet/proto/ProtoSchemaConverterTest.java b/src/test/java/parquet/proto/ProtoSchemaConverterTest.java new file mode 100644 index 0000000000..a41138d319 --- /dev/null +++ b/src/test/java/parquet/proto/ProtoSchemaConverterTest.java @@ -0,0 +1,101 @@ +/** + * Copyright 2012 Twitter, Inc. + * + * Licensed 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 KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package parquet.proto; + +import com.google.protobuf.Message; +import org.junit.Before; +import org.junit.Test; +import parquet.protobuf.test.TestProtobuf; +import parquet.schema.MessageType; +import parquet.schema.MessageTypeParser; + +import static org.junit.Assert.assertEquals; + +public class ProtoSchemaConverterTest { + + private void testConversion(Class pbClass, String parquetSchemaString) throws + Exception { + ProtoSchemaConverter protoSchemaConverter = new ProtoSchemaConverter(); + MessageType schema = protoSchemaConverter.convert(pbClass); + MessageType expectedMT = MessageTypeParser.parseMessageType(parquetSchemaString); + assertEquals(expectedMT.toString(), schema.toString()); + } + + @Before + public void setUp() throws Exception { + + } + + + /** + * Tests that all protobuffer datatypes are converted to correct parquet datatypes. + * */ + @Test + public void testConvertAllDatatypes() throws Exception { + String expectedSchema = + "message TestProtobuf.SchemaConverterAllDatatypes {\n" + + " optional double optionalDouble;\n" + + " optional float optionalFloat;\n" + + " optional int32 optionalInt32;\n" + + " optional int64 optionalInt64;\n" + + " optional int32 optionalUInt32;\n" + + " optional int64 optionalUInt64;\n" + + " optional int32 optionalSInt32;\n" + + " optional int64 optionalSInt64;\n" + + " optional int32 optionalFixed32;\n" + + " optional int64 optionalFixed64;\n" + + " optional int32 optionalSFixed32;\n" + + " optional int64 optionalSFixed64;\n" + + " optional boolean optionalBool;\n" + + " optional binary optionalString (UTF8);\n" + + " optional binary optionalBytes;\n" + + " optional group optionalGroup {\n" + + " optional int32 someId;\n" + + " }\n" + + " optional group group {\n" + + " optional int32 groupInt;\n" + + " }\n" + + "}"; + + testConversion(TestProtobuf.SchemaConverterAllDatatypes.class, expectedSchema); + } + + @Test + public void testConvertRepetition() throws Exception { + String expectedSchema = + "message TestProtobuf.SchemaConverterRepetition {\n" + + " optional int32 optionalPrimitive;\n" + + " required int32 requiredPrimitive;\n" + + " optional group repeatedPrimitive (LIST) {\n" + + " repeated int32 repeatedPrimitive_tuple;\n" + + " }\n" + + " optional group optionalMessage {\n" + + " optional int32 someId;\n" + + " }\n" + + " required group requiredMessage {" + + " optional int32 someId;\n" + + " }\n" + + " optional group repeatedMessage (LIST) {" + + " repeated group repeatedMessage_tuple {\n" + + " optional int32 someId;\n" + + " }\n" + + " }\n" + + "}"; + + testConversion(TestProtobuf.SchemaConverterRepetition.class, expectedSchema); + } +} diff --git a/src/test/resources/TestProtobuf.proto b/src/test/resources/TestProtobuf.proto index 5f7b1d5f13..ec34bf9f28 100644 --- a/src/test/resources/TestProtobuf.proto +++ b/src/test/resources/TestProtobuf.proto @@ -24,5 +24,46 @@ message Links { repeated int64 Forward = 2; } + +// begin - protobuffers for ProtoSchemaConverterTest + + message SchemaConverterSimpleMessage { + optional int32 someId = 3; + } + + message SchemaConverterAllDatatypes { + optional double optionalDouble = 1; + optional float optionalFloat = 2; + optional int32 optionalInt32 = 3; + optional int64 optionalInt64 = 4; + optional uint32 optionalUInt32 = 5; + optional uint64 optionalUInt64 = 6; + optional sint32 optionalSInt32 = 7; + optional sint64 optionalSInt64 = 8; + optional fixed32 optionalFixed32 = 9; + optional fixed64 optionalFixed64 = 10; + optional sfixed32 optionalSFixed32 = 11; + optional sfixed64 optionalSFixed64 = 12; + optional bool optionalBool = 13; + optional string optionalString = 14; + optional bytes optionalBytes = 15; + optional SchemaConverterSimpleMessage optionalGroup = 16; + optional group PbGroup = 17 { + optional int32 groupInt = 2; + } + } + + message SchemaConverterRepetition { + optional int32 optionalPrimitive = 1; + required int32 requiredPrimitive = 2; + repeated int32 repeatedPrimitive = 3; + optional SchemaConverterSimpleMessage optionalMessage = 7; + required SchemaConverterSimpleMessage requiredMessage = 8; + repeated SchemaConverterSimpleMessage repeatedMessage = 9; + } + +// end - protobuffers for ProtoSchemaConverterTest + + //please place your unit test protobuffer definitions here. From 861016bc72b36074ac35c7c8b9f5eea7df87f2ad Mon Sep 17 00:00:00 2001 From: Lukas Date: Fri, 20 Dec 2013 21:55:51 +0100 Subject: [PATCH 18/31] Removing hadoop-core dependency conflict --- pom.xml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/pom.xml b/pom.xml index a2332e5f18..6e2a87eb70 100644 --- a/pom.xml +++ b/pom.xml @@ -34,6 +34,12 @@ com.twitter.elephantbird elephant-bird-core ${elephant-bird.version} + + + org.apache.hadoop + hadoop-core + + From dba65be43fd824266f947570ac310775187cef83 Mon Sep 17 00:00:00 2001 From: Lukas Date: Fri, 20 Dec 2013 21:59:42 +0100 Subject: [PATCH 19/31] tests for Input and Output Formats --- .../proto/ProtobufferRecordConverter.java | 12 +- .../proto/ProtoInputOutputFormatTest.java | 173 ++++++++++++++++++ src/test/resources/TestProtobuf.proto | 15 ++ 3 files changed, 196 insertions(+), 4 deletions(-) create mode 100644 src/test/java/parquet/proto/ProtoInputOutputFormatTest.java diff --git a/src/main/java/parquet/proto/ProtobufferRecordConverter.java b/src/main/java/parquet/proto/ProtobufferRecordConverter.java index a952e721bc..c974bb3d59 100644 --- a/src/main/java/parquet/proto/ProtobufferRecordConverter.java +++ b/src/main/java/parquet/proto/ProtobufferRecordConverter.java @@ -1,5 +1,5 @@ /** - * Copyright 2012 Twitter, Inc. + * Copyright 2013 Lukas Nalezenec * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -22,7 +22,9 @@ import parquet.schema.MessageType; /** - * Converts root protobuffer message. + * Converts data content of root message from protobuffer message to parquet message. + * It delegates conversion of inner fields to ProtoMessageConverter class using inheritance. + * Schema is converted in ProtoSchemaConverter class. * * @author Lukas Nalezenec */ @@ -31,7 +33,9 @@ class ProtobufferRecordConverter extends parquet.pro final Message.Builder reusedBuilder; boolean buildBefore; - /** We dont need to write message value at top level. */ + /** + * We dont need to write message value at top level. + * */ private static class SkipParentValueContainer extends ParentValueContainer { @Override public void add(Object a) { @@ -54,7 +58,7 @@ public void start() { @Override public void end() { - // do nothing, dont call ParentValueContainer. + // do nothing, dont call ParentValueContainer at top level. } T getCurrentRecord() { diff --git a/src/test/java/parquet/proto/ProtoInputOutputFormatTest.java b/src/test/java/parquet/proto/ProtoInputOutputFormatTest.java new file mode 100644 index 0000000000..af55bd33c3 --- /dev/null +++ b/src/test/java/parquet/proto/ProtoInputOutputFormatTest.java @@ -0,0 +1,173 @@ +/** + * Copyright 2013 Lukas Nalezenec + * + * Licensed 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 KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package parquet.proto; + +import com.google.protobuf.Message; +import com.google.protobuf.MessageOrBuilder; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.Mapper; +import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; +import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat; +import org.junit.After; +import org.junit.Test; +import parquet.Log; +import parquet.protobuf.test.TestProtobuf; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import static java.lang.Thread.sleep; +import static org.junit.Assert.assertEquals; + + +public class ProtoInputOutputFormatTest { + + + private static final Log LOG = Log.getLog(ProtoInputOutputFormatTest.class); + + private static List inputMessages; + private static List outputMessages; + + + /** + * Writes protobuffer using first MR job, reads written file using + * second job and compares input and output. + * */ + @Test + public void testInputOutput() throws Exception { + TestProtobuf.IOFormatMessage input; + { + TestProtobuf.IOFormatMessage.Builder msg = TestProtobuf.IOFormatMessage.newBuilder(); + msg.setOptionalDouble(666); + msg.addRepeatedString("Msg1"); + msg.addRepeatedString("Msg2"); + msg.getMsgBuilder().setSomeId(323); + input = msg.build(); + } + + List result = runMRJobs(TestProtobuf.IOFormatMessage.class, input); + + assertEquals(1, result.size()); + TestProtobuf.IOFormatMessage output = (TestProtobuf.IOFormatMessage) result.get(0); + + assertEquals(666, output.getOptionalDouble(), 0.00001); + assertEquals(323, output.getMsg().getSomeId()); + assertEquals("Msg1", output.getRepeatedString(0)); + assertEquals("Msg2", output.getRepeatedString(1)); + + assertEquals(input, output); + + } + + public static class WritingMapper extends Mapper { + + public void run(Context context) throws IOException,InterruptedException { + if (inputMessages == null || inputMessages.size() == 0) { + throw new RuntimeException("No mock data given"); + } else { + for (Message msg : inputMessages) { + context.write(null, msg); + LOG.debug("Reading msg from mock writing mapper" + msg); + } + } + } + } + + public static class ReadingMapper extends Mapper { + protected void map(Void key, MessageOrBuilder value, Context context) throws IOException ,InterruptedException { + Message clone = ((Message.Builder) value).build(); + outputMessages.add(clone); + } + } + + /** + * Runs job that writes input to file and then job reading data back. + * */ + public static synchronized List runMRJobs(Class pbClass, Message... messages) throws Exception { + final Configuration conf = new Configuration(); + final Path inputPath = new Path("src/test/java/parquet/proto/ProtoInputOutputFormatTest.java"); + final Path parquetPath = TestUtils.someTemporaryFilePath(); + + inputMessages = new ArrayList(); + + for (Message m: messages) { + inputMessages.add(m); + } + + inputMessages = Collections.unmodifiableList(inputMessages); + + { + final Job job = new Job(conf, "write"); + + // input not really used + TextInputFormat.addInputPath(job, inputPath); + job.setInputFormatClass(TextInputFormat.class); + + job.setMapperClass(WritingMapper.class); + job.setNumReduceTasks(0); + + job.setOutputFormatClass(ProtoParquetOutputFormat.class); + ProtoParquetOutputFormat.setOutputPath(job, parquetPath); + ProtoParquetOutputFormat.setProtobufferClass(job, pbClass); + + waitForJob(job); + } + inputMessages = null; + outputMessages = new ArrayList(); + { + final Job job = new Job(conf, "read"); + job.setInputFormatClass(ProtoParquetInputFormat.class); + ProtoParquetInputFormat.setInputPaths(job, parquetPath); + + job.setMapperClass(ReadingMapper.class); + job.setNumReduceTasks(0); + + job.setOutputFormatClass(NullOutputFormat.class); + + waitForJob(job); + } + + List result = Collections.unmodifiableList(outputMessages); + outputMessages = null; + return result; + } + + + @After + public void tearDown() throws Exception { + inputMessages = null; + outputMessages = null; + } + + + private static void waitForJob(Job job) throws Exception { + job.submit(); + while (!job.isComplete()) { + LOG.debug("waiting for job " + job.getJobName()); + sleep(100); + } + LOG.info("status for job " + job.getJobName() + ": " + (job.isSuccessful() ? "SUCCESS" : "FAILURE")); + if (!job.isSuccessful()) { + throw new RuntimeException("job failed " + job.getJobName()); + } + } +} diff --git a/src/test/resources/TestProtobuf.proto b/src/test/resources/TestProtobuf.proto index ec34bf9f28..7087b6d9ae 100644 --- a/src/test/resources/TestProtobuf.proto +++ b/src/test/resources/TestProtobuf.proto @@ -65,5 +65,20 @@ message Links { // end - protobuffers for ProtoSchemaConverterTest +//begin protobuffers for ProtoInputOutputFormatTest + +message InputOutputMsgFormat { + optional int32 someId = 3; +} + +message IOFormatMessage { + optional double optionalDouble = 1; + repeated string repeatedString = 2; + optional InputOutputMsgFormat msg = 3; + } + +//end protobuffers for ProtoInputOutputFormatTest + + //please place your unit test protobuffer definitions here. From 16b2f7362bfcf64eb23a28933098d32ce19cddaf Mon Sep 17 00:00:00 2001 From: Lukas Date: Fri, 20 Dec 2013 22:10:12 +0100 Subject: [PATCH 20/31] ProtoSchemaConverter Code Style --- .../proto/ProtoSchemaConverterTest.java | 92 +++++++++---------- 1 file changed, 45 insertions(+), 47 deletions(-) diff --git a/src/test/java/parquet/proto/ProtoSchemaConverterTest.java b/src/test/java/parquet/proto/ProtoSchemaConverterTest.java index a41138d319..9ebcb0eaff 100644 --- a/src/test/java/parquet/proto/ProtoSchemaConverterTest.java +++ b/src/test/java/parquet/proto/ProtoSchemaConverterTest.java @@ -1,5 +1,5 @@ /** - * Copyright 2012 Twitter, Inc. + * Copyright 2013 Lukas Nalezenec * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -27,6 +27,9 @@ public class ProtoSchemaConverterTest { + /** + * Converts given pbClass to parquet schema and compares it with expected parquet schema. + * */ private void testConversion(Class pbClass, String parquetSchemaString) throws Exception { ProtoSchemaConverter protoSchemaConverter = new ProtoSchemaConverter(); @@ -35,11 +38,6 @@ private void testConversion(Class pbClass, String parquetSche assertEquals(expectedMT.toString(), schema.toString()); } - @Before - public void setUp() throws Exception { - - } - /** * Tests that all protobuffer datatypes are converted to correct parquet datatypes. @@ -47,29 +45,29 @@ public void setUp() throws Exception { @Test public void testConvertAllDatatypes() throws Exception { String expectedSchema = - "message TestProtobuf.SchemaConverterAllDatatypes {\n" + - " optional double optionalDouble;\n" + - " optional float optionalFloat;\n" + - " optional int32 optionalInt32;\n" + - " optional int64 optionalInt64;\n" + - " optional int32 optionalUInt32;\n" + - " optional int64 optionalUInt64;\n" + - " optional int32 optionalSInt32;\n" + - " optional int64 optionalSInt64;\n" + - " optional int32 optionalFixed32;\n" + - " optional int64 optionalFixed64;\n" + - " optional int32 optionalSFixed32;\n" + - " optional int64 optionalSFixed64;\n" + - " optional boolean optionalBool;\n" + - " optional binary optionalString (UTF8);\n" + - " optional binary optionalBytes;\n" + - " optional group optionalGroup {\n" + - " optional int32 someId;\n" + - " }\n" + - " optional group group {\n" + - " optional int32 groupInt;\n" + - " }\n" + - "}"; + "message TestProtobuf.SchemaConverterAllDatatypes {\n" + + " optional double optionalDouble;\n" + + " optional float optionalFloat;\n" + + " optional int32 optionalInt32;\n" + + " optional int64 optionalInt64;\n" + + " optional int32 optionalUInt32;\n" + + " optional int64 optionalUInt64;\n" + + " optional int32 optionalSInt32;\n" + + " optional int64 optionalSInt64;\n" + + " optional int32 optionalFixed32;\n" + + " optional int64 optionalFixed64;\n" + + " optional int32 optionalSFixed32;\n" + + " optional int64 optionalSFixed64;\n" + + " optional boolean optionalBool;\n" + + " optional binary optionalString (UTF8);\n" + + " optional binary optionalBytes;\n" + + " optional group optionalGroup {\n" + + " optional int32 someId;\n" + + " }\n" + + " optional group group {\n" + + " optional int32 groupInt;\n" + + " }\n" + + "}"; testConversion(TestProtobuf.SchemaConverterAllDatatypes.class, expectedSchema); } @@ -77,24 +75,24 @@ public void testConvertAllDatatypes() throws Exception { @Test public void testConvertRepetition() throws Exception { String expectedSchema = - "message TestProtobuf.SchemaConverterRepetition {\n" + - " optional int32 optionalPrimitive;\n" + - " required int32 requiredPrimitive;\n" + - " optional group repeatedPrimitive (LIST) {\n" + - " repeated int32 repeatedPrimitive_tuple;\n" + - " }\n" + - " optional group optionalMessage {\n" + - " optional int32 someId;\n" + - " }\n" + - " required group requiredMessage {" + - " optional int32 someId;\n" + - " }\n" + - " optional group repeatedMessage (LIST) {" + - " repeated group repeatedMessage_tuple {\n" + - " optional int32 someId;\n" + - " }\n" + - " }\n" + - "}"; + "message TestProtobuf.SchemaConverterRepetition {\n" + + " optional int32 optionalPrimitive;\n" + + " required int32 requiredPrimitive;\n" + + " optional group repeatedPrimitive (LIST) {\n" + + " repeated int32 repeatedPrimitive_tuple;\n" + + " }\n" + + " optional group optionalMessage {\n" + + " optional int32 someId;\n" + + " }\n" + + " required group requiredMessage {" + + " optional int32 someId;\n" + + " }\n" + + " optional group repeatedMessage (LIST) {" + + " repeated group repeatedMessage_tuple {\n" + + " optional int32 someId;\n" + + " }\n" + + " }\n" + + "}"; testConversion(TestProtobuf.SchemaConverterRepetition.class, expectedSchema); } From 13942364d47d493fe10c66c17644d8284a84cbc7 Mon Sep 17 00:00:00 2001 From: Lukas Date: Fri, 20 Dec 2013 22:41:52 +0100 Subject: [PATCH 21/31] CodeStyle --- .../proto/ProtoParquetOutputFormat.java | 4 +- .../java/parquet/proto/ProtoWriteSupport.java | 4 +- .../proto/ProtobufferRecordConverter.java | 2 +- .../proto/ProtoInputOutputFormatTest.java | 26 +++--- .../proto/ProtoSchemaConverterTest.java | 87 +++++++++---------- src/test/java/parquet/proto/TestUtils.java | 23 +++-- 6 files changed, 76 insertions(+), 70 deletions(-) diff --git a/src/main/java/parquet/proto/ProtoParquetOutputFormat.java b/src/main/java/parquet/proto/ProtoParquetOutputFormat.java index e2757f8beb..4c401d3222 100644 --- a/src/main/java/parquet/proto/ProtoParquetOutputFormat.java +++ b/src/main/java/parquet/proto/ProtoParquetOutputFormat.java @@ -23,9 +23,9 @@ /** * A Hadoop {@link org.apache.hadoop.mapreduce.OutputFormat} for Protobuffer Parquet files. - * + *

* Usage: - * + *

*

  * {@code
  * final Job job = new Job(conf, "Parquet writing job");
diff --git a/src/main/java/parquet/proto/ProtoWriteSupport.java b/src/main/java/parquet/proto/ProtoWriteSupport.java
index 605335dbb2..e78c1da479 100644
--- a/src/main/java/parquet/proto/ProtoWriteSupport.java
+++ b/src/main/java/parquet/proto/ProtoWriteSupport.java
@@ -138,7 +138,7 @@ private void writeRecordFields(GroupType parquetSchema, T record) {
 
   private  void writeArray(GroupType schema, Descriptors.FieldDescriptor fieldDescriptor,
                               List array) {
-    if (!schema.getName().equals(fieldDescriptor.getName())) throw  new RuntimeException("Mismatch");//TODO remove me
+    if (!schema.getName().equals(fieldDescriptor.getName())) throw new RuntimeException("Mismatch");//TODO remove me
 
     recordConsumer.startGroup();
     if (array.iterator().hasNext()) {
@@ -172,7 +172,7 @@ private void writeScalarValue(Type type, Descriptors.FieldDescriptor fieldDescri
     } else if (javaType.equals(Descriptors.FieldDescriptor.JavaType.STRING)) {
       recordConsumer.addBinary(stringToBinary(value));
     } else if (javaType.equals(Descriptors.FieldDescriptor.JavaType.MESSAGE)) {
-      writeMessage(type.asGroupType(), (T) value);// patched
+      writeMessage(type.asGroupType(), (T) value);
     } else if (javaType.equals(Descriptors.FieldDescriptor.JavaType.ENUM)) {
       Descriptors.EnumValueDescriptor enumDescriptor = (Descriptors.EnumValueDescriptor) value;
       recordConsumer.addBinary(Binary.fromString(enumDescriptor.getName()));
diff --git a/src/main/java/parquet/proto/ProtobufferRecordConverter.java b/src/main/java/parquet/proto/ProtobufferRecordConverter.java
index c974bb3d59..f9a7a3e192 100644
--- a/src/main/java/parquet/proto/ProtobufferRecordConverter.java
+++ b/src/main/java/parquet/proto/ProtobufferRecordConverter.java
@@ -35,7 +35,7 @@ class ProtobufferRecordConverter extends parquet.pro
 
   /**
    * We dont need to write message value at top level.
-   * */
+   */
   private static class SkipParentValueContainer extends ParentValueContainer {
     @Override
     public void add(Object a) {
diff --git a/src/test/java/parquet/proto/ProtoInputOutputFormatTest.java b/src/test/java/parquet/proto/ProtoInputOutputFormatTest.java
index af55bd33c3..7faea37e15 100644
--- a/src/test/java/parquet/proto/ProtoInputOutputFormatTest.java
+++ b/src/test/java/parquet/proto/ProtoInputOutputFormatTest.java
@@ -51,18 +51,18 @@ public class ProtoInputOutputFormatTest {
   /**
    * Writes protobuffer using first MR job, reads written file using
    * second job and compares input and output.
-   * */
+   */
   @Test
   public void testInputOutput() throws Exception {
     TestProtobuf.IOFormatMessage input;
-      {
-        TestProtobuf.IOFormatMessage.Builder msg = TestProtobuf.IOFormatMessage.newBuilder();
-        msg.setOptionalDouble(666);
-        msg.addRepeatedString("Msg1");
-        msg.addRepeatedString("Msg2");
-        msg.getMsgBuilder().setSomeId(323);
-        input = msg.build();
-      }
+    {
+      TestProtobuf.IOFormatMessage.Builder msg = TestProtobuf.IOFormatMessage.newBuilder();
+      msg.setOptionalDouble(666);
+      msg.addRepeatedString("Msg1");
+      msg.addRepeatedString("Msg2");
+      msg.getMsgBuilder().setSomeId(323);
+      input = msg.build();
+    }
 
     List result = runMRJobs(TestProtobuf.IOFormatMessage.class, input);
 
@@ -80,7 +80,7 @@ public void testInputOutput() throws Exception {
 
   public static class WritingMapper extends Mapper {
 
-    public void run(Context context) throws IOException,InterruptedException {
+    public void run(Context context) throws IOException, InterruptedException {
       if (inputMessages == null || inputMessages.size() == 0) {
         throw new RuntimeException("No mock data given");
       } else {
@@ -93,7 +93,7 @@ public void run(Context context) throws IOException,InterruptedException {
   }
 
   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) throws IOException, InterruptedException {
       Message clone = ((Message.Builder) value).build();
       outputMessages.add(clone);
     }
@@ -101,7 +101,7 @@ protected void map(Void key, MessageOrBuilder value, Context context) throws IOE
 
   /**
    * Runs job that writes input to file and then job reading data back.
-   * */
+   */
   public static synchronized List runMRJobs(Class pbClass, Message... messages) throws Exception {
     final Configuration conf = new Configuration();
     final Path inputPath = new Path("src/test/java/parquet/proto/ProtoInputOutputFormatTest.java");
@@ -109,7 +109,7 @@ public static synchronized List runMRJobs(Class pbCl
 
     inputMessages = new ArrayList();
 
-    for (Message m: messages) {
+    for (Message m : messages) {
       inputMessages.add(m);
     }
 
diff --git a/src/test/java/parquet/proto/ProtoSchemaConverterTest.java b/src/test/java/parquet/proto/ProtoSchemaConverterTest.java
index 9ebcb0eaff..24190ed14d 100644
--- a/src/test/java/parquet/proto/ProtoSchemaConverterTest.java
+++ b/src/test/java/parquet/proto/ProtoSchemaConverterTest.java
@@ -17,7 +17,6 @@
 package parquet.proto;
 
 import com.google.protobuf.Message;
-import org.junit.Before;
 import org.junit.Test;
 import parquet.protobuf.test.TestProtobuf;
 import parquet.schema.MessageType;
@@ -29,7 +28,7 @@ public class ProtoSchemaConverterTest {
 
   /**
    * Converts given pbClass to parquet schema and compares it with expected parquet schema.
-   * */
+   */
   private void testConversion(Class pbClass, String parquetSchemaString) throws
           Exception {
     ProtoSchemaConverter protoSchemaConverter = new ProtoSchemaConverter();
@@ -41,33 +40,33 @@ private void testConversion(Class pbClass, String parquetSche
 
   /**
    * Tests that all protobuffer datatypes are converted to correct parquet datatypes.
-   * */
+   */
   @Test
   public void testConvertAllDatatypes() throws Exception {
     String expectedSchema =
-    "message TestProtobuf.SchemaConverterAllDatatypes {\n" +
-    "  optional double optionalDouble;\n" +
-    "  optional float optionalFloat;\n" +
-    "  optional int32 optionalInt32;\n" +
-    "  optional int64 optionalInt64;\n" +
-    "  optional int32 optionalUInt32;\n" +
-    "  optional int64 optionalUInt64;\n" +
-    "  optional int32 optionalSInt32;\n" +
-    "  optional int64 optionalSInt64;\n" +
-    "  optional int32 optionalFixed32;\n" +
-    "  optional int64 optionalFixed64;\n" +
-    "  optional int32 optionalSFixed32;\n" +
-    "  optional int64 optionalSFixed64;\n" +
-    "  optional boolean optionalBool;\n" +
-    "  optional binary optionalString (UTF8);\n" +
-    "  optional binary optionalBytes;\n" +
-    "  optional group optionalGroup {\n" +
-    "    optional int32 someId;\n" +
-    "  }\n" +
-    "  optional group group {\n" +
-    "    optional int32 groupInt;\n" +
-    "  }\n" +
-    "}";
+      "message TestProtobuf.SchemaConverterAllDatatypes {\n" +
+      "  optional double optionalDouble;\n" +
+      "  optional float optionalFloat;\n" +
+      "  optional int32 optionalInt32;\n" +
+      "  optional int64 optionalInt64;\n" +
+      "  optional int32 optionalUInt32;\n" +
+      "  optional int64 optionalUInt64;\n" +
+      "  optional int32 optionalSInt32;\n" +
+      "  optional int64 optionalSInt64;\n" +
+      "  optional int32 optionalFixed32;\n" +
+      "  optional int64 optionalFixed64;\n" +
+      "  optional int32 optionalSFixed32;\n" +
+      "  optional int64 optionalSFixed64;\n" +
+      "  optional boolean optionalBool;\n" +
+      "  optional binary optionalString (UTF8);\n" +
+      "  optional binary optionalBytes;\n" +
+      "  optional group optionalGroup {\n" +
+      "    optional int32 someId;\n" +
+      "  }\n" +
+      "  optional group group {\n" +
+      "    optional int32 groupInt;\n" +
+      "  }\n" +
+      "}";
 
     testConversion(TestProtobuf.SchemaConverterAllDatatypes.class, expectedSchema);
   }
@@ -75,24 +74,24 @@ public void testConvertAllDatatypes() throws Exception {
   @Test
   public void testConvertRepetition() throws Exception {
     String expectedSchema =
-    "message TestProtobuf.SchemaConverterRepetition {\n" +
-      "  optional int32 optionalPrimitive;\n" +
-      "  required int32 requiredPrimitive;\n" +
-      "  optional group repeatedPrimitive (LIST) {\n" +
-      "    repeated int32 repeatedPrimitive_tuple;\n" +
-      "  }\n" +
-      "  optional group optionalMessage {\n" +
-      "    optional int32 someId;\n" +
-      "  }\n" +
-      "  required group requiredMessage {" +
-      "    optional int32 someId;\n" +
-      "  }\n" +
-      "  optional group repeatedMessage (LIST) {" +
-      "    repeated group repeatedMessage_tuple {\n" +
-      "      optional int32 someId;\n" +
-      "    }\n" +
-      "  }\n" +
-      "}";
+      "message TestProtobuf.SchemaConverterRepetition {\n" +
+        "  optional int32 optionalPrimitive;\n" +
+        "  required int32 requiredPrimitive;\n" +
+        "  optional group repeatedPrimitive (LIST) {\n" +
+        "    repeated int32 repeatedPrimitive_tuple;\n" +
+        "  }\n" +
+        "  optional group optionalMessage {\n" +
+        "    optional int32 someId;\n" +
+        "  }\n" +
+        "  required group requiredMessage {" +
+        "    optional int32 someId;\n" +
+        "  }\n" +
+        "  optional group repeatedMessage (LIST) {" +
+        "    repeated group repeatedMessage_tuple {\n" +
+        "      optional int32 someId;\n" +
+        "    }\n" +
+        "  }\n" +
+        "}";
 
     testConversion(TestProtobuf.SchemaConverterRepetition.class, expectedSchema);
   }
diff --git a/src/test/java/parquet/proto/TestUtils.java b/src/test/java/parquet/proto/TestUtils.java
index 874c0a614f..086bb75aed 100644
--- a/src/test/java/parquet/proto/TestUtils.java
+++ b/src/test/java/parquet/proto/TestUtils.java
@@ -58,7 +58,7 @@ private static Class inferRecordsClass(MessageOrBuilder[] rec
 
   /**
    * Writes messages to file, reads messages from file and checks if everything is OK.
-   * */
+   */
   public static void testData(MessageOrBuilder... messages) throws IOException {
 
     checkSameBuilderInstance(messages);
@@ -71,9 +71,9 @@ public static void testData(MessageOrBuilder... messages) throws IOException {
   }
 
   private static List cloneList(MessageOrBuilder[] messages) {
-    List  result =  new ArrayList();
+    List result = new ArrayList();
 
-    for (MessageOrBuilder mob: messages) {
+    for (MessageOrBuilder mob : messages) {
       result.add(asMessage(mob));
     }
 
@@ -88,10 +88,13 @@ public static List asMessages(List mobs) {
 
     return result;
   }
-  /** Given message or builder returns same data as messagee*/
+
+  /**
+   * Given message or builder returns same data as message
+   */
   public static Message asMessage(MessageOrBuilder mob) {
     Message message;
-    if (mob instanceof  Message.Builder) {
+    if (mob instanceof Message.Builder) {
       message = ((Message.Builder) mob).build();
     } else {
       message = (Message) mob;
@@ -99,7 +102,9 @@ public static Message asMessage(MessageOrBuilder mob) {
     return message;
   }
 
-  /** Fails if some instance of builder is two times in list.*/
+  /**
+   * Fails if some instance of builder is two times in list.
+   */
   private static void checkSameBuilderInstance(MessageOrBuilder[] messages) {
     for (int i = 0; i < messages.length; i++) {
       MessageOrBuilder firstMessage = messages[i];
@@ -120,7 +125,9 @@ private static void checkSameBuilderInstance(MessageOrBuilder[] messages) {
     }
   }
 
-  /** Reads messages from given file. The file could/should be created by method writeMessages */
+  /**
+   * Reads messages from given file. The file could/should be created by method writeMessages
+   */
   public static  List readMessages(Path file) throws IOException {
     ProtoParquetReader reader = new ProtoParquetReader(file);
 
@@ -142,7 +149,7 @@ public static  List readMessages(Path file) throw
 
   /**
    * Writes messages to temporary file and returns its path.
-   * */
+   */
   public static Path writeMessages(MessageOrBuilder... records) throws IOException {
     return writeMessages(inferRecordsClass(records), records);
   }

From 1f75813a03305c30da1d2f9326affc92e541443a Mon Sep 17 00:00:00 2001
From: Lukas 
Date: Sat, 21 Dec 2013 00:05:07 +0100
Subject: [PATCH 22/31] junit test for enum schema conversion

---
 .../java/parquet/proto/converters/ProtoEnumConverter.java  | 7 ++++++-
 src/test/java/parquet/proto/ProtoSchemaConverterTest.java  | 3 ++-
 src/test/resources/TestProtobuf.proto                      | 5 +++++
 3 files changed, 13 insertions(+), 2 deletions(-)

diff --git a/src/main/java/parquet/proto/converters/ProtoEnumConverter.java b/src/main/java/parquet/proto/converters/ProtoEnumConverter.java
index 0403429ded..671acd1cf1 100644
--- a/src/main/java/parquet/proto/converters/ProtoEnumConverter.java
+++ b/src/main/java/parquet/proto/converters/ProtoEnumConverter.java
@@ -23,6 +23,7 @@
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 public final class ProtoEnumConverter extends PrimitiveConverter {
 
@@ -55,7 +56,11 @@ final public void addBinary(Binary value) {
     Descriptors.EnumValueDescriptor protoValue = enumLookup.get(value);
 
     if (protoValue == null) {
-      throw new RuntimeException("Illegal enum value " + value + " in protoBuffer " + fieldType.getFullName());
+      Set knownValues = enumLookup.keySet();
+      String msg = "Illegal enum value \"" + value + "\""
+         + " in protoBuffer \"" + fieldType.getFullName() + "\""
+         + " legal values are: \"" + knownValues + "\"";
+      throw new RuntimeException(msg);
     }
 
     parent.add(protoValue);
diff --git a/src/test/java/parquet/proto/ProtoSchemaConverterTest.java b/src/test/java/parquet/proto/ProtoSchemaConverterTest.java
index 24190ed14d..0c3dc8b5b7 100644
--- a/src/test/java/parquet/proto/ProtoSchemaConverterTest.java
+++ b/src/test/java/parquet/proto/ProtoSchemaConverterTest.java
@@ -63,9 +63,10 @@ public void testConvertAllDatatypes() throws Exception {
       "  optional group optionalGroup {\n" +
       "    optional int32 someId;\n" +
       "  }\n" +
-      "  optional group group {\n" +
+      "  optional group pbgroup {\n" +
       "    optional int32 groupInt;\n" +
       "  }\n" +
+      " optional binary optionalEnum (ENUM);" +
       "}";
 
     testConversion(TestProtobuf.SchemaConverterAllDatatypes.class, expectedSchema);
diff --git a/src/test/resources/TestProtobuf.proto b/src/test/resources/TestProtobuf.proto
index 7087b6d9ae..989c8b2394 100644
--- a/src/test/resources/TestProtobuf.proto
+++ b/src/test/resources/TestProtobuf.proto
@@ -51,6 +51,11 @@ message Links {
      optional group PbGroup  = 17 {
        optional int32 groupInt = 2;
      }
+    enum TestEnum {
+        FIRST = 0;
+        SECOND = 1;
+    }
+    optional TestEnum optionalEnum = 18;
  }
 
  message SchemaConverterRepetition {

From 51ca71a63e332e8976649f0631664de808529a25 Mon Sep 17 00:00:00 2001
From: Lukas 
Date: Sat, 21 Dec 2013 00:12:22 +0100
Subject: [PATCH 23/31] remove old package info

---
 src/main/java/parquet/proto/package-info.java | 89 -------------------
 1 file changed, 89 deletions(-)
 delete mode 100644 src/main/java/parquet/proto/package-info.java

diff --git a/src/main/java/parquet/proto/package-info.java b/src/main/java/parquet/proto/package-info.java
deleted file mode 100644
index 60e02d55c5..0000000000
--- a/src/main/java/parquet/proto/package-info.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/**
- * Copyright 2012 Twitter, Inc.
- *
- * Licensed 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 KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-/**
- *
- * 

- * Provides classes to store Avro data in Parquet files. Avro schemas are converted to - * parquet schemas as follows. Only record schemas are converted, - * other top-level schema types are not converted and attempting to do so will result - * in an error. Avro types are converted to Parquet types using the mapping shown here: - *

- * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - *
Avro typeParquet type
nullno type (the field is not encoded in Parquet), unless a null union
booleanboolean
intint32
longint64
floatfloat
doubledouble
bytesbinary
stringbinary (with original type UTF8)
recordgroup containing nested fields
enumbinary (with original type ENUM)
arraygroup (with original type LIST) containing one repeated group field
mapgroup (with original type MAP) containing one repeated group - * field (with original type MAP_KEY_VALUE) of (key, value)
fixedfixed_len_byte_array
unionan optional type, in the case of a null union, otherwise not supported
- */ -package parquet.proto; From 52ffcfe6eea65f54b4d06ee5a6680497f4905b2a Mon Sep 17 00:00:00 2001 From: Lukas Date: Sat, 21 Dec 2013 00:14:36 +0100 Subject: [PATCH 24/31] remove commented code --- .../java/parquet/proto/converters/ProtoMessageConverter.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/main/java/parquet/proto/converters/ProtoMessageConverter.java b/src/main/java/parquet/proto/converters/ProtoMessageConverter.java index 490919d8e3..3c100bec26 100644 --- a/src/main/java/parquet/proto/converters/ProtoMessageConverter.java +++ b/src/main/java/parquet/proto/converters/ProtoMessageConverter.java @@ -146,11 +146,10 @@ private static Converter newScalarConverter(ParentValueContainer pvc, Message.Bu throw new RuntimeException(fieldDescriptor.getFullName() + " is not inside " + parentBuilder.getDescriptorForType().getFullName()); } - GroupType parquetSubType = parquetType.asGroupType();//.getType(fieldDescriptor.getName()).asGroupType(); + GroupType parquetSubType = parquetType.asGroupType(); Message.Builder subBuilder; if (fieldDescriptor.isRepeated()) { subBuilder = parentBuilder.newBuilderForField(fieldDescriptor); - //subBuilder = parentBuilder. getFieldBuilder(fieldDescriptor); } else { subBuilder = parentBuilder.newBuilderForField(fieldDescriptor); } From f7a90232cab0da9121c327ad0b43c2d39811fb53 Mon Sep 17 00:00:00 2001 From: Lukas Date: Thu, 26 Dec 2013 14:15:12 +0100 Subject: [PATCH 25/31] correct byte[] storage --- .../java/parquet/proto/ProtoWriteSupport.java | 16 ++++++++++------ .../proto/converters/ProtoBinaryConverter.java | 6 ++++-- 2 files changed, 14 insertions(+), 8 deletions(-) diff --git a/src/main/java/parquet/proto/ProtoWriteSupport.java b/src/main/java/parquet/proto/ProtoWriteSupport.java index e78c1da479..dd283db622 100644 --- a/src/main/java/parquet/proto/ProtoWriteSupport.java +++ b/src/main/java/parquet/proto/ProtoWriteSupport.java @@ -15,6 +15,7 @@ */ package parquet.proto; +import com.google.protobuf.ByteString; import com.google.protobuf.DescriptorProtos; import com.google.protobuf.Descriptors; import com.google.protobuf.Message; @@ -28,7 +29,6 @@ import parquet.schema.MessageType; import parquet.schema.Type; -import java.nio.ByteBuffer; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -97,13 +97,13 @@ public void write(T record) { recordConsumer.endMessage(); } - private void writeMessage(GroupType schema, T message) { + private void writeMessage(GroupType schema, MessageOrBuilder message) { recordConsumer.startGroup(); writeRecordFields(schema, message); recordConsumer.endGroup(); } - private void writeRecordFields(GroupType parquetSchema, T record) { + private void writeRecordFields(GroupType parquetSchema, MessageOrBuilder record) { List fields = parquetSchema.getFields(); Map pbFields = record.getAllFields(); @@ -168,11 +168,15 @@ private void writeScalarValue(Type type, Descriptors.FieldDescriptor fieldDescri } else if (javaType.equals(Descriptors.FieldDescriptor.JavaType.DOUBLE)) { recordConsumer.addDouble(((Number) value).doubleValue()); } else if (javaType.equals(Descriptors.FieldDescriptor.JavaType.BYTE_STRING)) { - recordConsumer.addBinary(Binary.fromByteBuffer((ByteBuffer) value)); + ByteString byteString = (ByteString) value; + Binary binary = Binary.fromByteArray(byteString.toByteArray()); + recordConsumer.addBinary(binary); } else if (javaType.equals(Descriptors.FieldDescriptor.JavaType.STRING)) { - recordConsumer.addBinary(stringToBinary(value)); + Binary binary = stringToBinary(value); + recordConsumer.addBinary(binary); } else if (javaType.equals(Descriptors.FieldDescriptor.JavaType.MESSAGE)) { - writeMessage(type.asGroupType(), (T) value); + MessageOrBuilder msg = (MessageOrBuilder) value; + writeMessage(type.asGroupType(), msg); } else if (javaType.equals(Descriptors.FieldDescriptor.JavaType.ENUM)) { Descriptors.EnumValueDescriptor enumDescriptor = (Descriptors.EnumValueDescriptor) value; recordConsumer.addBinary(Binary.fromString(enumDescriptor.getName())); diff --git a/src/main/java/parquet/proto/converters/ProtoBinaryConverter.java b/src/main/java/parquet/proto/converters/ProtoBinaryConverter.java index 3149c412be..eb2a95c678 100644 --- a/src/main/java/parquet/proto/converters/ProtoBinaryConverter.java +++ b/src/main/java/parquet/proto/converters/ProtoBinaryConverter.java @@ -16,6 +16,7 @@ package parquet.proto.converters; +import com.google.protobuf.ByteString; import parquet.io.api.Binary; import parquet.io.api.PrimitiveConverter; @@ -28,7 +29,8 @@ public ProtoBinaryConverter(ParentValueContainer parent) { } @Override - public void addBinary(Binary value) { - parent.add(value); + public void addBinary(Binary binary) { + ByteString byteString = ByteString.copyFrom(binary.toByteBuffer()); + parent.add(byteString); } } From 5997bf5ce0f588c7ea8ad0e6786d7dc3105908b5 Mon Sep 17 00:00:00 2001 From: Lukas Date: Thu, 26 Dec 2013 16:18:22 +0100 Subject: [PATCH 26/31] #projection test --- .../proto/ProtoParquetInputFormat.java | 9 +- .../proto/ProtoInputOutputFormatTest.java | 125 ++++-------------- .../java/parquet/proto/utils/ReadUsingMR.java | 74 +++++++++++ .../parquet/proto/utils/WriteUsingMR.java | 104 +++++++++++++++ 4 files changed, 209 insertions(+), 103 deletions(-) create mode 100644 src/test/java/parquet/proto/utils/ReadUsingMR.java create mode 100644 src/test/java/parquet/proto/utils/WriteUsingMR.java diff --git a/src/main/java/parquet/proto/ProtoParquetInputFormat.java b/src/main/java/parquet/proto/ProtoParquetInputFormat.java index 4415e00eb1..497b9d1e18 100644 --- a/src/main/java/parquet/proto/ProtoParquetInputFormat.java +++ b/src/main/java/parquet/proto/ProtoParquetInputFormat.java @@ -1,5 +1,5 @@ /** - * Copyright 2012 Twitter, Inc. + * Copyright 2013 Lukas Nalezenec * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -16,9 +16,8 @@ package parquet.proto; import com.google.protobuf.MessageOrBuilder; -import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.conf.Configuration; import parquet.hadoop.ParquetInputFormat; -import parquet.hadoop.util.ContextUtil; /** * A Hadoop {@link org.apache.hadoop.mapreduce.InputFormat} for Parquet files. @@ -28,8 +27,8 @@ public ProtoParquetInputFormat() { super(ProtoReadSupport.class); } - public static void setRequestedProjection(Job job, String requestedProjection) { - ProtoReadSupport.setRequestedProjection(ContextUtil.getConfiguration(job), requestedProjection); + public static void setRequestedProjection(Configuration configuration, String requestedProjection) { + ProtoReadSupport.setRequestedProjection(configuration, requestedProjection); } } diff --git a/src/test/java/parquet/proto/ProtoInputOutputFormatTest.java b/src/test/java/parquet/proto/ProtoInputOutputFormatTest.java index 7faea37e15..3b276fd857 100644 --- a/src/test/java/parquet/proto/ProtoInputOutputFormatTest.java +++ b/src/test/java/parquet/proto/ProtoInputOutputFormatTest.java @@ -16,38 +16,24 @@ package parquet.proto; import com.google.protobuf.Message; -import com.google.protobuf.MessageOrBuilder; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.io.Text; -import org.apache.hadoop.mapreduce.Job; -import org.apache.hadoop.mapreduce.Mapper; -import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; -import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat; -import org.junit.After; import org.junit.Test; import parquet.Log; +import parquet.proto.utils.ReadUsingMR; +import parquet.proto.utils.WriteUsingMR; import parquet.protobuf.test.TestProtobuf; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; import java.util.List; -import static java.lang.Thread.sleep; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; public class ProtoInputOutputFormatTest { - private static final Log LOG = Log.getLog(ProtoInputOutputFormatTest.class); - private static List inputMessages; - private static List outputMessages; - - /** * Writes protobuffer using first MR job, reads written file using * second job and compares input and output. @@ -78,96 +64,39 @@ public void testInputOutput() throws Exception { } - public static class WritingMapper extends Mapper { - - public void run(Context context) throws IOException, InterruptedException { - if (inputMessages == null || inputMessages.size() == 0) { - throw new RuntimeException("No mock data given"); - } else { - for (Message msg : inputMessages) { - context.write(null, msg); - LOG.debug("Reading msg from mock writing mapper" + msg); - } - } - } - } - public static class ReadingMapper extends Mapper { - protected void map(Void key, MessageOrBuilder value, Context context) throws IOException, InterruptedException { - Message clone = ((Message.Builder) value).build(); - outputMessages.add(clone); - } - } - - /** - * Runs job that writes input to file and then job reading data back. - */ - public static synchronized List runMRJobs(Class pbClass, Message... messages) throws Exception { - final Configuration conf = new Configuration(); - final Path inputPath = new Path("src/test/java/parquet/proto/ProtoInputOutputFormatTest.java"); - final Path parquetPath = TestUtils.someTemporaryFilePath(); - - inputMessages = new ArrayList(); - - for (Message m : messages) { - inputMessages.add(m); - } - - inputMessages = Collections.unmodifiableList(inputMessages); - - { - final Job job = new Job(conf, "write"); - - // input not really used - TextInputFormat.addInputPath(job, inputPath); - job.setInputFormatClass(TextInputFormat.class); - - job.setMapperClass(WritingMapper.class); - job.setNumReduceTasks(0); - - job.setOutputFormatClass(ProtoParquetOutputFormat.class); - ProtoParquetOutputFormat.setOutputPath(job, parquetPath); - ProtoParquetOutputFormat.setProtobufferClass(job, pbClass); + @Test + public void testProjection() throws Exception { - waitForJob(job); - } - inputMessages = null; - outputMessages = new ArrayList(); - { - final Job job = new Job(conf, "read"); - job.setInputFormatClass(ProtoParquetInputFormat.class); - ProtoParquetInputFormat.setInputPaths(job, parquetPath); + TestProtobuf.Document.Builder writtenDocument = TestProtobuf.Document.newBuilder(); + writtenDocument.setDocId(12345); + writtenDocument.addNameBuilder().setUrl("http://goout.cz/"); - job.setMapperClass(ReadingMapper.class); - job.setNumReduceTasks(0); + Path outputPath = new WriteUsingMR().write(TestProtobuf.Document.class, writtenDocument.build()); - job.setOutputFormatClass(NullOutputFormat.class); + //lets prepare reading with schema + ReadUsingMR reader = new ReadUsingMR(); - waitForJob(job); - } + Configuration conf = new Configuration(); + reader.setConfiguration(conf); + String projection = "message Document {required int64 DocId; }"; + ProtoParquetInputFormat.setRequestedProjection(conf, projection); - List result = Collections.unmodifiableList(outputMessages); - outputMessages = null; - return result; - } + List output = reader.read(outputPath); + TestProtobuf.Document readDocument = (TestProtobuf.Document) output.get(0); - @After - public void tearDown() throws Exception { - inputMessages = null; - outputMessages = null; + //test that only requested fields were deserialized + assertTrue(readDocument.hasDocId()); + assertTrue("Found data outside projection.", readDocument.getNameCount() == 0); } - - private static void waitForJob(Job job) throws Exception { - job.submit(); - while (!job.isComplete()) { - LOG.debug("waiting for job " + job.getJobName()); - sleep(100); - } - LOG.info("status for job " + job.getJobName() + ": " + (job.isSuccessful() ? "SUCCESS" : "FAILURE")); - if (!job.isSuccessful()) { - throw new RuntimeException("job failed " + job.getJobName()); - } + /** + * Runs job that writes input to file and then job reading data back. + */ + public static List runMRJobs(Class pbClass, Message... messages) throws Exception { + Path outputPath = new WriteUsingMR().write(pbClass, messages); + List result = new ReadUsingMR().read(outputPath); + return result; } } diff --git a/src/test/java/parquet/proto/utils/ReadUsingMR.java b/src/test/java/parquet/proto/utils/ReadUsingMR.java new file mode 100644 index 0000000000..78a37e9ba5 --- /dev/null +++ b/src/test/java/parquet/proto/utils/ReadUsingMR.java @@ -0,0 +1,74 @@ +/** + * Copyright 2013 Lukas Nalezenec + * + * Licensed 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 KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package parquet.proto.utils; + +import com.google.protobuf.Message; +import com.google.protobuf.MessageOrBuilder; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.Mapper; +import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat; +import parquet.proto.ProtoParquetInputFormat; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + + +/** + * Reads data from given parquet file using MapReduce job. + */ +public class ReadUsingMR { + + Configuration conf; + + public void setConfiguration(Configuration conf) { + this.conf = conf; + } + + public List read(Path parquetPath) throws Exception { + + final List outputMessages = new ArrayList(); + + if (conf == null) conf = new Configuration(); + + final Job job = new Job(conf, "read"); + job.setInputFormatClass(ProtoParquetInputFormat.class); + ProtoParquetInputFormat.setInputPaths(job, parquetPath); + + class ReadingMapper extends Mapper { + protected void map(Void key, MessageOrBuilder value, Context context) throws IOException, InterruptedException { + Message clone = ((Message.Builder) value).build(); + outputMessages.add(clone); + } + } + + job.setMapperClass(ReadingMapper.class); + job.setNumReduceTasks(0); + + job.setOutputFormatClass(NullOutputFormat.class); + + WriteUsingMR.waitForJob(job); + + List result = Collections.unmodifiableList(outputMessages); + return result; + } + + +} diff --git a/src/test/java/parquet/proto/utils/WriteUsingMR.java b/src/test/java/parquet/proto/utils/WriteUsingMR.java new file mode 100644 index 0000000000..ab8ce264ce --- /dev/null +++ b/src/test/java/parquet/proto/utils/WriteUsingMR.java @@ -0,0 +1,104 @@ +/** + * Copyright 2013 Lukas Nalezenec + * + * Licensed 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 KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package parquet.proto.utils; + +import com.google.protobuf.Message; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapreduce.Job; +import org.apache.hadoop.mapreduce.Mapper; +import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; +import parquet.Log; +import parquet.proto.ProtoParquetOutputFormat; +import parquet.proto.TestUtils; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import static java.lang.Thread.sleep; + +/** + * Writes data to parquet file using MapReduce job. + */ +public class WriteUsingMR { + + private static final Log LOG = Log.getLog(WriteUsingMR.class); + Configuration conf; + + Path outputPath; + + public void setConfiguration(Configuration conf) { + this.conf = conf; + } + + public Path write(Class pbClass, Message... messages) throws Exception { + outputPath = TestUtils.someTemporaryFilePath(); + if (conf == null) conf = new Configuration(); + final Path inputPath = new Path("src/test/java/parquet/proto/ProtoInputOutputFormatTest.java"); + + final List inputMessages = new ArrayList(); + + for (Message m : messages) { + inputMessages.add(m); + } + + class WritingMapper extends Mapper { + public void run(Context context) throws IOException, InterruptedException { + if (inputMessages == null || inputMessages.size() == 0) { + throw new RuntimeException("No mock data given"); + } else { + for (Message msg : inputMessages) { + context.write(null, msg); + LOG.debug("Reading msg from mock writing mapper" + msg); + } + } + } + } + + final Job job = new Job(conf, "write"); + + // input not really used + TextInputFormat.addInputPath(job, inputPath); + job.setInputFormatClass(TextInputFormat.class); + + job.setMapperClass(WritingMapper.class); + job.setNumReduceTasks(0); + + job.setOutputFormatClass(ProtoParquetOutputFormat.class); + ProtoParquetOutputFormat.setOutputPath(job, outputPath); + ProtoParquetOutputFormat.setProtobufferClass(job, pbClass); + + waitForJob(job); + + return outputPath; + + } + + static void waitForJob(Job job) throws Exception { + job.submit(); + while (!job.isComplete()) { + LOG.debug("waiting for job " + job.getJobName()); + sleep(50); + } + LOG.info("status for job " + job.getJobName() + ": " + (job.isSuccessful() ? "SUCCESS" : "FAILURE")); + if (!job.isSuccessful()) { + throw new RuntimeException("job failed " + job.getJobName()); + } + } +} From 96f230019ae8e91f0eacf1490481a73161b4d8a2 Mon Sep 17 00:00:00 2001 From: Lukas Date: Thu, 26 Dec 2013 16:22:11 +0100 Subject: [PATCH 27/31] #projection test - fix - cannot use inner class as mapper --- .../java/parquet/proto/utils/ReadUsingMR.java | 41 ++++++----- .../parquet/proto/utils/WriteUsingMR.java | 69 +++++++++++-------- 2 files changed, 62 insertions(+), 48 deletions(-) diff --git a/src/test/java/parquet/proto/utils/ReadUsingMR.java b/src/test/java/parquet/proto/utils/ReadUsingMR.java index 78a37e9ba5..c92289433a 100644 --- a/src/test/java/parquet/proto/utils/ReadUsingMR.java +++ b/src/test/java/parquet/proto/utils/ReadUsingMR.java @@ -36,39 +36,44 @@ */ public class ReadUsingMR { + private static List outputMessages; + Configuration conf; public void setConfiguration(Configuration conf) { this.conf = conf; } + public static class ReadingMapper extends Mapper { + protected void map(Void key, MessageOrBuilder value, Context context) throws IOException, InterruptedException { + Message clone = ((Message.Builder) value).build(); + outputMessages.add(clone); + } + } + public List read(Path parquetPath) throws Exception { - final List outputMessages = new ArrayList(); + synchronized (WriteUsingMR.class) { + outputMessages = new ArrayList(); - if (conf == null) conf = new Configuration(); + if (conf == null) conf = new Configuration(); - final Job job = new Job(conf, "read"); - job.setInputFormatClass(ProtoParquetInputFormat.class); - ProtoParquetInputFormat.setInputPaths(job, parquetPath); - class ReadingMapper extends Mapper { - protected void map(Void key, MessageOrBuilder value, Context context) throws IOException, InterruptedException { - Message clone = ((Message.Builder) value).build(); - outputMessages.add(clone); - } - } + final Job job = new Job(conf, "read"); + job.setInputFormatClass(ProtoParquetInputFormat.class); + ProtoParquetInputFormat.setInputPaths(job, parquetPath); - job.setMapperClass(ReadingMapper.class); - job.setNumReduceTasks(0); + job.setMapperClass(ReadingMapper.class); + job.setNumReduceTasks(0); - job.setOutputFormatClass(NullOutputFormat.class); + job.setOutputFormatClass(NullOutputFormat.class); - WriteUsingMR.waitForJob(job); + WriteUsingMR.waitForJob(job); - List result = Collections.unmodifiableList(outputMessages); - return result; + List result = Collections.unmodifiableList(outputMessages); + outputMessages = null; + return result; + } } - } diff --git a/src/test/java/parquet/proto/utils/WriteUsingMR.java b/src/test/java/parquet/proto/utils/WriteUsingMR.java index ab8ce264ce..22119ac677 100644 --- a/src/test/java/parquet/proto/utils/WriteUsingMR.java +++ b/src/test/java/parquet/proto/utils/WriteUsingMR.java @@ -29,6 +29,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import static java.lang.Thread.sleep; @@ -40,54 +41,62 @@ public class WriteUsingMR { private static final Log LOG = Log.getLog(WriteUsingMR.class); Configuration conf; - + private static List inputMessages; Path outputPath; public void setConfiguration(Configuration conf) { this.conf = conf; } + public static class WritingMapper extends Mapper { + + public void run(Context context) throws IOException, InterruptedException { + if (inputMessages == null || inputMessages.size() == 0) { + throw new RuntimeException("No mock data given"); + } else { + for (Message msg : inputMessages) { + context.write(null, msg); + LOG.debug("Reading msg from mock writing mapper" + msg); + } + } + } + } + public Path write(Class pbClass, Message... messages) throws Exception { - outputPath = TestUtils.someTemporaryFilePath(); - if (conf == null) conf = new Configuration(); - final Path inputPath = new Path("src/test/java/parquet/proto/ProtoInputOutputFormatTest.java"); - final List inputMessages = new ArrayList(); + synchronized (WriteUsingMR.class) { - for (Message m : messages) { - inputMessages.add(m); - } + outputPath = TestUtils.someTemporaryFilePath(); + if (conf == null) conf = new Configuration(); + final Path inputPath = new Path("src/test/java/parquet/proto/ProtoInputOutputFormatTest.java"); - class WritingMapper extends Mapper { - public void run(Context context) throws IOException, InterruptedException { - if (inputMessages == null || inputMessages.size() == 0) { - throw new RuntimeException("No mock data given"); - } else { - for (Message msg : inputMessages) { - context.write(null, msg); - LOG.debug("Reading msg from mock writing mapper" + msg); - } - } + inputMessages = new ArrayList(); + + for (Message m : messages) { + inputMessages.add(m); } - } - final Job job = new Job(conf, "write"); + inputMessages = Collections.unmodifiableList(inputMessages); + - // input not really used - TextInputFormat.addInputPath(job, inputPath); - job.setInputFormatClass(TextInputFormat.class); + final Job job = new Job(conf, "write"); - job.setMapperClass(WritingMapper.class); - job.setNumReduceTasks(0); + // input not really used + TextInputFormat.addInputPath(job, inputPath); + job.setInputFormatClass(TextInputFormat.class); - job.setOutputFormatClass(ProtoParquetOutputFormat.class); - ProtoParquetOutputFormat.setOutputPath(job, outputPath); - ProtoParquetOutputFormat.setProtobufferClass(job, pbClass); + job.setMapperClass(WritingMapper.class); + job.setNumReduceTasks(0); - waitForJob(job); + job.setOutputFormatClass(ProtoParquetOutputFormat.class); + ProtoParquetOutputFormat.setOutputPath(job, outputPath); + ProtoParquetOutputFormat.setProtobufferClass(job, pbClass); - return outputPath; + waitForJob(job); + inputMessages = null; + return outputPath; + } } static void waitForJob(Job job) throws Exception { From 985002ee33ffe6c199ff6471b3df89394bb22f14 Mon Sep 17 00:00:00 2001 From: Lukas Date: Thu, 26 Dec 2013 21:00:30 +0100 Subject: [PATCH 28/31] Code cleanup --- .../parquet/proto/converters/ProtoEnumConverter.java | 6 +++--- .../proto/converters/ProtoMessageConverter.java | 10 ++++------ .../proto/converters/ProtobufStringConverter.java | 5 +++-- src/test/java/parquet/proto/utils/ReadUsingMR.java | 3 +-- 4 files changed, 11 insertions(+), 13 deletions(-) diff --git a/src/main/java/parquet/proto/converters/ProtoEnumConverter.java b/src/main/java/parquet/proto/converters/ProtoEnumConverter.java index 671acd1cf1..1bea9a6f3f 100644 --- a/src/main/java/parquet/proto/converters/ProtoEnumConverter.java +++ b/src/main/java/parquet/proto/converters/ProtoEnumConverter.java @@ -52,12 +52,12 @@ Map makeLookupStructure(Descriptors.Fie } @Override - final public void addBinary(Binary value) { - Descriptors.EnumValueDescriptor protoValue = enumLookup.get(value); + final public void addBinary(Binary binaryValue) { + Descriptors.EnumValueDescriptor protoValue = enumLookup.get(binaryValue); if (protoValue == null) { Set knownValues = enumLookup.keySet(); - String msg = "Illegal enum value \"" + value + "\"" + String msg = "Illegal enum value \"" + binaryValue + "\"" + " in protoBuffer \"" + fieldType.getFullName() + "\"" + " legal values are: \"" + knownValues + "\""; throw new RuntimeException(msg); diff --git a/src/main/java/parquet/proto/converters/ProtoMessageConverter.java b/src/main/java/parquet/proto/converters/ProtoMessageConverter.java index 3c100bec26..27fc0d76dd 100644 --- a/src/main/java/parquet/proto/converters/ProtoMessageConverter.java +++ b/src/main/java/parquet/proto/converters/ProtoMessageConverter.java @@ -38,10 +38,8 @@ public ProtoMessageConverter(ParentValueContainer pvc, Class this(pvc, Protobufs.getMessageBuilder(protoClass), parquetSchema); } - /** - * For usage in MSG arrays - * buidler je tam protoze predtim tam byl descriptor ale nejde jednoduse vytvorit builder z descriptoru - */ + + // For usage in message arrays public ProtoMessageConverter(ParentValueContainer pvc, Message.Builder builder, GroupType parquetSchema) { int schemaSize = parquetSchema.getFieldCount(); @@ -58,7 +56,7 @@ public ProtoMessageConverter(ParentValueContainer pvc, Message.Builder builder, Descriptors.Descriptor protoDescriptor = builder.getDescriptorForType(); - for (Type parquetField : parquetSchema.getFields()) { // ziskavat jeden field bych IMHO mel o uroven vis + for (Type parquetField : parquetSchema.getFields()) { Descriptors.FieldDescriptor protoField = protoDescriptor.findFieldByName(parquetField.getName()); if (parquetField.isRepetition(Type.Repetition.REPEATED)) { @@ -134,7 +132,7 @@ public void add(Object value) { } - private static Converter newScalarConverter(ParentValueContainer pvc, Message.Builder parentBuilder, Descriptors.FieldDescriptor fieldDescriptor, Type parquetType) { + private Converter newScalarConverter(ParentValueContainer pvc, Message.Builder parentBuilder, Descriptors.FieldDescriptor fieldDescriptor, Type parquetType) { Descriptors.FieldDescriptor.JavaType javaType = fieldDescriptor.getJavaType(); diff --git a/src/main/java/parquet/proto/converters/ProtobufStringConverter.java b/src/main/java/parquet/proto/converters/ProtobufStringConverter.java index 48cf08deab..53805ecaa8 100644 --- a/src/main/java/parquet/proto/converters/ProtobufStringConverter.java +++ b/src/main/java/parquet/proto/converters/ProtobufStringConverter.java @@ -29,8 +29,9 @@ public ProtobufStringConverter(ParentValueContainer parent) { @Override - public void addBinary(Binary value) { - parent.add(value.toStringUsingUTF8()); + public void addBinary(Binary binary) { + String str = binary.toStringUsingUTF8(); + parent.add(str); } } diff --git a/src/test/java/parquet/proto/utils/ReadUsingMR.java b/src/test/java/parquet/proto/utils/ReadUsingMR.java index c92289433a..87394f4ce5 100644 --- a/src/test/java/parquet/proto/utils/ReadUsingMR.java +++ b/src/test/java/parquet/proto/utils/ReadUsingMR.java @@ -53,12 +53,11 @@ protected void map(Void key, MessageOrBuilder value, Context context) throws IOE public List read(Path parquetPath) throws Exception { - synchronized (WriteUsingMR.class) { + synchronized (ReadUsingMR.class) { outputMessages = new ArrayList(); if (conf == null) conf = new Configuration(); - final Job job = new Job(conf, "read"); job.setInputFormatClass(ProtoParquetInputFormat.class); ProtoParquetInputFormat.setInputPaths(job, parquetPath); From b273684b5a4efdd49a5bc3e0e5aa70ad3abbb155 Mon Sep 17 00:00:00 2001 From: Lukas Date: Thu, 26 Dec 2013 21:00:45 +0100 Subject: [PATCH 29/31] ConverterTest --- .../proto/ProtoSchemaConverterTest.java | 2 +- .../proto/ProtobufferRecordConverterTest.java | 134 ++++++++++++++++++ src/test/java/parquet/proto/TestUtils.java | 8 +- src/test/resources/TestProtobuf.proto | 7 +- 4 files changed, 146 insertions(+), 5 deletions(-) create mode 100644 src/test/java/parquet/proto/ProtobufferRecordConverterTest.java diff --git a/src/test/java/parquet/proto/ProtoSchemaConverterTest.java b/src/test/java/parquet/proto/ProtoSchemaConverterTest.java index 0c3dc8b5b7..ebf11fee46 100644 --- a/src/test/java/parquet/proto/ProtoSchemaConverterTest.java +++ b/src/test/java/parquet/proto/ProtoSchemaConverterTest.java @@ -60,7 +60,7 @@ public void testConvertAllDatatypes() throws Exception { " optional boolean optionalBool;\n" + " optional binary optionalString (UTF8);\n" + " optional binary optionalBytes;\n" + - " optional group optionalGroup {\n" + + " optional group optionalMessage {\n" + " optional int32 someId;\n" + " }\n" + " optional group pbgroup {\n" + diff --git a/src/test/java/parquet/proto/ProtobufferRecordConverterTest.java b/src/test/java/parquet/proto/ProtobufferRecordConverterTest.java new file mode 100644 index 0000000000..f552393327 --- /dev/null +++ b/src/test/java/parquet/proto/ProtobufferRecordConverterTest.java @@ -0,0 +1,134 @@ +/** + * Copyright 2013 Lukas Nalezenec + * + * Licensed 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 KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package parquet.proto; + +import com.google.protobuf.ByteString; +import org.junit.Test; +import parquet.protobuf.test.TestProtobuf; + +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; +import static parquet.proto.TestUtils.testData; +import static parquet.protobuf.test.TestProtobuf.SchemaConverterAllDatatypes; + +public class ProtobufferRecordConverterTest { + + @Test + public void testAllTypes() throws Exception { + SchemaConverterAllDatatypes.Builder data; + data = SchemaConverterAllDatatypes.newBuilder(); + + data.setOptionalBool(true); + data.setOptionalBytes(ByteString.copyFrom("someText", "UTF-8")); + data.setOptionalDouble(0.577); + data.setOptionalFloat(3.1415f); + data.setOptionalEnum(SchemaConverterAllDatatypes.TestEnum.FIRST); + data.setOptionalFixed32(1000 * 1000 * 1); + data.setOptionalFixed64(1000 * 1000 * 1000 * 2); + data.setOptionalInt32(1000 * 1000 * 3); + data.setOptionalInt64(1000L * 1000 * 1000 * 4); + data.setOptionalSFixed32(1000 * 1000 * 5); + data.setOptionalSFixed64(1000L * 1000 * 1000 * 6); + data.setOptionalSInt32(1000 * 1000 * 56); + data.setOptionalSInt64(1000L * 1000 * 1000 * 7); + data.setOptionalString("Good Will Hunting"); + data.setOptionalUInt32(1000 * 1000 * 8); + data.setOptionalUInt64(1000L * 1000 * 1000 * 9); + data.getOptionalMessageBuilder().setSomeId(1984); + data.getPbGroupBuilder().setGroupInt(1492); + + SchemaConverterAllDatatypes dataBuilt = data.build(); + data.clear(); + + List result; + result = testData(dataBuilt); + + //data are fully checked in testData function. Lets do one more check. + SchemaConverterAllDatatypes o = result.get(0); + assertEquals("Good Will Hunting", o.getOptionalString()); + + assertEquals(true, o.getOptionalBool()); + assertEquals(ByteString.copyFrom("someText", "UTF-8"), o.getOptionalBytes()); + assertEquals(0.577, o.getOptionalDouble(), 0.00001); + assertEquals(3.1415f, o.getOptionalFloat(), 0.00001); + assertEquals(SchemaConverterAllDatatypes.TestEnum.FIRST, o.getOptionalEnum()); + assertEquals(1000 * 1000 * 1, o.getOptionalFixed32()); + assertEquals(1000 * 1000 * 1000 * 2, o.getOptionalFixed64()); + assertEquals(1000 * 1000 * 3, o.getOptionalInt32()); + assertEquals(1000L * 1000 * 1000 * 4, o.getOptionalInt64()); + assertEquals(1000 * 1000 * 5, o.getOptionalSFixed32()); + assertEquals(1000L * 1000 * 1000 * 6, o.getOptionalSFixed64()); + assertEquals(1000 * 1000 * 56, o.getOptionalSInt32()); + assertEquals(1000L * 1000 * 1000 * 7, o.getOptionalSInt64()); + assertEquals(1000 * 1000 * 8, o.getOptionalUInt32()); + assertEquals(1000L * 1000 * 1000 * 9, o.getOptionalUInt64()); + assertEquals(1984, o.getOptionalMessage().getSomeId()); + assertEquals(1492, o.getPbGroup().getGroupInt()); + } + + @Test + public void testAllTypesMultiple() throws Exception { + int count = 100; + SchemaConverterAllDatatypes[] input = new SchemaConverterAllDatatypes[count]; + + for (int i = 0; i < count; i++) { + SchemaConverterAllDatatypes.Builder d = SchemaConverterAllDatatypes.newBuilder(); + + if (i % 2 != 0) d.setOptionalBool(true); + if (i % 3 != 0) d.setOptionalBytes(ByteString.copyFrom("someText " + i, "UTF-8")); + if (i % 4 != 0) d.setOptionalDouble(0.577 * i); + if (i % 5 != 0) d.setOptionalFloat(3.1415f * i); + if (i % 6 != 0) d.setOptionalEnum(SchemaConverterAllDatatypes.TestEnum.FIRST); + if (i % 7 != 0) d.setOptionalFixed32(1000 * i * 1); + if (i % 8 != 0) d.setOptionalFixed64(1000 * i * 1000 * 2); + if (i % 9 != 0) d.setOptionalInt32(1000 * i * 3); + if (i % 2 != 1) d.setOptionalSFixed32(1000 * i * 5); + if (i % 3 != 1) d.setOptionalSFixed64(1000 * i * 1000 * 6); + if (i % 4 != 1) d.setOptionalSInt32(1000 * i * 56); + if (i % 5 != 1) d.setOptionalSInt64(1000 * i * 1000 * 7); + if (i % 6 != 1) d.setOptionalString("Good Will Hunting " + i); + if (i % 7 != 1) d.setOptionalUInt32(1000 * i * 8); + if (i % 8 != 1) d.setOptionalUInt64(1000 * i * 1000 * 9); + if (i % 9 != 1) d.getOptionalMessageBuilder().setSomeId(1984 * i); + if (i % 2 != 1) d.getPbGroupBuilder().setGroupInt(1492 * i); + if (i % 3 != 1) d.setOptionalInt64(1000 * i * 1000 * 4); + input[i] = d.build(); + } + + List result; + result = testData(input); + + //data are fully checked in testData function. Lets do one more check. + assertEquals("Good Will Hunting 0", result.get(0).getOptionalString()); + assertEquals("Good Will Hunting 90", result.get(90).getOptionalString()); + } + + + @Test + public void testDefaults() throws Exception { + SchemaConverterAllDatatypes.Builder data; + data = SchemaConverterAllDatatypes.newBuilder(); + + List result = testData(data.build()); + SchemaConverterAllDatatypes message = result.get(0); + assertEquals("", message.getOptionalString()); + assertEquals(false, message.getOptionalBool()); + assertEquals(0, message.getOptionalFixed32()); + } +} diff --git a/src/test/java/parquet/proto/TestUtils.java b/src/test/java/parquet/proto/TestUtils.java index 086bb75aed..3e97a76356 100644 --- a/src/test/java/parquet/proto/TestUtils.java +++ b/src/test/java/parquet/proto/TestUtils.java @@ -59,15 +59,17 @@ private static Class inferRecordsClass(MessageOrBuilder[] rec /** * Writes messages to file, reads messages from file and checks if everything is OK. */ - public static void testData(MessageOrBuilder... messages) throws IOException { + public static List testData(T... messages) throws IOException { checkSameBuilderInstance(messages); List input = cloneList(messages); - List output = writeAndRead(messages); + List output = (List) writeAndRead(messages); - assertEquals("The protobuffers are not same:\n", asMessages(input), asMessages(output)); + List outputAsMessages = asMessages(output); + assertEquals("The protobuffers are not same:\n", asMessages(input), outputAsMessages); + return (List) outputAsMessages; } private static List cloneList(MessageOrBuilder[] messages) { diff --git a/src/test/resources/TestProtobuf.proto b/src/test/resources/TestProtobuf.proto index 989c8b2394..2362d53ed1 100644 --- a/src/test/resources/TestProtobuf.proto +++ b/src/test/resources/TestProtobuf.proto @@ -47,7 +47,7 @@ message Links { optional bool optionalBool = 13; optional string optionalString = 14; optional bytes optionalBytes = 15; - optional SchemaConverterSimpleMessage optionalGroup = 16; + optional SchemaConverterSimpleMessage optionalMessage = 16; optional group PbGroup = 17 { optional int32 groupInt = 2; } @@ -58,6 +58,11 @@ message Links { optional TestEnum optionalEnum = 18; } +message RequiredWithDefaultMessage{ + // tohe je asi spatne, required nemuzou mit defaulty + required int32 optionalWD = 18 [default = 10]; + } + message SchemaConverterRepetition { optional int32 optionalPrimitive = 1; required int32 requiredPrimitive = 2; From 99b7e52a58d55d894dc244c93e5a4c4d183e411e Mon Sep 17 00:00:00 2001 From: Lukas Date: Thu, 26 Dec 2013 21:08:35 +0100 Subject: [PATCH 30/31] new root directory --- .idea/ant.xml | 7 +++++++ ...n__commons_beanutils_commons_beanutils_1_7_0.xml | 13 +++++++++++++ ...mmons_beanutils_commons_beanutils_core_1_8_0.xml | 13 +++++++++++++ ...ommons_collections_commons_collections_3_2_1.xml | 13 +++++++++++++ ...mons_configuration_commons_configuration_1_6.xml | 13 +++++++++++++ ...Maven__commons_digester_commons_digester_1_8.xml | 13 +++++++++++++ .../libraries/Maven__commons_io_commons_io_2_1.xml | 13 +++++++++++++ .../Maven__org_apache_commons_commons_math_2_1.xml | 13 +++++++++++++ .../Maven__org_apache_hadoop_hadoop_core_1_1_0.xml | 13 +++++++++++++ ..._org_codehaus_jackson_jackson_core_asl_1_0_1.xml | 13 +++++++++++++ ...rg_codehaus_jackson_jackson_mapper_asl_1_0_1.xml | 13 +++++++++++++ README.md => parquet-protobuf/README.md | 0 pom.xml => parquet-protobuf/pom.xml | 0 .../java/parquet/proto/ProtoParquetInputFormat.java | 0 .../parquet/proto/ProtoParquetOutputFormat.java | 0 .../main/java/parquet/proto/ProtoParquetReader.java | 0 .../main/java/parquet/proto/ProtoParquetWriter.java | 0 .../main/java/parquet/proto/ProtoReadSupport.java | 0 .../java/parquet/proto/ProtoRecordMaterializer.java | 0 .../java/parquet/proto/ProtoSchemaConverter.java | 0 .../main/java/parquet/proto/ProtoWriteSupport.java | 0 .../parquet/proto/ProtobufferRecordConverter.java | 0 .../proto/converters/ParentValueContainer.java | 0 .../proto/converters/ProtoArrayConverter.java | 0 .../proto/converters/ProtoBinaryConverter.java | 0 .../proto/converters/ProtoBooleanConverter.java | 0 .../proto/converters/ProtoDoubleConverter.java | 0 .../proto/converters/ProtoEnumConverter.java | 0 .../proto/converters/ProtoFloatConverter.java | 0 .../parquet/proto/converters/ProtoIntConverter.java | 0 .../proto/converters/ProtoLongConverter.java | 0 .../proto/converters/ProtoMessageConverter.java | 0 .../proto/converters/ProtobufStringConverter.java | 0 .../parquet/proto/ProtoInputOutputFormatTest.java | 0 .../parquet/proto/ProtoSchemaConverterTest.java | 0 .../proto/ProtobufferRecordConverterTest.java | 0 .../src}/test/java/parquet/proto/TestUtils.java | 0 .../test/java/parquet/proto/utils/ReadUsingMR.java | 0 .../test/java/parquet/proto/utils/WriteUsingMR.java | 0 .../src}/test/resources/TestProtobuf.proto | 0 40 files changed, 137 insertions(+) create mode 100644 .idea/ant.xml create mode 100644 .idea/libraries/Maven__commons_beanutils_commons_beanutils_1_7_0.xml create mode 100644 .idea/libraries/Maven__commons_beanutils_commons_beanutils_core_1_8_0.xml create mode 100644 .idea/libraries/Maven__commons_collections_commons_collections_3_2_1.xml create mode 100644 .idea/libraries/Maven__commons_configuration_commons_configuration_1_6.xml create mode 100644 .idea/libraries/Maven__commons_digester_commons_digester_1_8.xml create mode 100644 .idea/libraries/Maven__commons_io_commons_io_2_1.xml create mode 100644 .idea/libraries/Maven__org_apache_commons_commons_math_2_1.xml create mode 100644 .idea/libraries/Maven__org_apache_hadoop_hadoop_core_1_1_0.xml create mode 100644 .idea/libraries/Maven__org_codehaus_jackson_jackson_core_asl_1_0_1.xml create mode 100644 .idea/libraries/Maven__org_codehaus_jackson_jackson_mapper_asl_1_0_1.xml rename README.md => parquet-protobuf/README.md (100%) rename pom.xml => parquet-protobuf/pom.xml (100%) rename {src => parquet-protobuf/src}/main/java/parquet/proto/ProtoParquetInputFormat.java (100%) rename {src => parquet-protobuf/src}/main/java/parquet/proto/ProtoParquetOutputFormat.java (100%) rename {src => parquet-protobuf/src}/main/java/parquet/proto/ProtoParquetReader.java (100%) rename {src => parquet-protobuf/src}/main/java/parquet/proto/ProtoParquetWriter.java (100%) rename {src => parquet-protobuf/src}/main/java/parquet/proto/ProtoReadSupport.java (100%) rename {src => parquet-protobuf/src}/main/java/parquet/proto/ProtoRecordMaterializer.java (100%) rename {src => parquet-protobuf/src}/main/java/parquet/proto/ProtoSchemaConverter.java (100%) rename {src => parquet-protobuf/src}/main/java/parquet/proto/ProtoWriteSupport.java (100%) rename {src => parquet-protobuf/src}/main/java/parquet/proto/ProtobufferRecordConverter.java (100%) rename {src => parquet-protobuf/src}/main/java/parquet/proto/converters/ParentValueContainer.java (100%) rename {src => parquet-protobuf/src}/main/java/parquet/proto/converters/ProtoArrayConverter.java (100%) rename {src => parquet-protobuf/src}/main/java/parquet/proto/converters/ProtoBinaryConverter.java (100%) rename {src => parquet-protobuf/src}/main/java/parquet/proto/converters/ProtoBooleanConverter.java (100%) rename {src => parquet-protobuf/src}/main/java/parquet/proto/converters/ProtoDoubleConverter.java (100%) rename {src => parquet-protobuf/src}/main/java/parquet/proto/converters/ProtoEnumConverter.java (100%) rename {src => parquet-protobuf/src}/main/java/parquet/proto/converters/ProtoFloatConverter.java (100%) rename {src => parquet-protobuf/src}/main/java/parquet/proto/converters/ProtoIntConverter.java (100%) rename {src => parquet-protobuf/src}/main/java/parquet/proto/converters/ProtoLongConverter.java (100%) rename {src => parquet-protobuf/src}/main/java/parquet/proto/converters/ProtoMessageConverter.java (100%) rename {src => parquet-protobuf/src}/main/java/parquet/proto/converters/ProtobufStringConverter.java (100%) rename {src => parquet-protobuf/src}/test/java/parquet/proto/ProtoInputOutputFormatTest.java (100%) rename {src => parquet-protobuf/src}/test/java/parquet/proto/ProtoSchemaConverterTest.java (100%) rename {src => parquet-protobuf/src}/test/java/parquet/proto/ProtobufferRecordConverterTest.java (100%) rename {src => parquet-protobuf/src}/test/java/parquet/proto/TestUtils.java (100%) rename {src => parquet-protobuf/src}/test/java/parquet/proto/utils/ReadUsingMR.java (100%) rename {src => parquet-protobuf/src}/test/java/parquet/proto/utils/WriteUsingMR.java (100%) rename {src => parquet-protobuf/src}/test/resources/TestProtobuf.proto (100%) diff --git a/.idea/ant.xml b/.idea/ant.xml new file mode 100644 index 0000000000..2581ca3fe8 --- /dev/null +++ b/.idea/ant.xml @@ -0,0 +1,7 @@ + + + + + + + diff --git a/.idea/libraries/Maven__commons_beanutils_commons_beanutils_1_7_0.xml b/.idea/libraries/Maven__commons_beanutils_commons_beanutils_1_7_0.xml new file mode 100644 index 0000000000..1cb64edf6e --- /dev/null +++ b/.idea/libraries/Maven__commons_beanutils_commons_beanutils_1_7_0.xml @@ -0,0 +1,13 @@ + + + + + + + + + + + + + \ No newline at end of file diff --git a/.idea/libraries/Maven__commons_beanutils_commons_beanutils_core_1_8_0.xml b/.idea/libraries/Maven__commons_beanutils_commons_beanutils_core_1_8_0.xml new file mode 100644 index 0000000000..3fda2c0c1f --- /dev/null +++ b/.idea/libraries/Maven__commons_beanutils_commons_beanutils_core_1_8_0.xml @@ -0,0 +1,13 @@ + + + + + + + + + + + + + \ No newline at end of file diff --git a/.idea/libraries/Maven__commons_collections_commons_collections_3_2_1.xml b/.idea/libraries/Maven__commons_collections_commons_collections_3_2_1.xml new file mode 100644 index 0000000000..3caee7e547 --- /dev/null +++ b/.idea/libraries/Maven__commons_collections_commons_collections_3_2_1.xml @@ -0,0 +1,13 @@ + + + + + + + + + + + + + \ No newline at end of file diff --git a/.idea/libraries/Maven__commons_configuration_commons_configuration_1_6.xml b/.idea/libraries/Maven__commons_configuration_commons_configuration_1_6.xml new file mode 100644 index 0000000000..ae9678115a --- /dev/null +++ b/.idea/libraries/Maven__commons_configuration_commons_configuration_1_6.xml @@ -0,0 +1,13 @@ + + + + + + + + + + + + + \ No newline at end of file diff --git a/.idea/libraries/Maven__commons_digester_commons_digester_1_8.xml b/.idea/libraries/Maven__commons_digester_commons_digester_1_8.xml new file mode 100644 index 0000000000..49f1defe1f --- /dev/null +++ b/.idea/libraries/Maven__commons_digester_commons_digester_1_8.xml @@ -0,0 +1,13 @@ + + + + + + + + + + + + + \ No newline at end of file diff --git a/.idea/libraries/Maven__commons_io_commons_io_2_1.xml b/.idea/libraries/Maven__commons_io_commons_io_2_1.xml new file mode 100644 index 0000000000..3b7814235f --- /dev/null +++ b/.idea/libraries/Maven__commons_io_commons_io_2_1.xml @@ -0,0 +1,13 @@ + + + + + + + + + + + + + \ No newline at end of file diff --git a/.idea/libraries/Maven__org_apache_commons_commons_math_2_1.xml b/.idea/libraries/Maven__org_apache_commons_commons_math_2_1.xml new file mode 100644 index 0000000000..eaa8a65b4c --- /dev/null +++ b/.idea/libraries/Maven__org_apache_commons_commons_math_2_1.xml @@ -0,0 +1,13 @@ + + + + + + + + + + + + + \ No newline at end of file diff --git a/.idea/libraries/Maven__org_apache_hadoop_hadoop_core_1_1_0.xml b/.idea/libraries/Maven__org_apache_hadoop_hadoop_core_1_1_0.xml new file mode 100644 index 0000000000..a2af01f112 --- /dev/null +++ b/.idea/libraries/Maven__org_apache_hadoop_hadoop_core_1_1_0.xml @@ -0,0 +1,13 @@ + + + + + + + + + + + + + \ No newline at end of file diff --git a/.idea/libraries/Maven__org_codehaus_jackson_jackson_core_asl_1_0_1.xml b/.idea/libraries/Maven__org_codehaus_jackson_jackson_core_asl_1_0_1.xml new file mode 100644 index 0000000000..6b47b70df8 --- /dev/null +++ b/.idea/libraries/Maven__org_codehaus_jackson_jackson_core_asl_1_0_1.xml @@ -0,0 +1,13 @@ + + + + + + + + + + + + + \ No newline at end of file diff --git a/.idea/libraries/Maven__org_codehaus_jackson_jackson_mapper_asl_1_0_1.xml b/.idea/libraries/Maven__org_codehaus_jackson_jackson_mapper_asl_1_0_1.xml new file mode 100644 index 0000000000..598ed2cbba --- /dev/null +++ b/.idea/libraries/Maven__org_codehaus_jackson_jackson_mapper_asl_1_0_1.xml @@ -0,0 +1,13 @@ + + + + + + + + + + + + + \ No newline at end of file diff --git a/README.md b/parquet-protobuf/README.md similarity index 100% rename from README.md rename to parquet-protobuf/README.md diff --git a/pom.xml b/parquet-protobuf/pom.xml similarity index 100% rename from pom.xml rename to parquet-protobuf/pom.xml diff --git a/src/main/java/parquet/proto/ProtoParquetInputFormat.java b/parquet-protobuf/src/main/java/parquet/proto/ProtoParquetInputFormat.java similarity index 100% rename from src/main/java/parquet/proto/ProtoParquetInputFormat.java rename to parquet-protobuf/src/main/java/parquet/proto/ProtoParquetInputFormat.java diff --git a/src/main/java/parquet/proto/ProtoParquetOutputFormat.java b/parquet-protobuf/src/main/java/parquet/proto/ProtoParquetOutputFormat.java similarity index 100% rename from src/main/java/parquet/proto/ProtoParquetOutputFormat.java rename to parquet-protobuf/src/main/java/parquet/proto/ProtoParquetOutputFormat.java diff --git a/src/main/java/parquet/proto/ProtoParquetReader.java b/parquet-protobuf/src/main/java/parquet/proto/ProtoParquetReader.java similarity index 100% rename from src/main/java/parquet/proto/ProtoParquetReader.java rename to parquet-protobuf/src/main/java/parquet/proto/ProtoParquetReader.java diff --git a/src/main/java/parquet/proto/ProtoParquetWriter.java b/parquet-protobuf/src/main/java/parquet/proto/ProtoParquetWriter.java similarity index 100% rename from src/main/java/parquet/proto/ProtoParquetWriter.java rename to parquet-protobuf/src/main/java/parquet/proto/ProtoParquetWriter.java diff --git a/src/main/java/parquet/proto/ProtoReadSupport.java b/parquet-protobuf/src/main/java/parquet/proto/ProtoReadSupport.java similarity index 100% rename from src/main/java/parquet/proto/ProtoReadSupport.java rename to parquet-protobuf/src/main/java/parquet/proto/ProtoReadSupport.java diff --git a/src/main/java/parquet/proto/ProtoRecordMaterializer.java b/parquet-protobuf/src/main/java/parquet/proto/ProtoRecordMaterializer.java similarity index 100% rename from src/main/java/parquet/proto/ProtoRecordMaterializer.java rename to parquet-protobuf/src/main/java/parquet/proto/ProtoRecordMaterializer.java diff --git a/src/main/java/parquet/proto/ProtoSchemaConverter.java b/parquet-protobuf/src/main/java/parquet/proto/ProtoSchemaConverter.java similarity index 100% rename from src/main/java/parquet/proto/ProtoSchemaConverter.java rename to parquet-protobuf/src/main/java/parquet/proto/ProtoSchemaConverter.java diff --git a/src/main/java/parquet/proto/ProtoWriteSupport.java b/parquet-protobuf/src/main/java/parquet/proto/ProtoWriteSupport.java similarity index 100% rename from src/main/java/parquet/proto/ProtoWriteSupport.java rename to parquet-protobuf/src/main/java/parquet/proto/ProtoWriteSupport.java diff --git a/src/main/java/parquet/proto/ProtobufferRecordConverter.java b/parquet-protobuf/src/main/java/parquet/proto/ProtobufferRecordConverter.java similarity index 100% rename from src/main/java/parquet/proto/ProtobufferRecordConverter.java rename to parquet-protobuf/src/main/java/parquet/proto/ProtobufferRecordConverter.java diff --git a/src/main/java/parquet/proto/converters/ParentValueContainer.java b/parquet-protobuf/src/main/java/parquet/proto/converters/ParentValueContainer.java similarity index 100% rename from src/main/java/parquet/proto/converters/ParentValueContainer.java rename to parquet-protobuf/src/main/java/parquet/proto/converters/ParentValueContainer.java diff --git a/src/main/java/parquet/proto/converters/ProtoArrayConverter.java b/parquet-protobuf/src/main/java/parquet/proto/converters/ProtoArrayConverter.java similarity index 100% rename from src/main/java/parquet/proto/converters/ProtoArrayConverter.java rename to parquet-protobuf/src/main/java/parquet/proto/converters/ProtoArrayConverter.java diff --git a/src/main/java/parquet/proto/converters/ProtoBinaryConverter.java b/parquet-protobuf/src/main/java/parquet/proto/converters/ProtoBinaryConverter.java similarity index 100% rename from src/main/java/parquet/proto/converters/ProtoBinaryConverter.java rename to parquet-protobuf/src/main/java/parquet/proto/converters/ProtoBinaryConverter.java diff --git a/src/main/java/parquet/proto/converters/ProtoBooleanConverter.java b/parquet-protobuf/src/main/java/parquet/proto/converters/ProtoBooleanConverter.java similarity index 100% rename from src/main/java/parquet/proto/converters/ProtoBooleanConverter.java rename to parquet-protobuf/src/main/java/parquet/proto/converters/ProtoBooleanConverter.java diff --git a/src/main/java/parquet/proto/converters/ProtoDoubleConverter.java b/parquet-protobuf/src/main/java/parquet/proto/converters/ProtoDoubleConverter.java similarity index 100% rename from src/main/java/parquet/proto/converters/ProtoDoubleConverter.java rename to parquet-protobuf/src/main/java/parquet/proto/converters/ProtoDoubleConverter.java diff --git a/src/main/java/parquet/proto/converters/ProtoEnumConverter.java b/parquet-protobuf/src/main/java/parquet/proto/converters/ProtoEnumConverter.java similarity index 100% rename from src/main/java/parquet/proto/converters/ProtoEnumConverter.java rename to parquet-protobuf/src/main/java/parquet/proto/converters/ProtoEnumConverter.java diff --git a/src/main/java/parquet/proto/converters/ProtoFloatConverter.java b/parquet-protobuf/src/main/java/parquet/proto/converters/ProtoFloatConverter.java similarity index 100% rename from src/main/java/parquet/proto/converters/ProtoFloatConverter.java rename to parquet-protobuf/src/main/java/parquet/proto/converters/ProtoFloatConverter.java diff --git a/src/main/java/parquet/proto/converters/ProtoIntConverter.java b/parquet-protobuf/src/main/java/parquet/proto/converters/ProtoIntConverter.java similarity index 100% rename from src/main/java/parquet/proto/converters/ProtoIntConverter.java rename to parquet-protobuf/src/main/java/parquet/proto/converters/ProtoIntConverter.java diff --git a/src/main/java/parquet/proto/converters/ProtoLongConverter.java b/parquet-protobuf/src/main/java/parquet/proto/converters/ProtoLongConverter.java similarity index 100% rename from src/main/java/parquet/proto/converters/ProtoLongConverter.java rename to parquet-protobuf/src/main/java/parquet/proto/converters/ProtoLongConverter.java diff --git a/src/main/java/parquet/proto/converters/ProtoMessageConverter.java b/parquet-protobuf/src/main/java/parquet/proto/converters/ProtoMessageConverter.java similarity index 100% rename from src/main/java/parquet/proto/converters/ProtoMessageConverter.java rename to parquet-protobuf/src/main/java/parquet/proto/converters/ProtoMessageConverter.java diff --git a/src/main/java/parquet/proto/converters/ProtobufStringConverter.java b/parquet-protobuf/src/main/java/parquet/proto/converters/ProtobufStringConverter.java similarity index 100% rename from src/main/java/parquet/proto/converters/ProtobufStringConverter.java rename to parquet-protobuf/src/main/java/parquet/proto/converters/ProtobufStringConverter.java diff --git a/src/test/java/parquet/proto/ProtoInputOutputFormatTest.java b/parquet-protobuf/src/test/java/parquet/proto/ProtoInputOutputFormatTest.java similarity index 100% rename from src/test/java/parquet/proto/ProtoInputOutputFormatTest.java rename to parquet-protobuf/src/test/java/parquet/proto/ProtoInputOutputFormatTest.java diff --git a/src/test/java/parquet/proto/ProtoSchemaConverterTest.java b/parquet-protobuf/src/test/java/parquet/proto/ProtoSchemaConverterTest.java similarity index 100% rename from src/test/java/parquet/proto/ProtoSchemaConverterTest.java rename to parquet-protobuf/src/test/java/parquet/proto/ProtoSchemaConverterTest.java diff --git a/src/test/java/parquet/proto/ProtobufferRecordConverterTest.java b/parquet-protobuf/src/test/java/parquet/proto/ProtobufferRecordConverterTest.java similarity index 100% rename from src/test/java/parquet/proto/ProtobufferRecordConverterTest.java rename to parquet-protobuf/src/test/java/parquet/proto/ProtobufferRecordConverterTest.java diff --git a/src/test/java/parquet/proto/TestUtils.java b/parquet-protobuf/src/test/java/parquet/proto/TestUtils.java similarity index 100% rename from src/test/java/parquet/proto/TestUtils.java rename to parquet-protobuf/src/test/java/parquet/proto/TestUtils.java diff --git a/src/test/java/parquet/proto/utils/ReadUsingMR.java b/parquet-protobuf/src/test/java/parquet/proto/utils/ReadUsingMR.java similarity index 100% rename from src/test/java/parquet/proto/utils/ReadUsingMR.java rename to parquet-protobuf/src/test/java/parquet/proto/utils/ReadUsingMR.java diff --git a/src/test/java/parquet/proto/utils/WriteUsingMR.java b/parquet-protobuf/src/test/java/parquet/proto/utils/WriteUsingMR.java similarity index 100% rename from src/test/java/parquet/proto/utils/WriteUsingMR.java rename to parquet-protobuf/src/test/java/parquet/proto/utils/WriteUsingMR.java diff --git a/src/test/resources/TestProtobuf.proto b/parquet-protobuf/src/test/resources/TestProtobuf.proto similarity index 100% rename from src/test/resources/TestProtobuf.proto rename to parquet-protobuf/src/test/resources/TestProtobuf.proto From 94b2ec0b93c826b39a881f5e0653719ae78889ce Mon Sep 17 00:00:00 2001 From: Lukas Date: Thu, 26 Dec 2013 21:13:48 +0100 Subject: [PATCH 31/31] delete .idea directory --- .idea/ant.xml | 7 ------- ...n__commons_beanutils_commons_beanutils_1_7_0.xml | 13 ------------- ...mmons_beanutils_commons_beanutils_core_1_8_0.xml | 13 ------------- ...ommons_collections_commons_collections_3_2_1.xml | 13 ------------- ...mons_configuration_commons_configuration_1_6.xml | 13 ------------- ...Maven__commons_digester_commons_digester_1_8.xml | 13 ------------- .../libraries/Maven__commons_io_commons_io_2_1.xml | 13 ------------- .../Maven__org_apache_commons_commons_math_2_1.xml | 13 ------------- .../Maven__org_apache_hadoop_hadoop_core_1_1_0.xml | 13 ------------- ..._org_codehaus_jackson_jackson_core_asl_1_0_1.xml | 13 ------------- ...rg_codehaus_jackson_jackson_mapper_asl_1_0_1.xml | 13 ------------- 11 files changed, 137 deletions(-) delete mode 100644 .idea/ant.xml delete mode 100644 .idea/libraries/Maven__commons_beanutils_commons_beanutils_1_7_0.xml delete mode 100644 .idea/libraries/Maven__commons_beanutils_commons_beanutils_core_1_8_0.xml delete mode 100644 .idea/libraries/Maven__commons_collections_commons_collections_3_2_1.xml delete mode 100644 .idea/libraries/Maven__commons_configuration_commons_configuration_1_6.xml delete mode 100644 .idea/libraries/Maven__commons_digester_commons_digester_1_8.xml delete mode 100644 .idea/libraries/Maven__commons_io_commons_io_2_1.xml delete mode 100644 .idea/libraries/Maven__org_apache_commons_commons_math_2_1.xml delete mode 100644 .idea/libraries/Maven__org_apache_hadoop_hadoop_core_1_1_0.xml delete mode 100644 .idea/libraries/Maven__org_codehaus_jackson_jackson_core_asl_1_0_1.xml delete mode 100644 .idea/libraries/Maven__org_codehaus_jackson_jackson_mapper_asl_1_0_1.xml diff --git a/.idea/ant.xml b/.idea/ant.xml deleted file mode 100644 index 2581ca3fe8..0000000000 --- a/.idea/ant.xml +++ /dev/null @@ -1,7 +0,0 @@ - - - - - - - diff --git a/.idea/libraries/Maven__commons_beanutils_commons_beanutils_1_7_0.xml b/.idea/libraries/Maven__commons_beanutils_commons_beanutils_1_7_0.xml deleted file mode 100644 index 1cb64edf6e..0000000000 --- a/.idea/libraries/Maven__commons_beanutils_commons_beanutils_1_7_0.xml +++ /dev/null @@ -1,13 +0,0 @@ - - - - - - - - - - - - - \ No newline at end of file diff --git a/.idea/libraries/Maven__commons_beanutils_commons_beanutils_core_1_8_0.xml b/.idea/libraries/Maven__commons_beanutils_commons_beanutils_core_1_8_0.xml deleted file mode 100644 index 3fda2c0c1f..0000000000 --- a/.idea/libraries/Maven__commons_beanutils_commons_beanutils_core_1_8_0.xml +++ /dev/null @@ -1,13 +0,0 @@ - - - - - - - - - - - - - \ No newline at end of file diff --git a/.idea/libraries/Maven__commons_collections_commons_collections_3_2_1.xml b/.idea/libraries/Maven__commons_collections_commons_collections_3_2_1.xml deleted file mode 100644 index 3caee7e547..0000000000 --- a/.idea/libraries/Maven__commons_collections_commons_collections_3_2_1.xml +++ /dev/null @@ -1,13 +0,0 @@ - - - - - - - - - - - - - \ No newline at end of file diff --git a/.idea/libraries/Maven__commons_configuration_commons_configuration_1_6.xml b/.idea/libraries/Maven__commons_configuration_commons_configuration_1_6.xml deleted file mode 100644 index ae9678115a..0000000000 --- a/.idea/libraries/Maven__commons_configuration_commons_configuration_1_6.xml +++ /dev/null @@ -1,13 +0,0 @@ - - - - - - - - - - - - - \ No newline at end of file diff --git a/.idea/libraries/Maven__commons_digester_commons_digester_1_8.xml b/.idea/libraries/Maven__commons_digester_commons_digester_1_8.xml deleted file mode 100644 index 49f1defe1f..0000000000 --- a/.idea/libraries/Maven__commons_digester_commons_digester_1_8.xml +++ /dev/null @@ -1,13 +0,0 @@ - - - - - - - - - - - - - \ No newline at end of file diff --git a/.idea/libraries/Maven__commons_io_commons_io_2_1.xml b/.idea/libraries/Maven__commons_io_commons_io_2_1.xml deleted file mode 100644 index 3b7814235f..0000000000 --- a/.idea/libraries/Maven__commons_io_commons_io_2_1.xml +++ /dev/null @@ -1,13 +0,0 @@ - - - - - - - - - - - - - \ No newline at end of file diff --git a/.idea/libraries/Maven__org_apache_commons_commons_math_2_1.xml b/.idea/libraries/Maven__org_apache_commons_commons_math_2_1.xml deleted file mode 100644 index eaa8a65b4c..0000000000 --- a/.idea/libraries/Maven__org_apache_commons_commons_math_2_1.xml +++ /dev/null @@ -1,13 +0,0 @@ - - - - - - - - - - - - - \ No newline at end of file diff --git a/.idea/libraries/Maven__org_apache_hadoop_hadoop_core_1_1_0.xml b/.idea/libraries/Maven__org_apache_hadoop_hadoop_core_1_1_0.xml deleted file mode 100644 index a2af01f112..0000000000 --- a/.idea/libraries/Maven__org_apache_hadoop_hadoop_core_1_1_0.xml +++ /dev/null @@ -1,13 +0,0 @@ - - - - - - - - - - - - - \ No newline at end of file diff --git a/.idea/libraries/Maven__org_codehaus_jackson_jackson_core_asl_1_0_1.xml b/.idea/libraries/Maven__org_codehaus_jackson_jackson_core_asl_1_0_1.xml deleted file mode 100644 index 6b47b70df8..0000000000 --- a/.idea/libraries/Maven__org_codehaus_jackson_jackson_core_asl_1_0_1.xml +++ /dev/null @@ -1,13 +0,0 @@ - - - - - - - - - - - - - \ No newline at end of file diff --git a/.idea/libraries/Maven__org_codehaus_jackson_jackson_mapper_asl_1_0_1.xml b/.idea/libraries/Maven__org_codehaus_jackson_jackson_mapper_asl_1_0_1.xml deleted file mode 100644 index 598ed2cbba..0000000000 --- a/.idea/libraries/Maven__org_codehaus_jackson_jackson_mapper_asl_1_0_1.xml +++ /dev/null @@ -1,13 +0,0 @@ - - - - - - - - - - - - - \ No newline at end of file