Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-04-24 Thread via GitHub


xicm commented on code in PR #9723:
URL: https://github.com/apache/hudi/pull/9723#discussion_r1578761500


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/ttl/strategy/KeepByTimeStrategy.java:
##
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.action.ttl.strategy;
+
+import org.apache.hudi.common.model.FileSlice;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.HoodieTimer;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.table.HoodieTable;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static 
org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator.fixInstantTimeCompatibility;
+import static 
org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator.instantTimePlusMillis;
+
+/**
+ * KeepByTimeStrategy will return expired partitions by their lastCommitTime.
+ */
+public class KeepByTimeStrategy extends PartitionTTLStrategy {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(KeepByTimeStrategy.class);
+
+  protected final long ttlInMilis;
+
+  public KeepByTimeStrategy(HoodieTable hoodieTable, String instantTime) {
+super(hoodieTable, instantTime);
+this.ttlInMilis = writeConfig.getPartitionTTLStrategyDaysRetain() * 1000 * 
3600 * 24;
+  }
+
+  @Override
+  public List getExpiredPartitionPaths() {
+Option lastCompletedInstant = 
hoodieTable.getActiveTimeline().filterCompletedInstants().lastInstant();
+if (!lastCompletedInstant.isPresent() || ttlInMilis <= 0
+|| 
!hoodieTable.getMetaClient().getTableConfig().getPartitionFields().isPresent()) 
{
+  return Collections.emptyList();
+}
+List expiredPartitions = 
getExpiredPartitionsForTimeStrategy(getPartitionPathsForTTL());
+int limit = writeConfig.getPartitionTTLMaxPartitionsToDelete();
+LOG.info("Total expired partitions count {}, limit {}", 
expiredPartitions.size(), limit);
+return expiredPartitions.stream()
+.limit(limit) // Avoid a single replace commit too large
+.collect(Collectors.toList());
+  }
+
+  protected List getExpiredPartitionsForTimeStrategy(List 
partitionsForTTLManagement) {
+HoodieTimer timer = HoodieTimer.start();
+Map> lastCommitTimeForPartitions = 
getLastCommitTimeForPartitions(partitionsForTTLManagement);
+LOG.info("Collect last commit time for partitions cost {} ms", 
timer.endTimer());
+return lastCommitTimeForPartitions.entrySet()
+.stream()
+.filter(entry -> entry.getValue().isPresent())
+.filter(entry -> isPartitionExpired(entry.getValue().get()))
+.map(Map.Entry::getKey)
+.collect(Collectors.toList());
+  }
+
+  /**
+   * @param partitionPaths Partitions to collect stats.
+   */
+  private Map> 
getLastCommitTimeForPartitions(List partitionPaths) {
+int statsParallelism = Math.min(partitionPaths.size(), 200);
+return hoodieTable.getContext().map(partitionPaths, partitionPath -> {
+  Option partitionLastModifiedTime = hoodieTable.getHoodieView()
+  .getLatestFileSlicesBeforeOrOn(partitionPath, instantTime, true)
+  .map(FileSlice::getBaseInstantTime)
+  .max(Comparator.naturalOrder())
+  .map(Option::ofNullable)
+  .orElse(Option.empty());
+  return Pair.of(partitionPath, partitionLastModifiedTime);
+}, statsParallelism).stream().collect(Collectors.toMap(Pair::getKey, 
Pair::getValue));
+  }
+
+  /**
+   * Determines if a partition's reference time has exceeded its time-to-live 
(TTL).
+   * 
+   * This method checks if the current time has passed the TTL threshold based 
on a
+   * reference time, which could be the creation time or the last commit time 
of the partition.
+   *
+   * @param referenceTime last commit time or creation time for partition
+   */
+  protected boolean isPartitionExpired(String referenceTime) {
+String expiredTime = 
instantTimePlusMillis(fixInstantT

Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-04-24 Thread via GitHub


xicm commented on code in PR #9723:
URL: https://github.com/apache/hudi/pull/9723#discussion_r1578761500


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/ttl/strategy/KeepByTimeStrategy.java:
##
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.action.ttl.strategy;
+
+import org.apache.hudi.common.model.FileSlice;
+import org.apache.hudi.common.table.timeline.HoodieInstant;
+import org.apache.hudi.common.util.HoodieTimer;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.table.HoodieTable;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import static 
org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator.fixInstantTimeCompatibility;
+import static 
org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator.instantTimePlusMillis;
+
+/**
+ * KeepByTimeStrategy will return expired partitions by their lastCommitTime.
+ */
+public class KeepByTimeStrategy extends PartitionTTLStrategy {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(KeepByTimeStrategy.class);
+
+  protected final long ttlInMilis;
+
+  public KeepByTimeStrategy(HoodieTable hoodieTable, String instantTime) {
+super(hoodieTable, instantTime);
+this.ttlInMilis = writeConfig.getPartitionTTLStrategyDaysRetain() * 1000 * 
3600 * 24;
+  }
+
+  @Override
+  public List getExpiredPartitionPaths() {
+Option lastCompletedInstant = 
hoodieTable.getActiveTimeline().filterCompletedInstants().lastInstant();
+if (!lastCompletedInstant.isPresent() || ttlInMilis <= 0
+|| 
!hoodieTable.getMetaClient().getTableConfig().getPartitionFields().isPresent()) 
{
+  return Collections.emptyList();
+}
+List expiredPartitions = 
getExpiredPartitionsForTimeStrategy(getPartitionPathsForTTL());
+int limit = writeConfig.getPartitionTTLMaxPartitionsToDelete();
+LOG.info("Total expired partitions count {}, limit {}", 
expiredPartitions.size(), limit);
+return expiredPartitions.stream()
+.limit(limit) // Avoid a single replace commit too large
+.collect(Collectors.toList());
+  }
+
+  protected List getExpiredPartitionsForTimeStrategy(List 
partitionsForTTLManagement) {
+HoodieTimer timer = HoodieTimer.start();
+Map> lastCommitTimeForPartitions = 
getLastCommitTimeForPartitions(partitionsForTTLManagement);
+LOG.info("Collect last commit time for partitions cost {} ms", 
timer.endTimer());
+return lastCommitTimeForPartitions.entrySet()
+.stream()
+.filter(entry -> entry.getValue().isPresent())
+.filter(entry -> isPartitionExpired(entry.getValue().get()))
+.map(Map.Entry::getKey)
+.collect(Collectors.toList());
+  }
+
+  /**
+   * @param partitionPaths Partitions to collect stats.
+   */
+  private Map> 
getLastCommitTimeForPartitions(List partitionPaths) {
+int statsParallelism = Math.min(partitionPaths.size(), 200);
+return hoodieTable.getContext().map(partitionPaths, partitionPath -> {
+  Option partitionLastModifiedTime = hoodieTable.getHoodieView()
+  .getLatestFileSlicesBeforeOrOn(partitionPath, instantTime, true)
+  .map(FileSlice::getBaseInstantTime)
+  .max(Comparator.naturalOrder())
+  .map(Option::ofNullable)
+  .orElse(Option.empty());
+  return Pair.of(partitionPath, partitionLastModifiedTime);
+}, statsParallelism).stream().collect(Collectors.toMap(Pair::getKey, 
Pair::getValue));
+  }
+
+  /**
+   * Determines if a partition's reference time has exceeded its time-to-live 
(TTL).
+   * 
+   * This method checks if the current time has passed the TTL threshold based 
on a
+   * reference time, which could be the creation time or the last commit time 
of the partition.
+   *
+   * @param referenceTime last commit time or creation time for partition
+   */
+  protected boolean isPartitionExpired(String referenceTime) {
+String expiredTime = 
instantTimePlusMillis(fixInstantT

Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-02-27 Thread via GitHub


leesf merged PR #9723:
URL: https://github.com/apache/hudi/pull/9723


-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-02-27 Thread via GitHub


hudi-bot commented on PR #9723:
URL: https://github.com/apache/hudi/pull/9723#issuecomment-1966459840

   
   ## CI report:
   
   * ae3572005431da476574d5cbdf6a324ba93d4725 UNKNOWN
   * 5167205276199e9ed7886d361167ed6cb310cd32 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22668)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-02-27 Thread via GitHub


hudi-bot commented on PR #9723:
URL: https://github.com/apache/hudi/pull/9723#issuecomment-1966264411

   
   ## CI report:
   
   * ae3572005431da476574d5cbdf6a324ba93d4725 UNKNOWN
   * 8712b4dd774255ccf17d9d7c8c354892d1caa677 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22574)
 
   * 5167205276199e9ed7886d361167ed6cb310cd32 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22668)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-02-27 Thread via GitHub


hudi-bot commented on PR #9723:
URL: https://github.com/apache/hudi/pull/9723#issuecomment-1966180129

   
   ## CI report:
   
   * ae3572005431da476574d5cbdf6a324ba93d4725 UNKNOWN
   * 8712b4dd774255ccf17d9d7c8c354892d1caa677 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22574)
 
   * 5167205276199e9ed7886d361167ed6cb310cd32 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-02-27 Thread via GitHub


stream2000 commented on code in PR #9723:
URL: https://github.com/apache/hudi/pull/9723#discussion_r1503923899


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/ttl/strategy/TTLStrategy.java:
##
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.action.ttl.strategy;
+
+/**
+ * Strategy for ttl management.
+ */
+public interface TTLStrategy {

Review Comment:
   Yes, it's reserved for row-level ttl management. 



-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-02-27 Thread via GitHub


leesf commented on code in PR #9723:
URL: https://github.com/apache/hudi/pull/9723#discussion_r1503904245


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/ttl/strategy/TTLStrategy.java:
##
@@ -0,0 +1,25 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.action.ttl.strategy;
+
+/**
+ * Strategy for ttl management.
+ */
+public interface TTLStrategy {

Review Comment:
   this interface is empty with no methods?



-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-02-22 Thread via GitHub


hudi-bot commented on PR #9723:
URL: https://github.com/apache/hudi/pull/9723#issuecomment-1960751397

   
   ## CI report:
   
   * ae3572005431da476574d5cbdf6a324ba93d4725 UNKNOWN
   * 8712b4dd774255ccf17d9d7c8c354892d1caa677 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22574)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-02-22 Thread via GitHub


hudi-bot commented on PR #9723:
URL: https://github.com/apache/hudi/pull/9723#issuecomment-1960688398

   
   ## CI report:
   
   * ae3572005431da476574d5cbdf6a324ba93d4725 UNKNOWN
   * 67ee450c21d566e068ef08696bc3a70a9dc1691d Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22558)
 
   * 8712b4dd774255ccf17d9d7c8c354892d1caa677 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22574)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-02-22 Thread via GitHub


hudi-bot commented on PR #9723:
URL: https://github.com/apache/hudi/pull/9723#issuecomment-1960684133

   
   ## CI report:
   
   * ae3572005431da476574d5cbdf6a324ba93d4725 UNKNOWN
   * 67ee450c21d566e068ef08696bc3a70a9dc1691d Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22558)
 
   * 8712b4dd774255ccf17d9d7c8c354892d1caa677 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-02-22 Thread via GitHub


