KnightChess commented on code in PR #11376:
URL: https://github.com/apache/hudi/pull/11376#discussion_r1624306145


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -174,32 +174,43 @@ class HoodieSparkSqlWriterInternal {
             sourceDf: DataFrame,
             streamingWritesParamsOpt: Option[StreamingWriteParams] = 
Option.empty,
             hoodieWriteClient: Option[SparkRDDWriteClient[_]] = Option.empty):
-
   (Boolean, HOption[String], HOption[String], HOption[String], 
SparkRDDWriteClient[_], HoodieTableConfig) = {
-    var succeeded = false
-    var counter = 0
-    val maxRetry: Integer = 
Integer.parseInt(optParams.getOrElse(HoodieWriteConfig.NUM_RETRIES_ON_CONFLICT_FAILURES.key(),
 HoodieWriteConfig.NUM_RETRIES_ON_CONFLICT_FAILURES.defaultValue().toString))
-    var toReturn: (Boolean, HOption[String], HOption[String], HOption[String], 
SparkRDDWriteClient[_], HoodieTableConfig) = null
 
-    while (counter <= maxRetry && !succeeded) {
-      try {
-        toReturn = writeInternal(sqlContext, mode, optParams, sourceDf, 
streamingWritesParamsOpt, hoodieWriteClient)
-        if (counter > 0) {
-          log.warn(s"Succeeded with attempt no $counter")
-        }
-        succeeded = true
-      } catch {
-        case e: HoodieWriteConflictException =>
-          val writeConcurrencyMode = 
optParams.getOrElse(HoodieWriteConfig.WRITE_CONCURRENCY_MODE.key(), 
HoodieWriteConfig.WRITE_CONCURRENCY_MODE.defaultValue())
-          if (WriteConcurrencyMode.supportsMultiWriter(writeConcurrencyMode) 
&& counter < maxRetry) {
-            counter += 1
-            log.warn(s"Conflict found. Retrying again for attempt no $counter")
-          } else {
-            throw e
+    val retryWrite: () => (Boolean, HOption[String], HOption[String], 
HOption[String], SparkRDDWriteClient[_], HoodieTableConfig) = () => {
+      var succeeded = false
+      var counter = 0
+      val maxRetry: Integer = 
Integer.parseInt(optParams.getOrElse(HoodieWriteConfig.NUM_RETRIES_ON_CONFLICT_FAILURES.key(),
 HoodieWriteConfig.NUM_RETRIES_ON_CONFLICT_FAILURES.defaultValue().toString))
+      var toReturn: (Boolean, HOption[String], HOption[String], 
HOption[String], SparkRDDWriteClient[_], HoodieTableConfig) = null
+
+      while (counter <= maxRetry && !succeeded) {
+        try {
+          toReturn = writeInternal(sqlContext, mode, optParams, sourceDf, 
streamingWritesParamsOpt, hoodieWriteClient)
+          if (counter > 0) {
+            log.warn(s"Succeeded with attempt no $counter")
           }
+          succeeded = true
+        } catch {
+          case e: HoodieWriteConflictException =>
+            val writeConcurrencyMode = 
optParams.getOrElse(HoodieWriteConfig.WRITE_CONCURRENCY_MODE.key(), 
HoodieWriteConfig.WRITE_CONCURRENCY_MODE.defaultValue())
+            if (WriteConcurrencyMode.supportsMultiWriter(writeConcurrencyMode) 
&& counter < maxRetry) {
+              counter += 1
+              log.warn(s"Conflict found. Retrying again for attempt no 
$counter")
+            } else {
+              throw e
+            }
+        }
       }
+      toReturn
+    }
+
+    val executionId = getExecutionId(sqlContext.sparkContext, 
sourceDf.queryExecution)
+    if (executionId.isEmpty) {
+      sparkAdapter.sqlExecutionWithNewExecutionId(sourceDf.sparkSession, 
sourceDf.queryExecution, Option("Hudi Command"))(

Review Comment:
   this executionId will be sub-list in rootExecutionId after this pr 
https://github.com/apache/spark/pull/40403, so ignore this `TODO` 
https://github.com/apache/hudi/pull/8233#discussion_r1298071684, cc @codope 



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