-
Notifications
You must be signed in to change notification settings - Fork 2.3k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Use compatible column name to set Parquet bloom filter #11799
Changes from 2 commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change | ||||
---|---|---|---|---|---|---|
|
@@ -55,6 +55,7 @@ | |||||
import java.util.Locale; | ||||||
import java.util.Map; | ||||||
import java.util.Objects; | ||||||
import java.util.function.BiConsumer; | ||||||
import java.util.function.Function; | ||||||
import java.util.stream.Collectors; | ||||||
import java.util.stream.IntStream; | ||||||
|
@@ -95,6 +96,7 @@ | |||||
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; | ||||||
import org.apache.iceberg.relocated.com.google.common.collect.Maps; | ||||||
import org.apache.iceberg.relocated.com.google.common.collect.Sets; | ||||||
import org.apache.iceberg.types.Types; | ||||||
import org.apache.iceberg.util.ArrayUtil; | ||||||
import org.apache.iceberg.util.ByteBuffers; | ||||||
import org.apache.iceberg.util.PropertyUtil; | ||||||
|
@@ -115,8 +117,12 @@ | |||||
import org.apache.parquet.hadoop.api.WriteSupport; | ||||||
import org.apache.parquet.hadoop.metadata.CompressionCodecName; | ||||||
import org.apache.parquet.schema.MessageType; | ||||||
import org.slf4j.Logger; | ||||||
import org.slf4j.LoggerFactory; | ||||||
|
||||||
public class Parquet { | ||||||
private static final Logger LOG = LoggerFactory.getLogger(Parquet.class); | ||||||
|
||||||
private Parquet() {} | ||||||
|
||||||
private static final Collection<String> READ_PROPERTIES_TO_REMOVE = | ||||||
|
@@ -266,6 +272,43 @@ private WriteBuilder createContextFunc( | |||||
return this; | ||||||
} | ||||||
|
||||||
private <T> void setBloomFilterConfig( | ||||||
Context context, | ||||||
MessageType parquetSchema, | ||||||
BiConsumer<String, Boolean> withBloomFilterEnabled, | ||||||
BiConsumer<String, Double> withBloomFilterFPP) { | ||||||
|
||||||
Map<Integer, String> fieldIdToParquetPath = | ||||||
parquetSchema.getColumns().stream() | ||||||
.collect( | ||||||
Collectors.toMap( | ||||||
col -> col.getPrimitiveType().getId().intValue(), | ||||||
col -> String.join(".", col.getPath()))); | ||||||
|
||||||
context | ||||||
.columnBloomFilterEnabled() | ||||||
.forEach( | ||||||
(colPath, isEnabled) -> { | ||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. [question] do we need to do anything for isEnabled as false ? or can parquet pro-actively decide if it should have a BF for a column and this isEnabled as false serves as explicit deny ? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If isEnable is true, iceberg will call withBloomFilterEnabled(String columnPath, boolean enabled). If isEnable is false, we don't need to do anything. |
||||||
Types.NestedField fieldId = schema.findField(colPath); | ||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. can we call this as field instead ?
Suggested change
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Updated. Thanks! |
||||||
if (fieldId == null) { | ||||||
LOG.warn("Skipping bloom filter config for missing field: {}", colPath); | ||||||
return; | ||||||
} | ||||||
|
||||||
String parquetColumnPath = fieldIdToParquetPath.get(fieldId.fieldId()); | ||||||
if (parquetColumnPath == null) { | ||||||
LOG.warn("Skipping bloom filter config for missing field: {}", fieldId); | ||||||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. should we update this message to say something like
Suggested change
mostly coming from the above log lines are identical mostly though at one part we add columnPath and the other we do fielId There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Fixed. Thanks! |
||||||
return; | ||||||
} | ||||||
|
||||||
withBloomFilterEnabled.accept(parquetColumnPath, Boolean.valueOf(isEnabled)); | ||||||
String fpp = context.columnBloomFilterFpp().get(colPath); | ||||||
if (fpp != null) { | ||||||
withBloomFilterFPP.accept(parquetColumnPath, Double.parseDouble(fpp)); | ||||||
} | ||||||
}); | ||||||
} | ||||||
|
||||||
public <D> FileAppender<D> build() throws IOException { | ||||||
Preconditions.checkNotNull(schema, "Schema is required"); | ||||||
Preconditions.checkNotNull(name, "Table name is required and cannot be null"); | ||||||
|
@@ -285,8 +328,6 @@ public <D> FileAppender<D> build() throws IOException { | |||||
int rowGroupCheckMinRecordCount = context.rowGroupCheckMinRecordCount(); | ||||||
int rowGroupCheckMaxRecordCount = context.rowGroupCheckMaxRecordCount(); | ||||||
int bloomFilterMaxBytes = context.bloomFilterMaxBytes(); | ||||||
Map<String, String> columnBloomFilterFpp = context.columnBloomFilterFpp(); | ||||||
Map<String, String> columnBloomFilterEnabled = context.columnBloomFilterEnabled(); | ||||||
boolean dictionaryEnabled = context.dictionaryEnabled(); | ||||||
|
||||||
if (compressionLevel != null) { | ||||||
|
@@ -343,17 +384,8 @@ public <D> FileAppender<D> build() throws IOException { | |||||
.withMaxRowCountForPageSizeCheck(rowGroupCheckMaxRecordCount) | ||||||
.withMaxBloomFilterBytes(bloomFilterMaxBytes); | ||||||
|
||||||
for (Map.Entry<String, String> entry : columnBloomFilterEnabled.entrySet()) { | ||||||
String colPath = entry.getKey(); | ||||||
String bloomEnabled = entry.getValue(); | ||||||
propsBuilder.withBloomFilterEnabled(colPath, Boolean.parseBoolean(bloomEnabled)); | ||||||
} | ||||||
|
||||||
for (Map.Entry<String, String> entry : columnBloomFilterFpp.entrySet()) { | ||||||
String colPath = entry.getKey(); | ||||||
String fpp = entry.getValue(); | ||||||
propsBuilder.withBloomFilterFPP(colPath, Double.parseDouble(fpp)); | ||||||
} | ||||||
setBloomFilterConfig( | ||||||
context, type, propsBuilder::withBloomFilterEnabled, propsBuilder::withBloomFilterFPP); | ||||||
|
||||||
ParquetProperties parquetProperties = propsBuilder.build(); | ||||||
|
||||||
|
@@ -386,17 +418,11 @@ public <D> FileAppender<D> build() throws IOException { | |||||
.withDictionaryPageSize(dictionaryPageSize) | ||||||
.withEncryption(fileEncryptionProperties); | ||||||
|
||||||
for (Map.Entry<String, String> entry : columnBloomFilterEnabled.entrySet()) { | ||||||
String colPath = entry.getKey(); | ||||||
String bloomEnabled = entry.getValue(); | ||||||
parquetWriteBuilder.withBloomFilterEnabled(colPath, Boolean.parseBoolean(bloomEnabled)); | ||||||
} | ||||||
|
||||||
for (Map.Entry<String, String> entry : columnBloomFilterFpp.entrySet()) { | ||||||
String colPath = entry.getKey(); | ||||||
String fpp = entry.getValue(); | ||||||
parquetWriteBuilder.withBloomFilterFPP(colPath, Double.parseDouble(fpp)); | ||||||
} | ||||||
setBloomFilterConfig( | ||||||
context, | ||||||
type, | ||||||
parquetWriteBuilder::withBloomFilterEnabled, | ||||||
parquetWriteBuilder::withBloomFilterFPP); | ||||||
|
||||||
return new ParquetWriteAdapter<>(parquetWriteBuilder.build(), metricsConfig); | ||||||
} | ||||||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -109,7 +109,7 @@ public class TestBloomRowGroupFilter { | |
optional(22, "timestamp", Types.TimestampType.withoutZone()), | ||
optional(23, "timestamptz", Types.TimestampType.withZone()), | ||
optional(24, "binary", Types.BinaryType.get()), | ||
optional(25, "int_decimal", Types.DecimalType.of(8, 2)), | ||
optional(25, "int-decimal", Types.DecimalType.of(8, 2)), | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Minor quibble here, let's not re-use this field and instead add a new field like we did with "non_bloom", "struct_not_null" etc ... optional(28, "incompatible-name", StringType.get()) There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Changed. Thanks |
||
optional(26, "long_decimal", Types.DecimalType.of(14, 2)), | ||
optional(27, "fixed_decimal", Types.DecimalType.of(31, 2))); | ||
|
||
|
@@ -140,7 +140,7 @@ public class TestBloomRowGroupFilter { | |
optional(22, "_timestamp", Types.TimestampType.withoutZone()), | ||
optional(23, "_timestamptz", Types.TimestampType.withZone()), | ||
optional(24, "_binary", Types.BinaryType.get()), | ||
optional(25, "_int_decimal", Types.DecimalType.of(8, 2)), | ||
optional(25, "_int-decimal", Types.DecimalType.of(8, 2)), | ||
optional(26, "_long_decimal", Types.DecimalType.of(14, 2)), | ||
optional(27, "_fixed_decimal", Types.DecimalType.of(31, 2))); | ||
|
||
|
@@ -193,6 +193,7 @@ public void createInputFile() throws IOException { | |
|
||
// build struct field schema | ||
org.apache.avro.Schema structSchema = AvroSchemaUtil.convert(UNDERSCORE_STRUCT_FIELD_TYPE); | ||
String compatibleFieldName = AvroSchemaUtil.makeCompatibleName("_int-decimal"); | ||
|
||
OutputFile outFile = Files.localOutput(temp); | ||
try (FileAppender<Record> appender = | ||
|
@@ -221,7 +222,7 @@ public void createInputFile() throws IOException { | |
.set(PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "_timestamp", "true") | ||
.set(PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "_timestamptz", "true") | ||
.set(PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "_binary", "true") | ||
.set(PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "_int_decimal", "true") | ||
.set(PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "_int-decimal", "true") | ||
.set(PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "_long_decimal", "true") | ||
.set(PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "_fixed_decimal", "true") | ||
.build()) { | ||
|
@@ -256,7 +257,7 @@ public void createInputFile() throws IOException { | |
builder.set("_timestamp", INSTANT.plusSeconds(i * 86400).toEpochMilli()); | ||
builder.set("_timestamptz", INSTANT.plusSeconds(i * 86400).toEpochMilli()); | ||
builder.set("_binary", RANDOM_BYTES.get(i)); | ||
builder.set("_int_decimal", new BigDecimal(String.valueOf(77.77 + i))); | ||
builder.set(compatibleFieldName, new BigDecimal(String.valueOf(77.77 + i))); | ||
builder.set("_long_decimal", new BigDecimal(String.valueOf(88.88 + i))); | ||
builder.set("_fixed_decimal", new BigDecimal(String.valueOf(99.99 + i))); | ||
|
||
|
@@ -683,23 +684,23 @@ public void testBytesEq() { | |
} | ||
|
||
@Test | ||
public void testIntDeciamlEq() { | ||
public void testIntDecimalEq() { | ||
for (int i = 0; i < INT_VALUE_COUNT; i++) { | ||
boolean shouldRead = | ||
new ParquetBloomRowGroupFilter( | ||
SCHEMA, equal("int_decimal", new BigDecimal(String.valueOf(77.77 + i)))) | ||
SCHEMA, equal("int-decimal", new BigDecimal(String.valueOf(77.77 + i)))) | ||
.shouldRead(parquetSchema, rowGroupMetadata, bloomStore); | ||
assertThat(shouldRead).as("Should read: decimal within range").isTrue(); | ||
} | ||
|
||
boolean shouldRead = | ||
new ParquetBloomRowGroupFilter(SCHEMA, equal("int_decimal", new BigDecimal("1234.56"))) | ||
new ParquetBloomRowGroupFilter(SCHEMA, equal("int-decimal", new BigDecimal("1234.56"))) | ||
.shouldRead(parquetSchema, rowGroupMetadata, bloomStore); | ||
assertThat(shouldRead).as("Should not read: decimal outside range").isFalse(); | ||
} | ||
|
||
@Test | ||
public void testLongDeciamlEq() { | ||
public void testLongDecimalEq() { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Let's keep this fix! |
||
for (int i = 0; i < INT_VALUE_COUNT; i++) { | ||
boolean shouldRead = | ||
new ParquetBloomRowGroupFilter( | ||
|
@@ -715,7 +716,7 @@ SCHEMA, equal("long_decimal", new BigDecimal(String.valueOf(88.88 + i)))) | |
} | ||
|
||
@Test | ||
public void testFixedDeciamlEq() { | ||
public void testFixedDecimalEq() { | ||
for (int i = 0; i < INT_VALUE_COUNT; i++) { | ||
boolean shouldRead = | ||
new ParquetBloomRowGroupFilter( | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is it possible to use the same Name compat function we use in the parquet writer to get the compatible column names? Just seems a bit odd to look up the paths when we know how to generate them?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
For example couldn't we keep the old layout of the code and just use
iceberg/core/src/main/java/org/apache/iceberg/avro/AvroSchemaUtil.java
Line 483 in afda8be
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Feel free to tell me if this is a bad idea, I think your approach is fine as well since it does seem to be a better validation
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It's simpler to use
makeCompatibleName
. I have made the changes. Thanks!