Skip to content

Commit d9a1962

Browse files
authored
PARQUET-1510: Fix notEq for optional columns with null values. (#603)
Dictionaries cannot contain null values, so notEq filters cannot conclude that a block cannot match using only the dictionary. Instead, it must also check whether the block may have at least one null value. If there are no null values, then the existing check is correct.
1 parent ddc7747 commit d9a1962

File tree

2 files changed

+19
-3
lines changed

2 files changed

+19
-3
lines changed

parquet-hadoop/src/main/java/org/apache/parquet/filter2/dictionarylevel/DictionaryFilter.java

Lines changed: 4 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -189,7 +189,10 @@ public <T extends Comparable<T>> Boolean visit(NotEq<T> notEq) {
189189

190190
try {
191191
Set<T> dictSet = expandDictionary(meta);
192-
if (dictSet != null && dictSet.size() == 1 && dictSet.contains(value)) {
192+
boolean mayContainNull = (meta.getStatistics() == null
193+
|| !meta.getStatistics().isNumNullsSet()
194+
|| meta.getStatistics().getNumNulls() > 0);
195+
if (dictSet != null && dictSet.size() == 1 && dictSet.contains(value) && !mayContainNull) {
193196
return BLOCK_CANNOT_MATCH;
194197
}
195198
} catch (IOException e) {

parquet-hadoop/src/test/java/org/apache/parquet/filter2/dictionarylevel/DictionaryFilterTest.java

Lines changed: 15 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -88,6 +88,7 @@ public class DictionaryFilterTest {
8888
"message test { "
8989
+ "required binary binary_field; "
9090
+ "required binary single_value_field; "
91+
+ "optional binary optional_single_value_field; "
9192
+ "required fixed_len_byte_array(17) fixed_field (DECIMAL(40,4)); "
9293
+ "required int32 int32_field; "
9394
+ "required int64 int64_field; "
@@ -165,6 +166,11 @@ private static void writeData(SimpleGroupFactory f, ParquetWriter<Group> writer)
165166
ALPHABET.substring(index, index+1) : UUID.randomUUID().toString())
166167
.append("int96_field", INT96_VALUES[i % INT96_VALUES.length]);
167168

169+
// 10% of the time, leave the field null
170+
if (index % 10 > 0) {
171+
group.append("optional_single_value_field", "sharp");
172+
}
173+
168174
writer.write(group);
169175
}
170176
writer.close();
@@ -256,7 +262,7 @@ public void testDictionaryEncodedColumns() throws Exception {
256262
@SuppressWarnings("deprecation")
257263
private void testDictionaryEncodedColumnsV1() throws Exception {
258264
Set<String> dictionaryEncodedColumns = new HashSet<String>(Arrays.asList(
259-
"binary_field", "single_value_field", "int32_field", "int64_field",
265+
"binary_field", "single_value_field", "optional_single_value_field", "int32_field", "int64_field",
260266
"double_field", "float_field", "int96_field"));
261267
for (ColumnChunkMetaData column : ccmd) {
262268
String name = column.getPath().toDotString();
@@ -281,7 +287,7 @@ private void testDictionaryEncodedColumnsV1() throws Exception {
281287

282288
private void testDictionaryEncodedColumnsV2() throws Exception {
283289
Set<String> dictionaryEncodedColumns = new HashSet<String>(Arrays.asList(
284-
"binary_field", "single_value_field", "fixed_field", "int32_field",
290+
"binary_field", "single_value_field", "optional_single_value_field", "fixed_field", "int32_field",
285291
"int64_field", "double_field", "float_field", "int96_field"));
286292
for (ColumnChunkMetaData column : ccmd) {
287293
EncodingStats encStats = column.getEncodingStats();
@@ -355,6 +361,7 @@ public void testEqInt96() throws Exception {
355361
@Test
356362
public void testNotEqBinary() throws Exception {
357363
BinaryColumn sharp = binaryColumn("single_value_field");
364+
BinaryColumn sharpAndNull = binaryColumn("optional_single_value_field");
358365
BinaryColumn b = binaryColumn("binary_field");
359366

360367
assertTrue("Should drop block with only the excluded value",
@@ -363,6 +370,12 @@ public void testNotEqBinary() throws Exception {
363370
assertFalse("Should not drop block with any other value",
364371
canDrop(notEq(sharp, Binary.fromString("applause")), ccmd, dictionaries));
365372

373+
assertFalse("Should not drop block with only the excluded value and null",
374+
canDrop(notEq(sharpAndNull, Binary.fromString("sharp")), ccmd, dictionaries));
375+
376+
assertFalse("Should not drop block with any other value",
377+
canDrop(notEq(sharpAndNull, Binary.fromString("applause")), ccmd, dictionaries));
378+
366379
assertFalse("Should not drop block with a known value",
367380
canDrop(notEq(b, Binary.fromString("x")), ccmd, dictionaries));
368381

0 commit comments

Comments
 (0)