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


##########
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 am adding to extraMetadata but Siva suggested a better way to handle this. 
Now a config "_streaming_sink_checkpoint" is added for this purpose. Any config 
prefixed by `_` gets added to extra metadata.



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