Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[HUDI-3922] parse record key + partition path config consistently between keygens and HiveSync #5520

Closed
wants to merge 3 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -21,8 +21,7 @@
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;

import java.util.Arrays;
import java.util.stream.Collectors;
import java.util.Collections;

/**
* Avro complex key generator, which takes names of fields to be used for recordKey and partitionPath as configs.
Expand All @@ -32,14 +31,10 @@ public class ComplexAvroKeyGenerator extends BaseKeyGenerator {

public ComplexAvroKeyGenerator(TypedProperties props) {
super(props);
this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()).split(","))
.map(String::trim)
.filter(s -> !s.isEmpty())
.collect(Collectors.toList());
this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key()).split(","))
.map(String::trim)
.filter(s -> !s.isEmpty())
.collect(Collectors.toList());
this.recordKeyFields = props.getStringList(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), ",",
Collections.singletonList(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.defaultValue()));
this.partitionPathFields = props.getStringList(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), ",",
Collections.emptyList());
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,8 +25,7 @@
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;

import java.io.IOException;
import java.util.Arrays;
import java.util.stream.Collectors;
import java.util.Collections;

/**
* This is a generic implementation of KeyGenerator where users can configure record key as a single field or a combination of fields. Similarly partition path can be configured to have multiple
Expand Down Expand Up @@ -55,8 +54,10 @@ public enum PartitionKeyType {

public CustomAvroKeyGenerator(TypedProperties props) {
super(props);
this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()).split(",")).map(String::trim).collect(Collectors.toList());
this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key()).split(",")).map(String::trim).collect(Collectors.toList());
this.recordKeyFields = props.getStringList(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), ",",
Collections.singletonList(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.defaultValue()));
this.partitionPathFields = props.getStringList(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), ",",
Collections.emptyList());
}

@Override
Expand All @@ -69,7 +70,7 @@ public String getPartitionPath(GenericRecord record) {
StringBuilder partitionPath = new StringBuilder();

//Corresponds to no partition case
if (getPartitionPathFields().size() == 1 && getPartitionPathFields().get(0).isEmpty()) {
if (getPartitionPathFields().isEmpty()) {
return "";
}
for (String field : getPartitionPathFields()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,8 +25,7 @@
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.types.StructType;

import java.util.Arrays;
import java.util.stream.Collectors;
import java.util.Collections;

/**
* Complex key generator, which takes names of fields to be used for recordKey and partitionPath as configs.
Expand All @@ -37,14 +36,10 @@ public class ComplexKeyGenerator extends BuiltinKeyGenerator {

public ComplexKeyGenerator(TypedProperties props) {
super(props);
this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()).split(","))
.map(String::trim)
.filter(s -> !s.isEmpty())
.collect(Collectors.toList());
this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key()).split(","))
.map(String::trim)
.filter(s -> !s.isEmpty())
.collect(Collectors.toList());
this.recordKeyFields = props.getStringList(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), ",",
Collections.singletonList(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.defaultValue()));
qjqqyy marked this conversation as resolved.
Show resolved Hide resolved
this.partitionPathFields = props.getStringList(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), ",",
Collections.emptyList());
complexAvroKeyGenerator = new ComplexAvroKeyGenerator(props);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,8 +31,7 @@
import org.apache.spark.sql.types.StructType;

import java.io.IOException;
import java.util.Arrays;
import java.util.stream.Collectors;
import java.util.Collections;

/**
* This is a generic implementation of KeyGenerator where users can configure record key as a single field or a combination of fields. Similarly partition path can be configured to have multiple
Expand All @@ -53,8 +52,10 @@ public class CustomKeyGenerator extends BuiltinKeyGenerator {

public CustomKeyGenerator(TypedProperties props) {
super(props);
this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()).split(",")).map(String::trim).collect(Collectors.toList());
this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key()).split(",")).map(String::trim).collect(Collectors.toList());
this.recordKeyFields = props.getStringList(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), ",",
Collections.singletonList(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.defaultValue()));
this.partitionPathFields = props.getStringList(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), ",",
Collections.emptyList());
customAvroKeyGenerator = new CustomAvroKeyGenerator(props);
}

Expand Down Expand Up @@ -95,7 +96,7 @@ private String getPartitionPath(Option<GenericRecord> record, Option<Row> row, O
StringBuilder partitionPath = new StringBuilder();

//Corresponds to no partition case
if (getPartitionPathFields().size() == 1 && getPartitionPathFields().get(0).isEmpty()) {
if (getPartitionPathFields().isEmpty()) {
return "";
}
for (String field : getPartitionPathFields()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,10 +26,8 @@
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.types.StructType;

import java.util.Arrays;
import java.util.Collections;
import java.util.List;
import java.util.stream.Collectors;

/**
* Simple Key generator for non-partitioned Hive Tables.
Expand All @@ -40,8 +38,8 @@ public class NonpartitionedKeyGenerator extends BuiltinKeyGenerator {

public NonpartitionedKeyGenerator(TypedProperties props) {
super(props);
this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key())
.split(",")).map(String::trim).collect(Collectors.toList());
this.recordKeyFields = props.getStringList(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key(), ",",
Collections.singletonList(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.defaultValue()));
this.partitionPathFields = Collections.emptyList();
nonpartitionedAvroKeyGenerator = new NonpartitionedAvroKeyGenerator(props);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -73,7 +73,7 @@ public List<String> getStringList(String property, String delimiter, List<String
if (!containsKey(property)) {
return defaultVal;
}
return Arrays.stream(getProperty(property).split(delimiter)).map(String::trim).collect(Collectors.toList());
return Arrays.stream(getProperty(property).split(delimiter)).map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList());
}

public int getInteger(String property) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -71,12 +71,15 @@ private TypedProperties getProps() {

@Test
public void testNullPartitionPathFields() {
Assertions.assertThrows(IllegalArgumentException.class, () -> new ComplexKeyGenerator(getPropertiesWithoutPartitionPathProp()));
ComplexKeyGenerator compositeKeyGenerator = new ComplexKeyGenerator(getPropertiesWithoutPartitionPathProp());
assertEquals(compositeKeyGenerator.getPartitionPathFields().size(), 0);
}

@Test
public void testNullRecordKeyFields() {
Assertions.assertThrows(IllegalArgumentException.class, () -> new ComplexKeyGenerator(getPropertiesWithoutRecordKeyProp()));
ComplexKeyGenerator compositeKeyGenerator = new ComplexKeyGenerator(getPropertiesWithoutRecordKeyProp());
assertEquals(compositeKeyGenerator.getRecordKeyFields().size(), 1);
assertEquals(compositeKeyGenerator.getRecordKeyFields().get(0), KeyGeneratorOptions.RECORDKEY_FIELD_NAME.defaultValue());
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -240,59 +240,22 @@ public void testInvalidPartitionKeyType(TypedProperties props) {
}

@Test
public void testNoRecordKeyFieldPropWithKeyGeneratorClass() {
public void testNoRecordKeyFieldPropWithKeyGeneratorClass() throws IOException {
testNoRecordKeyFieldProp(true);
}

@Test
public void testNoRecordKeyFieldPropWithKeyGeneratorType() {
public void testNoRecordKeyFieldPropWithKeyGeneratorType() throws IOException {
testNoRecordKeyFieldProp(false);
}

public void testNoRecordKeyFieldProp(boolean useKeyGeneratorClassName) {
public void testNoRecordKeyFieldProp(boolean useKeyGeneratorClassName) throws IOException {
TypedProperties propsWithoutRecordKeyFieldProps = getPropsWithoutRecordKeyFieldProps(useKeyGeneratorClassName);
try {
BuiltinKeyGenerator keyGenerator =
(BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(propsWithoutRecordKeyFieldProps);

keyGenerator.getKey(getRecord());
Assertions.fail("should fail when record key field is not provided!");
} catch (Exception e) {
if (useKeyGeneratorClassName) {
// "Property hoodie.datasource.write.recordkey.field not found" exception cause CustomKeyGenerator init fail
Assertions.assertTrue(e
.getCause()
.getCause()
.getCause()
.getMessage()
.contains("Property hoodie.datasource.write.recordkey.field not found"));
} else {
Assertions.assertTrue(stackTraceToString(e).contains("Property hoodie.datasource.write.recordkey.field not found"));
}

}

try {
BuiltinKeyGenerator keyGenerator =
(BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(propsWithoutRecordKeyFieldProps);
BuiltinKeyGenerator keyGenerator =
(BuiltinKeyGenerator) HoodieSparkKeyGeneratorFactory.createKeyGenerator(propsWithoutRecordKeyFieldProps);

GenericRecord record = getRecord();
Row row = KeyGeneratorTestUtilities.getRow(record);
keyGenerator.getRecordKey(row);
Assertions.fail("should fail when record key field is not provided!");
} catch (Exception e) {
if (useKeyGeneratorClassName) {
// "Property hoodie.datasource.write.recordkey.field not found" exception cause CustomKeyGenerator init fail
Assertions.assertTrue(e
.getCause()
.getCause()
.getCause()
.getMessage()
.contains("Property hoodie.datasource.write.recordkey.field not found"));
} else {
Assertions.assertTrue(stackTraceToString(e).contains("Property hoodie.datasource.write.recordkey.field not found"));
}
}
Assertions.assertEquals(keyGenerator.getRecordKeyFields().size(), 1);
Assertions.assertEquals(keyGenerator.getRecordKeyFields().get(0), KeyGeneratorOptions.RECORDKEY_FIELD_NAME.defaultValue());
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -277,50 +277,19 @@ class TestDataSourceDefaults {
assertEquals("field1/name1", keyGen.getPartitionPath(baseRow))

// partition path field not specified
try {
val props = new TypedProperties()
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD.key, "field1")
new ComplexKeyGenerator(props).getKey(baseRecord)
fail("Should have errored out")
} catch {
case e: IllegalArgumentException =>
// do nothing
}

// partition path field not specified using Row
try {
keyGen = {
val props = new TypedProperties()
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD.key, "field1")
val keyGen = new ComplexKeyGenerator(props)
keyGen.getRecordKey(baseRow)
fail("Should have errored out")
} catch {
case e: IllegalArgumentException =>
// do nothing
new ComplexKeyGenerator(props)
}

// recordkey field not specified
try {
val props = new TypedProperties()
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "partitionField")
new ComplexKeyGenerator(props).getKey(baseRecord)
fail("Should have errored out")
} catch {
case e: IllegalArgumentException =>
// do nothing
}
val nonPartitionedHk = keyGen.getKey(baseRecord)
assertEquals("field1:field1", nonPartitionedHk.getRecordKey)
assertEquals("", nonPartitionedHk.getPartitionPath)

// recordkey field not specified
try {
val props = new TypedProperties()
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "partitionField")
val keyGen = new ComplexKeyGenerator(props)
keyGen.getPartitionPath(baseRow)
fail("Should have errored out")
} catch {
case e: IllegalArgumentException =>
// do nothing
}
// partition path field not specified using Row
assertEquals("field1:field1", keyGen.getRecordKey(baseRow))
assertEquals("", keyGen.getPartitionPath(baseRow))

// nested field as record key and partition path
keyGen = new ComplexKeyGenerator(getKeyConfig("testNestedRecord.userId,testNestedRecord.isAdmin", "testNestedRecord.userId,testNestedRecord.isAdmin", "false"))
Expand Down