Skip to content

Commit fb59f44

Browse files
committed
apacheGH-3125: Add CLI for SizeStatistics
1 parent 7a99d86 commit fb59f44

File tree

5 files changed

+167
-4
lines changed

5 files changed

+167
-4
lines changed

parquet-cli/src/main/java/org/apache/parquet/cli/Main.java

+2
Original file line numberDiff line numberDiff line change
@@ -51,6 +51,7 @@
5151
import org.apache.parquet.cli.commands.ShowDictionaryCommand;
5252
import org.apache.parquet.cli.commands.ShowFooterCommand;
5353
import org.apache.parquet.cli.commands.ShowPagesCommand;
54+
import org.apache.parquet.cli.commands.ShowSizeStatisticsCommand;
5455
import org.apache.parquet.cli.commands.ToAvroCommand;
5556
import org.apache.parquet.cli.commands.TransCompressionCommand;
5657
import org.slf4j.Logger;
@@ -105,6 +106,7 @@ public class Main extends Configured implements Tool {
105106
jc.addCommand("bloom-filter", new ShowBloomFilterCommand(console));
106107
jc.addCommand("scan", new ScanCommand(console));
107108
jc.addCommand("rewrite", new RewriteCommand(console));
109+
jc.addCommand("size-stats", new ShowSizeStatisticsCommand(console));
108110
}
109111

110112
@Override
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,116 @@
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.cli.commands;
21+
22+
import static org.apache.parquet.cli.Util.humanReadable;
23+
24+
import com.beust.jcommander.Parameter;
25+
import com.beust.jcommander.Parameters;
26+
import com.google.common.base.Preconditions;
27+
import com.google.common.collect.Lists;
28+
import java.io.IOException;
29+
import java.util.List;
30+
import org.apache.commons.text.TextStringBuilder;
31+
import org.apache.parquet.cli.BaseCommand;
32+
import org.apache.parquet.column.statistics.SizeStatistics;
33+
import org.apache.parquet.hadoop.ParquetFileReader;
34+
import org.apache.parquet.hadoop.metadata.BlockMetaData;
35+
import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
36+
import org.apache.parquet.hadoop.metadata.ParquetMetadata;
37+
import org.apache.parquet.schema.MessageType;
38+
import org.slf4j.Logger;
39+
40+
@Parameters(commandDescription = "Print size statistics for a Parquet file")
41+
public class ShowSizeStatisticsCommand extends BaseCommand {
42+
43+
public ShowSizeStatisticsCommand(Logger console) {
44+
super(console);
45+
}
46+
47+
@Parameter(description = "<parquet path>")
48+
List<String> targets;
49+
50+
@Override
51+
@SuppressWarnings("unchecked")
52+
public int run() throws IOException {
53+
Preconditions.checkArgument(targets != null && !targets.isEmpty(), "A Parquet file is required.");
54+
Preconditions.checkArgument(targets.size() == 1, "Cannot process multiple Parquet files.");
55+
56+
String source = targets.get(0);
57+
try (ParquetFileReader reader = ParquetFileReader.open(getConf(), qualifiedPath(source))) {
58+
ParquetMetadata footer = reader.getFooter();
59+
MessageType schema = footer.getFileMetaData().getSchema();
60+
61+
console.info("\nFile path: {}", source);
62+
63+
List<BlockMetaData> rowGroups = footer.getBlocks();
64+
for (int index = 0, n = rowGroups.size(); index < n; index++) {
65+
printRowGroupSizeStats(console, index, rowGroups.get(index), schema);
66+
console.info("");
67+
}
68+
}
69+
70+
return 0;
71+
}
72+
73+
private void printRowGroupSizeStats(Logger console, int index, BlockMetaData rowGroup, MessageType schema) {
74+
int maxColumnWidth = Math.max(
75+
"column".length(),
76+
rowGroup.getColumns().stream()
77+
.map(col -> col.getPath().toString().length())
78+
.max(Integer::compare)
79+
.orElse(0));
80+
81+
console.info(String.format("\nRow group %d\n%s", index, new TextStringBuilder(80).appendPadding(80, '-')));
82+
83+
String formatString = String.format("%%-%ds %%-15s %%-40s %%-40s", maxColumnWidth);
84+
console.info(
85+
String.format(formatString, "column", "unencoded bytes", "rep level histogram", "def level histogram"));
86+
87+
for (ColumnChunkMetaData column : rowGroup.getColumns()) {
88+
printColumnSizeStats(console, column, schema, maxColumnWidth);
89+
}
90+
}
91+
92+
private void printColumnSizeStats(Logger console, ColumnChunkMetaData column, MessageType schema, int columnWidth) {
93+
SizeStatistics stats = column.getSizeStatistics();
94+
95+
if (stats != null && stats.isValid()) {
96+
String unencodedBytes = stats.getUnencodedByteArrayDataBytes().isPresent()
97+
? humanReadable(stats.getUnencodedByteArrayDataBytes().get())
98+
: "-";
99+
List<Long> repLevels = stats.getRepetitionLevelHistogram();
100+
String repLevelsString = (repLevels != null && !repLevels.isEmpty()) ? repLevels.toString() : "-";
101+
List<Long> defLevels = stats.getDefinitionLevelHistogram();
102+
String defLevelsString = (defLevels != null && !defLevels.isEmpty()) ? defLevels.toString() : "-";
103+
String formatString = String.format("%%-%ds %%-15s %%-40s %%-40s", columnWidth);
104+
console.info(
105+
String.format(formatString, column.getPath(), unencodedBytes, repLevelsString, defLevelsString));
106+
} else {
107+
String formatString = String.format("%%-%ds %%-15s %%-40s %%-40s", columnWidth);
108+
console.info(String.format(formatString, column.getPath(), "-", "-", "-"));
109+
}
110+
}
111+
112+
@Override
113+
public List<String> getExamples() {
114+
return Lists.newArrayList("# Show size statistics for a Parquet file", "sample.parquet");
115+
}
116+
}
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,37 @@
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+
package org.apache.parquet.cli.commands;
20+
21+
import java.io.File;
22+
import java.io.IOException;
23+
import java.util.Arrays;
24+
import org.apache.hadoop.conf.Configuration;
25+
import org.junit.Assert;
26+
import org.junit.Test;
27+
28+
public class ShowSizeStatisticsCommandTest extends ParquetFileTest {
29+
@Test
30+
public void testShowSizeStatisticsCommand() throws IOException {
31+
File file = parquetFile();
32+
ShowSizeStatisticsCommand command = new ShowSizeStatisticsCommand(createLogger());
33+
command.targets = Arrays.asList(file.getAbsolutePath());
34+
command.setConf(new Configuration());
35+
Assert.assertEquals(0, command.run());
36+
}
37+
}

