You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by "stream2000 (via GitHub)" <gi...@apache.org> on 2023/02/27 11:09:47 UTC

[GitHub] [hudi] stream2000 opened a new pull request, #8062: [HUDI-5823] RFC for Partition TTL Management

stream2000 opened a new pull request, #8062:
URL: https://github.com/apache/hudi/pull/8062

   ### Change Logs
   
   RFC-65  Partition TTL Management
   
   ### Impact
   
   None
   
   ### Risk level (write none, low medium or high below)
   
   Low
   
   ### Documentation Update
   
   None
   
   ### Contributor's checklist
   
   - [ ] Read through [contributor's guide](https://hudi.apache.org/contribute/how-to-contribute)
   - [ ] Change Logs and Impact were stated clearly
   - [ ] Adequate tests were added if applicable
   - [ ] CI passed
   


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


[GitHub] [hudi] stream2000 commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "stream2000 (via GitHub)" <gi...@apache.org>.
stream2000 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1209106782


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.
+    2. **KEEP_BY_COUNT**. Keep N sub-partitions for a  high-level partition. When sub partition count exceeds, delete the partitions with smaller partition values until the sub-partition count meets the policy configuration.
+    3. **KEEP_BY_SIZE**. Similar to KEEP_BY_COUNT, but to ensure that the sum of the data size of all sub-partitions does not exceed the policy configuration.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned by two fields (user_id, ts). For partition(user_id='1'), we set the policy to keep 100G data for all sub-partitions, and for partition(user_id='2') we set the policy that all sub-partitions will expire 10 days after their last modified time.
+3. It's possible that there are a lot of high-level partitions in the user's table,  and they don't want to set TTL policies for all the high-level partitions. So we need to provide a default policy mechanism so that users can set a default policy for all high-level partitions and add some explicit policies for some of them if needed. Explicit policies will override the default policy.
+
+So here we have the TTL policy definition:
+```java
+public class HoodiePartitionTTLPolicy {
+  public enum TTLPolicy {
+    KEEP_BY_TIME, KEEP_BY_SIZE, KEEP_BY_COUNT
+  }
+
+  // Partition spec for which the policy takes effect
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.  Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as follows.
+
+```sql
+-- Set default policy for all user_id, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');
+ 
+--For partition user_id=1/, keep 10 sub partitions.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=1/', policy => 'KEEP_BY_COUNT', policyValue => '10');
+
+--For partition user_id=2/, keep 100GB data in total
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=2/', policy => 'KEEP_BY_SIZE', policyValue => '107374182400');
+
+--For partition user_id=3/, keep the data for 7 day.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=3/', policy => 'KEEP_BY_TIME', policyValue => '7');
+
+-- Show all the TTL policies including default and explicit policies
+call show_ttl_policies(table => 'test');
+user_id=*/	KEEP_BY_TIME	30
+user_id=1/	KEEP_BY_COUNT	10
+user_id=2/	KEEP_BY_SIZE	107374182400
+user_id=3/	KEEP_BY_TIME	7
+```
+
+### Storage for TTL policy
+The partition TTL policies will be stored in `hoodie.properties`since it is part of table metadata. The policy configs in `hoodie.properties`are defined as follows. Explicit policies are defined using a JSON array while default policy is defined using separate configs.

Review Comment:
   I agree that we can treat the TTL policy as write configs, but in many cases, we need to persistent the TTL policy config otherwise the users may need to store the policy by themselves. Also, without persisting the TTL config we can't share the config in multiple processes like one for gathering metrics while the other is for doing the ttl management. 
   
   So I think the TTL config can also be treated as a table config like the secondary index config? We can store the TTL in  table config or somewhere else but maybe hoodie table config is the better way.. Looking forward for your opinion. 
   
   
   
   
   
   



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


[GitHub] [hudi] stream2000 commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "stream2000 (via GitHub)" <gi...@apache.org>.
stream2000 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1211260244


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.

Review Comment:
   Maybe we can add the stash/restore mechanism to replace commit/clean process of hudi instead of dealing with it in TTL management? TTL management should only decide which partitions are outdated and call `delete_partition` to delete them. If we want to retain the deleted data we can add extra mechanism in the `delete_parrtition` method. 



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "stream2000 (via GitHub)" <gi...@apache.org>.
stream2000 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1399985395


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,248 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a lifecycle management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces partition lifecycle management strategies to hudi, people can config the strategies by write
+configs. With proper configs set, Hudi can find out which partitions are expired and delete them.
+
+This proposal introduces partition lifecycle management service to hudi. Lifecycle management is like other table
+services such as Clean/Compaction/Clustering.
+Users can config their partition lifecycle management strategies through write configs and Hudi will help users find
+expired partitions and delete them automatically.
+
+## Background
+
+Lifecycle management mechanism is an important feature for databases. Hudi already provides a `delete_partition`
+interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of partition lifecycle
+management strategies, find expired partitions and call `delete_partition` by themselves. As the scale of installations
+grew, it is becoming increasingly important to implement a user-friendly lifecycle management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition lifecycle management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or
+  asynchronous table services.
+
+### Strategy Definition
+
+The lifecycle strategies is similar to existing table service strategies. We can define lifecycle strategies like
+defining a clustering/clean/compaction strategy:
+
+```properties
+hoodie.partition.lifecycle.management.strategy=KEEP_BY_TIME
+hoodie.partition.lifecycle.management.strategy.class=org.apache.hudi.table.action.lifecycle.strategy.KeepByTimePartitionLifecycleManagementStrategy
+hoodie.partition.lifecycle.days.retain=10
+```
+
+The config `hoodie.partition.lifecycle.management.strategy.class` is to provide a strategy class (subclass
+of `PartitionLifecycleManagementStrategy`) to get expired partition paths to delete.
+And `hoodie.partition.lifecycle.days.retain` is the strategy value used
+by `KeepByTimePartitionLifecycleManagementStrategy` which means that we will expire partitions that haven't been
+modified for this strategy value set. We will cover the `KeepByTimePartitionLifecycleManagementStrategy` strategy in
+detail in the next section.
+
+The core definition of `PartitionLifecycleManagementStrategy` looks like this:
+
+```java
+/**
+ * Strategy for partition-level lifecycle management.
+ */
+public abstract class PartitionLifecycleManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition lifecycle management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionLifecycleManagementStrategy` and hudi will help delete the
+expired partitions.
+
+### KeepByTimePartitionLifecycleManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionLifecycleManagementStrategy` in the first version of partition
+lifecycle management implementation.
+
+The `KeepByTimePartitionLifecycleManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If
+duration between now and 'lastModifiedTime' for the partition is larger than
+what `hoodie.partition.lifecycle.days.retain` configured, `KeepByTimePartitionLifecycleManagementStrategy` will mark
+this partition as an expired partition. We use day as the unit of expired time since it is very common-used for
+datalakes. Open to ideas for this.
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the
+partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However,
+we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy.
+And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example
+using metadata table.
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may
+have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for
+partitions under `product=2` they want to keep for 7 days only.
+
+For the first version of partition lifecycle management, we do not plan to implement a complicated strategy (For
+example, use an array to store strategies, introduce partition regex etc.). Instead, we add a new abstract
+method `getPartitionPathsForLifecycleManagement` in `PartitionLifecycleManagementStrategy` and provides a new
+config `hoodie.partition.lifecycle.management.partition.selected`.
+
+If `hoodie.partition.lifecycle.management.partition.selected` is set, `getPartitionPathsForLifecycleManagement` will
+return partitions provided by this config. If not, `getPartitionPathsForLifecycleManagement` will return all partitions
+in the hudi table.
+
+lifecycle management strategies will only be applied for partitions return by `getPartitionPathsForLifecycleManagement`.
+
+Thus, if users want to apply different strategies for different partitions, they can do the partition lifecycle
+management multiple times, selecting different partitions and apply corresponding strategies. And we may provide a batch
+interface in the future to simplify this.
+
+The `getPartitionPathsForLifecycleManagement` method will look like this:
+
+```java
+/**
+ * Strategy for partition-level lifecycle management.
+ */
+public abstract class PartitionLifecycleManagementStrategy {
+  /**
+   * Scan and list all partitions for partition lifecycle management.
+   *
+   * @return Partitions to apply lifecycle management strategy
+   */
+  protected List<String> getPartitionPathsForLifecycleManagement() {
+    if (StringUtils.isNullOrEmpty(config.getLifecycleManagementPartitionSelected())) {
+      return getMatchedPartitions();
+    } else {
+      // Return All partition paths
+      return FSUtils.getAllPartitionPaths(hoodieTable.getContext(), config.getMetadataConfig(), config.getBasePath());
+    }
+  }
+}
+``` 
+
+### Executing partition lifecycle management
+
+Once we already have a proper `PartitionLifecycleManagementStrategy` implementation, it's easy to execute the partition
+lifecycle management.
+
+```java
+public class SparkPartitionLifecycleManagementActionExecutor<T> extends BaseSparkCommitActionExecutor<T> {
+  @Override
+  public HoodieWriteMetadata<HoodieData<WriteStatus>> execute() {
+    // Construct PartitionLifecycleManagementStrategy
+    PartitionLifecycleManagementStrategy strategy = (PartitionLifecycleManagementStrategy) ReflectionUtils.loadClass(
+        PartitionLifecycleManagementStrategy.checkAndGetPartitionLifecycleManagementStrategy(config),
+        new Class<?>[] {HoodieTable.class, HoodieEngineContext.class, HoodieWriteConfig.class}, table, context, config);
+
+    // Get expired partition paths
+    List<String> expiredPartitions = strategy.getExpiredPartitionPaths();
+
+    // Delete them reusing SparkDeletePartitionCommitActionExecutor
+    return new SparkDeletePartitionCommitActionExecutor<>(context, config, table, instantTime,
+        expiredPartitions).execute();
+  }
+}
+```
+
+We will add a new method `managePartitionLifecycle` in `HoodieTable` and `HoodieSparkCopyOnWriteTable` can implement it
+like this:
+
+```java
+@Override
+public HoodieWriteMetadata<HoodieData<WriteStatus>>managePartitionLifecycle(HoodieEngineContext context,String instantTime){
+    return new SparkPartitionLifecycleManagementActionExecutor<>(context,config,this,instantTime).execute();
+    }
+```
+
+We can call `hoodieTable.managePartitionLifecycle` in independent flink/spark job, in async/sync inline table services
+like clustering/compaction/clean etc.
+
+### User interface for Partition lifecycle Management
+
+We can do partition lifecycle management inline with streaming ingestion job or do it with a independent batch job, for
+both spark and flink engine.
+
+#### Run inline with Streaming Ingestion
+
+Since we can run clustering inline with streaming ingestion job through the following config:
+
+```properties
+hoodie.clustering.async.enabled=true
+hoodie.clustering.async.max.commits=5
+```
+
+We can do similar thing for partition lifecycle management. The config for async partition lifecycle management are:
+
+| Config key                                              | Remarks                                                                                                                        | Default |
+|---------------------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------|---------|
+| hoodie.partition.lifecycle.management.async.enabled     | Enable running of lifecycle management service, asynchronously as writes happen on the table.                                  | False   |

Review Comment:
   Yes, we will use `delete partiton` command to delete the expired partitions, this is mentioned in `Background` section. And conflict resolution for the operation is the same with what we do in `delete partiton` command. 



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,248 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a lifecycle management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces partition lifecycle management strategies to hudi, people can config the strategies by write
+configs. With proper configs set, Hudi can find out which partitions are expired and delete them.
+
+This proposal introduces partition lifecycle management service to hudi. Lifecycle management is like other table
+services such as Clean/Compaction/Clustering.
+Users can config their partition lifecycle management strategies through write configs and Hudi will help users find
+expired partitions and delete them automatically.
+
+## Background
+
+Lifecycle management mechanism is an important feature for databases. Hudi already provides a `delete_partition`
+interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of partition lifecycle
+management strategies, find expired partitions and call `delete_partition` by themselves. As the scale of installations
+grew, it is becoming increasingly important to implement a user-friendly lifecycle management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition lifecycle management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or
+  asynchronous table services.
+
+### Strategy Definition
+
+The lifecycle strategies is similar to existing table service strategies. We can define lifecycle strategies like
+defining a clustering/clean/compaction strategy:
+
+```properties
+hoodie.partition.lifecycle.management.strategy=KEEP_BY_TIME
+hoodie.partition.lifecycle.management.strategy.class=org.apache.hudi.table.action.lifecycle.strategy.KeepByTimePartitionLifecycleManagementStrategy

Review Comment:
   Use the word `ttl` means that we can't apply time unrelative partition deletion strategy in the future. However it is more suitable for current implementation since we only provide a time based strategy. What do you think? Should we use ttl directly or remain some extensibility? 



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1413454783


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 

Review Comment:
   > We need to care about policies defined in different level, for example we have a,b,c,d as partition fields, can we define ttl policies in any level for the partition?
   
   If partition suitable for the specified policy spec then check `lastUpdateTime`, no matter on which level this partition is. I suppose it is more clear behavior.



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


[GitHub] [hudi] stream2000 commented on pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "stream2000 (via GitHub)" <gi...@apache.org>.
stream2000 commented on PR #8062:
URL: https://github.com/apache/hudi/pull/8062#issuecomment-1566798066

   @codope @nsivabalan @SteNicholas @nbalajee Sorry for the delayed response, I've been busy with other things these days.  Will address comments and update the RFC. 


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


[GitHub] [hudi] SteNicholas commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "SteNicholas (via GitHub)" <gi...@apache.org>.
SteNicholas commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1211584715


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,163 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and
+maintain proper statistics to find expired partitions by themselves. As the scale of installations grew, it's more
+important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Gathering proper partition statistics for TTL management
+- Executing TTL management
+
+### TTL Policy Definition
+
+We have four main considerations when designing TTL policy:
+
+1. Usually user hopes to manage partition TTL by expired time, so we support the policy `KEEP_BY_TIME` that partitions
+   will expire N days after their last modified time. Note that the last modified time here means any inserts/updates to
+   the partition. Also, we will make it easy to extend new policies so that users can implement their own polices to
+   decide which partitions are expired.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned
+   by `user_id`. For partition(user_id='1'), we set the policy that this partition will expire 7 days after its last
+   modified time, while for
+   partition(user_id='2') we can set the policy that it will expire 30 days after its last modified time.
+3. It's possible that users have multi-fields partitioning, and they don't want set policy for all partition. So we need
+   to support partition regex when defining
+   partition TTL policies. For example, for a hudi table partitioned by (user_id, ts), we can first set a default
+   policy(which policy value contains `*`)
+   that for all partitions matched (user_id=*/) will expire in 7 days, and explicitly set a policy that for partition
+   matched user_id=3 will expire in 30 days. Explicit policy will override default polices, which means default policies
+   only takes effects on partitions that do not match any explicit ttl policies.
+4. We may need to support record level ttl policy in the future. So partition TTL policy should be an implementation of
+   HoodieTTLPolicy.
+
+So we have the TTL policy definition like this (may change when implementing):
+
+```java
+public class HoodiePartitionTTLPolicy implements HoodieTTLPolicy {
+  public enum TTLPolicy {
+    // We supports keep by last modified time at the first version
+    KEEP_BY_TIME
+  }
+
+  // Partition spec for which the policy takes effect, could be a regex or a static partition value
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy Definition
+
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.
+Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as
+follows.
+
+```sparksql
+-- Set policy for all user_id using partition regex, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');
+
+--For partition user_id=3/, keep the data for 7 day.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=3/', policy => 'KEEP_BY_TIME', policyValue => '7');
+
+-- Show all the TTL policies including default and explicit policies
+call show_ttl_policies(table => 'test');
+user_id=*/	KEEP_BY_TIME	30
+user_id=3/	KEEP_BY_TIME	7
+```
+
+### Storage for TTL policy
+
+We need persistent partition ttt policies in hudi table config and users should interact with hudi only by
+setting/getting ttl policies. Hudi then will takes care of doing the ttl management in an async table service.
+
+The partition TTL policies will be stored in `hoodie.properties` and they are defined as follows. Explicit policies are
+defined using a JSON array, note we support regex here.
+
+```properties
+# TTL policy definition
+hoodie.partition.ttl.policies=[{"partitionSpec"\:"user_id\=*/","policy"\:"KEEP_BY_TIME","policyValue"\:7},{"partitionSpec"\:"user_id\=3/","policy"\:"KEEP_BY_TIME","policyValue"\:30}]
+```
+
+### Partition Statistics
+
+We need to gather proper partition statistics for different kinds of partition ttl policies. Specifically, for
+KEEP_BY_TIME policy that we want to support first, we need to know the last modified time for every partition in hudi
+table.
+
+To simplify the design, we decide to use the largest commit time of committed file groups in the partition as partition
+last modified time. But for file groups generated by replace commit, it may not reveal the real insert/update time for
+the file group. We can also introduce other mechanism to get last modified time of the partition, like add a new kind of
+partition metadata or add a new field in metadata table. Open to ideas for this design choice.
+
+### Executing TTL management

Review Comment:
   What does TTL management include? IMO, TTL management includes the ttl policy management like add, update, delete and query ttl policy, and the ttl policy execution.



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


[GitHub] [hudi] geserdugarov commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1334923947


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 

Review Comment:
   We can allow user to write specification,
   `spec = "*/product=1/*", TTL = 30 DAYS`
   `spec = "*/product=2/*, TTL = 7 DAYS`
   It also allows to set default TTL by:
   `spec = "*", TTL = 2 MONTHS`
   
   In this case we will have multiple policies for some partitons. To resolve conflict automatically, we could add  `hoodie.ttl.policies.conflict.resolution.rule` property with available options:
   `MIN_TTL` - policy with min TTL value would be choosen, used in the case of "prefer cleaning" to minimize disk space costs,
   `MAX_TTL` - policy with max TTL value would be choosen, used in the case of "prefer to save data" to prevent deletion of any data in case of ambiguity.
   
   `spec` is unique and can be used to drop some policies.



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 merged PR #8062:
URL: https://github.com/apache/hudi/pull/8062


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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1411531558


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,210 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();

Review Comment:
   Do we need to pass around basic info like table path or meta client so that user can customize more easily? For example, how can a customized class knows the table path?



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1418323849


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,210 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+For 1.0.0 and later hudi version which supports efficient completion time queries on the timeline(#9565), we can get partition's `lastModifiedTime` by scanning the timeline and get the last write commit for the partition. Also for efficiency, we can store the partitions' last modified time and current completion time in the replace commit metadata. The next time we need to calculate the partitions' last modified time, we can build incrementally from the replace commit metadata of the last ttl management.
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 
+
+For the first version of TTL management, we do not plan to implement a complicated strategy (For example, use an array to store strategies, introduce partition regex etc.). Instead, we add a new abstract method `getPartitionPathsForTTLManagement` in `PartitionTTLManagementStrategy` and provides a new config `hoodie.partition.ttl.management.partition.selected`. 
+
+If `hoodie.partition.ttl.management.partition.selected` is set, `getPartitionPathsForTTLManagement` will return partitions provided by this config. If not, `getPartitionPathsForTTLManagement` will return all partitions in the hudi table. 
+
+TTL management strategies will only be applied for partitions return by `getPartitionPathsForTTLManagement`. 
+
+Thus, if users want to apply different strategies for different partitions, they can do the TTL management multiple times, selecting different partitions and apply corresponding strategies. And we may provide a batch interface in the future to simplify this. 
+
+The `getPartitionPathsForTTLManagement` method will look like this:
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+   /**
+    * Scan and list all partitions for partition TTL management.
+    *
+    * @return Partitions to apply TTL management strategy
+    */
+   protected List<String> getPartitionPathsForTTLManagement() {
+      if (StringUtils.isNullOrEmpty(config.getTTLManagementPartitionSelected())) {
+        return getMatchedPartitions();
+      } else {
+        // Return All partition paths
+        return FSUtils.getAllPartitionPaths(hoodieTable.getContext(), config.getMetadataConfig(), config.getBasePath());
+      }

Review Comment:
   Let's make the pseudocode clear.



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


[GitHub] [hudi] SteNicholas commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "SteNicholas (via GitHub)" <gi...@apache.org>.
SteNicholas commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1198517777


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.
+    2. **KEEP_BY_COUNT**. Keep N sub-partitions for a  high-level partition. When sub partition count exceeds, delete the partitions with smaller partition values until the sub-partition count meets the policy configuration.
+    3. **KEEP_BY_SIZE**. Similar to KEEP_BY_COUNT, but to ensure that the sum of the data size of all sub-partitions does not exceed the policy configuration.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned by two fields (user_id, ts). For partition(user_id='1'), we set the policy to keep 100G data for all sub-partitions, and for partition(user_id='2') we set the policy that all sub-partitions will expire 10 days after their last modified time.
+3. It's possible that there are a lot of high-level partitions in the user's table,  and they don't want to set TTL policies for all the high-level partitions. So we need to provide a default policy mechanism so that users can set a default policy for all high-level partitions and add some explicit policies for some of them if needed. Explicit policies will override the default policy.
+
+So here we have the TTL policy definition:
+```java
+public class HoodiePartitionTTLPolicy {
+  public enum TTLPolicy {
+    KEEP_BY_TIME, KEEP_BY_SIZE, KEEP_BY_COUNT
+  }
+
+  // Partition spec for which the policy takes effect
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.  Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as follows.
+
+```sql
+-- Set default policy for all user_id, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');
+ 
+--For partition user_id=1/, keep 10 sub partitions.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=1/', policy => 'KEEP_BY_COUNT', policyValue => '10');
+
+--For partition user_id=2/, keep 100GB data in total
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=2/', policy => 'KEEP_BY_SIZE', policyValue => '107374182400');
+
+--For partition user_id=3/, keep the data for 7 day.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=3/', policy => 'KEEP_BY_TIME', policyValue => '7');
+
+-- Show all the TTL policies including default and explicit policies
+call show_ttl_policies(table => 'test');
+user_id=*/	KEEP_BY_TIME	30
+user_id=1/	KEEP_BY_COUNT	10
+user_id=2/	KEEP_BY_SIZE	107374182400
+user_id=3/	KEEP_BY_TIME	7
+```
+
+### Storage for TTL policy
+The partition TTL policies will be stored in `hoodie.properties`since it is part of table metadata. The policy configs in `hoodie.properties`are defined as follows. Explicit policies are defined using a JSON array while default policy is defined using separate configs.
+
+```properties
+# Default TTL policy definition
+hoodie.partition.ttl.management.default.policy=KEEP_BY_TIME
+hoodie.partition.ttl.management.default.fields=user_id
+hoodie.partition.ttl.management.default.policy.value=30
+
+# Explicit TTL policy definition
+hoodie.partition.ttl.policies=[{"partitionSpec"\:"user_id\=2/","policy"\:"KEEP_BY_SIZE","policyValue"\:107374182400},{"partitionSpec"\:"user_id\=1/","policy"\:"KEEP_BY_COUNT","policyValue"\:10},{"partitionSpec"\:"user_id\=3/","policy"\:"KEEP_BY_TIME","policyValue"\:7}]

Review Comment:
   Could one partition have multiple ttl policy? If could, which ttl policy would be the main?



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.
+    2. **KEEP_BY_COUNT**. Keep N sub-partitions for a  high-level partition. When sub partition count exceeds, delete the partitions with smaller partition values until the sub-partition count meets the policy configuration.
+    3. **KEEP_BY_SIZE**. Similar to KEEP_BY_COUNT, but to ensure that the sum of the data size of all sub-partitions does not exceed the policy configuration.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned by two fields (user_id, ts). For partition(user_id='1'), we set the policy to keep 100G data for all sub-partitions, and for partition(user_id='2') we set the policy that all sub-partitions will expire 10 days after their last modified time.
+3. It's possible that there are a lot of high-level partitions in the user's table,  and they don't want to set TTL policies for all the high-level partitions. So we need to provide a default policy mechanism so that users can set a default policy for all high-level partitions and add some explicit policies for some of them if needed. Explicit policies will override the default policy.
+
+So here we have the TTL policy definition:
+```java
+public class HoodiePartitionTTLPolicy {
+  public enum TTLPolicy {
+    KEEP_BY_TIME, KEEP_BY_SIZE, KEEP_BY_COUNT
+  }
+
+  // Partition spec for which the policy takes effect
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.  Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as follows.
+
+```sql
+-- Set default policy for all user_id, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');
+ 
+--For partition user_id=1/, keep 10 sub partitions.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=1/', policy => 'KEEP_BY_COUNT', policyValue => '10');
+
+--For partition user_id=2/, keep 100GB data in total
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=2/', policy => 'KEEP_BY_SIZE', policyValue => '107374182400');
+
+--For partition user_id=3/, keep the data for 7 day.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=3/', policy => 'KEEP_BY_TIME', policyValue => '7');
+
+-- Show all the TTL policies including default and explicit policies
+call show_ttl_policies(table => 'test');
+user_id=*/	KEEP_BY_TIME	30
+user_id=1/	KEEP_BY_COUNT	10
+user_id=2/	KEEP_BY_SIZE	107374182400
+user_id=3/	KEEP_BY_TIME	7
+```
+
+### Storage for TTL policy
+The partition TTL policies will be stored in `hoodie.properties`since it is part of table metadata. The policy configs in `hoodie.properties`are defined as follows. Explicit policies are defined using a JSON array while default policy is defined using separate configs.
+
+```properties
+# Default TTL policy definition
+hoodie.partition.ttl.management.default.policy=KEEP_BY_TIME
+hoodie.partition.ttl.management.default.fields=user_id
+hoodie.partition.ttl.management.default.policy.value=30
+
+# Explicit TTL policy definition
+hoodie.partition.ttl.policies=[{"partitionSpec"\:"user_id\=2/","policy"\:"KEEP_BY_SIZE","policyValue"\:107374182400},{"partitionSpec"\:"user_id\=1/","policy"\:"KEEP_BY_COUNT","policyValue"\:10},{"partitionSpec"\:"user_id\=3/","policy"\:"KEEP_BY_TIME","policyValue"\:7}]

Review Comment:
   @stream2000, could one partition have multiple ttl policy? If could, which ttl policy would be the main?



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


[GitHub] [hudi] SteNicholas commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "SteNicholas (via GitHub)" <gi...@apache.org>.
SteNicholas commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1198516655


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.
+    2. **KEEP_BY_COUNT**. Keep N sub-partitions for a  high-level partition. When sub partition count exceeds, delete the partitions with smaller partition values until the sub-partition count meets the policy configuration.
+    3. **KEEP_BY_SIZE**. Similar to KEEP_BY_COUNT, but to ensure that the sum of the data size of all sub-partitions does not exceed the policy configuration.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned by two fields (user_id, ts). For partition(user_id='1'), we set the policy to keep 100G data for all sub-partitions, and for partition(user_id='2') we set the policy that all sub-partitions will expire 10 days after their last modified time.
+3. It's possible that there are a lot of high-level partitions in the user's table,  and they don't want to set TTL policies for all the high-level partitions. So we need to provide a default policy mechanism so that users can set a default policy for all high-level partitions and add some explicit policies for some of them if needed. Explicit policies will override the default policy.
+
+So here we have the TTL policy definition:
+```java
+public class HoodiePartitionTTLPolicy {
+  public enum TTLPolicy {
+    KEEP_BY_TIME, KEEP_BY_SIZE, KEEP_BY_COUNT
+  }
+
+  // Partition spec for which the policy takes effect
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.  Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as follows.
+
+```sql
+-- Set default policy for all user_id, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');

Review Comment:
   Could the `add_ttl_policy` procedure add the `type` to specify the ttl policy type, which value could be `partition` etc.



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.
+    2. **KEEP_BY_COUNT**. Keep N sub-partitions for a  high-level partition. When sub partition count exceeds, delete the partitions with smaller partition values until the sub-partition count meets the policy configuration.
+    3. **KEEP_BY_SIZE**. Similar to KEEP_BY_COUNT, but to ensure that the sum of the data size of all sub-partitions does not exceed the policy configuration.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned by two fields (user_id, ts). For partition(user_id='1'), we set the policy to keep 100G data for all sub-partitions, and for partition(user_id='2') we set the policy that all sub-partitions will expire 10 days after their last modified time.
+3. It's possible that there are a lot of high-level partitions in the user's table,  and they don't want to set TTL policies for all the high-level partitions. So we need to provide a default policy mechanism so that users can set a default policy for all high-level partitions and add some explicit policies for some of them if needed. Explicit policies will override the default policy.
+
+So here we have the TTL policy definition:
+```java
+public class HoodiePartitionTTLPolicy {
+  public enum TTLPolicy {
+    KEEP_BY_TIME, KEEP_BY_SIZE, KEEP_BY_COUNT
+  }
+
+  // Partition spec for which the policy takes effect
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.  Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as follows.
+
+```sql
+-- Set default policy for all user_id, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');

Review Comment:
   Could the `add_ttl_policy` procedure add the `type` to specify the ttl policy type, which value could be `partition` etc?



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


[GitHub] [hudi] SteNicholas commented on pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "SteNicholas (via GitHub)" <gi...@apache.org>.
SteNicholas commented on PR #8062:
URL: https://github.com/apache/hudi/pull/8062#issuecomment-1582405189

   @stream2000, do you have any updates?


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


[GitHub] [hudi] stream2000 commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "stream2000 (via GitHub)" <gi...@apache.org>.
stream2000 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1211364813


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,163 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and
+maintain proper statistics to find expired partitions by themselves. As the scale of installations grew, it's more
+important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Gathering proper partition statistics for TTL management
+- Executing TTL management
+
+### TTL Policy Definition
+
+We have four main considerations when designing TTL policy:
+
+1. Usually user hopes to manage partition TTL by expired time, so we support the policy `KEEP_BY_TIME` that partitions
+   will expire N days after their last modified time. Note that the last modified time here means any inserts/updates to
+   the partition. Also, we will make it easy to extend new policies so that users can implement their own polices to
+   decide which partitions are expired.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned
+   by `user_id`. For partition(user_id='1'), we set the policy that this partition will expire 7 days after its last
+   modified time, while for
+   partition(user_id='2') we can set the policy that it will expire 30 days after its last modified time.
+3. It's possible that users have multi-fields partitioning, and they don't want set policy for all partition. So we need
+   to support partition regex when defining
+   partition TTL policies. For example, for a hudi table partitioned by (user_id, ts), we can first set a default
+   policy(which policy value contains `*`)
+   that for all partitions matched (user_id=*/) will expire in 7 days, and explicitly set a policy that for partition
+   matched user_id=3 will expire in 30 days. Explicit policy will override default polices, which means default policies
+   only takes effects on partitions that do not match any explicit ttl policies.
+4. We may need to support record level ttl policy in the future. So partition TTL policy should be an implementation of
+   HoodieTTLPolicy.
+
+So we have the TTL policy definition like this (may change when implementing):
+
+```java
+public class HoodiePartitionTTLPolicy implements HoodieTTLPolicy {
+  public enum TTLPolicy {
+    // We supports keep by last modified time at the first version
+    KEEP_BY_TIME
+  }
+
+  // Partition spec for which the policy takes effect, could be a regex or a static partition value
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy Definition
+
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.
+Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as
+follows.
+
+```sparksql
+-- Set policy for all user_id using partition regex, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');
+
+--For partition user_id=3/, keep the data for 7 day.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=3/', policy => 'KEEP_BY_TIME', policyValue => '7');
+
+-- Show all the TTL policies including default and explicit policies
+call show_ttl_policies(table => 'test');
+user_id=*/	KEEP_BY_TIME	30
+user_id=3/	KEEP_BY_TIME	7
+```
+
+### Storage for TTL policy
+
+We need persistent partition ttt policies in hudi table config and users should interact with hudi only by
+setting/getting ttl policies. Hudi then will takes care of doing the ttl management in an async table service.
+
+The partition TTL policies will be stored in `hoodie.properties` and they are defined as follows. Explicit policies are
+defined using a JSON array, note we support regex here.
+
+```properties
+# TTL policy definition
+hoodie.partition.ttl.policies=[{"partitionSpec"\:"user_id\=*/","policy"\:"KEEP_BY_TIME","policyValue"\:7},{"partitionSpec"\:"user_id\=3/","policy"\:"KEEP_BY_TIME","policyValue"\:30}]
+```
+
+### Partition Statistics
+
+We need to gather proper partition statistics for different kinds of partition ttl policies. Specifically, for
+KEEP_BY_TIME policy that we want to support first, we need to know the last modified time for every partition in hudi
+table.
+
+To simplify the design, we decide to use the largest commit time of committed file groups in the partition as partition
+last modified time. But for file groups generated by replace commit, it may not reveal the real insert/update time for
+the file group. We can also introduce other mechanism to get last modified time of the partition, like add a new kind of

Review Comment:
   Sorry, I can't get what you mean by get partition statistics be confirmed. Do you mean how to get the real insert/update time of the file group? 



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


[GitHub] [hudi] SteNicholas commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "SteNicholas (via GitHub)" <gi...@apache.org>.
SteNicholas commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1211560398


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,163 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and
+maintain proper statistics to find expired partitions by themselves. As the scale of installations grew, it's more
+important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Gathering proper partition statistics for TTL management
+- Executing TTL management
+
+### TTL Policy Definition
+
+We have four main considerations when designing TTL policy:
+
+1. Usually user hopes to manage partition TTL by expired time, so we support the policy `KEEP_BY_TIME` that partitions
+   will expire N days after their last modified time. Note that the last modified time here means any inserts/updates to
+   the partition. Also, we will make it easy to extend new policies so that users can implement their own polices to
+   decide which partitions are expired.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned
+   by `user_id`. For partition(user_id='1'), we set the policy that this partition will expire 7 days after its last
+   modified time, while for
+   partition(user_id='2') we can set the policy that it will expire 30 days after its last modified time.
+3. It's possible that users have multi-fields partitioning, and they don't want set policy for all partition. So we need
+   to support partition regex when defining
+   partition TTL policies. For example, for a hudi table partitioned by (user_id, ts), we can first set a default
+   policy(which policy value contains `*`)
+   that for all partitions matched (user_id=*/) will expire in 7 days, and explicitly set a policy that for partition
+   matched user_id=3 will expire in 30 days. Explicit policy will override default polices, which means default policies
+   only takes effects on partitions that do not match any explicit ttl policies.
+4. We may need to support record level ttl policy in the future. So partition TTL policy should be an implementation of
+   HoodieTTLPolicy.
+
+So we have the TTL policy definition like this (may change when implementing):
+
+```java
+public class HoodiePartitionTTLPolicy implements HoodieTTLPolicy {
+  public enum TTLPolicy {
+    // We supports keep by last modified time at the first version
+    KEEP_BY_TIME
+  }
+
+  // Partition spec for which the policy takes effect, could be a regex or a static partition value
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy Definition
+
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.
+Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as
+follows.
+
+```sparksql
+-- Set policy for all user_id using partition regex, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');
+
+--For partition user_id=3/, keep the data for 7 day.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=3/', policy => 'KEEP_BY_TIME', policyValue => '7');
+
+-- Show all the TTL policies including default and explicit policies
+call show_ttl_policies(table => 'test');
+user_id=*/	KEEP_BY_TIME	30
+user_id=3/	KEEP_BY_TIME	7
+```
+
+### Storage for TTL policy
+
+We need persistent partition ttt policies in hudi table config and users should interact with hudi only by
+setting/getting ttl policies. Hudi then will takes care of doing the ttl management in an async table service.
+
+The partition TTL policies will be stored in `hoodie.properties` and they are defined as follows. Explicit policies are
+defined using a JSON array, note we support regex here.
+
+```properties
+# TTL policy definition
+hoodie.partition.ttl.policies=[{"partitionSpec"\:"user_id\=*/","policy"\:"KEEP_BY_TIME","policyValue"\:7},{"partitionSpec"\:"user_id\=3/","policy"\:"KEEP_BY_TIME","policyValue"\:30}]
+```
+
+### Partition Statistics
+
+We need to gather proper partition statistics for different kinds of partition ttl policies. Specifically, for
+KEEP_BY_TIME policy that we want to support first, we need to know the last modified time for every partition in hudi
+table.
+
+To simplify the design, we decide to use the largest commit time of committed file groups in the partition as partition
+last modified time. But for file groups generated by replace commit, it may not reveal the real insert/update time for
+the file group. We can also introduce other mechanism to get last modified time of the partition, like add a new kind of

Review Comment:
   @stream2000, I mean that could you confirm where to get partition statistics in this RFC.



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,163 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and
+maintain proper statistics to find expired partitions by themselves. As the scale of installations grew, it's more
+important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Gathering proper partition statistics for TTL management
+- Executing TTL management
+
+### TTL Policy Definition
+
+We have four main considerations when designing TTL policy:
+
+1. Usually user hopes to manage partition TTL by expired time, so we support the policy `KEEP_BY_TIME` that partitions
+   will expire N days after their last modified time. Note that the last modified time here means any inserts/updates to
+   the partition. Also, we will make it easy to extend new policies so that users can implement their own polices to
+   decide which partitions are expired.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned
+   by `user_id`. For partition(user_id='1'), we set the policy that this partition will expire 7 days after its last
+   modified time, while for
+   partition(user_id='2') we can set the policy that it will expire 30 days after its last modified time.
+3. It's possible that users have multi-fields partitioning, and they don't want set policy for all partition. So we need
+   to support partition regex when defining
+   partition TTL policies. For example, for a hudi table partitioned by (user_id, ts), we can first set a default
+   policy(which policy value contains `*`)
+   that for all partitions matched (user_id=*/) will expire in 7 days, and explicitly set a policy that for partition
+   matched user_id=3 will expire in 30 days. Explicit policy will override default polices, which means default policies
+   only takes effects on partitions that do not match any explicit ttl policies.
+4. We may need to support record level ttl policy in the future. So partition TTL policy should be an implementation of
+   HoodieTTLPolicy.
+
+So we have the TTL policy definition like this (may change when implementing):
+
+```java
+public class HoodiePartitionTTLPolicy implements HoodieTTLPolicy {
+  public enum TTLPolicy {
+    // We supports keep by last modified time at the first version
+    KEEP_BY_TIME
+  }
+
+  // Partition spec for which the policy takes effect, could be a regex or a static partition value
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy Definition
+
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.
+Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as
+follows.
+
+```sparksql
+-- Set policy for all user_id using partition regex, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');
+
+--For partition user_id=3/, keep the data for 7 day.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=3/', policy => 'KEEP_BY_TIME', policyValue => '7');
+
+-- Show all the TTL policies including default and explicit policies
+call show_ttl_policies(table => 'test');
+user_id=*/	KEEP_BY_TIME	30
+user_id=3/	KEEP_BY_TIME	7
+```
+
+### Storage for TTL policy
+
+We need persistent partition ttt policies in hudi table config and users should interact with hudi only by
+setting/getting ttl policies. Hudi then will takes care of doing the ttl management in an async table service.
+
+The partition TTL policies will be stored in `hoodie.properties` and they are defined as follows. Explicit policies are
+defined using a JSON array, note we support regex here.
+
+```properties
+# TTL policy definition
+hoodie.partition.ttl.policies=[{"partitionSpec"\:"user_id\=*/","policy"\:"KEEP_BY_TIME","policyValue"\:7},{"partitionSpec"\:"user_id\=3/","policy"\:"KEEP_BY_TIME","policyValue"\:30}]
+```
+
+### Partition Statistics
+
+We need to gather proper partition statistics for different kinds of partition ttl policies. Specifically, for
+KEEP_BY_TIME policy that we want to support first, we need to know the last modified time for every partition in hudi
+table.
+
+To simplify the design, we decide to use the largest commit time of committed file groups in the partition as partition
+last modified time. But for file groups generated by replace commit, it may not reveal the real insert/update time for
+the file group. We can also introduce other mechanism to get last modified time of the partition, like add a new kind of

Review Comment:
   @stream2000, I mean that could you confirm where to get partition statistics in this RFC?



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


[GitHub] [hudi] SteNicholas commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "SteNicholas (via GitHub)" <gi...@apache.org>.
SteNicholas commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1211553559


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,163 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and
+maintain proper statistics to find expired partitions by themselves. As the scale of installations grew, it's more
+important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Gathering proper partition statistics for TTL management
+- Executing TTL management
+
+### TTL Policy Definition
+
+We have four main considerations when designing TTL policy:
+
+1. Usually user hopes to manage partition TTL by expired time, so we support the policy `KEEP_BY_TIME` that partitions
+   will expire N days after their last modified time. Note that the last modified time here means any inserts/updates to
+   the partition. Also, we will make it easy to extend new policies so that users can implement their own polices to
+   decide which partitions are expired.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned
+   by `user_id`. For partition(user_id='1'), we set the policy that this partition will expire 7 days after its last
+   modified time, while for
+   partition(user_id='2') we can set the policy that it will expire 30 days after its last modified time.
+3. It's possible that users have multi-fields partitioning, and they don't want set policy for all partition. So we need
+   to support partition regex when defining
+   partition TTL policies. For example, for a hudi table partitioned by (user_id, ts), we can first set a default
+   policy(which policy value contains `*`)
+   that for all partitions matched (user_id=*/) will expire in 7 days, and explicitly set a policy that for partition
+   matched user_id=3 will expire in 30 days. Explicit policy will override default polices, which means default policies
+   only takes effects on partitions that do not match any explicit ttl policies.
+4. We may need to support record level ttl policy in the future. So partition TTL policy should be an implementation of
+   HoodieTTLPolicy.
+
+So we have the TTL policy definition like this (may change when implementing):
+
+```java
+public class HoodiePartitionTTLPolicy implements HoodieTTLPolicy {
+  public enum TTLPolicy {
+    // We supports keep by last modified time at the first version
+    KEEP_BY_TIME
+  }
+
+  // Partition spec for which the policy takes effect, could be a regex or a static partition value
+  private String partitionSpec;

Review Comment:
   @stream2000, partitionSpec isn't enough for partition ttl management. For example, there is a Hudi table partitioned by date and hour. Meanwhile, the user want to configure ttl with a year. How could user configure this ttl with current policy definition?



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


[GitHub] [hudi] geserdugarov commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1334923947


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 

Review Comment:
   We can allow user to write specification,
   `spec = "*/product=1/*", TTL = 30 DAYS`
   `spec = "*/product=2/*, TTL = 7 DAYS`
   It also allows to set default TTL by:
   `spec = "*", TTL = 2 MONTHS`
   
   In this case we will have multiple policies for some partitons. To resolve conflict automatically, we could add  `hoodie.ttl.policies.conflict.resolution.rule` property with available options:
   `MIN_TTL` - policy with min TTL value would be choosen, used in the case of "prefer cleaning" to minimize disk space costs,
   `MAX_TTL` - policy with max TTL value would be choosen, used in the case of "prefer to save data" to prevent deletion of any data in the case of ambiguity.
   
   `spec` is unique and can be used to drop some policies.



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition TTL Management [hudi]

Posted by "stream2000 (via GitHub)" <gi...@apache.org>.
stream2000 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1351764739


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME

Review Comment:
   We can change the word `TTL` to `lifecycle` if we want to support other types of data expiration policies.  In current version this RFC, we do not plan to support other types of policies like `KEEP_BY_SIZE` since they rely on the condition that the partition values are comparable which is not always true. However we do have some users who have this kind of requirement so we need to make the policy extensible.



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.

Review Comment:
   We should always think twice before making format changes. We have considered using `HoodiePartitionMetadata` to provide the `lastModifiedTime`, however, the current implementation of partition metadata does not support transactions, and will never be modified since the first creation.  Not sure it is a good choice for maintaining the `lastModifiedTime`



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.

Review Comment:
   Thanks for your review~ 
   
   > it's just providing TTL functionality by custom implementation of PartitionTTLManagementStrategy is not user friendly.
   
    For common users that want to do TTL management, they just need to add some write configs to their ingestion job. In the simplest situation they may just add the following config to enable async ttl management: 
   
   ```properties
   hoodie.partition.ttl.days.retain=10
   hoodie.ttl.management.async.enabled=true
   ```
   
   The TTL policy will default to `KEEP_BY_TIME` and we can automatically detect expired partitions by their last modified time  and delete them. 
   
   However, this simple policy may not suit for all users, they may need to implement their own partition TTL policy so we provide an abstraction `PartitionTTLManagementStrategy` that returns expired partitions as users need and hudi will help delete them. 
   
   > This is a main scope for TTL, and we shouldn't allow to have more flexibility.
   Customized implementation of PartitionTTLManagementStrategy will allow to do anything with partitions. It still could be PartitionManagementStrategy, but then we shouldn't named it with TTL part.
   
   Customized implementation of PartitionTTLManagementStrategy will only provide a list of expired partitions, and hudi will help delete them.  We can rename the word `PartitionTTLManagementStrategy` to `PartitionLifecycleManagementStrategy` since we do not always delete the partitions by time. 
    



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 

Review Comment:
   We can implement two independent ttl policies, one is simple while another is advanced as I commented below. 



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 
+
+For the first version of TTL management, we do not plan to implement a complicated strategy (For example, use an array to store strategies, introduce partition regex etc.). Instead, we add a new abstract method `getPartitionPathsForTTLManagement` in `PartitionTTLManagementStrategy` and provides a new config `hoodie.partition.ttl.management.partition.selected`. 
+
+If `hoodie.partition.ttl.management.partition.selected` is set, `getPartitionPathsForTTLManagement` will return partitions provided by this config. If not, `getPartitionPathsForTTLManagement` will return all partitions in the hudi table. 
+
+TTL management strategies will only be applied for partitions return by `getPartitionPathsForTTLManagement`. 
+
+Thus, if users want to apply different strategies for different partitions, they can do the TTL management multiple times, selecting different partitions and apply corresponding strategies. And we may provide a batch interface in the future to simplify this. 

Review Comment:
   > I offer to allow user set only spec and TTL values with support of add, show, and delete operations. 
   
   > So, if I want to implement 6 partition policies, then I should prepare 6 PartitionTTLManagementStrategys and call them 6 times? 
   
   We can implement the advanced TTL management policy in the future to provide the ability to set different policies for different partitions.  To achieve this, we can introduce a new `hoodie.partition.ttl.management.strategy` and implement the logic you mentioned. For most users, providing a simple policy that TTL any partition whose last mod time (last time when data was added/updated), is greater than the TTL time specified will be enough. 
   
    @geserdugarov  @nsivabalan What do you think about this?  
   
   



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 
+
+For the first version of TTL management, we do not plan to implement a complicated strategy (For example, use an array to store strategies, introduce partition regex etc.). Instead, we add a new abstract method `getPartitionPathsForTTLManagement` in `PartitionTTLManagementStrategy` and provides a new config `hoodie.partition.ttl.management.partition.selected`. 
+
+If `hoodie.partition.ttl.management.partition.selected` is set, `getPartitionPathsForTTLManagement` will return partitions provided by this config. If not, `getPartitionPathsForTTLManagement` will return all partitions in the hudi table. 

Review Comment:
   It will be a write config, and we will only apply TTL policy to the partition specified by this config. If this config is not  specified, we will apply ttl policy to all the partitions in hudi table. 



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1415523349


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 

Review Comment:
   Ok, I wrote all my ideas in a separate RFC:
   https://github.com/apache/hudi/pull/10248
   @stream2000 , @danny0405 if it wouldn't bother you, could you, please, look at it?



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 

Review Comment:
   Ok, I have wrote all my ideas in a separate RFC:
   https://github.com/apache/hudi/pull/10248
   @stream2000 , @danny0405 if it wouldn't bother you, could you, please, look at it?



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


[GitHub] [hudi] stream2000 commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "stream2000 (via GitHub)" <gi...@apache.org>.
stream2000 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1154134187


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.
+    2. **KEEP_BY_COUNT**. Keep N sub-partitions for a  high-level partition. When sub partition count exceeds, delete the partitions with smaller partition values until the sub-partition count meets the policy configuration.
+    3. **KEEP_BY_SIZE**. Similar to KEEP_BY_COUNT, but to ensure that the sum of the data size of all sub-partitions does not exceed the policy configuration.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned by two fields (user_id, ts). For partition(user_id='1'), we set the policy to keep 100G data for all sub-partitions, and for partition(user_id='2') we set the policy that all sub-partitions will expire 10 days after their last modified time.
+3. It's possible that there are a lot of high-level partitions in the user's table,  and they don't want to set TTL policies for all the high-level partitions. So we need to provide a default policy mechanism so that users can set a default policy for all high-level partitions and add some explicit policies for some of them if needed. Explicit policies will override the default policy.
+
+So here we have the TTL policy definition:
+```java
+public class HoodiePartitionTTLPolicy {
+  public enum TTLPolicy {
+    KEEP_BY_TIME, KEEP_BY_SIZE, KEEP_BY_COUNT
+  }
+
+  // Partition spec for which the policy takes effect
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.  Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as follows.
+
+```sql
+-- Set default policy for all user_id, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');
+ 
+--For partition user_id=1/, keep 10 sub partitions.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=1/', policy => 'KEEP_BY_COUNT', policyValue => '10');
+
+--For partition user_id=2/, keep 100GB data in total
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=2/', policy => 'KEEP_BY_SIZE', policyValue => '107374182400');
+
+--For partition user_id=3/, keep the data for 7 day.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=3/', policy => 'KEEP_BY_TIME', policyValue => '7');
+
+-- Show all the TTL policies including default and explicit policies
+call show_ttl_policies(table => 'test');
+user_id=*/	KEEP_BY_TIME	30
+user_id=1/	KEEP_BY_COUNT	10
+user_id=2/	KEEP_BY_SIZE	107374182400
+user_id=3/	KEEP_BY_TIME	7
+```
+
+### Storage for TTL policy
+The partition TTL policies will be stored in `hoodie.properties`since it is part of table metadata. The policy configs in `hoodie.properties`are defined as follows. Explicit policies are defined using a JSON array while default policy is defined using separate configs.
+
+```properties
+# Default TTL policy definition
+hoodie.partition.ttl.management.default.policy=KEEP_BY_TIME
+hoodie.partition.ttl.management.default.fields=user_id
+hoodie.partition.ttl.management.default.policy.value=30
+
+# Explicit TTL policy definition
+hoodie.partition.ttl.policies=[{"partitionSpec"\:"user_id\=2/","policy"\:"KEEP_BY_SIZE","policyValue"\:107374182400},{"partitionSpec"\:"user_id\=1/","policy"\:"KEEP_BY_COUNT","policyValue"\:10},{"partitionSpec"\:"user_id\=3/","policy"\:"KEEP_BY_TIME","policyValue"\:7}]
+```
+
+### Partition Statistics
+#### Partition Statistics Entity
+We need need to maintain a partition stat for every partition in the hudi table. The stat will contain three fields:
+
+- RelativePath. Relative path to the base path, or we can call it PartitionPath.
+- LastModifiedTime. Last instant time that modified the partition. We need this information to support  the `KEEP_BY_TIME` policy.
+- PartitionTotalFileSize. The sum of all valid data file sizes in the partition, to support the `KEEP_BY_SIZE`policy. We calculate only the latest file slices in the partition currently.
+#### Gathering Partition Statistics
+The simplest way to get partition statistics is that reading information from the metadata table. However, the write and compaction of the metadata table will severely affect the throughput and latency of data ingestion.  So we design an asynchronous partition statistics as follows.
+
+- At the first time we gather the partitions statistics, we list all partitions in the hudi table and calculate `PartitionTotalFileSize`and `LastModifiedTime`for each partition. Then we store all the partition stats in persistent storage along with the instant time we do the stats.  For example, we can directly store all partition stats in a JSON file whose file name contains the instant time.
+- After initializing the partition statistics, we can list only affected partitions by reading timeline metadata and store the new partition statistics back to the storage with new instant time.
+- Note that deleted partition will be deleted from partition statistics too. If a partition was deleted before and have no new data, we will remove it from partition statistics so it won't calculated as expired partition again.
+### Appling Policies
+Once we have defined the TTL policies and have proper partition statistics,  it's easy to apply the policies and delete expired partitions.
+
+1. Gather partitions statistics.
+2. Apply each TTL policy. For explicit policies, find sub-partitions matched `PartitionSpec`defined in the policy and check if there are expired partitions according to the policy type and  size. For default policy, find partitions that do not match any explicit policy and check if they are expired.
+3. For all expired partitions, call delete_partition to delete them, which will generate a replace commit and mark all files in those partitions as replaced. For pending clustering and compaction that affect the target partition to delete,  [HUDI-5553](https://issues.apache.org/jira/browse/HUDI-5553) introduces a pre-check that will throw an exception, and further improvement could be discussed in [HUDI-5663](https://issues.apache.org/jira/browse/HUDI-5663).
+4. Clean then will delete all replaced file groups.

Review Comment:
   delete_partition has been already implemented in current hudi master branch.  When we call delete_partition to delete a list partitions, the executor will list all files for the partitions to delete and store them in the replacecommit commit metadata. After the replace commit committed, all the filesystem views that have seen the replace commit will exclude files that were replaced in the replace commit.



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


[GitHub] [hudi] geserdugarov commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1334248487


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.

Review Comment:
   Sorry to be difficult, it's just providing TTL functionality by custom implementation of `PartitionTTLManagementStrategy` is not user friendly.
   We want to automate detection of outdated partitions and calling `delete_partition`. Could we just allow user to set partition path specification with TTL value, and implement everything internally?
   
   From my point of view, there are two main entities in TTL:
   - object
   In our case, it's partition, we define it by using `spec`.
   - definition of outdating
   It should be time or something time-dependent. In our case, we use current time, `_hoodie_commit_time`, and user-defined delta value.
   
   This is a main scope for TTL, and we shouldn't allow to have more flexibility.
   Customized implementation of `PartitionTTLManagementStrategy` will allow to do anything with partitions. It could be `PartitionManagementStrategy`, but we shouldn't named it with `TTL` part.



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 
+
+For the first version of TTL management, we do not plan to implement a complicated strategy (For example, use an array to store strategies, introduce partition regex etc.). Instead, we add a new abstract method `getPartitionPathsForTTLManagement` in `PartitionTTLManagementStrategy` and provides a new config `hoodie.partition.ttl.management.partition.selected`. 
+
+If `hoodie.partition.ttl.management.partition.selected` is set, `getPartitionPathsForTTLManagement` will return partitions provided by this config. If not, `getPartitionPathsForTTLManagement` will return all partitions in the hudi table. 

Review Comment:
   Is it Hudi table new property? If we want to apply some default TTL for all partitions then it would contain list of all partitions?



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME

Review Comment:
   Do we really need to consider other policies? It's really confusing, that TTL (Time-To-Live) could consider anything else then time or time-dependent. In your first commit I found that there were KEEP_BY_SIZE and KEEP_BY_COUNT. But KEEP_BY_SIZE looks like we wanna delete some huge partition with a lot of data just because it is big, and KEEP_BY_COUNT is not suitable due to Hudi's partitioning model.



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.

Review Comment:
   In current realization `HoodiePartitionMetadata` provides only `commitTime` (partition created commit time) and `partitionDepth` properties. We can add new `lastModifiedTime` property in `.hoodie_partition_metadata`, which is updated on every commit/deltacommit to corresponding partition.



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 
+
+For the first version of TTL management, we do not plan to implement a complicated strategy (For example, use an array to store strategies, introduce partition regex etc.). Instead, we add a new abstract method `getPartitionPathsForTTLManagement` in `PartitionTTLManagementStrategy` and provides a new config `hoodie.partition.ttl.management.partition.selected`. 
+
+If `hoodie.partition.ttl.management.partition.selected` is set, `getPartitionPathsForTTLManagement` will return partitions provided by this config. If not, `getPartitionPathsForTTLManagement` will return all partitions in the hudi table. 
+
+TTL management strategies will only be applied for partitions return by `getPartitionPathsForTTLManagement`. 
+
+Thus, if users want to apply different strategies for different partitions, they can do the TTL management multiple times, selecting different partitions and apply corresponding strategies. And we may provide a batch interface in the future to simplify this. 

Review Comment:
   So, if I want to implement 6 partition policies, then I should prepare 6 `PartitionTTLManagementStrategy`s and call them 6 times?
   
   I offer to allow user set only `spec` and `TTL value` with support of `add`, `show`, and `delete` operations. We can also implement `dry run` for checking partition list.
   Then during TTL checking:
   - Iterate through a list of all partitions and compare them to `spec`.
   Later listing of all partitions could be optimized by storing it in separate data structure. But the most hard task would be consistency providing in the case of concurrent writing with new partition creating and removing some partitions.
   - Compare partition to the list of policies.
   This list would be ordered by TTL value in ascending or descending order. It would be defined by a resolution strategy for multiple policies conflict, use MIN_TTL or MAX_TTL.
   - Then compare current time with `lastModifiedTime`.
   Later it also could be optimized by partition related data structure for decreasing reading from HDFS.



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 

Review Comment:
   We can allow user to write specification,
   `spec = "*/product=1/*", TTL = 30 DAYS`
   `spec = "*/product=2/*, TTL = 7 DAYS`
   It also allows to set default TTL by:
   `spec = "*", TTL = 2 MONTHS`
   
   In this case we will have multiple policies for some partitons. To resolve conflict automatically, we could add  `hoodie.ttl.policies.conflict.resolution.rule` property with available options:
   `MIN_TTL` - policy with min TTL value would be choosen, used in the case of "prefer cleaning" to minimize disk space costs,
   `MAX_TTL` - policy with max TTL value would be choosen, used in the case of "prefer to save data" to prevent deletion of any data in case of ambiguity.
   
   Specification is unique and can be used to drop some policies.



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


[GitHub] [hudi] geserdugarov commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1334927622


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 
+
+For the first version of TTL management, we do not plan to implement a complicated strategy (For example, use an array to store strategies, introduce partition regex etc.). Instead, we add a new abstract method `getPartitionPathsForTTLManagement` in `PartitionTTLManagementStrategy` and provides a new config `hoodie.partition.ttl.management.partition.selected`. 
+
+If `hoodie.partition.ttl.management.partition.selected` is set, `getPartitionPathsForTTLManagement` will return partitions provided by this config. If not, `getPartitionPathsForTTLManagement` will return all partitions in the hudi table. 

Review Comment:
   Is it Hudi table new property? Would it contain the list of all partitions if we want to apply some default TTL?



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 
+
+For the first version of TTL management, we do not plan to implement a complicated strategy (For example, use an array to store strategies, introduce partition regex etc.). Instead, we add a new abstract method `getPartitionPathsForTTLManagement` in `PartitionTTLManagementStrategy` and provides a new config `hoodie.partition.ttl.management.partition.selected`. 
+
+If `hoodie.partition.ttl.management.partition.selected` is set, `getPartitionPathsForTTLManagement` will return partitions provided by this config. If not, `getPartitionPathsForTTLManagement` will return all partitions in the hudi table. 

Review Comment:
   Is it Hudi table's new property? Would it contain the list of all partitions if we want to apply some default TTL?



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


[GitHub] [hudi] geserdugarov commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1334928958


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 
+
+For the first version of TTL management, we do not plan to implement a complicated strategy (For example, use an array to store strategies, introduce partition regex etc.). Instead, we add a new abstract method `getPartitionPathsForTTLManagement` in `PartitionTTLManagementStrategy` and provides a new config `hoodie.partition.ttl.management.partition.selected`. 
+
+If `hoodie.partition.ttl.management.partition.selected` is set, `getPartitionPathsForTTLManagement` will return partitions provided by this config. If not, `getPartitionPathsForTTLManagement` will return all partitions in the hudi table. 
+
+TTL management strategies will only be applied for partitions return by `getPartitionPathsForTTLManagement`. 
+
+Thus, if users want to apply different strategies for different partitions, they can do the TTL management multiple times, selecting different partitions and apply corresponding strategies. And we may provide a batch interface in the future to simplify this. 

Review Comment:
   So, if I want to implement 6 partition policies, then I should prepare 6 `PartitionTTLManagementStrategy`s and call them 6 times? Again, TTL is a simple property and I don't think that it should so hard to set it.
   
   I offer to allow user set only `spec` and `TTL` values with support of `add`, `show`, and `delete` operations. We can also implement `dry run` for checking partition list.
   Then during TTL checking:
   - Iterate through a list of all partitions and compare them to `spec`.
   Later, listing of all partitions could be optimized by storing it in a separate data structure. But the tough part in working with this data structure would be providing consistency in the case of concurrent writing, when new partitions are created and some old partitions are removed.
   - Compare partition to the list of policies.
   We can order this list by TTL value in ascending or descending order. The order would be defined by a resolution strategy for multiple policies conflict: `MIN_TTL` or `MAX_TTL` correspondingly.
   - And then we can compare current time with `lastModifiedTime`.
   Later, storing of this property could be optimized to decrease reading from HDFS, for instance, by adding this data to the mentioned partition related data structure.



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


[GitHub] [hudi] SteNicholas commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "SteNicholas (via GitHub)" <gi...@apache.org>.
SteNicholas commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1211558787


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,163 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and
+maintain proper statistics to find expired partitions by themselves. As the scale of installations grew, it's more
+important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Gathering proper partition statistics for TTL management
+- Executing TTL management
+
+### TTL Policy Definition
+
+We have four main considerations when designing TTL policy:
+
+1. Usually user hopes to manage partition TTL by expired time, so we support the policy `KEEP_BY_TIME` that partitions
+   will expire N days after their last modified time. Note that the last modified time here means any inserts/updates to
+   the partition. Also, we will make it easy to extend new policies so that users can implement their own polices to
+   decide which partitions are expired.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned
+   by `user_id`. For partition(user_id='1'), we set the policy that this partition will expire 7 days after its last
+   modified time, while for
+   partition(user_id='2') we can set the policy that it will expire 30 days after its last modified time.
+3. It's possible that users have multi-fields partitioning, and they don't want set policy for all partition. So we need
+   to support partition regex when defining
+   partition TTL policies. For example, for a hudi table partitioned by (user_id, ts), we can first set a default
+   policy(which policy value contains `*`)
+   that for all partitions matched (user_id=*/) will expire in 7 days, and explicitly set a policy that for partition
+   matched user_id=3 will expire in 30 days. Explicit policy will override default polices, which means default policies
+   only takes effects on partitions that do not match any explicit ttl policies.
+4. We may need to support record level ttl policy in the future. So partition TTL policy should be an implementation of
+   HoodieTTLPolicy.
+
+So we have the TTL policy definition like this (may change when implementing):
+
+```java
+public class HoodiePartitionTTLPolicy implements HoodieTTLPolicy {
+  public enum TTLPolicy {
+    // We supports keep by last modified time at the first version
+    KEEP_BY_TIME
+  }
+
+  // Partition spec for which the policy takes effect, could be a regex or a static partition value
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy Definition
+
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.
+Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as
+follows.
+
+```sparksql
+-- Set policy for all user_id using partition regex, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');
+
+--For partition user_id=3/, keep the data for 7 day.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=3/', policy => 'KEEP_BY_TIME', policyValue => '7');
+
+-- Show all the TTL policies including default and explicit policies
+call show_ttl_policies(table => 'test');

Review Comment:
   Could this procedure add the `partitionSpec` param?



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


[GitHub] [hudi] SteNicholas commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "SteNicholas (via GitHub)" <gi...@apache.org>.
SteNicholas commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1211467622


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,163 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and
+maintain proper statistics to find expired partitions by themselves. As the scale of installations grew, it's more
+important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Gathering proper partition statistics for TTL management
+- Executing TTL management
+
+### TTL Policy Definition
+
+We have four main considerations when designing TTL policy:
+
+1. Usually user hopes to manage partition TTL by expired time, so we support the policy `KEEP_BY_TIME` that partitions
+   will expire N days after their last modified time. Note that the last modified time here means any inserts/updates to
+   the partition. Also, we will make it easy to extend new policies so that users can implement their own polices to
+   decide which partitions are expired.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned
+   by `user_id`. For partition(user_id='1'), we set the policy that this partition will expire 7 days after its last
+   modified time, while for
+   partition(user_id='2') we can set the policy that it will expire 30 days after its last modified time.
+3. It's possible that users have multi-fields partitioning, and they don't want set policy for all partition. So we need
+   to support partition regex when defining
+   partition TTL policies. For example, for a hudi table partitioned by (user_id, ts), we can first set a default
+   policy(which policy value contains `*`)
+   that for all partitions matched (user_id=*/) will expire in 7 days, and explicitly set a policy that for partition
+   matched user_id=3 will expire in 30 days. Explicit policy will override default polices, which means default policies
+   only takes effects on partitions that do not match any explicit ttl policies.
+4. We may need to support record level ttl policy in the future. So partition TTL policy should be an implementation of
+   HoodieTTLPolicy.
+
+So we have the TTL policy definition like this (may change when implementing):
+
+```java
+public class HoodiePartitionTTLPolicy implements HoodieTTLPolicy {
+  public enum TTLPolicy {
+    // We supports keep by last modified time at the first version
+    KEEP_BY_TIME
+  }
+
+  // Partition spec for which the policy takes effect, could be a regex or a static partition value
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy Definition
+
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.
+Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as
+follows.
+
+```sparksql
+-- Set policy for all user_id using partition regex, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');
+
+--For partition user_id=3/, keep the data for 7 day.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=3/', policy => 'KEEP_BY_TIME', policyValue => '7');
+
+-- Show all the TTL policies including default and explicit policies
+call show_ttl_policies(table => 'test');
+user_id=*/	KEEP_BY_TIME	30
+user_id=3/	KEEP_BY_TIME	7
+```
+
+### Storage for TTL policy
+
+We need persistent partition ttt policies in hudi table config and users should interact with hudi only by
+setting/getting ttl policies. Hudi then will takes care of doing the ttl management in an async table service.
+
+The partition TTL policies will be stored in `hoodie.properties` and they are defined as follows. Explicit policies are

Review Comment:
   @stream2000, IMO, we could introduce `hoodie.ttl.properties` to store the `hoodie.partition.ttl.policies`, which have no compatibility problem here.



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1405590162


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.

Review Comment:
   How to fetch the info when a commit is archived?



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1418307128


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,248 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a lifecycle management mechanism to prevent the
+dataset from growing infinitely.

Review Comment:
   To clarify: this is very different, the proposal for this doc only yields target partitons, it does not remove the whole partition.



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1413436193


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.

Review Comment:
   > The TTL policy will default to KEEP_BY_TIME and we can automatically detect expired partitions by their last modified time and delete them.
   
   Unfortunately, I don't see any other strategies except KEEP_BY_TIME for partition level TTL.
   
   > Customized implementation of PartitionTTLManagementStrategy will only provide a list of expired partitions, and hudi will help delete them. We can rename the word PartitionTTLManagementStrategy to PartitionLifecycleManagementStrategy since we do not always delete the partitions by time.
   
   I see a partition on different object levels as:
   | level              | time | size                                                                           |
   | ------------------------ | ------ | ------------------------------------------------------------------- |
   | the whole partition | TTL | not applicable, could be trigger for lower levels        |
   | file groups              | managed by | other services |
   | file slices                | managed by | other services |
   | records                   | TTL | oldest could be removed by partition size threshold |
   
   So, we don't need anything else except KEEP_BY_TIME for partition level.
   



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on PR #8062:
URL: https://github.com/apache/hudi/pull/8062#issuecomment-1840714919

   > @danny0405 @geserdugarov Thanks for your review! I have addressed all comments, hope for your opinions.
   
   @stream2000 Thank you for reply. As you mentioned, I've wrote all my ideas in a separate RFC to simplify details discussion: https://github.com/apache/hudi/pull/10248
   @danny0405 @stream2000 Do you mind me asking for your comments?
   


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


[GitHub] [hudi] SteNicholas commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "SteNicholas (via GitHub)" <gi...@apache.org>.
SteNicholas commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1198516655


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.
+    2. **KEEP_BY_COUNT**. Keep N sub-partitions for a  high-level partition. When sub partition count exceeds, delete the partitions with smaller partition values until the sub-partition count meets the policy configuration.
+    3. **KEEP_BY_SIZE**. Similar to KEEP_BY_COUNT, but to ensure that the sum of the data size of all sub-partitions does not exceed the policy configuration.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned by two fields (user_id, ts). For partition(user_id='1'), we set the policy to keep 100G data for all sub-partitions, and for partition(user_id='2') we set the policy that all sub-partitions will expire 10 days after their last modified time.
+3. It's possible that there are a lot of high-level partitions in the user's table,  and they don't want to set TTL policies for all the high-level partitions. So we need to provide a default policy mechanism so that users can set a default policy for all high-level partitions and add some explicit policies for some of them if needed. Explicit policies will override the default policy.
+
+So here we have the TTL policy definition:
+```java
+public class HoodiePartitionTTLPolicy {
+  public enum TTLPolicy {
+    KEEP_BY_TIME, KEEP_BY_SIZE, KEEP_BY_COUNT
+  }
+
+  // Partition spec for which the policy takes effect
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.  Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as follows.
+
+```sql
+-- Set default policy for all user_id, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');

Review Comment:
   @stream2000, could the `add_ttl_policy` procedure add the `type` to specify the ttl policy type, which value could be `partition` etc?



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.
+    2. **KEEP_BY_COUNT**. Keep N sub-partitions for a  high-level partition. When sub partition count exceeds, delete the partitions with smaller partition values until the sub-partition count meets the policy configuration.
+    3. **KEEP_BY_SIZE**. Similar to KEEP_BY_COUNT, but to ensure that the sum of the data size of all sub-partitions does not exceed the policy configuration.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned by two fields (user_id, ts). For partition(user_id='1'), we set the policy to keep 100G data for all sub-partitions, and for partition(user_id='2') we set the policy that all sub-partitions will expire 10 days after their last modified time.
+3. It's possible that there are a lot of high-level partitions in the user's table,  and they don't want to set TTL policies for all the high-level partitions. So we need to provide a default policy mechanism so that users can set a default policy for all high-level partitions and add some explicit policies for some of them if needed. Explicit policies will override the default policy.
+
+So here we have the TTL policy definition:
+```java
+public class HoodiePartitionTTLPolicy {

Review Comment:
   @stream2000, could we introduce `HoodieTTLPolicy` interface? Then `HoodiePartitionTTLPolicy` implements the `HoodieTTLPolicy`. `HoodieRecordTTLPolicy` could also implement this interface in feature.



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


[GitHub] [hudi] nbalajee commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "nbalajee (via GitHub)" <gi...@apache.org>.
nbalajee commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1198202805


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.

Review Comment:
   When retiring the old/unused/not-accessed partitions, another approach we are taking internally is:
   (a) stash the partitions to be cleaned up in .stashedForDeletion folder (at .hoodie level).
   (b) partitions stashed for deletion will wait in the folder for a week (or time dictated by the policy) before actually getting deleted.  In cases, where we realize that something has been accidentally deleted (like a bad policy configuration,  TTL exclusion not configured etc), we can always move back from the stash to quickly recover from the TTL event.
   (c) We shall configure policies for .stashedForDeletion/<partition>/ subfolders to manage for appropriate tiering level (whether to be moved to a warm/cold tier etc)
   (d) in addition to the deletePartitions() API, which would stash the folder (instead of deleting) based on the configs, we would need a restore API to move the subfolder/files back to their original location. 
   (e) Metadata left by the delete operation to be synced with MDT to keep the file listing metadata in sync with the file system.  (In cases where replication to a different region is supported, this also would warrant applying the changes on the replicated copies of data).



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


[GitHub] [hudi] stream2000 commented on pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "stream2000 (via GitHub)" <gi...@apache.org>.
stream2000 commented on PR #8062:
URL: https://github.com/apache/hudi/pull/8062#issuecomment-1569696943

   @nsivabalan @SteNicholas Hi, thanks for your review. I have updated the RFC according to your comments. 
   1. Simplify the TTL policy, now we will only support `KEEP_BY_TIME` so we don't need to compare partition values between partitions. 
   2. Remove the persistent JSON stats mechanism. We can gather partition stats every time we do TTL management. 
   3. Add more detail about executing TTL management. We will support async table services for TTL management and both spark and flink engine can execute the async service. 
   4. Consider Record level TTL policy when design Pattition TTL Policy.
   
   And here are the answer to your remaining question. 
   > Why store the policy in hoodie.properties instead of using write config? 
   I think we need to store the policy in hudi somewhere otherwise user needs to store the policy themselves. Think when the user has 1000 partitions and wants to set TTL policies for 500 of them. It's better to store the policy metadata in hudi. 
   
   Hope for your another round of review! 
   


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


[GitHub] [hudi] SteNicholas commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "SteNicholas (via GitHub)" <gi...@apache.org>.
SteNicholas commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1211587004


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,163 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and
+maintain proper statistics to find expired partitions by themselves. As the scale of installations grew, it's more
+important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Gathering proper partition statistics for TTL management
+- Executing TTL management
+
+### TTL Policy Definition
+
+We have four main considerations when designing TTL policy:
+
+1. Usually user hopes to manage partition TTL by expired time, so we support the policy `KEEP_BY_TIME` that partitions
+   will expire N days after their last modified time. Note that the last modified time here means any inserts/updates to
+   the partition. Also, we will make it easy to extend new policies so that users can implement their own polices to
+   decide which partitions are expired.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned
+   by `user_id`. For partition(user_id='1'), we set the policy that this partition will expire 7 days after its last
+   modified time, while for
+   partition(user_id='2') we can set the policy that it will expire 30 days after its last modified time.
+3. It's possible that users have multi-fields partitioning, and they don't want set policy for all partition. So we need
+   to support partition regex when defining
+   partition TTL policies. For example, for a hudi table partitioned by (user_id, ts), we can first set a default
+   policy(which policy value contains `*`)
+   that for all partitions matched (user_id=*/) will expire in 7 days, and explicitly set a policy that for partition
+   matched user_id=3 will expire in 30 days. Explicit policy will override default polices, which means default policies
+   only takes effects on partitions that do not match any explicit ttl policies.
+4. We may need to support record level ttl policy in the future. So partition TTL policy should be an implementation of
+   HoodieTTLPolicy.
+
+So we have the TTL policy definition like this (may change when implementing):
+
+```java
+public class HoodiePartitionTTLPolicy implements HoodieTTLPolicy {
+  public enum TTLPolicy {
+    // We supports keep by last modified time at the first version
+    KEEP_BY_TIME
+  }
+
+  // Partition spec for which the policy takes effect, could be a regex or a static partition value
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy Definition
+
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.
+Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as
+follows.
+
+```sparksql
+-- Set policy for all user_id using partition regex, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');
+
+--For partition user_id=3/, keep the data for 7 day.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=3/', policy => 'KEEP_BY_TIME', policyValue => '7');
+
+-- Show all the TTL policies including default and explicit policies
+call show_ttl_policies(table => 'test');
+user_id=*/	KEEP_BY_TIME	30
+user_id=3/	KEEP_BY_TIME	7
+```
+
+### Storage for TTL policy
+
+We need persistent partition ttt policies in hudi table config and users should interact with hudi only by
+setting/getting ttl policies. Hudi then will takes care of doing the ttl management in an async table service.
+
+The partition TTL policies will be stored in `hoodie.properties` and they are defined as follows. Explicit policies are
+defined using a JSON array, note we support regex here.
+
+```properties
+# TTL policy definition
+hoodie.partition.ttl.policies=[{"partitionSpec"\:"user_id\=*/","policy"\:"KEEP_BY_TIME","policyValue"\:7},{"partitionSpec"\:"user_id\=3/","policy"\:"KEEP_BY_TIME","policyValue"\:30}]
+```
+
+### Partition Statistics
+
+We need to gather proper partition statistics for different kinds of partition ttl policies. Specifically, for
+KEEP_BY_TIME policy that we want to support first, we need to know the last modified time for every partition in hudi
+table.
+
+To simplify the design, we decide to use the largest commit time of committed file groups in the partition as partition
+last modified time. But for file groups generated by replace commit, it may not reveal the real insert/update time for
+the file group. We can also introduce other mechanism to get last modified time of the partition, like add a new kind of
+partition metadata or add a new field in metadata table. Open to ideas for this design choice.
+
+### Executing TTL management
+
+Once we have defined the TTL policies and have proper partition statistics, it's easy to apply the policies and delete
+expired partitions. The process of TTL management is very similar to other table services in hudi, we will provide a new
+method called `managePartitionTTL` in `BaseHoodieTableServiceClient` and we can invoke this method in async table
+service, SparkSQL Call Command, cli, JAVA code etc.
+
+We will provide an Async Table Service as the default interface to the users that want to do partition TTL management
+and we can add more interfaces in the future.
+
+User will config the async ttl management service as follows. TTL management will be trigger N commits after last
+TTL replace commit.
+
+```properties
+hoodie.partition.ttl.management.async=true
+hoodie.partition.ttl.management.min.commits=10
+```
+
+The process of manage partition TTL is as follows:
+
+1. Gather partitions statistics, list all partitions and the largest commit time of committed
+   file groups of each partition as their last modified time.
+2. Apply each TTL policy. For explicit policies, find sub-partitions matched `PartitionSpec` defined in the policy and
+   check if there are expired partitions according to the policy type and size. For default policy, find partitions that
+   do not match any explicit policy and check if they are expired.
+3. For all expired partitions, call delete_partition to delete them, which will generate a replace commit and mark all
+   files in those partitions as replaced. For pending clustering and compaction that affect the target partition to

Review Comment:
   If there is pending clustering and compaction, the ttl policy execution should clean and delete the expired file groups after clustering and compaction. The user doesn't need do any behavior for pending clustering and compaction case.



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


[GitHub] [hudi] codope commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "codope (via GitHub)" <gi...@apache.org>.
codope commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1151672933


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.
+    2. **KEEP_BY_COUNT**. Keep N sub-partitions for a  high-level partition. When sub partition count exceeds, delete the partitions with smaller partition values until the sub-partition count meets the policy configuration.
+    3. **KEEP_BY_SIZE**. Similar to KEEP_BY_COUNT, but to ensure that the sum of the data size of all sub-partitions does not exceed the policy configuration.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned by two fields (user_id, ts). For partition(user_id='1'), we set the policy to keep 100G data for all sub-partitions, and for partition(user_id='2') we set the policy that all sub-partitions will expire 10 days after their last modified time.
+3. It's possible that there are a lot of high-level partitions in the user's table,  and they don't want to set TTL policies for all the high-level partitions. So we need to provide a default policy mechanism so that users can set a default policy for all high-level partitions and add some explicit policies for some of them if needed. Explicit policies will override the default policy.
+
+So here we have the TTL policy definition:
+```java
+public class HoodiePartitionTTLPolicy {
+  public enum TTLPolicy {
+    KEEP_BY_TIME, KEEP_BY_SIZE, KEEP_BY_COUNT
+  }
+
+  // Partition spec for which the policy takes effect
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.  Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as follows.
+
+```sql
+-- Set default policy for all user_id, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');
+ 
+--For partition user_id=1/, keep 10 sub partitions.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=1/', policy => 'KEEP_BY_COUNT', policyValue => '10');
+
+--For partition user_id=2/, keep 100GB data in total
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=2/', policy => 'KEEP_BY_SIZE', policyValue => '107374182400');
+
+--For partition user_id=3/, keep the data for 7 day.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=3/', policy => 'KEEP_BY_TIME', policyValue => '7');
+
+-- Show all the TTL policies including default and explicit policies
+call show_ttl_policies(table => 'test');
+user_id=*/	KEEP_BY_TIME	30
+user_id=1/	KEEP_BY_COUNT	10
+user_id=2/	KEEP_BY_SIZE	107374182400
+user_id=3/	KEEP_BY_TIME	7
+```
+
+### Storage for TTL policy
+The partition TTL policies will be stored in `hoodie.properties`since it is part of table metadata. The policy configs in `hoodie.properties`are defined as follows. Explicit policies are defined using a JSON array while default policy is defined using separate configs.
+
+```properties
+# Default TTL policy definition
+hoodie.partition.ttl.management.default.policy=KEEP_BY_TIME
+hoodie.partition.ttl.management.default.fields=user_id
+hoodie.partition.ttl.management.default.policy.value=30
+
+# Explicit TTL policy definition
+hoodie.partition.ttl.policies=[{"partitionSpec"\:"user_id\=2/","policy"\:"KEEP_BY_SIZE","policyValue"\:107374182400},{"partitionSpec"\:"user_id\=1/","policy"\:"KEEP_BY_COUNT","policyValue"\:10},{"partitionSpec"\:"user_id\=3/","policy"\:"KEEP_BY_TIME","policyValue"\:7}]
+```
+
+### Partition Statistics
+#### Partition Statistics Entity
+We need need to maintain a partition stat for every partition in the hudi table. The stat will contain three fields:
+
+- RelativePath. Relative path to the base path, or we can call it PartitionPath.
+- LastModifiedTime. Last instant time that modified the partition. We need this information to support  the `KEEP_BY_TIME` policy.
+- PartitionTotalFileSize. The sum of all valid data file sizes in the partition, to support the `KEEP_BY_SIZE`policy. We calculate only the latest file slices in the partition currently.

Review Comment:
   Can you define `valid data file`? Is it just the latest version of each file id i.e. latest file slice?



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.
+    2. **KEEP_BY_COUNT**. Keep N sub-partitions for a  high-level partition. When sub partition count exceeds, delete the partitions with smaller partition values until the sub-partition count meets the policy configuration.
+    3. **KEEP_BY_SIZE**. Similar to KEEP_BY_COUNT, but to ensure that the sum of the data size of all sub-partitions does not exceed the policy configuration.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned by two fields (user_id, ts). For partition(user_id='1'), we set the policy to keep 100G data for all sub-partitions, and for partition(user_id='2') we set the policy that all sub-partitions will expire 10 days after their last modified time.
+3. It's possible that there are a lot of high-level partitions in the user's table,  and they don't want to set TTL policies for all the high-level partitions. So we need to provide a default policy mechanism so that users can set a default policy for all high-level partitions and add some explicit policies for some of them if needed. Explicit policies will override the default policy.
+
+So here we have the TTL policy definition:
+```java
+public class HoodiePartitionTTLPolicy {
+  public enum TTLPolicy {
+    KEEP_BY_TIME, KEEP_BY_SIZE, KEEP_BY_COUNT
+  }
+
+  // Partition spec for which the policy takes effect
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.  Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as follows.
+
+```sql
+-- Set default policy for all user_id, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');
+ 
+--For partition user_id=1/, keep 10 sub partitions.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=1/', policy => 'KEEP_BY_COUNT', policyValue => '10');
+
+--For partition user_id=2/, keep 100GB data in total
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=2/', policy => 'KEEP_BY_SIZE', policyValue => '107374182400');
+
+--For partition user_id=3/, keep the data for 7 day.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=3/', policy => 'KEEP_BY_TIME', policyValue => '7');
+
+-- Show all the TTL policies including default and explicit policies
+call show_ttl_policies(table => 'test');
+user_id=*/	KEEP_BY_TIME	30
+user_id=1/	KEEP_BY_COUNT	10
+user_id=2/	KEEP_BY_SIZE	107374182400
+user_id=3/	KEEP_BY_TIME	7
+```
+
+### Storage for TTL policy
+The partition TTL policies will be stored in `hoodie.properties`since it is part of table metadata. The policy configs in `hoodie.properties`are defined as follows. Explicit policies are defined using a JSON array while default policy is defined using separate configs.
+
+```properties
+# Default TTL policy definition
+hoodie.partition.ttl.management.default.policy=KEEP_BY_TIME
+hoodie.partition.ttl.management.default.fields=user_id
+hoodie.partition.ttl.management.default.policy.value=30
+
+# Explicit TTL policy definition
+hoodie.partition.ttl.policies=[{"partitionSpec"\:"user_id\=2/","policy"\:"KEEP_BY_SIZE","policyValue"\:107374182400},{"partitionSpec"\:"user_id\=1/","policy"\:"KEEP_BY_COUNT","policyValue"\:10},{"partitionSpec"\:"user_id\=3/","policy"\:"KEEP_BY_TIME","policyValue"\:7}]
+```
+
+### Partition Statistics
+#### Partition Statistics Entity
+We need need to maintain a partition stat for every partition in the hudi table. The stat will contain three fields:
+
+- RelativePath. Relative path to the base path, or we can call it PartitionPath.
+- LastModifiedTime. Last instant time that modified the partition. We need this information to support  the `KEEP_BY_TIME` policy.
+- PartitionTotalFileSize. The sum of all valid data file sizes in the partition, to support the `KEEP_BY_SIZE`policy. We calculate only the latest file slices in the partition currently.
+#### Gathering Partition Statistics
+The simplest way to get partition statistics is that reading information from the metadata table. However, the write and compaction of the metadata table will severely affect the throughput and latency of data ingestion.  So we design an asynchronous partition statistics as follows.
+
+- At the first time we gather the partitions statistics, we list all partitions in the hudi table and calculate `PartitionTotalFileSize`and `LastModifiedTime`for each partition. Then we store all the partition stats in persistent storage along with the instant time we do the stats.  For example, we can directly store all partition stats in a JSON file whose file name contains the instant time.

Review Comment:
   Why a separate json file? Why not augment the delete partition commit metadata to have all this info?



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.
+    2. **KEEP_BY_COUNT**. Keep N sub-partitions for a  high-level partition. When sub partition count exceeds, delete the partitions with smaller partition values until the sub-partition count meets the policy configuration.
+    3. **KEEP_BY_SIZE**. Similar to KEEP_BY_COUNT, but to ensure that the sum of the data size of all sub-partitions does not exceed the policy configuration.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned by two fields (user_id, ts). For partition(user_id='1'), we set the policy to keep 100G data for all sub-partitions, and for partition(user_id='2') we set the policy that all sub-partitions will expire 10 days after their last modified time.
+3. It's possible that there are a lot of high-level partitions in the user's table,  and they don't want to set TTL policies for all the high-level partitions. So we need to provide a default policy mechanism so that users can set a default policy for all high-level partitions and add some explicit policies for some of them if needed. Explicit policies will override the default policy.
+
+So here we have the TTL policy definition:
+```java
+public class HoodiePartitionTTLPolicy {
+  public enum TTLPolicy {
+    KEEP_BY_TIME, KEEP_BY_SIZE, KEEP_BY_COUNT
+  }
+
+  // Partition spec for which the policy takes effect
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.  Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as follows.
+
+```sql
+-- Set default policy for all user_id, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');
+ 
+--For partition user_id=1/, keep 10 sub partitions.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=1/', policy => 'KEEP_BY_COUNT', policyValue => '10');
+
+--For partition user_id=2/, keep 100GB data in total
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=2/', policy => 'KEEP_BY_SIZE', policyValue => '107374182400');
+
+--For partition user_id=3/, keep the data for 7 day.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=3/', policy => 'KEEP_BY_TIME', policyValue => '7');
+
+-- Show all the TTL policies including default and explicit policies
+call show_ttl_policies(table => 'test');
+user_id=*/	KEEP_BY_TIME	30
+user_id=1/	KEEP_BY_COUNT	10
+user_id=2/	KEEP_BY_SIZE	107374182400
+user_id=3/	KEEP_BY_TIME	7
+```
+
+### Storage for TTL policy
+The partition TTL policies will be stored in `hoodie.properties`since it is part of table metadata. The policy configs in `hoodie.properties`are defined as follows. Explicit policies are defined using a JSON array while default policy is defined using separate configs.
+
+```properties
+# Default TTL policy definition
+hoodie.partition.ttl.management.default.policy=KEEP_BY_TIME
+hoodie.partition.ttl.management.default.fields=user_id
+hoodie.partition.ttl.management.default.policy.value=30
+
+# Explicit TTL policy definition
+hoodie.partition.ttl.policies=[{"partitionSpec"\:"user_id\=2/","policy"\:"KEEP_BY_SIZE","policyValue"\:107374182400},{"partitionSpec"\:"user_id\=1/","policy"\:"KEEP_BY_COUNT","policyValue"\:10},{"partitionSpec"\:"user_id\=3/","policy"\:"KEEP_BY_TIME","policyValue"\:7}]
+```
+
+### Partition Statistics
+#### Partition Statistics Entity
+We need need to maintain a partition stat for every partition in the hudi table. The stat will contain three fields:
+
+- RelativePath. Relative path to the base path, or we can call it PartitionPath.
+- LastModifiedTime. Last instant time that modified the partition. We need this information to support  the `KEEP_BY_TIME` policy.
+- PartitionTotalFileSize. The sum of all valid data file sizes in the partition, to support the `KEEP_BY_SIZE`policy. We calculate only the latest file slices in the partition currently.
+#### Gathering Partition Statistics
+The simplest way to get partition statistics is that reading information from the metadata table. However, the write and compaction of the metadata table will severely affect the throughput and latency of data ingestion.  So we design an asynchronous partition statistics as follows.
+
+- At the first time we gather the partitions statistics, we list all partitions in the hudi table and calculate `PartitionTotalFileSize`and `LastModifiedTime`for each partition. Then we store all the partition stats in persistent storage along with the instant time we do the stats.  For example, we can directly store all partition stats in a JSON file whose file name contains the instant time.
+- After initializing the partition statistics, we can list only affected partitions by reading timeline metadata and store the new partition statistics back to the storage with new instant time.
+- Note that deleted partition will be deleted from partition statistics too. If a partition was deleted before and have no new data, we will remove it from partition statistics so it won't calculated as expired partition again.
+### Appling Policies

Review Comment:
   ```suggestion
   ### Applying Policies
   ```



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.
+    2. **KEEP_BY_COUNT**. Keep N sub-partitions for a  high-level partition. When sub partition count exceeds, delete the partitions with smaller partition values until the sub-partition count meets the policy configuration.
+    3. **KEEP_BY_SIZE**. Similar to KEEP_BY_COUNT, but to ensure that the sum of the data size of all sub-partitions does not exceed the policy configuration.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned by two fields (user_id, ts). For partition(user_id='1'), we set the policy to keep 100G data for all sub-partitions, and for partition(user_id='2') we set the policy that all sub-partitions will expire 10 days after their last modified time.
+3. It's possible that there are a lot of high-level partitions in the user's table,  and they don't want to set TTL policies for all the high-level partitions. So we need to provide a default policy mechanism so that users can set a default policy for all high-level partitions and add some explicit policies for some of them if needed. Explicit policies will override the default policy.
+
+So here we have the TTL policy definition:
+```java
+public class HoodiePartitionTTLPolicy {
+  public enum TTLPolicy {
+    KEEP_BY_TIME, KEEP_BY_SIZE, KEEP_BY_COUNT
+  }
+
+  // Partition spec for which the policy takes effect
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.  Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as follows.
+
+```sql
+-- Set default policy for all user_id, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');
+ 
+--For partition user_id=1/, keep 10 sub partitions.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=1/', policy => 'KEEP_BY_COUNT', policyValue => '10');
+
+--For partition user_id=2/, keep 100GB data in total
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=2/', policy => 'KEEP_BY_SIZE', policyValue => '107374182400');
+
+--For partition user_id=3/, keep the data for 7 day.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=3/', policy => 'KEEP_BY_TIME', policyValue => '7');
+
+-- Show all the TTL policies including default and explicit policies
+call show_ttl_policies(table => 'test');
+user_id=*/	KEEP_BY_TIME	30
+user_id=1/	KEEP_BY_COUNT	10
+user_id=2/	KEEP_BY_SIZE	107374182400
+user_id=3/	KEEP_BY_TIME	7
+```
+
+### Storage for TTL policy
+The partition TTL policies will be stored in `hoodie.properties`since it is part of table metadata. The policy configs in `hoodie.properties`are defined as follows. Explicit policies are defined using a JSON array while default policy is defined using separate configs.
+
+```properties
+# Default TTL policy definition
+hoodie.partition.ttl.management.default.policy=KEEP_BY_TIME
+hoodie.partition.ttl.management.default.fields=user_id
+hoodie.partition.ttl.management.default.policy.value=30
+
+# Explicit TTL policy definition
+hoodie.partition.ttl.policies=[{"partitionSpec"\:"user_id\=2/","policy"\:"KEEP_BY_SIZE","policyValue"\:107374182400},{"partitionSpec"\:"user_id\=1/","policy"\:"KEEP_BY_COUNT","policyValue"\:10},{"partitionSpec"\:"user_id\=3/","policy"\:"KEEP_BY_TIME","policyValue"\:7}]

Review Comment:
   I suggest that we just keep `hoodie.partition.ttl.policies` in the table config and infer the other hoodie.partition.ttl.* configs from that. We need not expose the default to the user in table config.



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.

Review Comment:
   Does the proposal consider running this as an async [table service](https://github.com/apache/hudi/blob/master/hudi-common/src/main/java/org/apache/hudi/common/model/TableServiceType.java)?



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.
+    2. **KEEP_BY_COUNT**. Keep N sub-partitions for a  high-level partition. When sub partition count exceeds, delete the partitions with smaller partition values until the sub-partition count meets the policy configuration.
+    3. **KEEP_BY_SIZE**. Similar to KEEP_BY_COUNT, but to ensure that the sum of the data size of all sub-partitions does not exceed the policy configuration.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned by two fields (user_id, ts). For partition(user_id='1'), we set the policy to keep 100G data for all sub-partitions, and for partition(user_id='2') we set the policy that all sub-partitions will expire 10 days after their last modified time.
+3. It's possible that there are a lot of high-level partitions in the user's table,  and they don't want to set TTL policies for all the high-level partitions. So we need to provide a default policy mechanism so that users can set a default policy for all high-level partitions and add some explicit policies for some of them if needed. Explicit policies will override the default policy.
+
+So here we have the TTL policy definition:
+```java
+public class HoodiePartitionTTLPolicy {
+  public enum TTLPolicy {
+    KEEP_BY_TIME, KEEP_BY_SIZE, KEEP_BY_COUNT
+  }
+
+  // Partition spec for which the policy takes effect
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.  Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as follows.
+
+```sql
+-- Set default policy for all user_id, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');
+ 
+--For partition user_id=1/, keep 10 sub partitions.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=1/', policy => 'KEEP_BY_COUNT', policyValue => '10');
+
+--For partition user_id=2/, keep 100GB data in total
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=2/', policy => 'KEEP_BY_SIZE', policyValue => '107374182400');
+
+--For partition user_id=3/, keep the data for 7 day.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=3/', policy => 'KEEP_BY_TIME', policyValue => '7');
+
+-- Show all the TTL policies including default and explicit policies
+call show_ttl_policies(table => 'test');
+user_id=*/	KEEP_BY_TIME	30
+user_id=1/	KEEP_BY_COUNT	10
+user_id=2/	KEEP_BY_SIZE	107374182400
+user_id=3/	KEEP_BY_TIME	7
+```
+
+### Storage for TTL policy
+The partition TTL policies will be stored in `hoodie.properties`since it is part of table metadata. The policy configs in `hoodie.properties`are defined as follows. Explicit policies are defined using a JSON array while default policy is defined using separate configs.
+
+```properties
+# Default TTL policy definition
+hoodie.partition.ttl.management.default.policy=KEEP_BY_TIME
+hoodie.partition.ttl.management.default.fields=user_id
+hoodie.partition.ttl.management.default.policy.value=30
+
+# Explicit TTL policy definition
+hoodie.partition.ttl.policies=[{"partitionSpec"\:"user_id\=2/","policy"\:"KEEP_BY_SIZE","policyValue"\:107374182400},{"partitionSpec"\:"user_id\=1/","policy"\:"KEEP_BY_COUNT","policyValue"\:10},{"partitionSpec"\:"user_id\=3/","policy"\:"KEEP_BY_TIME","policyValue"\:7}]
+```
+
+### Partition Statistics
+#### Partition Statistics Entity
+We need need to maintain a partition stat for every partition in the hudi table. The stat will contain three fields:
+
+- RelativePath. Relative path to the base path, or we can call it PartitionPath.
+- LastModifiedTime. Last instant time that modified the partition. We need this information to support  the `KEEP_BY_TIME` policy.
+- PartitionTotalFileSize. The sum of all valid data file sizes in the partition, to support the `KEEP_BY_SIZE`policy. We calculate only the latest file slices in the partition currently.
+#### Gathering Partition Statistics
+The simplest way to get partition statistics is that reading information from the metadata table. However, the write and compaction of the metadata table will severely affect the throughput and latency of data ingestion.  So we design an asynchronous partition statistics as follows.
+
+- At the first time we gather the partitions statistics, we list all partitions in the hudi table and calculate `PartitionTotalFileSize`and `LastModifiedTime`for each partition. Then we store all the partition stats in persistent storage along with the instant time we do the stats.  For example, we can directly store all partition stats in a JSON file whose file name contains the instant time.
+- After initializing the partition statistics, we can list only affected partitions by reading timeline metadata and store the new partition statistics back to the storage with new instant time.
+- Note that deleted partition will be deleted from partition statistics too. If a partition was deleted before and have no new data, we will remove it from partition statistics so it won't calculated as expired partition again.
+### Appling Policies
+Once we have defined the TTL policies and have proper partition statistics,  it's easy to apply the policies and delete expired partitions.
+
+1. Gather partitions statistics.
+2. Apply each TTL policy. For explicit policies, find sub-partitions matched `PartitionSpec`defined in the policy and check if there are expired partitions according to the policy type and  size. For default policy, find partitions that do not match any explicit policy and check if they are expired.
+3. For all expired partitions, call delete_partition to delete them, which will generate a replace commit and mark all files in those partitions as replaced. For pending clustering and compaction that affect the target partition to delete,  [HUDI-5553](https://issues.apache.org/jira/browse/HUDI-5553) introduces a pre-check that will throw an exception, and further improvement could be discussed in [HUDI-5663](https://issues.apache.org/jira/browse/HUDI-5663).
+4. Clean then will delete all replaced file groups.

Review Comment:
   Would be great if you can also discuss how the read path will work? How do we maintain a consistent filesystem view for readers given that delete partition operation can take time?



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


[GitHub] [hudi] stream2000 commented on pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "stream2000 (via GitHub)" <gi...@apache.org>.
stream2000 commented on PR #8062:
URL: https://github.com/apache/hudi/pull/8062#issuecomment-1718999994

   > Can this feature be supported in advance? Especially hope for the next version after 0.14.
   
   I hope so, too! Will work on this feature in the few weeks and push a POC code to my personal repo ASAP. 


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


[GitHub] [hudi] geserdugarov commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1334927622


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 
+
+For the first version of TTL management, we do not plan to implement a complicated strategy (For example, use an array to store strategies, introduce partition regex etc.). Instead, we add a new abstract method `getPartitionPathsForTTLManagement` in `PartitionTTLManagementStrategy` and provides a new config `hoodie.partition.ttl.management.partition.selected`. 
+
+If `hoodie.partition.ttl.management.partition.selected` is set, `getPartitionPathsForTTLManagement` will return partitions provided by this config. If not, `getPartitionPathsForTTLManagement` will return all partitions in the hudi table. 

Review Comment:
   Is it Hudi table new property? If we want to apply some default TTL for all partitions then it would contain the list of all partitions?



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1398634206


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,248 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a lifecycle management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces partition lifecycle management strategies to hudi, people can config the strategies by write
+configs. With proper configs set, Hudi can find out which partitions are expired and delete them.
+
+This proposal introduces partition lifecycle management service to hudi. Lifecycle management is like other table
+services such as Clean/Compaction/Clustering.
+Users can config their partition lifecycle management strategies through write configs and Hudi will help users find
+expired partitions and delete them automatically.
+
+## Background
+
+Lifecycle management mechanism is an important feature for databases. Hudi already provides a `delete_partition`
+interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of partition lifecycle
+management strategies, find expired partitions and call `delete_partition` by themselves. As the scale of installations
+grew, it is becoming increasingly important to implement a user-friendly lifecycle management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition lifecycle management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or
+  asynchronous table services.
+
+### Strategy Definition
+
+The lifecycle strategies is similar to existing table service strategies. We can define lifecycle strategies like
+defining a clustering/clean/compaction strategy:
+
+```properties
+hoodie.partition.lifecycle.management.strategy=KEEP_BY_TIME
+hoodie.partition.lifecycle.management.strategy.class=org.apache.hudi.table.action.lifecycle.strategy.KeepByTimePartitionLifecycleManagementStrategy
+hoodie.partition.lifecycle.days.retain=10
+```
+
+The config `hoodie.partition.lifecycle.management.strategy.class` is to provide a strategy class (subclass
+of `PartitionLifecycleManagementStrategy`) to get expired partition paths to delete.
+And `hoodie.partition.lifecycle.days.retain` is the strategy value used
+by `KeepByTimePartitionLifecycleManagementStrategy` which means that we will expire partitions that haven't been
+modified for this strategy value set. We will cover the `KeepByTimePartitionLifecycleManagementStrategy` strategy in
+detail in the next section.
+
+The core definition of `PartitionLifecycleManagementStrategy` looks like this:
+
+```java
+/**
+ * Strategy for partition-level lifecycle management.
+ */
+public abstract class PartitionLifecycleManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition lifecycle management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionLifecycleManagementStrategy` and hudi will help delete the
+expired partitions.
+
+### KeepByTimePartitionLifecycleManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionLifecycleManagementStrategy` in the first version of partition
+lifecycle management implementation.
+
+The `KeepByTimePartitionLifecycleManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If
+duration between now and 'lastModifiedTime' for the partition is larger than
+what `hoodie.partition.lifecycle.days.retain` configured, `KeepByTimePartitionLifecycleManagementStrategy` will mark
+this partition as an expired partition. We use day as the unit of expired time since it is very common-used for
+datalakes. Open to ideas for this.
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the
+partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However,
+we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy.
+And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example
+using metadata table.
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may
+have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for
+partitions under `product=2` they want to keep for 7 days only.
+
+For the first version of partition lifecycle management, we do not plan to implement a complicated strategy (For
+example, use an array to store strategies, introduce partition regex etc.). Instead, we add a new abstract
+method `getPartitionPathsForLifecycleManagement` in `PartitionLifecycleManagementStrategy` and provides a new
+config `hoodie.partition.lifecycle.management.partition.selected`.
+
+If `hoodie.partition.lifecycle.management.partition.selected` is set, `getPartitionPathsForLifecycleManagement` will
+return partitions provided by this config. If not, `getPartitionPathsForLifecycleManagement` will return all partitions
+in the hudi table.
+
+lifecycle management strategies will only be applied for partitions return by `getPartitionPathsForLifecycleManagement`.
+
+Thus, if users want to apply different strategies for different partitions, they can do the partition lifecycle
+management multiple times, selecting different partitions and apply corresponding strategies. And we may provide a batch
+interface in the future to simplify this.
+
+The `getPartitionPathsForLifecycleManagement` method will look like this:
+
+```java
+/**
+ * Strategy for partition-level lifecycle management.
+ */
+public abstract class PartitionLifecycleManagementStrategy {
+  /**
+   * Scan and list all partitions for partition lifecycle management.
+   *
+   * @return Partitions to apply lifecycle management strategy
+   */
+  protected List<String> getPartitionPathsForLifecycleManagement() {
+    if (StringUtils.isNullOrEmpty(config.getLifecycleManagementPartitionSelected())) {
+      return getMatchedPartitions();
+    } else {
+      // Return All partition paths
+      return FSUtils.getAllPartitionPaths(hoodieTable.getContext(), config.getMetadataConfig(), config.getBasePath());
+    }
+  }
+}
+``` 
+
+### Executing partition lifecycle management
+
+Once we already have a proper `PartitionLifecycleManagementStrategy` implementation, it's easy to execute the partition
+lifecycle management.
+
+```java
+public class SparkPartitionLifecycleManagementActionExecutor<T> extends BaseSparkCommitActionExecutor<T> {
+  @Override
+  public HoodieWriteMetadata<HoodieData<WriteStatus>> execute() {
+    // Construct PartitionLifecycleManagementStrategy
+    PartitionLifecycleManagementStrategy strategy = (PartitionLifecycleManagementStrategy) ReflectionUtils.loadClass(
+        PartitionLifecycleManagementStrategy.checkAndGetPartitionLifecycleManagementStrategy(config),
+        new Class<?>[] {HoodieTable.class, HoodieEngineContext.class, HoodieWriteConfig.class}, table, context, config);
+
+    // Get expired partition paths
+    List<String> expiredPartitions = strategy.getExpiredPartitionPaths();
+
+    // Delete them reusing SparkDeletePartitionCommitActionExecutor
+    return new SparkDeletePartitionCommitActionExecutor<>(context, config, table, instantTime,
+        expiredPartitions).execute();
+  }
+}
+```
+
+We will add a new method `managePartitionLifecycle` in `HoodieTable` and `HoodieSparkCopyOnWriteTable` can implement it
+like this:
+
+```java
+@Override
+public HoodieWriteMetadata<HoodieData<WriteStatus>>managePartitionLifecycle(HoodieEngineContext context,String instantTime){
+    return new SparkPartitionLifecycleManagementActionExecutor<>(context,config,this,instantTime).execute();
+    }
+```
+
+We can call `hoodieTable.managePartitionLifecycle` in independent flink/spark job, in async/sync inline table services
+like clustering/compaction/clean etc.
+
+### User interface for Partition lifecycle Management
+
+We can do partition lifecycle management inline with streaming ingestion job or do it with a independent batch job, for
+both spark and flink engine.
+
+#### Run inline with Streaming Ingestion
+
+Since we can run clustering inline with streaming ingestion job through the following config:
+
+```properties
+hoodie.clustering.async.enabled=true
+hoodie.clustering.async.max.commits=5
+```
+
+We can do similar thing for partition lifecycle management. The config for async partition lifecycle management are:
+
+| Config key                                              | Remarks                                                                                                                        | Default |
+|---------------------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------|---------|
+| hoodie.partition.lifecycle.management.async.enabled     | Enable running of lifecycle management service, asynchronously as writes happen on the table.                                  | False   |

Review Comment:
   What I really care about is the conflict resolution for the operation, does OCC applies here? Do we have some support for the basic concurrent modification.



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1411532246


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.

Review Comment:
   We have no notion of `lastModifiedTime`, do you mean `lastCommitTime` or `latestCommitTime`?



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1413436193


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.

Review Comment:
   > The TTL policy will default to KEEP_BY_TIME and we can automatically detect expired partitions by their last modified time and delete them.
   Unfortunately, I don't see any other strategies except KEEP_BY_TIME for partition level TTL.



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.

Review Comment:
   > The TTL policy will default to KEEP_BY_TIME and we can automatically detect expired partitions by their last modified time and delete them.
   
   Unfortunately, I don't see any other strategies except KEEP_BY_TIME for partition level TTL.



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1413444578


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.

Review Comment:
   Currently, HoodiePartitionMetadata, `.hoodie_partition_metadata` files, provides only `commitTime` and `partitionDepth` properties. This file is written during partition creation and is not updated later. We can make this file more usable by adding a new property, for instance, `lastUpdateTime` for saving time of the last upsert/delete operation on the partition with each commit/deltacommit/replacecommit. And use this property to handle for Partition TTL check.



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1415515672


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME

Review Comment:
   > For example, user wants to add a new strategy that delete partitions by their specified logic.
   
   Do you have any example of this logic? Unfortunately, I couldn't come up with any of them.



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


[GitHub] [hudi] stream2000 commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "stream2000 (via GitHub)" <gi...@apache.org>.
stream2000 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1211346749


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,163 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and
+maintain proper statistics to find expired partitions by themselves. As the scale of installations grew, it's more
+important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Gathering proper partition statistics for TTL management
+- Executing TTL management
+
+### TTL Policy Definition
+
+We have four main considerations when designing TTL policy:
+
+1. Usually user hopes to manage partition TTL by expired time, so we support the policy `KEEP_BY_TIME` that partitions
+   will expire N days after their last modified time. Note that the last modified time here means any inserts/updates to
+   the partition. Also, we will make it easy to extend new policies so that users can implement their own polices to
+   decide which partitions are expired.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned
+   by `user_id`. For partition(user_id='1'), we set the policy that this partition will expire 7 days after its last
+   modified time, while for
+   partition(user_id='2') we can set the policy that it will expire 30 days after its last modified time.
+3. It's possible that users have multi-fields partitioning, and they don't want set policy for all partition. So we need
+   to support partition regex when defining
+   partition TTL policies. For example, for a hudi table partitioned by (user_id, ts), we can first set a default
+   policy(which policy value contains `*`)
+   that for all partitions matched (user_id=*/) will expire in 7 days, and explicitly set a policy that for partition
+   matched user_id=3 will expire in 30 days. Explicit policy will override default polices, which means default policies
+   only takes effects on partitions that do not match any explicit ttl policies.
+4. We may need to support record level ttl policy in the future. So partition TTL policy should be an implementation of
+   HoodieTTLPolicy.
+
+So we have the TTL policy definition like this (may change when implementing):
+
+```java
+public class HoodiePartitionTTLPolicy implements HoodieTTLPolicy {
+  public enum TTLPolicy {
+    // We supports keep by last modified time at the first version
+    KEEP_BY_TIME
+  }
+
+  // Partition spec for which the policy takes effect, could be a regex or a static partition value
+  private String partitionSpec;

Review Comment:
   Yes, it supports regex expressions for partitions and static partition value



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


[GitHub] [hudi] SteNicholas commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "SteNicholas (via GitHub)" <gi...@apache.org>.
SteNicholas commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1211556787


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,163 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and
+maintain proper statistics to find expired partitions by themselves. As the scale of installations grew, it's more
+important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Gathering proper partition statistics for TTL management
+- Executing TTL management
+
+### TTL Policy Definition
+
+We have four main considerations when designing TTL policy:
+
+1. Usually user hopes to manage partition TTL by expired time, so we support the policy `KEEP_BY_TIME` that partitions
+   will expire N days after their last modified time. Note that the last modified time here means any inserts/updates to
+   the partition. Also, we will make it easy to extend new policies so that users can implement their own polices to
+   decide which partitions are expired.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned
+   by `user_id`. For partition(user_id='1'), we set the policy that this partition will expire 7 days after its last
+   modified time, while for
+   partition(user_id='2') we can set the policy that it will expire 30 days after its last modified time.
+3. It's possible that users have multi-fields partitioning, and they don't want set policy for all partition. So we need
+   to support partition regex when defining
+   partition TTL policies. For example, for a hudi table partitioned by (user_id, ts), we can first set a default
+   policy(which policy value contains `*`)
+   that for all partitions matched (user_id=*/) will expire in 7 days, and explicitly set a policy that for partition
+   matched user_id=3 will expire in 30 days. Explicit policy will override default polices, which means default policies
+   only takes effects on partitions that do not match any explicit ttl policies.
+4. We may need to support record level ttl policy in the future. So partition TTL policy should be an implementation of
+   HoodieTTLPolicy.
+
+So we have the TTL policy definition like this (may change when implementing):
+
+```java
+public class HoodiePartitionTTLPolicy implements HoodieTTLPolicy {
+  public enum TTLPolicy {
+    // We supports keep by last modified time at the first version
+    KEEP_BY_TIME
+  }
+
+  // Partition spec for which the policy takes effect, could be a regex or a static partition value
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy Definition
+
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.
+Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as
+follows.
+
+```sparksql
+-- Set policy for all user_id using partition regex, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');

Review Comment:
   What the time unit of `policyValue`?



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


[GitHub] [hudi] SteNicholas commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "SteNicholas (via GitHub)" <gi...@apache.org>.
SteNicholas commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1211710556


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,163 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and
+maintain proper statistics to find expired partitions by themselves. As the scale of installations grew, it's more
+important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Gathering proper partition statistics for TTL management
+- Executing TTL management
+
+### TTL Policy Definition
+
+We have four main considerations when designing TTL policy:
+
+1. Usually user hopes to manage partition TTL by expired time, so we support the policy `KEEP_BY_TIME` that partitions
+   will expire N days after their last modified time. Note that the last modified time here means any inserts/updates to
+   the partition. Also, we will make it easy to extend new policies so that users can implement their own polices to
+   decide which partitions are expired.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned
+   by `user_id`. For partition(user_id='1'), we set the policy that this partition will expire 7 days after its last
+   modified time, while for
+   partition(user_id='2') we can set the policy that it will expire 30 days after its last modified time.
+3. It's possible that users have multi-fields partitioning, and they don't want set policy for all partition. So we need
+   to support partition regex when defining
+   partition TTL policies. For example, for a hudi table partitioned by (user_id, ts), we can first set a default
+   policy(which policy value contains `*`)
+   that for all partitions matched (user_id=*/) will expire in 7 days, and explicitly set a policy that for partition
+   matched user_id=3 will expire in 30 days. Explicit policy will override default polices, which means default policies
+   only takes effects on partitions that do not match any explicit ttl policies.
+4. We may need to support record level ttl policy in the future. So partition TTL policy should be an implementation of
+   HoodieTTLPolicy.
+
+So we have the TTL policy definition like this (may change when implementing):
+
+```java
+public class HoodiePartitionTTLPolicy implements HoodieTTLPolicy {
+  public enum TTLPolicy {
+    // We supports keep by last modified time at the first version
+    KEEP_BY_TIME
+  }
+
+  // Partition spec for which the policy takes effect, could be a regex or a static partition value
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy Definition
+
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.
+Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as
+follows.
+
+```sparksql
+-- Set policy for all user_id using partition regex, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');
+
+--For partition user_id=3/, keep the data for 7 day.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=3/', policy => 'KEEP_BY_TIME', policyValue => '7');

Review Comment:
   If adding different ttl policy on the same partition, which ttl policy would valid?



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


[GitHub] [hudi] stream2000 commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "stream2000 (via GitHub)" <gi...@apache.org>.
stream2000 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1154120123


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.
+    2. **KEEP_BY_COUNT**. Keep N sub-partitions for a  high-level partition. When sub partition count exceeds, delete the partitions with smaller partition values until the sub-partition count meets the policy configuration.
+    3. **KEEP_BY_SIZE**. Similar to KEEP_BY_COUNT, but to ensure that the sum of the data size of all sub-partitions does not exceed the policy configuration.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned by two fields (user_id, ts). For partition(user_id='1'), we set the policy to keep 100G data for all sub-partitions, and for partition(user_id='2') we set the policy that all sub-partitions will expire 10 days after their last modified time.
+3. It's possible that there are a lot of high-level partitions in the user's table,  and they don't want to set TTL policies for all the high-level partitions. So we need to provide a default policy mechanism so that users can set a default policy for all high-level partitions and add some explicit policies for some of them if needed. Explicit policies will override the default policy.
+
+So here we have the TTL policy definition:
+```java
+public class HoodiePartitionTTLPolicy {
+  public enum TTLPolicy {
+    KEEP_BY_TIME, KEEP_BY_SIZE, KEEP_BY_COUNT
+  }
+
+  // Partition spec for which the policy takes effect
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.  Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as follows.
+
+```sql
+-- Set default policy for all user_id, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');
+ 
+--For partition user_id=1/, keep 10 sub partitions.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=1/', policy => 'KEEP_BY_COUNT', policyValue => '10');
+
+--For partition user_id=2/, keep 100GB data in total
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=2/', policy => 'KEEP_BY_SIZE', policyValue => '107374182400');
+
+--For partition user_id=3/, keep the data for 7 day.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=3/', policy => 'KEEP_BY_TIME', policyValue => '7');
+
+-- Show all the TTL policies including default and explicit policies
+call show_ttl_policies(table => 'test');
+user_id=*/	KEEP_BY_TIME	30
+user_id=1/	KEEP_BY_COUNT	10
+user_id=2/	KEEP_BY_SIZE	107374182400
+user_id=3/	KEEP_BY_TIME	7
+```
+
+### Storage for TTL policy
+The partition TTL policies will be stored in `hoodie.properties`since it is part of table metadata. The policy configs in `hoodie.properties`are defined as follows. Explicit policies are defined using a JSON array while default policy is defined using separate configs.
+
+```properties
+# Default TTL policy definition
+hoodie.partition.ttl.management.default.policy=KEEP_BY_TIME
+hoodie.partition.ttl.management.default.fields=user_id
+hoodie.partition.ttl.management.default.policy.value=30
+
+# Explicit TTL policy definition
+hoodie.partition.ttl.policies=[{"partitionSpec"\:"user_id\=2/","policy"\:"KEEP_BY_SIZE","policyValue"\:107374182400},{"partitionSpec"\:"user_id\=1/","policy"\:"KEEP_BY_COUNT","policyValue"\:10},{"partitionSpec"\:"user_id\=3/","policy"\:"KEEP_BY_TIME","policyValue"\:7}]
+```
+
+### Partition Statistics
+#### Partition Statistics Entity
+We need need to maintain a partition stat for every partition in the hudi table. The stat will contain three fields:
+
+- RelativePath. Relative path to the base path, or we can call it PartitionPath.
+- LastModifiedTime. Last instant time that modified the partition. We need this information to support  the `KEEP_BY_TIME` policy.
+- PartitionTotalFileSize. The sum of all valid data file sizes in the partition, to support the `KEEP_BY_SIZE`policy. We calculate only the latest file slices in the partition currently.

Review Comment:
   Yes, we calculate only the latest file slice. If we want to calculate all the file slices instead of just the latest file slice, we can add a config to control the behavior or adding another stat field. Here we choose to calculate only the latest file slice because we think it reveals the real data size of the file group.



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


[GitHub] [hudi] geserdugarov commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1334928958


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 
+
+For the first version of TTL management, we do not plan to implement a complicated strategy (For example, use an array to store strategies, introduce partition regex etc.). Instead, we add a new abstract method `getPartitionPathsForTTLManagement` in `PartitionTTLManagementStrategy` and provides a new config `hoodie.partition.ttl.management.partition.selected`. 
+
+If `hoodie.partition.ttl.management.partition.selected` is set, `getPartitionPathsForTTLManagement` will return partitions provided by this config. If not, `getPartitionPathsForTTLManagement` will return all partitions in the hudi table. 
+
+TTL management strategies will only be applied for partitions return by `getPartitionPathsForTTLManagement`. 
+
+Thus, if users want to apply different strategies for different partitions, they can do the TTL management multiple times, selecting different partitions and apply corresponding strategies. And we may provide a batch interface in the future to simplify this. 

Review Comment:
   So, if I want to implement 6 partition policies, then I should prepare 6 `PartitionTTLManagementStrategy`s and call them 6 times? Again, TTL is a simple property and I don't think that it should be so hard to set it.
   
   I offer to allow user set only `spec` and `TTL` values with support of `add`, `show`, and `delete` operations. We can also implement `dry run` for checking partition list.
   Then during TTL checking:
   - Iterate through a list of all partitions and compare them to `spec`.
   Later, listing of all partitions could be optimized by storing it in a separate data structure. But the tough part in working with this data structure would be providing consistency in the case of concurrent writing, when new partitions are created and some old partitions are removed.
   - Compare partition to the list of policies.
   We can order this list by TTL value in ascending or descending order. The order would be defined by a resolution strategy for multiple policies conflict: `MIN_TTL` or `MAX_TTL` correspondingly.
   - And then we can compare current time with `lastModifiedTime`.
   Later, storing of this property could be optimized to decrease reading from HDFS, for instance, by adding this data to the mentioned partition related data structure.



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "zyclove (via GitHub)" <gi...@apache.org>.
zyclove commented on PR #8062:
URL: https://github.com/apache/hudi/pull/8062#issuecomment-1826719670

   Will version 1.0 support this feature? This feature is greatly needed.
   @danny0405 @stream2000 


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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1398623725


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,248 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a lifecycle management mechanism to prevent the
+dataset from growing infinitely.

Review Comment:
   Guess you are talking about GDPR?



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1405592067


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 

Review Comment:
   Kind if think it is valuable if the specification could be a wildcard matching. We need to handle the lex parsing of the spec string though.



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1413459990


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 
+
+For the first version of TTL management, we do not plan to implement a complicated strategy (For example, use an array to store strategies, introduce partition regex etc.). Instead, we add a new abstract method `getPartitionPathsForTTLManagement` in `PartitionTTLManagementStrategy` and provides a new config `hoodie.partition.ttl.management.partition.selected`. 
+
+If `hoodie.partition.ttl.management.partition.selected` is set, `getPartitionPathsForTTLManagement` will return partitions provided by this config. If not, `getPartitionPathsForTTLManagement` will return all partitions in the hudi table. 
+
+TTL management strategies will only be applied for partitions return by `getPartitionPathsForTTLManagement`. 
+
+Thus, if users want to apply different strategies for different partitions, they can do the TTL management multiple times, selecting different partitions and apply corresponding strategies. And we may provide a batch interface in the future to simplify this. 

Review Comment:
   I think it's better to start with simple partition TTL. If you want I can do it. I have ideas about design with specs supported wildcards, simple conflict resolution strategy, and I've already tested the implementation. All my ideas I've mentioned in the corresponding comments.



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1411530672


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,210 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy

Review Comment:
   hoodie.partition.ttl.strategy.class ?



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1411530357


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME

Review Comment:
   `hoodie.partition.ttl.strategy` ?



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


[GitHub] [hudi] xicm commented on pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "xicm (via GitHub)" <gi...@apache.org>.
xicm commented on PR #8062:
URL: https://github.com/apache/hudi/pull/8062#issuecomment-1645047203

   Partition TTL can cover most scenarios, will bring a lot of convenience.
    I have a simple record level TTL idea. We configure a general TTL, update the query api to ensure that expired data can't not queried, and then delete the expired record when we do  compaction or clustering.


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


[GitHub] [hudi] SteNicholas commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "SteNicholas (via GitHub)" <gi...@apache.org>.
SteNicholas commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1198516174


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.
+    2. **KEEP_BY_COUNT**. Keep N sub-partitions for a  high-level partition. When sub partition count exceeds, delete the partitions with smaller partition values until the sub-partition count meets the policy configuration.
+    3. **KEEP_BY_SIZE**. Similar to KEEP_BY_COUNT, but to ensure that the sum of the data size of all sub-partitions does not exceed the policy configuration.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned by two fields (user_id, ts). For partition(user_id='1'), we set the policy to keep 100G data for all sub-partitions, and for partition(user_id='2') we set the policy that all sub-partitions will expire 10 days after their last modified time.
+3. It's possible that there are a lot of high-level partitions in the user's table,  and they don't want to set TTL policies for all the high-level partitions. So we need to provide a default policy mechanism so that users can set a default policy for all high-level partitions and add some explicit policies for some of them if needed. Explicit policies will override the default policy.
+
+So here we have the TTL policy definition:
+```java
+public class HoodiePartitionTTLPolicy {

Review Comment:
   Could we introduce `HoodieTTLPolicy` interface? Then `HoodiePartitionTTLPolicy` implements the `HoodieTTLPolicy`. `HoodieRecordTTLPolicy` could also implement this interface in feature.



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


[GitHub] [hudi] stream2000 commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "stream2000 (via GitHub)" <gi...@apache.org>.
stream2000 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1154110772


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.
+    2. **KEEP_BY_COUNT**. Keep N sub-partitions for a  high-level partition. When sub partition count exceeds, delete the partitions with smaller partition values until the sub-partition count meets the policy configuration.
+    3. **KEEP_BY_SIZE**. Similar to KEEP_BY_COUNT, but to ensure that the sum of the data size of all sub-partitions does not exceed the policy configuration.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned by two fields (user_id, ts). For partition(user_id='1'), we set the policy to keep 100G data for all sub-partitions, and for partition(user_id='2') we set the policy that all sub-partitions will expire 10 days after their last modified time.
+3. It's possible that there are a lot of high-level partitions in the user's table,  and they don't want to set TTL policies for all the high-level partitions. So we need to provide a default policy mechanism so that users can set a default policy for all high-level partitions and add some explicit policies for some of them if needed. Explicit policies will override the default policy.
+
+So here we have the TTL policy definition:
+```java
+public class HoodiePartitionTTLPolicy {
+  public enum TTLPolicy {
+    KEEP_BY_TIME, KEEP_BY_SIZE, KEEP_BY_COUNT
+  }
+
+  // Partition spec for which the policy takes effect
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.  Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as follows.
+
+```sql
+-- Set default policy for all user_id, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');
+ 
+--For partition user_id=1/, keep 10 sub partitions.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=1/', policy => 'KEEP_BY_COUNT', policyValue => '10');
+
+--For partition user_id=2/, keep 100GB data in total
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=2/', policy => 'KEEP_BY_SIZE', policyValue => '107374182400');
+
+--For partition user_id=3/, keep the data for 7 day.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=3/', policy => 'KEEP_BY_TIME', policyValue => '7');
+
+-- Show all the TTL policies including default and explicit policies
+call show_ttl_policies(table => 'test');
+user_id=*/	KEEP_BY_TIME	30
+user_id=1/	KEEP_BY_COUNT	10
+user_id=2/	KEEP_BY_SIZE	107374182400
+user_id=3/	KEEP_BY_TIME	7
+```
+
+### Storage for TTL policy
+The partition TTL policies will be stored in `hoodie.properties`since it is part of table metadata. The policy configs in `hoodie.properties`are defined as follows. Explicit policies are defined using a JSON array while default policy is defined using separate configs.
+
+```properties
+# Default TTL policy definition
+hoodie.partition.ttl.management.default.policy=KEEP_BY_TIME
+hoodie.partition.ttl.management.default.fields=user_id
+hoodie.partition.ttl.management.default.policy.value=30
+
+# Explicit TTL policy definition
+hoodie.partition.ttl.policies=[{"partitionSpec"\:"user_id\=2/","policy"\:"KEEP_BY_SIZE","policyValue"\:107374182400},{"partitionSpec"\:"user_id\=1/","policy"\:"KEEP_BY_COUNT","policyValue"\:10},{"partitionSpec"\:"user_id\=3/","policy"\:"KEEP_BY_TIME","policyValue"\:7}]

Review Comment:
   Yes, sure. We can infer default policy from a single hoodie.partition.ttl.policies config. 



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


[GitHub] [hudi] stream2000 commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "stream2000 (via GitHub)" <gi...@apache.org>.
stream2000 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1209111046


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.

Review Comment:
   Yes, inserts/updates will be treated as modification to the partition. And we track them by looking the commit/deltacommit write stats. 



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.
+    2. **KEEP_BY_COUNT**. Keep N sub-partitions for a  high-level partition. When sub partition count exceeds, delete the partitions with smaller partition values until the sub-partition count meets the policy configuration.
+    3. **KEEP_BY_SIZE**. Similar to KEEP_BY_COUNT, but to ensure that the sum of the data size of all sub-partitions does not exceed the policy configuration.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned by two fields (user_id, ts). For partition(user_id='1'), we set the policy to keep 100G data for all sub-partitions, and for partition(user_id='2') we set the policy that all sub-partitions will expire 10 days after their last modified time.
+3. It's possible that there are a lot of high-level partitions in the user's table,  and they don't want to set TTL policies for all the high-level partitions. So we need to provide a default policy mechanism so that users can set a default policy for all high-level partitions and add some explicit policies for some of them if needed. Explicit policies will override the default policy.
+
+So here we have the TTL policy definition:
+```java
+public class HoodiePartitionTTLPolicy {
+  public enum TTLPolicy {
+    KEEP_BY_TIME, KEEP_BY_SIZE, KEEP_BY_COUNT
+  }
+
+  // Partition spec for which the policy takes effect
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.  Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as follows.
+
+```sql
+-- Set default policy for all user_id, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');
+ 
+--For partition user_id=1/, keep 10 sub partitions.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=1/', policy => 'KEEP_BY_COUNT', policyValue => '10');
+
+--For partition user_id=2/, keep 100GB data in total
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=2/', policy => 'KEEP_BY_SIZE', policyValue => '107374182400');
+
+--For partition user_id=3/, keep the data for 7 day.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=3/', policy => 'KEEP_BY_TIME', policyValue => '7');
+
+-- Show all the TTL policies including default and explicit policies
+call show_ttl_policies(table => 'test');
+user_id=*/	KEEP_BY_TIME	30
+user_id=1/	KEEP_BY_COUNT	10
+user_id=2/	KEEP_BY_SIZE	107374182400
+user_id=3/	KEEP_BY_TIME	7
+```
+
+### Storage for TTL policy
+The partition TTL policies will be stored in `hoodie.properties`since it is part of table metadata. The policy configs in `hoodie.properties`are defined as follows. Explicit policies are defined using a JSON array while default policy is defined using separate configs.
+
+```properties
+# Default TTL policy definition
+hoodie.partition.ttl.management.default.policy=KEEP_BY_TIME
+hoodie.partition.ttl.management.default.fields=user_id
+hoodie.partition.ttl.management.default.policy.value=30
+
+# Explicit TTL policy definition
+hoodie.partition.ttl.policies=[{"partitionSpec"\:"user_id\=2/","policy"\:"KEEP_BY_SIZE","policyValue"\:107374182400},{"partitionSpec"\:"user_id\=1/","policy"\:"KEEP_BY_COUNT","policyValue"\:10},{"partitionSpec"\:"user_id\=3/","policy"\:"KEEP_BY_TIME","policyValue"\:7}]

Review Comment:
   > @stream2000, could one partition have multiple ttl policy? If could, which ttl policy would be the main?
   
   @SteNicholas If we support multiple ttl policies for one partition we do need to find the main one or just make sure the result meets all the policies.  We will support only one policy for one partition in the first version and we can extend it to support more policy for one partition in the future. 



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.
+    2. **KEEP_BY_COUNT**. Keep N sub-partitions for a  high-level partition. When sub partition count exceeds, delete the partitions with smaller partition values until the sub-partition count meets the policy configuration.
+    3. **KEEP_BY_SIZE**. Similar to KEEP_BY_COUNT, but to ensure that the sum of the data size of all sub-partitions does not exceed the policy configuration.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned by two fields (user_id, ts). For partition(user_id='1'), we set the policy to keep 100G data for all sub-partitions, and for partition(user_id='2') we set the policy that all sub-partitions will expire 10 days after their last modified time.
+3. It's possible that there are a lot of high-level partitions in the user's table,  and they don't want to set TTL policies for all the high-level partitions. So we need to provide a default policy mechanism so that users can set a default policy for all high-level partitions and add some explicit policies for some of them if needed. Explicit policies will override the default policy.
+
+So here we have the TTL policy definition:
+```java
+public class HoodiePartitionTTLPolicy {

Review Comment:
   @SteNicholas Nice suggestions!  I will take it into consideration when implementing the policy and make sure we can integrate more type of TTL policy in the future



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.
+    2. **KEEP_BY_COUNT**. Keep N sub-partitions for a  high-level partition. When sub partition count exceeds, delete the partitions with smaller partition values until the sub-partition count meets the policy configuration.
+    3. **KEEP_BY_SIZE**. Similar to KEEP_BY_COUNT, but to ensure that the sum of the data size of all sub-partitions does not exceed the policy configuration.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned by two fields (user_id, ts). For partition(user_id='1'), we set the policy to keep 100G data for all sub-partitions, and for partition(user_id='2') we set the policy that all sub-partitions will expire 10 days after their last modified time.
+3. It's possible that there are a lot of high-level partitions in the user's table,  and they don't want to set TTL policies for all the high-level partitions. So we need to provide a default policy mechanism so that users can set a default policy for all high-level partitions and add some explicit policies for some of them if needed. Explicit policies will override the default policy.
+
+So here we have the TTL policy definition:
+```java
+public class HoodiePartitionTTLPolicy {
+  public enum TTLPolicy {
+    KEEP_BY_TIME, KEEP_BY_SIZE, KEEP_BY_COUNT
+  }
+
+  // Partition spec for which the policy takes effect
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.  Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as follows.
+
+```sql
+-- Set default policy for all user_id, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');

Review Comment:
   Sure! We can add a  parameter named `type` which defaults to 'partition'



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


[GitHub] [hudi] stream2000 commented on pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "stream2000 (via GitHub)" <gi...@apache.org>.
stream2000 commented on PR #8062:
URL: https://github.com/apache/hudi/pull/8062#issuecomment-1566859694

   > @stream2000, could we also introduce record ttl management? Partition ttl management and record ttl management both need the ttl policy.
   
   @SteNicholas It will be nice to have record-level ttl management~ We can open another RFC for record-level TTL management and concentrate on Partition TTL Management in this RFC. We can also discuss more on how to make it easy to integrate other types of ttl policy in this RFC. 


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


[GitHub] [hudi] SteNicholas commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "SteNicholas (via GitHub)" <gi...@apache.org>.
SteNicholas commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1211477923


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,163 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and
+maintain proper statistics to find expired partitions by themselves. As the scale of installations grew, it's more
+important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation

Review Comment:
   Could we introduce the public interfaces before `Implementation` section? Like ttl management service interfaces, ttl execution interface etc?



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


[GitHub] [hudi] SteNicholas commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "SteNicholas (via GitHub)" <gi...@apache.org>.
SteNicholas commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1211549183


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,163 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.

Review Comment:
   Calling commands doesn't work for Flink engine. What's the way of partition ttl management for Flink side.



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition TTL Management [hudi]

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1334928958


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 
+
+For the first version of TTL management, we do not plan to implement a complicated strategy (For example, use an array to store strategies, introduce partition regex etc.). Instead, we add a new abstract method `getPartitionPathsForTTLManagement` in `PartitionTTLManagementStrategy` and provides a new config `hoodie.partition.ttl.management.partition.selected`. 
+
+If `hoodie.partition.ttl.management.partition.selected` is set, `getPartitionPathsForTTLManagement` will return partitions provided by this config. If not, `getPartitionPathsForTTLManagement` will return all partitions in the hudi table. 
+
+TTL management strategies will only be applied for partitions return by `getPartitionPathsForTTLManagement`. 
+
+Thus, if users want to apply different strategies for different partitions, they can do the TTL management multiple times, selecting different partitions and apply corresponding strategies. And we may provide a batch interface in the future to simplify this. 

Review Comment:
   So, if I want to implement 6 partition policies, then I should prepare 6 `PartitionTTLManagementStrategy`s and call them 6 times? TTL is a simple property and I don't think that it should be so hard to set it.
   
   I offer to allow user set only `spec` and `TTL` values with support of `add`, `show`, and `delete` operations. We can also implement `dry run` for checking partition list.
   Then during TTL checking:
   - Iterate through a list of all partitions and compare them to `spec`.
   Later, listing of all partitions could be optimized by storing it in a separate data structure. But the tough part in working with this data structure would be providing consistency in the case of concurrent writing, when new partitions are created and some old partitions are removed.
   - Compare partition to the list of policies.
   We can order this list by TTL value in ascending or descending order. The order would be defined by a resolution strategy for multiple policies conflict: `MIN_TTL` or `MAX_TTL` correspondingly.
   - And then we can compare current time with `lastModifiedTime`.
   Later, storing of this property could be optimized to decrease reading from HDFS, for instance, by adding this data to the mentioned partition related data structure.



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition TTL Management [hudi]

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1334248487


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.

Review Comment:
   Sorry to be difficult, it's just providing TTL functionality by custom implementation of `PartitionTTLManagementStrategy` is not user friendly.
   We want to automate detection of outdated partitions and calling `delete_partition`. Could we just allow user to set partition path specification with TTL value, and implement everything internally?
   
   From my point of view, there are two main entities in TTL:
   - object
   In our case, it's partition, we define it by using `spec`.
   - definition of outdating
   It should be time or something time-dependent. In our case, we could compare difference of a current time and `_hoodie_commit_time` with user-defined delta value.
   
   This is a main scope for TTL, and we shouldn't allow to have more flexibility.
   Customized implementation of `PartitionTTLManagementStrategy` will allow to do anything with partitions. It still could be `PartitionManagementStrategy`, but then we shouldn't named it with `TTL` part.



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1411533324


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,210 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+For 1.0.0 and later hudi version which supports efficient completion time queries on the timeline(#9565), we can get partition's `lastModifiedTime` by scanning the timeline and get the last write commit for the partition. Also for efficiency, we can store the partitions' last modified time and current completion time in the replace commit metadata. The next time we need to calculate the partitions' last modified time, we can build incrementally from the replace commit metadata of the last ttl management.

Review Comment:
   We will only support the feature in 1.0 right? Before 1.0, we do not have good manner to know when a log file is committed in a file slice and that might be a snag when calcurating the `lastCommitTime`.



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1411533758


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,210 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+For 1.0.0 and later hudi version which supports efficient completion time queries on the timeline(#9565), we can get partition's `lastModifiedTime` by scanning the timeline and get the last write commit for the partition. Also for efficiency, we can store the partitions' last modified time and current completion time in the replace commit metadata. The next time we need to calculate the partitions' last modified time, we can build incrementally from the replace commit metadata of the last ttl management.
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 
+
+For the first version of TTL management, we do not plan to implement a complicated strategy (For example, use an array to store strategies, introduce partition regex etc.). Instead, we add a new abstract method `getPartitionPathsForTTLManagement` in `PartitionTTLManagementStrategy` and provides a new config `hoodie.partition.ttl.management.partition.selected`. 
+
+If `hoodie.partition.ttl.management.partition.selected` is set, `getPartitionPathsForTTLManagement` will return partitions provided by this config. If not, `getPartitionPathsForTTLManagement` will return all partitions in the hudi table. 
+
+TTL management strategies will only be applied for partitions return by `getPartitionPathsForTTLManagement`. 
+
+Thus, if users want to apply different strategies for different partitions, they can do the TTL management multiple times, selecting different partitions and apply corresponding strategies. And we may provide a batch interface in the future to simplify this. 
+
+The `getPartitionPathsForTTLManagement` method will look like this:
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+   /**
+    * Scan and list all partitions for partition TTL management.
+    *
+    * @return Partitions to apply TTL management strategy
+    */
+   protected List<String> getPartitionPathsForTTLManagement() {
+      if (StringUtils.isNullOrEmpty(config.getTTLManagementPartitionSelected())) {
+        return getMatchedPartitions();
+      } else {
+        // Return All partition paths
+        return FSUtils.getAllPartitionPaths(hoodieTable.getContext(), config.getMetadataConfig(), config.getBasePath());
+      }

Review Comment:
   Where does the `hoodieTable.` got instantiated?



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1413444578


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.

Review Comment:
   Currently, HoodiePartitionMetadata, `.hoodie_partition_metadata` files, provides only commitTime, commit time of partition creation, and partitionDepth properties. This file is written during partition creation and is not updated later.
   
   We can add new property, for instance, `lastUpdateTime` to save time of the last upsert/delete operation on the partition with each commit/deltacommit/replacecommit. And use this property to handle for Partition TTL check.



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1413459990


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 
+
+For the first version of TTL management, we do not plan to implement a complicated strategy (For example, use an array to store strategies, introduce partition regex etc.). Instead, we add a new abstract method `getPartitionPathsForTTLManagement` in `PartitionTTLManagementStrategy` and provides a new config `hoodie.partition.ttl.management.partition.selected`. 
+
+If `hoodie.partition.ttl.management.partition.selected` is set, `getPartitionPathsForTTLManagement` will return partitions provided by this config. If not, `getPartitionPathsForTTLManagement` will return all partitions in the hudi table. 
+
+TTL management strategies will only be applied for partitions return by `getPartitionPathsForTTLManagement`. 
+
+Thus, if users want to apply different strategies for different partitions, they can do the TTL management multiple times, selecting different partitions and apply corresponding strategies. And we may provide a batch interface in the future to simplify this. 

Review Comment:
   I think it's better to start with simple partition TTL. If you want I can do it. I have ideas about design with specs supported wildcards, simple conflict resolution strategy, and I've already tested the implementation. All ideas I've mentioned in the corresponding comments.



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "stream2000 (via GitHub)" <gi...@apache.org>.
stream2000 commented on PR #8062:
URL: https://github.com/apache/hudi/pull/8062#issuecomment-1798178547

   Hi community, I have implemeted a poc version of partition lifecycle management in this PR: #9723. This PR provide a simple `KEEP_BY_TIME` strategy and can do the partition lifecycle management inline after commit. This POC implementation will help understand the design mentioned in this RFC. Hope for your review😄


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


[GitHub] [hudi] zyclove commented on pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "zyclove (via GitHub)" <gi...@apache.org>.
zyclove commented on PR #8062:
URL: https://github.com/apache/hudi/pull/8062#issuecomment-1718990106

   Can this feature be supported in advance? Especially hope for the next version after 0.14.
   


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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition TTL Management [hudi]

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1334257222


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.

Review Comment:
   In the current realization `HoodiePartitionMetadata` provides only `commitTime` (partition created commit time) and `partitionDepth` properties. We can add new `lastModifiedTime` property in `.hoodie_partition_metadata`, which is updated on every commit/deltacommit to corresponding partition.
   
   We need only to think about migration from version without partition level TTL to a new one with this feature.



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


[GitHub] [hudi] stream2000 commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "stream2000 (via GitHub)" <gi...@apache.org>.
stream2000 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1211349775


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,163 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and
+maintain proper statistics to find expired partitions by themselves. As the scale of installations grew, it's more
+important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Gathering proper partition statistics for TTL management
+- Executing TTL management
+
+### TTL Policy Definition
+
+We have four main considerations when designing TTL policy:
+
+1. Usually user hopes to manage partition TTL by expired time, so we support the policy `KEEP_BY_TIME` that partitions
+   will expire N days after their last modified time. Note that the last modified time here means any inserts/updates to
+   the partition. Also, we will make it easy to extend new policies so that users can implement their own polices to
+   decide which partitions are expired.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned
+   by `user_id`. For partition(user_id='1'), we set the policy that this partition will expire 7 days after its last
+   modified time, while for
+   partition(user_id='2') we can set the policy that it will expire 30 days after its last modified time.
+3. It's possible that users have multi-fields partitioning, and they don't want set policy for all partition. So we need
+   to support partition regex when defining
+   partition TTL policies. For example, for a hudi table partitioned by (user_id, ts), we can first set a default
+   policy(which policy value contains `*`)
+   that for all partitions matched (user_id=*/) will expire in 7 days, and explicitly set a policy that for partition
+   matched user_id=3 will expire in 30 days. Explicit policy will override default polices, which means default policies
+   only takes effects on partitions that do not match any explicit ttl policies.
+4. We may need to support record level ttl policy in the future. So partition TTL policy should be an implementation of
+   HoodieTTLPolicy.
+
+So we have the TTL policy definition like this (may change when implementing):
+
+```java
+public class HoodiePartitionTTLPolicy implements HoodieTTLPolicy {
+  public enum TTLPolicy {
+    // We supports keep by last modified time at the first version
+    KEEP_BY_TIME
+  }
+
+  // Partition spec for which the policy takes effect, could be a regex or a static partition value
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy Definition
+
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.
+Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as
+follows.
+
+```sparksql
+-- Set policy for all user_id using partition regex, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');
+
+--For partition user_id=3/, keep the data for 7 day.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=3/', policy => 'KEEP_BY_TIME', policyValue => '7');
+
+-- Show all the TTL policies including default and explicit policies
+call show_ttl_policies(table => 'test');
+user_id=*/	KEEP_BY_TIME	30
+user_id=3/	KEEP_BY_TIME	7
+```
+
+### Storage for TTL policy
+
+We need persistent partition ttt policies in hudi table config and users should interact with hudi only by
+setting/getting ttl policies. Hudi then will takes care of doing the ttl management in an async table service.
+
+The partition TTL policies will be stored in `hoodie.properties` and they are defined as follows. Explicit policies are
+defined using a JSON array, note we support regex here.
+
+```properties
+# TTL policy definition
+hoodie.partition.ttl.policies=[{"partitionSpec"\:"user_id\=*/","policy"\:"KEEP_BY_TIME","policyValue"\:7},{"partitionSpec"\:"user_id\=3/","policy"\:"KEEP_BY_TIME","policyValue"\:30}]
+```
+
+### Partition Statistics
+
+We need to gather proper partition statistics for different kinds of partition ttl policies. Specifically, for
+KEEP_BY_TIME policy that we want to support first, we need to know the last modified time for every partition in hudi
+table.
+
+To simplify the design, we decide to use the largest commit time of committed file groups in the partition as partition
+last modified time. But for file groups generated by replace commit, it may not reveal the real insert/update time for
+the file group. We can also introduce other mechanism to get last modified time of the partition, like add a new kind of
+partition metadata or add a new field in metadata table. Open to ideas for this design choice.
+
+### Executing TTL management
+
+Once we have defined the TTL policies and have proper partition statistics, it's easy to apply the policies and delete
+expired partitions. The process of TTL management is very similar to other table services in hudi, we will provide a new
+method called `managePartitionTTL` in `BaseHoodieTableServiceClient` and we can invoke this method in async table
+service, SparkSQL Call Command, cli, JAVA code etc.
+
+We will provide an Async Table Service as the default interface to the users that want to do partition TTL management
+and we can add more interfaces in the future.
+
+User will config the async ttl management service as follows. TTL management will be trigger N commits after last
+TTL replace commit.
+
+```properties
+hoodie.partition.ttl.management.async=true

Review Comment:
   Yes, we can support synchronously managing the partition TTL. Just need to call `tableServiceClient.managePartitionTTL`. 



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


[GitHub] [hudi] stream2000 commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "stream2000 (via GitHub)" <gi...@apache.org>.
stream2000 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1211353656


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,163 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and
+maintain proper statistics to find expired partitions by themselves. As the scale of installations grew, it's more
+important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Gathering proper partition statistics for TTL management
+- Executing TTL management
+
+### TTL Policy Definition
+
+We have four main considerations when designing TTL policy:
+
+1. Usually user hopes to manage partition TTL by expired time, so we support the policy `KEEP_BY_TIME` that partitions
+   will expire N days after their last modified time. Note that the last modified time here means any inserts/updates to
+   the partition. Also, we will make it easy to extend new policies so that users can implement their own polices to
+   decide which partitions are expired.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned
+   by `user_id`. For partition(user_id='1'), we set the policy that this partition will expire 7 days after its last
+   modified time, while for
+   partition(user_id='2') we can set the policy that it will expire 30 days after its last modified time.
+3. It's possible that users have multi-fields partitioning, and they don't want set policy for all partition. So we need
+   to support partition regex when defining
+   partition TTL policies. For example, for a hudi table partitioned by (user_id, ts), we can first set a default
+   policy(which policy value contains `*`)
+   that for all partitions matched (user_id=*/) will expire in 7 days, and explicitly set a policy that for partition
+   matched user_id=3 will expire in 30 days. Explicit policy will override default polices, which means default policies
+   only takes effects on partitions that do not match any explicit ttl policies.
+4. We may need to support record level ttl policy in the future. So partition TTL policy should be an implementation of
+   HoodieTTLPolicy.
+
+So we have the TTL policy definition like this (may change when implementing):
+
+```java
+public class HoodiePartitionTTLPolicy implements HoodieTTLPolicy {
+  public enum TTLPolicy {
+    // We supports keep by last modified time at the first version
+    KEEP_BY_TIME
+  }
+
+  // Partition spec for which the policy takes effect, could be a regex or a static partition value
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy Definition
+
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.
+Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as
+follows.
+
+```sparksql
+-- Set policy for all user_id using partition regex, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');
+
+--For partition user_id=3/, keep the data for 7 day.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=3/', policy => 'KEEP_BY_TIME', policyValue => '7');
+
+-- Show all the TTL policies including default and explicit policies
+call show_ttl_policies(table => 'test');
+user_id=*/	KEEP_BY_TIME	30
+user_id=3/	KEEP_BY_TIME	7
+```
+
+### Storage for TTL policy
+
+We need persistent partition ttt policies in hudi table config and users should interact with hudi only by
+setting/getting ttl policies. Hudi then will takes care of doing the ttl management in an async table service.
+
+The partition TTL policies will be stored in `hoodie.properties` and they are defined as follows. Explicit policies are
+defined using a JSON array, note we support regex here.
+
+```properties
+# TTL policy definition
+hoodie.partition.ttl.policies=[{"partitionSpec"\:"user_id\=*/","policy"\:"KEEP_BY_TIME","policyValue"\:7},{"partitionSpec"\:"user_id\=3/","policy"\:"KEEP_BY_TIME","policyValue"\:30}]
+```
+
+### Partition Statistics
+
+We need to gather proper partition statistics for different kinds of partition ttl policies. Specifically, for
+KEEP_BY_TIME policy that we want to support first, we need to know the last modified time for every partition in hudi
+table.
+
+To simplify the design, we decide to use the largest commit time of committed file groups in the partition as partition
+last modified time. But for file groups generated by replace commit, it may not reveal the real insert/update time for
+the file group. We can also introduce other mechanism to get last modified time of the partition, like add a new kind of
+partition metadata or add a new field in metadata table. Open to ideas for this design choice.
+
+### Executing TTL management
+
+Once we have defined the TTL policies and have proper partition statistics, it's easy to apply the policies and delete
+expired partitions. The process of TTL management is very similar to other table services in hudi, we will provide a new
+method called `managePartitionTTL` in `BaseHoodieTableServiceClient` and we can invoke this method in async table
+service, SparkSQL Call Command, cli, JAVA code etc.
+
+We will provide an Async Table Service as the default interface to the users that want to do partition TTL management
+and we can add more interfaces in the future.
+
+User will config the async ttl management service as follows. TTL management will be trigger N commits after last
+TTL replace commit.
+
+```properties
+hoodie.partition.ttl.management.async=true
+hoodie.partition.ttl.management.min.commits=10
+```
+
+The process of manage partition TTL is as follows:
+
+1. Gather partitions statistics, list all partitions and the largest commit time of committed
+   file groups of each partition as their last modified time.
+2. Apply each TTL policy. For explicit policies, find sub-partitions matched `PartitionSpec` defined in the policy and
+   check if there are expired partitions according to the policy type and size. For default policy, find partitions that
+   do not match any explicit policy and check if they are expired.
+3. For all expired partitions, call delete_partition to delete them, which will generate a replace commit and mark all
+   files in those partitions as replaced. For pending clustering and compaction that affect the target partition to
+   delete,  [HUDI-5553](https://issues.apache.org/jira/browse/HUDI-5553) introduces a pre-check that will throw an
+   exception, and further improvement could be discussed
+   in [HUDI-5663](https://issues.apache.org/jira/browse/HUDI-5663).
+4. Clean then will delete all replaced file groups.
+

Review Comment:
   > What is the definition of the procedure for ttl management?
   
   What do you mean by 'definition of the procedure ttl management'?  We can support run TTL management by SparkSQL and Hudi Cli, it's easy to implement once we have a proper interface in BaseHoodieTableServiceClient.



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


[GitHub] [hudi] stream2000 commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "stream2000 (via GitHub)" <gi...@apache.org>.
stream2000 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1211359555


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,163 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and
+maintain proper statistics to find expired partitions by themselves. As the scale of installations grew, it's more
+important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Gathering proper partition statistics for TTL management
+- Executing TTL management
+
+### TTL Policy Definition
+
+We have four main considerations when designing TTL policy:
+
+1. Usually user hopes to manage partition TTL by expired time, so we support the policy `KEEP_BY_TIME` that partitions
+   will expire N days after their last modified time. Note that the last modified time here means any inserts/updates to
+   the partition. Also, we will make it easy to extend new policies so that users can implement their own polices to
+   decide which partitions are expired.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned
+   by `user_id`. For partition(user_id='1'), we set the policy that this partition will expire 7 days after its last
+   modified time, while for
+   partition(user_id='2') we can set the policy that it will expire 30 days after its last modified time.
+3. It's possible that users have multi-fields partitioning, and they don't want set policy for all partition. So we need
+   to support partition regex when defining
+   partition TTL policies. For example, for a hudi table partitioned by (user_id, ts), we can first set a default
+   policy(which policy value contains `*`)
+   that for all partitions matched (user_id=*/) will expire in 7 days, and explicitly set a policy that for partition
+   matched user_id=3 will expire in 30 days. Explicit policy will override default polices, which means default policies
+   only takes effects on partitions that do not match any explicit ttl policies.
+4. We may need to support record level ttl policy in the future. So partition TTL policy should be an implementation of
+   HoodieTTLPolicy.
+
+So we have the TTL policy definition like this (may change when implementing):
+
+```java
+public class HoodiePartitionTTLPolicy implements HoodieTTLPolicy {
+  public enum TTLPolicy {
+    // We supports keep by last modified time at the first version
+    KEEP_BY_TIME
+  }
+
+  // Partition spec for which the policy takes effect, could be a regex or a static partition value
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy Definition
+
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.
+Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as
+follows.
+
+```sparksql
+-- Set policy for all user_id using partition regex, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');
+
+--For partition user_id=3/, keep the data for 7 day.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=3/', policy => 'KEEP_BY_TIME', policyValue => '7');
+
+-- Show all the TTL policies including default and explicit policies
+call show_ttl_policies(table => 'test');
+user_id=*/	KEEP_BY_TIME	30
+user_id=3/	KEEP_BY_TIME	7
+```
+
+### Storage for TTL policy
+
+We need persistent partition ttt policies in hudi table config and users should interact with hudi only by
+setting/getting ttl policies. Hudi then will takes care of doing the ttl management in an async table service.
+
+The partition TTL policies will be stored in `hoodie.properties` and they are defined as follows. Explicit policies are

Review Comment:
   Thanks for reminding me about this!  We can follow the common process of adding a new field to hoodie.properties. Will take it into consideration when implementing



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


[GitHub] [hudi] stream2000 commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "stream2000 (via GitHub)" <gi...@apache.org>.
stream2000 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1211233703


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.
+    2. **KEEP_BY_COUNT**. Keep N sub-partitions for a  high-level partition. When sub partition count exceeds, delete the partitions with smaller partition values until the sub-partition count meets the policy configuration.
+    3. **KEEP_BY_SIZE**. Similar to KEEP_BY_COUNT, but to ensure that the sum of the data size of all sub-partitions does not exceed the policy configuration.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned by two fields (user_id, ts). For partition(user_id='1'), we set the policy to keep 100G data for all sub-partitions, and for partition(user_id='2') we set the policy that all sub-partitions will expire 10 days after their last modified time.
+3. It's possible that there are a lot of high-level partitions in the user's table,  and they don't want to set TTL policies for all the high-level partitions. So we need to provide a default policy mechanism so that users can set a default policy for all high-level partitions and add some explicit policies for some of them if needed. Explicit policies will override the default policy.
+
+So here we have the TTL policy definition:
+```java
+public class HoodiePartitionTTLPolicy {
+  public enum TTLPolicy {
+    KEEP_BY_TIME, KEEP_BY_SIZE, KEEP_BY_COUNT
+  }
+
+  // Partition spec for which the policy takes effect
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.  Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as follows.
+
+```sql
+-- Set default policy for all user_id, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');
+ 
+--For partition user_id=1/, keep 10 sub partitions.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=1/', policy => 'KEEP_BY_COUNT', policyValue => '10');
+
+--For partition user_id=2/, keep 100GB data in total
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=2/', policy => 'KEEP_BY_SIZE', policyValue => '107374182400');
+
+--For partition user_id=3/, keep the data for 7 day.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=3/', policy => 'KEEP_BY_TIME', policyValue => '7');
+
+-- Show all the TTL policies including default and explicit policies
+call show_ttl_policies(table => 'test');
+user_id=*/	KEEP_BY_TIME	30
+user_id=1/	KEEP_BY_COUNT	10
+user_id=2/	KEEP_BY_SIZE	107374182400
+user_id=3/	KEEP_BY_TIME	7
+```
+
+### Storage for TTL policy
+The partition TTL policies will be stored in `hoodie.properties`since it is part of table metadata. The policy configs in `hoodie.properties`are defined as follows. Explicit policies are defined using a JSON array while default policy is defined using separate configs.
+
+```properties
+# Default TTL policy definition
+hoodie.partition.ttl.management.default.policy=KEEP_BY_TIME
+hoodie.partition.ttl.management.default.fields=user_id
+hoodie.partition.ttl.management.default.policy.value=30
+
+# Explicit TTL policy definition
+hoodie.partition.ttl.policies=[{"partitionSpec"\:"user_id\=2/","policy"\:"KEEP_BY_SIZE","policyValue"\:107374182400},{"partitionSpec"\:"user_id\=1/","policy"\:"KEEP_BY_COUNT","policyValue"\:10},{"partitionSpec"\:"user_id\=3/","policy"\:"KEEP_BY_TIME","policyValue"\:7}]
+```
+
+### Partition Statistics
+#### Partition Statistics Entity
+We need need to maintain a partition stat for every partition in the hudi table. The stat will contain three fields:
+
+- RelativePath. Relative path to the base path, or we can call it PartitionPath.
+- LastModifiedTime. Last instant time that modified the partition. We need this information to support  the `KEEP_BY_TIME` policy.
+- PartitionTotalFileSize. The sum of all valid data file sizes in the partition, to support the `KEEP_BY_SIZE`policy. We calculate only the latest file slices in the partition currently.
+#### Gathering Partition Statistics
+The simplest way to get partition statistics is that reading information from the metadata table. However, the write and compaction of the metadata table will severely affect the throughput and latency of data ingestion.  So we design an asynchronous partition statistics as follows.
+
+- At the first time we gather the partitions statistics, we list all partitions in the hudi table and calculate `PartitionTotalFileSize`and `LastModifiedTime`for each partition. Then we store all the partition stats in persistent storage along with the instant time we do the stats.  For example, we can directly store all partition stats in a JSON file whose file name contains the instant time.

Review Comment:
   > Why a separate json file? Why not augment the delete partition commit metadata to have all this info?
   
   In our case, the number of partitions is so large that we can only get partition stats in an incremental manner. So we introduce a service that will periodically gather stats of modified partitions from timeline metadata and store the stats in the filesystem to accelerate ttl management. 
   
   However, when the partition number is not so large, listing all partitions to get partition stats is acceptable. So the JSON file can be deemed as a kind of optimization and we may not need it in the first community version. 



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


[GitHub] [hudi] SteNicholas commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "SteNicholas (via GitHub)" <gi...@apache.org>.
SteNicholas commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1211557516


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,163 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and
+maintain proper statistics to find expired partitions by themselves. As the scale of installations grew, it's more
+important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Gathering proper partition statistics for TTL management
+- Executing TTL management
+
+### TTL Policy Definition
+
+We have four main considerations when designing TTL policy:
+
+1. Usually user hopes to manage partition TTL by expired time, so we support the policy `KEEP_BY_TIME` that partitions
+   will expire N days after their last modified time. Note that the last modified time here means any inserts/updates to
+   the partition. Also, we will make it easy to extend new policies so that users can implement their own polices to
+   decide which partitions are expired.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned
+   by `user_id`. For partition(user_id='1'), we set the policy that this partition will expire 7 days after its last
+   modified time, while for
+   partition(user_id='2') we can set the policy that it will expire 30 days after its last modified time.
+3. It's possible that users have multi-fields partitioning, and they don't want set policy for all partition. So we need
+   to support partition regex when defining
+   partition TTL policies. For example, for a hudi table partitioned by (user_id, ts), we can first set a default
+   policy(which policy value contains `*`)
+   that for all partitions matched (user_id=*/) will expire in 7 days, and explicitly set a policy that for partition
+   matched user_id=3 will expire in 30 days. Explicit policy will override default polices, which means default policies
+   only takes effects on partitions that do not match any explicit ttl policies.
+4. We may need to support record level ttl policy in the future. So partition TTL policy should be an implementation of
+   HoodieTTLPolicy.
+
+So we have the TTL policy definition like this (may change when implementing):
+
+```java
+public class HoodiePartitionTTLPolicy implements HoodieTTLPolicy {
+  public enum TTLPolicy {
+    // We supports keep by last modified time at the first version
+    KEEP_BY_TIME
+  }
+
+  // Partition spec for which the policy takes effect, could be a regex or a static partition value
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy Definition
+
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.
+Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as
+follows.
+
+```sparksql
+-- Set policy for all user_id using partition regex, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');
+
+--For partition user_id=3/, keep the data for 7 day.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=3/', policy => 'KEEP_BY_TIME', policyValue => '7');
+

Review Comment:
   How could user remove certain ttl policy?



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1413452143


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 

Review Comment:
   We can decrease complexity by not implementing different strategies for TTL processing and implement only one, which will use only time. And add processing of policies by wildcards.



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1413444578


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.

Review Comment:
   Currently, HoodiePartitionMetadata, `.hoodie_partition_metadata` files, provides only commitTime and partitionDepth properties. This file is written during partition creation and is not updated later. We can make this file more usable by adding a new property, for instance, `lastUpdateTime` for saving time of the last upsert/delete operation on the partition with each commit/deltacommit/replacecommit. And use this property to handle for Partition TTL check.



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "stream2000 (via GitHub)" <gi...@apache.org>.
stream2000 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1414833910


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.

Review Comment:
   Again, leverage `.hoodie_partition_metadata` will bring format change, and it doesn't support any kind of transaction currently. As discussed with @danny0405 , In 1.0.0 and later version which supports efficient completion time queries on the timeline(#9565), we will have a more elegant way to get the `lastCommitTime`.
   You can see the the updated RFC for details. 



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME

Review Comment:
   It's better to provide the `hoodie.partition.ttl.strategy` abtraction so that users can implement their own strategies and reuse the infrastructures provided by RFC-65 to delete expired partitions. For example, user wants to add a new strategy that delete partitions by their specified logic. 
   
   In current design, we will implement `KEEP_BY_TIME` only.  However,  it doesn't mean that we don't need any other kind of ttl strategy in the future. 



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,210 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+For 1.0.0 and later hudi version which supports efficient completion time queries on the timeline(#9565), we can get partition's `lastModifiedTime` by scanning the timeline and get the last write commit for the partition. Also for efficiency, we can store the partitions' last modified time and current completion time in the replace commit metadata. The next time we need to calculate the partitions' last modified time, we can build incrementally from the replace commit metadata of the last ttl management.

Review Comment:
   You're right. We should support this feature in 1.0 only to have an accurate `lastCommitTime`.



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,210 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();

Review Comment:
   see poc code: https://github.com/apache/hudi/pull/9723/files#diff-854a305c2cf1bbe8545bbe78866f53e84a489b60390ea1cfc7f64ed61165a0aaR33
   
   Will provide a  constructor like: 
   
   ``` java 
   
   public PartitionTTLManagementStrategy(HoodieTable hoodieTable) {
       this.writeConfig = hoodieTable.getConfig();
       this.hoodieTable = hoodieTable;
    }
   
   
   



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 

Review Comment:
   You can fire an RFC about your design of the wildcard spec version of TTL strategy,  make it clear about how user should use this kind of strategy, the data structure of the strategy, how to store them(and problems about it), etc. We can have a discussion about issues of this kind strategy in that RFC.
   
   And implement of this advanced strategy can be added in the future by implementing a new kind of ttl strategy. 
   
   I prefer to provide a simple strategy that we can configure with a few simple and easy-to-understand write configs.  @danny0405 What do you think? 



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 
+
+For the first version of TTL management, we do not plan to implement a complicated strategy (For example, use an array to store strategies, introduce partition regex etc.). Instead, we add a new abstract method `getPartitionPathsForTTLManagement` in `PartitionTTLManagementStrategy` and provides a new config `hoodie.partition.ttl.management.partition.selected`. 
+
+If `hoodie.partition.ttl.management.partition.selected` is set, `getPartitionPathsForTTLManagement` will return partitions provided by this config. If not, `getPartitionPathsForTTLManagement` will return all partitions in the hudi table. 

Review Comment:
   We can remove this `hoodie.partition.ttl.management.partition.selected` after  we  implement the advance strategy in the future. 



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,210 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy

Review Comment:
   Yes, sure. Will change it later.



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.

Review Comment:
   > Unfortunately, I don't see any other strategies except KEEP_BY_TIME for partition level TTL.
   
   > size: not applicable, could be trigger for lower levels
   
   We need to make the design extensible and help users with their own expire logic easy to use. I think it's better to have a strategy-level abstraction. 



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 
+
+For the first version of TTL management, we do not plan to implement a complicated strategy (For example, use an array to store strategies, introduce partition regex etc.). Instead, we add a new abstract method `getPartitionPathsForTTLManagement` in `PartitionTTLManagementStrategy` and provides a new config `hoodie.partition.ttl.management.partition.selected`. 
+
+If `hoodie.partition.ttl.management.partition.selected` is set, `getPartitionPathsForTTLManagement` will return partitions provided by this config. If not, `getPartitionPathsForTTLManagement` will return all partitions in the hudi table. 
+
+TTL management strategies will only be applied for partitions return by `getPartitionPathsForTTLManagement`. 
+
+Thus, if users want to apply different strategies for different partitions, they can do the TTL management multiple times, selecting different partitions and apply corresponding strategies. And we may provide a batch interface in the future to simplify this. 

Review Comment:
   You can fire an RFC about this, and we can have a discussion about your design in detail. And it's not conflict with the design in RFC-65. 



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,210 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+For 1.0.0 and later hudi version which supports efficient completion time queries on the timeline(#9565), we can get partition's `lastModifiedTime` by scanning the timeline and get the last write commit for the partition. Also for efficiency, we can store the partitions' last modified time and current completion time in the replace commit metadata. The next time we need to calculate the partitions' last modified time, we can build incrementally from the replace commit metadata of the last ttl management.
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 
+
+For the first version of TTL management, we do not plan to implement a complicated strategy (For example, use an array to store strategies, introduce partition regex etc.). Instead, we add a new abstract method `getPartitionPathsForTTLManagement` in `PartitionTTLManagementStrategy` and provides a new config `hoodie.partition.ttl.management.partition.selected`. 
+
+If `hoodie.partition.ttl.management.partition.selected` is set, `getPartitionPathsForTTLManagement` will return partitions provided by this config. If not, `getPartitionPathsForTTLManagement` will return all partitions in the hudi table. 
+
+TTL management strategies will only be applied for partitions return by `getPartitionPathsForTTLManagement`. 
+
+Thus, if users want to apply different strategies for different partitions, they can do the TTL management multiple times, selecting different partitions and apply corresponding strategies. And we may provide a batch interface in the future to simplify this. 
+
+The `getPartitionPathsForTTLManagement` method will look like this:
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+   /**
+    * Scan and list all partitions for partition TTL management.
+    *
+    * @return Partitions to apply TTL management strategy
+    */
+   protected List<String> getPartitionPathsForTTLManagement() {
+      if (StringUtils.isNullOrEmpty(config.getTTLManagementPartitionSelected())) {
+        return getMatchedPartitions();
+      } else {
+        // Return All partition paths
+        return FSUtils.getAllPartitionPaths(hoodieTable.getContext(), config.getMetadataConfig(), config.getBasePath());
+      }

Review Comment:
   In the constructor of `PartitionTTLManagementStrategy` like
   
   ```java
     protected final HoodieTable hoodieTable;
     protected final HoodieWriteConfig writeConfig;
     public PartitionTTLManagementStrategy(HoodieTable hoodieTable) {
       this.writeConfig = hoodieTable.getConfig();
       this.hoodieTable = hoodieTable;
     }
   ```



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1415523349


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 

Review Comment:
   Ok, I have wrote all my ideas in a separate RFC: https://github.com/apache/hudi/pull/10248
   @stream2000 , @danny0405 if it wouldn't bother you, could you, please, look at it?



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1411533758


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,210 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+For 1.0.0 and later hudi version which supports efficient completion time queries on the timeline(#9565), we can get partition's `lastModifiedTime` by scanning the timeline and get the last write commit for the partition. Also for efficiency, we can store the partitions' last modified time and current completion time in the replace commit metadata. The next time we need to calculate the partitions' last modified time, we can build incrementally from the replace commit metadata of the last ttl management.
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 
+
+For the first version of TTL management, we do not plan to implement a complicated strategy (For example, use an array to store strategies, introduce partition regex etc.). Instead, we add a new abstract method `getPartitionPathsForTTLManagement` in `PartitionTTLManagementStrategy` and provides a new config `hoodie.partition.ttl.management.partition.selected`. 
+
+If `hoodie.partition.ttl.management.partition.selected` is set, `getPartitionPathsForTTLManagement` will return partitions provided by this config. If not, `getPartitionPathsForTTLManagement` will return all partitions in the hudi table. 
+
+TTL management strategies will only be applied for partitions return by `getPartitionPathsForTTLManagement`. 
+
+Thus, if users want to apply different strategies for different partitions, they can do the TTL management multiple times, selecting different partitions and apply corresponding strategies. And we may provide a batch interface in the future to simplify this. 
+
+The `getPartitionPathsForTTLManagement` method will look like this:
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+   /**
+    * Scan and list all partitions for partition TTL management.
+    *
+    * @return Partitions to apply TTL management strategy
+    */
+   protected List<String> getPartitionPathsForTTLManagement() {
+      if (StringUtils.isNullOrEmpty(config.getTTLManagementPartitionSelected())) {
+        return getMatchedPartitions();
+      } else {
+        // Return All partition paths
+        return FSUtils.getAllPartitionPaths(hoodieTable.getContext(), config.getMetadataConfig(), config.getBasePath());
+      }

Review Comment:
   Where does the `hoodieTable.` instantiated in?



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


[GitHub] [hudi] SteNicholas commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "SteNicholas (via GitHub)" <gi...@apache.org>.
SteNicholas commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1211553559


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,163 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and
+maintain proper statistics to find expired partitions by themselves. As the scale of installations grew, it's more
+important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Gathering proper partition statistics for TTL management
+- Executing TTL management
+
+### TTL Policy Definition
+
+We have four main considerations when designing TTL policy:
+
+1. Usually user hopes to manage partition TTL by expired time, so we support the policy `KEEP_BY_TIME` that partitions
+   will expire N days after their last modified time. Note that the last modified time here means any inserts/updates to
+   the partition. Also, we will make it easy to extend new policies so that users can implement their own polices to
+   decide which partitions are expired.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned
+   by `user_id`. For partition(user_id='1'), we set the policy that this partition will expire 7 days after its last
+   modified time, while for
+   partition(user_id='2') we can set the policy that it will expire 30 days after its last modified time.
+3. It's possible that users have multi-fields partitioning, and they don't want set policy for all partition. So we need
+   to support partition regex when defining
+   partition TTL policies. For example, for a hudi table partitioned by (user_id, ts), we can first set a default
+   policy(which policy value contains `*`)
+   that for all partitions matched (user_id=*/) will expire in 7 days, and explicitly set a policy that for partition
+   matched user_id=3 will expire in 30 days. Explicit policy will override default polices, which means default policies
+   only takes effects on partitions that do not match any explicit ttl policies.
+4. We may need to support record level ttl policy in the future. So partition TTL policy should be an implementation of
+   HoodieTTLPolicy.
+
+So we have the TTL policy definition like this (may change when implementing):
+
+```java
+public class HoodiePartitionTTLPolicy implements HoodieTTLPolicy {
+  public enum TTLPolicy {
+    // We supports keep by last modified time at the first version
+    KEEP_BY_TIME
+  }
+
+  // Partition spec for which the policy takes effect, could be a regex or a static partition value
+  private String partitionSpec;

Review Comment:
   @stream2000, for example, there is a Hudi table partitioned by date and hour. Meanwhile, the user want to configure ttl with a year. How could user configure this ttl with current policy definition? Sets the `policyValue ` with a year and `partitionSpec` set to `*/*`?



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


[GitHub] [hudi] stream2000 commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "stream2000 (via GitHub)" <gi...@apache.org>.
stream2000 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1211359555


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,163 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and
+maintain proper statistics to find expired partitions by themselves. As the scale of installations grew, it's more
+important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Gathering proper partition statistics for TTL management
+- Executing TTL management
+
+### TTL Policy Definition
+
+We have four main considerations when designing TTL policy:
+
+1. Usually user hopes to manage partition TTL by expired time, so we support the policy `KEEP_BY_TIME` that partitions
+   will expire N days after their last modified time. Note that the last modified time here means any inserts/updates to
+   the partition. Also, we will make it easy to extend new policies so that users can implement their own polices to
+   decide which partitions are expired.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned
+   by `user_id`. For partition(user_id='1'), we set the policy that this partition will expire 7 days after its last
+   modified time, while for
+   partition(user_id='2') we can set the policy that it will expire 30 days after its last modified time.
+3. It's possible that users have multi-fields partitioning, and they don't want set policy for all partition. So we need
+   to support partition regex when defining
+   partition TTL policies. For example, for a hudi table partitioned by (user_id, ts), we can first set a default
+   policy(which policy value contains `*`)
+   that for all partitions matched (user_id=*/) will expire in 7 days, and explicitly set a policy that for partition
+   matched user_id=3 will expire in 30 days. Explicit policy will override default polices, which means default policies
+   only takes effects on partitions that do not match any explicit ttl policies.
+4. We may need to support record level ttl policy in the future. So partition TTL policy should be an implementation of
+   HoodieTTLPolicy.
+
+So we have the TTL policy definition like this (may change when implementing):
+
+```java
+public class HoodiePartitionTTLPolicy implements HoodieTTLPolicy {
+  public enum TTLPolicy {
+    // We supports keep by last modified time at the first version
+    KEEP_BY_TIME
+  }
+
+  // Partition spec for which the policy takes effect, could be a regex or a static partition value
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy Definition
+
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.
+Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as
+follows.
+
+```sparksql
+-- Set policy for all user_id using partition regex, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');
+
+--For partition user_id=3/, keep the data for 7 day.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=3/', policy => 'KEEP_BY_TIME', policyValue => '7');
+
+-- Show all the TTL policies including default and explicit policies
+call show_ttl_policies(table => 'test');
+user_id=*/	KEEP_BY_TIME	30
+user_id=3/	KEEP_BY_TIME	7
+```
+
+### Storage for TTL policy
+
+We need persistent partition ttt policies in hudi table config and users should interact with hudi only by
+setting/getting ttl policies. Hudi then will takes care of doing the ttl management in an async table service.
+
+The partition TTL policies will be stored in `hoodie.properties` and they are defined as follows. Explicit policies are

Review Comment:
   Thanks for reminding!  We can follow the common process of adding a new field to hoodie.properties. Will take it into consideration when implementing



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


[GitHub] [hudi] nsivabalan commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "nsivabalan (via GitHub)" <gi...@apache.org>.
nsivabalan commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1154738916


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.
+    2. **KEEP_BY_COUNT**. Keep N sub-partitions for a  high-level partition. When sub partition count exceeds, delete the partitions with smaller partition values until the sub-partition count meets the policy configuration.

Review Comment:
   can you help us understand the use-case here. I mean, I am trying to get an understanding of the sub-partitions here. in hudi, we have only one partitioning, but if could be multi-leveled. so, trying to see, if we can keep it high level. 



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.
+    2. **KEEP_BY_COUNT**. Keep N sub-partitions for a  high-level partition. When sub partition count exceeds, delete the partitions with smaller partition values until the sub-partition count meets the policy configuration.
+    3. **KEEP_BY_SIZE**. Similar to KEEP_BY_COUNT, but to ensure that the sum of the data size of all sub-partitions does not exceed the policy configuration.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned by two fields (user_id, ts). For partition(user_id='1'), we set the policy to keep 100G data for all sub-partitions, and for partition(user_id='2') we set the policy that all sub-partitions will expire 10 days after their last modified time.

Review Comment:
   we should be able to add regex and achieve this.
   for eg, 
   Map<{PartitionRegex/Static Partitions{ -> {TTL policy} >
   so, this map can have multiple entries as well.



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.
+    2. **KEEP_BY_COUNT**. Keep N sub-partitions for a  high-level partition. When sub partition count exceeds, delete the partitions with smaller partition values until the sub-partition count meets the policy configuration.

Review Comment:
   I feel, both (2) and (3) is very much catered towards multi-field partitioning like an ProductId/datstr based partitioning. can we layout high level strategies for one level partitioning as well in addition to multi-field partitioning. 



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.
+    2. **KEEP_BY_COUNT**. Keep N sub-partitions for a  high-level partition. When sub partition count exceeds, delete the partitions with smaller partition values until the sub-partition count meets the policy configuration.
+    3. **KEEP_BY_SIZE**. Similar to KEEP_BY_COUNT, but to ensure that the sum of the data size of all sub-partitions does not exceed the policy configuration.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned by two fields (user_id, ts). For partition(user_id='1'), we set the policy to keep 100G data for all sub-partitions, and for partition(user_id='2') we set the policy that all sub-partitions will expire 10 days after their last modified time.
+3. It's possible that there are a lot of high-level partitions in the user's table,  and they don't want to set TTL policies for all the high-level partitions. So we need to provide a default policy mechanism so that users can set a default policy for all high-level partitions and add some explicit policies for some of them if needed. Explicit policies will override the default policy.
+
+So here we have the TTL policy definition:
+```java
+public class HoodiePartitionTTLPolicy {
+  public enum TTLPolicy {
+    KEEP_BY_TIME, KEEP_BY_SIZE, KEEP_BY_COUNT
+  }
+
+  // Partition spec for which the policy takes effect
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.  Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as follows.
+
+```sql
+-- Set default policy for all user_id, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');
+ 
+--For partition user_id=1/, keep 10 sub partitions.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=1/', policy => 'KEEP_BY_COUNT', policyValue => '10');
+
+--For partition user_id=2/, keep 100GB data in total
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=2/', policy => 'KEEP_BY_SIZE', policyValue => '107374182400');
+
+--For partition user_id=3/, keep the data for 7 day.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=3/', policy => 'KEEP_BY_TIME', policyValue => '7');
+
+-- Show all the TTL policies including default and explicit policies
+call show_ttl_policies(table => 'test');
+user_id=*/	KEEP_BY_TIME	30
+user_id=1/	KEEP_BY_COUNT	10
+user_id=2/	KEEP_BY_SIZE	107374182400
+user_id=3/	KEEP_BY_TIME	7
+```
+
+### Storage for TTL policy
+The partition TTL policies will be stored in `hoodie.properties`since it is part of table metadata. The policy configs in `hoodie.properties`are defined as follows. Explicit policies are defined using a JSON array while default policy is defined using separate configs.

Review Comment:
   we should avoid using hoodie.properties for storing write configs. for eg, we don't store cleaning/compaction sceduling/execution strategies in hoodie.properties. 
   Users can start w/ 100GB as TTL policy and later change it to 50GB for instance. So, these are strictly write configs in my opinion. 
   



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.

Review Comment:
   what is last mod time. is it referring to new inserts, or updates as well ? 



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.
+    2. **KEEP_BY_COUNT**. Keep N sub-partitions for a  high-level partition. When sub partition count exceeds, delete the partitions with smaller partition values until the sub-partition count meets the policy configuration.

Review Comment:
   is it possible to simplify the strategies where in we can achieve it for both single or multi field partitioning. for eg, 
   TTL any partition whose last mod time (last time when data was added/updated), is > 1 month for eg. this will work for both a single field partitioning (datestr), or multi-field (productId/datestr). 
   Open to ideas. 



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.
+    2. **KEEP_BY_COUNT**. Keep N sub-partitions for a  high-level partition. When sub partition count exceeds, delete the partitions with smaller partition values until the sub-partition count meets the policy configuration.

Review Comment:
   we should also call out that the sub-partitioning might work only for day based or time based sub-partitioning right. for eg, lets say, if partitioning is datestr/productId. how do we know out of 1000 productIds under a given day, which 100 is older or newer (assuming all 1000 was created in same commit). 



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


[GitHub] [hudi] leesf commented on pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "leesf (via GitHub)" <gi...@apache.org>.
leesf commented on PR #8062:
URL: https://github.com/apache/hudi/pull/8062#issuecomment-1720873143

   @nsivabalan @nbalajee @codope @SteNicholas would you please take another pass since the RFC have been simplified to make it more clear to implement.


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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1398624021


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,248 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a lifecycle management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces partition lifecycle management strategies to hudi, people can config the strategies by write
+configs. With proper configs set, Hudi can find out which partitions are expired and delete them.
+
+This proposal introduces partition lifecycle management service to hudi. Lifecycle management is like other table
+services such as Clean/Compaction/Clustering.
+Users can config their partition lifecycle management strategies through write configs and Hudi will help users find
+expired partitions and delete them automatically.
+
+## Background
+
+Lifecycle management mechanism is an important feature for databases. Hudi already provides a `delete_partition`
+interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of partition lifecycle
+management strategies, find expired partitions and call `delete_partition` by themselves. As the scale of installations
+grew, it is becoming increasingly important to implement a user-friendly lifecycle management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition lifecycle management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or
+  asynchronous table services.
+
+### Strategy Definition
+
+The lifecycle strategies is similar to existing table service strategies. We can define lifecycle strategies like
+defining a clustering/clean/compaction strategy:
+
+```properties
+hoodie.partition.lifecycle.management.strategy=KEEP_BY_TIME
+hoodie.partition.lifecycle.management.strategy.class=org.apache.hudi.table.action.lifecycle.strategy.KeepByTimePartitionLifecycleManagementStrategy

Review Comment:
   `hoodie.partition.lifecycle.management.strategy` -> `hoodie.partition.ttl.strategy` ?



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1398633783


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,248 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a lifecycle management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces partition lifecycle management strategies to hudi, people can config the strategies by write
+configs. With proper configs set, Hudi can find out which partitions are expired and delete them.
+
+This proposal introduces partition lifecycle management service to hudi. Lifecycle management is like other table
+services such as Clean/Compaction/Clustering.
+Users can config their partition lifecycle management strategies through write configs and Hudi will help users find
+expired partitions and delete them automatically.
+
+## Background
+
+Lifecycle management mechanism is an important feature for databases. Hudi already provides a `delete_partition`
+interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of partition lifecycle
+management strategies, find expired partitions and call `delete_partition` by themselves. As the scale of installations
+grew, it is becoming increasingly important to implement a user-friendly lifecycle management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition lifecycle management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or
+  asynchronous table services.
+
+### Strategy Definition
+
+The lifecycle strategies is similar to existing table service strategies. We can define lifecycle strategies like
+defining a clustering/clean/compaction strategy:
+
+```properties
+hoodie.partition.lifecycle.management.strategy=KEEP_BY_TIME
+hoodie.partition.lifecycle.management.strategy.class=org.apache.hudi.table.action.lifecycle.strategy.KeepByTimePartitionLifecycleManagementStrategy
+hoodie.partition.lifecycle.days.retain=10
+```
+
+The config `hoodie.partition.lifecycle.management.strategy.class` is to provide a strategy class (subclass
+of `PartitionLifecycleManagementStrategy`) to get expired partition paths to delete.
+And `hoodie.partition.lifecycle.days.retain` is the strategy value used
+by `KeepByTimePartitionLifecycleManagementStrategy` which means that we will expire partitions that haven't been
+modified for this strategy value set. We will cover the `KeepByTimePartitionLifecycleManagementStrategy` strategy in
+detail in the next section.
+
+The core definition of `PartitionLifecycleManagementStrategy` looks like this:
+
+```java
+/**
+ * Strategy for partition-level lifecycle management.
+ */
+public abstract class PartitionLifecycleManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition lifecycle management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionLifecycleManagementStrategy` and hudi will help delete the
+expired partitions.
+
+### KeepByTimePartitionLifecycleManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionLifecycleManagementStrategy` in the first version of partition
+lifecycle management implementation.
+
+The `KeepByTimePartitionLifecycleManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If
+duration between now and 'lastModifiedTime' for the partition is larger than
+what `hoodie.partition.lifecycle.days.retain` configured, `KeepByTimePartitionLifecycleManagementStrategy` will mark
+this partition as an expired partition. We use day as the unit of expired time since it is very common-used for
+datalakes. Open to ideas for this.
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the
+partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However,
+we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy.
+And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example
+using metadata table.
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may
+have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for
+partitions under `product=2` they want to keep for 7 days only.
+
+For the first version of partition lifecycle management, we do not plan to implement a complicated strategy (For
+example, use an array to store strategies, introduce partition regex etc.). Instead, we add a new abstract
+method `getPartitionPathsForLifecycleManagement` in `PartitionLifecycleManagementStrategy` and provides a new
+config `hoodie.partition.lifecycle.management.partition.selected`.
+
+If `hoodie.partition.lifecycle.management.partition.selected` is set, `getPartitionPathsForLifecycleManagement` will
+return partitions provided by this config. If not, `getPartitionPathsForLifecycleManagement` will return all partitions
+in the hudi table.
+
+lifecycle management strategies will only be applied for partitions return by `getPartitionPathsForLifecycleManagement`.
+
+Thus, if users want to apply different strategies for different partitions, they can do the partition lifecycle
+management multiple times, selecting different partitions and apply corresponding strategies. And we may provide a batch
+interface in the future to simplify this.
+
+The `getPartitionPathsForLifecycleManagement` method will look like this:
+
+```java
+/**
+ * Strategy for partition-level lifecycle management.
+ */
+public abstract class PartitionLifecycleManagementStrategy {
+  /**
+   * Scan and list all partitions for partition lifecycle management.
+   *
+   * @return Partitions to apply lifecycle management strategy
+   */
+  protected List<String> getPartitionPathsForLifecycleManagement() {
+    if (StringUtils.isNullOrEmpty(config.getLifecycleManagementPartitionSelected())) {
+      return getMatchedPartitions();
+    } else {
+      // Return All partition paths
+      return FSUtils.getAllPartitionPaths(hoodieTable.getContext(), config.getMetadataConfig(), config.getBasePath());
+    }
+  }
+}
+``` 
+
+### Executing partition lifecycle management
+
+Once we already have a proper `PartitionLifecycleManagementStrategy` implementation, it's easy to execute the partition
+lifecycle management.
+
+```java
+public class SparkPartitionLifecycleManagementActionExecutor<T> extends BaseSparkCommitActionExecutor<T> {
+  @Override
+  public HoodieWriteMetadata<HoodieData<WriteStatus>> execute() {
+    // Construct PartitionLifecycleManagementStrategy
+    PartitionLifecycleManagementStrategy strategy = (PartitionLifecycleManagementStrategy) ReflectionUtils.loadClass(
+        PartitionLifecycleManagementStrategy.checkAndGetPartitionLifecycleManagementStrategy(config),
+        new Class<?>[] {HoodieTable.class, HoodieEngineContext.class, HoodieWriteConfig.class}, table, context, config);
+
+    // Get expired partition paths
+    List<String> expiredPartitions = strategy.getExpiredPartitionPaths();
+
+    // Delete them reusing SparkDeletePartitionCommitActionExecutor
+    return new SparkDeletePartitionCommitActionExecutor<>(context, config, table, instantTime,
+        expiredPartitions).execute();
+  }
+}
+```
+
+We will add a new method `managePartitionLifecycle` in `HoodieTable` and `HoodieSparkCopyOnWriteTable` can implement it
+like this:
+
+```java
+@Override
+public HoodieWriteMetadata<HoodieData<WriteStatus>>managePartitionLifecycle(HoodieEngineContext context,String instantTime){
+    return new SparkPartitionLifecycleManagementActionExecutor<>(context,config,this,instantTime).execute();
+    }
+```
+
+We can call `hoodieTable.managePartitionLifecycle` in independent flink/spark job, in async/sync inline table services
+like clustering/compaction/clean etc.
+
+### User interface for Partition lifecycle Management
+
+We can do partition lifecycle management inline with streaming ingestion job or do it with a independent batch job, for
+both spark and flink engine.
+
+#### Run inline with Streaming Ingestion
+
+Since we can run clustering inline with streaming ingestion job through the following config:
+
+```properties
+hoodie.clustering.async.enabled=true
+hoodie.clustering.async.max.commits=5
+```
+
+We can do similar thing for partition lifecycle management. The config for async partition lifecycle management are:
+
+| Config key                                              | Remarks                                                                                                                        | Default |
+|---------------------------------------------------------|--------------------------------------------------------------------------------------------------------------------------------|---------|
+| hoodie.partition.lifecycle.management.async.enabled     | Enable running of lifecycle management service, asynchronously as writes happen on the table.                                  | False   |

Review Comment:
   SHould we handle the cleaning duty just to the cleaner service? Like we do to the `delete partition` cmds, you just generate a delete partiton commit metadata on the timeline and the cleaner would take care of it.



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "stream2000 (via GitHub)" <gi...@apache.org>.
stream2000 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1406076695


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 

Review Comment:
   >  Kind if think it is valuable if the specification could be a wildcard matching. We need to handle the lex parsing of the spec string though. 
   
   Yes, it's valuable. However, it will introduce more complexity into the design. In our inner version which have implement a wild card match ttl policy, we need to deal with things like: 
   
   1. provide a spark SQL procedure for defining ttl policies, user can add/delete/update policies to hudi table
   2. policies will be stored in `hoodie.properties`.  We need to consider the concurrent modification issue for `hoodie.properties`. 
   3. We need a default policy, so that partitions that does not explicitly define a ttl policy uses the default policy. So, when implement it, we need to first find which partitions have defined TTL, and then find which partitions shoud apply the default policy. 
   4. We need to care about policies defined in different level, for example we have `a,b,c,d` as partition fields, can we define ttl policies in any level for the partition? ( We do not allow this in our inner version) 
   
   However, for most of our users,  they just need a simple ttl policy that just care about last modified time for each partition and do not need the complicated ttl policy ( which need to understand the multi-level ttl policy definition, the procedure to set/unset policies and so on),  so we simplify the design.  
   
   In the simplified version of partition lifecycle management, we can define the policy just by some write configs, and we do not need a default policy or complicated regex rules. Partition lifecycle management now is more like a simple syntactic sugar for `delete_partition` which is easy to understand and use by normal users.  For advanced users, we can provide an advanced policy in the future that takes care of things I mentioned above. 



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1413434722


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME

Review Comment:
   I suppose it's better to implement only one processing for Partition TTL with accounting time only. For KEEP_BY_SIZE partition level looks not suitable. It's appropriate for record level processing.
   So, we don't need this setting: `hoodie.partition.ttl.strategy`.



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1413434722


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME

Review Comment:
   I suppose it's better to implement only one processing for Partition TTL with accounting time only. For KEEP_BY_SIZE partition level looks not suitable. It's appropriate for record level processing.
   So, we don't need `hoodie.partition.ttl.strategy` setting.



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1413454783


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 

Review Comment:
   > We need to care about policies defined in different level, for example we have a,b,c,d as partition fields, can we define ttl policies in any level for the partition?
   
   If partition suitable for the specified policy `spec` then check `lastUpdateTime`, no matter on which level this partition is. I suppose it is more clear behavior.



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1413453307


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 

Review Comment:
   > We need a default policy, so that partitions that does not explicitly define a ttl policy uses the default policy.
   
   But if we have wildcards, then `spec=*` will be a default policy. And it is really confusing that we have some policies in `hoodie.properties` and there is one default policy somewhere else. We need to keep them all in one place.



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1418322239


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,210 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();

Review Comment:
   Then make it clear in the pseudocode code.



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on PR #8062:
URL: https://github.com/apache/hudi/pull/8062#issuecomment-1846473187

   @geserdugarov I'm gonna merge this one first, can you fire your suggestion chagne PR incrementally based on this one? And we can move the discussions there.


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


[GitHub] [hudi] stream2000 commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "stream2000 (via GitHub)" <gi...@apache.org>.
stream2000 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1154123924


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.

Review Comment:
   We can run it in both sync and async manner.



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


[GitHub] [hudi] SteNicholas commented on pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "SteNicholas (via GitHub)" <gi...@apache.org>.
SteNicholas commented on PR #8062:
URL: https://github.com/apache/hudi/pull/8062#issuecomment-1553957249

   @stream2000, could we also introduce record ttl management? Partition ttl management and record ttl management both need the ttl policy.


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


[GitHub] [hudi] geserdugarov commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1334928958


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 
+
+For the first version of TTL management, we do not plan to implement a complicated strategy (For example, use an array to store strategies, introduce partition regex etc.). Instead, we add a new abstract method `getPartitionPathsForTTLManagement` in `PartitionTTLManagementStrategy` and provides a new config `hoodie.partition.ttl.management.partition.selected`. 
+
+If `hoodie.partition.ttl.management.partition.selected` is set, `getPartitionPathsForTTLManagement` will return partitions provided by this config. If not, `getPartitionPathsForTTLManagement` will return all partitions in the hudi table. 
+
+TTL management strategies will only be applied for partitions return by `getPartitionPathsForTTLManagement`. 
+
+Thus, if users want to apply different strategies for different partitions, they can do the TTL management multiple times, selecting different partitions and apply corresponding strategies. And we may provide a batch interface in the future to simplify this. 

Review Comment:
   So, if I want to implement 6 partition policies, then I should prepare 6 `PartitionTTLManagementStrategy`s and call them 6 times?
   
   I offer to allow user set only `spec` and `TTL` values with support of `add`, `show`, and `delete` operations. We can also implement `dry run` for checking partition list.
   Then during TTL checking:
   - Iterate through a list of all partitions and compare them to `spec`.
   Later listing of all partitions could be optimized by storing it in a separate data structure. But the most hard task with this data structure would be providing consistency in the case of concurrent writing with new partition creation and removing some partitions.
   - Compare partition to the list of policies.
   This list would be ordered by TTL value in ascending or descending order. It would be defined by a resolution strategy for multiple policies conflict: use `MIN_TTL` or `MAX_TTL`.
   - Then compare current time with `lastModifiedTime`.
   Later it also could be optimized for decreasing reading from HDFS by adding this data to the mentioned partition related data structure.



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "stream2000 (via GitHub)" <gi...@apache.org>.
stream2000 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1406043167


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,110 @@
+## Proposers
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+## Approvers
+## Status
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+## Abstract
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period of time. The outdated data is useless and costly,  we need a TTL(Time-To-Live) management mechanism to prevent the dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+## Background
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to delete outdated partitions. However, users still need to detect which partitions are outdated and call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and maintain proper statistics to find expired partitions by themself. As the scale of installations grew,  it's more important to implement a user-friendly TTL management mechanism for hudi.
+## Implementation
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Partition statistics for TTL management
+- Appling policies
+### TTL Policy Definition
+We have three main considerations when designing TTL policy:
+
+1. User hopes to manage partition TTL not only by  expired time but also by sub-partitions count and sub-partitions size. So we need to support the following three different TTL policy types.
+    1. **KEEP_BY_TIME**. Partitions will expire N days after their last modified time.

Review Comment:
   In the latest version of the RFC, we use the max instant time of the committed file slices in the partition as the partition's last modified time for simplicity. Otherwise, we need some extra mechanism to get the last modified time. In our inner version, we maintain an extra JSON file and update it incrementally as new instants committed to get the real modified time for the partition. Also, we can use metadata table to track the last modify time. What do you think about this? 



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "stream2000 (via GitHub)" <gi...@apache.org>.
stream2000 commented on PR #8062:
URL: https://github.com/apache/hudi/pull/8062#issuecomment-1833238919

   For 1.0.0 and later hudi version which supports efficient completion time queries on the timeline(#9565), we can get partition's `lastModifiedTime` by scanning the timeline and get the last write commit for the partition. Also for efficiency, we can store the partitions' last modified time and current completion time in the replace commit metadata. The next time we need to calculate the partitions' last modified time, we can build incrementally from the replace commit metadata of the last ttl management.
   
   @danny0405 Added new `lastModifiedTime` calculation method for 1.0.0 and later hudi version. We plan to implement the file listing based `lastModifiedTime` at first and implement the timeline-based `lastModifiedTime` calculation in a separate PR. This will help users with earlier hudi versions easy to pick the function to their code base. 
   
   I have addressed all comments according to online/offline discussions. If there is no other concern, we can move on this~ 


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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1413457079


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 
+
+For the first version of TTL management, we do not plan to implement a complicated strategy (For example, use an array to store strategies, introduce partition regex etc.). Instead, we add a new abstract method `getPartitionPathsForTTLManagement` in `PartitionTTLManagementStrategy` and provides a new config `hoodie.partition.ttl.management.partition.selected`. 
+
+If `hoodie.partition.ttl.management.partition.selected` is set, `getPartitionPathsForTTLManagement` will return partitions provided by this config. If not, `getPartitionPathsForTTLManagement` will return all partitions in the hudi table. 

Review Comment:
   Again, it is too complicated. You have some default TTL, but there are a lot of policies in `hoodie.properties`. And also you have some filter in another place, `hoodie.partition.ttl.management.partition.selected`, which will intersect with all settings mentioned above.
   
   But all this functionality could be done by specs with wildcards. You don't need anything else. And all policies would be stored in one place.



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1413454783


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 

Review Comment:
   > We need to care about policies defined in different level, for example we have a,b,c,d as partition fields, can we define ttl policies in any level for the partition?
   
   If partition path is suitable for the some `spec` policy then compare `lastUpdateTime` of the partition with settings of the policy, no matter on which level this partition is. I suppose it is more clear behavior.



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1413457079


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,209 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management strategies to hudi, people can config the strategies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+
+This proposal introduces Partition TTL Management service to hudi. TTL management is like other table services such as Clean/Compaction/Clustering.
+The user can config their ttl strategies through write configs and Hudi will help users find expired partitions and delete them automatically.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a `delete_partition` interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL strategies, find expired partitions and call call `delete_partition` by themselves. As the scale of installations grew, it is becoming increasingly important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+Our main goals are as follows:
+
+* Providing an extensible framework for partition TTL management.
+* Implement a simple KEEP_BY_TIME strategy, which can be executed through independent Spark job, synchronous or asynchronous table services.
+
+### Strategy Definition
+
+The TTL strategies is similar to existing table service strategies. We can define TTL strategies like defining a clustering/clean/compaction strategy: 
+
+```properties
+hoodie.partition.ttl.management.strategy=KEEP_BY_TIME
+hoodie.partition.ttl.management.strategy.class=org.apache.hudi.table.action.ttl.strategy.KeepByTimePartitionTTLManagementStrategy
+hoodie.partition.ttl.days.retain=10
+```
+
+The config `hoodie.partition.ttl.management.strategy.class` is to provide a strategy class (subclass of `PartitionTTLManagementStrategy`) to get expired partition paths to delete. And `hoodie.partition.ttl.days.retain` is the strategy value used by `KeepByTimePartitionTTLManagementStrategy` which means that we will expire partitions that haven't been modified for this strategy value set. We will cover the `KeepByTimeTTLManagementStrategy` strategy in detail in the next section.
+
+The core definition of `PartitionTTLManagementStrategy` looks like this: 
+
+```java
+/**
+ * Strategy for partition-level TTL management.
+ */
+public abstract class PartitionTTLManagementStrategy {
+  /**
+   * Get expired partition paths for a specific partition ttl management strategy.
+   *
+   * @return Expired partition paths.
+   */
+  public abstract List<String> getExpiredPartitionPaths();
+}
+```
+
+Users can provide their own implementation of `PartitionTTLManagementStrategy` and hudi will help delete the expired partitions.
+
+### KeepByTimeTTLManagementStrategy
+
+We will provide a strategy call `KeepByTimePartitionTTLManagementStrategy` in the first version of partition TTL management implementation.
+
+The `KeepByTimePartitionTTLManagementStrategy` will calculate the `lastModifiedTime` for each input partitions. If duration between now and 'lastModifiedTime' for the partition is larger than what `hoodie.partition.ttl.days.retain` configured, `KeepByTimePartitionTTLManagementStrategy` will mark this partition as an expired partition. We use day as the unit of expired time since it is very common-used for datalakes. Open to ideas for this. 
+
+we will to use the largest commit time of committed file groups in the partition as the partition's
+`lastModifiedTime`. So any write (including normal DMLs, clustering etc.) with larger instant time will change the partition's `lastModifiedTime`.
+
+For file groups generated by replace commit, it may not reveal the real insert/update time for the file group. However, we can assume that we won't do clustering for a partition without new writes for a long time when using the strategy. And in the future, we may introduce a more accurate mechanism to get `lastModifiedTime` of a partition, for example using metadata table. 
+
+### Apply different strategies for different partitions
+
+For some specific users, they may want to apply different strategies for different partitions. For example, they may have multi partition fileds(productId, day). For partitions under `product=1` they want to keep for 30 days while for partitions under `product=2` they want to keep for 7 days only. 
+
+For the first version of TTL management, we do not plan to implement a complicated strategy (For example, use an array to store strategies, introduce partition regex etc.). Instead, we add a new abstract method `getPartitionPathsForTTLManagement` in `PartitionTTLManagementStrategy` and provides a new config `hoodie.partition.ttl.management.partition.selected`. 
+
+If `hoodie.partition.ttl.management.partition.selected` is set, `getPartitionPathsForTTLManagement` will return partitions provided by this config. If not, `getPartitionPathsForTTLManagement` will return all partitions in the hudi table. 

Review Comment:
   Again, it is too complicated. You have some default TTL, but there are a lot of policies in `hoodie.properties`. And also you have some filter in another place, `hoodie.partition.ttl.management.partition.selected`, which will intersect with all mentioned settings.
   
   But all this functionality could be done by specs with wildcards. You don't need anything else. And it would be stored in one place.



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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on PR #8062:
URL: https://github.com/apache/hudi/pull/8062#issuecomment-1837977625

   > @geserdugarov Thanks very much for your review! I think the most important part of the design to be confirmed is whether we need to provide a feature-rich but complicated TTL policy in the first place or just implement a simple but extensible policy.
   > 
   > @geserdugarov @codope @nsivabalan @vinothchandar Hope for your opinion for this ~
   
   Yes, you're right about the main difference in our ideas.


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


Re: [PR] [HUDI-5823][RFC-65] RFC for Partition Lifecycle Management [hudi]

Posted by "geserdugarov (via GitHub)" <gi...@apache.org>.
geserdugarov commented on PR #8062:
URL: https://github.com/apache/hudi/pull/8062#issuecomment-1846475981

   > @geserdugarov I'm gonna merge this one first, can you fire your suggestion chagne PR incrementally based on this one? And we can move the discussions there.
   @danny0405 Ok, I will work on it.
   


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


[GitHub] [hudi] SteNicholas commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "SteNicholas (via GitHub)" <gi...@apache.org>.
SteNicholas commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1211553559


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,163 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and
+maintain proper statistics to find expired partitions by themselves. As the scale of installations grew, it's more
+important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Gathering proper partition statistics for TTL management
+- Executing TTL management
+
+### TTL Policy Definition
+
+We have four main considerations when designing TTL policy:
+
+1. Usually user hopes to manage partition TTL by expired time, so we support the policy `KEEP_BY_TIME` that partitions
+   will expire N days after their last modified time. Note that the last modified time here means any inserts/updates to
+   the partition. Also, we will make it easy to extend new policies so that users can implement their own polices to
+   decide which partitions are expired.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned
+   by `user_id`. For partition(user_id='1'), we set the policy that this partition will expire 7 days after its last
+   modified time, while for
+   partition(user_id='2') we can set the policy that it will expire 30 days after its last modified time.
+3. It's possible that users have multi-fields partitioning, and they don't want set policy for all partition. So we need
+   to support partition regex when defining
+   partition TTL policies. For example, for a hudi table partitioned by (user_id, ts), we can first set a default
+   policy(which policy value contains `*`)
+   that for all partitions matched (user_id=*/) will expire in 7 days, and explicitly set a policy that for partition
+   matched user_id=3 will expire in 30 days. Explicit policy will override default polices, which means default policies
+   only takes effects on partitions that do not match any explicit ttl policies.
+4. We may need to support record level ttl policy in the future. So partition TTL policy should be an implementation of
+   HoodieTTLPolicy.
+
+So we have the TTL policy definition like this (may change when implementing):
+
+```java
+public class HoodiePartitionTTLPolicy implements HoodieTTLPolicy {
+  public enum TTLPolicy {
+    // We supports keep by last modified time at the first version
+    KEEP_BY_TIME
+  }
+
+  // Partition spec for which the policy takes effect, could be a regex or a static partition value
+  private String partitionSpec;

Review Comment:
   @stream2000, for example, there is a Hudi table partitioned by date and hour. Meanwhile, the user want to configure ttl with a year. How could user configure this ttl with current policy definition? Sets the `policyValue ` with a year and `partitionSpec` with `*/*`?



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


[GitHub] [hudi] stream2000 commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "stream2000 (via GitHub)" <gi...@apache.org>.
stream2000 commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1211354010


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,163 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and
+maintain proper statistics to find expired partitions by themselves. As the scale of installations grew, it's more
+important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Gathering proper partition statistics for TTL management
+- Executing TTL management
+
+### TTL Policy Definition
+
+We have four main considerations when designing TTL policy:
+
+1. Usually user hopes to manage partition TTL by expired time, so we support the policy `KEEP_BY_TIME` that partitions
+   will expire N days after their last modified time. Note that the last modified time here means any inserts/updates to
+   the partition. Also, we will make it easy to extend new policies so that users can implement their own polices to
+   decide which partitions are expired.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned
+   by `user_id`. For partition(user_id='1'), we set the policy that this partition will expire 7 days after its last
+   modified time, while for
+   partition(user_id='2') we can set the policy that it will expire 30 days after its last modified time.
+3. It's possible that users have multi-fields partitioning, and they don't want set policy for all partition. So we need
+   to support partition regex when defining
+   partition TTL policies. For example, for a hudi table partitioned by (user_id, ts), we can first set a default
+   policy(which policy value contains `*`)
+   that for all partitions matched (user_id=*/) will expire in 7 days, and explicitly set a policy that for partition
+   matched user_id=3 will expire in 30 days. Explicit policy will override default polices, which means default policies
+   only takes effects on partitions that do not match any explicit ttl policies.
+4. We may need to support record level ttl policy in the future. So partition TTL policy should be an implementation of
+   HoodieTTLPolicy.
+
+So we have the TTL policy definition like this (may change when implementing):
+
+```java
+public class HoodiePartitionTTLPolicy implements HoodieTTLPolicy {

Review Comment:
   Will add more details later



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


[GitHub] [hudi] SteNicholas commented on a diff in pull request #8062: [HUDI-5823][RFC-65] RFC for Partition TTL Management

Posted by "SteNicholas (via GitHub)" <gi...@apache.org>.
SteNicholas commented on code in PR #8062:
URL: https://github.com/apache/hudi/pull/8062#discussion_r1211323130


##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,163 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and
+maintain proper statistics to find expired partitions by themselves. As the scale of installations grew, it's more
+important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Gathering proper partition statistics for TTL management
+- Executing TTL management
+
+### TTL Policy Definition
+
+We have four main considerations when designing TTL policy:
+
+1. Usually user hopes to manage partition TTL by expired time, so we support the policy `KEEP_BY_TIME` that partitions
+   will expire N days after their last modified time. Note that the last modified time here means any inserts/updates to
+   the partition. Also, we will make it easy to extend new policies so that users can implement their own polices to
+   decide which partitions are expired.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned
+   by `user_id`. For partition(user_id='1'), we set the policy that this partition will expire 7 days after its last
+   modified time, while for
+   partition(user_id='2') we can set the policy that it will expire 30 days after its last modified time.
+3. It's possible that users have multi-fields partitioning, and they don't want set policy for all partition. So we need
+   to support partition regex when defining
+   partition TTL policies. For example, for a hudi table partitioned by (user_id, ts), we can first set a default
+   policy(which policy value contains `*`)
+   that for all partitions matched (user_id=*/) will expire in 7 days, and explicitly set a policy that for partition
+   matched user_id=3 will expire in 30 days. Explicit policy will override default polices, which means default policies
+   only takes effects on partitions that do not match any explicit ttl policies.
+4. We may need to support record level ttl policy in the future. So partition TTL policy should be an implementation of
+   HoodieTTLPolicy.
+
+So we have the TTL policy definition like this (may change when implementing):
+
+```java
+public class HoodiePartitionTTLPolicy implements HoodieTTLPolicy {
+  public enum TTLPolicy {
+    // We supports keep by last modified time at the first version
+    KEEP_BY_TIME
+  }
+
+  // Partition spec for which the policy takes effect, could be a regex or a static partition value
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy Definition
+
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.
+Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as
+follows.
+
+```sparksql
+-- Set policy for all user_id using partition regex, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');
+
+--For partition user_id=3/, keep the data for 7 day.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=3/', policy => 'KEEP_BY_TIME', policyValue => '7');
+
+-- Show all the TTL policies including default and explicit policies
+call show_ttl_policies(table => 'test');
+user_id=*/	KEEP_BY_TIME	30
+user_id=3/	KEEP_BY_TIME	7
+```
+
+### Storage for TTL policy
+
+We need persistent partition ttt policies in hudi table config and users should interact with hudi only by
+setting/getting ttl policies. Hudi then will takes care of doing the ttl management in an async table service.
+
+The partition TTL policies will be stored in `hoodie.properties` and they are defined as follows. Explicit policies are
+defined using a JSON array, note we support regex here.
+
+```properties
+# TTL policy definition
+hoodie.partition.ttl.policies=[{"partitionSpec"\:"user_id\=*/","policy"\:"KEEP_BY_TIME","policyValue"\:7},{"partitionSpec"\:"user_id\=3/","policy"\:"KEEP_BY_TIME","policyValue"\:30}]
+```
+
+### Partition Statistics
+
+We need to gather proper partition statistics for different kinds of partition ttl policies. Specifically, for
+KEEP_BY_TIME policy that we want to support first, we need to know the last modified time for every partition in hudi
+table.
+
+To simplify the design, we decide to use the largest commit time of committed file groups in the partition as partition
+last modified time. But for file groups generated by replace commit, it may not reveal the real insert/update time for
+the file group. We can also introduce other mechanism to get last modified time of the partition, like add a new kind of
+partition metadata or add a new field in metadata table. Open to ideas for this design choice.
+
+### Executing TTL management
+
+Once we have defined the TTL policies and have proper partition statistics, it's easy to apply the policies and delete
+expired partitions. The process of TTL management is very similar to other table services in hudi, we will provide a new
+method called `managePartitionTTL` in `BaseHoodieTableServiceClient` and we can invoke this method in async table
+service, SparkSQL Call Command, cli, JAVA code etc.
+
+We will provide an Async Table Service as the default interface to the users that want to do partition TTL management
+and we can add more interfaces in the future.
+
+User will config the async ttl management service as follows. TTL management will be trigger N commits after last
+TTL replace commit.
+
+```properties
+hoodie.partition.ttl.management.async=true
+hoodie.partition.ttl.management.min.commits=10
+```
+
+The process of manage partition TTL is as follows:
+
+1. Gather partitions statistics, list all partitions and the largest commit time of committed
+   file groups of each partition as their last modified time.
+2. Apply each TTL policy. For explicit policies, find sub-partitions matched `PartitionSpec` defined in the policy and
+   check if there are expired partitions according to the policy type and size. For default policy, find partitions that
+   do not match any explicit policy and check if they are expired.
+3. For all expired partitions, call delete_partition to delete them, which will generate a replace commit and mark all
+   files in those partitions as replaced. For pending clustering and compaction that affect the target partition to
+   delete,  [HUDI-5553](https://issues.apache.org/jira/browse/HUDI-5553) introduces a pre-check that will throw an
+   exception, and further improvement could be discussed
+   in [HUDI-5663](https://issues.apache.org/jira/browse/HUDI-5663).
+4. Clean then will delete all replaced file groups.
+

Review Comment:
   What is the definition of the procedure for ttl management?



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,163 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and
+maintain proper statistics to find expired partitions by themselves. As the scale of installations grew, it's more
+important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Gathering proper partition statistics for TTL management
+- Executing TTL management
+
+### TTL Policy Definition
+
+We have four main considerations when designing TTL policy:
+
+1. Usually user hopes to manage partition TTL by expired time, so we support the policy `KEEP_BY_TIME` that partitions
+   will expire N days after their last modified time. Note that the last modified time here means any inserts/updates to
+   the partition. Also, we will make it easy to extend new policies so that users can implement their own polices to
+   decide which partitions are expired.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned
+   by `user_id`. For partition(user_id='1'), we set the policy that this partition will expire 7 days after its last
+   modified time, while for
+   partition(user_id='2') we can set the policy that it will expire 30 days after its last modified time.
+3. It's possible that users have multi-fields partitioning, and they don't want set policy for all partition. So we need
+   to support partition regex when defining
+   partition TTL policies. For example, for a hudi table partitioned by (user_id, ts), we can first set a default
+   policy(which policy value contains `*`)
+   that for all partitions matched (user_id=*/) will expire in 7 days, and explicitly set a policy that for partition
+   matched user_id=3 will expire in 30 days. Explicit policy will override default polices, which means default policies
+   only takes effects on partitions that do not match any explicit ttl policies.
+4. We may need to support record level ttl policy in the future. So partition TTL policy should be an implementation of
+   HoodieTTLPolicy.
+
+So we have the TTL policy definition like this (may change when implementing):
+
+```java
+public class HoodiePartitionTTLPolicy implements HoodieTTLPolicy {
+  public enum TTLPolicy {
+    // We supports keep by last modified time at the first version
+    KEEP_BY_TIME
+  }
+
+  // Partition spec for which the policy takes effect, could be a regex or a static partition value
+  private String partitionSpec;

Review Comment:
   Could the `partitionSpec` support multiple level partition?



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,163 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and
+maintain proper statistics to find expired partitions by themselves. As the scale of installations grew, it's more
+important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Gathering proper partition statistics for TTL management
+- Executing TTL management
+
+### TTL Policy Definition
+
+We have four main considerations when designing TTL policy:
+
+1. Usually user hopes to manage partition TTL by expired time, so we support the policy `KEEP_BY_TIME` that partitions
+   will expire N days after their last modified time. Note that the last modified time here means any inserts/updates to
+   the partition. Also, we will make it easy to extend new policies so that users can implement their own polices to
+   decide which partitions are expired.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned
+   by `user_id`. For partition(user_id='1'), we set the policy that this partition will expire 7 days after its last
+   modified time, while for
+   partition(user_id='2') we can set the policy that it will expire 30 days after its last modified time.
+3. It's possible that users have multi-fields partitioning, and they don't want set policy for all partition. So we need
+   to support partition regex when defining
+   partition TTL policies. For example, for a hudi table partitioned by (user_id, ts), we can first set a default
+   policy(which policy value contains `*`)
+   that for all partitions matched (user_id=*/) will expire in 7 days, and explicitly set a policy that for partition
+   matched user_id=3 will expire in 30 days. Explicit policy will override default polices, which means default policies
+   only takes effects on partitions that do not match any explicit ttl policies.
+4. We may need to support record level ttl policy in the future. So partition TTL policy should be an implementation of
+   HoodieTTLPolicy.
+
+So we have the TTL policy definition like this (may change when implementing):
+
+```java
+public class HoodiePartitionTTLPolicy implements HoodieTTLPolicy {

Review Comment:
   What's the definition of `HoodieTTLPolicy`?



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,163 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and
+maintain proper statistics to find expired partitions by themselves. As the scale of installations grew, it's more
+important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Gathering proper partition statistics for TTL management
+- Executing TTL management
+
+### TTL Policy Definition
+
+We have four main considerations when designing TTL policy:
+
+1. Usually user hopes to manage partition TTL by expired time, so we support the policy `KEEP_BY_TIME` that partitions
+   will expire N days after their last modified time. Note that the last modified time here means any inserts/updates to
+   the partition. Also, we will make it easy to extend new policies so that users can implement their own polices to
+   decide which partitions are expired.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned
+   by `user_id`. For partition(user_id='1'), we set the policy that this partition will expire 7 days after its last
+   modified time, while for
+   partition(user_id='2') we can set the policy that it will expire 30 days after its last modified time.
+3. It's possible that users have multi-fields partitioning, and they don't want set policy for all partition. So we need
+   to support partition regex when defining
+   partition TTL policies. For example, for a hudi table partitioned by (user_id, ts), we can first set a default
+   policy(which policy value contains `*`)
+   that for all partitions matched (user_id=*/) will expire in 7 days, and explicitly set a policy that for partition
+   matched user_id=3 will expire in 30 days. Explicit policy will override default polices, which means default policies
+   only takes effects on partitions that do not match any explicit ttl policies.
+4. We may need to support record level ttl policy in the future. So partition TTL policy should be an implementation of
+   HoodieTTLPolicy.
+
+So we have the TTL policy definition like this (may change when implementing):
+
+```java
+public class HoodiePartitionTTLPolicy implements HoodieTTLPolicy {
+  public enum TTLPolicy {
+    // We supports keep by last modified time at the first version
+    KEEP_BY_TIME
+  }
+
+  // Partition spec for which the policy takes effect, could be a regex or a static partition value
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy Definition
+
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.
+Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as
+follows.
+
+```sparksql
+-- Set policy for all user_id using partition regex, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');
+
+--For partition user_id=3/, keep the data for 7 day.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=3/', policy => 'KEEP_BY_TIME', policyValue => '7');
+
+-- Show all the TTL policies including default and explicit policies
+call show_ttl_policies(table => 'test');
+user_id=*/	KEEP_BY_TIME	30
+user_id=3/	KEEP_BY_TIME	7
+```
+
+### Storage for TTL policy
+
+We need persistent partition ttt policies in hudi table config and users should interact with hudi only by
+setting/getting ttl policies. Hudi then will takes care of doing the ttl management in an async table service.
+
+The partition TTL policies will be stored in `hoodie.properties` and they are defined as follows. Explicit policies are
+defined using a JSON array, note we support regex here.
+
+```properties
+# TTL policy definition
+hoodie.partition.ttl.policies=[{"partitionSpec"\:"user_id\=*/","policy"\:"KEEP_BY_TIME","policyValue"\:7},{"partitionSpec"\:"user_id\=3/","policy"\:"KEEP_BY_TIME","policyValue"\:30}]
+```
+
+### Partition Statistics
+
+We need to gather proper partition statistics for different kinds of partition ttl policies. Specifically, for
+KEEP_BY_TIME policy that we want to support first, we need to know the last modified time for every partition in hudi
+table.
+
+To simplify the design, we decide to use the largest commit time of committed file groups in the partition as partition
+last modified time. But for file groups generated by replace commit, it may not reveal the real insert/update time for
+the file group. We can also introduce other mechanism to get last modified time of the partition, like add a new kind of
+partition metadata or add a new field in metadata table. Open to ideas for this design choice.
+
+### Executing TTL management
+
+Once we have defined the TTL policies and have proper partition statistics, it's easy to apply the policies and delete
+expired partitions. The process of TTL management is very similar to other table services in hudi, we will provide a new
+method called `managePartitionTTL` in `BaseHoodieTableServiceClient` and we can invoke this method in async table
+service, SparkSQL Call Command, cli, JAVA code etc.
+
+We will provide an Async Table Service as the default interface to the users that want to do partition TTL management
+and we can add more interfaces in the future.
+
+User will config the async ttl management service as follows. TTL management will be trigger N commits after last
+TTL replace commit.
+
+```properties
+hoodie.partition.ttl.management.async=true

Review Comment:
   Is there any case that synchronously manages the hoodie.partition.ttl?



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,163 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and
+maintain proper statistics to find expired partitions by themselves. As the scale of installations grew, it's more
+important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Gathering proper partition statistics for TTL management
+- Executing TTL management
+
+### TTL Policy Definition
+
+We have four main considerations when designing TTL policy:
+
+1. Usually user hopes to manage partition TTL by expired time, so we support the policy `KEEP_BY_TIME` that partitions
+   will expire N days after their last modified time. Note that the last modified time here means any inserts/updates to
+   the partition. Also, we will make it easy to extend new policies so that users can implement their own polices to
+   decide which partitions are expired.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned
+   by `user_id`. For partition(user_id='1'), we set the policy that this partition will expire 7 days after its last
+   modified time, while for
+   partition(user_id='2') we can set the policy that it will expire 30 days after its last modified time.
+3. It's possible that users have multi-fields partitioning, and they don't want set policy for all partition. So we need
+   to support partition regex when defining
+   partition TTL policies. For example, for a hudi table partitioned by (user_id, ts), we can first set a default
+   policy(which policy value contains `*`)
+   that for all partitions matched (user_id=*/) will expire in 7 days, and explicitly set a policy that for partition
+   matched user_id=3 will expire in 30 days. Explicit policy will override default polices, which means default policies
+   only takes effects on partitions that do not match any explicit ttl policies.
+4. We may need to support record level ttl policy in the future. So partition TTL policy should be an implementation of
+   HoodieTTLPolicy.
+
+So we have the TTL policy definition like this (may change when implementing):
+
+```java
+public class HoodiePartitionTTLPolicy implements HoodieTTLPolicy {
+  public enum TTLPolicy {
+    // We supports keep by last modified time at the first version
+    KEEP_BY_TIME
+  }
+
+  // Partition spec for which the policy takes effect, could be a regex or a static partition value
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy Definition
+
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.
+Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as
+follows.
+
+```sparksql
+-- Set policy for all user_id using partition regex, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');
+
+--For partition user_id=3/, keep the data for 7 day.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=3/', policy => 'KEEP_BY_TIME', policyValue => '7');
+
+-- Show all the TTL policies including default and explicit policies
+call show_ttl_policies(table => 'test');
+user_id=*/	KEEP_BY_TIME	30
+user_id=3/	KEEP_BY_TIME	7
+```
+
+### Storage for TTL policy
+
+We need persistent partition ttt policies in hudi table config and users should interact with hudi only by
+setting/getting ttl policies. Hudi then will takes care of doing the ttl management in an async table service.
+
+The partition TTL policies will be stored in `hoodie.properties` and they are defined as follows. Explicit policies are

Review Comment:
   Is there any compatibility problem in storing the `hoodie.partition.ttl.policies` in `hoodie.properties`?



##########
rfc/rfc-65/rfc-65.md:
##########
@@ -0,0 +1,163 @@
+## Proposers
+
+- @stream2000
+- @hujincalrin
+- @huberylee
+- @YuweiXiao
+
+## Approvers
+
+## Status
+
+JIRA: [HUDI-5823](https://issues.apache.org/jira/browse/HUDI-5823)
+
+## Abstract
+
+In some classic hudi use cases, users partition hudi data by time and are only interested in data from a recent period
+of time. The outdated data is useless and costly, we need a TTL(Time-To-Live) management mechanism to prevent the
+dataset from growing infinitely.
+This proposal introduces Partition TTL Management policies to hudi, people can config the policies by table config
+directly or by call commands. With proper configs set, Hudi can find out which partitions are outdated and delete them.
+
+## Background
+
+TTL management mechanism is an important feature for databases. Hudi already provides a delete_partition interface to
+delete outdated partitions. However, users still need to detect which partitions are outdated and
+call `delete_partition` manually, which means that users need to define and implement some kind of TTL policies and
+maintain proper statistics to find expired partitions by themselves. As the scale of installations grew, it's more
+important to implement a user-friendly TTL management mechanism for hudi.
+
+## Implementation
+
+There are 3 components to implement Partition TTL Management
+
+- TTL policy definition & storage
+- Gathering proper partition statistics for TTL management
+- Executing TTL management
+
+### TTL Policy Definition
+
+We have four main considerations when designing TTL policy:
+
+1. Usually user hopes to manage partition TTL by expired time, so we support the policy `KEEP_BY_TIME` that partitions
+   will expire N days after their last modified time. Note that the last modified time here means any inserts/updates to
+   the partition. Also, we will make it easy to extend new policies so that users can implement their own polices to
+   decide which partitions are expired.
+2. User need to set different policies for different partitions. For example, the hudi table is partitioned
+   by `user_id`. For partition(user_id='1'), we set the policy that this partition will expire 7 days after its last
+   modified time, while for
+   partition(user_id='2') we can set the policy that it will expire 30 days after its last modified time.
+3. It's possible that users have multi-fields partitioning, and they don't want set policy for all partition. So we need
+   to support partition regex when defining
+   partition TTL policies. For example, for a hudi table partitioned by (user_id, ts), we can first set a default
+   policy(which policy value contains `*`)
+   that for all partitions matched (user_id=*/) will expire in 7 days, and explicitly set a policy that for partition
+   matched user_id=3 will expire in 30 days. Explicit policy will override default polices, which means default policies
+   only takes effects on partitions that do not match any explicit ttl policies.
+4. We may need to support record level ttl policy in the future. So partition TTL policy should be an implementation of
+   HoodieTTLPolicy.
+
+So we have the TTL policy definition like this (may change when implementing):
+
+```java
+public class HoodiePartitionTTLPolicy implements HoodieTTLPolicy {
+  public enum TTLPolicy {
+    // We supports keep by last modified time at the first version
+    KEEP_BY_TIME
+  }
+
+  // Partition spec for which the policy takes effect, could be a regex or a static partition value
+  private String partitionSpec;
+
+  private TTLPolicy policy;
+
+  private long policyValue;
+}
+```
+
+### User Interface for TTL policy Definition
+
+Users can config partition TTL management policies through SparkSQL Call Command and through table config directly.
+Assume that the user has a hudi table partitioned by two fields(user_id, ts), he can config partition TTL policies as
+follows.
+
+```sparksql
+-- Set policy for all user_id using partition regex, which keeps the data for 30 days.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=*/', policy => 'KEEP_BY_TIME', policyValue => '30');
+
+--For partition user_id=3/, keep the data for 7 day.
+call add_ttl_policy(table => 'test', partitionSpec => 'user_id=3/', policy => 'KEEP_BY_TIME', policyValue => '7');
+
+-- Show all the TTL policies including default and explicit policies
+call show_ttl_policies(table => 'test');
+user_id=*/	KEEP_BY_TIME	30
+user_id=3/	KEEP_BY_TIME	7
+```
+
+### Storage for TTL policy
+
+We need persistent partition ttt policies in hudi table config and users should interact with hudi only by
+setting/getting ttl policies. Hudi then will takes care of doing the ttl management in an async table service.
+
+The partition TTL policies will be stored in `hoodie.properties` and they are defined as follows. Explicit policies are
+defined using a JSON array, note we support regex here.
+
+```properties
+# TTL policy definition
+hoodie.partition.ttl.policies=[{"partitionSpec"\:"user_id\=*/","policy"\:"KEEP_BY_TIME","policyValue"\:7},{"partitionSpec"\:"user_id\=3/","policy"\:"KEEP_BY_TIME","policyValue"\:30}]
+```
+
+### Partition Statistics
+
+We need to gather proper partition statistics for different kinds of partition ttl policies. Specifically, for
+KEEP_BY_TIME policy that we want to support first, we need to know the last modified time for every partition in hudi
+table.
+
+To simplify the design, we decide to use the largest commit time of committed file groups in the partition as partition
+last modified time. But for file groups generated by replace commit, it may not reveal the real insert/update time for
+the file group. We can also introduce other mechanism to get last modified time of the partition, like add a new kind of

Review Comment:
   Could how to get partition statistics be confirmed? From the `_hoodie_parition_metadata` or metadata table?



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