leesf commented on code in PR #9723:
URL: https://github.com/apache/hudi/pull/9723#discussion_r1499042769


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java:
##
@@ -583,6 +594,12 @@ protected void runTableServicesInline(HoodieTable table, 
HoodieCommitMetadata me
   
metadata.addMetadata(HoodieClusteringConfig.SCHEDULE_INLINE_CLUSTERING.key(), 
"true");
   inlineScheduleClustering(extraMetadata);
 }
+
+//  Do an inline partition ttl management if enabled
+if (config.isAutoPartitionTTL()) {

Review Comment:
isInlinePartitionTTL would be better?



##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java:
##
@@ -583,6 +594,12 @@ protected void runTableServicesInline(HoodieTable table, 
HoodieCommitMetadata me
   
metadata.addMetadata(HoodieClusteringConfig.SCHEDULE_INLINE_CLUSTERING.key(), 
"true");
   inlineScheduleClustering(extraMetadata);
 }
+
+//  Do an inline partition ttl management if enabled
+if (config.isAutoPartitionTTL()) {

Review Comment:
isInlinePartitionTTLEnabled would be better?



-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-02-22 Thread via GitHub


leesf commented on code in PR #9723:
URL: https://github.com/apache/hudi/pull/9723#discussion_r1499037855


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/ttl/strategy/PartitionTTLStrategyType.java:
##
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hudi.table.action.ttl.strategy;
+
+import org.apache.hudi.common.config.HoodieConfig;
+import org.apache.hudi.keygen.constant.KeyGeneratorType;
+
+import javax.annotation.Nullable;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import static 
org.apache.hudi.config.HoodieTTLConfig.PARTITION_TTL_STRATEGY_CLASS_NAME;
+import static 
org.apache.hudi.config.HoodieTTLConfig.PARTITION_TTL_STRATEGY_TYPE;
+
+/**
+ * Types of {@link PartitionTTLStrategy}.
+ */
+public enum PartitionTTLStrategyType {
+  KEEP_BY_TIME("org.apache.hudi.table.action.ttl.strategy.KeepByTimeStrategy"),
+  
KEEP_BY_CREATION_TIME("org.apache.hudi.table.action.ttl.strategy.KeepByCreationTimeStrategy");
+
+  private final String className;
+
+  PartitionTTLStrategyType(String className) {
+this.className = className;
+  }
+
+  public String getClassName() {
+return className;
+  }
+
+  public static PartitionTTLStrategyType fromClassName(String className) {
+for (PartitionTTLStrategyType type : PartitionTTLStrategyType.values()) {
+  if (type.getClassName().equals(className)) {
+return type;
+  }
+}
+throw new IllegalArgumentException("No PartitionTTLStrategyType found for 
class name: " + className);
+  }
+
+  public static List getNames() {

Review Comment:
   rename to getPartitionTTLStrategyNames?



-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-02-22 Thread via GitHub


hudi-bot commented on PR #9723:
URL: https://github.com/apache/hudi/pull/9723#issuecomment-1959139977

   
   ## CI report:
   
   * ae3572005431da476574d5cbdf6a324ba93d4725 UNKNOWN
   * 67ee450c21d566e068ef08696bc3a70a9dc1691d Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22558)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-02-22 Thread via GitHub


hudi-bot commented on PR #9723:
URL: https://github.com/apache/hudi/pull/9723#issuecomment-1959043307

   
   ## CI report:
   
   * ae3572005431da476574d5cbdf6a324ba93d4725 UNKNOWN
   * 0aa63bde940a79f3440e5e6924f99e53d7a368c0 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22505)
 
   * 67ee450c21d566e068ef08696bc3a70a9dc1691d Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22558)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-02-22 Thread via GitHub


hudi-bot commented on PR #9723:
URL: https://github.com/apache/hudi/pull/9723#issuecomment-1959029114

   
   ## CI report:
   
   * ae3572005431da476574d5cbdf6a324ba93d4725 UNKNOWN
   * 0aa63bde940a79f3440e5e6924f99e53d7a368c0 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22505)
 
   * 67ee450c21d566e068ef08696bc3a70a9dc1691d UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-02-19 Thread via GitHub


hudi-bot commented on PR #9723:
URL: https://github.com/apache/hudi/pull/9723#issuecomment-1953549510

   
   ## CI report:
   
   * ae3572005431da476574d5cbdf6a324ba93d4725 UNKNOWN
   * 0aa63bde940a79f3440e5e6924f99e53d7a368c0 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22505)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-02-19 Thread via GitHub


