1
0

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

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

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

View File

@@ -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<PartitionEvent> getPartitionEvents(List<Partition> tablePartitions, List<String> partitionStoragePartitions, boolean isDropPartition) {
Map<String, String> paths = new HashMap<>();
for (Partition tablePartition : tablePartitions) {
List<String> hivePartitionValues = tablePartition.getValues();
String fullTablePartitionPath =
Path.getPathWithoutSchemeAndAuthority(new Path(tablePartition.getStorageLocation())).toUri().getPath();
paths.put(String.join(", ", hivePartitionValues), fullTablePartitionPath);
}
List<PartitionEvent> 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<String> 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<Partition> 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<FieldSchema> getTableCommentUsingMetastoreClient(String tableName);
public abstract void updateTableComments(String tableName, List<FieldSchema> oldSchema, List<Schema.Field> newSchema);
public abstract void updateTableComments(String tableName, List<FieldSchema> oldSchema, Map<String, String> 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.
*/
}

View File

@@ -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;

View File

@@ -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<String> 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<String> HIVE_USER = ConfigProperty
.key("hoodie.datasource.hive_sync.username")
.defaultValue("hive")
.withDocumentation("hive user name to use");
public static final ConfigProperty<String> HIVE_PASS = ConfigProperty
.key("hoodie.datasource.hive_sync.password")
.defaultValue("hive")
.withDocumentation("hive password to use");
public static final ConfigProperty<String> HIVE_URL = ConfigProperty
.key("hoodie.datasource.hive_sync.jdbcurl")
.defaultValue("jdbc:hive2://localhost:10000")
.withDocumentation("Hive metastore url");
public static final ConfigProperty<String> 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<String> 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<String> METASTORE_URIS = ConfigProperty
.key("hoodie.datasource.hive_sync.metastore.uris")
.defaultValue("thrift://localhost:9083")
.withDocumentation("Hive metastore url");
public static final ConfigProperty<String> 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<String> HIVE_IGNORE_EXCEPTIONS = ConfigProperty
.key("hoodie.datasource.hive_sync.ignore_exceptions")
.defaultValue("false")
.withDocumentation("Ignore exceptions when syncing with Hive.");
public static final ConfigProperty<String> 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<String> 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<String> HIVE_TABLE_PROPERTIES = ConfigProperty
.key("hoodie.datasource.hive_sync.table_properties")
.noDefaultValue()
.withDocumentation("Additional properties to store with table.");
public static final ConfigProperty<String> HIVE_TABLE_SERDE_PROPERTIES = ConfigProperty
.key("hoodie.datasource.hive_sync.serde_properties")
.noDefaultValue()
.withDocumentation("Serde properties to hive table.");
public static final ConfigProperty<String> HIVE_SYNC_AS_DATA_SOURCE_TABLE = ConfigProperty
.key("hoodie.datasource.hive_sync.sync_as_datasource")
.defaultValue("true")
.withDocumentation("");
public static final ConfigProperty<Integer> 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<Boolean> 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<Integer> 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<String> 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<Boolean> 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<String> 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<String> 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() {
this(new TypedProperties());
}
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);
}
@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 static String getBucketSpec(String bucketCols, int bucketNum) {
return "CLUSTERED BY (" + bucketCols + " INTO " + bucketNum + " BUCKETS";
}
public HiveSyncConfig(Properties props) {
super(props);
}
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 HiveConf getHiveConf() {
return (HiveConf) getHadoopConf();
}
public boolean useBucketSync() {
return getBooleanOrDefault(HIVE_SYNC_BUCKET_SYNC);
}
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;
}
}
}

View File

