Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
6 changes: 3 additions & 3 deletions website/docs/clustering.md
Original file line number Diff line number Diff line change
Expand Up @@ -161,11 +161,11 @@ Users can leverage [HoodieClusteringJob](https://cwiki.apache.org/confluence/dis
to setup 2-step asynchronous clustering.

### HoodieClusteringJob
With the release of Hudi version 0.9.0, we can schedule as well as execute clustering in the same step. We just need to
specify the `—mode` or `-m` option. There are three modes:
By specifying the `scheduleAndExecute` mode both schedule as well as clustering can be achieved in the same step.
The appropriate mode can be specified using `-mode` or `-m` option. There are three modes:

1. `schedule`: Make a clustering plan. This gives an instant which can be passed in execute mode.
2. `execute`: Execute a clustering plan at given instant which means --instant-time is required here.
2. `execute`: Execute a clustering plan at a particular instant. If no instant-time is specified, HoodieClusteringJob will execute for the earliest instant on the Hudi timeline.
3. `scheduleAndExecute`: Make a clustering plan first and execute that plan immediately.

Note that to run this job while the original writer is still running, please enable multi-writing:
Expand Down
54 changes: 35 additions & 19 deletions website/docs/compaction.md
Original file line number Diff line number Diff line change
Expand Up @@ -5,11 +5,7 @@ toc: true
last_modified_at:
---

For Merge-On-Read table, data is stored using a combination of columnar (e.g parquet) + row based (e.g avro) file formats.
Updates are logged to delta files & later compacted to produce new versions of columnar files synchronously or
asynchronously. One of the main motivations behind Merge-On-Read is to reduce data latency when ingesting records.
Hence, it makes sense to run compaction asynchronously without blocking ingestion.

Compaction is executed asynchronously with Hudi by default.

## Async Compaction

Expand All @@ -19,15 +15,13 @@ Async Compaction is performed in 2 steps:
slices** to be compacted. A compaction plan is finally written to Hudi timeline.
1. ***Compaction Execution***: A separate process reads the compaction plan and performs compaction of file slices.


## Deployment Models

There are few ways by which we can execute compactions asynchronously.

### Spark Structured Streaming

With 0.6.0, we now have support for running async compactions in Spark
Structured Streaming jobs. Compactions are scheduled and executed asynchronously inside the
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.

Expand Down Expand Up @@ -74,22 +68,44 @@ spark-submit --packages org.apache.hudi:hudi-utilities-bundle_2.11:0.6.0 \
--continous
```

### Hudi Compactor Utility
Hudi provides a standalone tool to execute specific compactions asynchronously. Below is an example and you can read more in the [deployment guide](/docs/deployment#compactions)

Example:
```properties
spark-submit --packages org.apache.hudi:hudi-utilities-bundle_2.11:0.6.0 \
--class org.apache.hudi.utilities.HoodieCompactor \
--base-path <base_path> \
--table-name <table_name> \
--schema-file <schema_file> \
--instant-time <compaction_instant>
```

Note, the `instant-time` parameter is now optional for the Hudi Compactor Utility. If using the utility without `--instant time`,
the spark-submit will execute the earliest scheduled compaction on the Hudi timeline.

### Hudi CLI
Hudi CLI is yet another way to execute specific compactions asynchronously. Here is an example
Hudi CLI is yet another way to execute specific compactions asynchronously. Here is an example and you can read more in the [deployment guide](/docs/cli#compactions)

Example:
```properties
hudi:trips->compaction run --tableName <table_name> --parallelism <parallelism> --compactionInstant <InstantTime>
...
```

### Hudi Compactor Script
Hudi provides a standalone tool to also execute specific compactions asynchronously. Below is an example and you can read more in the [deployment guide](/docs/next/deployment#compactions)
## Synchronous Compaction
By default, compaction is run asynchronously.

```properties
spark-submit --packages org.apache.hudi:hudi-utilities-bundle_2.11:0.6.0 \
--class org.apache.hudi.utilities.HoodieCompactor \
--base-path <base_path> \
--table-name <table_name> \
--instant-time <compaction_instant> \
--schema-file <schema_file>
```
If latency of ingesting records is important for you, you are most likely using Merge-On-Read tables.
Merge-On-Read tables store data using a combination of columnar (e.g parquet) + row based (e.g avro) file formats.
Updates are logged to delta files & later compacted to produce new versions of columnar files.
To improve ingestion latency, Async Compaction is the default configuration.

If immediate read performance of a new commit is important for you, or you want simplicity of not managing separate compaction jobs,
you may want Synchronous compaction, which means that as a commit is written it is also compacted by the same job.

Compaction is run synchronously by passing the flag "--disable-compaction" (Meaning to disable async compaction scheduling).
When both ingestion and compaction is running in the same spark context, you can use resource allocation configuration
in DeltaStreamer CLI such as ("--delta-sync-scheduling-weight",
"--compact-scheduling-weight", ""--delta-sync-scheduling-minshare", and "--compact-scheduling-minshare")
to control executor allocation between ingestion and compaction.
5 changes: 5 additions & 0 deletions website/docs/hoodie_deltastreamer.md
Original file line number Diff line number Diff line change
Expand Up @@ -209,3 +209,8 @@ A deltastreamer job can then be triggered as follows:
```

Read more in depth about concurrency control in the [concurrency control concepts](/docs/concurrency_control) section

## Hudi Kafka Connect Sink
If you want to perform streaming ingestion into Hudi format similar to HoodieDeltaStreamer, but you don't want to depend on Spark,
try out the new experimental release of Hudi Kafka Connect Sink. Read the [ReadMe](https://github.com/apache/hudi/tree/master/hudi-kafka-connect)
for full documentation.
Loading