xushiyan commented on code in PR #7901:
URL: https://github.com/apache/hudi/pull/7901#discussion_r1120657881


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -1037,17 +1043,23 @@ object HoodieSparkSqlWriter {
   private def mergeParamsAndGetHoodieConfig(optParams: Map[String, String],
                                             tableConfig: HoodieTableConfig, 
mode: SaveMode): (Map[String, String], HoodieConfig) = {
     val translatedOptions = 
DataSourceWriteOptions.translateSqlOptions(optParams)
-    val mergedParams = mutable.Map.empty ++ 
HoodieWriterUtils.parametersWithWriteDefaults(translatedOptions)
+    var translatedOptsWithMappedTableConfig = mutable.Map.empty ++ 
translatedOptions.toMap
+    if (tableConfig != null && mode != SaveMode.Overwrite) {
+      // for missing write configs corresponding to table configs, fill them 
up.
+      fetchMissingWriteConfigsFromTableConfig(tableConfig, 
optParams).foreach((kv) => translatedOptsWithMappedTableConfig += (kv._1 -> 
kv._2))
+    }
+    val mergedParams = mutable.Map.empty ++ 
HoodieWriterUtils.parametersWithWriteDefaults(translatedOptsWithMappedTableConfig.toMap)
     if (!mergedParams.contains(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key)
       && mergedParams.contains(KEYGENERATOR_CLASS_NAME.key)) {
-      mergedParams(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key) = 
mergedParams(KEYGENERATOR_CLASS_NAME.key)
+      mergedParams(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key) = 
mergedParams(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key)

Review Comment:
   should just keep using the static import `KEYGENERATOR_CLASS_NAME`



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndex.scala:
##########
@@ -95,6 +96,7 @@ class TestColumnStatsIndex extends HoodieSparkClientTestBase {
       DataSourceWriteOptions.TABLE_TYPE.key -> testCase.tableType.toString,
       RECORDKEY_FIELD.key -> "c1",
       PRECOMBINE_FIELD.key -> "c1",
+      DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key() -> 
classOf[NonpartitionedKeyGenerator].getCanonicalName,

Review Comment:
   not getting this additionally set config - we don't expect any user-side 
changes, as in we only re-use table configs if not set. am i missing sth here?



##########
hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCDataFrameSuite.scala:
##########
@@ -433,6 +434,7 @@ class TestCDCDataFrameSuite extends HoodieCDCTestBase {
   def testDataSourceWriteWithPartitionField(tableType: String, loggingMode: 
String): Unit = {
     val options = commonOpts ++ Map(
       DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
+      DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key() -> 
classOf[SimpleKeyGenerator].getCanonicalName,

Review Comment:
   ditto



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -202,7 +206,7 @@ object HoodieSparkSqlWriter {
           .setRecordKeyFields(hoodieConfig.getString(RECORDKEY_FIELD))
           
.setCDCEnabled(hoodieConfig.getBooleanOrDefault(HoodieTableConfig.CDC_ENABLED))
           
.setCDCSupplementalLoggingMode(hoodieConfig.getStringOrDefault(HoodieTableConfig.CDC_SUPPLEMENTAL_LOGGING_MODE))
-          .setKeyGeneratorClassProp(originKeyGeneratorClassName)
+          
.setKeyGeneratorClassProp(hoodieConfig.getString(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key))

Review Comment:
   static import to reduce verbosity



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to