@@ -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<String> 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<String> HIVE_USER = ConfigProperty
.key("hoodie.datasource.hive_sync.username")
.defaultValue("hive")
.withDocumentation("hive user name to use");
public static final ConfigProperty<String> HIVE_PASS = ConfigProperty
.key("hoodie.datasource.hive_sync.password")
.defaultValue("hive")
.withDocumentation("hive password to use");
public static final ConfigProperty<String> HIVE_URL = ConfigProperty
.key("hoodie.datasource.hive_sync.jdbcurl")
.defaultValue("jdbc:hive2://localhost:10000")
.withDocumentation("Hive metastore url");
public static final ConfigProperty<String> 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<String> 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<String> METASTORE_URIS = ConfigProperty
.key("hoodie.datasource.hive_sync.metastore.uris")
.defaultValue("thrift://localhost:9083")
.withDocumentation("Hive metastore url");
public static final ConfigProperty<String> 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<String> HIVE_IGNORE_EXCEPTIONS = ConfigProperty
.key("hoodie.datasource.hive_sync.ignore_exceptions")
.defaultValue("false")
.withDocumentation("Ignore exceptions when syncing with Hive.");
public static final ConfigProperty<String> 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<String> 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<String> HIVE_TABLE_PROPERTIES = ConfigProperty
.key("hoodie.datasource.hive_sync.table_properties")
.noDefaultValue()
.withDocumentation("Additional properties to store with table.");
public static final ConfigProperty<String> HIVE_TABLE_SERDE_PROPERTIES = ConfigProperty
.key("hoodie.datasource.hive_sync.serde_properties")
.noDefaultValue()
.withDocumentation("Serde properties to hive table.");
public static final ConfigProperty<String> HIVE_SYNC_AS_DATA_SOURCE_TABLE = ConfigProperty
.key("hoodie.datasource.hive_sync.sync_as_datasource")
.defaultValue("true")
.withDocumentation("");
public static final ConfigProperty<Integer> 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<Boolean> 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<Integer> 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<String> 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<Boolean> 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<String> 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<String> 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.");
}

View File

