forked from apache/parquet-java
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
PARQUET-363: Allow empty schema groups.
This removes the check added in PARQUET-278 that rejects schema groups that have no fields. Selecting 0 columns from a file is allowed and used by Hive and SparkSQL to implement queries like `select count(1) ...` Author: Ryan Blue <[email protected]> Closes apache#263 from rdblue/PARQUET-363-allow-empty-groups and squashes the following commits: ab370f1 [Ryan Blue] PARQUET-363: Update Type builder tests to allow empty groups. 926932b [Ryan Blue] PARQUET-363: Add write-side schema validation. 365f30d [Ryan Blue] PARQUET-363: Allow empty schema groups.
- Loading branch information
Showing
11 changed files
with
307 additions
and
50 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
45 changes: 45 additions & 0 deletions
45
parquet-column/src/main/java/org/apache/parquet/schema/TypeUtil.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,45 @@ | ||
/** | ||
* 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 | ||
* KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations | ||
* under the License. | ||
*/ | ||
package org.apache.parquet.schema; | ||
|
||
public class TypeUtil { | ||
public static void checkValidWriteSchema(GroupType schema) { | ||
schema.accept(new TypeVisitor() { | ||
@Override | ||
public void visit(GroupType groupType) { | ||
if (groupType.getFieldCount() <= 0) { | ||
throw new InvalidSchemaException( | ||
"Cannot write a schema with an empty group: " + groupType); | ||
} | ||
for (Type type : groupType.getFields()) { | ||
type.accept(this); | ||
} | ||
} | ||
|
||
@Override | ||
public void visit(MessageType messageType) { | ||
visit((GroupType) messageType); | ||
} | ||
|
||
@Override | ||
public void visit(PrimitiveType primitiveType) { | ||
} | ||
}); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
89 changes: 89 additions & 0 deletions
89
parquet-column/src/test/java/org/apache/parquet/schema/TestTypeUtil.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,89 @@ | ||
/** | ||
* 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 | ||
* KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations | ||
* under the License. | ||
*/ | ||
package org.apache.parquet.schema; | ||
|
||
import org.junit.Test; | ||
|
||
import java.util.concurrent.Callable; | ||
|
||
import static org.apache.parquet.schema.OriginalType.UTF8; | ||
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.REPEATED; | ||
|
||
public class TestTypeUtil { | ||
@Test | ||
public void testWriteCheckMessageType() { | ||
TypeUtil.checkValidWriteSchema(Types.buildMessage() | ||
.required(INT32).named("a") | ||
.optional(BINARY).as(UTF8).named("b") | ||
.named("valid_schema")); | ||
|
||
TestTypeBuilders.assertThrows("Should complain about empty MessageType", | ||
InvalidSchemaException.class, | ||
new Callable<Void>() { | ||
@Override | ||
public Void call() throws Exception { | ||
TypeUtil.checkValidWriteSchema(new MessageType("invalid_schema")); | ||
return null; | ||
} | ||
}); | ||
} | ||
|
||
@Test | ||
public void testWriteCheckGroupType() { | ||
TypeUtil.checkValidWriteSchema(Types.repeatedGroup() | ||
.required(INT32).named("a") | ||
.optional(BINARY).as(UTF8).named("b") | ||
.named("valid_group")); | ||
|
||
TestTypeBuilders.assertThrows("Should complain about empty GroupType", | ||
InvalidSchemaException.class, | ||
new Callable<Void>() { | ||
@Override | ||
public Void call() throws Exception { | ||
TypeUtil.checkValidWriteSchema( | ||
new GroupType(REPEATED, "invalid_group")); | ||
return null; | ||
} | ||
}); | ||
} | ||
|
||
@Test | ||
public void testWriteCheckNestedGroupType() { | ||
TypeUtil.checkValidWriteSchema(Types.buildMessage() | ||
.repeatedGroup() | ||
.required(INT32).named("a") | ||
.optional(BINARY).as(UTF8).named("b") | ||
.named("valid_group") | ||
.named("valid_message")); | ||
|
||
TestTypeBuilders.assertThrows("Should complain about empty GroupType", | ||
InvalidSchemaException.class, | ||
new Callable<Void>() { | ||
@Override | ||
public Void call() throws Exception { | ||
TypeUtil.checkValidWriteSchema(Types.buildMessage() | ||
.addField(new GroupType(REPEATED, "invalid_group")) | ||
.named("invalid_message")); | ||
return null; | ||
} | ||
}); | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
93 changes: 93 additions & 0 deletions
93
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/example/ExampleParquetWriter.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,93 @@ | ||
/** | ||
* 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 | ||
* KIND, either express or implied. See the License for the | ||
* specific language governing permissions and limitations | ||
* under the License. | ||
*/ | ||
package org.apache.parquet.hadoop.example; | ||
|
||
import org.apache.hadoop.conf.Configuration; | ||
import org.apache.hadoop.fs.Path; | ||
import org.apache.parquet.column.ParquetProperties; | ||
import org.apache.parquet.example.data.Group; | ||
import org.apache.parquet.hadoop.ParquetWriter; | ||
import org.apache.parquet.hadoop.api.WriteSupport; | ||
import org.apache.parquet.hadoop.metadata.CompressionCodecName; | ||
import org.apache.parquet.schema.MessageType; | ||
import java.io.IOException; | ||
|
||
/** | ||
* An example file writer class. | ||
* THIS IS AN EXAMPLE ONLY AND NOT INTENDED FOR USE. | ||
*/ | ||
public class ExampleParquetWriter extends ParquetWriter<Group> { | ||
|
||
/** | ||
* Creates a Builder for configuring ParquetWriter with the example object | ||
* model. THIS IS AN EXAMPLE ONLY AND NOT INTENDED FOR USE. | ||
* | ||
* @param file the output file to create | ||
* @return a {@link Builder} to create a {@link ParquetWriter} | ||
*/ | ||
public static Builder builder(Path file) { | ||
return new Builder(file); | ||
} | ||
|
||
/** | ||
* Create a new {@link ExampleParquetWriter}. | ||
* | ||
* @param file The file name to write to. | ||
* @param writeSupport The schema to write with. | ||
* @param compressionCodecName Compression code to use, or CompressionCodecName.UNCOMPRESSED | ||
* @param blockSize the block size threshold. | ||
* @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. | ||
* @param conf The Configuration to use. | ||
* @throws IOException | ||
*/ | ||
ExampleParquetWriter(Path file, WriteSupport<Group> writeSupport, | ||
CompressionCodecName compressionCodecName, | ||
int blockSize, int pageSize, boolean enableDictionary, | ||
boolean enableValidation, | ||
ParquetProperties.WriterVersion writerVersion, | ||
Configuration conf) | ||
throws IOException { | ||
super(file, writeSupport, compressionCodecName, blockSize, pageSize, | ||
pageSize, enableDictionary, enableValidation, writerVersion, conf); | ||
} | ||
|
||
public static class Builder extends ParquetWriter.Builder<Group, Builder> { | ||
private MessageType type = null; | ||
|
||
private Builder(Path file) { | ||
super(file); | ||
} | ||
|
||
public Builder withType(MessageType type) { | ||
this.type = type; | ||
return this; | ||
} | ||
|
||
@Override | ||
protected Builder self() { | ||
return this; | ||
} | ||
|
||
@Override | ||
protected WriteSupport<Group> getWriteSupport(Configuration conf) { | ||
return new GroupWriteSupport(type); | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.