yihua commented on code in PR #8093:
URL: https://github.com/apache/hudi/pull/8093#discussion_r1128405129
##########
website/docs/file_sizing.md:
##########
@@ -3,51 +3,76 @@ title: "File Sizing"
toc: true
---
-This doc will show you how Apache Hudi overcomes the dreaded small files
problem. A key design decision in Hudi was to
-avoid creating small files in the first place and always write properly sized
files.
-There are 2 ways to manage small files in Hudi and below will describe the
advantages and trade-offs of each.
-
-## Auto-Size During ingestion
-
-You can automatically manage size of files during ingestion. This solution
adds a little latency during ingestion, but
-it ensures that read queries are always efficient as soon as a write is
committed. If you don't
-manage file sizing as you write and instead try to periodically run a
file-sizing clean-up, your queries will be slow until that resize cleanup is
periodically performed.
-
-(Note: [bulk_insert](/docs/next/write_operations) write operation does not
provide auto-sizing during ingestion)
-
-### For Copy-On-Write
-This is as simple as configuring the [maximum size for a base/parquet
file](/docs/configurations#hoodieparquetmaxfilesize)
-and the [soft limit](/docs/configurations#hoodieparquetsmallfilelimit) below
which a file should
-be considered a small file. For the initial bootstrap of a Hudi table, tuning
record size estimate is also important to
-ensure sufficient records are bin-packed in a parquet file. For subsequent
writes, Hudi automatically uses average
-record size based on previous commit. Hudi will try to add enough records to a
small file at write time to get it to the
-configured maximum limit. For e.g , with `compactionSmallFileSize=100MB` and
limitFileSize=120MB, Hudi will pick all
-files < 100MB and try to get them upto 120MB.
-
-### For Merge-On-Read
-MergeOnRead works differently for different INDEX choices so there are few
more configs to set:
-
-- Indexes with **canIndexLogFiles = true** : Inserts of new data go directly
to log files. In this case, you can
-configure the [maximum log size](/docs/configurations#hoodielogfilemaxsize)
and a
-[factor](/docs/configurations#hoodielogfiletoparquetcompressionratio) that
denotes reduction in
-size when data moves from avro to parquet files.
-- Indexes with **canIndexLogFiles = false** : Inserts of new data go only to
parquet files. In this case, the
-same configurations as above for the COPY_ON_WRITE case applies.
-
-NOTE : In either case, small files will be auto sized only if there is no
PENDING compaction or associated log file for
-that particular file slice. For example, for case 1: If you had a log file and
a compaction C1 was scheduled to convert
-that log file to parquet, no more inserts can go into that log file. For case
2: If you had a parquet file and an update
-ended up creating an associated delta log file, no more inserts can go into
that parquet file. Only after the compaction
-has been performed and there are NO log files associated with the base parquet
file, can new inserts be sent to auto size that parquet file.
-
-## Auto-Size With Clustering
-**[Clustering](/docs/next/clustering)** is a feature in Hudi to group
-small files into larger ones either synchronously or asynchronously. Since
first solution of auto-sizing small files has
-a tradeoff on ingestion speed (since the small files are sized during
ingestion), if your use-case is very sensitive to
-ingestion latency where you don't want to compromise on ingestion speed which
may end up creating a lot of small files,
-clustering comes to the rescue. Clustering can be scheduled through the
ingestion job and an asynchronus job can stitch
-small files together in the background to generate larger files. NOTE that
during this, ingestion can continue to run concurrently.
-
-*Please note that Hudi always creates immutable files on disk. To be able to
do auto-sizing or clustering, Hudi will
-always create a newer version of the smaller file, resulting in 2 versions of
the same file.
-The [cleaner service](/docs/next/hoodie_cleaner) will later kick in and delete
the older version small file and keep the latest one.*
\ No newline at end of file
+A fundamental problem when writing data to a source is having a lot of small
files. This is also known as a small file problem. If you don’t size the files
appropriately, you can slow down the query performance and work with stale
analytics. Some of the issues you may encounter with small files include the
following:
+- **Reads slow down**: You’ll have to scan through many small files to
retrieve data for a query. It’s a very inefficient way of accessing and
utilizing the data.
+
+- **Processes slow down**: You can slow down your i.e., Spark or Hive jobs;
the more files you have, the more tasks you create for reading each file.
+
+- **Storage increases**: When working with a lot of data, you can be
inefficient in using your storage. For example, many small files can have a
lower compression ratio, leading to more data on storage. If you’re indexing
the data, that also takes up more storage space inside the Parquet files. If
you’re working with a small amount of data, you might not see a significant
impact with storage. However, when dealing with petabyte and exabyte data,
you’ll need to be efficient in managing storage resources.
+
+All these challenges inevitably lead to stale analytics and scalability
challenges:
+- Query performance slows down.
+- Jobs could be running faster.
+- You’re utilizing way more resources.
+
+A critical design decision in the Hudi architecture is to avoid small file
creation. Hudi is uniquely designed to write appropriately sized files
automatically. This document will show you how Apache Hudi overcomes the
dreaded small files problem. There are two ways to manage small files in Hudi:
+
+- Auto-size during ingestion
+- Clustering
+
+Below, we will describe the advantages and trade-offs of each.
+
+## Auto-sizing during ingestion
+
+You can manage file sizes through Hudi’s auto-sizing capability during
ingestion. The default targeted file size for Parquet base files is 120MB,
which can be configured by hoodie.parquet.max.file.size. Auto-sizing may add
some data latency, but it ensures that the read queries are always efficient as
soon as a write transaction is committed. It’s important to note that if you
don’t manage file sizing as you write and, instead, try to run clustering to
fix your file sizing periodically, your queries might be slow until the point
when the clustering finishes.
+
+**Note**: the bulk_insert write operation does not have auto-sizing
capabilities during ingestion
+
+If you need to customize the file sizing, i.e., increase the target file size
or change how small files are identified, follow the instructions below for
Copy-On-Write and Merge-On-Read.
+
+### Copy-On-Write (COW)
+To tune the file sizing for a COW table, you can set the small file limit and
the maximum Parquet file size. Hudi will try to add enough records to a small
file at write time to get it to the configured maximum limit.
+
+ - For example, if the `hoodie.parquet.small.file.limit=104857600` (100MB) and
`hoodie.parquet.max.file.size=125829120` (120MB), Hudi will pick all files <
100MB and try to get them up to 120MB.
+
+For creating a Hudi table initially, setting an accurate record size estimate
is vital to ensure Hudi can adequately estimate how many records need to be
bin-packed in a Parquet file for the first ingestion batch. Then, Hudi
automatically uses the average record size for subsequent writes based on
previous commits.
+
+Here are the important configuration of interest:
+
+## Merge-On-Read (MOR)
+As a MOR table aims to reduce the write amplification, compared to a COW
table, when writing to a MOR table, Hudi limits the number of Parquet base
files to one for auto file sizing during insert and upsert operation. This
limits the number of rewritten files. This can be configured through
`hoodie.merge.small.file.group.candidates.limit`.
+
+In addition to file sizing Parquet base files for a MOR table, you can also
tune the log files file-sizing with `hoodie.logfile.max.size`.
+
+**NOTE**: Small files in file groups included in the requested or inflight
compaction or clustering under the active timeline, or small files with
associated log files are not auto-sized with incoming inserts until the
compaction or clustering is complete. For example:
+
+- In case 1: If you had a log file and a compaction, C1, was scheduled to
convert that log file to Parquet, no more inserts can go into the same file
slice.
+
+- In case 2: If the Hudi table has a file group with a Parquet base file and
an associated log file from updates, or this file group is under a requested or
inflight compaction, no more inserts can go into this file group to
automatically size the Parquet file. Only after the compaction has been
performed, and there are NO log files associated with the base Parquet file,
can new inserts be sent to auto-size that parquet file.
+
+Here are the essential configurations:
+
+| Parameter Name | Default | Description | Scope | Since Version
|
+|----------------|--------|----------|---------------|--------------------------------------|
+| hoodie.parquet.small.file.limit | 104857600 (100MB) | During an insert and
upsert operation, we opportunistically expand existing small files on storage
instead of writing new files to keep the number of files optimum. This config
sets the file size limit below which a storage file becomes a candidate to be
selected as such a `small file`. By default, treat any file <= 100MB as a small
file. Also note that if this is set to <= 0, Hudi will not try to get small
files and directly write new files. | Write COW, MOR | 0.4.0 |
Review Comment:
backticks for configs
##########
website/docs/file_sizing.md:
##########
@@ -3,51 +3,76 @@ title: "File Sizing"
toc: true
---
-This doc will show you how Apache Hudi overcomes the dreaded small files
problem. A key design decision in Hudi was to
-avoid creating small files in the first place and always write properly sized
files.
-There are 2 ways to manage small files in Hudi and below will describe the
advantages and trade-offs of each.
-
-## Auto-Size During ingestion
-
-You can automatically manage size of files during ingestion. This solution
adds a little latency during ingestion, but
-it ensures that read queries are always efficient as soon as a write is
committed. If you don't
-manage file sizing as you write and instead try to periodically run a
file-sizing clean-up, your queries will be slow until that resize cleanup is
periodically performed.
-
-(Note: [bulk_insert](/docs/next/write_operations) write operation does not
provide auto-sizing during ingestion)
-
-### For Copy-On-Write
-This is as simple as configuring the [maximum size for a base/parquet
file](/docs/configurations#hoodieparquetmaxfilesize)
-and the [soft limit](/docs/configurations#hoodieparquetsmallfilelimit) below
which a file should
-be considered a small file. For the initial bootstrap of a Hudi table, tuning
record size estimate is also important to
-ensure sufficient records are bin-packed in a parquet file. For subsequent
writes, Hudi automatically uses average
-record size based on previous commit. Hudi will try to add enough records to a
small file at write time to get it to the
-configured maximum limit. For e.g , with `compactionSmallFileSize=100MB` and
limitFileSize=120MB, Hudi will pick all
-files < 100MB and try to get them upto 120MB.
-
-### For Merge-On-Read
-MergeOnRead works differently for different INDEX choices so there are few
more configs to set:
-
-- Indexes with **canIndexLogFiles = true** : Inserts of new data go directly
to log files. In this case, you can
-configure the [maximum log size](/docs/configurations#hoodielogfilemaxsize)
and a
-[factor](/docs/configurations#hoodielogfiletoparquetcompressionratio) that
denotes reduction in
-size when data moves from avro to parquet files.
-- Indexes with **canIndexLogFiles = false** : Inserts of new data go only to
parquet files. In this case, the
-same configurations as above for the COPY_ON_WRITE case applies.
-
-NOTE : In either case, small files will be auto sized only if there is no
PENDING compaction or associated log file for
-that particular file slice. For example, for case 1: If you had a log file and
a compaction C1 was scheduled to convert
-that log file to parquet, no more inserts can go into that log file. For case
2: If you had a parquet file and an update
-ended up creating an associated delta log file, no more inserts can go into
that parquet file. Only after the compaction
-has been performed and there are NO log files associated with the base parquet
file, can new inserts be sent to auto size that parquet file.
-
-## Auto-Size With Clustering
-**[Clustering](/docs/next/clustering)** is a feature in Hudi to group
-small files into larger ones either synchronously or asynchronously. Since
first solution of auto-sizing small files has
-a tradeoff on ingestion speed (since the small files are sized during
ingestion), if your use-case is very sensitive to
-ingestion latency where you don't want to compromise on ingestion speed which
may end up creating a lot of small files,
-clustering comes to the rescue. Clustering can be scheduled through the
ingestion job and an asynchronus job can stitch
-small files together in the background to generate larger files. NOTE that
during this, ingestion can continue to run concurrently.
-
-*Please note that Hudi always creates immutable files on disk. To be able to
do auto-sizing or clustering, Hudi will
-always create a newer version of the smaller file, resulting in 2 versions of
the same file.
-The [cleaner service](/docs/next/hoodie_cleaner) will later kick in and delete
the older version small file and keep the latest one.*
\ No newline at end of file
+A fundamental problem when writing data to a source is having a lot of small
files. This is also known as a small file problem. If you don’t size the files
appropriately, you can slow down the query performance and work with stale
analytics. Some of the issues you may encounter with small files include the
following:
+- **Reads slow down**: You’ll have to scan through many small files to
retrieve data for a query. It’s a very inefficient way of accessing and
utilizing the data.
Review Comment:
Add this to the end: "As the cloud storage like S3 enforces
[rate-limiting](https://docs.aws.amazon.com/AmazonS3/latest/userguide/optimizing-performance.html)
on how many requests can be processed per second per prefix in a bucket, a
higher number of files, i.e., at least one request per file regardless of the
file size, increases the chance of encountering the rate-limiting, causing the
reader to slow down."
##########
website/docs/file_sizing.md:
##########
@@ -3,51 +3,76 @@ title: "File Sizing"
toc: true
---
-This doc will show you how Apache Hudi overcomes the dreaded small files
problem. A key design decision in Hudi was to
-avoid creating small files in the first place and always write properly sized
files.
-There are 2 ways to manage small files in Hudi and below will describe the
advantages and trade-offs of each.
-
-## Auto-Size During ingestion
-
-You can automatically manage size of files during ingestion. This solution
adds a little latency during ingestion, but
-it ensures that read queries are always efficient as soon as a write is
committed. If you don't
-manage file sizing as you write and instead try to periodically run a
file-sizing clean-up, your queries will be slow until that resize cleanup is
periodically performed.
-
-(Note: [bulk_insert](/docs/next/write_operations) write operation does not
provide auto-sizing during ingestion)
-
-### For Copy-On-Write
-This is as simple as configuring the [maximum size for a base/parquet
file](/docs/configurations#hoodieparquetmaxfilesize)
-and the [soft limit](/docs/configurations#hoodieparquetsmallfilelimit) below
which a file should
-be considered a small file. For the initial bootstrap of a Hudi table, tuning
record size estimate is also important to
-ensure sufficient records are bin-packed in a parquet file. For subsequent
writes, Hudi automatically uses average
-record size based on previous commit. Hudi will try to add enough records to a
small file at write time to get it to the
-configured maximum limit. For e.g , with `compactionSmallFileSize=100MB` and
limitFileSize=120MB, Hudi will pick all
-files < 100MB and try to get them upto 120MB.
-
-### For Merge-On-Read
-MergeOnRead works differently for different INDEX choices so there are few
more configs to set:
-
-- Indexes with **canIndexLogFiles = true** : Inserts of new data go directly
to log files. In this case, you can
-configure the [maximum log size](/docs/configurations#hoodielogfilemaxsize)
and a
-[factor](/docs/configurations#hoodielogfiletoparquetcompressionratio) that
denotes reduction in
-size when data moves from avro to parquet files.
-- Indexes with **canIndexLogFiles = false** : Inserts of new data go only to
parquet files. In this case, the
-same configurations as above for the COPY_ON_WRITE case applies.
-
-NOTE : In either case, small files will be auto sized only if there is no
PENDING compaction or associated log file for
-that particular file slice. For example, for case 1: If you had a log file and
a compaction C1 was scheduled to convert
-that log file to parquet, no more inserts can go into that log file. For case
2: If you had a parquet file and an update
-ended up creating an associated delta log file, no more inserts can go into
that parquet file. Only after the compaction
-has been performed and there are NO log files associated with the base parquet
file, can new inserts be sent to auto size that parquet file.
-
-## Auto-Size With Clustering
-**[Clustering](/docs/next/clustering)** is a feature in Hudi to group
-small files into larger ones either synchronously or asynchronously. Since
first solution of auto-sizing small files has
-a tradeoff on ingestion speed (since the small files are sized during
ingestion), if your use-case is very sensitive to
-ingestion latency where you don't want to compromise on ingestion speed which
may end up creating a lot of small files,
-clustering comes to the rescue. Clustering can be scheduled through the
ingestion job and an asynchronus job can stitch
-small files together in the background to generate larger files. NOTE that
during this, ingestion can continue to run concurrently.
-
-*Please note that Hudi always creates immutable files on disk. To be able to
do auto-sizing or clustering, Hudi will
-always create a newer version of the smaller file, resulting in 2 versions of
the same file.
-The [cleaner service](/docs/next/hoodie_cleaner) will later kick in and delete
the older version small file and keep the latest one.*
\ No newline at end of file
+A fundamental problem when writing data to a source is having a lot of small
files. This is also known as a small file problem. If you don’t size the files
appropriately, you can slow down the query performance and work with stale
analytics. Some of the issues you may encounter with small files include the
following:
+- **Reads slow down**: You’ll have to scan through many small files to
retrieve data for a query. It’s a very inefficient way of accessing and
utilizing the data.
+
+- **Processes slow down**: You can slow down your i.e., Spark or Hive jobs;
the more files you have, the more tasks you create for reading each file.
+
+- **Storage increases**: When working with a lot of data, you can be
inefficient in using your storage. For example, many small files can have a
lower compression ratio, leading to more data on storage. If you’re indexing
the data, that also takes up more storage space inside the Parquet files. If
you’re working with a small amount of data, you might not see a significant
impact with storage. However, when dealing with petabyte and exabyte data,
you’ll need to be efficient in managing storage resources.
+
+All these challenges inevitably lead to stale analytics and scalability
challenges:
+- Query performance slows down.
+- Jobs could be running faster.
+- You’re utilizing way more resources.
+
+A critical design decision in the Hudi architecture is to avoid small file
creation. Hudi is uniquely designed to write appropriately sized files
automatically. This document will show you how Apache Hudi overcomes the
dreaded small files problem. There are two ways to manage small files in Hudi:
+
+- Auto-size during ingestion
+- Clustering
+
+Below, we will describe the advantages and trade-offs of each.
+
+## Auto-sizing during ingestion
+
+You can manage file sizes through Hudi’s auto-sizing capability during
ingestion. The default targeted file size for Parquet base files is 120MB,
which can be configured by hoodie.parquet.max.file.size. Auto-sizing may add
some data latency, but it ensures that the read queries are always efficient as
soon as a write transaction is committed. It’s important to note that if you
don’t manage file sizing as you write and, instead, try to run clustering to
fix your file sizing periodically, your queries might be slow until the point
when the clustering finishes.
+
+**Note**: the bulk_insert write operation does not have auto-sizing
capabilities during ingestion
+
+If you need to customize the file sizing, i.e., increase the target file size
or change how small files are identified, follow the instructions below for
Copy-On-Write and Merge-On-Read.
+
+### Copy-On-Write (COW)
+To tune the file sizing for a COW table, you can set the small file limit and
the maximum Parquet file size. Hudi will try to add enough records to a small
file at write time to get it to the configured maximum limit.
+
+ - For example, if the `hoodie.parquet.small.file.limit=104857600` (100MB) and
`hoodie.parquet.max.file.size=125829120` (120MB), Hudi will pick all files <
100MB and try to get them up to 120MB.
+
+For creating a Hudi table initially, setting an accurate record size estimate
is vital to ensure Hudi can adequately estimate how many records need to be
bin-packed in a Parquet file for the first ingestion batch. Then, Hudi
automatically uses the average record size for subsequent writes based on
previous commits.
+
+Here are the important configuration of interest:
+
+## Merge-On-Read (MOR)
+As a MOR table aims to reduce the write amplification, compared to a COW
table, when writing to a MOR table, Hudi limits the number of Parquet base
files to one for auto file sizing during insert and upsert operation. This
limits the number of rewritten files. This can be configured through
`hoodie.merge.small.file.group.candidates.limit`.
+
+In addition to file sizing Parquet base files for a MOR table, you can also
tune the log files file-sizing with `hoodie.logfile.max.size`.
+
+**NOTE**: Small files in file groups included in the requested or inflight
compaction or clustering under the active timeline, or small files with
associated log files are not auto-sized with incoming inserts until the
compaction or clustering is complete. For example:
+
+- In case 1: If you had a log file and a compaction, C1, was scheduled to
convert that log file to Parquet, no more inserts can go into the same file
slice.
+
+- In case 2: If the Hudi table has a file group with a Parquet base file and
an associated log file from updates, or this file group is under a requested or
inflight compaction, no more inserts can go into this file group to
automatically size the Parquet file. Only after the compaction has been
performed, and there are NO log files associated with the base Parquet file,
can new inserts be sent to auto-size that parquet file.
+
+Here are the essential configurations:
+
+| Parameter Name | Default | Description | Scope | Since Version
|
+|----------------|--------|----------|---------------|--------------------------------------|
+| hoodie.parquet.small.file.limit | 104857600 (100MB) | During an insert and
upsert operation, we opportunistically expand existing small files on storage
instead of writing new files to keep the number of files optimum. This config
sets the file size limit below which a storage file becomes a candidate to be
selected as such a `small file`. By default, treat any file <= 100MB as a small
file. Also note that if this is set to <= 0, Hudi will not try to get small
files and directly write new files. | Write COW, MOR | 0.4.0 |
Review Comment:
For scope, make them into two lines:
```
Write
COW, MOR
```
##########
website/docs/file_sizing.md:
##########
@@ -3,51 +3,76 @@ title: "File Sizing"
toc: true
---
-This doc will show you how Apache Hudi overcomes the dreaded small files
problem. A key design decision in Hudi was to
-avoid creating small files in the first place and always write properly sized
files.
-There are 2 ways to manage small files in Hudi and below will describe the
advantages and trade-offs of each.
-
-## Auto-Size During ingestion
-
-You can automatically manage size of files during ingestion. This solution
adds a little latency during ingestion, but
-it ensures that read queries are always efficient as soon as a write is
committed. If you don't
-manage file sizing as you write and instead try to periodically run a
file-sizing clean-up, your queries will be slow until that resize cleanup is
periodically performed.
-
-(Note: [bulk_insert](/docs/next/write_operations) write operation does not
provide auto-sizing during ingestion)
-
-### For Copy-On-Write
-This is as simple as configuring the [maximum size for a base/parquet
file](/docs/configurations#hoodieparquetmaxfilesize)
-and the [soft limit](/docs/configurations#hoodieparquetsmallfilelimit) below
which a file should
-be considered a small file. For the initial bootstrap of a Hudi table, tuning
record size estimate is also important to
-ensure sufficient records are bin-packed in a parquet file. For subsequent
writes, Hudi automatically uses average
-record size based on previous commit. Hudi will try to add enough records to a
small file at write time to get it to the
-configured maximum limit. For e.g , with `compactionSmallFileSize=100MB` and
limitFileSize=120MB, Hudi will pick all
-files < 100MB and try to get them upto 120MB.
-
-### For Merge-On-Read
-MergeOnRead works differently for different INDEX choices so there are few
more configs to set:
-
-- Indexes with **canIndexLogFiles = true** : Inserts of new data go directly
to log files. In this case, you can
-configure the [maximum log size](/docs/configurations#hoodielogfilemaxsize)
and a
-[factor](/docs/configurations#hoodielogfiletoparquetcompressionratio) that
denotes reduction in
-size when data moves from avro to parquet files.
-- Indexes with **canIndexLogFiles = false** : Inserts of new data go only to
parquet files. In this case, the
-same configurations as above for the COPY_ON_WRITE case applies.
-
-NOTE : In either case, small files will be auto sized only if there is no
PENDING compaction or associated log file for
-that particular file slice. For example, for case 1: If you had a log file and
a compaction C1 was scheduled to convert
-that log file to parquet, no more inserts can go into that log file. For case
2: If you had a parquet file and an update
-ended up creating an associated delta log file, no more inserts can go into
that parquet file. Only after the compaction
-has been performed and there are NO log files associated with the base parquet
file, can new inserts be sent to auto size that parquet file.
-
-## Auto-Size With Clustering
-**[Clustering](/docs/next/clustering)** is a feature in Hudi to group
-small files into larger ones either synchronously or asynchronously. Since
first solution of auto-sizing small files has
-a tradeoff on ingestion speed (since the small files are sized during
ingestion), if your use-case is very sensitive to
-ingestion latency where you don't want to compromise on ingestion speed which
may end up creating a lot of small files,
-clustering comes to the rescue. Clustering can be scheduled through the
ingestion job and an asynchronus job can stitch
-small files together in the background to generate larger files. NOTE that
during this, ingestion can continue to run concurrently.
-
-*Please note that Hudi always creates immutable files on disk. To be able to
do auto-sizing or clustering, Hudi will
-always create a newer version of the smaller file, resulting in 2 versions of
the same file.
-The [cleaner service](/docs/next/hoodie_cleaner) will later kick in and delete
the older version small file and keep the latest one.*
\ No newline at end of file
+A fundamental problem when writing data to a source is having a lot of small
files. This is also known as a small file problem. If you don’t size the files
appropriately, you can slow down the query performance and work with stale
analytics. Some of the issues you may encounter with small files include the
following:
+- **Reads slow down**: You’ll have to scan through many small files to
retrieve data for a query. It’s a very inefficient way of accessing and
utilizing the data.
+
+- **Processes slow down**: You can slow down your i.e., Spark or Hive jobs;
the more files you have, the more tasks you create for reading each file.
+
+- **Storage increases**: When working with a lot of data, you can be
inefficient in using your storage. For example, many small files can have a
lower compression ratio, leading to more data on storage. If you’re indexing
the data, that also takes up more storage space inside the Parquet files. If
you’re working with a small amount of data, you might not see a significant
impact with storage. However, when dealing with petabyte and exabyte data,
you’ll need to be efficient in managing storage resources.
+
+All these challenges inevitably lead to stale analytics and scalability
challenges:
+- Query performance slows down.
+- Jobs could be running faster.
+- You’re utilizing way more resources.
+
+A critical design decision in the Hudi architecture is to avoid small file
creation. Hudi is uniquely designed to write appropriately sized files
automatically. This document will show you how Apache Hudi overcomes the
dreaded small files problem. There are two ways to manage small files in Hudi:
+
+- Auto-size during ingestion
+- Clustering
+
+Below, we will describe the advantages and trade-offs of each.
+
+## Auto-sizing during ingestion
+
+You can manage file sizes through Hudi’s auto-sizing capability during
ingestion. The default targeted file size for Parquet base files is 120MB,
which can be configured by hoodie.parquet.max.file.size. Auto-sizing may add
some data latency, but it ensures that the read queries are always efficient as
soon as a write transaction is committed. It’s important to note that if you
don’t manage file sizing as you write and, instead, try to run clustering to
fix your file sizing periodically, your queries might be slow until the point
when the clustering finishes.
+
+**Note**: the bulk_insert write operation does not have auto-sizing
capabilities during ingestion
+
+If you need to customize the file sizing, i.e., increase the target file size
or change how small files are identified, follow the instructions below for
Copy-On-Write and Merge-On-Read.
+
+### Copy-On-Write (COW)
+To tune the file sizing for a COW table, you can set the small file limit and
the maximum Parquet file size. Hudi will try to add enough records to a small
file at write time to get it to the configured maximum limit.
+
+ - For example, if the `hoodie.parquet.small.file.limit=104857600` (100MB) and
`hoodie.parquet.max.file.size=125829120` (120MB), Hudi will pick all files <
100MB and try to get them up to 120MB.
+
+For creating a Hudi table initially, setting an accurate record size estimate
is vital to ensure Hudi can adequately estimate how many records need to be
bin-packed in a Parquet file for the first ingestion batch. Then, Hudi
automatically uses the average record size for subsequent writes based on
previous commits.
+
+Here are the important configuration of interest:
+
+## Merge-On-Read (MOR)
+As a MOR table aims to reduce the write amplification, compared to a COW
table, when writing to a MOR table, Hudi limits the number of Parquet base
files to one for auto file sizing during insert and upsert operation. This
limits the number of rewritten files. This can be configured through
`hoodie.merge.small.file.group.candidates.limit`.
+
+In addition to file sizing Parquet base files for a MOR table, you can also
tune the log files file-sizing with `hoodie.logfile.max.size`.
+
+**NOTE**: Small files in file groups included in the requested or inflight
compaction or clustering under the active timeline, or small files with
associated log files are not auto-sized with incoming inserts until the
compaction or clustering is complete. For example:
+
+- In case 1: If you had a log file and a compaction, C1, was scheduled to
convert that log file to Parquet, no more inserts can go into the same file
slice.
+
+- In case 2: If the Hudi table has a file group with a Parquet base file and
an associated log file from updates, or this file group is under a requested or
inflight compaction, no more inserts can go into this file group to
automatically size the Parquet file. Only after the compaction has been
performed, and there are NO log files associated with the base Parquet file,
can new inserts be sent to auto-size that parquet file.
+
+Here are the essential configurations:
+
+| Parameter Name | Default | Description | Scope | Since Version
|
+|----------------|--------|----------|---------------|--------------------------------------|
+| hoodie.parquet.small.file.limit | 104857600 (100MB) | During an insert and
upsert operation, we opportunistically expand existing small files on storage
instead of writing new files to keep the number of files optimum. This config
sets the file size limit below which a storage file becomes a candidate to be
selected as such a `small file`. By default, treat any file <= 100MB as a small
file. Also note that if this is set to <= 0, Hudi will not try to get small
files and directly write new files. | Write COW, MOR | 0.4.0 |
+| hoodie.parquet.max.file.size |125829120 (120MB) | This config is the target
size in bytes for parquet files produced by the Hudi write phases. For DFS,
this needs to be aligned with the underlying filesystem block size for optimal
performance. | Write COW, MOR | 0.4.0 |
+| hoodie.logfile.max.size | 1073741824 (1GB) | This is the log file max size
in bytes. This is the maximum size allowed for a log file before it is rolled
over to the next version. | Write MOR | 0.4.0 |
+| hoodie.merge.small.file.group.candidates.limit | 1 | This limits the number
of file groups, whose base file satisfies the small-file limit to be considered
for appending records during an upsert operation. This is only applicable for
MOR tables. | Write MOR | 0.4.0 |
+
+
+## Auto-Sizing With Clustering
+[Clustering](https://hudi.apache.org/docs/next/clustering) is a service that
allows you to combine small files into larger ones while at the same time
(optionally) changing the data layout by sorting or applying space-filling
curves like Z-order or Hilbert curve. We won’t go into all the details about
clustering here, but please refer to the [clustering
section](https://hudi.apache.org/docs/next/clustering) for more details.
+
+Clustering is very handy for file sizing so you can have faster queries. When
you ingest data, you may still have a lot of small files (depending on your
configurations and the data size from ingestion i.e., input batch). In this
case, you will want to cluster all the small files to larger files in one write
operation to improve query performance. Setting configs for this use case is
unnecessary, other than running a job (i.e., Spark job). Optionally, you can
customize the file sizing using the configs down below.
+An example where clustering might be very useful is when a user has a Hudi
table with many small files. Then, instead of waiting for multiple ingestion
batches to gradually auto-size files, a user can use the clustering service to
fix all the file sizes without ingesting any new data.
+
+Please note clustering in Hudi is not a blocking operation, and ingestion can
continue concurrently as long as no files need to be updated while the
clustering service is running. The writes will fail if files need to be updated
while the clustering service runs.
+
+Here are the critical file sizing configurations:
+
+| Parameter Name | Default | Description | Scope | Since Version
|
+|----------------|--------|----------|---------------|--------------------------------------|
+| hoodie.clustering.plan.strategy.small.file.limit | 314572800 (300MB) | Files
smaller than the size in bytes specified here are candidates for clustering. |
Clustering | 0.7.0 |
+| target.file.max.bytes |1073741824 (1GB) | This configures the target file
size in bytes for clustering.| Clustering | 0.7.0 |
+
+*Please note that Hudi always creates immutable files on disk. To be able to
do auto-sizing or clustering, Hudi will always create a newer version of the
smaller file, resulting in 2 versions of the same file. The [cleaner
service](/docs/next/hoodie_cleaner) will later kick in and delete the older
version small file and keep the latest one.*
Review Comment:
move to note box
##########
website/docs/timeline.md:
##########
@@ -3,40 +3,384 @@ title: Timeline
toc: true
---
-## Timeline
-At its core, Hudi maintains a `timeline` of all actions performed on the table
at different `instants` of time that helps provide instantaneous views of the
table,
-while also efficiently supporting retrieval of data in the order of arrival. A
Hudi instant consists of the following components
+A Hudi table maintains all operations happened to the table in a single
timeline comprised of two parts, an active timeline and an archived timeline.
The active timeline stores all the recent instant, while the archive timeline
stores the older instants. An instant is a transaction where all respective
partitions within a base path have been successfully updated by either a writer
or a table service. Instants that get older in the active timeline are moved to
archived timeline at various times.
-* `Instant action` : Type of action performed on the table
-* `Instant time` : Instant time is typically a timestamp (e.g:
20190117010349), which monotonically increases in the order of action's begin
time.
-* `state` : current state of the instant
+An instant can alter one or many partitions:
-Hudi guarantees that the actions performed on the timeline are atomic &
timeline consistent based on the instant time.
+- If you have one batch ingestion, you’ll see that as one commit in the
active timeline. When you open that commit file, you’ll see a JSON object with
metadata about how one or more partitions were altered.
+
+- If you’re ingesting streaming data, you might see multiple commits in the
active timeline. In this case, when you open a commit file, you might see
metadata about how one or more partition files were altered.
Review Comment:
Could we combine this two? The second sentence is almost the same.
"""
If you have one batch ingestion, you’ll see that as one commit in the active
timeline. If you ingest streaming data, you might see multiple commits in the
active timeline. When you open one commit file, you’ll see a JSON object with
metadata about how one or more partitions were altered.
"""
##########
website/docs/timeline.md:
##########
@@ -3,40 +3,384 @@ title: Timeline
toc: true
---
-## Timeline
-At its core, Hudi maintains a `timeline` of all actions performed on the table
at different `instants` of time that helps provide instantaneous views of the
table,
-while also efficiently supporting retrieval of data in the order of arrival. A
Hudi instant consists of the following components
+A Hudi table maintains all operations happened to the table in a single
timeline comprised of two parts, an active timeline and an archived timeline.
The active timeline stores all the recent instant, while the archive timeline
stores the older instants. An instant is a transaction where all respective
partitions within a base path have been successfully updated by either a writer
or a table service. Instants that get older in the active timeline are moved to
archived timeline at various times.
-* `Instant action` : Type of action performed on the table
-* `Instant time` : Instant time is typically a timestamp (e.g:
20190117010349), which monotonically increases in the order of action's begin
time.
-* `state` : current state of the instant
+An instant can alter one or many partitions:
-Hudi guarantees that the actions performed on the timeline are atomic &
timeline consistent based on the instant time.
+- If you have one batch ingestion, you’ll see that as one commit in the
active timeline. When you open that commit file, you’ll see a JSON object with
metadata about how one or more partitions were altered.
+
+- If you’re ingesting streaming data, you might see multiple commits in the
active timeline. In this case, when you open a commit file, you might see
metadata about how one or more partition files were altered.
-Key actions performed include
+We’ll go over some details and concepts about the active and archived timeline
below. All files in the timelines are immutable.
-* `COMMITS` - A commit denotes an **atomic write** of a batch of records into
a table.
-* `CLEANS` - Background activity that gets rid of older versions of files in
the table, that are no longer needed.
-* `DELTA_COMMIT` - A delta commit refers to an **atomic write** of a batch of
records into a MergeOnRead type table, where some/all of the data could be
just written to delta logs.
-* `COMPACTION` - Background activity to reconcile differential data structures
within Hudi e.g: moving updates from row based log files to columnar formats.
Internally, compaction manifests as a special commit on the timeline
-* `ROLLBACK` - Indicates that a commit/delta commit was unsuccessful & rolled
back, removing any partial files produced during such a write
-* `SAVEPOINT` - Marks certain file groups as "saved", such that cleaner will
not delete them. It helps restore the table to a point on the timeline, in case
of disaster/data recovery scenarios.
+**Note**: The user should never directly alter the timeline (i.e. manually
delete the commits).
-Any given instant can be
-in one of the following states
+## Active Timeline
-* `REQUESTED` - Denotes an action has been scheduled, but has not initiated
-* `INFLIGHT` - Denotes that the action is currently being performed
-* `COMPLETED` - Denotes completion of an action on the timeline
+The active timeline is a source of truth for all write operations: when an
action (described below) happens on a table, the timeline is responsible for
recording it. This guarantees a good table state, and Hudi can provide
read/write isolation based on the timeline. For example, when data is being
written to a Hudi table (i.e., requested, inflight), any data being written as
part of the transaction is not visible to a query engine until the write
transaction is completed. The query engine can still read older data, but the
data inflight won’t be exposed.
-<figure>
- <img className="docimage"
src={require("/assets/images/hudi_timeline.png").default}
alt="hudi_timeline.png" />
-</figure>
+The active timeline is under the `.hoodie` metadata folder. For example, when
you navigate to your Hudi project directory:
-Example above shows upserts happenings between 10:00 and 10:20 on a Hudi
table, roughly every 5 mins, leaving commit metadata on the Hudi timeline, along
-with other background cleaning/compactions. One key observation to make is
that the commit time indicates the `arrival time` of the data (10:20AM), while
the actual data
-organization reflects the actual time or `event time`, the data was intended
for (hourly buckets from 07:00). These are two key concepts when reasoning
about tradeoffs between latency and completeness of data.
+```sh
+cd $YOUR_HUDI_PROJECT_DIRECTORY && ls -a
+```
+
+You’ll see the `.hoodie` metadata folder:
+
+```sh
+ls -a
+. .. .hoodie americas asia
+```
+
+When you navigate inside the `.hoodie` folder, you’ll see a lot of files with
different suffixes and the archived timeline folder:
+
+```sh
+cd .hoodie && ls
+2023021018095339.commit
+20230210180953939.commit.requested
+20230210180953939.inflight
+archived
+```
+
+Before we go into what’s in the files or how the files are named, we’ll need
to cover some broader concepts:
+- actions
+- states
+- instants
Review Comment:
Add links to sections/subsections?
##########
website/docs/timeline.md:
##########
@@ -3,40 +3,384 @@ title: Timeline
toc: true
---
-## Timeline
-At its core, Hudi maintains a `timeline` of all actions performed on the table
at different `instants` of time that helps provide instantaneous views of the
table,
-while also efficiently supporting retrieval of data in the order of arrival. A
Hudi instant consists of the following components
+A Hudi table maintains all operations happened to the table in a single
timeline comprised of two parts, an active timeline and an archived timeline.
The active timeline stores all the recent instant, while the archive timeline
stores the older instants. An instant is a transaction where all respective
partitions within a base path have been successfully updated by either a writer
or a table service. Instants that get older in the active timeline are moved to
archived timeline at various times.
-* `Instant action` : Type of action performed on the table
-* `Instant time` : Instant time is typically a timestamp (e.g:
20190117010349), which monotonically increases in the order of action's begin
time.
-* `state` : current state of the instant
+An instant can alter one or many partitions:
-Hudi guarantees that the actions performed on the timeline are atomic &
timeline consistent based on the instant time.
+- If you have one batch ingestion, you’ll see that as one commit in the
active timeline. When you open that commit file, you’ll see a JSON object with
metadata about how one or more partitions were altered.
+
+- If you’re ingesting streaming data, you might see multiple commits in the
active timeline. In this case, when you open a commit file, you might see
metadata about how one or more partition files were altered.
-Key actions performed include
+We’ll go over some details and concepts about the active and archived timeline
below. All files in the timelines are immutable.
-* `COMMITS` - A commit denotes an **atomic write** of a batch of records into
a table.
-* `CLEANS` - Background activity that gets rid of older versions of files in
the table, that are no longer needed.
-* `DELTA_COMMIT` - A delta commit refers to an **atomic write** of a batch of
records into a MergeOnRead type table, where some/all of the data could be
just written to delta logs.
-* `COMPACTION` - Background activity to reconcile differential data structures
within Hudi e.g: moving updates from row based log files to columnar formats.
Internally, compaction manifests as a special commit on the timeline
-* `ROLLBACK` - Indicates that a commit/delta commit was unsuccessful & rolled
back, removing any partial files produced during such a write
-* `SAVEPOINT` - Marks certain file groups as "saved", such that cleaner will
not delete them. It helps restore the table to a point on the timeline, in case
of disaster/data recovery scenarios.
+**Note**: The user should never directly alter the timeline (i.e. manually
delete the commits).
-Any given instant can be
-in one of the following states
+## Active Timeline
-* `REQUESTED` - Denotes an action has been scheduled, but has not initiated
-* `INFLIGHT` - Denotes that the action is currently being performed
-* `COMPLETED` - Denotes completion of an action on the timeline
+The active timeline is a source of truth for all write operations: when an
action (described below) happens on a table, the timeline is responsible for
recording it. This guarantees a good table state, and Hudi can provide
read/write isolation based on the timeline. For example, when data is being
written to a Hudi table (i.e., requested, inflight), any data being written as
part of the transaction is not visible to a query engine until the write
transaction is completed. The query engine can still read older data, but the
data inflight won’t be exposed.
-<figure>
- <img className="docimage"
src={require("/assets/images/hudi_timeline.png").default}
alt="hudi_timeline.png" />
-</figure>
+The active timeline is under the `.hoodie` metadata folder. For example, when
you navigate to your Hudi project directory:
-Example above shows upserts happenings between 10:00 and 10:20 on a Hudi
table, roughly every 5 mins, leaving commit metadata on the Hudi timeline, along
-with other background cleaning/compactions. One key observation to make is
that the commit time indicates the `arrival time` of the data (10:20AM), while
the actual data
-organization reflects the actual time or `event time`, the data was intended
for (hourly buckets from 07:00). These are two key concepts when reasoning
about tradeoffs between latency and completeness of data.
+```sh
+cd $YOUR_HUDI_PROJECT_DIRECTORY && ls -a
+```
+
+You’ll see the `.hoodie` metadata folder:
+
+```sh
+ls -a
+. .. .hoodie americas asia
+```
+
+When you navigate inside the `.hoodie` folder, you’ll see a lot of files with
different suffixes and the archived timeline folder:
+
+```sh
+cd .hoodie && ls
+2023021018095339.commit
+20230210180953939.commit.requested
+20230210180953939.inflight
+archived
+```
+
+Before we go into what’s in the files or how the files are named, we’ll need
to cover some broader concepts:
+- actions
+- states
+- instants
+
+## Actions
+An action describes what and how transactions were changed. Hudi guarantees
that the actions performed on the timeline are atomic & consistent based on the
instant time.
Review Comment:
"&" -> "and"
##########
website/docs/flink_configuration.md:
##########
@@ -3,115 +3,179 @@ title: Flink Setup
toc: true
---
-## Global Configurations
-When using Flink, you can set some global configurations in
`$FLINK_HOME/conf/flink-conf.yaml`
+[Apache Flink](https://flink.apache.org/what-is-flink/flink-architecture/) is
a powerful streaming-batch integrated engine that provides a stream processing
framework. Flink can process events at an incredible speed with low latency.
Along with Hudi, users can use streaming ingestion like with Kafka; streaming
consumption like with Kafka; and also perform batch workloads like bulk ingest,
snapshot queries and incremental queries.
-### Parallelism
-
-| Option Name | Default | Type | Description |
-| ----------- | ------- | ------- | ------- |
-| `taskmanager.numberOfTaskSlots` | `1` | `Integer` | The number of parallel
operator or user function instances that a single TaskManager can run. We
recommend setting this value > 4, and the actual value needs to be set
according to the amount of data |
-| `parallelism.default` | `1` | `Integer` | The default parallelism used when
no parallelism is specified anywhere (default: 1). For example, If the value of
[`write.bucket_assign.tasks`](#parallelism-1) is not set, this value will be
used |
+There are three executions modes a user can configure for Flink:
+- Streaming
Review Comment:
+1
##########
website/docs/file_sizing.md:
##########
@@ -3,51 +3,76 @@ title: "File Sizing"
toc: true
---
-This doc will show you how Apache Hudi overcomes the dreaded small files
problem. A key design decision in Hudi was to
-avoid creating small files in the first place and always write properly sized
files.
-There are 2 ways to manage small files in Hudi and below will describe the
advantages and trade-offs of each.
-
-## Auto-Size During ingestion
-
-You can automatically manage size of files during ingestion. This solution
adds a little latency during ingestion, but
-it ensures that read queries are always efficient as soon as a write is
committed. If you don't
-manage file sizing as you write and instead try to periodically run a
file-sizing clean-up, your queries will be slow until that resize cleanup is
periodically performed.
-
-(Note: [bulk_insert](/docs/next/write_operations) write operation does not
provide auto-sizing during ingestion)
-
-### For Copy-On-Write
-This is as simple as configuring the [maximum size for a base/parquet
file](/docs/configurations#hoodieparquetmaxfilesize)
-and the [soft limit](/docs/configurations#hoodieparquetsmallfilelimit) below
which a file should
-be considered a small file. For the initial bootstrap of a Hudi table, tuning
record size estimate is also important to
-ensure sufficient records are bin-packed in a parquet file. For subsequent
writes, Hudi automatically uses average
-record size based on previous commit. Hudi will try to add enough records to a
small file at write time to get it to the
-configured maximum limit. For e.g , with `compactionSmallFileSize=100MB` and
limitFileSize=120MB, Hudi will pick all
-files < 100MB and try to get them upto 120MB.
-
-### For Merge-On-Read
-MergeOnRead works differently for different INDEX choices so there are few
more configs to set:
-
-- Indexes with **canIndexLogFiles = true** : Inserts of new data go directly
to log files. In this case, you can
-configure the [maximum log size](/docs/configurations#hoodielogfilemaxsize)
and a
-[factor](/docs/configurations#hoodielogfiletoparquetcompressionratio) that
denotes reduction in
-size when data moves from avro to parquet files.
-- Indexes with **canIndexLogFiles = false** : Inserts of new data go only to
parquet files. In this case, the
-same configurations as above for the COPY_ON_WRITE case applies.
-
-NOTE : In either case, small files will be auto sized only if there is no
PENDING compaction or associated log file for
-that particular file slice. For example, for case 1: If you had a log file and
a compaction C1 was scheduled to convert
-that log file to parquet, no more inserts can go into that log file. For case
2: If you had a parquet file and an update
-ended up creating an associated delta log file, no more inserts can go into
that parquet file. Only after the compaction
-has been performed and there are NO log files associated with the base parquet
file, can new inserts be sent to auto size that parquet file.
-
-## Auto-Size With Clustering
-**[Clustering](/docs/next/clustering)** is a feature in Hudi to group
-small files into larger ones either synchronously or asynchronously. Since
first solution of auto-sizing small files has
-a tradeoff on ingestion speed (since the small files are sized during
ingestion), if your use-case is very sensitive to
-ingestion latency where you don't want to compromise on ingestion speed which
may end up creating a lot of small files,
-clustering comes to the rescue. Clustering can be scheduled through the
ingestion job and an asynchronus job can stitch
-small files together in the background to generate larger files. NOTE that
during this, ingestion can continue to run concurrently.
-
-*Please note that Hudi always creates immutable files on disk. To be able to
do auto-sizing or clustering, Hudi will
-always create a newer version of the smaller file, resulting in 2 versions of
the same file.
-The [cleaner service](/docs/next/hoodie_cleaner) will later kick in and delete
the older version small file and keep the latest one.*
\ No newline at end of file
+A fundamental problem when writing data to a source is having a lot of small
files. This is also known as a small file problem. If you don’t size the files
appropriately, you can slow down the query performance and work with stale
analytics. Some of the issues you may encounter with small files include the
following:
+- **Reads slow down**: You’ll have to scan through many small files to
retrieve data for a query. It’s a very inefficient way of accessing and
utilizing the data.
+
+- **Processes slow down**: You can slow down your i.e., Spark or Hive jobs;
the more files you have, the more tasks you create for reading each file.
+
+- **Storage increases**: When working with a lot of data, you can be
inefficient in using your storage. For example, many small files can have a
lower compression ratio, leading to more data on storage. If you’re indexing
the data, that also takes up more storage space inside the Parquet files. If
you’re working with a small amount of data, you might not see a significant
impact with storage. However, when dealing with petabyte and exabyte data,
you’ll need to be efficient in managing storage resources.
+
+All these challenges inevitably lead to stale analytics and scalability
challenges:
+- Query performance slows down.
+- Jobs could be running faster.
+- You’re utilizing way more resources.
+
+A critical design decision in the Hudi architecture is to avoid small file
creation. Hudi is uniquely designed to write appropriately sized files
automatically. This document will show you how Apache Hudi overcomes the
dreaded small files problem. There are two ways to manage small files in Hudi:
+
+- Auto-size during ingestion
+- Clustering
+
+Below, we will describe the advantages and trade-offs of each.
+
+## Auto-sizing during ingestion
+
+You can manage file sizes through Hudi’s auto-sizing capability during
ingestion. The default targeted file size for Parquet base files is 120MB,
which can be configured by hoodie.parquet.max.file.size. Auto-sizing may add
some data latency, but it ensures that the read queries are always efficient as
soon as a write transaction is committed. It’s important to note that if you
don’t manage file sizing as you write and, instead, try to run clustering to
fix your file sizing periodically, your queries might be slow until the point
when the clustering finishes.
+
+**Note**: the bulk_insert write operation does not have auto-sizing
capabilities during ingestion
Review Comment:
use the following box for note:
```
:::note
The bulk_insert write operation does not have auto-sizing capabilities
during ingestion.
:::
```
<img width="994" alt="Screenshot 2023-03-07 at 10 52 09"
src="https://user-images.githubusercontent.com/2497195/223522409-83207266-9c0d-4aca-b24e-4821592a78e2.png">
##########
website/docs/file_sizing.md:
##########
@@ -3,51 +3,76 @@ title: "File Sizing"
toc: true
---
-This doc will show you how Apache Hudi overcomes the dreaded small files
problem. A key design decision in Hudi was to
-avoid creating small files in the first place and always write properly sized
files.
-There are 2 ways to manage small files in Hudi and below will describe the
advantages and trade-offs of each.
-
-## Auto-Size During ingestion
-
-You can automatically manage size of files during ingestion. This solution
adds a little latency during ingestion, but
-it ensures that read queries are always efficient as soon as a write is
committed. If you don't
-manage file sizing as you write and instead try to periodically run a
file-sizing clean-up, your queries will be slow until that resize cleanup is
periodically performed.
-
-(Note: [bulk_insert](/docs/next/write_operations) write operation does not
provide auto-sizing during ingestion)
-
-### For Copy-On-Write
-This is as simple as configuring the [maximum size for a base/parquet
file](/docs/configurations#hoodieparquetmaxfilesize)
-and the [soft limit](/docs/configurations#hoodieparquetsmallfilelimit) below
which a file should
-be considered a small file. For the initial bootstrap of a Hudi table, tuning
record size estimate is also important to
-ensure sufficient records are bin-packed in a parquet file. For subsequent
writes, Hudi automatically uses average
-record size based on previous commit. Hudi will try to add enough records to a
small file at write time to get it to the
-configured maximum limit. For e.g , with `compactionSmallFileSize=100MB` and
limitFileSize=120MB, Hudi will pick all
-files < 100MB and try to get them upto 120MB.
-
-### For Merge-On-Read
-MergeOnRead works differently for different INDEX choices so there are few
more configs to set:
-
-- Indexes with **canIndexLogFiles = true** : Inserts of new data go directly
to log files. In this case, you can
-configure the [maximum log size](/docs/configurations#hoodielogfilemaxsize)
and a
-[factor](/docs/configurations#hoodielogfiletoparquetcompressionratio) that
denotes reduction in
-size when data moves from avro to parquet files.
-- Indexes with **canIndexLogFiles = false** : Inserts of new data go only to
parquet files. In this case, the
-same configurations as above for the COPY_ON_WRITE case applies.
-
-NOTE : In either case, small files will be auto sized only if there is no
PENDING compaction or associated log file for
-that particular file slice. For example, for case 1: If you had a log file and
a compaction C1 was scheduled to convert
-that log file to parquet, no more inserts can go into that log file. For case
2: If you had a parquet file and an update
-ended up creating an associated delta log file, no more inserts can go into
that parquet file. Only after the compaction
-has been performed and there are NO log files associated with the base parquet
file, can new inserts be sent to auto size that parquet file.
-
-## Auto-Size With Clustering
-**[Clustering](/docs/next/clustering)** is a feature in Hudi to group
-small files into larger ones either synchronously or asynchronously. Since
first solution of auto-sizing small files has
-a tradeoff on ingestion speed (since the small files are sized during
ingestion), if your use-case is very sensitive to
-ingestion latency where you don't want to compromise on ingestion speed which
may end up creating a lot of small files,
-clustering comes to the rescue. Clustering can be scheduled through the
ingestion job and an asynchronus job can stitch
-small files together in the background to generate larger files. NOTE that
during this, ingestion can continue to run concurrently.
-
-*Please note that Hudi always creates immutable files on disk. To be able to
do auto-sizing or clustering, Hudi will
-always create a newer version of the smaller file, resulting in 2 versions of
the same file.
-The [cleaner service](/docs/next/hoodie_cleaner) will later kick in and delete
the older version small file and keep the latest one.*
\ No newline at end of file
+A fundamental problem when writing data to a source is having a lot of small
files. This is also known as a small file problem. If you don’t size the files
appropriately, you can slow down the query performance and work with stale
analytics. Some of the issues you may encounter with small files include the
following:
+- **Reads slow down**: You’ll have to scan through many small files to
retrieve data for a query. It’s a very inefficient way of accessing and
utilizing the data.
+
+- **Processes slow down**: You can slow down your i.e., Spark or Hive jobs;
the more files you have, the more tasks you create for reading each file.
+
+- **Storage increases**: When working with a lot of data, you can be
inefficient in using your storage. For example, many small files can have a
lower compression ratio, leading to more data on storage. If you’re indexing
the data, that also takes up more storage space inside the Parquet files. If
you’re working with a small amount of data, you might not see a significant
impact with storage. However, when dealing with petabyte and exabyte data,
you’ll need to be efficient in managing storage resources.
+
+All these challenges inevitably lead to stale analytics and scalability
challenges:
+- Query performance slows down.
+- Jobs could be running faster.
+- You’re utilizing way more resources.
+
+A critical design decision in the Hudi architecture is to avoid small file
creation. Hudi is uniquely designed to write appropriately sized files
automatically. This document will show you how Apache Hudi overcomes the
dreaded small files problem. There are two ways to manage small files in Hudi:
+
+- Auto-size during ingestion
+- Clustering
+
+Below, we will describe the advantages and trade-offs of each.
+
+## Auto-sizing during ingestion
+
+You can manage file sizes through Hudi’s auto-sizing capability during
ingestion. The default targeted file size for Parquet base files is 120MB,
which can be configured by hoodie.parquet.max.file.size. Auto-sizing may add
some data latency, but it ensures that the read queries are always efficient as
soon as a write transaction is committed. It’s important to note that if you
don’t manage file sizing as you write and, instead, try to run clustering to
fix your file sizing periodically, your queries might be slow until the point
when the clustering finishes.
Review Comment:
code block for `hoodie.parquet.max.file.size` by adding back ticks "`"
##########
website/docs/file_sizing.md:
##########
@@ -3,51 +3,76 @@ title: "File Sizing"
toc: true
---
-This doc will show you how Apache Hudi overcomes the dreaded small files
problem. A key design decision in Hudi was to
-avoid creating small files in the first place and always write properly sized
files.
-There are 2 ways to manage small files in Hudi and below will describe the
advantages and trade-offs of each.
-
-## Auto-Size During ingestion
-
-You can automatically manage size of files during ingestion. This solution
adds a little latency during ingestion, but
-it ensures that read queries are always efficient as soon as a write is
committed. If you don't
-manage file sizing as you write and instead try to periodically run a
file-sizing clean-up, your queries will be slow until that resize cleanup is
periodically performed.
-
-(Note: [bulk_insert](/docs/next/write_operations) write operation does not
provide auto-sizing during ingestion)
-
-### For Copy-On-Write
-This is as simple as configuring the [maximum size for a base/parquet
file](/docs/configurations#hoodieparquetmaxfilesize)
-and the [soft limit](/docs/configurations#hoodieparquetsmallfilelimit) below
which a file should
-be considered a small file. For the initial bootstrap of a Hudi table, tuning
record size estimate is also important to
-ensure sufficient records are bin-packed in a parquet file. For subsequent
writes, Hudi automatically uses average
-record size based on previous commit. Hudi will try to add enough records to a
small file at write time to get it to the
-configured maximum limit. For e.g , with `compactionSmallFileSize=100MB` and
limitFileSize=120MB, Hudi will pick all
-files < 100MB and try to get them upto 120MB.
-
-### For Merge-On-Read
-MergeOnRead works differently for different INDEX choices so there are few
more configs to set:
-
-- Indexes with **canIndexLogFiles = true** : Inserts of new data go directly
to log files. In this case, you can
-configure the [maximum log size](/docs/configurations#hoodielogfilemaxsize)
and a
-[factor](/docs/configurations#hoodielogfiletoparquetcompressionratio) that
denotes reduction in
-size when data moves from avro to parquet files.
-- Indexes with **canIndexLogFiles = false** : Inserts of new data go only to
parquet files. In this case, the
-same configurations as above for the COPY_ON_WRITE case applies.
-
-NOTE : In either case, small files will be auto sized only if there is no
PENDING compaction or associated log file for
-that particular file slice. For example, for case 1: If you had a log file and
a compaction C1 was scheduled to convert
-that log file to parquet, no more inserts can go into that log file. For case
2: If you had a parquet file and an update
-ended up creating an associated delta log file, no more inserts can go into
that parquet file. Only after the compaction
-has been performed and there are NO log files associated with the base parquet
file, can new inserts be sent to auto size that parquet file.
-
-## Auto-Size With Clustering
-**[Clustering](/docs/next/clustering)** is a feature in Hudi to group
-small files into larger ones either synchronously or asynchronously. Since
first solution of auto-sizing small files has
-a tradeoff on ingestion speed (since the small files are sized during
ingestion), if your use-case is very sensitive to
-ingestion latency where you don't want to compromise on ingestion speed which
may end up creating a lot of small files,
-clustering comes to the rescue. Clustering can be scheduled through the
ingestion job and an asynchronus job can stitch
-small files together in the background to generate larger files. NOTE that
during this, ingestion can continue to run concurrently.
-
-*Please note that Hudi always creates immutable files on disk. To be able to
do auto-sizing or clustering, Hudi will
-always create a newer version of the smaller file, resulting in 2 versions of
the same file.
-The [cleaner service](/docs/next/hoodie_cleaner) will later kick in and delete
the older version small file and keep the latest one.*
\ No newline at end of file
+A fundamental problem when writing data to a source is having a lot of small
files. This is also known as a small file problem. If you don’t size the files
appropriately, you can slow down the query performance and work with stale
analytics. Some of the issues you may encounter with small files include the
following:
+- **Reads slow down**: You’ll have to scan through many small files to
retrieve data for a query. It’s a very inefficient way of accessing and
utilizing the data.
+
+- **Processes slow down**: You can slow down your i.e., Spark or Hive jobs;
the more files you have, the more tasks you create for reading each file.
+
+- **Storage increases**: When working with a lot of data, you can be
inefficient in using your storage. For example, many small files can have a
lower compression ratio, leading to more data on storage. If you’re indexing
the data, that also takes up more storage space inside the Parquet files. If
you’re working with a small amount of data, you might not see a significant
impact with storage. However, when dealing with petabyte and exabyte data,
you’ll need to be efficient in managing storage resources.
+
+All these challenges inevitably lead to stale analytics and scalability
challenges:
+- Query performance slows down.
+- Jobs could be running faster.
+- You’re utilizing way more resources.
+
+A critical design decision in the Hudi architecture is to avoid small file
creation. Hudi is uniquely designed to write appropriately sized files
automatically. This document will show you how Apache Hudi overcomes the
dreaded small files problem. There are two ways to manage small files in Hudi:
+
+- Auto-size during ingestion
+- Clustering
+
+Below, we will describe the advantages and trade-offs of each.
+
+## Auto-sizing during ingestion
+
+You can manage file sizes through Hudi’s auto-sizing capability during
ingestion. The default targeted file size for Parquet base files is 120MB,
which can be configured by hoodie.parquet.max.file.size. Auto-sizing may add
some data latency, but it ensures that the read queries are always efficient as
soon as a write transaction is committed. It’s important to note that if you
don’t manage file sizing as you write and, instead, try to run clustering to
fix your file sizing periodically, your queries might be slow until the point
when the clustering finishes.
+
+**Note**: the bulk_insert write operation does not have auto-sizing
capabilities during ingestion
+
+If you need to customize the file sizing, i.e., increase the target file size
or change how small files are identified, follow the instructions below for
Copy-On-Write and Merge-On-Read.
+
+### Copy-On-Write (COW)
+To tune the file sizing for a COW table, you can set the small file limit and
the maximum Parquet file size. Hudi will try to add enough records to a small
file at write time to get it to the configured maximum limit.
+
+ - For example, if the `hoodie.parquet.small.file.limit=104857600` (100MB) and
`hoodie.parquet.max.file.size=125829120` (120MB), Hudi will pick all files <
100MB and try to get them up to 120MB.
+
+For creating a Hudi table initially, setting an accurate record size estimate
is vital to ensure Hudi can adequately estimate how many records need to be
bin-packed in a Parquet file for the first ingestion batch. Then, Hudi
automatically uses the average record size for subsequent writes based on
previous commits.
+
+Here are the important configuration of interest:
+
+## Merge-On-Read (MOR)
+As a MOR table aims to reduce the write amplification, compared to a COW
table, when writing to a MOR table, Hudi limits the number of Parquet base
files to one for auto file sizing during insert and upsert operation. This
limits the number of rewritten files. This can be configured through
`hoodie.merge.small.file.group.candidates.limit`.
+
+In addition to file sizing Parquet base files for a MOR table, you can also
tune the log files file-sizing with `hoodie.logfile.max.size`.
+
+**NOTE**: Small files in file groups included in the requested or inflight
compaction or clustering under the active timeline, or small files with
associated log files are not auto-sized with incoming inserts until the
compaction or clustering is complete. For example:
+
+- In case 1: If you had a log file and a compaction, C1, was scheduled to
convert that log file to Parquet, no more inserts can go into the same file
slice.
+
+- In case 2: If the Hudi table has a file group with a Parquet base file and
an associated log file from updates, or this file group is under a requested or
inflight compaction, no more inserts can go into this file group to
automatically size the Parquet file. Only after the compaction has been
performed, and there are NO log files associated with the base Parquet file,
can new inserts be sent to auto-size that parquet file.
+
+Here are the essential configurations:
+
+| Parameter Name | Default | Description | Scope | Since Version
|
Review Comment:
"Parameter Name" -> "Property Name"
##########
website/docs/file_sizing.md:
##########
@@ -3,51 +3,76 @@ title: "File Sizing"
toc: true
---
-This doc will show you how Apache Hudi overcomes the dreaded small files
problem. A key design decision in Hudi was to
-avoid creating small files in the first place and always write properly sized
files.
-There are 2 ways to manage small files in Hudi and below will describe the
advantages and trade-offs of each.
-
-## Auto-Size During ingestion
-
-You can automatically manage size of files during ingestion. This solution
adds a little latency during ingestion, but
-it ensures that read queries are always efficient as soon as a write is
committed. If you don't
-manage file sizing as you write and instead try to periodically run a
file-sizing clean-up, your queries will be slow until that resize cleanup is
periodically performed.
-
-(Note: [bulk_insert](/docs/next/write_operations) write operation does not
provide auto-sizing during ingestion)
-
-### For Copy-On-Write
-This is as simple as configuring the [maximum size for a base/parquet
file](/docs/configurations#hoodieparquetmaxfilesize)
-and the [soft limit](/docs/configurations#hoodieparquetsmallfilelimit) below
which a file should
-be considered a small file. For the initial bootstrap of a Hudi table, tuning
record size estimate is also important to
-ensure sufficient records are bin-packed in a parquet file. For subsequent
writes, Hudi automatically uses average
-record size based on previous commit. Hudi will try to add enough records to a
small file at write time to get it to the
-configured maximum limit. For e.g , with `compactionSmallFileSize=100MB` and
limitFileSize=120MB, Hudi will pick all
-files < 100MB and try to get them upto 120MB.
-
-### For Merge-On-Read
-MergeOnRead works differently for different INDEX choices so there are few
more configs to set:
-
-- Indexes with **canIndexLogFiles = true** : Inserts of new data go directly
to log files. In this case, you can
-configure the [maximum log size](/docs/configurations#hoodielogfilemaxsize)
and a
-[factor](/docs/configurations#hoodielogfiletoparquetcompressionratio) that
denotes reduction in
-size when data moves from avro to parquet files.
-- Indexes with **canIndexLogFiles = false** : Inserts of new data go only to
parquet files. In this case, the
-same configurations as above for the COPY_ON_WRITE case applies.
-
-NOTE : In either case, small files will be auto sized only if there is no
PENDING compaction or associated log file for
-that particular file slice. For example, for case 1: If you had a log file and
a compaction C1 was scheduled to convert
-that log file to parquet, no more inserts can go into that log file. For case
2: If you had a parquet file and an update
-ended up creating an associated delta log file, no more inserts can go into
that parquet file. Only after the compaction
-has been performed and there are NO log files associated with the base parquet
file, can new inserts be sent to auto size that parquet file.
-
-## Auto-Size With Clustering
-**[Clustering](/docs/next/clustering)** is a feature in Hudi to group
-small files into larger ones either synchronously or asynchronously. Since
first solution of auto-sizing small files has
-a tradeoff on ingestion speed (since the small files are sized during
ingestion), if your use-case is very sensitive to
-ingestion latency where you don't want to compromise on ingestion speed which
may end up creating a lot of small files,
-clustering comes to the rescue. Clustering can be scheduled through the
ingestion job and an asynchronus job can stitch
-small files together in the background to generate larger files. NOTE that
during this, ingestion can continue to run concurrently.
-
-*Please note that Hudi always creates immutable files on disk. To be able to
do auto-sizing or clustering, Hudi will
-always create a newer version of the smaller file, resulting in 2 versions of
the same file.
-The [cleaner service](/docs/next/hoodie_cleaner) will later kick in and delete
the older version small file and keep the latest one.*
\ No newline at end of file
+A fundamental problem when writing data to a source is having a lot of small
files. This is also known as a small file problem. If you don’t size the files
appropriately, you can slow down the query performance and work with stale
analytics. Some of the issues you may encounter with small files include the
following:
+- **Reads slow down**: You’ll have to scan through many small files to
retrieve data for a query. It’s a very inefficient way of accessing and
utilizing the data.
+
+- **Processes slow down**: You can slow down your i.e., Spark or Hive jobs;
the more files you have, the more tasks you create for reading each file.
+
+- **Storage increases**: When working with a lot of data, you can be
inefficient in using your storage. For example, many small files can have a
lower compression ratio, leading to more data on storage. If you’re indexing
the data, that also takes up more storage space inside the Parquet files. If
you’re working with a small amount of data, you might not see a significant
impact with storage. However, when dealing with petabyte and exabyte data,
you’ll need to be efficient in managing storage resources.
+
+All these challenges inevitably lead to stale analytics and scalability
challenges:
+- Query performance slows down.
+- Jobs could be running faster.
+- You’re utilizing way more resources.
+
+A critical design decision in the Hudi architecture is to avoid small file
creation. Hudi is uniquely designed to write appropriately sized files
automatically. This document will show you how Apache Hudi overcomes the
dreaded small files problem. There are two ways to manage small files in Hudi:
+
+- Auto-size during ingestion
+- Clustering
+
+Below, we will describe the advantages and trade-offs of each.
+
+## Auto-sizing during ingestion
+
+You can manage file sizes through Hudi’s auto-sizing capability during
ingestion. The default targeted file size for Parquet base files is 120MB,
which can be configured by hoodie.parquet.max.file.size. Auto-sizing may add
some data latency, but it ensures that the read queries are always efficient as
soon as a write transaction is committed. It’s important to note that if you
don’t manage file sizing as you write and, instead, try to run clustering to
fix your file sizing periodically, your queries might be slow until the point
when the clustering finishes.
+
+**Note**: the bulk_insert write operation does not have auto-sizing
capabilities during ingestion
+
+If you need to customize the file sizing, i.e., increase the target file size
or change how small files are identified, follow the instructions below for
Copy-On-Write and Merge-On-Read.
+
+### Copy-On-Write (COW)
+To tune the file sizing for a COW table, you can set the small file limit and
the maximum Parquet file size. Hudi will try to add enough records to a small
file at write time to get it to the configured maximum limit.
+
+ - For example, if the `hoodie.parquet.small.file.limit=104857600` (100MB) and
`hoodie.parquet.max.file.size=125829120` (120MB), Hudi will pick all files <
100MB and try to get them up to 120MB.
+
+For creating a Hudi table initially, setting an accurate record size estimate
is vital to ensure Hudi can adequately estimate how many records need to be
bin-packed in a Parquet file for the first ingestion batch. Then, Hudi
automatically uses the average record size for subsequent writes based on
previous commits.
+
+Here are the important configuration of interest:
+
+## Merge-On-Read (MOR)
+As a MOR table aims to reduce the write amplification, compared to a COW
table, when writing to a MOR table, Hudi limits the number of Parquet base
files to one for auto file sizing during insert and upsert operation. This
limits the number of rewritten files. This can be configured through
`hoodie.merge.small.file.group.candidates.limit`.
+
+In addition to file sizing Parquet base files for a MOR table, you can also
tune the log files file-sizing with `hoodie.logfile.max.size`.
+
+**NOTE**: Small files in file groups included in the requested or inflight
compaction or clustering under the active timeline, or small files with
associated log files are not auto-sized with incoming inserts until the
compaction or clustering is complete. For example:
+
+- In case 1: If you had a log file and a compaction, C1, was scheduled to
convert that log file to Parquet, no more inserts can go into the same file
slice.
+
+- In case 2: If the Hudi table has a file group with a Parquet base file and
an associated log file from updates, or this file group is under a requested or
inflight compaction, no more inserts can go into this file group to
automatically size the Parquet file. Only after the compaction has been
performed, and there are NO log files associated with the base Parquet file,
can new inserts be sent to auto-size that parquet file.
+
+Here are the essential configurations:
+
+| Parameter Name | Default | Description | Scope | Since Version
|
+|----------------|--------|----------|---------------|--------------------------------------|
+| hoodie.parquet.small.file.limit | 104857600 (100MB) | During an insert and
upsert operation, we opportunistically expand existing small files on storage
instead of writing new files to keep the number of files optimum. This config
sets the file size limit below which a storage file becomes a candidate to be
selected as such a `small file`. By default, treat any file <= 100MB as a small
file. Also note that if this is set to <= 0, Hudi will not try to get small
files and directly write new files. | Write COW, MOR | 0.4.0 |
+| hoodie.parquet.max.file.size |125829120 (120MB) | This config is the target
size in bytes for parquet files produced by the Hudi write phases. For DFS,
this needs to be aligned with the underlying filesystem block size for optimal
performance. | Write COW, MOR | 0.4.0 |
+| hoodie.logfile.max.size | 1073741824 (1GB) | This is the log file max size
in bytes. This is the maximum size allowed for a log file before it is rolled
over to the next version. | Write MOR | 0.4.0 |
+| hoodie.merge.small.file.group.candidates.limit | 1 | This limits the number
of file groups, whose base file satisfies the small-file limit to be considered
for appending records during an upsert operation. This is only applicable for
MOR tables. | Write MOR | 0.4.0 |
+
+
+## Auto-Sizing With Clustering
+[Clustering](https://hudi.apache.org/docs/next/clustering) is a service that
allows you to combine small files into larger ones while at the same time
(optionally) changing the data layout by sorting or applying space-filling
curves like Z-order or Hilbert curve. We won’t go into all the details about
clustering here, but please refer to the [clustering
section](https://hudi.apache.org/docs/next/clustering) for more details.
+
+Clustering is very handy for file sizing so you can have faster queries. When
you ingest data, you may still have a lot of small files (depending on your
configurations and the data size from ingestion i.e., input batch). In this
case, you will want to cluster all the small files to larger files in one write
operation to improve query performance. Setting configs for this use case is
unnecessary, other than running a job (i.e., Spark job). Optionally, you can
customize the file sizing using the configs down below.
+An example where clustering might be very useful is when a user has a Hudi
table with many small files. Then, instead of waiting for multiple ingestion
batches to gradually auto-size files, a user can use the clustering service to
fix all the file sizes without ingesting any new data.
+
+Please note clustering in Hudi is not a blocking operation, and ingestion can
continue concurrently as long as no files need to be updated while the
clustering service is running. The writes will fail if files need to be updated
while the clustering service runs.
Review Comment:
"files need to be updated" -> "there are updates to the data to be clustered"
##########
website/docs/file_sizing.md:
##########
@@ -3,51 +3,76 @@ title: "File Sizing"
toc: true
---
-This doc will show you how Apache Hudi overcomes the dreaded small files
problem. A key design decision in Hudi was to
-avoid creating small files in the first place and always write properly sized
files.
-There are 2 ways to manage small files in Hudi and below will describe the
advantages and trade-offs of each.
-
-## Auto-Size During ingestion
-
-You can automatically manage size of files during ingestion. This solution
adds a little latency during ingestion, but
-it ensures that read queries are always efficient as soon as a write is
committed. If you don't
-manage file sizing as you write and instead try to periodically run a
file-sizing clean-up, your queries will be slow until that resize cleanup is
periodically performed.
-
-(Note: [bulk_insert](/docs/next/write_operations) write operation does not
provide auto-sizing during ingestion)
-
-### For Copy-On-Write
-This is as simple as configuring the [maximum size for a base/parquet
file](/docs/configurations#hoodieparquetmaxfilesize)
-and the [soft limit](/docs/configurations#hoodieparquetsmallfilelimit) below
which a file should
-be considered a small file. For the initial bootstrap of a Hudi table, tuning
record size estimate is also important to
-ensure sufficient records are bin-packed in a parquet file. For subsequent
writes, Hudi automatically uses average
-record size based on previous commit. Hudi will try to add enough records to a
small file at write time to get it to the
-configured maximum limit. For e.g , with `compactionSmallFileSize=100MB` and
limitFileSize=120MB, Hudi will pick all
-files < 100MB and try to get them upto 120MB.
-
-### For Merge-On-Read
-MergeOnRead works differently for different INDEX choices so there are few
more configs to set:
-
-- Indexes with **canIndexLogFiles = true** : Inserts of new data go directly
to log files. In this case, you can
-configure the [maximum log size](/docs/configurations#hoodielogfilemaxsize)
and a
-[factor](/docs/configurations#hoodielogfiletoparquetcompressionratio) that
denotes reduction in
-size when data moves from avro to parquet files.
-- Indexes with **canIndexLogFiles = false** : Inserts of new data go only to
parquet files. In this case, the
-same configurations as above for the COPY_ON_WRITE case applies.
-
-NOTE : In either case, small files will be auto sized only if there is no
PENDING compaction or associated log file for
-that particular file slice. For example, for case 1: If you had a log file and
a compaction C1 was scheduled to convert
-that log file to parquet, no more inserts can go into that log file. For case
2: If you had a parquet file and an update
-ended up creating an associated delta log file, no more inserts can go into
that parquet file. Only after the compaction
-has been performed and there are NO log files associated with the base parquet
file, can new inserts be sent to auto size that parquet file.
-
-## Auto-Size With Clustering
-**[Clustering](/docs/next/clustering)** is a feature in Hudi to group
-small files into larger ones either synchronously or asynchronously. Since
first solution of auto-sizing small files has
-a tradeoff on ingestion speed (since the small files are sized during
ingestion), if your use-case is very sensitive to
-ingestion latency where you don't want to compromise on ingestion speed which
may end up creating a lot of small files,
-clustering comes to the rescue. Clustering can be scheduled through the
ingestion job and an asynchronus job can stitch
-small files together in the background to generate larger files. NOTE that
during this, ingestion can continue to run concurrently.
-
-*Please note that Hudi always creates immutable files on disk. To be able to
do auto-sizing or clustering, Hudi will
-always create a newer version of the smaller file, resulting in 2 versions of
the same file.
-The [cleaner service](/docs/next/hoodie_cleaner) will later kick in and delete
the older version small file and keep the latest one.*
\ No newline at end of file
+A fundamental problem when writing data to a source is having a lot of small
files. This is also known as a small file problem. If you don’t size the files
appropriately, you can slow down the query performance and work with stale
analytics. Some of the issues you may encounter with small files include the
following:
+- **Reads slow down**: You’ll have to scan through many small files to
retrieve data for a query. It’s a very inefficient way of accessing and
utilizing the data.
+
+- **Processes slow down**: You can slow down your i.e., Spark or Hive jobs;
the more files you have, the more tasks you create for reading each file.
+
+- **Storage increases**: When working with a lot of data, you can be
inefficient in using your storage. For example, many small files can have a
lower compression ratio, leading to more data on storage. If you’re indexing
the data, that also takes up more storage space inside the Parquet files. If
you’re working with a small amount of data, you might not see a significant
impact with storage. However, when dealing with petabyte and exabyte data,
you’ll need to be efficient in managing storage resources.
+
+All these challenges inevitably lead to stale analytics and scalability
challenges:
+- Query performance slows down.
+- Jobs could be running faster.
+- You’re utilizing way more resources.
+
+A critical design decision in the Hudi architecture is to avoid small file
creation. Hudi is uniquely designed to write appropriately sized files
automatically. This document will show you how Apache Hudi overcomes the
dreaded small files problem. There are two ways to manage small files in Hudi:
+
+- Auto-size during ingestion
+- Clustering
+
+Below, we will describe the advantages and trade-offs of each.
+
+## Auto-sizing during ingestion
+
+You can manage file sizes through Hudi’s auto-sizing capability during
ingestion. The default targeted file size for Parquet base files is 120MB,
which can be configured by hoodie.parquet.max.file.size. Auto-sizing may add
some data latency, but it ensures that the read queries are always efficient as
soon as a write transaction is committed. It’s important to note that if you
don’t manage file sizing as you write and, instead, try to run clustering to
fix your file sizing periodically, your queries might be slow until the point
when the clustering finishes.
+
+**Note**: the bulk_insert write operation does not have auto-sizing
capabilities during ingestion
+
+If you need to customize the file sizing, i.e., increase the target file size
or change how small files are identified, follow the instructions below for
Copy-On-Write and Merge-On-Read.
+
+### Copy-On-Write (COW)
+To tune the file sizing for a COW table, you can set the small file limit and
the maximum Parquet file size. Hudi will try to add enough records to a small
file at write time to get it to the configured maximum limit.
+
+ - For example, if the `hoodie.parquet.small.file.limit=104857600` (100MB) and
`hoodie.parquet.max.file.size=125829120` (120MB), Hudi will pick all files <
100MB and try to get them up to 120MB.
+
+For creating a Hudi table initially, setting an accurate record size estimate
is vital to ensure Hudi can adequately estimate how many records need to be
bin-packed in a Parquet file for the first ingestion batch. Then, Hudi
automatically uses the average record size for subsequent writes based on
previous commits.
+
+Here are the important configuration of interest:
+
+## Merge-On-Read (MOR)
+As a MOR table aims to reduce the write amplification, compared to a COW
table, when writing to a MOR table, Hudi limits the number of Parquet base
files to one for auto file sizing during insert and upsert operation. This
limits the number of rewritten files. This can be configured through
`hoodie.merge.small.file.group.candidates.limit`.
+
+In addition to file sizing Parquet base files for a MOR table, you can also
tune the log files file-sizing with `hoodie.logfile.max.size`.
+
+**NOTE**: Small files in file groups included in the requested or inflight
compaction or clustering under the active timeline, or small files with
associated log files are not auto-sized with incoming inserts until the
compaction or clustering is complete. For example:
+
+- In case 1: If you had a log file and a compaction, C1, was scheduled to
convert that log file to Parquet, no more inserts can go into the same file
slice.
+
+- In case 2: If the Hudi table has a file group with a Parquet base file and
an associated log file from updates, or this file group is under a requested or
inflight compaction, no more inserts can go into this file group to
automatically size the Parquet file. Only after the compaction has been
performed, and there are NO log files associated with the base Parquet file,
can new inserts be sent to auto-size that parquet file.
+
+Here are the essential configurations:
+
+| Parameter Name | Default | Description | Scope | Since Version
|
+|----------------|--------|----------|---------------|--------------------------------------|
+| hoodie.parquet.small.file.limit | 104857600 (100MB) | During an insert and
upsert operation, we opportunistically expand existing small files on storage
instead of writing new files to keep the number of files optimum. This config
sets the file size limit below which a storage file becomes a candidate to be
selected as such a `small file`. By default, treat any file <= 100MB as a small
file. Also note that if this is set to <= 0, Hudi will not try to get small
files and directly write new files. | Write COW, MOR | 0.4.0 |
+| hoodie.parquet.max.file.size |125829120 (120MB) | This config is the target
size in bytes for parquet files produced by the Hudi write phases. For DFS,
this needs to be aligned with the underlying filesystem block size for optimal
performance. | Write COW, MOR | 0.4.0 |
+| hoodie.logfile.max.size | 1073741824 (1GB) | This is the log file max size
in bytes. This is the maximum size allowed for a log file before it is rolled
over to the next version. | Write MOR | 0.4.0 |
+| hoodie.merge.small.file.group.candidates.limit | 1 | This limits the number
of file groups, whose base file satisfies the small-file limit to be considered
for appending records during an upsert operation. This is only applicable for
MOR tables. | Write MOR | 0.4.0 |
+
+
+## Auto-Sizing With Clustering
+[Clustering](https://hudi.apache.org/docs/next/clustering) is a service that
allows you to combine small files into larger ones while at the same time
(optionally) changing the data layout by sorting or applying space-filling
curves like Z-order or Hilbert curve. We won’t go into all the details about
clustering here, but please refer to the [clustering
section](https://hudi.apache.org/docs/next/clustering) for more details.
+
+Clustering is very handy for file sizing so you can have faster queries. When
you ingest data, you may still have a lot of small files (depending on your
configurations and the data size from ingestion i.e., input batch). In this
case, you will want to cluster all the small files to larger files in one write
operation to improve query performance. Setting configs for this use case is
unnecessary, other than running a job (i.e., Spark job). Optionally, you can
customize the file sizing using the configs down below.
+An example where clustering might be very useful is when a user has a Hudi
table with many small files. Then, instead of waiting for multiple ingestion
batches to gradually auto-size files, a user can use the clustering service to
fix all the file sizes without ingesting any new data.
+
+Please note clustering in Hudi is not a blocking operation, and ingestion can
continue concurrently as long as no files need to be updated while the
clustering service is running. The writes will fail if files need to be updated
while the clustering service runs.
+
+Here are the critical file sizing configurations:
+
+| Parameter Name | Default | Description | Scope | Since Version
|
+|----------------|--------|----------|---------------|--------------------------------------|
+| hoodie.clustering.plan.strategy.small.file.limit | 314572800 (300MB) | Files
smaller than the size in bytes specified here are candidates for clustering. |
Clustering | 0.7.0 |
Review Comment:
similar for property name and backticks. Fix these for all config tables.
##########
website/docs/timeline.md:
##########
@@ -3,40 +3,384 @@ title: Timeline
toc: true
---
-## Timeline
-At its core, Hudi maintains a `timeline` of all actions performed on the table
at different `instants` of time that helps provide instantaneous views of the
table,
-while also efficiently supporting retrieval of data in the order of arrival. A
Hudi instant consists of the following components
+A Hudi table maintains all operations happened to the table in a single
timeline comprised of two parts, an active timeline and an archived timeline.
The active timeline stores all the recent instant, while the archive timeline
stores the older instants. An instant is a transaction where all respective
partitions within a base path have been successfully updated by either a writer
or a table service. Instants that get older in the active timeline are moved to
archived timeline at various times.
-* `Instant action` : Type of action performed on the table
-* `Instant time` : Instant time is typically a timestamp (e.g:
20190117010349), which monotonically increases in the order of action's begin
time.
-* `state` : current state of the instant
+An instant can alter one or many partitions:
-Hudi guarantees that the actions performed on the timeline are atomic &
timeline consistent based on the instant time.
+- If you have one batch ingestion, you’ll see that as one commit in the
active timeline. When you open that commit file, you’ll see a JSON object with
metadata about how one or more partitions were altered.
+
+- If you’re ingesting streaming data, you might see multiple commits in the
active timeline. In this case, when you open a commit file, you might see
metadata about how one or more partition files were altered.
-Key actions performed include
+We’ll go over some details and concepts about the active and archived timeline
below. All files in the timelines are immutable.
-* `COMMITS` - A commit denotes an **atomic write** of a batch of records into
a table.
-* `CLEANS` - Background activity that gets rid of older versions of files in
the table, that are no longer needed.
-* `DELTA_COMMIT` - A delta commit refers to an **atomic write** of a batch of
records into a MergeOnRead type table, where some/all of the data could be
just written to delta logs.
-* `COMPACTION` - Background activity to reconcile differential data structures
within Hudi e.g: moving updates from row based log files to columnar formats.
Internally, compaction manifests as a special commit on the timeline
-* `ROLLBACK` - Indicates that a commit/delta commit was unsuccessful & rolled
back, removing any partial files produced during such a write
-* `SAVEPOINT` - Marks certain file groups as "saved", such that cleaner will
not delete them. It helps restore the table to a point on the timeline, in case
of disaster/data recovery scenarios.
+**Note**: The user should never directly alter the timeline (i.e. manually
delete the commits).
Review Comment:
use caution block:
```
:::caution
xyz
:::
```
##########
website/docs/timeline.md:
##########
@@ -3,40 +3,384 @@ title: Timeline
toc: true
---
-## Timeline
-At its core, Hudi maintains a `timeline` of all actions performed on the table
at different `instants` of time that helps provide instantaneous views of the
table,
-while also efficiently supporting retrieval of data in the order of arrival. A
Hudi instant consists of the following components
+A Hudi table maintains all operations happened to the table in a single
timeline comprised of two parts, an active timeline and an archived timeline.
The active timeline stores all the recent instant, while the archive timeline
stores the older instants. An instant is a transaction where all respective
partitions within a base path have been successfully updated by either a writer
or a table service. Instants that get older in the active timeline are moved to
archived timeline at various times.
-* `Instant action` : Type of action performed on the table
-* `Instant time` : Instant time is typically a timestamp (e.g:
20190117010349), which monotonically increases in the order of action's begin
time.
-* `state` : current state of the instant
+An instant can alter one or many partitions:
-Hudi guarantees that the actions performed on the timeline are atomic &
timeline consistent based on the instant time.
+- If you have one batch ingestion, you’ll see that as one commit in the
active timeline. When you open that commit file, you’ll see a JSON object with
metadata about how one or more partitions were altered.
+
+- If you’re ingesting streaming data, you might see multiple commits in the
active timeline. In this case, when you open a commit file, you might see
metadata about how one or more partition files were altered.
-Key actions performed include
+We’ll go over some details and concepts about the active and archived timeline
below. All files in the timelines are immutable.
-* `COMMITS` - A commit denotes an **atomic write** of a batch of records into
a table.
-* `CLEANS` - Background activity that gets rid of older versions of files in
the table, that are no longer needed.
-* `DELTA_COMMIT` - A delta commit refers to an **atomic write** of a batch of
records into a MergeOnRead type table, where some/all of the data could be
just written to delta logs.
-* `COMPACTION` - Background activity to reconcile differential data structures
within Hudi e.g: moving updates from row based log files to columnar formats.
Internally, compaction manifests as a special commit on the timeline
-* `ROLLBACK` - Indicates that a commit/delta commit was unsuccessful & rolled
back, removing any partial files produced during such a write
-* `SAVEPOINT` - Marks certain file groups as "saved", such that cleaner will
not delete them. It helps restore the table to a point on the timeline, in case
of disaster/data recovery scenarios.
+**Note**: The user should never directly alter the timeline (i.e. manually
delete the commits).
-Any given instant can be
-in one of the following states
+## Active Timeline
-* `REQUESTED` - Denotes an action has been scheduled, but has not initiated
-* `INFLIGHT` - Denotes that the action is currently being performed
-* `COMPLETED` - Denotes completion of an action on the timeline
+The active timeline is a source of truth for all write operations: when an
action (described below) happens on a table, the timeline is responsible for
recording it. This guarantees a good table state, and Hudi can provide
read/write isolation based on the timeline. For example, when data is being
written to a Hudi table (i.e., requested, inflight), any data being written as
part of the transaction is not visible to a query engine until the write
transaction is completed. The query engine can still read older data, but the
data inflight won’t be exposed.
-<figure>
- <img className="docimage"
src={require("/assets/images/hudi_timeline.png").default}
alt="hudi_timeline.png" />
-</figure>
+The active timeline is under the `.hoodie` metadata folder. For example, when
you navigate to your Hudi project directory:
-Example above shows upserts happenings between 10:00 and 10:20 on a Hudi
table, roughly every 5 mins, leaving commit metadata on the Hudi timeline, along
-with other background cleaning/compactions. One key observation to make is
that the commit time indicates the `arrival time` of the data (10:20AM), while
the actual data
-organization reflects the actual time or `event time`, the data was intended
for (hourly buckets from 07:00). These are two key concepts when reasoning
about tradeoffs between latency and completeness of data.
+```sh
+cd $YOUR_HUDI_PROJECT_DIRECTORY && ls -a
+```
+
+You’ll see the `.hoodie` metadata folder:
+
+```sh
+ls -a
+. .. .hoodie americas asia
+```
+
+When you navigate inside the `.hoodie` folder, you’ll see a lot of files with
different suffixes and the archived timeline folder:
+
+```sh
+cd .hoodie && ls
+2023021018095339.commit
+20230210180953939.commit.requested
+20230210180953939.inflight
+archived
+```
+
+Before we go into what’s in the files or how the files are named, we’ll need
to cover some broader concepts:
+- actions
+- states
+- instants
+
+## Actions
Review Comment:
The concept of action, state, and instant is common across the active
timeline and archived timeline. Shall we move them up to level 1 before
discussing the active timeline and archived timeline?
Some descriptions on the layout, e.g., the filenames, are specific to active
timeline.
##########
website/docs/file_sizing.md:
##########
@@ -3,51 +3,76 @@ title: "File Sizing"
toc: true
---
-This doc will show you how Apache Hudi overcomes the dreaded small files
problem. A key design decision in Hudi was to
-avoid creating small files in the first place and always write properly sized
files.
-There are 2 ways to manage small files in Hudi and below will describe the
advantages and trade-offs of each.
-
-## Auto-Size During ingestion
-
-You can automatically manage size of files during ingestion. This solution
adds a little latency during ingestion, but
-it ensures that read queries are always efficient as soon as a write is
committed. If you don't
-manage file sizing as you write and instead try to periodically run a
file-sizing clean-up, your queries will be slow until that resize cleanup is
periodically performed.
-
-(Note: [bulk_insert](/docs/next/write_operations) write operation does not
provide auto-sizing during ingestion)
-
-### For Copy-On-Write
-This is as simple as configuring the [maximum size for a base/parquet
file](/docs/configurations#hoodieparquetmaxfilesize)
-and the [soft limit](/docs/configurations#hoodieparquetsmallfilelimit) below
which a file should
-be considered a small file. For the initial bootstrap of a Hudi table, tuning
record size estimate is also important to
-ensure sufficient records are bin-packed in a parquet file. For subsequent
writes, Hudi automatically uses average
-record size based on previous commit. Hudi will try to add enough records to a
small file at write time to get it to the
-configured maximum limit. For e.g , with `compactionSmallFileSize=100MB` and
limitFileSize=120MB, Hudi will pick all
-files < 100MB and try to get them upto 120MB.
-
-### For Merge-On-Read
-MergeOnRead works differently for different INDEX choices so there are few
more configs to set:
-
-- Indexes with **canIndexLogFiles = true** : Inserts of new data go directly
to log files. In this case, you can
-configure the [maximum log size](/docs/configurations#hoodielogfilemaxsize)
and a
-[factor](/docs/configurations#hoodielogfiletoparquetcompressionratio) that
denotes reduction in
-size when data moves from avro to parquet files.
-- Indexes with **canIndexLogFiles = false** : Inserts of new data go only to
parquet files. In this case, the
-same configurations as above for the COPY_ON_WRITE case applies.
-
-NOTE : In either case, small files will be auto sized only if there is no
PENDING compaction or associated log file for
-that particular file slice. For example, for case 1: If you had a log file and
a compaction C1 was scheduled to convert
-that log file to parquet, no more inserts can go into that log file. For case
2: If you had a parquet file and an update
-ended up creating an associated delta log file, no more inserts can go into
that parquet file. Only after the compaction
-has been performed and there are NO log files associated with the base parquet
file, can new inserts be sent to auto size that parquet file.
-
-## Auto-Size With Clustering
-**[Clustering](/docs/next/clustering)** is a feature in Hudi to group
-small files into larger ones either synchronously or asynchronously. Since
first solution of auto-sizing small files has
-a tradeoff on ingestion speed (since the small files are sized during
ingestion), if your use-case is very sensitive to
-ingestion latency where you don't want to compromise on ingestion speed which
may end up creating a lot of small files,
-clustering comes to the rescue. Clustering can be scheduled through the
ingestion job and an asynchronus job can stitch
-small files together in the background to generate larger files. NOTE that
during this, ingestion can continue to run concurrently.
-
-*Please note that Hudi always creates immutable files on disk. To be able to
do auto-sizing or clustering, Hudi will
-always create a newer version of the smaller file, resulting in 2 versions of
the same file.
-The [cleaner service](/docs/next/hoodie_cleaner) will later kick in and delete
the older version small file and keep the latest one.*
\ No newline at end of file
+A fundamental problem when writing data to a source is having a lot of small
files. This is also known as a small file problem. If you don’t size the files
appropriately, you can slow down the query performance and work with stale
analytics. Some of the issues you may encounter with small files include the
following:
+- **Reads slow down**: You’ll have to scan through many small files to
retrieve data for a query. It’s a very inefficient way of accessing and
utilizing the data.
+
+- **Processes slow down**: You can slow down your i.e., Spark or Hive jobs;
the more files you have, the more tasks you create for reading each file.
+
+- **Storage increases**: When working with a lot of data, you can be
inefficient in using your storage. For example, many small files can have a
lower compression ratio, leading to more data on storage. If you’re indexing
the data, that also takes up more storage space inside the Parquet files. If
you’re working with a small amount of data, you might not see a significant
impact with storage. However, when dealing with petabyte and exabyte data,
you’ll need to be efficient in managing storage resources.
+
+All these challenges inevitably lead to stale analytics and scalability
challenges:
+- Query performance slows down.
+- Jobs could be running faster.
+- You’re utilizing way more resources.
+
+A critical design decision in the Hudi architecture is to avoid small file
creation. Hudi is uniquely designed to write appropriately sized files
automatically. This document will show you how Apache Hudi overcomes the
dreaded small files problem. There are two ways to manage small files in Hudi:
+
+- Auto-size during ingestion
+- Clustering
+
+Below, we will describe the advantages and trade-offs of each.
+
+## Auto-sizing during ingestion
+
+You can manage file sizes through Hudi’s auto-sizing capability during
ingestion. The default targeted file size for Parquet base files is 120MB,
which can be configured by hoodie.parquet.max.file.size. Auto-sizing may add
some data latency, but it ensures that the read queries are always efficient as
soon as a write transaction is committed. It’s important to note that if you
don’t manage file sizing as you write and, instead, try to run clustering to
fix your file sizing periodically, your queries might be slow until the point
when the clustering finishes.
+
+**Note**: the bulk_insert write operation does not have auto-sizing
capabilities during ingestion
+
+If you need to customize the file sizing, i.e., increase the target file size
or change how small files are identified, follow the instructions below for
Copy-On-Write and Merge-On-Read.
+
+### Copy-On-Write (COW)
+To tune the file sizing for a COW table, you can set the small file limit and
the maximum Parquet file size. Hudi will try to add enough records to a small
file at write time to get it to the configured maximum limit.
+
+ - For example, if the `hoodie.parquet.small.file.limit=104857600` (100MB) and
`hoodie.parquet.max.file.size=125829120` (120MB), Hudi will pick all files <
100MB and try to get them up to 120MB.
+
+For creating a Hudi table initially, setting an accurate record size estimate
is vital to ensure Hudi can adequately estimate how many records need to be
bin-packed in a Parquet file for the first ingestion batch. Then, Hudi
automatically uses the average record size for subsequent writes based on
previous commits.
+
+Here are the important configuration of interest:
Review Comment:
@nfarah86 You should use the markdown to add the config tables.
##########
website/docs/file_sizing.md:
##########
@@ -3,51 +3,76 @@ title: "File Sizing"
toc: true
---
-This doc will show you how Apache Hudi overcomes the dreaded small files
problem. A key design decision in Hudi was to
-avoid creating small files in the first place and always write properly sized
files.
-There are 2 ways to manage small files in Hudi and below will describe the
advantages and trade-offs of each.
-
-## Auto-Size During ingestion
-
-You can automatically manage size of files during ingestion. This solution
adds a little latency during ingestion, but
-it ensures that read queries are always efficient as soon as a write is
committed. If you don't
-manage file sizing as you write and instead try to periodically run a
file-sizing clean-up, your queries will be slow until that resize cleanup is
periodically performed.
-
-(Note: [bulk_insert](/docs/next/write_operations) write operation does not
provide auto-sizing during ingestion)
-
-### For Copy-On-Write
-This is as simple as configuring the [maximum size for a base/parquet
file](/docs/configurations#hoodieparquetmaxfilesize)
-and the [soft limit](/docs/configurations#hoodieparquetsmallfilelimit) below
which a file should
-be considered a small file. For the initial bootstrap of a Hudi table, tuning
record size estimate is also important to
-ensure sufficient records are bin-packed in a parquet file. For subsequent
writes, Hudi automatically uses average
-record size based on previous commit. Hudi will try to add enough records to a
small file at write time to get it to the
-configured maximum limit. For e.g , with `compactionSmallFileSize=100MB` and
limitFileSize=120MB, Hudi will pick all
-files < 100MB and try to get them upto 120MB.
-
-### For Merge-On-Read
-MergeOnRead works differently for different INDEX choices so there are few
more configs to set:
-
-- Indexes with **canIndexLogFiles = true** : Inserts of new data go directly
to log files. In this case, you can
-configure the [maximum log size](/docs/configurations#hoodielogfilemaxsize)
and a
-[factor](/docs/configurations#hoodielogfiletoparquetcompressionratio) that
denotes reduction in
-size when data moves from avro to parquet files.
-- Indexes with **canIndexLogFiles = false** : Inserts of new data go only to
parquet files. In this case, the
-same configurations as above for the COPY_ON_WRITE case applies.
-
-NOTE : In either case, small files will be auto sized only if there is no
PENDING compaction or associated log file for
-that particular file slice. For example, for case 1: If you had a log file and
a compaction C1 was scheduled to convert
-that log file to parquet, no more inserts can go into that log file. For case
2: If you had a parquet file and an update
-ended up creating an associated delta log file, no more inserts can go into
that parquet file. Only after the compaction
-has been performed and there are NO log files associated with the base parquet
file, can new inserts be sent to auto size that parquet file.
-
-## Auto-Size With Clustering
-**[Clustering](/docs/next/clustering)** is a feature in Hudi to group
-small files into larger ones either synchronously or asynchronously. Since
first solution of auto-sizing small files has
-a tradeoff on ingestion speed (since the small files are sized during
ingestion), if your use-case is very sensitive to
-ingestion latency where you don't want to compromise on ingestion speed which
may end up creating a lot of small files,
-clustering comes to the rescue. Clustering can be scheduled through the
ingestion job and an asynchronus job can stitch
-small files together in the background to generate larger files. NOTE that
during this, ingestion can continue to run concurrently.
-
-*Please note that Hudi always creates immutable files on disk. To be able to
do auto-sizing or clustering, Hudi will
-always create a newer version of the smaller file, resulting in 2 versions of
the same file.
-The [cleaner service](/docs/next/hoodie_cleaner) will later kick in and delete
the older version small file and keep the latest one.*
\ No newline at end of file
+A fundamental problem when writing data to a source is having a lot of small
files. This is also known as a small file problem. If you don’t size the files
appropriately, you can slow down the query performance and work with stale
analytics. Some of the issues you may encounter with small files include the
following:
+- **Reads slow down**: You’ll have to scan through many small files to
retrieve data for a query. It’s a very inefficient way of accessing and
utilizing the data.
+
+- **Processes slow down**: You can slow down your i.e., Spark or Hive jobs;
the more files you have, the more tasks you create for reading each file.
+
+- **Storage increases**: When working with a lot of data, you can be
inefficient in using your storage. For example, many small files can have a
lower compression ratio, leading to more data on storage. If you’re indexing
the data, that also takes up more storage space inside the Parquet files. If
you’re working with a small amount of data, you might not see a significant
impact with storage. However, when dealing with petabyte and exabyte data,
you’ll need to be efficient in managing storage resources.
Review Comment:
+1
--
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]