1
0

[HUDI-3995] Making perf optimizations for bulk insert row writer path (#5462)

- Avoid using udf for key generator for SimpleKeyGen and NonPartitionedKeyGen.
- Fixed NonPartitioned Key generator to directly fetch record key from row rather than involving GenericRecord.
- Other minor fixes around using static values instead of looking up hashmap.
This commit is contained in:
Sivabalan Narayanan
2022-05-09 12:40:22 -04:00
committed by GitHub
parent 6b47ef6ed2
commit 6285a239a3
20 changed files with 217 additions and 187 deletions

View File

@@ -68,8 +68,8 @@ public class HoodieRowCreateHandle implements Serializable {
private final HoodieTimer currTimer;
public HoodieRowCreateHandle(HoodieTable table, HoodieWriteConfig writeConfig, String partitionPath, String fileId,
String instantTime, int taskPartitionId, long taskId, long taskEpochId,
StructType structType) {
String instantTime, int taskPartitionId, long taskId, long taskEpochId,
StructType structType) {
this.partitionPath = partitionPath;
this.table = table;
this.writeConfig = writeConfig;
@@ -107,16 +107,15 @@ public class HoodieRowCreateHandle implements Serializable {
/**
* Writes an {@link InternalRow} to the underlying HoodieInternalRowFileWriter. Before writing, value for meta columns are computed as required
* and wrapped in {@link HoodieInternalRow}. {@link HoodieInternalRow} is what gets written to HoodieInternalRowFileWriter.
*
* @param record instance of {@link InternalRow} that needs to be written to the fileWriter.
* @throws IOException
*/
public void write(InternalRow record) throws IOException {
try {
String partitionPath = record.getUTF8String(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(
HoodieRecord.PARTITION_PATH_METADATA_FIELD)).toString();
String seqId = HoodieRecord.generateSequenceId(instantTime, taskPartitionId, SEQGEN.getAndIncrement());
String recordKey = record.getUTF8String(HoodieRecord.HOODIE_META_COLUMNS_NAME_TO_POS.get(
HoodieRecord.RECORD_KEY_METADATA_FIELD)).toString();
final String partitionPath = String.valueOf(record.getUTF8String(HoodieRecord.PARTITION_PATH_META_FIELD_POS));
final String seqId = HoodieRecord.generateSequenceId(instantTime, taskPartitionId, SEQGEN.getAndIncrement());
final String recordKey = String.valueOf(record.getUTF8String(HoodieRecord.RECORD_KEY_META_FIELD_POS));
HoodieInternalRow internalRow = new HoodieInternalRow(instantTime, seqId, recordKey, partitionPath, path.getName(),
record);
try {
@@ -141,6 +140,7 @@ public class HoodieRowCreateHandle implements Serializable {
/**
* Closes the {@link HoodieRowCreateHandle} and returns an instance of {@link HoodieInternalWriteStatus} containing the stats and
* status of the writes to this handle.
*
* @return the {@link HoodieInternalWriteStatus} containing the stats and status of the writes to this handle.
* @throws IOException
*/

View File

@@ -18,25 +18,25 @@
package org.apache.hudi.keygen;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.ApiMaturityLevel;
import org.apache.hudi.AvroConversionUtils;
import org.apache.hudi.HoodieSparkUtils;
import org.apache.hudi.PublicAPIMethod;
import org.apache.hudi.client.utils.SparkRowSerDe;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.HoodieKeyException;
import org.apache.avro.generic.GenericRecord;
import org.apache.spark.sql.Row;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.types.DataType;
import org.apache.spark.sql.types.StructType;
import scala.Function1;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicBoolean;
import scala.Function1;
/**
* Base class for the built-in key generators. Contains methods structured for
@@ -46,13 +46,12 @@ public abstract class BuiltinKeyGenerator extends BaseKeyGenerator implements Sp
private static final String STRUCT_NAME = "hoodieRowTopLevelField";
private static final String NAMESPACE = "hoodieRow";
private transient Function1<Row, GenericRecord> converterFn = null;
private SparkRowSerDe sparkRowSerDe;
private Function1<Row, GenericRecord> converterFn = null;
private final AtomicBoolean validatePartitionFields = new AtomicBoolean(false);
protected StructType structType;
protected Map<String, List<Integer>> recordKeyPositions = new HashMap<>();
protected Map<String, List<Integer>> partitionPathPositions = new HashMap<>();
protected Map<String, List<DataType>> partitionPathDataTypes = null;
protected Map<String, Pair<List<Integer>, DataType>> recordKeySchemaInfo = new HashMap<>();
protected Map<String, Pair<List<Integer>, DataType>> partitionPathSchemaInfo = new HashMap<>();
protected BuiltinKeyGenerator(TypedProperties config) {
super(config);
@@ -60,6 +59,7 @@ public abstract class BuiltinKeyGenerator extends BaseKeyGenerator implements Sp
/**
* Fetch record key from {@link Row}.
*
* @param row instance of {@link Row} from which record key is requested.
* @return the record key of interest from {@link Row}.
*/
@@ -74,6 +74,7 @@ public abstract class BuiltinKeyGenerator extends BaseKeyGenerator implements Sp
/**
* Fetch partition path from {@link Row}.
*
* @param row instance of {@link Row} from which partition path is requested
* @return the partition path of interest from {@link Row}.
*/
@@ -97,87 +98,41 @@ public abstract class BuiltinKeyGenerator extends BaseKeyGenerator implements Sp
@PublicAPIMethod(maturity = ApiMaturityLevel.EVOLVING)
public String getPartitionPath(InternalRow internalRow, StructType structType) {
try {
initDeserializer(structType);
Row row = sparkRowSerDe.deserializeRow(internalRow);
return getPartitionPath(row);
buildFieldSchemaInfoIfNeeded(structType);
return RowKeyGeneratorHelper.getPartitionPathFromInternalRow(internalRow, getPartitionPathFields(),
hiveStylePartitioning, partitionPathSchemaInfo);
} catch (Exception e) {
throw new HoodieIOException("Conversion of InternalRow to Row failed with exception " + e);
}
}
private void initDeserializer(StructType structType) {
if (sparkRowSerDe == null) {
sparkRowSerDe = HoodieSparkUtils.getDeserializer(structType);
}
}
void buildFieldPositionMapIfNeeded(StructType structType) {
void buildFieldSchemaInfoIfNeeded(StructType structType) {
if (this.structType == null) {
// parse simple fields
getRecordKeyFields().stream()
.filter(f -> !(f.contains(".")))
.forEach(f -> {
if (structType.getFieldIndex(f).isDefined()) {
recordKeyPositions.put(f, Collections.singletonList((Integer) (structType.getFieldIndex(f).get())));
} else {
throw new HoodieKeyException("recordKey value not found for field: \"" + f + "\"");
}
});
// parse nested fields
getRecordKeyFields().stream()
.filter(f -> f.contains("."))
.forEach(f -> recordKeyPositions.put(f, RowKeyGeneratorHelper.getNestedFieldIndices(structType, f, true)));
// parse simple fields
getRecordKeyFields()
.stream().filter(f -> !f.isEmpty())
.forEach(f -> recordKeySchemaInfo.put(f, RowKeyGeneratorHelper.getFieldSchemaInfo(structType, f, true)));
if (getPartitionPathFields() != null) {
getPartitionPathFields().stream().filter(f -> !f.isEmpty()).filter(f -> !(f.contains(".")))
.forEach(f -> {
if (structType.getFieldIndex(f).isDefined()) {
partitionPathPositions.put(f,
Collections.singletonList((Integer) (structType.getFieldIndex(f).get())));
} else {
partitionPathPositions.put(f, Collections.singletonList(-1));
}
});
// parse nested fields
getPartitionPathFields().stream().filter(f -> !f.isEmpty()).filter(f -> f.contains("."))
.forEach(f -> partitionPathPositions.put(f,
RowKeyGeneratorHelper.getNestedFieldIndices(structType, f, false)));
getPartitionPathFields().stream().filter(f -> !f.isEmpty())
.forEach(f -> partitionPathSchemaInfo.put(f, RowKeyGeneratorHelper.getFieldSchemaInfo(structType, f, false)));
}
this.structType = structType;
}
}
protected String getPartitionPathInternal(InternalRow row, StructType structType) {
buildFieldDataTypesMapIfNeeded(structType);
buildFieldSchemaInfoIfNeeded(structType);
validatePartitionFieldsForInternalRow();
return RowKeyGeneratorHelper.getPartitionPathFromInternalRow(row, getPartitionPathFields(),
hiveStylePartitioning, partitionPathPositions, partitionPathDataTypes);
hiveStylePartitioning, partitionPathSchemaInfo);
}
protected void validatePartitionFieldsForInternalRow() {
partitionPathPositions.entrySet().forEach(entry -> {
if (entry.getValue().size() > 1) {
throw new IllegalArgumentException("Nested column for partitioning is not supported with disabling meta columns");
}
});
}
void buildFieldDataTypesMapIfNeeded(StructType structType) {
buildFieldPositionMapIfNeeded(structType);
if (this.partitionPathDataTypes == null) {
this.partitionPathDataTypes = new HashMap<>();
if (getPartitionPathFields() != null) {
// populating simple fields are good enough
getPartitionPathFields().stream().filter(f -> !f.isEmpty()).filter(f -> !(f.contains(".")))
.forEach(f -> {
if (structType.getFieldIndex(f).isDefined()) {
partitionPathDataTypes.put(f,
Collections.singletonList((structType.fields()[structType.fieldIndex(f)].dataType())));
} else {
partitionPathDataTypes.put(f, Collections.singletonList(null));
}
});
}
if (!validatePartitionFields.getAndSet(true)) {
partitionPathSchemaInfo.values().forEach(entry -> {
if (entry.getKey().size() > 1) {
throw new IllegalArgumentException("Nested column for partitioning is not supported with disabling meta columns");
}
});
}
}
}

View File

@@ -60,15 +60,15 @@ public class ComplexKeyGenerator extends BuiltinKeyGenerator {
@Override
public String getRecordKey(Row row) {
buildFieldPositionMapIfNeeded(row.schema());
return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeyPositions, true);
buildFieldSchemaInfoIfNeeded(row.schema());
return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeySchemaInfo, true);
}
@Override
public String getPartitionPath(Row row) {
buildFieldPositionMapIfNeeded(row.schema());
buildFieldSchemaInfoIfNeeded(row.schema());
return RowKeyGeneratorHelper.getPartitionPathFromRow(row, getPartitionPathFields(),
hiveStylePartitioning, partitionPathPositions);
hiveStylePartitioning, partitionPathSchemaInfo);
}
@Override

View File

@@ -60,8 +60,8 @@ public class GlobalDeleteKeyGenerator extends BuiltinKeyGenerator {
@Override
public String getRecordKey(Row row) {
buildFieldPositionMapIfNeeded(row.schema());
return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeyPositions, true);
buildFieldSchemaInfoIfNeeded(row.schema());
return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeySchemaInfo, true);
}
@Override

View File

@@ -61,6 +61,12 @@ public class NonpartitionedKeyGenerator extends BuiltinKeyGenerator {
return nonpartitionedAvroKeyGenerator.getPartitionPathFields();
}
@Override
public String getRecordKey(Row row) {
buildFieldSchemaInfoIfNeeded(row.schema());
return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeySchemaInfo, false);
}
@Override
public String getPartitionPath(Row row) {
return nonpartitionedAvroKeyGenerator.getEmptyPartition();

View File

@@ -18,6 +18,7 @@
package org.apache.hudi.keygen;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.exception.HoodieKeyException;
import org.apache.spark.sql.Row;
@@ -52,17 +53,18 @@ public class RowKeyGeneratorHelper {
/**
* Generates record key for the corresponding {@link Row}.
* @param row instance of {@link Row} of interest
* @param recordKeyFields record key fields as a list
*
* @param row instance of {@link Row} of interest
* @param recordKeyFields record key fields as a list
* @param recordKeyPositions record key positions for the corresponding record keys in {@code recordKeyFields}
* @param prefixFieldName {@code true} if field name need to be prefixed in the returned result. {@code false} otherwise.
* @param prefixFieldName {@code true} if field name need to be prefixed in the returned result. {@code false} otherwise.
* @return the record key thus generated
*/
public static String getRecordKeyFromRow(Row row, List<String> recordKeyFields, Map<String, List<Integer>> recordKeyPositions, boolean prefixFieldName) {
public static String getRecordKeyFromRow(Row row, List<String> recordKeyFields, Map<String, Pair<List<Integer>, DataType>> recordKeyPositions, boolean prefixFieldName) {
AtomicBoolean keyIsNullOrEmpty = new AtomicBoolean(true);
String toReturn = recordKeyFields.stream().map(field -> {
String val = null;
List<Integer> fieldPositions = recordKeyPositions.get(field);
List<Integer> fieldPositions = recordKeyPositions.get(field).getKey();
if (fieldPositions.size() == 1) { // simple field
Integer fieldPos = fieldPositions.get(0);
if (row.isNullAt(fieldPos)) {
@@ -76,7 +78,7 @@ public class RowKeyGeneratorHelper {
}
}
} else { // nested fields
val = getNestedFieldVal(row, recordKeyPositions.get(field)).toString();
val = getNestedFieldVal(row, recordKeyPositions.get(field).getKey()).toString();
if (!val.contains(NULL_RECORDKEY_PLACEHOLDER) && !val.contains(EMPTY_RECORDKEY_PLACEHOLDER)) {
keyIsNullOrEmpty.set(false);
}
@@ -91,17 +93,18 @@ public class RowKeyGeneratorHelper {
/**
* Generates partition path for the corresponding {@link Row}.
* @param row instance of {@link Row} of interest
* @param partitionPathFields partition path fields as a list
* @param hiveStylePartitioning {@code true} if hive style partitioning is set. {@code false} otherwise
*
* @param row instance of {@link Row} of interest
* @param partitionPathFields partition path fields as a list
* @param hiveStylePartitioning {@code true} if hive style partitioning is set. {@code false} otherwise
* @param partitionPathPositions partition path positions for the corresponding fields in {@code partitionPathFields}
* @return the generated partition path for the row
*/
public static String getPartitionPathFromRow(Row row, List<String> partitionPathFields, boolean hiveStylePartitioning, Map<String, List<Integer>> partitionPathPositions) {
public static String getPartitionPathFromRow(Row row, List<String> partitionPathFields, boolean hiveStylePartitioning, Map<String, Pair<List<Integer>, DataType>> partitionPathPositions) {
return IntStream.range(0, partitionPathFields.size()).mapToObj(idx -> {
String field = partitionPathFields.get(idx);
String val = null;
List<Integer> fieldPositions = partitionPathPositions.get(field);
List<Integer> fieldPositions = partitionPathPositions.get(field).getKey();
if (fieldPositions.size() == 1) { // simple
Integer fieldPos = fieldPositions.get(0);
// for partition path, if field is not found, index will be set to -1
@@ -118,7 +121,7 @@ public class RowKeyGeneratorHelper {
val = field + "=" + val;
}
} else { // nested
Object data = getNestedFieldVal(row, partitionPathPositions.get(field));
Object data = getNestedFieldVal(row, partitionPathPositions.get(field).getKey());
data = convertToTimestampIfInstant(data);
if (data.toString().contains(NULL_RECORDKEY_PLACEHOLDER) || data.toString().contains(EMPTY_RECORDKEY_PLACEHOLDER)) {
val = hiveStylePartitioning ? field + "=" + HUDI_DEFAULT_PARTITION_PATH : HUDI_DEFAULT_PARTITION_PATH;
@@ -130,20 +133,20 @@ public class RowKeyGeneratorHelper {
}).collect(Collectors.joining(DEFAULT_PARTITION_PATH_SEPARATOR));
}
public static String getPartitionPathFromInternalRow(InternalRow row, List<String> partitionPathFields, boolean hiveStylePartitioning,
Map<String, List<Integer>> partitionPathPositions,
Map<String, List<DataType>> partitionPathDataTypes) {
public static String getPartitionPathFromInternalRow(InternalRow internalRow, List<String> partitionPathFields, boolean hiveStylePartitioning,
Map<String, Pair<List<Integer>, DataType>> partitionPathPositions) {
return IntStream.range(0, partitionPathFields.size()).mapToObj(idx -> {
String field = partitionPathFields.get(idx);
String val = null;
List<Integer> fieldPositions = partitionPathPositions.get(field);
List<Integer> fieldPositions = partitionPathPositions.get(field).getKey();
DataType dataType = partitionPathPositions.get(field).getValue();
if (fieldPositions.size() == 1) { // simple
Integer fieldPos = fieldPositions.get(0);
// for partition path, if field is not found, index will be set to -1
if (fieldPos == -1 || row.isNullAt(fieldPos)) {
if (fieldPos == -1 || internalRow.isNullAt(fieldPos)) {
val = HUDI_DEFAULT_PARTITION_PATH;
} else {
Object value = row.get(fieldPos, partitionPathDataTypes.get(field).get(0));
Object value = internalRow.get(fieldPos, dataType);
if (value == null || value.toString().isEmpty()) {
val = HUDI_DEFAULT_PARTITION_PATH;
} else {
@@ -180,22 +183,22 @@ public class RowKeyGeneratorHelper {
/**
* Fetch the field value located at the positions requested for.
*
* <p>
* The fetching logic recursively goes into the nested field based on the position list to get the field value.
* For example, given the row [4357686,key1,2020-03-21,pi,[val1,10]] with the following schema, which has the fourth
* field as a nested field, and positions list as [4,0],
*
* <p>
* 0 = "StructField(timestamp,LongType,false)"
* 1 = "StructField(_row_key,StringType,false)"
* 2 = "StructField(ts_ms,StringType,false)"
* 3 = "StructField(pii_col,StringType,false)"
* 4 = "StructField(nested_col,StructType(StructField(prop1,StringType,false), StructField(prop2,LongType,false)),false)"
*
* <p>
* the logic fetches the value from field nested_col.prop1.
* If any level of the nested field is null, {@link KeyGenUtils#NULL_RECORDKEY_PLACEHOLDER} is returned.
* If the field value is an empty String, {@link KeyGenUtils#EMPTY_RECORDKEY_PLACEHOLDER} is returned.
*
* @param row instance of {@link Row} of interest
* @param row instance of {@link Row} of interest
* @param positions tree style positions where the leaf node need to be fetched and returned
* @return the field value as per the positions requested for.
*/
@@ -234,13 +237,14 @@ public class RowKeyGeneratorHelper {
* @param structType schema of interest
* @param field field of interest for which the positions are requested for
* @param isRecordKey {@code true} if the field requested for is a record key. {@code false} in case of a partition path.
* @return the positions of the field as per the struct type.
* @return the positions of the field as per the struct type and the leaf field's datatype.
*/
public static List<Integer> getNestedFieldIndices(StructType structType, String field, boolean isRecordKey) {
public static Pair<List<Integer>, DataType> getFieldSchemaInfo(StructType structType, String field, boolean isRecordKey) {
String[] slices = field.split("\\.");
List<Integer> positions = new ArrayList<>();
int index = 0;
int totalCount = slices.length;
DataType leafFieldDataType = null;
while (index < totalCount) {
String slice = slices[index];
Option<Object> curIndexOpt = structType.getFieldIndex(slice);
@@ -258,6 +262,9 @@ public class RowKeyGeneratorHelper {
}
}
structType = (StructType) nestedField.dataType();
} else {
// leaf node.
leafFieldDataType = nestedField.dataType();
}
} else {
if (isRecordKey) {
@@ -269,7 +276,7 @@ public class RowKeyGeneratorHelper {
}
index++;
}
return positions;
return Pair.of(positions, leafFieldDataType);
}
private static Object convertToTimestampIfInstant(Object data) {

View File

@@ -65,15 +65,15 @@ public class SimpleKeyGenerator extends BuiltinKeyGenerator {
@Override
public String getRecordKey(Row row) {
buildFieldPositionMapIfNeeded(row.schema());
return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeyPositions, false);
buildFieldSchemaInfoIfNeeded(row.schema());
return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeySchemaInfo, false);
}
@Override
public String getPartitionPath(Row row) {
buildFieldPositionMapIfNeeded(row.schema());
buildFieldSchemaInfoIfNeeded(row.schema());
return RowKeyGeneratorHelper.getPartitionPathFromRow(row, getPartitionPathFields(),
hiveStylePartitioning, partitionPathPositions);
hiveStylePartitioning, partitionPathSchemaInfo);
}
@Override

View File

@@ -29,8 +29,8 @@ import org.apache.spark.sql.types.StructType;
import java.io.IOException;
import static org.apache.hudi.keygen.KeyGenUtils.HUDI_DEFAULT_PARTITION_PATH;
import static org.apache.hudi.keygen.KeyGenUtils.EMPTY_RECORDKEY_PLACEHOLDER;
import static org.apache.hudi.keygen.KeyGenUtils.HUDI_DEFAULT_PARTITION_PATH;
import static org.apache.hudi.keygen.KeyGenUtils.NULL_RECORDKEY_PLACEHOLDER;
/**
@@ -61,24 +61,24 @@ public class TimestampBasedKeyGenerator extends SimpleKeyGenerator {
@Override
public String getRecordKey(Row row) {
buildFieldPositionMapIfNeeded(row.schema());
return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeyPositions, false);
buildFieldSchemaInfoIfNeeded(row.schema());
return RowKeyGeneratorHelper.getRecordKeyFromRow(row, getRecordKeyFields(), recordKeySchemaInfo, false);
}
@Override
public String getPartitionPath(Row row) {
buildFieldPositionMapIfNeeded(row.schema());
Object partitionPathFieldVal = RowKeyGeneratorHelper.getNestedFieldVal(row, partitionPathPositions.get(getPartitionPathFields().get(0)));
buildFieldSchemaInfoIfNeeded(row.schema());
Object partitionPathFieldVal = RowKeyGeneratorHelper.getNestedFieldVal(row, partitionPathSchemaInfo.get(getPartitionPathFields().get(0)).getKey());
return getTimestampBasedPartitionPath(partitionPathFieldVal);
}
@Override
public String getPartitionPath(InternalRow internalRow, StructType structType) {
buildFieldDataTypesMapIfNeeded(structType);
buildFieldSchemaInfoIfNeeded(structType);
validatePartitionFieldsForInternalRow();
Object partitionPathFieldVal = RowKeyGeneratorHelper.getFieldValFromInternalRow(internalRow,
partitionPathPositions.get(getPartitionPathFields().get(0)).get(0),
partitionPathDataTypes.get(getPartitionPathFields().get(0)).get(0));
partitionPathSchemaInfo.get(getPartitionPathFields().get(0)).getKey().get(0),
partitionPathSchemaInfo.get(getPartitionPathFields().get(0)).getValue());
return getTimestampBasedPartitionPath(partitionPathFieldVal);
}