hudi-bot commented on PR #9723:
URL: https://github.com/apache/hudi/pull/9723#issuecomment-1953427726

   
   ## CI report:
   
   * ae3572005431da476574d5cbdf6a324ba93d4725 UNKNOWN
   * 6550cc1c2d572459de1f84199f1b9b9f0c821b4c Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22489)
 
   * 0aa63bde940a79f3440e5e6924f99e53d7a368c0 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22505)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-02-19 Thread via GitHub


hudi-bot commented on PR #9723:
URL: https://github.com/apache/hudi/pull/9723#issuecomment-1953422089

   
   ## CI report:
   
   * ae3572005431da476574d5cbdf6a324ba93d4725 UNKNOWN
   * 6550cc1c2d572459de1f84199f1b9b9f0c821b4c Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=22489)
 
   * 0aa63bde940a79f3440e5e6924f99e53d7a368c0 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-01-15 Thread via GitHub


hudi-bot commented on PR #9723:
URL: https://github.com/apache/hudi/pull/9723#issuecomment-1892014968

   
   ## CI report:
   
   * ae3572005431da476574d5cbdf6a324ba93d4725 UNKNOWN
   * 9ef8e0cdb437792add70aae990319099efb61cf8 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=21964)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-01-15 Thread via GitHub


hudi-bot commented on PR #9723:
URL: https://github.com/apache/hudi/pull/9723#issuecomment-1891618947

   
   ## CI report:
   
   * ae3572005431da476574d5cbdf6a324ba93d4725 UNKNOWN
   * eb82362f82cc417fd10fa1907c4a7668917b1d22 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=21855)
 
   * 9ef8e0cdb437792add70aae990319099efb61cf8 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=21964)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-01-15 Thread via GitHub


