You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by li...@apache.org on 2022/06/24 08:58:03 UTC

[doris] branch master updated: add dynamic partition DataProperty (#10338)

This is an automated email from the ASF dual-hosted git repository.

lihaopeng pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/doris.git


The following commit(s) were added to refs/heads/master by this push:
     new 8f2b2b4457 add dynamic partition DataProperty (#10338)
8f2b2b4457 is described below

commit 8f2b2b4457df33d3dafee51d2fda72744ee04558
Author: pengxiangyu <di...@163.com>
AuthorDate: Fri Jun 24 16:57:56 2022 +0800

    add dynamic partition DataProperty (#10338)
---
 .../org/apache/doris/catalog/DataProperty.java     | 22 ++++++++--
 .../doris/catalog/DynamicPartitionProperty.java    | 10 ++++-
 .../apache/doris/catalog/TabletInvertedIndex.java  |  9 +++-
 .../doris/clone/DynamicPartitionScheduler.java     | 19 +++++++-
 .../doris/common/util/DynamicPartitionUtil.java    | 30 +++++++++++++
 .../apache/doris/common/util/PropertyAnalyzer.java | 19 +++++++-
 .../org/apache/doris/policy/StoragePolicy.java     | 51 ++++++++++++++++++++--
 .../java/org/apache/doris/alter/AlterTest.java     | 12 ++---
 .../org/apache/doris/catalog/DataPropertyTest.java |  2 +-
 9 files changed, 155 insertions(+), 19 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/DataProperty.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/DataProperty.java
index 5af708384d..5c77653221 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/DataProperty.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/DataProperty.java
@@ -44,6 +44,9 @@ public class DataProperty implements Writable {
     private long cooldownTimeMs;
     @SerializedName(value = "remoteStoragePolicy")
     private String remoteStoragePolicy;
+    // cooldown time for remote storage
+    @SerializedName(value = "remoteCooldownTimeMs")
+    private long remoteCooldownTimeMs;
 
     private DataProperty() {
         // for persist
@@ -58,6 +61,7 @@ public class DataProperty implements Writable {
             this.cooldownTimeMs = MAX_COOLDOWN_TIME_MS;
         }
         this.remoteStoragePolicy = "";
+        this.remoteCooldownTimeMs = MAX_COOLDOWN_TIME_MS;
     }
 
     /**
@@ -66,11 +70,17 @@ public class DataProperty implements Writable {
      * @param medium storage medium for the init storage of the table
      * @param cooldown cool down time for SSD->HDD
      * @param remoteStoragePolicy remote storage policy for remote storage
+     * @param remoteCooldownTimeMs remote storage cooldown time
      */
-    public DataProperty(TStorageMedium medium, long cooldown, String remoteStoragePolicy) {
+    public DataProperty(TStorageMedium medium, long cooldown, String remoteStoragePolicy, long remoteCooldownTimeMs) {
         this.storageMedium = medium;
         this.cooldownTimeMs = cooldown;
         this.remoteStoragePolicy = remoteStoragePolicy;
+        if (remoteCooldownTimeMs > 0) {
+            this.remoteCooldownTimeMs = remoteCooldownTimeMs;
+        } else {
+            this.remoteCooldownTimeMs = MAX_COOLDOWN_TIME_MS;
+        }
     }
 
     public TStorageMedium getStorageMedium() {
@@ -85,6 +95,10 @@ public class DataProperty implements Writable {
         return remoteStoragePolicy;
     }
 
+    public long getRemoteCooldownTimeMs() {
+        return remoteCooldownTimeMs;
+    }
+
     public static DataProperty read(DataInput in) throws IOException {
         if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_108) {
             String json = Text.readString(in);
@@ -109,7 +123,7 @@ public class DataProperty implements Writable {
 
     @Override
     public int hashCode() {
-        return Objects.hash(storageMedium, cooldownTimeMs, remoteStoragePolicy);
+        return Objects.hash(storageMedium, cooldownTimeMs, remoteStoragePolicy, remoteCooldownTimeMs);
     }
 
     @Override
@@ -126,7 +140,8 @@ public class DataProperty implements Writable {
 
         return this.storageMedium == other.storageMedium
                 && this.cooldownTimeMs == other.cooldownTimeMs
-                && this.remoteStoragePolicy.equals(other.remoteStoragePolicy);
+                && this.remoteStoragePolicy.equals(other.remoteStoragePolicy)
+                && this.remoteCooldownTimeMs == other.remoteCooldownTimeMs;
     }
 
     @Override
@@ -135,6 +150,7 @@ public class DataProperty implements Writable {
         sb.append("Storage medium[").append(this.storageMedium).append("]. ");
         sb.append("cool down[").append(TimeUtils.longToTimeString(cooldownTimeMs)).append("]. ");
         sb.append("remote storage policy[").append(this.remoteStoragePolicy).append("]. ");
+        sb.append("remote cooldown time[").append(TimeUtils.longToTimeString(remoteCooldownTimeMs)).append("]. ");
         return sb.toString();
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/DynamicPartitionProperty.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/DynamicPartitionProperty.java
index 0756bc0e85..5b76d43927 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/DynamicPartitionProperty.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/DynamicPartitionProperty.java
@@ -46,6 +46,7 @@ public class DynamicPartitionProperty {
     public static final String HISTORY_PARTITION_NUM = "dynamic_partition.history_partition_num";
     public static final String HOT_PARTITION_NUM = "dynamic_partition.hot_partition_num";
     public static final String RESERVED_HISTORY_PERIODS = "dynamic_partition.reserved_history_periods";
+    public static final String REMOTE_STORAGE_POLICY = "dynamic_partition.remote_storage_policy";
 
     public static final int MIN_START_OFFSET = Integer.MIN_VALUE;
     public static final int MAX_END_OFFSET = Integer.MAX_VALUE;
@@ -72,6 +73,7 @@ public class DynamicPartitionProperty {
     // If not set, default is 0
     private int hotPartitionNum;
     private String reservedHistoryPeriods;
+    private String remoteStoragePolicy;
 
     public DynamicPartitionProperty(Map<String, String> properties) {
         if (properties != null && !properties.isEmpty()) {
@@ -91,6 +93,7 @@ public class DynamicPartitionProperty {
             this.hotPartitionNum = Integer.parseInt(properties.getOrDefault(HOT_PARTITION_NUM, "0"));
             this.reservedHistoryPeriods = properties.getOrDefault(
                     RESERVED_HISTORY_PERIODS, NOT_SET_RESERVED_HISTORY_PERIODS);
+            this.remoteStoragePolicy = properties.getOrDefault(REMOTE_STORAGE_POLICY, "");
             createStartOfs(properties);
         } else {
             this.exist = false;
@@ -170,6 +173,10 @@ public class DynamicPartitionProperty {
         return hotPartitionNum;
     }
 
+    public String getRemoteStoragePolicy() {
+        return remoteStoragePolicy;
+    }
+
     public String getStartOfInfo() {
         if (getTimeUnit().equalsIgnoreCase(TimeUnit.WEEK.toString())) {
             return startOfWeek.toDisplayInfo();
@@ -212,7 +219,8 @@ public class DynamicPartitionProperty {
                 + ",\n\"" + CREATE_HISTORY_PARTITION + "\" = \"" + createHistoryPartition + "\""
                 + ",\n\"" + HISTORY_PARTITION_NUM + "\" = \"" + historyPartitionNum + "\""
                 + ",\n\"" + HOT_PARTITION_NUM + "\" = \"" + hotPartitionNum + "\""
-                + ",\n\"" + RESERVED_HISTORY_PERIODS + "\" = \"" + reservedHistoryPeriods + "\"";
+                + ",\n\"" + RESERVED_HISTORY_PERIODS + "\" = \"" + reservedHistoryPeriods + "\""
+                + ",\n\"" + REMOTE_STORAGE_POLICY + "\" = \"" + remoteStoragePolicy + "\"";
         if (getTimeUnit().equalsIgnoreCase(TimeUnit.WEEK.toString())) {
             res += ",\n\"" + START_DAY_OF_WEEK + "\" = \"" + startOfWeek.dayOfWeek + "\"";
         } else if (getTimeUnit().equalsIgnoreCase(TimeUnit.MONTH.toString())) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletInvertedIndex.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletInvertedIndex.java
index 21c55fb4a8..b3ca22a5a5 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletInvertedIndex.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TabletInvertedIndex.java
@@ -185,7 +185,9 @@ public class TabletInvertedIndex {
                         if (!Config.disable_storage_medium_check) {
                             // check if need migration
                             TStorageMedium storageMedium = storageMediumMap.get(partitionId);
-                            if (storageMedium != null && backendTabletInfo.isSetStorageMedium()) {
+                            if (storageMedium != null && backendTabletInfo.isSetStorageMedium()
+                                    && isLocal(storageMedium) && isLocal(backendTabletInfo.getStorageMedium())
+                                    && isLocal(tabletMeta.getStorageMedium())) {
                                 if (storageMedium != backendTabletInfo.getStorageMedium()) {
                                     synchronized (tabletMigrationMap) {
                                         tabletMigrationMap.put(storageMedium, tabletId);
@@ -671,4 +673,9 @@ public class TabletInvertedIndex {
     public Map<Long, TabletMeta> getTabletMetaMap() {
         return tabletMetaMap;
     }
+
+    private boolean isLocal(TStorageMedium storageMedium) {
+        return storageMedium == TStorageMedium.HDD || storageMedium == TStorageMedium.SSD;
+    }
+
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java
index fef969f646..3a6c2fd508 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java
@@ -53,6 +53,7 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Range;
 import com.google.common.collect.Sets;
+import org.apache.commons.lang3.StringUtils;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -161,6 +162,7 @@ public class DynamicPartitionScheduler extends MasterDaemon {
             idx = 0;
         }
         int hotPartitionNum = dynamicPartitionProperty.getHotPartitionNum();
+        String storagePolicyName = dynamicPartitionProperty.getRemoteStoragePolicy();
 
         for (; idx <= dynamicPartitionProperty.getEnd(); idx++) {
             String prevBorder = DynamicPartitionUtil.getPartitionRangeString(
@@ -223,6 +225,10 @@ public class DynamicPartitionScheduler extends MasterDaemon {
                 setStorageMediumProperty(partitionProperties, dynamicPartitionProperty, now, hotPartitionNum, idx);
             }
 
+            if (StringUtils.isNotEmpty(storagePolicyName)) {
+                setStoragePolicyProperty(partitionProperties, dynamicPartitionProperty, now, idx, storagePolicyName);
+            }
+
             String partitionName = dynamicPartitionProperty.getPrefix()
                     + DynamicPartitionUtil.getFormattedPartitionName(dynamicPartitionProperty.getTimeZone(),
                     prevBorder, dynamicPartitionProperty.getTimeUnit());
@@ -253,11 +259,20 @@ public class DynamicPartitionScheduler extends MasterDaemon {
             return;
         }
         partitionProperties.put(PropertyAnalyzer.PROPERTIES_STORAGE_MEDIUM, TStorageMedium.SSD.name());
-        String cooldownTime = DynamicPartitionUtil.getPartitionRangeString(property, now, offset + hotPartitionNum,
-                DynamicPartitionUtil.DATETIME_FORMAT);
+        String cooldownTime = DynamicPartitionUtil.getPartitionRangeString(
+                property, now, offset + hotPartitionNum, DynamicPartitionUtil.DATETIME_FORMAT);
         partitionProperties.put(PropertyAnalyzer.PROPERTIES_STORAGE_COOLDOWN_TIME, cooldownTime);
     }
 
+    private void setStoragePolicyProperty(HashMap<String, String> partitionProperties,
+                                          DynamicPartitionProperty property, ZonedDateTime now, int offset,
+                                          String storagePolicyName) {
+        partitionProperties.put(PropertyAnalyzer.PROPERTIES_REMOTE_STORAGE_POLICY, storagePolicyName);
+        String baseTime = DynamicPartitionUtil.getPartitionRangeString(
+                property, now, offset, DynamicPartitionUtil.DATETIME_FORMAT);
+        partitionProperties.put(PropertyAnalyzer.PROPERTIES_DATA_BASE_TIME, baseTime);
+    }
+
     private Range<PartitionKey> getClosedRange(Database db, OlapTable olapTable, Column partitionColumn,
             String partitionFormat, String lowerBorderOfReservedHistory, String upperBorderOfReservedHistory) {
         Range<PartitionKey> reservedHistoryPartitionKeyRange = null;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/DynamicPartitionUtil.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/DynamicPartitionUtil.java
index 30cd75cd52..b35be18ab0 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/util/DynamicPartitionUtil.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/DynamicPartitionUtil.java
@@ -37,6 +37,8 @@ import org.apache.doris.common.ErrorReport;
 import org.apache.doris.common.FeConstants;
 import org.apache.doris.common.FeNameFormat;
 import org.apache.doris.common.UserException;
+import org.apache.doris.policy.PolicyTypeEnum;
+import org.apache.doris.policy.StoragePolicy;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
@@ -339,6 +341,26 @@ public class DynamicPartitionUtil {
         }
     }
 
+    private static void checkRemoteStoragePolicy(String val) throws DdlException {
+        if (Strings.isNullOrEmpty(val)) {
+            LOG.info(DynamicPartitionProperty.REMOTE_STORAGE_POLICY + " is null, remove this key");
+            return;
+        }
+        if (val.isEmpty()) {
+            throw new DdlException(DynamicPartitionProperty.REMOTE_STORAGE_POLICY + " is empty.");
+        }
+        StoragePolicy checkedPolicyCondition = new StoragePolicy(PolicyTypeEnum.STORAGE, val);
+        if (!Catalog.getCurrentCatalog().getPolicyMgr().existPolicy(checkedPolicyCondition)) {
+            throw new DdlException(DynamicPartitionProperty.REMOTE_STORAGE_POLICY + ": " + val + " doesn't exist.");
+        }
+        StoragePolicy storagePolicy = (StoragePolicy) Catalog.getCurrentCatalog()
+                .getPolicyMgr().getPolicy(checkedPolicyCondition);
+        if (Strings.isNullOrEmpty(storagePolicy.getCooldownTtl())) {
+            throw new DdlException("Storage policy cooldown type need to be cooldownTtl for properties "
+                    + DynamicPartitionProperty.REMOTE_STORAGE_POLICY + ": " + val);
+        }
+    }
+
     private static SimpleDateFormat getSimpleDateFormat(String timeUnit) {
         if (timeUnit.equalsIgnoreCase(TimeUnit.HOUR.toString())) {
             return new SimpleDateFormat(DATETIME_FORMAT);
@@ -588,6 +610,14 @@ public class DynamicPartitionUtil {
             properties.remove(DynamicPartitionProperty.RESERVED_HISTORY_PERIODS);
             analyzedProperties.put(DynamicPartitionProperty.RESERVED_HISTORY_PERIODS, reservedHistoryPeriods);
         }
+        if (properties.containsKey(DynamicPartitionProperty.REMOTE_STORAGE_POLICY)) {
+            String remoteStoragePolicy = properties.get(DynamicPartitionProperty.REMOTE_STORAGE_POLICY);
+            checkRemoteStoragePolicy(remoteStoragePolicy);
+            properties.remove(DynamicPartitionProperty.REMOTE_STORAGE_POLICY);
+            if (!Strings.isNullOrEmpty(remoteStoragePolicy)) {
+                analyzedProperties.put(DynamicPartitionProperty.REMOTE_STORAGE_POLICY, remoteStoragePolicy);
+            }
+        }
         return analyzedProperties;
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java
index 144576af6d..563706fcca 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java
@@ -60,6 +60,8 @@ public class PropertyAnalyzer {
     public static final String PROPERTIES_STORAGE_TYPE = "storage_type";
     public static final String PROPERTIES_STORAGE_MEDIUM = "storage_medium";
     public static final String PROPERTIES_STORAGE_COOLDOWN_TIME = "storage_cooldown_time";
+    // base time for the data in the partition
+    public static final String PROPERTIES_DATA_BASE_TIME = "data_base_time_ms";
     // for 1.x -> 2.x migration
     public static final String PROPERTIES_VERSION_INFO = "version_info";
     // for restore
@@ -129,7 +131,9 @@ public class PropertyAnalyzer {
         TStorageMedium storageMedium = oldDataProperty.getStorageMedium();
         long cooldownTimeStamp = oldDataProperty.getCooldownTimeMs();
         String remoteStoragePolicy = oldDataProperty.getRemoteStoragePolicy();
+        long remoteCooldownTimeMs = oldDataProperty.getRemoteCooldownTimeMs();
 
+        long dataBaseTimeMs = 0;
         for (Map.Entry<String, String> entry : properties.entrySet()) {
             String key = entry.getKey();
             String value = entry.getValue();
@@ -146,12 +150,18 @@ public class PropertyAnalyzer {
                 cooldownTimeStamp = dateLiteral.unixTimestamp(TimeUtils.getTimeZone());
             } else if (key.equalsIgnoreCase(PROPERTIES_REMOTE_STORAGE_POLICY)) {
                 remoteStoragePolicy = value;
+            } else if (key.equalsIgnoreCase(PROPERTIES_DATA_BASE_TIME)) {
+                DateLiteral dateLiteral = new DateLiteral(value, Type.DATETIME);
+                dataBaseTimeMs = dateLiteral.unixTimestamp(TimeUtils.getTimeZone());
             }
         } // end for properties
 
         properties.remove(PROPERTIES_STORAGE_MEDIUM);
         properties.remove(PROPERTIES_STORAGE_COOLDOWN_TIME);
         properties.remove(PROPERTIES_REMOTE_STORAGE_POLICY);
+        properties.remove(PROPERTIES_DATA_BASE_TIME);
+
+        Preconditions.checkNotNull(storageMedium);
 
         if (storageMedium == TStorageMedium.HDD) {
             cooldownTimeStamp = DataProperty.MAX_COOLDOWN_TIME_MS;
@@ -190,11 +200,16 @@ public class PropertyAnalyzer {
                     throw new AnalysisException("`remote_storage_cooldown_time`"
                             + " should later than `storage_cooldown_time`.");
                 }
+                remoteCooldownTimeMs = storagePolicy.getCooldownDatetime().getTime();
+            } else if (storagePolicy.getCooldownTtl() != null && dataBaseTimeMs > 0) {
+                remoteCooldownTimeMs = dataBaseTimeMs + storagePolicy.getCooldownTtlMs();
             }
         }
 
-        Preconditions.checkNotNull(storageMedium);
-        return new DataProperty(storageMedium, cooldownTimeStamp, remoteStoragePolicy);
+        if (dataBaseTimeMs <= 0) {
+            remoteCooldownTimeMs = DataProperty.MAX_COOLDOWN_TIME_MS;
+        }
+        return new DataProperty(storageMedium, cooldownTimeStamp, remoteStoragePolicy, remoteCooldownTimeMs);
     }
 
     public static short analyzeShortKeyColumnCount(Map<String, String> properties) throws AnalysisException {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/policy/StoragePolicy.java b/fe/fe-core/src/main/java/org/apache/doris/policy/StoragePolicy.java
index e6a1ef7c1a..e5883ee227 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/policy/StoragePolicy.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/policy/StoragePolicy.java
@@ -60,6 +60,16 @@ public class StoragePolicy extends Policy {
     private static final String COOLDOWN_DATETIME = "cooldown_datetime";
     private static final String COOLDOWN_TTL = "cooldown_ttl";
 
+    // for ttl format
+    private static final String TTL_WEEK = "week";
+    private static final String TTL_DAY = "day";
+    private static final String TTL_DAY_SIMPLE = "d";
+    private static final String TTL_HOUR = "hour";
+    private static final String TTL_HOUR_SIMPLE = "h";
+    private static final long ONE_HOUR_MS = 3600 * 1000;
+    private static final long ONE_DAY_MS = 24 * ONE_HOUR_MS;
+    private static final long ONE_WEEK_MS = 7 * ONE_DAY_MS;
+
     @SerializedName(value = "storageResource")
     private String storageResource = null;
 
@@ -69,6 +79,9 @@ public class StoragePolicy extends Policy {
     @SerializedName(value = "cooldownTtl")
     private String cooldownTtl = null;
 
+    @SerializedName(value = "cooldownTtlMs")
+    private long cooldownTtlMs = 0;
+
     private Map<String, String> props;
 
     public StoragePolicy() {}
@@ -81,13 +94,15 @@ public class StoragePolicy extends Policy {
      * @param storageResource resource name for storage
      * @param cooldownDatetime cool down time
      * @param cooldownTtl cool down time cost after partition is created
+     * @param cooldownTtlMs seconds for cooldownTtl
      */
     public StoragePolicy(final PolicyTypeEnum type, final String policyName, final String storageResource,
-                         final Date cooldownDatetime, final String cooldownTtl) {
+                         final Date cooldownDatetime, final String cooldownTtl, long cooldownTtlMs) {
         super(type, policyName);
         this.storageResource = storageResource;
         this.cooldownDatetime = cooldownDatetime;
         this.cooldownTtl = cooldownTtl;
+        this.cooldownTtlMs = cooldownTtlMs;
     }
 
     /**
@@ -126,6 +141,7 @@ public class StoragePolicy extends Policy {
         if (props.containsKey(COOLDOWN_TTL)) {
             hasCooldownTtl = true;
             this.cooldownTtl = props.get(COOLDOWN_TTL);
+            this.cooldownTtlMs = getMsByCooldownTtl(this.cooldownTtl);
         }
         if (hasCooldownDatetime && hasCooldownTtl) {
             throw new AnalysisException(COOLDOWN_DATETIME + " and " + COOLDOWN_TTL + " can't be set together.");
@@ -152,7 +168,7 @@ public class StoragePolicy extends Policy {
             cooldownDatetimeStr = df.format(this.cooldownDatetime);
         }
         return Lists.newArrayList(this.policyName, this.type.name(), this.storageResource,
-            cooldownDatetimeStr, this.cooldownTtl, props);
+                                  cooldownDatetimeStr, this.cooldownTtl, props);
     }
 
     @Override
@@ -161,7 +177,7 @@ public class StoragePolicy extends Policy {
     @Override
     public StoragePolicy clone() {
         return new StoragePolicy(this.type, this.policyName, this.storageResource,
-                                 this.cooldownDatetime, this.cooldownTtl);
+                                 this.cooldownDatetime, this.cooldownTtl, this.cooldownTtlMs);
     }
 
     @Override
@@ -199,4 +215,33 @@ public class StoragePolicy extends Policy {
     public boolean isInvalid() {
         return false;
     }
+
+    /**
+     * Get milliseconds by cooldownTtl, 1week=604800000 1day=1d=86400000, 1hour=1h=3600000
+     * @param cooldownTtl cooldown ttl
+     * @return millisecond for cooldownTtl
+     */
+    private static long getMsByCooldownTtl(String cooldownTtl) throws AnalysisException {
+        cooldownTtl = cooldownTtl.replace(TTL_DAY, TTL_DAY_SIMPLE).replace(TTL_HOUR, TTL_HOUR_SIMPLE);
+        long cooldownTtlMs = 0;
+        try {
+            if (cooldownTtl.endsWith(TTL_DAY_SIMPLE)) {
+                cooldownTtlMs = Long.parseLong(cooldownTtl.replace(TTL_DAY_SIMPLE, "").trim()) * ONE_DAY_MS;
+            } else if (cooldownTtl.endsWith(TTL_HOUR_SIMPLE)) {
+                cooldownTtlMs = Long.parseLong(cooldownTtl.replace(TTL_HOUR_SIMPLE, "").trim()) * ONE_HOUR_MS;
+            } else if (cooldownTtl.endsWith(TTL_WEEK)) {
+                cooldownTtlMs = Long.parseLong(cooldownTtl.replace(TTL_WEEK, "").trim()) * ONE_WEEK_MS;
+            } else {
+                cooldownTtlMs = Long.parseLong(cooldownTtl.trim()) * 1000;
+            }
+        } catch (NumberFormatException e) {
+            LOG.error("getSecByCooldownTtl failed.", e);
+            throw new AnalysisException("getSecByCooldownTtl failed.", e);
+        }
+        if (cooldownTtlMs < 0) {
+            LOG.error("cooldownTtl can't be less than 0");
+            throw new AnalysisException("cooldownTtl can't be less than 0");
+        }
+        return cooldownTtlMs;
+    }
 }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java b/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java
index e6cdb5fee4..b1684cb75a 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java
@@ -464,13 +464,13 @@ public class AlterTest {
         stmt = "alter table test.tbl4 modify partition (p3, p4) set ('storage_medium' = 'HDD')";
         DateLiteral dateLiteral = new DateLiteral("2999-12-31 00:00:00", Type.DATETIME);
         long cooldownTimeMs = dateLiteral.unixTimestamp(TimeUtils.getTimeZone());
-        DataProperty oldDataProperty = new DataProperty(TStorageMedium.SSD, cooldownTimeMs, "");
+        DataProperty oldDataProperty = new DataProperty(TStorageMedium.SSD, cooldownTimeMs, "", -1);
         partitionList = Lists.newArrayList(p3, p4);
         for (Partition partition : partitionList) {
             Assert.assertEquals(oldDataProperty, tbl4.getPartitionInfo().getDataProperty(partition.getId()));
         }
         alterTable(stmt, false);
-        DataProperty newDataProperty = new DataProperty(TStorageMedium.HDD, DataProperty.MAX_COOLDOWN_TIME_MS, "");
+        DataProperty newDataProperty = new DataProperty(TStorageMedium.HDD, DataProperty.MAX_COOLDOWN_TIME_MS, "", -1);
         for (Partition partition : partitionList) {
             Assert.assertEquals(newDataProperty, tbl4.getPartitionInfo().getDataProperty(partition.getId()));
         }
@@ -483,7 +483,7 @@ public class AlterTest {
 
         dateLiteral = new DateLiteral("2100-12-31 00:00:00", Type.DATETIME);
         cooldownTimeMs = dateLiteral.unixTimestamp(TimeUtils.getTimeZone());
-        DataProperty newDataProperty1 = new DataProperty(TStorageMedium.SSD, cooldownTimeMs, "");
+        DataProperty newDataProperty1 = new DataProperty(TStorageMedium.SSD, cooldownTimeMs, "", -1);
         partitionList = Lists.newArrayList(p1, p2);
         for (Partition partition : partitionList) {
             Assert.assertEquals(newDataProperty1, tbl4.getPartitionInfo().getDataProperty(partition.getId()));
@@ -511,7 +511,7 @@ public class AlterTest {
 
         DateLiteral dateLiteral = new DateLiteral("2100-05-09 00:00:00", Type.DATETIME);
         long cooldownTimeMs = dateLiteral.unixTimestamp(TimeUtils.getTimeZone());
-        DataProperty oldDataProperty = new DataProperty(TStorageMedium.SSD, cooldownTimeMs, "testPolicy");
+        DataProperty oldDataProperty = new DataProperty(TStorageMedium.SSD, cooldownTimeMs, "testPolicy", -1);
         List<Partition> partitionList = Lists.newArrayList(p2, p3, p4);
         for (Partition partition : partitionList) {
             Assert.assertEquals(oldDataProperty, tblRemote.getPartitionInfo().getDataProperty(partition.getId()));
@@ -522,7 +522,7 @@ public class AlterTest {
         alterTable(stmt, false);
         DateLiteral newDateLiteral = new DateLiteral("2100-04-01 22:22:22", Type.DATETIME);
         long newCooldownTimeMs = newDateLiteral.unixTimestamp(TimeUtils.getTimeZone());
-        DataProperty dataProperty2 = new DataProperty(TStorageMedium.SSD, newCooldownTimeMs, "testPolicy");
+        DataProperty dataProperty2 = new DataProperty(TStorageMedium.SSD, newCooldownTimeMs, "testPolicy", -1);
         for (Partition partition : partitionList) {
             Assert.assertEquals(dataProperty2, tblRemote.getPartitionInfo().getDataProperty(partition.getId()));
         }
@@ -532,7 +532,7 @@ public class AlterTest {
         stmt = "alter table test.tbl_remote modify partition (p2, p3, p4) set ('storage_medium' = 'HDD')";
         alterTable(stmt, false);
         DataProperty dataProperty1 = new DataProperty(
-                TStorageMedium.HDD, DataProperty.MAX_COOLDOWN_TIME_MS, "testPolicy");
+                TStorageMedium.HDD, DataProperty.MAX_COOLDOWN_TIME_MS, "testPolicy", -1);
         for (Partition partition : partitionList) {
             Assert.assertEquals(dataProperty1, tblRemote.getPartitionInfo().getDataProperty(partition.getId()));
         }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/DataPropertyTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/DataPropertyTest.java
index 5114142569..1ea2bd3d56 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/DataPropertyTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/DataPropertyTest.java
@@ -35,7 +35,7 @@ public class DataPropertyTest {
         Assert.assertNotEquals(DataProperty.MAX_COOLDOWN_TIME_MS, dataProperty.getCooldownTimeMs());
 
         long storageCooldownTimeMs = System.currentTimeMillis() + 24 * 3600 * 1000L;
-        dataProperty = new DataProperty(TStorageMedium.SSD, storageCooldownTimeMs, "");
+        dataProperty = new DataProperty(TStorageMedium.SSD, storageCooldownTimeMs, "", -1);
         Assert.assertEquals(storageCooldownTimeMs, dataProperty.getCooldownTimeMs());
 
         dataProperty = new DataProperty(TStorageMedium.HDD);


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org