1
0

[HUDI-2538] persist some configs to hoodie.properties when the first write (#3823)

This commit is contained in:
Yann Byron
2021-11-03 10:04:23 +08:00
committed by GitHub
parent 1f17467f73
commit 6351e5f4d0
25 changed files with 544 additions and 207 deletions

View File

@@ -19,14 +19,13 @@
package org.apache.hudi.keygen.factory;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.util.ReflectionUtils;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieKeyGeneratorException;
import org.apache.hudi.keygen.BuiltinKeyGenerator;
import org.apache.hudi.keygen.ComplexKeyGenerator;
import org.apache.hudi.keygen.CustomKeyGenerator;
import org.apache.hudi.keygen.GlobalDeleteKeyGenerator;
import org.apache.hudi.keygen.KeyGenUtils;
import org.apache.hudi.keygen.KeyGenerator;
import org.apache.hudi.keygen.NonpartitionedKeyGenerator;
import org.apache.hudi.keygen.SimpleKeyGenerator;
@@ -37,8 +36,9 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.HashMap;
import java.util.Locale;
import java.util.Objects;
import java.util.Map;
/**
* Factory help to create {@link org.apache.hudi.keygen.KeyGenerator}.
@@ -50,45 +50,73 @@ public class HoodieSparkKeyGeneratorFactory {
private static final Logger LOG = LoggerFactory.getLogger(HoodieSparkKeyGeneratorFactory.class);
private static final Map<String, String> COMMON_TO_SPARK_KEYGENERATOR = new HashMap<>();
static {
COMMON_TO_SPARK_KEYGENERATOR.put("org.apache.hudi.keygen.ComplexAvroKeyGenerator",
"org.apache.hudi.keygen.ComplexKeyGenerator");
COMMON_TO_SPARK_KEYGENERATOR.put("org.apache.hudi.keygen.CustomAvroKeyGenerator",
"org.apache.hudi.keygen.CustomKeyGenerator");
COMMON_TO_SPARK_KEYGENERATOR.put("org.apache.hudi.keygen.GlobalAvroDeleteKeyGenerator",
"org.apache.hudi.keygen.GlobalDeleteKeyGenerator");
COMMON_TO_SPARK_KEYGENERATOR.put("org.apache.hudi.keygen.NonpartitionedAvroKeyGenerator",
"org.apache.hudi.keygen.NonpartitionedKeyGenerator");
COMMON_TO_SPARK_KEYGENERATOR.put("org.apache.hudi.keygen.SimpleAvroKeyGenerator",
"org.apache.hudi.keygen.SimpleKeyGenerator");
COMMON_TO_SPARK_KEYGENERATOR.put("org.apache.hudi.keygen.TimestampBasedAvroKeyGenerator",
"org.apache.hudi.keygen.TimestampBasedKeyGenerator");
}
public static KeyGenerator createKeyGenerator(TypedProperties props) throws IOException {
// keyGenerator class name has higher priority
KeyGenerator keyGenerator = KeyGenUtils.createKeyGeneratorByClassName(props);
return Objects.isNull(keyGenerator) ? createKeyGeneratorByType(props) : keyGenerator;
}
private static BuiltinKeyGenerator createKeyGeneratorByType(TypedProperties props) throws IOException {
// Use KeyGeneratorType.SIMPLE as default keyGeneratorType
String keyGeneratorType =
props.getString(HoodieWriteConfig.KEYGENERATOR_TYPE.key(), null);
if (StringUtils.isNullOrEmpty(keyGeneratorType)) {
LOG.info("The value of {} is empty, use SIMPLE", HoodieWriteConfig.KEYGENERATOR_TYPE.key());
keyGeneratorType = KeyGeneratorType.SIMPLE.name();
}
KeyGeneratorType keyGeneratorTypeEnum;
String keyGeneratorClass = getKeyGeneratorClassName(props);
try {
keyGeneratorTypeEnum = KeyGeneratorType.valueOf(keyGeneratorType.toUpperCase(Locale.ROOT));
} catch (IllegalArgumentException e) {
throw new HoodieKeyGeneratorException("Unsupported keyGenerator Type " + keyGeneratorType);
}
switch (keyGeneratorTypeEnum) {
case SIMPLE:
return new SimpleKeyGenerator(props);
case COMPLEX:
return new ComplexKeyGenerator(props);
case TIMESTAMP:
return new TimestampBasedKeyGenerator(props);
case CUSTOM:
return new CustomKeyGenerator(props);
case NON_PARTITION:
return new NonpartitionedKeyGenerator(props);
case GLOBAL_DELETE:
return new GlobalDeleteKeyGenerator(props);
default:
throw new HoodieKeyGeneratorException("Unsupported keyGenerator Type " + keyGeneratorType);
return (KeyGenerator) ReflectionUtils.loadClass(keyGeneratorClass, props);
} catch (Throwable e) {
throw new IOException("Could not load key generator class " + keyGeneratorClass, e);
}
}
public static String getKeyGeneratorClassName(TypedProperties props) {
String keyGeneratorClass = props.getString(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), null);
if (StringUtils.isNullOrEmpty(keyGeneratorClass)) {
String keyGeneratorType = props.getString(HoodieWriteConfig.KEYGENERATOR_TYPE.key(), KeyGeneratorType.SIMPLE.name());
LOG.info("The value of {} is empty, use SIMPLE", HoodieWriteConfig.KEYGENERATOR_TYPE.key());
KeyGeneratorType keyGeneratorTypeEnum;
try {
keyGeneratorTypeEnum = KeyGeneratorType.valueOf(keyGeneratorType.toUpperCase(Locale.ROOT));
} catch (IllegalArgumentException e) {
throw new HoodieKeyGeneratorException("Unsupported keyGenerator Type " + keyGeneratorType);
}
switch (keyGeneratorTypeEnum) {
case SIMPLE:
keyGeneratorClass = SimpleKeyGenerator.class.getName();
break;
case COMPLEX:
keyGeneratorClass = ComplexKeyGenerator.class.getName();
break;
case TIMESTAMP:
keyGeneratorClass = TimestampBasedKeyGenerator.class.getName();
break;
case CUSTOM:
keyGeneratorClass = CustomKeyGenerator.class.getName();
break;
case NON_PARTITION:
keyGeneratorClass = NonpartitionedKeyGenerator.class.getName();
break;
case GLOBAL_DELETE:
keyGeneratorClass = GlobalDeleteKeyGenerator.class.getName();
break;
default:
throw new HoodieKeyGeneratorException("Unsupported keyGenerator Type " + keyGeneratorType);
}
}
return keyGeneratorClass;
}
/**
* Convert hoodie-common KeyGenerator to SparkKeyGeneratorInterface implement.
*/
public static String convertToSparkKeyGenerator(String keyGeneratorClassName) {
return COMMON_TO_SPARK_KEYGENERATOR.getOrDefault(keyGeneratorClassName, keyGeneratorClassName);
}
}

