|
18 | 18 | */ |
19 | 19 | package org.apache.parquet.cli; |
20 | 20 |
|
| 21 | +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY; |
| 22 | +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BOOLEAN; |
| 23 | +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.DOUBLE; |
| 24 | +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY; |
| 25 | +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FLOAT; |
| 26 | +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32; |
| 27 | +import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64; |
| 28 | +import static org.apache.parquet.schema.Type.Repetition.OPTIONAL; |
| 29 | +import static org.apache.parquet.schema.Type.Repetition.REPEATED; |
| 30 | +import static org.apache.parquet.schema.Type.Repetition.REQUIRED; |
| 31 | + |
21 | 32 | import java.io.File; |
| 33 | +import java.io.IOException; |
| 34 | +import java.util.Random; |
| 35 | +import org.apache.hadoop.conf.Configuration; |
| 36 | +import org.apache.hadoop.fs.Path; |
22 | 37 | import org.apache.parquet.cli.testing.CliTestBase; |
| 38 | +import org.apache.parquet.cli.testing.CliResult; |
| 39 | +import org.apache.parquet.example.data.simple.SimpleGroup; |
| 40 | +import org.apache.parquet.io.api.Binary; |
| 41 | +import org.apache.parquet.hadoop.ParquetWriter; |
| 42 | +import org.apache.parquet.hadoop.example.ExampleParquetWriter; |
| 43 | +import org.apache.parquet.hadoop.example.GroupWriteSupport; |
| 44 | +import org.apache.parquet.schema.MessageType; |
| 45 | +import org.apache.parquet.schema.PrimitiveType; |
23 | 46 | import org.junit.Test; |
24 | 47 |
|
25 | 48 | public class ShowSizeStatisticsCliTest extends CliTestBase { |
26 | 49 |
|
| 50 | + private final int numRecord = 10000; |
| 51 | + |
27 | 52 | @Test |
28 | 53 | public void showSizeStatistics() throws Exception { |
29 | | - File file = parquetFile(); |
| 54 | + File file = createParquetFileWithStats(); |
30 | 55 |
|
31 | 56 | cli("size-stats " + file.getAbsolutePath()) |
32 | 57 | .ok() |
33 | 58 | .matchOutputFromFile("src/test/resources/cli-outputs/size-stats.txt"); |
34 | 59 | } |
35 | 60 |
|
| 61 | + private File createParquetFileWithStats() throws IOException { |
| 62 | + MessageType schema = new MessageType( |
| 63 | + "schema", |
| 64 | + new PrimitiveType(REQUIRED, INT64, "DocId"), |
| 65 | + new PrimitiveType(REQUIRED, INT32, "CategoryId"), |
| 66 | + new PrimitiveType(OPTIONAL, BOOLEAN, "IsActive"), |
| 67 | + new PrimitiveType(REPEATED, FLOAT, "Prices"), |
| 68 | + new PrimitiveType(REPEATED, BINARY, "Tags"), |
| 69 | + new PrimitiveType(REQUIRED, BINARY, "ProductName"), |
| 70 | + new PrimitiveType(OPTIONAL, BINARY, "Description"), |
| 71 | + new PrimitiveType(REQUIRED, FIXED_LEN_BYTE_ARRAY, 16, "UUID")); |
| 72 | + |
| 73 | + Configuration conf = new Configuration(); |
| 74 | + conf.set(GroupWriteSupport.PARQUET_EXAMPLE_SCHEMA, schema.toString()); |
| 75 | + |
| 76 | + File file = new File(getTempFolder(), "test.parquet"); |
| 77 | + String filePath = file.getAbsolutePath(); |
| 78 | + ExampleParquetWriter.Builder builder = |
| 79 | + ExampleParquetWriter.builder(new Path(filePath)) |
| 80 | + .withType(schema) |
| 81 | + .withSizeStatisticsEnabled(true) |
| 82 | + .withPageRowCountLimit(50) |
| 83 | + .withMinRowCountForPageSizeCheck(5) |
| 84 | + .withDictionaryEncoding(true) |
| 85 | + .withValidation(false) |
| 86 | + .withConf(conf); |
| 87 | + |
| 88 | + Random rnd = new Random(42); |
| 89 | + try (ParquetWriter writer = builder.build()) { |
| 90 | + for (int i = 0; i < numRecord; i++) { |
| 91 | + SimpleGroup g = new SimpleGroup(schema); |
| 92 | + |
| 93 | + g.add("DocId", rnd.nextLong()); |
| 94 | + |
| 95 | + g.add("CategoryId", rnd.nextInt(100)); |
| 96 | + |
| 97 | + // Operations to generate some non null meaningful test statistics on the parquet file. |
| 98 | + if (i % 4 != 0) { |
| 99 | + g.add("IsActive", rnd.nextBoolean()); |
| 100 | + } |
| 101 | + |
| 102 | + int priceCount = rnd.nextInt(4); |
| 103 | + for (int p = 0; p < priceCount; p++) { |
| 104 | + g.add("Prices", rnd.nextFloat() * 1000); |
| 105 | + } |
| 106 | + |
| 107 | + String[] possibleTags = {"electronics", "bestseller", "new", "discount", "premium"}; |
| 108 | + int tagCount = rnd.nextInt(5); |
| 109 | + for (int t = 0; t < tagCount; t++) { |
| 110 | + g.add("Tags", Binary.fromString(possibleTags[rnd.nextInt(possibleTags.length)])); |
| 111 | + } |
| 112 | + |
| 113 | + String[] products = {"Laptop", "Mouse", "Keyboard", "Monitor", "Headphones", |
| 114 | + "Smartphone", "Tablet", "Camera", "Printer", "Speaker"}; |
| 115 | + g.add("ProductName", Binary.fromString(products[i % products.length] + "_Model_" + (i % 50))); |
| 116 | + |
| 117 | + if (i % 3 != 0) { |
| 118 | + StringBuilder desc = new StringBuilder(); |
| 119 | + desc.append("Product description for item ").append(i).append(": "); |
| 120 | + int descLength = rnd.nextInt(200) + 50; |
| 121 | + for (int j = 0; j < descLength; j++) { |
| 122 | + desc.append((char) ('a' + rnd.nextInt(26))); |
| 123 | + } |
| 124 | + g.add("Description", Binary.fromString(desc.toString())); |
| 125 | + } |
| 126 | + |
| 127 | + byte[] uuid = new byte[16]; |
| 128 | + rnd.nextBytes(uuid); |
| 129 | + g.add("UUID", Binary.fromConstantByteArray(uuid)); |
| 130 | + |
| 131 | + writer.write(g); |
| 132 | + } |
| 133 | + } |
| 134 | + |
| 135 | + return file; |
| 136 | + } |
| 137 | + |
36 | 138 | @Test |
37 | 139 | public void showsHelpMessage() throws Exception { |
38 | 140 | cli("help size-stats").ok().matchOutputFromFile("src/test/resources/cli-outputs/size-stats-help.txt"); |
39 | 141 | } |
40 | 142 |
|
41 | | - @Test |
42 | | - public void showsSchemaOutput() throws Exception { |
43 | | - File file = parquetFile(); |
44 | | - cli("schema " + file.getAbsolutePath()) |
45 | | - .ok() |
46 | | - .matchOutputFromFile("src/test/resources/cli-outputs/size-stats-column.txt"); |
47 | | - } |
48 | 143 | } |
0 commit comments