Added a filter function to filter the record keys in a parquet file
This commit is contained in:
committed by
vinoth chandar
parent
23d53763c4
commit
a97814462d
@@ -29,9 +29,12 @@ import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
import java.util.function.Function;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.commons.collections.CollectionUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.parquet.avro.AvroParquetReader;
|
||||
@@ -52,8 +55,26 @@ public class ParquetUtils {
|
||||
*
|
||||
* @param filePath The parquet file path.
|
||||
* @param configuration configuration to build fs object
|
||||
* @return Set Set of row keys
|
||||
*/
|
||||
public static Set<String> readRowKeysFromParquet(Configuration configuration, Path filePath) {
|
||||
return filterParquetRowKeys(configuration, filePath, new HashSet<>());
|
||||
}
|
||||
|
||||
/**
|
||||
* Read the rowKey list matching the given filter, from the given parquet file. If the filter is empty,
|
||||
* then this will return all the rowkeys.
|
||||
*
|
||||
* @param filePath The parquet file path.
|
||||
* @param configuration configuration to build fs object
|
||||
* @param filter record keys filter
|
||||
* @return Set Set of row keys matching candidateRecordKeys
|
||||
*/
|
||||
public static Set<String> filterParquetRowKeys(Configuration configuration, Path filePath, Set<String> filter) {
|
||||
Optional<RecordKeysFilterFunction> filterFunction = Optional.empty();
|
||||
if (CollectionUtils.isNotEmpty(filter)) {
|
||||
filterFunction = Optional.of(new RecordKeysFilterFunction(filter));
|
||||
}
|
||||
Configuration conf = new Configuration(configuration);
|
||||
conf.addResource(getFs(filePath.toString(), conf).getConf());
|
||||
Schema readSchema = HoodieAvroUtils.getRecordKeySchema();
|
||||
@@ -66,7 +87,10 @@ public class ParquetUtils {
|
||||
Object obj = reader.read();
|
||||
while (obj != null) {
|
||||
if (obj instanceof GenericRecord) {
|
||||
rowKeys.add(((GenericRecord) obj).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString());
|
||||
String recordKey = ((GenericRecord) obj).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
|
||||
if (!filterFunction.isPresent() || filterFunction.get().apply(recordKey)) {
|
||||
rowKeys.add(recordKey);
|
||||
}
|
||||
}
|
||||
obj = reader.read();
|
||||
}
|
||||
@@ -178,4 +202,20 @@ public class ParquetUtils {
|
||||
}
|
||||
return records;
|
||||
}
|
||||
|
||||
static class RecordKeysFilterFunction implements Function<String, Boolean> {
|
||||
private final Set<String> candidateKeys;
|
||||
|
||||
RecordKeysFilterFunction(Set<String> candidateKeys) {
|
||||
this.candidateKeys = candidateKeys;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Boolean apply(String recordKey) {
|
||||
if (candidateKeys.contains(recordKey)) {
|
||||
return true;
|
||||
}
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user