diff --git a/docker/demo/sparksql-incremental.commands b/docker/demo/sparksql-incremental.commands index 5ea4729b932f2..9ec586e49d854 100644 --- a/docker/demo/sparksql-incremental.commands +++ b/docker/demo/sparksql-incremental.commands @@ -21,7 +21,7 @@ import org.apache.hudi.DataSourceWriteOptions; import org.apache.spark.sql.SaveMode; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.HoodieDataSourceHelpers; -import org.apache.hudi.hive.HiveSyncConfig; +import org.apache.hudi.hive.HiveSyncConfigHolder; import org.apache.hudi.sync.common.HoodieSyncConfig; import org.apache.hudi.hive.MultiPartKeysValueExtractor; import org.apache.hadoop.fs.FileSystem; @@ -47,10 +47,10 @@ spark.sql("select key, `_hoodie_partition_path` as datestr, symbol, ts, open, cl option(HoodieWriteConfig.TBL_NAME.key(), "stock_ticks_derived_mor"). option(HoodieSyncConfig.META_SYNC_TABLE_NAME.key(), "stock_ticks_derived_mor"). option(HoodieSyncConfig.META_SYNC_DATABASE_NAME.key(), "default"). - option(HiveSyncConfig.HIVE_URL.key(), "jdbc:hive2://hiveserver:10000"). - option(HiveSyncConfig.HIVE_USER.key(), "hive"). - option(HiveSyncConfig.HIVE_PASS.key(), "hive"). - option(HiveSyncConfig.HIVE_SYNC_ENABLED.key(), "true"). + option(HiveSyncConfigHolder.HIVE_URL.key(), "jdbc:hive2://hiveserver:10000"). + option(HiveSyncConfigHolder.HIVE_USER.key(), "hive"). + option(HiveSyncConfigHolder.HIVE_PASS.key(), "hive"). + option(HiveSyncConfigHolder.HIVE_SYNC_ENABLED.key(), "true"). option(HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "datestr"). option(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), classOf[MultiPartKeysValueExtractor].getCanonicalName). option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key(), "true"). @@ -79,10 +79,10 @@ spark.sql("select key, `_hoodie_partition_path` as datestr, symbol, ts, open, cl option(HoodieWriteConfig.TBL_NAME.key(), "stock_ticks_derived_mor_bs"). option(HoodieSyncConfig.META_SYNC_TABLE_NAME.key(), "stock_ticks_derived_mor_bs"). option(HoodieSyncConfig.META_SYNC_DATABASE_NAME.key(), "default"). - option(HiveSyncConfig.HIVE_URL.key(), "jdbc:hive2://hiveserver:10000"). - option(HiveSyncConfig.HIVE_USER.key(), "hive"). - option(HiveSyncConfig.HIVE_PASS.key(), "hive"). - option(HiveSyncConfig.HIVE_SYNC_ENABLED.key(), "true"). + option(HiveSyncConfigHolder.HIVE_URL.key(), "jdbc:hive2://hiveserver:10000"). + option(HiveSyncConfigHolder.HIVE_USER.key(), "hive"). + option(HiveSyncConfigHolder.HIVE_PASS.key(), "hive"). + option(HiveSyncConfigHolder.HIVE_SYNC_ENABLED.key(), "true"). option(HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "datestr"). option(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), classOf[MultiPartKeysValueExtractor].getCanonicalName). option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key(), "true"). diff --git a/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncClient.java b/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncClient.java index e15a698f27e3a..f322ef79cafe0 100644 --- a/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncClient.java +++ b/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncClient.java @@ -21,8 +21,8 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; -import org.apache.hudi.hive.AbstractHiveSyncHoodieClient; import org.apache.hudi.hive.HiveSyncConfig; +import org.apache.hudi.sync.common.HoodieSyncClient; import org.apache.hudi.sync.common.model.Partition; import com.amazonaws.services.glue.AWSGlue; @@ -50,10 +50,6 @@ import com.amazonaws.services.glue.model.Table; import com.amazonaws.services.glue.model.TableInput; import com.amazonaws.services.glue.model.UpdateTableRequest; -import org.apache.avro.Schema; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.parquet.schema.MessageType; @@ -69,8 +65,12 @@ import static org.apache.hudi.aws.utils.S3Utils.s3aToS3; import static org.apache.hudi.common.util.MapUtils.nonEmpty; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_CREATE_MANAGED_TABLE; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE; import static org.apache.hudi.hive.util.HiveSchemaUtil.getPartitionKeyType; import static org.apache.hudi.hive.util.HiveSchemaUtil.parquetSchemaToMapSchema; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_PARTITION_FIELDS; import static org.apache.hudi.sync.common.util.TableUtils.tableId; /** @@ -79,7 +79,7 @@ * * @Experimental */ -public class AWSGlueCatalogSyncClient extends AbstractHiveSyncHoodieClient { +public class AWSGlueCatalogSyncClient extends HoodieSyncClient { private static final Logger LOG = LogManager.getLogger(AWSGlueCatalogSyncClient.class); private static final int MAX_PARTITIONS_PER_REQUEST = 100; @@ -87,10 +87,10 @@ public class AWSGlueCatalogSyncClient extends AbstractHiveSyncHoodieClient { private final AWSGlue awsGlue; private final String databaseName; - public AWSGlueCatalogSyncClient(HiveSyncConfig syncConfig, Configuration hadoopConf, FileSystem fs) { - super(syncConfig, hadoopConf, fs); + public AWSGlueCatalogSyncClient(HiveSyncConfig config) { + super(config); this.awsGlue = AWSGlueClientBuilder.standard().build(); - this.databaseName = syncConfig.databaseName; + this.databaseName = config.getStringOrDefault(META_SYNC_DATABASE_NAME); } @Override @@ -126,7 +126,7 @@ public void addPartitionsToTable(String tableName, List partitionsToAdd) StorageDescriptor sd = table.getStorageDescriptor(); List partitionInputs = partitionsToAdd.stream().map(partition -> { StorageDescriptor partitionSd = sd.clone(); - String fullPartitionPath = FSUtils.getPartitionPath(syncConfig.basePath, partition).toString(); + String fullPartitionPath = FSUtils.getPartitionPath(getBasePath(), partition).toString(); List partitionValues = partitionValueExtractor.extractPartitionValuesInPath(partition); partitionSd.setLocation(fullPartitionPath); return new PartitionInput().withValues(partitionValues).withStorageDescriptor(partitionSd); @@ -160,7 +160,7 @@ public void updatePartitionsToTable(String tableName, List changedPartit StorageDescriptor sd = table.getStorageDescriptor(); List updatePartitionEntries = changedPartitions.stream().map(partition -> { StorageDescriptor partitionSd = sd.clone(); - String fullPartitionPath = FSUtils.getPartitionPath(syncConfig.basePath, partition).toString(); + String fullPartitionPath = FSUtils.getPartitionPath(getBasePath(), partition).toString(); List partitionValues = partitionValueExtractor.extractPartitionValuesInPath(partition); sd.setLocation(fullPartitionPath); PartitionInput partitionInput = new PartitionInput().withValues(partitionValues).withStorageDescriptor(partitionSd); @@ -204,12 +204,12 @@ public void updateTableProperties(String tableName, Map tablePro } @Override - public void updateTableDefinition(String tableName, MessageType newSchema) { + public void updateTableSchema(String tableName, MessageType newSchema) { // ToDo Cascade is set in Hive meta sync, but need to investigate how to configure it for Glue meta - boolean cascade = syncConfig.partitionFields.size() > 0; + boolean cascade = config.getSplitStrings(META_SYNC_PARTITION_FIELDS).size() > 0; try { Table table = getTable(awsGlue, databaseName, tableName); - Map newSchemaMap = parquetSchemaToMapSchema(newSchema, syncConfig.supportTimestamp, false); + Map newSchemaMap = parquetSchemaToMapSchema(newSchema, config.getBoolean(HIVE_SUPPORT_TIMESTAMP_TYPE), false); List newColumns = newSchemaMap.keySet().stream().map(key -> { String keyType = getPartitionKeyType(newSchemaMap, key); return new Column().withName(key).withType(keyType.toLowerCase()).withComment(""); @@ -237,21 +237,6 @@ public void updateTableDefinition(String tableName, MessageType newSchema) { } } - @Override - public List getTableCommentUsingMetastoreClient(String tableName) { - throw new UnsupportedOperationException("Not supported: `getTableCommentUsingMetastoreClient`"); - } - - @Override - public void updateTableComments(String tableName, List oldSchema, List newSchema) { - throw new UnsupportedOperationException("Not supported: `updateTableComments`"); - } - - @Override - public void updateTableComments(String tableName, List oldSchema, Map newComments) { - throw new UnsupportedOperationException("Not supported: `updateTableComments`"); - } - @Override public void createTable(String tableName, MessageType storageSchema, @@ -265,26 +250,26 @@ public void createTable(String tableName, } CreateTableRequest request = new CreateTableRequest(); Map params = new HashMap<>(); - if (!syncConfig.createManagedTable) { + if (!config.getBoolean(HIVE_CREATE_MANAGED_TABLE)) { params.put("EXTERNAL", "TRUE"); } params.putAll(tableProperties); try { - Map mapSchema = parquetSchemaToMapSchema(storageSchema, syncConfig.supportTimestamp, false); + Map mapSchema = parquetSchemaToMapSchema(storageSchema, config.getBoolean(HIVE_SUPPORT_TIMESTAMP_TYPE), false); List schemaWithoutPartitionKeys = new ArrayList<>(); for (String key : mapSchema.keySet()) { String keyType = getPartitionKeyType(mapSchema, key); Column column = new Column().withName(key).withType(keyType.toLowerCase()).withComment(""); // In Glue, the full schema should exclude the partition keys - if (!syncConfig.partitionFields.contains(key)) { + if (!config.getSplitStrings(META_SYNC_PARTITION_FIELDS).contains(key)) { schemaWithoutPartitionKeys.add(column); } } // now create the schema partition - List schemaPartitionKeys = syncConfig.partitionFields.stream().map(partitionKey -> { + List schemaPartitionKeys = config.getSplitStrings(META_SYNC_PARTITION_FIELDS).stream().map(partitionKey -> { String keyType = getPartitionKeyType(mapSchema, partitionKey); return new Column().withName(partitionKey).withType(keyType.toLowerCase()).withComment(""); }).collect(Collectors.toList()); @@ -293,7 +278,7 @@ public void createTable(String tableName, serdeProperties.put("serialization.format", "1"); storageDescriptor .withSerdeInfo(new SerDeInfo().withSerializationLibrary(serdeClass).withParameters(serdeProperties)) - .withLocation(s3aToS3(syncConfig.basePath)) + .withLocation(s3aToS3(getBasePath())) .withInputFormat(inputFormatClass) .withOutputFormat(outputFormatClass) .withColumns(schemaWithoutPartitionKeys); @@ -320,7 +305,7 @@ public void createTable(String tableName, } @Override - public Map getTableSchema(String tableName) { + public Map getMetastoreSchema(String tableName) { try { // GlueMetastoreClient returns partition keys separate from Columns, hence get both and merge to // get the Schema of the table. @@ -340,11 +325,6 @@ public Map getTableSchema(String tableName) { } } - @Override - public boolean doesTableExist(String tableName) { - return tableExists(tableName); - } - @Override public boolean tableExists(String tableName) { GetTableRequest request = new GetTableRequest() @@ -412,11 +392,11 @@ public void close() { @Override public void updateLastCommitTimeSynced(String tableName) { - if (!activeTimeline.lastInstant().isPresent()) { + if (!getActiveTimeline().lastInstant().isPresent()) { LOG.warn("No commit in active timeline."); return; } - final String lastCommitTimestamp = activeTimeline.lastInstant().get().getTimestamp(); + final String lastCommitTimestamp = getActiveTimeline().lastInstant().get().getTimestamp(); try { updateTableParameters(awsGlue, databaseName, tableName, Collections.singletonMap(HOODIE_LAST_COMMIT_TIME_SYNC, lastCommitTimestamp), false); } catch (Exception e) { diff --git a/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AwsGlueCatalogSyncTool.java b/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncTool.java similarity index 58% rename from hudi-aws/src/main/java/org/apache/hudi/aws/sync/AwsGlueCatalogSyncTool.java rename to hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncTool.java index bb1be377c9cae..4d008159e2696 100644 --- a/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AwsGlueCatalogSyncTool.java +++ b/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncTool.java @@ -18,53 +18,44 @@ package org.apache.hudi.aws.sync; -import org.apache.hudi.common.config.TypedProperties; -import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.hive.HiveSyncConfig; import org.apache.hudi.hive.HiveSyncTool; import com.beust.jcommander.JCommander; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.hive.conf.HiveConf; + +import java.util.Properties; /** * Currently Experimental. Utility class that implements syncing a Hudi Table with the * AWS Glue Data Catalog (https://docs.aws.amazon.com/glue/latest/dg/populate-data-catalog.html) * to enable querying via Glue ETLs, Athena etc. - * + *

* Extends HiveSyncTool since most logic is similar to Hive syncing, * expect using a different client {@link AWSGlueCatalogSyncClient} that implements * the necessary functionality using Glue APIs. * * @Experimental */ -public class AwsGlueCatalogSyncTool extends HiveSyncTool { - - public AwsGlueCatalogSyncTool(TypedProperties props, Configuration conf, FileSystem fs) { - super(props, new HiveConf(conf, HiveConf.class), fs); - } +public class AWSGlueCatalogSyncTool extends HiveSyncTool { - public AwsGlueCatalogSyncTool(HiveSyncConfig hiveSyncConfig, HiveConf hiveConf, FileSystem fs) { - super(hiveSyncConfig, hiveConf, fs); + public AWSGlueCatalogSyncTool(Properties props, Configuration hadoopConf) { + super(props, hadoopConf); } @Override - protected void initClient(HiveSyncConfig hiveSyncConfig, HiveConf hiveConf) { - hoodieHiveClient = new AWSGlueCatalogSyncClient(hiveSyncConfig, hiveConf, fs); + protected void initSyncClient(HiveSyncConfig hiveSyncConfig) { + syncClient = new AWSGlueCatalogSyncClient(hiveSyncConfig); } public static void main(String[] args) { - // parse the params - final HiveSyncConfig cfg = new HiveSyncConfig(); - JCommander cmd = new JCommander(cfg, null, args); - if (cfg.help || args.length == 0) { + final HiveSyncConfig.HiveSyncConfigParams params = new HiveSyncConfig.HiveSyncConfigParams(); + JCommander cmd = JCommander.newBuilder().addObject(params).build(); + cmd.parse(args); + if (params.isHelp()) { cmd.usage(); - System.exit(1); + System.exit(0); } - FileSystem fs = FSUtils.getFs(cfg.basePath, new Configuration()); - HiveConf hiveConf = new HiveConf(); - hiveConf.addResource(fs.getConf()); - new AwsGlueCatalogSyncTool(cfg, hiveConf, fs).syncHoodieTable(); + new AWSGlueCatalogSyncTool(params.toProps(), new Configuration()).syncHoodieTable(); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java index 1aa0cfba5bc13..bacb4960b7f39 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java @@ -21,6 +21,7 @@ import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; +import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.exception.HoodieException; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -29,6 +30,7 @@ import java.io.Serializable; import java.lang.reflect.Modifier; import java.util.Arrays; +import java.util.List; import java.util.Properties; /** @@ -133,6 +135,14 @@ public String getString(ConfigProperty configProperty) { return rawValue.map(Object::toString).orElse(null); } + public List getSplitStrings(ConfigProperty configProperty) { + return getSplitStrings(configProperty, ","); + } + + public List getSplitStrings(ConfigProperty configProperty, String delimiter) { + return StringUtils.split(getString(configProperty), delimiter); + } + public String getString(String key) { return props.getProperty(key); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/config/TypedProperties.java b/hudi-common/src/main/java/org/apache/hudi/common/config/TypedProperties.java index 08015f61b2e04..f246b75be7aeb 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/config/TypedProperties.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/config/TypedProperties.java @@ -49,6 +49,12 @@ public TypedProperties(Properties defaults) { } } + public void setPropertyIfNonNull(String key, Object value) { + if (value != null) { + setProperty(key, value.toString()); + } + } + @Override public String getProperty(String key) { Object oval = super.get(key); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java index a18c3b20eb856..a4f2c62437caa 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/StringUtils.java @@ -19,6 +19,7 @@ package org.apache.hudi.common.util; import javax.annotation.Nullable; + import java.nio.ByteBuffer; import java.util.Collections; import java.util.List; @@ -65,6 +66,18 @@ public static String join(final String[] array, final String separator) { return org.apache.hadoop.util.StringUtils.join(separator, array); } + /** + * Wrapper of {@link java.lang.String#join(CharSequence, Iterable)}. + * + * Allow return {@code null} when {@code Iterable} is {@code null}. + */ + public static String join(CharSequence delimiter, Iterable elements) { + if (elements == null) { + return null; + } + return String.join(delimiter, elements); + } + public static String toHexString(byte[] bytes) { StringBuilder sb = new StringBuilder(bytes.length * 2); for (byte b : bytes) { @@ -77,6 +90,9 @@ public static boolean isNullOrEmpty(String str) { return str == null || str.length() == 0; } + public static boolean nonEmpty(String str) { + return !isNullOrEmpty(str); + } /** * Returns the given string if it is non-null; the empty string otherwise. diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestStringUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestStringUtils.java index a98e22ccbcb1e..bbaca74434dca 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/util/TestStringUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestStringUtils.java @@ -23,6 +23,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotEquals; @@ -45,6 +46,14 @@ public void testStringJoin() { assertNotEquals(null, StringUtils.join(STRINGS)); } + @Test + public void testStringJoinWithJavaImpl() { + assertNull(StringUtils.join(",", null)); + assertEquals("", String.join(",", Collections.singletonList(""))); + assertEquals(",", String.join(",", Arrays.asList("", ""))); + assertEquals("a,", String.join(",", Arrays.asList("a", ""))); + } + @Test public void testStringNullToEmpty() { String str = "This is a test"; diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java index 8973c405f3b49..e34adac580f70 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/utils/HiveSyncContext.java @@ -18,22 +18,43 @@ package org.apache.hudi.sink.utils; -import org.apache.hudi.aws.sync.AwsGlueCatalogSyncTool; +import org.apache.hudi.aws.sync.AWSGlueCatalogSyncTool; import org.apache.hudi.common.config.SerializableConfiguration; -import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.configuration.FlinkOptions; import org.apache.hudi.configuration.HadoopConfigurations; -import org.apache.hudi.hive.HiveSyncConfig; import org.apache.hudi.hive.HiveSyncTool; import org.apache.hudi.hive.ddl.HiveSyncMode; import org.apache.hudi.table.format.FilePathUtils; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.configuration.Configuration; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hive.conf.HiveConf; -import java.util.Arrays; +import java.util.Properties; + +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_AUTO_CREATE_DATABASE; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_IGNORE_EXCEPTIONS; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_PASS; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_MODE; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_TABLE_PROPERTIES; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_TABLE_SERDE_PROPERTIES; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_URL; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USER; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USE_JDBC; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USE_PRE_APACHE_INPUT_FORMAT; +import static org.apache.hudi.hive.HiveSyncConfigHolder.METASTORE_URIS; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_ASSUME_DATE_PARTITION; +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; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DECODE_PARTITION; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_PARTITION_FIELDS; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_TABLE_NAME; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_USE_FILE_LISTING_FROM_METADATA; /** * Hive synchronization context. @@ -41,63 +62,57 @@ *

Use this context to create the {@link HiveSyncTool} for synchronization. */ public class HiveSyncContext { - private final HiveSyncConfig syncConfig; + + private final Properties props; private final HiveConf hiveConf; - private final FileSystem fs; - private HiveSyncContext(HiveSyncConfig syncConfig, HiveConf hiveConf, FileSystem fs) { - this.syncConfig = syncConfig; + private HiveSyncContext(Properties props, HiveConf hiveConf) { + this.props = props; this.hiveConf = hiveConf; - this.fs = fs; } public HiveSyncTool hiveSyncTool() { - HiveSyncMode syncMode = HiveSyncMode.of(syncConfig.syncMode); + HiveSyncMode syncMode = HiveSyncMode.of(props.getProperty(HIVE_SYNC_MODE.key())); if (syncMode == HiveSyncMode.GLUE) { - return new AwsGlueCatalogSyncTool(this.syncConfig, this.hiveConf, this.fs); + return new AWSGlueCatalogSyncTool(props, hiveConf); } - return new HiveSyncTool(this.syncConfig, this.hiveConf, this.fs); + return new HiveSyncTool(props, hiveConf); } public static HiveSyncContext create(Configuration conf, SerializableConfiguration serConf) { - HiveSyncConfig syncConfig = buildSyncConfig(conf); + Properties props = buildSyncConfig(conf); org.apache.hadoop.conf.Configuration hadoopConf = HadoopConfigurations.getHadoopConf(conf); - String path = conf.getString(FlinkOptions.PATH); - FileSystem fs = FSUtils.getFs(path, hadoopConf); HiveConf hiveConf = new HiveConf(); - if (!FlinkOptions.isDefaultValueDefined(conf, FlinkOptions.HIVE_SYNC_METASTORE_URIS)) { - hadoopConf.set(HiveConf.ConfVars.METASTOREURIS.varname, conf.getString(FlinkOptions.HIVE_SYNC_METASTORE_URIS)); - } hiveConf.addResource(serConf.get()); hiveConf.addResource(hadoopConf); - return new HiveSyncContext(syncConfig, hiveConf, fs); + return new HiveSyncContext(props, hiveConf); } @VisibleForTesting - public static HiveSyncConfig buildSyncConfig(Configuration conf) { - HiveSyncConfig hiveSyncConfig = new HiveSyncConfig(); - hiveSyncConfig.basePath = conf.getString(FlinkOptions.PATH); - hiveSyncConfig.baseFileFormat = conf.getString(FlinkOptions.HIVE_SYNC_FILE_FORMAT); - hiveSyncConfig.usePreApacheInputFormat = false; - hiveSyncConfig.databaseName = conf.getString(FlinkOptions.HIVE_SYNC_DB); - hiveSyncConfig.tableName = conf.getString(FlinkOptions.HIVE_SYNC_TABLE); - hiveSyncConfig.syncMode = conf.getString(FlinkOptions.HIVE_SYNC_MODE); - hiveSyncConfig.hiveUser = conf.getString(FlinkOptions.HIVE_SYNC_USERNAME); - hiveSyncConfig.hivePass = conf.getString(FlinkOptions.HIVE_SYNC_PASSWORD); - hiveSyncConfig.tableProperties = conf.getString(FlinkOptions.HIVE_SYNC_TABLE_PROPERTIES); - hiveSyncConfig.serdeProperties = conf.getString(FlinkOptions.HIVE_SYNC_TABLE_SERDE_PROPERTIES); - hiveSyncConfig.jdbcUrl = conf.getString(FlinkOptions.HIVE_SYNC_JDBC_URL); - hiveSyncConfig.partitionFields = Arrays.asList(FilePathUtils.extractHivePartitionFields(conf)); - hiveSyncConfig.partitionValueExtractorClass = conf.getString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS_NAME); - hiveSyncConfig.useJdbc = conf.getBoolean(FlinkOptions.HIVE_SYNC_USE_JDBC); - hiveSyncConfig.useFileListingFromMetadata = conf.getBoolean(FlinkOptions.METADATA_ENABLED); - hiveSyncConfig.ignoreExceptions = conf.getBoolean(FlinkOptions.HIVE_SYNC_IGNORE_EXCEPTIONS); - hiveSyncConfig.supportTimestamp = conf.getBoolean(FlinkOptions.HIVE_SYNC_SUPPORT_TIMESTAMP); - hiveSyncConfig.autoCreateDatabase = conf.getBoolean(FlinkOptions.HIVE_SYNC_AUTO_CREATE_DB); - hiveSyncConfig.decodePartition = conf.getBoolean(FlinkOptions.URL_ENCODE_PARTITIONING); - hiveSyncConfig.skipROSuffix = conf.getBoolean(FlinkOptions.HIVE_SYNC_SKIP_RO_SUFFIX); - hiveSyncConfig.assumeDatePartitioning = conf.getBoolean(FlinkOptions.HIVE_SYNC_ASSUME_DATE_PARTITION); - hiveSyncConfig.withOperationField = conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED); - return hiveSyncConfig; + public static Properties buildSyncConfig(Configuration conf) { + TypedProperties props = new TypedProperties(); + props.setPropertyIfNonNull(META_SYNC_BASE_PATH.key(), conf.getString(FlinkOptions.PATH)); + props.setPropertyIfNonNull(META_SYNC_BASE_FILE_FORMAT.key(), conf.getString(FlinkOptions.HIVE_SYNC_FILE_FORMAT)); + props.setPropertyIfNonNull(HIVE_USE_PRE_APACHE_INPUT_FORMAT.key(), "false"); + props.setPropertyIfNonNull(META_SYNC_DATABASE_NAME.key(), conf.getString(FlinkOptions.HIVE_SYNC_DB)); + props.setPropertyIfNonNull(META_SYNC_TABLE_NAME.key(), conf.getString(FlinkOptions.HIVE_SYNC_TABLE)); + props.setPropertyIfNonNull(HIVE_SYNC_MODE.key(), conf.getString(FlinkOptions.HIVE_SYNC_MODE)); + props.setPropertyIfNonNull(HIVE_USER.key(), conf.getString(FlinkOptions.HIVE_SYNC_USERNAME)); + props.setPropertyIfNonNull(HIVE_PASS.key(), conf.getString(FlinkOptions.HIVE_SYNC_PASSWORD)); + props.setPropertyIfNonNull(HIVE_URL.key(), conf.getString(FlinkOptions.HIVE_SYNC_JDBC_URL)); + props.setPropertyIfNonNull(METASTORE_URIS.key(), conf.getString(FlinkOptions.HIVE_SYNC_METASTORE_URIS)); + props.setPropertyIfNonNull(HIVE_TABLE_PROPERTIES.key(), conf.getString(FlinkOptions.HIVE_SYNC_TABLE_PROPERTIES)); + props.setPropertyIfNonNull(HIVE_TABLE_SERDE_PROPERTIES.key(), conf.getString(FlinkOptions.HIVE_SYNC_TABLE_SERDE_PROPERTIES)); + props.setPropertyIfNonNull(META_SYNC_PARTITION_FIELDS.key(), String.join(",", FilePathUtils.extractHivePartitionFields(conf))); + props.setPropertyIfNonNull(META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), conf.getString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS_NAME)); + props.setPropertyIfNonNull(HIVE_USE_JDBC.key(), String.valueOf(conf.getBoolean(FlinkOptions.HIVE_SYNC_USE_JDBC))); + props.setPropertyIfNonNull(META_SYNC_USE_FILE_LISTING_FROM_METADATA.key(), String.valueOf(conf.getBoolean(FlinkOptions.METADATA_ENABLED))); + props.setPropertyIfNonNull(HIVE_IGNORE_EXCEPTIONS.key(), String.valueOf(conf.getBoolean(FlinkOptions.HIVE_SYNC_IGNORE_EXCEPTIONS))); + props.setPropertyIfNonNull(HIVE_SUPPORT_TIMESTAMP_TYPE.key(), String.valueOf(conf.getBoolean(FlinkOptions.HIVE_SYNC_SUPPORT_TIMESTAMP))); + props.setPropertyIfNonNull(HIVE_AUTO_CREATE_DATABASE.key(), String.valueOf(conf.getBoolean(FlinkOptions.HIVE_SYNC_AUTO_CREATE_DB))); + props.setPropertyIfNonNull(META_SYNC_DECODE_PARTITION.key(), String.valueOf(conf.getBoolean(FlinkOptions.URL_ENCODE_PARTITIONING))); + props.setPropertyIfNonNull(HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.key(), String.valueOf(conf.getBoolean(FlinkOptions.HIVE_SYNC_SKIP_RO_SUFFIX))); + props.setPropertyIfNonNull(META_SYNC_ASSUME_DATE_PARTITION.key(), String.valueOf(conf.getBoolean(FlinkOptions.HIVE_SYNC_ASSUME_DATE_PARTITION))); + return props; } } diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/TestHiveSyncContext.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/TestHiveSyncContext.java index 5226f55bffe9e..381b292c25f9f 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/TestHiveSyncContext.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/utils/TestHiveSyncContext.java @@ -19,14 +19,15 @@ package org.apache.hudi.sink.utils; import org.apache.hudi.configuration.FlinkOptions; -import org.apache.hudi.hive.HiveSyncConfig; import org.apache.flink.configuration.Configuration; import org.junit.jupiter.api.Test; import java.lang.reflect.Method; +import java.util.Properties; -import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_PARTITION_FIELDS; +import static org.junit.jupiter.api.Assertions.assertEquals; /** * Test cases for {@link HiveSyncContext}. @@ -51,11 +52,11 @@ public void testBuildSyncConfig() throws Exception { Method buildSyncConfigMethod = threadClazz.getDeclaredMethod("buildSyncConfig", Configuration.class); buildSyncConfigMethod.setAccessible(true); - HiveSyncConfig hiveSyncConfig1 = HiveSyncContext.buildSyncConfig(configuration1); - HiveSyncConfig hiveSyncConfig2 = HiveSyncContext.buildSyncConfig(configuration2); + Properties props1 = HiveSyncContext.buildSyncConfig(configuration1); + Properties props2 = HiveSyncContext.buildSyncConfig(configuration2); - assertTrue(hiveSyncConfig1.partitionFields.get(0).equals(hiveSyncPartitionField)); - assertTrue(hiveSyncConfig2.partitionFields.get(0).equals(partitionPathField)); + assertEquals(hiveSyncPartitionField, props1.getProperty(META_SYNC_PARTITION_FIELDS.key())); + assertEquals(partitionPathField, props2.getProperty(META_SYNC_PARTITION_FIELDS.key())); } } diff --git a/hudi-gcp/src/main/java/org/apache/hudi/gcp/bigquery/BigQuerySyncConfig.java b/hudi-gcp/src/main/java/org/apache/hudi/gcp/bigquery/BigQuerySyncConfig.java index 6aa9bc0b535a0..b46cd9a9f81f7 100644 --- a/hudi-gcp/src/main/java/org/apache/hudi/gcp/bigquery/BigQuerySyncConfig.java +++ b/hudi-gcp/src/main/java/org/apache/hudi/gcp/bigquery/BigQuerySyncConfig.java @@ -19,113 +19,121 @@ package org.apache.hudi.gcp.bigquery; -import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.config.ConfigProperty; +import org.apache.hudi.sync.common.HoodieSyncConfig; import com.beust.jcommander.Parameter; +import com.beust.jcommander.ParametersDelegate; import java.io.Serializable; import java.util.ArrayList; -import java.util.Collections; import java.util.List; +import java.util.Properties; /** * Configs needed to sync data into BigQuery. */ -public class BigQuerySyncConfig implements Serializable { - - public static String BIGQUERY_SYNC_PROJECT_ID = "hoodie.gcp.bigquery.sync.project_id"; - public static String BIGQUERY_SYNC_DATASET_NAME = "hoodie.gcp.bigquery.sync.dataset_name"; - public static String BIGQUERY_SYNC_DATASET_LOCATION = "hoodie.gcp.bigquery.sync.dataset_location"; - public static String BIGQUERY_SYNC_TABLE_NAME = "hoodie.gcp.bigquery.sync.table_name"; - public static String BIGQUERY_SYNC_SOURCE_URI = "hoodie.gcp.bigquery.sync.source_uri"; - public static String BIGQUERY_SYNC_SOURCE_URI_PREFIX = "hoodie.gcp.bigquery.sync.source_uri_prefix"; - public static String BIGQUERY_SYNC_SYNC_BASE_PATH = "hoodie.gcp.bigquery.sync.base_path"; - public static String BIGQUERY_SYNC_PARTITION_FIELDS = "hoodie.gcp.bigquery.sync.partition_fields"; - public static String BIGQUERY_SYNC_USE_FILE_LISTING_FROM_METADATA = "hoodie.gcp.bigquery.sync.use_file_listing_from_metadata"; - public static String BIGQUERY_SYNC_ASSUME_DATE_PARTITIONING = "hoodie.gcp.bigquery.sync.assume_date_partitioning"; - - @Parameter(names = {"--project-id"}, description = "name of the target project in BigQuery", required = true) - public String projectId; - @Parameter(names = {"--dataset-name"}, description = "name of the target dataset in BigQuery", required = true) - public String datasetName; - @Parameter(names = {"--dataset-location"}, description = "location of the target dataset in BigQuery", required = true) - public String datasetLocation; - @Parameter(names = {"--table-name"}, description = "name of the target table in BigQuery", required = true) - public String tableName; - @Parameter(names = {"--source-uri"}, description = "name of the source uri gcs path of the table", required = true) - public String sourceUri; - @Parameter(names = {"--source-uri-prefix"}, description = "name of the source uri gcs path prefix of the table", required = true) - public String sourceUriPrefix; - @Parameter(names = {"--base-path"}, description = "Base path of the hoodie table to sync", required = true) - public String basePath; - @Parameter(names = {"--partitioned-by"}, description = "Comma-delimited partition fields. Default to non-partitioned.") - public List partitionFields = new ArrayList<>(); - @Parameter(names = {"--use-file-listing-from-metadata"}, description = "Fetch file listing from Hudi's metadata") - public Boolean useFileListingFromMetadata = false; - @Parameter(names = {"--assume-date-partitioning"}, description = "Assume standard yyyy/mm/dd partitioning, this" - + " exists to support backward compatibility. If you use hoodie 0.3.x, do not set this parameter") - public Boolean assumeDatePartitioning = false; - @Parameter(names = {"--help", "-h"}, help = true) - public Boolean help = false; - - public static BigQuerySyncConfig copy(BigQuerySyncConfig cfg) { - BigQuerySyncConfig newConfig = new BigQuerySyncConfig(); - newConfig.projectId = cfg.projectId; - newConfig.datasetName = cfg.datasetName; - newConfig.datasetLocation = cfg.datasetLocation; - newConfig.tableName = cfg.tableName; - newConfig.sourceUri = cfg.sourceUri; - newConfig.sourceUriPrefix = cfg.sourceUriPrefix; - newConfig.basePath = cfg.basePath; - newConfig.partitionFields = cfg.partitionFields; - newConfig.useFileListingFromMetadata = cfg.useFileListingFromMetadata; - newConfig.assumeDatePartitioning = cfg.assumeDatePartitioning; - newConfig.help = cfg.help; - return newConfig; - } +public class BigQuerySyncConfig extends HoodieSyncConfig implements Serializable { - public TypedProperties toProps() { - TypedProperties properties = new TypedProperties(); - properties.put(BIGQUERY_SYNC_PROJECT_ID, projectId); - properties.put(BIGQUERY_SYNC_DATASET_NAME, datasetName); - properties.put(BIGQUERY_SYNC_DATASET_LOCATION, datasetLocation); - properties.put(BIGQUERY_SYNC_TABLE_NAME, tableName); - properties.put(BIGQUERY_SYNC_SOURCE_URI, sourceUri); - properties.put(BIGQUERY_SYNC_SOURCE_URI_PREFIX, sourceUriPrefix); - properties.put(BIGQUERY_SYNC_SYNC_BASE_PATH, basePath); - properties.put(BIGQUERY_SYNC_PARTITION_FIELDS, String.join(",", partitionFields)); - properties.put(BIGQUERY_SYNC_USE_FILE_LISTING_FROM_METADATA, useFileListingFromMetadata); - properties.put(BIGQUERY_SYNC_ASSUME_DATE_PARTITIONING, assumeDatePartitioning); - return properties; - } + public static final ConfigProperty BIGQUERY_SYNC_PROJECT_ID = ConfigProperty + .key("hoodie.gcp.bigquery.sync.project_id") + .noDefaultValue() + .withDocumentation("Name of the target project in BigQuery"); + + public static final ConfigProperty BIGQUERY_SYNC_DATASET_NAME = ConfigProperty + .key("hoodie.gcp.bigquery.sync.dataset_name") + .noDefaultValue() + .withDocumentation("Name of the target dataset in BigQuery"); + + public static final ConfigProperty BIGQUERY_SYNC_DATASET_LOCATION = ConfigProperty + .key("hoodie.gcp.bigquery.sync.dataset_location") + .noDefaultValue() + .withDocumentation("Location of the target dataset in BigQuery"); + + public static final ConfigProperty BIGQUERY_SYNC_TABLE_NAME = ConfigProperty + .key("hoodie.gcp.bigquery.sync.table_name") + .noDefaultValue() + .withDocumentation("Name of the target table in BigQuery"); + + public static final ConfigProperty BIGQUERY_SYNC_SOURCE_URI = ConfigProperty + .key("hoodie.gcp.bigquery.sync.source_uri") + .noDefaultValue() + .withDocumentation("Name of the source uri gcs path of the table"); + + public static final ConfigProperty BIGQUERY_SYNC_SOURCE_URI_PREFIX = ConfigProperty + .key("hoodie.gcp.bigquery.sync.source_uri_prefix") + .noDefaultValue() + .withDocumentation("Name of the source uri gcs path prefix of the table"); + + public static final ConfigProperty BIGQUERY_SYNC_SYNC_BASE_PATH = ConfigProperty + .key("hoodie.gcp.bigquery.sync.base_path") + .noDefaultValue() + .withDocumentation("Base path of the hoodie table to sync"); - public static BigQuerySyncConfig fromProps(TypedProperties props) { - BigQuerySyncConfig config = new BigQuerySyncConfig(); - config.projectId = props.getString(BIGQUERY_SYNC_PROJECT_ID); - config.datasetName = props.getString(BIGQUERY_SYNC_DATASET_NAME); - config.datasetLocation = props.getString(BIGQUERY_SYNC_DATASET_LOCATION); - config.tableName = props.getString(BIGQUERY_SYNC_TABLE_NAME); - config.sourceUri = props.getString(BIGQUERY_SYNC_SOURCE_URI); - config.sourceUriPrefix = props.getString(BIGQUERY_SYNC_SOURCE_URI_PREFIX); - config.basePath = props.getString(BIGQUERY_SYNC_SYNC_BASE_PATH); - config.partitionFields = props.getStringList(BIGQUERY_SYNC_PARTITION_FIELDS, ",", Collections.emptyList()); - config.useFileListingFromMetadata = props.getBoolean(BIGQUERY_SYNC_USE_FILE_LISTING_FROM_METADATA, false); - config.assumeDatePartitioning = props.getBoolean(BIGQUERY_SYNC_ASSUME_DATE_PARTITIONING, false); - return config; + public static final ConfigProperty BIGQUERY_SYNC_PARTITION_FIELDS = ConfigProperty + .key("hoodie.gcp.bigquery.sync.partition_fields") + .noDefaultValue() + .withDocumentation("Comma-delimited partition fields. Default to non-partitioned."); + + public static final ConfigProperty BIGQUERY_SYNC_USE_FILE_LISTING_FROM_METADATA = ConfigProperty + .key("hoodie.gcp.bigquery.sync.use_file_listing_from_metadata") + .defaultValue(false) + .withDocumentation("Fetch file listing from Hudi's metadata"); + + public static final ConfigProperty BIGQUERY_SYNC_ASSUME_DATE_PARTITIONING = ConfigProperty + .key("hoodie.gcp.bigquery.sync.assume_date_partitioning") + .defaultValue(false) + .withDocumentation("Assume standard yyyy/mm/dd partitioning, this" + + " exists to support backward compatibility. If you use hoodie 0.3.x, do not set this parameter"); + + public BigQuerySyncConfig(Properties props) { + super(props); } - @Override - public String toString() { - return "BigQuerySyncConfig{projectId='" + projectId - + "', datasetName='" + datasetName - + "', datasetLocation='" + datasetLocation - + "', tableName='" + tableName - + "', sourceUri='" + sourceUri - + "', sourceUriPrefix='" + sourceUriPrefix - + "', basePath='" + basePath + "'" - + ", partitionFields=" + partitionFields - + "', useFileListingFromMetadata='" + useFileListingFromMetadata - + "', assumeDataPartitioning='" + assumeDatePartitioning - + "', help=" + help + "}"; + public static class BigQuerySyncConfigParams { + + @ParametersDelegate() + public final HoodieSyncConfigParams hoodieSyncConfigParams = new HoodieSyncConfigParams(); + + @Parameter(names = {"--project-id"}, description = "Name of the target project in BigQuery", required = true) + public String projectId; + @Parameter(names = {"--dataset-name"}, description = "Name of the target dataset in BigQuery", required = true) + public String datasetName; + @Parameter(names = {"--dataset-location"}, description = "Location of the target dataset in BigQuery", required = true) + public String datasetLocation; + @Parameter(names = {"--table-name"}, description = "Name of the target table in BigQuery", required = true) + public String tableName; + @Parameter(names = {"--source-uri"}, description = "Name of the source uri gcs path of the table", required = true) + public String sourceUri; + @Parameter(names = {"--source-uri-prefix"}, description = "Name of the source uri gcs path prefix of the table", required = true) + public String sourceUriPrefix; + @Parameter(names = {"--base-path"}, description = "Base path of the hoodie table to sync", required = true) + public String basePath; + @Parameter(names = {"--partitioned-by"}, description = "Comma-delimited partition fields. Default to non-partitioned.") + public List partitionFields = new ArrayList<>(); + @Parameter(names = {"--use-file-listing-from-metadata"}, description = "Fetch file listing from Hudi's metadata") + public boolean useFileListingFromMetadata = false; + @Parameter(names = {"--assume-date-partitioning"}, description = "Assume standard yyyy/mm/dd partitioning, this" + + " exists to support backward compatibility. If you use hoodie 0.3.x, do not set this parameter") + public boolean assumeDatePartitioning = false; + + public boolean isHelp() { + return hoodieSyncConfigParams.isHelp(); + } + + public Properties toProps() { + final Properties props = hoodieSyncConfigParams.toProps(); + props.setProperty(BIGQUERY_SYNC_PROJECT_ID.key(), projectId); + props.setProperty(BIGQUERY_SYNC_DATASET_NAME.key(), datasetName); + props.setProperty(BIGQUERY_SYNC_DATASET_LOCATION.key(), datasetLocation); + props.setProperty(BIGQUERY_SYNC_TABLE_NAME.key(), tableName); + props.setProperty(BIGQUERY_SYNC_SOURCE_URI.key(), sourceUri); + props.setProperty(BIGQUERY_SYNC_SOURCE_URI_PREFIX.key(), sourceUriPrefix); + props.setProperty(BIGQUERY_SYNC_SYNC_BASE_PATH.key(), basePath); + props.setProperty(BIGQUERY_SYNC_PARTITION_FIELDS.key(), String.join(",", partitionFields)); + props.setProperty(BIGQUERY_SYNC_USE_FILE_LISTING_FROM_METADATA.key(), String.valueOf(useFileListingFromMetadata)); + props.setProperty(BIGQUERY_SYNC_ASSUME_DATE_PARTITIONING.key(), String.valueOf(assumeDatePartitioning)); + return props; + } } } diff --git a/hudi-gcp/src/main/java/org/apache/hudi/gcp/bigquery/BigQuerySyncTool.java b/hudi-gcp/src/main/java/org/apache/hudi/gcp/bigquery/BigQuerySyncTool.java index 0cb75eea89bad..4716d6e494997 100644 --- a/hudi-gcp/src/main/java/org/apache/hudi/gcp/bigquery/BigQuerySyncTool.java +++ b/hudi-gcp/src/main/java/org/apache/hudi/gcp/bigquery/BigQuerySyncTool.java @@ -19,19 +19,26 @@ package org.apache.hudi.gcp.bigquery; -import org.apache.hudi.common.config.TypedProperties; -import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.util.ValidationUtils; -import org.apache.hudi.sync.common.AbstractSyncTool; +import org.apache.hudi.sync.common.HoodieSyncTool; import org.apache.hudi.sync.common.util.ManifestFileWriter; import com.beust.jcommander.JCommander; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import java.util.Properties; + +import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_ASSUME_DATE_PARTITIONING; +import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_DATASET_NAME; +import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_PARTITION_FIELDS; +import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_SOURCE_URI; +import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_SOURCE_URI_PREFIX; +import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_SYNC_BASE_PATH; +import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_TABLE_NAME; +import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_USE_FILE_LISTING_FROM_METADATA; + /** * Tool to sync a hoodie table with a big query table. Either use it as an api * BigQuerySyncTool.syncHoodieTable(BigQuerySyncConfig) or as a command line java -cp hoodie-hive.jar BigQuerySyncTool [args] @@ -40,26 +47,28 @@ * * @Experimental */ -public class BigQuerySyncTool extends AbstractSyncTool { +public class BigQuerySyncTool extends HoodieSyncTool { private static final Logger LOG = LogManager.getLogger(BigQuerySyncTool.class); - public final BigQuerySyncConfig cfg; + public final BigQuerySyncConfig config; + public final String tableName; public final String manifestTableName; public final String versionsTableName; public final String snapshotViewName; - public BigQuerySyncTool(TypedProperties properties, Configuration conf, FileSystem fs) { - super(properties, conf, fs); - cfg = BigQuerySyncConfig.fromProps(properties); - manifestTableName = cfg.tableName + "_manifest"; - versionsTableName = cfg.tableName + "_versions"; - snapshotViewName = cfg.tableName; + public BigQuerySyncTool(Properties props) { + super(props); + this.config = new BigQuerySyncConfig(props); + this.tableName = config.getString(BIGQUERY_SYNC_TABLE_NAME); + this.manifestTableName = tableName + "_manifest"; + this.versionsTableName = tableName + "_versions"; + this.snapshotViewName = tableName; } @Override public void syncHoodieTable() { - try (HoodieBigQuerySyncClient bqSyncClient = new HoodieBigQuerySyncClient(BigQuerySyncConfig.fromProps(props), fs)) { + try (HoodieBigQuerySyncClient bqSyncClient = new HoodieBigQuerySyncClient(config)) { switch (bqSyncClient.getTableType()) { case COPY_ON_WRITE: syncCoWTable(bqSyncClient); @@ -69,7 +78,7 @@ public void syncHoodieTable() { throw new UnsupportedOperationException(bqSyncClient.getTableType() + " table type is not supported yet."); } } catch (Exception e) { - throw new HoodieBigQuerySyncException("Got runtime exception when big query syncing " + cfg.tableName, e); + throw new HoodieBigQuerySyncException("Failed to sync BigQuery for table:" + tableName, e); } } @@ -78,14 +87,14 @@ private void syncCoWTable(HoodieBigQuerySyncClient bqSyncClient) { LOG.info("Sync hoodie table " + snapshotViewName + " at base path " + bqSyncClient.getBasePath()); if (!bqSyncClient.datasetExists()) { - throw new HoodieBigQuerySyncException("Dataset not found: " + cfg); + throw new HoodieBigQuerySyncException("Dataset not found: " + config.getString(BIGQUERY_SYNC_DATASET_NAME)); } ManifestFileWriter manifestFileWriter = ManifestFileWriter.builder() - .setConf(conf) - .setBasePath(cfg.basePath) - .setUseFileListingFromMetadata(cfg.useFileListingFromMetadata) - .setAssumeDatePartitioning(cfg.assumeDatePartitioning) + .setConf(config.getHadoopConf()) + .setBasePath(config.getString(BIGQUERY_SYNC_SYNC_BASE_PATH)) + .setUseFileListingFromMetadata(config.getBoolean(BIGQUERY_SYNC_USE_FILE_LISTING_FROM_METADATA)) + .setAssumeDatePartitioning(config.getBoolean(BIGQUERY_SYNC_ASSUME_DATE_PARTITIONING)) .build(); manifestFileWriter.writeManifestFile(); @@ -94,7 +103,11 @@ private void syncCoWTable(HoodieBigQuerySyncClient bqSyncClient) { LOG.info("Manifest table creation complete for " + manifestTableName); } if (!bqSyncClient.tableExists(versionsTableName)) { - bqSyncClient.createVersionsTable(versionsTableName, cfg.sourceUri, cfg.sourceUriPrefix, cfg.partitionFields); + bqSyncClient.createVersionsTable( + versionsTableName, + config.getString(BIGQUERY_SYNC_SOURCE_URI), + config.getString(BIGQUERY_SYNC_SOURCE_URI_PREFIX), + config.getSplitStrings(BIGQUERY_SYNC_PARTITION_FIELDS)); LOG.info("Versions table creation complete for " + versionsTableName); } if (!bqSyncClient.tableExists(snapshotViewName)) { @@ -107,13 +120,13 @@ private void syncCoWTable(HoodieBigQuerySyncClient bqSyncClient) { } public static void main(String[] args) { - BigQuerySyncConfig cfg = new BigQuerySyncConfig(); - JCommander cmd = new JCommander(cfg, null, args); - if (cfg.help || args.length == 0) { + final BigQuerySyncConfig.BigQuerySyncConfigParams params = new BigQuerySyncConfig.BigQuerySyncConfigParams(); + JCommander cmd = JCommander.newBuilder().addObject(params).build(); + cmd.parse(args); + if (params.isHelp()) { cmd.usage(); - System.exit(1); + System.exit(0); } - FileSystem fs = FSUtils.getFs(cfg.basePath, new Configuration()); - new BigQuerySyncTool(cfg.toProps(), fs.getConf(), fs).syncHoodieTable(); + new BigQuerySyncTool(params.toProps()).syncHoodieTable(); } } diff --git a/hudi-gcp/src/main/java/org/apache/hudi/gcp/bigquery/HoodieBigQuerySyncClient.java b/hudi-gcp/src/main/java/org/apache/hudi/gcp/bigquery/HoodieBigQuerySyncClient.java index cb41ca22724c3..0dca74f6d1760 100644 --- a/hudi-gcp/src/main/java/org/apache/hudi/gcp/bigquery/HoodieBigQuerySyncClient.java +++ b/hudi-gcp/src/main/java/org/apache/hudi/gcp/bigquery/HoodieBigQuerySyncClient.java @@ -19,8 +19,7 @@ package org.apache.hudi.gcp.bigquery; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.sync.common.AbstractSyncHoodieClient; +import org.apache.hudi.sync.common.HoodieSyncClient; import com.google.cloud.bigquery.BigQuery; import com.google.cloud.bigquery.BigQueryException; @@ -38,25 +37,31 @@ import com.google.cloud.bigquery.TableId; import com.google.cloud.bigquery.TableInfo; import com.google.cloud.bigquery.ViewDefinition; -import org.apache.hadoop.fs.FileSystem; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; -import org.apache.parquet.schema.MessageType; import java.util.Collections; import java.util.List; import java.util.Map; -public class HoodieBigQuerySyncClient extends AbstractSyncHoodieClient { +import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_DATASET_LOCATION; +import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_DATASET_NAME; +import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_PROJECT_ID; + +public class HoodieBigQuerySyncClient extends HoodieSyncClient { + private static final Logger LOG = LogManager.getLogger(HoodieBigQuerySyncClient.class); - private final BigQuerySyncConfig syncConfig; + protected final BigQuerySyncConfig config; + private final String projectId; + private final String datasetName; private transient BigQuery bigquery; - public HoodieBigQuerySyncClient(final BigQuerySyncConfig syncConfig, final FileSystem fs) { - super(syncConfig.basePath, syncConfig.assumeDatePartitioning, syncConfig.useFileListingFromMetadata, - false, fs); - this.syncConfig = syncConfig; + public HoodieBigQuerySyncClient(final BigQuerySyncConfig config) { + super(config); + this.config = config; + this.projectId = config.getString(BIGQUERY_SYNC_PROJECT_ID); + this.datasetName = config.getString(BIGQUERY_SYNC_DATASET_NAME); this.createBigQueryConnection(); } @@ -65,7 +70,7 @@ private void createBigQueryConnection() { try { // Initialize client that will be used to send requests. This client only needs to be created // once, and can be reused for multiple requests. - bigquery = BigQueryOptions.newBuilder().setLocation(syncConfig.datasetLocation).build().getService(); + bigquery = BigQueryOptions.newBuilder().setLocation(config.getString(BIGQUERY_SYNC_DATASET_LOCATION)).build().getService(); LOG.info("Successfully established BigQuery connection."); } catch (BigQueryException e) { throw new HoodieBigQuerySyncException("Cannot create bigQuery connection ", e); @@ -73,16 +78,9 @@ private void createBigQueryConnection() { } } - @Override - public void createTable(final String tableName, final MessageType storageSchema, final String inputFormatClass, - final String outputFormatClass, final String serdeClass, - final Map serdeProperties, final Map tableProperties) { - // bigQuery create table arguments are different, so do nothing. - } - public void createManifestTable(String tableName, String sourceUri) { try { - TableId tableId = TableId.of(syncConfig.projectId, syncConfig.datasetName, tableName); + TableId tableId = TableId.of(projectId, datasetName, tableName); CsvOptions csvOptions = CsvOptions.newBuilder() .setFieldDelimiter(",") .setAllowJaggedRows(false) @@ -108,7 +106,7 @@ public void createManifestTable(String tableName, String sourceUri) { public void createVersionsTable(String tableName, String sourceUri, String sourceUriPrefix, List partitionFields) { try { ExternalTableDefinition customTable; - TableId tableId = TableId.of(syncConfig.projectId, syncConfig.datasetName, tableName); + TableId tableId = TableId.of(projectId, datasetName, tableName); if (partitionFields.isEmpty()) { customTable = @@ -143,16 +141,16 @@ public void createVersionsTable(String tableName, String sourceUri, String sourc public void createSnapshotView(String viewName, String versionsTableName, String manifestTableName) { try { - TableId tableId = TableId.of(syncConfig.projectId, syncConfig.datasetName, viewName); + TableId tableId = TableId.of(projectId, datasetName, viewName); String query = String.format( "SELECT * FROM `%s.%s.%s` WHERE _hoodie_file_name IN " + "(SELECT filename FROM `%s.%s.%s`)", - syncConfig.projectId, - syncConfig.datasetName, + projectId, + datasetName, versionsTableName, - syncConfig.projectId, - syncConfig.datasetName, + projectId, + datasetName, manifestTableName); ViewDefinition viewDefinition = @@ -166,78 +164,25 @@ public void createSnapshotView(String viewName, String versionsTableName, String } @Override - public Map getTableSchema(String tableName) { + public Map getMetastoreSchema(String tableName) { // TODO: Implement automatic schema evolution when you add a new column. return Collections.emptyMap(); } - @Override - public void addPartitionsToTable(final String tableName, final List partitionsToAdd) { - // bigQuery discovers the new partitions automatically, so do nothing. - throw new UnsupportedOperationException("No support for addPartitionsToTable yet."); - } - public boolean datasetExists() { - Dataset dataset = bigquery.getDataset(DatasetId.of(syncConfig.projectId, syncConfig.datasetName)); + Dataset dataset = bigquery.getDataset(DatasetId.of(projectId, datasetName)); return dataset != null; } - @Override - public boolean doesTableExist(final String tableName) { - return tableExists(tableName); - } - @Override public boolean tableExists(String tableName) { - TableId tableId = TableId.of(syncConfig.projectId, syncConfig.datasetName, tableName); + TableId tableId = TableId.of(projectId, datasetName, tableName); Table table = bigquery.getTable(tableId, BigQuery.TableOption.fields()); return table != null && table.exists(); } - @Override - public Option getLastCommitTimeSynced(final String tableName) { - // bigQuery doesn't support tblproperties, so do nothing. - throw new UnsupportedOperationException("Not support getLastCommitTimeSynced yet."); - } - - @Override - public void updateLastCommitTimeSynced(final String tableName) { - // bigQuery doesn't support tblproperties, so do nothing. - throw new UnsupportedOperationException("No support for updateLastCommitTimeSynced yet."); - } - - @Override - public Option getLastReplicatedTime(String tableName) { - // bigQuery doesn't support tblproperties, so do nothing. - throw new UnsupportedOperationException("Not support getLastReplicatedTime yet."); - } - - @Override - public void updateLastReplicatedTimeStamp(String tableName, String timeStamp) { - // bigQuery doesn't support tblproperties, so do nothing. - throw new UnsupportedOperationException("No support for updateLastReplicatedTimeStamp yet."); - } - - @Override - public void deleteLastReplicatedTimeStamp(String tableName) { - // bigQuery doesn't support tblproperties, so do nothing. - throw new UnsupportedOperationException("No support for deleteLastReplicatedTimeStamp yet."); - } - - @Override - public void updatePartitionsToTable(final String tableName, final List changedPartitions) { - // bigQuery updates the partitions automatically, so do nothing. - throw new UnsupportedOperationException("No support for updatePartitionsToTable yet."); - } - - @Override - public void dropPartitions(String tableName, List partitionsToDrop) { - // bigQuery discovers the new partitions automatically, so do nothing. - throw new UnsupportedOperationException("No support for dropPartitions yet."); - } - @Override public void close() { - // bigQuery has no connection close method, so do nothing. + bigquery = null; } } diff --git a/hudi-gcp/src/test/java/org/apache/hudi/gcp/bigquery/TestBigQuerySyncConfig.java b/hudi-gcp/src/test/java/org/apache/hudi/gcp/bigquery/TestBigQuerySyncConfig.java index 8b3250ccde0f6..82a8527738477 100644 --- a/hudi-gcp/src/test/java/org/apache/hudi/gcp/bigquery/TestBigQuerySyncConfig.java +++ b/hudi-gcp/src/test/java/org/apache/hudi/gcp/bigquery/TestBigQuerySyncConfig.java @@ -19,12 +19,11 @@ package org.apache.hudi.gcp.bigquery; -import org.apache.hudi.common.config.TypedProperties; - import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import java.util.Arrays; +import java.util.Properties; import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_ASSUME_DATE_PARTITIONING; import static org.apache.hudi.gcp.bigquery.BigQuerySyncConfig.BIGQUERY_SYNC_DATASET_LOCATION; @@ -44,75 +43,32 @@ public class TestBigQuerySyncConfig { @BeforeEach void setUp() { - syncConfig = new BigQuerySyncConfig(); - syncConfig.projectId = "fooproject"; - syncConfig.datasetName = "foodataset"; - syncConfig.datasetLocation = "US"; - syncConfig.tableName = "footable"; - syncConfig.sourceUri = "gs://test-bucket/dwh/table_name/dt=*"; - syncConfig.sourceUriPrefix = "gs://test-bucket/dwh/table_name/"; - syncConfig.basePath = "gs://test-bucket/dwh/table_name"; - syncConfig.partitionFields = Arrays.asList("a", "b"); - syncConfig.useFileListingFromMetadata = true; - syncConfig.assumeDatePartitioning = true; - syncConfig.help = true; - } - - @Test - public void testCopy() { - BigQuerySyncConfig copied = BigQuerySyncConfig.copy(syncConfig); - assertEquals(copied.partitionFields, syncConfig.partitionFields); - assertEquals(copied.basePath, syncConfig.basePath); - assertEquals(copied.projectId, syncConfig.projectId); - assertEquals(copied.datasetName, syncConfig.datasetName); - assertEquals(copied.datasetLocation, syncConfig.datasetLocation); - assertEquals(copied.tableName, syncConfig.tableName); - assertEquals(copied.sourceUri, syncConfig.sourceUri); - assertEquals(copied.sourceUriPrefix, syncConfig.sourceUriPrefix); - assertEquals(copied.useFileListingFromMetadata, syncConfig.useFileListingFromMetadata); - assertEquals(copied.assumeDatePartitioning, syncConfig.assumeDatePartitioning); - assertEquals(copied.help, syncConfig.help); + Properties props = new Properties(); + props.setProperty(BIGQUERY_SYNC_PROJECT_ID.key(), "fooproject"); + props.setProperty(BIGQUERY_SYNC_DATASET_NAME.key(), "foodataset"); + props.setProperty(BIGQUERY_SYNC_DATASET_LOCATION.key(), "US"); + props.setProperty(BIGQUERY_SYNC_TABLE_NAME.key(), "footable"); + props.setProperty(BIGQUERY_SYNC_SOURCE_URI.key(), "gs://test-bucket/dwh/table_name/dt=*"); + props.setProperty(BIGQUERY_SYNC_SOURCE_URI_PREFIX.key(), "gs://test-bucket/dwh/table_name/"); + props.setProperty(BIGQUERY_SYNC_SYNC_BASE_PATH.key(), "gs://test-bucket/dwh/table_name"); + props.setProperty(BIGQUERY_SYNC_PARTITION_FIELDS.key(), "a,b"); + props.setProperty(BIGQUERY_SYNC_USE_FILE_LISTING_FROM_METADATA.key(), "true"); + props.setProperty(BIGQUERY_SYNC_ASSUME_DATE_PARTITIONING.key(), "true"); + syncConfig = new BigQuerySyncConfig(props); } @Test - public void testToProps() { - TypedProperties props = syncConfig.toProps(); - assertEquals("fooproject", props.getString(BIGQUERY_SYNC_PROJECT_ID)); - assertEquals("foodataset", props.getString(BIGQUERY_SYNC_DATASET_NAME)); - assertEquals("US", props.getString(BIGQUERY_SYNC_DATASET_LOCATION)); - assertEquals("footable", props.getString(BIGQUERY_SYNC_TABLE_NAME)); - assertEquals("gs://test-bucket/dwh/table_name/dt=*", props.getString(BIGQUERY_SYNC_SOURCE_URI)); - assertEquals("gs://test-bucket/dwh/table_name/", props.getString(BIGQUERY_SYNC_SOURCE_URI_PREFIX)); - assertEquals("gs://test-bucket/dwh/table_name", props.getString(BIGQUERY_SYNC_SYNC_BASE_PATH)); - assertEquals("a,b", props.getString(BIGQUERY_SYNC_PARTITION_FIELDS)); - assertEquals("true", props.getString(BIGQUERY_SYNC_USE_FILE_LISTING_FROM_METADATA)); - assertEquals("true", props.getString(BIGQUERY_SYNC_ASSUME_DATE_PARTITIONING)); + public void testGetConfigs() { + assertEquals("fooproject", syncConfig.getString(BIGQUERY_SYNC_PROJECT_ID)); + assertEquals("foodataset", syncConfig.getString(BIGQUERY_SYNC_DATASET_NAME)); + assertEquals("US", syncConfig.getString(BIGQUERY_SYNC_DATASET_LOCATION)); + assertEquals("footable", syncConfig.getString(BIGQUERY_SYNC_TABLE_NAME)); + assertEquals("gs://test-bucket/dwh/table_name/dt=*", syncConfig.getString(BIGQUERY_SYNC_SOURCE_URI)); + assertEquals("gs://test-bucket/dwh/table_name/", syncConfig.getString(BIGQUERY_SYNC_SOURCE_URI_PREFIX)); + assertEquals("gs://test-bucket/dwh/table_name", syncConfig.getString(BIGQUERY_SYNC_SYNC_BASE_PATH)); + assertEquals(Arrays.asList("a", "b"), syncConfig.getSplitStrings(BIGQUERY_SYNC_PARTITION_FIELDS)); + assertEquals(true, syncConfig.getBoolean(BIGQUERY_SYNC_USE_FILE_LISTING_FROM_METADATA)); + assertEquals(true, syncConfig.getBoolean(BIGQUERY_SYNC_ASSUME_DATE_PARTITIONING)); } - @Test - public void fromProps() { - TypedProperties props = new TypedProperties(); - props.put(BIGQUERY_SYNC_PROJECT_ID, "fooproject"); - props.put(BIGQUERY_SYNC_DATASET_NAME, "foodataset"); - props.put(BIGQUERY_SYNC_DATASET_LOCATION, "US"); - props.put(BIGQUERY_SYNC_TABLE_NAME, "footable"); - props.put(BIGQUERY_SYNC_SOURCE_URI, "gs://test-bucket/dwh/table_name/dt=*"); - props.put(BIGQUERY_SYNC_SOURCE_URI_PREFIX, "gs://test-bucket/dwh/table_name/"); - props.put(BIGQUERY_SYNC_SYNC_BASE_PATH, "gs://test-bucket/dwh/table_name"); - props.put(BIGQUERY_SYNC_PARTITION_FIELDS, "a,b"); - props.put(BIGQUERY_SYNC_USE_FILE_LISTING_FROM_METADATA, true); - props.put(BIGQUERY_SYNC_ASSUME_DATE_PARTITIONING, true); - BigQuerySyncConfig cfg = BigQuerySyncConfig.fromProps(props); - - assertEquals(syncConfig.projectId, cfg.projectId); - assertEquals(syncConfig.datasetName, cfg.datasetName); - assertEquals(syncConfig.datasetLocation, cfg.datasetLocation); - assertEquals(syncConfig.tableName, cfg.tableName); - assertEquals(syncConfig.sourceUri, cfg.sourceUri); - assertEquals(syncConfig.sourceUriPrefix, cfg.sourceUriPrefix); - assertEquals(syncConfig.basePath, cfg.basePath); - assertEquals(syncConfig.partitionFields, cfg.partitionFields); - assertEquals(syncConfig.useFileListingFromMetadata, cfg.useFileListingFromMetadata); - assertEquals(syncConfig.assumeDatePartitioning, cfg.assumeDatePartitioning); - } } diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/HiveQueryNode.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/HiveQueryNode.java index 4736133f2c20c..2da5c558d2cb2 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/HiveQueryNode.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/HiveQueryNode.java @@ -32,6 +32,10 @@ import java.sql.SQLException; import java.sql.Statement; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_PASS; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_URL; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USER; + /** * A hive query node in the DAG of operations for a workflow. used to perform a hive query with given config. */ @@ -57,8 +61,8 @@ public void execute(ExecutionContext executionContext, int curItrCount) throws E .getDeltaSyncService().getDeltaSync().getCfg().baseFileFormat); HiveSyncConfig hiveSyncConfig = new HiveSyncConfig(properties); this.hiveServiceProvider.syncToLocalHiveIfNeeded(executionContext.getHoodieTestSuiteWriter()); - Connection con = DriverManager.getConnection(hiveSyncConfig.jdbcUrl, hiveSyncConfig.hiveUser, - hiveSyncConfig.hivePass); + Connection con = DriverManager.getConnection(hiveSyncConfig.getString(HIVE_URL), + hiveSyncConfig.getString(HIVE_USER), hiveSyncConfig.getString(HIVE_PASS)); Statement stmt = con.createStatement(); stmt.execute("set hive.input.format=org.apache.hadoop.hive.ql.io.HiveInputFormat"); for (String hiveProperty : this.config.getHiveProperties()) { diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/HiveServiceProvider.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/HiveServiceProvider.java index a10e4d31fa736..9dbfcf3860dde 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/HiveServiceProvider.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/HiveServiceProvider.java @@ -18,16 +18,16 @@ package org.apache.hudi.integ.testsuite.helpers; -import java.io.IOException; -import org.apache.hadoop.conf.Configuration; -import org.apache.hive.service.server.HiveServer2; - -import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.hive.HiveSyncTool; import org.apache.hudi.hive.testutils.HiveTestService; import org.apache.hudi.integ.testsuite.HoodieTestSuiteWriter; import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config; +import org.apache.hadoop.conf.Configuration; +import org.apache.hive.service.server.HiveServer2; + +import java.io.IOException; + /** * Hive Service provider. */ @@ -52,12 +52,10 @@ public void syncToLocalHiveIfNeeded(HoodieTestSuiteWriter writer) { HiveSyncTool hiveSyncTool; if (this.config.isHiveLocal()) { hiveSyncTool = new HiveSyncTool(writer.getWriteConfig().getProps(), - getLocalHiveServer().getHiveConf(), - FSUtils.getFs(writer.getWriteConfig().getBasePath(), getLocalHiveServer().getHiveConf())); + getLocalHiveServer().getHiveConf()); } else { hiveSyncTool = new HiveSyncTool(writer.getWriteConfig().getProps(), - getLocalHiveServer().getHiveConf(), - FSUtils.getFs(writer.getWriteConfig().getBasePath(), writer.getConfiguration())); + writer.getConfiguration()); } hiveSyncTool.syncHoodieTable(); } diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java index fd7e8ff472f00..485c43d4ebfe5 100644 --- a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java +++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java @@ -22,7 +22,6 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.hive.HiveSyncConfig; import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.integ.testsuite.HoodieTestSuiteJob; import org.apache.hudi.integ.testsuite.HoodieTestSuiteJob.HoodieTestSuiteConfig; @@ -55,6 +54,10 @@ import java.util.UUID; import java.util.stream.Stream; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_URL; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_PARTITION_FIELDS; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_TABLE_NAME; import static org.junit.jupiter.api.Assertions.assertEquals; /** @@ -177,10 +180,10 @@ private static TypedProperties getProperties() { // Make path selection test suite specific props.setProperty("hoodie.deltastreamer.source.input.selector", DFSTestSuitePathSelector.class.getName()); // Hive Configs - props.setProperty(HiveSyncConfig.HIVE_URL.key(), "jdbc:hive2://127.0.0.1:9999/"); - props.setProperty(HiveSyncConfig.META_SYNC_DATABASE_NAME.key(), "testdb1"); - props.setProperty(HiveSyncConfig.META_SYNC_TABLE_NAME.key(), "table1"); - props.setProperty(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "datestr"); + props.setProperty(HIVE_URL.key(), "jdbc:hive2://127.0.0.1:9999/"); + props.setProperty(META_SYNC_DATABASE_NAME.key(), "testdb1"); + props.setProperty(META_SYNC_TABLE_NAME.key(), "table1"); + props.setProperty(META_SYNC_PARTITION_FIELDS.key(), "datestr"); props.setProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(), TimestampBasedKeyGenerator.class.getName()); props.setProperty("hoodie.write.lock.provider", "org.apache.hudi.client.transaction.lock.ZookeeperBasedLockProvider"); diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java index c5b18576999d4..ca1474fa88abc 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java @@ -18,9 +18,6 @@ package org.apache.hudi; -import org.apache.avro.generic.GenericRecord; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.hudi.client.HoodieReadClient; import org.apache.hudi.client.HoodieWriteResult; import org.apache.hudi.client.SparkRDDWriteClient; @@ -36,30 +33,27 @@ import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.TablePathUtils; import org.apache.hudi.config.HoodieCompactionConfig; -import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodiePayloadConfig; import org.apache.hudi.config.HoodieStorageConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieNotSupportedException; import org.apache.hudi.exception.TableNotFoundException; -import org.apache.hudi.hive.HiveSyncConfig; -import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor; -import org.apache.hudi.sync.common.HoodieSyncConfig; import org.apache.hudi.table.BulkInsertPartitioner; import org.apache.hudi.util.DataTypeUtils; + +import org.apache.avro.generic.GenericRecord; +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.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.apache.spark.sql.hive.HiveExternalCatalog; import org.apache.spark.sql.types.StructType; import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -270,63 +264,6 @@ public static JavaRDD dropDuplicates(JavaSparkContext jssc, JavaRD return dropDuplicates(jssc, incomingHoodieRecords, writeConfig); } - /** - * @deprecated Use {@link HiveSyncConfig} constructor directly and provide the props, - * and set {@link HoodieSyncConfig#META_SYNC_BASE_PATH} and {@link HoodieSyncConfig#META_SYNC_BASE_FILE_FORMAT} instead. - */ - @Deprecated - public static HiveSyncConfig buildHiveSyncConfig(TypedProperties props, String basePath, String baseFileFormat) { - checkRequiredProperties(props, Collections.singletonList(DataSourceWriteOptions.HIVE_TABLE().key())); - HiveSyncConfig hiveSyncConfig = new HiveSyncConfig(); - hiveSyncConfig.basePath = basePath; - hiveSyncConfig.usePreApacheInputFormat = - props.getBoolean(DataSourceWriteOptions.HIVE_USE_PRE_APACHE_INPUT_FORMAT().key(), - Boolean.parseBoolean(DataSourceWriteOptions.HIVE_USE_PRE_APACHE_INPUT_FORMAT().defaultValue())); - hiveSyncConfig.databaseName = props.getString(DataSourceWriteOptions.HIVE_DATABASE().key(), - DataSourceWriteOptions.HIVE_DATABASE().defaultValue()); - hiveSyncConfig.tableName = props.getString(DataSourceWriteOptions.HIVE_TABLE().key()); - hiveSyncConfig.baseFileFormat = baseFileFormat; - hiveSyncConfig.hiveUser = - props.getString(DataSourceWriteOptions.HIVE_USER().key(), DataSourceWriteOptions.HIVE_USER().defaultValue()); - hiveSyncConfig.hivePass = - props.getString(DataSourceWriteOptions.HIVE_PASS().key(), DataSourceWriteOptions.HIVE_PASS().defaultValue()); - hiveSyncConfig.jdbcUrl = - props.getString(DataSourceWriteOptions.HIVE_URL().key(), DataSourceWriteOptions.HIVE_URL().defaultValue()); - hiveSyncConfig.metastoreUris = - props.getString(DataSourceWriteOptions.METASTORE_URIS().key(), DataSourceWriteOptions.METASTORE_URIS().defaultValue()); - hiveSyncConfig.partitionFields = - props.getStringList(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), ",", new ArrayList<>()); - hiveSyncConfig.partitionValueExtractorClass = - props.getString(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(), - SlashEncodedDayPartitionValueExtractor.class.getName()); - hiveSyncConfig.useJdbc = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_USE_JDBC().key(), - DataSourceWriteOptions.HIVE_USE_JDBC().defaultValue())); - if (props.containsKey(DataSourceWriteOptions.HIVE_SYNC_MODE().key())) { - hiveSyncConfig.syncMode = props.getString(DataSourceWriteOptions.HIVE_SYNC_MODE().key()); - } - hiveSyncConfig.autoCreateDatabase = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_AUTO_CREATE_DATABASE().key(), - DataSourceWriteOptions.HIVE_AUTO_CREATE_DATABASE().defaultValue())); - hiveSyncConfig.ignoreExceptions = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_IGNORE_EXCEPTIONS().key(), - DataSourceWriteOptions.HIVE_IGNORE_EXCEPTIONS().defaultValue())); - hiveSyncConfig.skipROSuffix = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE().key(), - DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE().defaultValue())); - hiveSyncConfig.supportTimestamp = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_SUPPORT_TIMESTAMP_TYPE().key(), - DataSourceWriteOptions.HIVE_SUPPORT_TIMESTAMP_TYPE().defaultValue())); - hiveSyncConfig.isConditionalSync = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_CONDITIONAL_SYNC().key(), - DataSourceWriteOptions.HIVE_CONDITIONAL_SYNC().defaultValue())); - hiveSyncConfig.bucketSpec = props.getBoolean(DataSourceWriteOptions.HIVE_SYNC_BUCKET_SYNC().key(), - DataSourceWriteOptions.HIVE_SYNC_BUCKET_SYNC().defaultValue()) - ? HiveSyncConfig.getBucketSpec(props.getString(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key()), - props.getInteger(HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS.key())) : null; - if (props.containsKey(HiveExternalCatalog.CREATED_SPARK_VERSION())) { - hiveSyncConfig.sparkVersion = props.getString(HiveExternalCatalog.CREATED_SPARK_VERSION()); - } - hiveSyncConfig.syncComment = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_SYNC_COMMENT().key(), - DataSourceWriteOptions.HIVE_SYNC_COMMENT().defaultValue())); - return hiveSyncConfig; - } - - /** * Checks whether default value (false) of "hoodie.parquet.writelegacyformat.enabled" should be * overridden in case: diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala index 819c4b55a9e4f..8ff82746b0ceb 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala @@ -26,7 +26,7 @@ import org.apache.hudi.common.table.HoodieTableConfig import org.apache.hudi.common.util.Option import org.apache.hudi.common.util.ValidationUtils.checkState import org.apache.hudi.config.{HoodieClusteringConfig, HoodieWriteConfig} -import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool} +import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncConfigHolder, HiveSyncTool} import org.apache.hudi.keygen.constant.KeyGeneratorOptions import org.apache.hudi.keygen.{ComplexKeyGenerator, CustomKeyGenerator, NonpartitionedKeyGenerator, SimpleKeyGenerator} import org.apache.hudi.sync.common.HoodieSyncConfig @@ -413,7 +413,7 @@ object DataSourceWriteOptions { * @deprecated Hive Specific Configs are moved to {@link HiveSyncConfig} */ @Deprecated - val HIVE_SYNC_ENABLED: ConfigProperty[String] = HiveSyncConfig.HIVE_SYNC_ENABLED + val HIVE_SYNC_ENABLED: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_SYNC_ENABLED @Deprecated val META_SYNC_ENABLED: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_ENABLED @Deprecated @@ -425,13 +425,13 @@ object DataSourceWriteOptions { @Deprecated val HIVE_BASE_FILE_FORMAT: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT @Deprecated - val HIVE_USER: ConfigProperty[String] = HiveSyncConfig.HIVE_USER + val HIVE_USER: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_USER @Deprecated - val HIVE_PASS: ConfigProperty[String] = HiveSyncConfig.HIVE_PASS + val HIVE_PASS: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_PASS @Deprecated - val HIVE_URL: ConfigProperty[String] = HiveSyncConfig.HIVE_URL + val HIVE_URL: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_URL @Deprecated - val METASTORE_URIS: ConfigProperty[String] = HiveSyncConfig.METASTORE_URIS + val METASTORE_URIS: ConfigProperty[String] = HiveSyncConfigHolder.METASTORE_URIS @Deprecated val hivePartitionFieldsInferFunc: JavaFunction[HoodieConfig, Option[String]] = HoodieSyncConfig.PARTITION_FIELDS_INFERENCE_FUNCTION @Deprecated @@ -443,19 +443,19 @@ object DataSourceWriteOptions { @Deprecated val HIVE_ASSUME_DATE_PARTITION: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_ASSUME_DATE_PARTITION @Deprecated - val HIVE_USE_PRE_APACHE_INPUT_FORMAT: ConfigProperty[String] = HiveSyncConfig.HIVE_USE_PRE_APACHE_INPUT_FORMAT + val HIVE_USE_PRE_APACHE_INPUT_FORMAT: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_USE_PRE_APACHE_INPUT_FORMAT /** @deprecated Use {@link HIVE_SYNC_MODE} instead of this config from 0.9.0 */ @Deprecated - val HIVE_USE_JDBC: ConfigProperty[String] = HiveSyncConfig.HIVE_USE_JDBC + val HIVE_USE_JDBC: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_USE_JDBC @Deprecated - val HIVE_AUTO_CREATE_DATABASE: ConfigProperty[String] = HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE + val HIVE_AUTO_CREATE_DATABASE: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_AUTO_CREATE_DATABASE @Deprecated - val HIVE_IGNORE_EXCEPTIONS: ConfigProperty[String] = HiveSyncConfig.HIVE_IGNORE_EXCEPTIONS + val HIVE_IGNORE_EXCEPTIONS: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_IGNORE_EXCEPTIONS @Deprecated - val HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE: ConfigProperty[String] = HiveSyncConfig.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE + val HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE @Deprecated - val HIVE_SUPPORT_TIMESTAMP_TYPE: ConfigProperty[String] = HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE + val HIVE_SUPPORT_TIMESTAMP_TYPE: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE /** * Flag to indicate whether to use conditional syncing in HiveSync. @@ -465,23 +465,23 @@ object DataSourceWriteOptions { @Deprecated val HIVE_CONDITIONAL_SYNC: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_CONDITIONAL_SYNC @Deprecated - val HIVE_TABLE_PROPERTIES: ConfigProperty[String] = HiveSyncConfig.HIVE_TABLE_PROPERTIES + val HIVE_TABLE_PROPERTIES: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_TABLE_PROPERTIES @Deprecated - val HIVE_TABLE_SERDE_PROPERTIES: ConfigProperty[String] = HiveSyncConfig.HIVE_TABLE_SERDE_PROPERTIES + val HIVE_TABLE_SERDE_PROPERTIES: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_TABLE_SERDE_PROPERTIES @Deprecated - val HIVE_SYNC_AS_DATA_SOURCE_TABLE: ConfigProperty[String] = HiveSyncConfig.HIVE_SYNC_AS_DATA_SOURCE_TABLE + val HIVE_SYNC_AS_DATA_SOURCE_TABLE: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_SYNC_AS_DATA_SOURCE_TABLE // Create table as managed table @Deprecated - val HIVE_CREATE_MANAGED_TABLE: ConfigProperty[java.lang.Boolean] = HiveSyncConfig.HIVE_CREATE_MANAGED_TABLE + val HIVE_CREATE_MANAGED_TABLE: ConfigProperty[java.lang.Boolean] = HiveSyncConfigHolder.HIVE_CREATE_MANAGED_TABLE @Deprecated - val HIVE_BATCH_SYNC_PARTITION_NUM: ConfigProperty[java.lang.Integer] = HiveSyncConfig.HIVE_BATCH_SYNC_PARTITION_NUM + val HIVE_BATCH_SYNC_PARTITION_NUM: ConfigProperty[java.lang.Integer] = HiveSyncConfigHolder.HIVE_BATCH_SYNC_PARTITION_NUM @Deprecated - val HIVE_SYNC_MODE: ConfigProperty[String] = HiveSyncConfig.HIVE_SYNC_MODE + val HIVE_SYNC_MODE: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_SYNC_MODE @Deprecated - val HIVE_SYNC_BUCKET_SYNC: ConfigProperty[java.lang.Boolean] = HiveSyncConfig.HIVE_SYNC_BUCKET_SYNC + val HIVE_SYNC_BUCKET_SYNC: ConfigProperty[java.lang.Boolean] = HiveSyncConfigHolder.HIVE_SYNC_BUCKET_SYNC @Deprecated - val HIVE_SYNC_COMMENT: ConfigProperty[String] = HiveSyncConfig.HIVE_SYNC_COMMENT; + val HIVE_SYNC_COMMENT: ConfigProperty[String] = HiveSyncConfigHolder.HIVE_SYNC_COMMENT; // Async Compaction - Enabled by default for MOR val ASYNC_COMPACT_ENABLE: ConfigProperty[String] = ConfigProperty @@ -506,16 +506,16 @@ object DataSourceWriteOptions { val HIVE_ASSUME_DATE_PARTITION_OPT_KEY = HoodieSyncConfig.META_SYNC_ASSUME_DATE_PARTITION.key() /** @deprecated Use {@link HIVE_USE_PRE_APACHE_INPUT_FORMAT} and its methods instead */ @Deprecated - val HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY = HiveSyncConfig.HIVE_USE_PRE_APACHE_INPUT_FORMAT.key() + val HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY = HiveSyncConfigHolder.HIVE_USE_PRE_APACHE_INPUT_FORMAT.key() /** @deprecated Use {@link HIVE_USE_JDBC} and its methods instead */ @Deprecated - val HIVE_USE_JDBC_OPT_KEY = HiveSyncConfig.HIVE_USE_JDBC.key() + val HIVE_USE_JDBC_OPT_KEY = HiveSyncConfigHolder.HIVE_USE_JDBC.key() /** @deprecated Use {@link HIVE_AUTO_CREATE_DATABASE} and its methods instead */ @Deprecated - val HIVE_AUTO_CREATE_DATABASE_OPT_KEY = HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.key() + val HIVE_AUTO_CREATE_DATABASE_OPT_KEY = HiveSyncConfigHolder.HIVE_AUTO_CREATE_DATABASE.key() /** @deprecated Use {@link HIVE_IGNORE_EXCEPTIONS} and its methods instead */ @Deprecated - val HIVE_IGNORE_EXCEPTIONS_OPT_KEY = HiveSyncConfig.HIVE_IGNORE_EXCEPTIONS.key() + val HIVE_IGNORE_EXCEPTIONS_OPT_KEY = HiveSyncConfigHolder.HIVE_IGNORE_EXCEPTIONS.key() /** @deprecated Use {@link STREAMING_IGNORE_FAILED_BATCH} and its methods instead */ @Deprecated val STREAMING_IGNORE_FAILED_BATCH_OPT_KEY = STREAMING_IGNORE_FAILED_BATCH.key() @@ -530,7 +530,7 @@ object DataSourceWriteOptions { val DEFAULT_META_SYNC_CLIENT_TOOL_CLASS = META_SYNC_CLIENT_TOOL_CLASS_NAME.defaultValue() /** @deprecated Use {@link HIVE_SYNC_ENABLED} and its methods instead */ @Deprecated - val HIVE_SYNC_ENABLED_OPT_KEY = HiveSyncConfig.HIVE_SYNC_ENABLED.key() + val HIVE_SYNC_ENABLED_OPT_KEY = HiveSyncConfigHolder.HIVE_SYNC_ENABLED.key() /** @deprecated Use {@link META_SYNC_ENABLED} and its methods instead */ @Deprecated val META_SYNC_ENABLED_OPT_KEY = HoodieSyncConfig.META_SYNC_ENABLED.key() @@ -545,13 +545,13 @@ object DataSourceWriteOptions { val HIVE_BASE_FILE_FORMAT_OPT_KEY = HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT.key() /** @deprecated Use {@link HIVE_USER} and its methods instead */ @Deprecated - val HIVE_USER_OPT_KEY = HiveSyncConfig.HIVE_USER.key() + val HIVE_USER_OPT_KEY = HiveSyncConfigHolder.HIVE_USER.key() /** @deprecated Use {@link HIVE_PASS} and its methods instead */ @Deprecated - val HIVE_PASS_OPT_KEY = HiveSyncConfig.HIVE_PASS.key() + val HIVE_PASS_OPT_KEY = HiveSyncConfigHolder.HIVE_PASS.key() /** @deprecated Use {@link HIVE_URL} and its methods instead */ @Deprecated - val HIVE_URL_OPT_KEY = HiveSyncConfig.HIVE_URL.key() + val HIVE_URL_OPT_KEY = HiveSyncConfigHolder.HIVE_URL.key() /** @deprecated Use {@link HIVE_PARTITION_FIELDS} and its methods instead */ @Deprecated val HIVE_PARTITION_FIELDS_OPT_KEY = HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key() @@ -667,7 +667,7 @@ object DataSourceWriteOptions { /** @deprecated Use {@link HIVE_SYNC_ENABLED} and its methods instead */ @Deprecated - val DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL = HiveSyncConfig.HIVE_SYNC_ENABLED.defaultValue() + val DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL = HiveSyncConfigHolder.HIVE_SYNC_ENABLED.defaultValue() /** @deprecated Use {@link META_SYNC_ENABLED} and its methods instead */ @Deprecated val DEFAULT_META_SYNC_ENABLED_OPT_VAL = HoodieSyncConfig.META_SYNC_ENABLED.defaultValue() @@ -682,13 +682,13 @@ object DataSourceWriteOptions { val DEFAULT_HIVE_BASE_FILE_FORMAT_OPT_VAL = HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT.defaultValue() /** @deprecated Use {@link HIVE_USER} and its methods instead */ @Deprecated - val DEFAULT_HIVE_USER_OPT_VAL = HiveSyncConfig.HIVE_USER.defaultValue() + val DEFAULT_HIVE_USER_OPT_VAL = HiveSyncConfigHolder.HIVE_USER.defaultValue() /** @deprecated Use {@link HIVE_PASS} and its methods instead */ @Deprecated - val DEFAULT_HIVE_PASS_OPT_VAL = HiveSyncConfig.HIVE_PASS.defaultValue() + val DEFAULT_HIVE_PASS_OPT_VAL = HiveSyncConfigHolder.HIVE_PASS.defaultValue() /** @deprecated Use {@link HIVE_URL} and its methods instead */ @Deprecated - val DEFAULT_HIVE_URL_OPT_VAL = HiveSyncConfig.HIVE_URL.defaultValue() + val DEFAULT_HIVE_URL_OPT_VAL = HiveSyncConfigHolder.HIVE_URL.defaultValue() /** @deprecated Use {@link HIVE_PARTITION_FIELDS} and its methods instead */ @Deprecated val DEFAULT_HIVE_PARTITION_FIELDS_OPT_VAL = HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.defaultValue() @@ -702,25 +702,25 @@ object DataSourceWriteOptions { val DEFAULT_USE_PRE_APACHE_INPUT_FORMAT_OPT_VAL = "false" /** @deprecated Use {@link HIVE_USE_JDBC} and its methods instead */ @Deprecated - val DEFAULT_HIVE_USE_JDBC_OPT_VAL = HiveSyncConfig.HIVE_USE_JDBC.defaultValue() + val DEFAULT_HIVE_USE_JDBC_OPT_VAL = HiveSyncConfigHolder.HIVE_USE_JDBC.defaultValue() /** @deprecated Use {@link HIVE_AUTO_CREATE_DATABASE} and its methods instead */ @Deprecated - val DEFAULT_HIVE_AUTO_CREATE_DATABASE_OPT_KEY = HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.defaultValue() + val DEFAULT_HIVE_AUTO_CREATE_DATABASE_OPT_KEY = HiveSyncConfigHolder.HIVE_AUTO_CREATE_DATABASE.defaultValue() /** @deprecated Use {@link HIVE_IGNORE_EXCEPTIONS} and its methods instead */ @Deprecated - val DEFAULT_HIVE_IGNORE_EXCEPTIONS_OPT_KEY = HiveSyncConfig.HIVE_IGNORE_EXCEPTIONS.defaultValue() + val DEFAULT_HIVE_IGNORE_EXCEPTIONS_OPT_KEY = HiveSyncConfigHolder.HIVE_IGNORE_EXCEPTIONS.defaultValue() /** @deprecated Use {@link HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE} and its methods instead */ @Deprecated - val HIVE_SKIP_RO_SUFFIX = HiveSyncConfig.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.key() + val HIVE_SKIP_RO_SUFFIX = HiveSyncConfigHolder.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.key() /** @deprecated Use {@link HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE} and its methods instead */ @Deprecated - val DEFAULT_HIVE_SKIP_RO_SUFFIX_VAL = HiveSyncConfig.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.defaultValue() + val DEFAULT_HIVE_SKIP_RO_SUFFIX_VAL = HiveSyncConfigHolder.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.defaultValue() /** @deprecated Use {@link HIVE_SUPPORT_TIMESTAMP_TYPE} and its methods instead */ @Deprecated - val HIVE_SUPPORT_TIMESTAMP = HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.key() + val HIVE_SUPPORT_TIMESTAMP = HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE.key() /** @deprecated Use {@link HIVE_SUPPORT_TIMESTAMP_TYPE} and its methods instead */ @Deprecated - val DEFAULT_HIVE_SUPPORT_TIMESTAMP = HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.defaultValue() + val DEFAULT_HIVE_SUPPORT_TIMESTAMP = HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE.defaultValue() /** @deprecated Use {@link ASYNC_COMPACT_ENABLE} and its methods instead */ @Deprecated val ASYNC_COMPACT_ENABLE_OPT_KEY = ASYNC_COMPACT_ENABLE.key() diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index 005d21d8624f6..510a45899bce5 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -36,7 +36,7 @@ import org.apache.hudi.config.HoodieBootstrapConfig.{BASE_PATH, INDEX_CLASS_NAME import org.apache.hudi.config.{HoodieInternalConfig, HoodieWriteConfig} import org.apache.hudi.exception.HoodieException import org.apache.hudi.execution.bulkinsert.{BulkInsertInternalPartitionerWithRowsFactory, NonSortPartitionerWithRows} -import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool} +import org.apache.hudi.hive.{HiveSyncConfigHolder, HiveSyncTool} import org.apache.hudi.index.SparkHoodieIndexFactory import org.apache.hudi.internal.DataSourceInternalWriterHelper import org.apache.hudi.internal.schema.InternalSchema @@ -600,7 +600,7 @@ object HoodieSparkSqlWriter { private def metaSync(spark: SparkSession, hoodieConfig: HoodieConfig, basePath: Path, schema: StructType): Boolean = { - val hiveSyncEnabled = hoodieConfig.getStringOrDefault(HiveSyncConfig.HIVE_SYNC_ENABLED).toBoolean + val hiveSyncEnabled = hoodieConfig.getStringOrDefault(HiveSyncConfigHolder.HIVE_SYNC_ENABLED).toBoolean var metaSyncEnabled = hoodieConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_ENABLED).toBoolean var syncClientToolClassSet = scala.collection.mutable.Set[String]() hoodieConfig.getString(META_SYNC_CLIENT_TOOL_CLASS_NAME).split(",").foreach(syncClass => syncClientToolClassSet += syncClass) @@ -616,7 +616,7 @@ object HoodieSparkSqlWriter { val baseFileFormat = hoodieConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT); val properties = new TypedProperties() properties.putAll(hoodieConfig.getProps) - properties.put(HiveSyncConfig.HIVE_SYNC_SCHEMA_STRING_LENGTH_THRESHOLD.key, spark.sessionState.conf.getConf(StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD).toString) + properties.put(HiveSyncConfigHolder.HIVE_SYNC_SCHEMA_STRING_LENGTH_THRESHOLD.key, spark.sessionState.conf.getConf(StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD).toString) properties.put(HoodieSyncConfig.META_SYNC_SPARK_VERSION.key, SPARK_VERSION) properties.put(HoodieSyncConfig.META_SYNC_USE_FILE_LISTING_FROM_METADATA.key, hoodieConfig.getBoolean(HoodieMetadataConfig.ENABLE)) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala index 49672126752f0..f9d8a60004a8b 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala @@ -17,20 +17,19 @@ package org.apache.hudi -import java.util.Properties import org.apache.hudi.DataSourceOptionsHelper.allAlternatives import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.common.config.HoodieMetadataConfig.ENABLE -import org.apache.hudi.common.config.{DFSPropertiesConfiguration, HoodieCommonConfig, HoodieConfig, TypedProperties} +import org.apache.hudi.common.config.{DFSPropertiesConfiguration, HoodieCommonConfig, HoodieConfig} import org.apache.hudi.common.table.HoodieTableConfig -import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.exception.HoodieException -import org.apache.hudi.hive.HiveSyncConfig +import org.apache.hudi.hive.HiveSyncConfigHolder import org.apache.hudi.keygen.{NonpartitionedKeyGenerator, SimpleKeyGenerator} import org.apache.hudi.sync.common.HoodieSyncConfig import org.apache.spark.sql.SparkSession import org.apache.spark.sql.hudi.command.SqlKeyGenerator +import java.util.Properties import scala.collection.JavaConversions.mapAsJavaMap import scala.collection.JavaConverters._ @@ -67,21 +66,21 @@ object HoodieWriterUtils { hoodieConfig.setDefaultValue(STREAMING_RETRY_INTERVAL_MS) hoodieConfig.setDefaultValue(STREAMING_IGNORE_FAILED_BATCH) hoodieConfig.setDefaultValue(META_SYNC_CLIENT_TOOL_CLASS_NAME) - hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_SYNC_ENABLED) + hoodieConfig.setDefaultValue(HiveSyncConfigHolder.HIVE_SYNC_ENABLED) hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_ENABLED) hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_DATABASE_NAME) hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_TABLE_NAME) hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT) - hoodieConfig.setDefaultValue(HiveSyncConfig.METASTORE_URIS) - hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_USER) - hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_PASS) - hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_URL) + hoodieConfig.setDefaultValue(HiveSyncConfigHolder.METASTORE_URIS) + hoodieConfig.setDefaultValue(HiveSyncConfigHolder.HIVE_USER) + hoodieConfig.setDefaultValue(HiveSyncConfigHolder.HIVE_PASS) + hoodieConfig.setDefaultValue(HiveSyncConfigHolder.HIVE_URL) hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_PARTITION_FIELDS) hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS) hoodieConfig.setDefaultValue(HIVE_STYLE_PARTITIONING) - hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_USE_JDBC) - hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_CREATE_MANAGED_TABLE) - hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_SYNC_AS_DATA_SOURCE_TABLE) + hoodieConfig.setDefaultValue(HiveSyncConfigHolder.HIVE_USE_JDBC) + hoodieConfig.setDefaultValue(HiveSyncConfigHolder.HIVE_CREATE_MANAGED_TABLE) + hoodieConfig.setDefaultValue(HiveSyncConfigHolder.HIVE_SYNC_AS_DATA_SOURCE_TABLE) hoodieConfig.setDefaultValue(ASYNC_COMPACT_ENABLE) hoodieConfig.setDefaultValue(INLINE_CLUSTERING_ENABLE) hoodieConfig.setDefaultValue(ASYNC_CLUSTERING_ENABLE) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala index 3f67d5017fc9c..cfb357ee90b1c 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/ProvidesHoodieConfig.scala @@ -24,7 +24,7 @@ import org.apache.hudi.common.table.HoodieTableConfig import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME import org.apache.hudi.config.{HoodieIndexConfig, HoodieWriteConfig} import org.apache.hudi.hive.ddl.HiveSyncMode -import org.apache.hudi.hive.{HiveSyncConfig, MultiPartKeysValueExtractor} +import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncConfigHolder, MultiPartKeysValueExtractor} import org.apache.hudi.keygen.ComplexKeyGenerator import org.apache.hudi.sql.InsertMode import org.apache.hudi.sync.common.HoodieSyncConfig @@ -38,9 +38,7 @@ import org.apache.spark.sql.hudi.command.{SqlKeyGenerator, ValidateDuplicateKeyP import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType -import java.util import java.util.Locale - import scala.collection.JavaConverters._ trait ProvidesHoodieConfig extends Logging { @@ -76,13 +74,13 @@ trait ProvidesHoodieConfig extends Logging { OPERATION.key -> UPSERT_OPERATION_OPT_VAL, PARTITIONPATH_FIELD.key -> tableConfig.getPartitionFieldProp, HoodieSyncConfig.META_SYNC_ENABLED.key -> enableHive.toString, - HiveSyncConfig.HIVE_SYNC_ENABLED.key -> enableHive.toString, - HiveSyncConfig.HIVE_SYNC_MODE.key -> hiveSyncConfig.syncMode, - HoodieSyncConfig.META_SYNC_DATABASE_NAME.key -> hiveSyncConfig.databaseName, - HoodieSyncConfig.META_SYNC_TABLE_NAME.key -> hiveSyncConfig.tableName, + HiveSyncConfigHolder.HIVE_SYNC_ENABLED.key -> enableHive.toString, + HiveSyncConfigHolder.HIVE_SYNC_MODE.key -> hiveSyncConfig.getStringOrDefault(HiveSyncConfigHolder.HIVE_SYNC_MODE, HiveSyncMode.HMS.name()), + HoodieSyncConfig.META_SYNC_DATABASE_NAME.key -> hiveSyncConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_DATABASE_NAME), + HoodieSyncConfig.META_SYNC_TABLE_NAME.key -> hiveSyncConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_TABLE_NAME), HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key -> tableConfig.getPartitionFieldProp, - HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key -> hiveSyncConfig.partitionValueExtractorClass, - HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> hiveSyncConfig.supportTimestamp.toString, + HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key -> hiveSyncConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS), + HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> hiveSyncConfig.getBoolean(HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE).toString, HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> hoodieProps.getString(HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key, "200"), SqlKeyGenerator.PARTITION_SCHEMA -> hoodieCatalogTable.partitionSchema.toDDL ) @@ -194,12 +192,12 @@ trait ProvidesHoodieConfig extends Logging { HoodieWriteConfig.COMBINE_BEFORE_INSERT.key -> String.valueOf(hasPrecombineColumn), HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key -> partitionFieldsStr, HoodieSyncConfig.META_SYNC_ENABLED.key -> enableHive.toString, - HiveSyncConfig.HIVE_SYNC_ENABLED.key -> enableHive.toString, - HiveSyncConfig.HIVE_SYNC_MODE.key -> hiveSyncConfig.syncMode, - HoodieSyncConfig.META_SYNC_DATABASE_NAME.key -> hiveSyncConfig.databaseName, - HoodieSyncConfig.META_SYNC_TABLE_NAME.key -> hiveSyncConfig.tableName, - HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> hiveSyncConfig.supportTimestamp.toString, - HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key -> hiveSyncConfig.partitionValueExtractorClass, + HiveSyncConfigHolder.HIVE_SYNC_ENABLED.key -> enableHive.toString, + HiveSyncConfigHolder.HIVE_SYNC_MODE.key -> hiveSyncConfig.getStringOrDefault(HiveSyncConfigHolder.HIVE_SYNC_MODE, HiveSyncMode.HMS.name()), + HoodieSyncConfig.META_SYNC_DATABASE_NAME.key -> hiveSyncConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_DATABASE_NAME), + HoodieSyncConfig.META_SYNC_TABLE_NAME.key -> hiveSyncConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_TABLE_NAME), + HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> hiveSyncConfig.getBoolean(HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE).toString, + HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key -> hiveSyncConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS), HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key -> hoodieProps.getString(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "200"), HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> hoodieProps.getString(HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key, "200"), SqlKeyGenerator.PARTITION_SCHEMA -> hoodieCatalogTable.partitionSchema.toDDL @@ -231,13 +229,13 @@ trait ProvidesHoodieConfig extends Logging { PRECOMBINE_FIELD.key -> hoodieCatalogTable.preCombineKey.getOrElse(""), PARTITIONPATH_FIELD.key -> partitionFields, HoodieSyncConfig.META_SYNC_ENABLED.key -> enableHive.toString, - HiveSyncConfig.HIVE_SYNC_ENABLED.key -> enableHive.toString, - HiveSyncConfig.HIVE_SYNC_MODE.key -> hiveSyncConfig.syncMode, - HoodieSyncConfig.META_SYNC_DATABASE_NAME.key -> hiveSyncConfig.databaseName, - HoodieSyncConfig.META_SYNC_TABLE_NAME.key -> hiveSyncConfig.tableName, - HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> hiveSyncConfig.supportTimestamp.toString, + HiveSyncConfigHolder.HIVE_SYNC_ENABLED.key -> enableHive.toString, + HiveSyncConfigHolder.HIVE_SYNC_MODE.key -> hiveSyncConfig.getStringOrDefault(HiveSyncConfigHolder.HIVE_SYNC_MODE, HiveSyncMode.HMS.name()), + HoodieSyncConfig.META_SYNC_DATABASE_NAME.key -> hiveSyncConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_DATABASE_NAME), + HoodieSyncConfig.META_SYNC_TABLE_NAME.key -> hiveSyncConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_TABLE_NAME), + HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> hiveSyncConfig.getBoolean(HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE).toString, HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key -> partitionFields, - HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key -> hiveSyncConfig.partitionValueExtractorClass + HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key -> hiveSyncConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS) ) .filter { case (_, v) => v != null } } @@ -273,9 +271,9 @@ trait ProvidesHoodieConfig extends Logging { OPERATION.key -> DataSourceWriteOptions.DELETE_OPERATION_OPT_VAL, PARTITIONPATH_FIELD.key -> tableConfig.getPartitionFieldProp, HoodieSyncConfig.META_SYNC_ENABLED.key -> enableHive.toString, - HiveSyncConfig.HIVE_SYNC_ENABLED.key -> enableHive.toString, - HiveSyncConfig.HIVE_SYNC_MODE.key -> hiveSyncConfig.syncMode, - HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> hiveSyncConfig.supportTimestamp.toString, + HiveSyncConfigHolder.HIVE_SYNC_ENABLED.key -> enableHive.toString, + HiveSyncConfigHolder.HIVE_SYNC_MODE.key -> hiveSyncConfig.getStringOrDefault(HiveSyncConfigHolder.HIVE_SYNC_MODE, HiveSyncMode.HMS.name()), + HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> hiveSyncConfig.getBoolean(HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE).toString, HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key -> hoodieProps.getString(HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key, "200"), SqlKeyGenerator.PARTITION_SCHEMA -> partitionSchema.toDDL ) @@ -289,33 +287,20 @@ trait ProvidesHoodieConfig extends Logging { } def buildHiveSyncConfig(props: TypedProperties, hoodieCatalogTable: HoodieCatalogTable): HiveSyncConfig = { - val hiveSyncConfig: HiveSyncConfig = new HiveSyncConfig - hiveSyncConfig.basePath = hoodieCatalogTable.tableLocation - hiveSyncConfig.baseFileFormat = hoodieCatalogTable.baseFileFormat - hiveSyncConfig.usePreApacheInputFormat = props.getBoolean(HiveSyncConfig.HIVE_USE_PRE_APACHE_INPUT_FORMAT.key, HiveSyncConfig.HIVE_USE_PRE_APACHE_INPUT_FORMAT.defaultValue.toBoolean) - hiveSyncConfig.databaseName = hoodieCatalogTable.table.identifier.database.getOrElse("default") - if (props.containsKey(HoodieSyncConfig.META_SYNC_TABLE_NAME.key)) { - hiveSyncConfig.tableName = props.getString(HoodieSyncConfig.META_SYNC_TABLE_NAME.key) - } else { - hiveSyncConfig.tableName = hoodieCatalogTable.table.identifier.table - } - hiveSyncConfig.syncMode = props.getString(HiveSyncConfig.HIVE_SYNC_MODE.key, HiveSyncMode.HMS.name()) - hiveSyncConfig.hiveUser = props.getString(HiveSyncConfig.HIVE_USER.key, HiveSyncConfig.HIVE_USER.defaultValue) - hiveSyncConfig.hivePass = props.getString(HiveSyncConfig.HIVE_PASS.key, HiveSyncConfig.HIVE_PASS.defaultValue) - hiveSyncConfig.jdbcUrl = props.getString(HiveSyncConfig.HIVE_URL.key, HiveSyncConfig.HIVE_URL.defaultValue) - hiveSyncConfig.metastoreUris = props.getString(HiveSyncConfig.METASTORE_URIS.key, HiveSyncConfig.METASTORE_URIS.defaultValue) - hiveSyncConfig.partitionFields = props.getStringList(HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key, ",", new util.ArrayList[String]) - hiveSyncConfig.partitionValueExtractorClass = props.getString(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key, classOf[MultiPartKeysValueExtractor].getName) - if (props.containsKey(HiveSyncConfig.HIVE_SYNC_MODE.key)) hiveSyncConfig.syncMode = props.getString(HiveSyncConfig.HIVE_SYNC_MODE.key) - hiveSyncConfig.autoCreateDatabase = props.getString(HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.key, HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.defaultValue).toBoolean - hiveSyncConfig.ignoreExceptions = props.getString(HiveSyncConfig.HIVE_IGNORE_EXCEPTIONS.key, HiveSyncConfig.HIVE_IGNORE_EXCEPTIONS.defaultValue).toBoolean - hiveSyncConfig.skipROSuffix = props.getString(HiveSyncConfig.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.key, HiveSyncConfig.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.defaultValue).toBoolean - hiveSyncConfig.supportTimestamp = props.getString(HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.key, "true").toBoolean - hiveSyncConfig.isConditionalSync = props.getString(HoodieSyncConfig.META_SYNC_CONDITIONAL_SYNC.key, HoodieSyncConfig.META_SYNC_CONDITIONAL_SYNC.defaultValue).toBoolean - hiveSyncConfig.bucketSpec = if (props.getBoolean(HiveSyncConfig.HIVE_SYNC_BUCKET_SYNC.key, HiveSyncConfig.HIVE_SYNC_BUCKET_SYNC.defaultValue)) HiveSyncConfig.getBucketSpec(props.getString(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key), props.getInteger(HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS.key)) - else null - if (props.containsKey(HiveExternalCatalog.CREATED_SPARK_VERSION)) hiveSyncConfig.sparkVersion = props.getString(HiveExternalCatalog.CREATED_SPARK_VERSION) - hiveSyncConfig.syncComment = props.getString(DataSourceWriteOptions.HIVE_SYNC_COMMENT.key, DataSourceWriteOptions.HIVE_SYNC_COMMENT.defaultValue).toBoolean + val hiveSyncConfig: HiveSyncConfig = new HiveSyncConfig(props) + hiveSyncConfig.setValue(HoodieSyncConfig.META_SYNC_BASE_PATH, hoodieCatalogTable.tableLocation) + hiveSyncConfig.setValue(HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT, hoodieCatalogTable.baseFileFormat) + hiveSyncConfig.setValue(HoodieSyncConfig.META_SYNC_DATABASE_NAME, hoodieCatalogTable.table.identifier.database.getOrElse("default")) + hiveSyncConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_TABLE_NAME, hoodieCatalogTable.table.identifier.table) + hiveSyncConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS, classOf[MultiPartKeysValueExtractor].getName) + hiveSyncConfig.setDefaultValue(HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE, "true") + if (hiveSyncConfig.useBucketSync()) + hiveSyncConfig.setValue(HiveSyncConfigHolder.HIVE_SYNC_BUCKET_SYNC_SPEC, + HiveSyncConfig.getBucketSpec(props.getString(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key), + props.getInteger(HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS.key))) + if (props.containsKey(HiveExternalCatalog.CREATED_SPARK_VERSION)) + hiveSyncConfig.setValue(HoodieSyncConfig.META_SYNC_SPARK_VERSION, + props.getString(HiveExternalCatalog.CREATED_SPARK_VERSION)) hiveSyncConfig } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/BootstrapExecutorUtils.java b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/BootstrapExecutorUtils.java index 0cae022967e6b..97d3cfc441618 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/BootstrapExecutorUtils.java +++ b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/BootstrapExecutorUtils.java @@ -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.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.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 BootstrapExecutorUtils(Config cfg, JavaSparkContext jssc, FileSystem fs, // 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 @@ private void syncHive() { 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(); } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableAsSelectCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableAsSelectCommand.scala index 733bd67a0b0d7..a4d0fa04a7c3f 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableAsSelectCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableAsSelectCommand.scala @@ -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" ) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala index 636599ce0cf48..b6447ebab8d6b 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala @@ -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"), diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaApp.java b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaApp.java index d25896bdd9779..54f31ee281d74 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaApp.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaApp.java @@ -50,6 +50,12 @@ 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 void run() throws Exception { private DataFrameWriter updateHiveSyncConfig(DataFrameWriter 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(), diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaGenerateApp.java b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaGenerateApp.java index 42c6a4fd89f41..491e164b9572d 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaGenerateApp.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaGenerateApp.java @@ -46,6 +46,12 @@ 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 @@ private HoodieTestDataGenerator getDataGenerate() { private DataFrameWriter updateHiveSyncConfig(DataFrameWriter 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(), diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java index 207a9492fdad0..af2dd7613c4f5 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java @@ -54,6 +54,12 @@ 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 void stream(Dataset streamingInput, String operationType, String che private DataStreamWriter updateHiveSyncConfig(DataStreamWriter 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(), diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java index 41c061d2bc91c..11f0fc97856e9 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestDataSourceUtils.java @@ -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.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.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.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 void testCreateHoodieConfigWithAsyncClustering() { }); } - @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) diff --git a/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/AbstractAdbSyncHoodieClient.java b/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/AbstractAdbSyncHoodieClient.java deleted file mode 100644 index 84316ddb1152b..0000000000000 --- a/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/AbstractAdbSyncHoodieClient.java +++ /dev/null @@ -1,128 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hudi.sync.adb; - -import org.apache.hudi.common.fs.FSUtils; -import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.util.StringUtils; -import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.hive.PartitionValueExtractor; -import org.apache.hudi.hive.SchemaDifference; -import org.apache.hudi.sync.common.AbstractSyncHoodieClient; - -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -public abstract class AbstractAdbSyncHoodieClient extends AbstractSyncHoodieClient { - protected AdbSyncConfig adbSyncConfig; - protected PartitionValueExtractor partitionValueExtractor; - protected HoodieTimeline activeTimeline; - - public AbstractAdbSyncHoodieClient(AdbSyncConfig syncConfig, FileSystem fs) { - super(syncConfig.basePath, syncConfig.assumeDatePartitioning, - syncConfig.useFileListingFromMetadata, false, fs); - this.adbSyncConfig = syncConfig; - final String clazz = adbSyncConfig.partitionValueExtractorClass; - try { - this.partitionValueExtractor = (PartitionValueExtractor) Class.forName(clazz).newInstance(); - } catch (Exception e) { - throw new HoodieException("Fail to init PartitionValueExtractor class " + clazz, e); - } - - activeTimeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); - } - - public List getPartitionEvents(Map, String> tablePartitions, - List partitionStoragePartitions) { - Map paths = new HashMap<>(); - - for (Map.Entry, String> entry : tablePartitions.entrySet()) { - List partitionValues = entry.getKey(); - String fullTablePartitionPath = entry.getValue(); - paths.put(String.join(", ", partitionValues), fullTablePartitionPath); - } - List events = new ArrayList<>(); - for (String storagePartition : partitionStoragePartitions) { - Path storagePartitionPath = FSUtils.getPartitionPath(adbSyncConfig.basePath, storagePartition); - String fullStoragePartitionPath = Path.getPathWithoutSchemeAndAuthority(storagePartitionPath).toUri().getPath(); - // Check if the partition values or if hdfs path is the same - List storagePartitionValues = partitionValueExtractor.extractPartitionValuesInPath(storagePartition); - if (adbSyncConfig.useHiveStylePartitioning) { - String partition = String.join("/", storagePartitionValues); - storagePartitionPath = FSUtils.getPartitionPath(adbSyncConfig.basePath, partition); - fullStoragePartitionPath = Path.getPathWithoutSchemeAndAuthority(storagePartitionPath).toUri().getPath(); - } - if (!storagePartitionValues.isEmpty()) { - String storageValue = String.join(", ", storagePartitionValues); - if (!paths.containsKey(storageValue)) { - events.add(PartitionEvent.newPartitionAddEvent(storagePartition)); - } else if (!paths.get(storageValue).equals(fullStoragePartitionPath)) { - events.add(PartitionEvent.newPartitionUpdateEvent(storagePartition)); - } - } - } - return events; - } - - public void close() { - - } - - public abstract Map, String> scanTablePartitions(String tableName) throws Exception; - - public abstract void updateTableDefinition(String tableName, SchemaDifference schemaDiff) throws Exception; - - public abstract boolean databaseExists(String databaseName) throws Exception; - - public abstract void createDatabase(String databaseName) throws Exception; - - public abstract void dropTable(String tableName); - - protected String getDatabasePath() { - String dbLocation = adbSyncConfig.dbLocation; - Path dbLocationPath; - if (StringUtils.isNullOrEmpty(dbLocation)) { - if (new Path(adbSyncConfig.basePath).isRoot()) { - dbLocationPath = new Path(adbSyncConfig.basePath); - } else { - dbLocationPath = new Path(adbSyncConfig.basePath).getParent(); - } - } else { - dbLocationPath = new Path(dbLocation); - } - return generateAbsolutePathStr(dbLocationPath); - } - - protected String generateAbsolutePathStr(Path path) { - String absolutePathStr = path.toString(); - if (path.toUri().getScheme() == null) { - absolutePathStr = getDefaultFs() + absolutePathStr; - } - return absolutePathStr.endsWith("/") ? absolutePathStr : absolutePathStr + "/"; - } - - protected String getDefaultFs() { - return fs.getConf().get("fs.defaultFS"); - } -} diff --git a/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/AdbSyncConfig.java b/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/AdbSyncConfig.java index ae2e7024e5870..5dca22254c6cb 100644 --- a/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/AdbSyncConfig.java +++ b/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/AdbSyncConfig.java @@ -20,62 +20,19 @@ import org.apache.hudi.common.config.ConfigProperty; import org.apache.hudi.common.config.TypedProperties; -import org.apache.hudi.sync.common.HoodieSyncConfig; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.hive.HiveSyncConfig; import com.beust.jcommander.Parameter; +import com.beust.jcommander.ParametersDelegate; +import org.apache.hadoop.fs.Path; + +import java.util.Properties; /** * Configs needed to sync data into Alibaba Cloud AnalyticDB(ADB). */ -public class AdbSyncConfig extends HoodieSyncConfig { - - @Parameter(names = {"--user"}, description = "Adb username", required = true) - public String adbUser; - - @Parameter(names = {"--pass"}, description = "Adb password", required = true) - public String adbPass; - - @Parameter(names = {"--jdbc-url"}, description = "Adb jdbc connect url", required = true) - public String jdbcUrl; - - @Parameter(names = {"--skip-ro-suffix"}, description = "Whether skip the `_ro` suffix for read optimized table when syncing") - public Boolean skipROSuffix; - - @Parameter(names = {"--skip-rt-sync"}, description = "Whether skip the rt table when syncing") - public Boolean skipRTSync; - - @Parameter(names = {"--hive-style-partitioning"}, description = "Whether use hive style partitioning, true if like the following style: field1=value1/field2=value2") - public Boolean useHiveStylePartitioning; - - @Parameter(names = {"--support-timestamp"}, description = "If true, converts int64(timestamp_micros) to timestamp type") - public Boolean supportTimestamp; - - @Parameter(names = {"--spark-datasource"}, description = "Whether sync this table as spark data source table") - public Boolean syncAsSparkDataSourceTable; - - @Parameter(names = {"--table-properties"}, description = "Table properties, to support read hoodie table as datasource table", required = true) - public String tableProperties; - - @Parameter(names = {"--serde-properties"}, description = "Serde properties, to support read hoodie table as datasource table", required = true) - public String serdeProperties; - - @Parameter(names = {"--spark-schema-length-threshold"}, description = "The maximum length allowed in a single cell when storing additional schema information in Hive's metastore") - public int sparkSchemaLengthThreshold; - - @Parameter(names = {"--db-location"}, description = "Database location") - public String dbLocation; - - @Parameter(names = {"--auto-create-database"}, description = "Whether auto create adb database") - public Boolean autoCreateDatabase = true; - - @Parameter(names = {"--skip-last-commit-time-sync"}, description = "Whether skip last commit time syncing") - public Boolean skipLastCommitTimeSync = false; - - @Parameter(names = {"--drop-table-before-creation"}, description = "Whether drop table before creation") - public Boolean dropTableBeforeCreation = false; - - @Parameter(names = {"--help", "-h"}, help = true) - public Boolean help = false; +public class AdbSyncConfig extends HiveSyncConfig { public static final ConfigProperty ADB_SYNC_USER = ConfigProperty .key("hoodie.datasource.adb.sync.username") @@ -152,89 +109,101 @@ public class AdbSyncConfig extends HoodieSyncConfig { .defaultValue(false) .withDocumentation("Whether drop table before creation"); - public AdbSyncConfig() { - this(new TypedProperties()); + public AdbSyncConfig(Properties props) { + super(props); + } + + @Override + public String getAbsoluteBasePath() { + return generateAbsolutePathStr(new Path(getString(META_SYNC_BASE_PATH))); } - public AdbSyncConfig(TypedProperties props) { - super(props); + public String getDatabasePath() { + Path basePath = new Path(getString(META_SYNC_BASE_PATH)); + Path dbLocationPath; + String dbLocation = getString(ADB_SYNC_DB_LOCATION); + if (StringUtils.isNullOrEmpty(dbLocation)) { + if (basePath.isRoot()) { + dbLocationPath = basePath; + } else { + dbLocationPath = basePath.getParent(); + } + } else { + dbLocationPath = new Path(dbLocation); + } + return generateAbsolutePathStr(dbLocationPath); + } - adbUser = getString(ADB_SYNC_USER); - adbPass = getString(ADB_SYNC_PASS); - jdbcUrl = getString(ADB_SYNC_JDBC_URL); - skipROSuffix = getBooleanOrDefault(ADB_SYNC_SKIP_RO_SUFFIX); - skipRTSync = getBooleanOrDefault(ADB_SYNC_SKIP_RT_SYNC); - useHiveStylePartitioning = getBooleanOrDefault(ADB_SYNC_USE_HIVE_STYLE_PARTITIONING); - supportTimestamp = getBooleanOrDefault(ADB_SYNC_SUPPORT_TIMESTAMP); - syncAsSparkDataSourceTable = getBooleanOrDefault(ADB_SYNC_SYNC_AS_SPARK_DATA_SOURCE_TABLE); - tableProperties = getString(ADB_SYNC_TABLE_PROPERTIES); - serdeProperties = getString(ADB_SYNC_SERDE_PROPERTIES); - sparkSchemaLengthThreshold = getIntOrDefault(ADB_SYNC_SCHEMA_STRING_LENGTH_THRESHOLD); - dbLocation = getString(ADB_SYNC_DB_LOCATION); - autoCreateDatabase = getBooleanOrDefault(ADB_SYNC_AUTO_CREATE_DATABASE); - skipLastCommitTimeSync = getBooleanOrDefault(ADB_SYNC_SKIP_LAST_COMMIT_TIME_SYNC); - dropTableBeforeCreation = getBooleanOrDefault(ADB_SYNC_DROP_TABLE_BEFORE_CREATION); + public String generateAbsolutePathStr(Path path) { + String absolutePathStr = path.toString(); + if (path.toUri().getScheme() == null) { + absolutePathStr = getDefaultFs() + absolutePathStr; + } + return absolutePathStr.endsWith("/") ? absolutePathStr : absolutePathStr + "/"; } - public static TypedProperties toProps(AdbSyncConfig cfg) { - TypedProperties properties = new TypedProperties(); - properties.put(META_SYNC_DATABASE_NAME.key(), cfg.databaseName); - properties.put(META_SYNC_TABLE_NAME.key(), cfg.tableName); - properties.put(ADB_SYNC_USER.key(), cfg.adbUser); - properties.put(ADB_SYNC_PASS.key(), cfg.adbPass); - properties.put(ADB_SYNC_JDBC_URL.key(), cfg.jdbcUrl); - properties.put(META_SYNC_BASE_PATH.key(), cfg.basePath); - properties.put(META_SYNC_PARTITION_FIELDS.key(), String.join(",", cfg.partitionFields)); - properties.put(META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), cfg.partitionValueExtractorClass); - properties.put(META_SYNC_ASSUME_DATE_PARTITION.key(), String.valueOf(cfg.assumeDatePartitioning)); - properties.put(ADB_SYNC_SKIP_RO_SUFFIX.key(), String.valueOf(cfg.skipROSuffix)); - properties.put(ADB_SYNC_SKIP_RT_SYNC.key(), String.valueOf(cfg.skipRTSync)); - properties.put(ADB_SYNC_USE_HIVE_STYLE_PARTITIONING.key(), String.valueOf(cfg.useHiveStylePartitioning)); - properties.put(META_SYNC_USE_FILE_LISTING_FROM_METADATA.key(), String.valueOf(cfg.useFileListingFromMetadata)); - properties.put(ADB_SYNC_SUPPORT_TIMESTAMP.key(), String.valueOf(cfg.supportTimestamp)); - properties.put(ADB_SYNC_TABLE_PROPERTIES.key(), cfg.tableProperties); - properties.put(ADB_SYNC_SERDE_PROPERTIES.key(), cfg.serdeProperties); - properties.put(ADB_SYNC_SYNC_AS_SPARK_DATA_SOURCE_TABLE.key(), String.valueOf(cfg.syncAsSparkDataSourceTable)); - properties.put(ADB_SYNC_SCHEMA_STRING_LENGTH_THRESHOLD.key(), String.valueOf(cfg.sparkSchemaLengthThreshold)); - properties.put(META_SYNC_SPARK_VERSION.key(), cfg.sparkVersion); - properties.put(ADB_SYNC_DB_LOCATION.key(), cfg.dbLocation); - properties.put(ADB_SYNC_AUTO_CREATE_DATABASE.key(), String.valueOf(cfg.autoCreateDatabase)); - properties.put(ADB_SYNC_SKIP_LAST_COMMIT_TIME_SYNC.key(), String.valueOf(cfg.skipLastCommitTimeSync)); - properties.put(ADB_SYNC_DROP_TABLE_BEFORE_CREATION.key(), String.valueOf(cfg.dropTableBeforeCreation)); - - return properties; + public String getDefaultFs() { + return getHadoopConf().get("fs.defaultFS"); } - @Override - public String toString() { - return "AdbSyncConfig{" - + "adbUser='" + adbUser + '\'' - + ", adbPass='" + adbPass + '\'' - + ", jdbcUrl='" + jdbcUrl + '\'' - + ", skipROSuffix=" + skipROSuffix - + ", skipRTSync=" + skipRTSync - + ", useHiveStylePartitioning=" + useHiveStylePartitioning - + ", supportTimestamp=" + supportTimestamp - + ", syncAsSparkDataSourceTable=" + syncAsSparkDataSourceTable - + ", tableProperties='" + tableProperties + '\'' - + ", serdeProperties='" + serdeProperties + '\'' - + ", sparkSchemaLengthThreshold=" + sparkSchemaLengthThreshold - + ", dbLocation='" + dbLocation + '\'' - + ", autoCreateDatabase=" + autoCreateDatabase - + ", skipLastCommitTimeSync=" + skipLastCommitTimeSync - + ", dropTableBeforeCreation=" + dropTableBeforeCreation - + ", help=" + help - + ", databaseName='" + databaseName + '\'' - + ", tableName='" + tableName + '\'' - + ", basePath='" + basePath + '\'' - + ", baseFileFormat='" + baseFileFormat + '\'' - + ", partitionFields=" + partitionFields - + ", partitionValueExtractorClass='" + partitionValueExtractorClass + '\'' - + ", assumeDatePartitioning=" + assumeDatePartitioning - + ", decodePartition=" + decodePartition - + ", useFileListingFromMetadata=" + useFileListingFromMetadata - + ", isConditionalSync=" + isConditionalSync - + ", sparkVersion='" + sparkVersion + '\'' - + '}'; + public static class AdbSyncConfigParams { + + @ParametersDelegate() + public HiveSyncConfig.HiveSyncConfigParams hiveSyncConfigParams = new HiveSyncConfig.HiveSyncConfigParams(); + + @Parameter(names = {"--support-timestamp"}, description = "If true, converts int64(timestamp_micros) to timestamp type") + public Boolean supportTimestamp; + @Parameter(names = {"--spark-datasource"}, description = "Whether sync this table as spark data source table") + public Boolean syncAsSparkDataSourceTable; + @Parameter(names = {"--table-properties"}, description = "Table properties, to support read hoodie table as datasource table", required = true) + public String tableProperties; + @Parameter(names = {"--serde-properties"}, description = "Serde properties, to support read hoodie table as datasource table", required = true) + public String serdeProperties; + @Parameter(names = {"--spark-schema-length-threshold"}, description = "The maximum length allowed in a single cell when storing additional schema information in Hive's metastore") + public int sparkSchemaLengthThreshold; + @Parameter(names = {"--hive-style-partitioning"}, description = "Whether use hive style partitioning, true if like the following style: field1=value1/field2=value2") + public Boolean useHiveStylePartitioning; + @Parameter(names = {"--skip-rt-sync"}, description = "Whether skip the rt table when syncing") + public Boolean skipRTSync; + @Parameter(names = {"--db-location"}, description = "Database location") + public String dbLocation; + @Parameter(names = {"--auto-create-database"}, description = "Whether auto create adb database") + public Boolean autoCreateDatabase = true; + @Parameter(names = {"--skip-last-commit-time-sync"}, description = "Whether skip last commit time syncing") + public Boolean skipLastCommitTimeSync = false; + @Parameter(names = {"--drop-table-before-creation"}, description = "Whether drop table before creation") + public Boolean dropTableBeforeCreation = false; + + public boolean isHelp() { + return hiveSyncConfigParams.isHelp(); + } + + public TypedProperties toProps() { + final TypedProperties props = hiveSyncConfigParams.toProps(); + props.setPropertyIfNonNull(META_SYNC_DATABASE_NAME.key(), hiveSyncConfigParams.hoodieSyncConfigParams.databaseName); + props.setPropertyIfNonNull(META_SYNC_TABLE_NAME.key(), hiveSyncConfigParams.hoodieSyncConfigParams.tableName); + props.setPropertyIfNonNull(ADB_SYNC_USER.key(), hiveSyncConfigParams.hiveUser); + props.setPropertyIfNonNull(ADB_SYNC_PASS.key(), hiveSyncConfigParams.hivePass); + props.setPropertyIfNonNull(ADB_SYNC_JDBC_URL.key(), hiveSyncConfigParams.jdbcUrl); + props.setPropertyIfNonNull(META_SYNC_BASE_PATH.key(), hiveSyncConfigParams.hoodieSyncConfigParams.basePath); + props.setPropertyIfNonNull(META_SYNC_PARTITION_FIELDS.key(), String.join(",", hiveSyncConfigParams.hoodieSyncConfigParams.partitionFields)); + props.setPropertyIfNonNull(META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), hiveSyncConfigParams.hoodieSyncConfigParams.partitionValueExtractorClass); + props.setPropertyIfNonNull(META_SYNC_ASSUME_DATE_PARTITION.key(), String.valueOf(hiveSyncConfigParams.hoodieSyncConfigParams.assumeDatePartitioning)); + props.setPropertyIfNonNull(ADB_SYNC_SKIP_RO_SUFFIX.key(), String.valueOf(hiveSyncConfigParams.skipROSuffix)); + props.setPropertyIfNonNull(ADB_SYNC_SKIP_RT_SYNC.key(), String.valueOf(skipRTSync)); + props.setPropertyIfNonNull(ADB_SYNC_USE_HIVE_STYLE_PARTITIONING.key(), String.valueOf(useHiveStylePartitioning)); + props.setPropertyIfNonNull(META_SYNC_USE_FILE_LISTING_FROM_METADATA.key(), String.valueOf(hiveSyncConfigParams.hoodieSyncConfigParams.useFileListingFromMetadata)); + props.setPropertyIfNonNull(ADB_SYNC_SUPPORT_TIMESTAMP.key(), String.valueOf(supportTimestamp)); + props.setPropertyIfNonNull(ADB_SYNC_TABLE_PROPERTIES.key(), tableProperties); + props.setPropertyIfNonNull(ADB_SYNC_SERDE_PROPERTIES.key(), serdeProperties); + props.setPropertyIfNonNull(ADB_SYNC_SYNC_AS_SPARK_DATA_SOURCE_TABLE.key(), String.valueOf(syncAsSparkDataSourceTable)); + props.setPropertyIfNonNull(ADB_SYNC_SCHEMA_STRING_LENGTH_THRESHOLD.key(), String.valueOf(sparkSchemaLengthThreshold)); + props.setPropertyIfNonNull(META_SYNC_SPARK_VERSION.key(), hiveSyncConfigParams.hoodieSyncConfigParams.sparkVersion); + props.setPropertyIfNonNull(ADB_SYNC_DB_LOCATION.key(), dbLocation); + props.setPropertyIfNonNull(ADB_SYNC_AUTO_CREATE_DATABASE.key(), String.valueOf(autoCreateDatabase)); + props.setPropertyIfNonNull(ADB_SYNC_SKIP_LAST_COMMIT_TIME_SYNC.key(), String.valueOf(skipLastCommitTimeSync)); + props.setPropertyIfNonNull(ADB_SYNC_DROP_TABLE_BEFORE_CREATION.key(), String.valueOf(dropTableBeforeCreation)); + return props; + } } } diff --git a/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/AdbSyncTool.java b/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/AdbSyncTool.java index 8c2f9e20451ca..1c578b102cfa0 100644 --- a/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/AdbSyncTool.java +++ b/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/AdbSyncTool.java @@ -18,22 +18,19 @@ package org.apache.hudi.sync.adb; -import org.apache.hudi.common.config.TypedProperties; -import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.util.Option; import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; import org.apache.hudi.hive.SchemaDifference; import org.apache.hudi.hive.util.HiveSchemaUtil; -import org.apache.hudi.sync.common.AbstractSyncHoodieClient.PartitionEvent; -import org.apache.hudi.sync.common.AbstractSyncHoodieClient.PartitionEvent.PartitionEventType; -import org.apache.hudi.sync.common.AbstractSyncTool; +import org.apache.hudi.sync.common.HoodieSyncTool; +import org.apache.hudi.sync.common.model.PartitionEvent; +import org.apache.hudi.sync.common.model.PartitionEvent.PartitionEventType; import org.apache.hudi.sync.common.util.ConfigUtils; +import org.apache.hudi.sync.common.util.SparkDataSourceTableUtils; import com.beust.jcommander.JCommander; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat; import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe; import org.apache.parquet.schema.MessageType; @@ -43,8 +40,25 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.stream.Collectors; +import static org.apache.hudi.sync.adb.AdbSyncConfig.ADB_SYNC_AUTO_CREATE_DATABASE; +import static org.apache.hudi.sync.adb.AdbSyncConfig.ADB_SYNC_DROP_TABLE_BEFORE_CREATION; +import static org.apache.hudi.sync.adb.AdbSyncConfig.ADB_SYNC_SCHEMA_STRING_LENGTH_THRESHOLD; +import static org.apache.hudi.sync.adb.AdbSyncConfig.ADB_SYNC_SERDE_PROPERTIES; +import static org.apache.hudi.sync.adb.AdbSyncConfig.ADB_SYNC_SKIP_LAST_COMMIT_TIME_SYNC; +import static org.apache.hudi.sync.adb.AdbSyncConfig.ADB_SYNC_SKIP_RO_SUFFIX; +import static org.apache.hudi.sync.adb.AdbSyncConfig.ADB_SYNC_SKIP_RT_SYNC; +import static org.apache.hudi.sync.adb.AdbSyncConfig.ADB_SYNC_SUPPORT_TIMESTAMP; +import static org.apache.hudi.sync.adb.AdbSyncConfig.ADB_SYNC_SYNC_AS_SPARK_DATA_SOURCE_TABLE; +import static org.apache.hudi.sync.adb.AdbSyncConfig.ADB_SYNC_TABLE_PROPERTIES; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_BASE_PATH; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_PARTITION_FIELDS; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_SPARK_VERSION; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_TABLE_NAME; + /** * Adb sync tool is mainly used to sync hoodie tables to Alibaba Cloud AnalyticDB(ADB), * it can be used as API `AdbSyncTool.syncHoodieTable(AdbSyncConfig)` or as command @@ -55,45 +69,52 @@ * incremental partitions will be synced as well. */ @SuppressWarnings("WeakerAccess") -public class AdbSyncTool extends AbstractSyncTool { +public class AdbSyncTool extends HoodieSyncTool { private static final Logger LOG = LoggerFactory.getLogger(AdbSyncTool.class); public static final String SUFFIX_SNAPSHOT_TABLE = "_rt"; public static final String SUFFIX_READ_OPTIMIZED_TABLE = "_ro"; - private final AdbSyncConfig adbSyncConfig; - private final AbstractAdbSyncHoodieClient hoodieAdbClient; + private final AdbSyncConfig config; + private final String databaseName; + private final String tableName; + private final HoodieAdbJdbcClient syncClient; private final String snapshotTableName; private final Option roTableTableName; - public AdbSyncTool(TypedProperties props, Configuration conf, FileSystem fs) { - super(props, conf, fs); - this.adbSyncConfig = new AdbSyncConfig(props); - this.hoodieAdbClient = getHoodieAdbClient(adbSyncConfig, fs); - switch (hoodieAdbClient.getTableType()) { + public AdbSyncTool(Properties props) { + super(props); + this.config = new AdbSyncConfig(props); + this.databaseName = config.getString(META_SYNC_DATABASE_NAME); + this.tableName = config.getString(META_SYNC_TABLE_NAME); + this.syncClient = new HoodieAdbJdbcClient(config); + switch (syncClient.getTableType()) { case COPY_ON_WRITE: - this.snapshotTableName = adbSyncConfig.tableName; + this.snapshotTableName = tableName; this.roTableTableName = Option.empty(); break; case MERGE_ON_READ: - this.snapshotTableName = adbSyncConfig.tableName + SUFFIX_SNAPSHOT_TABLE; - this.roTableTableName = adbSyncConfig.skipROSuffix ? Option.of(adbSyncConfig.tableName) - : Option.of(adbSyncConfig.tableName + SUFFIX_READ_OPTIMIZED_TABLE); + this.snapshotTableName = tableName + SUFFIX_SNAPSHOT_TABLE; + this.roTableTableName = config.getBoolean(ADB_SYNC_SKIP_RO_SUFFIX) ? Option.of(tableName) + : Option.of(tableName + SUFFIX_READ_OPTIMIZED_TABLE); break; default: - throw new HoodieAdbSyncException("Unknown table type:" + hoodieAdbClient.getTableType() - + ", basePath:" + hoodieAdbClient.getBasePath()); + throw new HoodieAdbSyncException("Unknown table type:" + syncClient.getTableType() + + ", basePath:" + syncClient.getBasePath()); } } - private AbstractAdbSyncHoodieClient getHoodieAdbClient(AdbSyncConfig adbSyncConfig, FileSystem fs) { - return new HoodieAdbJdbcClient(adbSyncConfig, fs); + @Override + public void close() { + if (syncClient != null) { + syncClient.close(); + } } @Override public void syncHoodieTable() { try { - switch (hoodieAdbClient.getTableType()) { + switch (syncClient.getTableType()) { case COPY_ON_WRITE: syncHoodieTable(snapshotTableName, false, false); break; @@ -101,39 +122,38 @@ public void syncHoodieTable() { // Sync a ro table for MOR table syncHoodieTable(roTableTableName.get(), false, true); // Sync a rt table for MOR table - if (!adbSyncConfig.skipRTSync) { + if (!config.getBoolean(ADB_SYNC_SKIP_RT_SYNC)) { syncHoodieTable(snapshotTableName, true, false); } break; default: - throw new HoodieAdbSyncException("Unknown table type:" + hoodieAdbClient.getTableType() - + ", basePath:" + hoodieAdbClient.getBasePath()); + throw new HoodieAdbSyncException("Unknown table type:" + syncClient.getTableType() + + ", basePath:" + syncClient.getBasePath()); } } catch (Exception re) { - throw new HoodieAdbSyncException("Sync hoodie table to ADB failed, tableName:" + adbSyncConfig.tableName, re); + throw new HoodieAdbSyncException("Sync hoodie table to ADB failed, tableName:" + tableName, re); } finally { - hoodieAdbClient.close(); + syncClient.close(); } } - private void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, - boolean readAsOptimized) throws Exception { + private void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, boolean readAsOptimized) throws Exception { LOG.info("Try to sync hoodie table, tableName:{}, path:{}, tableType:{}", - tableName, hoodieAdbClient.getBasePath(), hoodieAdbClient.getTableType()); + tableName, syncClient.getBasePath(), syncClient.getTableType()); - if (adbSyncConfig.autoCreateDatabase) { + if (config.getBoolean(ADB_SYNC_AUTO_CREATE_DATABASE)) { try { synchronized (AdbSyncTool.class) { - if (!hoodieAdbClient.databaseExists(adbSyncConfig.databaseName)) { - hoodieAdbClient.createDatabase(adbSyncConfig.databaseName); + if (!syncClient.databaseExists(databaseName)) { + syncClient.createDatabase(databaseName); } } } catch (Exception e) { - throw new HoodieAdbSyncException("Failed to create database:" + adbSyncConfig.databaseName + throw new HoodieAdbSyncException("Failed to create database:" + databaseName + ", useRealtimeInputFormat = " + useRealtimeInputFormat, e); } - } else if (!hoodieAdbClient.databaseExists(adbSyncConfig.databaseName)) { - throw new HoodieAdbSyncException("ADB database does not exists:" + adbSyncConfig.databaseName); + } else if (!syncClient.databaseExists(databaseName)) { + throw new HoodieAdbSyncException("ADB database does not exists:" + databaseName); } // Currently HoodieBootstrapRelation does support reading bootstrap MOR rt table, @@ -141,22 +161,22 @@ private void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, // by the data source way (which will use the HoodieBootstrapRelation). // TODO after we support bootstrap MOR rt table in HoodieBootstrapRelation[HUDI-2071], // we can remove this logical. - if (hoodieAdbClient.isBootstrap() - && hoodieAdbClient.getTableType() == HoodieTableType.MERGE_ON_READ + if (syncClient.isBootstrap() + && syncClient.getTableType() == HoodieTableType.MERGE_ON_READ && !readAsOptimized) { - adbSyncConfig.syncAsSparkDataSourceTable = false; + config.setValue(ADB_SYNC_SYNC_AS_SPARK_DATA_SOURCE_TABLE, "false"); LOG.info("Disable sync as spark datasource table for mor rt table:{}", tableName); } - if (adbSyncConfig.dropTableBeforeCreation) { + if (config.getBoolean(ADB_SYNC_DROP_TABLE_BEFORE_CREATION)) { LOG.info("Drop table before creation, tableName:{}", tableName); - hoodieAdbClient.dropTable(tableName); + syncClient.dropTable(tableName); } - boolean tableExists = hoodieAdbClient.tableExists(tableName); + boolean tableExists = syncClient.tableExists(tableName); // Get the parquet schema for this table looking at the latest commit - MessageType schema = hoodieAdbClient.getDataSchema(); + MessageType schema = syncClient.getStorageSchema(); // Sync schema if needed syncSchema(tableName, tableExists, useRealtimeInputFormat, readAsOptimized, schema); @@ -165,16 +185,16 @@ private void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, // Get the last time we successfully synced partitions Option lastCommitTimeSynced = Option.empty(); if (tableExists) { - lastCommitTimeSynced = hoodieAdbClient.getLastCommitTimeSynced(tableName); + lastCommitTimeSynced = syncClient.getLastCommitTimeSynced(tableName); } LOG.info("Last commit time synced was found:{}", lastCommitTimeSynced.orElse("null")); // Scan synced partitions List writtenPartitionsSince; - if (adbSyncConfig.partitionFields.isEmpty()) { + if (config.getSplitStrings(META_SYNC_PARTITION_FIELDS).isEmpty()) { writtenPartitionsSince = new ArrayList<>(); } else { - writtenPartitionsSince = hoodieAdbClient.getPartitionsWrittenToSince(lastCommitTimeSynced); + writtenPartitionsSince = syncClient.getPartitionsWrittenToSince(lastCommitTimeSynced); } LOG.info("Scan partitions complete, partitionNum:{}", writtenPartitionsSince.size()); @@ -183,8 +203,8 @@ private void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, // Update sync commit time // whether to skip syncing commit time stored in tbl properties, since it is time consuming. - if (!adbSyncConfig.skipLastCommitTimeSync) { - hoodieAdbClient.updateLastCommitTimeSynced(tableName); + if (!config.getBoolean(ADB_SYNC_SKIP_LAST_COMMIT_TIME_SYNC)) { + syncClient.updateLastCommitTimeSynced(tableName); } LOG.info("Sync complete for table:{}", tableName); } @@ -200,14 +220,14 @@ private void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, * @param schema The extracted schema */ private void syncSchema(String tableName, boolean tableExists, boolean useRealTimeInputFormat, - boolean readAsOptimized, MessageType schema) throws Exception { + boolean readAsOptimized, MessageType schema) { // Append spark table properties & serde properties - Map tableProperties = ConfigUtils.toMap(adbSyncConfig.tableProperties); - Map serdeProperties = ConfigUtils.toMap(adbSyncConfig.serdeProperties); - if (adbSyncConfig.syncAsSparkDataSourceTable) { - Map sparkTableProperties = getSparkTableProperties(adbSyncConfig.partitionFields, - adbSyncConfig.sparkVersion, adbSyncConfig.sparkSchemaLengthThreshold, schema); - Map sparkSerdeProperties = getSparkSerdeProperties(readAsOptimized, adbSyncConfig.basePath); + Map tableProperties = ConfigUtils.toMap(config.getString(ADB_SYNC_TABLE_PROPERTIES)); + Map serdeProperties = ConfigUtils.toMap(config.getString(ADB_SYNC_SERDE_PROPERTIES)); + if (config.getBoolean(ADB_SYNC_SYNC_AS_SPARK_DATA_SOURCE_TABLE)) { + Map sparkTableProperties = SparkDataSourceTableUtils.getSparkTableProperties(config.getSplitStrings(META_SYNC_PARTITION_FIELDS), + config.getString(META_SYNC_SPARK_VERSION), config.getInt(ADB_SYNC_SCHEMA_STRING_LENGTH_THRESHOLD), schema); + Map sparkSerdeProperties = SparkDataSourceTableUtils.getSparkSerdeProperties(readAsOptimized, config.getString(META_SYNC_BASE_PATH)); tableProperties.putAll(sparkTableProperties); serdeProperties.putAll(sparkSerdeProperties); LOG.info("Sync as spark datasource table, tableName:{}, tableExists:{}, tableProperties:{}, sederProperties:{}", @@ -222,16 +242,16 @@ private void syncSchema(String tableName, boolean tableExists, boolean useRealTi // Custom serde will not work with ALTER TABLE REPLACE COLUMNS // https://github.com/apache/hive/blob/release-1.1.0/ql/src/java/org/apache/hadoop/hive // /ql/exec/DDLTask.java#L3488 - hoodieAdbClient.createTable(tableName, schema, inputFormatClassName, MapredParquetOutputFormat.class.getName(), + syncClient.createTable(tableName, schema, inputFormatClassName, MapredParquetOutputFormat.class.getName(), ParquetHiveSerDe.class.getName(), serdeProperties, tableProperties); } else { // Check if the table schema has evolved - Map tableSchema = hoodieAdbClient.getTableSchema(tableName); - SchemaDifference schemaDiff = HiveSchemaUtil.getSchemaDifference(schema, tableSchema, adbSyncConfig.partitionFields, - adbSyncConfig.supportTimestamp); + Map tableSchema = syncClient.getMetastoreSchema(tableName); + SchemaDifference schemaDiff = HiveSchemaUtil.getSchemaDifference(schema, tableSchema, config.getSplitStrings(META_SYNC_PARTITION_FIELDS), + config.getBoolean(ADB_SYNC_SUPPORT_TIMESTAMP)); if (!schemaDiff.isEmpty()) { LOG.info("Schema difference found for table:{}", tableName); - hoodieAdbClient.updateTableDefinition(tableName, schemaDiff); + syncClient.updateTableDefinition(tableName, schemaDiff); } else { LOG.info("No Schema difference for table:{}", tableName); } @@ -244,19 +264,19 @@ private void syncSchema(String tableName, boolean tableExists, boolean useRealTi */ private void syncPartitions(String tableName, List writtenPartitionsSince) { try { - if (adbSyncConfig.partitionFields.isEmpty()) { + if (config.getSplitStrings(META_SYNC_PARTITION_FIELDS).isEmpty()) { LOG.info("Not a partitioned table."); return; } - Map, String> partitions = hoodieAdbClient.scanTablePartitions(tableName); - List partitionEvents = hoodieAdbClient.getPartitionEvents(partitions, writtenPartitionsSince); + Map, String> partitions = syncClient.scanTablePartitions(tableName); + List partitionEvents = syncClient.getPartitionEvents(partitions, writtenPartitionsSince); List newPartitions = filterPartitions(partitionEvents, PartitionEventType.ADD); LOG.info("New Partitions:{}", newPartitions); - hoodieAdbClient.addPartitionsToTable(tableName, newPartitions); + syncClient.addPartitionsToTable(tableName, newPartitions); List updatePartitions = filterPartitions(partitionEvents, PartitionEventType.UPDATE); LOG.info("Changed Partitions:{}", updatePartitions); - hoodieAdbClient.updatePartitionsToTable(tableName, updatePartitions); + syncClient.updatePartitionsToTable(tableName, updatePartitions); } catch (Exception e) { throw new HoodieAdbSyncException("Failed to sync partitions for table:" + tableName, e); } @@ -268,16 +288,13 @@ private List filterPartitions(List events, PartitionEven } public static void main(String[] args) { - // parse the params - final AdbSyncConfig cfg = new AdbSyncConfig(); - JCommander cmd = new JCommander(cfg, null, args); - if (cfg.help || args.length == 0) { + final AdbSyncConfig.AdbSyncConfigParams params = new AdbSyncConfig.AdbSyncConfigParams(); + JCommander cmd = JCommander.newBuilder().addObject(params).build(); + cmd.parse(args); + if (params.isHelp()) { cmd.usage(); - System.exit(1); + System.exit(0); } - - Configuration hadoopConf = new Configuration(); - FileSystem fs = FSUtils.getFs(cfg.basePath, hadoopConf); - new AdbSyncTool(AdbSyncConfig.toProps(cfg), hadoopConf, fs).syncHoodieTable(); + new AdbSyncTool(params.toProps()).syncHoodieTable(); } } diff --git a/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/HoodieAdbJdbcClient.java b/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/HoodieAdbJdbcClient.java index a347ba701110d..69ccc49528fd6 100644 --- a/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/HoodieAdbJdbcClient.java +++ b/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/HoodieAdbJdbcClient.java @@ -23,12 +23,12 @@ import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.hive.HiveSyncConfig; import org.apache.hudi.hive.HoodieHiveSyncException; import org.apache.hudi.hive.SchemaDifference; import org.apache.hudi.hive.util.HiveSchemaUtil; +import org.apache.hudi.sync.common.HoodieSyncClient; +import org.apache.hudi.sync.common.model.PartitionEvent; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.parquet.schema.MessageType; import org.slf4j.Logger; @@ -47,13 +47,21 @@ import java.util.Map; import java.util.function.Function; -public class HoodieAdbJdbcClient extends AbstractAdbSyncHoodieClient { +import static org.apache.hudi.sync.adb.AdbSyncConfig.ADB_SYNC_JDBC_URL; +import static org.apache.hudi.sync.adb.AdbSyncConfig.ADB_SYNC_PASS; +import static org.apache.hudi.sync.adb.AdbSyncConfig.ADB_SYNC_USER; +import static org.apache.hudi.sync.adb.AdbSyncConfig.ADB_SYNC_USE_HIVE_STYLE_PARTITIONING; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_BASE_PATH; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_PARTITION_FIELDS; + +public class HoodieAdbJdbcClient extends HoodieSyncClient { + private static final Logger LOG = LoggerFactory.getLogger(HoodieAdbJdbcClient.class); public static final String HOODIE_LAST_COMMIT_TIME_SYNC = "hoodie_last_sync"; // Make sure we have the jdbc driver in classpath private static final String DRIVER_NAME = "com.mysql.jdbc.Driver"; - public static final String ADB_ESCAPE_CHARACTER = ""; private static final String TBL_PROPERTIES_STR = "TBLPROPERTIES"; static { @@ -64,12 +72,16 @@ public class HoodieAdbJdbcClient extends AbstractAdbSyncHoodieClient { } } + private final AdbSyncConfig config; + private final String databaseName; private Connection connection; - public HoodieAdbJdbcClient(AdbSyncConfig syncConfig, FileSystem fs) { - super(syncConfig, fs); + public HoodieAdbJdbcClient(AdbSyncConfig config) { + super(config); + this.config = config; + this.databaseName = config.getString(META_SYNC_DATABASE_NAME); createAdbConnection(); - LOG.info("Init adb jdbc client success, jdbcUrl:{}", syncConfig.jdbcUrl); + LOG.info("Init adb jdbc client success, jdbcUrl:{}", config.getString(ADB_SYNC_JDBC_URL)); } private void createAdbConnection() { @@ -82,7 +94,9 @@ private void createAdbConnection() { } try { this.connection = DriverManager.getConnection( - adbSyncConfig.jdbcUrl, adbSyncConfig.adbUser, adbSyncConfig.adbPass); + config.getString(ADB_SYNC_JDBC_URL), + config.getString(ADB_SYNC_USER), + config.getString(ADB_SYNC_PASS)); } catch (SQLException e) { throw new HoodieException("Cannot create adb connection ", e); } @@ -91,12 +105,12 @@ private void createAdbConnection() { @Override public void createTable(String tableName, MessageType storageSchema, String inputFormatClass, - String outputFormatClass, String serdeClass, - Map serdeProperties, Map tableProperties) { + String outputFormatClass, String serdeClass, + Map serdeProperties, Map tableProperties) { try { LOG.info("Creating table:{}", tableName); String createSQLQuery = HiveSchemaUtil.generateCreateDDL(tableName, storageSchema, - getHiveSyncConfig(), inputFormatClass, outputFormatClass, serdeClass, serdeProperties, tableProperties); + config, inputFormatClass, outputFormatClass, serdeClass, serdeProperties, tableProperties); executeAdbSql(createSQLQuery); } catch (IOException e) { throw new HoodieException("Fail to create table:" + tableName, e); @@ -106,17 +120,18 @@ public void createTable(String tableName, MessageType storageSchema, String inpu @Override public void dropTable(String tableName) { LOG.info("Dropping table:{}", tableName); - String dropTable = "drop table if exists `" + adbSyncConfig.databaseName + "`.`" + tableName + "`"; + String dropTable = "drop table if exists `" + databaseName + "`.`" + tableName + "`"; executeAdbSql(dropTable); } - public Map getTableSchema(String tableName) { + @Override + public Map getMetastoreSchema(String tableName) { Map schema = new HashMap<>(); ResultSet result = null; try { DatabaseMetaData databaseMetaData = connection.getMetaData(); - result = databaseMetaData.getColumns(adbSyncConfig.databaseName, - adbSyncConfig.databaseName, tableName, null); + result = databaseMetaData.getColumns(databaseName, + databaseName, tableName, null); while (result.next()) { String columnName = result.getString(4); String columnType = result.getString(6); @@ -174,7 +189,7 @@ private T executeQuerySQL(String sql, Function function) { } public void createDatabase(String databaseName) { - String rootPath = getDatabasePath(); + String rootPath = config.getDatabasePath(); LOG.info("Creating database:{}, databaseLocation:{}", databaseName, rootPath); String sql = constructCreateDatabaseSql(rootPath); executeAdbSql(sql); @@ -197,7 +212,7 @@ public boolean databaseExists(String databaseName) { } @Override - public boolean doesTableExist(String tableName) { + public boolean tableExists(String tableName) { String sql = constructShowLikeTableSql(tableName); Function transform = resultSet -> { try { @@ -209,11 +224,6 @@ public boolean doesTableExist(String tableName) { return executeQuerySQL(sql, transform); } - @Override - public boolean tableExists(String tableName) { - return doesTableExist(tableName); - } - @Override public Option getLastCommitTimeSynced(String tableName) { String sql = constructShowCreateTableSql(tableName); @@ -251,7 +261,7 @@ public Option getLastCommitTimeSynced(String tableName) { @Override public void updateLastCommitTimeSynced(String tableName) { // Set the last commit time from the TBLProperties - String lastCommitSynced = activeTimeline.lastInstant().get().getTimestamp(); + String lastCommitSynced = getActiveTimeline().lastInstant().get().getTimestamp(); try { String sql = constructUpdateTblPropertiesSql(tableName, lastCommitSynced); executeAdbSql(sql); @@ -275,6 +285,11 @@ public void deleteLastReplicatedTimeStamp(String tableName) { throw new UnsupportedOperationException("Not support deleteLastReplicatedTimeStamp yet"); } + @Override + public void updateTableProperties(String tableName, Map tableProperties) { + throw new UnsupportedOperationException("Not support updateTableProperties yet"); + } + @Override public void updatePartitionsToTable(String tableName, List changedPartitions) { if (changedPartitions.isEmpty()) { @@ -294,6 +309,9 @@ public void dropPartitions(String tableName, List partitionsToDrop) { throw new UnsupportedOperationException("Not support dropPartitions yet."); } + /** + * TODO migrate to implementation of {@link #getAllPartitions(String)} + */ public Map, String> scanTablePartitions(String tableName) { String sql = constructShowPartitionSql(tableName); Function, String>> transform = resultSet -> { @@ -304,7 +322,7 @@ public Map, String> scanTablePartitions(String tableName) { String str = resultSet.getString(1); if (!StringUtils.isNullOrEmpty(str)) { List values = partitionValueExtractor.extractPartitionValuesInPath(str); - Path storagePartitionPath = FSUtils.getPartitionPath(adbSyncConfig.basePath, String.join("/", values)); + Path storagePartitionPath = FSUtils.getPartitionPath(config.getString(META_SYNC_BASE_PATH), String.join("/", values)); String fullStoragePartitionPath = Path.getPathWithoutSchemeAndAuthority(storagePartitionPath).toUri().getPath(); partitions.put(values, fullStoragePartitionPath); } @@ -318,6 +336,9 @@ public Map, String> scanTablePartitions(String tableName) { return executeQuerySQL(sql, transform); } + /** + * TODO align with {@link org.apache.hudi.sync.common.HoodieMetaSyncOperations#updateTableSchema} + */ public void updateTableDefinition(String tableName, SchemaDifference schemaDiff) { LOG.info("Adding columns for table:{}", tableName); schemaDiff.getAddColumnTypes().forEach((columnName, columnType) -> @@ -332,12 +353,12 @@ public void updateTableDefinition(String tableName, SchemaDifference schemaDiff) private String constructAddPartitionsSql(String tableName, List partitions) { StringBuilder sqlBuilder = new StringBuilder("alter table `"); - sqlBuilder.append(adbSyncConfig.databaseName).append("`").append(".`") + sqlBuilder.append(databaseName).append("`").append(".`") .append(tableName).append("`").append(" add if not exists "); for (String partition : partitions) { String partitionClause = getPartitionClause(partition); - Path partitionPath = FSUtils.getPartitionPath(adbSyncConfig.basePath, partition); - String fullPartitionPathStr = generateAbsolutePathStr(partitionPath); + Path partitionPath = FSUtils.getPartitionPath(config.getString(META_SYNC_BASE_PATH), partition); + String fullPartitionPathStr = config.generateAbsolutePathStr(partitionPath); sqlBuilder.append(" partition (").append(partitionClause).append(") location '") .append(fullPartitionPathStr).append("' "); } @@ -347,14 +368,14 @@ private String constructAddPartitionsSql(String tableName, List partitio private List constructChangePartitionsSql(String tableName, List partitions) { List changePartitions = new ArrayList<>(); - String useDatabase = "use `" + adbSyncConfig.databaseName + "`"; + String useDatabase = "use `" + databaseName + "`"; changePartitions.add(useDatabase); String alterTable = "alter table `" + tableName + "`"; for (String partition : partitions) { String partitionClause = getPartitionClause(partition); - Path partitionPath = FSUtils.getPartitionPath(adbSyncConfig.basePath, partition); - String fullPartitionPathStr = generateAbsolutePathStr(partitionPath); + Path partitionPath = FSUtils.getPartitionPath(config.getString(META_SYNC_BASE_PATH), partition); + String fullPartitionPathStr = config.generateAbsolutePathStr(partitionPath); String changePartition = alterTable + " add if not exists partition (" + partitionClause + ") location '" + fullPartitionPathStr + "'"; changePartitions.add(changePartition); @@ -371,32 +392,32 @@ private List constructChangePartitionsSql(String tableName, List */ private String getPartitionClause(String partition) { List partitionValues = partitionValueExtractor.extractPartitionValuesInPath(partition); - ValidationUtils.checkArgument(adbSyncConfig.partitionFields.size() == partitionValues.size(), - "Partition key parts " + adbSyncConfig.partitionFields + ValidationUtils.checkArgument(config.getSplitStrings(META_SYNC_PARTITION_FIELDS).size() == partitionValues.size(), + "Partition key parts " + config.getSplitStrings(META_SYNC_PARTITION_FIELDS) + " does not match with partition values " + partitionValues + ". Check partition strategy. "); List partBuilder = new ArrayList<>(); - for (int i = 0; i < adbSyncConfig.partitionFields.size(); i++) { - partBuilder.add(adbSyncConfig.partitionFields.get(i) + "='" + partitionValues.get(i) + "'"); + for (int i = 0; i < config.getSplitStrings(META_SYNC_PARTITION_FIELDS).size(); i++) { + partBuilder.add(config.getSplitStrings(META_SYNC_PARTITION_FIELDS).get(i) + "='" + partitionValues.get(i) + "'"); } return String.join(",", partBuilder); } private String constructShowPartitionSql(String tableName) { - return String.format("show partitions `%s`.`%s`", adbSyncConfig.databaseName, tableName); + return String.format("show partitions `%s`.`%s`", databaseName, tableName); } private String constructShowCreateTableSql(String tableName) { - return String.format("show create table `%s`.`%s`", adbSyncConfig.databaseName, tableName); + return String.format("show create table `%s`.`%s`", databaseName, tableName); } private String constructShowLikeTableSql(String tableName) { - return String.format("show tables from `%s` like '%s'", adbSyncConfig.databaseName, tableName); + return String.format("show tables from `%s` like '%s'", databaseName, tableName); } private String constructCreateDatabaseSql(String rootPath) { return String.format("create database if not exists `%s` with dbproperties(catalog = 'oss', location = '%s')", - adbSyncConfig.databaseName, rootPath); + databaseName, rootPath); } private String constructShowCreateDatabaseSql(String databaseName) { @@ -405,26 +426,69 @@ private String constructShowCreateDatabaseSql(String databaseName) { private String constructUpdateTblPropertiesSql(String tableName, String lastCommitSynced) { return String.format("alter table `%s`.`%s` set tblproperties('%s' = '%s')", - adbSyncConfig.databaseName, tableName, HOODIE_LAST_COMMIT_TIME_SYNC, lastCommitSynced); + databaseName, tableName, HOODIE_LAST_COMMIT_TIME_SYNC, lastCommitSynced); } private String constructAddColumnSql(String tableName, String columnName, String columnType) { return String.format("alter table `%s`.`%s` add columns(`%s` %s)", - adbSyncConfig.databaseName, tableName, columnName, columnType); + databaseName, tableName, columnName, columnType); } private String constructChangeColumnSql(String tableName, String columnName, String columnType) { return String.format("alter table `%s`.`%s` change `%s` `%s` %s", - adbSyncConfig.databaseName, tableName, columnName, columnName, columnType); + databaseName, tableName, columnName, columnName, columnType); + } + + /** + * TODO align with {@link HoodieSyncClient#getPartitionEvents} + */ + public List getPartitionEvents(Map, String> tablePartitions, List partitionStoragePartitions) { + Map paths = new HashMap<>(); + + for (Map.Entry, String> entry : tablePartitions.entrySet()) { + List partitionValues = entry.getKey(); + String fullTablePartitionPath = entry.getValue(); + paths.put(String.join(", ", partitionValues), fullTablePartitionPath); + } + List events = new ArrayList<>(); + for (String storagePartition : partitionStoragePartitions) { + Path storagePartitionPath = FSUtils.getPartitionPath(config.getString(META_SYNC_BASE_PATH), storagePartition); + String fullStoragePartitionPath = Path.getPathWithoutSchemeAndAuthority(storagePartitionPath).toUri().getPath(); + // Check if the partition values or if hdfs path is the same + List storagePartitionValues = partitionValueExtractor.extractPartitionValuesInPath(storagePartition); + if (config.getBoolean(ADB_SYNC_USE_HIVE_STYLE_PARTITIONING)) { + String partition = String.join("/", storagePartitionValues); + storagePartitionPath = FSUtils.getPartitionPath(config.getString(META_SYNC_BASE_PATH), partition); + fullStoragePartitionPath = Path.getPathWithoutSchemeAndAuthority(storagePartitionPath).toUri().getPath(); + } + if (!storagePartitionValues.isEmpty()) { + String storageValue = String.join(", ", storagePartitionValues); + if (!paths.containsKey(storageValue)) { + events.add(PartitionEvent.newPartitionAddEvent(storagePartition)); + } else if (!paths.get(storageValue).equals(fullStoragePartitionPath)) { + events.add(PartitionEvent.newPartitionUpdateEvent(storagePartition)); + } + } + } + return events; } - private HiveSyncConfig getHiveSyncConfig() { - HiveSyncConfig hiveSyncConfig = new HiveSyncConfig(); - hiveSyncConfig.partitionFields = adbSyncConfig.partitionFields; - hiveSyncConfig.databaseName = adbSyncConfig.databaseName; - Path basePath = new Path(adbSyncConfig.basePath); - hiveSyncConfig.basePath = generateAbsolutePathStr(basePath); - return hiveSyncConfig; + public void closeQuietly(ResultSet resultSet, Statement stmt) { + try { + if (stmt != null) { + stmt.close(); + } + } catch (SQLException e) { + LOG.warn("Could not close the statement opened ", e); + } + + try { + if (resultSet != null) { + resultSet.close(); + } + } catch (SQLException e) { + LOG.warn("Could not close the resultset opened ", e); + } } @Override diff --git a/hudi-sync/hudi-adb-sync/src/test/java/org/apache/hudi/sync/adb/TestAdbSyncConfig.java b/hudi-sync/hudi-adb-sync/src/test/java/org/apache/hudi/sync/adb/TestAdbSyncConfig.java index f4eb8fc7fc453..e8c1b07e30fd3 100644 --- a/hudi-sync/hudi-adb-sync/src/test/java/org/apache/hudi/sync/adb/TestAdbSyncConfig.java +++ b/hudi-sync/hudi-adb-sync/src/test/java/org/apache/hudi/sync/adb/TestAdbSyncConfig.java @@ -19,47 +19,72 @@ package org.apache.hudi.sync.adb; -import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.sync.common.util.ConfigUtils; import org.junit.jupiter.api.Test; import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Properties; +import static org.apache.hudi.sync.adb.AdbSyncConfig.ADB_SYNC_DB_LOCATION; +import static org.apache.hudi.sync.adb.AdbSyncConfig.ADB_SYNC_JDBC_URL; +import static org.apache.hudi.sync.adb.AdbSyncConfig.ADB_SYNC_PASS; +import static org.apache.hudi.sync.adb.AdbSyncConfig.ADB_SYNC_SERDE_PROPERTIES; +import static org.apache.hudi.sync.adb.AdbSyncConfig.ADB_SYNC_SKIP_RO_SUFFIX; +import static org.apache.hudi.sync.adb.AdbSyncConfig.ADB_SYNC_TABLE_PROPERTIES; +import static org.apache.hudi.sync.adb.AdbSyncConfig.ADB_SYNC_USER; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_ASSUME_DATE_PARTITION; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_BASE_PATH; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_PARTITION_FIELDS; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_TABLE_NAME; import static org.junit.jupiter.api.Assertions.assertEquals; + public class TestAdbSyncConfig { - @Test - public void testCopy() { - AdbSyncConfig adbSyncConfig = new AdbSyncConfig(); - adbSyncConfig.partitionFields = Arrays.asList("a", "b"); - adbSyncConfig.basePath = "/tmp"; - adbSyncConfig.assumeDatePartitioning = true; - adbSyncConfig.databaseName = "test"; - adbSyncConfig.tableName = "test"; - adbSyncConfig.adbUser = "adb"; - adbSyncConfig.adbPass = "adb"; - adbSyncConfig.jdbcUrl = "jdbc:mysql://localhost:3306"; - adbSyncConfig.skipROSuffix = false; - adbSyncConfig.tableProperties = "spark.sql.sources.provider= 'hudi'\\n" - + "spark.sql.sources.schema.numParts = '1'\\n " - + "spark.sql.sources.schema.part.0 ='xx'\\n " - + "spark.sql.sources.schema.numPartCols = '1'\\n" - + "spark.sql.sources.schema.partCol.0 = 'dt'"; - adbSyncConfig.serdeProperties = "'path'='/tmp/test_db/tbl'"; - adbSyncConfig.dbLocation = "file://tmp/test_db"; - TypedProperties props = AdbSyncConfig.toProps(adbSyncConfig); - AdbSyncConfig copied = new AdbSyncConfig(props); + @Test + public void testInitConfig() { + Properties props = new Properties(); + props.setProperty(META_SYNC_PARTITION_FIELDS.key(), "a,b"); + props.setProperty(META_SYNC_BASE_PATH.key(), "/tmp"); + props.setProperty(META_SYNC_ASSUME_DATE_PARTITION.key(), "true"); + props.setProperty(META_SYNC_DATABASE_NAME.key(), "test"); + props.setProperty(META_SYNC_TABLE_NAME.key(), "test"); + props.setProperty(ADB_SYNC_USER.key(), "adb"); + props.setProperty(ADB_SYNC_PASS.key(), "adb"); + props.setProperty(ADB_SYNC_JDBC_URL.key(), "jdbc:mysql://localhost:3306"); + props.setProperty(ADB_SYNC_SKIP_RO_SUFFIX.key(), "false"); + String tableProps = "spark.sql.sources.provider=hudi\n" + + "spark.sql.sources.schema.numParts=1\n" + + "spark.sql.sources.schema.part.0=xx\n" + + "spark.sql.sources.schema.numPartCols=1\n" + + "spark.sql.sources.schema.partCol.0=dt"; + props.setProperty(ADB_SYNC_TABLE_PROPERTIES.key(), tableProps); + props.setProperty(ADB_SYNC_SERDE_PROPERTIES.key(), "path=/tmp/test_db/tbl"); + props.setProperty(ADB_SYNC_DB_LOCATION.key(), "file://tmp/test_db"); - assertEquals(copied.partitionFields, adbSyncConfig.partitionFields); - assertEquals(copied.basePath, adbSyncConfig.basePath); - assertEquals(copied.assumeDatePartitioning, adbSyncConfig.assumeDatePartitioning); - assertEquals(copied.databaseName, adbSyncConfig.databaseName); - assertEquals(copied.tableName, adbSyncConfig.tableName); - assertEquals(copied.adbUser, adbSyncConfig.adbUser); - assertEquals(copied.adbPass, adbSyncConfig.adbPass); - assertEquals(copied.basePath, adbSyncConfig.basePath); - assertEquals(copied.jdbcUrl, adbSyncConfig.jdbcUrl); - assertEquals(copied.skipROSuffix, adbSyncConfig.skipROSuffix); - assertEquals(copied.supportTimestamp, adbSyncConfig.supportTimestamp); + AdbSyncConfig config = new AdbSyncConfig(props); + assertEquals(Arrays.asList("a", "b"), config.getSplitStrings(META_SYNC_PARTITION_FIELDS)); + assertEquals("/tmp", config.getString(META_SYNC_BASE_PATH)); + assertEquals(true, config.getBoolean(META_SYNC_ASSUME_DATE_PARTITION)); + assertEquals("test", config.getString(META_SYNC_DATABASE_NAME)); + assertEquals("test", config.getString(META_SYNC_TABLE_NAME)); + assertEquals("adb", config.getString(ADB_SYNC_USER)); + assertEquals("adb", config.getString(ADB_SYNC_PASS)); + assertEquals("jdbc:mysql://localhost:3306", config.getString(ADB_SYNC_JDBC_URL)); + assertEquals(false, config.getBoolean(ADB_SYNC_SKIP_RO_SUFFIX)); + Map tablePropsMap = new HashMap<>(); + tablePropsMap.put("spark.sql.sources.provider", "hudi"); + tablePropsMap.put("spark.sql.sources.schema.numParts", "1"); + tablePropsMap.put("spark.sql.sources.schema.part.0", "xx"); + tablePropsMap.put("spark.sql.sources.schema.numPartCols", "1"); + tablePropsMap.put("spark.sql.sources.schema.partCol.0", "dt"); + assertEquals(tablePropsMap, ConfigUtils.toMap(config.getString(ADB_SYNC_TABLE_PROPERTIES))); + Map serdePropsMap = new HashMap<>(); + serdePropsMap.put("path", "/tmp/test_db/tbl"); + assertEquals(serdePropsMap, ConfigUtils.toMap(config.getString(ADB_SYNC_SERDE_PROPERTIES))); + assertEquals("file://tmp/test_db", config.getString(ADB_SYNC_DB_LOCATION)); } } diff --git a/hudi-sync/hudi-datahub-sync/src/main/java/org/apache/hudi/sync/datahub/DataHubSyncClient.java b/hudi-sync/hudi-datahub-sync/src/main/java/org/apache/hudi/sync/datahub/DataHubSyncClient.java index 68569822cc30b..4c050451c5a22 100644 --- a/hudi-sync/hudi-datahub-sync/src/main/java/org/apache/hudi/sync/datahub/DataHubSyncClient.java +++ b/hudi-sync/hudi-datahub-sync/src/main/java/org/apache/hudi/sync/datahub/DataHubSyncClient.java @@ -21,9 +21,8 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; -import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; -import org.apache.hudi.sync.common.AbstractSyncHoodieClient; +import org.apache.hudi.sync.common.HoodieSyncClient; import org.apache.hudi.sync.common.HoodieSyncException; import org.apache.hudi.sync.datahub.config.DataHubSyncConfig; @@ -51,8 +50,6 @@ import datahub.event.MetadataChangeProposalWrapper; import org.apache.avro.AvroTypeException; import org.apache.avro.Schema; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; import org.apache.parquet.schema.MessageType; import java.util.Collections; @@ -60,40 +57,15 @@ import java.util.Map; import java.util.stream.Collectors; -public class DataHubSyncClient extends AbstractSyncHoodieClient { +public class DataHubSyncClient extends HoodieSyncClient { - private final HoodieTimeline activeTimeline; - private final DataHubSyncConfig syncConfig; - private final Configuration hadoopConf; + protected final DataHubSyncConfig config; private final DatasetUrn datasetUrn; - public DataHubSyncClient(DataHubSyncConfig syncConfig, Configuration hadoopConf, FileSystem fs) { - super(syncConfig.basePath, syncConfig.assumeDatePartitioning, syncConfig.useFileListingFromMetadata, false, fs); - this.syncConfig = syncConfig; - this.hadoopConf = hadoopConf; - this.datasetUrn = syncConfig.datasetIdentifier.getDatasetUrn(); - this.activeTimeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); - } - - @Override - public void createTable(String tableName, - MessageType storageSchema, - String inputFormatClass, - String outputFormatClass, - String serdeClass, - Map serdeProperties, - Map tableProperties) { - throw new UnsupportedOperationException("Not supported: `createTable`"); - } - - @Override - public boolean doesTableExist(String tableName) { - return tableExists(tableName); - } - - @Override - public boolean tableExists(String tableName) { - throw new UnsupportedOperationException("Not supported: `tableExists`"); + public DataHubSyncClient(DataHubSyncConfig config) { + super(config); + this.config = config; + this.datasetUrn = config.datasetIdentifier.getDatasetUrn(); } @Override @@ -103,37 +75,7 @@ public Option getLastCommitTimeSynced(String tableName) { @Override public void updateLastCommitTimeSynced(String tableName) { - updateTableProperties(tableName, Collections.singletonMap(HOODIE_LAST_COMMIT_TIME_SYNC, activeTimeline.lastInstant().get().getTimestamp())); - } - - @Override - public Option getLastReplicatedTime(String tableName) { - throw new UnsupportedOperationException("Not supported: `getLastReplicatedTime`"); - } - - @Override - public void updateLastReplicatedTimeStamp(String tableName, String timeStamp) { - throw new UnsupportedOperationException("Not supported: `updateLastReplicatedTimeStamp`"); - } - - @Override - public void deleteLastReplicatedTimeStamp(String tableName) { - throw new UnsupportedOperationException("Not supported: `deleteLastReplicatedTimeStamp`"); - } - - @Override - public void addPartitionsToTable(String tableName, List partitionsToAdd) { - throw new UnsupportedOperationException("Not supported: `addPartitionsToTable`"); - } - - @Override - public void updatePartitionsToTable(String tableName, List changedPartitions) { - throw new UnsupportedOperationException("Not supported: `updatePartitionsToTable`"); - } - - @Override - public void dropPartitions(String tableName, List partitionsToDrop) { - throw new UnsupportedOperationException("Not supported: `dropPartitions`"); + updateTableProperties(tableName, Collections.singletonMap(HOODIE_LAST_COMMIT_TIME_SYNC, getActiveTimeline().lastInstant().get().getTimestamp())); } @Override @@ -145,14 +87,15 @@ public void updateTableProperties(String tableName, Map tablePro .aspect(new DatasetProperties().setCustomProperties(new StringMap(tableProperties))) .build(); - try (RestEmitter emitter = syncConfig.getRestEmitter()) { + try (RestEmitter emitter = config.getRestEmitter()) { emitter.emit(propertiesChangeProposal, null).get(); } catch (Exception e) { throw new HoodieDataHubSyncException("Fail to change properties for Dataset " + datasetUrn + ": " + tableProperties, e); } } - public void updateTableDefinition(String tableName) { + @Override + public void updateTableSchema(String tableName, MessageType schema) { Schema avroSchema = getAvroSchemaWithoutMetadataFields(metaClient); List fields = avroSchema.getFields().stream().map(f -> new SchemaField() .setFieldPath(f.name()) @@ -175,7 +118,7 @@ public void updateTableDefinition(String tableName) { .setFields(new SchemaFieldArray(fields))) .build(); - try (RestEmitter emitter = syncConfig.getRestEmitter()) { + try (RestEmitter emitter = config.getRestEmitter()) { emitter.emit(schemaChangeProposal, null).get(); } catch (Exception e) { throw new HoodieDataHubSyncException("Fail to change schema for Dataset " + datasetUrn, e); @@ -183,7 +126,7 @@ public void updateTableDefinition(String tableName) { } @Override - public Map getTableSchema(String tableName) { + public Map getMetastoreSchema(String tableName) { throw new UnsupportedOperationException("Not supported: `getTableSchema`"); } diff --git a/hudi-sync/hudi-datahub-sync/src/main/java/org/apache/hudi/sync/datahub/DataHubSyncTool.java b/hudi-sync/hudi-datahub-sync/src/main/java/org/apache/hudi/sync/datahub/DataHubSyncTool.java index 9633d6b089f12..567f547a817a0 100644 --- a/hudi-sync/hudi-datahub-sync/src/main/java/org/apache/hudi/sync/datahub/DataHubSyncTool.java +++ b/hudi-sync/hudi-datahub-sync/src/main/java/org/apache/hudi/sync/datahub/DataHubSyncTool.java @@ -19,14 +19,14 @@ package org.apache.hudi.sync.datahub; -import org.apache.hudi.common.config.TypedProperties; -import org.apache.hudi.common.fs.FSUtils; -import org.apache.hudi.sync.common.AbstractSyncTool; +import org.apache.hudi.sync.common.HoodieSyncTool; import org.apache.hudi.sync.datahub.config.DataHubSyncConfig; import com.beust.jcommander.JCommander; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; + +import java.util.Properties; + +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_TABLE_NAME; /** * To sync with DataHub via REST APIs. @@ -34,17 +34,13 @@ * @Experimental * @see https://datahubproject.io/ */ -public class DataHubSyncTool extends AbstractSyncTool { +public class DataHubSyncTool extends HoodieSyncTool { - private final DataHubSyncConfig config; - - public DataHubSyncTool(TypedProperties props, Configuration conf, FileSystem fs) { - this(new DataHubSyncConfig(props), conf, fs); - } + protected final DataHubSyncConfig config; - public DataHubSyncTool(DataHubSyncConfig config, Configuration conf, FileSystem fs) { - super(config.getProps(), conf, fs); - this.config = config; + public DataHubSyncTool(Properties props) { + super(props); + this.config = new DataHubSyncConfig(props); } /** @@ -55,20 +51,20 @@ public DataHubSyncTool(DataHubSyncConfig config, Configuration conf, FileSystem */ @Override public void syncHoodieTable() { - try (DataHubSyncClient syncClient = new DataHubSyncClient(config, conf, fs)) { - syncClient.updateTableDefinition(config.tableName); - syncClient.updateLastCommitTimeSynced(config.tableName); + try (DataHubSyncClient syncClient = new DataHubSyncClient(config)) { + syncClient.updateTableSchema(config.getString(META_SYNC_TABLE_NAME), null); + syncClient.updateLastCommitTimeSynced(config.getString(META_SYNC_TABLE_NAME)); } } public static void main(String[] args) { - final DataHubSyncConfig cfg = new DataHubSyncConfig(); - JCommander cmd = new JCommander(cfg, null, args); - if (cfg.help || args.length == 0) { + final DataHubSyncConfig.DataHubSyncConfigParams params = new DataHubSyncConfig.DataHubSyncConfigParams(); + JCommander cmd = JCommander.newBuilder().addObject(params).build(); + cmd.parse(args); + if (params.isHelp()) { cmd.usage(); - System.exit(1); + System.exit(0); } - FileSystem fs = FSUtils.getFs(cfg.basePath, new Configuration()); - new DataHubSyncTool(cfg, fs.getConf(), fs).syncHoodieTable(); + new DataHubSyncTool(params.toProps()).syncHoodieTable(); } } diff --git a/hudi-sync/hudi-datahub-sync/src/main/java/org/apache/hudi/sync/datahub/config/DataHubSyncConfig.java b/hudi-sync/hudi-datahub-sync/src/main/java/org/apache/hudi/sync/datahub/config/DataHubSyncConfig.java index 1965b15cffb2a..53c6529b5c699 100644 --- a/hudi-sync/hudi-datahub-sync/src/main/java/org/apache/hudi/sync/datahub/config/DataHubSyncConfig.java +++ b/hudi-sync/hudi-datahub-sync/src/main/java/org/apache/hudi/sync/datahub/config/DataHubSyncConfig.java @@ -25,8 +25,11 @@ import org.apache.hudi.sync.common.HoodieSyncConfig; import com.beust.jcommander.Parameter; +import com.beust.jcommander.ParametersDelegate; import datahub.client.rest.RestEmitter; +import java.util.Properties; + public class DataHubSyncConfig extends HoodieSyncConfig { public static final ConfigProperty META_SYNC_DATAHUB_DATASET_IDENTIFIER_CLASS = ConfigProperty @@ -49,45 +52,52 @@ public class DataHubSyncConfig extends HoodieSyncConfig { .noDefaultValue() .withDocumentation("Pluggable class to supply a DataHub REST emitter to connect to the DataHub instance. This overwrites other emitter configs."); - @Parameter(names = {"--identifier-class"}, description = "Pluggable class to help provide info to identify a DataHub Dataset.") - public String identifierClass; + public final HoodieDataHubDatasetIdentifier datasetIdentifier; + + public DataHubSyncConfig(Properties props) { + super(props); + String identifierClass = getStringOrDefault(META_SYNC_DATAHUB_DATASET_IDENTIFIER_CLASS); + datasetIdentifier = (HoodieDataHubDatasetIdentifier) ReflectionUtils.loadClass(identifierClass, new Class[] {Properties.class}, props); + } - @Parameter(names = {"--emitter-server"}, description = "Server URL of the DataHub instance.") - public String emitterServer; + public RestEmitter getRestEmitter() { + if (contains(META_SYNC_DATAHUB_EMITTER_SUPPLIER_CLASS)) { + return ((DataHubEmitterSupplier) ReflectionUtils.loadClass(getString(META_SYNC_DATAHUB_EMITTER_SUPPLIER_CLASS))).get(); + } else if (contains(META_SYNC_DATAHUB_EMITTER_SERVER)) { + return RestEmitter.create(b -> b.server(getString(META_SYNC_DATAHUB_EMITTER_SERVER)).token(getStringOrDefault(META_SYNC_DATAHUB_EMITTER_TOKEN, null))); + } else { + return RestEmitter.createWithDefaults(); + } + } - @Parameter(names = {"--emitter-token"}, description = "Auth token to connect to the DataHub instance.") - public String emitterToken; + public static class DataHubSyncConfigParams { - @Parameter(names = {"--emitter-supplier-class"}, description = "Pluggable class to supply a DataHub REST emitter to connect to the DataHub instance. This overwrites other emitter configs.") - public String emitterSupplierClass; + @ParametersDelegate() + public final HoodieSyncConfigParams hoodieSyncConfigParams = new HoodieSyncConfigParams(); - @Parameter(names = {"--help", "-h"}, help = true) - public Boolean help = false; + @Parameter(names = {"--identifier-class"}, description = "Pluggable class to help provide info to identify a DataHub Dataset.") + public String identifierClass; - public final HoodieDataHubDatasetIdentifier datasetIdentifier; + @Parameter(names = {"--emitter-server"}, description = "Server URL of the DataHub instance.") + public String emitterServer; - public DataHubSyncConfig() { - this(new TypedProperties()); - } + @Parameter(names = {"--emitter-token"}, description = "Auth token to connect to the DataHub instance.") + public String emitterToken; - public DataHubSyncConfig(TypedProperties props) { - super(props); - identifierClass = getStringOrDefault(META_SYNC_DATAHUB_DATASET_IDENTIFIER_CLASS); - emitterServer = getStringOrDefault(META_SYNC_DATAHUB_EMITTER_SERVER, null); - emitterToken = getStringOrDefault(META_SYNC_DATAHUB_EMITTER_TOKEN, null); - emitterSupplierClass = getStringOrDefault(META_SYNC_DATAHUB_EMITTER_SUPPLIER_CLASS, null); + @Parameter(names = {"--emitter-supplier-class"}, description = "Pluggable class to supply a DataHub REST emitter to connect to the DataHub instance. This overwrites other emitter configs.") + public String emitterSupplierClass; - datasetIdentifier = (HoodieDataHubDatasetIdentifier) ReflectionUtils - .loadClass(identifierClass, new Class[] {TypedProperties.class}, props); - } + public boolean isHelp() { + return hoodieSyncConfigParams.isHelp(); + } - public RestEmitter getRestEmitter() { - if (emitterSupplierClass != null) { - return ((DataHubEmitterSupplier) ReflectionUtils.loadClass(emitterSupplierClass)).get(); - } else if (emitterServer != null) { - return RestEmitter.create(b -> b.server(emitterServer).token(emitterToken)); - } else { - return RestEmitter.createWithDefaults(); + public Properties toProps() { + final TypedProperties props = hoodieSyncConfigParams.toProps(); + props.setPropertyIfNonNull(META_SYNC_DATAHUB_DATASET_IDENTIFIER_CLASS.key(), identifierClass); + props.setPropertyIfNonNull(META_SYNC_DATAHUB_EMITTER_SERVER.key(), emitterServer); + props.setPropertyIfNonNull(META_SYNC_DATAHUB_EMITTER_TOKEN.key(), emitterToken); + props.setPropertyIfNonNull(META_SYNC_DATAHUB_EMITTER_SUPPLIER_CLASS.key(), emitterSupplierClass); + return props; } } } diff --git a/hudi-sync/hudi-datahub-sync/src/main/java/org/apache/hudi/sync/datahub/config/HoodieDataHubDatasetIdentifier.java b/hudi-sync/hudi-datahub-sync/src/main/java/org/apache/hudi/sync/datahub/config/HoodieDataHubDatasetIdentifier.java index e3c1ad486c887..c7e121ea0f001 100644 --- a/hudi-sync/hudi-datahub-sync/src/main/java/org/apache/hudi/sync/datahub/config/HoodieDataHubDatasetIdentifier.java +++ b/hudi-sync/hudi-datahub-sync/src/main/java/org/apache/hudi/sync/datahub/config/HoodieDataHubDatasetIdentifier.java @@ -19,12 +19,15 @@ package org.apache.hudi.sync.datahub.config; -import org.apache.hudi.common.config.TypedProperties; - import com.linkedin.common.FabricType; import com.linkedin.common.urn.DataPlatformUrn; import com.linkedin.common.urn.DatasetUrn; +import java.util.Properties; + +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_TABLE_NAME; + /** * Construct and provide the default {@link DatasetUrn} to identify the Dataset on DataHub. *

@@ -34,15 +37,15 @@ public class HoodieDataHubDatasetIdentifier { public static final String DEFAULT_HOODIE_DATAHUB_PLATFORM_NAME = "hudi"; - protected final TypedProperties props; + protected final Properties props; - public HoodieDataHubDatasetIdentifier(TypedProperties props) { + public HoodieDataHubDatasetIdentifier(Properties props) { this.props = props; } public DatasetUrn getDatasetUrn() { DataPlatformUrn dataPlatformUrn = new DataPlatformUrn(DEFAULT_HOODIE_DATAHUB_PLATFORM_NAME); DataHubSyncConfig config = new DataHubSyncConfig(props); - return new DatasetUrn(dataPlatformUrn, String.format("%s.%s", config.databaseName, config.tableName), FabricType.DEV); + return new DatasetUrn(dataPlatformUrn, String.format("%s.%s", config.getString(META_SYNC_DATABASE_NAME), config.getString(META_SYNC_TABLE_NAME)), FabricType.DEV); } } diff --git a/hudi-sync/hudi-datahub-sync/src/test/java/org/apache/hudi/sync/datahub/config/TestDataHubSyncConfig.java b/hudi-sync/hudi-datahub-sync/src/test/java/org/apache/hudi/sync/datahub/config/TestDataHubSyncConfig.java index 4fec62da739bb..9d92970c3b2f2 100644 --- a/hudi-sync/hudi-datahub-sync/src/test/java/org/apache/hudi/sync/datahub/config/TestDataHubSyncConfig.java +++ b/hudi-sync/hudi-datahub-sync/src/test/java/org/apache/hudi/sync/datahub/config/TestDataHubSyncConfig.java @@ -19,22 +19,32 @@ package org.apache.hudi.sync.datahub.config; -import org.apache.hudi.common.config.TypedProperties; - import com.linkedin.common.FabricType; import com.linkedin.common.urn.DatasetUrn; +import datahub.client.rest.RestEmitter; import org.junit.jupiter.api.Test; import java.net.URISyntaxException; +import java.util.Properties; import static org.apache.hudi.sync.datahub.config.DataHubSyncConfig.META_SYNC_DATAHUB_DATASET_IDENTIFIER_CLASS; +import static org.apache.hudi.sync.datahub.config.DataHubSyncConfig.META_SYNC_DATAHUB_EMITTER_SUPPLIER_CLASS; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; class TestDataHubSyncConfig { + @Test + void testGetEmitterFromSupplier() { + Properties props = new Properties(); + props.setProperty(META_SYNC_DATAHUB_EMITTER_SUPPLIER_CLASS.key(), DummySupplier.class.getName()); + DataHubSyncConfig syncConfig = new DataHubSyncConfig(props); + assertNotNull(syncConfig.getRestEmitter()); + } + @Test void testInstantiationWithProps() { - TypedProperties props = new TypedProperties(); + Properties props = new Properties(); props.setProperty(META_SYNC_DATAHUB_DATASET_IDENTIFIER_CLASS.key(), DummyIdentifier.class.getName()); DataHubSyncConfig syncConfig = new DataHubSyncConfig(props); DatasetUrn datasetUrn = syncConfig.datasetIdentifier.getDatasetUrn(); @@ -43,9 +53,17 @@ void testInstantiationWithProps() { assertEquals(FabricType.PROD, datasetUrn.getOriginEntity()); } + public static class DummySupplier implements DataHubEmitterSupplier { + + @Override + public RestEmitter get() { + return RestEmitter.createWithDefaults(); + } + } + public static class DummyIdentifier extends HoodieDataHubDatasetIdentifier { - public DummyIdentifier(TypedProperties props) { + public DummyIdentifier(Properties props) { super(props); } diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/AbstractHiveSyncHoodieClient.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/AbstractHiveSyncHoodieClient.java deleted file mode 100644 index f0641b6fc08b4..0000000000000 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/AbstractHiveSyncHoodieClient.java +++ /dev/null @@ -1,142 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hudi.hive; - -import org.apache.hudi.common.fs.FSUtils; -import org.apache.hudi.common.table.TableSchemaResolver; -import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.util.ReflectionUtils; -import org.apache.hudi.sync.common.AbstractSyncHoodieClient; -import org.apache.hudi.sync.common.HoodieSyncException; -import org.apache.hudi.sync.common.model.Partition; - -import org.apache.avro.Schema; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.parquet.schema.MessageType; - -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -/** - * Base class to sync Hudi tables with Hive based metastores, such as Hive server, HMS or managed Hive services. - */ -public abstract class AbstractHiveSyncHoodieClient extends AbstractSyncHoodieClient { - - protected final HoodieTimeline activeTimeline; - protected final HiveSyncConfig syncConfig; - protected final Configuration hadoopConf; - protected final PartitionValueExtractor partitionValueExtractor; - - public AbstractHiveSyncHoodieClient(HiveSyncConfig syncConfig, Configuration hadoopConf, FileSystem fs) { - super(syncConfig.basePath, syncConfig.assumeDatePartitioning, syncConfig.useFileListingFromMetadata, syncConfig.withOperationField, fs); - this.syncConfig = syncConfig; - this.hadoopConf = hadoopConf; - this.partitionValueExtractor = ReflectionUtils.loadClass(syncConfig.partitionValueExtractorClass); - this.activeTimeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); - } - - public HoodieTimeline getActiveTimeline() { - return activeTimeline; - } - - /** - * Iterate over the storage partitions and find if there are any new partitions that need to be added or updated. - * Generate a list of PartitionEvent based on the changes required. - */ - protected List getPartitionEvents(List tablePartitions, List partitionStoragePartitions, boolean isDropPartition) { - Map paths = new HashMap<>(); - for (Partition tablePartition : tablePartitions) { - List hivePartitionValues = tablePartition.getValues(); - String fullTablePartitionPath = - Path.getPathWithoutSchemeAndAuthority(new Path(tablePartition.getStorageLocation())).toUri().getPath(); - paths.put(String.join(", ", hivePartitionValues), fullTablePartitionPath); - } - - List events = new ArrayList<>(); - for (String storagePartition : partitionStoragePartitions) { - Path storagePartitionPath = FSUtils.getPartitionPath(syncConfig.basePath, storagePartition); - String fullStoragePartitionPath = Path.getPathWithoutSchemeAndAuthority(storagePartitionPath).toUri().getPath(); - // Check if the partition values or if hdfs path is the same - List storagePartitionValues = partitionValueExtractor.extractPartitionValuesInPath(storagePartition); - - if (isDropPartition) { - events.add(PartitionEvent.newPartitionDropEvent(storagePartition)); - } else { - if (!storagePartitionValues.isEmpty()) { - String storageValue = String.join(", ", storagePartitionValues); - if (!paths.containsKey(storageValue)) { - events.add(PartitionEvent.newPartitionAddEvent(storagePartition)); - } else if (!paths.get(storageValue).equals(fullStoragePartitionPath)) { - events.add(PartitionEvent.newPartitionUpdateEvent(storagePartition)); - } - } - } - } - return events; - } - - /** - * Get all partitions for the table in the metastore. - */ - public abstract List getAllPartitions(String tableName); - - /** - * Check if a database already exists in the metastore. - */ - public abstract boolean databaseExists(String databaseName); - - /** - * Create a database in the metastore. - */ - public abstract void createDatabase(String databaseName); - - /** - * Update schema for the table in the metastore. - */ - public abstract void updateTableDefinition(String tableName, MessageType newSchema); - - /* - * APIs below need to be re-worked by modeling field comment in hudi-sync-common, - * instead of relying on Avro or Hive schema class. - */ - - public Schema getAvroSchemaWithoutMetadataFields() { - try { - return new TableSchemaResolver(metaClient).getTableAvroSchemaWithoutMetadataFields(); - } catch (Exception e) { - throw new HoodieSyncException("Failed to read avro schema", e); - } - } - - public abstract List getTableCommentUsingMetastoreClient(String tableName); - - public abstract void updateTableComments(String tableName, List oldSchema, List newSchema); - - public abstract void updateTableComments(String tableName, List oldSchema, Map newComments); - - /* - * APIs above need to be re-worked by modeling field comment in hudi-sync-common, - * instead of relying on Avro or Hive schema class. - */ -} diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveStylePartitionValueExtractor.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveStylePartitionValueExtractor.java index 6741d46900077..11098698e8aeb 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveStylePartitionValueExtractor.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveStylePartitionValueExtractor.java @@ -18,6 +18,8 @@ package org.apache.hudi.hive; +import org.apache.hudi.sync.common.model.PartitionValueExtractor; + import java.util.Collections; import java.util.List; diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java index 36dba81a33a27..6f2cc50a0af69 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java @@ -18,269 +18,147 @@ package org.apache.hudi.hive; -import org.apache.hudi.common.config.ConfigProperty; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.sync.common.HoodieSyncConfig; import com.beust.jcommander.Parameter; +import com.beust.jcommander.ParametersDelegate; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; + +import java.util.Properties; + +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_AUTO_CREATE_DATABASE; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_BATCH_SYNC_PARTITION_NUM; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_CREATE_MANAGED_TABLE; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_IGNORE_EXCEPTIONS; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_PASS; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_AS_DATA_SOURCE_TABLE; +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.hive.HiveSyncConfigHolder.HIVE_SYNC_COMMENT; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_MODE; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_SCHEMA_STRING_LENGTH_THRESHOLD; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_TABLE_PROPERTIES; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_TABLE_SERDE_PROPERTIES; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_URL; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USER; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USE_JDBC; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USE_PRE_APACHE_INPUT_FORMAT; +import static org.apache.hudi.hive.HiveSyncConfigHolder.METASTORE_URIS; /** * Configs needed to sync data into the Hive Metastore. */ public class HiveSyncConfig extends HoodieSyncConfig { - @Parameter(names = {"--user"}, description = "Hive username") - public String hiveUser; - - @Parameter(names = {"--pass"}, description = "Hive password") - public String hivePass; - - @Parameter(names = {"--jdbc-url"}, description = "Hive jdbc connect url") - public String jdbcUrl; - - @Parameter(names = {"--metastore-uris"}, description = "Hive metastore uris") - public String metastoreUris; - - @Parameter(names = {"--use-pre-apache-input-format"}, - description = "Use InputFormat under com.uber.hoodie package " - + "instead of org.apache.hudi package. Use this when you are in the process of migrating from " - + "com.uber.hoodie to org.apache.hudi. Stop using this after you migrated the table definition to " - + "org.apache.hudi input format.") - public Boolean usePreApacheInputFormat; - - @Parameter(names = {"--bucket-spec"}, description = "bucket spec stored in metastore", required = false) - public String bucketSpec; - - @Deprecated - @Parameter(names = {"--use-jdbc"}, description = "Hive jdbc connect url") - public Boolean useJdbc; - - @Parameter(names = {"--sync-mode"}, description = "Mode to choose for Hive ops. Valid values are hms,glue,jdbc and hiveql") - public String syncMode; - - @Parameter(names = {"--auto-create-database"}, description = "Auto create hive database") - public Boolean autoCreateDatabase; - - @Parameter(names = {"--ignore-exceptions"}, description = "Ignore hive exceptions") - public Boolean ignoreExceptions; - - @Parameter(names = {"--skip-ro-suffix"}, description = "Skip the `_ro` suffix for Read optimized table, when registering") - public Boolean skipROSuffix; - - @Parameter(names = {"--table-properties"}, description = "Table properties to hive table") - public String tableProperties; - - @Parameter(names = {"--serde-properties"}, description = "Serde properties to hive table") - public String serdeProperties; - - @Parameter(names = {"--help", "-h"}, help = true) - public Boolean help = false; - - @Parameter(names = {"--support-timestamp"}, description = "'INT64' with original type TIMESTAMP_MICROS is converted to hive 'timestamp' type." - + "Disabled by default for backward compatibility.") - public Boolean supportTimestamp; - - @Parameter(names = {"--managed-table"}, description = "Create a managed table") - public Boolean createManagedTable; - - @Parameter(names = {"--batch-sync-num"}, description = "The number of partitions one batch when synchronous partitions to hive") - public Integer batchSyncNum; - - @Parameter(names = {"--spark-datasource"}, description = "Whether sync this table as spark data source table.") - public Boolean syncAsSparkDataSourceTable; - - @Parameter(names = {"--spark-schema-length-threshold"}, description = "The maximum length allowed in a single cell when storing additional schema information in Hive's metastore.") - public int sparkSchemaLengthThreshold; - - @Parameter(names = {"--with-operation-field"}, description = "Whether to include the '_hoodie_operation' field in the metadata fields") - public Boolean withOperationField = false; - - @Parameter(names = {"--sync-comment"}, description = "synchronize table comments to hive") - public boolean syncComment = false; - - // HIVE SYNC SPECIFIC CONFIGS - // NOTE: DO NOT USE uppercase for the keys as they are internally lower-cased. Using upper-cases causes - // unexpected issues with config getting reset - public static final ConfigProperty HIVE_SYNC_ENABLED = ConfigProperty - .key("hoodie.datasource.hive_sync.enable") - .defaultValue("false") - .withDocumentation("When set to true, register/sync the table to Apache Hive metastore."); - - public static final ConfigProperty HIVE_USER = ConfigProperty - .key("hoodie.datasource.hive_sync.username") - .defaultValue("hive") - .withDocumentation("hive user name to use"); - - public static final ConfigProperty HIVE_PASS = ConfigProperty - .key("hoodie.datasource.hive_sync.password") - .defaultValue("hive") - .withDocumentation("hive password to use"); - - public static final ConfigProperty HIVE_URL = ConfigProperty - .key("hoodie.datasource.hive_sync.jdbcurl") - .defaultValue("jdbc:hive2://localhost:10000") - .withDocumentation("Hive metastore url"); - - public static final ConfigProperty HIVE_USE_PRE_APACHE_INPUT_FORMAT = ConfigProperty - .key("hoodie.datasource.hive_sync.use_pre_apache_input_format") - .defaultValue("false") - .withDocumentation("Flag to choose InputFormat under com.uber.hoodie package instead of org.apache.hudi package. " - + "Use this when you are in the process of migrating from " - + "com.uber.hoodie to org.apache.hudi. Stop using this after you migrated the table definition to org.apache.hudi input format"); - - /** - * @deprecated Use {@link #HIVE_SYNC_MODE} instead of this config from 0.9.0 - */ - @Deprecated - public static final ConfigProperty HIVE_USE_JDBC = ConfigProperty - .key("hoodie.datasource.hive_sync.use_jdbc") - .defaultValue("true") - .deprecatedAfter("0.9.0") - .withDocumentation("Use JDBC when hive synchronization is enabled"); - - public static final ConfigProperty METASTORE_URIS = ConfigProperty - .key("hoodie.datasource.hive_sync.metastore.uris") - .defaultValue("thrift://localhost:9083") - .withDocumentation("Hive metastore url"); - - public static final ConfigProperty HIVE_AUTO_CREATE_DATABASE = ConfigProperty - .key("hoodie.datasource.hive_sync.auto_create_database") - .defaultValue("true") - .withDocumentation("Auto create hive database if does not exists"); - - public static final ConfigProperty HIVE_IGNORE_EXCEPTIONS = ConfigProperty - .key("hoodie.datasource.hive_sync.ignore_exceptions") - .defaultValue("false") - .withDocumentation("Ignore exceptions when syncing with Hive."); - - public static final ConfigProperty HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE = ConfigProperty - .key("hoodie.datasource.hive_sync.skip_ro_suffix") - .defaultValue("false") - .withDocumentation("Skip the _ro suffix for Read optimized table, when registering"); - - public static final ConfigProperty HIVE_SUPPORT_TIMESTAMP_TYPE = ConfigProperty - .key("hoodie.datasource.hive_sync.support_timestamp") - .defaultValue("false") - .withDocumentation("‘INT64’ with original type TIMESTAMP_MICROS is converted to hive ‘timestamp’ type. " - + "Disabled by default for backward compatibility."); - - public static final ConfigProperty HIVE_TABLE_PROPERTIES = ConfigProperty - .key("hoodie.datasource.hive_sync.table_properties") - .noDefaultValue() - .withDocumentation("Additional properties to store with table."); - - public static final ConfigProperty HIVE_TABLE_SERDE_PROPERTIES = ConfigProperty - .key("hoodie.datasource.hive_sync.serde_properties") - .noDefaultValue() - .withDocumentation("Serde properties to hive table."); - - public static final ConfigProperty HIVE_SYNC_AS_DATA_SOURCE_TABLE = ConfigProperty - .key("hoodie.datasource.hive_sync.sync_as_datasource") - .defaultValue("true") - .withDocumentation(""); - - public static final ConfigProperty HIVE_SYNC_SCHEMA_STRING_LENGTH_THRESHOLD = ConfigProperty - .key("hoodie.datasource.hive_sync.schema_string_length_thresh") - .defaultValue(4000) - .withDocumentation(""); - - // Create table as managed table - public static final ConfigProperty HIVE_CREATE_MANAGED_TABLE = ConfigProperty - .key("hoodie.datasource.hive_sync.create_managed_table") - .defaultValue(false) - .withDocumentation("Whether to sync the table as managed table."); - - public static final ConfigProperty HIVE_BATCH_SYNC_PARTITION_NUM = ConfigProperty - .key("hoodie.datasource.hive_sync.batch_num") - .defaultValue(1000) - .withDocumentation("The number of partitions one batch when synchronous partitions to hive."); - - public static final ConfigProperty HIVE_SYNC_MODE = ConfigProperty - .key("hoodie.datasource.hive_sync.mode") - .noDefaultValue() - .withDocumentation("Mode to choose for Hive ops. Valid values are hms, jdbc and hiveql."); - - public static final ConfigProperty HIVE_SYNC_BUCKET_SYNC = ConfigProperty - .key("hoodie.datasource.hive_sync.bucket_sync") - .defaultValue(false) - .withDocumentation("Whether sync hive metastore bucket specification when using bucket index." - + "The specification is 'CLUSTERED BY (trace_id) SORTED BY (trace_id ASC) INTO 65536 BUCKETS'"); - - public static final ConfigProperty HIVE_SYNC_BUCKET_SYNC_SPEC = ConfigProperty - .key("hoodie.datasource.hive_sync.bucket_sync_spec") - .defaultValue("") - .withDocumentation("The hive metastore bucket specification when using bucket index." - + "The specification is 'CLUSTERED BY (trace_id) SORTED BY (trace_id ASC) INTO 65536 BUCKETS'"); + public static String getBucketSpec(String bucketCols, int bucketNum) { + return "CLUSTERED BY (" + bucketCols + " INTO " + bucketNum + " BUCKETS"; + } - public static final ConfigProperty HIVE_SYNC_COMMENT = ConfigProperty - .key("hoodie.datasource.hive_sync.sync_comment") - .defaultValue("false") - .withDocumentation("Whether to sync the table column comments while syncing the table."); + public HiveSyncConfig(Properties props) { + super(props); + } - public HiveSyncConfig() { - this(new TypedProperties()); + public HiveSyncConfig(Properties props, Configuration hadoopConf) { + super(props, hadoopConf); + HiveConf hiveConf = new HiveConf(hadoopConf, HiveConf.class); + // HiveConf needs to load fs conf to allow instantiation via AWSGlueClientFactory + hiveConf.addResource(getHadoopFileSystem().getConf()); + setHadoopConf(hiveConf); } - public HiveSyncConfig(TypedProperties props) { - super(props); - this.hiveUser = getStringOrDefault(HIVE_USER); - this.hivePass = getStringOrDefault(HIVE_PASS); - this.jdbcUrl = getStringOrDefault(HIVE_URL); - this.usePreApacheInputFormat = getBooleanOrDefault(HIVE_USE_PRE_APACHE_INPUT_FORMAT); - this.useJdbc = getBooleanOrDefault(HIVE_USE_JDBC); - this.metastoreUris = getStringOrDefault(METASTORE_URIS); - this.syncMode = getString(HIVE_SYNC_MODE); - this.autoCreateDatabase = getBooleanOrDefault(HIVE_AUTO_CREATE_DATABASE); - this.ignoreExceptions = getBooleanOrDefault(HIVE_IGNORE_EXCEPTIONS); - this.skipROSuffix = getBooleanOrDefault(HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE); - this.tableProperties = getString(HIVE_TABLE_PROPERTIES); - this.serdeProperties = getString(HIVE_TABLE_SERDE_PROPERTIES); - this.supportTimestamp = getBooleanOrDefault(HIVE_SUPPORT_TIMESTAMP_TYPE); - this.batchSyncNum = getIntOrDefault(HIVE_BATCH_SYNC_PARTITION_NUM); - this.syncAsSparkDataSourceTable = getBooleanOrDefault(HIVE_SYNC_AS_DATA_SOURCE_TABLE); - this.sparkSchemaLengthThreshold = getIntOrDefault(HIVE_SYNC_SCHEMA_STRING_LENGTH_THRESHOLD); - this.createManagedTable = getBooleanOrDefault(HIVE_CREATE_MANAGED_TABLE); - this.bucketSpec = getStringOrDefault(HIVE_SYNC_BUCKET_SYNC_SPEC); - this.syncComment = getBooleanOrDefault(HIVE_SYNC_COMMENT); + public HiveConf getHiveConf() { + return (HiveConf) getHadoopConf(); } - @Override - public String toString() { - return "HiveSyncConfig{" - + "databaseName='" + databaseName + '\'' - + ", tableName='" + tableName + '\'' - + ", bucketSpec='" + bucketSpec + '\'' - + ", baseFileFormat='" + baseFileFormat + '\'' - + ", hiveUser='" + hiveUser + '\'' - + ", hivePass='" + hivePass + '\'' - + ", jdbcUrl='" + jdbcUrl + '\'' - + ", metastoreUris='" + metastoreUris + '\'' - + ", basePath='" + basePath + '\'' - + ", partitionFields=" + partitionFields - + ", partitionValueExtractorClass='" + partitionValueExtractorClass + '\'' - + ", assumeDatePartitioning=" + assumeDatePartitioning - + ", usePreApacheInputFormat=" + usePreApacheInputFormat - + ", useJdbc=" + useJdbc - + ", autoCreateDatabase=" + autoCreateDatabase - + ", ignoreExceptions=" + ignoreExceptions - + ", skipROSuffix=" + skipROSuffix - + ", useFileListingFromMetadata=" + useFileListingFromMetadata - + ", tableProperties='" + tableProperties + '\'' - + ", serdeProperties='" + serdeProperties + '\'' - + ", help=" + help - + ", supportTimestamp=" + supportTimestamp - + ", decodePartition=" + decodePartition - + ", createManagedTable=" + createManagedTable - + ", syncAsSparkDataSourceTable=" + syncAsSparkDataSourceTable - + ", sparkSchemaLengthThreshold=" + sparkSchemaLengthThreshold - + ", withOperationField=" + withOperationField - + ", isConditionalSync=" + isConditionalSync - + ", sparkVersion=" + sparkVersion - + ", syncComment=" + syncComment - + '}'; + public boolean useBucketSync() { + return getBooleanOrDefault(HIVE_SYNC_BUCKET_SYNC); } - public static String getBucketSpec(String bucketCols, int bucketNum) { - return "CLUSTERED BY (" + bucketCols + " INTO " + bucketNum + " BUCKETS"; + public static class HiveSyncConfigParams { + + @ParametersDelegate() + public final HoodieSyncConfigParams hoodieSyncConfigParams = new HoodieSyncConfigParams(); + + @Parameter(names = {"--user"}, description = "Hive username") + public String hiveUser; + @Parameter(names = {"--pass"}, description = "Hive password") + public String hivePass; + @Parameter(names = {"--jdbc-url"}, description = "Hive jdbc connect url") + public String jdbcUrl; + @Parameter(names = {"--use-pre-apache-input-format"}, + description = "Use InputFormat under com.uber.hoodie package " + + "instead of org.apache.hudi package. Use this when you are in the process of migrating from " + + "com.uber.hoodie to org.apache.hudi. Stop using this after you migrated the table definition to " + + "org.apache.hudi input format.") + public Boolean usePreApacheInputFormat; + @Deprecated + @Parameter(names = {"--use-jdbc"}, description = "Hive jdbc connect url") + public Boolean useJdbc; + @Parameter(names = {"--metastore-uris"}, description = "Hive metastore uris") + public String metastoreUris; + @Parameter(names = {"--sync-mode"}, description = "Mode to choose for Hive ops. Valid values are hms,glue,jdbc and hiveql") + public String syncMode; + @Parameter(names = {"--auto-create-database"}, description = "Auto create hive database") + public Boolean autoCreateDatabase; + @Parameter(names = {"--ignore-exceptions"}, description = "Ignore hive exceptions") + public Boolean ignoreExceptions; + @Parameter(names = {"--skip-ro-suffix"}, description = "Skip the `_ro` suffix for Read optimized table, when registering") + public Boolean skipROSuffix; + @Parameter(names = {"--table-properties"}, description = "Table properties to hive table") + public String tableProperties; + @Parameter(names = {"--serde-properties"}, description = "Serde properties to hive table") + public String serdeProperties; + @Parameter(names = {"--support-timestamp"}, description = "'INT64' with original type TIMESTAMP_MICROS is converted to hive 'timestamp' type." + + "Disabled by default for backward compatibility.") + public Boolean supportTimestamp; + @Parameter(names = {"--managed-table"}, description = "Create a managed table") + public Boolean createManagedTable; + @Parameter(names = {"--batch-sync-num"}, description = "The number of partitions one batch when synchronous partitions to hive") + public Integer batchSyncNum; + @Parameter(names = {"--spark-datasource"}, description = "Whether sync this table as spark data source table.") + public Boolean syncAsSparkDataSourceTable; + @Parameter(names = {"--spark-schema-length-threshold"}, description = "The maximum length allowed in a single cell when storing additional schema information in Hive's metastore.") + public Integer sparkSchemaLengthThreshold; + @Parameter(names = {"--bucket-sync"}, description = "use bucket sync") + public Boolean bucketSync; + @Parameter(names = {"--bucket-spec"}, description = "bucket spec stored in metastore") + public String bucketSpec; + @Parameter(names = {"--sync-comment"}, description = "synchronize table comments to hive") + public Boolean syncComment; + @Parameter(names = {"--with-operation-field"}, description = "Whether to include the '_hoodie_operation' field in the metadata fields") + public Boolean withOperationField; // TODO remove this as it's not used + + public boolean isHelp() { + return hoodieSyncConfigParams.isHelp(); + } + + public TypedProperties toProps() { + final TypedProperties props = hoodieSyncConfigParams.toProps(); + props.setPropertyIfNonNull(HIVE_USER.key(), hiveUser); + props.setPropertyIfNonNull(HIVE_PASS.key(), hivePass); + props.setPropertyIfNonNull(HIVE_URL.key(), jdbcUrl); + props.setPropertyIfNonNull(HIVE_USE_PRE_APACHE_INPUT_FORMAT.key(), usePreApacheInputFormat); + props.setPropertyIfNonNull(HIVE_USE_JDBC.key(), useJdbc); + props.setPropertyIfNonNull(HIVE_SYNC_MODE.key(), syncMode); + props.setPropertyIfNonNull(METASTORE_URIS.key(), metastoreUris); + props.setPropertyIfNonNull(HIVE_AUTO_CREATE_DATABASE.key(), autoCreateDatabase); + props.setPropertyIfNonNull(HIVE_IGNORE_EXCEPTIONS.key(), ignoreExceptions); + props.setPropertyIfNonNull(HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.key(), skipROSuffix); + props.setPropertyIfNonNull(HIVE_SUPPORT_TIMESTAMP_TYPE.key(), supportTimestamp); + props.setPropertyIfNonNull(HIVE_TABLE_PROPERTIES.key(), tableProperties); + props.setPropertyIfNonNull(HIVE_TABLE_SERDE_PROPERTIES.key(), serdeProperties); + props.setPropertyIfNonNull(HIVE_SYNC_AS_DATA_SOURCE_TABLE.key(), syncAsSparkDataSourceTable); + props.setPropertyIfNonNull(HIVE_SYNC_SCHEMA_STRING_LENGTH_THRESHOLD.key(), sparkSchemaLengthThreshold); + props.setPropertyIfNonNull(HIVE_CREATE_MANAGED_TABLE.key(), createManagedTable); + props.setPropertyIfNonNull(HIVE_BATCH_SYNC_PARTITION_NUM.key(), batchSyncNum); + props.setPropertyIfNonNull(HIVE_SYNC_BUCKET_SYNC.key(), bucketSync); + props.setPropertyIfNonNull(HIVE_SYNC_BUCKET_SYNC_SPEC.key(), bucketSpec); + props.setPropertyIfNonNull(HIVE_SYNC_COMMENT.key(), syncComment); + return props; + } } } diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfigHolder.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfigHolder.java new file mode 100644 index 0000000000000..3877782c92026 --- /dev/null +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfigHolder.java @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.hive; + +import org.apache.hudi.common.config.ConfigProperty; + +public class HiveSyncConfigHolder { + /* + * NOTE: below are HIVE SYNC SPECIFIC CONFIGS which should be under HiveSyncConfig.java + * But since DataSourceOptions.scala references constants to some of these, and HiveSyncConfig.java imports HiveConf, + * it causes HiveConf ClassNotFound issue for loading DataSourceOptions. + * + * NOTE: DO NOT USE uppercase for the keys as they are internally lower-cased. Using upper-cases causes + * unexpected issues with config getting reset + */ + public static final ConfigProperty HIVE_SYNC_ENABLED = ConfigProperty + .key("hoodie.datasource.hive_sync.enable") + .defaultValue("false") + .withDocumentation("When set to true, register/sync the table to Apache Hive metastore."); + public static final ConfigProperty HIVE_USER = ConfigProperty + .key("hoodie.datasource.hive_sync.username") + .defaultValue("hive") + .withDocumentation("hive user name to use"); + public static final ConfigProperty HIVE_PASS = ConfigProperty + .key("hoodie.datasource.hive_sync.password") + .defaultValue("hive") + .withDocumentation("hive password to use"); + public static final ConfigProperty HIVE_URL = ConfigProperty + .key("hoodie.datasource.hive_sync.jdbcurl") + .defaultValue("jdbc:hive2://localhost:10000") + .withDocumentation("Hive metastore url"); + public static final ConfigProperty HIVE_USE_PRE_APACHE_INPUT_FORMAT = ConfigProperty + .key("hoodie.datasource.hive_sync.use_pre_apache_input_format") + .defaultValue("false") + .withDocumentation("Flag to choose InputFormat under com.uber.hoodie package instead of org.apache.hudi package. " + + "Use this when you are in the process of migrating from " + + "com.uber.hoodie to org.apache.hudi. Stop using this after you migrated the table definition to org.apache.hudi input format"); + /** + * @deprecated Use {@link #HIVE_SYNC_MODE} instead of this config from 0.9.0 + */ + @Deprecated + public static final ConfigProperty HIVE_USE_JDBC = ConfigProperty + .key("hoodie.datasource.hive_sync.use_jdbc") + .defaultValue("true") + .deprecatedAfter("0.9.0") + .withDocumentation("Use JDBC when hive synchronization is enabled"); + public static final ConfigProperty METASTORE_URIS = ConfigProperty + .key("hoodie.datasource.hive_sync.metastore.uris") + .defaultValue("thrift://localhost:9083") + .withDocumentation("Hive metastore url"); + public static final ConfigProperty HIVE_AUTO_CREATE_DATABASE = ConfigProperty + .key("hoodie.datasource.hive_sync.auto_create_database") + .defaultValue("true") + .withDocumentation("Auto create hive database if does not exists"); + public static final ConfigProperty HIVE_IGNORE_EXCEPTIONS = ConfigProperty + .key("hoodie.datasource.hive_sync.ignore_exceptions") + .defaultValue("false") + .withDocumentation("Ignore exceptions when syncing with Hive."); + public static final ConfigProperty HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE = ConfigProperty + .key("hoodie.datasource.hive_sync.skip_ro_suffix") + .defaultValue("false") + .withDocumentation("Skip the _ro suffix for Read optimized table, when registering"); + public static final ConfigProperty HIVE_SUPPORT_TIMESTAMP_TYPE = ConfigProperty + .key("hoodie.datasource.hive_sync.support_timestamp") + .defaultValue("false") + .withDocumentation("‘INT64’ with original type TIMESTAMP_MICROS is converted to hive ‘timestamp’ type. " + + "Disabled by default for backward compatibility."); + public static final ConfigProperty HIVE_TABLE_PROPERTIES = ConfigProperty + .key("hoodie.datasource.hive_sync.table_properties") + .noDefaultValue() + .withDocumentation("Additional properties to store with table."); + public static final ConfigProperty HIVE_TABLE_SERDE_PROPERTIES = ConfigProperty + .key("hoodie.datasource.hive_sync.serde_properties") + .noDefaultValue() + .withDocumentation("Serde properties to hive table."); + public static final ConfigProperty HIVE_SYNC_AS_DATA_SOURCE_TABLE = ConfigProperty + .key("hoodie.datasource.hive_sync.sync_as_datasource") + .defaultValue("true") + .withDocumentation(""); + public static final ConfigProperty HIVE_SYNC_SCHEMA_STRING_LENGTH_THRESHOLD = ConfigProperty + .key("hoodie.datasource.hive_sync.schema_string_length_thresh") + .defaultValue(4000) + .withDocumentation(""); + // Create table as managed table + public static final ConfigProperty HIVE_CREATE_MANAGED_TABLE = ConfigProperty + .key("hoodie.datasource.hive_sync.create_managed_table") + .defaultValue(false) + .withDocumentation("Whether to sync the table as managed table."); + public static final ConfigProperty HIVE_BATCH_SYNC_PARTITION_NUM = ConfigProperty + .key("hoodie.datasource.hive_sync.batch_num") + .defaultValue(1000) + .withDocumentation("The number of partitions one batch when synchronous partitions to hive."); + public static final ConfigProperty HIVE_SYNC_MODE = ConfigProperty + .key("hoodie.datasource.hive_sync.mode") + .noDefaultValue() + .withDocumentation("Mode to choose for Hive ops. Valid values are hms, jdbc and hiveql."); + public static final ConfigProperty HIVE_SYNC_BUCKET_SYNC = ConfigProperty + .key("hoodie.datasource.hive_sync.bucket_sync") + .defaultValue(false) + .withDocumentation("Whether sync hive metastore bucket specification when using bucket index." + + "The specification is 'CLUSTERED BY (trace_id) SORTED BY (trace_id ASC) INTO 65536 BUCKETS'"); + public static final ConfigProperty HIVE_SYNC_BUCKET_SYNC_SPEC = ConfigProperty + .key("hoodie.datasource.hive_sync.bucket_sync_spec") + .defaultValue("") + .withDocumentation("The hive metastore bucket specification when using bucket index." + + "The specification is 'CLUSTERED BY (trace_id) SORTED BY (trace_id ASC) INTO 65536 BUCKETS'"); + public static final ConfigProperty HIVE_SYNC_COMMENT = ConfigProperty + .key("hoodie.datasource.hive_sync.sync_comment") + .defaultValue("false") + .withDocumentation("Whether to sync the table column comments while syncing the table."); +} diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java index 4d6fad033b1fd..b763416e8f255 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java @@ -18,37 +18,53 @@ package org.apache.hudi.hive; -import org.apache.hudi.common.config.TypedProperties; -import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.InvalidTableException; import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; -import org.apache.hudi.sync.common.util.ConfigUtils; import org.apache.hudi.hive.util.HiveSchemaUtil; -import org.apache.hudi.sync.common.AbstractSyncHoodieClient.PartitionEvent; -import org.apache.hudi.sync.common.AbstractSyncHoodieClient.PartitionEvent.PartitionEventType; -import org.apache.hudi.sync.common.AbstractSyncTool; +import org.apache.hudi.sync.common.HoodieSyncClient; +import org.apache.hudi.sync.common.HoodieSyncTool; +import org.apache.hudi.sync.common.model.FieldSchema; import org.apache.hudi.sync.common.model.Partition; +import org.apache.hudi.sync.common.model.PartitionEvent; +import org.apache.hudi.sync.common.model.PartitionEvent.PartitionEventType; +import org.apache.hudi.sync.common.util.ConfigUtils; +import org.apache.hudi.sync.common.util.SparkDataSourceTableUtils; import com.beust.jcommander.JCommander; -import org.apache.avro.Schema; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.parquet.schema.MessageType; -import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.stream.Collectors; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_AUTO_CREATE_DATABASE; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_IGNORE_EXCEPTIONS; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_AS_DATA_SOURCE_TABLE; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_COMMENT; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_SCHEMA_STRING_LENGTH_THRESHOLD; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_TABLE_PROPERTIES; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_TABLE_SERDE_PROPERTIES; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USE_PRE_APACHE_INPUT_FORMAT; +import static org.apache.hudi.hive.HiveSyncConfigHolder.METASTORE_URIS; +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; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_CONDITIONAL_SYNC; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_PARTITION_FIELDS; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_SPARK_VERSION; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_TABLE_NAME; +import static org.apache.hudi.sync.common.util.TableUtils.tableId; + /** * Tool to sync a hoodie HDFS table with a hive metastore table. Either use it as a api * HiveSyncTool.syncHoodieTable(HiveSyncConfig) or as a command line java -cp hoodie-hive-sync.jar HiveSyncTool [args] @@ -57,38 +73,34 @@ * partitions incrementally (all the partitions modified since the last commit) */ @SuppressWarnings("WeakerAccess") -public class HiveSyncTool extends AbstractSyncTool implements AutoCloseable { +public class HiveSyncTool extends HoodieSyncTool implements AutoCloseable { private static final Logger LOG = LogManager.getLogger(HiveSyncTool.class); public static final String SUFFIX_SNAPSHOT_TABLE = "_rt"; public static final String SUFFIX_READ_OPTIMIZED_TABLE = "_ro"; - protected HiveSyncConfig hiveSyncConfig; - protected AbstractHiveSyncHoodieClient hoodieHiveClient; - protected String snapshotTableName = null; - protected Option roTableName = null; - - public HiveSyncTool(TypedProperties props, Configuration conf, FileSystem fs) { - this(new HiveSyncConfig(props), new HiveConf(conf, HiveConf.class), fs); - } - - public HiveSyncTool(HiveSyncConfig hiveSyncConfig, HiveConf hiveConf, FileSystem fs) { - super(hiveSyncConfig.getProps(), hiveConf, fs); - // TODO: reconcile the way to set METASTOREURIS - if (StringUtils.isNullOrEmpty(hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname))) { - hiveConf.set(HiveConf.ConfVars.METASTOREURIS.varname, hiveSyncConfig.metastoreUris); - } - // HiveConf needs to load fs conf to allow instantiation via AWSGlueClientFactory - hiveConf.addResource(fs.getConf()); - initClient(hiveSyncConfig, hiveConf); - initConfig(hiveSyncConfig); + protected final HiveSyncConfig config; + protected final String databaseName; + protected final String tableName; + protected HoodieSyncClient syncClient; + protected String snapshotTableName; + protected Option roTableName; + + public HiveSyncTool(Properties props, Configuration hadoopConf) { + super(props, hadoopConf); + HiveSyncConfig config = new HiveSyncConfig(props, hadoopConf); + this.config = config; + this.databaseName = config.getStringOrDefault(META_SYNC_DATABASE_NAME); + this.tableName = config.getString(META_SYNC_TABLE_NAME); + initSyncClient(config); + initTableNameVars(config); } - protected void initClient(HiveSyncConfig hiveSyncConfig, HiveConf hiveConf) { + protected void initSyncClient(HiveSyncConfig config) { try { - this.hoodieHiveClient = new HoodieHiveClient(hiveSyncConfig, hiveConf, fs); + this.syncClient = new HoodieHiveSyncClient(config); } catch (RuntimeException e) { - if (hiveSyncConfig.ignoreExceptions) { + if (config.getBoolean(HIVE_IGNORE_EXCEPTIONS)) { LOG.error("Got runtime exception when hive syncing, but continuing as ignoreExceptions config is set ", e); } else { throw new HoodieHiveSyncException("Got runtime exception when hive syncing", e); @@ -96,28 +108,22 @@ protected void initClient(HiveSyncConfig hiveSyncConfig, HiveConf hiveConf) { } } - private void initConfig(HiveSyncConfig hiveSyncConfig) { - // Set partitionFields to empty, when the NonPartitionedExtractor is used - // TODO: HiveSyncConfig should be responsible for inferring config value - if (NonPartitionedExtractor.class.getName().equals(hiveSyncConfig.partitionValueExtractorClass)) { - LOG.warn("Set partitionFields to empty, since the NonPartitionedExtractor is used"); - hiveSyncConfig.partitionFields = new ArrayList<>(); - } - this.hiveSyncConfig = hiveSyncConfig; - if (hoodieHiveClient != null) { - switch (hoodieHiveClient.getTableType()) { + private void initTableNameVars(HiveSyncConfig config) { + if (syncClient != null) { + switch (syncClient.getTableType()) { case COPY_ON_WRITE: - this.snapshotTableName = hiveSyncConfig.tableName; + this.snapshotTableName = tableName; this.roTableName = Option.empty(); break; case MERGE_ON_READ: - this.snapshotTableName = hiveSyncConfig.tableName + SUFFIX_SNAPSHOT_TABLE; - this.roTableName = hiveSyncConfig.skipROSuffix ? Option.of(hiveSyncConfig.tableName) : - Option.of(hiveSyncConfig.tableName + SUFFIX_READ_OPTIMIZED_TABLE); + this.snapshotTableName = tableName + SUFFIX_SNAPSHOT_TABLE; + this.roTableName = config.getBoolean(HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE) + ? Option.of(tableName) + : Option.of(tableName + SUFFIX_READ_OPTIMIZED_TABLE); break; default: - LOG.error("Unknown table type " + hoodieHiveClient.getTableType()); - throw new InvalidTableException(hoodieHiveClient.getBasePath()); + LOG.error("Unknown table type " + syncClient.getTableType()); + throw new InvalidTableException(syncClient.getBasePath()); } } } @@ -125,21 +131,23 @@ private void initConfig(HiveSyncConfig hiveSyncConfig) { @Override public void syncHoodieTable() { try { - if (hoodieHiveClient != null) { - LOG.info("Syncing target hoodie table with hive table(" + hiveSyncConfig.tableName + "). Hive metastore URL :" - + hiveSyncConfig.jdbcUrl + ", basePath :" + hiveSyncConfig.basePath); + if (syncClient != null) { + LOG.info("Syncing target hoodie table with hive table(" + + tableId(databaseName, tableName) + "). Hive metastore URL :" + + config.getString(METASTORE_URIS) + ", basePath :" + + config.getString(META_SYNC_BASE_PATH)); doSync(); } } catch (RuntimeException re) { - throw new HoodieException("Got runtime exception when hive syncing " + hiveSyncConfig.tableName, re); + throw new HoodieException("Got runtime exception when hive syncing " + tableName, re); } finally { close(); } } protected void doSync() { - switch (hoodieHiveClient.getTableType()) { + switch (syncClient.getTableType()) { case COPY_ON_WRITE: syncHoodieTable(snapshotTableName, false, false); break; @@ -150,61 +158,60 @@ protected void doSync() { syncHoodieTable(snapshotTableName, true, false); break; default: - LOG.error("Unknown table type " + hoodieHiveClient.getTableType()); - throw new InvalidTableException(hoodieHiveClient.getBasePath()); + LOG.error("Unknown table type " + syncClient.getTableType()); + throw new InvalidTableException(syncClient.getBasePath()); } } @Override public void close() { - if (hoodieHiveClient != null) { + if (syncClient != null) { try { - hoodieHiveClient.close(); + syncClient.close(); } catch (Exception e) { throw new HoodieHiveSyncException("Fail to close sync client.", e); } } } - protected void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, - boolean readAsOptimized) { - LOG.info("Trying to sync hoodie table " + tableName + " with base path " + hoodieHiveClient.getBasePath() - + " of type " + hoodieHiveClient.getTableType()); + protected void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, boolean readAsOptimized) { + LOG.info("Trying to sync hoodie table " + tableName + " with base path " + syncClient.getBasePath() + + " of type " + syncClient.getTableType()); // check if the database exists else create it - if (hiveSyncConfig.autoCreateDatabase) { + if (config.getBoolean(HIVE_AUTO_CREATE_DATABASE)) { try { - if (!hoodieHiveClient.databaseExists(hiveSyncConfig.databaseName)) { - hoodieHiveClient.createDatabase(hiveSyncConfig.databaseName); + if (!syncClient.databaseExists(databaseName)) { + syncClient.createDatabase(databaseName); } } catch (Exception e) { // this is harmless since table creation will fail anyways, creation of DB is needed for in-memory testing LOG.warn("Unable to create database", e); } } else { - if (!hoodieHiveClient.databaseExists(hiveSyncConfig.databaseName)) { - LOG.error("Hive database does not exist " + hiveSyncConfig.databaseName); - throw new HoodieHiveSyncException("hive database does not exist " + hiveSyncConfig.databaseName); + if (!syncClient.databaseExists(databaseName)) { + LOG.error("Hive database does not exist " + databaseName); + throw new HoodieHiveSyncException("hive database does not exist " + databaseName); } } // Check if the necessary table exists - boolean tableExists = hoodieHiveClient.tableExists(tableName); + boolean tableExists = syncClient.tableExists(tableName); // check if isDropPartition - boolean isDropPartition = hoodieHiveClient.isDropPartition(); + boolean isDropPartition = syncClient.isDropPartition(); // Get the parquet schema for this table looking at the latest commit - MessageType schema = hoodieHiveClient.getDataSchema(); + MessageType schema = syncClient.getStorageSchema(); // Currently HoodieBootstrapRelation does support reading bootstrap MOR rt table, // so we disable the syncAsSparkDataSourceTable here to avoid read such kind table // by the data source way (which will use the HoodieBootstrapRelation). // TODO after we support bootstrap MOR rt table in HoodieBootstrapRelation[HUDI-2071], we can remove this logical. - if (hoodieHiveClient.isBootstrap() - && hoodieHiveClient.getTableType() == HoodieTableType.MERGE_ON_READ - && !readAsOptimized) { - hiveSyncConfig.syncAsSparkDataSourceTable = false; + if (syncClient.isBootstrap() + && syncClient.getTableType() == HoodieTableType.MERGE_ON_READ + && !readAsOptimized) { + config.setValue(HIVE_SYNC_AS_DATA_SOURCE_TABLE, "false"); } // Sync schema if needed @@ -214,17 +221,17 @@ protected void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, // Get the last time we successfully synced partitions Option lastCommitTimeSynced = Option.empty(); if (tableExists) { - lastCommitTimeSynced = hoodieHiveClient.getLastCommitTimeSynced(tableName); + lastCommitTimeSynced = syncClient.getLastCommitTimeSynced(tableName); } LOG.info("Last commit time synced was found to be " + lastCommitTimeSynced.orElse("null")); - List writtenPartitionsSince = hoodieHiveClient.getPartitionsWrittenToSince(lastCommitTimeSynced); + List writtenPartitionsSince = syncClient.getPartitionsWrittenToSince(lastCommitTimeSynced); LOG.info("Storage partitions scan complete. Found " + writtenPartitionsSince.size()); // Sync the partitions if needed boolean partitionsChanged = syncPartitions(tableName, writtenPartitionsSince, isDropPartition); boolean meetSyncConditions = schemaChanged || partitionsChanged; - if (!hiveSyncConfig.isConditionalSync || meetSyncConditions) { - hoodieHiveClient.updateLastCommitTimeSynced(tableName); + if (!config.getBoolean(META_SYNC_CONDITIONAL_SYNC) || meetSyncConditions) { + syncClient.updateLastCommitTimeSynced(tableName); } LOG.info("Sync complete for " + tableName); } @@ -233,18 +240,18 @@ protected void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, * Get the latest schema from the last commit and check if its in sync with the hive table schema. If not, evolves the * table schema. * - * @param tableExists - does table exist - * @param schema - extracted schema + * @param tableExists does table exist + * @param schema extracted schema */ private boolean syncSchema(String tableName, boolean tableExists, boolean useRealTimeInputFormat, - boolean readAsOptimized, MessageType schema) { + boolean readAsOptimized, MessageType schema) { // Append spark table properties & serde properties - Map tableProperties = ConfigUtils.toMap(hiveSyncConfig.tableProperties); - Map serdeProperties = ConfigUtils.toMap(hiveSyncConfig.serdeProperties); - if (hiveSyncConfig.syncAsSparkDataSourceTable) { - Map sparkTableProperties = getSparkTableProperties(hiveSyncConfig.partitionFields, - hiveSyncConfig.sparkVersion, hiveSyncConfig.sparkSchemaLengthThreshold, schema); - Map sparkSerdeProperties = getSparkSerdeProperties(readAsOptimized, hiveSyncConfig.basePath); + Map tableProperties = ConfigUtils.toMap(config.getString(HIVE_TABLE_PROPERTIES)); + Map serdeProperties = ConfigUtils.toMap(config.getString(HIVE_TABLE_SERDE_PROPERTIES)); + if (config.getBoolean(HIVE_SYNC_AS_DATA_SOURCE_TABLE)) { + Map sparkTableProperties = SparkDataSourceTableUtils.getSparkTableProperties(config.getSplitStrings(META_SYNC_PARTITION_FIELDS), + config.getStringOrDefault(META_SYNC_SPARK_VERSION), config.getIntOrDefault(HIVE_SYNC_SCHEMA_STRING_LENGTH_THRESHOLD), schema); + Map sparkSerdeProperties = SparkDataSourceTableUtils.getSparkSerdeProperties(readAsOptimized, config.getString(META_SYNC_BASE_PATH)); tableProperties.putAll(sparkTableProperties); serdeProperties.putAll(sparkSerdeProperties); } @@ -252,10 +259,10 @@ private boolean syncSchema(String tableName, boolean tableExists, boolean useRea // Check and sync schema if (!tableExists) { LOG.info("Hive table " + tableName + " is not found. Creating it"); - HoodieFileFormat baseFileFormat = HoodieFileFormat.valueOf(hiveSyncConfig.baseFileFormat.toUpperCase()); + HoodieFileFormat baseFileFormat = HoodieFileFormat.valueOf(config.getStringOrDefault(META_SYNC_BASE_FILE_FORMAT).toUpperCase()); String inputFormatClassName = HoodieInputFormatUtils.getInputFormatClassName(baseFileFormat, useRealTimeInputFormat); - if (baseFileFormat.equals(HoodieFileFormat.PARQUET) && hiveSyncConfig.usePreApacheInputFormat) { + if (baseFileFormat.equals(HoodieFileFormat.PARQUET) && config.getBooleanOrDefault(HIVE_USE_PRE_APACHE_INPUT_FORMAT)) { // Parquet input format had an InputFormat class visible under the old naming scheme. inputFormatClassName = useRealTimeInputFormat ? com.uber.hoodie.hadoop.realtime.HoodieRealtimeInputFormat.class.getName() @@ -268,19 +275,20 @@ private boolean syncSchema(String tableName, boolean tableExists, boolean useRea // Custom serde will not work with ALTER TABLE REPLACE COLUMNS // https://github.com/apache/hive/blob/release-1.1.0/ql/src/java/org/apache/hadoop/hive // /ql/exec/DDLTask.java#L3488 - hoodieHiveClient.createTable(tableName, schema, inputFormatClassName, + syncClient.createTable(tableName, schema, inputFormatClassName, outputFormatClassName, serDeFormatClassName, serdeProperties, tableProperties); schemaChanged = true; } else { // Check if the table schema has evolved - Map tableSchema = hoodieHiveClient.getTableSchema(tableName); - SchemaDifference schemaDiff = HiveSchemaUtil.getSchemaDifference(schema, tableSchema, hiveSyncConfig.partitionFields, hiveSyncConfig.supportTimestamp); + Map tableSchema = syncClient.getMetastoreSchema(tableName); + SchemaDifference schemaDiff = HiveSchemaUtil.getSchemaDifference(schema, tableSchema, config.getSplitStrings(META_SYNC_PARTITION_FIELDS), + config.getBooleanOrDefault(HIVE_SUPPORT_TIMESTAMP_TYPE)); if (!schemaDiff.isEmpty()) { LOG.info("Schema difference found for " + tableName); - hoodieHiveClient.updateTableDefinition(tableName, schema); + syncClient.updateTableSchema(tableName, schema); // Sync the table properties if the schema has changed - if (hiveSyncConfig.tableProperties != null || hiveSyncConfig.syncAsSparkDataSourceTable) { - hoodieHiveClient.updateTableProperties(tableName, tableProperties); + if (config.getString(HIVE_TABLE_PROPERTIES) != null || config.getBoolean(HIVE_SYNC_AS_DATA_SOURCE_TABLE)) { + syncClient.updateTableProperties(tableName, tableProperties); LOG.info("Sync table properties for " + tableName + ", table properties is: " + tableProperties); } schemaChanged = true; @@ -289,17 +297,10 @@ private boolean syncSchema(String tableName, boolean tableExists, boolean useRea } } - if (hiveSyncConfig.syncComment) { - Schema avroSchemaWithoutMetadataFields = hoodieHiveClient.getAvroSchemaWithoutMetadataFields(); - Map newComments = avroSchemaWithoutMetadataFields.getFields() - .stream().collect(Collectors.toMap(Schema.Field::name, field -> StringUtils.isNullOrEmpty(field.doc()) ? "" : field.doc())); - boolean allEmpty = newComments.values().stream().allMatch(StringUtils::isNullOrEmpty); - if (!allEmpty) { - List hiveSchema = hoodieHiveClient.getTableCommentUsingMetastoreClient(tableName); - hoodieHiveClient.updateTableComments(tableName, hiveSchema, avroSchemaWithoutMetadataFields.getFields()); - } else { - LOG.info(String.format("No comment %s need to add", tableName)); - } + if (config.getBoolean(HIVE_SYNC_COMMENT)) { + List fromMetastore = syncClient.getMetastoreFieldSchemas(tableName); + List fromStorage = syncClient.getStorageFieldSchemas(); + syncClient.updateTableComments(tableName, fromMetastore, fromStorage); } return schemaChanged; } @@ -311,26 +312,26 @@ private boolean syncSchema(String tableName, boolean tableExists, boolean useRea private boolean syncPartitions(String tableName, List writtenPartitionsSince, boolean isDropPartition) { boolean partitionsChanged; try { - List hivePartitions = hoodieHiveClient.getAllPartitions(tableName); + List hivePartitions = syncClient.getAllPartitions(tableName); List partitionEvents = - hoodieHiveClient.getPartitionEvents(hivePartitions, writtenPartitionsSince, isDropPartition); + syncClient.getPartitionEvents(hivePartitions, writtenPartitionsSince, isDropPartition); List newPartitions = filterPartitions(partitionEvents, PartitionEventType.ADD); if (!newPartitions.isEmpty()) { LOG.info("New Partitions " + newPartitions); - hoodieHiveClient.addPartitionsToTable(tableName, newPartitions); + syncClient.addPartitionsToTable(tableName, newPartitions); } List updatePartitions = filterPartitions(partitionEvents, PartitionEventType.UPDATE); if (!updatePartitions.isEmpty()) { LOG.info("Changed Partitions " + updatePartitions); - hoodieHiveClient.updatePartitionsToTable(tableName, updatePartitions); + syncClient.updatePartitionsToTable(tableName, updatePartitions); } List dropPartitions = filterPartitions(partitionEvents, PartitionEventType.DROP); if (!dropPartitions.isEmpty()) { LOG.info("Drop Partitions " + dropPartitions); - hoodieHiveClient.dropPartitions(tableName, dropPartitions); + syncClient.dropPartitions(tableName, dropPartitions); } partitionsChanged = !updatePartitions.isEmpty() || !newPartitions.isEmpty() || !dropPartitions.isEmpty(); @@ -346,16 +347,13 @@ private List filterPartitions(List events, PartitionEven } public static void main(String[] args) { - // parse the params - final HiveSyncConfig cfg = new HiveSyncConfig(); - JCommander cmd = new JCommander(cfg, null, args); - if (cfg.help || args.length == 0) { + final HiveSyncConfig.HiveSyncConfigParams params = new HiveSyncConfig.HiveSyncConfigParams(); + JCommander cmd = JCommander.newBuilder().addObject(params).build(); + cmd.parse(args); + if (params.isHelp()) { cmd.usage(); - System.exit(1); + System.exit(0); } - FileSystem fs = FSUtils.getFs(cfg.basePath, new Configuration()); - HiveConf hiveConf = new HiveConf(); - hiveConf.addResource(fs.getConf()); - new HiveSyncTool(cfg, hiveConf, fs).syncHoodieTable(); + new HiveSyncTool(params.toProps(), new Configuration()).syncHoodieTable(); } } diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveSyncClient.java similarity index 64% rename from hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java rename to hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveSyncClient.java index 539d18a213d70..d5a85adcbacc2 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveClient.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HoodieHiveSyncClient.java @@ -18,22 +18,21 @@ package org.apache.hudi.hive; +import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; -import org.apache.hudi.common.util.collection.ImmutablePair; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.hive.ddl.DDLExecutor; import org.apache.hudi.hive.ddl.HMSDDLExecutor; import org.apache.hudi.hive.ddl.HiveQueryDDLExecutor; import org.apache.hudi.hive.ddl.HiveSyncMode; import org.apache.hudi.hive.ddl.JDBCExecutor; +import org.apache.hudi.sync.common.HoodieSyncClient; +import org.apache.hudi.sync.common.model.FieldSchema; import org.apache.hudi.sync.common.model.Partition; -import org.apache.avro.Schema; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.FieldSchema; import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.ql.metadata.Hive; @@ -49,115 +48,100 @@ import java.util.stream.Collectors; import static org.apache.hudi.hadoop.utils.HoodieHiveUtils.GLOBALLY_CONSISTENT_READ_TIMESTAMP; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_MODE; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USE_JDBC; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME; import static org.apache.hudi.sync.common.util.TableUtils.tableId; /** * This class implements logic to sync a Hudi table with either the Hive server or the Hive Metastore. */ -public class HoodieHiveClient extends AbstractHiveSyncHoodieClient { +public class HoodieHiveSyncClient extends HoodieSyncClient { - private static final Logger LOG = LogManager.getLogger(HoodieHiveClient.class); + private static final Logger LOG = LogManager.getLogger(HoodieHiveSyncClient.class); + protected final HiveSyncConfig config; + private final String databaseName; DDLExecutor ddlExecutor; private IMetaStoreClient client; - public HoodieHiveClient(HiveSyncConfig cfg, HiveConf configuration, FileSystem fs) { - super(cfg, configuration, fs); + public HoodieHiveSyncClient(HiveSyncConfig config) { + super(config); + this.config = config; + this.databaseName = config.getStringOrDefault(META_SYNC_DATABASE_NAME); // Support JDBC, HiveQL and metastore based implementations for backwards compatibility. Future users should // disable jdbc and depend on metastore client for all hive registrations try { - if (!StringUtils.isNullOrEmpty(cfg.syncMode)) { - HiveSyncMode syncMode = HiveSyncMode.of(cfg.syncMode); + if (!StringUtils.isNullOrEmpty(config.getString(HIVE_SYNC_MODE))) { + HiveSyncMode syncMode = HiveSyncMode.of(config.getString(HIVE_SYNC_MODE)); switch (syncMode) { case HMS: - ddlExecutor = new HMSDDLExecutor(configuration, cfg, fs); + ddlExecutor = new HMSDDLExecutor(config); break; case HIVEQL: - ddlExecutor = new HiveQueryDDLExecutor(cfg, fs, configuration); + ddlExecutor = new HiveQueryDDLExecutor(config); break; case JDBC: - ddlExecutor = new JDBCExecutor(cfg, fs); + ddlExecutor = new JDBCExecutor(config); break; default: - throw new HoodieHiveSyncException("Invalid sync mode given " + cfg.syncMode); + throw new HoodieHiveSyncException("Invalid sync mode given " + config.getString(HIVE_SYNC_MODE)); } } else { - ddlExecutor = cfg.useJdbc ? new JDBCExecutor(cfg, fs) : new HiveQueryDDLExecutor(cfg, fs, configuration); + ddlExecutor = config.getBoolean(HIVE_USE_JDBC) ? new JDBCExecutor(config) : new HiveQueryDDLExecutor(config); } - this.client = Hive.get(configuration).getMSC(); + this.client = Hive.get(config.getHiveConf()).getMSC(); } catch (Exception e) { throw new HoodieHiveSyncException("Failed to create HiveMetaStoreClient", e); } } - /** - * Add the (NEW) partitions to the table. - */ @Override public void addPartitionsToTable(String tableName, List partitionsToAdd) { ddlExecutor.addPartitionsToTable(tableName, partitionsToAdd); } - /** - * Partition path has changed - update the path for te following partitions. - */ @Override public void updatePartitionsToTable(String tableName, List changedPartitions) { ddlExecutor.updatePartitionsToTable(tableName, changedPartitions); } - /** - * Partition path has changed - drop the following partitions. - */ @Override public void dropPartitions(String tableName, List partitionsToDrop) { ddlExecutor.dropPartitionsToTable(tableName, partitionsToDrop); } - /** - * Update the table properties to the table. - */ @Override public void updateTableProperties(String tableName, Map tableProperties) { if (tableProperties == null || tableProperties.isEmpty()) { return; } try { - Table table = client.getTable(syncConfig.databaseName, tableName); + Table table = client.getTable(databaseName, tableName); for (Map.Entry entry : tableProperties.entrySet()) { table.putToParameters(entry.getKey(), entry.getValue()); } - client.alter_table(syncConfig.databaseName, tableName, table); + client.alter_table(databaseName, tableName, table); } catch (Exception e) { throw new HoodieHiveSyncException("Failed to update table properties for table: " + tableName, e); } } - /** - * Scan table partitions. - * - * @deprecated Use {@link #getAllPartitions} instead. - */ - @Deprecated - public List scanTablePartitions(String tableName) throws TException { - return client.listPartitions(syncConfig.databaseName, tableName, (short) -1); - } - @Override - public void updateTableDefinition(String tableName, MessageType newSchema) { + public void updateTableSchema(String tableName, MessageType newSchema) { ddlExecutor.updateTableDefinition(tableName, newSchema); } @Override public List getAllPartitions(String tableName) { try { - return client.listPartitions(syncConfig.databaseName, tableName, (short) -1) + return client.listPartitions(databaseName, tableName, (short) -1) .stream() .map(p -> new Partition(p.getValues(), p.getSd().getLocation())) .collect(Collectors.toList()); } catch (TException e) { - throw new HoodieHiveSyncException("Failed to get all partitions for table " + tableId(syncConfig.databaseName, tableName), e); + throw new HoodieHiveSyncException("Failed to get all partitions for table " + tableId(databaseName, tableName), e); } } @@ -168,11 +152,8 @@ public void createTable(String tableName, MessageType storageSchema, String inpu ddlExecutor.createTable(tableName, storageSchema, inputFormatClass, outputFormatClass, serdeClass, serdeProperties, tableProperties); } - /** - * Get the table schema. - */ @Override - public Map getTableSchema(String tableName) { + public Map getMetastoreSchema(String tableName) { if (!tableExists(tableName)) { throw new IllegalArgumentException( "Failed to get schema for table " + tableName + " does not exist"); @@ -180,26 +161,15 @@ public Map getTableSchema(String tableName) { return ddlExecutor.getTableSchema(tableName); } - @Deprecated - @Override - public boolean doesTableExist(String tableName) { - return tableExists(tableName); - } - @Override public boolean tableExists(String tableName) { try { - return client.tableExists(syncConfig.databaseName, tableName); + return client.tableExists(databaseName, tableName); } catch (TException e) { throw new HoodieHiveSyncException("Failed to check if table exists " + tableName, e); } } - @Deprecated - public boolean doesDataBaseExist(String databaseName) { - return databaseExists(databaseName); - } - @Override public boolean databaseExists(String databaseName) { try { @@ -222,7 +192,7 @@ public void createDatabase(String databaseName) { public Option getLastCommitTimeSynced(String tableName) { // Get the last commit time from the TBLproperties try { - Table table = client.getTable(syncConfig.databaseName, tableName); + Table table = client.getTable(databaseName, tableName); return Option.ofNullable(table.getParameters().getOrDefault(HOODIE_LAST_COMMIT_TIME_SYNC, null)); } catch (Exception e) { throw new HoodieHiveSyncException("Failed to get the last commit time synced from the table " + tableName, e); @@ -232,10 +202,10 @@ public Option getLastCommitTimeSynced(String tableName) { public Option getLastReplicatedTime(String tableName) { // Get the last replicated time from the TBLproperties try { - Table table = client.getTable(syncConfig.databaseName, tableName); + Table table = client.getTable(databaseName, tableName); return Option.ofNullable(table.getParameters().getOrDefault(GLOBALLY_CONSISTENT_READ_TIMESTAMP, null)); } catch (NoSuchObjectException e) { - LOG.warn("the said table not found in hms " + syncConfig.databaseName + "." + tableName); + LOG.warn("the said table not found in hms " + tableId(databaseName, tableName)); return Option.empty(); } catch (Exception e) { throw new HoodieHiveSyncException("Failed to get the last replicated time from the table " + tableName, e); @@ -243,15 +213,14 @@ public Option getLastReplicatedTime(String tableName) { } public void updateLastReplicatedTimeStamp(String tableName, String timeStamp) { - if (!activeTimeline.filterCompletedInstants().getInstants() - .anyMatch(i -> i.getTimestamp().equals(timeStamp))) { + if (getActiveTimeline().getInstants().noneMatch(i -> i.getTimestamp().equals(timeStamp))) { throw new HoodieHiveSyncException( "Not a valid completed timestamp " + timeStamp + " for table " + tableName); } try { - Table table = client.getTable(syncConfig.databaseName, tableName); + Table table = client.getTable(databaseName, tableName); table.putToParameters(GLOBALLY_CONSISTENT_READ_TIMESTAMP, timeStamp); - client.alter_table(syncConfig.databaseName, tableName, table); + client.alter_table(databaseName, tableName, table); } catch (Exception e) { throw new HoodieHiveSyncException( "Failed to update last replicated time to " + timeStamp + " for " + tableName, e); @@ -260,9 +229,9 @@ public void updateLastReplicatedTimeStamp(String tableName, String timeStamp) { public void deleteLastReplicatedTimeStamp(String tableName) { try { - Table table = client.getTable(syncConfig.databaseName, tableName); + Table table = client.getTable(databaseName, tableName); String timestamp = table.getParameters().remove(GLOBALLY_CONSISTENT_READ_TIMESTAMP); - client.alter_table(syncConfig.databaseName, tableName, table); + client.alter_table(databaseName, tableName, table); if (timestamp != null) { LOG.info("deleted last replicated timestamp " + timestamp + " for table " + tableName); } @@ -290,12 +259,12 @@ public void close() { @Override public void updateLastCommitTimeSynced(String tableName) { // Set the last commit time from the TBLproperties - Option lastCommitSynced = activeTimeline.lastInstant().map(HoodieInstant::getTimestamp); + Option lastCommitSynced = getActiveTimeline().lastInstant().map(HoodieInstant::getTimestamp); if (lastCommitSynced.isPresent()) { try { - Table table = client.getTable(syncConfig.databaseName, tableName); + Table table = client.getTable(databaseName, tableName); table.putToParameters(HOODIE_LAST_COMMIT_TIME_SYNC, lastCommitSynced.get()); - client.alter_table(syncConfig.databaseName, tableName, table); + client.alter_table(databaseName, tableName, table); } catch (Exception e) { throw new HoodieHiveSyncException("Failed to get update last commit time synced to " + lastCommitSynced, e); } @@ -303,36 +272,48 @@ public void updateLastCommitTimeSynced(String tableName) { } @Override - public List getTableCommentUsingMetastoreClient(String tableName) { + public List getMetastoreFieldSchemas(String tableName) { try { - return client.getSchema(syncConfig.databaseName, tableName); + return client.getSchema(databaseName, tableName) + .stream() + .map(f -> new FieldSchema(f.getName(), f.getType(), f.getComment())) + .collect(Collectors.toList()); } catch (Exception e) { - throw new HoodieHiveSyncException("Failed to get table comments for : " + tableName, e); + throw new HoodieHiveSyncException("Failed to get field schemas from metastore for table : " + tableName, e); } } @Override - public void updateTableComments(String tableName, List oldSchema, List newSchema) { - Map newComments = newSchema.stream().collect(Collectors.toMap(field -> field.name().toLowerCase(Locale.ROOT), field -> StringUtils.isNullOrEmpty(field.doc()) ? "" : field.doc())); - updateTableComments(tableName,oldSchema,newComments); + public List getStorageFieldSchemas() { + try { + return new TableSchemaResolver(metaClient).getTableAvroSchema(false) + .getFields() + .stream() + .map(f -> new FieldSchema(f.name(), f.schema().getType().getName(), f.doc())) + .collect(Collectors.toList()); + } catch (Exception e) { + throw new HoodieHiveSyncException("Failed to get field schemas from storage : ", e); + } } @Override - public void updateTableComments(String tableName, List oldSchema, Map newComments) { - Map oldComments = oldSchema.stream().collect(Collectors.toMap(fieldSchema -> fieldSchema.getName().toLowerCase(Locale.ROOT), - fieldSchema -> StringUtils.isNullOrEmpty(fieldSchema.getComment()) ? "" : fieldSchema.getComment())); - Map types = oldSchema.stream().collect(Collectors.toMap(FieldSchema::getName, FieldSchema::getType)); - Map> alterComments = new HashMap<>(); - oldComments.forEach((name,comment) -> { - String newComment = newComments.getOrDefault(name,""); - if (!newComment.equals(comment)) { - alterComments.put(name,new ImmutablePair<>(types.get(name),newComment)); + public void updateTableComments(String tableName, List fromMetastore, List fromStorage) { + Map metastoreMap = fromMetastore.stream().collect(Collectors.toMap(f -> f.getName().toLowerCase(Locale.ROOT), f -> f)); + Map storageMap = fromStorage.stream().collect(Collectors.toMap(f -> f.getName().toLowerCase(Locale.ROOT), f -> f)); + Map> alterComments = new HashMap<>(); + metastoreMap.forEach((name, metastoreFieldSchema) -> { + if (storageMap.containsKey(name)) { + boolean updated = metastoreFieldSchema.updateComment(storageMap.get(name)); + if (updated) { + alterComments.put(name, Pair.of(metastoreFieldSchema.getType(), metastoreFieldSchema.getCommentOrEmpty())); + } } }); - if (alterComments.size() > 0) { - ddlExecutor.updateTableComments(tableName, alterComments); + if (alterComments.isEmpty()) { + LOG.info(String.format("No comment difference of %s ", tableName)); } else { - LOG.info(String.format("No comment difference of %s ",tableName)); + ddlExecutor.updateTableComments(tableName, alterComments); } } + } diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/MultiPartKeysValueExtractor.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/MultiPartKeysValueExtractor.java index ae8f63639b768..14d4c9099eeb6 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/MultiPartKeysValueExtractor.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/MultiPartKeysValueExtractor.java @@ -7,19 +7,21 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. */ package org.apache.hudi.hive; import java.util.Collections; import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.sync.common.model.PartitionValueExtractor; import java.util.Arrays; import java.util.List; diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/NonPartitionedExtractor.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/NonPartitionedExtractor.java index dc6243910e132..37b15c6a61bd0 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/NonPartitionedExtractor.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/NonPartitionedExtractor.java @@ -7,17 +7,20 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. */ package org.apache.hudi.hive; +import org.apache.hudi.sync.common.model.PartitionValueExtractor; + import java.util.ArrayList; import java.util.List; diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/SlashEncodedDayPartitionValueExtractor.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/SlashEncodedDayPartitionValueExtractor.java index d6cd3ba24dd74..85fda472754e7 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/SlashEncodedDayPartitionValueExtractor.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/SlashEncodedDayPartitionValueExtractor.java @@ -7,17 +7,20 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. */ package org.apache.hudi.hive; +import org.apache.hudi.sync.common.model.PartitionValueExtractor; + import java.time.LocalDateTime; import java.time.ZoneId; import java.time.ZonedDateTime; diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/SlashEncodedHourPartitionValueExtractor.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/SlashEncodedHourPartitionValueExtractor.java index c8207f582abad..a30296b403d71 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/SlashEncodedHourPartitionValueExtractor.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/SlashEncodedHourPartitionValueExtractor.java @@ -7,17 +7,20 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. */ package org.apache.hudi.hive; +import org.apache.hudi.sync.common.model.PartitionValueExtractor; + import java.time.LocalDateTime; import java.time.ZoneId; import java.time.ZonedDateTime; diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/DDLExecutor.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/DDLExecutor.java index 7b22e56d4538c..dac82911800c1 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/DDLExecutor.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/DDLExecutor.java @@ -18,7 +18,7 @@ package org.apache.hudi.hive.ddl; -import org.apache.hudi.common.util.collection.ImmutablePair; +import org.apache.hudi.common.util.collection.Pair; import org.apache.parquet.schema.MessageType; @@ -98,5 +98,5 @@ void createTable(String tableName, MessageType storageSchema, String inputFormat * @param tableName * @param newSchema Map key: field name, Map value: [field type, field comment] */ - void updateTableComments(String tableName, Map> newSchema); + void updateTableComments(String tableName, Map> newSchema); } diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HMSDDLExecutor.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HMSDDLExecutor.java index 868f59b4fe6c9..ea01ffe15f01a 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HMSDDLExecutor.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/HMSDDLExecutor.java @@ -20,17 +20,15 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.fs.StorageSchemes; -import org.apache.hudi.common.util.collection.ImmutablePair; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.hive.HiveSyncConfig; import org.apache.hudi.hive.HoodieHiveSyncException; -import org.apache.hudi.hive.PartitionValueExtractor; import org.apache.hudi.hive.util.HivePartitionUtil; import org.apache.hudi.hive.util.HiveSchemaUtil; +import org.apache.hudi.sync.common.model.PartitionValueExtractor; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.common.StatsSetupConst; -import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.IMetaStoreClient; import org.apache.hadoop.hive.metastore.TableType; import org.apache.hadoop.hive.metastore.api.Database; @@ -55,26 +53,35 @@ import java.util.Map; import java.util.stream.Collectors; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_CREATE_MANAGED_TABLE; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_BASE_PATH; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_PARTITION_FIELDS; + /** * DDLExecutor impl based on HMS which use HMS apis directly for all DDL tasks. */ public class HMSDDLExecutor implements DDLExecutor { + private static final Logger LOG = LogManager.getLogger(HMSDDLExecutor.class); + private final HiveSyncConfig syncConfig; - private final PartitionValueExtractor partitionValueExtractor; - private final FileSystem fs; + private final String databaseName; private final IMetaStoreClient client; + private final PartitionValueExtractor partitionValueExtractor; - public HMSDDLExecutor(HiveConf conf, HiveSyncConfig syncConfig, FileSystem fs) throws HiveException, MetaException { - this.client = Hive.get(conf).getMSC(); + public HMSDDLExecutor(HiveSyncConfig syncConfig) throws HiveException, MetaException { this.syncConfig = syncConfig; - this.fs = fs; + this.databaseName = syncConfig.getStringOrDefault(META_SYNC_DATABASE_NAME); + this.client = Hive.get(syncConfig.getHiveConf()).getMSC(); try { this.partitionValueExtractor = - (PartitionValueExtractor) Class.forName(syncConfig.partitionValueExtractorClass).newInstance(); + (PartitionValueExtractor) Class.forName(syncConfig.getStringOrDefault(META_SYNC_PARTITION_EXTRACTOR_CLASS)).newInstance(); } catch (Exception e) { throw new HoodieHiveSyncException( - "Failed to initialize PartitionValueExtractor class " + syncConfig.partitionValueExtractorClass, e); + "Failed to initialize PartitionValueExtractor class " + syncConfig.getStringOrDefault(META_SYNC_PARTITION_EXTRACTOR_CLASS), e); } } @@ -93,16 +100,16 @@ public void createDatabase(String databaseName) { public void createTable(String tableName, MessageType storageSchema, String inputFormatClass, String outputFormatClass, String serdeClass, Map serdeProperties, Map tableProperties) { try { - LinkedHashMap mapSchema = HiveSchemaUtil.parquetSchemaToMapSchema(storageSchema, syncConfig.supportTimestamp, false); + LinkedHashMap mapSchema = HiveSchemaUtil.parquetSchemaToMapSchema(storageSchema, syncConfig.getBoolean(HIVE_SUPPORT_TIMESTAMP_TYPE), false); List fieldSchema = HiveSchemaUtil.convertMapSchemaToHiveFieldSchema(mapSchema, syncConfig); - List partitionSchema = syncConfig.partitionFields.stream().map(partitionKey -> { + List partitionSchema = syncConfig.getSplitStrings(META_SYNC_PARTITION_FIELDS).stream().map(partitionKey -> { String partitionKeyType = HiveSchemaUtil.getPartitionKeyType(mapSchema, partitionKey); return new FieldSchema(partitionKey, partitionKeyType.toLowerCase(), ""); }).collect(Collectors.toList()); Table newTb = new Table(); - newTb.setDbName(syncConfig.databaseName); + newTb.setDbName(databaseName); newTb.setTableName(tableName); newTb.setOwner(UserGroupInformation.getCurrentUser().getShortUserName()); newTb.setCreateTime((int) System.currentTimeMillis()); @@ -110,13 +117,13 @@ public void createTable(String tableName, MessageType storageSchema, String inpu storageDescriptor.setCols(fieldSchema); storageDescriptor.setInputFormat(inputFormatClass); storageDescriptor.setOutputFormat(outputFormatClass); - storageDescriptor.setLocation(syncConfig.basePath); + storageDescriptor.setLocation(syncConfig.getString(META_SYNC_BASE_PATH)); serdeProperties.put("serialization.format", "1"); storageDescriptor.setSerdeInfo(new SerDeInfo(null, serdeClass, serdeProperties)); newTb.setSd(storageDescriptor); newTb.setPartitionKeys(partitionSchema); - if (!syncConfig.createManagedTable) { + if (!syncConfig.getBoolean(HIVE_CREATE_MANAGED_TABLE)) { newTb.putToParameters("EXTERNAL", "TRUE"); } @@ -134,9 +141,9 @@ public void createTable(String tableName, MessageType storageSchema, String inpu @Override public void updateTableDefinition(String tableName, MessageType newSchema) { try { - boolean cascade = syncConfig.partitionFields.size() > 0; + boolean cascade = syncConfig.getSplitStrings(META_SYNC_PARTITION_FIELDS).size() > 0; List fieldSchema = HiveSchemaUtil.convertParquetSchemaToHiveFieldSchema(newSchema, syncConfig); - Table table = client.getTable(syncConfig.databaseName, tableName); + Table table = client.getTable(databaseName, tableName); StorageDescriptor sd = table.getSd(); sd.setCols(fieldSchema); table.setSd(sd); @@ -145,7 +152,7 @@ public void updateTableDefinition(String tableName, MessageType newSchema) { LOG.info("partition table,need cascade"); environmentContext.putToProperties(StatsSetupConst.CASCADE, StatsSetupConst.TRUE); } - client.alter_table_with_environmentContext(syncConfig.databaseName, tableName, table, environmentContext); + client.alter_table_with_environmentContext(databaseName, tableName, table, environmentContext); } catch (Exception e) { LOG.error("Failed to update table for " + tableName, e); throw new HoodieHiveSyncException("Failed to update table for " + tableName, e); @@ -158,7 +165,7 @@ public Map getTableSchema(String tableName) { // HiveMetastoreClient returns partition keys separate from Columns, hence get both and merge to // get the Schema of the table. final long start = System.currentTimeMillis(); - Table table = this.client.getTable(syncConfig.databaseName, tableName); + Table table = this.client.getTable(databaseName, tableName); Map partitionKeysMap = table.getPartitionKeys().stream().collect(Collectors.toMap(FieldSchema::getName, f -> f.getType().toUpperCase())); @@ -184,22 +191,22 @@ public void addPartitionsToTable(String tableName, List partitionsToAdd) } LOG.info("Adding partitions " + partitionsToAdd.size() + " to table " + tableName); try { - StorageDescriptor sd = client.getTable(syncConfig.databaseName, tableName).getSd(); + StorageDescriptor sd = client.getTable(databaseName, tableName).getSd(); List partitionList = partitionsToAdd.stream().map(partition -> { StorageDescriptor partitionSd = new StorageDescriptor(); partitionSd.setCols(sd.getCols()); partitionSd.setInputFormat(sd.getInputFormat()); partitionSd.setOutputFormat(sd.getOutputFormat()); partitionSd.setSerdeInfo(sd.getSerdeInfo()); - String fullPartitionPath = FSUtils.getPartitionPath(syncConfig.basePath, partition).toString(); + String fullPartitionPath = FSUtils.getPartitionPath(syncConfig.getString(META_SYNC_BASE_PATH), partition).toString(); List partitionValues = partitionValueExtractor.extractPartitionValuesInPath(partition); partitionSd.setLocation(fullPartitionPath); - return new Partition(partitionValues, syncConfig.databaseName, tableName, 0, 0, partitionSd, null); + return new Partition(partitionValues, databaseName, tableName, 0, 0, partitionSd, null); }).collect(Collectors.toList()); client.add_partitions(partitionList, true, false); } catch (TException e) { - LOG.error(syncConfig.databaseName + "." + tableName + " add partition failed", e); - throw new HoodieHiveSyncException(syncConfig.databaseName + "." + tableName + " add partition failed", e); + LOG.error(databaseName + "." + tableName + " add partition failed", e); + throw new HoodieHiveSyncException(databaseName + "." + tableName + " add partition failed", e); } } @@ -211,20 +218,20 @@ public void updatePartitionsToTable(String tableName, List changedPartit } LOG.info("Changing partitions " + changedPartitions.size() + " on " + tableName); try { - StorageDescriptor sd = client.getTable(syncConfig.databaseName, tableName).getSd(); + StorageDescriptor sd = client.getTable(databaseName, tableName).getSd(); List partitionList = changedPartitions.stream().map(partition -> { - Path partitionPath = FSUtils.getPartitionPath(syncConfig.basePath, partition); + Path partitionPath = FSUtils.getPartitionPath(syncConfig.getString(META_SYNC_BASE_PATH), partition); String partitionScheme = partitionPath.toUri().getScheme(); String fullPartitionPath = StorageSchemes.HDFS.getScheme().equals(partitionScheme) - ? FSUtils.getDFSFullPartitionPath(fs, partitionPath) : partitionPath.toString(); + ? FSUtils.getDFSFullPartitionPath(syncConfig.getHadoopFileSystem(), partitionPath) : partitionPath.toString(); List partitionValues = partitionValueExtractor.extractPartitionValuesInPath(partition); sd.setLocation(fullPartitionPath); - return new Partition(partitionValues, syncConfig.databaseName, tableName, 0, 0, sd, null); + return new Partition(partitionValues, databaseName, tableName, 0, 0, sd, null); }).collect(Collectors.toList()); - client.alter_partitions(syncConfig.databaseName, tableName, partitionList, null); + client.alter_partitions(databaseName, tableName, partitionList, null); } catch (TException e) { - LOG.error(syncConfig.databaseName + "." + tableName + " update partition failed", e); - throw new HoodieHiveSyncException(syncConfig.databaseName + "." + tableName + " update partition failed", e); + LOG.error(databaseName + "." + tableName + " update partition failed", e); + throw new HoodieHiveSyncException(databaseName + "." + tableName + " update partition failed", e); } } @@ -241,20 +248,20 @@ public void dropPartitionsToTable(String tableName, List partitionsToDro if (HivePartitionUtil.partitionExists(client, tableName, dropPartition, partitionValueExtractor, syncConfig)) { String partitionClause = HivePartitionUtil.getPartitionClauseForDrop(dropPartition, partitionValueExtractor, syncConfig); - client.dropPartition(syncConfig.databaseName, tableName, partitionClause, false); + client.dropPartition(databaseName, tableName, partitionClause, false); } LOG.info("Drop partition " + dropPartition + " on " + tableName); } } catch (TException e) { - LOG.error(syncConfig.databaseName + "." + tableName + " drop partition failed", e); - throw new HoodieHiveSyncException(syncConfig.databaseName + "." + tableName + " drop partition failed", e); + LOG.error(databaseName + "." + tableName + " drop partition failed", e); + throw new HoodieHiveSyncException(databaseName + "." + tableName + " drop partition failed", e); } } @Override - public void updateTableComments(String tableName, Map> alterSchema) { + public void updateTableComments(String tableName, Map> alterSchema) { try { - Table table = client.getTable(syncConfig.databaseName, tableName); + Table table = client.getTable(databaseName, tableName); StorageDescriptor sd = new StorageDescriptor(table.getSd()); for (FieldSchema fieldSchema : sd.getCols()) { if (alterSchema.containsKey(fieldSchema.getName())) { @@ -264,7 +271,7 @@ public void updateTableComments(String tableName, Map getTableSchema(String tableName) { // HiveMetastoreClient returns partition keys separate from Columns, hence get both and merge to // get the Schema of the table. final long start = System.currentTimeMillis(); - Table table = metaStoreClient.getTable(config.databaseName, tableName); + Table table = metaStoreClient.getTable(databaseName, tableName); Map partitionKeysMap = table.getPartitionKeys().stream().collect(Collectors.toMap(FieldSchema::getName, f -> f.getType().toUpperCase())); @@ -141,13 +141,13 @@ public void dropPartitionsToTable(String tableName, List partitionsToDro config)) { String partitionClause = HivePartitionUtil.getPartitionClauseForDrop(dropPartition, partitionValueExtractor, config); - metaStoreClient.dropPartition(config.databaseName, tableName, partitionClause, false); + metaStoreClient.dropPartition(databaseName, tableName, partitionClause, false); } LOG.info("Drop partition " + dropPartition + " on " + tableName); } } catch (Exception e) { - LOG.error(config.databaseName + "." + tableName + " drop partition failed", e); - throw new HoodieHiveSyncException(config.databaseName + "." + tableName + " drop partition failed", e); + LOG.error(tableId(databaseName, tableName) + " drop partition failed", e); + throw new HoodieHiveSyncException(tableId(databaseName, tableName) + " drop partition failed", e); } } diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/JDBCExecutor.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/JDBCExecutor.java index 997d6e087c1b0..2673e46a9f745 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/JDBCExecutor.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/JDBCExecutor.java @@ -18,12 +18,9 @@ package org.apache.hudi.hive.ddl; -import static org.apache.hudi.hive.util.HiveSchemaUtil.HIVE_ESCAPE_CHARACTER; - import org.apache.hudi.hive.HiveSyncConfig; import org.apache.hudi.hive.HoodieHiveSyncException; -import org.apache.hadoop.fs.FileSystem; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -39,21 +36,27 @@ import java.util.Map; import java.util.Objects; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_BATCH_SYNC_PARTITION_NUM; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_PASS; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_URL; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USER; +import static org.apache.hudi.hive.util.HiveSchemaUtil.HIVE_ESCAPE_CHARACTER; + /** * This class offers DDL executor backed by the jdbc This class preserves the old useJDBC = true way of doing things. */ public class JDBCExecutor extends QueryBasedDDLExecutor { + private static final Logger LOG = LogManager.getLogger(QueryBasedDDLExecutor.class); - private final HiveSyncConfig config; + private Connection connection; - public JDBCExecutor(HiveSyncConfig config, FileSystem fs) { - super(config, fs); - Objects.requireNonNull(config.jdbcUrl, "--jdbc-url option is required for jdbc sync mode"); - Objects.requireNonNull(config.hiveUser, "--user option is required for jdbc sync mode"); - Objects.requireNonNull(config.hivePass, "--pass option is required for jdbc sync mode"); - this.config = config; - createHiveConnection(config.jdbcUrl, config.hiveUser, config.hivePass); + public JDBCExecutor(HiveSyncConfig config) { + super(config); + Objects.requireNonNull(config.getStringOrDefault(HIVE_URL), "--jdbc-url option is required for jdbc sync mode"); + Objects.requireNonNull(config.getStringOrDefault(HIVE_USER), "--user option is required for jdbc sync mode"); + Objects.requireNonNull(config.getStringOrDefault(HIVE_PASS), "--pass option is required for jdbc sync mode"); + createHiveConnection(config.getStringOrDefault(HIVE_URL), config.getStringOrDefault(HIVE_USER), config.getStringOrDefault(HIVE_PASS)); } @Override @@ -126,7 +129,7 @@ public Map getTableSchema(String tableName) { ResultSet result = null; try { DatabaseMetaData databaseMetaData = connection.getMetaData(); - result = databaseMetaData.getColumns(null, config.databaseName, tableName, null); + result = databaseMetaData.getColumns(null, databaseName, tableName, null); while (result.next()) { String columnName = result.getString(4); String columnType = result.getString(6); @@ -157,11 +160,11 @@ public void dropPartitionsToTable(String tableName, List partitionsToDro } private List constructDropPartitions(String tableName, List partitions) { - if (config.batchSyncNum <= 0) { + if (config.getIntOrDefault(HIVE_BATCH_SYNC_PARTITION_NUM) <= 0) { throw new HoodieHiveSyncException("batch-sync-num for sync hive table must be greater than 0, pls check your parameter"); } List result = new ArrayList<>(); - int batchSyncPartitionNum = config.batchSyncNum; + int batchSyncPartitionNum = config.getIntOrDefault(HIVE_BATCH_SYNC_PARTITION_NUM); StringBuilder alterSQL = getAlterTableDropPrefix(tableName); for (int i = 0; i < partitions.size(); i++) { @@ -186,7 +189,7 @@ private List constructDropPartitions(String tableName, List part public StringBuilder getAlterTableDropPrefix(String tableName) { StringBuilder alterSQL = new StringBuilder("ALTER TABLE "); - alterSQL.append(HIVE_ESCAPE_CHARACTER).append(config.databaseName) + alterSQL.append(HIVE_ESCAPE_CHARACTER).append(databaseName) .append(HIVE_ESCAPE_CHARACTER).append(".").append(HIVE_ESCAPE_CHARACTER) .append(tableName).append(HIVE_ESCAPE_CHARACTER).append(" DROP IF EXISTS "); return alterSQL; @@ -202,4 +205,4 @@ public void close() { LOG.error("Could not close connection ", e); } } -} \ No newline at end of file +} diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/QueryBasedDDLExecutor.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/QueryBasedDDLExecutor.java index d9b663ccb00ca..43dcc0d217c94 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/QueryBasedDDLExecutor.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/ddl/QueryBasedDDLExecutor.java @@ -22,13 +22,12 @@ import org.apache.hudi.common.fs.StorageSchemes; import org.apache.hudi.common.util.PartitionPathEncodeUtils; import org.apache.hudi.common.util.ValidationUtils; -import org.apache.hudi.common.util.collection.ImmutablePair; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.hive.HiveSyncConfig; import org.apache.hudi.hive.HoodieHiveSyncException; -import org.apache.hudi.hive.PartitionValueExtractor; import org.apache.hudi.hive.util.HiveSchemaUtil; +import org.apache.hudi.sync.common.model.PartitionValueExtractor; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -39,26 +38,35 @@ import java.util.List; import java.util.Map; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_BATCH_SYNC_PARTITION_NUM; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE; import static org.apache.hudi.hive.util.HiveSchemaUtil.HIVE_ESCAPE_CHARACTER; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_BASE_PATH; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DECODE_PARTITION; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_PARTITION_FIELDS; /** * This class adds functionality for all query based DDLExecutors. The classes extending it only have to provide runSQL(sql) functions. */ public abstract class QueryBasedDDLExecutor implements DDLExecutor { + private static final Logger LOG = LogManager.getLogger(QueryBasedDDLExecutor.class); - private final HiveSyncConfig config; - public final PartitionValueExtractor partitionValueExtractor; - private final FileSystem fs; - public QueryBasedDDLExecutor(HiveSyncConfig config, FileSystem fs) { - this.fs = fs; + protected final HiveSyncConfig config; + protected final String databaseName; + protected final PartitionValueExtractor partitionValueExtractor; + + public QueryBasedDDLExecutor(HiveSyncConfig config) { this.config = config; + this.databaseName = config.getStringOrDefault(META_SYNC_DATABASE_NAME); try { this.partitionValueExtractor = - (PartitionValueExtractor) Class.forName(config.partitionValueExtractorClass).newInstance(); + (PartitionValueExtractor) Class.forName(config.getStringOrDefault(META_SYNC_PARTITION_EXTRACTOR_CLASS)).newInstance(); } catch (Exception e) { throw new HoodieHiveSyncException( - "Failed to initialize PartitionValueExtractor class " + config.partitionValueExtractorClass, e); + "Failed to initialize PartitionValueExtractor class " + config.getStringOrDefault(META_SYNC_PARTITION_EXTRACTOR_CLASS), e); } } @@ -90,11 +98,11 @@ public void createTable(String tableName, MessageType storageSchema, String inpu @Override public void updateTableDefinition(String tableName, MessageType newSchema) { try { - String newSchemaStr = HiveSchemaUtil.generateSchemaString(newSchema, config.partitionFields, config.supportTimestamp); + String newSchemaStr = HiveSchemaUtil.generateSchemaString(newSchema, config.getSplitStrings(META_SYNC_PARTITION_FIELDS), config.getBoolean(HIVE_SUPPORT_TIMESTAMP_TYPE)); // Cascade clause should not be present for non-partitioned tables - String cascadeClause = config.partitionFields.size() > 0 ? " cascade" : ""; + String cascadeClause = config.getSplitStrings(HIVE_SUPPORT_TIMESTAMP_TYPE).size() > 0 ? " cascade" : ""; StringBuilder sqlBuilder = new StringBuilder("ALTER TABLE ").append(HIVE_ESCAPE_CHARACTER) - .append(config.databaseName).append(HIVE_ESCAPE_CHARACTER).append(".") + .append(databaseName).append(HIVE_ESCAPE_CHARACTER).append(".") .append(HIVE_ESCAPE_CHARACTER).append(tableName) .append(HIVE_ESCAPE_CHARACTER).append(" REPLACE COLUMNS(") .append(newSchemaStr).append(" )").append(cascadeClause); @@ -130,15 +138,15 @@ public void updatePartitionsToTable(String tableName, List changedPartit } @Override - public void updateTableComments(String tableName, Map> newSchema) { - for (Map.Entry> field : newSchema.entrySet()) { + public void updateTableComments(String tableName, Map> newSchema) { + for (Map.Entry> field : newSchema.entrySet()) { String name = field.getKey(); StringBuilder sql = new StringBuilder(); String type = field.getValue().getLeft(); String comment = field.getValue().getRight(); comment = comment.replace("'",""); sql.append("ALTER TABLE ").append(HIVE_ESCAPE_CHARACTER) - .append(config.databaseName).append(HIVE_ESCAPE_CHARACTER).append(".") + .append(databaseName).append(HIVE_ESCAPE_CHARACTER).append(".") .append(HIVE_ESCAPE_CHARACTER).append(tableName) .append(HIVE_ESCAPE_CHARACTER) .append(" CHANGE COLUMN `").append(name).append("` `").append(name) @@ -148,15 +156,15 @@ public void updateTableComments(String tableName, Map constructAddPartitions(String tableName, List partitions) { - if (config.batchSyncNum <= 0) { + if (config.getIntOrDefault(HIVE_BATCH_SYNC_PARTITION_NUM) <= 0) { throw new HoodieHiveSyncException("batch-sync-num for sync hive table must be greater than 0, pls check your parameter"); } List result = new ArrayList<>(); - int batchSyncPartitionNum = config.batchSyncNum; + int batchSyncPartitionNum = config.getIntOrDefault(HIVE_BATCH_SYNC_PARTITION_NUM); StringBuilder alterSQL = getAlterTablePrefix(tableName); for (int i = 0; i < partitions.size(); i++) { String partitionClause = getPartitionClause(partitions.get(i)); - String fullPartitionPath = FSUtils.getPartitionPath(config.basePath, partitions.get(i)).toString(); + String fullPartitionPath = FSUtils.getPartitionPath(config.getString(META_SYNC_BASE_PATH), partitions.get(i)).toString(); alterSQL.append(" PARTITION (").append(partitionClause).append(") LOCATION '").append(fullPartitionPath) .append("' "); if ((i + 1) % batchSyncPartitionNum == 0) { @@ -173,7 +181,7 @@ private List constructAddPartitions(String tableName, List parti private StringBuilder getAlterTablePrefix(String tableName) { StringBuilder alterSQL = new StringBuilder("ALTER TABLE "); - alterSQL.append(HIVE_ESCAPE_CHARACTER).append(config.databaseName) + alterSQL.append(HIVE_ESCAPE_CHARACTER).append(databaseName) .append(HIVE_ESCAPE_CHARACTER).append(".").append(HIVE_ESCAPE_CHARACTER) .append(tableName).append(HIVE_ESCAPE_CHARACTER).append(" ADD IF NOT EXISTS "); return alterSQL; @@ -181,18 +189,18 @@ private StringBuilder getAlterTablePrefix(String tableName) { public String getPartitionClause(String partition) { List partitionValues = partitionValueExtractor.extractPartitionValuesInPath(partition); - ValidationUtils.checkArgument(config.partitionFields.size() == partitionValues.size(), - "Partition key parts " + config.partitionFields + " does not match with partition values " + partitionValues + ValidationUtils.checkArgument(config.getSplitStrings(META_SYNC_PARTITION_FIELDS).size() == partitionValues.size(), + "Partition key parts " + config.getSplitStrings(META_SYNC_PARTITION_FIELDS) + " does not match with partition values " + partitionValues + ". Check partition strategy. "); List partBuilder = new ArrayList<>(); - for (int i = 0; i < config.partitionFields.size(); i++) { + for (int i = 0; i < config.getSplitStrings(META_SYNC_PARTITION_FIELDS).size(); i++) { String partitionValue = partitionValues.get(i); // decode the partition before sync to hive to prevent multiple escapes of HIVE - if (config.decodePartition) { + if (config.getBoolean(META_SYNC_DECODE_PARTITION)) { // This is a decode operator for encode in KeyGenUtils#getRecordPartitionPath partitionValue = PartitionPathEncodeUtils.unescapePathName(partitionValue); } - partBuilder.add("`" + config.partitionFields.get(i) + "`='" + partitionValue + "'"); + partBuilder.add("`" + config.getSplitStrings(META_SYNC_PARTITION_FIELDS).get(i) + "`='" + partitionValue + "'"); } return String.join(",", partBuilder); } @@ -200,15 +208,15 @@ public String getPartitionClause(String partition) { private List constructChangePartitions(String tableName, List partitions) { List changePartitions = new ArrayList<>(); // Hive 2.x doesn't like db.table name for operations, hence we need to change to using the database first - String useDatabase = "USE " + HIVE_ESCAPE_CHARACTER + config.databaseName + HIVE_ESCAPE_CHARACTER; + String useDatabase = "USE " + HIVE_ESCAPE_CHARACTER + databaseName + HIVE_ESCAPE_CHARACTER; changePartitions.add(useDatabase); String alterTable = "ALTER TABLE " + HIVE_ESCAPE_CHARACTER + tableName + HIVE_ESCAPE_CHARACTER; for (String partition : partitions) { String partitionClause = getPartitionClause(partition); - Path partitionPath = FSUtils.getPartitionPath(config.basePath, partition); + Path partitionPath = FSUtils.getPartitionPath(config.getString(META_SYNC_BASE_PATH), partition); String partitionScheme = partitionPath.toUri().getScheme(); String fullPartitionPath = StorageSchemes.HDFS.getScheme().equals(partitionScheme) - ? FSUtils.getDFSFullPartitionPath(fs, partitionPath) : partitionPath.toString(); + ? FSUtils.getDFSFullPartitionPath(config.getHadoopFileSystem(), partitionPath) : partitionPath.toString(); String changePartition = alterTable + " PARTITION (" + partitionClause + ") SET LOCATION '" + fullPartitionPath + "'"; changePartitions.add(changePartition); diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncConfig.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncConfig.java index 16c30a16aabc6..dc4355c57c3d7 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncConfig.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncConfig.java @@ -18,46 +18,44 @@ package org.apache.hudi.hive.replication; +import org.apache.hudi.common.config.ConfigProperty; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.hive.HiveSyncConfig; import com.beust.jcommander.Parameter; +import com.beust.jcommander.ParametersDelegate; +import org.apache.hadoop.conf.Configuration; + +import java.util.Properties; public class GlobalHiveSyncConfig extends HiveSyncConfig { - @Parameter(names = {"--replicated-timestamp"}, description = "Add globally replicated timestamp to enable consistent reads across clusters") - public String globallyReplicatedTimeStamp; - public GlobalHiveSyncConfig() { - } + public static final ConfigProperty META_SYNC_GLOBAL_REPLICATE_TIMESTAMP = ConfigProperty + .key("hoodie.meta_sync.global.replicate.timestamp") + .noDefaultValue() + .withDocumentation(""); - public GlobalHiveSyncConfig(TypedProperties props) { - super(props); + public GlobalHiveSyncConfig(Properties props, Configuration hadoopConf) { + super(props, hadoopConf); } - public static GlobalHiveSyncConfig copy(GlobalHiveSyncConfig cfg) { - GlobalHiveSyncConfig newConfig = new GlobalHiveSyncConfig(cfg.getProps()); - newConfig.basePath = cfg.basePath; - newConfig.assumeDatePartitioning = cfg.assumeDatePartitioning; - newConfig.databaseName = cfg.databaseName; - newConfig.hivePass = cfg.hivePass; - newConfig.hiveUser = cfg.hiveUser; - newConfig.partitionFields = cfg.partitionFields; - newConfig.partitionValueExtractorClass = cfg.partitionValueExtractorClass; - newConfig.jdbcUrl = cfg.jdbcUrl; - newConfig.tableName = cfg.tableName; - newConfig.usePreApacheInputFormat = cfg.usePreApacheInputFormat; - newConfig.useFileListingFromMetadata = cfg.useFileListingFromMetadata; - newConfig.supportTimestamp = cfg.supportTimestamp; - newConfig.decodePartition = cfg.decodePartition; - newConfig.batchSyncNum = cfg.batchSyncNum; - newConfig.globallyReplicatedTimeStamp = cfg.globallyReplicatedTimeStamp; - return newConfig; - } + public static class GlobalHiveSyncConfigParams { + + @ParametersDelegate() + public final HiveSyncConfigParams hiveSyncConfigParams = new HiveSyncConfigParams(); + + @Parameter(names = {"--replicated-timestamp"}, description = "Add globally replicated timestamp to enable consistent reads across clusters") + public String globallyReplicatedTimeStamp; + + public boolean isHelp() { + return hiveSyncConfigParams.isHelp(); + } - @Override - public String toString() { - return "GlobalHiveSyncConfig{" + super.toString() - + " globallyReplicatedTimeStamp=" + globallyReplicatedTimeStamp + "}"; + public TypedProperties toProps() { + final TypedProperties props = hiveSyncConfigParams.toProps(); + props.setPropertyIfNonNull(META_SYNC_GLOBAL_REPLICATE_TIMESTAMP.key(), globallyReplicatedTimeStamp); + return props; + } } } diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncTool.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncTool.java index a7d205962e25c..0198554006778 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncTool.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncTool.java @@ -18,26 +18,28 @@ package org.apache.hudi.hive.replication; -import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.util.Option; import org.apache.hudi.hive.HiveSyncTool; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.hive.conf.HiveConf; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import java.util.HashMap; import java.util.Map; +import java.util.Properties; + +import static org.apache.hudi.hive.replication.GlobalHiveSyncConfig.META_SYNC_GLOBAL_REPLICATE_TIMESTAMP; public class GlobalHiveSyncTool extends HiveSyncTool { - private static final Logger LOG = LogManager.getLogger(HiveSyncTool.class); + private static final Logger LOG = LogManager.getLogger(GlobalHiveSyncTool.class); + protected final GlobalHiveSyncConfig config; - public GlobalHiveSyncTool(GlobalHiveSyncConfig cfg, HiveConf configuration, FileSystem fs) { - super(cfg, configuration, fs); + public GlobalHiveSyncTool(Properties props, Configuration hadoopConf) { + super(props, hadoopConf); + this.config = new GlobalHiveSyncConfig(props, hadoopConf); } @Override @@ -48,19 +50,21 @@ public void syncHoodieTable() { @Override protected void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, boolean readAsOptimized) { super.syncHoodieTable(tableName, useRealtimeInputFormat, readAsOptimized); - if (((GlobalHiveSyncConfig) hiveSyncConfig).globallyReplicatedTimeStamp != null) { - hoodieHiveClient.updateLastReplicatedTimeStamp(tableName, - ((GlobalHiveSyncConfig) hiveSyncConfig).globallyReplicatedTimeStamp); + Option timestamp = Option.ofNullable(config.getString(META_SYNC_GLOBAL_REPLICATE_TIMESTAMP)); + if (timestamp.isPresent()) { + syncClient.updateLastReplicatedTimeStamp(tableName, timestamp.get()); + LOG.info("Sync complete for " + tableName); + } else { + LOG.warn("Sync skipped: " + META_SYNC_GLOBAL_REPLICATE_TIMESTAMP.key() + " is not set."); } - LOG.info("Sync complete for " + tableName); } public Map> getLastReplicatedTimeStampMap() { Map> timeStampMap = new HashMap<>(); - Option timeStamp = hoodieHiveClient.getLastReplicatedTime(snapshotTableName); + Option timeStamp = syncClient.getLastReplicatedTime(snapshotTableName); timeStampMap.put(snapshotTableName, timeStamp); - if (HoodieTableType.MERGE_ON_READ.equals(hoodieHiveClient.getTableType())) { - Option roTimeStamp = hoodieHiveClient.getLastReplicatedTime(roTableName.get()); + if (HoodieTableType.MERGE_ON_READ.equals(syncClient.getTableType())) { + Option roTimeStamp = syncClient.getLastReplicatedTime(roTableName.get()); timeStampMap.put(roTableName.get(), roTimeStamp); } return timeStampMap; @@ -70,18 +74,12 @@ public void setLastReplicatedTimeStamp(Map> timeStampMap) for (String tableName : timeStampMap.keySet()) { Option timestamp = timeStampMap.get(tableName); if (timestamp.isPresent()) { - hoodieHiveClient.updateLastReplicatedTimeStamp(tableName, timestamp.get()); + syncClient.updateLastReplicatedTimeStamp(tableName, timestamp.get()); LOG.info("updated timestamp for " + tableName + " to: " + timestamp.get()); } else { - hoodieHiveClient.deleteLastReplicatedTimeStamp(tableName); + syncClient.deleteLastReplicatedTimeStamp(tableName); LOG.info("deleted timestamp for " + tableName); } } } - - public static GlobalHiveSyncTool buildGlobalHiveSyncTool(GlobalHiveSyncConfig cfg, HiveConf hiveConf) { - FileSystem fs = FSUtils.getFs(cfg.basePath, new Configuration()); - hiveConf.addResource(fs.getConf()); - return new GlobalHiveSyncTool(cfg, hiveConf, fs); - } } diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/HiveSyncGlobalCommitConfig.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/HiveSyncGlobalCommitParams.java similarity index 65% rename from hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/HiveSyncGlobalCommitConfig.java rename to hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/HiveSyncGlobalCommitParams.java index c3dd2af3465be..58188f578e1bf 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/HiveSyncGlobalCommitConfig.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/HiveSyncGlobalCommitParams.java @@ -18,18 +18,22 @@ package org.apache.hudi.hive.replication; -import com.beust.jcommander.Parameter; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.util.StringUtils; +import com.beust.jcommander.Parameter; import com.beust.jcommander.Parameters; -import java.io.File; +import com.beust.jcommander.ParametersDelegate; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; -import java.io.OutputStream; import java.util.Properties; -import org.apache.hudi.common.util.StringUtils; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; + +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_URL; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_BASE_PATH; // TODO: stop extending HiveSyncConfig and take all the variables needed from config file @Parameters(commandDescription = "A tool to sync the hudi table to hive from different clusters. Similar to HiveSyncTool but syncs it to more" @@ -40,9 +44,9 @@ + " The tool tries to be transactional but does not guarantee it. If the sync fails midway in one cluster it will try to roll back the committed " + " timestamp from already successful sync on other clusters but that can also fail." + " The tool does not roll back any synced partitions but only the timestamp.") -public class HiveSyncGlobalCommitConfig extends GlobalHiveSyncConfig { +public class HiveSyncGlobalCommitParams { - private static final Logger LOG = LogManager.getLogger(HiveSyncGlobalCommitConfig.class); + private static final Logger LOG = LogManager.getLogger(HiveSyncGlobalCommitParams.class); public static String LOCAL_HIVE_SITE_URI = "hivesyncglobal.local_hive_site_uri"; public static String REMOTE_HIVE_SITE_URI = "hivesyncglobal.remote_hive_site_uri"; @@ -55,7 +59,14 @@ public class HiveSyncGlobalCommitConfig extends GlobalHiveSyncConfig { "--config-xml-file"}, description = "path to the config file in Hive", required = true) public String configFile; - public Properties properties = new Properties(); + @ParametersDelegate() + public final GlobalHiveSyncConfig.GlobalHiveSyncConfigParams globalHiveSyncConfigParams = new GlobalHiveSyncConfig.GlobalHiveSyncConfigParams(); + + public boolean isHelp() { + return globalHiveSyncConfigParams.isHelp(); + } + + public Properties loadedProps = new Properties(); private boolean finalize = false; @@ -64,33 +75,33 @@ public void load() throws IOException { throw new RuntimeException("trying to modify finalized config"); } finalize = true; - try (InputStream configStream = new FileInputStream(new File(configFile))) { - properties.loadFromXML(configStream); + try (InputStream configStream = new FileInputStream(configFile)) { + loadedProps.loadFromXML(configStream); } - if (StringUtils.isNullOrEmpty(globallyReplicatedTimeStamp)) { + if (StringUtils.isNullOrEmpty(globalHiveSyncConfigParams.globallyReplicatedTimeStamp)) { throw new RuntimeException("globally replicated timestamp not set"); } } - GlobalHiveSyncConfig mkGlobalHiveSyncConfig(boolean forRemote) { - GlobalHiveSyncConfig cfg = GlobalHiveSyncConfig.copy(this); - cfg.basePath = forRemote ? properties.getProperty(REMOTE_BASE_PATH) - : properties.getProperty(LOCAL_BASE_PATH, cfg.basePath); - cfg.jdbcUrl = forRemote ? properties.getProperty(REMOTE_HIVE_SERVER_JDBC_URLS) - : properties.getProperty(LOCAL_HIVE_SERVER_JDBC_URLS, cfg.jdbcUrl); - LOG.info("building hivesync config forRemote: " + forRemote + " " + cfg.jdbcUrl + " " - + cfg.basePath); - return cfg; + Properties mkGlobalHiveSyncProps(boolean forRemote) { + TypedProperties props = new TypedProperties(loadedProps); + props.putAll(globalHiveSyncConfigParams.toProps()); + String basePath = forRemote ? loadedProps.getProperty(REMOTE_BASE_PATH) + : loadedProps.getProperty(LOCAL_BASE_PATH, loadedProps.getProperty(META_SYNC_BASE_PATH.key())); + props.setPropertyIfNonNull(META_SYNC_BASE_PATH.key(), basePath); + String jdbcUrl = forRemote ? loadedProps.getProperty(REMOTE_HIVE_SERVER_JDBC_URLS) + : loadedProps.getProperty(LOCAL_HIVE_SERVER_JDBC_URLS, loadedProps.getProperty(HIVE_URL.key())); + props.setPropertyIfNonNull(HIVE_URL.key(), jdbcUrl); + LOG.info("building hivesync config forRemote: " + forRemote + " " + jdbcUrl + " " + + basePath); + return props; } @Override public String toString() { - return "HiveSyncGlobalCommitConfig{ " + "configFile=" + configFile + ", properties=" - + properties + ", " + super.toString() + return "HiveSyncGlobalCommitParams{ " + "configFile=" + configFile + ", properties=" + + loadedProps + ", " + super.toString() + " }"; } - public void storeToXML(OutputStream configStream) throws IOException { - this.properties.storeToXML(configStream, "hivesync global config"); - } } diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/HiveSyncGlobalCommitTool.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/HiveSyncGlobalCommitTool.java index a194eeb2e96ca..9ca842d159aac 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/HiveSyncGlobalCommitTool.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/HiveSyncGlobalCommitTool.java @@ -18,36 +18,37 @@ package org.apache.hudi.hive.replication; -import static org.apache.hudi.hive.replication.HiveSyncGlobalCommitConfig.LOCAL_HIVE_SITE_URI; -import static org.apache.hudi.hive.replication.HiveSyncGlobalCommitConfig.REMOTE_HIVE_SITE_URI; +import org.apache.hudi.hive.HoodieHiveSyncException; import com.beust.jcommander.JCommander; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; import org.apache.hadoop.fs.Path; import org.apache.hadoop.hive.conf.HiveConf; - -import org.apache.hudi.hive.HoodieHiveSyncException; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import static org.apache.hudi.hive.replication.HiveSyncGlobalCommitParams.LOCAL_HIVE_SITE_URI; +import static org.apache.hudi.hive.replication.HiveSyncGlobalCommitParams.REMOTE_HIVE_SITE_URI; + public class HiveSyncGlobalCommitTool implements HiveSyncGlobalCommit, AutoCloseable { private static final Logger LOG = LogManager.getLogger(HiveSyncGlobalCommitTool.class); - private final HiveSyncGlobalCommitConfig config; - private List replicationStateSyncList; + private final HiveSyncGlobalCommitParams params; + private final List replicationStateSyncList; - private ReplicationStateSync getReplicatedState(boolean forRemote) { + ReplicationStateSync getReplicatedState(boolean forRemote) { HiveConf hiveConf = new HiveConf(); // we probably just need to set the metastore URIs // TODO: figure out how to integrate this in production // how to load balance between piper HMS,HS2 // if we have list of uris, we can do something similar to createHiveConf in reairsync - hiveConf.addResource(new Path(config.properties.getProperty( + hiveConf.addResource(new Path(params.loadedProps.getProperty( forRemote ? REMOTE_HIVE_SITE_URI : LOCAL_HIVE_SITE_URI))); // TODO: get clusterId as input parameters - ReplicationStateSync state = new ReplicationStateSync(config.mkGlobalHiveSyncConfig(forRemote), + ReplicationStateSync state = new ReplicationStateSync(params.mkGlobalHiveSyncProps(forRemote), hiveConf, forRemote ? "REMOTESYNC" : "LOCALSYNC"); return state; } @@ -93,23 +94,24 @@ public boolean rollback() { return true; } - public HiveSyncGlobalCommitTool(HiveSyncGlobalCommitConfig config) { - this.config = config; + public HiveSyncGlobalCommitTool(HiveSyncGlobalCommitParams params) { + this.params = params; this.replicationStateSyncList = new ArrayList<>(2); this.replicationStateSyncList.add(getReplicatedState(false)); this.replicationStateSyncList.add(getReplicatedState(true)); } - private static HiveSyncGlobalCommitConfig getHiveSyncGlobalCommitConfig(String[] args) + private static HiveSyncGlobalCommitParams loadParams(String[] args) throws IOException { - HiveSyncGlobalCommitConfig cfg = new HiveSyncGlobalCommitConfig(); - JCommander cmd = new JCommander(cfg, null, args); - if (cfg.help || args.length == 0) { + final HiveSyncGlobalCommitParams params = new HiveSyncGlobalCommitParams(); + JCommander cmd = JCommander.newBuilder().addObject(params).build(); + cmd.parse(args); + if (params.isHelp()) { cmd.usage(); - System.exit(1); + System.exit(0); } - cfg.load(); - return cfg; + params.load(); + return params; } @Override @@ -120,8 +122,8 @@ public void close() { } public static void main(String[] args) throws IOException, HoodieHiveSyncException { - final HiveSyncGlobalCommitConfig cfg = getHiveSyncGlobalCommitConfig(args); - try (final HiveSyncGlobalCommitTool globalCommitTool = new HiveSyncGlobalCommitTool(cfg)) { + final HiveSyncGlobalCommitParams params = loadParams(args); + try (final HiveSyncGlobalCommitTool globalCommitTool = new HiveSyncGlobalCommitTool(params)) { boolean success = globalCommitTool.commit(); if (!success) { if (!globalCommitTool.rollback()) { diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/ReplicationStateSync.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/ReplicationStateSync.java index bf806fe4b134b..c2ed57e6becbe 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/ReplicationStateSync.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/ReplicationStateSync.java @@ -18,31 +18,26 @@ package org.apache.hudi.hive.replication; -import java.util.Map; -import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hudi.common.util.Option; -public class ReplicationStateSync { +import org.apache.hadoop.hive.conf.HiveConf; + +import java.util.Map; +import java.util.Properties; - private GlobalHiveSyncTool globalHiveSyncTool; - private final GlobalHiveSyncConfig globalHiveSyncConfig; - private final HiveConf hiveConf; +public class ReplicationStateSync implements AutoCloseable { + + protected GlobalHiveSyncTool globalHiveSyncTool; private Map> replicatedTimeStampMap; private Map> oldReplicatedTimeStampMap; private final String clusterId; - ReplicationStateSync(GlobalHiveSyncConfig conf, HiveConf hiveConf, String uid) { - this.globalHiveSyncConfig = conf; - this.hiveConf = hiveConf; - initGlobalHiveSyncTool(); + ReplicationStateSync(Properties props, HiveConf hiveConf, String uid) { + globalHiveSyncTool = new GlobalHiveSyncTool(props, hiveConf); replicatedTimeStampMap = globalHiveSyncTool.getLastReplicatedTimeStampMap(); clusterId = uid; } - private void initGlobalHiveSyncTool() { - globalHiveSyncTool = GlobalHiveSyncTool.buildGlobalHiveSyncTool(globalHiveSyncConfig, hiveConf); - } - public void sync() throws Exception { // the cluster maybe down by the time we reach here so we refresh our replication // state right before we set the oldReplicatedTimeStamp to narrow this window. this is a @@ -80,6 +75,7 @@ public String getClusterId() { return clusterId; } + @Override public void close() { if (globalHiveSyncTool != null) { globalHiveSyncTool.close(); diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveMetastoreBasedLockProvider.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/transaction/lock/HiveMetastoreBasedLockProvider.java similarity index 96% rename from hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveMetastoreBasedLockProvider.java rename to hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/transaction/lock/HiveMetastoreBasedLockProvider.java index f5bca97c3d4ac..aeeccf906fe96 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveMetastoreBasedLockProvider.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/transaction/lock/HiveMetastoreBasedLockProvider.java @@ -7,16 +7,17 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. */ -package org.apache.hudi.hive; +package org.apache.hudi.hive.transaction.lock; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.conf.HiveConf; diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/HivePartitionUtil.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/HivePartitionUtil.java index 0258cfc5efa53..8c2c86699b990 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/HivePartitionUtil.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/HivePartitionUtil.java @@ -18,20 +18,26 @@ package org.apache.hudi.hive.util; -import java.util.ArrayList; -import java.util.List; -import org.apache.hadoop.hive.metastore.IMetaStoreClient; -import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; -import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hudi.common.util.PartitionPathEncodeUtils; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.hive.HiveSyncConfig; import org.apache.hudi.hive.HoodieHiveSyncException; -import org.apache.hudi.hive.PartitionValueExtractor; +import org.apache.hudi.sync.common.model.PartitionValueExtractor; + +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; +import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.thrift.TException; +import java.util.ArrayList; +import java.util.List; + +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DECODE_PARTITION; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_PARTITION_FIELDS; + public class HivePartitionUtil { private static final Logger LOG = LogManager.getLogger(HivePartitionUtil.class); @@ -40,18 +46,18 @@ public class HivePartitionUtil { */ public static String getPartitionClauseForDrop(String partition, PartitionValueExtractor partitionValueExtractor, HiveSyncConfig config) { List partitionValues = partitionValueExtractor.extractPartitionValuesInPath(partition); - ValidationUtils.checkArgument(config.partitionFields.size() == partitionValues.size(), - "Partition key parts " + config.partitionFields + " does not match with partition values " + partitionValues + ValidationUtils.checkArgument(config.getSplitStrings(META_SYNC_PARTITION_FIELDS).size() == partitionValues.size(), + "Partition key parts " + config.getSplitStrings(META_SYNC_PARTITION_FIELDS) + " does not match with partition values " + partitionValues + ". Check partition strategy. "); List partBuilder = new ArrayList<>(); - for (int i = 0; i < config.partitionFields.size(); i++) { + for (int i = 0; i < config.getSplitStrings(META_SYNC_PARTITION_FIELDS).size(); i++) { String partitionValue = partitionValues.get(i); // decode the partition before sync to hive to prevent multiple escapes of HIVE - if (config.decodePartition) { + if (config.getBoolean(META_SYNC_DECODE_PARTITION)) { // This is a decode operator for encode in KeyGenUtils#getRecordPartitionPath partitionValue = PartitionPathEncodeUtils.unescapePathName(partitionValue); } - partBuilder.add(config.partitionFields.get(i) + "=" + partitionValue); + partBuilder.add(config.getSplitStrings(META_SYNC_PARTITION_FIELDS).get(i) + "=" + partitionValue); } return String.join("/", partBuilder); } @@ -61,7 +67,7 @@ public static Boolean partitionExists(IMetaStoreClient client, String tableName, Partition newPartition; try { List partitionValues = partitionValueExtractor.extractPartitionValuesInPath(partitionPath); - newPartition = client.getPartition(config.databaseName, tableName, partitionValues); + newPartition = client.getPartition(config.getStringOrDefault(META_SYNC_DATABASE_NAME), tableName, partitionValues); } catch (NoSuchObjectException ignored) { newPartition = null; } catch (TException e) { diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/HiveSchemaUtil.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/HiveSchemaUtil.java index 2d700596f055b..50a6aabd2bbd9 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/HiveSchemaUtil.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/util/HiveSchemaUtil.java @@ -42,6 +42,12 @@ import java.util.Set; import java.util.stream.Collectors; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_CREATE_MANAGED_TABLE; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_BUCKET_SYNC_SPEC; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_PARTITION_FIELDS; + /** * Schema Utilities. */ @@ -156,7 +162,7 @@ public static Map convertParquetSchemaToHiveSchema(MessageType m * @return : Hive Table schema read from parquet file List[FieldSchema] without partitionField */ public static List convertParquetSchemaToHiveFieldSchema(MessageType messageType, HiveSyncConfig syncConfig) throws IOException { - return convertMapSchemaToHiveFieldSchema(parquetSchemaToMapSchema(messageType, syncConfig.supportTimestamp, false), syncConfig); + return convertMapSchemaToHiveFieldSchema(parquetSchemaToMapSchema(messageType, syncConfig.getBoolean(HIVE_SUPPORT_TIMESTAMP_TYPE), false), syncConfig); } /** @@ -202,7 +208,7 @@ public static Map convertMapSchemaToHiveSchema(LinkedHashMap convertMapSchemaToHiveFieldSchema(LinkedHashMap schema, HiveSyncConfig syncConfig) throws IOException { return schema.keySet().stream() .map(key -> new FieldSchema(key, schema.get(key).toLowerCase(), "")) - .filter(field -> !syncConfig.partitionFields.contains(field.getName())) + .filter(field -> !syncConfig.getSplitStrings(META_SYNC_PARTITION_FIELDS).contains(field.getName())) .collect(Collectors.toList()); } @@ -448,11 +454,11 @@ public static String generateSchemaString(MessageType storageSchema, List serdeProperties, Map tableProperties) throws IOException { - Map hiveSchema = convertParquetSchemaToHiveSchema(storageSchema, config.supportTimestamp); - String columns = generateSchemaString(storageSchema, config.partitionFields, config.supportTimestamp); + Map hiveSchema = convertParquetSchemaToHiveSchema(storageSchema, config.getBoolean(HIVE_SUPPORT_TIMESTAMP_TYPE)); + String columns = generateSchemaString(storageSchema, config.getSplitStrings(META_SYNC_PARTITION_FIELDS), config.getBoolean(HIVE_SUPPORT_TIMESTAMP_TYPE)); List partitionFields = new ArrayList<>(); - for (String partitionKey : config.partitionFields) { + for (String partitionKey : config.getSplitStrings(META_SYNC_PARTITION_FIELDS)) { String partitionKeyWithTicks = tickSurround(partitionKey); partitionFields.add(new StringBuilder().append(partitionKeyWithTicks).append(" ") .append(getPartitionKeyType(hiveSchema, partitionKeyWithTicks)).toString()); @@ -460,26 +466,26 @@ public static String generateCreateDDL(String tableName, MessageType storageSche String partitionsStr = String.join(",", partitionFields); StringBuilder sb = new StringBuilder(); - if (config.createManagedTable) { + if (config.getBoolean(HIVE_CREATE_MANAGED_TABLE)) { sb.append("CREATE TABLE IF NOT EXISTS "); } else { sb.append("CREATE EXTERNAL TABLE IF NOT EXISTS "); } - sb.append(HIVE_ESCAPE_CHARACTER).append(config.databaseName).append(HIVE_ESCAPE_CHARACTER) + sb.append(HIVE_ESCAPE_CHARACTER).append(config.getStringOrDefault(META_SYNC_DATABASE_NAME)).append(HIVE_ESCAPE_CHARACTER) .append(".").append(HIVE_ESCAPE_CHARACTER).append(tableName).append(HIVE_ESCAPE_CHARACTER); sb.append("( ").append(columns).append(")"); - if (!config.partitionFields.isEmpty()) { + if (!config.getSplitStrings(META_SYNC_PARTITION_FIELDS).isEmpty()) { sb.append(" PARTITIONED BY (").append(partitionsStr).append(")"); } - if (config.bucketSpec != null) { - sb.append(' ' + config.bucketSpec + ' '); + if (config.getString(HIVE_SYNC_BUCKET_SYNC_SPEC) != null) { + sb.append(' ' + config.getString(HIVE_SYNC_BUCKET_SYNC_SPEC) + ' '); } sb.append(" ROW FORMAT SERDE '").append(serdeClass).append("'"); if (serdeProperties != null && !serdeProperties.isEmpty()) { sb.append(" WITH SERDEPROPERTIES (").append(propertyToString(serdeProperties)).append(")"); } sb.append(" STORED AS INPUTFORMAT '").append(inputFormatClass).append("'"); - sb.append(" OUTPUTFORMAT '").append(outputFormatClass).append("' LOCATION '").append(config.basePath).append("'"); + sb.append(" OUTPUTFORMAT '").append(outputFormatClass).append("' LOCATION '").append(config.getAbsoluteBasePath()).append("'"); if (tableProperties != null && !tableProperties.isEmpty()) { sb.append(" TBLPROPERTIES(").append(propertyToString(tableProperties)).append(")"); diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncGlobalCommitTool.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncGlobalCommitTool.java deleted file mode 100644 index 937243393f7f0..0000000000000 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncGlobalCommitTool.java +++ /dev/null @@ -1,128 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hudi.hive; - -import static org.apache.hudi.hadoop.utils.HoodieHiveUtils.GLOBALLY_CONSISTENT_READ_TIMESTAMP; -import static org.apache.hudi.hive.replication.HiveSyncGlobalCommitConfig.LOCAL_BASE_PATH; -import static org.apache.hudi.hive.replication.HiveSyncGlobalCommitConfig.LOCAL_HIVE_SERVER_JDBC_URLS; -import static org.apache.hudi.hive.replication.HiveSyncGlobalCommitConfig.LOCAL_HIVE_SITE_URI; -import static org.apache.hudi.hive.replication.HiveSyncGlobalCommitConfig.REMOTE_BASE_PATH; -import static org.apache.hudi.hive.replication.HiveSyncGlobalCommitConfig.REMOTE_HIVE_SERVER_JDBC_URLS; -import static org.apache.hudi.hive.replication.HiveSyncGlobalCommitConfig.REMOTE_HIVE_SITE_URI; - -import java.util.Collections; -import org.apache.hadoop.fs.Path; -import org.apache.hudi.hive.replication.HiveSyncGlobalCommitConfig; -import org.apache.hudi.hive.replication.HiveSyncGlobalCommitTool; -import org.apache.hudi.hive.testutils.TestCluster; -import org.junit.jupiter.api.extension.RegisterExtension; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -public class TestHiveSyncGlobalCommitTool { - - @RegisterExtension - public static TestCluster localCluster = new TestCluster(); - @RegisterExtension - public static TestCluster remoteCluster = new TestCluster(); - - private static String DB_NAME = "foo"; - private static String TBL_NAME = "bar"; - - private HiveSyncGlobalCommitConfig getGlobalCommitConfig( - String commitTime, String dbName, String tblName) throws Exception { - HiveSyncGlobalCommitConfig config = new HiveSyncGlobalCommitConfig(); - config.properties.setProperty(LOCAL_HIVE_SITE_URI, localCluster.getHiveSiteXmlLocation()); - config.properties.setProperty(REMOTE_HIVE_SITE_URI, remoteCluster.getHiveSiteXmlLocation()); - config.properties.setProperty(LOCAL_HIVE_SERVER_JDBC_URLS, localCluster.getHiveJdBcUrl()); - config.properties.setProperty(REMOTE_HIVE_SERVER_JDBC_URLS, remoteCluster.getHiveJdBcUrl()); - config.properties.setProperty(LOCAL_BASE_PATH, localCluster.tablePath(dbName, tblName)); - config.properties.setProperty(REMOTE_BASE_PATH, remoteCluster.tablePath(dbName, tblName)); - config.globallyReplicatedTimeStamp = commitTime; - config.hiveUser = System.getProperty("user.name"); - config.hivePass = ""; - config.databaseName = dbName; - config.tableName = tblName; - config.basePath = localCluster.tablePath(dbName, tblName); - config.assumeDatePartitioning = true; - config.usePreApacheInputFormat = false; - config.partitionFields = Collections.singletonList("datestr"); - return config; - } - - private void compareEqualLastReplicatedTimeStamp(HiveSyncGlobalCommitConfig config) throws Exception { - Assertions.assertEquals(localCluster.getHMSClient() - .getTable(config.databaseName, config.tableName).getParameters() - .get(GLOBALLY_CONSISTENT_READ_TIMESTAMP), remoteCluster.getHMSClient() - .getTable(config.databaseName, config.tableName).getParameters() - .get(GLOBALLY_CONSISTENT_READ_TIMESTAMP), "compare replicated timestamps"); - } - - @BeforeEach - public void setUp() throws Exception { - localCluster.forceCreateDb(DB_NAME); - remoteCluster.forceCreateDb(DB_NAME); - localCluster.dfsCluster.getFileSystem().delete(new Path(localCluster.tablePath(DB_NAME, TBL_NAME)), true); - remoteCluster.dfsCluster.getFileSystem().delete(new Path(remoteCluster.tablePath(DB_NAME, TBL_NAME)), true); - } - - @AfterEach - public void clear() throws Exception { - localCluster.getHMSClient().dropTable(DB_NAME, TBL_NAME); - remoteCluster.getHMSClient().dropTable(DB_NAME, TBL_NAME); - } - - @Test - public void testBasicGlobalCommit() throws Exception { - String commitTime = "100"; - localCluster.createCOWTable(commitTime, 5, DB_NAME, TBL_NAME); - // simulate drs - remoteCluster.createCOWTable(commitTime, 5, DB_NAME, TBL_NAME); - HiveSyncGlobalCommitConfig config = getGlobalCommitConfig(commitTime, DB_NAME, TBL_NAME); - HiveSyncGlobalCommitTool tool = new HiveSyncGlobalCommitTool(config); - Assertions.assertTrue(tool.commit()); - compareEqualLastReplicatedTimeStamp(config); - } - - @Test - public void testBasicRollback() throws Exception { - String commitTime = "100"; - localCluster.createCOWTable(commitTime, 5, DB_NAME, TBL_NAME); - // simulate drs - remoteCluster.createCOWTable(commitTime, 5, DB_NAME, TBL_NAME); - HiveSyncGlobalCommitConfig config = getGlobalCommitConfig(commitTime, DB_NAME, TBL_NAME); - HiveSyncGlobalCommitTool tool = new HiveSyncGlobalCommitTool(config); - Assertions.assertFalse(localCluster.getHMSClient().tableExists(DB_NAME, TBL_NAME)); - Assertions.assertFalse(remoteCluster.getHMSClient().tableExists(DB_NAME, TBL_NAME)); - // stop the remote cluster hive server to simulate cluster going down - remoteCluster.stopHiveServer2(); - Assertions.assertFalse(tool.commit()); - Assertions.assertEquals(commitTime, localCluster.getHMSClient() - .getTable(config.databaseName, config.tableName).getParameters() - .get(GLOBALLY_CONSISTENT_READ_TIMESTAMP)); - Assertions.assertTrue(tool.rollback()); // do a rollback - Assertions.assertNotEquals(commitTime, localCluster.getHMSClient() - .getTable(config.databaseName, config.tableName).getParameters() - .get(GLOBALLY_CONSISTENT_READ_TIMESTAMP)); - Assertions.assertFalse(remoteCluster.getHMSClient().tableExists(DB_NAME, TBL_NAME)); - remoteCluster.startHiveServer2(); - } -} diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java index 167c35a124ab6..072feeb663160 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java @@ -27,19 +27,18 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.ImmutablePair; +import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.hive.testutils.HiveTestUtil; +import org.apache.hudi.sync.common.model.FieldSchema; +import org.apache.hudi.sync.common.model.Partition; +import org.apache.hudi.sync.common.model.PartitionEvent; +import org.apache.hudi.sync.common.model.PartitionEvent.PartitionEventType; import org.apache.hudi.sync.common.util.ConfigUtils; -import org.apache.hudi.sync.common.AbstractSyncHoodieClient.PartitionEvent; -import org.apache.hudi.sync.common.AbstractSyncHoodieClient.PartitionEvent.PartitionEventType; import org.apache.avro.Schema; import org.apache.avro.Schema.Field; import org.apache.hadoop.fs.Path; -import org.apache.hadoop.hive.metastore.api.FieldSchema; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.ql.Driver; -import org.apache.hadoop.hive.ql.metadata.HiveException; import org.apache.hadoop.hive.ql.session.SessionState; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; @@ -53,17 +52,30 @@ import java.time.ZonedDateTime; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Locale; import java.util.Map; import java.util.stream.Collectors; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_AUTO_CREATE_DATABASE; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_CREATE_MANAGED_TABLE; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_IGNORE_EXCEPTIONS; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_AS_DATA_SOURCE_TABLE; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_COMMENT; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_MODE; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_TABLE_PROPERTIES; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_TABLE_SERDE_PROPERTIES; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_URL; import static org.apache.hudi.hive.testutils.HiveTestUtil.basePath; import static org.apache.hudi.hive.testutils.HiveTestUtil.ddlExecutor; -import static org.apache.hudi.hive.testutils.HiveTestUtil.fileSystem; import static org.apache.hudi.hive.testutils.HiveTestUtil.getHiveConf; import static org.apache.hudi.hive.testutils.HiveTestUtil.hiveSyncProps; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_CONDITIONAL_SYNC; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_PARTITION_FIELDS; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -92,7 +104,7 @@ private static Iterable syncModeAndSchemaFromCommitMetadata() { } private HiveSyncTool hiveSyncTool; - private HoodieHiveClient hiveClient; + private HoodieHiveSyncClient hiveClient; @AfterAll public static void cleanUpClass() { @@ -131,7 +143,7 @@ public void teardown() throws Exception { @ParameterizedTest @MethodSource({"syncModeAndSchemaFromCommitMetadata"}) public void testBasicSync(boolean useSchemaFromCommitMetadata, String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + hiveSyncProps.setProperty(HIVE_SYNC_MODE.key(), syncMode); String instantTime = "100"; HiveTestUtil.createCOWTable(instantTime, 5, useSchemaFromCommitMetadata); @@ -144,29 +156,29 @@ public void testBasicSync(boolean useSchemaFromCommitMetadata, String syncMode) assertTrue(hiveClient.tableExists(HiveTestUtil.TABLE_NAME), "Table " + HiveTestUtil.TABLE_NAME + " should exist after sync completes"); - assertEquals(hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(), - hiveClient.getDataSchema().getColumns().size() + 1, + assertEquals(hiveClient.getMetastoreSchema(HiveTestUtil.TABLE_NAME).size(), + hiveClient.getStorageSchema().getColumns().size() + 1, "Hive Schema should match the table schema + partition field"); - assertEquals(5, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + assertEquals(5, hiveClient.getAllPartitions(HiveTestUtil.TABLE_NAME).size(), "Table partitions should match the number of partitions we wrote"); assertEquals(instantTime, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); // Adding of new partitions - List newPartition = Arrays.asList("2050/01/01"); - hiveClient.addPartitionsToTable(HiveTestUtil.TABLE_NAME, Arrays.asList()); - assertEquals(5, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + List newPartition = Collections.singletonList("2050/01/01"); + hiveClient.addPartitionsToTable(HiveTestUtil.TABLE_NAME, Collections.emptyList()); + assertEquals(5, hiveClient.getAllPartitions(HiveTestUtil.TABLE_NAME).size(), "No new partition should be added"); hiveClient.addPartitionsToTable(HiveTestUtil.TABLE_NAME, newPartition); - assertEquals(6, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + assertEquals(6, hiveClient.getAllPartitions(HiveTestUtil.TABLE_NAME).size(), "New partition should be added"); // Update partitions - hiveClient.updatePartitionsToTable(HiveTestUtil.TABLE_NAME, Arrays.asList()); - assertEquals(6, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + hiveClient.updatePartitionsToTable(HiveTestUtil.TABLE_NAME, Collections.emptyList()); + assertEquals(6, hiveClient.getAllPartitions(HiveTestUtil.TABLE_NAME).size(), "Partition count should remain the same"); hiveClient.updatePartitionsToTable(HiveTestUtil.TABLE_NAME, newPartition); - assertEquals(6, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + assertEquals(6, hiveClient.getAllPartitions(HiveTestUtil.TABLE_NAME).size(), "Partition count should remain the same"); // Alter partitions @@ -175,7 +187,7 @@ public void testBasicSync(boolean useSchemaFromCommitMetadata, String syncMode) ddlExecutor.runSQL("ALTER TABLE `" + HiveTestUtil.TABLE_NAME + "` PARTITION (`datestr`='2050-01-01') SET LOCATION '/some/new/location'"); - List hivePartitions = hiveClient.getAllPartitions(HiveTestUtil.TABLE_NAME); + List hivePartitions = hiveClient.getAllPartitions(HiveTestUtil.TABLE_NAME); List writtenPartitionsSince = hiveClient.getPartitionsWrittenToSince(Option.empty()); List partitionEvents = hiveClient.getPartitionEvents(hivePartitions, writtenPartitionsSince, false); assertEquals(1, partitionEvents.size(), "There should be only one partition event"); @@ -186,7 +198,7 @@ public void testBasicSync(boolean useSchemaFromCommitMetadata, String syncMode) reSyncHiveTable(); // Sync should update the changed partition to correct path - List tablePartitions = hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME); + List tablePartitions = hiveClient.getAllPartitions(HiveTestUtil.TABLE_NAME); assertEquals(6, tablePartitions.size(), "The one partition we wrote should be added to hive"); assertEquals(instantTime, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), "The last commit that was synced should be 100"); @@ -195,33 +207,33 @@ public void testBasicSync(boolean useSchemaFromCommitMetadata, String syncMode) @ParameterizedTest @MethodSource({"syncMode"}) public void testSyncDataBase(String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + hiveSyncProps.setProperty(HIVE_SYNC_MODE.key(), syncMode); String instantTime = "100"; HiveTestUtil.createCOWTable(instantTime, 5, true); - hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_DATABASE_NAME.key(), HiveTestUtil.DB_NAME); + hiveSyncProps.setProperty(META_SYNC_DATABASE_NAME.key(), HiveTestUtil.DB_NAME); // while autoCreateDatabase is false and database not exists; - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.key(), "false"); + hiveSyncProps.setProperty(HIVE_AUTO_CREATE_DATABASE.key(), "false"); reinitHiveSyncClient(); // Lets do the sync assertThrows(Exception.class, (this::reSyncHiveTable)); // while autoCreateDatabase is true and database not exists; - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.key(), "true"); + hiveSyncProps.setProperty(HIVE_AUTO_CREATE_DATABASE.key(), "true"); reinitHiveSyncClient(); assertDoesNotThrow((this::reSyncHiveTable)); assertTrue(hiveClient.databaseExists(HiveTestUtil.DB_NAME), "DataBases " + HiveTestUtil.DB_NAME + " should exist after sync completes"); // while autoCreateDatabase is false and database exists; - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.key(), "false"); + hiveSyncProps.setProperty(HIVE_AUTO_CREATE_DATABASE.key(), "false"); reinitHiveSyncClient(); assertDoesNotThrow((this::reSyncHiveTable)); assertTrue(hiveClient.databaseExists(HiveTestUtil.DB_NAME), "DataBases " + HiveTestUtil.DB_NAME + " should exist after sync completes"); // while autoCreateDatabase is true and database exists; - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.key(), "true"); + hiveSyncProps.setProperty(HIVE_AUTO_CREATE_DATABASE.key(), "true"); assertDoesNotThrow((this::reSyncHiveTable)); assertTrue(hiveClient.databaseExists(HiveTestUtil.DB_NAME), "DataBases " + HiveTestUtil.DB_NAME + " should exist after sync completes"); @@ -244,10 +256,10 @@ public void testSyncCOWTableWithProperties(boolean useSchemaFromCommitMetadata, put("tp_1", "p1"); } }; - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_AS_DATA_SOURCE_TABLE.key(), String.valueOf(syncAsDataSourceTable)); - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_TABLE_SERDE_PROPERTIES.key(), ConfigUtils.configToString(serdeProperties)); - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_TABLE_PROPERTIES.key(), ConfigUtils.configToString(tableProperties)); + hiveSyncProps.setProperty(HIVE_SYNC_MODE.key(), syncMode); + hiveSyncProps.setProperty(HIVE_SYNC_AS_DATA_SOURCE_TABLE.key(), String.valueOf(syncAsDataSourceTable)); + hiveSyncProps.setProperty(HIVE_TABLE_SERDE_PROPERTIES.key(), ConfigUtils.configToString(serdeProperties)); + hiveSyncProps.setProperty(HIVE_TABLE_PROPERTIES.key(), ConfigUtils.configToString(tableProperties)); String instantTime = "100"; HiveTestUtil.createCOWTable(instantTime, 5, useSchemaFromCommitMetadata); @@ -335,10 +347,10 @@ public void testSyncMORTableWithProperties(boolean useSchemaFromCommitMetadata, put("tp_1", "p1"); } }; - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_AS_DATA_SOURCE_TABLE.key(), String.valueOf(syncAsDataSourceTable)); - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_TABLE_SERDE_PROPERTIES.key(), ConfigUtils.configToString(serdeProperties)); - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_TABLE_PROPERTIES.key(), ConfigUtils.configToString(tableProperties)); + hiveSyncProps.setProperty(HIVE_SYNC_MODE.key(), syncMode); + hiveSyncProps.setProperty(HIVE_SYNC_AS_DATA_SOURCE_TABLE.key(), String.valueOf(syncAsDataSourceTable)); + hiveSyncProps.setProperty(HIVE_TABLE_SERDE_PROPERTIES.key(), ConfigUtils.configToString(serdeProperties)); + hiveSyncProps.setProperty(HIVE_TABLE_PROPERTIES.key(), ConfigUtils.configToString(tableProperties)); String instantTime = "100"; String deltaCommitTime = "101"; @@ -394,8 +406,8 @@ public void testSyncMORTableWithProperties(boolean useSchemaFromCommitMetadata, public void testSyncManagedTable(boolean useSchemaFromCommitMetadata, boolean isManagedTable, String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_CREATE_MANAGED_TABLE.key(), String.valueOf(isManagedTable)); + hiveSyncProps.setProperty(HIVE_SYNC_MODE.key(), syncMode); + hiveSyncProps.setProperty(HIVE_CREATE_MANAGED_TABLE.key(), String.valueOf(isManagedTable)); String instantTime = "100"; HiveTestUtil.createCOWTable(instantTime, 5, useSchemaFromCommitMetadata); @@ -422,13 +434,13 @@ public void testSyncManagedTable(boolean useSchemaFromCommitMetadata, @ParameterizedTest @MethodSource("syncMode") public void testSyncWithSchema(String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + hiveSyncProps.setProperty(HIVE_SYNC_MODE.key(), syncMode); String commitTime = "100"; HiveTestUtil.createCOWTableWithSchema(commitTime, "/complex.schema.avsc"); reinitHiveSyncClient(); reSyncHiveTable(); - assertEquals(1, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + assertEquals(1, hiveClient.getAllPartitions(HiveTestUtil.TABLE_NAME).size(), "Table partitions should match the number of partitions we wrote"); assertEquals(commitTime, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); @@ -437,12 +449,12 @@ public void testSyncWithSchema(String syncMode) throws Exception { @ParameterizedTest @MethodSource("syncMode") public void testSyncIncremental(String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + hiveSyncProps.setProperty(HIVE_SYNC_MODE.key(), syncMode); String commitTime1 = "100"; HiveTestUtil.createCOWTable(commitTime1, 5, true); reinitHiveSyncClient(); reSyncHiveTable(); - assertEquals(5, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + assertEquals(5, hiveClient.getAllPartitions(HiveTestUtil.TABLE_NAME).size(), "Table partitions should match the number of partitions we wrote"); assertEquals(commitTime1, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); @@ -463,7 +475,7 @@ public void testSyncIncremental(String syncMode) throws Exception { // Sync should add the one partition reSyncHiveTable(); - assertEquals(6, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + assertEquals(6, hiveClient.getAllPartitions(HiveTestUtil.TABLE_NAME).size(), "The one partition we wrote should be added to hive"); assertEquals(commitTime2, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), "The last commit that was synced should be 101"); @@ -472,13 +484,13 @@ public void testSyncIncremental(String syncMode) throws Exception { @ParameterizedTest @MethodSource("syncMode") public void testSyncIncrementalWithSchemaEvolution(String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + hiveSyncProps.setProperty(HIVE_SYNC_MODE.key(), syncMode); String commitTime1 = "100"; HiveTestUtil.createCOWTable(commitTime1, 5, true); reinitHiveSyncClient(); reSyncHiveTable(); - int fields = hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(); + int fields = hiveClient.getMetastoreSchema(HiveTestUtil.TABLE_NAME).size(); // Now lets create more partitions and these are the only ones which needs to be synced ZonedDateTime dateTime = ZonedDateTime.now().plusDays(6); @@ -488,15 +500,15 @@ public void testSyncIncrementalWithSchemaEvolution(String syncMode) throws Excep // Lets do the sync reinitHiveSyncClient(); reSyncHiveTable(); - assertEquals(fields + 3, hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(), + assertEquals(fields + 3, hiveClient.getMetastoreSchema(HiveTestUtil.TABLE_NAME).size(), "Hive Schema has evolved and should not be 3 more field"); - assertEquals("BIGINT", hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).get("favorite_number"), + assertEquals("BIGINT", hiveClient.getMetastoreSchema(HiveTestUtil.TABLE_NAME).get("favorite_number"), "Hive Schema has evolved - Field favorite_number has evolved from int to long"); - assertTrue(hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).containsKey("favorite_movie"), + assertTrue(hiveClient.getMetastoreSchema(HiveTestUtil.TABLE_NAME).containsKey("favorite_movie"), "Hive Schema has evolved - Field favorite_movie was added"); // Sync should add the one partition - assertEquals(6, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + assertEquals(6, hiveClient.getAllPartitions(HiveTestUtil.TABLE_NAME).size(), "The one partition we wrote should be added to hive"); assertEquals(commitTime2, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), "The last commit that was synced should be 101"); @@ -505,13 +517,13 @@ public void testSyncIncrementalWithSchemaEvolution(String syncMode) throws Excep @ParameterizedTest @MethodSource("syncMode") public void testUpdateTableComments(String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + hiveSyncProps.setProperty(HIVE_SYNC_MODE.key(), syncMode); String commitTime = "100"; HiveTestUtil.createCOWTableWithSchema(commitTime, "/simple-test.avsc"); reinitHiveSyncClient(); reSyncHiveTable(); - Map> alterCommentSchema = new HashMap<>(); + Map> alterCommentSchema = new HashMap<>(); //generate commented schema field Schema schema = SchemaTestUtil.getSchemaFromResource(HiveTestUtil.class, "/simple-test.avsc"); Schema commentedSchema = SchemaTestUtil.getSchemaFromResource(HiveTestUtil.class, "/simple-test-doced.avsc"); @@ -521,16 +533,16 @@ public void testUpdateTableComments(String syncMode) throws Exception { String name = field.name().toLowerCase(Locale.ROOT); String comment = fieldsNameAndDoc.get(name); if (fieldsNameAndDoc.containsKey(name) && !comment.equals(field.doc())) { - alterCommentSchema.put(name, new ImmutablePair<>(field.schema().getType().name(),comment)); + alterCommentSchema.put(name, new ImmutablePair<>(field.schema().getType().name(), comment)); } } ddlExecutor.updateTableComments(HiveTestUtil.TABLE_NAME, alterCommentSchema); - List fieldSchemas = hiveClient.getTableCommentUsingMetastoreClient(HiveTestUtil.TABLE_NAME); + List fieldSchemas = hiveClient.getMetastoreFieldSchemas(HiveTestUtil.TABLE_NAME); int commentCnt = 0; for (FieldSchema fieldSchema : fieldSchemas) { - if (!StringUtils.isNullOrEmpty(fieldSchema.getComment())) { + if (StringUtils.nonEmpty(fieldSchema.getCommentOrEmpty())) { commentCnt++; } } @@ -540,29 +552,29 @@ public void testUpdateTableComments(String syncMode) throws Exception { @ParameterizedTest @MethodSource("syncMode") public void testSyncWithCommentedSchema(String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_COMMENT.key(), "false"); + hiveSyncProps.setProperty(HIVE_SYNC_MODE.key(), syncMode); + hiveSyncProps.setProperty(HIVE_SYNC_COMMENT.key(), "false"); String commitTime = "100"; HiveTestUtil.createCOWTableWithSchema(commitTime, "/simple-test-doced.avsc"); reinitHiveSyncClient(); reSyncHiveTable(); - List fieldSchemas = hiveClient.getTableCommentUsingMetastoreClient(HiveTestUtil.TABLE_NAME); + List fieldSchemas = hiveClient.getMetastoreFieldSchemas(HiveTestUtil.TABLE_NAME); int commentCnt = 0; for (FieldSchema fieldSchema : fieldSchemas) { - if (!StringUtils.isNullOrEmpty(fieldSchema.getComment())) { + if (StringUtils.nonEmpty(fieldSchema.getCommentOrEmpty())) { commentCnt++; } } assertEquals(0, commentCnt, "hive schema field comment numbers should match the avro schema field doc numbers"); - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_COMMENT.key(), "true"); + hiveSyncProps.setProperty(HIVE_SYNC_COMMENT.key(), "true"); reinitHiveSyncClient(); reSyncHiveTable(); - fieldSchemas = hiveClient.getTableCommentUsingMetastoreClient(HiveTestUtil.TABLE_NAME); + fieldSchemas = hiveClient.getMetastoreFieldSchemas(HiveTestUtil.TABLE_NAME); commentCnt = 0; for (FieldSchema fieldSchema : fieldSchemas) { - if (!StringUtils.isNullOrEmpty(fieldSchema.getComment())) { + if (StringUtils.nonEmpty(fieldSchema.getCommentOrEmpty())) { commentCnt++; } } @@ -572,7 +584,7 @@ public void testSyncWithCommentedSchema(String syncMode) throws Exception { @ParameterizedTest @MethodSource("syncModeAndSchemaFromCommitMetadata") public void testSyncMergeOnRead(boolean useSchemaFromCommitMetadata, String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + hiveSyncProps.setProperty(HIVE_SYNC_MODE.key(), syncMode); String instantTime = "100"; String deltaCommitTime = "101"; HiveTestUtil.createMORTable(instantTime, deltaCommitTime, 5, true, @@ -587,18 +599,18 @@ public void testSyncMergeOnRead(boolean useSchemaFromCommitMetadata, String sync assertTrue(hiveClient.tableExists(roTableName), "Table " + roTableName + " should exist after sync completes"); if (useSchemaFromCommitMetadata) { - assertEquals(hiveClient.getTableSchema(roTableName).size(), + assertEquals(hiveClient.getMetastoreSchema(roTableName).size(), SchemaTestUtil.getSimpleSchema().getFields().size() + getPartitionFieldSize() + HoodieRecord.HOODIE_META_COLUMNS.size(), "Hive Schema should match the table schema + partition field"); } else { // The data generated and schema in the data file do not have metadata columns, so we need a separate check. - assertEquals(hiveClient.getTableSchema(roTableName).size(), + assertEquals(hiveClient.getMetastoreSchema(roTableName).size(), SchemaTestUtil.getSimpleSchema().getFields().size() + getPartitionFieldSize(), "Hive Schema should match the table schema + partition field"); } - assertEquals(5, hiveClient.scanTablePartitions(roTableName).size(), + assertEquals(5, hiveClient.getAllPartitions(roTableName).size(), "Table partitions should match the number of partitions we wrote"); assertEquals(deltaCommitTime, hiveClient.getLastCommitTimeSynced(roTableName).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); @@ -616,18 +628,18 @@ public void testSyncMergeOnRead(boolean useSchemaFromCommitMetadata, String sync reSyncHiveTable(); if (useSchemaFromCommitMetadata) { - assertEquals(hiveClient.getTableSchema(roTableName).size(), + assertEquals(hiveClient.getMetastoreSchema(roTableName).size(), SchemaTestUtil.getEvolvedSchema().getFields().size() + getPartitionFieldSize() + HoodieRecord.HOODIE_META_COLUMNS.size(), "Hive Schema should match the evolved table schema + partition field"); } else { // The data generated and schema in the data file do not have metadata columns, so we need a separate check. - assertEquals(hiveClient.getTableSchema(roTableName).size(), + assertEquals(hiveClient.getMetastoreSchema(roTableName).size(), SchemaTestUtil.getEvolvedSchema().getFields().size() + getPartitionFieldSize(), "Hive Schema should match the evolved table schema + partition field"); } // Sync should add the one partition - assertEquals(6, hiveClient.scanTablePartitions(roTableName).size(), + assertEquals(6, hiveClient.getAllPartitions(roTableName).size(), "The 2 partitions we wrote should be added to hive"); assertEquals(deltaCommitTime2, hiveClient.getLastCommitTimeSynced(roTableName).get(), "The last commit that was synced should be 103"); @@ -636,7 +648,7 @@ public void testSyncMergeOnRead(boolean useSchemaFromCommitMetadata, String sync @ParameterizedTest @MethodSource("syncModeAndSchemaFromCommitMetadata") public void testSyncMergeOnReadRT(boolean useSchemaFromCommitMetadata, String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + hiveSyncProps.setProperty(HIVE_SYNC_MODE.key(), syncMode); String instantTime = "100"; String deltaCommitTime = "101"; String snapshotTableName = HiveTestUtil.TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE; @@ -654,18 +666,18 @@ public void testSyncMergeOnReadRT(boolean useSchemaFromCommitMetadata, String sy + " should exist after sync completes"); if (useSchemaFromCommitMetadata) { - assertEquals(hiveClient.getTableSchema(snapshotTableName).size(), + assertEquals(hiveClient.getMetastoreSchema(snapshotTableName).size(), SchemaTestUtil.getSimpleSchema().getFields().size() + getPartitionFieldSize() + HoodieRecord.HOODIE_META_COLUMNS.size(), "Hive Schema should match the table schema + partition field"); } else { // The data generated and schema in the data file do not have metadata columns, so we need a separate check. - assertEquals(hiveClient.getTableSchema(snapshotTableName).size(), + assertEquals(hiveClient.getMetastoreSchema(snapshotTableName).size(), SchemaTestUtil.getSimpleSchema().getFields().size() + getPartitionFieldSize(), "Hive Schema should match the table schema + partition field"); } - assertEquals(5, hiveClient.scanTablePartitions(snapshotTableName).size(), + assertEquals(5, hiveClient.getAllPartitions(snapshotTableName).size(), "Table partitions should match the number of partitions we wrote"); assertEquals(deltaCommitTime, hiveClient.getLastCommitTimeSynced(snapshotTableName).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); @@ -682,18 +694,18 @@ public void testSyncMergeOnReadRT(boolean useSchemaFromCommitMetadata, String sy reSyncHiveTable(); if (useSchemaFromCommitMetadata) { - assertEquals(hiveClient.getTableSchema(snapshotTableName).size(), + assertEquals(hiveClient.getMetastoreSchema(snapshotTableName).size(), SchemaTestUtil.getEvolvedSchema().getFields().size() + getPartitionFieldSize() + HoodieRecord.HOODIE_META_COLUMNS.size(), "Hive Schema should match the evolved table schema + partition field"); } else { // The data generated and schema in the data file do not have metadata columns, so we need a separate check. - assertEquals(hiveClient.getTableSchema(snapshotTableName).size(), + assertEquals(hiveClient.getMetastoreSchema(snapshotTableName).size(), SchemaTestUtil.getEvolvedSchema().getFields().size() + getPartitionFieldSize(), "Hive Schema should match the evolved table schema + partition field"); } // Sync should add the one partition - assertEquals(6, hiveClient.scanTablePartitions(snapshotTableName).size(), + assertEquals(6, hiveClient.getAllPartitions(snapshotTableName).size(), "The 2 partitions we wrote should be added to hive"); assertEquals(deltaCommitTime2, hiveClient.getLastCommitTimeSynced(snapshotTableName).get(), "The last commit that was synced should be 103"); @@ -702,12 +714,12 @@ public void testSyncMergeOnReadRT(boolean useSchemaFromCommitMetadata, String sy @ParameterizedTest @MethodSource("syncMode") public void testMultiPartitionKeySync(String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + hiveSyncProps.setProperty(HIVE_SYNC_MODE.key(), syncMode); String instantTime = "100"; HiveTestUtil.createCOWTable(instantTime, 5, true); - hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), MultiPartKeysValueExtractor.class.getCanonicalName()); - hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "year,month,day"); + hiveSyncProps.setProperty(META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), MultiPartKeysValueExtractor.class.getCanonicalName()); + hiveSyncProps.setProperty(META_SYNC_PARTITION_FIELDS.key(), "year,month,day"); HiveTestUtil.getCreatedTablesSet().add(HiveTestUtil.DB_NAME + "." + HiveTestUtil.TABLE_NAME); @@ -718,15 +730,15 @@ public void testMultiPartitionKeySync(String syncMode) throws Exception { reSyncHiveTable(); assertTrue(hiveClient.tableExists(HiveTestUtil.TABLE_NAME), "Table " + HiveTestUtil.TABLE_NAME + " should exist after sync completes"); - assertEquals(hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(), - hiveClient.getDataSchema().getColumns().size() + 3, + assertEquals(hiveClient.getMetastoreSchema(HiveTestUtil.TABLE_NAME).size(), + hiveClient.getStorageSchema().getColumns().size() + 3, "Hive Schema should match the table schema + partition fields"); - assertEquals(5, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + assertEquals(5, hiveClient.getAllPartitions(HiveTestUtil.TABLE_NAME).size(), "Table partitions should match the number of partitions we wrote"); assertEquals(instantTime, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); - // HoodieHiveClient had a bug where partition vals were sorted + // HoodieHiveSyncClient had a bug where partition vals were sorted // and stored as keys in a map. The following tests this particular case. // Now lets create partition "2010/01/02" and followed by "2010/02/01". String commitTime2 = "101"; @@ -742,7 +754,7 @@ public void testMultiPartitionKeySync(String syncMode) throws Exception { reSyncHiveTable(); // Sync should add the one partition - assertEquals(6, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + assertEquals(6, hiveClient.getAllPartitions(HiveTestUtil.TABLE_NAME).size(), "Table partitions should match the number of partitions we wrote"); assertEquals(commitTime2, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), "The last commit that was synced should be 101"); @@ -756,10 +768,10 @@ public void testMultiPartitionKeySync(String syncMode) throws Exception { reSyncHiveTable(); assertTrue(hiveClient.tableExists(HiveTestUtil.TABLE_NAME), "Table " + HiveTestUtil.TABLE_NAME + " should exist after sync completes"); - assertEquals(hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(), - hiveClient.getDataSchema().getColumns().size() + 3, + assertEquals(hiveClient.getMetastoreSchema(HiveTestUtil.TABLE_NAME).size(), + hiveClient.getStorageSchema().getColumns().size() + 3, "Hive Schema should match the table schema + partition fields"); - assertEquals(7, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + assertEquals(7, hiveClient.getAllPartitions(HiveTestUtil.TABLE_NAME).size(), "Table partitions should match the number of partitions we wrote"); assertEquals(commitTime3, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); @@ -769,7 +781,7 @@ public void testMultiPartitionKeySync(String syncMode) throws Exception { @ParameterizedTest @MethodSource("syncMode") public void testDropPartitionKeySync(String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + hiveSyncProps.setProperty(HIVE_SYNC_MODE.key(), syncMode); String instantTime = "100"; HiveTestUtil.createCOWTable(instantTime, 1, true); @@ -782,21 +794,21 @@ public void testDropPartitionKeySync(String syncMode) throws Exception { assertTrue(hiveClient.tableExists(HiveTestUtil.TABLE_NAME), "Table " + HiveTestUtil.TABLE_NAME + " should exist after sync completes"); - assertEquals(hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(), - hiveClient.getDataSchema().getColumns().size() + 1, + assertEquals(hiveClient.getMetastoreSchema(HiveTestUtil.TABLE_NAME).size(), + hiveClient.getStorageSchema().getColumns().size() + 1, "Hive Schema should match the table schema + partition field"); - assertEquals(1, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + assertEquals(1, hiveClient.getAllPartitions(HiveTestUtil.TABLE_NAME).size(), "Table partitions should match the number of partitions we wrote"); assertEquals(instantTime, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); // Adding of new partitions - List newPartition = Arrays.asList("2050/01/01"); - hiveClient.addPartitionsToTable(HiveTestUtil.TABLE_NAME, Arrays.asList()); - assertEquals(1, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + List newPartition = Collections.singletonList("2050/01/01"); + hiveClient.addPartitionsToTable(HiveTestUtil.TABLE_NAME, Collections.emptyList()); + assertEquals(1, hiveClient.getAllPartitions(HiveTestUtil.TABLE_NAME).size(), "No new partition should be added"); hiveClient.addPartitionsToTable(HiveTestUtil.TABLE_NAME, newPartition); - assertEquals(2, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + assertEquals(2, hiveClient.getAllPartitions(HiveTestUtil.TABLE_NAME).size(), "New partition should be added"); reSyncHiveTable(); @@ -805,7 +817,7 @@ public void testDropPartitionKeySync(String syncMode) throws Exception { ddlExecutor.runSQL("ALTER TABLE `" + HiveTestUtil.TABLE_NAME + "` DROP PARTITION (`datestr`='2050-01-01')"); - List hivePartitions = hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME); + List hivePartitions = hiveClient.getAllPartitions(HiveTestUtil.TABLE_NAME); assertEquals(1, hivePartitions.size(), "Table should have 1 partition because of the drop 1 partition"); } @@ -813,7 +825,7 @@ public void testDropPartitionKeySync(String syncMode) throws Exception { @ParameterizedTest @MethodSource("syncMode") public void testDropPartition(String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + hiveSyncProps.setProperty(HIVE_SYNC_MODE.key(), syncMode); String instantTime = "100"; HiveTestUtil.createCOWTable(instantTime, 1, true); @@ -825,15 +837,15 @@ public void testDropPartition(String syncMode) throws Exception { reSyncHiveTable(); assertTrue(hiveClient.tableExists(HiveTestUtil.TABLE_NAME), "Table " + HiveTestUtil.TABLE_NAME + " should exist after sync completes"); - assertEquals(hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(), - hiveClient.getDataSchema().getColumns().size() + 1, + assertEquals(hiveClient.getMetastoreSchema(HiveTestUtil.TABLE_NAME).size(), + hiveClient.getStorageSchema().getColumns().size() + 1, "Hive Schema should match the table schema + partition field"); - List partitions = hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME); + List partitions = hiveClient.getAllPartitions(HiveTestUtil.TABLE_NAME); assertEquals(1, partitions.size(), "Table partitions should match the number of partitions we wrote"); assertEquals(instantTime, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); - String partitiontoDelete = partitions.get(0).getValues().get(0).replace("-","/"); + String partitiontoDelete = partitions.get(0).getValues().get(0).replace("-", "/"); // create a replace commit to delete current partitions+ HiveTestUtil.createReplaceCommit("101", partitiontoDelete, WriteOperationType.DELETE_PARTITION, true, true); @@ -841,7 +853,7 @@ public void testDropPartition(String syncMode) throws Exception { reinitHiveSyncClient(); reSyncHiveTable(); - List hivePartitions = hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME); + List hivePartitions = hiveClient.getAllPartitions(HiveTestUtil.TABLE_NAME); assertEquals(0, hivePartitions.size(), "Table should have 0 partition because of the drop the only one partition"); } @@ -849,12 +861,12 @@ public void testDropPartition(String syncMode) throws Exception { @ParameterizedTest @MethodSource("syncMode") public void testNonPartitionedSync(String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + hiveSyncProps.setProperty(HIVE_SYNC_MODE.key(), syncMode); String instantTime = "100"; HiveTestUtil.createCOWTable(instantTime, 5, true); // Set partition value extractor to NonPartitionedExtractor - hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), NonPartitionedExtractor.class.getCanonicalName()); - hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "year, month, day"); + hiveSyncProps.setProperty(META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), NonPartitionedExtractor.class.getCanonicalName()); + hiveSyncProps.setProperty(META_SYNC_PARTITION_FIELDS.key(), ""); HiveTestUtil.getCreatedTablesSet().add(HiveTestUtil.DB_NAME + "." + HiveTestUtil.TABLE_NAME); @@ -865,17 +877,17 @@ public void testNonPartitionedSync(String syncMode) throws Exception { reSyncHiveTable(); assertTrue(hiveClient.tableExists(HiveTestUtil.TABLE_NAME), "Table " + HiveTestUtil.TABLE_NAME + " should exist after sync completes"); - assertEquals(hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(), - hiveClient.getDataSchema().getColumns().size(), + assertEquals(hiveClient.getMetastoreSchema(HiveTestUtil.TABLE_NAME).size(), + hiveClient.getStorageSchema().getColumns().size(), "Hive Schema should match the table schema,ignoring the partition fields"); - assertEquals(0, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + assertEquals(0, hiveClient.getAllPartitions(HiveTestUtil.TABLE_NAME).size(), "Table should not have partitions because of the NonPartitionedExtractor"); } @ParameterizedTest @MethodSource("syncMode") public void testReadSchemaForMOR(String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + hiveSyncProps.setProperty(HIVE_SYNC_MODE.key(), syncMode); String commitTime = "100"; String snapshotTableName = HiveTestUtil.TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE; HiveTestUtil.createMORTable(commitTime, "", 5, false, true); @@ -891,11 +903,11 @@ public void testReadSchemaForMOR(String syncMode) throws Exception { + " should exist after sync completes"); // Schema being read from compacted base files - assertEquals(hiveClient.getTableSchema(snapshotTableName).size(), + assertEquals(hiveClient.getMetastoreSchema(snapshotTableName).size(), SchemaTestUtil.getSimpleSchema().getFields().size() + getPartitionFieldSize() + HoodieRecord.HOODIE_META_COLUMNS.size(), "Hive Schema should match the table schema + partition field"); - assertEquals(5, hiveClient.scanTablePartitions(snapshotTableName).size(), "Table partitions should match the number of partitions we wrote"); + assertEquals(5, hiveClient.getAllPartitions(snapshotTableName).size(), "Table partitions should match the number of partitions we wrote"); // Now lets create more partitions and these are the only ones which needs to be synced ZonedDateTime dateTime = ZonedDateTime.now().plusDays(6); @@ -908,28 +920,28 @@ public void testReadSchemaForMOR(String syncMode) throws Exception { reSyncHiveTable(); // Schema being read from the log filesTestHiveSyncTool - assertEquals(hiveClient.getTableSchema(snapshotTableName).size(), + assertEquals(hiveClient.getMetastoreSchema(snapshotTableName).size(), SchemaTestUtil.getEvolvedSchema().getFields().size() + getPartitionFieldSize() + HoodieRecord.HOODIE_META_COLUMNS.size(), "Hive Schema should match the evolved table schema + partition field"); // Sync should add the one partition - assertEquals(6, hiveClient.scanTablePartitions(snapshotTableName).size(), "The 1 partition we wrote should be added to hive"); + assertEquals(6, hiveClient.getAllPartitions(snapshotTableName).size(), "The 1 partition we wrote should be added to hive"); assertEquals(deltaCommitTime2, hiveClient.getLastCommitTimeSynced(snapshotTableName).get(), "The last commit that was synced should be 103"); } @Test - public void testConnectExceptionIgnoreConfigSet() throws IOException, URISyntaxException, HiveException, MetaException { + public void testConnectExceptionIgnoreConfigSet() throws IOException, URISyntaxException { String instantTime = "100"; HiveTestUtil.createCOWTable(instantTime, 5, false); reinitHiveSyncClient(); - HoodieHiveClient prevHiveClient = hiveClient; + HoodieHiveSyncClient prevHiveClient = hiveClient; assertFalse(hiveClient.tableExists(HiveTestUtil.TABLE_NAME), "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially"); // Lets do the sync - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_IGNORE_EXCEPTIONS.key(), "true"); - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_URL.key(), hiveSyncProps.getString(HiveSyncConfig.HIVE_URL.key()) + hiveSyncProps.setProperty(HIVE_IGNORE_EXCEPTIONS.key(), "true"); + hiveSyncProps.setProperty(HIVE_URL.key(), hiveSyncProps.getString(HIVE_URL.key()) .replace(String.valueOf(HiveTestUtil.hiveTestService.getHiveServerPort()), String.valueOf(NetworkTestUtils.nextFreePort()))); reinitHiveSyncClient(); reSyncHiveTable(); @@ -939,12 +951,12 @@ public void testConnectExceptionIgnoreConfigSet() throws IOException, URISyntaxE "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially"); } - private void verifyOldParquetFileTest(HoodieHiveClient hiveClient, String emptyCommitTime) throws Exception { + private void verifyOldParquetFileTest(HoodieHiveSyncClient hiveClient, String emptyCommitTime) throws Exception { assertTrue(hiveClient.tableExists(HiveTestUtil.TABLE_NAME), "Table " + HiveTestUtil.TABLE_NAME + " should exist after sync completes"); - assertEquals(hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(), - hiveClient.getDataSchema().getColumns().size() + 1, + assertEquals(hiveClient.getMetastoreSchema(HiveTestUtil.TABLE_NAME).size(), + hiveClient.getStorageSchema().getColumns().size() + 1, "Hive Schema should match the table schema + partition field"); - assertEquals(1, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), "Table partitions should match the number of partitions we wrote"); + assertEquals(1, hiveClient.getAllPartitions(HiveTestUtil.TABLE_NAME).size(), "Table partitions should match the number of partitions we wrote"); assertEquals(emptyCommitTime, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); @@ -952,19 +964,19 @@ private void verifyOldParquetFileTest(HoodieHiveClient hiveClient, String emptyC Schema schema = SchemaTestUtil.getSimpleSchema(); for (Field field : schema.getFields()) { assertEquals(field.schema().getType().getName(), - hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).get(field.name()).toLowerCase(), + hiveClient.getMetastoreSchema(HiveTestUtil.TABLE_NAME).get(field.name()).toLowerCase(), String.format("Hive Schema Field %s was added", field)); } assertEquals("string", - hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).get("datestr").toLowerCase(), "Hive Schema Field datestr was added"); + hiveClient.getMetastoreSchema(HiveTestUtil.TABLE_NAME).get("datestr").toLowerCase(), "Hive Schema Field datestr was added"); assertEquals(schema.getFields().size() + 1 + HoodieRecord.HOODIE_META_COLUMNS.size(), - hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(), "Hive Schema fields size"); + hiveClient.getMetastoreSchema(HiveTestUtil.TABLE_NAME).size(), "Hive Schema fields size"); } @ParameterizedTest @MethodSource("syncMode") public void testPickingOlderParquetFileIfLatestIsEmptyCommit(String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + hiveSyncProps.setProperty(HIVE_SYNC_MODE.key(), syncMode); final String commitTime = "100"; HiveTestUtil.createCOWTable(commitTime, 1, true); HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); @@ -983,7 +995,7 @@ public void testPickingOlderParquetFileIfLatestIsEmptyCommit(String syncMode) th @ParameterizedTest @MethodSource("syncMode") public void testNotPickingOlderParquetFileWhenLatestCommitReadFails(String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + hiveSyncProps.setProperty(HIVE_SYNC_MODE.key(), syncMode); final String commitTime = "100"; HiveTestUtil.createCOWTable(commitTime, 1, true); HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); @@ -1001,7 +1013,7 @@ public void testNotPickingOlderParquetFileWhenLatestCommitReadFails(String syncM assertFalse( hiveClient.tableExists(HiveTestUtil.TABLE_NAME), "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially"); - HiveSyncTool tool = new HiveSyncTool(hiveSyncProps, getHiveConf(), fileSystem); + HiveSyncTool tool = new HiveSyncTool(hiveSyncProps, getHiveConf()); // now delete the evolved commit instant Path fullPath = new Path(HiveTestUtil.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + hiveClient.getActiveTimeline().getInstants() @@ -1022,7 +1034,7 @@ public void testNotPickingOlderParquetFileWhenLatestCommitReadFails(String syncM @ParameterizedTest @MethodSource("syncMode") public void testNotPickingOlderParquetFileWhenLatestCommitReadFailsForExistingTable(String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + hiveSyncProps.setProperty(HIVE_SYNC_MODE.key(), syncMode); final String commitTime = "100"; HiveTestUtil.createCOWTable(commitTime, 1, true); HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); @@ -1067,7 +1079,7 @@ public void testNotPickingOlderParquetFileWhenLatestCommitReadFailsForExistingTa @ParameterizedTest @MethodSource("syncMode") public void testTypeConverter(String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + hiveSyncProps.setProperty(HIVE_SYNC_MODE.key(), syncMode); HiveTestUtil.createCOWTable("100", 5, true); // create database. ddlExecutor.runSQL("create database " + HiveTestUtil.DB_NAME); @@ -1082,24 +1094,24 @@ public void testTypeConverter(String syncMode) throws Exception { // test one column in DECIMAL String oneTargetColumnSql = createTableSqlPrefix + "(`decimal_col` DECIMAL(9,8), `bigint_col` BIGINT)"; ddlExecutor.runSQL(oneTargetColumnSql); - System.out.println(hiveClient.getTableSchema(tableName)); - assertTrue(hiveClient.getTableSchema(tableName).containsValue("DECIMAL(9,8)"), errorMsg); + System.out.println(hiveClient.getMetastoreSchema(tableName)); + assertTrue(hiveClient.getMetastoreSchema(tableName).containsValue("DECIMAL(9,8)"), errorMsg); ddlExecutor.runSQL(dropTableSql); // test multiple columns in DECIMAL String multipleTargetColumnSql = createTableSqlPrefix + "(`decimal_col1` DECIMAL(9,8), `bigint_col` BIGINT, `decimal_col2` DECIMAL(7,4))"; ddlExecutor.runSQL(multipleTargetColumnSql); - System.out.println(hiveClient.getTableSchema(tableName)); - assertTrue(hiveClient.getTableSchema(tableName).containsValue("DECIMAL(9,8)") - && hiveClient.getTableSchema(tableName).containsValue("DECIMAL(7,4)"), errorMsg); + System.out.println(hiveClient.getMetastoreSchema(tableName)); + assertTrue(hiveClient.getMetastoreSchema(tableName).containsValue("DECIMAL(9,8)") + && hiveClient.getMetastoreSchema(tableName).containsValue("DECIMAL(7,4)"), errorMsg); ddlExecutor.runSQL(dropTableSql); // test no columns in DECIMAL String noTargetColumnsSql = createTableSqlPrefix + "(`bigint_col` BIGINT)"; ddlExecutor.runSQL(noTargetColumnsSql); - System.out.println(hiveClient.getTableSchema(tableName)); - assertTrue(hiveClient.getTableSchema(tableName).size() == 1 && hiveClient.getTableSchema(tableName) + System.out.println(hiveClient.getMetastoreSchema(tableName)); + assertTrue(hiveClient.getMetastoreSchema(tableName).size() == 1 && hiveClient.getMetastoreSchema(tableName) .containsValue("BIGINT"), errorMsg); ddlExecutor.runSQL(dropTableSql); } @@ -1108,8 +1120,8 @@ public void testTypeConverter(String syncMode) throws Exception { @MethodSource("syncMode") public void testSyncWithoutDiffs(String syncMode) throws Exception { String tableName = HiveTestUtil.TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE; - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); - hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_CONDITIONAL_SYNC.key(), "true"); + hiveSyncProps.setProperty(HIVE_SYNC_MODE.key(), syncMode); + hiveSyncProps.setProperty(META_SYNC_CONDITIONAL_SYNC.key(), "true"); String commitTime0 = "100"; String commitTime1 = "101"; @@ -1136,11 +1148,11 @@ private void reSyncHiveTable() { } private void reinitHiveSyncClient() { - hiveSyncTool = new HiveSyncTool(hiveSyncProps, HiveTestUtil.getHiveConf(), fileSystem); - hiveClient = (HoodieHiveClient) hiveSyncTool.hoodieHiveClient; + hiveSyncTool = new HiveSyncTool(hiveSyncProps, HiveTestUtil.getHiveConf()); + hiveClient = (HoodieHiveSyncClient) hiveSyncTool.syncClient; } private int getPartitionFieldSize() { - return hiveSyncProps.getString(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key()).split(",").length; + return hiveSyncProps.getString(META_SYNC_PARTITION_FIELDS.key()).split(",").length; } } diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/functional/TestHiveMetastoreBasedLockProvider.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/functional/TestHiveMetastoreBasedLockProvider.java index 8ae002ce6bc68..3d129c5899e65 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/functional/TestHiveMetastoreBasedLockProvider.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/functional/TestHiveMetastoreBasedLockProvider.java @@ -21,7 +21,7 @@ import org.apache.hudi.common.config.LockConfiguration; import org.apache.hudi.common.config.TypedProperties; -import org.apache.hudi.hive.HiveMetastoreBasedLockProvider; +import org.apache.hudi.hive.transaction.lock.HiveMetastoreBasedLockProvider; import org.apache.hudi.hive.testutils.HiveSyncFunctionalTestHarness; import org.apache.hadoop.hive.metastore.api.DataOperationType; diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/replication/TestHiveSyncGlobalCommitTool.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/replication/TestHiveSyncGlobalCommitTool.java new file mode 100644 index 0000000000000..a11b23762e73a --- /dev/null +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/replication/TestHiveSyncGlobalCommitTool.java @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.hive.replication; + +import org.apache.hudi.hive.testutils.TestCluster; + +import org.apache.hadoop.fs.Path; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +import static org.apache.hudi.hadoop.utils.HoodieHiveUtils.GLOBALLY_CONSISTENT_READ_TIMESTAMP; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_PASS; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USER; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USE_PRE_APACHE_INPUT_FORMAT; +import static org.apache.hudi.hive.replication.GlobalHiveSyncConfig.META_SYNC_GLOBAL_REPLICATE_TIMESTAMP; +import static org.apache.hudi.hive.replication.HiveSyncGlobalCommitParams.LOCAL_BASE_PATH; +import static org.apache.hudi.hive.replication.HiveSyncGlobalCommitParams.LOCAL_HIVE_SERVER_JDBC_URLS; +import static org.apache.hudi.hive.replication.HiveSyncGlobalCommitParams.LOCAL_HIVE_SITE_URI; +import static org.apache.hudi.hive.replication.HiveSyncGlobalCommitParams.REMOTE_BASE_PATH; +import static org.apache.hudi.hive.replication.HiveSyncGlobalCommitParams.REMOTE_HIVE_SERVER_JDBC_URLS; +import static org.apache.hudi.hive.replication.HiveSyncGlobalCommitParams.REMOTE_HIVE_SITE_URI; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_ASSUME_DATE_PARTITION; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_BASE_PATH; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_PARTITION_FIELDS; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_TABLE_NAME; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestHiveSyncGlobalCommitTool { + + @RegisterExtension + public static TestCluster localCluster = new TestCluster(); + @RegisterExtension + public static TestCluster remoteCluster = new TestCluster(); + + private static final String DB_NAME = "foo"; + private static final String TBL_NAME = "bar"; + + private HiveSyncGlobalCommitParams getGlobalCommitConfig(String commitTime) throws Exception { + HiveSyncGlobalCommitParams params = new HiveSyncGlobalCommitParams(); + params.loadedProps.setProperty(LOCAL_HIVE_SITE_URI, localCluster.getHiveSiteXmlLocation()); + params.loadedProps.setProperty(REMOTE_HIVE_SITE_URI, remoteCluster.getHiveSiteXmlLocation()); + params.loadedProps.setProperty(LOCAL_HIVE_SERVER_JDBC_URLS, localCluster.getHiveJdBcUrl()); + params.loadedProps.setProperty(REMOTE_HIVE_SERVER_JDBC_URLS, remoteCluster.getHiveJdBcUrl()); + params.loadedProps.setProperty(LOCAL_BASE_PATH, localCluster.tablePath(DB_NAME, TBL_NAME)); + params.loadedProps.setProperty(REMOTE_BASE_PATH, remoteCluster.tablePath(DB_NAME, TBL_NAME)); + params.loadedProps.setProperty(META_SYNC_GLOBAL_REPLICATE_TIMESTAMP.key(), commitTime); + params.loadedProps.setProperty(HIVE_USER.key(), System.getProperty("user.name")); + params.loadedProps.setProperty(HIVE_PASS.key(), ""); + params.loadedProps.setProperty(META_SYNC_DATABASE_NAME.key(), DB_NAME); + params.loadedProps.setProperty(META_SYNC_TABLE_NAME.key(), TBL_NAME); + params.loadedProps.setProperty(META_SYNC_BASE_PATH.key(), localCluster.tablePath(DB_NAME, TBL_NAME)); + params.loadedProps.setProperty(META_SYNC_ASSUME_DATE_PARTITION.key(), "true"); + params.loadedProps.setProperty(HIVE_USE_PRE_APACHE_INPUT_FORMAT.key(), "false"); + params.loadedProps.setProperty(META_SYNC_PARTITION_FIELDS.key(), "datestr"); + return params; + } + + private void compareEqualLastReplicatedTimeStamp(HiveSyncGlobalCommitParams config) throws Exception { + assertEquals(localCluster.getHMSClient() + .getTable(DB_NAME, TBL_NAME).getParameters() + .get(GLOBALLY_CONSISTENT_READ_TIMESTAMP), remoteCluster.getHMSClient() + .getTable(DB_NAME, TBL_NAME).getParameters() + .get(GLOBALLY_CONSISTENT_READ_TIMESTAMP), "compare replicated timestamps"); + } + + @BeforeEach + public void setUp() throws Exception { + localCluster.forceCreateDb(DB_NAME); + remoteCluster.forceCreateDb(DB_NAME); + localCluster.dfsCluster.getFileSystem().delete(new Path(localCluster.tablePath(DB_NAME, TBL_NAME)), true); + remoteCluster.dfsCluster.getFileSystem().delete(new Path(remoteCluster.tablePath(DB_NAME, TBL_NAME)), true); + } + + @AfterEach + public void clear() throws Exception { + localCluster.getHMSClient().dropTable(DB_NAME, TBL_NAME); + remoteCluster.getHMSClient().dropTable(DB_NAME, TBL_NAME); + } + + @Test + public void testHiveConfigShouldMatchClusterConf() throws Exception { + String commitTime = "100"; + localCluster.createCOWTable(commitTime, 5, DB_NAME, TBL_NAME); + // simulate drs + remoteCluster.createCOWTable(commitTime, 5, DB_NAME, TBL_NAME); + HiveSyncGlobalCommitParams params = getGlobalCommitConfig(commitTime); + HiveSyncGlobalCommitTool tool = new HiveSyncGlobalCommitTool(params); + ReplicationStateSync localReplicationStateSync = tool.getReplicatedState(false); + ReplicationStateSync remoteReplicationStateSync = tool.getReplicatedState(true); + assertEquals(localReplicationStateSync.globalHiveSyncTool.config.getHiveConf().get("hive.metastore.uris"), + localCluster.getHiveConf().get("hive.metastore.uris")); + assertEquals(remoteReplicationStateSync.globalHiveSyncTool.config.getHiveConf().get("hive.metastore.uris"), + remoteCluster.getHiveConf().get("hive.metastore.uris")); + } + + @Test + public void testBasicGlobalCommit() throws Exception { + String commitTime = "100"; + localCluster.createCOWTable(commitTime, 5, DB_NAME, TBL_NAME); + // simulate drs + remoteCluster.createCOWTable(commitTime, 5, DB_NAME, TBL_NAME); + HiveSyncGlobalCommitParams params = getGlobalCommitConfig(commitTime); + HiveSyncGlobalCommitTool tool = new HiveSyncGlobalCommitTool(params); + assertTrue(tool.commit()); + compareEqualLastReplicatedTimeStamp(params); + } + + @Test + public void testBasicRollback() throws Exception { + String commitTime = "100"; + localCluster.createCOWTable(commitTime, 5, DB_NAME, TBL_NAME); + // simulate drs + remoteCluster.createCOWTable(commitTime, 5, DB_NAME, TBL_NAME); + HiveSyncGlobalCommitParams params = getGlobalCommitConfig(commitTime); + HiveSyncGlobalCommitTool tool = new HiveSyncGlobalCommitTool(params); + assertFalse(localCluster.getHMSClient().tableExists(DB_NAME, TBL_NAME)); + assertFalse(remoteCluster.getHMSClient().tableExists(DB_NAME, TBL_NAME)); + // stop the remote cluster hive server to simulate cluster going down + remoteCluster.stopHiveServer2(); + assertFalse(tool.commit()); + assertEquals(commitTime, localCluster.getHMSClient() + .getTable(DB_NAME, TBL_NAME).getParameters() + .get(GLOBALLY_CONSISTENT_READ_TIMESTAMP)); + assertTrue(tool.rollback()); // do a rollback + assertNotEquals(commitTime, localCluster.getHMSClient() + .getTable(DB_NAME, TBL_NAME).getParameters() + .get(GLOBALLY_CONSISTENT_READ_TIMESTAMP)); + assertFalse(remoteCluster.getHMSClient().tableExists(DB_NAME, TBL_NAME)); + remoteCluster.startHiveServer2(); + } +} diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveSyncFunctionalTestHarness.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveSyncFunctionalTestHarness.java index b6adcb29825e4..e8981c9b67fbf 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveSyncFunctionalTestHarness.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveSyncFunctionalTestHarness.java @@ -24,7 +24,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.testutils.minicluster.ZookeeperTestService; import org.apache.hudi.hive.HiveSyncConfig; -import org.apache.hudi.hive.HoodieHiveClient; +import org.apache.hudi.hive.HoodieHiveSyncClient; import org.apache.hudi.hive.ddl.HiveQueryDDLExecutor; import org.apache.hadoop.conf.Configuration; @@ -39,7 +39,17 @@ import java.io.IOException; import java.nio.file.Files; import java.time.Instant; -import java.util.Collections; +import java.util.Properties; + +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_PASS; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_URL; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USER; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USE_PRE_APACHE_INPUT_FORMAT; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_ASSUME_DATE_PARTITION; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_BASE_PATH; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_PARTITION_FIELDS; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_TABLE_NAME; public class HiveSyncFunctionalTestHarness { @@ -79,42 +89,42 @@ public ZookeeperTestService zkService() { } public HiveSyncConfig hiveSyncConf() throws IOException { - HiveSyncConfig conf = new HiveSyncConfig(); - conf.jdbcUrl = hiveTestService.getJdbcHive2Url(); - conf.hiveUser = ""; - conf.hivePass = ""; - conf.databaseName = "hivesynctestdb"; - conf.tableName = "hivesynctesttable"; - conf.basePath = Files.createDirectories(tempDir.resolve("hivesynctestcase-" + Instant.now().toEpochMilli())).toUri().toString(); - conf.assumeDatePartitioning = true; - conf.usePreApacheInputFormat = false; - conf.partitionFields = Collections.singletonList("datestr"); - return conf; + Properties props = new Properties(); + props.setProperty(HIVE_URL.key(), hiveTestService.getJdbcHive2Url()); + props.setProperty(HIVE_USER.key(), ""); + props.setProperty(HIVE_PASS.key(), ""); + props.setProperty(META_SYNC_DATABASE_NAME.key(), "hivesynctestdb"); + props.setProperty(META_SYNC_TABLE_NAME.key(), "hivesynctesttable"); + props.setProperty(META_SYNC_BASE_PATH.key(), Files.createDirectories(tempDir.resolve("hivesynctestcase-" + Instant.now().toEpochMilli())).toUri().toString()); + props.setProperty(META_SYNC_ASSUME_DATE_PARTITION.key(), "true"); + props.setProperty(HIVE_USE_PRE_APACHE_INPUT_FORMAT.key(), "false"); + props.setProperty(META_SYNC_PARTITION_FIELDS.key(), "datestr"); + return new HiveSyncConfig(props, hiveConf()); } - public HoodieHiveClient hiveClient(HiveSyncConfig hiveSyncConfig) throws IOException { + public HoodieHiveSyncClient hiveClient(HiveSyncConfig hiveSyncConfig) throws IOException { HoodieTableMetaClient.withPropertyBuilder() .setTableType(HoodieTableType.COPY_ON_WRITE) - .setTableName(hiveSyncConfig.tableName) + .setTableName(hiveSyncConfig.getString(META_SYNC_TABLE_NAME)) .setPayloadClass(HoodieAvroPayload.class) - .initTable(hadoopConf, hiveSyncConfig.basePath); - return new HoodieHiveClient(hiveSyncConfig, hiveConf(), fs()); + .initTable(hadoopConf, hiveSyncConfig.getString(META_SYNC_BASE_PATH)); + return new HoodieHiveSyncClient(hiveSyncConfig); } public void dropTables(String database, String... tables) throws IOException, HiveException, MetaException { HiveSyncConfig hiveSyncConfig = hiveSyncConf(); - hiveSyncConfig.databaseName = database; + hiveSyncConfig.setValue(META_SYNC_DATABASE_NAME, database); for (String table : tables) { - hiveSyncConfig.tableName = table; - new HiveQueryDDLExecutor(hiveSyncConfig, fs(), hiveConf()).runSQL("drop table if exists " + table); + hiveSyncConfig.setValue(META_SYNC_TABLE_NAME, table); + new HiveQueryDDLExecutor(hiveSyncConfig).runSQL("drop table if exists " + table); } } public void dropDatabases(String... databases) throws IOException, HiveException, MetaException { HiveSyncConfig hiveSyncConfig = hiveSyncConf(); for (String database : databases) { - hiveSyncConfig.databaseName = database; - new HiveQueryDDLExecutor(hiveSyncConfig, fs(), hiveConf()).runSQL("drop database if exists " + database); + hiveSyncConfig.setValue(META_SYNC_DATABASE_NAME, database); + new HiveQueryDDLExecutor(hiveSyncConfig).runSQL("drop database if exists " + database); } } diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java index 8be2ace89f8f1..9687e557928bd 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java @@ -84,6 +84,16 @@ import java.util.Set; import java.util.UUID; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_BATCH_SYNC_PARTITION_NUM; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_PASS; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_URL; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USER; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USE_PRE_APACHE_INPUT_FORMAT; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_ASSUME_DATE_PARTITION; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_BASE_PATH; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_PARTITION_FIELDS; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_TABLE_NAME; import static org.junit.jupiter.api.Assertions.fail; @SuppressWarnings("SameParameterValue") @@ -120,21 +130,21 @@ public static void setUp() throws IOException, InterruptedException, HiveExcepti basePath = Files.createTempDirectory("hivesynctest" + Instant.now().toEpochMilli()).toUri().toString(); hiveSyncProps = new TypedProperties(); - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_URL.key(), hiveTestService.getJdbcHive2Url()); - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_USER.key(), ""); - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_PASS.key(), ""); - hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_DATABASE_NAME.key(), DB_NAME); - hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_TABLE_NAME.key(), TABLE_NAME); - hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_BASE_PATH.key(), basePath); - hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_ASSUME_DATE_PARTITION.key(), "true"); - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_USE_PRE_APACHE_INPUT_FORMAT.key(), "false"); - hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "datestr"); - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_BATCH_SYNC_PARTITION_NUM.key(), "3"); - - hiveSyncConfig = new HiveSyncConfig(hiveSyncProps); + hiveSyncProps.setProperty(HIVE_URL.key(), hiveTestService.getJdbcHive2Url()); + hiveSyncProps.setProperty(HIVE_USER.key(), ""); + hiveSyncProps.setProperty(HIVE_PASS.key(), ""); + hiveSyncProps.setProperty(META_SYNC_DATABASE_NAME.key(), DB_NAME); + hiveSyncProps.setProperty(META_SYNC_TABLE_NAME.key(), TABLE_NAME); + hiveSyncProps.setProperty(META_SYNC_BASE_PATH.key(), basePath); + hiveSyncProps.setProperty(META_SYNC_ASSUME_DATE_PARTITION.key(), "true"); + hiveSyncProps.setProperty(HIVE_USE_PRE_APACHE_INPUT_FORMAT.key(), "false"); + hiveSyncProps.setProperty(META_SYNC_PARTITION_FIELDS.key(), "datestr"); + hiveSyncProps.setProperty(HIVE_BATCH_SYNC_PARTITION_NUM.key(), "3"); + + hiveSyncConfig = new HiveSyncConfig(hiveSyncProps, configuration); dtfOut = DateTimeFormatter.ofPattern("yyyy/MM/dd"); - ddlExecutor = new HiveQueryDDLExecutor(hiveSyncConfig, fileSystem, getHiveConf()); + ddlExecutor = new HiveQueryDDLExecutor(hiveSyncConfig); clear(); } diff --git a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/AbstractSyncHoodieClient.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/AbstractSyncHoodieClient.java deleted file mode 100644 index 8eec327890ca4..0000000000000 --- a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/AbstractSyncHoodieClient.java +++ /dev/null @@ -1,276 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hudi.sync.common; - -import java.io.Serializable; -import java.sql.ResultSet; -import java.sql.SQLException; -import java.sql.Statement; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hudi.common.engine.HoodieLocalEngineContext; -import org.apache.hudi.common.fs.FSUtils; -import org.apache.hudi.common.model.HoodieCommitMetadata; -import org.apache.hudi.common.model.HoodieTableType; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.TableSchemaResolver; -import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.TimelineUtils; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.ValidationUtils; -import org.apache.hudi.metadata.HoodieTableMetadataUtil; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; -import org.apache.parquet.schema.MessageType; - -public abstract class AbstractSyncHoodieClient implements AutoCloseable { - - private static final Logger LOG = LogManager.getLogger(AbstractSyncHoodieClient.class); - - public static final String HOODIE_LAST_COMMIT_TIME_SYNC = "last_commit_time_sync"; - public static final TypeConverter TYPE_CONVERTOR = new TypeConverter() {}; - - protected final HoodieTableMetaClient metaClient; - protected final HoodieTableType tableType; - protected final FileSystem fs; - private final String basePath; - private final boolean assumeDatePartitioning; - private final boolean useFileListingFromMetadata; - private final boolean withOperationField; - - @Deprecated - public AbstractSyncHoodieClient(String basePath, boolean assumeDatePartitioning, boolean useFileListingFromMetadata, - boolean verifyMetadataFileListing, boolean withOperationField, FileSystem fs) { - this(basePath, assumeDatePartitioning, useFileListingFromMetadata, withOperationField, fs); - } - - public AbstractSyncHoodieClient(String basePath, boolean assumeDatePartitioning, boolean useFileListingFromMetadata, - boolean withOperationField, FileSystem fs) { - this.metaClient = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(basePath).setLoadActiveTimelineOnLoad(true).build(); - this.tableType = metaClient.getTableType(); - this.basePath = basePath; - this.assumeDatePartitioning = assumeDatePartitioning; - this.useFileListingFromMetadata = useFileListingFromMetadata; - this.withOperationField = withOperationField; - this.fs = fs; - } - - /** - * Create the table. - * @param tableName The table name. - * @param storageSchema The table schema. - * @param inputFormatClass The input format class of this table. - * @param outputFormatClass The output format class of this table. - * @param serdeClass The serde class of this table. - * @param serdeProperties The serde properties of this table. - * @param tableProperties The table properties for this table. - */ - public abstract void createTable(String tableName, MessageType storageSchema, - String inputFormatClass, String outputFormatClass, - String serdeClass, Map serdeProperties, - Map tableProperties); - - /** - * @deprecated Use {@link #tableExists} instead. - */ - @Deprecated - public abstract boolean doesTableExist(String tableName); - - public abstract boolean tableExists(String tableName); - - public abstract Option getLastCommitTimeSynced(String tableName); - - public abstract void updateLastCommitTimeSynced(String tableName); - - public abstract Option getLastReplicatedTime(String tableName); - - public abstract void updateLastReplicatedTimeStamp(String tableName, String timeStamp); - - public abstract void deleteLastReplicatedTimeStamp(String tableName); - - public abstract void addPartitionsToTable(String tableName, List partitionsToAdd); - - public abstract void updatePartitionsToTable(String tableName, List changedPartitions); - - public abstract void dropPartitions(String tableName, List partitionsToDrop); - - public void updateTableProperties(String tableName, Map tableProperties) {} - - public abstract Map getTableSchema(String tableName); - - public HoodieTableType getTableType() { - return tableType; - } - - public String getBasePath() { - return metaClient.getBasePath(); - } - - public FileSystem getFs() { - return fs; - } - - public boolean isBootstrap() { - return metaClient.getTableConfig().getBootstrapBasePath().isPresent(); - } - - public void closeQuietly(ResultSet resultSet, Statement stmt) { - try { - if (stmt != null) { - stmt.close(); - } - } catch (SQLException e) { - LOG.warn("Could not close the statement opened ", e); - } - - try { - if (resultSet != null) { - resultSet.close(); - } - } catch (SQLException e) { - LOG.warn("Could not close the resultset opened ", e); - } - } - - /** - * Gets the schema for a hoodie table. Depending on the type of table, try to read schema from commit metadata if - * present, else fallback to reading from any file written in the latest commit. We will assume that the schema has - * not changed within a single atomic write. - * - * @return Parquet schema for this table - */ - public MessageType getDataSchema() { - try { - return new TableSchemaResolver(metaClient).getTableParquetSchema(); - } catch (Exception e) { - throw new HoodieSyncException("Failed to read data schema", e); - } - } - - public boolean isDropPartition() { - try { - Option hoodieCommitMetadata = HoodieTableMetadataUtil.getLatestCommitMetadata(metaClient); - - if (hoodieCommitMetadata.isPresent() - && WriteOperationType.DELETE_PARTITION.equals(hoodieCommitMetadata.get().getOperationType())) { - return true; - } - } catch (Exception e) { - throw new HoodieSyncException("Failed to get commit metadata", e); - } - return false; - } - - @SuppressWarnings("OptionalUsedAsFieldOrParameterType") - public List getPartitionsWrittenToSince(Option lastCommitTimeSynced) { - if (!lastCommitTimeSynced.isPresent()) { - LOG.info("Last commit time synced is not known, listing all partitions in " + basePath + ",FS :" + fs); - HoodieLocalEngineContext engineContext = new HoodieLocalEngineContext(metaClient.getHadoopConf()); - return FSUtils.getAllPartitionPaths(engineContext, basePath, useFileListingFromMetadata, assumeDatePartitioning); - } else { - LOG.info("Last commit time synced is " + lastCommitTimeSynced.get() + ", Getting commits since then"); - return TimelineUtils.getPartitionsWritten(metaClient.getActiveTimeline().getCommitsTimeline() - .findInstantsAfter(lastCommitTimeSynced.get(), Integer.MAX_VALUE)); - } - } - - public abstract static class TypeConverter implements Serializable { - - static final String DEFAULT_TARGET_TYPE = "DECIMAL"; - - protected String targetType; - - public TypeConverter() { - this.targetType = DEFAULT_TARGET_TYPE; - } - - public TypeConverter(String targetType) { - ValidationUtils.checkArgument(Objects.nonNull(targetType)); - this.targetType = targetType; - } - - public void doConvert(ResultSet resultSet, Map schema) throws SQLException { - schema.put(getColumnName(resultSet), targetType.equalsIgnoreCase(getColumnType(resultSet)) - ? convert(resultSet) : getColumnType(resultSet)); - } - - public String convert(ResultSet resultSet) throws SQLException { - String columnType = getColumnType(resultSet); - int columnSize = resultSet.getInt("COLUMN_SIZE"); - int decimalDigits = resultSet.getInt("DECIMAL_DIGITS"); - return columnType + String.format("(%s,%s)", columnSize, decimalDigits); - } - - public String getColumnName(ResultSet resultSet) throws SQLException { - return resultSet.getString(4); - } - - public String getColumnType(ResultSet resultSet) throws SQLException { - return resultSet.getString(6); - } - } - - /** - * Read the schema from the log file on path. - */ - @SuppressWarnings("OptionalUsedAsFieldOrParameterType") - private MessageType readSchemaFromLogFile(Option lastCompactionCommitOpt, Path path) throws Exception { - MessageType messageType = TableSchemaResolver.readSchemaFromLogFile(fs, path); - // Fall back to read the schema from last compaction - if (messageType == null) { - LOG.info("Falling back to read the schema from last compaction " + lastCompactionCommitOpt); - return new TableSchemaResolver(this.metaClient).readSchemaFromLastCompaction(lastCompactionCommitOpt); - } - return messageType; - } - - /** - * Partition Event captures any partition that needs to be added or updated. - */ - public static class PartitionEvent { - - public enum PartitionEventType { - ADD, UPDATE, DROP - } - - public PartitionEventType eventType; - public String storagePartition; - - PartitionEvent(PartitionEventType eventType, String storagePartition) { - this.eventType = eventType; - this.storagePartition = storagePartition; - } - - public static PartitionEvent newPartitionAddEvent(String storagePartition) { - return new PartitionEvent(PartitionEventType.ADD, storagePartition); - } - - public static PartitionEvent newPartitionUpdateEvent(String storagePartition) { - return new PartitionEvent(PartitionEventType.UPDATE, storagePartition); - } - - public static PartitionEvent newPartitionDropEvent(String storagePartition) { - return new PartitionEvent(PartitionEventType.DROP, storagePartition); - } - } -} diff --git a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieMetaSyncOperations.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieMetaSyncOperations.java new file mode 100644 index 0000000000000..1c16dd13edaa4 --- /dev/null +++ b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieMetaSyncOperations.java @@ -0,0 +1,196 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.sync.common; + +import org.apache.hudi.common.util.Option; +import org.apache.hudi.sync.common.model.FieldSchema; +import org.apache.hudi.sync.common.model.Partition; + +import org.apache.parquet.schema.MessageType; + +import java.util.Collections; +import java.util.List; +import java.util.Map; + +public interface HoodieMetaSyncOperations { + + String HOODIE_LAST_COMMIT_TIME_SYNC = "last_commit_time_sync"; + + /** + * Create the table. + * + * @param tableName The table name. + * @param storageSchema The table schema. + * @param inputFormatClass The input format class of this table. + * @param outputFormatClass The output format class of this table. + * @param serdeClass The serde class of this table. + * @param serdeProperties The serde properties of this table. + * @param tableProperties The table properties for this table. + */ + default void createTable(String tableName, + MessageType storageSchema, + String inputFormatClass, + String outputFormatClass, + String serdeClass, + Map serdeProperties, + Map tableProperties) { + + } + + /** + * Check if table exists in metastore. + */ + default boolean tableExists(String tableName) { + return false; + } + + /** + * Drop table from metastore. + */ + default void dropTable(String tableName) { + + } + + /** + * Add partitions to the table in metastore. + */ + default void addPartitionsToTable(String tableName, List partitionsToAdd) { + + } + + /** + * Update partitions to the table in metastore. + */ + default void updatePartitionsToTable(String tableName, List changedPartitions) { + + } + + /** + * Drop partitions from the table in metastore. + */ + default void dropPartitions(String tableName, List partitionsToDrop) { + + } + + /** + * Get all partitions for the table in the metastore. + */ + default List getAllPartitions(String tableName) { + return Collections.emptyList(); + } + + /** + * Check if a database already exists in the metastore. + */ + default boolean databaseExists(String databaseName) { + return false; + } + + /** + * Create a database in the metastore. + */ + default void createDatabase(String databaseName) { + + } + + /** + * Get the schema from metastore. + */ + default Map getMetastoreSchema(String tableName) { + return Collections.emptyMap(); + } + + /** + * Get the schema from the Hudi table on storage. + */ + default MessageType getStorageSchema() { + return null; + } + + /** + * Update schema for the table in the metastore. + */ + default void updateTableSchema(String tableName, MessageType newSchema) { + + } + + /** + * Get the list of field schemas from metastore. + */ + default List getMetastoreFieldSchemas(String tableName) { + return Collections.emptyList(); + } + + /** + * Get the list of field schema from the Hudi table on storage. + */ + default List getStorageFieldSchemas() { + return Collections.emptyList(); + } + + /** + * Update the field comments for table in metastore, by using the ones from storage. + */ + default void updateTableComments(String tableName, List fromMetastore, List fromStorage) { + + } + + /** + * Get the timestamp of last sync. + */ + default Option getLastCommitTimeSynced(String tableName) { + return Option.empty(); + } + + /** + * Update the timestamp of last sync. + */ + default void updateLastCommitTimeSynced(String tableName) { + + } + + /** + * Update the table properties in metastore. + */ + default void updateTableProperties(String tableName, Map tableProperties) { + + } + + /** + * Get the timestamp of last replication. + */ + default Option getLastReplicatedTime(String tableName) { + return Option.empty(); + } + + /** + * Update the timestamp of last replication. + */ + default void updateLastReplicatedTimeStamp(String tableName, String timeStamp) { + + } + + /** + * Delete the timestamp of last replication. + */ + default void deleteLastReplicatedTimeStamp(String tableName) { + + } +} diff --git a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncClient.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncClient.java new file mode 100644 index 0000000000000..32ade18d08117 --- /dev/null +++ b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncClient.java @@ -0,0 +1,161 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.sync.common; + +import org.apache.hudi.common.engine.HoodieLocalEngineContext; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.TableSchemaResolver; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ReflectionUtils; +import org.apache.hudi.metadata.HoodieTableMetadataUtil; +import org.apache.hudi.sync.common.model.Partition; +import org.apache.hudi.sync.common.model.PartitionEvent; +import org.apache.hudi.sync.common.model.PartitionValueExtractor; + +import org.apache.hadoop.fs.Path; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.parquet.schema.MessageType; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_ASSUME_DATE_PARTITION; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_BASE_PATH; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_USE_FILE_LISTING_FROM_METADATA; + +public abstract class HoodieSyncClient implements HoodieMetaSyncOperations, AutoCloseable { + + private static final Logger LOG = LogManager.getLogger(HoodieSyncClient.class); + + protected final HoodieSyncConfig config; + protected final PartitionValueExtractor partitionValueExtractor; + protected final HoodieTableMetaClient metaClient; + + public HoodieSyncClient(HoodieSyncConfig config) { + this.config = config; + this.partitionValueExtractor = ReflectionUtils.loadClass(config.getStringOrDefault(META_SYNC_PARTITION_EXTRACTOR_CLASS)); + this.metaClient = HoodieTableMetaClient.builder() + .setConf(config.getHadoopConf()) + .setBasePath(config.getString(META_SYNC_BASE_PATH)) + .setLoadActiveTimelineOnLoad(true) + .build(); + } + + public HoodieTimeline getActiveTimeline() { + return metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); + } + + public HoodieTableType getTableType() { + return metaClient.getTableType(); + } + + public String getBasePath() { + return metaClient.getBasePathV2().toString(); + } + + public boolean isBootstrap() { + return metaClient.getTableConfig().getBootstrapBasePath().isPresent(); + } + + public boolean isDropPartition() { + try { + Option hoodieCommitMetadata = HoodieTableMetadataUtil.getLatestCommitMetadata(metaClient); + + if (hoodieCommitMetadata.isPresent() + && WriteOperationType.DELETE_PARTITION.equals(hoodieCommitMetadata.get().getOperationType())) { + return true; + } + } catch (Exception e) { + throw new HoodieSyncException("Failed to get commit metadata", e); + } + return false; + } + + @Override + public MessageType getStorageSchema() { + try { + return new TableSchemaResolver(metaClient).getTableParquetSchema(); + } catch (Exception e) { + throw new HoodieSyncException("Failed to read schema from storage.", e); + } + } + + public List getPartitionsWrittenToSince(Option lastCommitTimeSynced) { + if (!lastCommitTimeSynced.isPresent()) { + LOG.info("Last commit time synced is not known, listing all partitions in " + + config.getString(META_SYNC_BASE_PATH) + + ",FS :" + config.getHadoopFileSystem()); + HoodieLocalEngineContext engineContext = new HoodieLocalEngineContext(metaClient.getHadoopConf()); + return FSUtils.getAllPartitionPaths(engineContext, + config.getString(META_SYNC_BASE_PATH), + config.getBoolean(META_SYNC_USE_FILE_LISTING_FROM_METADATA), + config.getBoolean(META_SYNC_ASSUME_DATE_PARTITION)); + } else { + LOG.info("Last commit time synced is " + lastCommitTimeSynced.get() + ", Getting commits since then"); + return TimelineUtils.getPartitionsWritten(metaClient.getActiveTimeline().getCommitsTimeline() + .findInstantsAfter(lastCommitTimeSynced.get(), Integer.MAX_VALUE)); + } + } + + /** + * Iterate over the storage partitions and find if there are any new partitions that need to be added or updated. + * Generate a list of PartitionEvent based on the changes required. + */ + public List getPartitionEvents(List tablePartitions, List partitionStoragePartitions, boolean isDropPartition) { + Map paths = new HashMap<>(); + for (Partition tablePartition : tablePartitions) { + List hivePartitionValues = tablePartition.getValues(); + String fullTablePartitionPath = + Path.getPathWithoutSchemeAndAuthority(new Path(tablePartition.getStorageLocation())).toUri().getPath(); + paths.put(String.join(", ", hivePartitionValues), fullTablePartitionPath); + } + + List events = new ArrayList<>(); + for (String storagePartition : partitionStoragePartitions) { + Path storagePartitionPath = FSUtils.getPartitionPath(config.getString(META_SYNC_BASE_PATH), storagePartition); + String fullStoragePartitionPath = Path.getPathWithoutSchemeAndAuthority(storagePartitionPath).toUri().getPath(); + // Check if the partition values or if hdfs path is the same + List storagePartitionValues = partitionValueExtractor.extractPartitionValuesInPath(storagePartition); + + if (isDropPartition) { + events.add(PartitionEvent.newPartitionDropEvent(storagePartition)); + } else { + if (!storagePartitionValues.isEmpty()) { + String storageValue = String.join(", ", storagePartitionValues); + if (!paths.containsKey(storageValue)) { + events.add(PartitionEvent.newPartitionAddEvent(storagePartition)); + } else if (!paths.get(storageValue).equals(fullStoragePartitionPath)) { + events.add(PartitionEvent.newPartitionUpdateEvent(storagePartition)); + } + } + } + } + return events; + } +} diff --git a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncConfig.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncConfig.java index dc2b21ba4504f..ba763ddc14bb7 100644 --- a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncConfig.java +++ b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncConfig.java @@ -22,14 +22,19 @@ import org.apache.hudi.common.config.HoodieConfig; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; +import org.apache.hudi.sync.common.util.ConfigUtils; import com.beust.jcommander.Parameter; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; -import java.util.Collections; import java.util.List; +import java.util.Properties; import java.util.function.Function; /** @@ -37,41 +42,6 @@ */ public class HoodieSyncConfig extends HoodieConfig { - @Parameter(names = {"--database"}, description = "name of the target database in meta store", required = true) - public String databaseName; - - @Parameter(names = {"--table"}, description = "name of the target table in meta store", required = true) - public String tableName; - - @Parameter(names = {"--base-path"}, description = "Base path of the hoodie table to sync", required = true) - public String basePath; - - @Parameter(names = {"--base-file-format"}, description = "Format of the base files (PARQUET (or) HFILE)") - public String baseFileFormat; - - @Parameter(names = "--partitioned-by", description = "Fields in the schema partitioned by") - public List partitionFields; - - @Parameter(names = "--partition-value-extractor", description = "Class which implements PartitionValueExtractor " - + "to extract the partition values from HDFS path") - public String partitionValueExtractorClass; - - @Parameter(names = {"--assume-date-partitioning"}, description = "Assume standard yyyy/mm/dd partitioning, this" - + " exists to support backward compatibility. If you use hoodie 0.3.x, do not set this parameter") - public Boolean assumeDatePartitioning; - - @Parameter(names = {"--decode-partition"}, description = "Decode the partition value if the partition has encoded during writing") - public Boolean decodePartition; - - @Parameter(names = {"--use-file-listing-from-metadata"}, description = "Fetch file listing from Hudi's metadata") - public Boolean useFileListingFromMetadata; - - @Parameter(names = {"--conditional-sync"}, description = "If true, only sync on conditions like schema change or partition change.") - public Boolean isConditionalSync; - - @Parameter(names = {"--spark-version"}, description = "The spark version") - public String sparkVersion; - public static final ConfigProperty META_SYNC_BASE_PATH = ConfigProperty .key("hoodie.datasource.meta.sync.base.path") .defaultValue("") @@ -150,6 +120,11 @@ public class HoodieSyncConfig extends HoodieConfig { .defaultValue("false") .withDocumentation("Assume partitioning is yyyy/mm/dd"); + public static final ConfigProperty META_SYNC_DECODE_PARTITION = ConfigProperty + .key("hoodie.meta.sync.decode_partition") + .defaultValue(false) // TODO infer from url encode option + .withDocumentation(""); + public static final ConfigProperty META_SYNC_USE_FILE_LISTING_FROM_METADATA = ConfigProperty .key("hoodie.meta.sync.metadata_file_listing") .defaultValue(HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS) @@ -165,24 +140,85 @@ public class HoodieSyncConfig extends HoodieConfig { .defaultValue("") .withDocumentation("The spark version used when syncing with a metastore."); - public HoodieSyncConfig(TypedProperties props) { + private Configuration hadoopConf; + + public HoodieSyncConfig(Properties props) { + this(props, ConfigUtils.createHadoopConf(props)); + } + + public HoodieSyncConfig(Properties props, Configuration hadoopConf) { super(props); - setDefaults(); - - this.basePath = getStringOrDefault(META_SYNC_BASE_PATH); - this.databaseName = getStringOrDefault(META_SYNC_DATABASE_NAME); - this.tableName = getStringOrDefault(META_SYNC_TABLE_NAME); - this.baseFileFormat = getStringOrDefault(META_SYNC_BASE_FILE_FORMAT); - this.partitionFields = props.getStringList(META_SYNC_PARTITION_FIELDS.key(), ",", Collections.emptyList()); - this.partitionValueExtractorClass = getStringOrDefault(META_SYNC_PARTITION_EXTRACTOR_CLASS); - this.assumeDatePartitioning = getBooleanOrDefault(META_SYNC_ASSUME_DATE_PARTITION); - this.decodePartition = getBooleanOrDefault(KeyGeneratorOptions.URL_ENCODE_PARTITIONING); - this.useFileListingFromMetadata = getBooleanOrDefault(META_SYNC_USE_FILE_LISTING_FROM_METADATA); - this.isConditionalSync = getBooleanOrDefault(META_SYNC_CONDITIONAL_SYNC); - this.sparkVersion = getStringOrDefault(META_SYNC_SPARK_VERSION); + this.hadoopConf = hadoopConf; + } + + public void setHadoopConf(Configuration hadoopConf) { + this.hadoopConf = hadoopConf; + } + + public Configuration getHadoopConf() { + return hadoopConf; + } + + public FileSystem getHadoopFileSystem() { + return FSUtils.getFs(getString(META_SYNC_BASE_PATH), getHadoopConf()); } - protected void setDefaults() { - this.setDefaultValue(META_SYNC_TABLE_NAME); + public String getAbsoluteBasePath() { + return getString(META_SYNC_BASE_PATH); + } + + @Override + public String toString() { + return props.toString(); + } + + public static class HoodieSyncConfigParams { + @Parameter(names = {"--database"}, description = "name of the target database in meta store", required = true) + public String databaseName; + @Parameter(names = {"--table"}, description = "name of the target table in meta store", required = true) + public String tableName; + @Parameter(names = {"--base-path"}, description = "Base path of the hoodie table to sync", required = true) + public String basePath; + @Parameter(names = {"--base-file-format"}, description = "Format of the base files (PARQUET (or) HFILE)") + public String baseFileFormat; + @Parameter(names = "--partitioned-by", description = "Fields in the schema partitioned by") + public List partitionFields; + @Parameter(names = "--partition-value-extractor", description = "Class which implements PartitionValueExtractor " + + "to extract the partition values from HDFS path") + public String partitionValueExtractorClass; + @Parameter(names = {"--assume-date-partitioning"}, description = "Assume standard yyyy/mm/dd partitioning, this" + + " exists to support backward compatibility. If you use hoodie 0.3.x, do not set this parameter") + public Boolean assumeDatePartitioning; + @Parameter(names = {"--decode-partition"}, description = "Decode the partition value if the partition has encoded during writing") + public Boolean decodePartition; + @Parameter(names = {"--use-file-listing-from-metadata"}, description = "Fetch file listing from Hudi's metadata") + public Boolean useFileListingFromMetadata; + @Parameter(names = {"--conditional-sync"}, description = "If true, only sync on conditions like schema change or partition change.") + public Boolean isConditionalSync; + @Parameter(names = {"--spark-version"}, description = "The spark version") + public String sparkVersion; + + @Parameter(names = {"--help", "-h"}, help = true) + public boolean help = false; + + public boolean isHelp() { + return help; + } + + public TypedProperties toProps() { + final TypedProperties props = new TypedProperties(); + props.setPropertyIfNonNull(META_SYNC_BASE_PATH.key(), basePath); + props.setPropertyIfNonNull(META_SYNC_DATABASE_NAME.key(), databaseName); + props.setPropertyIfNonNull(META_SYNC_TABLE_NAME.key(), tableName); + props.setPropertyIfNonNull(META_SYNC_BASE_FILE_FORMAT.key(), baseFileFormat); + props.setPropertyIfNonNull(META_SYNC_PARTITION_FIELDS.key(), StringUtils.join(",", partitionFields)); + props.setPropertyIfNonNull(META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), partitionValueExtractorClass); + props.setPropertyIfNonNull(META_SYNC_ASSUME_DATE_PARTITION.key(), assumeDatePartitioning); + props.setPropertyIfNonNull(META_SYNC_DECODE_PARTITION.key(), decodePartition); + props.setPropertyIfNonNull(META_SYNC_USE_FILE_LISTING_FROM_METADATA.key(), useFileListingFromMetadata); + props.setPropertyIfNonNull(META_SYNC_CONDITIONAL_SYNC.key(), isConditionalSync); + props.setPropertyIfNonNull(META_SYNC_SPARK_VERSION.key(), sparkVersion); + return props; + } } } diff --git a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncTool.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncTool.java new file mode 100644 index 0000000000000..dd55480eb86fe --- /dev/null +++ b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncTool.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.sync.common; + +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.sync.common.util.ConfigUtils; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; + +import java.util.Properties; + +/** + * Base class to sync metadata with metastores to make + * Hudi table queryable through external systems. + */ +public abstract class HoodieSyncTool implements AutoCloseable { + + protected Properties props; + protected Configuration hadoopConf; + + public HoodieSyncTool(Properties props) { + this(props, ConfigUtils.createHadoopConf(props)); + } + + public HoodieSyncTool(Properties props, Configuration hadoopConf) { + this.props = props; + this.hadoopConf = hadoopConf; + } + + @Deprecated + public HoodieSyncTool(TypedProperties props, Configuration conf, FileSystem fs) { + this(props, conf); + } + + @Deprecated + public HoodieSyncTool(Properties props, FileSystem fileSystem) { + this(props, fileSystem.getConf()); + } + + public abstract void syncHoodieTable(); + + @Override + public void close() throws Exception { + // no op + } +} diff --git a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/model/FieldSchema.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/model/FieldSchema.java new file mode 100644 index 0000000000000..d9506be0809ac --- /dev/null +++ b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/model/FieldSchema.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.sync.common.model; + +import org.apache.hudi.common.util.Option; + +import java.util.Objects; + +public class FieldSchema { + + private final String name; + private String type; + private Option comment; + + public FieldSchema(String name, String type) { + this(name, type, Option.empty()); + } + + public FieldSchema(String name, String type, String comment) { + this(name, type, Option.ofNullable(comment)); + } + + public FieldSchema(String name, String type, Option comment) { + this.name = name; + this.type = type; + this.comment = comment; + } + + public String getName() { + return name; + } + + public String getType() { + return type; + } + + public Option getComment() { + return comment; + } + + public String getCommentOrEmpty() { + return comment.orElse(""); + } + + public void setType(String type) { + this.type = type; + } + + public void setComment(Option comment) { + this.comment = comment; + } + + public void setComment(String comment) { + this.comment = Option.ofNullable(comment); + } + + public boolean updateComment(FieldSchema another) { + if (Objects.equals(name, another.getName()) + && !Objects.equals(getCommentOrEmpty(), another.getCommentOrEmpty())) { + setComment(another.getComment()); + return true; + } else { + return false; + } + } +} diff --git a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/model/PartitionEvent.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/model/PartitionEvent.java new file mode 100644 index 0000000000000..ebdc90af4bfb9 --- /dev/null +++ b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/model/PartitionEvent.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.sync.common.model; + +/** + * Partition Event captures any partition that needs to be added or updated. + */ +public class PartitionEvent { + + public enum PartitionEventType { + ADD, UPDATE, DROP + } + + public PartitionEventType eventType; + public String storagePartition; + + PartitionEvent(PartitionEventType eventType, String storagePartition) { + this.eventType = eventType; + this.storagePartition = storagePartition; + } + + public static PartitionEvent newPartitionAddEvent(String storagePartition) { + return new PartitionEvent(PartitionEventType.ADD, storagePartition); + } + + public static PartitionEvent newPartitionUpdateEvent(String storagePartition) { + return new PartitionEvent(PartitionEventType.UPDATE, storagePartition); + } + + public static PartitionEvent newPartitionDropEvent(String storagePartition) { + return new PartitionEvent(PartitionEventType.DROP, storagePartition); + } +} diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/PartitionValueExtractor.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/model/PartitionValueExtractor.java similarity index 69% rename from hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/PartitionValueExtractor.java rename to hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/model/PartitionValueExtractor.java index f4820e3162f1d..60d080a0ffdd8 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/PartitionValueExtractor.java +++ b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/model/PartitionValueExtractor.java @@ -7,16 +7,17 @@ * "License"); you may not use this file except in compliance * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. */ -package org.apache.hudi.hive; +package org.apache.hudi.sync.common.model; import java.io.Serializable; import java.util.List; diff --git a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/ConfigUtils.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/ConfigUtils.java index ca5224aef4697..9c9979d5539bf 100644 --- a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/ConfigUtils.java +++ b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/ConfigUtils.java @@ -18,9 +18,13 @@ package org.apache.hudi.sync.common.util; +import org.apache.hudi.common.util.StringUtils; + +import org.apache.hadoop.conf.Configuration; + import java.util.HashMap; import java.util.Map; -import org.apache.hudi.common.util.StringUtils; +import java.util.Properties; public class ConfigUtils { /** @@ -32,6 +36,7 @@ public class ConfigUtils { /** * Convert the key-value config to a map.The format of the config * is a key-value pair just like "k1=v1\nk2=v2\nk3=v3". + * * @param keyValueConfig * @return */ @@ -49,7 +54,7 @@ public static Map toMap(String keyValueConfig) { tableProperties.put(key, value); } else { throw new IllegalArgumentException("Bad key-value config: " + keyValue + ", must be the" - + " format 'key = value'"); + + " format 'key = value'"); } } return tableProperties; @@ -58,6 +63,7 @@ public static Map toMap(String keyValueConfig) { /** * Convert map config to key-value string.The format of the config * is a key-value pair just like "k1=v1\nk2=v2\nk3=v3". + * * @param config * @return */ @@ -75,4 +81,10 @@ public static String configToString(Map config) { return sb.toString(); } + public static Configuration createHadoopConf(Properties props) { + Configuration hadoopConf = new Configuration(); + props.stringPropertyNames().forEach(k -> hadoopConf.set(k, props.getProperty(k))); + return hadoopConf; + } + } diff --git a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/AbstractSyncTool.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/SparkDataSourceTableUtils.java similarity index 69% rename from hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/AbstractSyncTool.java rename to hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/SparkDataSourceTableUtils.java index 972ae1f96c512..31c845397d700 100644 --- a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/AbstractSyncTool.java +++ b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/SparkDataSourceTableUtils.java @@ -1,12 +1,13 @@ /* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at * - * http://www.apache.org/licenses/LICENSE-2.0 + * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, @@ -15,15 +16,10 @@ * limitations under the License. */ -package org.apache.hudi.sync.common; +package org.apache.hudi.sync.common.util; -import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.util.StringUtils; -import org.apache.hudi.sync.common.util.ConfigUtils; -import org.apache.hudi.sync.common.util.Parquet2SparkSchemaUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; import org.apache.parquet.schema.GroupType; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; @@ -33,40 +29,18 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Properties; import static org.apache.parquet.schema.OriginalType.UTF8; import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY; -/** - * Base class to sync Hudi meta data with Metastores to make - * Hudi table queryable through external systems. - */ -public abstract class AbstractSyncTool { - protected final Configuration conf; - protected final FileSystem fs; - protected TypedProperties props; - - public AbstractSyncTool(TypedProperties props, Configuration conf, FileSystem fs) { - this.props = props; - this.conf = conf; - this.fs = fs; - } - - @Deprecated - public AbstractSyncTool(Properties props, FileSystem fileSystem) { - this(new TypedProperties(props), fileSystem.getConf(), fileSystem); - } - - public abstract void syncHoodieTable(); - +public class SparkDataSourceTableUtils { /** * Get Spark Sql related table properties. This is used for spark datasource table. * @param schema The schema to write to the table. * @return A new parameters added the spark's table properties. */ - protected Map getSparkTableProperties(List partitionNames, String sparkVersion, - int schemaLengthThreshold, MessageType schema) { + public static Map getSparkTableProperties(List partitionNames, String sparkVersion, + int schemaLengthThreshold, MessageType schema) { // Convert the schema and partition info used by spark sql to hive table properties. // The following code refers to the spark code in // https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala @@ -122,7 +96,7 @@ protected Map getSparkTableProperties(List partitionName return sparkProperties; } - protected Map getSparkSerdeProperties(boolean readAsOptimized, String basePath) { + public static Map getSparkSerdeProperties(boolean readAsOptimized, String basePath) { Map sparkSerdeProperties = new HashMap<>(); sparkSerdeProperties.put("path", basePath); sparkSerdeProperties.put(ConfigUtils.IS_QUERY_AS_RO_TABLE, String.valueOf(readAsOptimized)); diff --git a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/SyncUtilHelpers.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/SyncUtilHelpers.java index def85c5b805d9..25b19be924003 100644 --- a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/SyncUtilHelpers.java +++ b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/SyncUtilHelpers.java @@ -22,13 +22,11 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.sync.common.AbstractSyncTool; import org.apache.hudi.sync.common.HoodieSyncConfig; +import org.apache.hudi.sync.common.HoodieSyncTool; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; -import org.apache.log4j.LogManager; -import org.apache.log4j.Logger; import java.util.Properties; @@ -36,13 +34,12 @@ * Helper class for syncing Hudi commit data with external metastores. */ public class SyncUtilHelpers { - private static final Logger LOG = LogManager.getLogger(SyncUtilHelpers.class); /** - * Create an instance of an implementation of {@link AbstractSyncTool} that will sync all the relevant meta information + * Create an instance of an implementation of {@link HoodieSyncTool} that will sync all the relevant meta information * with an external metastore such as Hive etc. to ensure Hoodie tables can be queried or read via external systems. * - * @param metaSyncFQCN The class that implements the sync of the metadata. + * @param metaSyncFQCN The class that implements the sync of the metadata. * @param props property map. * @param hadoopConfig Hadoop confs. * @param fs Filesystem used. @@ -62,30 +59,40 @@ public static void runHoodieMetaSync(String metaSyncFQCN, } } - static AbstractSyncTool instantiateMetaSyncTool(String metaSyncFQCN, - TypedProperties props, - Configuration hadoopConfig, - FileSystem fs, - String targetBasePath, - String baseFileFormat) { + static HoodieSyncTool instantiateMetaSyncTool(String metaSyncFQCN, + TypedProperties props, + Configuration hadoopConfig, + FileSystem fs, + String targetBasePath, + String baseFileFormat) { TypedProperties properties = new TypedProperties(); properties.putAll(props); properties.put(HoodieSyncConfig.META_SYNC_BASE_PATH.key(), targetBasePath); properties.put(HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT.key(), baseFileFormat); if (ReflectionUtils.hasConstructor(metaSyncFQCN, + new Class[] {Properties.class, Configuration.class})) { + return ((HoodieSyncTool) ReflectionUtils.loadClass(metaSyncFQCN, + new Class[] {Properties.class, Configuration.class}, + properties, hadoopConfig)); + } else if (ReflectionUtils.hasConstructor(metaSyncFQCN, + new Class[] {Properties.class})) { + return ((HoodieSyncTool) ReflectionUtils.loadClass(metaSyncFQCN, + new Class[] {Properties.class}, + properties)); + } else if (ReflectionUtils.hasConstructor(metaSyncFQCN, new Class[] {TypedProperties.class, Configuration.class, FileSystem.class})) { - return ((AbstractSyncTool) ReflectionUtils.loadClass(metaSyncFQCN, + return ((HoodieSyncTool) ReflectionUtils.loadClass(metaSyncFQCN, new Class[] {TypedProperties.class, Configuration.class, FileSystem.class}, properties, hadoopConfig, fs)); + } else if (ReflectionUtils.hasConstructor(metaSyncFQCN, + new Class[] {Properties.class, FileSystem.class})) { + return ((HoodieSyncTool) ReflectionUtils.loadClass(metaSyncFQCN, + new Class[] {Properties.class, FileSystem.class}, + properties, fs)); } else { - LOG.warn("Falling back to deprecated constructor for class: " + metaSyncFQCN); - try { - return ((AbstractSyncTool) ReflectionUtils.loadClass(metaSyncFQCN, - new Class[] {Properties.class, FileSystem.class}, properties, fs)); - } catch (Throwable t) { - throw new HoodieException("Could not load meta sync class " + metaSyncFQCN, t); - } + throw new HoodieException("Could not load meta sync class " + metaSyncFQCN + + ": no valid constructor found."); } } } diff --git a/hudi-sync/hudi-sync-common/src/test/java/org/apache/hudi/sync/common/util/TestSyncUtilHelpers.java b/hudi-sync/hudi-sync-common/src/test/java/org/apache/hudi/sync/common/util/TestSyncUtilHelpers.java index dc9dee8b42ea4..5e9f71590203e 100644 --- a/hudi-sync/hudi-sync-common/src/test/java/org/apache/hudi/sync/common/util/TestSyncUtilHelpers.java +++ b/hudi-sync/hudi-sync-common/src/test/java/org/apache/hudi/sync/common/util/TestSyncUtilHelpers.java @@ -20,16 +20,19 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.sync.common.AbstractSyncTool; +import org.apache.hudi.sync.common.HoodieSyncTool; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import java.io.IOException; import java.util.Properties; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -46,42 +49,44 @@ public void setUp() throws IOException { hadoopConf = fileSystem.getConf(); } - @Test - public void testCreateValidSyncClass() { - AbstractSyncTool metaSyncTool = SyncUtilHelpers.instantiateMetaSyncTool( - ValidMetaSyncClass.class.getName(), + @ParameterizedTest + @ValueSource(classes = {DummySyncTool1.class, DummySyncTool2.class}) + public void testCreateValidSyncClass(Class clazz) { + HoodieSyncTool syncTool = SyncUtilHelpers.instantiateMetaSyncTool( + clazz.getName(), new TypedProperties(), hadoopConf, fileSystem, BASE_PATH, BASE_FORMAT ); - assertTrue(metaSyncTool instanceof ValidMetaSyncClass); + assertTrue(clazz.isAssignableFrom(syncTool.getClass())); } /** - * Ensure it still works for the deprecated constructor of {@link AbstractSyncTool} + * Ensure it still works for the deprecated constructor of {@link HoodieSyncTool} * as we implemented the fallback. */ - @Test - public void testCreateDeprecatedSyncClass() { + @ParameterizedTest + @ValueSource(classes = {DeprecatedSyncTool1.class, DeprecatedSyncTool2.class}) + public void testCreateDeprecatedSyncClass(Class clazz) { Properties properties = new Properties(); - AbstractSyncTool deprecatedMetaSyncClass = SyncUtilHelpers.instantiateMetaSyncTool( - DeprecatedMetaSyncClass.class.getName(), + HoodieSyncTool syncTool = SyncUtilHelpers.instantiateMetaSyncTool( + clazz.getName(), new TypedProperties(properties), hadoopConf, fileSystem, BASE_PATH, BASE_FORMAT ); - assertTrue(deprecatedMetaSyncClass instanceof DeprecatedMetaSyncClass); + assertTrue(clazz.isAssignableFrom(syncTool.getClass())); } @Test public void testCreateInvalidSyncClass() { - Exception exception = assertThrows(HoodieException.class, () -> { + Throwable t = assertThrows(HoodieException.class, () -> { SyncUtilHelpers.instantiateMetaSyncTool( - InvalidSyncClass.class.getName(), + InvalidSyncTool.class.getName(), new TypedProperties(), hadoopConf, fileSystem, @@ -90,14 +95,36 @@ public void testCreateInvalidSyncClass() { ); }); - String expectedMessage = "Could not load meta sync class " + InvalidSyncClass.class.getName(); - assertTrue(exception.getMessage().contains(expectedMessage)); + String expectedMessage = "Could not load meta sync class " + InvalidSyncTool.class.getName() + + ": no valid constructor found."; + assertEquals(expectedMessage, t.getMessage()); + } + + public static class DummySyncTool1 extends HoodieSyncTool { + public DummySyncTool1(Properties props, Configuration hadoopConf) { + super(props, hadoopConf); + } + @Override + public void syncHoodieTable() { + throw new HoodieException("Method unimplemented as its a test class"); + } + } + + public static class DummySyncTool2 extends HoodieSyncTool { + public DummySyncTool2(Properties props, Configuration hadoopConf) { + super(props, hadoopConf); + } + + @Override + public void syncHoodieTable() { + throw new HoodieException("Method unimplemented as its a test class"); + } } - public static class ValidMetaSyncClass extends AbstractSyncTool { - public ValidMetaSyncClass(TypedProperties props, Configuration conf, FileSystem fs) { - super(props, conf, fs); + public static class DeprecatedSyncTool1 extends HoodieSyncTool { + public DeprecatedSyncTool1(TypedProperties props, Configuration hadoopConf, FileSystem fs) { + super(props, hadoopConf, fs); } @Override @@ -106,9 +133,9 @@ public void syncHoodieTable() { } } - public static class DeprecatedMetaSyncClass extends AbstractSyncTool { - public DeprecatedMetaSyncClass(Properties props, FileSystem fileSystem) { - super(props, fileSystem); + public static class DeprecatedSyncTool2 extends HoodieSyncTool { + public DeprecatedSyncTool2(Properties props, FileSystem fs) { + super(props, fs); } @Override @@ -117,8 +144,8 @@ public void syncHoodieTable() { } } - public static class InvalidSyncClass { - public InvalidSyncClass(Properties props) { + public static class InvalidSyncTool { + public InvalidSyncTool(Properties props, FileSystem fs, Configuration hadoopConf) { } } } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BootstrapExecutor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BootstrapExecutor.java index 7e605dbd36a0a..828dd6af5f444 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BootstrapExecutor.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BootstrapExecutor.java @@ -33,7 +33,6 @@ 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.hudi.utilities.UtilHelpers; import org.apache.hudi.utilities.schema.SchemaProvider; @@ -49,6 +48,10 @@ import java.util.HashMap; import static org.apache.hudi.common.table.HoodieTableConfig.ARCHIVELOG_FOLDER; +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. @@ -162,14 +165,14 @@ private void syncHive() { if (cfg.enableHiveSync || cfg.enableMetaSync) { TypedProperties metaProps = new TypedProperties(); metaProps.putAll(props); - metaProps.put(HoodieSyncConfig.META_SYNC_BASE_PATH.key(), cfg.targetBasePath); - 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()), + metaProps.put(META_SYNC_BASE_PATH.key(), cfg.targetBasePath); + 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(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key()), props.getInteger(HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS.key()))); } - new HiveSyncTool(metaProps, configuration, fs).syncHoodieTable(); + new HiveSyncTool(metaProps, configuration).syncHoodieTable(); } } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java index 736e416162d21..5940ab1d46f25 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java @@ -114,6 +114,8 @@ import static org.apache.hudi.config.HoodieWriteConfig.AUTO_COMMIT_ENABLE; import static org.apache.hudi.config.HoodieWriteConfig.COMBINE_BEFORE_INSERT; import static org.apache.hudi.config.HoodieWriteConfig.COMBINE_BEFORE_UPSERT; +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.utilities.deltastreamer.HoodieDeltaStreamer.CHECKPOINT_KEY; import static org.apache.hudi.utilities.deltastreamer.HoodieDeltaStreamer.CHECKPOINT_RESET_KEY; import static org.apache.hudi.utilities.schema.RowBasedSchemaProvider.HOODIE_RECORD_NAMESPACE; @@ -629,7 +631,7 @@ private Pair, JavaRDD> writeToSink(JavaRDD syncClientToolClasses = new HashSet<>(Arrays.asList(cfg.syncClientToolClassNames.split(","))); // for backward compatibility if (cfg.enableHiveSync) { @@ -703,8 +705,8 @@ private void syncMeta(HoodieDeltaStreamerMetrics metrics) { TypedProperties metaProps = new TypedProperties(); metaProps.putAll(props); - 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()), + 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(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key()), props.getInteger(HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS.key()))); } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHiveIncrementalPuller.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHiveIncrementalPuller.java index d6837a384aa0d..1dcca13a82575 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHiveIncrementalPuller.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHiveIncrementalPuller.java @@ -18,16 +18,15 @@ package org.apache.hudi.utilities; -import org.apache.hadoop.hive.metastore.api.MetaException; -import org.apache.hadoop.hive.ql.metadata.HiveException; - import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.hive.HiveSyncConfig; import org.apache.hudi.hive.HiveSyncTool; -import org.apache.hudi.hive.HoodieHiveClient; +import org.apache.hudi.hive.HoodieHiveSyncClient; import org.apache.hudi.hive.testutils.HiveTestUtil; -import org.apache.hudi.sync.common.HoodieSyncConfig; import org.apache.hudi.utilities.exception.HoodieIncrementalPullSQLException; + +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.ql.metadata.HiveException; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -40,8 +39,14 @@ import java.nio.file.Paths; import java.time.Instant; -import static org.apache.hudi.hive.testutils.HiveTestUtil.fileSystem; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_PASS; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_MODE; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_URL; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USER; import static org.apache.hudi.hive.testutils.HiveTestUtil.hiveSyncProps; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_BASE_PATH; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_TABLE_NAME; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -72,12 +77,12 @@ public void testInitHiveIncrementalPuller() { } private HiveIncrementalPuller.Config getHivePullerConfig(String incrementalSql) throws IOException { - config.hiveJDBCUrl = hiveSyncProps.getString(HiveSyncConfig.HIVE_URL.key()); - config.hiveUsername = hiveSyncProps.getString(HiveSyncConfig.HIVE_USER.key()); - config.hivePassword = hiveSyncProps.getString(HiveSyncConfig.HIVE_PASS.key()); + config.hiveJDBCUrl = hiveSyncProps.getString(HIVE_URL.key()); + config.hiveUsername = hiveSyncProps.getString(HIVE_USER.key()); + config.hivePassword = hiveSyncProps.getString(HIVE_PASS.key()); config.hoodieTmpDir = Files.createTempDirectory("hivePullerTest").toUri().toString(); - config.sourceDb = hiveSyncProps.getString(HoodieSyncConfig.META_SYNC_DATABASE_NAME.key()); - config.sourceTable = hiveSyncProps.getString(HoodieSyncConfig.META_SYNC_TABLE_NAME.key()); + config.sourceDb = hiveSyncProps.getString(META_SYNC_DATABASE_NAME.key()); + config.sourceTable = hiveSyncProps.getString(META_SYNC_TABLE_NAME.key()); config.targetDb = "tgtdb"; config.targetTable = "test2"; config.tmpDb = "tmp_db"; @@ -101,9 +106,8 @@ private void createIncrementalSqlFile(String text, HiveIncrementalPuller.Config private void createSourceTable() throws IOException, URISyntaxException { String instantTime = "101"; HiveTestUtil.createCOWTable(instantTime, 5, true); - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), "jdbc"); - - HiveSyncTool tool = new HiveSyncTool(hiveSyncProps, HiveTestUtil.getHiveConf(), fileSystem); + hiveSyncProps.setProperty(HIVE_SYNC_MODE.key(), "jdbc"); + HiveSyncTool tool = new HiveSyncTool(hiveSyncProps, HiveTestUtil.getHiveConf()); tool.syncHoodieTable(); } @@ -112,23 +116,23 @@ private void createTargetTable() throws IOException, URISyntaxException { targetBasePath = Files.createTempDirectory("hivesynctest1" + Instant.now().toEpochMilli()).toUri().toString(); HiveTestUtil.createCOWTable(instantTime, 5, true, targetBasePath, "tgtdb", "test2"); - HiveSyncTool tool = new HiveSyncTool(getTargetHiveSyncConfig(targetBasePath), HiveTestUtil.getHiveConf(), fileSystem); + HiveSyncTool tool = new HiveSyncTool(getTargetHiveSyncConfig(targetBasePath), HiveTestUtil.getHiveConf()); tool.syncHoodieTable(); } private TypedProperties getTargetHiveSyncConfig(String basePath) { TypedProperties targetHiveSyncProps = new TypedProperties(hiveSyncProps); - targetHiveSyncProps.setProperty(HoodieSyncConfig.META_SYNC_DATABASE_NAME.key(), "tgtdb"); - targetHiveSyncProps.setProperty(HoodieSyncConfig.META_SYNC_TABLE_NAME.key(), "test2"); - targetHiveSyncProps.setProperty(HoodieSyncConfig.META_SYNC_BASE_PATH.key(), basePath); - targetHiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), "jdbc"); + targetHiveSyncProps.setProperty(META_SYNC_DATABASE_NAME.key(), "tgtdb"); + targetHiveSyncProps.setProperty(META_SYNC_TABLE_NAME.key(), "test2"); + targetHiveSyncProps.setProperty(META_SYNC_BASE_PATH.key(), basePath); + targetHiveSyncProps.setProperty(HIVE_SYNC_MODE.key(), "jdbc"); return targetHiveSyncProps; } private TypedProperties getAssertionSyncConfig(String databaseName) { TypedProperties assertHiveSyncProps = new TypedProperties(hiveSyncProps); - assertHiveSyncProps.setProperty(HoodieSyncConfig.META_SYNC_DATABASE_NAME.key(), databaseName); + assertHiveSyncProps.setProperty(META_SYNC_DATABASE_NAME.key(), databaseName); return assertHiveSyncProps; } @@ -161,11 +165,11 @@ public void testPullerWithoutSourceInSql() throws IOException, URISyntaxExceptio public void testPuller() throws IOException, URISyntaxException { createTables(); HiveIncrementalPuller.Config cfg = getHivePullerConfig("select name from testdb.test1 where `_hoodie_commit_time` > '%s'"); - HoodieHiveClient hiveClient = new HoodieHiveClient(new HiveSyncConfig(hiveSyncProps), HiveTestUtil.getHiveConf(), fileSystem); + HoodieHiveSyncClient hiveClient = new HoodieHiveSyncClient(new HiveSyncConfig(hiveSyncProps, HiveTestUtil.getHiveConf())); hiveClient.createDatabase(cfg.tmpDb); HiveIncrementalPuller puller = new HiveIncrementalPuller(cfg); puller.saveDelta(); - HoodieHiveClient assertingClient = new HoodieHiveClient(new HiveSyncConfig(getAssertionSyncConfig(cfg.tmpDb)), HiveTestUtil.getHiveConf(), fileSystem); + HoodieHiveSyncClient assertingClient = new HoodieHiveSyncClient(new HiveSyncConfig(getAssertionSyncConfig(cfg.tmpDb), HiveTestUtil.getHiveConf())); String tmpTable = cfg.targetTable + "__" + cfg.sourceTable; assertTrue(assertingClient.tableExists(tmpTable)); } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/HoodieDeltaStreamerTestBase.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/HoodieDeltaStreamerTestBase.java index cc6f90790c732..b4497289fd34a 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/HoodieDeltaStreamerTestBase.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/HoodieDeltaStreamerTestBase.java @@ -27,7 +27,6 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; -import org.apache.hudi.hive.HiveSyncConfig; import org.apache.hudi.hive.MultiPartKeysValueExtractor; import org.apache.hudi.utilities.schema.FilebasedSchemaProvider; import org.apache.hudi.utilities.sources.TestDataSource; @@ -48,6 +47,13 @@ import java.util.Map; import java.util.Random; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_URL; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_ASSUME_DATE_PARTITION; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_PARTITION_FIELDS; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_TABLE_NAME; + public class HoodieDeltaStreamerTestBase extends UtilitiesTestBase { @@ -180,11 +186,11 @@ protected static void writeCommonPropsToFile(FileSystem dfs, String dfsBasePath) props.setProperty("hoodie.deltastreamer.schemaprovider.target.schema.file", dfsBasePath + "/target.avsc"); // Hive Configs - props.setProperty(HiveSyncConfig.HIVE_URL.key(), "jdbc:hive2://127.0.0.1:9999/"); - props.setProperty(HiveSyncConfig.META_SYNC_DATABASE_NAME.key(), "testdb1"); - props.setProperty(HiveSyncConfig.META_SYNC_TABLE_NAME.key(), "hive_trips"); - props.setProperty(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "datestr"); - props.setProperty(HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), + props.setProperty(HIVE_URL.key(), "jdbc:hive2://127.0.0.1:9999/"); + props.setProperty(META_SYNC_DATABASE_NAME.key(), "testdb1"); + props.setProperty(META_SYNC_TABLE_NAME.key(), "hive_trips"); + props.setProperty(META_SYNC_PARTITION_FIELDS.key(), "datestr"); + props.setProperty(META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), MultiPartKeysValueExtractor.class.getName()); UtilitiesTestBase.Helpers.savePropsToDFS(props, dfs, dfsBasePath + "/" + PROPS_FILENAME_TEST_SOURCE); } @@ -240,11 +246,11 @@ protected static void populateCommonKafkaProps(TypedProperties props, String bro protected static void populateCommonHiveProps(TypedProperties props) { // Hive Configs - props.setProperty(HiveSyncConfig.HIVE_URL.key(), "jdbc:hive2://127.0.0.1:9999/"); - props.setProperty(HiveSyncConfig.META_SYNC_DATABASE_NAME.key(), "testdb2"); - props.setProperty(HiveSyncConfig.META_SYNC_ASSUME_DATE_PARTITION.key(), "false"); - props.setProperty(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "datestr"); - props.setProperty(HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), + props.setProperty(HIVE_URL.key(), "jdbc:hive2://127.0.0.1:9999/"); + props.setProperty(META_SYNC_DATABASE_NAME.key(), "testdb2"); + props.setProperty(META_SYNC_ASSUME_DATE_PARTITION.key(), "false"); + props.setProperty(META_SYNC_PARTITION_FIELDS.key(), "datestr"); + props.setProperty(META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), MultiPartKeysValueExtractor.class.getName()); } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java index ae38968187793..45d8a427c0436 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHoodieDeltaStreamer.java @@ -43,7 +43,6 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestUtils; -import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieClusteringConfig; @@ -53,7 +52,7 @@ import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.TableNotFoundException; import org.apache.hudi.hive.HiveSyncConfig; -import org.apache.hudi.hive.HoodieHiveClient; +import org.apache.hudi.hive.HoodieHiveSyncClient; import org.apache.hudi.keygen.SimpleKeyGenerator; import org.apache.hudi.utilities.DummySchemaProvider; import org.apache.hudi.utilities.HoodieClusteringJob; @@ -133,6 +132,8 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_PARTITION_FIELDS; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_TABLE_NAME; import static org.apache.hudi.utilities.UtilHelpers.EXECUTE; import static org.apache.hudi.utilities.UtilHelpers.SCHEDULE; import static org.apache.hudi.utilities.UtilHelpers.SCHEDULE_AND_EXECUTE; @@ -1355,13 +1356,15 @@ public void testBulkInsertsAndUpsertsWithSQLBasedTransformerFor2StepPipeline() t // Test Hive integration HiveSyncConfig hiveSyncConfig = getHiveSyncConfig(tableBasePath, "hive_trips"); - hiveSyncConfig.partitionFields = CollectionUtils.createImmutableList("year", "month", "day"); - HoodieHiveClient hiveClient = new HoodieHiveClient(hiveSyncConfig, hiveServer.getHiveConf(), dfs); - assertTrue(hiveClient.tableExists(hiveSyncConfig.tableName), "Table " + hiveSyncConfig.tableName + " should exist"); - assertEquals(3, hiveClient.getAllPartitions(hiveSyncConfig.tableName).size(), + hiveSyncConfig.setValue(META_SYNC_PARTITION_FIELDS, "year,month,day"); + hiveSyncConfig.setHadoopConf(hiveServer.getHiveConf()); + HoodieHiveSyncClient hiveClient = new HoodieHiveSyncClient(hiveSyncConfig); + final String tableName = hiveSyncConfig.getString(META_SYNC_TABLE_NAME); + assertTrue(hiveClient.tableExists(tableName), "Table " + tableName + " should exist"); + assertEquals(3, hiveClient.getAllPartitions(tableName).size(), "Table partitions should match the number of partitions we wrote"); assertEquals(lastInstantForUpstreamTable, - hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get(), + hiveClient.getLastCommitTimeSynced(tableName).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java index 7df6e11014bc6..8e7bce944f91a 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/testutils/UtilitiesTestBase.java @@ -30,7 +30,6 @@ import org.apache.hudi.common.testutils.minicluster.HdfsTestService; import org.apache.hudi.common.testutils.minicluster.ZookeeperTestService; import org.apache.hudi.common.util.AvroOrcUtils; -import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.hive.HiveSyncConfig; @@ -86,6 +85,17 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Properties; + +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_PASS; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_URL; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USER; +import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USE_PRE_APACHE_INPUT_FORMAT; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_ASSUME_DATE_PARTITION; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_BASE_PATH; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_PARTITION_FIELDS; +import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_TABLE_NAME; /** * Abstract test that provides a dfs & spark contexts. @@ -183,17 +193,17 @@ public void teardown() throws Exception { * @return */ protected static HiveSyncConfig getHiveSyncConfig(String basePath, String tableName) { - HiveSyncConfig hiveSyncConfig = new HiveSyncConfig(); - hiveSyncConfig.jdbcUrl = "jdbc:hive2://127.0.0.1:9999/"; - hiveSyncConfig.hiveUser = ""; - hiveSyncConfig.hivePass = ""; - hiveSyncConfig.databaseName = "testdb1"; - hiveSyncConfig.tableName = tableName; - hiveSyncConfig.basePath = basePath; - hiveSyncConfig.assumeDatePartitioning = false; - hiveSyncConfig.usePreApacheInputFormat = false; - hiveSyncConfig.partitionFields = CollectionUtils.createImmutableList("datestr"); - return hiveSyncConfig; + Properties props = new Properties(); + props.setProperty(HIVE_URL.key(),"jdbc:hive2://127.0.0.1:9999/"); + props.setProperty(HIVE_USER.key(), ""); + props.setProperty(HIVE_PASS.key(), ""); + props.setProperty(META_SYNC_DATABASE_NAME.key(), "testdb1"); + props.setProperty(META_SYNC_TABLE_NAME.key(), tableName); + props.setProperty(META_SYNC_BASE_PATH.key(), basePath); + props.setProperty(META_SYNC_ASSUME_DATE_PARTITION.key(), "false"); + props.setProperty(HIVE_USE_PRE_APACHE_INPUT_FORMAT.key(), "false"); + props.setProperty(META_SYNC_PARTITION_FIELDS.key(), "datestr"); + return new HiveSyncConfig(props); } /** @@ -206,14 +216,15 @@ private static void clearHiveDb() throws Exception { // Create Dummy hive sync config HiveSyncConfig hiveSyncConfig = getHiveSyncConfig("/dummy", "dummy"); hiveConf.addResource(hiveServer.getHiveConf()); + hiveSyncConfig.setHadoopConf(hiveConf); HoodieTableMetaClient.withPropertyBuilder() .setTableType(HoodieTableType.COPY_ON_WRITE) - .setTableName(hiveSyncConfig.tableName) - .initTable(dfs.getConf(), hiveSyncConfig.basePath); + .setTableName(hiveSyncConfig.getString(META_SYNC_TABLE_NAME)) + .initTable(dfs.getConf(), hiveSyncConfig.getString(META_SYNC_BASE_PATH)); - QueryBasedDDLExecutor ddlExecutor = new JDBCExecutor(hiveSyncConfig, dfs); - ddlExecutor.runSQL("drop database if exists " + hiveSyncConfig.databaseName); - ddlExecutor.runSQL("create database " + hiveSyncConfig.databaseName); + QueryBasedDDLExecutor ddlExecutor = new JDBCExecutor(hiveSyncConfig); + ddlExecutor.runSQL("drop database if exists " + hiveSyncConfig.getString(META_SYNC_DATABASE_NAME)); + ddlExecutor.runSQL("create database " + hiveSyncConfig.getString(META_SYNC_DATABASE_NAME)); ddlExecutor.close(); } diff --git a/packaging/hudi-aws-bundle/pom.xml b/packaging/hudi-aws-bundle/pom.xml index e9c17a4892699..c366159331f29 100644 --- a/packaging/hudi-aws-bundle/pom.xml +++ b/packaging/hudi-aws-bundle/pom.xml @@ -279,7 +279,7 @@ hudi-aws ${project.version} - org.apache.parquet