1
0

[HUDI-1533] Make SerializableSchema work for large schemas and add ability to sortBy numeric values (#2453)

This commit is contained in:
satishkotha
2021-01-17 12:36:55 -08:00
committed by GitHub
parent a43e191d6c
commit 3d1d5d00b0
3 changed files with 40 additions and 9 deletions

View File

@@ -20,6 +20,7 @@ package org.apache.hudi.execution.bulkinsert;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.common.config.SerializableSchema;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
@@ -58,18 +59,22 @@ public class RDDCustomColumnsSortPartitioner<T extends HoodieRecordPayload>
public boolean arePartitionRecordsSorted() {
return true;
}
private static String getRecordSortColumnValues(HoodieRecord<? extends HoodieRecordPayload> record,
private static Object getRecordSortColumnValues(HoodieRecord<? extends HoodieRecordPayload> record,
String[] sortColumns,
SerializableSchema schema) {
try {
GenericRecord genericRecord = (GenericRecord) record.getData().getInsertValue(schema.get()).get();
StringBuilder sb = new StringBuilder();
for (String col : sortColumns) {
sb.append(genericRecord.get(col));
}
if (sortColumns.length == 1) {
return HoodieAvroUtils.getNestedFieldVal(genericRecord, sortColumns[0], true);
} else {
StringBuilder sb = new StringBuilder();
for (String col : sortColumns) {
sb.append(HoodieAvroUtils.getNestedFieldValAsString(genericRecord, col, true));
}
return sb.toString();
return sb.toString();
}
} catch (IOException e) {
throw new HoodieIOException("Unable to read record with key:" + record.getKey(), e);
}