This is an automated email from the ASF dual-hosted git repository.

yihua pushed a commit to branch release-0.14.1-spark35-scala213
in repository https://gitbox.apache.org/repos/asf/hudi.git

commit 0f783f1a66fb2a0650901a354cdfd829b8f1f0d5
Author: Aditya Goenka <[email protected]>
AuthorDate: Sat Feb 3 03:59:58 2024 +0530

    [HUDI-6868] Support extracting passwords from credential store for Hive 
Sync (#10577)
    
    Co-authored-by: Danny Chan <[email protected]>
---
 .../scala/org/apache/hudi/HoodieSparkSqlWriter.scala     | 16 +++++++++++++++-
 1 file changed, 15 insertions(+), 1 deletion(-)

diff --git 
a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala
 
b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala
index 03a2cbb3b02..c6958242620 100644
--- 
a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala
+++ 
b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala
@@ -21,6 +21,8 @@ import org.apache.avro.Schema
 import org.apache.avro.generic.GenericData
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.hive.conf.HiveConf
+import org.apache.hadoop.hive.shims.ShimLoader
 import 
org.apache.hudi.AutoRecordKeyGenerationUtils.mayBeValidateParamsForAutoGenerationOfRecordKeys
 import org.apache.hudi.AvroConversionUtils.{convertAvroSchemaToStructType, 
convertStructTypeToAvroSchema, getAvroRecordNameAndNamespace}
 import 
org.apache.hudi.DataSourceOptionsHelper.fetchMissingWriteConfigsFromTableConfig
@@ -1006,7 +1008,19 @@ class HoodieSparkSqlWriterInternal {
       
properties.put(HiveSyncConfigHolder.HIVE_SYNC_SCHEMA_STRING_LENGTH_THRESHOLD.key,
 
spark.sessionState.conf.getConf(StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD).toString)
       properties.put(HoodieSyncConfig.META_SYNC_SPARK_VERSION.key, 
SPARK_VERSION)
       
properties.put(HoodieSyncConfig.META_SYNC_USE_FILE_LISTING_FROM_METADATA.key, 
hoodieConfig.getBoolean(HoodieMetadataConfig.ENABLE))
-
+      if ((fs.getConf.get(HiveConf.ConfVars.METASTOREPWD.varname) == null || 
fs.getConf.get(HiveConf.ConfVars.METASTOREPWD.varname).isEmpty) &&
+        (properties.get(HiveSyncConfigHolder.HIVE_PASS.key()) == null || 
properties.get(HiveSyncConfigHolder.HIVE_PASS.key()).toString.isEmpty)){
+        try {
+          val passwd = 
ShimLoader.getHadoopShims.getPassword(spark.sparkContext.hadoopConfiguration, 
HiveConf.ConfVars.METASTOREPWD.varname)
+          if (passwd != null && !passwd.isEmpty) {
+            fs.getConf.set(HiveConf.ConfVars.METASTOREPWD.varname, passwd)
+            properties.put(HiveSyncConfigHolder.HIVE_PASS.key(), passwd)
+          }
+        } catch {
+          case e: Exception =>
+            log.info("Exception while trying to get Meta Sync password from 
hadoop credential store", e)
+        }
+      }
       // Collect exceptions in list because we want all sync to run. Then we 
can throw
       val failedMetaSyncs = new mutable.HashMap[String,HoodieException]()
       syncClientToolClassSet.foreach(impl => {

Reply via email to