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


##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala:
##########
@@ -78,26 +82,62 @@ class HoodieStreamingSink(sqlContext: SQLContext,
       log.error("Async clustering service shutdown unexpectedly")
       throw new IllegalStateException("Async clustering service shutdown 
unexpectedly")
     }
+
+    // get the latest checkpoint from the commit metadata to check if the 
microbatch has already been prcessed or not
+    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(SinkCheckpointKey)
+        if (!StringUtils.isNullOrEmpty(lastCheckpoint)) {
+          latestBatchId = lastCheckpoint.toLong
+        }
+      }
+    }
+    if (latestBatchId >= batchId) {
+      val queryId = 
sqlContext.sparkContext.getLocalProperty(StreamExecution.QUERY_ID_KEY)
+      assert(queryId != null)
+      log.warn(s"Skipping already completed batch $batchId in query $queryId")
+      return
+    }
+
     // Override to use direct markers. In Structured streaming, timeline 
server is closed after
     // first micro-batch and subsequent micro-batches do not have timeline 
server running.
     // Thus, we can't use timeline-server-based markers.
     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")
+    }
+    // add batchId as checkpoint to the extra metadata
+    updatedOptions = updatedOptions.updated(SinkCheckpointKey, 
batchId.toString)
 
     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"

Review Comment:
   ah yes good catch.. thanks



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