Skip to content

Commit 5110562

Browse files
Alexey Kudinkinyuzhaojing
authored andcommitted
[HUDI-4992] Fixing invalid min/max record key stats in Parquet metadata (#6883)
1 parent 28cb191 commit 5110562

10 files changed

Lines changed: 361 additions & 221 deletions

File tree

hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/storage/HoodieOrcWriter.java

Lines changed: 4 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@
2323
import org.apache.avro.generic.IndexedRecord;
2424
import org.apache.hadoop.conf.Configuration;
2525
import org.apache.hadoop.fs.Path;
26+
import org.apache.hudi.avro.HoodieBloomFilterWriteSupport;
2627
import org.apache.hudi.common.bloom.BloomFilter;
2728
import org.apache.hudi.common.bloom.HoodieDynamicBoundedBloomFilter;
2829
import org.apache.hudi.common.engine.TaskContextSupplier;
@@ -44,9 +45,6 @@
4445
import java.util.concurrent.atomic.AtomicLong;
4546

4647
import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY;
47-
import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_BLOOM_FILTER_TYPE_CODE;
48-
import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_MAX_RECORD_KEY_FOOTER;
49-
import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_MIN_RECORD_KEY_FOOTER;
5048

5149
public class HoodieOrcWriter<T extends HoodieRecordPayload, R extends IndexedRecord>
5250
implements HoodieFileWriter<R>, Closeable {
@@ -155,11 +153,11 @@ public void close() throws IOException {
155153
final BloomFilter bloomFilter = orcConfig.getBloomFilter();
156154
writer.addUserMetadata(HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY, ByteBuffer.wrap(bloomFilter.serializeToString().getBytes()));
157155
if (minRecordKey != null && maxRecordKey != null) {
158-
writer.addUserMetadata(HOODIE_MIN_RECORD_KEY_FOOTER, ByteBuffer.wrap(minRecordKey.getBytes()));
159-
writer.addUserMetadata(HOODIE_MAX_RECORD_KEY_FOOTER, ByteBuffer.wrap(maxRecordKey.getBytes()));
156+
writer.addUserMetadata(HoodieBloomFilterWriteSupport.HOODIE_MIN_RECORD_KEY_FOOTER, ByteBuffer.wrap(minRecordKey.getBytes()));
157+
writer.addUserMetadata(HoodieBloomFilterWriteSupport.HOODIE_MAX_RECORD_KEY_FOOTER, ByteBuffer.wrap(maxRecordKey.getBytes()));
160158
}
161159
if (bloomFilter.getBloomFilterTypeCode().name().contains(HoodieDynamicBoundedBloomFilter.TYPE_CODE_PREFIX)) {
162-
writer.addUserMetadata(HOODIE_BLOOM_FILTER_TYPE_CODE, ByteBuffer.wrap(bloomFilter.getBloomFilterTypeCode().name().getBytes()));
160+
writer.addUserMetadata(HoodieBloomFilterWriteSupport.HOODIE_BLOOM_FILTER_TYPE_CODE, ByteBuffer.wrap(bloomFilter.getBloomFilterTypeCode().name().getBytes()));
163161
}
164162
}
165163
writer.addUserMetadata(HoodieOrcConfig.AVRO_SCHEMA_METADATA_KEY, ByteBuffer.wrap(avroSchema.toString().getBytes()));
Lines changed: 118 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,118 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one
3+
* or more contributor license agreements. See the NOTICE file
4+
* distributed with this work for additional information
5+
* regarding copyright ownership. The ASF licenses this file
6+
* to you under the Apache License, Version 2.0 (the
7+
* "License"); you may not use this file except in compliance
8+
* with the License. You may obtain a copy of the License at
9+
*
10+
* http://www.apache.org/licenses/LICENSE-2.0
11+
*
12+
* Unless required by applicable law or agreed to in writing, software
13+
* distributed under the License is distributed on an "AS IS" BASIS,
14+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15+
* See the License for the specific language governing permissions and
16+
* limitations under the License.
17+
*/
18+
19+
package org.apache.hudi.avro;
20+
21+
import org.apache.avro.Schema;
22+
import org.apache.avro.generic.GenericRecord;
23+
import org.apache.hadoop.conf.Configuration;
24+
import org.apache.hadoop.fs.Path;
25+
import org.apache.hudi.DummyTaskContextSupplier;
26+
import org.apache.hudi.common.bloom.BloomFilter;
27+
import org.apache.hudi.common.bloom.BloomFilterFactory;
28+
import org.apache.hudi.common.bloom.BloomFilterTypeCode;
29+
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
30+
import org.apache.hudi.common.util.Option;
31+
import org.apache.hudi.common.util.ParquetUtils;
32+
import org.apache.hudi.io.storage.HoodieAvroParquetWriter;
33+
import org.apache.hudi.io.storage.HoodieParquetConfig;
34+
import org.apache.parquet.avro.AvroSchemaConverter;
35+
import org.apache.parquet.hadoop.ParquetWriter;
36+
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
37+
import org.apache.parquet.hadoop.metadata.FileMetaData;
38+
import org.junit.jupiter.api.Test;
39+
import org.junit.jupiter.api.io.TempDir;
40+
41+
import java.io.IOException;
42+
import java.util.Comparator;
43+
import java.util.List;
44+
import java.util.Map;
45+
import java.util.stream.Collectors;
46+
47+
import static org.junit.jupiter.api.Assertions.assertEquals;
48+
import static org.junit.jupiter.api.Assertions.assertTrue;
49+
50+
public class TestHoodieAvroParquetWriter {
51+
52+
@TempDir java.nio.file.Path tmpDir;
53+
54+
@Test
55+
public void testProperWriting() throws IOException {
56+
Configuration hadoopConf = new Configuration();
57+
58+
HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(0xDEED);
59+
List<GenericRecord> records = dataGen.generateGenericRecords(10);
60+
61+
Schema schema = records.get(0).getSchema();
62+
63+
BloomFilter filter = BloomFilterFactory.createBloomFilter(1000, 0.0001, 10000,
64+
BloomFilterTypeCode.DYNAMIC_V0.name());
65+
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema),
66+
schema, Option.of(filter));
67+
68+
HoodieParquetConfig<HoodieAvroWriteSupport> parquetConfig =
69+
new HoodieParquetConfig(writeSupport, CompressionCodecName.GZIP, ParquetWriter.DEFAULT_BLOCK_SIZE,
70+
ParquetWriter.DEFAULT_PAGE_SIZE, 1024 * 1024 * 1024, hadoopConf, 0.1);
71+
72+
Path filePath = new Path(tmpDir.resolve("test.parquet").toAbsolutePath().toString());
73+
74+
try (HoodieAvroParquetWriter<GenericRecord> writer =
75+
new HoodieAvroParquetWriter<>(filePath, parquetConfig, "001", new DummyTaskContextSupplier(), true)) {
76+
for (GenericRecord record : records) {
77+
writer.writeAvro((String) record.get("_row_key"), record);
78+
}
79+
}
80+
81+
ParquetUtils utils = new ParquetUtils();
82+
83+
// Step 1: Make sure records are written appropriately
84+
List<GenericRecord> readRecords = utils.readAvroRecords(hadoopConf, filePath);
85+
86+
assertEquals(toJson(records), toJson(readRecords));
87+
88+
// Step 2: Assert Parquet metadata was written appropriately
89+
List<String> recordKeys = records.stream().map(r -> (String) r.get("_row_key")).collect(Collectors.toList());
90+
91+
String minKey = recordKeys.stream().min(Comparator.naturalOrder()).get();
92+
String maxKey = recordKeys.stream().max(Comparator.naturalOrder()).get();
93+
94+
FileMetaData parquetMetadata = ParquetUtils.readMetadata(hadoopConf, filePath).getFileMetaData();
95+
96+
Map<String, String> extraMetadata = parquetMetadata.getKeyValueMetaData();
97+
98+
assertEquals(extraMetadata.get(HoodieBloomFilterWriteSupport.HOODIE_MIN_RECORD_KEY_FOOTER), minKey);
99+
assertEquals(extraMetadata.get(HoodieBloomFilterWriteSupport.HOODIE_MAX_RECORD_KEY_FOOTER), maxKey);
100+
assertEquals(extraMetadata.get(HoodieBloomFilterWriteSupport.HOODIE_BLOOM_FILTER_TYPE_CODE), BloomFilterTypeCode.DYNAMIC_V0.name());
101+
102+
// Step 3: Make sure Bloom Filter contains all the record keys
103+
BloomFilter bloomFilter = utils.readBloomFilterFromMetadata(hadoopConf, filePath);
104+
recordKeys.forEach(recordKey -> {
105+
assertTrue(bloomFilter.mightContain(recordKey));
106+
});
107+
}
108+
109+
private static List<String> toJson(List<GenericRecord> records) {
110+
return records.stream().map(r -> {
111+
try {
112+
return new String(HoodieAvroUtils.avroToJson(r, true));
113+
} catch (IOException e) {
114+
throw new RuntimeException(e);
115+
}
116+
}).collect(Collectors.toList());
117+
}
118+
}

