1
0

[HUDI-4039] Make sure all builtin KeyGenerators properly implement Spark specific APIs (#5523)

This set of changes makes sure that all builtin KeyGenerators properly implement Spark-specific APIs in a performant way (minimizing key-generators overhead)
This commit is contained in:
Alexey Kudinkin
2022-07-22 08:35:07 -07:00
committed by GitHub
parent d5c904e10e
commit eea4a692c0
52 changed files with 1507 additions and 1363 deletions

View File

@@ -388,7 +388,7 @@ public class HoodieAvroUtils {
copyOldValueOrSetDefault(genericRecord, newRecord, f);
}
// do not preserve FILENAME_METADATA_FIELD
newRecord.put(HoodieRecord.FILENAME_META_FIELD_POS, fileName);
newRecord.put(HoodieRecord.FILENAME_META_FIELD_ORD, fileName);
if (!GenericData.get().validate(newSchema, newRecord)) {
throw new SchemaCompatibilityException(
"Unable to validate the rewritten record " + genericRecord + " against schema " + newSchema);
@@ -400,7 +400,7 @@ public class HoodieAvroUtils {
public static GenericRecord rewriteEvolutionRecordWithMetadata(GenericRecord genericRecord, Schema newSchema, String fileName) {
GenericRecord newRecord = HoodieAvroUtils.rewriteRecordWithNewSchema(genericRecord, newSchema, new HashMap<>());
// do not preserve FILENAME_METADATA_FIELD
newRecord.put(HoodieRecord.FILENAME_META_FIELD_POS, fileName);
newRecord.put(HoodieRecord.FILENAME_META_FIELD_ORD, fileName);
return newRecord;
}

View File

@@ -57,9 +57,9 @@ public abstract class HoodieRecord<T> implements Serializable {
IntStream.range(0, HOODIE_META_COLUMNS.size()).mapToObj(idx -> Pair.of(HOODIE_META_COLUMNS.get(idx), idx))
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
public static int RECORD_KEY_META_FIELD_POS = HOODIE_META_COLUMNS_NAME_TO_POS.get(RECORD_KEY_METADATA_FIELD);
public static int PARTITION_PATH_META_FIELD_POS = HOODIE_META_COLUMNS_NAME_TO_POS.get(PARTITION_PATH_METADATA_FIELD);
public static int FILENAME_META_FIELD_POS = HOODIE_META_COLUMNS_NAME_TO_POS.get(FILENAME_METADATA_FIELD);
public static int RECORD_KEY_META_FIELD_ORD = HOODIE_META_COLUMNS_NAME_TO_POS.get(RECORD_KEY_METADATA_FIELD);
public static int PARTITION_PATH_META_FIELD_ORD = HOODIE_META_COLUMNS_NAME_TO_POS.get(PARTITION_PATH_METADATA_FIELD);
public static int FILENAME_META_FIELD_ORD = HOODIE_META_COLUMNS_NAME_TO_POS.get(FILENAME_METADATA_FIELD);
/**
* Identifies the record across the table.

View File

@@ -402,13 +402,13 @@ public class HoodieTableMetaClient implements Serializable {
throw new HoodieException(HoodieTableConfig.POPULATE_META_FIELDS.key() + " already disabled for the table. Can't be re-enabled back");
}
// meta fields can be disabled only with SimpleKeyGenerator, NonPartitioned and ComplexKeyGen.
// Meta fields can be disabled only when either {@code SimpleKeyGenerator}, {@code ComplexKeyGenerator}, {@code NonpartitionedKeyGenerator} is used
if (!getTableConfig().populateMetaFields()) {
String keyGenClass = properties.getProperty(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key(), "org.apache.hudi.keygen.SimpleKeyGenerator");
if (!keyGenClass.equals("org.apache.hudi.keygen.SimpleKeyGenerator") && !keyGenClass.equals("org.apache.hudi.keygen.NonpartitionedKeyGenerator")
if (!keyGenClass.equals("org.apache.hudi.keygen.SimpleKeyGenerator")
&& !keyGenClass.equals("org.apache.hudi.keygen.NonpartitionedKeyGenerator")
&& !keyGenClass.equals("org.apache.hudi.keygen.ComplexKeyGenerator")) {
throw new HoodieException("Only simple, non partitioned and complex key generator is supported when meta fields are disabled. KeyGenerator used : "
+ properties.getProperty(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key()));
throw new HoodieException("Only simple, non-partitioned or complex key generator are supported when meta-fields are disabled. Used: " + keyGenClass);
}
}
}

View File

@@ -40,6 +40,8 @@ import java.util.stream.IntStream;
import java.util.stream.Stream;
import java.util.stream.StreamSupport;
import static org.apache.hudi.common.util.ValidationUtils.checkArgument;
public class CollectionUtils {
public static final Properties EMPTY_PROPERTIES = new Properties();
@@ -52,6 +54,14 @@ public class CollectionUtils {
return !isNullOrEmpty(c);
}
/**
* Returns last element of the array of {@code T}
*/
public static <T> T tail(T[] ts) {
checkArgument(ts.length > 0);
return ts[ts.length - 1];
}
/**
* Collects provided {@link Iterator} to a {@link Stream}
*/
@@ -143,7 +153,7 @@ public class CollectionUtils {
}
public static <E> Stream<List<E>> batchesAsStream(List<E> list, int batchSize) {
ValidationUtils.checkArgument(batchSize > 0, "batch size must be positive.");
checkArgument(batchSize > 0, "batch size must be positive.");
int total = list.size();
if (total <= 0) {
return Stream.empty();

View File

@@ -32,11 +32,15 @@ public class HoodieTimer {
// Ordered stack of TimeInfo's to make sure stopping the timer returns the correct elapsed time
private final Deque<TimeInfo> timeInfoDeque = new ArrayDeque<>();
/**
* @deprecated please use either {@link HoodieTimer#start} or {@link HoodieTimer#create} APIs
*/
@Deprecated
public HoodieTimer() {
this(false);
}
public HoodieTimer(boolean shouldStart) {
private HoodieTimer(boolean shouldStart) {
if (shouldStart) {
startTimer();
}
@@ -79,4 +83,12 @@ public class HoodieTimer {
}
return timeInfoDeque.pop().stop();
}
public static HoodieTimer start() {
return new HoodieTimer(true);
}
public static HoodieTimer create() {
return new HoodieTimer(false);
}
}

View File

@@ -168,7 +168,7 @@ public class ParquetUtils extends BaseFileUtils {
conf.addResource(FSUtils.getFs(filePath.toString(), conf).getConf());
Schema readSchema = keyGeneratorOpt.map(keyGenerator -> {
List<String> fields = new ArrayList<>();
fields.addAll(keyGenerator.getRecordKeyFields());
fields.addAll(keyGenerator.getRecordKeyFieldNames());
fields.addAll(keyGenerator.getPartitionPathFields());
return HoodieAvroUtils.getSchemaForFields(readAvroSchema(conf, filePath), fields);
})

View File

@@ -24,7 +24,6 @@ import org.apache.hudi.exception.HoodieKeyException;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import java.util.List;
import java.util.stream.Collectors;
public abstract class BaseKeyGenerator extends KeyGenerator {
@@ -59,23 +58,14 @@ public abstract class BaseKeyGenerator extends KeyGenerator {
*/
@Override
public final HoodieKey getKey(GenericRecord record) {
if (getRecordKeyFields() == null || getPartitionPathFields() == null) {
if (getRecordKeyFieldNames() == null || getPartitionPathFields() == null) {
throw new HoodieKeyException("Unable to find field names for record key or partition path in cfg");
}
return new HoodieKey(getRecordKey(record), getPartitionPath(record));
}
@Override
public final List<String> getRecordKeyFieldNames() {
// For nested columns, pick top level column name
// TODO materialize
return getRecordKeyFields().stream().map(k -> {
int idx = k.indexOf('.');
return idx > 0 ? k.substring(0, idx) : k;
}).collect(Collectors.toList());
}
public List<String> getRecordKeyFields() {
public List<String> getRecordKeyFieldNames() {
return recordKeyFields;
}

View File

@@ -34,7 +34,7 @@ import java.util.List;
@PublicAPIClass(maturity = ApiMaturityLevel.STABLE)
public abstract class KeyGenerator implements KeyGeneratorInterface {
protected TypedProperties config;
protected final TypedProperties config;
protected KeyGenerator(TypedProperties config) {
this.config = config;

View File

@@ -259,7 +259,7 @@ public class TestParquetUtils extends HoodieCommonTestHarness {
}
@Override
public List<String> getRecordKeyFields() {
public List<String> getRecordKeyFieldNames() {
return Arrays.asList(new String[]{recordKeyField});
}