You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/07/18 12:17:27 UTC

[GitHub] [hudi] YuweiXiao commented on a diff in pull request #6098: [HUDI-4389] Make HoodieStreamingSink idempotent

YuweiXiao commented on code in PR #6098:
URL: https://github.com/apache/hudi/pull/6098#discussion_r923306507


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala:
##########
@@ -84,20 +96,62 @@ class HoodieStreamingSink(sqlContext: SQLContext,
     var updatedOptions = options.updated(HoodieWriteConfig.MARKERS_TYPE.key(), MarkerType.DIRECT.name())
     // we need auto adjustment enabled for streaming sink since async table services are feasible within the same JVM.
     updatedOptions = updatedOptions.updated(HoodieWriteConfig.AUTO_ADJUST_LOCK_CONFIGS.key, "true")
+    // disable row writer bulk insert of write stream
+    if (options.getOrDefault(OPERATION.key, UPSERT_OPERATION_OPT_VAL).equalsIgnoreCase(BULK_INSERT_OPERATION_OPT_VAL)) {
+      updatedOptions = updatedOptions.updated(ENABLE_ROW_WRITER.key, "false")
+    }
+
+    val queryId = sqlContext.sparkContext.getLocalProperty(StreamExecution.QUERY_ID_KEY)
+    assert(queryId != null)
+    log.warn(s"Query id: $queryId")
+
+    if (metaClient != null) {
+      val lastCommit = metaClient.getActiveTimeline.getCommitsTimeline.filterCompletedInstants().lastInstant()
+      if (lastCommit.isPresent) {
+        val commitMetadata = HoodieCommitMetadata.fromBytes(metaClient.getActiveTimeline.getInstantDetails(lastCommit.get()).get(), classOf[HoodieCommitMetadata])
+        val lastCheckpoint = commitMetadata.getMetadata(HoodieWriteConfig.DELTASTREAMER_CHECKPOINT_KEY)
+        if (!StringUtils.isNullOrEmpty(lastCheckpoint)) {
+          latestBatchId = lastCheckpoint.toLong
+        }
+      }
+    }
+
+    if (latestBatchId >= batchId) {
+      log.warn(s"Skipping already completed batch $batchId in query $queryId")
+      return
+    }
 
     retry(retryCnt, retryIntervalMs)(
       Try(
         HoodieSparkSqlWriter.write(
           sqlContext, mode, updatedOptions, data, hoodieTableConfig, writeClient, Some(triggerAsyncCompactor), Some(triggerAsyncClustering))
-      ) match {
+      )
+      match {
         case Success((true, commitOps, compactionInstantOps, clusteringInstant, client, tableConfig)) =>
-          log.info(s"Micro batch id=$batchId succeeded"
-            + (commitOps.isPresent match {
-            case true => s" for commit=${commitOps.get()}"
-            case _ => s" with no new commits"
+          log.warn(s"Micro batch id=$batchId succeeded"
+            + (if (commitOps.isPresent) {
+            s" for commit=${commitOps.get()}"
+          } else {
+            s" with no new commits"
           }))
+          log.warn(s"Current value of latestBatchId: $latestBatchId")
+          log.warn(s"Setting latestBatchId to batchId $batchId")
+          latestBatchId = batchId
           writeClient = Some(client)
           hoodieTableConfig = Some(tableConfig)
+          metaClient = HoodieTableMetaClient.builder()
+            .setConf(sqlContext.sparkContext.hadoopConfiguration)
+            .setBasePath(client.getConfig.getBasePath)
+            .build()
+          // let's update batchId as checkpoint for this commit
+          if (commitOps.isPresent) {
+            val instant = metaClient.getActiveTimeline.getCompletedInstantForTimestamp(commitOps.get())

Review Comment:
   I guess we could add the `batchId` to the `extraMetadata` field of the commit metadata. I did the similar thing in my own streaming job.



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org