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/04/26 02:53:49 UTC

[GitHub] [hudi] nsivabalan commented on a diff in pull request #5304: [DOCS] Add faq for async compaction options

nsivabalan commented on code in PR #5304:
URL: https://github.com/apache/hudi/pull/5304#discussion_r858192316


##########
website/learn/faq.md:
##########
@@ -253,6 +253,25 @@ Simplest way to run compaction on MOR dataset is to run the [compaction inline](
 
 That said, for obvious reasons of not blocking ingesting for compaction, you may want to run it asynchronously as well. This can be done either via a separate [compaction job](https://github.com/apache/hudi/blob/master/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java) that is scheduled by your workflow scheduler/notebook independently. If you are using delta streamer, then you can run in [continuous mode](https://github.com/apache/hudi/blob/d3edac4612bde2fa9deca9536801dbc48961fb95/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java#L241) where the ingestion and compaction are both managed concurrently in a single spark run time.
 
+### What options do I have for asynchronous/offline compactions on MOR dataset?
+
+There are a couple of options depending on how you write to Hudi. But first let us understand briefly what is involved. There are two parts to compaction
+- Scheduling: In this step, Hudi scans the partitions and selects file slices to be compacted. A compaction plan is finally written to Hudi timeline. Scheduling needs tighter coordination with other writers (regular ingestion is considered one of the writers). If scheduling is done inline with the ingestion job, this coordination is automatically taken care of. Else when scheduling happens asynchronously a lock provider needs to be configured for this coordination among multiple writers.
+- Execution: A separate process reads the compaction plan and performs compaction of file slices. Execution doesnt need the same level of coordination with other writers as Scheduling step and can be decoupled from ingestion job easily.
+
+Depending on how you write to Hudi these are the possible options currently.
+- DeltaStreamer:
+   - In Continuous mode asynchronous compaction is achieved by default. Here scheduling is done by the ingestion job inline and compaction execution is achieved asynchronously by a separate parallel thread.
+- Spark datasource:
+   - Async scheduling and async execution can be achieved by periodically running an offline Hudi Compactor Utility or Hudi CLI. However this needs a lock provider to be configured.
+   - Alternately to avoid dependency on lock providers, scheduling alone can be done inline by regular writer using the config `hoodie.compact.schedule.inline` . And compaction execution can be done offline by periodically triggering the Hudi Compactor Utility or Hudi CLI.

Review Comment:
   can we add a line that this is from 0.11



##########
website/learn/faq.md:
##########
@@ -253,6 +253,25 @@ Simplest way to run compaction on MOR dataset is to run the [compaction inline](
 
 That said, for obvious reasons of not blocking ingesting for compaction, you may want to run it asynchronously as well. This can be done either via a separate [compaction job](https://github.com/apache/hudi/blob/master/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java) that is scheduled by your workflow scheduler/notebook independently. If you are using delta streamer, then you can run in [continuous mode](https://github.com/apache/hudi/blob/d3edac4612bde2fa9deca9536801dbc48961fb95/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java#L241) where the ingestion and compaction are both managed concurrently in a single spark run time.
 
+### What options do I have for asynchronous/offline compactions on MOR dataset?
+
+There are a couple of options depending on how you write to Hudi. But first let us understand briefly what is involved. There are two parts to compaction
+- Scheduling: In this step, Hudi scans the partitions and selects file slices to be compacted. A compaction plan is finally written to Hudi timeline. Scheduling needs tighter coordination with other writers (regular ingestion is considered one of the writers). If scheduling is done inline with the ingestion job, this coordination is automatically taken care of. Else when scheduling happens asynchronously a lock provider needs to be configured for this coordination among multiple writers.
+- Execution: A separate process reads the compaction plan and performs compaction of file slices. Execution doesnt need the same level of coordination with other writers as Scheduling step and can be decoupled from ingestion job easily.
+
+Depending on how you write to Hudi these are the possible options currently.
+- DeltaStreamer:
+   - In Continuous mode asynchronous compaction is achieved by default. Here scheduling is done by the ingestion job inline and compaction execution is achieved asynchronously by a separate parallel thread.
+- Spark datasource:
+   - Async scheduling and async execution can be achieved by periodically running an offline Hudi Compactor Utility or Hudi CLI. However this needs a lock provider to be configured.

Review Comment:
   can we add that, this was the case until 0.10



##########
website/learn/faq.md:
##########
@@ -253,6 +253,25 @@ Simplest way to run compaction on MOR dataset is to run the [compaction inline](
 
 That said, for obvious reasons of not blocking ingesting for compaction, you may want to run it asynchronously as well. This can be done either via a separate [compaction job](https://github.com/apache/hudi/blob/master/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java) that is scheduled by your workflow scheduler/notebook independently. If you are using delta streamer, then you can run in [continuous mode](https://github.com/apache/hudi/blob/d3edac4612bde2fa9deca9536801dbc48961fb95/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java#L241) where the ingestion and compaction are both managed concurrently in a single spark run time.
 
+### What options do I have for asynchronous/offline compactions on MOR dataset?
+
+There are a couple of options depending on how you write to Hudi. But first let us understand briefly what is involved. There are two parts to compaction
+- Scheduling: In this step, Hudi scans the partitions and selects file slices to be compacted. A compaction plan is finally written to Hudi timeline. Scheduling needs tighter coordination with other writers (regular ingestion is considered one of the writers). If scheduling is done inline with the ingestion job, this coordination is automatically taken care of. Else when scheduling happens asynchronously a lock provider needs to be configured for this coordination among multiple writers.
+- Execution: A separate process reads the compaction plan and performs compaction of file slices. Execution doesnt need the same level of coordination with other writers as Scheduling step and can be decoupled from ingestion job easily.

Review Comment:
   "A seperate process" -> don't really need to a separate process. can we re-word this a bit



##########
website/learn/faq.md:
##########
@@ -253,6 +253,25 @@ Simplest way to run compaction on MOR dataset is to run the [compaction inline](
 
 That said, for obvious reasons of not blocking ingesting for compaction, you may want to run it asynchronously as well. This can be done either via a separate [compaction job](https://github.com/apache/hudi/blob/master/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java) that is scheduled by your workflow scheduler/notebook independently. If you are using delta streamer, then you can run in [continuous mode](https://github.com/apache/hudi/blob/d3edac4612bde2fa9deca9536801dbc48961fb95/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java#L241) where the ingestion and compaction are both managed concurrently in a single spark run time.
 
+### What options do I have for asynchronous/offline compactions on MOR dataset?
+
+There are a couple of options depending on how you write to Hudi. But first let us understand briefly what is involved. There are two parts to compaction
+- Scheduling: In this step, Hudi scans the partitions and selects file slices to be compacted. A compaction plan is finally written to Hudi timeline. Scheduling needs tighter coordination with other writers (regular ingestion is considered one of the writers). If scheduling is done inline with the ingestion job, this coordination is automatically taken care of. Else when scheduling happens asynchronously a lock provider needs to be configured for this coordination among multiple writers.
+- Execution: A separate process reads the compaction plan and performs compaction of file slices. Execution doesnt need the same level of coordination with other writers as Scheduling step and can be decoupled from ingestion job easily.
+
+Depending on how you write to Hudi these are the possible options currently.
+- DeltaStreamer:
+   - In Continuous mode asynchronous compaction is achieved by default. Here scheduling is done by the ingestion job inline and compaction execution is achieved asynchronously by a separate parallel thread.
+- Spark datasource:
+   - Async scheduling and async execution can be achieved by periodically running an offline Hudi Compactor Utility or Hudi CLI. However this needs a lock provider to be configured.
+   - Alternately to avoid dependency on lock providers, scheduling alone can be done inline by regular writer using the config `hoodie.compact.schedule.inline` . And compaction execution can be done offline by periodically triggering the Hudi Compactor Utility or Hudi CLI.
+- Spark structured streaming:
+   - Compactions are scheduled and executed asynchronously inside the streaming job. Async Compactions are enabled by default for structured streaming jobs on Merge-On-Read table.

Review Comment:
   please add that its not possible to disable async compaction for MOR w/ spark structured streaming



##########
website/learn/faq.md:
##########
@@ -253,6 +253,25 @@ Simplest way to run compaction on MOR dataset is to run the [compaction inline](
 
 That said, for obvious reasons of not blocking ingesting for compaction, you may want to run it asynchronously as well. This can be done either via a separate [compaction job](https://github.com/apache/hudi/blob/master/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java) that is scheduled by your workflow scheduler/notebook independently. If you are using delta streamer, then you can run in [continuous mode](https://github.com/apache/hudi/blob/d3edac4612bde2fa9deca9536801dbc48961fb95/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java#L241) where the ingestion and compaction are both managed concurrently in a single spark run time.
 
+### What options do I have for asynchronous/offline compactions on MOR dataset?
+
+There are a couple of options depending on how you write to Hudi. But first let us understand briefly what is involved. There are two parts to compaction
+- Scheduling: In this step, Hudi scans the partitions and selects file slices to be compacted. A compaction plan is finally written to Hudi timeline. Scheduling needs tighter coordination with other writers (regular ingestion is considered one of the writers). If scheduling is done inline with the ingestion job, this coordination is automatically taken care of. Else when scheduling happens asynchronously a lock provider needs to be configured for this coordination among multiple writers.
+- Execution: A separate process reads the compaction plan and performs compaction of file slices. Execution doesnt need the same level of coordination with other writers as Scheduling step and can be decoupled from ingestion job easily.
+
+Depending on how you write to Hudi these are the possible options currently.
+- DeltaStreamer:
+   - In Continuous mode asynchronous compaction is achieved by default. Here scheduling is done by the ingestion job inline and compaction execution is achieved asynchronously by a separate parallel thread.

Review Comment:
   "," after Continuous mode. 
   i.e. 
   ```
   In Continuous mode, asynchronous
   ```



##########
website/learn/faq.md:
##########
@@ -253,6 +253,25 @@ Simplest way to run compaction on MOR dataset is to run the [compaction inline](
 
 That said, for obvious reasons of not blocking ingesting for compaction, you may want to run it asynchronously as well. This can be done either via a separate [compaction job](https://github.com/apache/hudi/blob/master/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java) that is scheduled by your workflow scheduler/notebook independently. If you are using delta streamer, then you can run in [continuous mode](https://github.com/apache/hudi/blob/d3edac4612bde2fa9deca9536801dbc48961fb95/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamer.java#L241) where the ingestion and compaction are both managed concurrently in a single spark run time.
 
+### What options do I have for asynchronous/offline compactions on MOR dataset?
+
+There are a couple of options depending on how you write to Hudi. But first let us understand briefly what is involved. There are two parts to compaction
+- Scheduling: In this step, Hudi scans the partitions and selects file slices to be compacted. A compaction plan is finally written to Hudi timeline. Scheduling needs tighter coordination with other writers (regular ingestion is considered one of the writers). If scheduling is done inline with the ingestion job, this coordination is automatically taken care of. Else when scheduling happens asynchronously a lock provider needs to be configured for this coordination among multiple writers.
+- Execution: A separate process reads the compaction plan and performs compaction of file slices. Execution doesnt need the same level of coordination with other writers as Scheduling step and can be decoupled from ingestion job easily.
+
+Depending on how you write to Hudi these are the possible options currently.
+- DeltaStreamer:
+   - In Continuous mode asynchronous compaction is achieved by default. Here scheduling is done by the ingestion job inline and compaction execution is achieved asynchronously by a separate parallel thread.

Review Comment:
   we can also add a line, that is users wish to disable compaction, they can do so with --disable-compaction config with delta streamer. 



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