View File

@@ -22,6 +22,7 @@ import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.testutils.HoodieMetadataTestTable;
@@ -33,6 +34,7 @@ import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.config.metrics.HoodieMetricsConfig;
import org.apache.hudi.config.metrics.HoodieMetricsGraphiteConfig;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.keygen.SimpleKeyGenerator;
import org.apache.hudi.metadata.HoodieTableMetadata;
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
@@ -50,6 +52,7 @@ import java.io.IOException;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Properties;
import static java.util.Arrays.asList;
import static java.util.Collections.emptyList;
@@ -268,6 +271,8 @@ public class TestHoodieMetadataBase extends HoodieClientTestHarness {
protected HoodieWriteConfig.Builder getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy policy, boolean autoCommit, boolean useFileListingMetadata,
boolean enableMetrics, boolean enableFullScan) {
Properties properties = new Properties();
properties.put(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key(), SimpleKeyGenerator.class.getName());
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA)
.withParallelism(2, 2).withDeleteParallelism(2).withRollbackParallelism(2).withFinalizeWriteParallelism(2)
.withAutoCommit(autoCommit)
@@ -287,7 +292,8 @@ public class TestHoodieMetadataBase extends HoodieClientTestHarness {
.withMetricsConfig(HoodieMetricsConfig.newBuilder().on(enableMetrics)
.withExecutorMetrics(true).build())
.withMetricsGraphiteConfig(HoodieMetricsGraphiteConfig.newBuilder()
.usePrefix("unit-test").build());
.usePrefix("unit-test").build())
.withProperties(properties);
}
}

View File

@@ -33,6 +33,8 @@ import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.io.PrintWriter;
import java.io.StringWriter;
public class TestCustomKeyGenerator extends KeyGeneratorTestUtilities {
@@ -122,6 +124,13 @@ public class TestCustomKeyGenerator extends KeyGeneratorTestUtilities {
return properties;
}
private String stackTraceToString(Throwable e) {
StringWriter sw = new StringWriter();
PrintWriter pw = new PrintWriter(sw);
e.printStackTrace(pw);
return sw.toString();
}
@Test
public void testSimpleKeyGeneratorWithKeyGeneratorClass() throws IOException {
testSimpleKeyGenerator(getPropertiesForSimpleKeyGen(true));
@@ -259,7 +268,7 @@ public class TestCustomKeyGenerator extends KeyGeneratorTestUtilities {
.getMessage()
.contains("Property hoodie.datasource.write.recordkey.field not found"));
} else {
Assertions.assertTrue(e.getMessage().contains("Property hoodie.datasource.write.recordkey.field not found"));
Assertions.assertTrue(stackTraceToString(e).contains("Property hoodie.datasource.write.recordkey.field not found"));
}
}
@@ -282,7 +291,7 @@ public class TestCustomKeyGenerator extends KeyGeneratorTestUtilities {
.getMessage()
.contains("Property hoodie.datasource.write.recordkey.field not found"));
} else {
Assertions.assertTrue(e.getMessage().contains("Property hoodie.datasource.write.recordkey.field not found"));
Assertions.assertTrue(stackTraceToString(e).contains("Property hoodie.datasource.write.recordkey.field not found"));
}
}
}

View File

@@ -249,7 +249,15 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
initMetaClient(getTableType());
}
protected void initMetaClient(Properties properties) throws IOException {
initMetaClient(getTableType(), properties);
}
protected void initMetaClient(HoodieTableType tableType) throws IOException {
initMetaClient(tableType, new Properties());
}
protected void initMetaClient(HoodieTableType tableType, Properties properties) throws IOException {
if (basePath == null) {
throw new IllegalStateException("The base path has not been initialized.");
}
@@ -258,7 +266,10 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
throw new IllegalStateException("The Spark context has not been initialized.");
}
metaClient = HoodieTestUtils.init(hadoopConf, basePath, tableType);
if (tableName != null && !tableName.isEmpty()) {
properties.put(HoodieTableConfig.NAME.key(), tableName);
}
metaClient = HoodieTestUtils.init(hadoopConf, basePath, tableType, properties);
}
protected Properties getPropertiesForKeyGen() {