Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[PARQUET-951] Pull request for handling protobuf field id #410

Closed
wants to merge 6 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
14 changes: 0 additions & 14 deletions .travis.yml
Original file line number Diff line number Diff line change
Expand Up @@ -2,20 +2,6 @@ language: java
before_install:
- sudo apt-get update -qq
- sudo apt-get install build-essential
- mkdir protobuf_install
- pushd protobuf_install
- wget https://github.com/google/protobuf/archive/v3.2.0.tar.gz -O protobuf-3.2.0.tar.gz
- tar xzf protobuf-3.2.0.tar.gz
- cd protobuf-3.2.0
- sudo apt-get install autoconf automake libtool curl make g++ unzip
- ./autogen.sh
- ./configure
- make
- make check
- sudo make install
- sudo ldconfig
- protoc --version
- popd
- pwd
- sudo apt-get install -qq libboost-dev libboost-test-dev libboost-program-options-dev libevent-dev automake libtool flex bison pkg-config g++ libssl-dev
- wget -nv http://archive.apache.org/dist/thrift/0.7.0/thrift-0.7.0.tar.gz
Expand Down
Original file line number Diff line number Diff line change
@@ -1,14 +1,14 @@
/*
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
*
* http://www.apache.org/licenses/LICENSE-2.0
*
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
Expand Down Expand Up @@ -394,4 +394,36 @@ List<Type> mergeFields(GroupType toMerge, boolean strict) {
}
return newFields;
}

/**
* Check the projection schema is contained by the origin, and decorate projection schema with the origin field id if original schema has field id.
* This is needed because people usually do not put the field id when they describe a projection schema.
*
* @param projection Projection schema (or subschema of the projection)
* @param builder Builder for constructing a corresponding schema with field id (if origin contains) regarding to projection
* @param <T> Type, which could be {@link MessageType} or {@link org.apache.parquet.schema.Types.GroupBuilder}
* @return builder or the final {@link MessageType}.
* @throws InvalidRecordException if check fails.
*/
protected <T> T checkSubTypeAndDecorateWithFieldId(GroupType projection, Types.GroupBuilder<T> builder) {
if (this.getRepetition() != projection.getRepetition()) {
throw new InvalidRecordException(projection + " found: expected " + this);
}
for (Type field : projection.getFields()) {
Type fieldInFile = this.getType(field.getName());
if (fieldInFile == null) {
throw new InvalidRecordException(projection + " found: expected " + this);
}
if (fieldInFile.isPrimitive()) {
fieldInFile.checkContains(field);
builder.addField(fieldInFile.getId() == null ? field : field.withId(fieldInFile.getId().intValue()));
} else {
fieldInFile.asGroupType().checkSubTypeAndDecorateWithFieldId(field.asGroupType(), builder.group(field.getRepetition()));
}
}
if (this.getId() != null) {
builder.id(this.getId().intValue());
}
return builder.as(this.getOriginalType()).named(this.getName());
}
}
Original file line number Diff line number Diff line change
@@ -1,14 +1,14 @@
/*
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
*
* http://www.apache.org/licenses/LICENSE-2.0
*
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
Expand Down Expand Up @@ -41,14 +41,14 @@ public MessageType(String name, Type... fields) {
super(Repetition.REPEATED, name, fields);
}

/**
*
* @param name the name of the type
* @param fields the fields contained by this message
*/
public MessageType(String name, List<Type> fields) {
super(Repetition.REPEATED, name, fields);
}
/**
*
* @param name the name of the type
* @param fields the fields contained by this message
*/
public MessageType(String name, List<Type> fields) {
super(Repetition.REPEATED, name, fields);
}

/**
* {@inheritDoc}
Expand Down Expand Up @@ -145,4 +145,15 @@ public MessageType union(MessageType toMerge, boolean strict) {
return new MessageType(this.getName(), mergeFields(toMerge, strict));
}

/**
* Check projection schema is subschema of this and append field id to projection if this contains field id.
* This is because we often just don't put field id for projection schema, but that is necessary when trying to read fields by id.
*
* @param projection
* @return Projection schema with field id.
* @throws InvalidRecordException if check fails.
*/
public MessageType checkSubTypeAndDecorateWithId(MessageType projection) {
return checkSubTypeAndDecorateWithFieldId(projection, Types.buildMessage());
}
}
15 changes: 12 additions & 3 deletions parquet-column/src/main/java/org/apache/parquet/schema/Type.java
Original file line number Diff line number Diff line change
@@ -1,14 +1,14 @@
/*
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
*
* http://www.apache.org/licenses/LICENSE-2.0
*
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
Expand Down Expand Up @@ -49,6 +49,15 @@ public int intValue() {
return id;
}

/**
* This method is solely used by the jackson json inspector to serialize an {@link ID} object.
*
* @return Int value of this id.
*/
public int getValue() {
return id;
}

