You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/04/21 21:21:21 UTC

[GitHub] [hudi] vinothchandar commented on a diff in pull request #5113: [HUDI-3625] [RFC-48] Optimized storage layout for Cloud Object Stores

vinothchandar commented on code in PR #5113:
URL: https://github.com/apache/hudi/pull/5113#discussion_r855589377


##########
rfc/rfc-48/rfc-48.md:
##########
@@ -0,0 +1,171 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+# RFC-[48]: Optimized storage layout for Cloud Object Stores
+
+## Proposers
+- @umehrot2
+
+## Approvers
+- @vinoth
+- @shivnarayan
+
+## Status
+
+JIRA: [https://issues.apache.org/jira/browse/HUDI-3625](https://issues.apache.org/jira/browse/HUDI-3625)
+
+## Abstract
+
+As you scale your Apache Hudi workloads over Cloud object stores like Amazon S3, there is potential of hitting request
+throttling limits which in-turn impacts performance. In this RFC, we are proposing to support an alternate storage
+layout that is optimized for Amazon S3 and other cloud object stores, which helps achieve maximum throughput and
+significantly reduce throttling.
+
+## Background
+
+Apache Hudi follows the traditional Hive storage layout while writing files on storage:
+- Partitioned Tables: The files are distributed across multiple physical partition folders, under the table's base path.
+- Non Partitioned Tables: The files are stored directly under the table's base path.
+
+While this storage layout scales well for HDFS, it increases the probability of hitting request throttle limits when
+working with cloud object stores like Amazon S3 and others. This is because Amazon S3 and other cloud stores [throttle
+requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/).
+Amazon S3 does scale based on request patterns for different prefixes and adds internal partitions (with their own request limits),
+but there can be a 30 - 60 minute wait time before new partitions are created. Thus, all files/objects stored under the
+same table path prefix could result in these request limits being hit for the table prefix, specially as workloads
+scale, and there are several thousands of files being written/updated concurrently. This hurts performance due to
+re-trying of failed requests affecting throughput, and result in occasional failures if the retries are not able to
+succeed either and continue to be throttled.
+
+The high level proposal here is to introduce a new storage layout, where all files are distributed evenly across multiple
+randomly generated prefixes under the Amazon S3 bucket, instead of being stored under a common table path/prefix. This
+would help distribute the requests evenly across different prefixes, resulting in Amazon S3 to create partitions for
+the prefixes each with its own request limit. This significantly reduces the possibility of hitting the request limit
+for a specific prefix/partition.
+
+## Design
+
+### Generating file paths
+
+We want to distribute files evenly across multiple random prefixes, instead of following the traditional Hive storage
+layout of keeping them under a common table path/prefix. In addition to the `Table Path`, for this new layout user will
+configure another `Table Storage Path` under which the actual data files will be distributed. The original `Table Path` will
+be used to maintain the table/partitions Hudi metadata.
+
+For the purpose of this documentation lets assume:
+```
+Table Path => s3://<table_bucket>/<hudi_table_name>/
+
+Table Storage Path => s3://<table_storage_bucket>/
+```
+Note: `Table Storage Path` can be a path in the same Amazon S3 bucket or a different bucket. For best results,
+`Table Storage Path` should be a bucket instead of a prefix under the bucket as it allows for S3 to partition sooner.
+
+We will use a Hashing function on the `File Name` to map them to a prefix generated under `Table Storage Path`:
+```
+s3://<table_storage_bucket>/<hash_prefix>/..
+```
+
+In addition, under the hash prefix we will follow a folder structure by appending Hudi Table Name and Partition. This
+folder structuring would be useful if we ever have to do a file system listing to re-create the metadata file list for
+the table (discussed more in the next section). Here is how the final layout would look like for `partitioned` tables:
+```
+s3://<table_storage_bucket>/01f50736/<hudi_table_name>/country=usa/075f3295-def8-4a42-a927-07fd2dd2976c-0_7-11-49_20220301005056692.parquet
+s3://<table_storage_bucket>/01f50736/<hudi_table_name>/country=india/4b0c6b40-2ac0-4a1c-a26f-6338aa4db22e-0_6-11-48_20220301005056692.parquet
+...
+s3://<table_storage_bucket>/0bfb3d6e/<hudi_table_name>/country=india/9320889c-8537-4aa7-a63e-ef088b9a21ce-0_9-11-51_20220301005056692.parquet
+s3://<table_storage_bucket>/0bfb3d6e/<hudi_table_name>/country=uk/a62aa56b-d55e-4a2b-88a6-d603ef26775c-0_8-11-50_20220301005056692.parquet
+...
+```
+
+For `non-partitioned` tables, this is how it would look:
+```
+s3://<table_storage_bucket>/01f50736/<hudi_table_name>/075f3295-def8-4a42-a927-07fd2dd2976c-0_7-11-49_20220301005056692.parquet
+s3://<table_storage_bucket>/01f50736/<hudi_table_name>/4b0c6b40-2ac0-4a1c-a26f-6338aa4db22e-0_6-11-48_20220301005056692.parquet
+s3://<table_storage_bucket>/0bfb3d6e/<hudi_table_name>/9320889c-8537-4aa7-a63e-ef088b9a21ce-0_9-11-51_20220301005056692.parquet
+s3://<table_storage_bucket>/0bfb3d6e/<hudi_table_name>/a62aa56b-d55e-4a2b-88a6-d603ef26775c-0_8-11-50_20220301005056692.parquet
+...
+```
+
+The original table path will continue to store the `metadata folder` and `partition metadata` files:
+```
+s3://<table_bucket>/<hudi_table_name>/.hoodie/...
+s3://<table_bucket>/<hudi_table_name>/country=usa/.hoodie_partition_metadata
+s3://<table_bucket>/<hudi_table_name>/country=india/.hoodie_partition_metadata
+s3://<table_bucket>/<hudi_table_name>/country=uk/.hoodie_partition_metadata
+...
+```
+
+#### Hashing
+
+To generate the prefixes we can use `Murmur 32 bit` hash on the `File Names`, which is known for being fast and provides
+good distribution guarantees. We might have to further do bucketing and re-hash it to reduce the number of possible hashes
+from 2^32 to a slightly lower number, as it may be overkill to have that many unique hashes, which might result in scenarios
+where each file is under a different prefix.
+
+### Maintain mapping to files
+
+In [RFC-15](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=147427331), we introduced an internal
+Metadata Table with a `files` partition that maintains mapping from partitions to list of files in the partition stored
+under `Table Path`. This mapping is kept up to date, as operations are performed on the original table. We will leverage
+the same to now maintain mappings to files stored at `Table Storage Path` under different prefixes.
+
+Here are some of the design considerations:
+
+1. Metadata table is a pre-requisite and will be automatically enabled if object storage layout is enabled.
+
+2. The object storage layout cannot be enabled on an existing table that is already bootstrapped with Hive storage layout.
+To switch to object storage layout, the table will need to be re-bootstrapped with the new layout.
+
+3. The Instant metadata (`HoodieCommitMetadata`,`HoodieCleanMetadata` etc.) will always act as the source of file listing
+for metadata table to be populated.
+
+4. `HoodieCommitMetadata` currently stores `file name` instead of complete `file path`. We will have to modify commit

Review Comment:
   can we instead just derive the hash on the fly? i.e if the table has a storage path configured, then hash(file_name) and then look under `storage_path/hash/...`? Do we need to store it?



##########
rfc/rfc-48/rfc-48.md:
##########
@@ -0,0 +1,171 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+# RFC-[48]: Optimized storage layout for Cloud Object Stores
+
+## Proposers
+- @umehrot2
+
+## Approvers
+- @vinoth
+- @shivnarayan
+
+## Status
+
+JIRA: [https://issues.apache.org/jira/browse/HUDI-3625](https://issues.apache.org/jira/browse/HUDI-3625)
+
+## Abstract
+
+As you scale your Apache Hudi workloads over Cloud object stores like Amazon S3, there is potential of hitting request
+throttling limits which in-turn impacts performance. In this RFC, we are proposing to support an alternate storage
+layout that is optimized for Amazon S3 and other cloud object stores, which helps achieve maximum throughput and
+significantly reduce throttling.
+
+## Background
+
+Apache Hudi follows the traditional Hive storage layout while writing files on storage:
+- Partitioned Tables: The files are distributed across multiple physical partition folders, under the table's base path.
+- Non Partitioned Tables: The files are stored directly under the table's base path.
+
+While this storage layout scales well for HDFS, it increases the probability of hitting request throttle limits when
+working with cloud object stores like Amazon S3 and others. This is because Amazon S3 and other cloud stores [throttle
+requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/).
+Amazon S3 does scale based on request patterns for different prefixes and adds internal partitions (with their own request limits),
+but there can be a 30 - 60 minute wait time before new partitions are created. Thus, all files/objects stored under the
+same table path prefix could result in these request limits being hit for the table prefix, specially as workloads
+scale, and there are several thousands of files being written/updated concurrently. This hurts performance due to
+re-trying of failed requests affecting throughput, and result in occasional failures if the retries are not able to
+succeed either and continue to be throttled.
+
+The high level proposal here is to introduce a new storage layout, where all files are distributed evenly across multiple
+randomly generated prefixes under the Amazon S3 bucket, instead of being stored under a common table path/prefix. This
+would help distribute the requests evenly across different prefixes, resulting in Amazon S3 to create partitions for
+the prefixes each with its own request limit. This significantly reduces the possibility of hitting the request limit
+for a specific prefix/partition.
+
+## Design
+
+### Generating file paths
+
+We want to distribute files evenly across multiple random prefixes, instead of following the traditional Hive storage
+layout of keeping them under a common table path/prefix. In addition to the `Table Path`, for this new layout user will
+configure another `Table Storage Path` under which the actual data files will be distributed. The original `Table Path` will
+be used to maintain the table/partitions Hudi metadata.
+
+For the purpose of this documentation lets assume:
+```
+Table Path => s3://<table_bucket>/<hudi_table_name>/
+
+Table Storage Path => s3://<table_storage_bucket>/
+```
+Note: `Table Storage Path` can be a path in the same Amazon S3 bucket or a different bucket. For best results,
+`Table Storage Path` should be a bucket instead of a prefix under the bucket as it allows for S3 to partition sooner.
+
+We will use a Hashing function on the `File Name` to map them to a prefix generated under `Table Storage Path`:
+```
+s3://<table_storage_bucket>/<hash_prefix>/..
+```
+
+In addition, under the hash prefix we will follow a folder structure by appending Hudi Table Name and Partition. This
+folder structuring would be useful if we ever have to do a file system listing to re-create the metadata file list for
+the table (discussed more in the next section). Here is how the final layout would look like for `partitioned` tables:
+```
+s3://<table_storage_bucket>/01f50736/<hudi_table_name>/country=usa/075f3295-def8-4a42-a927-07fd2dd2976c-0_7-11-49_20220301005056692.parquet
+s3://<table_storage_bucket>/01f50736/<hudi_table_name>/country=india/4b0c6b40-2ac0-4a1c-a26f-6338aa4db22e-0_6-11-48_20220301005056692.parquet
+...
+s3://<table_storage_bucket>/0bfb3d6e/<hudi_table_name>/country=india/9320889c-8537-4aa7-a63e-ef088b9a21ce-0_9-11-51_20220301005056692.parquet
+s3://<table_storage_bucket>/0bfb3d6e/<hudi_table_name>/country=uk/a62aa56b-d55e-4a2b-88a6-d603ef26775c-0_8-11-50_20220301005056692.parquet
+...
+```
+
+For `non-partitioned` tables, this is how it would look:
+```
+s3://<table_storage_bucket>/01f50736/<hudi_table_name>/075f3295-def8-4a42-a927-07fd2dd2976c-0_7-11-49_20220301005056692.parquet
+s3://<table_storage_bucket>/01f50736/<hudi_table_name>/4b0c6b40-2ac0-4a1c-a26f-6338aa4db22e-0_6-11-48_20220301005056692.parquet
+s3://<table_storage_bucket>/0bfb3d6e/<hudi_table_name>/9320889c-8537-4aa7-a63e-ef088b9a21ce-0_9-11-51_20220301005056692.parquet
+s3://<table_storage_bucket>/0bfb3d6e/<hudi_table_name>/a62aa56b-d55e-4a2b-88a6-d603ef26775c-0_8-11-50_20220301005056692.parquet
+...
+```
+
+The original table path will continue to store the `metadata folder` and `partition metadata` files:
+```
+s3://<table_bucket>/<hudi_table_name>/.hoodie/...
+s3://<table_bucket>/<hudi_table_name>/country=usa/.hoodie_partition_metadata
+s3://<table_bucket>/<hudi_table_name>/country=india/.hoodie_partition_metadata
+s3://<table_bucket>/<hudi_table_name>/country=uk/.hoodie_partition_metadata
+...
+```
+
+#### Hashing
+
+To generate the prefixes we can use `Murmur 32 bit` hash on the `File Names`, which is known for being fast and provides

Review Comment:
   See `HashID` class, which already has some fast hashing functions!
   



##########
rfc/rfc-48/rfc-48.md:
##########
@@ -0,0 +1,171 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+# RFC-[48]: Optimized storage layout for Cloud Object Stores
+
+## Proposers
+- @umehrot2
+
+## Approvers
+- @vinoth
+- @shivnarayan
+
+## Status
+
+JIRA: [https://issues.apache.org/jira/browse/HUDI-3625](https://issues.apache.org/jira/browse/HUDI-3625)
+
+## Abstract
+
+As you scale your Apache Hudi workloads over Cloud object stores like Amazon S3, there is potential of hitting request
+throttling limits which in-turn impacts performance. In this RFC, we are proposing to support an alternate storage
+layout that is optimized for Amazon S3 and other cloud object stores, which helps achieve maximum throughput and
+significantly reduce throttling.
+
+## Background
+
+Apache Hudi follows the traditional Hive storage layout while writing files on storage:
+- Partitioned Tables: The files are distributed across multiple physical partition folders, under the table's base path.
+- Non Partitioned Tables: The files are stored directly under the table's base path.
+
+While this storage layout scales well for HDFS, it increases the probability of hitting request throttle limits when
+working with cloud object stores like Amazon S3 and others. This is because Amazon S3 and other cloud stores [throttle
+requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/).
+Amazon S3 does scale based on request patterns for different prefixes and adds internal partitions (with their own request limits),
+but there can be a 30 - 60 minute wait time before new partitions are created. Thus, all files/objects stored under the
+same table path prefix could result in these request limits being hit for the table prefix, specially as workloads
+scale, and there are several thousands of files being written/updated concurrently. This hurts performance due to
+re-trying of failed requests affecting throughput, and result in occasional failures if the retries are not able to
+succeed either and continue to be throttled.
+
+The high level proposal here is to introduce a new storage layout, where all files are distributed evenly across multiple
+randomly generated prefixes under the Amazon S3 bucket, instead of being stored under a common table path/prefix. This
+would help distribute the requests evenly across different prefixes, resulting in Amazon S3 to create partitions for
+the prefixes each with its own request limit. This significantly reduces the possibility of hitting the request limit
+for a specific prefix/partition.
+
+## Design
+
+### Generating file paths
+
+We want to distribute files evenly across multiple random prefixes, instead of following the traditional Hive storage
+layout of keeping them under a common table path/prefix. In addition to the `Table Path`, for this new layout user will
+configure another `Table Storage Path` under which the actual data files will be distributed. The original `Table Path` will
+be used to maintain the table/partitions Hudi metadata.
+
+For the purpose of this documentation lets assume:
+```
+Table Path => s3://<table_bucket>/<hudi_table_name>/
+
+Table Storage Path => s3://<table_storage_bucket>/
+```
+Note: `Table Storage Path` can be a path in the same Amazon S3 bucket or a different bucket. For best results,
+`Table Storage Path` should be a bucket instead of a prefix under the bucket as it allows for S3 to partition sooner.
+
+We will use a Hashing function on the `File Name` to map them to a prefix generated under `Table Storage Path`:
+```
+s3://<table_storage_bucket>/<hash_prefix>/..
+```
+
+In addition, under the hash prefix we will follow a folder structure by appending Hudi Table Name and Partition. This
+folder structuring would be useful if we ever have to do a file system listing to re-create the metadata file list for
+the table (discussed more in the next section). Here is how the final layout would look like for `partitioned` tables:
+```
+s3://<table_storage_bucket>/01f50736/<hudi_table_name>/country=usa/075f3295-def8-4a42-a927-07fd2dd2976c-0_7-11-49_20220301005056692.parquet
+s3://<table_storage_bucket>/01f50736/<hudi_table_name>/country=india/4b0c6b40-2ac0-4a1c-a26f-6338aa4db22e-0_6-11-48_20220301005056692.parquet
+...
+s3://<table_storage_bucket>/0bfb3d6e/<hudi_table_name>/country=india/9320889c-8537-4aa7-a63e-ef088b9a21ce-0_9-11-51_20220301005056692.parquet
+s3://<table_storage_bucket>/0bfb3d6e/<hudi_table_name>/country=uk/a62aa56b-d55e-4a2b-88a6-d603ef26775c-0_8-11-50_20220301005056692.parquet
+...
+```
+
+For `non-partitioned` tables, this is how it would look:
+```
+s3://<table_storage_bucket>/01f50736/<hudi_table_name>/075f3295-def8-4a42-a927-07fd2dd2976c-0_7-11-49_20220301005056692.parquet
+s3://<table_storage_bucket>/01f50736/<hudi_table_name>/4b0c6b40-2ac0-4a1c-a26f-6338aa4db22e-0_6-11-48_20220301005056692.parquet
+s3://<table_storage_bucket>/0bfb3d6e/<hudi_table_name>/9320889c-8537-4aa7-a63e-ef088b9a21ce-0_9-11-51_20220301005056692.parquet
+s3://<table_storage_bucket>/0bfb3d6e/<hudi_table_name>/a62aa56b-d55e-4a2b-88a6-d603ef26775c-0_8-11-50_20220301005056692.parquet
+...
+```
+
+The original table path will continue to store the `metadata folder` and `partition metadata` files:
+```
+s3://<table_bucket>/<hudi_table_name>/.hoodie/...
+s3://<table_bucket>/<hudi_table_name>/country=usa/.hoodie_partition_metadata
+s3://<table_bucket>/<hudi_table_name>/country=india/.hoodie_partition_metadata
+s3://<table_bucket>/<hudi_table_name>/country=uk/.hoodie_partition_metadata
+...
+```
+
+#### Hashing
+
+To generate the prefixes we can use `Murmur 32 bit` hash on the `File Names`, which is known for being fast and provides
+good distribution guarantees. We might have to further do bucketing and re-hash it to reduce the number of possible hashes
+from 2^32 to a slightly lower number, as it may be overkill to have that many unique hashes, which might result in scenarios
+where each file is under a different prefix.
+
+### Maintain mapping to files
+
+In [RFC-15](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=147427331), we introduced an internal
+Metadata Table with a `files` partition that maintains mapping from partitions to list of files in the partition stored
+under `Table Path`. This mapping is kept up to date, as operations are performed on the original table. We will leverage
+the same to now maintain mappings to files stored at `Table Storage Path` under different prefixes.
+
+Here are some of the design considerations:
+
+1. Metadata table is a pre-requisite and will be automatically enabled if object storage layout is enabled.
+
+2. The object storage layout cannot be enabled on an existing table that is already bootstrapped with Hive storage layout.
+To switch to object storage layout, the table will need to be re-bootstrapped with the new layout.
+
+3. The Instant metadata (`HoodieCommitMetadata`,`HoodieCleanMetadata` etc.) will always act as the source of file listing
+for metadata table to be populated.
+
+4. `HoodieCommitMetadata` currently stores `file name` instead of complete `file path`. We will have to modify commit
+metadata to store the complete file path instead of just file name, as the files are now distributed across several random
+prefix paths instead of a derivable table/partition path.
+
+5. If there is an error reading from Metadata table, we will not fall back listing from file system.
+
+6. In case of metadata table getting corrupted or lost, we need to have a solution here to reconstruct metadata table
+from the files which distributed using object storage layout. We will likely have to implement a file system listing
+logic, that can get all the partition to files mapping by listing all the prefixes under the `Table Storage Path`.
+Following the folder structure of adding table name/partitions under the prefix will help in getting the listing and
+identifying the table/partition they belong to.
+
+### Query Side Integration
+
+Spark, Hive and [Presto](https://github.com/prestodb/presto/commit/ef1fd25c582631513ccdd097e0a654cda44ec3dc) are
+already integrated to use metadata based listing. Hudi connector for [Trino](https://github.com/prestodb/presto/commit/ef1fd25c582631513ccdd097e0a654cda44ec3dc)
+is currently in process of being contributed, which also integrates with metadata based listing. In general, since these
+query engines are able to use Hudi's metadata table there should ideally be no changes required in
+terms of making them work with Object storage layout. Here are some considerations:
+
+- Spark DataSource and Spark SQL queries have been integrated with metadata based listing via the Hudi's custom implementation
+of Spark's FileIndex interface. However, if Spark DataSource queries are used with globbed paths then the FileIndex path

Review Comment:
   We can end support for globbed paths if need be. is that problematic?



##########
rfc/rfc-48/rfc-48.md:
##########
@@ -0,0 +1,171 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+# RFC-[48]: Optimized storage layout for Cloud Object Stores
+
+## Proposers
+- @umehrot2
+
+## Approvers
+- @vinoth
+- @shivnarayan
+
+## Status
+
+JIRA: [https://issues.apache.org/jira/browse/HUDI-3625](https://issues.apache.org/jira/browse/HUDI-3625)
+
+## Abstract
+
+As you scale your Apache Hudi workloads over Cloud object stores like Amazon S3, there is potential of hitting request
+throttling limits which in-turn impacts performance. In this RFC, we are proposing to support an alternate storage
+layout that is optimized for Amazon S3 and other cloud object stores, which helps achieve maximum throughput and
+significantly reduce throttling.
+
+## Background
+
+Apache Hudi follows the traditional Hive storage layout while writing files on storage:
+- Partitioned Tables: The files are distributed across multiple physical partition folders, under the table's base path.
+- Non Partitioned Tables: The files are stored directly under the table's base path.
+
+While this storage layout scales well for HDFS, it increases the probability of hitting request throttle limits when
+working with cloud object stores like Amazon S3 and others. This is because Amazon S3 and other cloud stores [throttle
+requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/).
+Amazon S3 does scale based on request patterns for different prefixes and adds internal partitions (with their own request limits),
+but there can be a 30 - 60 minute wait time before new partitions are created. Thus, all files/objects stored under the
+same table path prefix could result in these request limits being hit for the table prefix, specially as workloads
+scale, and there are several thousands of files being written/updated concurrently. This hurts performance due to
+re-trying of failed requests affecting throughput, and result in occasional failures if the retries are not able to
+succeed either and continue to be throttled.
+
+The high level proposal here is to introduce a new storage layout, where all files are distributed evenly across multiple
+randomly generated prefixes under the Amazon S3 bucket, instead of being stored under a common table path/prefix. This
+would help distribute the requests evenly across different prefixes, resulting in Amazon S3 to create partitions for
+the prefixes each with its own request limit. This significantly reduces the possibility of hitting the request limit
+for a specific prefix/partition.
+
+## Design
+
+### Generating file paths
+
+We want to distribute files evenly across multiple random prefixes, instead of following the traditional Hive storage
+layout of keeping them under a common table path/prefix. In addition to the `Table Path`, for this new layout user will
+configure another `Table Storage Path` under which the actual data files will be distributed. The original `Table Path` will
+be used to maintain the table/partitions Hudi metadata.
+
+For the purpose of this documentation lets assume:
+```
+Table Path => s3://<table_bucket>/<hudi_table_name>/
+
+Table Storage Path => s3://<table_storage_bucket>/
+```
+Note: `Table Storage Path` can be a path in the same Amazon S3 bucket or a different bucket. For best results,
+`Table Storage Path` should be a bucket instead of a prefix under the bucket as it allows for S3 to partition sooner.
+
+We will use a Hashing function on the `File Name` to map them to a prefix generated under `Table Storage Path`:
+```
+s3://<table_storage_bucket>/<hash_prefix>/..
+```
+
+In addition, under the hash prefix we will follow a folder structure by appending Hudi Table Name and Partition. This
+folder structuring would be useful if we ever have to do a file system listing to re-create the metadata file list for
+the table (discussed more in the next section). Here is how the final layout would look like for `partitioned` tables:
+```
+s3://<table_storage_bucket>/01f50736/<hudi_table_name>/country=usa/075f3295-def8-4a42-a927-07fd2dd2976c-0_7-11-49_20220301005056692.parquet
+s3://<table_storage_bucket>/01f50736/<hudi_table_name>/country=india/4b0c6b40-2ac0-4a1c-a26f-6338aa4db22e-0_6-11-48_20220301005056692.parquet
+...
+s3://<table_storage_bucket>/0bfb3d6e/<hudi_table_name>/country=india/9320889c-8537-4aa7-a63e-ef088b9a21ce-0_9-11-51_20220301005056692.parquet
+s3://<table_storage_bucket>/0bfb3d6e/<hudi_table_name>/country=uk/a62aa56b-d55e-4a2b-88a6-d603ef26775c-0_8-11-50_20220301005056692.parquet
+...
+```
+
+For `non-partitioned` tables, this is how it would look:
+```
+s3://<table_storage_bucket>/01f50736/<hudi_table_name>/075f3295-def8-4a42-a927-07fd2dd2976c-0_7-11-49_20220301005056692.parquet
+s3://<table_storage_bucket>/01f50736/<hudi_table_name>/4b0c6b40-2ac0-4a1c-a26f-6338aa4db22e-0_6-11-48_20220301005056692.parquet
+s3://<table_storage_bucket>/0bfb3d6e/<hudi_table_name>/9320889c-8537-4aa7-a63e-ef088b9a21ce-0_9-11-51_20220301005056692.parquet
+s3://<table_storage_bucket>/0bfb3d6e/<hudi_table_name>/a62aa56b-d55e-4a2b-88a6-d603ef26775c-0_8-11-50_20220301005056692.parquet
+...
+```
+
+The original table path will continue to store the `metadata folder` and `partition metadata` files:
+```
+s3://<table_bucket>/<hudi_table_name>/.hoodie/...
+s3://<table_bucket>/<hudi_table_name>/country=usa/.hoodie_partition_metadata
+s3://<table_bucket>/<hudi_table_name>/country=india/.hoodie_partition_metadata
+s3://<table_bucket>/<hudi_table_name>/country=uk/.hoodie_partition_metadata
+...
+```
+
+#### Hashing
+
+To generate the prefixes we can use `Murmur 32 bit` hash on the `File Names`, which is known for being fast and provides
+good distribution guarantees. We might have to further do bucketing and re-hash it to reduce the number of possible hashes
+from 2^32 to a slightly lower number, as it may be overkill to have that many unique hashes, which might result in scenarios
+where each file is under a different prefix.
+
+### Maintain mapping to files
+
+In [RFC-15](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=147427331), we introduced an internal
+Metadata Table with a `files` partition that maintains mapping from partitions to list of files in the partition stored
+under `Table Path`. This mapping is kept up to date, as operations are performed on the original table. We will leverage
+the same to now maintain mappings to files stored at `Table Storage Path` under different prefixes.
+
+Here are some of the design considerations:
+
+1. Metadata table is a pre-requisite and will be automatically enabled if object storage layout is enabled.
+
+2. The object storage layout cannot be enabled on an existing table that is already bootstrapped with Hive storage layout.
+To switch to object storage layout, the table will need to be re-bootstrapped with the new layout.
+
+3. The Instant metadata (`HoodieCommitMetadata`,`HoodieCleanMetadata` etc.) will always act as the source of file listing
+for metadata table to be populated.
+
+4. `HoodieCommitMetadata` currently stores `file name` instead of complete `file path`. We will have to modify commit
+metadata to store the complete file path instead of just file name, as the files are now distributed across several random
+prefix paths instead of a derivable table/partition path.
+
+5. If there is an error reading from Metadata table, we will not fall back listing from file system.
+
+6. In case of metadata table getting corrupted or lost, we need to have a solution here to reconstruct metadata table
+from the files which distributed using object storage layout. We will likely have to implement a file system listing
+logic, that can get all the partition to files mapping by listing all the prefixes under the `Table Storage Path`.
+Following the folder structure of adding table name/partitions under the prefix will help in getting the listing and
+identifying the table/partition they belong to.
+
+### Query Side Integration
+
+Spark, Hive and [Presto](https://github.com/prestodb/presto/commit/ef1fd25c582631513ccdd097e0a654cda44ec3dc) are
+already integrated to use metadata based listing. Hudi connector for [Trino](https://github.com/prestodb/presto/commit/ef1fd25c582631513ccdd097e0a654cda44ec3dc)
+is currently in process of being contributed, which also integrates with metadata based listing. In general, since these
+query engines are able to use Hudi's metadata table there should ideally be no changes required in
+terms of making them work with Object storage layout. Here are some considerations:
+
+- Spark DataSource and Spark SQL queries have been integrated with metadata based listing via the Hudi's custom implementation
+of Spark's FileIndex interface. However, if Spark DataSource queries are used with globbed paths then the FileIndex path
+does not kick in, and it would rely on Spark's `InMemoryFileIndex` to do the file listing with Hudi's path filter applied. Thus,
+these Spark DataSource queries with globbed paths would not work with object storage layout.
+
+- Query engines should be able to determine that Object storage layout is configured, and rely on metadata to list files. It

Review Comment:
   yes we can add a check to fail if storagePath is configured, but metadata table is not



##########
rfc/rfc-48/rfc-48.md:
##########
@@ -0,0 +1,171 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+# RFC-[48]: Optimized storage layout for Cloud Object Stores
+
+## Proposers
+- @umehrot2
+
+## Approvers
+- @vinoth
+- @shivnarayan
+
+## Status
+
+JIRA: [https://issues.apache.org/jira/browse/HUDI-3625](https://issues.apache.org/jira/browse/HUDI-3625)
+
+## Abstract
+
+As you scale your Apache Hudi workloads over Cloud object stores like Amazon S3, there is potential of hitting request
+throttling limits which in-turn impacts performance. In this RFC, we are proposing to support an alternate storage
+layout that is optimized for Amazon S3 and other cloud object stores, which helps achieve maximum throughput and
+significantly reduce throttling.
+
+## Background
+
+Apache Hudi follows the traditional Hive storage layout while writing files on storage:
+- Partitioned Tables: The files are distributed across multiple physical partition folders, under the table's base path.
+- Non Partitioned Tables: The files are stored directly under the table's base path.
+
+While this storage layout scales well for HDFS, it increases the probability of hitting request throttle limits when
+working with cloud object stores like Amazon S3 and others. This is because Amazon S3 and other cloud stores [throttle
+requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/).
+Amazon S3 does scale based on request patterns for different prefixes and adds internal partitions (with their own request limits),
+but there can be a 30 - 60 minute wait time before new partitions are created. Thus, all files/objects stored under the
+same table path prefix could result in these request limits being hit for the table prefix, specially as workloads
+scale, and there are several thousands of files being written/updated concurrently. This hurts performance due to

Review Comment:
   the bigger problem is this thundering herd, bursty puts/gets from engines. 



##########
rfc/rfc-48/rfc-48.md:
##########
@@ -0,0 +1,171 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+# RFC-[48]: Optimized storage layout for Cloud Object Stores
+
+## Proposers
+- @umehrot2
+
+## Approvers
+- @vinoth
+- @shivnarayan
+
+## Status
+
+JIRA: [https://issues.apache.org/jira/browse/HUDI-3625](https://issues.apache.org/jira/browse/HUDI-3625)
+
+## Abstract
+
+As you scale your Apache Hudi workloads over Cloud object stores like Amazon S3, there is potential of hitting request
+throttling limits which in-turn impacts performance. In this RFC, we are proposing to support an alternate storage
+layout that is optimized for Amazon S3 and other cloud object stores, which helps achieve maximum throughput and
+significantly reduce throttling.
+
+## Background
+
+Apache Hudi follows the traditional Hive storage layout while writing files on storage:
+- Partitioned Tables: The files are distributed across multiple physical partition folders, under the table's base path.
+- Non Partitioned Tables: The files are stored directly under the table's base path.
+
+While this storage layout scales well for HDFS, it increases the probability of hitting request throttle limits when
+working with cloud object stores like Amazon S3 and others. This is because Amazon S3 and other cloud stores [throttle
+requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/).
+Amazon S3 does scale based on request patterns for different prefixes and adds internal partitions (with their own request limits),
+but there can be a 30 - 60 minute wait time before new partitions are created. Thus, all files/objects stored under the
+same table path prefix could result in these request limits being hit for the table prefix, specially as workloads
+scale, and there are several thousands of files being written/updated concurrently. This hurts performance due to
+re-trying of failed requests affecting throughput, and result in occasional failures if the retries are not able to
+succeed either and continue to be throttled.
+
+The high level proposal here is to introduce a new storage layout, where all files are distributed evenly across multiple
+randomly generated prefixes under the Amazon S3 bucket, instead of being stored under a common table path/prefix. This
+would help distribute the requests evenly across different prefixes, resulting in Amazon S3 to create partitions for
+the prefixes each with its own request limit. This significantly reduces the possibility of hitting the request limit
+for a specific prefix/partition.
+
+## Design
+
+### Generating file paths
+
+We want to distribute files evenly across multiple random prefixes, instead of following the traditional Hive storage
+layout of keeping them under a common table path/prefix. In addition to the `Table Path`, for this new layout user will
+configure another `Table Storage Path` under which the actual data files will be distributed. The original `Table Path` will
+be used to maintain the table/partitions Hudi metadata.
+
+For the purpose of this documentation lets assume:
+```
+Table Path => s3://<table_bucket>/<hudi_table_name>/
+
+Table Storage Path => s3://<table_storage_bucket>/
+```
+Note: `Table Storage Path` can be a path in the same Amazon S3 bucket or a different bucket. For best results,
+`Table Storage Path` should be a bucket instead of a prefix under the bucket as it allows for S3 to partition sooner.
+
+We will use a Hashing function on the `File Name` to map them to a prefix generated under `Table Storage Path`:
+```
+s3://<table_storage_bucket>/<hash_prefix>/..
+```
+
+In addition, under the hash prefix we will follow a folder structure by appending Hudi Table Name and Partition. This
+folder structuring would be useful if we ever have to do a file system listing to re-create the metadata file list for
+the table (discussed more in the next section). Here is how the final layout would look like for `partitioned` tables:
+```
+s3://<table_storage_bucket>/01f50736/<hudi_table_name>/country=usa/075f3295-def8-4a42-a927-07fd2dd2976c-0_7-11-49_20220301005056692.parquet
+s3://<table_storage_bucket>/01f50736/<hudi_table_name>/country=india/4b0c6b40-2ac0-4a1c-a26f-6338aa4db22e-0_6-11-48_20220301005056692.parquet
+...
+s3://<table_storage_bucket>/0bfb3d6e/<hudi_table_name>/country=india/9320889c-8537-4aa7-a63e-ef088b9a21ce-0_9-11-51_20220301005056692.parquet
+s3://<table_storage_bucket>/0bfb3d6e/<hudi_table_name>/country=uk/a62aa56b-d55e-4a2b-88a6-d603ef26775c-0_8-11-50_20220301005056692.parquet
+...
+```
+
+For `non-partitioned` tables, this is how it would look:
+```
+s3://<table_storage_bucket>/01f50736/<hudi_table_name>/075f3295-def8-4a42-a927-07fd2dd2976c-0_7-11-49_20220301005056692.parquet
+s3://<table_storage_bucket>/01f50736/<hudi_table_name>/4b0c6b40-2ac0-4a1c-a26f-6338aa4db22e-0_6-11-48_20220301005056692.parquet
+s3://<table_storage_bucket>/0bfb3d6e/<hudi_table_name>/9320889c-8537-4aa7-a63e-ef088b9a21ce-0_9-11-51_20220301005056692.parquet
+s3://<table_storage_bucket>/0bfb3d6e/<hudi_table_name>/a62aa56b-d55e-4a2b-88a6-d603ef26775c-0_8-11-50_20220301005056692.parquet
+...
+```
+
+The original table path will continue to store the `metadata folder` and `partition metadata` files:
+```
+s3://<table_bucket>/<hudi_table_name>/.hoodie/...
+s3://<table_bucket>/<hudi_table_name>/country=usa/.hoodie_partition_metadata
+s3://<table_bucket>/<hudi_table_name>/country=india/.hoodie_partition_metadata
+s3://<table_bucket>/<hudi_table_name>/country=uk/.hoodie_partition_metadata
+...
+```
+
+#### Hashing
+
+To generate the prefixes we can use `Murmur 32 bit` hash on the `File Names`, which is known for being fast and provides
+good distribution guarantees. We might have to further do bucketing and re-hash it to reduce the number of possible hashes
+from 2^32 to a slightly lower number, as it may be overkill to have that many unique hashes, which might result in scenarios
+where each file is under a different prefix.
+
+### Maintain mapping to files
+
+In [RFC-15](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=147427331), we introduced an internal
+Metadata Table with a `files` partition that maintains mapping from partitions to list of files in the partition stored
+under `Table Path`. This mapping is kept up to date, as operations are performed on the original table. We will leverage
+the same to now maintain mappings to files stored at `Table Storage Path` under different prefixes.
+
+Here are some of the design considerations:
+
+1. Metadata table is a pre-requisite and will be automatically enabled if object storage layout is enabled.
+
+2. The object storage layout cannot be enabled on an existing table that is already bootstrapped with Hive storage layout.
+To switch to object storage layout, the table will need to be re-bootstrapped with the new layout.
+
+3. The Instant metadata (`HoodieCommitMetadata`,`HoodieCleanMetadata` etc.) will always act as the source of file listing
+for metadata table to be populated.
+
+4. `HoodieCommitMetadata` currently stores `file name` instead of complete `file path`. We will have to modify commit
+metadata to store the complete file path instead of just file name, as the files are now distributed across several random
+prefix paths instead of a derivable table/partition path.
+
+5. If there is an error reading from Metadata table, we will not fall back listing from file system.

Review Comment:
   +1



##########
rfc/rfc-48/rfc-48.md:
##########
@@ -0,0 +1,171 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+# RFC-[48]: Optimized storage layout for Cloud Object Stores
+
+## Proposers
+- @umehrot2
+
+## Approvers
+- @vinoth
+- @shivnarayan
+
+## Status
+
+JIRA: [https://issues.apache.org/jira/browse/HUDI-3625](https://issues.apache.org/jira/browse/HUDI-3625)
+
+## Abstract
+
+As you scale your Apache Hudi workloads over Cloud object stores like Amazon S3, there is potential of hitting request
+throttling limits which in-turn impacts performance. In this RFC, we are proposing to support an alternate storage
+layout that is optimized for Amazon S3 and other cloud object stores, which helps achieve maximum throughput and
+significantly reduce throttling.
+
+## Background
+
+Apache Hudi follows the traditional Hive storage layout while writing files on storage:
+- Partitioned Tables: The files are distributed across multiple physical partition folders, under the table's base path.
+- Non Partitioned Tables: The files are stored directly under the table's base path.
+
+While this storage layout scales well for HDFS, it increases the probability of hitting request throttle limits when
+working with cloud object stores like Amazon S3 and others. This is because Amazon S3 and other cloud stores [throttle
+requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/).
+Amazon S3 does scale based on request patterns for different prefixes and adds internal partitions (with their own request limits),
+but there can be a 30 - 60 minute wait time before new partitions are created. Thus, all files/objects stored under the
+same table path prefix could result in these request limits being hit for the table prefix, specially as workloads
+scale, and there are several thousands of files being written/updated concurrently. This hurts performance due to
+re-trying of failed requests affecting throughput, and result in occasional failures if the retries are not able to
+succeed either and continue to be throttled.
+
+The high level proposal here is to introduce a new storage layout, where all files are distributed evenly across multiple
+randomly generated prefixes under the Amazon S3 bucket, instead of being stored under a common table path/prefix. This
+would help distribute the requests evenly across different prefixes, resulting in Amazon S3 to create partitions for
+the prefixes each with its own request limit. This significantly reduces the possibility of hitting the request limit
+for a specific prefix/partition.
+
+## Design
+
+### Generating file paths
+
+We want to distribute files evenly across multiple random prefixes, instead of following the traditional Hive storage
+layout of keeping them under a common table path/prefix. In addition to the `Table Path`, for this new layout user will
+configure another `Table Storage Path` under which the actual data files will be distributed. The original `Table Path` will
+be used to maintain the table/partitions Hudi metadata.
+
+For the purpose of this documentation lets assume:
+```
+Table Path => s3://<table_bucket>/<hudi_table_name>/
+
+Table Storage Path => s3://<table_storage_bucket>/
+```
+Note: `Table Storage Path` can be a path in the same Amazon S3 bucket or a different bucket. For best results,
+`Table Storage Path` should be a bucket instead of a prefix under the bucket as it allows for S3 to partition sooner.
+
+We will use a Hashing function on the `File Name` to map them to a prefix generated under `Table Storage Path`:
+```
+s3://<table_storage_bucket>/<hash_prefix>/..
+```
+
+In addition, under the hash prefix we will follow a folder structure by appending Hudi Table Name and Partition. This
+folder structuring would be useful if we ever have to do a file system listing to re-create the metadata file list for
+the table (discussed more in the next section). Here is how the final layout would look like for `partitioned` tables:
+```
+s3://<table_storage_bucket>/01f50736/<hudi_table_name>/country=usa/075f3295-def8-4a42-a927-07fd2dd2976c-0_7-11-49_20220301005056692.parquet
+s3://<table_storage_bucket>/01f50736/<hudi_table_name>/country=india/4b0c6b40-2ac0-4a1c-a26f-6338aa4db22e-0_6-11-48_20220301005056692.parquet
+...
+s3://<table_storage_bucket>/0bfb3d6e/<hudi_table_name>/country=india/9320889c-8537-4aa7-a63e-ef088b9a21ce-0_9-11-51_20220301005056692.parquet
+s3://<table_storage_bucket>/0bfb3d6e/<hudi_table_name>/country=uk/a62aa56b-d55e-4a2b-88a6-d603ef26775c-0_8-11-50_20220301005056692.parquet
+...
+```
+
+For `non-partitioned` tables, this is how it would look:
+```
+s3://<table_storage_bucket>/01f50736/<hudi_table_name>/075f3295-def8-4a42-a927-07fd2dd2976c-0_7-11-49_20220301005056692.parquet
+s3://<table_storage_bucket>/01f50736/<hudi_table_name>/4b0c6b40-2ac0-4a1c-a26f-6338aa4db22e-0_6-11-48_20220301005056692.parquet
+s3://<table_storage_bucket>/0bfb3d6e/<hudi_table_name>/9320889c-8537-4aa7-a63e-ef088b9a21ce-0_9-11-51_20220301005056692.parquet
+s3://<table_storage_bucket>/0bfb3d6e/<hudi_table_name>/a62aa56b-d55e-4a2b-88a6-d603ef26775c-0_8-11-50_20220301005056692.parquet
+...
+```
+
+The original table path will continue to store the `metadata folder` and `partition metadata` files:
+```
+s3://<table_bucket>/<hudi_table_name>/.hoodie/...
+s3://<table_bucket>/<hudi_table_name>/country=usa/.hoodie_partition_metadata
+s3://<table_bucket>/<hudi_table_name>/country=india/.hoodie_partition_metadata
+s3://<table_bucket>/<hudi_table_name>/country=uk/.hoodie_partition_metadata
+...
+```
+
+#### Hashing
+
+To generate the prefixes we can use `Murmur 32 bit` hash on the `File Names`, which is known for being fast and provides
+good distribution guarantees. We might have to further do bucketing and re-hash it to reduce the number of possible hashes
+from 2^32 to a slightly lower number, as it may be overkill to have that many unique hashes, which might result in scenarios
+where each file is under a different prefix.
+
+### Maintain mapping to files
+
+In [RFC-15](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=147427331), we introduced an internal
+Metadata Table with a `files` partition that maintains mapping from partitions to list of files in the partition stored
+under `Table Path`. This mapping is kept up to date, as operations are performed on the original table. We will leverage
+the same to now maintain mappings to files stored at `Table Storage Path` under different prefixes.
+
+Here are some of the design considerations:
+
+1. Metadata table is a pre-requisite and will be automatically enabled if object storage layout is enabled.
+
+2. The object storage layout cannot be enabled on an existing table that is already bootstrapped with Hive storage layout.
+To switch to object storage layout, the table will need to be re-bootstrapped with the new layout.
+
+3. The Instant metadata (`HoodieCommitMetadata`,`HoodieCleanMetadata` etc.) will always act as the source of file listing
+for metadata table to be populated.
+
+4. `HoodieCommitMetadata` currently stores `file name` instead of complete `file path`. We will have to modify commit
+metadata to store the complete file path instead of just file name, as the files are now distributed across several random
+prefix paths instead of a derivable table/partition path.
+
+5. If there is an error reading from Metadata table, we will not fall back listing from file system.
+
+6. In case of metadata table getting corrupted or lost, we need to have a solution here to reconstruct metadata table
+from the files which distributed using object storage layout. We will likely have to implement a file system listing
+logic, that can get all the partition to files mapping by listing all the prefixes under the `Table Storage Path`.
+Following the folder structure of adding table name/partitions under the prefix will help in getting the listing and
+identifying the table/partition they belong to.
+
+### Query Side Integration

Review Comment:
   Thinking out loud, the main thing here is - even files under the same partition will not be spread across different prefixes. I think all the fileSystemView methods that fetch file slices for a given partition, can be made to work as-is. 
   Something to watch for is code that assumes all files under a logical partition now lives the same folder i.e using `partitionPath + fileName` as the path. Using `BaseFile#fullPath`  consistently should help address this



##########
rfc/rfc-48/rfc-48.md:
##########
@@ -0,0 +1,171 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+# RFC-[48]: Optimized storage layout for Cloud Object Stores
+
+## Proposers
+- @umehrot2
+
+## Approvers
+- @vinoth
+- @shivnarayan
+
+## Status
+
+JIRA: [https://issues.apache.org/jira/browse/HUDI-3625](https://issues.apache.org/jira/browse/HUDI-3625)
+
+## Abstract
+
+As you scale your Apache Hudi workloads over Cloud object stores like Amazon S3, there is potential of hitting request
+throttling limits which in-turn impacts performance. In this RFC, we are proposing to support an alternate storage
+layout that is optimized for Amazon S3 and other cloud object stores, which helps achieve maximum throughput and
+significantly reduce throttling.
+
+## Background
+
+Apache Hudi follows the traditional Hive storage layout while writing files on storage:
+- Partitioned Tables: The files are distributed across multiple physical partition folders, under the table's base path.
+- Non Partitioned Tables: The files are stored directly under the table's base path.
+
+While this storage layout scales well for HDFS, it increases the probability of hitting request throttle limits when
+working with cloud object stores like Amazon S3 and others. This is because Amazon S3 and other cloud stores [throttle
+requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/).
+Amazon S3 does scale based on request patterns for different prefixes and adds internal partitions (with their own request limits),
+but there can be a 30 - 60 minute wait time before new partitions are created. Thus, all files/objects stored under the

Review Comment:
   Good explanation. Looks like, in practice this is a soft limit. but this 60 min internal replication period can be very disruptive for big tables. 
   
   The other thing I would like to point out here. Hudi writes out well sized files. Assuming 100 MB files, read out fully in 10 seconds, someone must be reading 60 GB/sec or something in aggregate from a bucket to hit throttles for gets. 



##########
rfc/rfc-48/rfc-48.md:
##########
@@ -0,0 +1,171 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+# RFC-[48]: Optimized storage layout for Cloud Object Stores
+
+## Proposers
+- @umehrot2
+
+## Approvers
+- @vinoth
+- @shivnarayan
+
+## Status
+
+JIRA: [https://issues.apache.org/jira/browse/HUDI-3625](https://issues.apache.org/jira/browse/HUDI-3625)
+
+## Abstract
+
+As you scale your Apache Hudi workloads over Cloud object stores like Amazon S3, there is potential of hitting request
+throttling limits which in-turn impacts performance. In this RFC, we are proposing to support an alternate storage
+layout that is optimized for Amazon S3 and other cloud object stores, which helps achieve maximum throughput and
+significantly reduce throttling.
+
+## Background
+
+Apache Hudi follows the traditional Hive storage layout while writing files on storage:
+- Partitioned Tables: The files are distributed across multiple physical partition folders, under the table's base path.
+- Non Partitioned Tables: The files are stored directly under the table's base path.
+
+While this storage layout scales well for HDFS, it increases the probability of hitting request throttle limits when
+working with cloud object stores like Amazon S3 and others. This is because Amazon S3 and other cloud stores [throttle

Review Comment:
   do other cloud storage throttle as well?



##########
rfc/rfc-48/rfc-48.md:
##########
@@ -0,0 +1,171 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+# RFC-[48]: Optimized storage layout for Cloud Object Stores
+
+## Proposers
+- @umehrot2
+
+## Approvers
+- @vinoth
+- @shivnarayan
+
+## Status
+
+JIRA: [https://issues.apache.org/jira/browse/HUDI-3625](https://issues.apache.org/jira/browse/HUDI-3625)
+
+## Abstract
+
+As you scale your Apache Hudi workloads over Cloud object stores like Amazon S3, there is potential of hitting request
+throttling limits which in-turn impacts performance. In this RFC, we are proposing to support an alternate storage
+layout that is optimized for Amazon S3 and other cloud object stores, which helps achieve maximum throughput and
+significantly reduce throttling.
+
+## Background
+
+Apache Hudi follows the traditional Hive storage layout while writing files on storage:
+- Partitioned Tables: The files are distributed across multiple physical partition folders, under the table's base path.
+- Non Partitioned Tables: The files are stored directly under the table's base path.
+
+While this storage layout scales well for HDFS, it increases the probability of hitting request throttle limits when
+working with cloud object stores like Amazon S3 and others. This is because Amazon S3 and other cloud stores [throttle
+requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/).
+Amazon S3 does scale based on request patterns for different prefixes and adds internal partitions (with their own request limits),
+but there can be a 30 - 60 minute wait time before new partitions are created. Thus, all files/objects stored under the
+same table path prefix could result in these request limits being hit for the table prefix, specially as workloads
+scale, and there are several thousands of files being written/updated concurrently. This hurts performance due to
+re-trying of failed requests affecting throughput, and result in occasional failures if the retries are not able to
+succeed either and continue to be throttled.
+
+The high level proposal here is to introduce a new storage layout, where all files are distributed evenly across multiple
+randomly generated prefixes under the Amazon S3 bucket, instead of being stored under a common table path/prefix. This
+would help distribute the requests evenly across different prefixes, resulting in Amazon S3 to create partitions for
+the prefixes each with its own request limit. This significantly reduces the possibility of hitting the request limit
+for a specific prefix/partition.
+
+## Design
+
+### Generating file paths
+
+We want to distribute files evenly across multiple random prefixes, instead of following the traditional Hive storage
+layout of keeping them under a common table path/prefix. In addition to the `Table Path`, for this new layout user will
+configure another `Table Storage Path` under which the actual data files will be distributed. The original `Table Path` will
+be used to maintain the table/partitions Hudi metadata.
+
+For the purpose of this documentation lets assume:
+```
+Table Path => s3://<table_bucket>/<hudi_table_name>/
+
+Table Storage Path => s3://<table_storage_bucket>/
+```
+Note: `Table Storage Path` can be a path in the same Amazon S3 bucket or a different bucket. For best results,
+`Table Storage Path` should be a bucket instead of a prefix under the bucket as it allows for S3 to partition sooner.
+
+We will use a Hashing function on the `File Name` to map them to a prefix generated under `Table Storage Path`:
+```
+s3://<table_storage_bucket>/<hash_prefix>/..
+```
+
+In addition, under the hash prefix we will follow a folder structure by appending Hudi Table Name and Partition. This
+folder structuring would be useful if we ever have to do a file system listing to re-create the metadata file list for
+the table (discussed more in the next section). Here is how the final layout would look like for `partitioned` tables:
+```
+s3://<table_storage_bucket>/01f50736/<hudi_table_name>/country=usa/075f3295-def8-4a42-a927-07fd2dd2976c-0_7-11-49_20220301005056692.parquet
+s3://<table_storage_bucket>/01f50736/<hudi_table_name>/country=india/4b0c6b40-2ac0-4a1c-a26f-6338aa4db22e-0_6-11-48_20220301005056692.parquet
+...
+s3://<table_storage_bucket>/0bfb3d6e/<hudi_table_name>/country=india/9320889c-8537-4aa7-a63e-ef088b9a21ce-0_9-11-51_20220301005056692.parquet
+s3://<table_storage_bucket>/0bfb3d6e/<hudi_table_name>/country=uk/a62aa56b-d55e-4a2b-88a6-d603ef26775c-0_8-11-50_20220301005056692.parquet
+...
+```
+
+For `non-partitioned` tables, this is how it would look:
+```
+s3://<table_storage_bucket>/01f50736/<hudi_table_name>/075f3295-def8-4a42-a927-07fd2dd2976c-0_7-11-49_20220301005056692.parquet
+s3://<table_storage_bucket>/01f50736/<hudi_table_name>/4b0c6b40-2ac0-4a1c-a26f-6338aa4db22e-0_6-11-48_20220301005056692.parquet
+s3://<table_storage_bucket>/0bfb3d6e/<hudi_table_name>/9320889c-8537-4aa7-a63e-ef088b9a21ce-0_9-11-51_20220301005056692.parquet
+s3://<table_storage_bucket>/0bfb3d6e/<hudi_table_name>/a62aa56b-d55e-4a2b-88a6-d603ef26775c-0_8-11-50_20220301005056692.parquet
+...
+```
+
+The original table path will continue to store the `metadata folder` and `partition metadata` files:
+```
+s3://<table_bucket>/<hudi_table_name>/.hoodie/...
+s3://<table_bucket>/<hudi_table_name>/country=usa/.hoodie_partition_metadata
+s3://<table_bucket>/<hudi_table_name>/country=india/.hoodie_partition_metadata
+s3://<table_bucket>/<hudi_table_name>/country=uk/.hoodie_partition_metadata
+...
+```
+
+#### Hashing
+
+To generate the prefixes we can use `Murmur 32 bit` hash on the `File Names`, which is known for being fast and provides
+good distribution guarantees. We might have to further do bucketing and re-hash it to reduce the number of possible hashes
+from 2^32 to a slightly lower number, as it may be overkill to have that many unique hashes, which might result in scenarios
+where each file is under a different prefix.
+
+### Maintain mapping to files
+
+In [RFC-15](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=147427331), we introduced an internal
+Metadata Table with a `files` partition that maintains mapping from partitions to list of files in the partition stored
+under `Table Path`. This mapping is kept up to date, as operations are performed on the original table. We will leverage
+the same to now maintain mappings to files stored at `Table Storage Path` under different prefixes.
+
+Here are some of the design considerations:
+
+1. Metadata table is a pre-requisite and will be automatically enabled if object storage layout is enabled.
+
+2. The object storage layout cannot be enabled on an existing table that is already bootstrapped with Hive storage layout.

Review Comment:
   I think its a fair call for the initial cut



##########
rfc/rfc-48/rfc-48.md:
##########
@@ -0,0 +1,171 @@
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+# RFC-[48]: Optimized storage layout for Cloud Object Stores
+
+## Proposers
+- @umehrot2
+
+## Approvers
+- @vinoth
+- @shivnarayan
+
+## Status
+
+JIRA: [https://issues.apache.org/jira/browse/HUDI-3625](https://issues.apache.org/jira/browse/HUDI-3625)
+
+## Abstract
+
+As you scale your Apache Hudi workloads over Cloud object stores like Amazon S3, there is potential of hitting request
+throttling limits which in-turn impacts performance. In this RFC, we are proposing to support an alternate storage
+layout that is optimized for Amazon S3 and other cloud object stores, which helps achieve maximum throughput and
+significantly reduce throttling.
+
+## Background
+
+Apache Hudi follows the traditional Hive storage layout while writing files on storage:
+- Partitioned Tables: The files are distributed across multiple physical partition folders, under the table's base path.
+- Non Partitioned Tables: The files are stored directly under the table's base path.
+
+While this storage layout scales well for HDFS, it increases the probability of hitting request throttle limits when
+working with cloud object stores like Amazon S3 and others. This is because Amazon S3 and other cloud stores [throttle
+requests based on object prefix](https://aws.amazon.com/premiumsupport/knowledge-center/s3-request-limit-avoid-throttling/).
+Amazon S3 does scale based on request patterns for different prefixes and adds internal partitions (with their own request limits),
+but there can be a 30 - 60 minute wait time before new partitions are created. Thus, all files/objects stored under the
+same table path prefix could result in these request limits being hit for the table prefix, specially as workloads
+scale, and there are several thousands of files being written/updated concurrently. This hurts performance due to
+re-trying of failed requests affecting throughput, and result in occasional failures if the retries are not able to
+succeed either and continue to be throttled.
+
+The high level proposal here is to introduce a new storage layout, where all files are distributed evenly across multiple
+randomly generated prefixes under the Amazon S3 bucket, instead of being stored under a common table path/prefix. This
+would help distribute the requests evenly across different prefixes, resulting in Amazon S3 to create partitions for
+the prefixes each with its own request limit. This significantly reduces the possibility of hitting the request limit
+for a specific prefix/partition.
+
+## Design
+
+### Generating file paths
+
+We want to distribute files evenly across multiple random prefixes, instead of following the traditional Hive storage
+layout of keeping them under a common table path/prefix. In addition to the `Table Path`, for this new layout user will
+configure another `Table Storage Path` under which the actual data files will be distributed. The original `Table Path` will
+be used to maintain the table/partitions Hudi metadata.
+
+For the purpose of this documentation lets assume:
+```
+Table Path => s3://<table_bucket>/<hudi_table_name>/
+
+Table Storage Path => s3://<table_storage_bucket>/
+```
+Note: `Table Storage Path` can be a path in the same Amazon S3 bucket or a different bucket. For best results,
+`Table Storage Path` should be a bucket instead of a prefix under the bucket as it allows for S3 to partition sooner.
+
+We will use a Hashing function on the `File Name` to map them to a prefix generated under `Table Storage Path`:
+```
+s3://<table_storage_bucket>/<hash_prefix>/..
+```
+
+In addition, under the hash prefix we will follow a folder structure by appending Hudi Table Name and Partition. This
+folder structuring would be useful if we ever have to do a file system listing to re-create the metadata file list for
+the table (discussed more in the next section). Here is how the final layout would look like for `partitioned` tables:
+```
+s3://<table_storage_bucket>/01f50736/<hudi_table_name>/country=usa/075f3295-def8-4a42-a927-07fd2dd2976c-0_7-11-49_20220301005056692.parquet

Review Comment:
   what about log files? Should keep it under the same prefix? Can we write the RFC/code with also MOR considered from the start



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org