This is an automated email from the ASF dual-hosted git repository.
gangwu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/parquet-java.git
The following commit(s) were added to refs/heads/master by this push:
new 4b6fbc1fb GH-3273: Add scoped chunk level statistics to avoid
unbounded output (#3274)
4b6fbc1fb is described below
commit 4b6fbc1fb636f5553416b6bfd9ce7767ed058bbb
Author: Arnav Balyan <[email protected]>
AuthorDate: Sun Aug 31 07:12:57 2025 +0530
GH-3273: Add scoped chunk level statistics to avoid unbounded output (#3274)
---
.../cli/commands/ShowSizeStatisticsCommand.java | 45 ++++++++++++++++++++--
.../commands/ShowSizeStatisticsCommandTest.java | 20 ++++++++++
2 files changed, 61 insertions(+), 4 deletions(-)
diff --git
a/parquet-cli/src/main/java/org/apache/parquet/cli/commands/ShowSizeStatisticsCommand.java
b/parquet-cli/src/main/java/org/apache/parquet/cli/commands/ShowSizeStatisticsCommand.java
index 0821d260e..1f50509a0 100644
---
a/parquet-cli/src/main/java/org/apache/parquet/cli/commands/ShowSizeStatisticsCommand.java
+++
b/parquet-cli/src/main/java/org/apache/parquet/cli/commands/ShowSizeStatisticsCommand.java
@@ -26,7 +26,11 @@ import com.beust.jcommander.Parameters;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import java.io.IOException;
+import java.util.Collection;
+import java.util.HashSet;
import java.util.List;
+import java.util.Set;
+import javax.annotation.Nullable;
import org.apache.commons.text.TextStringBuilder;
import org.apache.parquet.cli.BaseCommand;
import org.apache.parquet.column.statistics.SizeStatistics;
@@ -40,6 +44,15 @@ import org.slf4j.Logger;
@Parameters(commandDescription = "Print size statistics for a Parquet file")
public class ShowSizeStatisticsCommand extends BaseCommand {
+ private static <T> Set<T> filterOrNull(Collection<T> values) {
+ if (values == null || values.isEmpty()) return null;
+ return new HashSet<>(values);
+ }
+
+ private static boolean includesOrAll(@Nullable Set<?> filter, Object value) {
+ return filter == null || filter.contains(value);
+ }
+
public ShowSizeStatisticsCommand(Logger console) {
super(console);
}
@@ -47,6 +60,16 @@ public class ShowSizeStatisticsCommand extends BaseCommand {
@Parameter(description = "<parquet path>")
List<String> targets;
+ @Parameter(
+ names = {"-c", "--column", "--columns"},
+ description = "List of columns (dot paths) to include")
+ List<String> columns;
+
+ @Parameter(
+ names = {"-r", "--row-group", "--row-groups"},
+ description = "List of row-group indexes to include (0-based)")
+ List<Integer> rowGroups;
+
@Override
@SuppressWarnings("unchecked")
public int run() throws IOException {
@@ -60,9 +83,13 @@ public class ShowSizeStatisticsCommand extends BaseCommand {
console.info("\nFile path: {}", source);
- List<BlockMetaData> rowGroups = footer.getBlocks();
- for (int index = 0, n = rowGroups.size(); index < n; index++) {
- printRowGroupSizeStats(console, index, rowGroups.get(index), schema);
+ List<BlockMetaData> blocks = footer.getBlocks();
+
+ final Set<Integer> rowGroupFilter = filterOrNull(this.rowGroups);
+
+ for (int i = 0, n = blocks.size(); i < n; i++) {
+ if (!includesOrAll(rowGroupFilter, i)) continue;
+ printRowGroupSizeStats(console, i, blocks.get(i), schema);
console.info("");
}
}
@@ -84,7 +111,11 @@ public class ShowSizeStatisticsCommand extends BaseCommand {
console.info(
String.format(formatString, "column", "unencoded bytes", "rep level
histogram", "def level histogram"));
+ final Set<String> columnFilter = filterOrNull(this.columns);
+
for (ColumnChunkMetaData column : rowGroup.getColumns()) {
+ String dotPath = column.getPath().toDotString();
+ if (!includesOrAll(columnFilter, dotPath)) continue;
printColumnSizeStats(console, column, schema, maxColumnWidth);
}
}
@@ -111,6 +142,12 @@ public class ShowSizeStatisticsCommand extends BaseCommand
{
@Override
public List<String> getExamples() {
- return Lists.newArrayList("# Show size statistics for a Parquet file",
"sample.parquet");
+ return Lists.newArrayList(
+ "# Show size statistics for a Parquet file",
+ "sample.parquet",
+ "# Show size statistics for selected columns",
+ "sample.parquet -c name,tags",
+ "# Show size statistics for a specific row-group",
+ "sample.parquet -r 0");
}
}
diff --git
a/parquet-cli/src/test/java/org/apache/parquet/cli/commands/ShowSizeStatisticsCommandTest.java
b/parquet-cli/src/test/java/org/apache/parquet/cli/commands/ShowSizeStatisticsCommandTest.java
index 55d4f9d6e..ff1733e90 100644
---
a/parquet-cli/src/test/java/org/apache/parquet/cli/commands/ShowSizeStatisticsCommandTest.java
+++
b/parquet-cli/src/test/java/org/apache/parquet/cli/commands/ShowSizeStatisticsCommandTest.java
@@ -34,4 +34,24 @@ public class ShowSizeStatisticsCommandTest extends
ParquetFileTest {
command.setConf(new Configuration());
Assert.assertEquals(0, command.run());
}
+
+ @Test
+ public void testShowSizeStatisticsWithColumnFilter() throws IOException {
+ File file = parquetFile();
+ ShowSizeStatisticsCommand command = new
ShowSizeStatisticsCommand(createLogger());
+ command.targets = Arrays.asList(file.getAbsolutePath());
+ command.columns = Arrays.asList(INT32_FIELD, INT64_FIELD);
+ command.setConf(new Configuration());
+ Assert.assertEquals(0, command.run());
+ }
+
+ @Test
+ public void testShowSizeStatisticsWithRowGroupFilter() throws IOException {
+ File file = parquetFile();
+ ShowSizeStatisticsCommand command = new
ShowSizeStatisticsCommand(createLogger());
+ command.targets = Arrays.asList(file.getAbsolutePath());
+ command.rowGroups = Arrays.asList(0);
+ command.setConf(new Configuration());
+ Assert.assertEquals(0, command.run());
+ }
}