Moving depedencies off cdh to apache + Hive2 support
- Tests redone in the process - Main changes are to RealtimeRecordReader and how it treats maps/arrays - Make hive sync work with Hive 1/2 and CDH environments - Fixes to make corner cases for Hive queries - Spark Hive integration - Working version across Apache and CDH versions - Known Issue - https://github.com/uber/hudi/issues/439
This commit is contained in:
committed by
vinoth chandar
parent
2b1af18941
commit
a5359662be
@@ -32,6 +32,8 @@ import com.uber.hoodie.index.HoodieIndex;
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
import org.apache.avro.Schema.Field;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
@@ -47,7 +49,8 @@ public class DataSourceUtils {
|
||||
public static String getNestedFieldValAsString(GenericRecord record, String fieldName) {
|
||||
String[] parts = fieldName.split("\\.");
|
||||
GenericRecord valueNode = record;
|
||||
for (int i = 0; i < parts.length; i++) {
|
||||
int i = 0;
|
||||
for (;i < parts.length; i++) {
|
||||
String part = parts[i];
|
||||
Object val = valueNode.get(part);
|
||||
if (val == null) {
|
||||
@@ -65,7 +68,9 @@ public class DataSourceUtils {
|
||||
valueNode = (GenericRecord) val;
|
||||
}
|
||||
}
|
||||
throw new HoodieException(fieldName + " field not found in record");
|
||||
throw new HoodieException(fieldName + "(Part -" + parts[i] + ") field not found in record. "
|
||||
+ "Acceptable fields were :" + valueNode.getSchema().getFields()
|
||||
.stream().map(Field::name).collect(Collectors.toList()));
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -19,6 +19,7 @@
|
||||
package com.uber.hoodie
|
||||
|
||||
import com.uber.hoodie.common.model.HoodieTableType
|
||||
import com.uber.hoodie.hive.SlashEncodedDayPartitionValueExtractor
|
||||
|
||||
/**
|
||||
* List of options that can be passed to the Hoodie datasource,
|
||||
@@ -143,4 +144,28 @@ object DataSourceWriteOptions {
|
||||
*/
|
||||
val COMMIT_METADATA_KEYPREFIX_OPT_KEY = "hoodie.datasource.write.commitmeta.key.prefix"
|
||||
val DEFAULT_COMMIT_METADATA_KEYPREFIX_OPT_VAL = "_"
|
||||
|
||||
// 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 HIVE_DATABASE_OPT_KEY = "hoodie.datasource.hive_sync.database"
|
||||
val HIVE_TABLE_OPT_KEY = "hoodie.datasource.hive_sync.table"
|
||||
val HIVE_USER_OPT_KEY = "hoodie.datasource.hive_sync.username"
|
||||
val HIVE_PASS_OPT_KEY = "hoodie.datasource.hive_sync.password"
|
||||
val HIVE_URL_OPT_KEY = "hoodie.datasource.hive_sync.jdbcUrl"
|
||||
val HIVE_PARTITION_FIELDS_OPT_KEY = "hoodie.datasource.hive_sync.partition_fields"
|
||||
val HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY = "hoodie.datasource.hive_sync.partition_extractor_class"
|
||||
val HIVE_ASSUME_DATE_PARTITION_OPT_KEY = "hoodie.datasource.hive_sync.assume_date_partitioning"
|
||||
|
||||
// DEFAULT FOR HIVE SPECIFIC CONFIGS
|
||||
val DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL = "false"
|
||||
val DEFAULT_HIVE_DATABASE_OPT_VAL = "default"
|
||||
val DEFAULT_HIVE_TABLE_OPT_VAL = "unknown"
|
||||
val DEFAULT_HIVE_USER_OPT_VAL = "hive"
|
||||
val DEFAULT_HIVE_PASS_OPT_VAL = "hive"
|
||||
val DEFAULT_HIVE_URL_OPT_VAL = "jdbc:hive2://localhost:10000"
|
||||
val DEFAULT_HIVE_PARTITION_FIELDS_OPT_VAL = ""
|
||||
val DEFAULT_HIVE_PARTITION_EXTRACTOR_CLASS_OPT_VAL = classOf[SlashEncodedDayPartitionValueExtractor].getCanonicalName
|
||||
val DEFAULT_HIVE_ASSUME_DATE_PARTITION_OPT_VAL = "false"
|
||||
}
|
||||
|
||||
@@ -25,11 +25,12 @@ import java.util.{Optional, Properties}
|
||||
import com.uber.hoodie.DataSourceReadOptions._
|
||||
import com.uber.hoodie.DataSourceWriteOptions._
|
||||
import com.uber.hoodie.common.table.{HoodieTableConfig, HoodieTableMetaClient}
|
||||
import com.uber.hoodie.common.util.TypedProperties
|
||||
import com.uber.hoodie.common.util.{FSUtils, TypedProperties}
|
||||
import com.uber.hoodie.config.HoodieWriteConfig
|
||||
import com.uber.hoodie.exception.HoodieException
|
||||
import org.apache.avro.generic.GenericRecord
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.hadoop.fs.{FileSystem, Path}
|
||||
import org.apache.hadoop.hive.conf.HiveConf
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
import org.apache.spark.rdd.RDD
|
||||
@@ -39,6 +40,7 @@ import org.apache.spark.sql.types.StructType
|
||||
import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.collection.mutable.ListBuffer
|
||||
|
||||
/**
|
||||
* Hoodie Spark Datasource, for reading and writing hoodie datasets
|
||||
@@ -92,6 +94,7 @@ class DefaultSource extends RelationProvider
|
||||
classOf[com.uber.hoodie.hadoop.HoodieROTablePathFilter],
|
||||
classOf[org.apache.hadoop.fs.PathFilter]);
|
||||
|
||||
log.info("Constructing hoodie (as parquet) data source with options :" + parameters)
|
||||
// simply return as a regular parquet relation
|
||||
DataSource.apply(
|
||||
sparkSession = sqlContext.sparkSession,
|
||||
@@ -118,6 +121,15 @@ class DefaultSource extends RelationProvider
|
||||
defaultsMap.putIfAbsent(PARTITIONPATH_FIELD_OPT_KEY, DEFAULT_PARTITIONPATH_FIELD_OPT_VAL)
|
||||
defaultsMap.putIfAbsent(KEYGENERATOR_CLASS_OPT_KEY, DEFAULT_KEYGENERATOR_CLASS_OPT_VAL)
|
||||
defaultsMap.putIfAbsent(COMMIT_METADATA_KEYPREFIX_OPT_KEY, DEFAULT_COMMIT_METADATA_KEYPREFIX_OPT_VAL)
|
||||
defaultsMap.putIfAbsent(HIVE_SYNC_ENABLED_OPT_KEY, DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL)
|
||||
defaultsMap.putIfAbsent(HIVE_DATABASE_OPT_KEY, DEFAULT_HIVE_DATABASE_OPT_VAL)
|
||||
defaultsMap.putIfAbsent(HIVE_TABLE_OPT_KEY, DEFAULT_HIVE_TABLE_OPT_VAL)
|
||||
defaultsMap.putIfAbsent(HIVE_USER_OPT_KEY, DEFAULT_HIVE_USER_OPT_VAL)
|
||||
defaultsMap.putIfAbsent(HIVE_PASS_OPT_KEY, DEFAULT_HIVE_PASS_OPT_VAL)
|
||||
defaultsMap.putIfAbsent(HIVE_URL_OPT_KEY, DEFAULT_HIVE_URL_OPT_VAL)
|
||||
defaultsMap.putIfAbsent(HIVE_PARTITION_FIELDS_OPT_KEY, DEFAULT_HIVE_PARTITION_FIELDS_OPT_VAL)
|
||||
defaultsMap.putIfAbsent(HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY, DEFAULT_HIVE_PARTITION_EXTRACTOR_CLASS_OPT_VAL)
|
||||
defaultsMap.putIfAbsent(HIVE_ASSUME_DATE_PARTITION_OPT_KEY, DEFAULT_HIVE_ASSUME_DATE_PARTITION_OPT_VAL)
|
||||
mapAsScalaMap(defaultsMap)
|
||||
}
|
||||
|
||||
@@ -200,7 +212,8 @@ class DefaultSource extends RelationProvider
|
||||
}
|
||||
|
||||
// Create a HoodieWriteClient & issue the write.
|
||||
val client = DataSourceUtils.createHoodieClient(new JavaSparkContext(sparkContext),
|
||||
val jsc = new JavaSparkContext(sparkContext);
|
||||
val client = DataSourceUtils.createHoodieClient(jsc,
|
||||
schema.toString,
|
||||
path.get,
|
||||
tblName.get,
|
||||
@@ -228,6 +241,13 @@ class DefaultSource extends RelationProvider
|
||||
else {
|
||||
log.info("Commit " + commitTime + " failed!")
|
||||
}
|
||||
|
||||
val hiveSyncEnabled = parameters.get(HIVE_SYNC_ENABLED_OPT_KEY).map(r => r.toBoolean).getOrElse(false)
|
||||
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)
|
||||
}
|
||||
client.close
|
||||
} else {
|
||||
log.error(s"Upsert failed with ${errorCount} errors :");
|
||||
@@ -247,5 +267,28 @@ class DefaultSource extends RelationProvider
|
||||
createRelation(sqlContext, parameters, df.schema)
|
||||
}
|
||||
|
||||
private def syncHive(basePath: Path, fs: FileSystem, parameters: Map[String, String]): Boolean = {
|
||||
val hiveSyncConfig: HiveSyncConfig = buildSyncConfig(basePath, parameters)
|
||||
val hiveConf: HiveConf = new HiveConf()
|
||||
hiveConf.addResource(fs.getConf)
|
||||
new HiveSyncTool(hiveSyncConfig, hiveConf, fs).syncHoodieTable()
|
||||
true
|
||||
}
|
||||
|
||||
private def buildSyncConfig(basePath: Path, parameters: Map[String, String]): HiveSyncConfig = {
|
||||
val hiveSyncConfig: HiveSyncConfig = new HiveSyncConfig()
|
||||
hiveSyncConfig.basePath = basePath.toString
|
||||
hiveSyncConfig.assumeDatePartitioning =
|
||||
parameters.get(HIVE_ASSUME_DATE_PARTITION_OPT_KEY).exists(r => r.toBoolean)
|
||||
hiveSyncConfig.databaseName = parameters(HIVE_DATABASE_OPT_KEY)
|
||||
hiveSyncConfig.tableName = parameters(HIVE_TABLE_OPT_KEY)
|
||||
hiveSyncConfig.hiveUser = parameters(HIVE_USER_OPT_KEY)
|
||||
hiveSyncConfig.hivePass = parameters(HIVE_PASS_OPT_KEY)
|
||||
hiveSyncConfig.jdbcUrl = parameters(HIVE_URL_OPT_KEY)
|
||||
hiveSyncConfig.partitionFields =
|
||||
ListBuffer(parameters(HIVE_PARTITION_FIELDS_OPT_KEY).split(",").map(_.trim).toList: _*)
|
||||
hiveSyncConfig.partitionValueExtractorClass = parameters(HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY)
|
||||
hiveSyncConfig
|
||||
}
|
||||
override def shortName(): String = "hoodie"
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user