parquet-column/src/main/java/org/apache/parquet/column/statistics/SizeStatistics.java

+4-2
Original file line numberDiff line numberDiff line change
@@ -136,8 +136,10 @@ public SizeStatistics(
136136
List<Long> definitionLevelHistogram) {
137137
this.type = type;
138138
this.unencodedByteArrayDataBytes = unencodedByteArrayDataBytes;
139-
this.repetitionLevelHistogram = repetitionLevelHistogram;
140-
this.definitionLevelHistogram = definitionLevelHistogram;
139+
this.repetitionLevelHistogram =
140+
repetitionLevelHistogram == null ? Collections.emptyList() : repetitionLevelHistogram;
141+
this.definitionLevelHistogram =
142+
definitionLevelHistogram == null ? Collections.emptyList() : definitionLevelHistogram;
141143
}
142144

143145
/**

parquet-hadoop/src/main/java/org/apache/parquet/format/converter/ParquetMetadataConverter.java

+8-2
Original file line numberDiff line numberDiff line change
@@ -2382,8 +2382,14 @@ public static SizeStatistics toParquetSizeStatistics(org.apache.parquet.column.s
23822382
formatStats.setUnencoded_byte_array_data_bytes(
23832383
stats.getUnencodedByteArrayDataBytes().get());
23842384
}
2385-
formatStats.setRepetition_level_histogram(stats.getRepetitionLevelHistogram());
2386-
formatStats.setDefinition_level_histogram(stats.getDefinitionLevelHistogram());
2385+
List<Long> repLevelHistogram = stats.getRepetitionLevelHistogram();
2386+
if (repLevelHistogram != null && !repLevelHistogram.isEmpty()) {
2387+
formatStats.setRepetition_level_histogram(repLevelHistogram);
2388+
}
2389+
List<Long> defLevelHistogram = stats.getDefinitionLevelHistogram();
2390+
if (defLevelHistogram != null && !defLevelHistogram.isEmpty()) {
2391+
formatStats.setDefinition_level_histogram(defLevelHistogram);
2392+
}
23872393
return formatStats;
23882394
}
23892395
}

0 commit comments

Comments
 (0)