hudi-bot commented on PR #9723:
URL: https://github.com/apache/hudi/pull/9723#issuecomment-1891606025

   
   ## CI report:
   
   * ae3572005431da476574d5cbdf6a324ba93d4725 UNKNOWN
   * eb82362f82cc417fd10fa1907c4a7668917b1d22 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=21855)
 
   * 9ef8e0cdb437792add70aae990319099efb61cf8 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-01-07 Thread via GitHub


hudi-bot commented on PR #9723:
URL: https://github.com/apache/hudi/pull/9723#issuecomment-1880440934

   
   ## CI report:
   
   * ae3572005431da476574d5cbdf6a324ba93d4725 UNKNOWN
   * eb82362f82cc417fd10fa1907c4a7668917b1d22 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=21855)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-01-07 Thread via GitHub


hudi-bot commented on PR #9723:
URL: https://github.com/apache/hudi/pull/9723#issuecomment-1880340742

   
   ## CI report:
   
   * ae3572005431da476574d5cbdf6a324ba93d4725 UNKNOWN
   * 7b4d31ec9e571ad4e14c1e7e228424675fa20d50 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=21850)
 
   * eb82362f82cc417fd10fa1907c4a7668917b1d22 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=21855)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-01-07 Thread via GitHub


hudi-bot commented on PR #9723:
URL: https://github.com/apache/hudi/pull/9723#issuecomment-1880336264

   
   ## CI report:
   
   * ae3572005431da476574d5cbdf6a324ba93d4725 UNKNOWN
   * 7b4d31ec9e571ad4e14c1e7e228424675fa20d50 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=21850)
 
   * eb82362f82cc417fd10fa1907c4a7668917b1d22 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-01-06 Thread via GitHub


