You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@parquet.apache.org by ga...@apache.org on 2020/03/30 08:51:35 UTC
[parquet-mr] branch master updated: PARQUET-1805: Refactor the
configuration for bloom filters (#763)
This is an automated email from the ASF dual-hosted git repository.
gabor 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 7ddfb4d PARQUET-1805: Refactor the configuration for bloom filters (#763)
7ddfb4d is described below
commit 7ddfb4d9d89fc7ae1d9ea0ef86aaa6dd5e81aa59
Author: Gabor Szadovszky <ga...@apache.org>
AuthorDate: Mon Mar 30 10:51:25 2020 +0200
PARQUET-1805: Refactor the configuration for bloom filters (#763)
---
.../apache/parquet/column/ParquetProperties.java | 84 +++++++++++++---------
.../parquet/column/impl/ColumnWriteStoreBase.java | 2 +-
.../parquet/column/impl/ColumnWriterBase.java | 23 ++----
.../apache/parquet/hadoop/ParquetOutputFormat.java | 46 +++---------
.../org/apache/parquet/hadoop/ParquetWriter.java | 37 ++++++----
.../apache/parquet/hadoop/TestBloomFiltering.java | 12 ++--
.../apache/parquet/hadoop/TestParquetWriter.java | 2 +-
7 files changed, 96 insertions(+), 110 deletions(-)
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/ParquetProperties.java b/parquet-column/src/main/java/org/apache/parquet/column/ParquetProperties.java
index d9238de..7e55ebc 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/ParquetProperties.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/ParquetProperties.java
@@ -18,13 +18,8 @@
*/
package org.apache.parquet.column;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
import java.util.Objects;
-import java.util.Set;
+import java.util.OptionalLong;
import org.apache.parquet.Preconditions;
import org.apache.parquet.bytes.ByteBufferAllocator;
@@ -61,6 +56,7 @@ public class ParquetProperties {
public static final int DEFAULT_STATISTICS_TRUNCATE_LENGTH = Integer.MAX_VALUE;
public static final int DEFAULT_PAGE_ROW_COUNT_LIMIT = 20_000;
public static final int DEFAULT_MAX_BLOOM_FILTER_BYTES = 1024 * 1024;
+ public static final boolean DEFAULT_BLOOM_FILTER_ENABLED = false;
public static final boolean DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED = true;
@@ -102,10 +98,10 @@ public class ParquetProperties {
private final int columnIndexTruncateLength;
private final int statisticsTruncateLength;
- // The key-value pair represents the column name and its expected distinct number of values in a row group.
- private final Map<String, Long> bloomFilterExpectedDistinctNumbers;
+ // The expected NDV (number of distinct values) for each columns
+ private final ColumnProperty<Long> bloomFilterNDVs;
private final int maxBloomFilterBytes;
- private final List<String> bloomFilterColumns;
+ private final ColumnProperty<Boolean> bloomFilterEnabled;
private final int pageRowCountLimit;
private final boolean pageWriteChecksumEnabled;
private final boolean enableByteStreamSplit;
@@ -125,8 +121,8 @@ public class ParquetProperties {
this.valuesWriterFactory = builder.valuesWriterFactory;
this.columnIndexTruncateLength = builder.columnIndexTruncateLength;
this.statisticsTruncateLength = builder.statisticsTruncateLength;
- this.bloomFilterExpectedDistinctNumbers = builder.bloomFilterColumnExpectedNDVs;
- this.bloomFilterColumns = builder.bloomFilterColumns;
+ this.bloomFilterNDVs = builder.bloomFilterNDVs.build();
+ this.bloomFilterEnabled = builder.bloomFilterEnabled.build();
this.maxBloomFilterBytes = builder.maxBloomFilterBytes;
this.pageRowCountLimit = builder.pageRowCountLimit;
this.pageWriteChecksumEnabled = builder.pageWriteChecksumEnabled;
@@ -257,16 +253,13 @@ public class ParquetProperties {
return pageWriteChecksumEnabled;
}
- public Map<String, Long> getBloomFilterColumnExpectedNDVs() {
- return bloomFilterExpectedDistinctNumbers;
+ public OptionalLong getBloomFilterNDV(ColumnDescriptor column) {
+ Long ndv = bloomFilterNDVs.getValue(column);
+ return ndv == null ? OptionalLong.empty() : OptionalLong.of(ndv);
}
- public Set<String> getBloomFilterColumns() {
- if (bloomFilterColumns != null && bloomFilterColumns.size() > 0){
- return new HashSet<>(bloomFilterColumns);
- }
-
- return bloomFilterExpectedDistinctNumbers.keySet();
+ public boolean isBloomFilterEnabled(ColumnDescriptor column) {
+ return bloomFilterEnabled.getValue(column);
}
public int getMaxBloomFilterBytes() {
@@ -292,9 +285,9 @@ public class ParquetProperties {
+ "Max row count for page size check is: " + getMaxRowCountForPageSizeCheck() + '\n'
+ "Truncate length for column indexes is: " + getColumnIndexTruncateLength() + '\n'
+ "Truncate length for statistics min/max is: " + getStatisticsTruncateLength() + '\n'
- + "Bloom filter enabled column names are: " + getBloomFilterColumns() + '\n'
+ + "Bloom filter enabled: " + bloomFilterEnabled + '\n'
+ "Max Bloom filter size for a column is " + getMaxBloomFilterBytes() + '\n'
- + "Bloom filter enabled column expected number of distinct values are: " + getBloomFilterColumnExpectedNDVs().values() + '\n'
+ + "Bloom filter expected number of distinct values are: " + bloomFilterNDVs + '\n'
+ "Page row count limit to " + getPageRowCountLimit() + '\n'
+ "Writing page checksums is: " + (getPageWriteChecksumEnabled() ? "on" : "off");
}
@@ -311,15 +304,17 @@ public class ParquetProperties {
private ValuesWriterFactory valuesWriterFactory = DEFAULT_VALUES_WRITER_FACTORY;
private int columnIndexTruncateLength = DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH;
private int statisticsTruncateLength = DEFAULT_STATISTICS_TRUNCATE_LENGTH;
- private Map<String, Long> bloomFilterColumnExpectedNDVs = new HashMap<>();
+ private final ColumnProperty.Builder<Long> bloomFilterNDVs;
private int maxBloomFilterBytes = DEFAULT_MAX_BLOOM_FILTER_BYTES;
- private List<String> bloomFilterColumns = new ArrayList<>();
+ private final ColumnProperty.Builder<Boolean> bloomFilterEnabled;
private int pageRowCountLimit = DEFAULT_PAGE_ROW_COUNT_LIMIT;
private boolean pageWriteChecksumEnabled = DEFAULT_PAGE_WRITE_CHECKSUM_ENABLED;
private boolean enableByteStreamSplit = DEFAULT_IS_BYTE_STREAM_SPLIT_ENABLED;
private Builder() {
enableDict = ColumnProperty.<Boolean>builder().withDefaultValue(DEFAULT_IS_DICTIONARY_ENABLED);
+ bloomFilterEnabled = ColumnProperty.<Boolean>builder().withDefaultValue(DEFAULT_BLOOM_FILTER_ENABLED);
+ bloomFilterNDVs = ColumnProperty.<Long>builder().withDefaultValue(null);
}
private Builder(ParquetProperties toCopy) {
@@ -334,8 +329,8 @@ public class ParquetProperties {
this.allocator = toCopy.allocator;
this.pageRowCountLimit = toCopy.pageRowCountLimit;
this.pageWriteChecksumEnabled = toCopy.pageWriteChecksumEnabled;
- this.bloomFilterColumnExpectedNDVs = toCopy.bloomFilterExpectedDistinctNumbers;
- this.bloomFilterColumns = toCopy.bloomFilterColumns;
+ this.bloomFilterNDVs = ColumnProperty.<Long>builder(toCopy.bloomFilterNDVs);
+ this.bloomFilterEnabled = ColumnProperty.<Boolean>builder(toCopy.bloomFilterEnabled);
this.maxBloomFilterBytes = toCopy.maxBloomFilterBytes;
this.enableByteStreamSplit = toCopy.enableByteStreamSplit;
}
@@ -459,26 +454,49 @@ public class ParquetProperties {
}
/**
- * Set Bloom filter column names and expected NDVs.
+ * Set Bloom filter NDV (number of distinct values) for the specified column.
+ * If set for a column then the writing of the bloom filter for that column will be automatically enabled (see
+ * {@link #withBloomFilterEnabled(String, boolean)}).
+ *
+ * @param columnPath the path of the column (dot-string)
+ * @param ndv the NDV of the column
+ *
+ * @return this builder for method chaining
+ */
+ public Builder withBloomFilterNDV(String columnPath, long ndv) {
+ Preconditions.checkArgument(ndv > 0, "Invalid NDV for column \"%s\": %d", columnPath, ndv);
+ this.bloomFilterNDVs.withValue(columnPath, ndv);
+ // Setting an NDV for a column implies writing a bloom filter
+ this.bloomFilterEnabled.withValue(columnPath, true);
+ return this;
+ }
+
+ /**
+ * Enable or disable the bloom filter for the columns not specified by
+ * {@link #withBloomFilterEnabled(String, boolean)}.
*
- * @param columnToNDVMap the columns which has bloom filter enabled.
+ * @param enabled whether bloom filter shall be enabled for all columns
*
* @return this builder for method chaining
*/
- public Builder withBloomFilterColumnToNDVMap(Map<String, Long> columnToNDVMap) {
- this.bloomFilterColumnExpectedNDVs = columnToNDVMap;
+ public Builder withBloomFilterEnabled(boolean enabled) {
+ this.bloomFilterEnabled.withDefaultValue(enabled);
return this;
}
/**
- * Set Bloom filter column names.
+ * Enable or disable the bloom filter for the specified column.
+ * One may either disable bloom filters for all columns by invoking {@link #withBloomFilterEnabled(boolean)} with a
+ * {@code false} value and then enable the bloom filters for the required columns one-by-one by invoking this
+ * method or vice versa.
*
- * @param columns the columns which has bloom filter enabled.
+ * @param columnPath the path of the column (dot-string)
+ * @param enabled whether bloom filter shall be enabled
*
* @return this builder for method chaining
*/
- public Builder withBloomFilterColumnNames(List<String> columns) {
- this.bloomFilterColumns = columns;
+ public Builder withBloomFilterEnabled(String columnPath, boolean enabled) {
+ this.bloomFilterEnabled.withValue(columnPath, enabled);
return this;
}
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreBase.java b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreBase.java
index 8740099..a5e7836 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreBase.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreBase.java
@@ -118,7 +118,7 @@ abstract class ColumnWriteStoreBase implements ColumnWriteStore {
Map<ColumnDescriptor, ColumnWriterBase> mcolumns = new TreeMap<>();
for (ColumnDescriptor path : schema.getColumns()) {
PageWriter pageWriter = pageWriteStore.getPageWriter(path);
- if (props.getBloomFilterColumns() != null && props.getBloomFilterColumns().size() > 0) {
+ if (props.isBloomFilterEnabled(path)) {
BloomFilterWriter bloomFilterWriter = bloomFilterWriteStore.getBloomFilterWriter(path);
mcolumns.put(path, createColumnWriter(path, pageWriter, bloomFilterWriter, props));
} else {
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterBase.java b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterBase.java
index c46b26a..f45e10d 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterBase.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterBase.java
@@ -19,8 +19,7 @@
package org.apache.parquet.column.impl;
import java.io.IOException;
-import java.util.Map;
-import java.util.Set;
+import java.util.OptionalLong;
import org.apache.parquet.column.ColumnDescriptor;
import org.apache.parquet.column.ColumnWriter;
@@ -83,25 +82,17 @@ abstract class ColumnWriterBase implements ColumnWriter {
this.dataColumn = props.newValuesWriter(path);
this.bloomFilterWriter = bloomFilterWriter;
- Set<String> bloomFilterColumns = props.getBloomFilterColumns();
- String column = String.join(".", path.getPath());
- if (!bloomFilterColumns.contains(column)) {
+ if (bloomFilterWriter == null) {
this.bloomFilter = null;
return;
}
int maxBloomFilterSize = props.getMaxBloomFilterBytes();
- Map<String, Long> bloomFilterColumnExpectedNDVs = props.getBloomFilterColumnExpectedNDVs();
- if (bloomFilterColumnExpectedNDVs.size() > 0) {
- // If user specify the column NDV, we construct Bloom filter from it.
- if (bloomFilterColumnExpectedNDVs.containsKey(column)) {
- int optimalNumOfBits = BlockSplitBloomFilter.optimalNumOfBits(
- bloomFilterColumnExpectedNDVs.get(column).intValue(), BlockSplitBloomFilter.DEFAULT_FPP);
-
- this.bloomFilter = new BlockSplitBloomFilter(optimalNumOfBits / 8, maxBloomFilterSize);
- } else {
- this.bloomFilter = null;
- }
+ OptionalLong ndv = props.getBloomFilterNDV(path);
+ // If user specify the column NDV, we construct Bloom filter from it.
+ if (ndv.isPresent()) {
+ int optimalNumOfBits = BlockSplitBloomFilter.optimalNumOfBits(ndv.getAsLong(), BlockSplitBloomFilter.DEFAULT_FPP);
+ this.bloomFilter = new BlockSplitBloomFilter(optimalNumOfBits / 8, maxBloomFilterSize);
} else {
this.bloomFilter = new BlockSplitBloomFilter(maxBloomFilterSize);
}
diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java
index d925e1d..a4c8e45 100644
--- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java
+++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetOutputFormat.java
@@ -18,16 +18,12 @@
*/
package org.apache.parquet.hadoop;
+import static org.apache.parquet.column.ParquetProperties.DEFAULT_BLOOM_FILTER_ENABLED;
import static org.apache.parquet.hadoop.ParquetWriter.DEFAULT_BLOCK_SIZE;
import static org.apache.parquet.hadoop.util.ContextUtil.getConfiguration;
import java.io.IOException;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
import java.util.Objects;
-import java.util.Set;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
@@ -150,7 +146,7 @@ public class ParquetOutputFormat<T> extends FileOutputFormat<Void, T> {
public static final String ESTIMATE_PAGE_SIZE_CHECK = "parquet.page.size.check.estimate";
public static final String COLUMN_INDEX_TRUNCATE_LENGTH = "parquet.columnindex.truncate.length";
public static final String STATISTICS_TRUNCATE_LENGTH = "parquet.statistics.truncate.length";
- public static final String BLOOM_FILTER_COLUMN_NAMES = "parquet.bloom.filter.column.names";
+ public static final String BLOOM_FILTER_ENABLED = "parquet.bloom.filter.enabled";
public static final String BLOOM_FILTER_EXPECTED_NDV = "parquet.bloom.filter.expected.ndv";
public static final String BLOOM_FILTER_MAX_BYTES = "parquet.bloom.filter.max.bytes";
public static final String PAGE_ROW_COUNT_LIMIT = "parquet.page.row.count.limit";
@@ -225,38 +221,9 @@ public class ParquetOutputFormat<T> extends FileOutputFormat<Void, T> {
ParquetProperties.DEFAULT_MAX_BLOOM_FILTER_BYTES);
}
- public static Set<String> getBloomFilterColumns(Configuration conf) {
- String columnNames = conf.get(BLOOM_FILTER_COLUMN_NAMES);
- if (columnNames != null) {
- return new HashSet<>(Arrays.asList(columnNames.split(",")));
- } else {
- return new HashSet<>();
- }
+ public static boolean getBloomFilterEnabled(Configuration conf) {
+ return conf.getBoolean(BLOOM_FILTER_ENABLED, DEFAULT_BLOOM_FILTER_ENABLED);
}
-
- public static Map<String, Long> getBloomFilterColumnExpectedNDVs(Configuration conf) {
- Map<String, Long> kv = new HashMap<>();
- String columnNamesConf = conf.get(BLOOM_FILTER_COLUMN_NAMES);
- String expectedNDVsConf = conf.get(BLOOM_FILTER_EXPECTED_NDV);
-
- if (columnNamesConf == null || expectedNDVsConf == null) {
- return kv;
- }
-
- String[] columnNames = columnNamesConf.split(",");
- String[] expectedNDVs = expectedNDVsConf.split(",");
-
- if (columnNames.length == expectedNDVs.length) {
- for (int i = 0; i < columnNames.length; i++) {
- kv.put(columnNames[i], Long.parseLong(expectedNDVs[i]));
- }
- } else {
- LOG.warn("Bloom filter column names are not match expected NDVs");
- }
-
- return kv;
- }
-
public static int getBlockSize(JobContext jobContext) {
return getBlockSize(getConfiguration(jobContext));
}
@@ -481,11 +448,14 @@ public class ParquetOutputFormat<T> extends FileOutputFormat<Void, T> {
.withColumnIndexTruncateLength(getColumnIndexTruncateLength(conf))
.withStatisticsTruncateLength(getStatisticsTruncateLength(conf))
.withMaxBloomFilterBytes(getBloomFilterMaxBytes(conf))
- .withBloomFilterColumnToNDVMap(getBloomFilterColumnExpectedNDVs(conf))
+ .withBloomFilterEnabled(getBloomFilterEnabled(conf))
.withPageRowCountLimit(getPageRowCountLimit(conf))
.withPageWriteChecksumEnabled(getPageWriteChecksumEnabled(conf));
new ColumnConfigParser()
.withColumnConfig(ENABLE_DICTIONARY, key -> conf.getBoolean(key, false), propsBuilder::withDictionaryEncoding)
+ .withColumnConfig(BLOOM_FILTER_ENABLED, key -> conf.getBoolean(key, false),
+ propsBuilder::withBloomFilterEnabled)
+ .withColumnConfig(BLOOM_FILTER_EXPECTED_NDV, key -> conf.getLong(key, -1L), propsBuilder::withBloomFilterNDV)
.parseConfig(conf);
ParquetProperties props = propsBuilder.build();
diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java
index 4637069..c6b2828 100644
--- a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java
+++ b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetWriter.java
@@ -20,8 +20,6 @@ package org.apache.parquet.hadoop;
import java.io.Closeable;
import java.io.IOException;
-import java.util.Arrays;
-import java.util.Map;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
@@ -558,30 +556,39 @@ public class ParquetWriter<T> implements Closeable {
}
/**
- * Specified the column and the NDV map for the bloom filter.
+ * Sets the NDV (number of distinct values) for the specified column.
*
- * @param columnNDVMap the column to NDV map
+ * @param columnPath the path of the column (dot-string)
+ * @param ndv the NDV of the column
*
* @return this builder for method chaining.
*/
- public SELF withBloomFilterColumnToNDVMap(Map<String, Long> columnNDVMap) {
- if (columnNDVMap != null) {
- encodingPropsBuilder.withBloomFilterColumnToNDVMap(columnNDVMap);
- }
-
+ public SELF withBloomFilterNDV(String columnPath, long ndv) {
+ encodingPropsBuilder.withBloomFilterNDV(columnPath, ndv);
return self();
}
/**
- * Specified the column names for the bloom filter.
+ * Sets the bloom filter enabled/disabled
*
- * @return this builder for method chaining.
+ * @param enabled whether to write bloom filters
+ * @return this builder for method chaining
*/
- public SELF withBloomFilterColumnNames(String...columns) {
- if (columns != null) {
- encodingPropsBuilder.withBloomFilterColumnNames(Arrays.asList(columns));
- }
+ public SELF withBloomFilterEnabled(boolean enabled) {
+ encodingPropsBuilder.withBloomFilterEnabled(enabled);
+ return self();
+ }
+ /**
+ * Sets the bloom filter enabled/disabled for the specified column. If not set for the column specifically the
+ * default enabled/disabled state will take place. See {@link #withBloomFilterEnabled(boolean)}.
+ *
+ * @param columnPath the path of the column (dot-string)
+ * @param enabled whether to write bloom filter for the column
+ * @return this builder for method chaining
+ */
+ public SELF withBloomFilterEnabled(String columnPath, boolean enabled) {
+ encodingPropsBuilder.withBloomFilterEnabled(columnPath, enabled);
return self();
}
diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestBloomFiltering.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestBloomFiltering.java
index 7246d36..3a6a002 100644
--- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestBloomFiltering.java
+++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestBloomFiltering.java
@@ -207,22 +207,22 @@ public class TestBloomFiltering {
public static void createFile() throws IOException {
int pageSize = DATA.size() / 100; // Ensure that several pages will be created
int rowGroupSize = pageSize * 4; // Ensure that there are more row-groups created
- Map<String, Long> column2NDVMap = new HashMap<>();
- column2NDVMap.put("location.lat", 10000L);
- column2NDVMap.put("name", 10000L);
- column2NDVMap.put("id", 10000L);
PhoneBookWriter.write(ExampleParquetWriter.builder(FILE_V1)
.withWriteMode(OVERWRITE)
.withRowGroupSize(rowGroupSize)
.withPageSize(pageSize)
- .withBloomFilterColumnToNDVMap(column2NDVMap)
+ .withBloomFilterNDV("location.lat", 10000L)
+ .withBloomFilterNDV("name", 10000L)
+ .withBloomFilterNDV("id", 10000L)
.withWriterVersion(ParquetProperties.WriterVersion.PARQUET_1_0),
DATA);
PhoneBookWriter.write(ExampleParquetWriter.builder(FILE_V2)
.withWriteMode(OVERWRITE)
.withRowGroupSize(rowGroupSize)
.withPageSize(pageSize)
- .withBloomFilterColumnToNDVMap(column2NDVMap)
+ .withBloomFilterNDV("location.lat", 10000L)
+ .withBloomFilterNDV("name", 10000L)
+ .withBloomFilterNDV("id", 10000L)
.withWriterVersion(ParquetProperties.WriterVersion.PARQUET_2_0),
DATA);
}
diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriter.java b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriter.java
index fde3fa0..868af8e 100644
--- a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriter.java
+++ b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestParquetWriter.java
@@ -221,7 +221,7 @@ public class TestParquetWriter {
.withPageRowCountLimit(10)
.withConf(conf)
.withDictionaryEncoding(false)
- .withBloomFilterColumnNames("name")
+ .withBloomFilterEnabled("name", true)
.build()) {
for (String testName : testNames) {
writer.write(factory.newGroup().append("name", testName));