1
0

[HUDI-3730] Improve meta sync class design and hierarchies (#5854)

* [HUDI-3730] Improve meta sync class design and hierarchies (#5754)
* Implements class design proposed in RFC-55

Co-authored-by: jian.feng <fengjian428@gmial.com>
Co-authored-by: jian.feng <jian.feng@shopee.com>
This commit is contained in:
Shiyan Xu
2022-07-03 04:17:25 -05:00
committed by GitHub
parent c00ea84985
commit c0e1587966
86 changed files with 2977 additions and 2877 deletions

View File

@@ -18,9 +18,6 @@
package org.apache.hudi.cli;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
@@ -38,7 +35,10 @@ import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.hive.HiveSyncConfig;
import org.apache.hudi.hive.HiveSyncTool;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.sync.common.HoodieSyncConfig;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext;
@@ -48,6 +48,12 @@ import java.io.Serializable;
import java.util.HashMap;
import static org.apache.hudi.common.table.HoodieTableConfig.ARCHIVELOG_FOLDER;
import static org.apache.hudi.config.HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD;
import static org.apache.hudi.config.HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_BUCKET_SYNC;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_BUCKET_SYNC_SPEC;
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT;
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_BASE_PATH;
/**
* Performs bootstrap from a non-hudi source.
@@ -115,7 +121,7 @@ public class BootstrapExecutorUtils implements Serializable {
// Add more defaults if full bootstrap requested
this.props.putIfAbsent(DataSourceWriteOptions.PAYLOAD_CLASS_NAME().key(),
DataSourceWriteOptions.PAYLOAD_CLASS_NAME().defaultValue());
/**
/*
* Schema provider that supplies the command for reading the input and writing out the target table.
*/
SchemaProvider schemaProvider = createSchemaProvider(cfg.schemaProviderClass, props, jssc);
@@ -165,14 +171,14 @@ public class BootstrapExecutorUtils implements Serializable {
if (cfg.enableHiveSync) {
TypedProperties metaProps = new TypedProperties();
metaProps.putAll(props);
metaProps.put(HoodieSyncConfig.META_SYNC_BASE_PATH.key(), cfg.basePath);
metaProps.put(HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT.key(), cfg.baseFileFormat);
if (props.getBoolean(HiveSyncConfig.HIVE_SYNC_BUCKET_SYNC.key(), HiveSyncConfig.HIVE_SYNC_BUCKET_SYNC.defaultValue())) {
metaProps.put(HiveSyncConfig.HIVE_SYNC_BUCKET_SYNC_SPEC.key(), HiveSyncConfig.getBucketSpec(props.getString(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key()),
props.getInteger(HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS.key())));
metaProps.put(META_SYNC_BASE_PATH.key(), cfg.basePath);
metaProps.put(META_SYNC_BASE_FILE_FORMAT.key(), cfg.baseFileFormat);
if (props.getBoolean(HIVE_SYNC_BUCKET_SYNC.key(), HIVE_SYNC_BUCKET_SYNC.defaultValue())) {
metaProps.put(HIVE_SYNC_BUCKET_SYNC_SPEC.key(), HiveSyncConfig.getBucketSpec(props.getString(BUCKET_INDEX_HASH_FIELD.key()),
props.getInteger(BUCKET_INDEX_NUM_BUCKETS.key())));
}
new HiveSyncTool(metaProps, configuration, fs).syncHoodieTable();
new HiveSyncTool(metaProps, configuration).syncHoodieTable();
}
}

View File

@@ -20,11 +20,11 @@ package org.apache.spark.sql.hudi.command
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hudi.DataSourceWriteOptions
import org.apache.hudi.hive.HiveSyncConfig
import org.apache.hudi.hive.HiveSyncConfigHolder
import org.apache.hudi.sql.InsertMode
import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, HoodieCatalogTable}
import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable.needFilterProps
import org.apache.hudi.sync.common.util.ConfigUtils
import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable.needFilterProps
import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType, HoodieCatalogTable}
import org.apache.spark.sql.catalyst.plans.QueryPlan
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
import org.apache.spark.sql.hudi.HoodieSqlCommonUtils
@@ -94,9 +94,9 @@ case class CreateHoodieTableAsSelectCommand(
val tblProperties = hoodieCatalogTable.catalogProperties
val options = Map(
HiveSyncConfig.HIVE_CREATE_MANAGED_TABLE.key -> (table.tableType == CatalogTableType.MANAGED).toString,
HiveSyncConfig.HIVE_TABLE_SERDE_PROPERTIES.key -> ConfigUtils.configToString(tblProperties.asJava),
HiveSyncConfig.HIVE_TABLE_PROPERTIES.key -> ConfigUtils.configToString(newTable.properties.asJava),
HiveSyncConfigHolder.HIVE_CREATE_MANAGED_TABLE.key -> (table.tableType == CatalogTableType.MANAGED).toString,
HiveSyncConfigHolder.HIVE_TABLE_SERDE_PROPERTIES.key -> ConfigUtils.configToString(tblProperties.asJava),
HiveSyncConfigHolder.HIVE_TABLE_PROPERTIES.key -> ConfigUtils.configToString(newTable.properties.asJava),
DataSourceWriteOptions.SQL_INSERT_MODE.key -> InsertMode.NON_STRICT.value(),
DataSourceWriteOptions.SQL_ENABLE_BULK_INSERT.key -> "true"
)

View File

@@ -22,7 +22,7 @@ import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.common.util.StringUtils
import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME
import org.apache.hudi.hive.HiveSyncConfig
import org.apache.hudi.hive.HiveSyncConfigHolder
import org.apache.hudi.sync.common.HoodieSyncConfig
import org.apache.hudi.{AvroConversionUtils, DataSourceWriteOptions, HoodieSparkSqlWriter, SparkAdapterSupport}
import org.apache.spark.sql._
@@ -467,13 +467,13 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Hoodie
KEYGENERATOR_CLASS_NAME.key -> classOf[SqlKeyGenerator].getCanonicalName,
SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME -> tableConfig.getKeyGeneratorClassName,
HoodieSyncConfig.META_SYNC_ENABLED.key -> enableHive.toString,
HiveSyncConfig.HIVE_SYNC_ENABLED.key -> enableHive.toString,
HiveSyncConfig.HIVE_SYNC_MODE.key -> hiveSyncConfig.syncMode,
HiveSyncConfigHolder.HIVE_SYNC_MODE.key -> hiveSyncConfig.getString(HiveSyncConfigHolder.HIVE_SYNC_MODE),
HiveSyncConfigHolder.HIVE_SYNC_ENABLED.key -> enableHive.toString,
HoodieSyncConfig.META_SYNC_DATABASE_NAME.key -> targetTableDb,
HoodieSyncConfig.META_SYNC_TABLE_NAME.key -> targetTableName,
HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> hiveSyncConfig.supportTimestamp.toString,
HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> hiveSyncConfig.getBoolean(HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE).toString,
HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key -> tableConfig.getPartitionFieldProp,
HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key -> hiveSyncConfig.partitionValueExtractorClass,
HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key -> hiveSyncConfig.getString(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS),
HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key -> hoodieProps.getString(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "200"), // set the default parallelism to 200 for sql
HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> hoodieProps.getString(HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key, "200"),
HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key -> hoodieProps.getString(HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key, "200"),

View File

@@ -50,6 +50,12 @@ import java.util.stream.Collectors;
import static org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings;
import static org.apache.hudi.common.testutils.Transformations.randomSelectAsHoodieKeys;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_PASS;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_ENABLED;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_URL;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USER;
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME;
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_TABLE_NAME;
/**
* Sample program that writes & reads hoodie tables via the Spark datasource.
@@ -256,12 +262,12 @@ public class HoodieJavaApp {
private DataFrameWriter<Row> updateHiveSyncConfig(DataFrameWriter<Row> writer) {
if (enableHiveSync) {
LOG.info("Enabling Hive sync to " + hiveJdbcUrl);
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");
writer = writer.option(META_SYNC_TABLE_NAME.key(), hiveTable)
.option(META_SYNC_DATABASE_NAME.key(), hiveDB)
.option(HIVE_URL.key(), hiveJdbcUrl)
.option(HIVE_USER.key(), hiveUser)
.option(HIVE_PASS.key(), hivePass)
.option(HIVE_SYNC_ENABLED.key(), "true");
if (nonPartitionedTable) {
writer = writer
.option(HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(),

View File

@@ -46,6 +46,12 @@ import java.util.ArrayList;
import java.util.List;
import static org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_PASS;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_ENABLED;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_URL;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USER;
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME;
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_TABLE_NAME;
public class HoodieJavaGenerateApp {
@Parameter(names = {"--table-path", "-p"}, description = "Path for Hoodie sample table")
@@ -126,12 +132,12 @@ public class HoodieJavaGenerateApp {
private DataFrameWriter<Row> updateHiveSyncConfig(DataFrameWriter<Row> writer) {
if (enableHiveSync) {
LOG.info("Enabling Hive sync to " + hiveJdbcUrl);
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");
writer = writer.option(META_SYNC_TABLE_NAME.key(), hiveTable)
.option(META_SYNC_DATABASE_NAME.key(), hiveDB)
.option(HIVE_URL.key(), hiveJdbcUrl)
.option(HIVE_USER.key(), hiveUser)
.option(HIVE_PASS.key(), hivePass)
.option(HIVE_SYNC_ENABLED.key(), "true");
if (nonPartitionedTable) {
writer = writer
.option(HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(),

View File

@@ -54,6 +54,12 @@ import java.util.concurrent.Future;
import java.util.stream.Collectors;
import static org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_PASS;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_ENABLED;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_URL;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USER;
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME;
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_TABLE_NAME;
/**
* Sample program that writes & reads hoodie tables via the Spark datasource streaming.
@@ -383,12 +389,12 @@ public class HoodieJavaStreamingApp {
private DataStreamWriter<Row> updateHiveSyncConfig(DataStreamWriter<Row> writer) {
if (enableHiveSync) {
LOG.info("Enabling Hive sync to " + hiveJdbcUrl);
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");
writer = writer.option(META_SYNC_TABLE_NAME.key(), hiveTable)
.option(META_SYNC_DATABASE_NAME.key(), hiveDB)
.option(HIVE_URL.key(), hiveJdbcUrl)
.option(HIVE_USER.key(), hiveUser)
.option(HIVE_PASS.key(), hivePass)
.option(HIVE_SYNC_ENABLED.key(), "true");
if (useMultiPartitionKeys) {
writer = writer.option(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "year,month,day").option(
HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(),

View File

@@ -18,12 +18,6 @@
package org.apache.hudi;
import org.apache.avro.Conversions;
import org.apache.avro.LogicalTypes;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericFixed;
import org.apache.avro.generic.GenericRecord;
import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.common.config.TypedProperties;
@@ -37,8 +31,14 @@ import org.apache.hudi.config.HoodieStorageConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.execution.bulkinsert.RDDCustomColumnsSortPartitioner;
import org.apache.hudi.hive.HiveSyncConfig;
import org.apache.hudi.table.BulkInsertPartitioner;
import org.apache.avro.Conversions;
import org.apache.avro.LogicalTypes;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericData;
import org.apache.avro.generic.GenericFixed;
import org.apache.avro.generic.GenericRecord;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.sql.Dataset;
import org.apache.spark.sql.Row;
@@ -54,7 +54,6 @@ import org.junit.jupiter.api.extension.ExtendWith;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.Arguments;
import org.junit.jupiter.params.provider.MethodSource;
import org.junit.jupiter.params.provider.ValueSource;
import org.mockito.ArgumentCaptor;
import org.mockito.Captor;
import org.mockito.Mock;
@@ -70,18 +69,13 @@ import java.util.Map;
import java.util.stream.Stream;
import static org.apache.hudi.DataSourceUtils.tryOverrideParquetWriteLegacyFormatProperty;
import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET;
import static org.apache.hudi.hive.ddl.HiveSyncMode.HMS;
import static org.hamcrest.CoreMatchers.containsString;
import static org.hamcrest.CoreMatchers.equalTo;
import static org.hamcrest.CoreMatchers.instanceOf;
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertThrows;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.ArgumentMatchers.any;
import static org.mockito.ArgumentMatchers.anyString;
import static org.mockito.Mockito.times;
@@ -252,29 +246,6 @@ public class TestDataSourceUtils {
});
}
@ParameterizedTest
@ValueSource(booleans = {true, false})
public void testBuildHiveSyncConfig(boolean useSyncMode) {
TypedProperties props = new TypedProperties();
if (useSyncMode) {
props.setProperty(DataSourceWriteOptions.HIVE_SYNC_MODE().key(), HMS.name());
props.setProperty(DataSourceWriteOptions.HIVE_USE_JDBC().key(), String.valueOf(false));
}
props.setProperty(DataSourceWriteOptions.HIVE_DATABASE().key(), HIVE_DATABASE);
props.setProperty(DataSourceWriteOptions.HIVE_TABLE().key(), HIVE_TABLE);
HiveSyncConfig hiveSyncConfig = DataSourceUtils.buildHiveSyncConfig(props, config.getBasePath(), PARQUET.name());
if (useSyncMode) {
assertFalse(hiveSyncConfig.useJdbc);
assertEquals(HMS.name(), hiveSyncConfig.syncMode);
} else {
assertTrue(hiveSyncConfig.useJdbc);
assertNull(hiveSyncConfig.syncMode);
}
assertEquals(HIVE_DATABASE, hiveSyncConfig.databaseName);
assertEquals(HIVE_TABLE, hiveSyncConfig.tableName);
}
private void setAndVerifyHoodieWriteClientWith(final String partitionerClassName) {
config = HoodieWriteConfig.newBuilder().withPath(config.getBasePath())
.withUserDefinedBulkInsertPartitionerClass(partitionerClassName)