This is an automated email from the ASF dual-hosted git repository.

shangxinli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/parquet-mr.git


The following commit(s) were added to refs/heads/master by this push:
     new e990eb3f1 PARQUET-2138: Add ShowBloomFilterCommand to parquet-cli 
(#958)
e990eb3f1 is described below

commit e990eb3f14c39273e46a9fce07ec85d2edf7fccb
Author: WangGuangxin <[email protected]>
AuthorDate: Sun Jul 3 00:51:40 2022 +0800

    PARQUET-2138: Add ShowBloomFilterCommand to parquet-cli (#958)
    
    * PARQUET-2138: Add ShowBloomFilterCommand to parquet-cli
    
    * address comments
---
 .../src/main/java/org/apache/parquet/cli/Main.java |   2 +
 .../cli/commands/ShowBloomFilterCommand.java       | 133 +++++++++++++++++++++
 .../cli/commands/ShowBloomFilterCommandTest.java   |  41 +++++++
 3 files changed, 176 insertions(+)

diff --git a/parquet-cli/src/main/java/org/apache/parquet/cli/Main.java 
b/parquet-cli/src/main/java/org/apache/parquet/cli/Main.java
index d656476f5..450fa6e01 100644
--- a/parquet-cli/src/main/java/org/apache/parquet/cli/Main.java
+++ b/parquet-cli/src/main/java/org/apache/parquet/cli/Main.java
@@ -35,6 +35,7 @@ import org.apache.parquet.cli.commands.ConvertCommand;
 import org.apache.parquet.cli.commands.ParquetMetadataCommand;
 import org.apache.parquet.cli.commands.PruneColumnsCommand;
 import org.apache.parquet.cli.commands.SchemaCommand;
+import org.apache.parquet.cli.commands.ShowBloomFilterCommand;
 import org.apache.parquet.cli.commands.ShowColumnIndexCommand;
 import org.apache.parquet.cli.commands.ShowDictionaryCommand;
 import org.apache.parquet.cli.commands.ShowFooterCommand;
@@ -99,6 +100,7 @@ public class Main extends Configured implements Tool {
     jc.addCommand("trans-compression", new TransCompressionCommand(console));
     jc.addCommand("masking", new ColumnMaskingCommand(console));
     jc.addCommand("footer", new ShowFooterCommand(console));
+    jc.addCommand("bloom-filter", new ShowBloomFilterCommand(console));
   }
 
   @Override
diff --git 
a/parquet-cli/src/main/java/org/apache/parquet/cli/commands/ShowBloomFilterCommand.java
 
b/parquet-cli/src/main/java/org/apache/parquet/cli/commands/ShowBloomFilterCommand.java
new file mode 100644
index 000000000..82bafde54
--- /dev/null
+++ 
b/parquet-cli/src/main/java/org/apache/parquet/cli/commands/ShowBloomFilterCommand.java
@@ -0,0 +1,133 @@
+/*
+ * 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.cli.commands;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.util.List;
+import java.util.Optional;
+import org.apache.commons.text.TextStringBuilder;
+import org.apache.parquet.cli.BaseCommand;
+import org.apache.parquet.cli.Util;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.hadoop.BloomFilterReader;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.io.InputFile;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.PrimitiveType;
+import org.slf4j.Logger;
+
+@Parameters(commandDescription="Check bloom filters for a Parquet column")
+public class ShowBloomFilterCommand extends BaseCommand {
+
+  public ShowBloomFilterCommand(Logger console) {
+    super(console);
+  }
+
+  @Parameter(description = "<parquet path>")
+  String file;
+
+  @Parameter(
+    names = { "-c", "--column" },
+    description = "Check the bloom filter indexes for the given column",
+    required = true)
+  String columnPath;
+
+  @Parameter(
+    names = { "-v", "--values" },
+    description = "Check if the given values match bloom filter",
+    required = true)
+  List<String> testValues;
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public int run() throws IOException {
+    Preconditions.checkArgument(file != null,
+      "A Parquet file is required.");
+
+    InputFile in = HadoopInputFile.fromPath(qualifiedPath(file), getConf());
+
+    try (ParquetFileReader reader = ParquetFileReader.open(in)) {
+      MessageType schema = reader.getFileMetaData().getSchema();
+      PrimitiveType type = Util.primitive(columnPath, schema);
+
+      int rowGroupIndex = 0;
+      for (BlockMetaData block : reader.getFooter().getBlocks()) {
+        console.info(String.format("\nRow group %d: \n%s",
+          rowGroupIndex,
+          new TextStringBuilder(80).appendPadding(80, '-')));
+
+        Optional<ColumnChunkMetaData> maybeColumnMeta = 
block.getColumns().stream()
+          .filter(c -> 
columnPath.equals(c.getPath().toDotString())).findFirst();
+        if (!maybeColumnMeta.isPresent()) {
+          console.info("column {} doesn't exist.", columnPath);
+        } else {
+          BloomFilterReader bloomFilterReader = 
reader.getBloomFilterDataReader(block);
+          BloomFilter bloomFilter = 
bloomFilterReader.readBloomFilter(maybeColumnMeta.get());
+          if (bloomFilter == null) {
+            console.info("column {} has no bloom filter", columnPath);
+          } else {
+            for (String value: testValues) {
+              if (bloomFilter.findHash(bloomFilter.hash(getOriginalType(value, 
type)))) {
+                console.info("value {} maybe exists.", value);
+              } else {
+                console.info("value {} NOT exists.", value);
+              }
+            }
+          }
+        }
+        ++rowGroupIndex;
+      }
+    }
+    return 0;
+  }
+
+  private Object getOriginalType(String value, PrimitiveType type) {
+    switch(type.getPrimitiveTypeName()) {
+      case BINARY:
+        return Binary.fromString(value);
+      case INT32:
+        return Integer.valueOf(value);
+      case INT64:
+        return Long.valueOf(value);
+      case FLOAT:
+        return Float.valueOf(value);
+      case DOUBLE:
+        return Double.valueOf(value);
+      default:
+        throw new IllegalArgumentException(
+          "Unknown type: " + type.getPrimitiveTypeName());
+    }
+  }
+
+  @Override
+  public List<String> getExamples() {
+    return Lists.newArrayList(
+      "# Show bloom filter for column 'col' from a Parquet file",
+      "-c col -v 1,2,3 -i sample.parquet");
+  }
+}
diff --git 
a/parquet-cli/src/test/java/org/apache/parquet/cli/commands/ShowBloomFilterCommandTest.java
 
b/parquet-cli/src/test/java/org/apache/parquet/cli/commands/ShowBloomFilterCommandTest.java
new file mode 100644
index 000000000..0b36a514f
--- /dev/null
+++ 
b/parquet-cli/src/test/java/org/apache/parquet/cli/commands/ShowBloomFilterCommandTest.java
@@ -0,0 +1,41 @@
+/*
+ * 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.cli.commands;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.Arrays;
+import org.apache.hadoop.conf.Configuration;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class ShowBloomFilterCommandTest extends ParquetFileTest {
+  @Test
+  public void testShowBloomFilterCommand() throws IOException {
+    File file = parquetFile();
+    ShowBloomFilterCommand command = new 
ShowBloomFilterCommand(createLogger());
+    command.file = file.getAbsolutePath();
+    command.columnPath = INT32_FIELD;
+    command.testValues = Arrays.asList(new String[] {"1"});
+    command.setConf(new Configuration());
+    Assert.assertEquals(0, command.run());
+  }
+}
+

Reply via email to