1
0

[HUDI-875] Abstract hudi-sync-common, and support hudi-hive-sync, hudi-dla-sync (#1810)

- Generalize the hive-sync module for syncing to multiple metastores
- Added new options for datasource
- Added new command line for delta streamer 

Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
lw0090
2020-08-06 12:34:55 +08:00
committed by GitHub
parent c21209cb58
commit 51ea27d665
44 changed files with 1663 additions and 145 deletions

View File

@@ -19,6 +19,7 @@ package org.apache.hudi
import org.apache.hudi.common.model.HoodieTableType
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload
import org.apache.hudi.hive.HiveSyncTool
import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor
import org.apache.hudi.keygen.SimpleKeyGenerator
import org.apache.log4j.LogManager
@@ -251,11 +252,14 @@ object DataSourceWriteOptions {
*/
val STREAMING_IGNORE_FAILED_BATCH_OPT_KEY = "hoodie.datasource.write.streaming.ignore.failed.batch"
val DEFAULT_STREAMING_IGNORE_FAILED_BATCH_OPT_VAL = "true"
val META_SYNC_CLIENT_TOOL_CLASS = "hoodie.meta.sync.client.tool.class"
val DEFAULT_META_SYNC_CLIENT_TOOL_CLASS = classOf[HiveSyncTool].getName
// 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
val HIVE_SYNC_ENABLED_OPT_KEY = "hoodie.datasource.hive_sync.enable"
val META_SYNC_ENABLED_OPT_KEY = "hoodie.datasource.meta.sync.enable"
val HIVE_DATABASE_OPT_KEY = "hoodie.datasource.hive_sync.database"
val HIVE_TABLE_OPT_KEY = "hoodie.datasource.hive_sync.table"
val HIVE_BASE_FILE_FORMAT_OPT_KEY = "hoodie.datasource.hive_sync.base_file_format"
@@ -270,6 +274,7 @@ object DataSourceWriteOptions {
// DEFAULT FOR HIVE SPECIFIC CONFIGS
val DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL = "false"
val DEFAULT_META_SYNC_ENABLED_OPT_VAL = "false"
val DEFAULT_HIVE_DATABASE_OPT_VAL = "default"
val DEFAULT_HIVE_TABLE_OPT_VAL = "unknown"
val DEFAULT_HIVE_BASE_FILE_FORMAT_OPT_VAL = "PARQUET"

View File

@@ -18,6 +18,7 @@
package org.apache.hudi
import java.util
import java.util.Properties
import org.apache.avro.Schema
import org.apache.avro.generic.GenericRecord
@@ -32,9 +33,11 @@ import org.apache.hudi.common.fs.FSUtils
import org.apache.hudi.common.model.{HoodieRecordPayload, HoodieTableType}
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
import org.apache.hudi.common.util.ReflectionUtils
import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.exception.HoodieException
import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool}
import org.apache.hudi.sync.common.AbstractSyncTool
import org.apache.log4j.LogManager
import org.apache.spark.api.java.{JavaRDD, JavaSparkContext}
import org.apache.spark.rdd.RDD
@@ -240,7 +243,10 @@ private[hudi] object HoodieSparkSqlWriter {
STREAMING_RETRY_CNT_OPT_KEY -> DEFAULT_STREAMING_RETRY_CNT_OPT_VAL,
STREAMING_RETRY_INTERVAL_MS_OPT_KEY -> DEFAULT_STREAMING_RETRY_INTERVAL_MS_OPT_VAL,
STREAMING_IGNORE_FAILED_BATCH_OPT_KEY -> DEFAULT_STREAMING_IGNORE_FAILED_BATCH_OPT_VAL,
META_SYNC_CLIENT_TOOL_CLASS -> DEFAULT_META_SYNC_CLIENT_TOOL_CLASS,
//just for backwards compatiblity
HIVE_SYNC_ENABLED_OPT_KEY -> DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL,
META_SYNC_ENABLED_OPT_KEY -> DEFAULT_META_SYNC_ENABLED_OPT_VAL,
HIVE_DATABASE_OPT_KEY -> DEFAULT_HIVE_DATABASE_OPT_VAL,
HIVE_TABLE_OPT_KEY -> DEFAULT_HIVE_TABLE_OPT_VAL,
HIVE_BASE_FILE_FORMAT_OPT_KEY -> DEFAULT_HIVE_BASE_FILE_FORMAT_OPT_VAL,
@@ -287,6 +293,44 @@ private[hudi] object HoodieSparkSqlWriter {
hiveSyncConfig
}
private def metaSync(parameters: Map[String, String],
basePath: Path,
hadoopConf: Configuration): Boolean = {
val hiveSyncEnabled = parameters.get(HIVE_SYNC_ENABLED_OPT_KEY).exists(r => r.toBoolean)
var metaSyncEnabled = parameters.get(META_SYNC_ENABLED_OPT_KEY).exists(r => r.toBoolean)
var syncClientToolClassSet = scala.collection.mutable.Set[String]()
parameters(META_SYNC_CLIENT_TOOL_CLASS).split(",").foreach(syncClass => syncClientToolClassSet += syncClass)
// for backward compatibility
if (hiveSyncEnabled) {
metaSyncEnabled = true
syncClientToolClassSet += classOf[HiveSyncTool].getName
}
var metaSyncSuccess = true
if (metaSyncEnabled) {
val fs = basePath.getFileSystem(hadoopConf)
syncClientToolClassSet.foreach(impl => {
val syncSuccess = impl.trim match {
case "org.apache.hudi.hive.HiveSyncTool" => {
log.info("Syncing to Hive Metastore (URL: " + parameters(HIVE_URL_OPT_KEY) + ")")
syncHive(basePath, fs, parameters)
true
}
case _ => {
val properties = new Properties();
properties.putAll(parameters)
properties.put("basePath", basePath.toString)
val syncHoodie = ReflectionUtils.loadClass(impl.trim, Array[Class[_]](classOf[Properties], classOf[FileSystem]), properties, fs).asInstanceOf[AbstractSyncTool]
syncHoodie.syncHoodieTable()
true
}
}
metaSyncSuccess = metaSyncSuccess && syncSuccess
})
}
metaSyncSuccess
}
private def commitAndPerformPostOperations(writeStatuses: JavaRDD[WriteStatus],
parameters: Map[String, String],
client: HoodieWriteClient[HoodieRecordPayload[Nothing]],
@@ -323,20 +367,13 @@ private[hudi] object HoodieSparkSqlWriter {
}
log.info(s"Compaction Scheduled is $compactionInstant")
val hiveSyncEnabled = parameters.get(HIVE_SYNC_ENABLED_OPT_KEY).exists(r => r.toBoolean)
val syncHiveSucess = if (hiveSyncEnabled) {
log.info("Syncing to Hive Metastore (URL: " + parameters(HIVE_URL_OPT_KEY) + ")")
val fs = FSUtils.getFs(basePath.toString, jsc.hadoopConfiguration)
syncHive(basePath, fs, parameters)
} else {
true
}
val metaSyncSuccess = metaSync(parameters, basePath, jsc.hadoopConfiguration())
log.info(s"Is Async Compaction Enabled ? $asyncCompactionEnabled")
if (!asyncCompactionEnabled) {
client.close()
}
(commitSuccess && syncHiveSucess, compactionInstant)
(commitSuccess && metaSyncSuccess, compactionInstant)
} else {
log.error(s"$operation failed with $errorCount errors :")
if (log.isTraceEnabled) {