raviMoengage opened a new issue, #5565:
URL: https://github.com/apache/hudi/issues/5565

   
   **Describe the problem you faced**
   
   We are unable to make async compaction work on the MOR table using spark 
streaming.
   
   **Expected behavior**
   
   As per the 
[documentation](https://hudi.apache.org/docs/compaction#spark-structured-streaming)
 spark-structured streaming should have async compaction enabled by default for 
MOR . 
   
   **Environment Description**
   
   * Hudi version : 0.11.0
   
   * Spark version : 3.1.3
   
   * Hadoop version : 3.2.0
   
   * Storage (HDFS/S3/GCS..) : S3
   
   * Running on Docker? (yes/no) : No
   
   
   ### Async compaction
    These configuration are used for enabling async compaction.
   
   ```
   hoodie.datasource.compaction.async.enable = true
   hoodie.compact.inline.max.delta.commits = 1 
   ```
   
   Async compaction is not enabled, here are the sample logs
   ```
   22/05/12 00:49:22 INFO HoodieSparkSqlWriter$: Commit 20220512003040185 
successful!
   22/05/12 00:49:22 INFO HoodieSparkSqlWriter$: Config.inlineCompactionEnabled 
? false
   22/05/12 00:49:22 INFO HoodieSparkSqlWriter$: Compaction Scheduled is 
Optional.empty
   22/05/12 00:49:22 INFO HoodieSparkSqlWriter$: Config.asyncClusteringEnabled 
? false
   22/05/12 00:49:22 INFO HoodieSparkSqlWriter$: Clustering Scheduled is 
Optional.empty
   22/05/12 00:49:22 INFO HoodieSparkSqlWriter$: Is Async Compaction Enabled ? 
false
   ```
   
   **Context**
   
   - The default value of 
[asyncCompactionTriggerFnDefined](https://github.com/apache/hudi/blob/release-0.11.0/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala#L64)
 is false.
   
   - Since `asyncCompactionTriggerFn` has Option.empty as default value hence 
it stays false here [HoodieSparkSqlWriter.scala 
](https://github.com/apache/hudi/blob/release-0.11.0/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala#L100)
   
   - So `isAsyncCompactionEnabled` function return false because 
`asyncCompactionTriggerFnDefined` is false. [HoodieSparkSqlWriter.scala 
](https://github.com/apache/hudi/blob/release-0.11.0/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala#L709)
   
   
   Hudi Configuration.
   ```
   hudi-conf = {
   "hoodie.table.name" = table_name
   "hoodie.datasource.write.table.name" = table_name
   "hoodie.datasource.write.table.type" = MERGE_ON_READ
   "hoodie.datasource.write.operation" = upsert
   "hoodie.datasource.write.recordkey.field" = record_key
   "hoodie.datasource.write.precombine.field" = time
   "hoodie.datasource.write.hive_style_partitioning" = true
   "hoodie.datasource.write.partitionpath.field" = key_part
   "hoodie.datasource.write.streaming.ignore.failed.batch" = true
   "hoodie.file.index.enable" = true
   "hoodie.index.type" = SIMPLE
   "hoodie.cleaner.policy" = KEEP_LATEST_COMMITS
   "hoodie.cleaner.delete.bootstrap.base.file" = false
   "hoodie.clean.async" = true
   "hoodie.clean.automatic" = true
   "hoodie.cleaner.commits.retained" = 10
   "hoodie.cleaner.parallelism" = 300
   "hoodie.cleaner.incremental.mode" = true
   "hoodie.cleaner.policy.failed.writes" = LAZY
   "hoodie.datasource.compaction.async.enable"=true
   "hoodie.compact.inline.max.delta.commits"=1
   "hoodie.insert.shuffle.parallelism" = 300
   "hoodie.upsert.shuffle.parallelism" = 300
   "hoodie.write.concurrency.mode" = optimistic_concurrency_control
   "hoodie.write.lock.provider" = 
"org.apache.hudi.client.transaction.lock.ZookeeperBasedLockProvider"
   "hoodie.write.lock.zookeeper.url" = localhost
   "hoodie.write.lock.zookeeper.port" = 2181
   "hoodie.write.lock.zookeeper.lock_key" = device
   "hoodie.write.lock.zookeeper.base_path" = /hudi-datalake
   }
   ```
   


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