hudi-bot commented on PR #9723:
URL: https://github.com/apache/hudi/pull/9723#issuecomment-1879590453

   
   ## CI report:
   
   * ae3572005431da476574d5cbdf6a324ba93d4725 UNKNOWN
   * 7b4d31ec9e571ad4e14c1e7e228424675fa20d50 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=21850)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-01-05 Thread via GitHub


hudi-bot commented on PR #9723:
URL: https://github.com/apache/hudi/pull/9723#issuecomment-1879569043

   
   ## CI report:
   
   * ae3572005431da476574d5cbdf6a324ba93d4725 UNKNOWN
   * da26f51a6f2f97adba00012a6cd0eb9a1ce99d56 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=21838)
 
   * 7b4d31ec9e571ad4e14c1e7e228424675fa20d50 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=21850)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-01-05 Thread via GitHub


hudi-bot commented on PR #9723:
URL: https://github.com/apache/hudi/pull/9723#issuecomment-1879560487

   
   ## CI report:
   
   * ae3572005431da476574d5cbdf6a324ba93d4725 UNKNOWN
   * da26f51a6f2f97adba00012a6cd0eb9a1ce99d56 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=21838)
 
   * 7b4d31ec9e571ad4e14c1e7e228424675fa20d50 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-01-05 Thread via GitHub


