1
0

[HUDI-3640] Set SimpleKeyGenerator as default in 2to3 table upgrade for Spark engine (#5075)

This commit is contained in:
Y Ethan Guo
2022-03-21 17:35:06 -07:00
committed by GitHub
parent ca0931d332
commit 9b6e138af2
4 changed files with 104 additions and 9 deletions

View File

@@ -38,6 +38,8 @@ import org.apache.hudi.common.testutils.HoodieTestUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.keygen.SimpleKeyGenerator;
import org.apache.hudi.keygen.TimestampBasedKeyGenerator;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.marker.WriteMarkers;
@@ -117,6 +119,16 @@ public class TestUpgradeDowngrade extends HoodieClientTestBase {
return Stream.of(data).map(Arguments::of);
}
public static Stream<Arguments> twoToThreeUpgradeConfigParams() {
Object[][] data = new Object[][] {
{HoodieTableType.COPY_ON_WRITE, Option.empty()},
{HoodieTableType.COPY_ON_WRITE, Option.of(TimestampBasedKeyGenerator.class.getName())},
{HoodieTableType.MERGE_ON_READ, Option.empty()},
{HoodieTableType.MERGE_ON_READ, Option.of(TimestampBasedKeyGenerator.class.getName())}
};
return Stream.of(data).map(Arguments::of);
}
@BeforeEach
public void setUp() throws Exception {
initSparkContexts();
@@ -232,6 +244,51 @@ public class TestUpgradeDowngrade extends HoodieClientTestBase {
assertTableProps(cfg);
}
@ParameterizedTest
@MethodSource("twoToThreeUpgradeConfigParams")
public void testUpgradeTwoToThree(
HoodieTableType tableType, Option<String> keyGeneratorClass) throws IOException {
// init config, table and client.
Map<String, String> params = new HashMap<>();
addNewTableParamsToProps(params);
if (tableType == HoodieTableType.MERGE_ON_READ) {
params.put(TYPE.key(), HoodieTableType.MERGE_ON_READ.name());
metaClient = HoodieTestUtils.init(hadoopConf, basePath, HoodieTableType.MERGE_ON_READ);
}
HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder()
.withAutoCommit(false).withRollbackUsingMarkers(false).withProps(params);
if (keyGeneratorClass.isPresent()) {
cfgBuilder.withKeyGenerator(keyGeneratorClass.get());
}
HoodieWriteConfig cfg = cfgBuilder.build();
SparkRDDWriteClient client = getHoodieWriteClient(cfg);
// Write inserts
doInsert(client);
// downgrade table props
downgradeTableConfigsFromThreeToTwo(cfg);
// perform upgrade
new UpgradeDowngrade(metaClient, cfg, context, SparkUpgradeDowngradeHelper.getInstance())
.run(HoodieTableVersion.THREE, null);
// verify hoodie.table.version got upgraded
metaClient = HoodieTableMetaClient.builder().setConf(context.getHadoopConf().get()).setBasePath(cfg.getBasePath())
.setLayoutVersion(Option.of(new TimelineLayoutVersion(cfg.getTimelineLayoutVersion()))).build();
assertEquals(metaClient.getTableConfig().getTableVersion().versionCode(), HoodieTableVersion.THREE.versionCode());
assertTableVersionFromPropertyFile(HoodieTableVersion.THREE);
// verify table props
HoodieTableConfig tableConfig = metaClient.getTableConfig();
Properties originalProps = cfg.getProps();
assertEquals(tableConfig.getUrlEncodePartitioning(),
cfg.getStringOrDefault(HoodieTableConfig.URL_ENCODE_PARTITIONING));
assertEquals(tableConfig.getHiveStylePartitioningEnable(),
cfg.getStringOrDefault(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE));
assertEquals(tableConfig.getKeyGeneratorClassName(), originalProps.getOrDefault(
HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), SimpleKeyGenerator.class.getName()));
}
@Test
public void testUpgradeDowngradeBetweenThreeAndCurrentVersion() throws IOException {
// init config, table and client.
@@ -298,6 +355,19 @@ public class TestUpgradeDowngrade extends HoodieClientTestBase {
metaClient.getTableConfig().setTableVersion(HoodieTableVersion.ONE);
}
private void downgradeTableConfigsFromThreeToTwo(HoodieWriteConfig cfg) throws IOException {
Properties properties = new Properties(cfg.getProps());
properties.remove(HoodieTableConfig.URL_ENCODE_PARTITIONING.key());
properties.remove(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key());
properties.remove(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key());
properties.remove(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key());
properties.setProperty(HoodieTableConfig.VERSION.key(), "2");
metaClient = HoodieTestUtils.init(hadoopConf, basePath, getTableType(), properties);
// set hoodie.table.version to 2 in hoodie.properties file
metaClient.getTableConfig().setTableVersion(HoodieTableVersion.TWO);
}
private void assertTableProps(HoodieWriteConfig cfg) {
HoodieTableConfig tableConfig = metaClient.getTableConfig();
Properties originalProps = cfg.getProps();