@Override
public boolean equals(Object obj) {
return (obj instanceof ID) && ((ID)obj).id == id;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,16 +18,17 @@
*/
package org.apache.parquet.schema;

import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY;
import static org.apache.parquet.schema.OriginalType.LIST;
import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.*;
import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32;
import static org.apache.parquet.schema.Type.Repetition.OPTIONAL;
import static org.apache.parquet.schema.Type.Repetition.REPEATED;
import static org.apache.parquet.schema.Type.Repetition.REQUIRED;

import org.apache.parquet.io.InvalidRecordException;
import org.junit.Test;

import org.apache.parquet.example.Paper;
Expand Down Expand Up @@ -200,4 +201,66 @@ public void testIDs() throws Exception {
assertEquals(schema, schema2);
assertEquals(schema.toString(), schema2.toString());
}

@Test
public void testDecorateProjectionSchema() {
MessageType schema = new MessageType("Document",
new PrimitiveType(REQUIRED, INT64, "DocId").withId(1),
new GroupType(OPTIONAL, "Links",
new PrimitiveType(REPEATED, INT64, "Backward").withId(1),
new PrimitiveType(REPEATED, INT64, "Forward").withId(2)
).withId(2),
new GroupType(REPEATED, "Name",
new GroupType(REPEATED, "Language",
new PrimitiveType(REQUIRED, BINARY, "Code").withId(1),
new PrimitiveType(OPTIONAL, BINARY, "Country").withId(2))
.withId(1),
new PrimitiveType(OPTIONAL, BINARY, "Url").withId(2))
.withId(3));

MessageType projection = new MessageType("Document",
new PrimitiveType(REQUIRED, INT64, "DocId"),
new GroupType(REPEATED, "Name",
new GroupType(REPEATED, "Language",
new PrimitiveType(OPTIONAL, BINARY, "Country"))));

MessageType projectionWithId = schema.checkSubTypeAndDecorateWithId(projection);
projection.checkContains(projectionWithId);
projectionWithId.checkContains(projection);
assertFieldIdNotNull(projectionWithId);
}

@Test(expected = InvalidRecordException.class)
public void testCheckProjectionSchema() {
MessageType schema = new MessageType("Document",
new PrimitiveType(REQUIRED, INT64, "DocId").withId(1),
new GroupType(OPTIONAL, "Links",
new PrimitiveType(REPEATED, INT64, "Backward").withId(1),
new PrimitiveType(REPEATED, INT64, "Forward").withId(2)
).withId(2),
new GroupType(REPEATED, "Name",
new GroupType(REPEATED, "Language",
new PrimitiveType(REQUIRED, BINARY, "Code").withId(1),
new PrimitiveType(OPTIONAL, BINARY, "Country").withId(2))
.withId(1),
new PrimitiveType(OPTIONAL, BINARY, "Url").withId(2))
.withId(3));

MessageType projection = new MessageType("Document",
new PrimitiveType(REQUIRED, INT64, "DocId"),
new GroupType(REPEATED, "Name",
new GroupType(REQUIRED, "Language", // repetition wrong
new PrimitiveType(OPTIONAL, BINARY, "Country"))));

schema.checkSubTypeAndDecorateWithId(projection);
}

private void assertFieldIdNotNull(GroupType projectionWithId) {
for (Type field : projectionWithId.getFields()) {
assertTrue(field.getId() != null);
if (!field.isPrimitive()) {
assertFieldIdNotNull(field.asGroupType());
}
}
}
}
Original file line number Diff line number Diff line change
@@ -1,14 +1,14 @@
/*
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
*
* http://www.apache.org/licenses/LICENSE-2.0
*
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
Expand Down Expand Up @@ -36,6 +36,7 @@
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;

import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.parquet.CorruptStatistics;
import org.apache.parquet.format.PageEncodingStats;
Expand All @@ -62,14 +63,9 @@
import org.apache.parquet.column.EncodingStats;
import org.apache.parquet.hadoop.metadata.ParquetMetadata;
import org.apache.parquet.io.ParquetDecodingException;
import org.apache.parquet.schema.GroupType;
import org.apache.parquet.schema.MessageType;
import org.apache.parquet.schema.OriginalType;
import org.apache.parquet.schema.PrimitiveType;
import org.apache.parquet.schema.*;
import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName;
import org.apache.parquet.schema.Type.Repetition;
import org.apache.parquet.schema.TypeVisitor;
import org.apache.parquet.schema.Types;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand All @@ -80,6 +76,7 @@ public class ParquetMetadataConverter {
public static final MetadataFilter NO_FILTER = new NoFilter();
public static final MetadataFilter SKIP_ROW_GROUPS = new SkipMetadataFilter();
public static final long MAX_STATS_SIZE = 4096; // limit stats to 4k
public static final String PARQUET_SCHEMA_FIELD_WITH_ID = "parquet.schema.field.with.id";

private static final Logger LOG = LoggerFactory.getLogger(ParquetMetadataConverter.class);

Expand Down Expand Up @@ -115,7 +112,7 @@ public FileMetaData toParquetMetadata(int currentVersion, ParquetMetadata parque
}
FileMetaData fileMetaData = new FileMetaData(
currentVersion,
toParquetSchema(parquetMetadata.getFileMetaData().getSchema()),
toParquetSchema(parquetMetadata.getFileMetaData()),
numRows,
rowGroups);

Expand All @@ -129,17 +126,18 @@ public FileMetaData toParquetMetadata(int currentVersion, ParquetMetadata parque
}

// Visible for testing
List<SchemaElement> toParquetSchema(MessageType schema) {
List<SchemaElement> toParquetSchema(org.apache.parquet.hadoop.metadata.FileMetaData fileMetaData) {
List<SchemaElement> result = new ArrayList<SchemaElement>();
addToList(result, schema);
String withId = fileMetaData.getKeyValueMetaData().get(PARQUET_SCHEMA_FIELD_WITH_ID);
addToList(result, fileMetaData.getSchema(), !StringUtils.isBlank(withId) && Boolean.valueOf(withId));
return result;
}

private void addToList(final List<SchemaElement> result, org.apache.parquet.schema.Type field) {
private void addToList(final List<SchemaElement> result, final org.apache.parquet.schema.Type field, final boolean withId) {
field.accept(new TypeVisitor() {
@Override
public void visit(PrimitiveType primitiveType) {
SchemaElement element = new SchemaElement(primitiveType.getName());
SchemaElement element = schemaElementfromField(primitiveType, withId);
element.setRepetition_type(toParquetRepetition(primitiveType.getRepetition()));
element.setType(getType(primitiveType.getPrimitiveTypeName()));
if (primitiveType.getOriginalType() != null) {
Expand All @@ -163,7 +161,7 @@ public void visit(MessageType messageType) {

@Override
public void visit(GroupType groupType) {
SchemaElement element = new SchemaElement(groupType.getName());
SchemaElement element = schemaElementfromField(groupType, withId);
element.setRepetition_type(toParquetRepetition(groupType.getRepetition()));
if (groupType.getOriginalType() != null) {
element.setConverted_type(getConvertedType(groupType.getOriginalType()));
Expand All @@ -176,12 +174,30 @@ private void visitChildren(final List<SchemaElement> result,
element.setNum_children(groupType.getFieldCount());
result.add(element);
for (org.apache.parquet.schema.Type field : groupType.getFields()) {
addToList(result, field);
addToList(result, field, withId);
}
}
});
}

/**
* Build a {@link SchemaElement} from {@link org.apache.parquet.schema.Type} with the field's name, and keep the field
* id if the field has one.
*
* @param field a field of the parquet schema
* @return SchemaElement
*/
private static SchemaElement schemaElementfromField(org.apache.parquet.schema.Type field, boolean withId) {
SchemaElement element = new SchemaElement(field.getName());
if (withId) {
if (field.getId() == null) {
throw new InvalidSchemaException("Field id is required, but not found in " + field.toString());
}
element.setField_id(field.getId().intValue());
}
return element;
}

private void addRowGroup(ParquetMetadata parquetMetadata, List<RowGroup> rowGroups, BlockMetaData block) {
//rowGroup.total_byte_size = ;
List<ColumnChunkMetaData> columns = block.getColumns();
Expand Down
Loading