diff --git a/docs/zh/data_source/catalog/external_cooldown.md b/docs/zh/data_source/catalog/external_cooldown.md new file mode 100644 index 0000000000000..d532429a2a023 --- /dev/null +++ b/docs/zh/data_source/catalog/external_cooldown.md @@ -0,0 +1,135 @@ +--- +displayed_sidebar: docs +--- + +# 分区降冷 + +分区降冷是将本地OLAP表分区导出到外部存储(比如iceberg表)的过程,降冷之后的数据可以通过连外部catalog查询(后续PR支持使用一张表查询降冷后的数据)。 +本文介绍如何配置降冷相关属性及触发分区下沉。 + +## 分区降冷表属性 + +- `external_cooldown_target` 降冷目标表,格式: `{catalog}.{db}.{tbl}` +- `external_cooldown_schedule` 降冷调度配置,比如配置 `START 23:00 END 08:00 EVERY INTERVAL 1 MINUTE` 表示晚上23:00到第二天早上8:00(不包含08:00)这个时间范围可以执行自动降冷任务, 默认空,不会自动生成降冷任务 +- `external_cooldown_wait_second` 降冷等待时间,分区最后一次修改(以`VisibleVersionTime`为准)之后过`external_cooldown_wait_second`时间之后才可以降冷,默认值0, external_cooldown_wait_second不大于0时不会自动降冷 + +```SQL +ALTER TABLE olap_tbl SET ( + 'external_cooldown_target'='iceberg.db1.tbl1', + 'external_cooldown_schedule'='START 01:00 END 08:00 EVERY INTERVAL 1 MINUTE', + 'external_cooldown_wait_second'='3600' +); +``` + + 或 + +```SQL +alter table olap_tbl set('external_cooldown_target'='iceberg.db1.tbl1'); +alter table olap_tbl set('external_cooldown_schedule'='START 01:00 END 08:00 EVERY INTERVAL 1 MINUTE'); +alter table olap_tbl set('external_cooldown_wait_second'='3600'); +``` + +2. 可以通过查看建表语句查看分区降冷相关属性 + +```SQL +mysql> show create table olap_tbl \G +*************************** 1. row *************************** + Table: olap_tbl +Create Table: CREATE TABLE `olap_tbl` ( + `event` varchar(65533) NULL COMMENT "", + `dteventtime` datetime NULL COMMENT "", + `f_date` date NULL COMMENT "", + `f_int` int(11) NULL COMMENT "", + `f_bigint` bigint(20) NULL COMMENT "", + `f_double` double NULL COMMENT "", + `f_boolean` boolean NULL COMMENT "", + `f_decimal` decimal(12, 4) NULL COMMENT "" +) ENGINE=OLAP +DUPLICATE KEY(`event`, `dteventtime`) +COMMENT "OLAP" +PARTITION BY RANGE(`dteventtime`) +(PARTITION p20220417 VALUES [("2022-04-17 00:00:00"), ("2022-04-18 00:00:00")), +PARTITION p20220418 VALUES [("2022-04-18 00:00:00"), ("2022-04-19 00:00:00")), +PARTITION p20220419 VALUES [("2022-04-19 00:00:00"), ("2022-04-20 00:00:00")), +PARTITION p20220420 VALUES [("2022-04-20 00:00:00"), ("2022-04-21 00:00:00")), +PARTITION p20220421 VALUES [("2022-04-21 00:00:00"), ("2022-04-22 00:00:00"))) +DISTRIBUTED BY HASH(`event`) BUCKETS 1 +PROPERTIES ( +"compression" = "LZ4", +"dynamic_partition.buckets" = "1", +"dynamic_partition.enable" = "false", +"dynamic_partition.end" = "2", +"dynamic_partition.history_partition_num" = "0", +"dynamic_partition.prefix" = "p", +"dynamic_partition.start" = "-2147483648", +"dynamic_partition.time_unit" = "DAY", +"dynamic_partition.time_zone" = "Asia/Shanghai", +"external_cooldown_schedule" = "START 14:00 END 19:59 EVERY INTERVAL 180 SECOND", +"external_cooldown_target" = "iceberg.db1.tbl1", +"external_cooldown_wait_second" = "0", +"fast_schema_evolution" = "true", +"replicated_storage" = "true", +"replication_num" = "1" +); +1 row in set (0.01 sec) +``` + +## 手动触发下沉 + +- 降冷整个表 + +```SQL +cooldown table olap_tbl; +``` + +- 降冷分区时间范围在2022-04-18 00:00:00到2022-04-19 00:00:00之间的分区, 如果已降冷不会重复降冷,空分区不会降冷 + +```SQL +cooldown table olap_tbl partition start ('2022-04-18 00:00:00') end ('2022-04-19 00:00:00'); +``` + +- 强制降冷分区时间范围在2022-04-18 00:00:00到2022-04-19 00:00:00之间的分区, 如果已下沉不会重复降冷,空分区不会降冷 + +```SQL +cooldown table olap_tbl partition start ('2022-04-18 00:00:00') end ('2022-04-19 00:00:00') force; +``` + +## 查看降冷后分区状态 + +```SQL +mysql> show partitions from olap_tbl where partitionname='p20220419' \G +*************************** 1. row *************************** + PartitionId: 10743 + PartitionName: p20220419 + VisibleVersion: 2 + VisibleVersionTime: 2024-02-28 16:52:54 + VisibleVersionHash: 0 + State: NORMAL + PartitionKey: dteventtime + Range: [types: [DATETIME]; keys: [2022-04-19 00:00:00]; ..types: [DATETIME]; keys: [2022-04-20 00:00:00]; ) + DistributionKey: event + Buckets: 1 + ReplicationNum: 1 + StorageMedium: HDD + CooldownTime: 9999-12-31 23:59:59 + LastConsistencyCheckTime: NULL + DataSize: 2.1KB + IsInMemory: false + RowCount: 0 + DataVersion: 2 + VersionEpoch: 313464788426424322 + VersionTxnType: TXN_NORMAL + ExternalCoolDownSyncedTime: 2024-02-28 16:52:54 + ExternalCoolDownConsistencyCheckTime: NULL +ExternalCoolDownConsistencyCheckDifference: NULL +1 row in set (0.00 sec) +``` + +## 查看降冷task + +通过查询tasks和task_runs表可查看降冷任务执行情况 + +```SQL +select * from information_schema.tasks; +select * from information_schema.task_runs; +``` diff --git a/fe/fe-core/src/main/java/com/starrocks/alter/AlterJobExecutor.java b/fe/fe-core/src/main/java/com/starrocks/alter/AlterJobExecutor.java index 3ec505b2b8a4b..182e6c9861c0f 100644 --- a/fe/fe-core/src/main/java/com/starrocks/alter/AlterJobExecutor.java +++ b/fe/fe-core/src/main/java/com/starrocks/alter/AlterJobExecutor.java @@ -34,6 +34,7 @@ import com.starrocks.catalog.PartitionType; import com.starrocks.catalog.RangePartitionInfo; import com.starrocks.catalog.Table; +import com.starrocks.catalog.TableProperty; import com.starrocks.catalog.Type; import com.starrocks.common.AnalysisException; import com.starrocks.common.DdlException; @@ -432,6 +433,9 @@ public Void visitModifyTablePropertiesClause(ModifyTablePropertiesClause clause, } GlobalStateMgr.getCurrentState().getLocalMetastore() .modifyTableDynamicPartition(db, olapTable, properties); + } else if (TableProperty.isSamePrefixProperties(properties, + PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_PREFIX)) { + GlobalStateMgr.getCurrentState().getLocalMetastore().alterTableProperties(db, olapTable, properties); } else if (properties.containsKey("default." + PropertyAnalyzer.PROPERTIES_REPLICATION_NUM)) { Preconditions.checkNotNull(properties.get(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM)); GlobalStateMgr.getCurrentState().getLocalMetastore() @@ -722,6 +726,12 @@ private void modifyPartitionsProperty(Database db, TTabletType tTabletType = PropertyAnalyzer.analyzeTabletType(properties); + // 5. external cooldown synced time + long coolDownSyncedTimeMs = PropertyAnalyzer.analyzeExternalCooldownSyncedTimeMs(properties); + + // 6. external cooldown consistency check time + long coolDownCheckTimeMs = PropertyAnalyzer.analyzeExternalCooldownConsistencyCheckTimeMs(properties); + // modify meta here for (String partitionName : partitionNames) { Partition partition = olapTable.getPartition(partitionName); @@ -772,8 +782,19 @@ private void modifyPartitionsProperty(Database db, if (tTabletType != partitionInfo.getTabletType(partition.getId())) { partitionInfo.setTabletType(partition.getId(), tTabletType); } + Long preCoolDownSyncedTimeMs = partitionInfo.getExternalCoolDownSyncedTimeMs(partition.getId()); + if (coolDownSyncedTimeMs != -1L && + (preCoolDownSyncedTimeMs == null || coolDownSyncedTimeMs != preCoolDownSyncedTimeMs)) { + partitionInfo.setExternalCoolDownSyncedTimeMs(partition.getId(), coolDownSyncedTimeMs); + } + Long preCoolDownCheckTimeMs = partitionInfo.getExternalCoolDownConsistencyCheckTimeMs(partition.getId()); + if (coolDownCheckTimeMs != -1L && + (preCoolDownCheckTimeMs == null || coolDownCheckTimeMs != preCoolDownCheckTimeMs)) { + partitionInfo.setExternalCoolDownConsistencyCheckTimeMs(partition.getId(), coolDownCheckTimeMs); + } ModifyPartitionInfo info = new ModifyPartitionInfo(db.getId(), olapTable.getId(), partition.getId(), - newDataProperty, newReplicationNum, hasInMemory ? newInMemory : oldInMemory); + newDataProperty, newReplicationNum, hasInMemory ? newInMemory : oldInMemory, + coolDownSyncedTimeMs, coolDownCheckTimeMs); modifyPartitionInfos.add(info); } diff --git a/fe/fe-core/src/main/java/com/starrocks/alter/AlterJobMgr.java b/fe/fe-core/src/main/java/com/starrocks/alter/AlterJobMgr.java index c98a26fb224b9..e7a58bc71e4fa 100644 --- a/fe/fe-core/src/main/java/com/starrocks/alter/AlterJobMgr.java +++ b/fe/fe-core/src/main/java/com/starrocks/alter/AlterJobMgr.java @@ -563,6 +563,14 @@ public void replayModifyPartition(ModifyPartitionInfo info) { } } partitionInfo.setIsInMemory(info.getPartitionId(), info.isInMemory()); + if (info.getExternalCoolDownSyncedTimeMs() != -1L) { + partitionInfo.setExternalCoolDownSyncedTimeMs( + info.getPartitionId(), info.getExternalCoolDownSyncedTimeMs()); + } + if (info.getExternalCoolDownConsistencyCheckTimeMs() != -1L) { + partitionInfo.setExternalCoolDownConsistencyCheckTimeMs( + info.getPartitionId(), info.getExternalCoolDownConsistencyCheckTimeMs()); + } } finally { locker.unLockTablesWithIntensiveDbLock(db.getId(), Lists.newArrayList(olapTable.getId()), LockType.WRITE); } diff --git a/fe/fe-core/src/main/java/com/starrocks/catalog/OlapTable.java b/fe/fe-core/src/main/java/com/starrocks/catalog/OlapTable.java index f1b0c2daaf6d5..3a3a0e7517a51 100644 --- a/fe/fe-core/src/main/java/com/starrocks/catalog/OlapTable.java +++ b/fe/fe-core/src/main/java/com/starrocks/catalog/OlapTable.java @@ -36,6 +36,7 @@ import com.google.common.base.Joiner; import com.google.common.base.Preconditions; +import com.google.common.base.Splitter; import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; @@ -87,6 +88,7 @@ import com.starrocks.common.util.Util; import com.starrocks.common.util.WriteQuorum; import com.starrocks.common.util.concurrent.MarkedCountDownLatch; +import com.starrocks.externalcooldown.ExternalCooldownConfig; import com.starrocks.lake.DataCacheInfo; import com.starrocks.lake.StarOSAgent; import com.starrocks.lake.StorageInfo; @@ -485,6 +487,31 @@ public void setBinlogTxnId(long binlogTxnId) { this.binlogTxnId = binlogTxnId; } + public ExternalCooldownConfig getCurExternalCoolDownConfig() { + if (tableProperty != null) { + return tableProperty.getExternalCoolDownConfig(); + } + return null; + } + + public void setCurExternalCoolDownConfig(ExternalCooldownConfig externalCoolDownConfig) { + if (tableProperty == null) { + tableProperty = new TableProperty(Maps.newHashMap()); + } + tableProperty.modifyTableProperties(externalCoolDownConfig.getValidProperties()); + tableProperty.setExternalCoolDownConfig(externalCoolDownConfig); + } + + public boolean containsExternalCoolDownConfig() { + if (tableProperty == null || + tableProperty.getExternalCoolDownConfig() == null) { + return false; + } + + return tableProperty.getExternalCoolDownConfig().getTarget() != null && + !tableProperty.getExternalCoolDownConfig().getTarget().isEmpty(); + } + public void setTableProperty(TableProperty tableProperty) { this.tableProperty = tableProperty; } @@ -3339,6 +3366,10 @@ public Map getProperties() { // unique properties properties.putAll(getUniqueProperties()); + if (getCurExternalCoolDownConfig() != null) { + properties.putAll(getCurExternalCoolDownConfig().getValidProperties()); + } + return properties; } @@ -3421,6 +3452,45 @@ public boolean hasRowStorageType() { return TStorageType.ROW == getStorageType() || TStorageType.COLUMN_WITH_ROW == getStorageType(); } + public String getExternalCoolDownTarget() { + if (tableProperty != null) { + return tableProperty.getExternalCoolDownTarget(); + } + return null; + } + + public TableName getExternalCoolDownTargetTableName() { + String tableName = getExternalCoolDownTarget(); + if (Strings.isNullOrEmpty(tableName)) { + return null; + } + List pieces = Splitter.on(".").splitToList(tableName); + return new TableName(pieces.get(0), pieces.get(1), pieces.get(2)); + } + + public Table getExternalCoolDownTable() { + TableName tableName = getExternalCoolDownTargetTableName(); + if (tableName == null) { + return null; + } + Optional table = GlobalStateMgr.getCurrentState().getMetadataMgr().getTable(tableName); + return table.orElse(null); + } + + public String getExternalCoolDownSchedule() { + if (tableProperty != null) { + return tableProperty.getExternalCoolDownSchedule(); + } + return null; + } + + public Long getExternalCoolDownWaitSecond() { + if (tableProperty != null) { + return tableProperty.getExternalCoolDownWaitSecond(); + } + return null; + } + // ------ for lake table and lake materialized view start ------ @Nullable public FilePathInfo getDefaultFilePathInfo() { diff --git a/fe/fe-core/src/main/java/com/starrocks/catalog/PartitionInfo.java b/fe/fe-core/src/main/java/com/starrocks/catalog/PartitionInfo.java index d0322134711f5..715e9d81316b3 100644 --- a/fe/fe-core/src/main/java/com/starrocks/catalog/PartitionInfo.java +++ b/fe/fe-core/src/main/java/com/starrocks/catalog/PartitionInfo.java @@ -37,6 +37,7 @@ import com.google.gson.annotations.SerializedName; import com.starrocks.common.DdlException; import com.starrocks.common.io.JsonWriter; +import com.starrocks.common.util.TimeUtils; import com.starrocks.lake.DataCacheInfo; import com.starrocks.persist.gson.GsonPostProcessable; import com.starrocks.persist.gson.GsonPreProcessable; @@ -84,6 +85,16 @@ public class PartitionInfo extends JsonWriter implements Cloneable, GsonPreProce // storage cache, ttl and enable_async_write_back @SerializedName(value = "idToStorageCacheInfo") protected Map idToStorageCacheInfo; + // partition id -> last external cool down time + @SerializedName(value = "idToExternalCoolDownSyncedTimeMs") + protected Map idToExternalCoolDownSyncedTimeMs; + + // partition id -> last external cool down data consistency check time + @SerializedName(value = "idToExternalCoolDownConsistencyCheckTimeMs") + protected Map idToExternalCoolDownConsistencyCheckTimeMs; + // partition id -> last consistency cool down data consistency check difference (olap - iceberg) + @SerializedName(value = "idToExternalCoolDownConsistencyCheckDifference") + protected Map idToExternalCoolDownConsistencyCheckDifference; public PartitionInfo() { @@ -92,6 +103,9 @@ public PartitionInfo() { this.idToInMemory = new HashMap<>(); this.idToTabletType = new HashMap<>(); this.idToStorageCacheInfo = new HashMap<>(); + this.idToExternalCoolDownSyncedTimeMs = new HashMap<>(); + this.idToExternalCoolDownConsistencyCheckTimeMs = new HashMap<>(); + this.idToExternalCoolDownConsistencyCheckDifference = new HashMap<>(); } public PartitionInfo(PartitionType type) { @@ -101,6 +115,9 @@ public PartitionInfo(PartitionType type) { this.idToInMemory = new HashMap<>(); this.idToTabletType = new HashMap<>(); this.idToStorageCacheInfo = new HashMap<>(); + this.idToExternalCoolDownSyncedTimeMs = new HashMap<>(); + this.idToExternalCoolDownConsistencyCheckTimeMs = new HashMap<>(); + this.idToExternalCoolDownConsistencyCheckDifference = new HashMap<>(); } public PartitionType getType() { @@ -199,11 +216,38 @@ public void setDataCacheInfo(long partitionId, DataCacheInfo dataCacheInfo) { idToStorageCacheInfo.put(partitionId, dataCacheInfo); } + public Long getExternalCoolDownSyncedTimeMs(long partitionId) { + return idToExternalCoolDownSyncedTimeMs.get(partitionId); + } + + public void setExternalCoolDownSyncedTimeMs(long partitionId, long coldDownTimeMs) { + idToExternalCoolDownSyncedTimeMs.put(partitionId, coldDownTimeMs); + } + + public Long getExternalCoolDownConsistencyCheckTimeMs(long partitionId) { + return idToExternalCoolDownConsistencyCheckTimeMs.get(partitionId); + } + + public void setExternalCoolDownConsistencyCheckTimeMs(long partitionId, long checkTimeMs) { + idToExternalCoolDownConsistencyCheckTimeMs.put(partitionId, checkTimeMs); + } + + public Long getExternalCoolDownConsistencyCheckDifference(long partitionId) { + return idToExternalCoolDownConsistencyCheckDifference.get(partitionId); + } + + public void setCoolDownConsistencyCheckDifference(long partitionId, long difference) { + idToExternalCoolDownConsistencyCheckDifference.put(partitionId, difference); + } + public void dropPartition(long partitionId) { idToDataProperty.remove(partitionId); idToReplicationNum.remove(partitionId); idToInMemory.remove(partitionId); idToStorageCacheInfo.remove(partitionId); + idToExternalCoolDownSyncedTimeMs.remove(partitionId); + idToExternalCoolDownConsistencyCheckTimeMs.remove(partitionId); + idToExternalCoolDownConsistencyCheckDifference.remove(partitionId); } public void moveRangeFromTempToFormal(long tempPartitionId) { @@ -215,6 +259,24 @@ public void addPartition(long partitionId, DataProperty dataProperty, idToDataProperty.put(partitionId, dataProperty); idToReplicationNum.put(partitionId, replicationNum); idToInMemory.put(partitionId, isInMemory); + idToExternalCoolDownSyncedTimeMs.put(partitionId, 0L); + idToExternalCoolDownConsistencyCheckTimeMs.put(partitionId, 0L); + idToExternalCoolDownConsistencyCheckDifference.put(partitionId, 0L); + } + + public boolean couldUseExternalCoolDownPartition(Partition partition) { + return partition.getVisibleVersionTime() <= this.getExternalCoolDownSyncedTimeMs(partition.getId()) && + this.getExternalCoolDownSyncedTimeMs(partition.getId()) > 0; + } + + public boolean couldUseExternalCoolDownPartition(Partition partition, long visibleVersionTime) { + // just change from 1(initial version) to 2 + if (partition.getVisibleVersion() == 2) { + return true; + } + + return visibleVersionTime <= this.getExternalCoolDownSyncedTimeMs(partition.getId()) && + this.getExternalCoolDownSyncedTimeMs(partition.getId()) > 0; } public void addPartition(long partitionId, DataProperty dataProperty, @@ -284,6 +346,19 @@ public String toString() { buff.append(" data_property: ").append(entry.getValue().toString()); buff.append(" replica number: ").append(idToReplicationNum.get(entry.getKey())); buff.append(" in memory: ").append(idToInMemory.get(entry.getKey())); + + if (idToExternalCoolDownSyncedTimeMs.get(entry.getKey()) != null) { + buff.append(" external cool down time: ").append( + TimeUtils.longToTimeString(idToExternalCoolDownSyncedTimeMs.get(entry.getKey()))); + } + if (idToExternalCoolDownConsistencyCheckTimeMs.get(entry.getKey()) != null) { + buff.append(" external cool down consistency check time: ").append( + TimeUtils.longToTimeString(idToExternalCoolDownConsistencyCheckTimeMs.get(entry.getKey()))); + } + if (idToExternalCoolDownConsistencyCheckDifference.get(entry.getKey()) != null) { + buff.append(" external cool down consistency check result: ").append( + idToExternalCoolDownConsistencyCheckDifference.get(entry.getKey())); + } } return buff.toString(); @@ -306,6 +381,9 @@ protected Object clone() { p.idToInMemory = new HashMap<>(this.idToInMemory); p.idToTabletType = new HashMap<>(this.idToTabletType); p.idToStorageCacheInfo = new HashMap<>(this.idToStorageCacheInfo); + p.idToExternalCoolDownSyncedTimeMs = this.idToExternalCoolDownSyncedTimeMs; + p.idToExternalCoolDownConsistencyCheckTimeMs = this.idToExternalCoolDownConsistencyCheckTimeMs; + p.idToExternalCoolDownConsistencyCheckDifference = this.idToExternalCoolDownConsistencyCheckDifference; return p; } catch (CloneNotSupportedException e) { throw new RuntimeException(e); diff --git a/fe/fe-core/src/main/java/com/starrocks/catalog/TableProperty.java b/fe/fe-core/src/main/java/com/starrocks/catalog/TableProperty.java index 0243d3ff70cad..bc2bec65d1355 100644 --- a/fe/fe-core/src/main/java/com/starrocks/catalog/TableProperty.java +++ b/fe/fe-core/src/main/java/com/starrocks/catalog/TableProperty.java @@ -53,6 +53,7 @@ import com.starrocks.common.util.PropertyAnalyzer; import com.starrocks.common.util.TimeUtils; import com.starrocks.common.util.WriteQuorum; +import com.starrocks.externalcooldown.ExternalCooldownConfig; import com.starrocks.lake.StorageInfo; import com.starrocks.persist.OperationType; import com.starrocks.persist.gson.GsonPostProcessable; @@ -296,6 +297,7 @@ public static String valueList() { private boolean useFastSchemaEvolution; private PeriodDuration dataCachePartitionDuration; + private ExternalCooldownConfig externalCoolDownConfig; private Multimap location; @@ -387,12 +389,19 @@ public TableProperty buildProperty(short opCode) { case OperationType.OP_MODIFY_TABLE_CONSTRAINT_PROPERTY: buildConstraint(); break; + case OperationType.OP_MODIFY_EXTERNAL_COOLDOWN_CONFIG: + buildExternalCooldownConfig(); + break; default: break; } return this; } + public TableProperty buildExternalCoolDownProperties() { + return this; + } + // TODO: refactor the postProcessing code into listener-based instead of procedure-oriented public TableProperty buildMvProperties() { buildPartitionTTL(); @@ -426,6 +435,22 @@ public void setBinlogConfig(BinlogConfig binlogConfig) { this.binlogConfig = binlogConfig; } + public TableProperty buildExternalCooldownConfig() { + String externalCoolDownTarget = properties.get(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_TARGET); + String externalCoolDownSchedule = properties.get(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_SCHEDULE); + long externalCoolDownWaitSecond = Long.parseLong(properties.getOrDefault( + PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_WAIT_SECOND, + String.valueOf(-1L))); + externalCoolDownConfig = new ExternalCooldownConfig( + externalCoolDownTarget, externalCoolDownSchedule, externalCoolDownWaitSecond); + return this; + } + + // just modify externalCoolDownConfig, not properties + public void setExternalCoolDownConfig(ExternalCooldownConfig externalCoolDownConfig) { + this.externalCoolDownConfig = externalCoolDownConfig; + } + public TableProperty buildDynamicProperty() { HashMap dynamicPartitionProperties = new HashMap<>(); for (Map.Entry entry : properties.entrySet()) { @@ -988,6 +1013,10 @@ public BinlogConfig getBinlogConfig() { return binlogConfig; } + public ExternalCooldownConfig getExternalCoolDownConfig() { + return externalCoolDownConfig; + } + public List getUniqueConstraints() { return uniqueConstraints; } @@ -1036,6 +1065,27 @@ public boolean getUseFastSchemaEvolution() { return useFastSchemaEvolution; } + public String getExternalCoolDownTarget() { + if (externalCoolDownConfig == null) { + return null; + } + return externalCoolDownConfig.getTarget(); + } + + public String getExternalCoolDownSchedule() { + if (externalCoolDownConfig == null) { + return null; + } + return externalCoolDownConfig.getSchedule(); + } + + public long getExternalCoolDownWaitSecond() { + if (externalCoolDownConfig == null) { + return 0; + } + return externalCoolDownConfig.getWaitSecond(); + } + @Override public void write(DataOutput out) throws IOException { Text.writeString(out, GsonUtils.GSON.toJson(this)); @@ -1073,5 +1123,6 @@ public void gsonPostProcess() throws IOException { buildMvProperties(); buildLocation(); buildBaseCompactionForbiddenTimeRanges(); + buildExternalCooldownConfig(); } } diff --git a/fe/fe-core/src/main/java/com/starrocks/common/proc/PartitionsProcDir.java b/fe/fe-core/src/main/java/com/starrocks/common/proc/PartitionsProcDir.java index b62ab7b874271..f6cdba39e8d23 100644 --- a/fe/fe-core/src/main/java/com/starrocks/common/proc/PartitionsProcDir.java +++ b/fe/fe-core/src/main/java/com/starrocks/common/proc/PartitionsProcDir.java @@ -81,6 +81,7 @@ import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.stream.Collectors; /* @@ -148,7 +149,10 @@ private void createTitleNames() { .add("RowCount") .add("DataVersion") .add("VersionEpoch") - .add("VersionTxnType"); + .add("VersionTxnType") + .add("ExternalCoolDownSyncedTime") + .add("ExternalCoolDownConsistencyCheckTime") + .add("ExternalCoolDownConsistencyCheckDifference"); this.titleNames = builder.build(); } } @@ -370,6 +374,23 @@ private List getOlapPartitionInfo(PartitionInfo tblPartitionInfo, partitionInfo.add(physicalPartition.getVersionEpoch()); // VersionEpoch partitionInfo.add(physicalPartition.getVersionTxnType()); // VersionTxnType + Long externalCoolDownSyncedTimeMs = tblPartitionInfo.getExternalCoolDownSyncedTimeMs(partition.getId()); + if (externalCoolDownSyncedTimeMs == null) { + partitionInfo.add("NULL"); + } else { + partitionInfo.add(TimeUtils.longToTimeString(externalCoolDownSyncedTimeMs)); + } + + Long externalCoolDownConsistencyCheckTimeMs = tblPartitionInfo.getExternalCoolDownConsistencyCheckTimeMs( + partition.getId()); + if (externalCoolDownConsistencyCheckTimeMs == null) { + partitionInfo.add("NULL"); + } else { + partitionInfo.add(TimeUtils.longToTimeString(externalCoolDownConsistencyCheckTimeMs)); + } + Long diff = tblPartitionInfo.getExternalCoolDownConsistencyCheckDifference(partition.getId()); + partitionInfo.add(Objects.requireNonNullElse(diff, "NULL")); + return partitionInfo; } diff --git a/fe/fe-core/src/main/java/com/starrocks/common/util/PropertyAnalyzer.java b/fe/fe-core/src/main/java/com/starrocks/common/util/PropertyAnalyzer.java index 98b0d5d08825b..29bb8d678248a 100644 --- a/fe/fe-core/src/main/java/com/starrocks/common/util/PropertyAnalyzer.java +++ b/fe/fe-core/src/main/java/com/starrocks/common/util/PropertyAnalyzer.java @@ -53,11 +53,13 @@ import com.starrocks.catalog.ColumnId; import com.starrocks.catalog.DataProperty; import com.starrocks.catalog.Database; +import com.starrocks.catalog.IcebergTable; import com.starrocks.catalog.InternalCatalog; import com.starrocks.catalog.KeysType; import com.starrocks.catalog.MaterializedView; import com.starrocks.catalog.OlapTable; import com.starrocks.catalog.Partition; +import com.starrocks.catalog.PrimitiveType; import com.starrocks.catalog.Table; import com.starrocks.catalog.TableProperty; import com.starrocks.catalog.Type; @@ -70,6 +72,8 @@ import com.starrocks.common.ErrorReport; import com.starrocks.common.FeConstants; import com.starrocks.common.Pair; +import com.starrocks.externalcooldown.ExternalCooldownConfig; +import com.starrocks.externalcooldown.ExternalCooldownSchedule; import com.starrocks.lake.DataCacheInfo; import com.starrocks.qe.ConnectContext; import com.starrocks.server.GlobalStateMgr; @@ -249,6 +253,21 @@ public class PropertyAnalyzer { */ public static final String MULTI_LOCATION_LABELS_REGEX = "\\s*" + SINGLE_LOCATION_LABEL_REGEX + "\\s*(,\\s*" + SINGLE_LOCATION_LABEL_REGEX + "){0,9}\\s*"; + // external cooldown prefix + public static final String PROPERTIES_EXTERNAL_COOLDOWN_PREFIX = "external_cooldown"; + // "external_cooldown_target"="iceberg_catalog.iceberg_db.iceberg_tbl", + public static final String PROPERTIES_EXTERNAL_COOLDOWN_TARGET = "external_cooldown_target"; + public static final String PROPERTIES_EXTERNAL_COOLDOWN_CONFIG = "external_cooldown_config"; + + // "external_cooldown_schedule"="START END EVERY INTERVAL " + public static final String PROPERTIES_EXTERNAL_COOLDOWN_SCHEDULE = "external_cooldown_schedule"; + + // "external_cooldown_wait_second"="86400" + public static final String PROPERTIES_EXTERNAL_COOLDOWN_WAIT_SECOND = "external_cooldown_wait_second"; + // external cool down synced time(ms) + public static final String PROPERTIES_EXTERNAL_COOLDOWN_SYNCED_TIME = "external_cooldown_synced_time"; + // external cool down consistency check time(ms) + public static final String PROPERTIES_EXTERNAL_COOLDOWN_CONSISTENCY_CHECK_TIME = "external_cooldown_consistency_check_time"; public static DataProperty analyzeDataProperty(Map properties, DataProperty inferredDataProperty, @@ -1665,6 +1684,54 @@ public static DataProperty analyzeMVDataProperty(MaterializedView materializedVi return dataProperty; } + public static ExternalCooldownConfig analyzeExternalCoolDownConfig(Map properties) + throws DdlException { + ExternalCooldownConfig externalCoolDownConfig = new ExternalCooldownConfig(); + + if (properties.containsKey(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_TARGET)) { + String target = properties.get(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_TARGET); + properties.remove(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_TARGET); + if (!target.isEmpty()) { + Pattern targetPattern = Pattern.compile("^\\w+\\.\\w+\\.\\w+$", Pattern.CASE_INSENSITIVE); + if (!targetPattern.matcher(target).find()) { + throw new DdlException(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_TARGET + + " must be format of {catalog}.{db}.{tbl}"); + } + String[] parts = target.split("\\."); + Table table = GlobalStateMgr.getCurrentState().getMetadataMgr().getTable(parts[0], parts[1], parts[2]); + if (table == null) { + throw new DdlException(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_TARGET + + " table " + target + " not exist"); + } + if (!(table instanceof IcebergTable)) { + throw new DdlException(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_TARGET + + " only support iceberg table"); + } + } + externalCoolDownConfig.setTarget(target); + } + if (properties.containsKey(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_SCHEDULE)) { + String schedule = properties.get(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_SCHEDULE); + properties.remove(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_SCHEDULE); + if (!ExternalCooldownSchedule.validateScheduleString(schedule)) { + throw new DdlException(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_SCHEDULE + + " must be format like `START 23:00 END 08:00 EVERY INTERVAL 1 MINUTE`"); + } + externalCoolDownConfig.setSchedule(schedule); + } + if (properties.containsKey(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_WAIT_SECOND)) { + long waitSecond; + try { + waitSecond = PropertyAnalyzer.analyzeLongProp(properties, + PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_WAIT_SECOND, 0); + } catch (AnalysisException e) { + throw new DdlException(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_WAIT_SECOND + " must be long"); + } + externalCoolDownConfig.setWaitSecond(waitSecond); + } + return externalCoolDownConfig; + } + /** * Generate a string representation of properties like ('a'='1', 'b'='2') */ @@ -1684,4 +1751,51 @@ public static String stringifyProperties(Map properties) { sb.append(")"); return sb.toString(); } + + public static long analyzeDatetimeProp(Map properties, + String propKey, long defaultVal) throws AnalysisException { + String text = properties.get(propKey); + if (text == null) { + return defaultVal; + } + properties.remove(propKey); + if (text.trim().isEmpty()) { + return 0L; + } + return TimeUtils.parseDate(text, PrimitiveType.DATETIME).getTime(); + } + + public static long analyzeExternalCooldownSyncedTimeMs(Map properties) throws AnalysisException { + long coldDownSyncedTimeMs = -1L; + if (properties != null && properties.containsKey(PROPERTIES_EXTERNAL_COOLDOWN_SYNCED_TIME)) { + String coldDownSyncedTimeMsStr = properties.get(PROPERTIES_EXTERNAL_COOLDOWN_SYNCED_TIME); + if (coldDownSyncedTimeMsStr.isEmpty()) { + coldDownSyncedTimeMs = 0L; + } else { + coldDownSyncedTimeMs = TimeUtils.timeStringToLong(coldDownSyncedTimeMsStr); + if (coldDownSyncedTimeMs == -1) { + throw new AnalysisException(PROPERTIES_EXTERNAL_COOLDOWN_SYNCED_TIME + " format error."); + } + } + } + + return coldDownSyncedTimeMs; + } + + public static long analyzeExternalCooldownConsistencyCheckTimeMs(Map properties) throws AnalysisException { + long coldDownConsistencyCheckTimeMs = -1L; + if (properties != null && properties.containsKey(PROPERTIES_EXTERNAL_COOLDOWN_CONSISTENCY_CHECK_TIME)) { + String coldDownConsistencyCheckTimeMsStr = properties.get(PROPERTIES_EXTERNAL_COOLDOWN_CONSISTENCY_CHECK_TIME); + if (coldDownConsistencyCheckTimeMsStr.isEmpty()) { + coldDownConsistencyCheckTimeMs = 0L; + } else { + coldDownConsistencyCheckTimeMs = TimeUtils.timeStringToLong(coldDownConsistencyCheckTimeMsStr); + if (coldDownConsistencyCheckTimeMs == -1) { + throw new AnalysisException(PROPERTIES_EXTERNAL_COOLDOWN_CONSISTENCY_CHECK_TIME + " format error."); + } + } + } + + return coldDownConsistencyCheckTimeMs; + } } diff --git a/fe/fe-core/src/main/java/com/starrocks/externalcooldown/ExternalCooldownConfig.java b/fe/fe-core/src/main/java/com/starrocks/externalcooldown/ExternalCooldownConfig.java new file mode 100644 index 0000000000000..b80fec3664491 --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/externalcooldown/ExternalCooldownConfig.java @@ -0,0 +1,153 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed 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 +// +// https://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 com.starrocks.externalcooldown; + +import com.google.gson.annotations.SerializedName; +import com.starrocks.common.DdlException; +import com.starrocks.common.io.Text; +import com.starrocks.common.io.Writable; +import com.starrocks.common.util.PropertyAnalyzer; +import com.starrocks.persist.gson.GsonUtils; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + +public class ExternalCooldownConfig implements Writable { + @SerializedName("target") + private String target; + + @SerializedName("schedule") + private String schedule; + + @SerializedName("waitSecond") + private Long waitSecond; + + public ExternalCooldownConfig(String target, String schedule, Long waitSecond) { + this.target = target; + this.schedule = schedule; + this.waitSecond = waitSecond; + } + + public ExternalCooldownConfig(ExternalCooldownConfig externalCoolDownConfig) { + if (externalCoolDownConfig != null) { + target = externalCoolDownConfig.target; + schedule = externalCoolDownConfig.schedule; + waitSecond = externalCoolDownConfig.waitSecond; + } else { + target = null; + schedule = null; + waitSecond = null; + } + } + + public ExternalCooldownConfig() { + this(null, null, null); + } + + public boolean isReadyForAutoCooldown() { + if (target == null || target.isEmpty()) { + return false; + } + if (waitSecond == null || waitSecond <= 0) { + return false; + } + return schedule != null && !schedule.isEmpty(); + } + + public String getTarget() { + return target; + } + + public String getSchedule() { + return schedule; + } + + public Long getWaitSecond() { + return waitSecond; + } + + public void setTarget(String target) { + this.target = target; + } + + public void setSchedule(String schedule) { + this.schedule = schedule; + } + + public void setWaitSecond(Long waitSecond) { + this.waitSecond = waitSecond; + } + + public Map getValidProperties() { + Map properties = new HashMap<>(); + if (target != null && !target.isEmpty()) { + properties.put(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_TARGET, target); + } + if (schedule != null && !schedule.isEmpty()) { + properties.put(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_SCHEDULE, schedule); + } + if (waitSecond != null && waitSecond > 0) { + properties.put(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_WAIT_SECOND, String.valueOf(waitSecond)); + } + return properties; + } + + @Override + public void write(DataOutput out) throws IOException { + Text.writeString(out, GsonUtils.GSON.toJson(this)); + } + + public static ExternalCooldownConfig read(DataInput in) throws IOException { + return GsonUtils.GSON.fromJson(Text.readString(in), ExternalCooldownConfig.class); + } + + public void mergeUpdateFromProperties(Map properties) throws DdlException { + ExternalCooldownConfig externalCoolDownConfig = PropertyAnalyzer.analyzeExternalCoolDownConfig(properties); + if (externalCoolDownConfig.getSchedule() != null) { + this.setSchedule(externalCoolDownConfig.getSchedule()); + } + if (externalCoolDownConfig.getWaitSecond() != null) { + this.setWaitSecond(externalCoolDownConfig.getWaitSecond()); + } + if (externalCoolDownConfig.getTarget() != null) { + this.setTarget(externalCoolDownConfig.getTarget()); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ExternalCooldownConfig that = (ExternalCooldownConfig) o; + return Objects.equals(target, that.target) && + Objects.equals(schedule, that.schedule) && + Objects.equals(waitSecond, that.waitSecond); + } + + @Override + public String toString() { + return String.format("{ target : %s,\n " + + "schedule : %s,\n " + + "wait second : %d }", target, schedule, waitSecond); + } +} diff --git a/fe/fe-core/src/main/java/com/starrocks/externalcooldown/ExternalCooldownPartitionSelector.java b/fe/fe-core/src/main/java/com/starrocks/externalcooldown/ExternalCooldownPartitionSelector.java new file mode 100644 index 0000000000000..bcacc93fc26bd --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/externalcooldown/ExternalCooldownPartitionSelector.java @@ -0,0 +1,310 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed 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 +// +// https://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 com.starrocks.externalcooldown; + +import com.google.common.collect.Range; +import com.google.common.collect.Sets; +import com.starrocks.catalog.Column; +import com.starrocks.catalog.IcebergTable; +import com.starrocks.catalog.ListPartitionInfo; +import com.starrocks.catalog.OlapTable; +import com.starrocks.catalog.Partition; +import com.starrocks.catalog.PartitionInfo; +import com.starrocks.catalog.PartitionKey; +import com.starrocks.catalog.RangePartitionInfo; +import com.starrocks.catalog.SinglePartitionInfo; +import com.starrocks.catalog.Table; +import com.starrocks.common.AnalysisException; +import com.starrocks.common.util.PropertyAnalyzer; +import com.starrocks.sql.common.PListCell; +import org.apache.commons.lang3.StringUtils; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static com.starrocks.sql.common.SyncPartitionUtils.createRange; + +public class ExternalCooldownPartitionSelector { + private static final Logger LOG = LogManager.getLogger(ExternalCooldownPartitionSelector.class); + private final OlapTable olapTable; + private org.apache.iceberg.Table targetIcebergTable; + private long externalCoolDownWaitSeconds; + private boolean tableSatisfied; + private String tableName; + private final String partitionStart; + private final String partitionEnd; + private final boolean isForce; + private PartitionInfo partitionInfo; + private List satisfiedPartitions; + + public ExternalCooldownPartitionSelector(OlapTable olapTable) { + this(olapTable, null, null, false); + } + + public ExternalCooldownPartitionSelector(OlapTable olapTable, + String partitionStart, String partitionEnd, boolean isForce) { + this.olapTable = olapTable; + this.partitionStart = partitionStart; + this.partitionEnd = partitionEnd; + this.isForce = isForce; + this.satisfiedPartitions = new ArrayList<>(); + this.init(); + } + + public void init() { + tableName = olapTable.getName(); + reloadSatisfiedPartitions(); + } + + public void reloadSatisfiedPartitions() { + tableSatisfied = true; + partitionInfo = olapTable.getPartitionInfo(); + + // check table has external cool down wait second + Long waitSeconds = olapTable.getExternalCoolDownWaitSecond(); + if (waitSeconds == null || waitSeconds <= 0) { + LOG.info("table[{}] has no set `{}` property or not satisfied. ignore", tableName, + PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_WAIT_SECOND); + tableSatisfied = false; + return; + } + + Table targetTable = olapTable.getExternalCoolDownTable(); + if (targetTable == null) { + LOG.debug("table[{}]'s `{}` not found. ignore", tableName, + PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_TARGET); + tableSatisfied = false; + } + if (!(targetTable instanceof IcebergTable)) { + LOG.debug("table[{}]'s `{}` property is not iceberg table. ignore", tableName, + PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_TARGET); + tableSatisfied = false; + return; + } + targetIcebergTable = ((IcebergTable) targetTable).getNativeTable(); + if (targetIcebergTable == null) { + LOG.debug("table[{}]'s `{}` property related native iceberg table not found. ignore", + tableName, PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_TARGET); + tableSatisfied = false; + } + + externalCoolDownWaitSeconds = waitSeconds; + satisfiedPartitions = this.computeSatisfiedPartitions(-1); + } + + public boolean isTableSatisfied() { + return tableSatisfied; + } + + /** + * check whether partition satisfy external cool down condition + */ + protected boolean isPartitionSatisfied(Partition partition) { + try { + return checkPartitionSatisfied(partition); + } catch (Exception e) { + LOG.warn("check partition [{}-{}] satisfy external cool down condition failed", + tableName, partition.getName(), e); + return false; + } + } + + private boolean checkPartitionSatisfied(Partition partition) { + // force cooldown don't need check cooldown state and consistency check result, + // and initial partition could also be cooldown + if (isForce) { + return true; + } + + long externalCoolDownWaitMillis = externalCoolDownWaitSeconds * 1000L; + if (externalCoolDownWaitSeconds <= 0) { + return false; + } + // partition with init version has no data, doesn't need do external cool down + if (partition.getVisibleVersion() == Partition.PARTITION_INIT_VERSION) { + LOG.debug("table[{}] partition[{}] is init version. ignore", tableName, partition.getName()); + return false; + } + + // after partition update, should wait a while to avoid unnecessary duplicate external cool down + long changedMillis = System.currentTimeMillis() - partition.getVisibleVersionTime(); + if (changedMillis <= externalCoolDownWaitMillis) { + LOG.debug("partition[{}]'s changed time hasn't reach {} {}. ignore", partition.getId(), + PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_WAIT_SECOND, externalCoolDownWaitSeconds); + return false; + } + + // check if this partition has changed since last external cool down + Long partitionCoolDownSyncedTimeMs = partitionInfo.getExternalCoolDownSyncedTimeMs(partition.getId()); + Long consistencyCheckTimeMs = partitionInfo.getExternalCoolDownConsistencyCheckTimeMs(partition.getId()); + // partition has never do external cool down or has changed since last external cool down + if (partitionCoolDownSyncedTimeMs == null || partitionCoolDownSyncedTimeMs < partition.getVisibleVersionTime()) { + return true; + } + // wait consistency check if has done external cooldown + if (consistencyCheckTimeMs == null || partitionCoolDownSyncedTimeMs > consistencyCheckTimeMs) { + // wait to do consistency check after external cool down + LOG.debug("table [{}] partition[{}] external cool down time newer then consistency check time", + tableName, partition.getName()); + return false; + } + Long diff = partitionInfo.getExternalCoolDownConsistencyCheckDifference(partition.getId()); + if (diff == null || diff == 0) { + // has consistency check after external cool down, and check result ok + LOG.debug("table [{}] partition[{}] external cool down consistency check result ok", + tableName, partition.getName()); + return false; + } + + return true; + } + + public Partition getOneSatisfiedPartition() { + if (satisfiedPartitions.isEmpty()) { + reloadSatisfiedPartitions(); + if (satisfiedPartitions.isEmpty()) { + return null; + } + } + return satisfiedPartitions.remove(0); + } + + public Partition getNextSatisfiedPartition(Partition currentPartition) { + if (satisfiedPartitions.isEmpty()) { + return null; + } + for (Partition partition : satisfiedPartitions) { + if (partition.getName().compareTo(currentPartition.getName()) <= 0) { + continue; + } + return partition; + } + return null; + } + + public Set getPartitionNamesByListWithPartitionLimit() { + boolean hasPartitionRange = StringUtils.isNoneEmpty(partitionStart) || StringUtils.isNoneEmpty(partitionEnd); + + if (hasPartitionRange) { + Set result = Sets.newHashSet(); + + Map listMap = olapTable.getValidListPartitionMap(-1); + for (Map.Entry entry : listMap.entrySet()) { + if (entry.getKey().compareTo(partitionStart) >= 0 && entry.getKey().compareTo(partitionEnd) <= 0) { + result.add(entry.getKey()); + } + } + return result; + } + + return olapTable.getValidListPartitionMap(-1).keySet(); + } + + public Set getPartitionNamesByRangeWithPartitionLimit() + throws AnalysisException { + boolean hasPartitionRange = StringUtils.isNoneEmpty(partitionStart) || StringUtils.isNoneEmpty(partitionEnd); + + if (hasPartitionRange) { + Set result = Sets.newHashSet(); + List partitionColumns = olapTable.getPartitionInfo().getPartitionColumns(olapTable.getIdToColumn()); + Column partitionColumn = partitionColumns.get(0); + Range rangeToInclude = createRange(partitionStart, partitionEnd, partitionColumn); + Map> rangeMap = olapTable.getValidRangePartitionMap(-1); + for (Map.Entry> entry : rangeMap.entrySet()) { + Range rangeToCheck = entry.getValue(); + int lowerCmp = rangeToInclude.lowerEndpoint().compareTo(rangeToCheck.upperEndpoint()); + int upperCmp = rangeToInclude.upperEndpoint().compareTo(rangeToCheck.lowerEndpoint()); + if (!(lowerCmp >= 0 || upperCmp <= 0)) { + result.add(entry.getKey()); + } + } + return result; + } + + return olapTable.getValidRangePartitionMap(-1).keySet(); + } + + // get partition names in range [partitionStart, partitionEnd] + private Set getPartitionsInRange() throws AnalysisException { + if (partitionStart == null && partitionEnd == null) { + if (partitionInfo instanceof SinglePartitionInfo) { + return olapTable.getVisiblePartitionNames(); + } else { + if (partitionInfo instanceof ListPartitionInfo) { + return olapTable.getValidListPartitionMap(-1).keySet(); + } else { + return olapTable.getValidRangePartitionMap(-1).keySet(); + } + } + } + + if (partitionInfo instanceof SinglePartitionInfo) { + return olapTable.getVisiblePartitionNames(); + } else if (partitionInfo instanceof RangePartitionInfo) { + return getPartitionNamesByRangeWithPartitionLimit(); + } else { + // ListPartitionInfo + return getPartitionNamesByListWithPartitionLimit(); + } + } + + public List computeSatisfiedPartitions(int limit) { + boolean isOlapTablePartitioned = olapTable.getPartitions().size() > 1 || olapTable.dynamicPartitionExists(); + if (!isOlapTablePartitioned && targetIcebergTable.spec() != null + && !targetIcebergTable.spec().fields().isEmpty()) { + LOG.warn("source table: {} is a none partitioned table, target table shouldn't partitionSpec fields", + tableName); + return new ArrayList<>(); + } + + Set partitionNames; + try { + partitionNames = getPartitionsInRange(); + } catch (AnalysisException e) { + LOG.warn("table: {} get partitions in range failed, {}", olapTable.getName(), e); + return new ArrayList<>(); + } + + List sortedPartitionNames = new ArrayList<>(partitionNames); + Collections.sort(sortedPartitionNames); + List chosenPartitions = new ArrayList<>(); + for (String partitionName : sortedPartitionNames) { + Partition partition = olapTable.getPartition(partitionName); + if (partition == null) { + continue; + } + + boolean isSatisfied = isPartitionSatisfied(partition); + if (isSatisfied) { + LOG.info("choose partition[{}-{}] to external cool down", tableName, partition.getName()); + chosenPartitions.add(partition); + } + if (limit > 0 && chosenPartitions.size() >= limit) { + LOG.info("stop choose partition as no remain jobs"); + return chosenPartitions; + } + } + return chosenPartitions; + } + + public boolean hasPartitionSatisfied() { + return tableSatisfied && !satisfiedPartitions.isEmpty(); + } +} diff --git a/fe/fe-core/src/main/java/com/starrocks/externalcooldown/ExternalCooldownSchedule.java b/fe/fe-core/src/main/java/com/starrocks/externalcooldown/ExternalCooldownSchedule.java new file mode 100644 index 0000000000000..a05cbc261bf93 --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/externalcooldown/ExternalCooldownSchedule.java @@ -0,0 +1,149 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed 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 +// +// https://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 com.starrocks.externalcooldown; + +import java.text.SimpleDateFormat; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + + +public class ExternalCooldownSchedule { + private static final Pattern SCHEDULE_PATTERN = Pattern.compile( + "\\s*START\\s+(?\\d{2}:\\d{2})\\s+END\\s+(?\\d{2}:\\d{2})\\s+EVERY\\s+" + + "INTERVAL\\s+(?\\d+)\\s+(?(HOUR|MINUTE|SECOND))\\s*", + Pattern.CASE_INSENSITIVE); + private final SimpleDateFormat timeFormat = new SimpleDateFormat("HH:mm"); + + private final String start; + private final String end; + private final long interval; + private final String unit; + private long lastScheduleMs = 0L; + + + public ExternalCooldownSchedule(String start, String end, long interval, String unit) { + this.start = start; + this.end = end; + this.interval = interval; + this.unit = unit; + } + + public static boolean validateScheduleString(String schedule) { + Matcher matcher = SCHEDULE_PATTERN.matcher(schedule); + if (!matcher.matches()) { + return false; + } + ExternalCooldownSchedule externalCooldownSchedule = fromString(schedule); + if (externalCooldownSchedule == null) { + return false; + } + String start = externalCooldownSchedule.getStart(); + if (start == null) { + return false; + } + if (start.compareTo("23:59") > 0 || start.compareTo("00:00") < 0) { + return false; + } + String end = externalCooldownSchedule.getEnd(); + return end.compareTo("23:59") <= 0 && end.compareTo("00:00") >= 0; + } + + public long getIntervalSeconds() { + long intervalSeconds; + switch (unit.toUpperCase()) { + case "HOUR": + intervalSeconds = interval * 3600; + break; + case "MINUTE": + intervalSeconds = interval * 60; + break; + case "SECOND": + intervalSeconds = interval; + break; + default: + intervalSeconds = interval; + break; + } + return intervalSeconds; + } + + public static ExternalCooldownSchedule fromString(String schedule) { + Matcher matcher = SCHEDULE_PATTERN.matcher(schedule); + if (!matcher.matches()) { + return null; + } + String start = matcher.group("start"); + String end = matcher.group("end"); + String interval = matcher.group("interval"); + String unit = matcher.group("unit"); + long intervalValue = Long.parseLong(interval); + return new ExternalCooldownSchedule(start, end, intervalValue, unit); + } + + public String getStart() { + return start; + } + + public String getEnd() { + return end; + } + + public long getInterval() { + return interval; + } + + public String getUnit() { + return unit; + } + + @Override + public String toString() { + return String.format("START %s END %s EVERY INTERVAL %s %s", start, end, interval, unit); + } + + public long getLastScheduleMs() { + return lastScheduleMs; + } + + public void setLastScheduleMs(long lastScheduleMs) { + this.lastScheduleMs = lastScheduleMs; + } + + public boolean trySchedule(long currentMs) { + String s = timeFormat.format(currentMs); + if (end.compareTo(start) < 0) { + // ex: [start=23:00, end=07:00) + if (!(s.compareTo(start) >= 0 || s.compareTo(end) < 0)) { + return false; + } + } else if (end.compareTo(start) > 0) { + // ex: [start=01:00, end=07:00) + if (s.compareTo(start) < 0) { + return false; + } + if (s.compareTo(end) >= 0) { + return false; + } + } else { + // never schedule if start == end + return false; + } + if ((currentMs - getLastScheduleMs()) / 1000 < getIntervalSeconds()) { + return false; + } + lastScheduleMs = currentMs; + return true; + } +} diff --git a/fe/fe-core/src/main/java/com/starrocks/journal/JournalEntity.java b/fe/fe-core/src/main/java/com/starrocks/journal/JournalEntity.java index 2f7f1f204e71b..1d71fcea2f85b 100644 --- a/fe/fe-core/src/main/java/com/starrocks/journal/JournalEntity.java +++ b/fe/fe-core/src/main/java/com/starrocks/journal/JournalEntity.java @@ -131,6 +131,7 @@ import com.starrocks.persist.gson.GsonUtils; import com.starrocks.plugin.PluginInfo; import com.starrocks.scheduler.Task; +import com.starrocks.scheduler.externalcooldown.ExternalCooldownMaintenanceJob; import com.starrocks.scheduler.mv.MVEpoch; import com.starrocks.scheduler.mv.MVMaintenanceJob; import com.starrocks.scheduler.persist.ArchiveTaskRunsLog; @@ -765,6 +766,14 @@ public void readFields(DataInput in) throws IOException { data = new Text(Text.readBinary(in)); break; } + case OperationType.OP_MODIFY_EXTERNAL_COOLDOWN_CONFIG: { + data = ModifyTablePropertyOperationLog.read(in); + break; + } + case OperationType.OP_EXTERNAL_COOLDOWN_JOB_STATE: { + data = ExternalCooldownMaintenanceJob.read(in); + break; + } default: { if (Config.metadata_ignore_unknown_operation_type) { LOG.warn("UNKNOWN Operation Type {}", opCode); diff --git a/fe/fe-core/src/main/java/com/starrocks/persist/EditLog.java b/fe/fe-core/src/main/java/com/starrocks/persist/EditLog.java index abb4fd45a0d96..7454b027a06da 100644 --- a/fe/fe-core/src/main/java/com/starrocks/persist/EditLog.java +++ b/fe/fe-core/src/main/java/com/starrocks/persist/EditLog.java @@ -82,6 +82,7 @@ import com.starrocks.proto.EncryptionKeyPB; import com.starrocks.replication.ReplicationJob; import com.starrocks.scheduler.Task; +import com.starrocks.scheduler.externalcooldown.ExternalCooldownMaintenanceJob; import com.starrocks.scheduler.mv.MVEpoch; import com.starrocks.scheduler.mv.MVMaintenanceJob; import com.starrocks.scheduler.persist.ArchiveTaskRunsLog; @@ -727,7 +728,8 @@ public void loadJournal(GlobalStateMgr globalStateMgr, JournalEntity journal) case OperationType.OP_MODIFY_ENABLE_PERSISTENT_INDEX: case OperationType.OP_MODIFY_PRIMARY_INDEX_CACHE_EXPIRE_SEC: case OperationType.OP_ALTER_TABLE_PROPERTIES: - case OperationType.OP_MODIFY_TABLE_CONSTRAINT_PROPERTY: { + case OperationType.OP_MODIFY_TABLE_CONSTRAINT_PROPERTY: + case OperationType.OP_MODIFY_EXTERNAL_COOLDOWN_CONFIG: { ModifyTablePropertyOperationLog modifyTablePropertyOperationLog = (ModifyTablePropertyOperationLog) journal.getData(); globalStateMgr.getLocalMetastore().replayModifyTableProperty(opCode, modifyTablePropertyOperationLog); @@ -1089,6 +1091,11 @@ public void loadJournal(GlobalStateMgr globalStateMgr, JournalEntity journal) GlobalStateMgr.getCurrentState().getKeyMgr().replayAddKey(keyPB); break; } + case OperationType.OP_EXTERNAL_COOLDOWN_JOB_STATE: { + ExternalCooldownMaintenanceJob job = (ExternalCooldownMaintenanceJob) journal.getData(); + GlobalStateMgr.getCurrentState().getExternalCooldownMgr().replay(job); + break; + } default: { if (Config.metadata_ignore_unknown_operation_type) { LOG.warn("UNKNOWN Operation Type {}", opCode); @@ -1842,6 +1849,10 @@ public void logModifyBinlogAvailableVersion(ModifyTablePropertyOperationLog log) logEdit(OperationType.OP_MODIFY_BINLOG_AVAILABLE_VERSION, log); } + public void logModifyExternalCoolDownConfig(ModifyTablePropertyOperationLog log) { + logEdit(OperationType.OP_MODIFY_EXTERNAL_COOLDOWN_CONFIG, log); + } + public void logMVJobState(MVMaintenanceJob job) { logEdit(OperationType.OP_MV_JOB_STATE, job); } @@ -1931,4 +1942,8 @@ public void logCancelDisableDisk(CancelDisableDiskInfo info) { public void logRecoverPartitionVersion(PartitionVersionRecoveryInfo info) { logEdit(OperationType.OP_RECOVER_PARTITION_VERSION, info); } + + public void logExternalCooldownJobState(ExternalCooldownMaintenanceJob job) { + logEdit(OperationType.OP_EXTERNAL_COOLDOWN_JOB_STATE, job); + } } diff --git a/fe/fe-core/src/main/java/com/starrocks/persist/ModifyPartitionInfo.java b/fe/fe-core/src/main/java/com/starrocks/persist/ModifyPartitionInfo.java index 39259106161a1..e97ae8acf1c48 100644 --- a/fe/fe-core/src/main/java/com/starrocks/persist/ModifyPartitionInfo.java +++ b/fe/fe-core/src/main/java/com/starrocks/persist/ModifyPartitionInfo.java @@ -58,6 +58,10 @@ public class ModifyPartitionInfo implements Writable { private short replicationNum; @SerializedName(value = "isInMemory") private boolean isInMemory; + @SerializedName(value = "externalCoolDownSyncedTimeMs") + private long externalCoolDownSyncedTimeMs; + @SerializedName(value = "externalCoolDownConsistencyCheckTimeMs") + private long externalCoolDownConsistencyCheckTimeMs; public ModifyPartitionInfo() { // for persist @@ -66,12 +70,22 @@ public ModifyPartitionInfo() { public ModifyPartitionInfo(long dbId, long tableId, long partitionId, DataProperty dataProperty, short replicationNum, boolean isInMemory) { + this(dbId, tableId, partitionId, dataProperty, replicationNum, isInMemory, -1L, -1L); + } + + public ModifyPartitionInfo(long dbId, long tableId, long partitionId, + DataProperty dataProperty, short replicationNum, + boolean isInMemory, + long externalCoolDownSyncedTimeMs, + long externalCoolDownConsistencyCheckTimeMs) { this.dbId = dbId; this.tableId = tableId; this.partitionId = partitionId; this.dataProperty = dataProperty; this.replicationNum = replicationNum; this.isInMemory = isInMemory; + this.externalCoolDownSyncedTimeMs = externalCoolDownSyncedTimeMs; + this.externalCoolDownConsistencyCheckTimeMs = externalCoolDownConsistencyCheckTimeMs; } public long getDbId() { @@ -98,6 +112,14 @@ public boolean isInMemory() { return isInMemory; } + public long getExternalCoolDownSyncedTimeMs() { + return externalCoolDownSyncedTimeMs; + } + + public long getExternalCoolDownConsistencyCheckTimeMs() { + return externalCoolDownConsistencyCheckTimeMs; + } + @Override public int hashCode() { return Objects.hashCode(dbId, tableId); @@ -114,7 +136,9 @@ public boolean equals(Object other) { ModifyPartitionInfo otherInfo = (ModifyPartitionInfo) other; return dbId == otherInfo.getDbId() && tableId == otherInfo.getTableId() && dataProperty.equals(otherInfo.getDataProperty()) && replicationNum == otherInfo.getReplicationNum() - && isInMemory == otherInfo.isInMemory(); + && isInMemory == otherInfo.isInMemory() + && externalCoolDownSyncedTimeMs == otherInfo.externalCoolDownSyncedTimeMs + && externalCoolDownConsistencyCheckTimeMs == otherInfo.externalCoolDownConsistencyCheckTimeMs; } @Override diff --git a/fe/fe-core/src/main/java/com/starrocks/persist/OperationType.java b/fe/fe-core/src/main/java/com/starrocks/persist/OperationType.java index 5470752c4e46d..a2c04ecff51af 100644 --- a/fe/fe-core/src/main/java/com/starrocks/persist/OperationType.java +++ b/fe/fe-core/src/main/java/com/starrocks/persist/OperationType.java @@ -583,7 +583,11 @@ public class OperationType { @IgnorableOnReplayFailed public static final short OP_ADD_KEY = 13512; - + // External cool down config + @IgnorableOnReplayFailed + public static final short OP_MODIFY_EXTERNAL_COOLDOWN_CONFIG = 13600; + @IgnorableOnReplayFailed + public static final short OP_EXTERNAL_COOLDOWN_JOB_STATE = 13601; /** * NOTICE: OperationType cannot use a value exceeding 20000, and an error will be reported if it exceeds */ diff --git a/fe/fe-core/src/main/java/com/starrocks/persist/metablock/SRMetaBlockID.java b/fe/fe-core/src/main/java/com/starrocks/persist/metablock/SRMetaBlockID.java index f4db1e9e235f9..2395b57120a11 100644 --- a/fe/fe-core/src/main/java/com/starrocks/persist/metablock/SRMetaBlockID.java +++ b/fe/fe-core/src/main/java/com/starrocks/persist/metablock/SRMetaBlockID.java @@ -97,6 +97,8 @@ public int getId() { public static final SRMetaBlockID PIPE_MGR = new SRMetaBlockID(32); + public static final SRMetaBlockID EXTERNAL_COOLDOWN_MGR = new SRMetaBlockID(33); + @Override public String toString() { return String.valueOf(id); diff --git a/fe/fe-core/src/main/java/com/starrocks/qe/DDLStmtExecutor.java b/fe/fe-core/src/main/java/com/starrocks/qe/DDLStmtExecutor.java index 1748df7281383..83cb052b68054 100644 --- a/fe/fe-core/src/main/java/com/starrocks/qe/DDLStmtExecutor.java +++ b/fe/fe-core/src/main/java/com/starrocks/qe/DDLStmtExecutor.java @@ -19,7 +19,10 @@ import com.starrocks.alter.SystemHandler; import com.starrocks.analysis.FunctionName; import com.starrocks.analysis.ParseNode; +import com.starrocks.analysis.TableName; import com.starrocks.catalog.Database; +import com.starrocks.catalog.OlapTable; +import com.starrocks.catalog.Table; import com.starrocks.common.AlreadyExistsException; import com.starrocks.common.Config; import com.starrocks.common.DdlException; @@ -33,8 +36,11 @@ import com.starrocks.load.EtlJobType; import com.starrocks.plugin.PluginInfo; import com.starrocks.scheduler.Constants; +import com.starrocks.scheduler.ExecuteOption; import com.starrocks.scheduler.Task; +import com.starrocks.scheduler.TaskBuilder; import com.starrocks.scheduler.TaskManager; +import com.starrocks.scheduler.TaskRun; import com.starrocks.server.GlobalStateMgr; import com.starrocks.sql.analyzer.SemanticException; import com.starrocks.sql.ast.AdminCancelRepairTableStmt; @@ -66,6 +72,7 @@ import com.starrocks.sql.ast.CancelBackupStmt; import com.starrocks.sql.ast.CancelCompactionStmt; import com.starrocks.sql.ast.CancelExportStmt; +import com.starrocks.sql.ast.CancelExternalCooldownStmt; import com.starrocks.sql.ast.CancelLoadStmt; import com.starrocks.sql.ast.CancelRefreshDictionaryStmt; import com.starrocks.sql.ast.CancelRefreshMaterializedViewStmt; @@ -76,6 +83,7 @@ import com.starrocks.sql.ast.CreateDataCacheRuleStmt; import com.starrocks.sql.ast.CreateDbStmt; import com.starrocks.sql.ast.CreateDictionaryStmt; +import com.starrocks.sql.ast.CreateExternalCooldownStmt; import com.starrocks.sql.ast.CreateFileStmt; import com.starrocks.sql.ast.CreateFunctionStmt; import com.starrocks.sql.ast.CreateMaterializedViewStatement; @@ -114,6 +122,7 @@ import com.starrocks.sql.ast.GrantRoleStmt; import com.starrocks.sql.ast.InstallPluginStmt; import com.starrocks.sql.ast.LoadStmt; +import com.starrocks.sql.ast.PartitionRangeDesc; import com.starrocks.sql.ast.PauseRoutineLoadStmt; import com.starrocks.sql.ast.RecoverDbStmt; import com.starrocks.sql.ast.RecoverPartitionStmt; @@ -149,8 +158,10 @@ import java.time.LocalDateTime; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; public class DDLStmtExecutor { @@ -392,6 +403,62 @@ public ShowResultSet visitCancelRefreshMaterializedViewStatement(CancelRefreshMa return null; } + public static ExecuteOption getCooldownExecuteOption(CreateExternalCooldownStmt externalCooldownStmt) { + boolean force = externalCooldownStmt.isForce(); + PartitionRangeDesc range = externalCooldownStmt.getPartitionRangeDesc(); + HashMap taskRunProperties = new HashMap<>(); + taskRunProperties.put(TaskRun.PARTITION_START, range == null ? null : range.getPartitionStart()); + taskRunProperties.put(TaskRun.PARTITION_END, range == null ? null : range.getPartitionEnd()); + taskRunProperties.put(TaskRun.FORCE, Boolean.toString(force)); + ExecuteOption executeOption = new ExecuteOption( + Constants.TaskRunPriority.HIGH.value(), false, taskRunProperties); + executeOption.setManual(true); + executeOption.setSync(force); + return executeOption; + } + + @Override + public ShowResultSet visitCreateExternalCooldownStatement(CreateExternalCooldownStmt stmt, + ConnectContext context) { + ErrorReport.wrapWithRuntimeException(() -> { + TaskManager taskManager = GlobalStateMgr.getCurrentState().getTaskManager(); + TableName tableName = stmt.getTableName(); + Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getTable( + tableName.getDb(), tableName.getTbl()); + if (!(table instanceof OlapTable)) { + throw new SemanticException("only support cooldown for olap table, got " + table.getType()); + } + final String cooldownTaskName = TaskBuilder.getExternalCooldownTaskName(table.getId()); + if (!taskManager.containTask(cooldownTaskName)) { + Task task = TaskBuilder.buildExternalCooldownTask(stmt); + taskManager.createTask(task, false); + } + ExecuteOption executeOption = getCooldownExecuteOption(stmt); + taskManager.executeTask(cooldownTaskName, executeOption); + }); + return null; + } + + @Override + public ShowResultSet visitCancelExternalCooldownStatement(CancelExternalCooldownStmt stmt, + ConnectContext context) { + ErrorReport.wrapWithRuntimeException(() -> { + Optional
table = GlobalStateMgr.getCurrentState().getMetadataMgr().getTable(stmt.getTableName()); + if (table.isEmpty()) { + throw new MetaNotFoundException("table " + stmt.getTableName() + " not found"); + } + if (!(table.get() instanceof OlapTable)) { + throw new DdlException("only support cooldown for olap table, got " + table.get().getType()); + } + TaskManager taskManager = GlobalStateMgr.getCurrentState().getTaskManager(); + Task cooldownTask = taskManager.getTask(TaskBuilder.getExternalCooldownTaskName(table.get().getId())); + if (cooldownTask != null) { + taskManager.killTask(cooldownTask.getName(), false); + } + }); + return null; + } + @Override public ShowResultSet visitAlterTableStatement(AlterTableStmt stmt, ConnectContext context) { ErrorReport.wrapWithRuntimeException(() -> { diff --git a/fe/fe-core/src/main/java/com/starrocks/qe/StmtExecutor.java b/fe/fe-core/src/main/java/com/starrocks/qe/StmtExecutor.java index 40d55dca1d022..9520c3e2cec73 100644 --- a/fe/fe-core/src/main/java/com/starrocks/qe/StmtExecutor.java +++ b/fe/fe-core/src/main/java/com/starrocks/qe/StmtExecutor.java @@ -143,6 +143,8 @@ import com.starrocks.sql.ast.AnalyzeHistogramDesc; import com.starrocks.sql.ast.AnalyzeProfileStmt; import com.starrocks.sql.ast.AnalyzeStmt; +import com.starrocks.sql.ast.CancelExternalCooldownStmt; +import com.starrocks.sql.ast.CreateExternalCooldownStmt; import com.starrocks.sql.ast.CreateTableAsSelectStmt; import com.starrocks.sql.ast.CreateTemporaryTableAsSelectStmt; import com.starrocks.sql.ast.CreateTemporaryTableStmt; @@ -554,6 +556,10 @@ public void execute() throws Exception { throw e; } } + } else if (parsedStmt instanceof CreateExternalCooldownStmt) { + com.starrocks.sql.analyzer.Analyzer.analyze(parsedStmt, context); + } else if (parsedStmt instanceof CancelExternalCooldownStmt) { + com.starrocks.sql.analyzer.Analyzer.analyze(parsedStmt, context); } else { execPlan = StatementPlanner.plan(parsedStmt, context); if (parsedStmt instanceof QueryStatement && context.shouldDumpQuery()) { diff --git a/fe/fe-core/src/main/java/com/starrocks/scheduler/Constants.java b/fe/fe-core/src/main/java/com/starrocks/scheduler/Constants.java index f3952530cfdc9..49383ed5204c8 100644 --- a/fe/fe-core/src/main/java/com/starrocks/scheduler/Constants.java +++ b/fe/fe-core/src/main/java/com/starrocks/scheduler/Constants.java @@ -45,7 +45,8 @@ public enum TaskSource { MV, INSERT, PIPE, - DATACACHE_SELECT; + DATACACHE_SELECT, + EXTERNAL_COOLDOWN; // Whether the task source is mergeable, only MV is mergeable by default. public boolean isMergeable() { diff --git a/fe/fe-core/src/main/java/com/starrocks/scheduler/PartitionBasedCooldownProcessor.java b/fe/fe-core/src/main/java/com/starrocks/scheduler/PartitionBasedCooldownProcessor.java new file mode 100644 index 0000000000000..8193d8ac528bf --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/scheduler/PartitionBasedCooldownProcessor.java @@ -0,0 +1,202 @@ +// 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 com.starrocks.scheduler; + +import com.google.common.base.Strings; +import com.starrocks.analysis.TableName; +import com.starrocks.catalog.Database; +import com.starrocks.catalog.IcebergTable; +import com.starrocks.catalog.OlapTable; +import com.starrocks.catalog.Partition; +import com.starrocks.catalog.PartitionInfo; +import com.starrocks.catalog.Table; +import com.starrocks.common.DdlException; +import com.starrocks.externalcooldown.ExternalCooldownPartitionSelector; +import com.starrocks.persist.EditLog; +import com.starrocks.persist.ModifyPartitionInfo; +import com.starrocks.qe.ConnectContext; +import com.starrocks.qe.OriginStatement; +import com.starrocks.qe.QueryState; +import com.starrocks.qe.StmtExecutor; +import com.starrocks.server.GlobalStateMgr; +import com.starrocks.sql.ast.StatementBase; +import com.starrocks.sql.common.DmlException; +import com.starrocks.sql.parser.SqlParser; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.Map; + + +public class PartitionBasedCooldownProcessor extends BaseTaskRunProcessor { + private static final Logger LOG = LogManager.getLogger(PartitionBasedCooldownProcessor.class); + public static final String TABLE_ID = "TABLE_ID"; + public static final String PARTITION_ID = "PARTITION_ID"; + public static final String PARTITION_START = "PARTITION_START"; + public static final String PARTITION_END = "PARTITION_END"; + public static final String FORCE = "FORCE"; + private static final String TABLE_PREFIX = "Table "; + private Database db; + private OlapTable olapTable; + private Partition partition; + private String externalTableName; + private ExternalCooldownPartitionSelector partitionSelector; + + private void prepare(TaskRunContext context) { + db = GlobalStateMgr.getCurrentState().getMetadataMgr().getDb(context.ctx.getCurrentCatalog(), context.ctx.getDatabase()); + long tableId = Long.parseLong(context.getProperties().get(TABLE_ID)); + String partitionStart = context.getProperties().get(PARTITION_START); + String partitionEnd = context.getProperties().get(PARTITION_END); + boolean isForce = Boolean.parseBoolean(context.getProperties().get(FORCE)); + Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(db.getId(), tableId); + if (table == null) { + throw new DmlException("Table with id[" + tableId + "] not found"); + } + if (!(table instanceof OlapTable)) { + throw new DmlException(TABLE_PREFIX + table.getName() + " not found or not olap table"); + } + olapTable = (OlapTable) table; + externalTableName = olapTable.getExternalCoolDownTarget(); + if (Strings.isNullOrEmpty(externalTableName)) { + throw new DmlException(TABLE_PREFIX + table.getName() + " not found external cool down target table"); + } + Table iTable = olapTable.getExternalCoolDownTable(); + if (iTable == null) { + throw new DmlException(TABLE_PREFIX + table.getName() + " get external cool down target table failed"); + } + if (!(iTable instanceof IcebergTable)) { + throw new DmlException(TABLE_PREFIX + table.getName() + "'s external table is not iceberg table"); + } + + partitionSelector = new ExternalCooldownPartitionSelector(olapTable, partitionStart, partitionEnd, isForce); + if (!partitionSelector.isTableSatisfied()) { + throw new DmlException(TABLE_PREFIX + table.getName() + " don't satisfy external cool down condition"); + } + if (!partitionSelector.hasPartitionSatisfied()) { + throw new DmlException(TABLE_PREFIX + table.getName() + " has no partition satisfy external cool down condition"); + } + if (context.getProperties().containsKey(PARTITION_ID)) { + long partitionId = Long.parseLong(context.getProperties().get(PARTITION_ID)); + partition = olapTable.getPartition(partitionId); + } else { + partition = partitionSelector.getOneSatisfiedPartition(); + } + } + + private boolean generateNextTaskRun(TaskRunContext context) { + this.partition = partitionSelector.getNextSatisfiedPartition(this.partition); + if (this.partition == null) { + return false; + } + + TaskManager taskManager = GlobalStateMgr.getCurrentState().getTaskManager(); + Map newProperties = context.getProperties(); + newProperties.put(TABLE_ID, String.valueOf(olapTable.getId())); + newProperties.put(PARTITION_ID, String.valueOf(partition.getId())); + ExecuteOption option = new ExecuteOption(Constants.TaskRunPriority.NORMAL.value(), true, newProperties); + String taskName = TaskBuilder.getExternalCooldownTaskName(olapTable.getId()); + taskManager.executeTask(taskName, option); + return true; + } + + private String generateInsertSql(ConnectContext ctx) { + ctx.setThreadLocalInfo(); + TableName externalTable = TableName.fromString(externalTableName); + TableName olapTableName = new TableName(db.getFullName(), olapTable.getName()); + return String.format("INSERT OVERWRITE %s SELECT * FROM %s PARTITION(%s)", + externalTable.toSql(), olapTableName.toSql(), partition.getName()); + } + + @Override + public void processTaskRun(TaskRunContext context) throws Exception { + prepare(context); + StmtExecutor executor = null; + try { + ConnectContext ctx = context.getCtx(); + String definition = generateInsertSql(ctx); + context.setDefinition(definition); + ctx.getState().setOk(); + ctx.getAuditEventBuilder().reset(); + ctx.getAuditEventBuilder() + .setTimestamp(System.currentTimeMillis()) + .setClientIp(context.getRemoteIp()) + .setUser(ctx.getQualifiedUser()) + .setDb(ctx.getDatabase()) + .setCatalog(ctx.getCurrentCatalog()) + .setStmt(definition); + + long visibleVersionTime = partition.getVisibleVersionTime(); + StatementBase sqlStmt = SqlParser.parse(definition, ctx.getSessionVariable()).get(0); + sqlStmt.setOrigStmt(new OriginStatement(definition, 0)); + executor = new StmtExecutor(ctx, sqlStmt); + ctx.setExecutor(executor); + ctx.setThreadLocalInfo(); + + LOG.info("[QueryId:{}] start to external cooldown table {} partition {} with sql {}", + ctx.getQueryId(), olapTable.getName(), partition.getName(), definition); + try { + executor.execute(); + } catch (Exception e) { + LOG.warn("[QueryId:{}] execute external cooldown table {} partition {} failed, err: {}", ctx.getQueryId(), + olapTable.getName(), partition.getName(), e.getMessage()); + throw new DdlException(e.getMessage()); + } finally { + LOG.info("[QueryId:{}] finished to external cooldown table {} partition {}", ctx.getQueryId(), + olapTable.getName(), partition.getName()); + auditAfterExec(context, executor.getParsedStmt(), executor.getQueryStatisticsForAuditLog()); + } + + if (ctx.getState().getStateType() == QueryState.MysqlStateType.ERR) { + LOG.warn("[QueryId:{}] external cooldown task table {} partition {} failed, err: {}", + ctx.getQueryId().toString(), olapTable.getName(), + partition.getName(), ctx.getState().getErrorMessage()); + throw new DdlException(ctx.getState().getErrorMessage()); + } + + PartitionInfo partitionInfo = olapTable.getPartitionInfo(); + long partitionId = partition.getId(); + + partitionInfo.setExternalCoolDownSyncedTimeMs(partitionId, visibleVersionTime); + // update olap table partition external cool down time + EditLog editLog = GlobalStateMgr.getCurrentState().getEditLog(); + // persist editlog + editLog.logModifyPartition(new ModifyPartitionInfo(db.getId(), olapTable.getId(), partitionId, + partitionInfo.getDataProperty(partitionId), + partitionInfo.getReplicationNum(partitionId), + partitionInfo.getIsInMemory(partitionId), + visibleVersionTime, + -1L)); + LOG.info("[QueryId:{}] finished modify partition cooldown flag as " + + "external cooldown table {} partition {} finished", + ctx.getQueryId(), olapTable.getName(), partition.getName()); + + if (!generateNextTaskRun(context)) { + LOG.info("[QueryId:{}] finished cooldown table {} " + + "as no partition satisfied cooldown condition", + ctx.getQueryId(), olapTable.getName()); + } + } finally { + if (executor != null) { + auditAfterExec(context, executor.getParsedStmt(), executor.getQueryStatisticsForAuditLog()); + } else { + // executor can be null if we encounter analysis error. + auditAfterExec(context, null, null); + } + } + } +} diff --git a/fe/fe-core/src/main/java/com/starrocks/scheduler/TaskBuilder.java b/fe/fe-core/src/main/java/com/starrocks/scheduler/TaskBuilder.java index 0ac5de20440ca..78e917870d59a 100644 --- a/fe/fe-core/src/main/java/com/starrocks/scheduler/TaskBuilder.java +++ b/fe/fe-core/src/main/java/com/starrocks/scheduler/TaskBuilder.java @@ -16,9 +16,18 @@ import com.google.common.base.Preconditions; import com.google.common.collect.Maps; +import com.google.common.collect.Range; import com.starrocks.alter.OptimizeTask; import com.starrocks.analysis.IntLiteral; +import com.starrocks.analysis.TableName; +import com.starrocks.catalog.Database; import com.starrocks.catalog.MaterializedView; +import com.starrocks.catalog.OlapTable; +import com.starrocks.catalog.Partition; +import com.starrocks.catalog.PartitionInfo; +import com.starrocks.catalog.PartitionKey; +import com.starrocks.catalog.RangePartitionInfo; +import com.starrocks.catalog.Table; import com.starrocks.common.Config; import com.starrocks.common.DdlException; import com.starrocks.common.FeConstants; @@ -32,15 +41,19 @@ import com.starrocks.server.GlobalStateMgr; import com.starrocks.sql.analyzer.SemanticException; import com.starrocks.sql.ast.AsyncRefreshSchemeDesc; +import com.starrocks.sql.ast.CreateExternalCooldownStmt; import com.starrocks.sql.ast.IntervalLiteral; import com.starrocks.sql.ast.RefreshSchemeClause; import com.starrocks.sql.ast.SubmitTaskStmt; import com.starrocks.sql.optimizer.Utils; import com.starrocks.warehouse.Warehouse; +import org.jetbrains.annotations.NotNull; +import java.util.HashMap; import java.util.Map; import java.util.concurrent.TimeUnit; +import static com.starrocks.scheduler.PartitionBasedCooldownProcessor.PARTITION_ID; import static com.starrocks.scheduler.TaskRun.MV_ID; // TaskBuilder is responsible for converting Stmt to Task Class @@ -276,7 +289,100 @@ public static void rebuildMVTask(String dbName, } } + public static Task buildExternalCooldownTask(CreateExternalCooldownStmt externalCooldownStmt) { + TableName tableName = externalCooldownStmt.getTableName(); + Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb(tableName.getDb()); + Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(tableName.getDb(), tableName.getTbl()); + if (!(table instanceof OlapTable)) { + throw new SemanticException("only support cooldown for olap table, got " + tableName); + } + OlapTable olapTable = (OlapTable) table; + + Task task = new Task(getExternalCooldownTaskName(table.getId())); + task.setSource(Constants.TaskSource.EXTERNAL_COOLDOWN); + task.setDbName(db.getOriginName()); + Map taskProperties = getExternalCooldownTaskProperties(externalCooldownStmt, table); + + task.setDefinition(String.format("INSERT OVERWRITE %s SELECT * FROM %s", + TableName.fromString(olapTable.getExternalCoolDownTarget()).toSql(), + externalCooldownStmt.getTableName().toSql())); + task.setProperties(taskProperties); + task.setExpireTime(0L); + handleSpecialTaskProperties(task); + return task; + } + + public static Task buildExternalCooldownTask(Database db, OlapTable olapTable, Partition partition) { + Task task = new Task(getExternalCooldownTaskName(olapTable.getId(), partition.getId()) + + "-" + System.currentTimeMillis()); + task.setSource(Constants.TaskSource.EXTERNAL_COOLDOWN); + task.setDbName(db.getOriginName()); + Map taskProperties = getExternalCooldownTaskProperties(olapTable, partition); + task.setDefinition(String.format("INSERT OVERWRITE %s SELECT * FROM %s", + olapTable.getExternalCoolDownTargetTableName().toSql(), + (new TableName(db.getOriginName(), olapTable.getName())).toSql() + )); + task.setProperties(taskProperties); + task.setExpireTime(0L); + handleSpecialTaskProperties(task); + return task; + } + + @NotNull + private static Map getExternalCooldownTaskProperties(OlapTable table, Partition partition) { + Map taskProperties = Maps.newHashMap(); + taskProperties.put(PartitionBasedCooldownProcessor.TABLE_ID, String.valueOf(table.getId())); + PartitionInfo partitionInfo = table.getPartitionInfo(); + RangePartitionInfo rangePartitionInfo = (RangePartitionInfo) partitionInfo; + Range range = rangePartitionInfo.getRange(partition.getId()); + taskProperties.put(PARTITION_ID, String.valueOf(partition.getId())); + taskProperties.put(PartitionBasedCooldownProcessor.PARTITION_START, + String.valueOf(range.lowerEndpoint().getKeys().get(0).getStringValue())); + taskProperties.put(PartitionBasedCooldownProcessor.PARTITION_END, + String.valueOf(range.upperEndpoint().getKeys().get(0).getStringValue())); + taskProperties.put(PartitionBasedCooldownProcessor.TABLE_ID, String.valueOf(table.getId())); + return taskProperties; + } + + public static ExecuteOption getCooldownExecuteOption(OlapTable table, Partition partition) { + PartitionInfo partitionInfo = table.getPartitionInfo(); + RangePartitionInfo rangePartitionInfo = (RangePartitionInfo) partitionInfo; + Range range = rangePartitionInfo.getRange(partition.getId()); + HashMap taskRunProperties = new HashMap<>(); + taskRunProperties.put(PARTITION_ID, String.valueOf(partition.getId())); + taskRunProperties.put(TaskRun.PARTITION_START, range.lowerEndpoint().getKeys().get(0).getStringValue()); + taskRunProperties.put(TaskRun.PARTITION_END, range.upperEndpoint().getKeys().get(0).getStringValue()); + taskRunProperties.put(TaskRun.FORCE, Boolean.toString(false)); + ExecuteOption executeOption = new ExecuteOption( + Constants.TaskRunPriority.HIGH.value(), false, taskRunProperties); + executeOption.setManual(true); + executeOption.setSync(false); + return executeOption; + } + + @NotNull + private static Map getExternalCooldownTaskProperties(CreateExternalCooldownStmt stmt, Table table) { + Map taskProperties = Maps.newHashMap(); + taskProperties.put(PartitionBasedCooldownProcessor.TABLE_ID, String.valueOf(table.getId())); + if (stmt.getPartitionRangeDesc() != null) { + taskProperties.put(PartitionBasedCooldownProcessor.PARTITION_START, + String.valueOf(stmt.getPartitionRangeDesc().getPartitionStart())); + taskProperties.put(PartitionBasedCooldownProcessor.PARTITION_END, + String.valueOf(stmt.getPartitionRangeDesc().getPartitionEnd())); + } + taskProperties.put(PartitionBasedCooldownProcessor.TABLE_ID, String.valueOf(table.getId())); + return taskProperties; + } + public static String getMvTaskName(long mvId) { return "mv-" + mvId; } + + public static String getExternalCooldownTaskName(Long tableId) { + return "external-cooldown-" + tableId; + } + + public static String getExternalCooldownTaskName(Long tableId, long partitionId) { + return "external-cooldown-" + tableId + "-" + partitionId; + } } diff --git a/fe/fe-core/src/main/java/com/starrocks/scheduler/TaskRunBuilder.java b/fe/fe-core/src/main/java/com/starrocks/scheduler/TaskRunBuilder.java index 9ed8815a75ecf..0ad4d8a8526ea 100644 --- a/fe/fe-core/src/main/java/com/starrocks/scheduler/TaskRunBuilder.java +++ b/fe/fe-core/src/main/java/com/starrocks/scheduler/TaskRunBuilder.java @@ -52,6 +52,8 @@ public TaskRun build() { taskRun.setProcessor(new PartitionBasedMvRefreshProcessor()); } else if (task.getSource().equals(Constants.TaskSource.DATACACHE_SELECT)) { taskRun.setProcessor(new DataCacheSelectProcessor()); + } else if (task.getSource().equals(Constants.TaskSource.EXTERNAL_COOLDOWN)) { + taskRun.setProcessor(new PartitionBasedCooldownProcessor()); } else { taskRun.setProcessor(new SqlTaskRunProcessor()); } diff --git a/fe/fe-core/src/main/java/com/starrocks/scheduler/externalcooldown/ExternalCooldownJobExecutor.java b/fe/fe-core/src/main/java/com/starrocks/scheduler/externalcooldown/ExternalCooldownJobExecutor.java new file mode 100644 index 0000000000000..797a47ff6ba2a --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/scheduler/externalcooldown/ExternalCooldownJobExecutor.java @@ -0,0 +1,68 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed 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 +// +// https://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 com.starrocks.scheduler.externalcooldown; + +import com.starrocks.common.DdlException; +import com.starrocks.common.util.FrontendDaemon; +import com.starrocks.server.GlobalStateMgr; +import org.apache.commons.collections.CollectionUtils; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.util.List; +import java.util.stream.Collectors; + + +public class ExternalCooldownJobExecutor extends FrontendDaemon { + private static final long EXECUTOR_INTERVAL_MILLIS = 5000; + private static final Logger LOG = LogManager.getLogger(ExternalCooldownJobExecutor.class); + + public ExternalCooldownJobExecutor() { + super("External Cooldown Job Executor", EXECUTOR_INTERVAL_MILLIS); + } + + @Override + protected void runAfterCatalogReady() { + // initialize if need + GlobalStateMgr.getCurrentState().getExternalCooldownMgr().doInitializeIfNeed(); + + try { + runImpl(); + } catch (Throwable e) { + LOG.error("Failed to run the ExternalCooldownJobExecutor ", e); + } + } + + private void runImpl() { + List jobs = GlobalStateMgr.getCurrentState().getExternalCooldownMgr().getRunnableJobs(); + if (CollectionUtils.isEmpty(jobs)) { + return; + } + + long startMillis = System.currentTimeMillis(); + for (ExternalCooldownMaintenanceJob job : jobs) { + try { + job.onSchedule(System.currentTimeMillis()); + } catch (DdlException e) { + LOG.warn("[ExternalCooldownJobExecutor] execute job got exception", e); + } + } + + String jobNameList = jobs.stream().map(x -> x.getOlapTable().getName()).collect(Collectors.joining(", ")); + long duration = System.currentTimeMillis() - startMillis; + LOG.info("[ExternalCooldownJobExecutor] finish schedule batch of jobs in {}ms: {}", duration, jobNameList); + } +} diff --git a/fe/fe-core/src/main/java/com/starrocks/scheduler/externalcooldown/ExternalCooldownMaintenanceJob.java b/fe/fe-core/src/main/java/com/starrocks/scheduler/externalcooldown/ExternalCooldownMaintenanceJob.java new file mode 100644 index 0000000000000..fb286d830a581 --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/scheduler/externalcooldown/ExternalCooldownMaintenanceJob.java @@ -0,0 +1,212 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed 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 +// +// https://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 com.starrocks.scheduler.externalcooldown; + +import com.google.common.base.Preconditions; +import com.google.gson.annotations.SerializedName; +import com.starrocks.catalog.Database; +import com.starrocks.catalog.OlapTable; +import com.starrocks.catalog.Partition; +import com.starrocks.catalog.Table; +import com.starrocks.common.DdlException; +import com.starrocks.common.io.Text; +import com.starrocks.common.io.Writable; +import com.starrocks.externalcooldown.ExternalCooldownConfig; +import com.starrocks.externalcooldown.ExternalCooldownPartitionSelector; +import com.starrocks.externalcooldown.ExternalCooldownSchedule; +import com.starrocks.persist.gson.GsonUtils; +import com.starrocks.scheduler.Constants; +import com.starrocks.scheduler.ExecuteOption; +import com.starrocks.scheduler.Task; +import com.starrocks.scheduler.TaskBuilder; +import com.starrocks.scheduler.TaskManager; +import com.starrocks.server.GlobalStateMgr; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; + + +/** + * Long-running job responsible for external cooldown maintenance. + */ +public class ExternalCooldownMaintenanceJob implements Writable { + private static final Logger LOG = LogManager.getLogger(ExternalCooldownMaintenanceJob.class); + + // Persisted state + @SerializedName("jobId") + private final long jobId; + @SerializedName("dbId") + private final long dbId; + @SerializedName("tableId") + private final long tableId; + + // Runtime ephemeral state + // At most one thread could execute this job, this flag indicates is someone scheduling this job + private OlapTable olapTable; + private ExternalCooldownPartitionSelector partitionSelector; + private ExternalCooldownSchedule schedule; + private String lastRunTaskName = null; + + public ExternalCooldownMaintenanceJob(OlapTable olapTable, long dbId) { + this.jobId = olapTable.getId(); + this.tableId = olapTable.getId(); + this.olapTable = olapTable; + this.dbId = dbId; + } + + public static ExternalCooldownMaintenanceJob read(DataInput input) throws IOException { + return GsonUtils.GSON.fromJson( + Text.readString(input), ExternalCooldownMaintenanceJob.class); + } + + public void restore() { + if (olapTable == null) { + Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(dbId, tableId); + Preconditions.checkState(table != null && table.isOlapTable()); + this.olapTable = (OlapTable) table; + } + + partitionSelector = new ExternalCooldownPartitionSelector(olapTable); + if (schedule == null) { + schedule = ExternalCooldownSchedule.fromString(olapTable.getExternalCoolDownSchedule()); + } else { + ExternalCooldownSchedule tmpSchedule = ExternalCooldownSchedule.fromString(olapTable.getExternalCoolDownSchedule()); + if (tmpSchedule != null) { + tmpSchedule.setLastScheduleMs(schedule.getLastScheduleMs()); + schedule = tmpSchedule; + } + } + } + + public void stopJob() { + if (lastRunTaskName != null) { + TaskManager taskManager = GlobalStateMgr.getCurrentState().getTaskManager(); + Task task = taskManager.getTask(lastRunTaskName); + if (task != null) { + taskManager.killTask(lastRunTaskName, true); + } + } + } + + public void onSchedule(long currentMs) throws DdlException { + if (!isRunnable()) { + LOG.warn("Job {} external cooldown config not satisfied ", this); + return; + } + if (lastRunTaskName != null) { + Task task = GlobalStateMgr.getCurrentState().getTaskManager().getTask(lastRunTaskName); + if (task != null && task.getState() != Constants.TaskState.ACTIVE) { + return; + } + } + if (schedule == null || !schedule.trySchedule(currentMs)) { + LOG.debug("current time not match external cooldown schedule, skip"); + return; + } + if (!partitionSelector.hasPartitionSatisfied()) { + partitionSelector.reloadSatisfiedPartitions(); + if (!partitionSelector.hasPartitionSatisfied()) { + return; + } + } + Partition partition = partitionSelector.getOneSatisfiedPartition(); + if (partition == null) { + return; + } + LOG.info("create external cooldown task for partition {}", partition); + Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb(dbId); + Task task = TaskBuilder.buildExternalCooldownTask(db, olapTable, partition); + TaskManager taskManager = GlobalStateMgr.getCurrentState().getTaskManager(); + List taskIds = new ArrayList<>(); + taskIds.add(task.getId()); + taskManager.dropTasks(taskIds, false); + taskManager.createTask(task, false); + ExecuteOption executeOption = TaskBuilder.getCooldownExecuteOption(olapTable, partition); + taskManager.executeTask(task.getName(), executeOption); + lastRunTaskName = task.getName(); + } + + public boolean isRunnable() { + if (olapTable == null) { + return false; + } + ExternalCooldownConfig config = olapTable.getCurExternalCoolDownConfig(); + if (config == null) { + return false; + } + return config.isReadyForAutoCooldown(); + } + + public OlapTable getOlapTable() { + return olapTable; + } + + public long getJobId() { + return jobId; + } + + public long getTableId() { + return tableId; + } + + public long getDbId() { + return dbId; + } + + public String getLastRunTaskName() { + return lastRunTaskName; + } + + public ExternalCooldownSchedule getSchedule() { + return schedule; + } + + public ExternalCooldownPartitionSelector getPartitionSelector() { + return partitionSelector; + } + + @Override + public String toString() { + return String.format("ExternalCooldownJob id=%s,dbId=%s,tableId=%d", jobId, dbId, tableId); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ExternalCooldownMaintenanceJob that = (ExternalCooldownMaintenanceJob) o; + return jobId == that.jobId && tableId == that.tableId; + } + + @Override + public int hashCode() { + return Objects.hash(jobId, tableId); + } + + @Override + public void write(DataOutput out) throws IOException { + Text.writeString(out, GsonUtils.GSON.toJson(this)); + } +} diff --git a/fe/fe-core/src/main/java/com/starrocks/scheduler/externalcooldown/ExternalCooldownMgr.java b/fe/fe-core/src/main/java/com/starrocks/scheduler/externalcooldown/ExternalCooldownMgr.java new file mode 100644 index 0000000000000..3a22398f52853 --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/scheduler/externalcooldown/ExternalCooldownMgr.java @@ -0,0 +1,190 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed 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 +// +// https://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 com.starrocks.scheduler.externalcooldown; + +import com.google.common.base.Preconditions; +import com.google.gson.annotations.SerializedName; +import com.starrocks.catalog.OlapTable; +import com.starrocks.common.io.Text; +import com.starrocks.persist.ImageWriter; +import com.starrocks.persist.gson.GsonUtils; +import com.starrocks.persist.metablock.SRMetaBlockEOFException; +import com.starrocks.persist.metablock.SRMetaBlockException; +import com.starrocks.persist.metablock.SRMetaBlockID; +import com.starrocks.persist.metablock.SRMetaBlockReader; +import com.starrocks.persist.metablock.SRMetaBlockWriter; +import com.starrocks.server.GlobalStateMgr; +import org.apache.commons.collections.CollectionUtils; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.EOFException; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; + +/** + * Manage lifetime of external cooldown, including create, build, refresh, destroy + */ +public class ExternalCooldownMgr { + private static final Logger LOG = LogManager.getLogger(ExternalCooldownMgr.class); + + private final Map jobMap = new ConcurrentHashMap<>(); + + private boolean initialized = false; + + /** + * Reload jobs from meta store + */ + public long reload(DataInputStream input, long checksum) throws IOException { + Preconditions.checkState(jobMap.isEmpty()); + + try { + String str = Text.readString(input); + SerializedJobs data = GsonUtils.GSON.fromJson(str, SerializedJobs.class); + if (CollectionUtils.isNotEmpty(data.jobList)) { + for (ExternalCooldownMaintenanceJob job : data.jobList) { + job.restore(); + jobMap.put(job.getTableId(), job); + } + LOG.info("reload external cooldown maintenance jobs: {}", data.jobList); + LOG.debug("reload external cooldown maintenance job details: {}", str); + } + checksum ^= data.jobList.size(); + } catch (EOFException e) { + LOG.warn("reload external cooldown maintenance job details: {}", e.getMessage()); + } + return checksum; + } + + /** + * Replay from journal + */ + public void replay(ExternalCooldownMaintenanceJob job) { + try { + job.restore(); + jobMap.put(job.getTableId(), job); + LOG.info("Replay external cooldown maintenance jobs: {}", job); + } catch (Exception e) { + LOG.warn("Replay external cooldown maintenance job failed: {}", job); + LOG.warn("Failed to replay external cooldown maintenance job", e); + } + } + + /** + * Store jobs in meta store + */ + public long store(DataOutputStream output, long checksum) throws IOException { + SerializedJobs data = new SerializedJobs(); + data.jobList = new ArrayList<>(jobMap.values()); + String json = GsonUtils.GSON.toJson(data); + Text.writeString(output, json); + checksum ^= data.jobList.size(); + return checksum; + } + + public void prepareMaintenanceWork(long dbId, OlapTable olapTable) { + ExternalCooldownMaintenanceJob job = jobMap.get(olapTable.getId()); + if (job != null) { + job.restore(); + return; + } + try { + // Create the job but not execute it + job = new ExternalCooldownMaintenanceJob(olapTable, dbId); + Preconditions.checkState(jobMap.putIfAbsent(job.getTableId(), job) == null, "job already existed"); + + GlobalStateMgr.getCurrentState().getEditLog().logExternalCooldownJobState(job); + LOG.info("create the maintenance job for external cooldown table: {}", olapTable.getName()); + } catch (Exception e) { + jobMap.remove(olapTable.getId()); + LOG.warn("prepare external cooldown for {} failed, ", olapTable.getName(), e); + } + + if (job != null) { + job.restore(); + } + } + + /** + * Stop the maintenance job for external cooldown after dropped + */ + public void stopMaintainExternalCooldown(OlapTable olapTable) { + ExternalCooldownMaintenanceJob job = getJob(olapTable.getId()); + if (job == null) { + LOG.warn("external cooldown job not exists {}", olapTable.getName()); + return; + } + job.stopJob(); + jobMap.remove(olapTable.getId()); + LOG.info("Remove maintenance job for external cooldown: {}", olapTable.getName()); + } + + private ExternalCooldownMaintenanceJob getJob(long tblId) { + return jobMap.get(tblId); + } + + public List getRunnableJobs() { + return this.jobMap.values().stream().filter(ExternalCooldownMaintenanceJob::isRunnable).collect(Collectors.toList()); + } + + static class SerializedJobs { + @SerializedName("jobList") + List jobList; + } + + public void save(ImageWriter imageWriter) throws IOException, SRMetaBlockException { + int numJson = 1 + jobMap.size(); + SRMetaBlockWriter writer = imageWriter.getBlockWriter(SRMetaBlockID.EXTERNAL_COOLDOWN_MGR, numJson); + writer.writeInt(jobMap.size()); + for (ExternalCooldownMaintenanceJob job : jobMap.values()) { + writer.writeJson(job); + } + + writer.close(); + } + + public void load(SRMetaBlockReader reader) throws IOException, SRMetaBlockEOFException { + int numJson = reader.readInt(); + for (int i = 0; i < numJson; ++i) { + ExternalCooldownMaintenanceJob job = reader.readJson(ExternalCooldownMaintenanceJob.class); + job.restore(); + jobMap.put(job.getTableId(), job); + } + } + + public void doInitializeIfNeed() { + if (!initialized) { + reloadJobs(); + this.initialized = true; + } + } + + public void reloadJobs() { + for (ExternalCooldownMaintenanceJob job : jobMap.values()) { + try { + job.restore(); + } catch (IllegalStateException e) { + LOG.warn("restore job {} failed, will, ", job.getTableId(), e); + this.jobMap.remove(job.getTableId()); + } + } + } +} diff --git a/fe/fe-core/src/main/java/com/starrocks/server/GlobalStateMgr.java b/fe/fe-core/src/main/java/com/starrocks/server/GlobalStateMgr.java index a41c6260c6488..bdf7f52baefb9 100644 --- a/fe/fe-core/src/main/java/com/starrocks/server/GlobalStateMgr.java +++ b/fe/fe-core/src/main/java/com/starrocks/server/GlobalStateMgr.java @@ -199,6 +199,8 @@ import com.starrocks.rpc.ThriftRPCRequestExecutor; import com.starrocks.scheduler.MVActiveChecker; import com.starrocks.scheduler.TaskManager; +import com.starrocks.scheduler.externalcooldown.ExternalCooldownJobExecutor; +import com.starrocks.scheduler.externalcooldown.ExternalCooldownMgr; import com.starrocks.scheduler.history.TableKeeper; import com.starrocks.scheduler.mv.MVJobExecutor; import com.starrocks.scheduler.mv.MaterializedViewMgr; @@ -304,6 +306,7 @@ public class GlobalStateMgr { private final StreamLoadMgr streamLoadMgr; private final ExportMgr exportMgr; private final MaterializedViewMgr materializedViewMgr; + private final ExternalCooldownMgr externalCooldownMgr; private final ConsistencyChecker consistencyChecker; private final BackupHandler backupHandler; @@ -401,6 +404,8 @@ public class GlobalStateMgr { private final MVJobExecutor mvMVJobExecutor; + private final ExternalCooldownJobExecutor externalCooldownJobExecutor; + private final SmallFileMgr smallFileMgr; private final DynamicPartitionScheduler dynamicPartitionScheduler; @@ -636,6 +641,7 @@ private GlobalStateMgr(boolean isCkptGlobalState, NodeMgr nodeMgr) { this.routineLoadMgr = new RoutineLoadMgr(); this.exportMgr = new ExportMgr(); this.materializedViewMgr = new MaterializedViewMgr(); + this.externalCooldownMgr = new ExternalCooldownMgr(); this.consistencyChecker = new ConsistencyChecker(); this.lock = new QueryableReentrantLock(true); @@ -704,6 +710,7 @@ private GlobalStateMgr(boolean isCkptGlobalState, NodeMgr nodeMgr) { this.routineLoadScheduler = new RoutineLoadScheduler(routineLoadMgr); this.routineLoadTaskScheduler = new RoutineLoadTaskScheduler(routineLoadMgr); this.mvMVJobExecutor = new MVJobExecutor(); + this.externalCooldownJobExecutor = new ExternalCooldownJobExecutor(); this.smallFileMgr = new SmallFileMgr(); @@ -1396,6 +1403,7 @@ private void startLeaderOnlyDaemonThreads() { taskManager.start(); taskCleaner.start(); mvMVJobExecutor.start(); + externalCooldownJobExecutor.start(); pipeListener.start(); pipeScheduler.start(); mvActiveChecker.start(); @@ -1537,6 +1545,7 @@ public void loadImage(String imageDir) throws IOException { .put(SRMetaBlockID.REPLICATION_MGR, replicationMgr::load) .put(SRMetaBlockID.KEY_MGR, keyMgr::load) .put(SRMetaBlockID.PIPE_MGR, pipeManager.getRepo()::load) + .put(SRMetaBlockID.EXTERNAL_COOLDOWN_MGR, externalCooldownMgr::load) .build(); Set metaMgrMustExists = new HashSet<>(loadImages.keySet()); @@ -1736,6 +1745,7 @@ public void saveImage(ImageWriter imageWriter, File curFile) throws IOException replicationMgr.save(imageWriter); keyMgr.save(imageWriter); pipeManager.getRepo().save(imageWriter); + externalCooldownMgr.save(imageWriter); } catch (SRMetaBlockException e) { LOG.error("Save meta block failed ", e); throw new IOException("Save meta block failed ", e); @@ -2158,6 +2168,14 @@ public MaterializedViewMgr getMaterializedViewMgr() { return this.materializedViewMgr; } + public ExternalCooldownMgr getExternalCooldownMgr() { + return this.externalCooldownMgr; + } + + public ExternalCooldownJobExecutor getExternalCooldownJobExecutor() { + return this.externalCooldownJobExecutor; + } + public SmallFileMgr getSmallFileMgr() { return this.smallFileMgr; } diff --git a/fe/fe-core/src/main/java/com/starrocks/server/LocalMetastore.java b/fe/fe-core/src/main/java/com/starrocks/server/LocalMetastore.java index 02ee8ff344edd..a20ab42380963 100644 --- a/fe/fe-core/src/main/java/com/starrocks/server/LocalMetastore.java +++ b/fe/fe-core/src/main/java/com/starrocks/server/LocalMetastore.java @@ -121,6 +121,7 @@ import com.starrocks.common.util.concurrent.lock.Locker; import com.starrocks.connector.ConnectorMetadata; import com.starrocks.connector.exception.StarRocksConnectorException; +import com.starrocks.externalcooldown.ExternalCooldownConfig; import com.starrocks.lake.DataCacheInfo; import com.starrocks.lake.LakeMaterializedView; import com.starrocks.lake.LakeTable; @@ -265,6 +266,7 @@ import java.util.stream.Collectors; import javax.validation.constraints.NotNull; +import static com.starrocks.common.util.PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_SYNCED_TIME; import static com.starrocks.server.GlobalStateMgr.NEXT_ID_INIT_VALUE; import static com.starrocks.server.GlobalStateMgr.isCheckpointThread; @@ -3606,6 +3608,64 @@ public void alterTableProperties(Database db, OlapTable table, Map partitions = table.getPartitions(); + long newCooldownSyncedTimeMs = 0L; + List clearPartitionIds = new ArrayList<>(); + + for (Partition partition : partitions) { + long partitionId = partition.getId(); + Long oldCoolDownSyncedTimeMs = partitionInfo.getExternalCoolDownSyncedTimeMs(partitionId); + if (oldCoolDownSyncedTimeMs == null || oldCoolDownSyncedTimeMs <= 0) { + continue; + } + clearPartitionIds.add(partitionId); + partitionInfo.setExternalCoolDownSyncedTimeMs(partitionId, newCooldownSyncedTimeMs); + GlobalStateMgr.getCurrentState().getEditLog().logModifyPartition(new ModifyPartitionInfo( + db.getId(), table.getId(), partitionId, + partitionInfo.getDataProperty(partitionId), + partitionInfo.getReplicationNum(partitionId), + partitionInfo.getIsInMemory(partitionId), + newCooldownSyncedTimeMs, -1L + )); + } + if (!clearPartitionIds.isEmpty()) { + LOG.info("clear {} for partition {} as external cooldown target changed", + PROPERTIES_EXTERNAL_COOLDOWN_SYNCED_TIME, clearPartitionIds); + } + } + } } } @@ -3654,6 +3714,15 @@ private Map validateToBeModifiedProps(Map proper String locations = PropertyAnalyzer.analyzeLocation(properties, true); results.put(PropertyAnalyzer.PROPERTIES_LABELS_LOCATION, locations); } + if (properties.containsKey(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_TARGET) || + properties.containsKey(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_WAIT_SECOND) || + properties.containsKey(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_SCHEDULE)) { + if (!table.isOlapTable()) { + throw new DdlException("Cannot set external cooldown property for non olap table"); + } + ExternalCooldownConfig config = PropertyAnalyzer.analyzeExternalCoolDownConfig(properties); + results.put(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_CONFIG, config); + } if (!properties.isEmpty()) { throw new DdlException("Modify failed because unknown properties: " + properties); } @@ -3801,6 +3870,18 @@ public void modifyBinlogMeta(Database db, OlapTable table, BinlogConfig binlogCo table.setCurBinlogConfig(binlogConfig); } + public void modifyExternalCoolDownMeta(Database db, OlapTable table, ExternalCooldownConfig externalCoolDownConfig) { + Locker locker = new Locker(); + Preconditions.checkArgument(locker.isDbWriteLockHeldByCurrentThread(db)); + ModifyTablePropertyOperationLog log = new ModifyTablePropertyOperationLog( + db.getId(), + table.getId(), + externalCoolDownConfig.getValidProperties()); + GlobalStateMgr.getCurrentState().getEditLog().logModifyExternalCoolDownConfig(log); + + table.setCurExternalCoolDownConfig(externalCoolDownConfig); + } + // The caller need to hold the db write lock public void modifyTableInMemoryMeta(Database db, OlapTable table, Map properties) { Locker locker = new Locker(); @@ -4110,6 +4191,8 @@ public void replayModifyTableProperty(short opCode, ModifyTablePropertyOperation if (!olapTable.isBinlogEnabled()) { olapTable.clearBinlogAvailableVersion(); } + } else if (opCode == OperationType.OP_MODIFY_EXTERNAL_COOLDOWN_CONFIG) { + olapTable.setCurExternalCoolDownConfig(tableProperty.getExternalCoolDownConfig()); } } } catch (Exception ex) { diff --git a/fe/fe-core/src/main/java/com/starrocks/server/OlapTableFactory.java b/fe/fe-core/src/main/java/com/starrocks/server/OlapTableFactory.java index ad21d6c7f819b..5aca0b3ab7356 100644 --- a/fe/fe-core/src/main/java/com/starrocks/server/OlapTableFactory.java +++ b/fe/fe-core/src/main/java/com/starrocks/server/OlapTableFactory.java @@ -48,6 +48,7 @@ import com.starrocks.common.util.DynamicPartitionUtil; import com.starrocks.common.util.PropertyAnalyzer; import com.starrocks.common.util.Util; +import com.starrocks.externalcooldown.ExternalCooldownConfig; import com.starrocks.lake.DataCacheInfo; import com.starrocks.lake.LakeTable; import com.starrocks.lake.StorageInfo; @@ -389,6 +390,20 @@ public Table createTable(LocalMetastore metastore, Database db, CreateTableStmt } } + if (properties != null && (properties.containsKey(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_TARGET) || + properties.containsKey(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_SCHEDULE) || + properties.containsKey(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_WAIT_SECOND))) { + ExternalCooldownConfig externalCoolDownConfig = table.getCurExternalCoolDownConfig(); + if (externalCoolDownConfig == null) { + externalCoolDownConfig = new ExternalCooldownConfig(); + } + externalCoolDownConfig.mergeUpdateFromProperties(properties); + table.setCurExternalCoolDownConfig(externalCoolDownConfig); + LOG.info("create table {} set external cool down config, target = {}, schedule = {}, wait second = {}", + tableName, externalCoolDownConfig.getTarget(), externalCoolDownConfig.getSchedule(), + externalCoolDownConfig.getWaitSecond()); + } + try { long bucketSize = PropertyAnalyzer.analyzeLongProp(properties, PropertyAnalyzer.PROPERTIES_BUCKET_SIZE, Config.default_automatic_bucket_size); diff --git a/fe/fe-core/src/main/java/com/starrocks/sql/analyzer/AlterTableClauseAnalyzer.java b/fe/fe-core/src/main/java/com/starrocks/sql/analyzer/AlterTableClauseAnalyzer.java index ec8beb45e2ad8..3e6acc2171c23 100644 --- a/fe/fe-core/src/main/java/com/starrocks/sql/analyzer/AlterTableClauseAnalyzer.java +++ b/fe/fe-core/src/main/java/com/starrocks/sql/analyzer/AlterTableClauseAnalyzer.java @@ -19,6 +19,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import com.starrocks.alter.AlterOpType; import com.starrocks.analysis.ColumnPosition; import com.starrocks.analysis.Expr; import com.starrocks.analysis.FunctionCallExpr; @@ -33,6 +34,7 @@ import com.starrocks.catalog.DynamicPartitionProperty; import com.starrocks.catalog.ExpressionRangePartitionInfo; import com.starrocks.catalog.HashDistributionInfo; +import com.starrocks.catalog.IcebergTable; import com.starrocks.catalog.Index; import com.starrocks.catalog.KeysType; import com.starrocks.catalog.MaterializedView; @@ -51,7 +53,9 @@ import com.starrocks.common.util.PropertyAnalyzer; import com.starrocks.common.util.TimeUtils; import com.starrocks.common.util.WriteQuorum; +import com.starrocks.externalcooldown.ExternalCooldownSchedule; import com.starrocks.qe.ConnectContext; +import com.starrocks.server.GlobalStateMgr; import com.starrocks.server.RunMode; import com.starrocks.sql.ast.AddColumnClause; import com.starrocks.sql.ast.AddColumnsClause; @@ -113,8 +117,10 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.regex.Pattern; import java.util.stream.Collectors; +import static com.starrocks.common.util.PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_PREFIX; import static com.starrocks.sql.common.ErrorMsgProxy.PARSER_ERROR_MSG; public class AlterTableClauseAnalyzer implements AstVisitor { @@ -174,7 +180,8 @@ public Void visitModifyTablePropertiesClause(ModifyTablePropertiesClause clause, if (properties.size() != 1 && !(TableProperty.isSamePrefixProperties(properties, TableProperty.DYNAMIC_PARTITION_PROPERTY_PREFIX) - || TableProperty.isSamePrefixProperties(properties, TableProperty.BINLOG_PROPERTY_PREFIX))) { + || TableProperty.isSamePrefixProperties(properties, TableProperty.BINLOG_PROPERTY_PREFIX) + || TableProperty.isSamePrefixProperties(properties, PROPERTIES_EXTERNAL_COOLDOWN_PREFIX))) { ErrorReport.reportSemanticException(ErrorCode.ERR_COMMON_ERROR, "Can only set one table property at a time"); } @@ -317,6 +324,52 @@ public Void visitModifyTablePropertiesClause(ModifyTablePropertiesClause clause, } } } + } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_TARGET) || + properties.containsKey(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_SCHEDULE) || + properties.containsKey(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_WAIT_SECOND)) { + + if (properties.containsKey(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_TARGET)) { + String target = properties.get(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_TARGET); + if (!target.isEmpty()) { + Pattern targetPattern = Pattern.compile("^\\w+\\.\\w+\\.\\w+$", Pattern.CASE_INSENSITIVE); + if (!targetPattern.matcher(target).find()) { + ErrorReport.reportSemanticException(ErrorCode.ERR_COMMON_ERROR, + "Property " + PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_TARGET + + " must be format of {catalog}.{db}.{tbl}"); + } + String[] parts = target.split("\\."); + Table targetTable = GlobalStateMgr.getCurrentState().getMetadataMgr().getTable( + parts[0], parts[1], parts[2]); + if (targetTable == null) { + ErrorReport.reportSemanticException(ErrorCode.ERR_COMMON_ERROR, + "Property " + PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_TARGET + + " table " + target + " not exist"); + } + if (!(targetTable instanceof IcebergTable)) { + ErrorReport.reportSemanticException(ErrorCode.ERR_COMMON_ERROR, + "Property " + PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_TARGET + + " only support iceberg table"); + } + } + } + if (properties.containsKey(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_SCHEDULE)) { + String schedule = properties.get(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_SCHEDULE); + if (!ExternalCooldownSchedule.validateScheduleString(schedule)) { + ErrorReport.reportSemanticException(ErrorCode.ERR_COMMON_ERROR, + "Property " + PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_SCHEDULE + + " must be format like `START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE`"); + } + } + if (properties.containsKey(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_WAIT_SECOND)) { + try { + Long.parseLong(properties.get(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_WAIT_SECOND)); + } catch (NumberFormatException e) { + ErrorReport.reportSemanticException(ErrorCode.ERR_COMMON_ERROR, + "Property " + PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_WAIT_SECOND + + " must be long"); + } + } + clause.setOpType(AlterOpType.MODIFY_TABLE_PROPERTY_SYNC); } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_TABLET_TYPE)) { ErrorReport.reportSemanticException(ErrorCode.ERR_COMMON_ERROR, "Alter tablet type not supported"); } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_FOREIGN_KEY_CONSTRAINT) diff --git a/fe/fe-core/src/main/java/com/starrocks/sql/analyzer/Analyzer.java b/fe/fe-core/src/main/java/com/starrocks/sql/analyzer/Analyzer.java index 511fd59cfb45c..a6a3f1d4ab186 100644 --- a/fe/fe-core/src/main/java/com/starrocks/sql/analyzer/Analyzer.java +++ b/fe/fe-core/src/main/java/com/starrocks/sql/analyzer/Analyzer.java @@ -49,6 +49,7 @@ import com.starrocks.sql.ast.CancelAlterTableStmt; import com.starrocks.sql.ast.CancelCompactionStmt; import com.starrocks.sql.ast.CancelExportStmt; +import com.starrocks.sql.ast.CancelExternalCooldownStmt; import com.starrocks.sql.ast.CancelLoadStmt; import com.starrocks.sql.ast.CancelRefreshMaterializedViewStmt; import com.starrocks.sql.ast.ClearDataCacheRulesStmt; @@ -57,6 +58,7 @@ import com.starrocks.sql.ast.CreateDataCacheRuleStmt; import com.starrocks.sql.ast.CreateDbStmt; import com.starrocks.sql.ast.CreateDictionaryStmt; +import com.starrocks.sql.ast.CreateExternalCooldownStmt; import com.starrocks.sql.ast.CreateFileStmt; import com.starrocks.sql.ast.CreateFunctionStmt; import com.starrocks.sql.ast.CreateMaterializedViewStatement; @@ -1016,5 +1018,17 @@ public Void visitShowDictionaryStatement(ShowDictionaryStmt statement, ConnectCo DictionaryAnalyzer.analyze(statement, context); return null; } + + @Override + public Void visitCreateExternalCooldownStatement(CreateExternalCooldownStmt statement, ConnectContext context) { + ExternalCooldownAnalyzer.analyze(statement, context); + return null; + } + + @Override + public Void visitCancelExternalCooldownStatement(CancelExternalCooldownStmt statement, ConnectContext context) { + ExternalCooldownAnalyzer.analyze(statement, context); + return null; + } } } diff --git a/fe/fe-core/src/main/java/com/starrocks/sql/analyzer/AuthorizerStmtVisitor.java b/fe/fe-core/src/main/java/com/starrocks/sql/analyzer/AuthorizerStmtVisitor.java index 5d922d72a55f3..12d4cc7d4cb8a 100644 --- a/fe/fe-core/src/main/java/com/starrocks/sql/analyzer/AuthorizerStmtVisitor.java +++ b/fe/fe-core/src/main/java/com/starrocks/sql/analyzer/AuthorizerStmtVisitor.java @@ -82,12 +82,14 @@ import com.starrocks.sql.ast.CancelBackupStmt; import com.starrocks.sql.ast.CancelCompactionStmt; import com.starrocks.sql.ast.CancelExportStmt; +import com.starrocks.sql.ast.CancelExternalCooldownStmt; import com.starrocks.sql.ast.CancelLoadStmt; import com.starrocks.sql.ast.CancelRefreshMaterializedViewStmt; import com.starrocks.sql.ast.CleanTemporaryTableStmt; import com.starrocks.sql.ast.CreateAnalyzeJobStmt; import com.starrocks.sql.ast.CreateCatalogStmt; import com.starrocks.sql.ast.CreateDbStmt; +import com.starrocks.sql.ast.CreateExternalCooldownStmt; import com.starrocks.sql.ast.CreateFileStmt; import com.starrocks.sql.ast.CreateFunctionStmt; import com.starrocks.sql.ast.CreateMaterializedViewStatement; @@ -1725,6 +1727,16 @@ public Void visitDataCacheSelectStatement(DataCacheSelectStatement statement, Co return null; } + @Override + public Void visitCreateExternalCooldownStatement(CreateExternalCooldownStmt statement, ConnectContext context) { + return null; + } + + @Override + public Void visitCancelExternalCooldownStatement(CancelExternalCooldownStmt statement, ConnectContext context) { + return null; + } + @Override public Void visitShowAlterStatement(ShowAlterStmt statement, ConnectContext context) { // `show alter table` only show tables/views/mvs that user has any privilege on, we will check it in diff --git a/fe/fe-core/src/main/java/com/starrocks/sql/analyzer/CreateTableLikeAnalyzer.java b/fe/fe-core/src/main/java/com/starrocks/sql/analyzer/CreateTableLikeAnalyzer.java index 67cd34ebdbc78..55d8adaf43cf8 100644 --- a/fe/fe-core/src/main/java/com/starrocks/sql/analyzer/CreateTableLikeAnalyzer.java +++ b/fe/fe-core/src/main/java/com/starrocks/sql/analyzer/CreateTableLikeAnalyzer.java @@ -79,6 +79,7 @@ public static void analyze(CreateTableLikeStmt stmt, ConnectContext context) { } com.starrocks.sql.analyzer.Analyzer.analyze(parsedCreateTableStmt, context); + parsedCreateTableStmt.removeExternalCoolDownConfigProperties(); stmt.setCreateTableStmt(parsedCreateTableStmt); } else { ErrorReport.reportSemanticException(ErrorCode.ERROR_CREATE_TABLE_LIKE_UNSUPPORTED_VIEW); diff --git a/fe/fe-core/src/main/java/com/starrocks/sql/analyzer/ExternalCooldownAnalyzer.java b/fe/fe-core/src/main/java/com/starrocks/sql/analyzer/ExternalCooldownAnalyzer.java new file mode 100644 index 0000000000000..d90f7cade3d40 --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/sql/analyzer/ExternalCooldownAnalyzer.java @@ -0,0 +1,70 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed 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 +// +// https://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 com.starrocks.sql.analyzer; + +import com.google.common.base.Strings; +import com.starrocks.analysis.TableName; +import com.starrocks.common.ErrorCode; +import com.starrocks.common.ErrorReport; +import com.starrocks.qe.ConnectContext; +import com.starrocks.sql.ast.AstVisitor; +import com.starrocks.sql.ast.CancelExternalCooldownStmt; +import com.starrocks.sql.ast.CreateExternalCooldownStmt; +import com.starrocks.sql.ast.StatementBase; + + +public class ExternalCooldownAnalyzer { + public static void analyze(StatementBase stmt, ConnectContext session) { + new ExternalCooldownAnalyzer.ExternalCooldownAnalyzerVisitor().visit(stmt, session); + } + + private ExternalCooldownAnalyzer() { + throw new IllegalStateException("creating an instance is illegal"); + } + + static class ExternalCooldownAnalyzerVisitor implements AstVisitor { + @Override + public Void visitCreateExternalCooldownStatement(CreateExternalCooldownStmt createExternalCooldownStmt, + ConnectContext context) { + TableName tableName = createExternalCooldownStmt.getTableName(); + if (tableName.getDb() == null) { + String dbName = context.getDatabase(); + if (Strings.isNullOrEmpty(dbName)) { + ErrorReport.reportSemanticException(ErrorCode.ERR_NO_DB_ERROR); + } + tableName.setDb(dbName); + } + createExternalCooldownStmt.setTableName(tableName); + createExternalCooldownStmt.setPartitionRangeDesc(createExternalCooldownStmt.getPartitionRangeDesc()); + return null; + } + + @Override + public Void visitCancelExternalCooldownStatement( + CancelExternalCooldownStmt cancelExternalCooldownStmt, ConnectContext context) { + TableName tableName = cancelExternalCooldownStmt.getTableName(); + if (tableName.getDb() == null) { + String dbName = context.getDatabase(); + if (Strings.isNullOrEmpty(dbName)) { + ErrorReport.reportSemanticException(ErrorCode.ERR_NO_DB_ERROR); + } + tableName.setDb(dbName); + } + cancelExternalCooldownStmt.setTableName(tableName); + return null; + } + } +} diff --git a/fe/fe-core/src/main/java/com/starrocks/sql/ast/AstTraverser.java b/fe/fe-core/src/main/java/com/starrocks/sql/ast/AstTraverser.java index 3ce3d13167941..57bce1246c4fe 100644 --- a/fe/fe-core/src/main/java/com/starrocks/sql/ast/AstTraverser.java +++ b/fe/fe-core/src/main/java/com/starrocks/sql/ast/AstTraverser.java @@ -68,6 +68,16 @@ public R visitSubmitTaskStatement(SubmitTaskStmt statement, C context) { return null; } + @Override + public R visitCreateExternalCooldownStatement(CreateExternalCooldownStmt statement, C context) { + return null; + } + + @Override + public R visitCancelExternalCooldownStatement(CancelExternalCooldownStmt statement, C context) { + return null; + } + @Override public R visitCreatePipeStatement(CreatePipeStmt statement, C context) { if (statement.getInsertStmt() != null) { diff --git a/fe/fe-core/src/main/java/com/starrocks/sql/ast/AstVisitor.java b/fe/fe-core/src/main/java/com/starrocks/sql/ast/AstVisitor.java index 9c95b65327e56..ceabd2c7fec88 100644 --- a/fe/fe-core/src/main/java/com/starrocks/sql/ast/AstVisitor.java +++ b/fe/fe-core/src/main/java/com/starrocks/sql/ast/AstVisitor.java @@ -241,6 +241,16 @@ default R visitDropTaskStmt(DropTaskStmt statement, C context) { return visitDDLStatement(statement, context); } + // ---------------------------------------- Create External Cooldown Statement --------------------------------------------------------- + + default R visitCreateExternalCooldownStatement(CreateExternalCooldownStmt statement, C context) { + return visitStatement(statement, context); + } + + default R visitCancelExternalCooldownStatement(CancelExternalCooldownStmt statement, C context) { + return visitStatement(statement, context); + } + // ---------------------------------------- Partition Statement ---------------------------------------------------- default R visitRecoverPartitionStatement(RecoverPartitionStmt statement, C context) { diff --git a/fe/fe-core/src/main/java/com/starrocks/sql/ast/CancelExternalCooldownStmt.java b/fe/fe-core/src/main/java/com/starrocks/sql/ast/CancelExternalCooldownStmt.java new file mode 100644 index 0000000000000..20faca030bf82 --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/sql/ast/CancelExternalCooldownStmt.java @@ -0,0 +1,48 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed 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 +// +// https://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 com.starrocks.sql.ast; + +import com.starrocks.analysis.TableName; +import com.starrocks.sql.parser.NodePosition; + + +public class CancelExternalCooldownStmt extends DdlStmt { + + private TableName tableName; + + public CancelExternalCooldownStmt(TableName tableName, NodePosition pos) { + super(pos); + this.tableName = tableName; + } + + public TableName getTableName() { + return tableName; + } + + public void setTableName(TableName tableName) { + this.tableName = tableName; + } + + @Override + public R accept(AstVisitor visitor, C context) { + return visitor.visitCancelExternalCooldownStatement(this, context); + } + + @Override + public String toSql() { + return "CANCEL COOLDOWN TABLE " + tableName; + } +} diff --git a/fe/fe-core/src/main/java/com/starrocks/sql/ast/CreateExternalCooldownStmt.java b/fe/fe-core/src/main/java/com/starrocks/sql/ast/CreateExternalCooldownStmt.java new file mode 100644 index 0000000000000..551ce7a7958cf --- /dev/null +++ b/fe/fe-core/src/main/java/com/starrocks/sql/ast/CreateExternalCooldownStmt.java @@ -0,0 +1,79 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed 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 +// +// https://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 com.starrocks.sql.ast; + +import com.starrocks.analysis.TableName; +import com.starrocks.catalog.Column; +import com.starrocks.catalog.ScalarType; +import com.starrocks.qe.ShowResultSetMetaData; +import com.starrocks.sql.parser.NodePosition; + +public class CreateExternalCooldownStmt extends DdlStmt { + private TableName tableName; + private final boolean force; + private PartitionRangeDesc partitionRangeDesc; + + public static final ShowResultSetMetaData META_DATA = + ShowResultSetMetaData.builder() + .addColumn(new Column("QUERY_ID", ScalarType.createVarchar(60))) + .build(); + public CreateExternalCooldownStmt(TableName tableName, PartitionRangeDesc partitionRangeDesc, + boolean force, NodePosition pos) { + super(pos); + this.tableName = tableName; + this.partitionRangeDesc = partitionRangeDesc; + this.force = force; + } + + public TableName getTableName() { + return tableName; + } + + public void setTableName(TableName tableName) { + this.tableName = tableName; + } + + public PartitionRangeDesc getPartitionRangeDesc() { + return partitionRangeDesc; + } + + public void setPartitionRangeDesc(PartitionRangeDesc partitionRangeDesc) { + this.partitionRangeDesc = partitionRangeDesc; + } + + public boolean isForce() { + return force; + } + + @Override + public R accept(AstVisitor visitor, C context) { + return visitor.visitCreateExternalCooldownStatement(this, context); + } + + @Override + public String toSql() { + String sql = "COOLDOWN TABLE " + tableName.toSql(); + if (partitionRangeDesc != null) { + sql += " PARTITION START ('" + partitionRangeDesc.getPartitionStart() + + "') END ('" + partitionRangeDesc.getPartitionEnd() + "')"; + } + if (isForce()) { + sql += " FORCE"; + } + + return sql; + } +} diff --git a/fe/fe-core/src/main/java/com/starrocks/sql/ast/CreateTableStmt.java b/fe/fe-core/src/main/java/com/starrocks/sql/ast/CreateTableStmt.java index 5cf6e28d7d62f..26ec611a236ee 100644 --- a/fe/fe-core/src/main/java/com/starrocks/sql/ast/CreateTableStmt.java +++ b/fe/fe-core/src/main/java/com/starrocks/sql/ast/CreateTableStmt.java @@ -19,6 +19,7 @@ import com.starrocks.analysis.TableName; import com.starrocks.catalog.Column; import com.starrocks.catalog.Index; +import com.starrocks.common.util.PropertyAnalyzer; import com.starrocks.sql.common.EngineType; import com.starrocks.sql.parser.NodePosition; @@ -344,4 +345,12 @@ public static CreateTableStmt read(DataInput in) throws IOException { public R accept(AstVisitor visitor, C context) { return visitor.visitCreateTableStatement(this, context); } + + public void removeExternalCoolDownConfigProperties() { + if (this.properties != null) { + this.properties.remove(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_TARGET); + this.properties.remove(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_SCHEDULE); + this.properties.remove(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_WAIT_SECOND); + } + } } diff --git a/fe/fe-core/src/main/java/com/starrocks/sql/parser/AstBuilder.java b/fe/fe-core/src/main/java/com/starrocks/sql/parser/AstBuilder.java index cf20af37c1e62..0e0d82c95e0b8 100644 --- a/fe/fe-core/src/main/java/com/starrocks/sql/parser/AstBuilder.java +++ b/fe/fe-core/src/main/java/com/starrocks/sql/parser/AstBuilder.java @@ -174,6 +174,7 @@ import com.starrocks.sql.ast.CancelBackupStmt; import com.starrocks.sql.ast.CancelCompactionStmt; import com.starrocks.sql.ast.CancelExportStmt; +import com.starrocks.sql.ast.CancelExternalCooldownStmt; import com.starrocks.sql.ast.CancelLoadStmt; import com.starrocks.sql.ast.CancelRefreshDictionaryStmt; import com.starrocks.sql.ast.CancelRefreshMaterializedViewStmt; @@ -191,6 +192,7 @@ import com.starrocks.sql.ast.CreateDataCacheRuleStmt; import com.starrocks.sql.ast.CreateDbStmt; import com.starrocks.sql.ast.CreateDictionaryStmt; +import com.starrocks.sql.ast.CreateExternalCooldownStmt; import com.starrocks.sql.ast.CreateFileStmt; import com.starrocks.sql.ast.CreateFunctionStmt; import com.starrocks.sql.ast.CreateImageClause; @@ -1829,6 +1831,30 @@ public ParseNode visitDropTaskStatement(StarRocksParser.DropTaskStatementContext return new DropTaskStmt(taskName, force, createPos(context)); } + // ------------------------------------------- External Cooldown Statement ------------------------------------------------------ + + @Override + public ParseNode visitCooldownStatement(StarRocksParser.CooldownStatementContext context) { + QualifiedName qualifiedName = getQualifiedName(context.qualifiedName()); + TableName targetTableName = qualifiedNameToTableName(qualifiedName); + + PartitionRangeDesc partitionRangeDesc = null; + if (context.partitionRangeDesc() != null) { + partitionRangeDesc = + (PartitionRangeDesc) visit(context.partitionRangeDesc()); + } + + return new CreateExternalCooldownStmt(targetTableName, partitionRangeDesc, + context.FORCE() != null, createPos(context)); + } + + @Override + public ParseNode visitCancelCooldownStatement(StarRocksParser.CancelCooldownStatementContext context) { + QualifiedName qualifiedName = getQualifiedName(context.qualifiedName()); + TableName targetTableName = qualifiedNameToTableName(qualifiedName); + return new CancelExternalCooldownStmt(targetTableName, createPos(context)); + } + // ------------------------------------------- Materialized View Statement ----------------------------------------- public static final ImmutableList MATERIALIZEDVIEW_REFRESHSCHEME_SUPPORT_UNIT_IDENTIFIERS = diff --git a/fe/fe-core/src/main/java/com/starrocks/sql/parser/StarRocks.g4 b/fe/fe-core/src/main/java/com/starrocks/sql/parser/StarRocks.g4 index ab190be514b59..bdbdb7a496d7c 100644 --- a/fe/fe-core/src/main/java/com/starrocks/sql/parser/StarRocks.g4 +++ b/fe/fe-core/src/main/java/com/starrocks/sql/parser/StarRocks.g4 @@ -80,6 +80,10 @@ statement | submitTaskStatement | dropTaskStatement + // External Cooldown Statement + | cooldownStatement + | cancelCooldownStatement + // Materialized View Statement | createMaterializedViewStatement | showMaterializedViewsStatement @@ -623,6 +627,17 @@ taskScheduleDesc : SCHEDULE (START '(' string ')')? EVERY '(' taskInterval ')' ; +// ------------------------------------------- External Cool Down Statement ---------------------------------------------------------- + +cooldownStatement + : COOLDOWN TABLE qualifiedName? properties? + (PARTITION partitionRangeDesc)? FORCE? + ; + +cancelCooldownStatement + : CANCEL COOLDOWN TABLE qualifiedName FORCE? + ; + // ------------------------------------------- Materialized View Statement --------------------------------------------- createMaterializedViewStatement diff --git a/fe/fe-core/src/main/java/com/starrocks/sql/parser/StarRocksLex.g4 b/fe/fe-core/src/main/java/com/starrocks/sql/parser/StarRocksLex.g4 index f318b29550cba..065dd5f0f5c92 100644 --- a/fe/fe-core/src/main/java/com/starrocks/sql/parser/StarRocksLex.g4 +++ b/fe/fe-core/src/main/java/com/starrocks/sql/parser/StarRocksLex.g4 @@ -428,6 +428,7 @@ TABLET: 'TABLET'; TABLETS: 'TABLETS'; TAG: 'TAG'; TASK: 'TASK'; +COOLDOWN: 'COOLDOWN'; TEMPORARY: 'TEMPORARY'; TERMINATED: 'TERMINATED'; THAN: 'THAN'; diff --git a/fe/fe-core/src/test/java/com/starrocks/alter/AlterTableTest.java b/fe/fe-core/src/test/java/com/starrocks/alter/AlterTableTest.java index b481dc1bd495c..3c08d8d51c199 100644 --- a/fe/fe-core/src/test/java/com/starrocks/alter/AlterTableTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/alter/AlterTableTest.java @@ -14,9 +14,13 @@ package com.starrocks.alter; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.starrocks.catalog.DataProperty; import com.starrocks.catalog.Database; +import com.starrocks.catalog.IcebergTable; import com.starrocks.catalog.OlapTable; +import com.starrocks.catalog.PrimitiveType; import com.starrocks.catalog.RangePartitionInfo; import com.starrocks.catalog.Table; import com.starrocks.common.AnalysisException; @@ -26,6 +30,9 @@ import com.starrocks.common.Pair; import com.starrocks.common.util.PropertyAnalyzer; import com.starrocks.common.util.TimeUtils; +import com.starrocks.connector.ConnectorContext; +import com.starrocks.connector.ConnectorMgr; +import com.starrocks.connector.iceberg.IcebergMetadata; import com.starrocks.persist.ModifyTablePropertyOperationLog; import com.starrocks.persist.OperationType; import com.starrocks.qe.ConnectContext; @@ -39,12 +46,16 @@ import com.starrocks.thrift.TStorageType; import com.starrocks.utframe.StarRocksAssert; import com.starrocks.utframe.UtFrameUtils; +import mockit.Mock; +import mockit.MockUp; +import org.apache.iceberg.BaseTable; import org.junit.Assert; import org.junit.BeforeClass; import org.junit.Test; import org.threeten.extra.PeriodDuration; import java.util.List; +import java.util.Map; import java.util.Set; public class AlterTableTest { @@ -467,4 +478,158 @@ public void testAlterLocationPropTableToColocate() throws Exception { Assert.assertTrue(e.getMessage().contains("table has location property and cannot be colocated")); } } + + @Test + public void testAlterTableExternalCoolDownConfig() throws Exception { + new MockUp() { + @Mock + public Table getTable(String dbName, String tblName) { + return new IcebergTable(1, "table1", "iceberg_catalog", + "iceberg_catalog", "iceberg_db", + "table1", "", Lists.newArrayList(), new BaseTable(null, ""), Maps.newHashMap()); + } + }; + new MockUp() { + @Mock + public String getTableIdentifier() { + return "iceberg_catalog.iceberg_db.table1"; + } + }; + + ConnectorMgr connectorMgr = GlobalStateMgr.getCurrentState().getConnectorMgr(); + Map properties = Maps.newHashMap(); + + properties.put("type", "iceberg"); + properties.put("iceberg.catalog.type", "hive"); + properties.put("hive.metastore.uris", "thrift://127.0.0.1:9083"); + connectorMgr.createConnector(new ConnectorContext("iceberg_catalog", "iceberg", properties), false); + + starRocksAssert.useDatabase("test").withTable("CREATE TABLE test_external_cooldown (\n" + + "event_day DATE,\n" + + "site_id INT DEFAULT '10',\n" + + "city_code VARCHAR(100),\n" + + "user_name VARCHAR(32) DEFAULT '',\n" + + "pv BIGINT DEFAULT '0'\n" + + ")\n" + + "DUPLICATE KEY(event_day, site_id, city_code, user_name)\n" + + "PARTITION BY RANGE(event_day)(\n" + + "PARTITION p20200321 VALUES LESS THAN (\"2020-03-22\"),\n" + + "PARTITION p20200322 VALUES LESS THAN (\"2020-03-23\"),\n" + + "PARTITION p20200323 VALUES LESS THAN (\"2020-03-24\"),\n" + + "PARTITION p20200324 VALUES LESS THAN MAXVALUE\n" + + ")\n" + + "DISTRIBUTED BY HASH(event_day, site_id)\n" + + "PROPERTIES(\n" + + "\"replication_num\" = \"1\",\n" + + "\"external_cooldown_target\" = \"iceberg_catalog.db.external_iceberg_tbl\",\n" + + "\"external_cooldown_schedule\" = \"START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE\"\n," + + "\"external_cooldown_wait_second\" = \"3600\"\n" + + ");"); + + ConnectContext ctx = starRocksAssert.getCtx(); + Database db = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); + OlapTable olapTable = (OlapTable) db.getTable("test_external_cooldown"); + + Assert.assertNotNull(olapTable.getExternalCoolDownTarget()); + Assert.assertEquals("iceberg_catalog.db.external_iceberg_tbl", olapTable.getExternalCoolDownTarget()); + Assert.assertEquals("START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE", olapTable.getExternalCoolDownSchedule()); + Assert.assertEquals((Long) 3600L, olapTable.getExternalCoolDownWaitSecond()); + String sql = "ALTER TABLE test_external_cooldown SET(\"external_cooldown_target\" = \"iceberg_catalog.db.tbl\");"; + AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(sql, ctx); + GlobalStateMgr.getCurrentState().getLocalMetastore().alterTable(ctx, alterTableStmt); + Assert.assertEquals("iceberg_catalog.db.tbl", olapTable.getExternalCoolDownTarget()); + + String sql2 = "ALTER TABLE test_external_cooldown " + + "SET(\"external_cooldown_schedule\" = \"START 00:00 END 07:59 EVERY INTERVAL 2 MINUTE\");"; + AlterTableStmt alterTableStmt2 = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(sql2, ctx); + GlobalStateMgr.getCurrentState().getLocalMetastore().alterTable(ctx, alterTableStmt2); + Assert.assertEquals("START 00:00 END 07:59 EVERY INTERVAL 2 MINUTE", olapTable.getExternalCoolDownSchedule()); + + String sql3 = "ALTER TABLE test_external_cooldown SET(\"external_cooldown_wait_second\" = \"7200\");"; + AlterTableStmt alterTableStmt3 = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(sql3, ctx); + GlobalStateMgr.getCurrentState().getLocalMetastore().alterTable(ctx, alterTableStmt3); + Assert.assertEquals((Long) 7200L, olapTable.getExternalCoolDownWaitSecond()); + + String sql4 = "ALTER TABLE test_external_cooldown SET(\"external_cooldown_wait_second\" = \"abc\");"; + Assert.assertThrows(AnalysisException.class, () -> + UtFrameUtils.parseStmtWithNewParser(sql4, ctx)); + + String sql5 = "ALTER TABLE test_external_cooldown SET(\"external_cooldown_schedule\" = \"abc\");"; + Assert.assertThrows(AnalysisException.class, () -> + UtFrameUtils.parseStmtWithNewParser(sql5, ctx)); + + String sql6 = "ALTER TABLE test_external_cooldown SET(\"external_cooldown_target\" = \"abc\");"; + Assert.assertThrows(AnalysisException.class, () -> + UtFrameUtils.parseStmtWithNewParser(sql6, ctx)); + + String sql7 = "ALTER TABLE test_external_cooldown SET(\"external_cooldown_target\" = \"a.b.c\");"; + Assert.assertThrows(AnalysisException.class, () -> + UtFrameUtils.parseStmtWithNewParser(sql7, ctx)); + + String sql8 = "ALTER TABLE test_external_cooldown SET(\"external_cooldown_target\" " + + "= \"default_catalog.test.test_external_cooldown\");"; + Assert.assertThrows(AnalysisException.class, () -> + UtFrameUtils.parseStmtWithNewParser(sql8, ctx)); + } + + @Test + public void testAlterTableExternalCoolDownSyncedTime() throws Exception { + Config.default_replication_num = 1; + starRocksAssert.useDatabase("test").withTable("CREATE TABLE test_alter_external_cool_down_synced_time (\n" + + "event_day DATE,\n" + + "site_id INT DEFAULT '10',\n" + + "city_code VARCHAR(100),\n" + + "user_name VARCHAR(32) DEFAULT '',\n" + + "pv BIGINT DEFAULT '0'\n" + + ")\n" + + "DUPLICATE KEY(event_day, site_id, city_code, user_name)\n" + + "PARTITION BY RANGE(event_day)(\n" + + "PARTITION p20200321 VALUES LESS THAN (\"2020-03-22\"),\n" + + "PARTITION p20200322 VALUES LESS THAN (\"2020-03-23\"),\n" + + "PARTITION p20200323 VALUES LESS THAN (\"2020-03-24\"),\n" + + "PARTITION p20200324 VALUES LESS THAN MAXVALUE\n" + + ")\n" + + "DISTRIBUTED BY HASH(event_day, site_id);"); + + ConnectContext ctx = starRocksAssert.getCtx(); + Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getTable( + "test", "test_alter_external_cool_down_synced_time"); + OlapTable olapTable = (OlapTable) table; + RangePartitionInfo rangePartitionInfo = (RangePartitionInfo) olapTable.getPartitionInfo(); + + String sql = "ALTER TABLE test_alter_external_cool_down_synced_time\n" + + "MODIFY PARTITION (*) SET(\"external_cooldown_synced_time\" = \"2020-03-25 01:00:00\",\n" + + " \"external_cooldown_consistency_check_time\" = \"2020-03-25 02:00:00\");"; + AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(sql, ctx); + GlobalStateMgr.getCurrentState().getLocalMetastore().alterTable(ctx, alterTableStmt); + + Assert.assertEquals((Long) (TimeUtils.parseDate("2020-03-25 01:00:00", PrimitiveType.DATETIME).getTime()), + rangePartitionInfo.getExternalCoolDownSyncedTimeMs(olapTable.getPartition("p20200321").getId())); + Assert.assertEquals((Long) (TimeUtils.parseDate("2020-03-25 02:00:00", PrimitiveType.DATETIME).getTime()), + rangePartitionInfo.getExternalCoolDownConsistencyCheckTimeMs(olapTable.getPartition("p20200321").getId())); + + Assert.assertEquals((Long) (TimeUtils.parseDate("2020-03-25 01:00:00", PrimitiveType.DATETIME).getTime()), + rangePartitionInfo.getExternalCoolDownSyncedTimeMs(olapTable.getPartition("p20200322").getId())); + Assert.assertEquals((Long) (TimeUtils.parseDate("2020-03-25 02:00:00", PrimitiveType.DATETIME).getTime()), + rangePartitionInfo.getExternalCoolDownConsistencyCheckTimeMs(olapTable.getPartition("p20200322").getId())); + + Assert.assertEquals((Long) (TimeUtils.parseDate("2020-03-25 01:00:00", PrimitiveType.DATETIME).getTime()), + rangePartitionInfo.getExternalCoolDownSyncedTimeMs(olapTable.getPartition("p20200323").getId())); + Assert.assertEquals((Long) (TimeUtils.parseDate("2020-03-25 02:00:00", PrimitiveType.DATETIME).getTime()), + rangePartitionInfo.getExternalCoolDownConsistencyCheckTimeMs(olapTable.getPartition("p20200323").getId())); + + Assert.assertEquals((Long) (TimeUtils.parseDate("2020-03-25 01:00:00", PrimitiveType.DATETIME).getTime()), + rangePartitionInfo.getExternalCoolDownSyncedTimeMs(olapTable.getPartition("p20200324").getId())); + Assert.assertEquals((Long) (TimeUtils.parseDate("2020-03-25 02:00:00", PrimitiveType.DATETIME).getTime()), + rangePartitionInfo.getExternalCoolDownConsistencyCheckTimeMs(olapTable.getPartition("p20200324").getId())); + + String sql1 = "ALTER TABLE test_alter_external_cool_down_synced_time\n" + + "MODIFY PARTITION (p20200321) SET(\"external_cooldown_synced_time\" = \"\",\n" + + " \"external_cooldown_consistency_check_time\" = \"\");"; + AlterTableStmt alterTableStmt1 = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(sql1, ctx); + GlobalStateMgr.getCurrentState().getLocalMetastore().alterTable(ctx, alterTableStmt1); + long partitionId = olapTable.getPartition("p20200321").getId(); + Assert.assertEquals((Long) 0L, rangePartitionInfo.getExternalCoolDownSyncedTimeMs(partitionId)); + Assert.assertEquals((Long) 0L, rangePartitionInfo.getExternalCoolDownConsistencyCheckTimeMs(partitionId)); + } } diff --git a/fe/fe-core/src/test/java/com/starrocks/analysis/CancelExternalCooldownStmtTest.java b/fe/fe-core/src/test/java/com/starrocks/analysis/CancelExternalCooldownStmtTest.java new file mode 100644 index 0000000000000..d7d7e588dd67b --- /dev/null +++ b/fe/fe-core/src/test/java/com/starrocks/analysis/CancelExternalCooldownStmtTest.java @@ -0,0 +1,104 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed 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 +// +// https://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 com.starrocks.analysis; + +import com.starrocks.common.util.UUIDUtil; +import com.starrocks.connector.iceberg.MockIcebergMetadata; +import com.starrocks.pseudocluster.PseudoCluster; +import com.starrocks.qe.ConnectContext; +import com.starrocks.sql.analyzer.ExternalCooldownAnalyzer; +import com.starrocks.sql.ast.CancelExternalCooldownStmt; +import com.starrocks.sql.plan.ConnectorPlanTestBase; +import com.starrocks.statistic.StatisticsMetaManager; +import com.starrocks.utframe.StarRocksAssert; +import com.starrocks.utframe.UtFrameUtils; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + + +public class CancelExternalCooldownStmtTest { + + private static StarRocksAssert starRocksAssert; + + protected static PseudoCluster cluster; + @ClassRule + public static TemporaryFolder temp = new TemporaryFolder(); + + protected static long startSuiteTime = 0; + protected long startCaseTime = 0; + + protected static final String TEST_DB_NAME = "test"; + protected static final String TEST_TBL_NAME = "tbl1"; + + @BeforeClass + public static void beforeClass() throws Exception { + UtFrameUtils.createMinStarRocksCluster(); + ConnectContext connectContext = UtFrameUtils.createDefaultCtx(); + starRocksAssert = new StarRocksAssert(connectContext); + + // set default config for async mvs + UtFrameUtils.setDefaultConfigForAsyncMVTest(connectContext); + + if (!starRocksAssert.databaseExist("_statistics_")) { + StatisticsMetaManager m = new StatisticsMetaManager(); + m.createStatisticsTablesForTest(); + } + starRocksAssert.withDatabase(TEST_DB_NAME); + starRocksAssert.useDatabase(TEST_DB_NAME); + + ConnectorPlanTestBase.mockCatalog(connectContext, MockIcebergMetadata.MOCKED_ICEBERG_CATALOG_NAME); + + starRocksAssert.withDatabase(TEST_DB_NAME).useDatabase(TEST_DB_NAME) + .withTable("CREATE TABLE tbl1\n" + + "(\n" + + " id int,\n" + + " ts datetime,\n" + + " data string\n" + + ")\n" + + "DUPLICATE KEY(`id`, `ts`)\n" + + "PARTITION BY RANGE(`ts`)\n" + + "(\n" + + " PARTITION p20200101 VALUES [('2020-01-01 00:00:00'),('2020-01-02 00:00:00')),\n" + + " PARTITION p20200102 VALUES [('2020-01-02 00:00:00'),('2020-01-03 00:00:00')),\n" + + " PARTITION p20200103 VALUES [('2020-01-03 00:00:00'),('2020-01-04 00:00:00')),\n" + + " PARTITION p20200104 VALUES [('2020-01-04 00:00:00'),('2020-01-05 00:00:00')),\n" + + " PARTITION p20200105 VALUES [('2020-01-05 00:00:00'),('2020-01-06 00:00:00'))\n" + + ")\n" + + "DISTRIBUTED BY HASH(`id`) BUCKETS 1\n" + + "PROPERTIES(\n" + + "'external_cooldown_target'='iceberg0.partitioned_transforms_db.t0_day',\n" + + "'external_cooldown_schedule'='START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE',\n" + + "'external_cooldown_wait_second'='3600',\n" + + "'replication_num' = '1'\n" + + ");"); + } + + @Test + public void BasicCreateExternalCooldownTest() throws Exception { + ConnectContext ctx = starRocksAssert.getCtx(); + ctx.setExecutionId(UUIDUtil.toTUniqueId(UUIDUtil.genUUID())); + String createExternalCooldownSQL = "CANCEL COOLDOWN TABLE tbl1"; + CancelExternalCooldownStmt stmt = (CancelExternalCooldownStmt) UtFrameUtils.parseStmtWithNewParser( + createExternalCooldownSQL, ctx); + ExternalCooldownAnalyzer.analyze(stmt, ctx); + + Assert.assertEquals(TEST_DB_NAME, stmt.getTableName().getDb()); + Assert.assertEquals(TEST_TBL_NAME, stmt.getTableName().getTbl()); + Assert.assertEquals("CANCEL COOLDOWN TABLE test.tbl1", stmt.toSql()); + } +} diff --git a/fe/fe-core/src/test/java/com/starrocks/analysis/CreateExternalCooldownStmtTest.java b/fe/fe-core/src/test/java/com/starrocks/analysis/CreateExternalCooldownStmtTest.java new file mode 100644 index 0000000000000..d71e5fc5c0a86 --- /dev/null +++ b/fe/fe-core/src/test/java/com/starrocks/analysis/CreateExternalCooldownStmtTest.java @@ -0,0 +1,145 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed 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 +// +// https://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 com.starrocks.analysis; + +import com.starrocks.common.util.UUIDUtil; +import com.starrocks.connector.iceberg.MockIcebergMetadata; +import com.starrocks.pseudocluster.PseudoCluster; +import com.starrocks.qe.ConnectContext; +import com.starrocks.scheduler.Task; +import com.starrocks.scheduler.TaskBuilder; +import com.starrocks.sql.analyzer.ExternalCooldownAnalyzer; +import com.starrocks.sql.analyzer.SemanticException; +import com.starrocks.sql.ast.CreateExternalCooldownStmt; +import com.starrocks.sql.plan.ConnectorPlanTestBase; +import com.starrocks.statistic.StatisticsMetaManager; +import com.starrocks.utframe.StarRocksAssert; +import com.starrocks.utframe.UtFrameUtils; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + + +public class CreateExternalCooldownStmtTest { + + private static StarRocksAssert starRocksAssert; + + protected static PseudoCluster cluster; + @ClassRule + public static TemporaryFolder temp = new TemporaryFolder(); + + protected static long startSuiteTime = 0; + protected long startCaseTime = 0; + + protected static final String TEST_DB_NAME = "test"; + protected static final String TEST_TBL_NAME = "tbl1"; + + @BeforeClass + public static void beforeClass() throws Exception { + UtFrameUtils.createMinStarRocksCluster(); + ConnectContext connectContext = UtFrameUtils.createDefaultCtx(); + starRocksAssert = new StarRocksAssert(connectContext); + + // set default config for async mvs + UtFrameUtils.setDefaultConfigForAsyncMVTest(connectContext); + + if (!starRocksAssert.databaseExist("_statistics_")) { + StatisticsMetaManager m = new StatisticsMetaManager(); + m.createStatisticsTablesForTest(); + } + starRocksAssert.withDatabase(TEST_DB_NAME); + starRocksAssert.useDatabase(TEST_DB_NAME); + + ConnectorPlanTestBase.mockCatalog(connectContext, MockIcebergMetadata.MOCKED_ICEBERG_CATALOG_NAME); + + starRocksAssert.withDatabase(TEST_DB_NAME).useDatabase(TEST_DB_NAME) + .withTable("CREATE TABLE tbl1\n" + + "(\n" + + " id int,\n" + + " ts datetime,\n" + + " data string\n" + + ")\n" + + "DUPLICATE KEY(`id`, `ts`)\n" + + "PARTITION BY RANGE(`ts`)\n" + + "(\n" + + " PARTITION p20200101 VALUES [('2020-01-01 00:00:00'),('2020-01-02 00:00:00')),\n" + + " PARTITION p20200102 VALUES [('2020-01-02 00:00:00'),('2020-01-03 00:00:00')),\n" + + " PARTITION p20200103 VALUES [('2020-01-03 00:00:00'),('2020-01-04 00:00:00')),\n" + + " PARTITION p20200104 VALUES [('2020-01-04 00:00:00'),('2020-01-05 00:00:00')),\n" + + " PARTITION p20200105 VALUES [('2020-01-05 00:00:00'),('2020-01-06 00:00:00'))\n" + + ")\n" + + "DISTRIBUTED BY HASH(`id`) BUCKETS 1\n" + + "PROPERTIES(\n" + + "'external_cooldown_target'='iceberg0.partitioned_transforms_db.t0_day',\n" + + "'external_cooldown_schedule'='START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE',\n" + + "'external_cooldown_wait_second'='3600',\n" + + "'replication_num' = '1'\n" + + ");"); + } + + @Test + public void testBasicCreateExternalCooldown() throws Exception { + ConnectContext ctx = starRocksAssert.getCtx(); + ctx.setExecutionId(UUIDUtil.toTUniqueId(UUIDUtil.genUUID())); + String createExternalCooldownSQL = "COOLDOWN TABLE tbl1"; + CreateExternalCooldownStmt stmt = (CreateExternalCooldownStmt) UtFrameUtils.parseStmtWithNewParser( + createExternalCooldownSQL, ctx); + ExternalCooldownAnalyzer.analyze(stmt, ctx); + + Assert.assertEquals(TEST_DB_NAME, stmt.getTableName().getDb()); + Assert.assertEquals(TEST_TBL_NAME, stmt.getTableName().getTbl()); + Assert.assertNull(stmt.getPartitionRangeDesc()); + Assert.assertFalse(stmt.isForce()); + Assert.assertEquals("COOLDOWN TABLE `test`.`tbl1`", stmt.toSql()); + + String createSQL2 = "COOLDOWN TABLE tbl1 PARTITION START ('2020-01-01 00:00:00') END ('2020-01-13 00:00:00')"; + CreateExternalCooldownStmt stmt2 = (CreateExternalCooldownStmt) UtFrameUtils.parseStmtWithNewParser( + createSQL2, ctx); + ExternalCooldownAnalyzer.analyze(stmt2, ctx); + Assert.assertEquals(TEST_DB_NAME, stmt2.getTableName().getDb()); + Assert.assertEquals(TEST_TBL_NAME, stmt2.getTableName().getTbl()); + Assert.assertNotNull(stmt2.getPartitionRangeDesc()); + Assert.assertFalse(stmt2.isForce()); + + String createSQL3 = "COOLDOWN TABLE tbl1 PARTITION START ('2020-01-01 00:00:00') END ('2020-01-13 00:00:00') FORCE"; + CreateExternalCooldownStmt stmt3 = (CreateExternalCooldownStmt) UtFrameUtils.parseStmtWithNewParser( + createSQL3, ctx); + ExternalCooldownAnalyzer.analyze(stmt3, ctx); + Assert.assertEquals(TEST_DB_NAME, stmt3.getTableName().getDb()); + Assert.assertEquals(TEST_TBL_NAME, stmt3.getTableName().getTbl()); + Assert.assertNotNull(stmt3.getPartitionRangeDesc()); + Assert.assertTrue(stmt3.isForce()); + String s = "COOLDOWN TABLE `test`.`tbl1` PARTITION START ('2020-01-01 00:00:00') END ('2020-01-13 00:00:00') FORCE"; + Assert.assertEquals(s, stmt3.toSql()); + } + + @Test + public void testCreateExternalCooldownOnNonOlapTable() throws Exception { + ConnectContext ctx = starRocksAssert.getCtx(); + ctx.setExecutionId(UUIDUtil.toTUniqueId(UUIDUtil.genUUID())); + String createExternalCooldownSQL = "COOLDOWN TABLE iceberg0.partitioned_transforms_db.t0_day"; + CreateExternalCooldownStmt stmt = (CreateExternalCooldownStmt) UtFrameUtils.parseStmtWithNewParser( + createExternalCooldownSQL, ctx); + ExternalCooldownAnalyzer.analyze(stmt, ctx); + try { + TaskBuilder.buildExternalCooldownTask(stmt); + } catch (Exception e) { + Assert.assertTrue(e instanceof SemanticException); + Assert.assertTrue(e.getMessage().contains("only support cooldown for olap table")); + } + } +} diff --git a/fe/fe-core/src/test/java/com/starrocks/catalog/CreateTableWithExternalCooldownTest.java b/fe/fe-core/src/test/java/com/starrocks/catalog/CreateTableWithExternalCooldownTest.java new file mode 100644 index 0000000000000..fbcad4ebf20d9 --- /dev/null +++ b/fe/fe-core/src/test/java/com/starrocks/catalog/CreateTableWithExternalCooldownTest.java @@ -0,0 +1,144 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed 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 +// +// https://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. + +// This file is based on code available under the Apache license here: +// https://github.com/apache/incubator-doris/blob/master/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableTest.java + +// 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 com.starrocks.catalog; + +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.starrocks.common.Config; +import com.starrocks.common.FeConstants; +import com.starrocks.connector.ConnectorContext; +import com.starrocks.connector.ConnectorMgr; +import com.starrocks.connector.iceberg.IcebergMetadata; +import com.starrocks.externalcooldown.ExternalCooldownConfig; +import com.starrocks.qe.ConnectContext; +import com.starrocks.server.GlobalStateMgr; +import com.starrocks.utframe.StarRocksAssert; +import com.starrocks.utframe.UtFrameUtils; +import mockit.Mock; +import mockit.MockUp; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.PartitionSpec; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Map; + +/** + * Test create table with location property and check whether + * the replica is placed on the right backends with matching location label. + */ +public class CreateTableWithExternalCooldownTest { + private static ConnectContext connectContext; + private static StarRocksAssert starRocksAssert; + + @Test + public void testCreateTableWithExternalCooldownConfig() throws Exception { + FeConstants.runningUnitTest = true; + Config.alter_scheduler_interval_millisecond = 100; + Config.dynamic_partition_enable = true; + Config.dynamic_partition_check_interval_seconds = 1; + Config.enable_strict_storage_medium_check = false; + Config.enable_experimental_rowstore = true; + UtFrameUtils.createMinStarRocksCluster(); + // create connect context + connectContext = UtFrameUtils.createDefaultCtx(); + starRocksAssert = new StarRocksAssert(connectContext); + starRocksAssert.withDatabase("test").useDatabase("test"); + + Config.default_replication_num = 1; + + starRocksAssert.withDatabase("test"); + starRocksAssert.useDatabase("test"); + + new MockUp() { + @Mock + public Table getTable(String dbName, String tblName) { + return new IcebergTable(1, "iceberg_tbl", "iceberg_catalog", + "iceberg_catalog", "iceberg_db", + "table1", "", Lists.newArrayList(), new BaseTable(null, ""), Maps.newHashMap()); + } + }; + new MockUp() { + @Mock + public String getTableIdentifier() { + return "iceberg_catalog.iceberg_db.iceberg_tbl"; + } + }; + new MockUp() { + @Mock + public PartitionSpec spec() { + return PartitionSpec.unpartitioned(); + } + }; + + ConnectorMgr connectorMgr = GlobalStateMgr.getCurrentState().getConnectorMgr(); + Map properties = Maps.newHashMap(); + + properties.put("type", "iceberg"); + properties.put("iceberg.catalog.type", "hive"); + properties.put("hive.metastore.uris", "thrift://127.0.0.1:9083"); + connectorMgr.createConnector(new ConnectorContext("iceberg_catalog", "iceberg", properties), false); + + starRocksAssert.withTable("CREATE TABLE test.tbl1\n" + + "(\n" + + " k1 date,\n" + + " k2 int,\n" + + " v1 int sum\n" + + ")\n" + + "PARTITION BY RANGE(k1)\n" + + "(\n" + + " PARTITION p1 values [('2024-03-01 00:00:00'),('2024-03-02 00:00:00')),\n" + + " PARTITION p2 values [('2024-03-02 00:00:00'),('2024-03-03 00:00:00')),\n" + + " PARTITION p3 values [('2024-03-03 00:00:00'),('2024-03-04 00:00:00')),\n" + + " PARTITION p4 values [('2024-03-04 00:00:00'),('2024-03-05 00:00:00')),\n" + + " PARTITION p5 values [('2024-03-05 00:00:00'),('2024-03-06 00:00:00'))\n" + + ")\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 1\n" + + "PROPERTIES(" + + "'replication_num' = '1',\n" + + "'external_cooldown_target' = 'iceberg_catalog.iceberg_db.iceberg_tbl',\n" + + "'external_cooldown_schedule' = 'START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE',\n" + + "'external_cooldown_wait_second' = '60'\n" + + ");"); + Table table = starRocksAssert.getTable("test", "tbl1"); + Assert.assertTrue(table instanceof OlapTable); + OlapTable olapTable = (OlapTable) table; + ExternalCooldownConfig config = olapTable.getCurExternalCoolDownConfig(); + Assert.assertNotNull(config); + Assert.assertEquals("iceberg_catalog.iceberg_db.iceberg_tbl", config.getTarget()); + Assert.assertEquals("START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE", config.getSchedule()); + Assert.assertEquals((Long) 60L, config.getWaitSecond()); + } +} \ No newline at end of file diff --git a/fe/fe-core/src/test/java/com/starrocks/common/proc/PartitionsProcDirTest.java b/fe/fe-core/src/test/java/com/starrocks/common/proc/PartitionsProcDirTest.java index 6a4480464e21f..63c58d9ba0b91 100644 --- a/fe/fe-core/src/test/java/com/starrocks/common/proc/PartitionsProcDirTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/common/proc/PartitionsProcDirTest.java @@ -18,10 +18,12 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.starrocks.catalog.Column; +import com.starrocks.catalog.DataProperty; import com.starrocks.catalog.Database; import com.starrocks.catalog.ListPartitionInfo; import com.starrocks.catalog.MaterializedIndex; import com.starrocks.catalog.MaterializedIndex.IndexState; +import com.starrocks.catalog.OlapTable; import com.starrocks.catalog.Partition; import com.starrocks.catalog.PartitionInfo; import com.starrocks.catalog.PartitionType; @@ -42,6 +44,7 @@ public class PartitionsProcDirTest { private Database db; private LakeTable cloudNativTable; + private OlapTable olapTable; @Before public void setUp() throws DdlException, AnalysisException { @@ -59,6 +62,15 @@ public void setUp() throws DdlException, AnalysisException { cloudNativTable.addPartition(new Partition(partitionId, "p1", index, new RandomDistributionInfo(10))); db.registerTableUnlocked(cloudNativTable); + + MaterializedIndex index1 = new MaterializedIndex(1001L, IndexState.NORMAL); + Partition partition = new Partition(1028L, "p2", index1, new RandomDistributionInfo(10)); + List col1 = Lists.newArrayList(new Column("province", Type.VARCHAR)); + PartitionInfo listPartition1 = new ListPartitionInfo(PartitionType.LIST, col1); + listPartition1.addPartition(partition.getId(), DataProperty.getInferredDefaultDataProperty(), (short) 1, true); + olapTable = new OlapTable(1026L, "olap_table", col, null, listPartition1, null); + olapTable.addPartition(partition); + db.registerTableUnlocked(olapTable); } @Test @@ -74,4 +86,17 @@ public void testFetchResult() throws AnalysisException { Assert.assertEquals("NORMAL", list1.get(5)); Assert.assertEquals("province", list1.get(6)); } + + @Test + public void testFetchResult4OlapTable() throws AnalysisException { + BaseProcResult result = (BaseProcResult) new PartitionsProcDir(db, olapTable, false).fetchResult(); + List> rows = result.getRows(); + List list1 = rows.get(0); + Assert.assertEquals("1028", list1.get(0)); + Assert.assertEquals("p2", list1.get(1)); + Assert.assertEquals("1", list1.get(2)); + Assert.assertEquals("\\N", list1.get(20)); + Assert.assertEquals("\\N", list1.get(21)); + Assert.assertEquals("0", list1.get(22)); + } } diff --git a/fe/fe-core/src/test/java/com/starrocks/connector/iceberg/MockIcebergMetadata.java b/fe/fe-core/src/test/java/com/starrocks/connector/iceberg/MockIcebergMetadata.java index f864c9ec732c0..bfe9fb1923b72 100644 --- a/fe/fe-core/src/test/java/com/starrocks/connector/iceberg/MockIcebergMetadata.java +++ b/fe/fe-core/src/test/java/com/starrocks/connector/iceberg/MockIcebergMetadata.java @@ -87,6 +87,10 @@ public class MockIcebergMetadata implements ConnectorMetadata { public static final String MOCKED_PARTITIONED_HOUR_TZ_TABLE_NAME = "t0_hour_tz"; // partition table with partition evolutions public static final String MOCKED_PARTITIONED_EVOLUTION_DATE_MONTH_IDENTITY_TABLE_NAME = "t0_date_month_identity_evolution"; + // partition table with date transforms + public static final String MOCKED_PARTITIONED_DATE_YEAR_TABLE_NAME = "t0_year_dt"; + public static final String MOCKED_PARTITIONED_DATE_MONTH_TABLE_NAME = "t0_month_dt"; + public static final String MOCKED_PARTITIONED_DATE_DAY_TABLE_NAME = "t0_day_dt"; private static final List PARTITION_TABLE_NAMES = ImmutableList.of(MOCKED_PARTITIONED_TABLE_NAME1, MOCKED_STRING_PARTITIONED_TABLE_NAME1, MOCKED_STRING_PARTITIONED_TABLE_NAME2, @@ -98,7 +102,10 @@ public class MockIcebergMetadata implements ConnectorMetadata { MOCKED_PARTITIONED_BUCKET_TABLE_NAME, MOCKED_PARTITIONED_YEAR_TZ_TABLE_NAME, MOCKED_PARTITIONED_MONTH_TZ_TABLE_NAME, MOCKED_PARTITIONED_DAY_TZ_TABLE_NAME, MOCKED_PARTITIONED_HOUR_TZ_TABLE_NAME, - MOCKED_PARTITIONED_EVOLUTION_DATE_MONTH_IDENTITY_TABLE_NAME); + MOCKED_PARTITIONED_EVOLUTION_DATE_MONTH_IDENTITY_TABLE_NAME, + MOCKED_PARTITIONED_DATE_YEAR_TABLE_NAME, + MOCKED_PARTITIONED_DATE_MONTH_TABLE_NAME, + MOCKED_PARTITIONED_DATE_DAY_TABLE_NAME); private static final List PARTITION_NAMES_0 = Lists.newArrayList("date=2020-01-01", "date=2020-01-02", @@ -235,6 +242,10 @@ private static Schema getIcebergPartitionTransformSchema(String tblName) { return new Schema(required(3, "id", Types.IntegerType.get()), required(4, "data", Types.StringType.get()), required(5, "ts", Types.TimestampType.withZone())); + } else if (tblName.endsWith("dt")) { + return new Schema(required(3, "id", Types.IntegerType.get()), + required(4, "data", Types.StringType.get()), + required(5, "date", Types.DateType.get())); } else { return new Schema(required(3, "id", Types.IntegerType.get()), required(4, "data", Types.StringType.get()), @@ -352,6 +363,30 @@ private static TestTables.TestTable getPartitionTransformTable(String tblName, S table.ops().commit(table.ops().current(), evolutionMetaData); return table; } + case MOCKED_PARTITIONED_DATE_YEAR_TABLE_NAME: { + PartitionSpec spec = + PartitionSpec.builderFor(schema).year("date").build(); + return TestTables.create( + new File(getStarRocksHome() + "/" + MOCKED_PARTITIONED_TRANSFORMS_DB_NAME + "/" + + MOCKED_PARTITIONED_DATE_YEAR_TABLE_NAME), MOCKED_PARTITIONED_DATE_YEAR_TABLE_NAME, + schema, spec, 1); + } + case MOCKED_PARTITIONED_DATE_MONTH_TABLE_NAME: { + PartitionSpec spec = + PartitionSpec.builderFor(schema).month("date").build(); + return TestTables.create( + new File(getStarRocksHome() + "/" + MOCKED_PARTITIONED_TRANSFORMS_DB_NAME + "/" + + MOCKED_PARTITIONED_DATE_MONTH_TABLE_NAME), MOCKED_PARTITIONED_DATE_MONTH_TABLE_NAME, + schema, spec, 1); + } + case MOCKED_PARTITIONED_DATE_DAY_TABLE_NAME: { + PartitionSpec spec = + PartitionSpec.builderFor(schema).day("date").build(); + return TestTables.create( + new File(getStarRocksHome() + "/" + MOCKED_PARTITIONED_TRANSFORMS_DB_NAME + "/" + + MOCKED_PARTITIONED_DATE_DAY_TABLE_NAME), MOCKED_PARTITIONED_DATE_DAY_TABLE_NAME, + schema, spec, 1); + } } return null; } @@ -380,6 +415,15 @@ public static List getTransformTablePartitionNames(String tblName) { case MOCKED_PARTITIONED_EVOLUTION_DATE_MONTH_IDENTITY_TABLE_NAME: return Lists.newArrayList("ts=2024-01-01", "ts_month=2024-01", "ts=2024-02", "ts=2024-03"); + case MOCKED_PARTITIONED_DATE_YEAR_TABLE_NAME: + return Lists.newArrayList("date_day=2022-01-01", "date_day=2022-01-02", + "date_day=2022-01-03", "date_day=2022-01-04", "date_day=2022-01-05"); + case MOCKED_PARTITIONED_DATE_MONTH_TABLE_NAME: + return Lists.newArrayList("date_month=2022-01", "date_month=2022-02", + "date_month=2022-03", "date_month=2022-04", "date_month=2022-05"); + case MOCKED_PARTITIONED_DATE_DAY_TABLE_NAME: + return Lists.newArrayList("date_day=2022-01-01", "date_day=2022-01-02", + "date_day=2022-01-03", "date_day=2022-01-04", "date_day=2022-01-05"); } return null; } diff --git a/fe/fe-core/src/test/java/com/starrocks/externalcooldown/ExternalCooldownConfigTest.java b/fe/fe-core/src/test/java/com/starrocks/externalcooldown/ExternalCooldownConfigTest.java new file mode 100644 index 0000000000000..6c74cd17ef2c3 --- /dev/null +++ b/fe/fe-core/src/test/java/com/starrocks/externalcooldown/ExternalCooldownConfigTest.java @@ -0,0 +1,134 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed 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 +// +// https://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 com.starrocks.externalcooldown; + +import com.starrocks.common.util.PropertyAnalyzer; +import com.starrocks.connector.iceberg.MockIcebergMetadata; +import com.starrocks.sql.plan.ConnectorPlanTestBase; +import com.starrocks.utframe.UtFrameUtils; +import org.junit.Assert; +import org.junit.Test; + +import java.util.Map; + + +public class ExternalCooldownConfigTest { + + @Test + public void testPartitionStartEnd() { + ExternalCooldownConfig config = new ExternalCooldownConfig( + "iceberg.db1.tbl1", "START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE", 3600L); + + ExternalCooldownSchedule schedule = ExternalCooldownSchedule.fromString(config.getSchedule()); + Assert.assertNotNull(schedule); + Assert.assertEquals("01:00", schedule.getStart()); + Assert.assertEquals("07:59", schedule.getEnd()); + Assert.assertEquals(1L, schedule.getInterval()); + Assert.assertEquals("MINUTE", schedule.getUnit()); + Assert.assertEquals(60L, schedule.getIntervalSeconds()); + + Assert.assertTrue(config.isReadyForAutoCooldown()); + + Map properties = config.getValidProperties(); + Assert.assertNotNull(properties); + Assert.assertEquals(3, properties.size()); + Assert.assertTrue(properties.containsKey(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_TARGET)); + Assert.assertEquals("iceberg.db1.tbl1", + properties.get(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_TARGET)); + Assert.assertTrue(properties.containsKey(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_SCHEDULE)); + Assert.assertEquals("START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE", + properties.get(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_SCHEDULE)); + Assert.assertTrue(properties.containsKey(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_WAIT_SECOND)); + Assert.assertEquals("3600", + properties.get(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_WAIT_SECOND)); + + String str = "{ target : iceberg.db1.tbl1,\n schedule : START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE,\n " + + "wait second : 3600 }"; + Assert.assertEquals(str, config.toString()); + } + + @Test + public void testReadyForAutoCooldown() { + ExternalCooldownConfig config = new ExternalCooldownConfig( + "iceberg.db1.tbl1", "START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE", 3600L); + Assert.assertTrue(config.isReadyForAutoCooldown()); + + config.setWaitSecond(null); + Assert.assertFalse(config.isReadyForAutoCooldown()); + Assert.assertEquals(2, config.getValidProperties().size()); + config.setWaitSecond(0L); + Assert.assertFalse(config.isReadyForAutoCooldown()); + Assert.assertEquals(2, config.getValidProperties().size()); + config.setWaitSecond(3600L); + Assert.assertEquals(3, config.getValidProperties().size()); + + config.setSchedule(null); + Assert.assertFalse(config.isReadyForAutoCooldown()); + Assert.assertEquals(2, config.getValidProperties().size()); + config.setSchedule(""); + Assert.assertFalse(config.isReadyForAutoCooldown()); + Assert.assertEquals(2, config.getValidProperties().size()); + config.setSchedule("START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE"); + Assert.assertEquals(3, config.getValidProperties().size()); + + config.setTarget(null); + Assert.assertFalse(config.isReadyForAutoCooldown()); + Assert.assertEquals(2, config.getValidProperties().size()); + config.setTarget(""); + Assert.assertFalse(config.isReadyForAutoCooldown()); + Assert.assertEquals(2, config.getValidProperties().size()); + config.setTarget("iceberg.db1.tbl1"); + Assert.assertEquals(3, config.getValidProperties().size()); + } + + @Test + public void testConstructor() { + ExternalCooldownConfig config = new ExternalCooldownConfig( + "iceberg.db1.tbl1", "START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE", 3600L); + Assert.assertTrue(config.isReadyForAutoCooldown()); + Assert.assertTrue(config.equals(config)); + + ExternalCooldownConfig config2 = new ExternalCooldownConfig(config); + Assert.assertTrue(config2.isReadyForAutoCooldown()); + + ExternalCooldownConfig config3 = new ExternalCooldownConfig(); + Assert.assertNull(config3.getWaitSecond()); + Assert.assertNull(config3.getSchedule()); + Assert.assertNull(config3.getTarget()); + } + + @Test + public void testMergeUpdateFromProperties() throws Exception { + UtFrameUtils.createMinStarRocksCluster(); + ConnectorPlanTestBase.mockCatalog(UtFrameUtils.createDefaultCtx(), MockIcebergMetadata.MOCKED_ICEBERG_CATALOG_NAME); + + ExternalCooldownConfig config = new ExternalCooldownConfig( + "iceberg.db1.tbl1", "START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE", 3600L); + + ExternalCooldownConfig config2 = new ExternalCooldownConfig( + "iceberg0.partitioned_transforms_db.t0_day", + "START 01:00 END 07:59 EVERY INTERVAL 5 MINUTE", 1800L); + config.mergeUpdateFromProperties(config2.getValidProperties()); + Assert.assertEquals("iceberg0.partitioned_transforms_db.t0_day", config.getTarget()); + Assert.assertEquals("START 01:00 END 07:59 EVERY INTERVAL 5 MINUTE", config.getSchedule()); + Assert.assertEquals((Long) 1800L, config.getWaitSecond()); + + ExternalCooldownConfig config3 = new ExternalCooldownConfig(); + config.mergeUpdateFromProperties(config3.getValidProperties()); + Assert.assertEquals("iceberg0.partitioned_transforms_db.t0_day", config.getTarget()); + Assert.assertEquals("START 01:00 END 07:59 EVERY INTERVAL 5 MINUTE", config.getSchedule()); + Assert.assertEquals((Long) 1800L, config.getWaitSecond()); + } +} \ No newline at end of file diff --git a/fe/fe-core/src/test/java/com/starrocks/externalcooldown/ExternalCooldownPartitionSelectorTest.java b/fe/fe-core/src/test/java/com/starrocks/externalcooldown/ExternalCooldownPartitionSelectorTest.java new file mode 100644 index 0000000000000..e15a40445e68e --- /dev/null +++ b/fe/fe-core/src/test/java/com/starrocks/externalcooldown/ExternalCooldownPartitionSelectorTest.java @@ -0,0 +1,603 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed 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 +// +// https://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 com.starrocks.externalcooldown; + +import com.starrocks.catalog.IcebergTable; +import com.starrocks.catalog.OlapTable; +import com.starrocks.catalog.Partition; +import com.starrocks.catalog.PartitionInfo; +import com.starrocks.catalog.Table; +import com.starrocks.common.Config; +import com.starrocks.common.FeConstants; +import com.starrocks.connector.iceberg.MockIcebergMetadata; +import com.starrocks.qe.ConnectContext; +import com.starrocks.server.GlobalStateMgr; +import com.starrocks.sql.ast.AlterTableStmt; +import com.starrocks.sql.plan.ConnectorPlanTestBase; +import com.starrocks.utframe.StarRocksAssert; +import com.starrocks.utframe.UtFrameUtils; +import mockit.Mock; +import mockit.MockUp; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + + +public class ExternalCooldownPartitionSelectorTest { + private static ConnectContext connectContext; + private static StarRocksAssert starRocksAssert; + + @BeforeClass + public static void beforeClass() throws Exception { + FeConstants.runningUnitTest = true; + Config.alter_scheduler_interval_millisecond = 100; + Config.dynamic_partition_enable = true; + Config.dynamic_partition_check_interval_seconds = 1; + Config.enable_strict_storage_medium_check = false; + Config.enable_experimental_rowstore = true; + UtFrameUtils.createMinStarRocksCluster(); + // create connect context + connectContext = UtFrameUtils.createDefaultCtx(); + starRocksAssert = new StarRocksAssert(connectContext); + starRocksAssert.withDatabase("test").useDatabase("test"); + + Config.default_replication_num = 1; + + starRocksAssert.withDatabase("test"); + starRocksAssert.useDatabase("test"); + + ConnectorPlanTestBase.mockCatalog(connectContext, MockIcebergMetadata.MOCKED_ICEBERG_CATALOG_NAME); + + starRocksAssert.withTable("CREATE TABLE test.tbl1\n" + + "(\n" + + " k1 date,\n" + + " k2 int,\n" + + " v1 int sum\n" + + ")\n" + + "PARTITION BY RANGE(k1)\n" + + "(\n" + + " PARTITION p1 values [('2024-03-01 00:00:00'),('2024-03-02 00:00:00')),\n" + + " PARTITION p2 values [('2024-03-02 00:00:00'),('2024-03-03 00:00:00')),\n" + + " PARTITION p3 values [('2024-03-03 00:00:00'),('2024-03-04 00:00:00')),\n" + + " PARTITION p4 values [('2024-03-04 00:00:00'),('2024-03-05 00:00:00')),\n" + + " PARTITION p5 values [('2024-03-05 00:00:00'),('2024-03-06 00:00:00'))\n" + + ")\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 1\n" + + "PROPERTIES(" + + "'replication_num' = '1',\n" + + "'external_cooldown_target' = 'iceberg0.partitioned_transforms_db.t0_day_dt',\n" + + "'external_cooldown_schedule' = 'START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE'\n," + + "'external_cooldown_wait_second' = '1'\n" + + ");"); + starRocksAssert.withTable("CREATE TABLE test.tbl2\n" + + "(\n" + + " k1 date,\n" + + " k2 int,\n" + + " v1 int sum\n" + + ")\n" + + "PARTITION BY RANGE(k1)\n" + + "(\n" + + " PARTITION p1 values [('2024-03-01 00:00:00'),('2024-03-02 00:00:00')),\n" + + " PARTITION p2 values [('2024-03-02 00:00:00'),('2024-03-03 00:00:00')),\n" + + " PARTITION p3 values [('2024-03-03 00:00:00'),('2024-03-04 00:00:00')),\n" + + " PARTITION p4 values [('2024-03-04 00:00:00'),('2024-03-05 00:00:00')),\n" + + " PARTITION p5 values [('2024-03-05 00:00:00'),('2024-03-06 00:00:00'))\n" + + ")\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 1\n" + + "PROPERTIES(" + + "'replication_num' = '1',\n" + + "'external_cooldown_target' = 'iceberg0.partitioned_transforms_db.t0_day_dt',\n" + + "'external_cooldown_schedule' = 'START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE'\n," + + "'external_cooldown_wait_second' = '1'\n" + + ");"); + starRocksAssert.withTable("CREATE TABLE test.tbl3\n" + + "(\n" + + " k1 date,\n" + + " k2 int,\n" + + " v1 int sum\n" + + ")\n" + + "PARTITION BY RANGE(k1)\n" + + "(\n" + + " PARTITION p1 values [('2024-03-01 00:00:00'),('2024-03-02 00:00:00')),\n" + + " PARTITION p2 values [('2024-03-02 00:00:00'),('2024-03-03 00:00:00')),\n" + + " PARTITION p3 values [('2024-03-03 00:00:00'),('2024-03-04 00:00:00')),\n" + + " PARTITION p4 values [('2024-03-04 00:00:00'),('2024-03-05 00:00:00')),\n" + + " PARTITION p5 values [('2024-03-05 00:00:00'),('2024-03-06 00:00:00'))\n" + + ")\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 1\n" + + "PROPERTIES(" + + "'replication_num' = '1',\n" + + "'external_cooldown_target' = 'iceberg0.partitioned_transforms_db.t0_day_dt',\n" + + "'external_cooldown_schedule' = 'START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE'\n," + + "'external_cooldown_wait_second' = '1'\n" + + ");"); + starRocksAssert.withTable("CREATE TABLE test.tbl4\n" + + "(\n" + + " k1 date,\n" + + " k2 int,\n" + + " v1 int sum\n" + + ")\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 1\n" + + "PROPERTIES(" + + "'replication_num' = '1',\n" + + "'external_cooldown_target' = 'iceberg0.unpartitioned_db.t0',\n" + + "'external_cooldown_schedule' = 'START 01:00 END 07:59 EVERY INTERVAL 10 SECOND'\n," + + "'external_cooldown_wait_second' = '1'\n" + + ");"); + + starRocksAssert.withTable("CREATE TABLE test.tbl5 (\n" + + " id BIGINT,\n" + + " age SMALLINT,\n" + + " dt VARCHAR(10),\n" + + " province VARCHAR(64) not null\n" + + ")\n" + + "ENGINE=olap\n" + + "DUPLICATE KEY(id)\n" + + "PARTITION BY LIST (province) (\n" + + " PARTITION p1 VALUES IN ('beijing','chongqing') ,\n" + + " PARTITION p2 VALUES IN ('shenzhen','hainan') ,\n" + + " PARTITION p3 VALUES IN ('guangdong') \n" + + ")\n" + + "DISTRIBUTED BY HASH(id) BUCKETS 1\n" + + "PROPERTIES (\n" + + " 'replication_num' = '1',\n" + + " 'external_cooldown_target' = 'iceberg0.partitioned_transforms_db.t0_day_dt',\n" + + " 'external_cooldown_schedule' = 'START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE'\n," + + " 'external_cooldown_wait_second' = '1'\n" + + ")"); + + starRocksAssert.withTable("CREATE TABLE test.tbl6 (\n" + + " id BIGINT,\n" + + " age SMALLINT,\n" + + " dt VARCHAR(10),\n" + + " province VARCHAR(64) not null\n" + + ")\n" + + "ENGINE=olap\n" + + "DUPLICATE KEY(id)\n" + + "PARTITION BY LIST (province) (\n" + + " PARTITION p1 VALUES IN ('beijing','chongqing') ,\n" + + " PARTITION p2 VALUES IN ('shenzhen','hainan') ,\n" + + " PARTITION p3 VALUES IN ('guangdong') \n" + + ")\n" + + "DISTRIBUTED BY HASH(id) BUCKETS 1\n" + + "PROPERTIES (\n" + + " 'replication_num' = '1',\n" + + " 'external_cooldown_target' = 'iceberg0.partitioned_transforms_db.t0_day_dt',\n" + + " 'external_cooldown_schedule' = 'START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE'\n," + + " 'external_cooldown_wait_second' = '1'\n" + + ")"); + + starRocksAssert.withTable("CREATE TABLE test.tbl7 (\n" + + " id BIGINT,\n" + + " age SMALLINT,\n" + + " dt VARCHAR(10),\n" + + " province VARCHAR(64) not null\n" + + ")\n" + + "ENGINE=olap\n" + + "DUPLICATE KEY(id)\n" + + "PARTITION BY LIST (province) (\n" + + " PARTITION p1 VALUES IN ('beijing','chongqing') ,\n" + + " PARTITION p2 VALUES IN ('shenzhen','hainan') ,\n" + + " PARTITION p3 VALUES IN ('guangdong') \n" + + ")\n" + + "DISTRIBUTED BY HASH(id) BUCKETS 1\n" + + "PROPERTIES (\n" + + " 'replication_num' = '1',\n" + + " 'external_cooldown_target' = 'iceberg0.partitioned_transforms_db.t0_day_dt',\n" + + " 'external_cooldown_schedule' = 'START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE'\n," + + " 'external_cooldown_wait_second' = '1'\n" + + ")"); + + starRocksAssert.withTable("CREATE TABLE test.tbl8 (\n" + + " id BIGINT,\n" + + " age SMALLINT,\n" + + " dt VARCHAR(10),\n" + + " province VARCHAR(64) not null\n" + + ")\n" + + "ENGINE=olap\n" + + "DUPLICATE KEY(id)\n" + + "DISTRIBUTED BY HASH(id) BUCKETS 1\n" + + "PROPERTIES (\n" + + " 'replication_num' = '1',\n" + + " 'external_cooldown_target' = 'iceberg0.partitioned_transforms_db.t0_day_dt',\n" + + " 'external_cooldown_schedule' = 'START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE'\n," + + " 'external_cooldown_wait_second' = '1'\n" + + ")"); + + starRocksAssert.withTable("CREATE TABLE test.tbl9 (\n" + + " id BIGINT,\n" + + " age SMALLINT,\n" + + " dt VARCHAR(10),\n" + + " province VARCHAR(64) not null\n" + + ")\n" + + "ENGINE=olap\n" + + "DUPLICATE KEY(id)\n" + + "PARTITION BY LIST (province) (\n" + + " PARTITION p1 VALUES IN ('beijing','chongqing') ,\n" + + " PARTITION p2 VALUES IN ('shenzhen','hainan') ,\n" + + " PARTITION p3 VALUES IN ('shanghai','wuxi') ,\n" + + " PARTITION p4 VALUES IN ('wuhan') \n" + + ")\n" + + "DISTRIBUTED BY HASH(id) BUCKETS 1\n" + + "PROPERTIES (\n" + + " 'replication_num' = '1',\n" + + " 'external_cooldown_target' = 'iceberg0.partitioned_db.part_tbl1',\n" + + " 'external_cooldown_schedule' = 'START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE'\n," + + " 'external_cooldown_wait_second' = '1'\n" + + ")"); + + starRocksAssert.withTable("CREATE TABLE test.tbl10 (\n" + + " id BIGINT,\n" + + " age SMALLINT,\n" + + " dt VARCHAR(10),\n" + + " province VARCHAR(64) not null\n" + + ")\n" + + "ENGINE=olap\n" + + "DUPLICATE KEY(id)\n" + + "PARTITION BY LIST (province) (\n" + + " PARTITION p1 VALUES IN ('beijing','chongqing') ,\n" + + " PARTITION p2 VALUES IN ('shenzhen','hainan') ,\n" + + " PARTITION p3 VALUES IN ('shanghai','wuxi') ,\n" + + " PARTITION p4 VALUES IN ('wuhan') \n" + + ")\n" + + "DISTRIBUTED BY HASH(id) BUCKETS 1\n" + + "PROPERTIES (\n" + + " 'replication_num' = '1',\n" + + " 'external_cooldown_target' = 'iceberg0.partitioned_db.part_tbl1',\n" + + " 'external_cooldown_schedule' = 'START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE'\n," + + " 'external_cooldown_wait_second' = '1'\n" + + ")"); + starRocksAssert.withTable("CREATE TABLE test.tbl11\n" + + "(\n" + + " k1 date,\n" + + " k2 int,\n" + + " v1 int sum\n" + + ")\n" + + "PARTITION BY RANGE(k1)\n" + + "(\n" + + " PARTITION p1 values [('2024-03-01 00:00:00'),('2024-03-02 00:00:00')),\n" + + " PARTITION p2 values [('2024-03-02 00:00:00'),('2024-03-03 00:00:00')),\n" + + " PARTITION p3 values [('2024-03-03 00:00:00'),('2024-03-04 00:00:00')),\n" + + " PARTITION p4 values [('2024-03-04 00:00:00'),('2024-03-05 00:00:00')),\n" + + " PARTITION p5 values [('2024-03-05 00:00:00'),('2024-03-06 00:00:00'))\n" + + ")\n" + + "DISTRIBUTED BY HASH(k2) BUCKETS 1\n" + + "PROPERTIES(" + + "'replication_num' = '1',\n" + + "'external_cooldown_target' = 'iceberg0.partitioned_transforms_db.t0_day_dt',\n" + + "'external_cooldown_schedule' = 'START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE'\n," + + "'external_cooldown_wait_second' = '1'\n" + + ");"); + } + + @Test + public void testCooldownWaitSecond() throws Exception { + Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getTable("test", "tbl1"); + + long updateTime = System.currentTimeMillis() - 2000; + Partition p1 = table.getPartition("p1"); + p1.updateVisibleVersion(p1.getVisibleVersion() + 1, updateTime); + Partition p2 = table.getPartition("p2"); + p2.updateVisibleVersion(p2.getVisibleVersion() + 1, updateTime); + Partition p3 = table.getPartition("p3"); + p3.updateVisibleVersion(p3.getVisibleVersion() + 1, updateTime); + Partition p5 = table.getPartition("p5"); + p5.updateVisibleVersion(p5.getVisibleVersion() + 1, System.currentTimeMillis()); + + ConnectContext ctx = starRocksAssert.getCtx(); + String sql3 = "ALTER TABLE test.tbl1 SET(\"external_cooldown_wait_second\" = \"1\");"; + AlterTableStmt alterTableStmt3 = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(sql3, ctx); + GlobalStateMgr.getCurrentState().getLocalMetastore().alterTable(ctx, alterTableStmt3); + + ExternalCooldownPartitionSelector selector = new ExternalCooldownPartitionSelector((OlapTable) table); + Assert.assertTrue(selector.isTableSatisfied()); + + List partitions = selector.computeSatisfiedPartitions(-1); + Assert.assertEquals(3, partitions.size()); + Set satisfiedPartitionNames = partitions.stream().map(Partition::getName).collect(Collectors.toSet()); + Assert.assertTrue("p1 satisfied cooldown condition", satisfiedPartitionNames.contains("p1")); + Assert.assertTrue("p2 satisfied cooldown condition", satisfiedPartitionNames.contains("p2")); + Assert.assertTrue("p3 satisfied cooldown condition", satisfiedPartitionNames.contains("p3")); + Assert.assertFalse("p4 not satisfied cooldown condition", satisfiedPartitionNames.contains("p4")); + Assert.assertFalse("p5 not satisfied cooldown condition", satisfiedPartitionNames.contains("p5")); + } + + @Test + public void testPartitionStartEnd() throws Exception { + Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getTable("test", "tbl2"); + + long updateTime = System.currentTimeMillis() - 2000; + Partition p1 = table.getPartition("p1"); + p1.updateVisibleVersion(p1.getVisibleVersion() + 1, updateTime); + Partition p2 = table.getPartition("p2"); + p2.updateVisibleVersion(p2.getVisibleVersion() + 1, updateTime); + Partition p3 = table.getPartition("p3"); + p3.updateVisibleVersion(p3.getVisibleVersion() + 1, updateTime); + Partition p5 = table.getPartition("p5"); + p5.updateVisibleVersion(p5.getVisibleVersion() + 1, updateTime); + + ConnectContext ctx = starRocksAssert.getCtx(); + String sql3 = "ALTER TABLE test.tbl2 SET(\"external_cooldown_wait_second\" = \"1\");"; + AlterTableStmt alterTableStmt3 = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(sql3, ctx); + GlobalStateMgr.getCurrentState().getLocalMetastore().alterTable(ctx, alterTableStmt3); + + ExternalCooldownPartitionSelector selector = new ExternalCooldownPartitionSelector( + (OlapTable) table, "2024-03-02 00:00:00", "2024-03-05 00:00:00", false); + Assert.assertTrue(selector.isTableSatisfied()); + + List partitions = selector.computeSatisfiedPartitions(-1); + Assert.assertEquals(2, partitions.size()); + Set satisfiedPartitionNames = partitions.stream().map(Partition::getName).collect(Collectors.toSet()); + Assert.assertFalse("p1 not in partition range", satisfiedPartitionNames.contains("p1")); + Assert.assertTrue("p2 in partition range", satisfiedPartitionNames.contains("p2")); + Assert.assertTrue("p3 in partition range", satisfiedPartitionNames.contains("p3")); + Assert.assertFalse("p4 not in partition range", satisfiedPartitionNames.contains("p4")); + Assert.assertFalse("p5 not in partition range", satisfiedPartitionNames.contains("p5")); + } + + @Test + public void testForce() throws Exception { + Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getTable("test", "tbl3"); + + Partition p1 = table.getPartition("p1"); + p1.updateVisibleVersion(p1.getVisibleVersion() + 1); + Partition p2 = table.getPartition("p2"); + p2.updateVisibleVersion(p2.getVisibleVersion() + 1); + Partition p4 = table.getPartition("p4"); + p4.updateVisibleVersion(p4.getVisibleVersion() + 1); + Partition p5 = table.getPartition("p5"); + p5.updateVisibleVersion(p5.getVisibleVersion() + 1); + + ConnectContext ctx = starRocksAssert.getCtx(); + String sql3 = "ALTER TABLE test.tbl3 SET(\"external_cooldown_wait_second\" = \"7200\");"; + AlterTableStmt alterTableStmt3 = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(sql3, ctx); + GlobalStateMgr.getCurrentState().getLocalMetastore().alterTable(ctx, alterTableStmt3); + + ExternalCooldownPartitionSelector selector = new ExternalCooldownPartitionSelector( + (OlapTable) table, "2024-03-02 00:00:00", "2024-03-05 00:00:00", true); + Assert.assertTrue(selector.isTableSatisfied()); + + List partitions = selector.computeSatisfiedPartitions(-1); + Assert.assertEquals(3, partitions.size()); + Set satisfiedPartitionNames = partitions.stream().map(Partition::getName).collect(Collectors.toSet()); + Assert.assertFalse("p1 not in partition range", satisfiedPartitionNames.contains("p1")); + Assert.assertTrue("p2 in partition range", satisfiedPartitionNames.contains("p2")); + Assert.assertTrue("p3 satisfied condition", satisfiedPartitionNames.contains("p3")); + Assert.assertTrue("p4 in partition range", satisfiedPartitionNames.contains("p4")); + Assert.assertFalse("p5 not in partition range", satisfiedPartitionNames.contains("p5")); + } + + @Test + public void testSinglePartitionTable() { + Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getTable("test", "tbl4"); + OlapTable olapTable = (OlapTable) table; + + long updateTime = System.currentTimeMillis() - 2000; + Partition p1 = table.getPartition("tbl4"); + p1.updateVisibleVersion(p1.getVisibleVersion() + 1, updateTime); + + ExternalCooldownPartitionSelector selector = new ExternalCooldownPartitionSelector(olapTable); + Assert.assertTrue(selector.isTableSatisfied()); + List partitions = selector.computeSatisfiedPartitions(-1); + Assert.assertEquals(1, partitions.size()); + Set satisfiedPartitionNames = partitions.stream().map(Partition::getName).collect(Collectors.toSet()); + Assert.assertTrue("tbl4 in partition range", satisfiedPartitionNames.contains("tbl4")); + + ExternalCooldownPartitionSelector selector1 = new ExternalCooldownPartitionSelector(olapTable, null, null, true); + Assert.assertTrue(selector1.isTableSatisfied()); + List partitions1 = selector1.computeSatisfiedPartitions(-1); + Assert.assertEquals(1, partitions1.size()); + Set satisfiedPartitionNames1 = partitions1.stream().map(Partition::getName).collect(Collectors.toSet()); + Assert.assertTrue("tbl4 in partition range", satisfiedPartitionNames1.contains("tbl4")); + + ExternalCooldownPartitionSelector selector2 = new ExternalCooldownPartitionSelector(olapTable, "tbl4", "tbl4", false); + Assert.assertTrue(selector2.isTableSatisfied()); + List partitions2 = selector2.computeSatisfiedPartitions(-1); + Assert.assertEquals(1, partitions2.size()); + Set satisfiedPartitionNames2 = partitions2.stream().map(Partition::getName).collect(Collectors.toSet()); + Assert.assertTrue("tbl4 in partition range", satisfiedPartitionNames2.contains("tbl4")); + } + + @Test + public void testListPartitionTable() { + Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getTable("test", "tbl5"); + + long updateTime = System.currentTimeMillis() - 2000; + Partition p1 = table.getPartition("p1"); + p1.updateVisibleVersion(p1.getVisibleVersion() + 1, updateTime); + Partition p3 = table.getPartition("p3"); + p3.updateVisibleVersion(p3.getVisibleVersion() + 1, updateTime); + + ExternalCooldownPartitionSelector selector = new ExternalCooldownPartitionSelector((OlapTable) table); + Assert.assertTrue(selector.isTableSatisfied()); + + List partitions = selector.computeSatisfiedPartitions(-1); + Assert.assertEquals(2, partitions.size()); + Set satisfiedPartitionNames = partitions.stream().map(Partition::getName).collect(Collectors.toSet()); + Assert.assertTrue("p1 satisfied condition", satisfiedPartitionNames.contains("p1")); + Assert.assertFalse("p2 not satisfied condition", satisfiedPartitionNames.contains("p2")); + Assert.assertTrue("p3 satisfied condition", satisfiedPartitionNames.contains("p3")); + } + + @Test + public void testReloadSatisfiedPartitions() throws Exception { + ConnectContext ctx = starRocksAssert.getCtx(); + Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getTable("test", "tbl6"); + + long updateTime = System.currentTimeMillis() - 2000; + Partition p1 = table.getPartition("p1"); + p1.updateVisibleVersion(p1.getVisibleVersion() + 1, updateTime); + Partition p3 = table.getPartition("p3"); + p3.updateVisibleVersion(p3.getVisibleVersion() + 1, updateTime); + + ExternalCooldownPartitionSelector selector = new ExternalCooldownPartitionSelector((OlapTable) table); + Assert.assertTrue(selector.isTableSatisfied()); + + // wait seconds not satisfied + String sql = "ALTER TABLE tbl6 SET(\"external_cooldown_wait_second\" = \"0\");"; + AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(sql, ctx); + GlobalStateMgr.getCurrentState().getLocalMetastore().alterTable(ctx, alterTableStmt); + selector.reloadSatisfiedPartitions(); + Assert.assertFalse(selector.isTableSatisfied()); + + // recover wait seconds not satisfied + String sql1 = "ALTER TABLE tbl6 SET(\"external_cooldown_wait_second\" = \"1\");"; + AlterTableStmt alterTableStmt1 = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(sql1, ctx); + GlobalStateMgr.getCurrentState().getLocalMetastore().alterTable(ctx, alterTableStmt1); + selector.reloadSatisfiedPartitions(); + Assert.assertTrue(selector.isTableSatisfied()); + + // target table not satisfied (null) + String sql2 = "ALTER TABLE tbl6 SET(\"external_cooldown_target\" = \"\");"; + AlterTableStmt alterTableStmt2 = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(sql2, ctx); + GlobalStateMgr.getCurrentState().getLocalMetastore().alterTable(ctx, alterTableStmt2); + selector.reloadSatisfiedPartitions(); + Assert.assertFalse(selector.isTableSatisfied()); + + // recover target table not satisfied + String sql3 = "ALTER TABLE tbl6 SET(\"external_cooldown_target\" = \"iceberg0.partitioned_transforms_db.t0_day_dt\");"; + AlterTableStmt alterTableStmt3 = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(sql3, ctx); + GlobalStateMgr.getCurrentState().getLocalMetastore().alterTable(ctx, alterTableStmt3); + selector.reloadSatisfiedPartitions(); + Assert.assertTrue(selector.isTableSatisfied()); + + // target table not satisfied (null) + ExternalCooldownConfig config = new ExternalCooldownConfig( + "default_catalog.test.tbl5", "START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE", 1L); + ((OlapTable) table).setCurExternalCoolDownConfig(config); + selector.reloadSatisfiedPartitions(); + Assert.assertFalse(selector.isTableSatisfied()); + + // recover target table not satisfied + String sql5 = "ALTER TABLE tbl6 SET(\"external_cooldown_target\" = \"iceberg0.partitioned_transforms_db.t0_day_dt\");"; + AlterTableStmt alterTableStmt5 = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(sql5, ctx); + GlobalStateMgr.getCurrentState().getLocalMetastore().alterTable(ctx, alterTableStmt5); + selector.reloadSatisfiedPartitions(); + Assert.assertTrue(selector.isTableSatisfied()); + List satisfiedPartitions = selector.computeSatisfiedPartitions(-1); + Assert.assertEquals(2, satisfiedPartitions.size()); + List satisfiedPartitionsLimited = selector.computeSatisfiedPartitions(1); + Assert.assertEquals(1, satisfiedPartitionsLimited.size()); + } + + @Test + public void testGetNextSatisfiedPartition() { + Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getTable("test", "tbl7"); + + long updateTime = System.currentTimeMillis() - 2000; + Partition p1 = table.getPartition("p1"); + p1.updateVisibleVersion(p1.getVisibleVersion() + 1, updateTime); + Partition p3 = table.getPartition("p3"); + p3.updateVisibleVersion(p3.getVisibleVersion() + 1, updateTime); + + ExternalCooldownPartitionSelector selector = new ExternalCooldownPartitionSelector((OlapTable) table); + Partition partition = selector.getOneSatisfiedPartition(); + Assert.assertEquals("p1", partition.getName()); + Partition partition2 = selector.getNextSatisfiedPartition(partition); + Assert.assertEquals("p3", partition2.getName()); + Partition partition3 = selector.getNextSatisfiedPartition(partition2); + Assert.assertNull(partition3); + } + + @Test + public void testNonPartitionTableWithIcebergPartitioned() throws Exception { + Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getTable("test", "tbl8"); + long updateTime = System.currentTimeMillis() - 2000; + Partition p1 = table.getPartition("tbl8"); + p1.updateVisibleVersion(p1.getVisibleVersion() + 1, updateTime); + + ExternalCooldownPartitionSelector selector = new ExternalCooldownPartitionSelector((OlapTable) table); + List satisfiedPartitions = selector.computeSatisfiedPartitions(-1); + Assert.assertEquals(0, satisfiedPartitions.size()); + } + + @Test + public void testGetPartitionsInRange() { + Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getTable("test", "tbl9"); + long updateTime = System.currentTimeMillis() - 2000; + Partition p1 = table.getPartition("p1"); + p1.updateVisibleVersion(p1.getVisibleVersion() + 1, updateTime); + Partition p3 = table.getPartition("p3"); + p3.updateVisibleVersion(p3.getVisibleVersion() + 1, updateTime); + Partition p4 = table.getPartition("p4"); + p4.updateVisibleVersion(p4.getVisibleVersion() + 1, updateTime); + + ExternalCooldownPartitionSelector selector = new ExternalCooldownPartitionSelector( + (OlapTable) table, "p1", "p3", false); + List satisfiedPartitions = selector.computeSatisfiedPartitions(-1); + Assert.assertEquals(2, satisfiedPartitions.size()); + + ExternalCooldownPartitionSelector selector1 = new ExternalCooldownPartitionSelector( + (OlapTable) table, "p1", "p3", true); + List satisfiedPartitions1 = selector1.computeSatisfiedPartitions(-1); + Assert.assertEquals(3, satisfiedPartitions1.size()); + } + + @Test + public void testGetNativeTableNull() { + Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getTable("test", "tbl10"); + ExternalCooldownPartitionSelector selector = new ExternalCooldownPartitionSelector((OlapTable) table); + + new MockUp() { + @Mock + public org.apache.iceberg.Table getNativeTable() { + return null; + } + }; + selector.reloadSatisfiedPartitions(); + Assert.assertFalse(selector.isTableSatisfied()); + } + + @Test + public void testWaitConsistencyCheck() throws Exception { + OlapTable table = (OlapTable) GlobalStateMgr.getCurrentState().getLocalMetastore().getTable( + "test", "tbl11"); + + PartitionInfo partitionInfo = table.getPartitionInfo(); + ExternalCooldownPartitionSelector selector = new ExternalCooldownPartitionSelector((OlapTable) table); + long updateTime = System.currentTimeMillis() - 2000; + Partition p1 = table.getPartition("p1"); + p1.updateVisibleVersion(p1.getVisibleVersion() + 1, updateTime); + partitionInfo.setExternalCoolDownSyncedTimeMs(p1.getId(), updateTime); + partitionInfo.setExternalCoolDownConsistencyCheckTimeMs(p1.getId(), updateTime + 1000); + partitionInfo.setCoolDownConsistencyCheckDifference(p1.getId(), 0); + Assert.assertFalse(selector.isPartitionSatisfied(p1)); + + Partition p2 = table.getPartition("p2"); + p2.updateVisibleVersion(p2.getVisibleVersion() + 1, updateTime); + partitionInfo.setExternalCoolDownSyncedTimeMs(p2.getId(), updateTime); + partitionInfo.setExternalCoolDownConsistencyCheckTimeMs(p2.getId(), updateTime - 1000); + partitionInfo.setCoolDownConsistencyCheckDifference(p2.getId(), 0); + Assert.assertFalse(selector.isPartitionSatisfied(p2)); + + Partition p3 = table.getPartition("p3"); + p3.updateVisibleVersion(p3.getVisibleVersion() + 1, updateTime); + partitionInfo.setExternalCoolDownSyncedTimeMs(p3.getId(), updateTime); + partitionInfo.setExternalCoolDownConsistencyCheckTimeMs(p3.getId(), updateTime + 1000); + partitionInfo.setCoolDownConsistencyCheckDifference(p3.getId(), 1); + Assert.assertTrue(selector.isPartitionSatisfied(p3)); + + Partition p4 = table.getPartition("p4"); + p4.updateVisibleVersion(p4.getVisibleVersion() + 1, updateTime); + partitionInfo.setExternalCoolDownSyncedTimeMs(p4.getId(), updateTime); + partitionInfo.setExternalCoolDownConsistencyCheckTimeMs(p4.getId(), updateTime - 1000); + partitionInfo.setCoolDownConsistencyCheckDifference(p4.getId(), -1); + Assert.assertFalse(selector.isPartitionSatisfied(p4)); + } +} \ No newline at end of file diff --git a/fe/fe-core/src/test/java/com/starrocks/externalcooldown/ExternalCooldownScheduleTest.java b/fe/fe-core/src/test/java/com/starrocks/externalcooldown/ExternalCooldownScheduleTest.java new file mode 100644 index 0000000000000..3e0dcfe190810 --- /dev/null +++ b/fe/fe-core/src/test/java/com/starrocks/externalcooldown/ExternalCooldownScheduleTest.java @@ -0,0 +1,34 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed 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 +// +// https://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 com.starrocks.externalcooldown; + +import org.junit.Assert; +import org.junit.Test; + + +public class ExternalCooldownScheduleTest { + + @Test + public void testPartitionStartEnd() { + String scheduleStr = "START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE"; + ExternalCooldownSchedule schedule = ExternalCooldownSchedule.fromString(scheduleStr); + Assert.assertNotNull(schedule); + Assert.assertEquals("01:00", schedule.getStart()); + Assert.assertEquals("07:59", schedule.getEnd()); + Assert.assertEquals(1L, schedule.getInterval()); + Assert.assertEquals("MINUTE", schedule.getUnit()); + Assert.assertEquals(60L, schedule.getIntervalSeconds()); + } +} \ No newline at end of file diff --git a/fe/fe-core/src/test/java/com/starrocks/persist/ExternalCooldownConfigTest.java b/fe/fe-core/src/test/java/com/starrocks/persist/ExternalCooldownConfigTest.java new file mode 100644 index 0000000000000..31111efc1b4fb --- /dev/null +++ b/fe/fe-core/src/test/java/com/starrocks/persist/ExternalCooldownConfigTest.java @@ -0,0 +1,70 @@ +// 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 com.starrocks.persist; + +import com.starrocks.externalcooldown.ExternalCooldownConfig; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; + + +public class ExternalCooldownConfigTest { + private String fileName = "./ExternalCooldownConfigTest"; + private String fileName1 = "./ExternalCooldownConfigTest1"; + + @After + public void tearDown() { + File file = new File(fileName); + file.delete(); + + File file1 = new File(fileName1); + file1.delete(); + } + + @Test + public void testNormal() throws IOException { + // 1. Write objects to file + File file = new File(fileName); + file.createNewFile(); + DataOutputStream out = new DataOutputStream(new FileOutputStream(file)); + + ExternalCooldownConfig config = new ExternalCooldownConfig( + "iceberg.db1.tbl1", "START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE", 3600L); + + config.write(out); + out.flush(); + out.close(); + + // 2. Read objects from file + DataInputStream in = new DataInputStream(new FileInputStream(file)); + ExternalCooldownConfig config2 = ExternalCooldownConfig.read(in); + Assert.assertEquals(config, config2); + + in.close(); + + Assert.assertNotEquals(config, null); + Assert.assertNotEquals(config, 0); + } +} \ No newline at end of file diff --git a/fe/fe-core/src/test/java/com/starrocks/persist/ExternalCooldownMaintenanceJobTest.java b/fe/fe-core/src/test/java/com/starrocks/persist/ExternalCooldownMaintenanceJobTest.java new file mode 100644 index 0000000000000..b9e071d2cfb09 --- /dev/null +++ b/fe/fe-core/src/test/java/com/starrocks/persist/ExternalCooldownMaintenanceJobTest.java @@ -0,0 +1,79 @@ +// 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 com.starrocks.persist; + +import com.starrocks.catalog.OlapTable; +import com.starrocks.common.io.Writable; +import com.starrocks.journal.JournalEntity; +import com.starrocks.scheduler.externalcooldown.ExternalCooldownMaintenanceJob; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; + +import static com.starrocks.persist.OperationType.OP_EXTERNAL_COOLDOWN_JOB_STATE; + +public class ExternalCooldownMaintenanceJobTest { + private String fileName = "./ExternalCooldownMaintenanceJobTest"; + + @After + public void tearDown() { + File file = new File(fileName); + file.delete(); + } + + @Test + public void testNormal() throws IOException { + // 1. Write objects to file + File file = new File(fileName); + file.createNewFile(); + DataOutputStream out = new DataOutputStream(new FileOutputStream(file)); + + OlapTable table = new OlapTable(2000L, "tb2", null, null, null, null); + ExternalCooldownMaintenanceJob job = new ExternalCooldownMaintenanceJob(table, 1001L); + + out.writeShort(OP_EXTERNAL_COOLDOWN_JOB_STATE); + job.write(out); + out.flush(); + out.close(); + + // 2. Read objects from file + DataInputStream in = new DataInputStream(new FileInputStream(file)); + JournalEntity journalEntity = new JournalEntity(); + journalEntity.readFields(in); + Writable data = journalEntity.getData(); + Assert.assertEquals(OP_EXTERNAL_COOLDOWN_JOB_STATE, journalEntity.getOpCode()); + Assert.assertTrue(data instanceof ExternalCooldownMaintenanceJob); + + ExternalCooldownMaintenanceJob recoverJob = (ExternalCooldownMaintenanceJob) data; + Assert.assertEquals(1001L, recoverJob.getDbId()); + Assert.assertEquals(2000L, recoverJob.getTableId()); + Assert.assertEquals(2000L, recoverJob.getJobId()); + + in.close(); + + Assert.assertNotEquals(recoverJob, null); + Assert.assertNotEquals(recoverJob, 0); + } +} diff --git a/fe/fe-core/src/test/java/com/starrocks/persist/ModifyExternalCooldownConfigTest.java b/fe/fe-core/src/test/java/com/starrocks/persist/ModifyExternalCooldownConfigTest.java new file mode 100644 index 0000000000000..8bb82ca512adf --- /dev/null +++ b/fe/fe-core/src/test/java/com/starrocks/persist/ModifyExternalCooldownConfigTest.java @@ -0,0 +1,79 @@ +// 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 com.starrocks.persist; + +import com.starrocks.common.io.Writable; +import com.starrocks.common.util.PropertyAnalyzer; +import com.starrocks.journal.JournalEntity; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; +import java.util.HashMap; + +import static com.starrocks.persist.OperationType.OP_MODIFY_EXTERNAL_COOLDOWN_CONFIG; + +public class ModifyExternalCooldownConfigTest { + private String fileName = "./ModifyExternalCooldownConfigTest"; + + @After + public void tearDown() { + File file = new File(fileName); + file.delete(); + } + + @Test + public void testNormal() throws IOException { + // 1. Write objects to file + File file = new File(fileName); + file.createNewFile(); + DataOutputStream out = new DataOutputStream(new FileOutputStream(file)); + + HashMap properties = new HashMap<>(); + properties.put(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_TARGET, "iceberg.db1.tbl1"); + properties.put(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_WAIT_SECOND, "3600"); + properties.put(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_SCHEDULE, + "START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE"); + ModifyTablePropertyOperationLog modifyTablePropertyOperationLog = + new ModifyTablePropertyOperationLog(100L, 200L, properties); + out.writeShort(OP_MODIFY_EXTERNAL_COOLDOWN_CONFIG); + modifyTablePropertyOperationLog.write(out); + out.flush(); + out.close(); + + // 2. Read objects from file + DataInputStream in = new DataInputStream(new FileInputStream(file)); + JournalEntity journalEntity = new JournalEntity(); + journalEntity.readFields(in); + Writable data = journalEntity.getData(); + Assert.assertEquals(OP_MODIFY_EXTERNAL_COOLDOWN_CONFIG, journalEntity.getOpCode()); + Assert.assertTrue(data instanceof ModifyTablePropertyOperationLog); + + ModifyTablePropertyOperationLog readModifyExternalCooldownConfigInfo = (ModifyTablePropertyOperationLog) data; + Assert.assertEquals(100L, readModifyExternalCooldownConfigInfo.getDbId()); + Assert.assertEquals(200L, readModifyExternalCooldownConfigInfo.getTableId()); + Assert.assertEquals(readModifyExternalCooldownConfigInfo.getProperties(), properties); + in.close(); + } +} diff --git a/fe/fe-core/src/test/java/com/starrocks/persist/OperationTypeTest.java b/fe/fe-core/src/test/java/com/starrocks/persist/OperationTypeTest.java index 689b382eb1ce2..6ee0acc68c1a2 100644 --- a/fe/fe-core/src/test/java/com/starrocks/persist/OperationTypeTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/persist/OperationTypeTest.java @@ -150,6 +150,7 @@ public void testRecoverableOperations() { Assert.assertTrue(OperationType.IGNORABLE_OPERATIONS.contains(OperationType.OP_REPLICATION_JOB)); Assert.assertTrue(OperationType.IGNORABLE_OPERATIONS.contains(OperationType.OP_DELETE_REPLICATION_JOB)); Assert.assertTrue(OperationType.IGNORABLE_OPERATIONS.contains(OperationType.OP_RESET_FRONTENDS)); + Assert.assertTrue(OperationType.IGNORABLE_OPERATIONS.contains(OperationType.OP_EXTERNAL_COOLDOWN_JOB_STATE)); } @Test diff --git a/fe/fe-core/src/test/java/com/starrocks/qe/ExternalCooldownExecutorTest.java b/fe/fe-core/src/test/java/com/starrocks/qe/ExternalCooldownExecutorTest.java new file mode 100644 index 0000000000000..af0bfae370a59 --- /dev/null +++ b/fe/fe-core/src/test/java/com/starrocks/qe/ExternalCooldownExecutorTest.java @@ -0,0 +1,96 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed 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 +// +// https://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 com.starrocks.qe; + +import com.starrocks.catalog.IcebergTable; +import com.starrocks.catalog.Table; +import com.starrocks.scheduler.Task; +import com.starrocks.scheduler.TaskBuilder; +import com.starrocks.server.GlobalStateMgr; +import com.starrocks.server.MetadataMgr; +import com.starrocks.sql.ast.StatementBase; +import com.starrocks.sql.plan.ConnectorPlanTestBase; +import com.starrocks.statistic.StatisticsMetaManager; +import com.starrocks.utframe.StarRocksAssert; +import com.starrocks.utframe.UtFrameUtils; +import mockit.Mock; +import mockit.MockUp; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class ExternalCooldownExecutorTest { + @ClassRule + public static TemporaryFolder temp = new TemporaryFolder(); + + private static ConnectContext connectContext; + private static StarRocksAssert starRocksAssert; + + @BeforeClass + public static void beforeClass() throws Exception { + ConnectorPlanTestBase.doInit(temp.newFolder().toURI().toString()); + connectContext = UtFrameUtils.createDefaultCtx(); + starRocksAssert = new StarRocksAssert(connectContext); + if (!starRocksAssert.databaseExist("_statistics_")) { + StatisticsMetaManager m = new StatisticsMetaManager(); + m.createStatisticsTablesForTest(); + } + + new MockUp() { + @Mock + public Table getTable(String catalogName, String dbName, String tblName) { + return new IcebergTable(); + } + }; + starRocksAssert.withDatabase("test").useDatabase("test") + .withTable("CREATE TABLE test.tbl1\n" + + "(\n" + + " k1 int,\n" + + " k2 datetime,\n" + + " v1 int\n" + + ")\n" + + "DUPLICATE KEY(`k1`, `k2`)\n" + + "PARTITION BY RANGE(`k2`)\n" + + "(\n" + + " PARTITION p1 VALUES [('2020-01-01 00:00:00'),('2020-01-02 00:00:00')),\n" + + " PARTITION p2 VALUES [('2020-01-02 00:00:00'),('2020-01-03 00:00:00'))\n" + + ")\n" + + "DISTRIBUTED BY HASH(k1) BUCKETS 1\n" + + "PROPERTIES(\n" + + "'external_cooldown_target'='iceberg.db1.tbl1',\n" + + "'external_cooldown_schedule'='START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE',\n" + + "'external_cooldown_wait_second'='3600',\n" + + "'replication_num' = '1'\n" + + ");"); + } + + @Test + public void testCooldownTable() throws Exception { + StatementBase stmt = UtFrameUtils.parseStmtWithNewParser("cooldown table test.tbl1", connectContext); + StmtExecutor executor = new StmtExecutor(connectContext, stmt); + executor.execute(); + + Table table = starRocksAssert.getTable("test", "tbl1"); + String taskName = TaskBuilder.getExternalCooldownTaskName(table.getId()); + Task task = GlobalStateMgr.getCurrentState().getTaskManager().getTask(taskName); + Assert.assertNotNull(task); + + StatementBase stmt1 = UtFrameUtils.parseStmtWithNewParser("cancel cooldown table test.tbl1", connectContext); + StmtExecutor executor1 = new StmtExecutor(connectContext, stmt1); + executor1.execute(); + } +} diff --git a/fe/fe-core/src/test/java/com/starrocks/scheduler/PartitionBasedCooldownProcessorTest.java b/fe/fe-core/src/test/java/com/starrocks/scheduler/PartitionBasedCooldownProcessorTest.java new file mode 100644 index 0000000000000..b2c864af57264 --- /dev/null +++ b/fe/fe-core/src/test/java/com/starrocks/scheduler/PartitionBasedCooldownProcessorTest.java @@ -0,0 +1,153 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed 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 +// +// https://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 com.starrocks.scheduler; + +import com.starrocks.catalog.Database; +import com.starrocks.catalog.OlapTable; +import com.starrocks.catalog.Partition; +import com.starrocks.catalog.PartitionInfo; +import com.starrocks.common.util.UUIDUtil; +import com.starrocks.connector.iceberg.MockIcebergMetadata; +import com.starrocks.pseudocluster.PseudoCluster; +import com.starrocks.qe.ConnectContext; +import com.starrocks.qe.StmtExecutor; +import com.starrocks.server.GlobalStateMgr; +import com.starrocks.sql.plan.ConnectorPlanTestBase; +import com.starrocks.sql.plan.ExecPlan; +import com.starrocks.statistic.StatisticsMetaManager; +import com.starrocks.thrift.TExplainLevel; +import com.starrocks.utframe.StarRocksAssert; +import com.starrocks.utframe.UtFrameUtils; +import mockit.Mock; +import mockit.MockUp; +import org.apache.commons.lang3.StringUtils; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.FixMethodOrder; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runners.MethodSorters; + +import java.time.Instant; + +@FixMethodOrder(MethodSorters.NAME_ASCENDING) +public class PartitionBasedCooldownProcessorTest { + protected static ConnectContext connectContext; + protected static PseudoCluster cluster; + protected static StarRocksAssert starRocksAssert; + @ClassRule + public static TemporaryFolder temp = new TemporaryFolder(); + + protected static long startSuiteTime = 0; + protected long startCaseTime = 0; + + protected static final String TEST_DB_NAME = "test"; + + @BeforeClass + public static void beforeClass() throws Exception { + UtFrameUtils.createMinStarRocksCluster(); + connectContext = UtFrameUtils.createDefaultCtx(); + starRocksAssert = new StarRocksAssert(connectContext); + + // set default config for async mvs + UtFrameUtils.setDefaultConfigForAsyncMVTest(connectContext); + + if (!starRocksAssert.databaseExist("_statistics_")) { + StatisticsMetaManager m = new StatisticsMetaManager(); + m.createStatisticsTablesForTest(); + } + starRocksAssert.withDatabase(TEST_DB_NAME); + starRocksAssert.useDatabase(TEST_DB_NAME); + + ConnectorPlanTestBase.mockCatalog(connectContext, MockIcebergMetadata.MOCKED_ICEBERG_CATALOG_NAME); + + + starRocksAssert.withDatabase(TEST_DB_NAME).useDatabase(TEST_DB_NAME) + .withTable("CREATE TABLE tbl1\n" + + "(\n" + + " id int,\n" + + " ts datetime,\n" + + " data string\n" + + ")\n" + + "DUPLICATE KEY(`id`, `ts`)\n" + + "PARTITION BY RANGE(`ts`)\n" + + "(\n" + + " PARTITION p20200101 VALUES [('2020-01-01 00:00:00'),('2020-01-02 00:00:00')),\n" + + " PARTITION p20200102 VALUES [('2020-01-02 00:00:00'),('2020-01-03 00:00:00')),\n" + + " PARTITION p20200103 VALUES [('2020-01-03 00:00:00'),('2020-01-04 00:00:00')),\n" + + " PARTITION p20200104 VALUES [('2020-01-04 00:00:00'),('2020-01-05 00:00:00')),\n" + + " PARTITION p20200105 VALUES [('2020-01-05 00:00:00'),('2020-01-06 00:00:00'))\n" + + ")\n" + + "DISTRIBUTED BY HASH(`id`) BUCKETS 1\n" + + "PROPERTIES(\n" + + "'external_cooldown_target'='iceberg0.partitioned_transforms_db.t0_day',\n" + + "'external_cooldown_schedule'='START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE',\n" + + "'external_cooldown_wait_second'='3600',\n" + + "'replication_num' = '1'\n" + + ");"); + } + + @Before + public void before() { + startCaseTime = Instant.now().getEpochSecond(); + } + + protected static void initAndExecuteTaskRun(TaskRun taskRun) throws Exception { + taskRun.initStatus(UUIDUtil.genUUID().toString(), System.currentTimeMillis()); + + new MockUp() { + @Mock + public void execute() throws Exception { + return; + } + }; + taskRun.executeTaskRun(); + } + + private static void triggerExternalCooldown(Database testDb, OlapTable table, Partition partition) throws Exception { + Task task = TaskBuilder.buildExternalCooldownTask(testDb, table, partition); + TaskRun taskRun = TaskRunBuilder.newBuilder(task).build(); + + initAndExecuteTaskRun(taskRun); + } + + protected void assertPlanContains(ExecPlan execPlan, String... explain) { + String explainString = execPlan.getExplainString(TExplainLevel.NORMAL); + + for (String expected : explain) { + Assert.assertTrue("expected is: " + expected + " but plan is \n" + explainString, + StringUtils.containsIgnoreCase(explainString.toLowerCase(), expected)); + } + } + + @Test + public void testCooldownSinglePartition() throws Exception { + OlapTable olapTable = ((OlapTable) GlobalStateMgr.getCurrentState().getLocalMetastore() + .getTable(TEST_DB_NAME, "tbl1")); + Partition partition = olapTable.getPartition("p20200101"); + partition.setVisibleVersion(partition.getVisibleVersion() + 1, + System.currentTimeMillis() - 3600 * 1000); + + Database testDb = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb(TEST_DB_NAME); + triggerExternalCooldown(testDb, olapTable, partition); + + PartitionInfo partitionInfo = olapTable.getPartitionInfo(); + Long syncedTimeMs = partitionInfo.getExternalCoolDownSyncedTimeMs(partition.getId()); + Assert.assertNotNull(syncedTimeMs); + Assert.assertEquals((Long) partition.getVisibleVersionTime(), syncedTimeMs); + } +} \ No newline at end of file diff --git a/fe/fe-core/src/test/java/com/starrocks/scheduler/externalcooldown/ExternalCooldownExecutorTest.java b/fe/fe-core/src/test/java/com/starrocks/scheduler/externalcooldown/ExternalCooldownExecutorTest.java new file mode 100644 index 0000000000000..b2d063043517f --- /dev/null +++ b/fe/fe-core/src/test/java/com/starrocks/scheduler/externalcooldown/ExternalCooldownExecutorTest.java @@ -0,0 +1,152 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed 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 +// +// https://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 com.starrocks.scheduler.externalcooldown; + +import com.starrocks.catalog.OlapTable; +import com.starrocks.catalog.Partition; +import com.starrocks.catalog.Table; +import com.starrocks.common.DdlException; +import com.starrocks.connector.iceberg.MockIcebergMetadata; +import com.starrocks.externalcooldown.ExternalCooldownSchedule; +import com.starrocks.pseudocluster.PseudoCluster; +import com.starrocks.qe.ConnectContext; +import com.starrocks.server.GlobalStateMgr; +import com.starrocks.sql.ast.AlterTableStmt; +import com.starrocks.sql.plan.ConnectorPlanTestBase; +import com.starrocks.statistic.StatisticsMetaManager; +import com.starrocks.utframe.StarRocksAssert; +import com.starrocks.utframe.UtFrameUtils; +import mockit.Mock; +import mockit.MockUp; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.text.SimpleDateFormat; + + +public class ExternalCooldownExecutorTest { + @ClassRule + public static TemporaryFolder temp = new TemporaryFolder(); + + private static StarRocksAssert starRocksAssert; + + protected static PseudoCluster cluster; + + protected static final String TEST_DB_NAME = "test"; + private final SimpleDateFormat timeFormat = new SimpleDateFormat("HH:mm"); + + @BeforeClass + public static void beforeClass() throws Exception { + UtFrameUtils.createMinStarRocksCluster(); + ConnectContext connectContext = UtFrameUtils.createDefaultCtx(); + starRocksAssert = new StarRocksAssert(connectContext); + + // set default config for async mvs + UtFrameUtils.setDefaultConfigForAsyncMVTest(connectContext); + + if (!starRocksAssert.databaseExist("_statistics_")) { + StatisticsMetaManager m = new StatisticsMetaManager(); + m.createStatisticsTablesForTest(); + } + starRocksAssert.withDatabase(TEST_DB_NAME); + starRocksAssert.useDatabase(TEST_DB_NAME); + + ConnectorPlanTestBase.mockCatalog(connectContext, MockIcebergMetadata.MOCKED_ICEBERG_CATALOG_NAME); + GlobalStateMgr.getCurrentState().getExternalCooldownJobExecutor().setStop(); + + starRocksAssert.withDatabase(TEST_DB_NAME).useDatabase(TEST_DB_NAME) + .withTable("CREATE TABLE tbl1\n" + + "(\n" + + " id int,\n" + + " ts datetime,\n" + + " data string\n" + + ")\n" + + "DUPLICATE KEY(`id`, `ts`)\n" + + "PARTITION BY RANGE(`ts`)\n" + + "(\n" + + " PARTITION p20200101 VALUES [('2020-01-01 00:00:00'),('2020-01-02 00:00:00')),\n" + + " PARTITION p20200102 VALUES [('2020-01-02 00:00:00'),('2020-01-03 00:00:00')),\n" + + " PARTITION p20200103 VALUES [('2020-01-03 00:00:00'),('2020-01-04 00:00:00')),\n" + + " PARTITION p20200104 VALUES [('2020-01-04 00:00:00'),('2020-01-05 00:00:00')),\n" + + " PARTITION p20200105 VALUES [('2020-01-05 00:00:00'),('2020-01-06 00:00:00'))\n" + + ")\n" + + "DISTRIBUTED BY HASH(`id`) BUCKETS 1\n" + + "PROPERTIES(\n" + + "'external_cooldown_target'='iceberg0.partitioned_transforms_db.t0_day',\n" + + "'external_cooldown_schedule'='START 22:00 END 08:00 EVERY INTERVAL 1 MINUTE',\n" + + "'external_cooldown_wait_second'='3600',\n" + + "'replication_num' = '1'\n" + + ");"); + } + + @Test + public void testExternalCooldownNormalExecute() throws Exception { + ConnectContext ctx = starRocksAssert.getCtx(); + ExternalCooldownJobExecutor executor = GlobalStateMgr.getCurrentState().getExternalCooldownJobExecutor(); + ExternalCooldownMgr mgr = GlobalStateMgr.getCurrentState().getExternalCooldownMgr(); + mgr.doInitializeIfNeed(); + + // compute a valid schedule start and end + long now = System.currentTimeMillis(); + String start = timeFormat.format(now - 3600 * 1000); + String end = timeFormat.format(now + 3600 * 1000); + String schedule = String.format("START %s END %s EVERY INTERVAL 1 MINUTE", start, end); + + // alter schedule + String sql1 = "ALTER TABLE tbl1 SET(\"external_cooldown_schedule\" = \"" + schedule + "\");"; + AlterTableStmt alterTableStmt1 = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(sql1, ctx); + GlobalStateMgr.getCurrentState().getLocalMetastore().alterTable(ctx, alterTableStmt1); + + Table table = GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(TEST_DB_NAME, "tbl1"); + Partition p1 = table.getPartition("p20200101"); + p1.updateVisibleVersion(p1.getVisibleVersion() + 1); + Partition p2 = table.getPartition("p20200102"); + p2.updateVisibleVersion(p2.getVisibleVersion() + 1); + Partition p4 = table.getPartition("p20200104"); + p4.updateVisibleVersion(p4.getVisibleVersion() + 1); + Partition p5 = table.getPartition("p20200105"); + p5.updateVisibleVersion(p5.getVisibleVersion() + 1); + + // run + executor.runAfterCatalogReady(); + + ExternalCooldownSchedule scheduleObj = mgr.getRunnableJobs().get(0).getSchedule(); + Assert.assertNotEquals(0, scheduleObj.getLastScheduleMs()); + + // test no jobs + new MockUp() { + @Mock + public void onSchedule(long currentMs) throws DdlException { + throw new DdlException("test"); + } + }; + executor.runAfterCatalogReady(); + + new MockUp() { + @Mock + public void onSchedule(long currentMs) throws DdlException { + throw new RuntimeException("test"); + } + }; + executor.runAfterCatalogReady(); + + mgr.stopMaintainExternalCooldown((OlapTable) table); + executor.runAfterCatalogReady(); + } +} \ No newline at end of file diff --git a/fe/fe-core/src/test/java/com/starrocks/scheduler/externalcooldown/ExternalCooldownMaintenanceJobTest.java b/fe/fe-core/src/test/java/com/starrocks/scheduler/externalcooldown/ExternalCooldownMaintenanceJobTest.java new file mode 100644 index 0000000000000..1f700f26a6f17 --- /dev/null +++ b/fe/fe-core/src/test/java/com/starrocks/scheduler/externalcooldown/ExternalCooldownMaintenanceJobTest.java @@ -0,0 +1,148 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed 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 +// +// https://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 com.starrocks.scheduler.externalcooldown; + +import com.starrocks.catalog.Database; +import com.starrocks.catalog.OlapTable; +import com.starrocks.catalog.Partition; +import com.starrocks.connector.iceberg.MockIcebergMetadata; +import com.starrocks.externalcooldown.ExternalCooldownConfig; +import com.starrocks.pseudocluster.PseudoCluster; +import com.starrocks.qe.ConnectContext; +import com.starrocks.server.GlobalStateMgr; +import com.starrocks.sql.plan.ConnectorPlanTestBase; +import com.starrocks.statistic.StatisticsMetaManager; +import com.starrocks.utframe.StarRocksAssert; +import com.starrocks.utframe.UtFrameUtils; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.FileOutputStream; +import java.io.IOException; + + +public class ExternalCooldownMaintenanceJobTest { + @ClassRule + public static TemporaryFolder temp = new TemporaryFolder(); + + private static ConnectContext connectContext; + private static StarRocksAssert starRocksAssert; + + protected static PseudoCluster cluster; + + protected static final String TEST_DB_NAME = "test"; + + @BeforeClass + public static void beforeClass() throws Exception { + UtFrameUtils.createMinStarRocksCluster(); + connectContext = UtFrameUtils.createDefaultCtx(); + starRocksAssert = new StarRocksAssert(connectContext); + + // set default config for async mvs + UtFrameUtils.setDefaultConfigForAsyncMVTest(connectContext); + + if (!starRocksAssert.databaseExist("_statistics_")) { + StatisticsMetaManager m = new StatisticsMetaManager(); + m.createStatisticsTablesForTest(); + } + starRocksAssert.withDatabase(TEST_DB_NAME); + starRocksAssert.useDatabase(TEST_DB_NAME); + + ConnectorPlanTestBase.mockCatalog(connectContext, MockIcebergMetadata.MOCKED_ICEBERG_CATALOG_NAME); + + + starRocksAssert.withDatabase(TEST_DB_NAME).useDatabase(TEST_DB_NAME) + .withTable("CREATE TABLE tbl1\n" + + "(\n" + + " id int,\n" + + " ts datetime,\n" + + " data string\n" + + ")\n" + + "DUPLICATE KEY(`id`, `ts`)\n" + + "PARTITION BY RANGE(`ts`)\n" + + "(\n" + + " PARTITION p20200101 VALUES [('2020-01-01 00:00:00'),('2020-01-02 00:00:00')),\n" + + " PARTITION p20200102 VALUES [('2020-01-02 00:00:00'),('2020-01-03 00:00:00')),\n" + + " PARTITION p20200103 VALUES [('2020-01-03 00:00:00'),('2020-01-04 00:00:00')),\n" + + " PARTITION p20200104 VALUES [('2020-01-04 00:00:00'),('2020-01-05 00:00:00')),\n" + + " PARTITION p20200105 VALUES [('2020-01-05 00:00:00'),('2020-01-06 00:00:00'))\n" + + ")\n" + + "DISTRIBUTED BY HASH(`id`) BUCKETS 1\n" + + "PROPERTIES(\n" + + "'external_cooldown_target'='iceberg0.partitioned_transforms_db.t0_day',\n" + + "'external_cooldown_schedule'='START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE',\n" + + "'external_cooldown_wait_second'='3600',\n" + + "'replication_num' = '1'\n" + + ");"); + } + + @Test + public void testNormal() throws Exception { + Database testDb = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); + OlapTable table = (OlapTable) GlobalStateMgr.getCurrentState().getLocalMetastore().getTable("test", "tbl1"); + ExternalCooldownMaintenanceJob job = new ExternalCooldownMaintenanceJob(table, testDb.getId()); + Assert.assertEquals(table.getId(), job.getJobId()); + String str = String.format("ExternalCooldownJob id=%s,dbId=%s,tableId=%d", + job.getJobId(), job.getDbId(), job.getTableId()); + Assert.assertEquals(str, job.toString()); + ExternalCooldownMaintenanceJob job2 = new ExternalCooldownMaintenanceJob(table, testDb.getId()); + Assert.assertEquals(job, job2); + Assert.assertEquals(job.hashCode(), job2.hashCode()); + } + + @Test + public void testRestore() throws IOException { + Database testDb = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb("test"); + OlapTable table = (OlapTable) GlobalStateMgr.getCurrentState().getLocalMetastore().getTable("test", "tbl1"); + long updateTime = System.currentTimeMillis() - 3600 * 1000; + Partition p1 = table.getPartition("p20200101"); + p1.updateVisibleVersion(p1.getVisibleVersion() + 1, updateTime); + + // 1. Write job to file + String fileName = "./ExternalCooldownMaintenanceJobRestoreTest"; + File file = new File(fileName); + file.createNewFile(); + DataOutputStream out = new DataOutputStream(new FileOutputStream(file)); + ExternalCooldownMaintenanceJob job = new ExternalCooldownMaintenanceJob(table, testDb.getId()); + job.write(out); + out.flush(); + out.close(); + + // 2. Read objects from file + DataInputStream in = new DataInputStream(new FileInputStream(file)); + ExternalCooldownMaintenanceJob jobRestore = ExternalCooldownMaintenanceJob.read(in); + in.close(); + jobRestore.restore(); + Assert.assertNotNull(jobRestore.getSchedule()); + Assert.assertNotNull(jobRestore.getOlapTable()); + Assert.assertNotNull(jobRestore.getPartitionSelector()); + Assert.assertEquals(1, jobRestore.getPartitionSelector().computeSatisfiedPartitions(-1).size()); + + // 3. Check reload when config changed + ExternalCooldownConfig config = table.getCurExternalCoolDownConfig(); + config.setWaitSecond(0L); + table.setCurExternalCoolDownConfig(config); + jobRestore.restore(); + Assert.assertEquals(0, jobRestore.getPartitionSelector().computeSatisfiedPartitions(-1).size()); + } +} \ No newline at end of file diff --git a/fe/fe-core/src/test/java/com/starrocks/scheduler/externalcooldown/ExternalCooldownMgrTest.java b/fe/fe-core/src/test/java/com/starrocks/scheduler/externalcooldown/ExternalCooldownMgrTest.java new file mode 100644 index 0000000000000..81d0e306b250f --- /dev/null +++ b/fe/fe-core/src/test/java/com/starrocks/scheduler/externalcooldown/ExternalCooldownMgrTest.java @@ -0,0 +1,135 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed 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 +// +// https://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 com.starrocks.scheduler.externalcooldown; + +import com.starrocks.catalog.Database; +import com.starrocks.catalog.OlapTable; +import com.starrocks.catalog.Partition; +import com.starrocks.connector.iceberg.MockIcebergMetadata; +import com.starrocks.pseudocluster.PseudoCluster; +import com.starrocks.qe.ConnectContext; +import com.starrocks.scheduler.Task; +import com.starrocks.scheduler.TaskManager; +import com.starrocks.server.GlobalStateMgr; +import com.starrocks.sql.ast.AlterTableStmt; +import com.starrocks.sql.plan.ConnectorPlanTestBase; +import com.starrocks.statistic.StatisticsMetaManager; +import com.starrocks.utframe.StarRocksAssert; +import com.starrocks.utframe.UtFrameUtils; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.text.SimpleDateFormat; +import java.util.Date; + + +public class ExternalCooldownMgrTest { + @ClassRule + public static TemporaryFolder temp = new TemporaryFolder(); + + private static ConnectContext connectContext; + private static StarRocksAssert starRocksAssert; + + protected static PseudoCluster cluster; + + protected static final String TEST_DB_NAME = "test"; + + @BeforeClass + public static void beforeClass() throws Exception { + UtFrameUtils.createMinStarRocksCluster(); + connectContext = UtFrameUtils.createDefaultCtx(); + starRocksAssert = new StarRocksAssert(connectContext); + + // set default config for async mvs + UtFrameUtils.setDefaultConfigForAsyncMVTest(connectContext); + + if (!starRocksAssert.databaseExist("_statistics_")) { + StatisticsMetaManager m = new StatisticsMetaManager(); + m.createStatisticsTablesForTest(); + } + starRocksAssert.withDatabase(TEST_DB_NAME); + starRocksAssert.useDatabase(TEST_DB_NAME); + + ConnectorPlanTestBase.mockCatalog(connectContext, MockIcebergMetadata.MOCKED_ICEBERG_CATALOG_NAME); + + starRocksAssert.withDatabase(TEST_DB_NAME).useDatabase(TEST_DB_NAME) + .withTable("CREATE TABLE tbl1\n" + + "(\n" + + " id int,\n" + + " ts datetime,\n" + + " data string\n" + + ")\n" + + "DUPLICATE KEY(`id`, `ts`)\n" + + "PARTITION BY RANGE(`ts`)\n" + + "(\n" + + " PARTITION p20200101 VALUES [('2020-01-01 00:00:00'),('2020-01-02 00:00:00')),\n" + + " PARTITION p20200102 VALUES [('2020-01-02 00:00:00'),('2020-01-03 00:00:00')),\n" + + " PARTITION p20200103 VALUES [('2020-01-03 00:00:00'),('2020-01-04 00:00:00')),\n" + + " PARTITION p20200104 VALUES [('2020-01-04 00:00:00'),('2020-01-05 00:00:00')),\n" + + " PARTITION p20200105 VALUES [('2020-01-05 00:00:00'),('2020-01-06 00:00:00'))\n" + + ")\n" + + "DISTRIBUTED BY HASH(`id`) BUCKETS 1\n" + + "PROPERTIES(\n" + + "'external_cooldown_target'='iceberg0.partitioned_transforms_db.t0_day',\n" + + "'external_cooldown_schedule'='START 22:00 END 08:00 EVERY INTERVAL 1 MINUTE',\n" + + "'external_cooldown_wait_second'='3600',\n" + + "'replication_num' = '1'\n" + + ");"); + } + + @Test + public void testReload() throws Exception { + ExternalCooldownMgr mgr = new ExternalCooldownMgr(); + mgr.doInitializeIfNeed(); + + Database testDb = GlobalStateMgr.getCurrentState().getLocalMetastore().getDb(TEST_DB_NAME); + OlapTable table = (OlapTable) GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(TEST_DB_NAME, "tbl1"); + mgr.prepareMaintenanceWork(testDb.getId(), table); + Assert.assertEquals(1, mgr.getRunnableJobs().size()); + + String sql = "ALTER TABLE tbl1 SET(\"external_cooldown_target\" = \"\");"; + ConnectContext ctx = starRocksAssert.getCtx(); + AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(sql, ctx); + GlobalStateMgr.getCurrentState().getLocalMetastore().alterTable(ctx, alterTableStmt); + + mgr.reloadJobs(); + Assert.assertEquals(0, mgr.getRunnableJobs().size()); + + long updateTime = System.currentTimeMillis() - 3600 * 1000; + Partition p1 = table.getPartition("p20200101"); + p1.updateVisibleVersion(p1.getVisibleVersion() + 1, updateTime); + + String sql1 = "ALTER TABLE tbl1 SET(\"external_cooldown_target\" = \"iceberg0.partitioned_transforms_db.t0_day\");"; + AlterTableStmt alterTableStmt1 = (AlterTableStmt) UtFrameUtils.parseStmtWithNewParser(sql1, ctx); + GlobalStateMgr.getCurrentState().getLocalMetastore().alterTable(ctx, alterTableStmt1); + + mgr.prepareMaintenanceWork(testDb.getId(), table); + ExternalCooldownMaintenanceJob job = mgr.getRunnableJobs().get(0); + SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd hh:mm:ss.SSS"); + Date parsedDate = dateFormat.parse("2024-10-13 23:50:00.000"); + job.onSchedule(parsedDate.getTime()); + mgr.stopMaintainExternalCooldown(table); + String lastRunTaskName = job.getLastRunTaskName(); + Assert.assertNotNull(lastRunTaskName); + TaskManager taskManager = GlobalStateMgr.getCurrentState().getTaskManager(); + Task task = taskManager.getTask(lastRunTaskName); + boolean isRunning = taskManager.getTaskRunScheduler().isTaskRunning(task.getId()); + Assert.assertFalse(isRunning); + } +} \ No newline at end of file diff --git a/fe/fe-core/src/test/java/com/starrocks/server/LocalMetaStoreTest.java b/fe/fe-core/src/test/java/com/starrocks/server/LocalMetaStoreTest.java index cd551d08a6e55..fbdac4e843155 100644 --- a/fe/fe-core/src/test/java/com/starrocks/server/LocalMetaStoreTest.java +++ b/fe/fe-core/src/test/java/com/starrocks/server/LocalMetaStoreTest.java @@ -41,6 +41,8 @@ import com.starrocks.common.util.UUIDUtil; import com.starrocks.common.util.concurrent.lock.LockType; import com.starrocks.common.util.concurrent.lock.Locker; +import com.starrocks.connector.iceberg.MockIcebergMetadata; +import com.starrocks.externalcooldown.ExternalCooldownConfig; import com.starrocks.persist.EditLog; import com.starrocks.persist.ModifyPartitionInfo; import com.starrocks.persist.PhysicalPartitionPersistInfoV2; @@ -49,6 +51,7 @@ import com.starrocks.persist.metablock.SRMetaBlockReaderV2; import com.starrocks.qe.ConnectContext; import com.starrocks.sql.ast.ColumnRenameClause; +import com.starrocks.sql.plan.ConnectorPlanTestBase; import com.starrocks.thrift.TStorageMedium; import com.starrocks.utframe.StarRocksAssert; import com.starrocks.utframe.UtFrameUtils; @@ -312,4 +315,24 @@ public void testRenameColumnException() throws Exception { } } + @Test + public void testAlterTableExternalCooldownProperties() throws Exception { + ConnectorPlanTestBase.mockCatalog(connectContext, MockIcebergMetadata.MOCKED_ICEBERG_CATALOG_NAME); + String icebergTable = "iceberg0.partitioned_transforms_db.t0_day"; + + Database db = connectContext.getGlobalStateMgr().getLocalMetastore().getDb("test"); + OlapTable table = (OlapTable) GlobalStateMgr.getCurrentState().getLocalMetastore().getTable(db.getFullName(), "t1"); + Map properties = Maps.newHashMap(); + properties.put(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_SCHEDULE, + "START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE"); + properties.put(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_TARGET, icebergTable); + properties.put(PropertyAnalyzer.PROPERTIES_EXTERNAL_COOLDOWN_WAIT_SECOND, "3600"); + LocalMetastore localMetastore = connectContext.getGlobalStateMgr().getLocalMetastore(); + localMetastore.alterTableProperties(db, table, properties); + ExternalCooldownConfig config = table.getCurExternalCoolDownConfig(); + Assert.assertNotNull(config); + Assert.assertEquals(config.getTarget(), icebergTable); + Assert.assertEquals("START 01:00 END 07:59 EVERY INTERVAL 1 MINUTE", config.getSchedule()); + Assert.assertEquals(config.getWaitSecond(), (Long) 3600L); + } } diff --git a/fe/fe-core/src/test/java/com/starrocks/sql/analyzer/CreateExternalCooldownStmtAnalyzerTest.java b/fe/fe-core/src/test/java/com/starrocks/sql/analyzer/CreateExternalCooldownStmtAnalyzerTest.java new file mode 100644 index 0000000000000..57ed08edb0999 --- /dev/null +++ b/fe/fe-core/src/test/java/com/starrocks/sql/analyzer/CreateExternalCooldownStmtAnalyzerTest.java @@ -0,0 +1,51 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed 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 +// +// https://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 com.starrocks.sql.analyzer; + +import com.starrocks.qe.ConnectContext; +import com.starrocks.sql.ast.CancelExternalCooldownStmt; +import com.starrocks.sql.ast.CreateExternalCooldownStmt; +import com.starrocks.sql.ast.UserIdentity; +import com.starrocks.utframe.StarRocksAssert; +import com.starrocks.utframe.UtFrameUtils; +import org.junit.BeforeClass; +import org.junit.Test; + +public class CreateExternalCooldownStmtAnalyzerTest { + private static StarRocksAssert starRocksAssert; + private static ConnectContext connectContext; + + @BeforeClass + public static void beforeClass() throws Exception { + UtFrameUtils.createMinStarRocksCluster(); + connectContext = UtFrameUtils.initCtxForNewPrivilege(UserIdentity.ROOT); + starRocksAssert = new StarRocksAssert(connectContext); + AnalyzeTestUtil.init(); + } + + @Test(expected = Throwable.class) + public void testCreateExternalCooldownStmt() { + CreateExternalCooldownStmt stmt = (CreateExternalCooldownStmt) com.starrocks.sql.parser.SqlParser.parse( + "COOLDOWN TABLE tbl1", 32).get(0); + ExternalCooldownAnalyzer.analyze(stmt, connectContext); + } + + @Test(expected = Throwable.class) + public void testCancelExternalCooldownStmt() { + CancelExternalCooldownStmt stmt = (CancelExternalCooldownStmt) com.starrocks.sql.parser.SqlParser.parse( + "CANCEL COOLDOWN tbl1", 32).get(0); + ExternalCooldownAnalyzer.analyze(stmt, connectContext); + } +} diff --git a/fe/fe-core/src/test/java/com/starrocks/sql/ast/CancelExternalCooldownStmtTest.java b/fe/fe-core/src/test/java/com/starrocks/sql/ast/CancelExternalCooldownStmtTest.java new file mode 100644 index 0000000000000..153a9e2c8fbaf --- /dev/null +++ b/fe/fe-core/src/test/java/com/starrocks/sql/ast/CancelExternalCooldownStmtTest.java @@ -0,0 +1,29 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed 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 +// +// https://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 com.starrocks.sql.ast; + +import org.junit.Assert; +import org.junit.Test; + +public class CancelExternalCooldownStmtTest { + @Test + public void testNormal() throws Exception { + String sql = "CANCEL COOLDOWN TABLE tbl"; + CancelExternalCooldownStmt stmt = (CancelExternalCooldownStmt) com.starrocks.sql.parser.SqlParser.parse( + sql, 32).get(0); + + Assert.assertNull(stmt.getTableName().getDb()); + Assert.assertEquals("tbl", stmt.getTableName().getTbl()); + } +} diff --git a/fe/fe-core/src/test/java/com/starrocks/sql/ast/CreateExternalCooldownStmtTest.java b/fe/fe-core/src/test/java/com/starrocks/sql/ast/CreateExternalCooldownStmtTest.java new file mode 100644 index 0000000000000..45e247af306f4 --- /dev/null +++ b/fe/fe-core/src/test/java/com/starrocks/sql/ast/CreateExternalCooldownStmtTest.java @@ -0,0 +1,62 @@ +// Copyright 2021-present StarRocks, Inc. All rights reserved. +// +// Licensed 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 +// +// https://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 com.starrocks.sql.ast; + +import com.starrocks.qe.ConnectContext; +import mockit.Mocked; +import org.junit.Assert; +import org.junit.Test; + +public class CreateExternalCooldownStmtTest { + @Mocked + private ConnectContext ctx; + + @Test + public void testNormal() throws Exception { + String sql = "COOLDOWN TABLE tbl"; + CreateExternalCooldownStmt stmt = (CreateExternalCooldownStmt) com.starrocks.sql.parser.SqlParser.parse( + sql, 32).get(0); + + Assert.assertNull(stmt.getTableName().getDb()); + Assert.assertEquals("tbl", stmt.getTableName().getTbl()); + Assert.assertNull(stmt.getPartitionRangeDesc()); + Assert.assertFalse(stmt.isForce()); + } + + @Test + public void testPartitionRange() { + String sql = "COOLDOWN TABLE tbl PARTITION START ('2022-04-18 00:00:00') END ('2022-04-19 00:00:00')"; + CreateExternalCooldownStmt stmt = (CreateExternalCooldownStmt) com.starrocks.sql.parser.SqlParser.parse( + sql, 32).get(0); + + Assert.assertNull(stmt.getTableName().getDb()); + Assert.assertEquals("tbl", stmt.getTableName().getTbl()); + Assert.assertNotNull(stmt.getPartitionRangeDesc()); + Assert.assertFalse(stmt.isForce()); + Assert.assertEquals("2022-04-18 00:00:00", stmt.getPartitionRangeDesc().getPartitionStart()); + Assert.assertEquals("2022-04-19 00:00:00", stmt.getPartitionRangeDesc().getPartitionEnd()); + } + + @Test + public void testForce() { + String sql = "COOLDOWN TABLE tbl FORCE"; + CreateExternalCooldownStmt stmt = (CreateExternalCooldownStmt) com.starrocks.sql.parser.SqlParser.parse( + sql, 32).get(0); + + Assert.assertNull(stmt.getTableName().getDb()); + Assert.assertEquals("tbl", stmt.getTableName().getTbl()); + Assert.assertNull(stmt.getPartitionRangeDesc()); + Assert.assertTrue(stmt.isForce()); + } +} diff --git a/gensrc/thrift/AgentService.thrift b/gensrc/thrift/AgentService.thrift index 50bca7f323a27..96b09718a3006 100644 --- a/gensrc/thrift/AgentService.thrift +++ b/gensrc/thrift/AgentService.thrift @@ -420,7 +420,8 @@ enum TTabletMetaType { STORAGE_TYPE, MUTABLE_BUCKET_NUM, ENABLE_LOAD_PROFILE, - BASE_COMPACTION_FORBIDDEN_TIME_RANGES + BASE_COMPACTION_FORBIDDEN_TIME_RANGES, + EXTERNAL_COOLDOWN_CONFIG } struct TTabletMetaInfo {