Skip to content

Commit e05447e

Browse files
Add Statistics Test for Parquet Columns
1 parent 2f956f4 commit e05447e

File tree

3 files changed

+688
-0
lines changed

3 files changed

+688
-0
lines changed
Lines changed: 82 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,82 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing,
13+
* software distributed under the License is distributed on an
14+
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
15+
* KIND, either express or implied. See the License for the
16+
* specific language governing permissions and limitations
17+
* under the License.
18+
*/
19+
20+
package org.apache.parquet.statistics;
21+
22+
import org.apache.hadoop.conf.Configuration;
23+
import org.apache.hadoop.fs.Path;
24+
import org.apache.parquet.column.ParquetProperties;
25+
import org.apache.parquet.example.data.Group;
26+
import org.apache.parquet.hadoop.ParquetWriter;
27+
import org.apache.parquet.hadoop.example.GroupWriteSupport;
28+
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
29+
import org.apache.parquet.schema.MessageType;
30+
31+
import java.io.IOException;
32+
33+
public class DataGenerationContext {
34+
public static abstract class WriteContext {
35+
protected final String path;
36+
protected final Path fsPath;
37+
protected final MessageType schema;
38+
protected final int blockSize;
39+
protected final int pageSize;
40+
protected final boolean enableDictionary;
41+
protected final boolean enableValidation;
42+
protected final ParquetProperties.WriterVersion version;
43+
44+
public WriteContext(String path, MessageType schema, int blockSize, int pageSize, boolean enableDictionary, boolean enableValidation, ParquetProperties.WriterVersion version) throws IOException {
45+
this.path = path;
46+
this.fsPath = new Path(path);
47+
this.schema = schema;
48+
this.blockSize = blockSize;
49+
this.pageSize = pageSize;
50+
this.enableDictionary = enableDictionary;
51+
this.enableValidation = enableValidation;
52+
this.version = version;
53+
}
54+
55+
public abstract void write(ParquetWriter<Group> writer) throws IOException;
56+
public abstract void test() throws IOException;
57+
}
58+
59+
public static void write(WriteContext context) throws IOException {
60+
// Create the configuration, and then apply the schema to our configuration.
61+
Configuration configuration = new Configuration();
62+
GroupWriteSupport.setSchema(context.schema, configuration);
63+
GroupWriteSupport groupWriteSupport = new GroupWriteSupport();
64+
65+
// Create the writer properties
66+
final int blockSize = context.blockSize;
67+
final int pageSize = context.pageSize;
68+
final int dictionaryPageSize = pageSize;
69+
final boolean enableDictionary = context.enableDictionary;
70+
final boolean enableValidation = context.enableValidation;
71+
ParquetProperties.WriterVersion writerVersion = context.version;
72+
CompressionCodecName codec = CompressionCodecName.UNCOMPRESSED;
73+
74+
ParquetWriter<Group> writer = new ParquetWriter<Group>(context.fsPath, groupWriteSupport, codec, blockSize,
75+
pageSize, dictionaryPageSize, enableDictionary, enableValidation, writerVersion, configuration);
76+
77+
context.write(writer);
78+
writer.close();
79+
80+
context.test();
81+
}
82+
}

0 commit comments

Comments
 (0)