[HUDI-2883] Refactor hive sync tool / config to use reflection and standardize configs (#4175)
- Refactor hive sync tool / config to use reflection and standardize configs Co-authored-by: sivabalan <n.siva.b@gmail.com> Co-authored-by: Rajesh Mahindra <rmahindra@Rajeshs-MacBook-Pro.local> Co-authored-by: Raymond Xu <2701446+xushiyan@users.noreply.github.com>
This commit is contained in:
@@ -24,6 +24,7 @@ import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.hive.HiveSyncConfig;
|
||||
import org.apache.hudi.hive.MultiPartKeysValueExtractor;
|
||||
import org.apache.hudi.hive.NonPartitionedExtractor;
|
||||
import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor;
|
||||
@@ -255,24 +256,24 @@ public class HoodieJavaApp {
|
||||
private DataFrameWriter<Row> updateHiveSyncConfig(DataFrameWriter<Row> writer) {
|
||||
if (enableHiveSync) {
|
||||
LOG.info("Enabling Hive sync to " + hiveJdbcUrl);
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_TABLE().key(), hiveTable)
|
||||
.option(DataSourceWriteOptions.HIVE_DATABASE().key(), hiveDB)
|
||||
.option(DataSourceWriteOptions.HIVE_URL().key(), hiveJdbcUrl)
|
||||
.option(DataSourceWriteOptions.HIVE_USER().key(), hiveUser)
|
||||
.option(DataSourceWriteOptions.HIVE_PASS().key(), hivePass)
|
||||
.option(DataSourceWriteOptions.HIVE_SYNC_ENABLED().key(), "true");
|
||||
writer = writer.option(HiveSyncConfig.META_SYNC_TABLE_NAME.key(), hiveTable)
|
||||
.option(HiveSyncConfig.META_SYNC_DATABASE_NAME.key(), hiveDB)
|
||||
.option(HiveSyncConfig.HIVE_URL.key(), hiveJdbcUrl)
|
||||
.option(HiveSyncConfig.HIVE_USER.key(), hiveUser)
|
||||
.option(HiveSyncConfig.HIVE_PASS.key(), hivePass)
|
||||
.option(HiveSyncConfig.HIVE_SYNC_ENABLED.key(), "true");
|
||||
if (nonPartitionedTable) {
|
||||
writer = writer
|
||||
.option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(),
|
||||
.option(HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(),
|
||||
NonPartitionedExtractor.class.getCanonicalName())
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), "");
|
||||
} else if (useMultiPartitionKeys) {
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "year,month,day").option(
|
||||
DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(),
|
||||
writer = writer.option(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "year,month,day").option(
|
||||
HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(),
|
||||
MultiPartKeysValueExtractor.class.getCanonicalName());
|
||||
} else {
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "dateStr").option(
|
||||
DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(),
|
||||
writer = writer.option(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "dateStr").option(
|
||||
HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(),
|
||||
SlashEncodedDayPartitionValueExtractor.class.getCanonicalName());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -23,6 +23,7 @@ import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.hive.HiveSyncConfig;
|
||||
import org.apache.hudi.hive.MultiPartKeysValueExtractor;
|
||||
import org.apache.hudi.hive.NonPartitionedExtractor;
|
||||
import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor;
|
||||
@@ -125,24 +126,24 @@ public class HoodieJavaGenerateApp {
|
||||
private DataFrameWriter<Row> updateHiveSyncConfig(DataFrameWriter<Row> writer) {
|
||||
if (enableHiveSync) {
|
||||
LOG.info("Enabling Hive sync to " + hiveJdbcUrl);
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_TABLE().key(), hiveTable)
|
||||
.option(DataSourceWriteOptions.HIVE_DATABASE().key(), hiveDB)
|
||||
.option(DataSourceWriteOptions.HIVE_URL().key(), hiveJdbcUrl)
|
||||
.option(DataSourceWriteOptions.HIVE_USER().key(), hiveUser)
|
||||
.option(DataSourceWriteOptions.HIVE_PASS().key(), hivePass)
|
||||
.option(DataSourceWriteOptions.HIVE_SYNC_ENABLED().key(), "true");
|
||||
writer = writer.option(HiveSyncConfig.META_SYNC_TABLE_NAME.key(), hiveTable)
|
||||
.option(HiveSyncConfig.META_SYNC_DATABASE_NAME.key(), hiveDB)
|
||||
.option(HiveSyncConfig.HIVE_URL.key(), hiveJdbcUrl)
|
||||
.option(HiveSyncConfig.HIVE_USER.key(), hiveUser)
|
||||
.option(HiveSyncConfig.HIVE_PASS.key(), hivePass)
|
||||
.option(HiveSyncConfig.HIVE_SYNC_ENABLED.key(), "true");
|
||||
if (nonPartitionedTable) {
|
||||
writer = writer
|
||||
.option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(),
|
||||
.option(HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(),
|
||||
NonPartitionedExtractor.class.getCanonicalName())
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), "");
|
||||
} else if (useMultiPartitionKeys) {
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "year,month,day").option(
|
||||
DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(),
|
||||
writer = writer.option(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "year,month,day").option(
|
||||
HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(),
|
||||
MultiPartKeysValueExtractor.class.getCanonicalName());
|
||||
} else {
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "dateStr").option(
|
||||
DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(),
|
||||
writer = writer.option(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "dateStr").option(
|
||||
HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(),
|
||||
SlashEncodedDayPartitionValueExtractor.class.getCanonicalName());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -27,6 +27,7 @@ import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.TableNotFoundException;
|
||||
import org.apache.hudi.hive.HiveSyncConfig;
|
||||
import org.apache.hudi.hive.MultiPartKeysValueExtractor;
|
||||
import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor;
|
||||
|
||||
@@ -382,19 +383,19 @@ public class HoodieJavaStreamingApp {
|
||||
private DataStreamWriter<Row> updateHiveSyncConfig(DataStreamWriter<Row> writer) {
|
||||
if (enableHiveSync) {
|
||||
LOG.info("Enabling Hive sync to " + hiveJdbcUrl);
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_TABLE().key(), hiveTable)
|
||||
.option(DataSourceWriteOptions.HIVE_DATABASE().key(), hiveDB)
|
||||
.option(DataSourceWriteOptions.HIVE_URL().key(), hiveJdbcUrl)
|
||||
.option(DataSourceWriteOptions.HIVE_USER().key(), hiveUser)
|
||||
.option(DataSourceWriteOptions.HIVE_PASS().key(), hivePass)
|
||||
.option(DataSourceWriteOptions.HIVE_SYNC_ENABLED().key(), "true");
|
||||
writer = writer.option(HiveSyncConfig.META_SYNC_TABLE_NAME.key(), hiveTable)
|
||||
.option(HiveSyncConfig.META_SYNC_DATABASE_NAME.key(), hiveDB)
|
||||
.option(HiveSyncConfig.HIVE_URL.key(), hiveJdbcUrl)
|
||||
.option(HiveSyncConfig.HIVE_USER.key(), hiveUser)
|
||||
.option(HiveSyncConfig.HIVE_PASS.key(), hivePass)
|
||||
.option(HiveSyncConfig.HIVE_SYNC_ENABLED.key(), "true");
|
||||
if (useMultiPartitionKeys) {
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "year,month,day").option(
|
||||
DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(),
|
||||
writer = writer.option(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "year,month,day").option(
|
||||
HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(),
|
||||
MultiPartKeysValueExtractor.class.getCanonicalName());
|
||||
} else {
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "dateStr").option(
|
||||
DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(),
|
||||
writer = writer.option(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "dateStr").option(
|
||||
HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(),
|
||||
SlashEncodedDayPartitionValueExtractor.class.getCanonicalName());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -20,6 +20,7 @@ package org.apache.hudi
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.hive.{HiveStylePartitionValueExtractor, MultiPartKeysValueExtractor}
|
||||
import org.apache.hudi.keygen.{ComplexKeyGenerator, SimpleKeyGenerator}
|
||||
import org.apache.hudi.sync.common.HoodieSyncConfig
|
||||
import org.junit.jupiter.api.Assertions.assertEquals
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
@@ -31,10 +32,10 @@ class TestDataSourceOptions {
|
||||
)
|
||||
val modifiedOptions1 = HoodieWriterUtils.parametersWithWriteDefaults(inputOptions1)
|
||||
assertEquals(classOf[ComplexKeyGenerator].getName, modifiedOptions1(KEYGENERATOR_CLASS_NAME.key))
|
||||
assertEquals("hudi_table", modifiedOptions1(HIVE_TABLE.key))
|
||||
assertEquals("year,month", modifiedOptions1(HIVE_PARTITION_FIELDS.key))
|
||||
assertEquals("hudi_table", modifiedOptions1(HoodieSyncConfig.META_SYNC_TABLE_NAME.key))
|
||||
assertEquals("year,month", modifiedOptions1(HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key))
|
||||
assertEquals(classOf[MultiPartKeysValueExtractor].getName,
|
||||
modifiedOptions1(HIVE_PARTITION_EXTRACTOR_CLASS.key))
|
||||
modifiedOptions1(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key))
|
||||
|
||||
val inputOptions2 = Map(
|
||||
TABLE_NAME.key -> "hudi_table",
|
||||
@@ -43,9 +44,9 @@ class TestDataSourceOptions {
|
||||
)
|
||||
val modifiedOptions2 = HoodieWriterUtils.parametersWithWriteDefaults(inputOptions2)
|
||||
assertEquals(classOf[SimpleKeyGenerator].getName, modifiedOptions2(KEYGENERATOR_CLASS_NAME.key))
|
||||
assertEquals("hudi_table", modifiedOptions2(HIVE_TABLE.key))
|
||||
assertEquals("year", modifiedOptions2(HIVE_PARTITION_FIELDS.key))
|
||||
assertEquals("hudi_table", modifiedOptions2(HoodieSyncConfig.META_SYNC_TABLE_NAME.key))
|
||||
assertEquals("year", modifiedOptions2(HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key))
|
||||
assertEquals(classOf[HiveStylePartitionValueExtractor].getName,
|
||||
modifiedOptions2(HIVE_PARTITION_EXTRACTOR_CLASS.key))
|
||||
modifiedOptions2(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key))
|
||||
}
|
||||
}
|
||||
|
||||
@@ -664,55 +664,6 @@ class TestHoodieSparkSqlWriter {
|
||||
assertEquals(expectedSchema, actualSchema)
|
||||
}
|
||||
|
||||
/**
|
||||
* Test case for build sync config for spark sql.
|
||||
*/
|
||||
@Test
|
||||
def testBuildSyncConfigForSparkSql(): Unit = {
|
||||
val params = Map(
|
||||
"path" -> tempBasePath,
|
||||
DataSourceWriteOptions.TABLE_NAME.key -> "test_hoodie",
|
||||
DataSourceWriteOptions.HIVE_PARTITION_FIELDS.key -> "partition",
|
||||
DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.key -> "true",
|
||||
DataSourceWriteOptions.HIVE_CREATE_MANAGED_TABLE.key -> "true"
|
||||
)
|
||||
val parameters = HoodieWriterUtils.parametersWithWriteDefaults(params)
|
||||
val hoodieConfig = HoodieWriterUtils.convertMapToHoodieConfig(parameters)
|
||||
|
||||
val buildSyncConfigMethod =
|
||||
HoodieSparkSqlWriter.getClass.getDeclaredMethod("buildSyncConfig", classOf[Path],
|
||||
classOf[HoodieConfig], classOf[SQLConf])
|
||||
buildSyncConfigMethod.setAccessible(true)
|
||||
|
||||
val hiveSyncConfig = buildSyncConfigMethod.invoke(HoodieSparkSqlWriter,
|
||||
new Path(tempBasePath), hoodieConfig, spark.sessionState.conf).asInstanceOf[HiveSyncConfig]
|
||||
assertTrue(hiveSyncConfig.skipROSuffix)
|
||||
assertTrue(hiveSyncConfig.createManagedTable)
|
||||
assertTrue(hiveSyncConfig.syncAsSparkDataSourceTable)
|
||||
assertResult(spark.sessionState.conf.getConf(StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD))(hiveSyncConfig.sparkSchemaLengthThreshold)
|
||||
}
|
||||
|
||||
/**
|
||||
* Test case for build sync config for skip Ro Suffix values.
|
||||
*/
|
||||
@Test
|
||||
def testBuildSyncConfigForSkipRoSuffixValues(): Unit = {
|
||||
val params = Map(
|
||||
"path" -> tempBasePath,
|
||||
DataSourceWriteOptions.TABLE_NAME.key -> "test_hoodie",
|
||||
DataSourceWriteOptions.HIVE_PARTITION_FIELDS.key -> "partition"
|
||||
)
|
||||
val parameters = HoodieWriterUtils.parametersWithWriteDefaults(params)
|
||||
val hoodieConfig = HoodieWriterUtils.convertMapToHoodieConfig(parameters)
|
||||
val buildSyncConfigMethod =
|
||||
HoodieSparkSqlWriter.getClass.getDeclaredMethod("buildSyncConfig", classOf[Path],
|
||||
classOf[HoodieConfig], classOf[SQLConf])
|
||||
buildSyncConfigMethod.setAccessible(true)
|
||||
val hiveSyncConfig = buildSyncConfigMethod.invoke(HoodieSparkSqlWriter,
|
||||
new Path(tempBasePath), hoodieConfig, spark.sessionState.conf).asInstanceOf[HiveSyncConfig]
|
||||
assertFalse(hiveSyncConfig.skipROSuffix)
|
||||
}
|
||||
|
||||
/**
|
||||
* Test case for incremental view with replacement.
|
||||
*/
|
||||
|
||||
Reference in New Issue
Block a user