hudi-client/hudi-client-common/src/test/java/org/apache/hudi/io/storage/TestHoodieOrcReaderWriter.java

Lines changed: 3 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -18,6 +18,7 @@
1818

1919
package org.apache.hudi.io.storage;
2020

21+
import org.apache.hudi.avro.HoodieBloomFilterWriteSupport;
2122
import org.apache.hudi.common.bloom.BloomFilter;
2223
import org.apache.hudi.common.bloom.BloomFilterFactory;
2324
import org.apache.hudi.common.bloom.BloomFilterTypeCode;
@@ -37,8 +38,6 @@
3738
import java.util.function.Supplier;
3839

3940
import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY;
40-
import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_MAX_RECORD_KEY_FOOTER;
41-
import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_MIN_RECORD_KEY_FOOTER;
4241
import static org.apache.hudi.io.storage.HoodieOrcConfig.AVRO_SCHEMA_METADATA_KEY;
4342
import static org.junit.jupiter.api.Assertions.assertEquals;
4443
import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -78,8 +77,8 @@ protected HoodieFileReader<GenericRecord> createReader(
7877
protected void verifyMetadata(Configuration conf) throws IOException {
7978
Reader orcReader = OrcFile.createReader(getFilePath(), OrcFile.readerOptions(conf));
8079
assertEquals(4, orcReader.getMetadataKeys().size());
81-
assertTrue(orcReader.getMetadataKeys().contains(HOODIE_MIN_RECORD_KEY_FOOTER));
82-
assertTrue(orcReader.getMetadataKeys().contains(HOODIE_MAX_RECORD_KEY_FOOTER));
80+
assertTrue(orcReader.getMetadataKeys().contains(HoodieBloomFilterWriteSupport.HOODIE_MIN_RECORD_KEY_FOOTER));
81+
assertTrue(orcReader.getMetadataKeys().contains(HoodieBloomFilterWriteSupport.HOODIE_MAX_RECORD_KEY_FOOTER));
8382
assertTrue(orcReader.getMetadataKeys().contains(HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY));
8483
assertTrue(orcReader.getMetadataKeys().contains(AVRO_SCHEMA_METADATA_KEY));
8584
assertEquals(CompressionKind.ZLIB.name(), orcReader.getCompressionKind().toString());

hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowDataParquetWriteSupport.java

Lines changed: 23 additions & 32 deletions
Original file line numberDiff line numberDiff line change
@@ -18,35 +18,32 @@
1818

1919
package org.apache.hudi.io.storage.row;
2020

21+
import org.apache.hudi.avro.HoodieBloomFilterWriteSupport;
2122
import org.apache.hudi.common.bloom.BloomFilter;
22-
import org.apache.hudi.common.bloom.HoodieDynamicBoundedBloomFilter;
2323

2424
import org.apache.flink.table.data.RowData;
2525
import org.apache.flink.table.types.logical.RowType;
2626
import org.apache.hadoop.conf.Configuration;
27+
import org.apache.hudi.common.util.Option;
2728
import org.apache.parquet.hadoop.api.WriteSupport;
2829

29-
import java.util.HashMap;
30-
31-
import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY;
32-
import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_BLOOM_FILTER_TYPE_CODE;
33-
import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_MAX_RECORD_KEY_FOOTER;
34-
import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_MIN_RECORD_KEY_FOOTER;
30+
import java.nio.charset.StandardCharsets;
31+
import java.util.Collections;
32+
import java.util.Map;
3533

3634
/**
3735
* Hoodie Write Support for directly writing {@link RowData} to Parquet.
3836
*/
3937
public class HoodieRowDataParquetWriteSupport extends RowDataParquetWriteSupport {
4038

4139
private final Configuration hadoopConf;
42-
private final BloomFilter bloomFilter;
43-
private String minRecordKey;
44-
private String maxRecordKey;
40+
private final Option<HoodieBloomFilterWriteSupport<String>> bloomFilterWriteSupportOpt;
4541

4642
public HoodieRowDataParquetWriteSupport(Configuration conf, RowType rowType, BloomFilter bloomFilter) {
4743
super(rowType);
4844
this.hadoopConf = new Configuration(conf);
49-
this.bloomFilter = bloomFilter;
45+
this.bloomFilterWriteSupportOpt = Option.ofNullable(bloomFilter)
46+
.map(HoodieBloomFilterRowDataWriteSupport::new);
5047
}
5148

5249
public Configuration getHadoopConf() {
@@ -55,32 +52,26 @@ public Configuration getHadoopConf() {
5552

5653
@Override
5754
public WriteSupport.FinalizedWriteContext finalizeWrite() {
58-
HashMap<String, String> extraMetaData = new HashMap<>();
59-
if (bloomFilter != null) {
60-
extraMetaData.put(HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY, bloomFilter.serializeToString());
61-
if (minRecordKey != null && maxRecordKey != null) {
62-
extraMetaData.put(HOODIE_MIN_RECORD_KEY_FOOTER, minRecordKey);
63-
extraMetaData.put(HOODIE_MAX_RECORD_KEY_FOOTER, maxRecordKey);
64-
}
65-
if (bloomFilter.getBloomFilterTypeCode().name().contains(HoodieDynamicBoundedBloomFilter.TYPE_CODE_PREFIX)) {
66-
extraMetaData.put(HOODIE_BLOOM_FILTER_TYPE_CODE, bloomFilter.getBloomFilterTypeCode().name());
67-
}
68-
}
69-
return new WriteSupport.FinalizedWriteContext(extraMetaData);
55+
Map<String, String> extraMetadata =
56+
bloomFilterWriteSupportOpt.map(HoodieBloomFilterWriteSupport::finalizeMetadata)
57+
.orElse(Collections.emptyMap());
58+
59+
return new WriteSupport.FinalizedWriteContext(extraMetadata);
7060
}
7161

7262
public void add(String recordKey) {
73-
this.bloomFilter.add(recordKey);
74-
if (minRecordKey != null) {
75-
minRecordKey = minRecordKey.compareTo(recordKey) <= 0 ? minRecordKey : recordKey;
76-
} else {
77-
minRecordKey = recordKey;
63+
this.bloomFilterWriteSupportOpt.ifPresent(bloomFilterWriteSupport ->
64+
bloomFilterWriteSupport.addKey(recordKey));
65+
}
66+
67+
private static class HoodieBloomFilterRowDataWriteSupport extends HoodieBloomFilterWriteSupport<String> {
68+
public HoodieBloomFilterRowDataWriteSupport(BloomFilter bloomFilter) {
69+
super(bloomFilter);
7870
}
7971

80-
if (maxRecordKey != null) {
81-
maxRecordKey = maxRecordKey.compareTo(recordKey) >= 0 ? maxRecordKey : recordKey;
82-
} else {
83-
maxRecordKey = recordKey;
72+
@Override
73+
protected byte[] getUTF8Bytes(String key) {
74+
return key.getBytes(StandardCharsets.UTF_8);
8475
}
8576
}
8677
}

hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java

Lines changed: 29 additions & 32 deletions
Original file line numberDiff line numberDiff line change
@@ -19,41 +19,35 @@
1919
package org.apache.hudi.io.storage.row;
2020

2121
import org.apache.hadoop.conf.Configuration;
22+
import org.apache.hudi.avro.HoodieBloomFilterWriteSupport;
2223
import org.apache.hudi.common.bloom.BloomFilter;
23-
import org.apache.hudi.common.bloom.HoodieDynamicBoundedBloomFilter;
2424
import org.apache.hudi.common.util.Option;
2525
import org.apache.hudi.config.HoodieWriteConfig;
2626
import org.apache.parquet.hadoop.api.WriteSupport;
2727
import org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport;
2828
import org.apache.spark.sql.types.StructType;
2929
import org.apache.spark.unsafe.types.UTF8String;
3030

31-
import java.util.HashMap;
32-
33-
import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY;
34-
import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_BLOOM_FILTER_TYPE_CODE;
35-
import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_MAX_RECORD_KEY_FOOTER;
36-
import static org.apache.hudi.avro.HoodieAvroWriteSupport.HOODIE_MIN_RECORD_KEY_FOOTER;
31+
import java.util.Collections;
32+
import java.util.Map;
3733

3834
/**
3935
* Hoodie Write Support for directly writing Row to Parquet.
4036
*/
4137
public class HoodieRowParquetWriteSupport extends ParquetWriteSupport {
4238

4339
private final Configuration hadoopConf;
44-
private final BloomFilter bloomFilter;
45-
46-
private UTF8String minRecordKey;
47-
private UTF8String maxRecordKey;
40+
private final Option<HoodieBloomFilterWriteSupport<UTF8String>> bloomFilterWriteSupportOpt;
4841

4942
public HoodieRowParquetWriteSupport(Configuration conf, StructType structType, Option<BloomFilter> bloomFilterOpt, HoodieWriteConfig writeConfig) {
5043
Configuration hadoopConf = new Configuration(conf);
5144
hadoopConf.set("spark.sql.parquet.writeLegacyFormat", writeConfig.parquetWriteLegacyFormatEnabled());
5245
hadoopConf.set("spark.sql.parquet.outputTimestampType", writeConfig.parquetOutputTimestampType());
5346
hadoopConf.set("spark.sql.parquet.fieldId.write.enabled", writeConfig.parquetFieldIdWriteEnabled());
54-
this.hadoopConf = hadoopConf;
5547
setSchema(structType, hadoopConf);
56-
this.bloomFilter = bloomFilterOpt.orElse(null);
48+
49+
this.hadoopConf = hadoopConf;
50+
this.bloomFilterWriteSupportOpt = bloomFilterOpt.map(HoodieBloomFilterRowWriteSupport::new);
5751
}
5852

5953
public Configuration getHadoopConf() {
@@ -62,32 +56,35 @@ public Configuration getHadoopConf() {
6256

6357
@Override
6458
public WriteSupport.FinalizedWriteContext finalizeWrite() {
65-
HashMap<String, String> extraMetaData = new HashMap<>();
66-
if (bloomFilter != null) {
67-
extraMetaData.put(HOODIE_AVRO_BLOOM_FILTER_METADATA_KEY, bloomFilter.serializeToString());
68-
if (minRecordKey != null && maxRecordKey != null) {
69-
extraMetaData.put(HOODIE_MIN_RECORD_KEY_FOOTER, minRecordKey.toString());
70-
extraMetaData.put(HOODIE_MAX_RECORD_KEY_FOOTER, maxRecordKey.toString());
71-
}
72-
if (bloomFilter.getBloomFilterTypeCode().name().contains(HoodieDynamicBoundedBloomFilter.TYPE_CODE_PREFIX)) {
73-
extraMetaData.put(HOODIE_BLOOM_FILTER_TYPE_CODE, bloomFilter.getBloomFilterTypeCode().name());
74-
}
75-
}
76-
return new WriteSupport.FinalizedWriteContext(extraMetaData);
59+
Map<String, String> extraMetadata =
60+
bloomFilterWriteSupportOpt.map(HoodieBloomFilterWriteSupport::finalizeMetadata)
61+
.orElse(Collections.emptyMap());
62+
63+
return new WriteSupport.FinalizedWriteContext(extraMetadata);
7764
}
7865

7966
public void add(UTF8String recordKey) {
80-
this.bloomFilter.add(recordKey.getBytes());
67+
this.bloomFilterWriteSupportOpt.ifPresent(bloomFilterWriteSupport ->
68+
bloomFilterWriteSupport.addKey(recordKey));
69+
}
8170

82-
if (minRecordKey == null || minRecordKey.compareTo(recordKey) < 0) {
71+
private static class HoodieBloomFilterRowWriteSupport extends HoodieBloomFilterWriteSupport<UTF8String> {
72+
public HoodieBloomFilterRowWriteSupport(BloomFilter bloomFilter) {
73+
super(bloomFilter);
74+
}
75+
76+
@Override
77+
protected byte[] getUTF8Bytes(UTF8String key) {
78+
return key.getBytes();
79+
}
80+
81+
@Override
82+
protected UTF8String dereference(UTF8String key) {
8383
// NOTE: [[clone]] is performed here (rather than [[copy]]) to only copy underlying buffer in
8484
// cases when [[UTF8String]] is pointing into a buffer storing the whole containing record,
8585
// and simply do a pass over when it holds a (immutable) buffer holding just the string
86-
minRecordKey = recordKey.clone();
87-
}
88-
89-
if (maxRecordKey == null || maxRecordKey.compareTo(recordKey) > 0) {
90-
maxRecordKey = recordKey.clone();
86+
return key.clone();
9187
}
9288
}
89+
9390
}

0 commit comments

Comments
 (0)