hudi-bot commented on PR #9723:
URL: https://github.com/apache/hudi/pull/9723#issuecomment-1878569717

   
   ## CI report:
   
   * ae3572005431da476574d5cbdf6a324ba93d4725 UNKNOWN
   * da26f51a6f2f97adba00012a6cd0eb9a1ce99d56 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=21838)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-01-05 Thread via GitHub


hudi-bot commented on PR #9723:
URL: https://github.com/apache/hudi/pull/9723#issuecomment-1878420511

   
   ## CI report:
   
   * 3bdc9bbe380b07cb8e55656714570405cb78 Azure: 
[CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=21718)
 
   * aed6440819d5c894a404977c68d4dc308206dcf0 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=21837)
 
   * ae3572005431da476574d5cbdf6a324ba93d4725 UNKNOWN
   * da26f51a6f2f97adba00012a6cd0eb9a1ce99d56 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=21838)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-01-05 Thread via GitHub


hudi-bot commented on PR #9723:
URL: https://github.com/apache/hudi/pull/9723#issuecomment-1878365424

   
   ## CI report:
   
   * 3bdc9bbe380b07cb8e55656714570405cb78 Azure: 
[CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=21718)
 
   * aed6440819d5c894a404977c68d4dc308206dcf0 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=21837)
 
   * ae3572005431da476574d5cbdf6a324ba93d4725 UNKNOWN
   * da26f51a6f2f97adba00012a6cd0eb9a1ce99d56 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-01-05 Thread via GitHub


hudi-bot commented on PR #9723:
URL: https://github.com/apache/hudi/pull/9723#issuecomment-1878356019

   
   ## CI report:
   
   * 3bdc9bbe380b07cb8e55656714570405cb78 Azure: 
[CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=21718)
 
   * aed6440819d5c894a404977c68d4dc308206dcf0 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=21837)
 
   * ae3572005431da476574d5cbdf6a324ba93d4725 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-01-05 Thread via GitHub


hudi-bot commented on PR #9723:
URL: https://github.com/apache/hudi/pull/9723#issuecomment-1878296567

   
   ## CI report:
   
   * 3bdc9bbe380b07cb8e55656714570405cb78 Azure: 
[CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=21718)
 
   * aed6440819d5c894a404977c68d4dc308206dcf0 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=21837)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2024-01-05 Thread via GitHub


hudi-bot commented on PR #9723:
URL: https://github.com/apache/hudi/pull/9723#issuecomment-1878287788

   
   ## CI report:
   
   * 3bdc9bbe380b07cb8e55656714570405cb78 Azure: 
[CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=21718)
 
   * aed6440819d5c894a404977c68d4dc308206dcf0 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2023-12-27 Thread via GitHub


hudi-bot commented on PR #9723:
URL: https://github.com/apache/hudi/pull/9723#issuecomment-1870457088

   
   ## CI report:
   
   * 3bdc9bbe380b07cb8e55656714570405cb78 Azure: 
[CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=21718)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2023-12-27 Thread via GitHub


hudi-bot commented on PR #9723:
URL: https://github.com/apache/hudi/pull/9723#issuecomment-1870324881

   
   ## CI report:
   
   * 2d82832731aa3c4c80b5d4902b2a1ca267d871c0 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=21674)
 
   * 3bdc9bbe380b07cb8e55656714570405cb78 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=21718)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



Re: [PR] [HUDI-5823] Partition ttl management [hudi]

2023-12-27 Thread via GitHub


hudi-bot commented on PR #9723:
URL: https://github.com/apache/hudi/pull/9723#issuecomment-1870318790

   
   ## CI report:
   
   * 2d82832731aa3c4c80b5d4902b2a1ca267d871c0 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=21674)
 
   * 3bdc9bbe380b07cb8e55656714570405cb78 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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