From 68f5f07f8879359bd622e10b1c49e31bf6909d09 Mon Sep 17 00:00:00 2001 From: Junjie Chen Date: Fri, 15 Feb 2019 04:31:34 +0800 Subject: [PATCH 1/3] PARQUET-1391: Integrate Bloom filter logic --- .../values/bloomfilter/BloomFilterReader.java | 10 +- .../bloomfilter/BloomFilterUtility.java | 81 ++++++++ .../BloomFilterLevel/BloomFilterImpl.java | 194 ++++++++++++++++++ .../filter2/compat/RowGroupFilter.java | 18 +- .../parquet/hadoop/BloomFilterDataReader.java | 35 +++- 5 files changed, 331 insertions(+), 7 deletions(-) create mode 100644 parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilterUtility.java create mode 100644 parquet-hadoop/src/main/java/org/apache/parquet/filter2/BloomFilterLevel/BloomFilterImpl.java diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilterReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilterReader.java index 7a430581dd..02188fe42f 100644 --- a/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilterReader.java +++ b/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilterReader.java @@ -26,7 +26,15 @@ public interface BloomFilterReader { * Returns a {@link BloomFilter} for the given column descriptor. * * @param path the descriptor of the column - * @return the bloomFilter dta for that column, or null if there isn't one + * @return the bloomFilter data for the column, or null if there isn't one */ BloomFilter readBloomFilter(ColumnDescriptor path); + + /** + * Returns a {@link BloomFilterUtility} for the given column descriptor. + * + * @param path the descriptor of the column + * @return a concrete bloom filter utility + */ + BloomFilterUtility buildBloomFilterUtility(ColumnDescriptor path); } diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilterUtility.java b/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilterUtility.java new file mode 100644 index 0000000000..d7ceb679e3 --- /dev/null +++ b/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilterUtility.java @@ -0,0 +1,81 @@ +/* + * 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.column.values.bloomfilter; + +import org.apache.parquet.io.api.Binary; + +public abstract class BloomFilterUtility> { + public BloomFilter bloomFilter; + public abstract boolean contains(T value); + + public static class IntBloomFilter extends BloomFilterUtility { + public IntBloomFilter(BloomFilter bloomFilter) { + this.bloomFilter = bloomFilter; + } + + @Override + public boolean contains(Integer value) { + return bloomFilter.findHash(bloomFilter.hash(value.intValue())); + } + } + + public static class LongBloomFilter extends BloomFilterUtility { + public LongBloomFilter(BloomFilter bloomFilter) { + this.bloomFilter = bloomFilter; + } + + @Override + public boolean contains(Long value) { + return bloomFilter.findHash(bloomFilter.hash(value.longValue())); + } + } + + public static class DoubleBloomFilter extends BloomFilterUtility { + public DoubleBloomFilter(BloomFilter bloomFilter) { + this.bloomFilter = bloomFilter; + } + + @Override + public boolean contains(Double value) { + return bloomFilter.findHash(bloomFilter.hash(value.doubleValue())); + } + } + + public static class FloatBloomFilter extends BloomFilterUtility { + public FloatBloomFilter(BloomFilter bloomFilter) { + this.bloomFilter = bloomFilter; + } + + @Override + public boolean contains(Float value) { + return bloomFilter.findHash(bloomFilter.hash(value.floatValue())); + } + } + + public static class BinaryBloomFilter extends BloomFilterUtility { + public BinaryBloomFilter(BloomFilter bloomFilter) { + this.bloomFilter = bloomFilter; + } + + @Override + public boolean contains(Binary value) { + return bloomFilter.findHash(bloomFilter.hash(value)); + } + } +} diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/filter2/BloomFilterLevel/BloomFilterImpl.java b/parquet-hadoop/src/main/java/org/apache/parquet/filter2/BloomFilterLevel/BloomFilterImpl.java new file mode 100644 index 0000000000..9266c74cf4 --- /dev/null +++ b/parquet-hadoop/src/main/java/org/apache/parquet/filter2/BloomFilterLevel/BloomFilterImpl.java @@ -0,0 +1,194 @@ +/* + * 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.filter2.BloomFilterLevel; + +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.values.bloomfilter.BloomFilterReader; +import org.apache.parquet.column.values.bloomfilter.BloomFilterUtility; +import org.apache.parquet.filter2.predicate.FilterPredicate; +import org.apache.parquet.filter2.predicate.Operators; +import org.apache.parquet.filter2.predicate.UserDefinedPredicate; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.ColumnPath; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.parquet.Preconditions.checkNotNull; + + +public class BloomFilterImpl implements FilterPredicate.Visitor{ + private static final boolean BLOCK_MIGHT_MATCH = false; + private static final boolean BLOCK_CANNOT_MATCH = true; + + private final Map columns = new HashMap(); + + public static boolean canDrop(FilterPredicate pred, List columns, BloomFilterReader bloomFilterReader) { + checkNotNull(pred, "pred"); + checkNotNull(columns, "columns"); + return pred.accept(new BloomFilterImpl(columns, bloomFilterReader)); + } + + private BloomFilterImpl(List columnsList, BloomFilterReader bloomFilterReader) { + for (ColumnChunkMetaData chunk : columnsList) { + columns.put(chunk.getPath(), chunk); + } + + this.bloomFilterReader = bloomFilterReader; + } + + private BloomFilterReader bloomFilterReader; + + private ColumnChunkMetaData getColumnChunk(ColumnPath columnPath) { + return columns.get(columnPath); + } + + // is this column chunk composed entirely of nulls? + // assumes the column chunk's statistics is not empty + private boolean isAllNulls(ColumnChunkMetaData column) { + return column.getStatistics().getNumNulls() == column.getValueCount(); + } + + @Override + public > Boolean visit(Operators.Eq eq) { + T value = eq.getValue(); + + if (value == null) { + // the bloom filter bitset contains only non-null values so isn't helpful. this + // could check the column stats, but the StatisticsFilter is responsible + return BLOCK_MIGHT_MATCH; + } + + Operators.Column filterColumn = eq.getColumn(); + ColumnChunkMetaData meta = getColumnChunk(filterColumn.getColumnPath()); + ColumnDescriptor col = new ColumnDescriptor(meta.getPath().toArray(), meta.getType(), -1, -1); + if (meta == null) { + // the column isn't in this file so all values are null, but the value + // must be non-null because of the above check. + return BLOCK_CANNOT_MATCH; + } + + BloomFilterUtility bloomFilterUtility = bloomFilterReader.buildBloomFilterUtility(col); + if (bloomFilterUtility != null && !bloomFilterUtility.contains(value)) { + return BLOCK_CANNOT_MATCH; + } + + return BLOCK_MIGHT_MATCH; + } + + @Override + public > Boolean visit(Operators.NotEq notEq) { + Operators.Column filterColumn = notEq.getColumn(); + ColumnChunkMetaData meta = getColumnChunk(filterColumn.getColumnPath()); + + T value = notEq.getValue(); + + if (meta == null) { + if (value == null) { + // null is always equal to null + return BLOCK_CANNOT_MATCH; + } + return BLOCK_MIGHT_MATCH; + } + + if (value == null) { + // we are looking for records where v notEq(null) + // so, if this is a column of all nulls, we can drop it + return isAllNulls(meta); + } + + return BLOCK_MIGHT_MATCH; + } + + @Override + public > Boolean visit(Operators.Lt lt) { + return BLOCK_MIGHT_MATCH; + } + + @Override + public > Boolean visit(Operators.LtEq ltEq) { + return BLOCK_MIGHT_MATCH; + } + + @Override + public > Boolean visit(Operators.Gt gt) { + return BLOCK_MIGHT_MATCH; + } + + @Override + public > Boolean visit(Operators.GtEq gtEq) { + return BLOCK_MIGHT_MATCH; + } + + @Override + public Boolean visit(Operators.And and) { + return and.getLeft().accept(this) || and.getRight().accept(this); + } + + @Override + public Boolean visit(Operators.Or or) { + return or.getLeft().accept(this) && or.getRight().accept(this); + } + + @Override + public Boolean visit(Operators.Not not) { + throw new IllegalArgumentException( + "This predicate contains a not! Did you forget to run this predicate through LogicalInverseRewriter? " + not); + } + + + private , U extends UserDefinedPredicate> Boolean visit(Operators.UserDefined ud, boolean inverted) { + Operators.Column filterColumn = ud.getColumn(); + ColumnChunkMetaData columnChunk = getColumnChunk(filterColumn.getColumnPath()); + U udp = ud.getUserDefinedPredicate(); + + if (columnChunk == null) { + // the column isn't in this file so all values are null. + // lets run the udp with null value to see if it keeps null or not. + if (inverted) { + return udp.keep(null); + } else { + return !udp.keep(null); + } + } + + if (isAllNulls(columnChunk)) { + // lets run the udp with null value to see if it keeps null or not. + if (inverted) { + return udp.keep(null); + } else { + return !udp.keep(null); + } + } + + return BLOCK_MIGHT_MATCH; + } + + + @Override + public , U extends UserDefinedPredicate> Boolean visit(Operators.UserDefined udp) { + return visit(udp, false); + } + + @Override + public , U extends UserDefinedPredicate> Boolean visit(Operators.LogicalNotUserDefined udp) { + return visit(udp.getUserDefined(), true); + } +} diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/RowGroupFilter.java b/parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/RowGroupFilter.java index d1d40e9d78..fe6f637bff 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/RowGroupFilter.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/RowGroupFilter.java @@ -1,4 +1,4 @@ -/* +/* * 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 @@ -6,9 +6,9 @@ * 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 @@ -22,6 +22,7 @@ import java.util.Collections; import java.util.List; +import org.apache.parquet.filter2.BloomFilterLevel.BloomFilterImpl; import org.apache.parquet.filter2.compat.FilterCompat.Filter; import org.apache.parquet.filter2.compat.FilterCompat.NoOpFilter; import org.apache.parquet.filter2.compat.FilterCompat.Visitor; @@ -32,6 +33,8 @@ import org.apache.parquet.hadoop.ParquetFileReader; import org.apache.parquet.hadoop.metadata.BlockMetaData; import org.apache.parquet.schema.MessageType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static org.apache.parquet.Preconditions.checkNotNull; @@ -45,10 +48,12 @@ public class RowGroupFilter implements Visitor> { private final MessageType schema; private final List levels; private final ParquetFileReader reader; + private Logger logger = LoggerFactory.getLogger(RowGroupFilter.class); public enum FilterLevel { STATISTICS, - DICTIONARY + DICTIONARY, + BLOOMFILTER } /** @@ -104,6 +109,11 @@ public List visit(FilterCompat.FilterPredicateCompat filterPredic drop = DictionaryFilter.canDrop(filterPredicate, block.getColumns(), reader.getDictionaryReader(block)); } + if (!drop && levels.contains(FilterLevel.BLOOMFILTER)) { + drop = BloomFilterImpl.canDrop(filterPredicate, block.getColumns(), reader.getBloomFilterDataReader(block)); + if (drop) logger.info("Block drop by Bloom filter"); + } + if(!drop) { filteredBlocks.add(block); } diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/BloomFilterDataReader.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/BloomFilterDataReader.java index 96e258fe40..23a7dc4159 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/BloomFilterDataReader.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/BloomFilterDataReader.java @@ -24,6 +24,7 @@ import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.column.values.bloomfilter.BloomFilter; import org.apache.parquet.column.values.bloomfilter.BloomFilterReader; +import org.apache.parquet.column.values.bloomfilter.BloomFilterUtility; import org.apache.parquet.hadoop.metadata.BlockMetaData; import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; import org.apache.parquet.io.ParquetDecodingException; @@ -36,6 +37,7 @@ public class BloomFilterDataReader implements BloomFilterReader { private final ParquetFileReader reader; private final Map columns; private final Map cache = new HashMap<>(); + public BloomFilterDataReader(ParquetFileReader fileReader, BlockMetaData block) { this.reader = fileReader; this.columns = new HashMap<>(); @@ -43,13 +45,14 @@ public BloomFilterDataReader(ParquetFileReader fileReader, BlockMetaData block) columns.put(column.getPath().toDotString(), column); } } + @Override public BloomFilter readBloomFilter(ColumnDescriptor descriptor) { String dotPath = Strings.join(descriptor.getPath(), "."); ColumnChunkMetaData column = columns.get(dotPath); if (column == null) { throw new ParquetDecodingException( - "Cannot load Bloom filter data, unknown column: " + dotPath); + "Cannot read Bloom filter data from meta data of column: " + dotPath); } if (cache.containsKey(dotPath)) { return cache.get(dotPath); @@ -65,7 +68,35 @@ public BloomFilter readBloomFilter(ColumnDescriptor descriptor) { return cache.get(dotPath); } catch (IOException e) { throw new ParquetDecodingException( - "Failed to read Bloom data", e); + "Failed to read Bloom filter data", e); } } + + @Override + public BloomFilterUtility buildBloomFilterUtility(ColumnDescriptor desc) { + BloomFilter bloomFilterData = readBloomFilter(desc); + + String dotPath = Strings.join(desc.getPath(), "."); + ColumnChunkMetaData meta = columns.get(dotPath); + if (meta == null) { + throw new ParquetDecodingException( + "Cannot read Bloom filter data from meta data of column: " + dotPath); + } + + switch (meta.getType()){ + case INT32: + return new BloomFilterUtility.IntBloomFilter(bloomFilterData); + case INT64: + return new BloomFilterUtility.LongBloomFilter(bloomFilterData); + case FLOAT: + return new BloomFilterUtility.FloatBloomFilter(bloomFilterData); + case DOUBLE: + return new BloomFilterUtility.DoubleBloomFilter(bloomFilterData); + case BINARY: + return new BloomFilterUtility.BinaryBloomFilter(bloomFilterData); + default: + return null; + } + } + } From db9a7bee2dff06a2478931b84de779d265f103a1 Mon Sep 17 00:00:00 2001 From: Junjie Chen Date: Sun, 10 Mar 2019 19:13:43 +0800 Subject: [PATCH 2/3] Address Gabor'm comments --- .../bloomfilter/BlockSplitBloomFilter.java | 55 +++++----- .../values/bloomfilter/BloomFilter.java | 45 ++------ .../bloomfilter/BloomFilterReadStore.java | 34 ------ .../values/bloomfilter/BloomFilterReader.java | 40 ------- .../bloomfilter/BloomFilterUtility.java | 81 -------------- .../BloomFilterLevel/BloomFilterImpl.java | 76 ++++--------- .../parquet/hadoop/BloomFilterDataReader.java | 102 ------------------ .../parquet/hadoop/BloomFilterReader.java | 66 ++++++++++++ .../parquet/hadoop/ParquetFileReader.java | 4 +- .../parquet/hadoop/TestParquetFileWriter.java | 9 +- 10 files changed, 123 insertions(+), 389 deletions(-) delete mode 100644 parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilterReadStore.java delete mode 100644 parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilterReader.java delete mode 100644 parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilterUtility.java delete mode 100644 parquet-hadoop/src/main/java/org/apache/parquet/hadoop/BloomFilterDataReader.java create mode 100644 parquet-hadoop/src/main/java/org/apache/parquet/hadoop/BloomFilterReader.java diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BlockSplitBloomFilter.java b/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BlockSplitBloomFilter.java index b6378976c3..34200d9f1d 100644 --- a/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BlockSplitBloomFilter.java +++ b/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BlockSplitBloomFilter.java @@ -18,11 +18,13 @@ */ package org.apache.parquet.column.values.bloomfilter; + import com.google.common.hash.HashFunction; import com.google.common.hash.Hashing; import org.apache.parquet.Preconditions; import org.apache.parquet.bytes.BytesUtils; import org.apache.parquet.io.api.Binary; + import java.io.IOException; import java.io.OutputStream; import java.nio.ByteBuffer; @@ -139,7 +141,7 @@ private BlockSplitBloomFilter(byte[] bitset, HashStrategy hashStrategy) { hashFunction = Hashing.murmur3_128(DEFAULT_SEED); break; default: - throw new RuntimeException("Not supported hash strategy"); + throw new RuntimeException("Unsupported hash strategy"); } } @@ -255,40 +257,35 @@ public static int optimalNumOfBits(long n, double p) { } @Override - public long hash(int value) { - ByteBuffer plain = ByteBuffer.allocate(Integer.SIZE/Byte.SIZE); - plain.order(ByteOrder.LITTLE_ENDIAN).putInt(value); - return hashFunction.hashBytes(plain.array()).asLong(); + public long getBitsetSize() { + return this.bitset.length; } - @Override - public long hash(long value) { - ByteBuffer plain = ByteBuffer.allocate(Long.SIZE/Byte.SIZE); - plain.order(ByteOrder.LITTLE_ENDIAN).putLong(value); - return hashFunction.hashBytes(plain.array()).asLong(); - } @Override - public long hash(double value) { - ByteBuffer plain = ByteBuffer.allocate(Double.SIZE/Byte.SIZE); - plain.order(ByteOrder.LITTLE_ENDIAN).putDouble(value); - return hashFunction.hashBytes(plain.array()).asLong(); - } + public long hash(Object value) { + ByteBuffer plain = null; - @Override - public long hash(float value) { - ByteBuffer plain = ByteBuffer.allocate(Float.SIZE/Byte.SIZE); - plain.order(ByteOrder.LITTLE_ENDIAN).putFloat(value); - return hashFunction.hashBytes(plain.array()).asLong(); - } + if (value instanceof Binary) { + return hashFunction.hashBytes(((Binary) value).getBytes()).asLong(); + } - @Override - public long hash(Binary value) { - return hashFunction.hashBytes(value.getBytes()).asLong(); - } + if (value instanceof Integer) { + plain = ByteBuffer.allocate(Integer.SIZE/Byte.SIZE); + plain.order(ByteOrder.LITTLE_ENDIAN).putInt(((Integer)value).intValue()); + } else if (value instanceof Long) { + plain = ByteBuffer.allocate(Long.SIZE/Byte.SIZE); + plain.order(ByteOrder.LITTLE_ENDIAN).putLong(((Long)value).longValue()); + } else if (value instanceof Float) { + plain = ByteBuffer.allocate(Float.SIZE/Byte.SIZE); + plain.order(ByteOrder.LITTLE_ENDIAN).putFloat(((Float)value).floatValue()); + } else if (value instanceof Double) { + plain = ByteBuffer.allocate(Double.SIZE/ Byte.SIZE); + plain.order(ByteOrder.LITTLE_ENDIAN).putDouble(((Double)value).doubleValue()); + } else { + throw new RuntimeException("Parquet Bloom filter: Not supported type"); + } - @Override - public long getBitsetSize() { - return this.bitset.length; + return hashFunction.hashBytes(plain.array()).asLong(); } } diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilter.java b/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilter.java index 3ec192e3e0..7d993b2835 100644 --- a/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilter.java +++ b/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilter.java @@ -18,7 +18,6 @@ */ package org.apache.parquet.column.values.bloomfilter; -import org.apache.parquet.io.api.Binary; import java.io.IOException; import java.io.OutputStream; @@ -27,7 +26,7 @@ * in a set. The Bloom filter usually consists of a bit set that represents a elements set, * a hash strategy and a Bloom filter algorithm. */ -public interface BloomFilter { +public interface BloomFilter { // Bloom filter Hash strategy. enum HashStrategy { MURMUR3_X64_128(0); @@ -71,49 +70,17 @@ enum Algorithm { boolean findHash(long hash); /** - * Compute hash for int value by using its plain encoding result. - * - * @param value the value to hash - * @return hash result - */ - long hash(int value); - - /** - * Compute hash for long value by using its plain encoding result. - * - * @param value the value to hash - * @return hash result - */ - long hash(long value) ; - - /** - * Compute hash for double value by using its plain encoding result. - * - * @param value the value to hash - * @return hash result - */ - long hash(double value); - - /** - * Compute hash for float value by using its plain encoding result. + * Get the number of bytes for bitset in this Bloom filter. * - * @param value the value to hash - * @return hash result + * @return The number of bytes for bitset in this Bloom filter. */ - long hash(float value); + long getBitsetSize(); /** - * Compute hash for Binary value by using its plain encoding result. + * Compute hash for value by using its plain encoding result. * * @param value the value to hash * @return hash result */ - long hash(Binary value); - - /** - * Get the number of bytes for bitset in this Bloom filter. - * - * @return The number of bytes for bitset in this Bloom filter. - */ - long getBitsetSize(); + long hash (T value); } diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilterReadStore.java b/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilterReadStore.java deleted file mode 100644 index 3373bc1a0e..0000000000 --- a/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilterReadStore.java +++ /dev/null @@ -1,34 +0,0 @@ -/* - * 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.column.values.bloomfilter; - -import org.apache.parquet.column.ColumnDescriptor; - -/** - * contains all the bloom filter reader for all columns of a row group - */ -public interface BloomFilterReadStore { - /** - * Get a Bloom filter reader of a column - * - * @param path the descriptor of the column - * @return the corresponding Bloom filter writer - */ - BloomFilterReader getBloomFilterReader(ColumnDescriptor path); -} diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilterReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilterReader.java deleted file mode 100644 index 02188fe42f..0000000000 --- a/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilterReader.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * 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.column.values.bloomfilter; - -import org.apache.parquet.column.ColumnDescriptor; - -public interface BloomFilterReader { - /** - * Returns a {@link BloomFilter} for the given column descriptor. - * - * @param path the descriptor of the column - * @return the bloomFilter data for the column, or null if there isn't one - */ - BloomFilter readBloomFilter(ColumnDescriptor path); - - /** - * Returns a {@link BloomFilterUtility} for the given column descriptor. - * - * @param path the descriptor of the column - * @return a concrete bloom filter utility - */ - BloomFilterUtility buildBloomFilterUtility(ColumnDescriptor path); -} diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilterUtility.java b/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilterUtility.java deleted file mode 100644 index d7ceb679e3..0000000000 --- a/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilterUtility.java +++ /dev/null @@ -1,81 +0,0 @@ -/* - * 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.column.values.bloomfilter; - -import org.apache.parquet.io.api.Binary; - -public abstract class BloomFilterUtility> { - public BloomFilter bloomFilter; - public abstract boolean contains(T value); - - public static class IntBloomFilter extends BloomFilterUtility { - public IntBloomFilter(BloomFilter bloomFilter) { - this.bloomFilter = bloomFilter; - } - - @Override - public boolean contains(Integer value) { - return bloomFilter.findHash(bloomFilter.hash(value.intValue())); - } - } - - public static class LongBloomFilter extends BloomFilterUtility { - public LongBloomFilter(BloomFilter bloomFilter) { - this.bloomFilter = bloomFilter; - } - - @Override - public boolean contains(Long value) { - return bloomFilter.findHash(bloomFilter.hash(value.longValue())); - } - } - - public static class DoubleBloomFilter extends BloomFilterUtility { - public DoubleBloomFilter(BloomFilter bloomFilter) { - this.bloomFilter = bloomFilter; - } - - @Override - public boolean contains(Double value) { - return bloomFilter.findHash(bloomFilter.hash(value.doubleValue())); - } - } - - public static class FloatBloomFilter extends BloomFilterUtility { - public FloatBloomFilter(BloomFilter bloomFilter) { - this.bloomFilter = bloomFilter; - } - - @Override - public boolean contains(Float value) { - return bloomFilter.findHash(bloomFilter.hash(value.floatValue())); - } - } - - public static class BinaryBloomFilter extends BloomFilterUtility { - public BinaryBloomFilter(BloomFilter bloomFilter) { - this.bloomFilter = bloomFilter; - } - - @Override - public boolean contains(Binary value) { - return bloomFilter.findHash(bloomFilter.hash(value)); - } - } -} diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/filter2/BloomFilterLevel/BloomFilterImpl.java b/parquet-hadoop/src/main/java/org/apache/parquet/filter2/BloomFilterLevel/BloomFilterImpl.java index 9266c74cf4..68e40fcc27 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/filter2/BloomFilterLevel/BloomFilterImpl.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/filter2/BloomFilterLevel/BloomFilterImpl.java @@ -16,25 +16,27 @@ * specific language governing permissions and limitations * under the License. */ + package org.apache.parquet.filter2.BloomFilterLevel; -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.values.bloomfilter.BloomFilterReader; -import org.apache.parquet.column.values.bloomfilter.BloomFilterUtility; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.parquet.column.values.bloomfilter.BloomFilter; import org.apache.parquet.filter2.predicate.FilterPredicate; import org.apache.parquet.filter2.predicate.Operators; import org.apache.parquet.filter2.predicate.UserDefinedPredicate; +import org.apache.parquet.hadoop.BloomFilterReader; import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; import org.apache.parquet.hadoop.metadata.ColumnPath; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - import static org.apache.parquet.Preconditions.checkNotNull; - public class BloomFilterImpl implements FilterPredicate.Visitor{ + private static final Logger LOG = LoggerFactory.getLogger(BloomFilterImpl.class); private static final boolean BLOCK_MIGHT_MATCH = false; private static final boolean BLOCK_CANNOT_MATCH = true; @@ -63,7 +65,7 @@ private ColumnChunkMetaData getColumnChunk(ColumnPath columnPath) { // is this column chunk composed entirely of nulls? // assumes the column chunk's statistics is not empty private boolean isAllNulls(ColumnChunkMetaData column) { - return column.getStatistics().getNumNulls() == column.getValueCount(); + return BLOCK_MIGHT_MATCH; } @Override @@ -78,16 +80,20 @@ public > Boolean visit(Operators.Eq eq) { Operators.Column filterColumn = eq.getColumn(); ColumnChunkMetaData meta = getColumnChunk(filterColumn.getColumnPath()); - ColumnDescriptor col = new ColumnDescriptor(meta.getPath().toArray(), meta.getType(), -1, -1); if (meta == null) { // the column isn't in this file so all values are null, but the value // must be non-null because of the above check. return BLOCK_CANNOT_MATCH; } - BloomFilterUtility bloomFilterUtility = bloomFilterReader.buildBloomFilterUtility(col); - if (bloomFilterUtility != null && !bloomFilterUtility.contains(value)) { - return BLOCK_CANNOT_MATCH; + try { + BloomFilter bloomFilter = bloomFilterReader.readBloomFilter(meta); + if (bloomFilter != null && !bloomFilter.findHash(bloomFilter.hash(value))) { + return BLOCK_CANNOT_MATCH; + } + } catch (RuntimeException e) { + LOG.warn(e.getMessage()); + return BLOCK_MIGHT_MATCH; } return BLOCK_MIGHT_MATCH; @@ -95,25 +101,6 @@ public > Boolean visit(Operators.Eq eq) { @Override public > Boolean visit(Operators.NotEq notEq) { - Operators.Column filterColumn = notEq.getColumn(); - ColumnChunkMetaData meta = getColumnChunk(filterColumn.getColumnPath()); - - T value = notEq.getValue(); - - if (meta == null) { - if (value == null) { - // null is always equal to null - return BLOCK_CANNOT_MATCH; - } - return BLOCK_MIGHT_MATCH; - } - - if (value == null) { - // we are looking for records where v notEq(null) - // so, if this is a column of all nulls, we can drop it - return isAllNulls(meta); - } - return BLOCK_MIGHT_MATCH; } @@ -153,35 +140,10 @@ public Boolean visit(Operators.Not not) { "This predicate contains a not! Did you forget to run this predicate through LogicalInverseRewriter? " + not); } - private , U extends UserDefinedPredicate> Boolean visit(Operators.UserDefined ud, boolean inverted) { - Operators.Column filterColumn = ud.getColumn(); - ColumnChunkMetaData columnChunk = getColumnChunk(filterColumn.getColumnPath()); - U udp = ud.getUserDefinedPredicate(); - - if (columnChunk == null) { - // the column isn't in this file so all values are null. - // lets run the udp with null value to see if it keeps null or not. - if (inverted) { - return udp.keep(null); - } else { - return !udp.keep(null); - } - } - - if (isAllNulls(columnChunk)) { - // lets run the udp with null value to see if it keeps null or not. - if (inverted) { - return udp.keep(null); - } else { - return !udp.keep(null); - } - } - return BLOCK_MIGHT_MATCH; } - @Override public , U extends UserDefinedPredicate> Boolean visit(Operators.UserDefined udp) { return visit(udp, false); diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/BloomFilterDataReader.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/BloomFilterDataReader.java deleted file mode 100644 index 23a7dc4159..0000000000 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/BloomFilterDataReader.java +++ /dev/null @@ -1,102 +0,0 @@ -/* - * 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.hadoop; -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; -import org.apache.parquet.Strings; -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.column.values.bloomfilter.BloomFilter; -import org.apache.parquet.column.values.bloomfilter.BloomFilterReader; -import org.apache.parquet.column.values.bloomfilter.BloomFilterUtility; -import org.apache.parquet.hadoop.metadata.BlockMetaData; -import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; -import org.apache.parquet.io.ParquetDecodingException; -/** - * A {@link BloomFilterReader} implementation that reads Bloom filter data from - * an open {@link ParquetFileReader}. - * - */ -public class BloomFilterDataReader implements BloomFilterReader { - private final ParquetFileReader reader; - private final Map columns; - private final Map cache = new HashMap<>(); - - public BloomFilterDataReader(ParquetFileReader fileReader, BlockMetaData block) { - this.reader = fileReader; - this.columns = new HashMap<>(); - for (ColumnChunkMetaData column : block.getColumns()) { - columns.put(column.getPath().toDotString(), column); - } - } - - @Override - public BloomFilter readBloomFilter(ColumnDescriptor descriptor) { - String dotPath = Strings.join(descriptor.getPath(), "."); - ColumnChunkMetaData column = columns.get(dotPath); - if (column == null) { - throw new ParquetDecodingException( - "Cannot read Bloom filter data from meta data of column: " + dotPath); - } - if (cache.containsKey(dotPath)) { - return cache.get(dotPath); - } - try { - synchronized (cache) { - if (!cache.containsKey(dotPath)) { - BloomFilter bloomFilter = reader.readBloomFilter(column); - if (bloomFilter == null) return null; - cache.put(dotPath, bloomFilter); - } - } - return cache.get(dotPath); - } catch (IOException e) { - throw new ParquetDecodingException( - "Failed to read Bloom filter data", e); - } - } - - @Override - public BloomFilterUtility buildBloomFilterUtility(ColumnDescriptor desc) { - BloomFilter bloomFilterData = readBloomFilter(desc); - - String dotPath = Strings.join(desc.getPath(), "."); - ColumnChunkMetaData meta = columns.get(dotPath); - if (meta == null) { - throw new ParquetDecodingException( - "Cannot read Bloom filter data from meta data of column: " + dotPath); - } - - switch (meta.getType()){ - case INT32: - return new BloomFilterUtility.IntBloomFilter(bloomFilterData); - case INT64: - return new BloomFilterUtility.LongBloomFilter(bloomFilterData); - case FLOAT: - return new BloomFilterUtility.FloatBloomFilter(bloomFilterData); - case DOUBLE: - return new BloomFilterUtility.DoubleBloomFilter(bloomFilterData); - case BINARY: - return new BloomFilterUtility.BinaryBloomFilter(bloomFilterData); - default: - return null; - } - } - -} diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/BloomFilterReader.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/BloomFilterReader.java new file mode 100644 index 0000000000..3ad91ce01e --- /dev/null +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/BloomFilterReader.java @@ -0,0 +1,66 @@ +/* + * 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.hadoop; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import org.apache.parquet.column.values.bloomfilter.BloomFilter; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.hadoop.metadata.ColumnPath; + +/** + * Bloom filter reader that reads Bloom filter data from an open {@link ParquetFileReader}. + * + */ +public class BloomFilterReader { + private final ParquetFileReader reader; + private final Map columns; + private final Map cache = new HashMap<>(); + + public BloomFilterReader(ParquetFileReader fileReader, BlockMetaData block) { + this.reader = fileReader; + this.columns = new HashMap<>(); + for (ColumnChunkMetaData column : block.getColumns()) { + columns.put(column.getPath(), column); + } + } + + public BloomFilter readBloomFilter(ColumnChunkMetaData meta) { + if (cache.containsKey(meta.getPath())) { + return cache.get(meta.getPath()); + } + try { + synchronized (cache) { + if (!cache.containsKey(meta.getPath())) { + BloomFilter bloomFilter = reader.readBloomFilter(meta); + if (bloomFilter == null) return null; + cache.put(meta.getPath(), bloomFilter); + } + } + return cache.get(meta.getPath()); + } catch (IOException e) { + throw new RuntimeException( + "Failed to read Bloom filter data", e); + } + } + +} diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java index 7fe0e410ee..860e044041 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java @@ -1049,8 +1049,8 @@ private DictionaryPage readCompressedDictionary( converter.getEncoding(dictHeader.getEncoding())); } - public BloomFilterDataReader getBloomFilterDataReader(BlockMetaData block) { - return new BloomFilterDataReader(this, block); + public BloomFilterReader getBloomFilterDataReader(BlockMetaData block) { + return new BloomFilterReader(this, block); } /** diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetFileWriter.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetFileWriter.java index 71ca5ea93d..e19e35cfac 100644 --- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetFileWriter.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetFileWriter.java @@ -29,7 +29,6 @@ import org.apache.parquet.bytes.BytesUtils; import org.apache.parquet.column.values.bloomfilter.BlockSplitBloomFilter; import org.apache.parquet.column.values.bloomfilter.BloomFilter; -import org.apache.parquet.column.values.bloomfilter.BloomFilterReader; import org.apache.parquet.hadoop.ParquetOutputFormat.JobSummaryLevel; import org.junit.Assume; import org.junit.Rule; @@ -249,10 +248,10 @@ public void testBloomWriteRead() throws Exception { ParquetMetadata readFooter = ParquetFileReader.readFooter(configuration, path); ParquetFileReader r = new ParquetFileReader(configuration, readFooter.getFileMetaData(), path, Arrays.asList(readFooter.getBlocks().get(0)), Arrays.asList(schema.getColumnDescription(colPath))); - BloomFilterReader bloomFilterReader = r.getBloomFilterDataReader(readFooter.getBlocks().get(0)); - BloomFilter bloomDataRead = bloomFilterReader.readBloomFilter(col); - assertTrue(bloomDataRead.findHash(bloomData.hash(Binary.fromString("hello")))); - assertTrue(bloomDataRead.findHash(bloomData.hash(Binary.fromString("world")))); + BloomFilterReader bloomFilterReader = r.getBloomFilterDataReader(readFooter.getBlocks().get(0)); + BloomFilter bloomFilter = bloomFilterReader.readBloomFilter(readFooter.getBlocks().get(0).getColumns().get(0)); + assertTrue(bloomFilter.findHash(bloomData.hash(Binary.fromString("hello")))); + assertTrue(bloomFilter.findHash(bloomData.hash(Binary.fromString("world")))); } @Test From 36225a2300faa733749fd4a7a91131cdf72bf327 Mon Sep 17 00:00:00 2001 From: Junjie Chen Date: Fri, 15 Mar 2019 23:07:10 +0800 Subject: [PATCH 3/3] fix performance issue when writing values --- .../bloomfilter/BlockSplitBloomFilter.java | 36 +++++++++++++- .../values/bloomfilter/BloomFilter.java | 48 +++++++++++++++++-- .../BloomFilterLevel/BloomFilterImpl.java | 6 --- 3 files changed, 79 insertions(+), 11 deletions(-) diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BlockSplitBloomFilter.java b/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BlockSplitBloomFilter.java index 34200d9f1d..d8ac0b435a 100644 --- a/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BlockSplitBloomFilter.java +++ b/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BlockSplitBloomFilter.java @@ -261,10 +261,9 @@ public long getBitsetSize() { return this.bitset.length; } - @Override public long hash(Object value) { - ByteBuffer plain = null; + ByteBuffer plain; if (value instanceof Binary) { return hashFunction.hashBytes(((Binary) value).getBytes()).asLong(); @@ -288,4 +287,37 @@ public long hash(Object value) { return hashFunction.hashBytes(plain.array()).asLong(); } + + @Override + public long hash(int value) { + ByteBuffer plain = ByteBuffer.allocate(Integer.SIZE/Byte.SIZE); + plain.order(ByteOrder.LITTLE_ENDIAN).putInt(value); + return hashFunction.hashBytes(plain.array()).asLong(); + } + + @Override + public long hash(long value) { + ByteBuffer plain = ByteBuffer.allocate(Long.SIZE/Byte.SIZE); + plain.order(ByteOrder.LITTLE_ENDIAN).putLong(value); + return hashFunction.hashBytes(plain.array()).asLong(); + } + + @Override + public long hash(double value) { + ByteBuffer plain = ByteBuffer.allocate(Double.SIZE/Byte.SIZE); + plain.order(ByteOrder.LITTLE_ENDIAN).putDouble(value); + return hashFunction.hashBytes(plain.array()).asLong(); + } + + @Override + public long hash(float value) { + ByteBuffer plain = ByteBuffer.allocate(Float.SIZE/Byte.SIZE); + plain.order(ByteOrder.LITTLE_ENDIAN).putFloat(value); + return hashFunction.hashBytes(plain.array()).asLong(); + } + + @Override + public long hash(Binary value) { + return hashFunction.hashBytes(value.getBytes()).asLong(); + } } diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilter.java b/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilter.java index 7d993b2835..a6e548ffb2 100644 --- a/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilter.java +++ b/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BloomFilter.java @@ -18,6 +18,8 @@ */ package org.apache.parquet.column.values.bloomfilter; +import org.apache.parquet.io.api.Binary; + import java.io.IOException; import java.io.OutputStream; @@ -26,7 +28,7 @@ * in a set. The Bloom filter usually consists of a bit set that represents a elements set, * a hash strategy and a Bloom filter algorithm. */ -public interface BloomFilter { +public interface BloomFilter { // Bloom filter Hash strategy. enum HashStrategy { MURMUR3_X64_128(0); @@ -77,10 +79,50 @@ enum Algorithm { long getBitsetSize(); /** - * Compute hash for value by using its plain encoding result. + * Compute hash for int value by using its plain encoding result. + * + * @param value the value to hash + * @return hash result + */ + long hash(int value); + + /** + * Compute hash for long value by using its plain encoding result. + * + * @param value the value to hash + * @return hash result + */ + long hash(long value) ; + + /** + * Compute hash for double value by using its plain encoding result. + * + * @param value the value to hash + * @return hash result + */ + long hash(double value); + + /** + * Compute hash for float value by using its plain encoding result. + * + * @param value the value to hash + * @return hash result + */ + long hash(float value); + + /** + * Compute hash for Binary value by using its plain encoding result. + * + * @param value the value to hash + * @return hash result + */ + long hash(Binary value); + + /** + * Compute hash for Object value by using its plain encoding result. * * @param value the value to hash * @return hash result */ - long hash (T value); + long hash(Object value); } diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/filter2/BloomFilterLevel/BloomFilterImpl.java b/parquet-hadoop/src/main/java/org/apache/parquet/filter2/BloomFilterLevel/BloomFilterImpl.java index 68e40fcc27..c1e377403e 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/filter2/BloomFilterLevel/BloomFilterImpl.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/filter2/BloomFilterLevel/BloomFilterImpl.java @@ -62,12 +62,6 @@ private ColumnChunkMetaData getColumnChunk(ColumnPath columnPath) { return columns.get(columnPath); } - // is this column chunk composed entirely of nulls? - // assumes the column chunk's statistics is not empty - private boolean isAllNulls(ColumnChunkMetaData column) { - return BLOCK_MIGHT_MATCH; - } - @Override public > Boolean visit(Operators.Eq eq) { T value = eq.getValue();