@@ -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 @@ import java.util.stream.Collectors;
* 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<String> roTableName = null;
protected final HiveSyncConfig config;
protected final String databaseName;
protected final String tableName;
protected HoodieSyncClient syncClient;
protected String snapshotTableName;
protected Option<String> roTableName;
public HiveSyncTool(TypedProperties props, Configuration conf, FileSystem fs) {
this(new HiveSyncConfig(props), new HiveConf(conf, HiveConf.class), fs);
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);
}
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 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 @@ public class HiveSyncTool extends AbstractSyncTool implements AutoCloseable {
}
}
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 @@ public class HiveSyncTool extends AbstractSyncTool implements AutoCloseable {
@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 @@ public class HiveSyncTool extends AbstractSyncTool implements AutoCloseable {
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 @@ public class HiveSyncTool extends AbstractSyncTool implements AutoCloseable {
// Get the last time we successfully synced partitions
Option<String> 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<String> writtenPartitionsSince = hoodieHiveClient.getPartitionsWrittenToSince(lastCommitTimeSynced);
List<String> 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 @@ public class HiveSyncTool extends AbstractSyncTool implements AutoCloseable {
* 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<String, String> tableProperties = ConfigUtils.toMap(hiveSyncConfig.tableProperties);
Map<String, String> serdeProperties = ConfigUtils.toMap(hiveSyncConfig.serdeProperties);
if (hiveSyncConfig.syncAsSparkDataSourceTable) {
Map<String, String> sparkTableProperties = getSparkTableProperties(hiveSyncConfig.partitionFields,
hiveSyncConfig.sparkVersion, hiveSyncConfig.sparkSchemaLengthThreshold, schema);
Map<String, String> sparkSerdeProperties = getSparkSerdeProperties(readAsOptimized, hiveSyncConfig.basePath);
Map<String, String> tableProperties = ConfigUtils.toMap(config.getString(HIVE_TABLE_PROPERTIES));
Map<String, String> serdeProperties = ConfigUtils.toMap(config.getString(HIVE_TABLE_SERDE_PROPERTIES));
if (config.getBoolean(HIVE_SYNC_AS_DATA_SOURCE_TABLE)) {
Map<String, String> 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<String, String> sparkSerdeProperties = SparkDataSourceTableUtils.getSparkSerdeProperties(readAsOptimized, config.getString(META_SYNC_BASE_PATH));
tableProperties.putAll(sparkTableProperties);
serdeProperties.putAll(sparkSerdeProperties);
}
@@ -252,10 +259,10 @@ public class HiveSyncTool extends AbstractSyncTool implements AutoCloseable {
// 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 @@ public class HiveSyncTool extends AbstractSyncTool implements AutoCloseable {
// 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<String, String> tableSchema = hoodieHiveClient.getTableSchema(tableName);
SchemaDifference schemaDiff = HiveSchemaUtil.getSchemaDifference(schema, tableSchema, hiveSyncConfig.partitionFields, hiveSyncConfig.supportTimestamp);
Map<String, String> 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 @@ public class HiveSyncTool extends AbstractSyncTool implements AutoCloseable {
}
}
if (hiveSyncConfig.syncComment) {
Schema avroSchemaWithoutMetadataFields = hoodieHiveClient.getAvroSchemaWithoutMetadataFields();
Map<String, String> 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<FieldSchema> 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<FieldSchema> fromMetastore = syncClient.getMetastoreFieldSchemas(tableName);
List<FieldSchema> fromStorage = syncClient.getStorageFieldSchemas();
syncClient.updateTableComments(tableName, fromMetastore, fromStorage);
}
return schemaChanged;
}
@@ -311,26 +312,26 @@ public class HiveSyncTool extends AbstractSyncTool implements AutoCloseable {
private boolean syncPartitions(String tableName, List<String> writtenPartitionsSince, boolean isDropPartition) {
boolean partitionsChanged;
try {
List<Partition> hivePartitions = hoodieHiveClient.getAllPartitions(tableName);
List<Partition> hivePartitions = syncClient.getAllPartitions(tableName);
List<PartitionEvent> partitionEvents =
hoodieHiveClient.getPartitionEvents(hivePartitions, writtenPartitionsSince, isDropPartition);
syncClient.getPartitionEvents(hivePartitions, writtenPartitionsSince, isDropPartition);
List<String> newPartitions = filterPartitions(partitionEvents, PartitionEventType.ADD);
if (!newPartitions.isEmpty()) {
LOG.info("New Partitions " + newPartitions);
hoodieHiveClient.addPartitionsToTable(tableName, newPartitions);
syncClient.addPartitionsToTable(tableName, newPartitions);
}
List<String> updatePartitions = filterPartitions(partitionEvents, PartitionEventType.UPDATE);
if (!updatePartitions.isEmpty()) {
LOG.info("Changed Partitions " + updatePartitions);
hoodieHiveClient.updatePartitionsToTable(tableName, updatePartitions);
syncClient.updatePartitionsToTable(tableName, updatePartitions);
}
List<String> 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 @@ public class HiveSyncTool extends AbstractSyncTool implements AutoCloseable {
}
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();
}
}

View File

@@ -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.Map;
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<String> partitionsToAdd) {
ddlExecutor.addPartitionsToTable(tableName, partitionsToAdd);
}
/**
* Partition path has changed - update the path for te following partitions.
*/
@Override
public void updatePartitionsToTable(String tableName, List<String> changedPartitions) {
ddlExecutor.updatePartitionsToTable(tableName, changedPartitions);
}
/**
* Partition path has changed - drop the following partitions.
*/
@Override
public void dropPartitions(String tableName, List<String> partitionsToDrop) {
ddlExecutor.dropPartitionsToTable(tableName, partitionsToDrop);
}
/**
* Update the table properties to the table.
*/
@Override
public void updateTableProperties(String tableName, Map<String, String> tableProperties) {
if (tableProperties == null || tableProperties.isEmpty()) {
return;
}
try {
Table table = client.getTable(syncConfig.databaseName, tableName);
Table table = client.getTable(databaseName, tableName);
for (Map.Entry<String, String> 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<org.apache.hadoop.hive.metastore.api.Partition> 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<Partition> 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 class HoodieHiveClient extends AbstractHiveSyncHoodieClient {
ddlExecutor.createTable(tableName, storageSchema, inputFormatClass, outputFormatClass, serdeClass, serdeProperties, tableProperties);
}
/**
* Get the table schema.
*/
@Override
public Map<String, String> getTableSchema(String tableName) {
public Map<String, String> getMetastoreSchema(String tableName) {
if (!tableExists(tableName)) {
throw new IllegalArgumentException(
"Failed to get schema for table " + tableName + " does not exist");
@@ -180,26 +161,15 @@ public class HoodieHiveClient extends AbstractHiveSyncHoodieClient {
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 class HoodieHiveClient extends AbstractHiveSyncHoodieClient {
public Option<String> 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 class HoodieHiveClient extends AbstractHiveSyncHoodieClient {
public Option<String> 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 class HoodieHiveClient extends AbstractHiveSyncHoodieClient {
}
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 class HoodieHiveClient extends AbstractHiveSyncHoodieClient {
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 class HoodieHiveClient extends AbstractHiveSyncHoodieClient {
@Override
public void updateLastCommitTimeSynced(String tableName) {
// Set the last commit time from the TBLproperties
Option<String> lastCommitSynced = activeTimeline.lastInstant().map(HoodieInstant::getTimestamp);
Option<String> 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 class HoodieHiveClient extends AbstractHiveSyncHoodieClient {
}
@Override
public List<FieldSchema> getTableCommentUsingMetastoreClient(String tableName) {
public List<FieldSchema> 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<FieldSchema> oldSchema, List<Schema.Field> newSchema) {
Map<String,String> 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<FieldSchema> 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<FieldSchema> oldSchema, Map<String,String> newComments) {
Map<String,String> oldComments = oldSchema.stream().collect(Collectors.toMap(fieldSchema -> fieldSchema.getName().toLowerCase(Locale.ROOT),
fieldSchema -> StringUtils.isNullOrEmpty(fieldSchema.getComment()) ? "" : fieldSchema.getComment()));
Map<String,String> types = oldSchema.stream().collect(Collectors.toMap(FieldSchema::getName, FieldSchema::getType));
Map<String, ImmutablePair<String,String>> 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<FieldSchema> fromMetastore, List<FieldSchema> fromStorage) {
Map<String, FieldSchema> metastoreMap = fromMetastore.stream().collect(Collectors.toMap(f -> f.getName().toLowerCase(Locale.ROOT), f -> f));
Map<String, FieldSchema> storageMap = fromStorage.stream().collect(Collectors.toMap(f -> f.getName().toLowerCase(Locale.ROOT), f -> f));
Map<String, Pair<String, String>> 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);
}
}
}

View File

@@ -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;

View File

@@ -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;

View File

@@ -1,33 +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 java.io.Serializable;
import java.util.List;
/**
* HDFS Path contain hive partition values for the keys it is partitioned on. This mapping is not straight forward and
* requires a pluggable implementation to extract the partition value from HDFS path.
* <p>
* e.g. Hive table partitioned by datestr=yyyy-mm-dd and hdfs path /app/hoodie/dataset1/YYYY=[yyyy]/MM=[mm]/DD=[dd]
*/
public interface PartitionValueExtractor extends Serializable {
List<String> extractPartitionValuesInPath(String partitionPath);
}

View File

@@ -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;

View File

@@ -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;

View File

@@ -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 @@ public interface DDLExecutor extends AutoCloseable {
* @param tableName
* @param newSchema Map key: field name, Map value: [field type, field comment]
*/
void updateTableComments(String tableName, Map<String, ImmutablePair<String, String>> newSchema);
void updateTableComments(String tableName, Map<String, Pair<String, String>> newSchema);
}

View File

@@ -20,17 +20,15 @@ package org.apache.hudi.hive.ddl;
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.List;
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 IMetaStoreClient client;
public HMSDDLExecutor(HiveConf conf, HiveSyncConfig syncConfig, FileSystem fs) throws HiveException, MetaException {
this.client = Hive.get(conf).getMSC();
private static final Logger LOG = LogManager.getLogger(HMSDDLExecutor.class);
private final HiveSyncConfig syncConfig;
private final String databaseName;
private final IMetaStoreClient client;
private final PartitionValueExtractor partitionValueExtractor;
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 class HMSDDLExecutor implements DDLExecutor {
public void createTable(String tableName, MessageType storageSchema, String inputFormatClass, String outputFormatClass, String serdeClass, Map<String, String> serdeProperties,
Map<String, String> tableProperties) {
try {
LinkedHashMap<String, String> mapSchema = HiveSchemaUtil.parquetSchemaToMapSchema(storageSchema, syncConfig.supportTimestamp, false);
LinkedHashMap<String, String> mapSchema = HiveSchemaUtil.parquetSchemaToMapSchema(storageSchema, syncConfig.getBoolean(HIVE_SUPPORT_TIMESTAMP_TYPE), false);
List<FieldSchema> fieldSchema = HiveSchemaUtil.convertMapSchemaToHiveFieldSchema(mapSchema, syncConfig);
List<FieldSchema> partitionSchema = syncConfig.partitionFields.stream().map(partitionKey -> {
List<FieldSchema> 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 class HMSDDLExecutor implements DDLExecutor {
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 class HMSDDLExecutor implements DDLExecutor {
@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> 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 class HMSDDLExecutor implements DDLExecutor {
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 class HMSDDLExecutor implements DDLExecutor {
// 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<String, String> partitionKeysMap =
table.getPartitionKeys().stream().collect(Collectors.toMap(FieldSchema::getName, f -> f.getType().toUpperCase()));
@@ -184,22 +191,22 @@ public class HMSDDLExecutor implements DDLExecutor {
}
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<Partition> 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<String> 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 class HMSDDLExecutor implements DDLExecutor {
}
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<Partition> 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<String> 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 class HMSDDLExecutor implements DDLExecutor {
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<String, ImmutablePair<String,String>> alterSchema) {
public void updateTableComments(String tableName, Map<String, Pair<String, String>> 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 class HMSDDLExecutor implements DDLExecutor {
}
table.setSd(sd);
EnvironmentContext environmentContext = new EnvironmentContext();
client.alter_table_with_environmentContext(syncConfig.databaseName, tableName, table, environmentContext);
client.alter_table_with_environmentContext(databaseName, tableName, table, environmentContext);
sd.clear();
} catch (Exception e) {
LOG.error("Failed to update table comments for " + tableName, e);

View File

@@ -21,9 +21,8 @@ package org.apache.hudi.hive.ddl;
import org.apache.hudi.common.util.HoodieTimer;
import org.apache.hudi.hive.HiveSyncConfig;
import org.apache.hudi.hive.HoodieHiveSyncException;
import org.apache.hudi.hive.util.HivePartitionUtil;
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.MetaException;
@@ -34,7 +33,6 @@ import org.apache.hadoop.hive.ql.metadata.HiveException;
import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
import org.apache.hadoop.hive.ql.session.SessionState;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hudi.hive.util.HivePartitionUtil;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -46,26 +44,28 @@ import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import static org.apache.hudi.sync.common.util.TableUtils.tableId;
/**
* This class offers DDL executor backed by the hive.ql Driver This class preserves the old useJDBC = false way of doing things.
*/
public class HiveQueryDDLExecutor extends QueryBasedDDLExecutor {
private static final Logger LOG = LogManager.getLogger(HiveQueryDDLExecutor.class);
private final HiveSyncConfig config;
private final IMetaStoreClient metaStoreClient;
private SessionState sessionState = null;
private Driver hiveDriver = null;
public HiveQueryDDLExecutor(HiveSyncConfig config, FileSystem fs, HiveConf configuration) throws HiveException, MetaException {
super(config, fs);
this.config = config;
this.metaStoreClient = Hive.get(configuration).getMSC();
private static final Logger LOG = LogManager.getLogger(HiveQueryDDLExecutor.class);
private final IMetaStoreClient metaStoreClient;
private SessionState sessionState;
private Driver hiveDriver;
public HiveQueryDDLExecutor(HiveSyncConfig config) throws HiveException, MetaException {
super(config);
this.metaStoreClient = Hive.get(config.getHiveConf()).getMSC();
try {
this.sessionState = new SessionState(configuration,
this.sessionState = new SessionState(config.getHiveConf(),
UserGroupInformation.getCurrentUser().getShortUserName());
SessionState.start(this.sessionState);
this.sessionState.setCurrentDatabase(config.databaseName);
hiveDriver = new org.apache.hadoop.hive.ql.Driver(configuration);
this.sessionState.setCurrentDatabase(databaseName);
this.hiveDriver = new org.apache.hadoop.hive.ql.Driver(config.getHiveConf());
} catch (Exception e) {
if (sessionState != null) {
try {
@@ -109,7 +109,7 @@ public class HiveQueryDDLExecutor extends QueryBasedDDLExecutor {
// 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<String, String> partitionKeysMap =
table.getPartitionKeys().stream().collect(Collectors.toMap(FieldSchema::getName, f -> f.getType().toUpperCase()));
@@ -141,13 +141,13 @@ public class HiveQueryDDLExecutor extends QueryBasedDDLExecutor {
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);
}
}

View File

@@ -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.List;
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 class JDBCExecutor extends QueryBasedDDLExecutor {
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 class JDBCExecutor extends QueryBasedDDLExecutor {
}
private List<String> constructDropPartitions(String tableName, List<String> 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<String> 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 @@ public class JDBCExecutor extends QueryBasedDDLExecutor {
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 class JDBCExecutor extends QueryBasedDDLExecutor {
LOG.error("Could not close connection ", e);
}
}
}
}

View File

@@ -22,13 +22,12 @@ import org.apache.hudi.common.fs.FSUtils;
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.ArrayList;
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;
private static final Logger LOG = LogManager.getLogger(QueryBasedDDLExecutor.class);
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 abstract class QueryBasedDDLExecutor implements DDLExecutor {
@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 abstract class QueryBasedDDLExecutor implements DDLExecutor {
}
@Override
public void updateTableComments(String tableName, Map<String, ImmutablePair<String,String>> newSchema) {
for (Map.Entry<String, ImmutablePair<String,String>> field : newSchema.entrySet()) {
public void updateTableComments(String tableName, Map<String, Pair<String, String>> newSchema) {
for (Map.Entry<String, Pair<String,String>> 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 abstract class QueryBasedDDLExecutor implements DDLExecutor {
}
private List<String> constructAddPartitions(String tableName, List<String> 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<String> 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 @@ public abstract class QueryBasedDDLExecutor implements DDLExecutor {
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 @@ public abstract class QueryBasedDDLExecutor implements DDLExecutor {
public String getPartitionClause(String partition) {
List<String> 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<String> 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 abstract class QueryBasedDDLExecutor implements DDLExecutor {
private List<String> constructChangePartitions(String tableName, List<String> partitions) {
List<String> 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);

View File

@@ -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<String> META_SYNC_GLOBAL_REPLICATE_TIMESTAMP = ConfigProperty
.key("hoodie.meta_sync.global.replicate.timestamp")
.noDefaultValue()
.withDocumentation("");
public GlobalHiveSyncConfig(Properties props, Configuration hadoopConf) {
super(props, hadoopConf);
}
public GlobalHiveSyncConfig(TypedProperties props) {
super(props);
}
public static class GlobalHiveSyncConfigParams {
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;
}
@ParametersDelegate()
public final HiveSyncConfigParams hiveSyncConfigParams = new HiveSyncConfigParams();
@Override
public String toString() {
return "GlobalHiveSyncConfig{" + super.toString()
+ " globallyReplicatedTimeStamp=" + globallyReplicatedTimeStamp + "}";
@Parameter(names = {"--replicated-timestamp"}, description = "Add globally replicated timestamp to enable consistent reads across clusters")
public String globallyReplicatedTimeStamp;
public boolean isHelp() {
return hiveSyncConfigParams.isHelp();
}
public TypedProperties toProps() {
final TypedProperties props = hiveSyncConfigParams.toProps();
props.setPropertyIfNonNull(META_SYNC_GLOBAL_REPLICATE_TIMESTAMP.key(), globallyReplicatedTimeStamp);
return props;
}
}
}

View File

@@ -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 class GlobalHiveSyncTool extends HiveSyncTool {
@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<String> 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<String, Option<String>> getLastReplicatedTimeStampMap() {
Map<String, Option<String>> timeStampMap = new HashMap<>();
Option<String> timeStamp = hoodieHiveClient.getLastReplicatedTime(snapshotTableName);
Option<String> timeStamp = syncClient.getLastReplicatedTime(snapshotTableName);
timeStampMap.put(snapshotTableName, timeStamp);
if (HoodieTableType.MERGE_ON_READ.equals(hoodieHiveClient.getTableType())) {
Option<String> roTimeStamp = hoodieHiveClient.getLastReplicatedTime(roTableName.get());
if (HoodieTableType.MERGE_ON_READ.equals(syncClient.getTableType())) {
Option<String> roTimeStamp = syncClient.getLastReplicatedTime(roTableName.get());
timeStampMap.put(roTableName.get(), roTimeStamp);
}
return timeStampMap;
@@ -70,18 +74,12 @@ public class GlobalHiveSyncTool extends HiveSyncTool {
for (String tableName : timeStampMap.keySet()) {
Option<String> 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);
}
}

View File

@@ -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 @@ import org.apache.log4j.Logger;
+ " 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 class HiveSyncGlobalCommitConfig extends GlobalHiveSyncConfig {
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");
}
}

View File

@@ -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 org.apache.hadoop.fs.Path;
import org.apache.hadoop.hive.conf.HiveConf;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
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 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<ReplicationStateSync> replicationStateSyncList;
private final HiveSyncGlobalCommitParams params;
private final List<ReplicationStateSync> 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 class HiveSyncGlobalCommitTool implements HiveSyncGlobalCommit, AutoClose
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 class HiveSyncGlobalCommitTool implements HiveSyncGlobalCommit, AutoClose
}
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()) {

View File

@@ -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;
private GlobalHiveSyncTool globalHiveSyncTool;
private final GlobalHiveSyncConfig globalHiveSyncConfig;
private final HiveConf hiveConf;
import java.util.Map;
import java.util.Properties;
public class ReplicationStateSync implements AutoCloseable {
protected GlobalHiveSyncTool globalHiveSyncTool;
private Map<String, Option<String>> replicatedTimeStampMap;
private Map<String, Option<String>> 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 class ReplicationStateSync {
return clusterId;
}
@Override
public void close() {
if (globalHiveSyncTool != null) {
globalHiveSyncTool.close();

View File

@@ -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;

View File

@@ -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<String> 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<String> 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 class HivePartitionUtil {
Partition newPartition;
try {
List<String> 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) {

View File

@@ -42,6 +42,12 @@ import java.util.Map;
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 class HiveSchemaUtil {
* @return : Hive Table schema read from parquet file List[FieldSchema] without partitionField
*/
public static List<FieldSchema> 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 class HiveSchemaUtil {
public static List<FieldSchema> convertMapSchemaToHiveFieldSchema(LinkedHashMap<String, String> 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 class HiveSchemaUtil {
public static String generateCreateDDL(String tableName, MessageType storageSchema, HiveSyncConfig config, String inputFormatClass,
String outputFormatClass, String serdeClass, Map<String, String> serdeProperties,
Map<String, String> tableProperties) throws IOException {
Map<String, String> hiveSchema = convertParquetSchemaToHiveSchema(storageSchema, config.supportTimestamp);
String columns = generateSchemaString(storageSchema, config.partitionFields, config.supportTimestamp);
Map<String, String> 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<String> 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 class HiveSchemaUtil {
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(")");

View File

@@ -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();
}
}

View File

@@ -27,19 +27,18 @@ import org.apache.hudi.common.testutils.SchemaTestUtil;
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.net.URISyntaxException;
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 @@ public class TestHiveSyncTool {
}
private HiveSyncTool hiveSyncTool;
private HoodieHiveClient hiveClient;
private HoodieHiveSyncClient hiveClient;
@AfterAll
public static void cleanUpClass() {
@@ -131,7 +143,7 @@ public class TestHiveSyncTool {
@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 class TestHiveSyncTool {
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<String> newPartition = Arrays.asList("2050/01/01");
hiveClient.addPartitionsToTable(HiveTestUtil.TABLE_NAME, Arrays.asList());
assertEquals(5, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(),
List<String> 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 class TestHiveSyncTool {
ddlExecutor.runSQL("ALTER TABLE `" + HiveTestUtil.TABLE_NAME
+ "` PARTITION (`datestr`='2050-01-01') SET LOCATION '/some/new/location'");
List<org.apache.hudi.sync.common.model.Partition> hivePartitions = hiveClient.getAllPartitions(HiveTestUtil.TABLE_NAME);
List<Partition> hivePartitions = hiveClient.getAllPartitions(HiveTestUtil.TABLE_NAME);
List<String> writtenPartitionsSince = hiveClient.getPartitionsWrittenToSince(Option.empty());
List<PartitionEvent> partitionEvents = hiveClient.getPartitionEvents(hivePartitions, writtenPartitionsSince, false);
assertEquals(1, partitionEvents.size(), "There should be only one partition event");
@@ -186,7 +198,7 @@ public class TestHiveSyncTool {
reSyncHiveTable();
// Sync should update the changed partition to correct path
List<Partition> tablePartitions = hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME);
List<Partition> 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 class TestHiveSyncTool {
@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 class TestHiveSyncTool {
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 class TestHiveSyncTool {
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 class TestHiveSyncTool {
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 class TestHiveSyncTool {
@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 class TestHiveSyncTool {
@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 class TestHiveSyncTool {
// 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 class TestHiveSyncTool {
@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 class TestHiveSyncTool {
// 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 class TestHiveSyncTool {
@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<String, ImmutablePair<String,String>> alterCommentSchema = new HashMap<>();
Map<String, Pair<String, String>> 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 class TestHiveSyncTool {
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<FieldSchema> fieldSchemas = hiveClient.getTableCommentUsingMetastoreClient(HiveTestUtil.TABLE_NAME);
List<FieldSchema> 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 class TestHiveSyncTool {
@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<FieldSchema> fieldSchemas = hiveClient.getTableCommentUsingMetastoreClient(HiveTestUtil.TABLE_NAME);
List<FieldSchema> 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 class TestHiveSyncTool {
@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 class TestHiveSyncTool {
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 class TestHiveSyncTool {
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 class TestHiveSyncTool {
@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 class TestHiveSyncTool {
+ " 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 class TestHiveSyncTool {
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 class TestHiveSyncTool {
@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 class TestHiveSyncTool {
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 class TestHiveSyncTool {
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 class TestHiveSyncTool {
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 class TestHiveSyncTool {
@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 class TestHiveSyncTool {
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<String> newPartition = Arrays.asList("2050/01/01");
hiveClient.addPartitionsToTable(HiveTestUtil.TABLE_NAME, Arrays.asList());
assertEquals(1, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(),
List<String> 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 class TestHiveSyncTool {
ddlExecutor.runSQL("ALTER TABLE `" + HiveTestUtil.TABLE_NAME
+ "` DROP PARTITION (`datestr`='2050-01-01')");
List<Partition> hivePartitions = hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME);
List<Partition> 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 class TestHiveSyncTool {
@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 class TestHiveSyncTool {
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<Partition> partitions = hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME);
List<Partition> 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 class TestHiveSyncTool {
reinitHiveSyncClient();
reSyncHiveTable();
List<Partition> hivePartitions = hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME);
List<Partition> 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 class TestHiveSyncTool {
@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 class TestHiveSyncTool {
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 schemaignoring 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 class TestHiveSyncTool {
+ " 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 class TestHiveSyncTool {
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 class TestHiveSyncTool {
"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 @@ public class TestHiveSyncTool {
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 class TestHiveSyncTool {
@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 class TestHiveSyncTool {
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 class TestHiveSyncTool {
@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 class TestHiveSyncTool {
@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 class TestHiveSyncTool {
// 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 class TestHiveSyncTool {
@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 @@ public class TestHiveSyncTool {
}
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;
}
}

View File

@@ -21,7 +21,7 @@ package org.apache.hudi.hive.functional;
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;

View File

@@ -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();
}
}

View File

@@ -24,7 +24,7 @@ import org.apache.hudi.common.model.HoodieTableType;
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 org.junit.jupiter.api.io.TempDir;
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 class HiveSyncFunctionalTestHarness {
}
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);
}
}

View File

@@ -84,6 +84,16 @@ import java.util.Map.Entry;
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 class HiveTestUtil {
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");
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);
hiveSyncConfig = new HiveSyncConfig(hiveSyncProps, configuration);
dtfOut = DateTimeFormatter.ofPattern("yyyy/MM/dd");
ddlExecutor = new HiveQueryDDLExecutor(hiveSyncConfig, fileSystem, getHiveConf());
ddlExecutor = new HiveQueryDDLExecutor(hiveSyncConfig);
clear();
}