1
0

[HUDI-407] Adding Simple Index to Hoodie. (#1402)

This index finds the location by joining incoming records with records from base files.
This commit is contained in:
Sivabalan Narayanan
2020-05-17 21:32:24 -04:00
committed by GitHub
parent 3c9da2e5f0
commit 29edf4b3b8
16 changed files with 1381 additions and 65 deletions

View File

@@ -198,6 +198,24 @@ public class HoodieAvroUtils {
return RECORD_KEY_SCHEMA;
}
/**
* Fetch schema for record key and partition path.
*/
public static Schema getRecordKeyPartitionPathSchema() {
List<Schema.Field> toBeAddedFields = new ArrayList<>();
Schema recordSchema = Schema.createRecord("HoodieRecordKey", "", "", false);
Schema.Field recordKeyField =
new Schema.Field(HoodieRecord.RECORD_KEY_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", NullNode.getInstance());
Schema.Field partitionPathField =
new Schema.Field(HoodieRecord.PARTITION_PATH_METADATA_FIELD, METADATA_FIELD_SCHEMA, "", NullNode.getInstance());
toBeAddedFields.add(recordKeyField);
toBeAddedFields.add(partitionPathField);
recordSchema.setFields(toBeAddedFields);
return recordSchema;
}
public static GenericRecord addHoodieKeyToRecord(GenericRecord record, String recordKey, String partitionPath,
String fileName) {
record.put(HoodieRecord.FILENAME_METADATA_FIELD, fileName);

View File

@@ -24,6 +24,7 @@ import org.apache.hudi.common.bloom.BloomFilter;
import org.apache.hudi.common.bloom.BloomFilterFactory;
import org.apache.hudi.common.bloom.BloomFilterTypeCode;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
@@ -76,13 +77,27 @@ public class ParquetUtils {
* @return Set Set of row keys matching candidateRecordKeys
*/
public static Set<String> filterParquetRowKeys(Configuration configuration, Path filePath, Set<String> filter) {
return filterParquetRowKeys(configuration, filePath, filter, HoodieAvroUtils.getRecordKeySchema());
}
/**
* 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
* @param readSchema schema of columns to be read
* @return Set Set of row keys matching candidateRecordKeys
*/
private static Set<String> filterParquetRowKeys(Configuration configuration, Path filePath, Set<String> filter,
Schema readSchema) {
Option<RecordKeysFilterFunction> filterFunction = Option.empty();
if (filter != null && !filter.isEmpty()) {
filterFunction = Option.of(new RecordKeysFilterFunction(filter));
}
Configuration conf = new Configuration(configuration);
conf.addResource(FSUtils.getFs(filePath.toString(), conf).getConf());
Schema readSchema = HoodieAvroUtils.getRecordKeySchema();
AvroReadSupport.setAvroReadSchema(conf, readSchema);
AvroReadSupport.setRequestedProjection(conf, readSchema);
Set<String> rowKeys = new HashSet<>();
@@ -105,6 +120,41 @@ public class ParquetUtils {
return rowKeys;
}
/**
* Fetch {@link HoodieKey}s from the given parquet file.
*
* @param filePath The parquet file path.
* @param configuration configuration to build fs object
* @return {@link List} of {@link HoodieKey}s fetched from the parquet file
*/
public static List<HoodieKey> fetchRecordKeyPartitionPathFromParquet(Configuration configuration, Path filePath) {
List<HoodieKey> hoodieKeys = new ArrayList<>();
try {
if (!filePath.getFileSystem(configuration).exists(filePath)) {
return new ArrayList<>();
}
Configuration conf = new Configuration(configuration);
conf.addResource(FSUtils.getFs(filePath.toString(), conf).getConf());
Schema readSchema = HoodieAvroUtils.getRecordKeyPartitionPathSchema();
AvroReadSupport.setAvroReadSchema(conf, readSchema);
AvroReadSupport.setRequestedProjection(conf, readSchema);
ParquetReader reader = AvroParquetReader.builder(filePath).withConf(conf).build();
Object obj = reader.read();
while (obj != null) {
if (obj instanceof GenericRecord) {
String recordKey = ((GenericRecord) obj).get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
String partitionPath = ((GenericRecord) obj).get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
hoodieKeys.add(new HoodieKey(recordKey, partitionPath));
obj = reader.read();
}
}
} catch (IOException e) {
throw new HoodieIOException("Failed to read from Parquet file " + filePath, e);
}
return hoodieKeys;
}
public static ParquetMetadata readMetadata(Configuration conf, Path parquetFilePath) {
ParquetMetadata footer;
try {