1
0

[HUDI-1653] Add support for composite keys in NonpartitionedKeyGenerator (#2627)

* [HUDI-1653] Add support for composite keys in NonpartitionedKeyGenerator

* update NonpartitionedKeyGenerator to support composite record keys

* update NonpartitionedKeyGenerator
This commit is contained in:
Jintao Guan
2021-03-18 15:33:31 -07:00
committed by GitHub
parent e602e5dfb9
commit 1277c62398
3 changed files with 170 additions and 5 deletions

View File

@@ -20,20 +20,32 @@ package org.apache.hudi.keygen;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import org.apache.spark.sql.Row;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.stream.Collectors;
/**
* Simple Key generator for unpartitioned Hive Tables.
*/
public class NonpartitionedKeyGenerator extends SimpleKeyGenerator {
public class NonpartitionedKeyGenerator extends BuiltinKeyGenerator {
private final NonpartitionedAvroKeyGenerator nonpartitionedAvroKeyGenerator;
public NonpartitionedKeyGenerator(TypedProperties config) {
super(config);
nonpartitionedAvroKeyGenerator = new NonpartitionedAvroKeyGenerator(config);
public NonpartitionedKeyGenerator(TypedProperties props) {
super(props);
this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY)
.split(",")).map(String::trim).collect(Collectors.toList());
this.partitionPathFields = Collections.emptyList();
nonpartitionedAvroKeyGenerator = new NonpartitionedAvroKeyGenerator(props);
}
@Override
public String getRecordKey(GenericRecord record) {
return nonpartitionedAvroKeyGenerator.getRecordKey(record);
}
@Override