1
0

Added a filter function to filter the record keys in a parquet file

This commit is contained in:
Sunil Ramaiah
2018-05-17 15:40:47 -07:00
committed by vinoth chandar
parent 23d53763c4
commit a97814462d
3 changed files with 93 additions and 27 deletions

View File

@@ -26,7 +26,9 @@ import com.uber.hoodie.common.model.HoodieTestUtils;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.UUID;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData;
@@ -54,28 +56,13 @@ public class TestParquetUtils {
@Test
public void testHoodieWriteSupport() throws Exception {
List<String> rowKeys = new ArrayList<>();
for (int i = 0; i < 1000; i++) {
rowKeys.add(UUID.randomUUID().toString());
}
// Write out a parquet file
Schema schema = HoodieAvroUtils.getRecordKeySchema();
BloomFilter filter = new BloomFilter(1000, 0.0001);
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema,
filter);
String filePath = basePath + "/test.parquet";
ParquetWriter writer = new ParquetWriter(new Path(filePath), writeSupport, CompressionCodecName.GZIP,
120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE);
for (String rowKey : rowKeys) {
GenericRecord rec = new GenericData.Record(schema);
rec.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, rowKey);
writer.write(rec);
filter.add(rowKey);
}
writer.close();
writeParquetFile(filePath, rowKeys);
// Read and verify
List<String> rowKeysInFile = new ArrayList<>(
@@ -90,4 +77,49 @@ public class TestParquetUtils {
assertTrue("key should be found in bloom filter", filterInFile.mightContain(rowKey));
}
}
@Test
public void testFilterParquetRowKeys() throws Exception {
List<String> rowKeys = new ArrayList<>();
Set<String> filter = new HashSet<>();
for (int i = 0; i < 1000; i++) {
String rowKey = UUID.randomUUID().toString();
rowKeys.add(rowKey);
if (i % 100 == 0) {
filter.add(rowKey);
}
}
String filePath = basePath + "/test.parquet";
writeParquetFile(filePath, rowKeys);
// Read and verify
Set<String> filtered = ParquetUtils.filterParquetRowKeys(HoodieTestUtils.getDefaultHadoopConf(),
new Path(filePath),
filter);
assertEquals("Filtered count does not match", filter.size(), filtered.size());
for (String rowKey : filtered) {
assertTrue("filtered key must be in the given filter", filter.contains(rowKey));
}
}
private void writeParquetFile(String filePath,
List<String> rowKeys) throws Exception {
// Write out a parquet file
Schema schema = HoodieAvroUtils.getRecordKeySchema();
BloomFilter filter = new BloomFilter(1000, 0.0001);
HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(new AvroSchemaConverter().convert(schema), schema,
filter);
ParquetWriter writer = new ParquetWriter(new Path(filePath), writeSupport, CompressionCodecName.GZIP,
120 * 1024 * 1024, ParquetWriter.DEFAULT_PAGE_SIZE);
for (String rowKey : rowKeys) {
GenericRecord rec = new GenericData.Record(schema);
rec.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, rowKey);
writer.write(rec);
filter.add(rowKey);
}
writer.close();
}
}