You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by GitBox <gi...@apache.org> on 2022/10/28 11:06:09 UTC

[GitHub] [doris] catpineapple opened a new pull request, #13772: [feature](planner) add multi partition

catpineapple opened a new pull request, #13772:
URL: https://github.com/apache/doris/pull/13772

   # Proposed changes
   
   Issue Number: close #13123 
   
   ## Problem summary
   
   Create partitions use :PARTITIONS START ("2022-11-01") END ("2022-12-01") EVERY (1 DAY) ,can create a month's date partitions in a batch.
   
   ## Checklist(Required)
   
   1. Does it affect the original behavior: 
       - [ ] Yes
       - [ ] No
       - [ ] I don't know
   2. Has unit tests been added:
       - [ ] Yes
       - [ ] No
       - [ ] No Need
   3. Has document been added or modified:
       - [ ] Yes
       - [ ] No
       - [ ] No Need
   4. Does it need to update dependencies:
       - [ ] Yes
       - [ ] No
   5. Are there any changes that cannot be rolled back:
       - [ ] Yes (If Yes, please explain WHY)
       - [ ] No
   
   ## Further comments
   
   If this is a relatively large or complex change, kick off the discussion at [dev@doris.apache.org](mailto:dev@doris.apache.org) by explaining why you chose the solution you did and what alternatives you considered, etc...
   
   


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

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

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


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


[GitHub] [doris] catpineapple commented on a diff in pull request #13772: [feature](planner) add multi partition

Posted by GitBox <gi...@apache.org>.
catpineapple commented on code in PR #13772:
URL: https://github.com/apache/doris/pull/13772#discussion_r1014976532


##########
fe/fe-core/src/main/java/org/apache/doris/planner/DateTools.java:
##########
@@ -0,0 +1,48 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.planner;
+
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoField;
+import java.time.temporal.TemporalAccessor;
+
+public class DateTools {
+    // format string DateTime  And Full Zero for hour,minute,second
+    public static LocalDateTime formatDateTimeAndFullZero(String datetime, DateTimeFormatter formatter) {

Review Comment:
   I will



##########
fe/fe-core/src/main/cup/sql_parser.cup:
##########
@@ -2927,6 +2938,22 @@ single_partition_desc ::=
     :}
     ;
 
+multi_partition_desc ::=
+    fixed_multi_partition_key_desc:desc
+        opt_key_value_map:properties
+    {:
+        RESULT = new MultiPartitionDesc(desc, properties);
+    :}
+    ;
+
+fixed_multi_partition_key_desc ::=
+    // FROM (lower) TO (upper) INTERVAL time_interval time_type
+    KW_FROM LPAREN partition_key_list:lower RPAREN KW_TO LPAREN partition_key_list:upper RPAREN KW_INTERVAL INTEGER_LITERAL:time_interval ident:time_type

Review Comment:
   I will



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

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

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


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


[GitHub] [doris] hello-stephen commented on pull request #13772: [feature](planner) add multi partition

Posted by GitBox <gi...@apache.org>.
hello-stephen commented on PR #13772:
URL: https://github.com/apache/doris/pull/13772#issuecomment-1294888684

   TeamCity pipeline, clickbench performance test result:
    the sum of best hot time: 39.07 seconds
    load time: 578 seconds
    storage size: 17154810728 Bytes
    https://doris-community-test-1308700295.cos.ap-hongkong.myqcloud.com/tmp/20221028113540_clickbench_pr_35376.html


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

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

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


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


[GitHub] [doris] morningman merged pull request #13772: [feature](planner) add multi partition

Posted by GitBox <gi...@apache.org>.
morningman merged PR #13772:
URL: https://github.com/apache/doris/pull/13772


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

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

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


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


[GitHub] [doris] catpineapple commented on a diff in pull request #13772: [feature](planner) add multi partition

Posted by GitBox <gi...@apache.org>.
catpineapple commented on code in PR #13772:
URL: https://github.com/apache/doris/pull/13772#discussion_r1014976699


##########
fe/fe-core/src/main/java/org/apache/doris/analysis/MultiPartitionDesc.java:
##########
@@ -0,0 +1,316 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.analysis;
+
+import org.apache.doris.analysis.TimestampArithmeticExpr.TimeUnit;
+import org.apache.doris.catalog.DynamicPartitionProperty;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.Config;
+import org.apache.doris.common.DdlException;
+import org.apache.doris.common.util.DynamicPartitionUtil;
+import org.apache.doris.planner.DateTools;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+
+import java.time.DayOfWeek;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoField;
+import java.time.temporal.WeekFields;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+// to describe the key list partition's information in create table stmt
+public class MultiPartitionDesc implements AllPartitionDesc {
+    public static final String HOURS_FORMAT = "yyyyMMddHH";
+    public static final String HOUR_FORMAT = "yyyy-MM-dd HH";
+    public static final String DATES_FORMAT = "yyyyMMdd";
+    public static final String DATE_FORMAT = "yyyy-MM-dd";
+    public static final String MONTHS_FORMAT = "yyyyMM";
+    public static final String MONTH_FORMAT = "yyyy-MM";
+    public static final String YEAR_FORMAT = "yyyy";
+    public static final String DATETIME_FORMAT = "yyyy-MM-dd HH:mm:ss";
+
+
+
+    private final String partitionPrefix = "p_";
+    private LocalDateTime startTime;
+    private LocalDateTime endTime;
+
+    private DateTimeFormatter startDateTimeFormat;
+    private DateTimeFormatter endDateTimeFormat;
+
+
+    private Long timeInterval;
+    private final PartitionKeyDesc partitionKeyDesc;
+    private TimestampArithmeticExpr.TimeUnit timeUnitType;
+    private final Map<String, String> properties;
+    private final List<SinglePartitionDesc> singlePartitionDescList = Lists.newArrayList();
+
+    private final ImmutableSet<TimestampArithmeticExpr.TimeUnit> timeUnitTypeMultiPartition = ImmutableSet.of(
+            TimestampArithmeticExpr.TimeUnit.HOUR,
+            TimestampArithmeticExpr.TimeUnit.DAY,
+            TimestampArithmeticExpr.TimeUnit.WEEK,
+            TimestampArithmeticExpr.TimeUnit.MONTH,
+            TimestampArithmeticExpr.TimeUnit.YEAR
+    );
+
+    private final Integer maxAllowedLimit = Config.max_multi_partition_num;
+
+    public MultiPartitionDesc(PartitionKeyDesc partitionKeyDesc,
+                          Map<String, String> properties) throws AnalysisException {
+        this.partitionKeyDesc = partitionKeyDesc;
+        this.properties = properties;
+        this.timeIntervalTrans();
+        this.timeTrans();
+    }
+
+    public List<SinglePartitionDesc> getSinglePartitionDescList() throws AnalysisException {
+        if (singlePartitionDescList.size() == 0) {
+            buildMultiPartitionToSinglePartitionDescs();
+        }
+        return singlePartitionDescList;
+    }
+
+    private List<SinglePartitionDesc> buildMultiPartitionToSinglePartitionDescs() throws AnalysisException {
+        String partitionName;
+        long countNum = 0;
+        int dayOfWeek = 1;
+        int dayOfMonth = 1;
+        String partitionPrefix = this.partitionPrefix;
+        LocalDateTime startTime = this.startTime;
+        if (properties != null) {
+            if (properties.containsKey(DynamicPartitionProperty.START_DAY_OF_WEEK)) {
+                String dayOfWeekStr = properties.get(DynamicPartitionProperty.START_DAY_OF_WEEK);
+                try {
+                    DynamicPartitionUtil.checkStartDayOfWeek(dayOfWeekStr);
+                } catch (DdlException e) {
+                    throw new AnalysisException(e.getMessage());
+                }
+                dayOfWeek = Integer.parseInt(dayOfWeekStr);
+            }
+            if (properties.containsKey(DynamicPartitionProperty.START_DAY_OF_MONTH)) {
+                String dayOfMonthStr = properties.get(DynamicPartitionProperty.START_DAY_OF_MONTH);
+                try {
+                    DynamicPartitionUtil.checkStartDayOfMonth(dayOfMonthStr);
+                } catch (DdlException e) {
+                    throw new AnalysisException(e.getMessage());
+                }
+                dayOfMonth = Integer.parseInt(dayOfMonthStr);
+            }
+
+            if (properties.containsKey(DynamicPartitionProperty.CREATE_HISTORY_PARTITION)) {
+                properties.put(DynamicPartitionProperty.CREATE_HISTORY_PARTITION, "false");
+            }
+            if (properties.containsKey(DynamicPartitionProperty.PREFIX)) {
+                partitionPrefix = properties.get(DynamicPartitionProperty.PREFIX);
+                try {
+                    DynamicPartitionUtil.checkPrefix(partitionPrefix);
+                } catch (DdlException e) {
+                    throw new AnalysisException(e.getMessage());
+                }
+            }
+        }
+        WeekFields weekFields = WeekFields.of(DayOfWeek.of(dayOfWeek), 1);
+        while (startTime.isBefore(this.endTime)) {
+            System.out.println("---" + startTime.format(dateTypeFormat()));
+            PartitionValue lowerPartitionValue = new PartitionValue(startTime.format(dateTypeFormat()));
+            switch (this.timeUnitType) {
+                case HOUR:
+                    partitionName = partitionPrefix + startTime.format(DateTimeFormatter.ofPattern(HOURS_FORMAT));
+                    startTime = startTime.plusHours(timeInterval);
+                    break;
+                case DAY:
+                    partitionName = partitionPrefix + startTime.format(DateTimeFormatter.ofPattern(DATES_FORMAT));
+                    startTime = startTime.plusDays(timeInterval);
+                    break;
+                case WEEK:
+                    LocalDate localDate = LocalDate.of(startTime.getYear(), startTime.getMonthValue(),
+                            startTime.getDayOfMonth());
+                    int weekOfYear = localDate.get(weekFields.weekOfYear());
+                    partitionName = String.format("%s%s_%02d", partitionPrefix,
+                            startTime.format(DateTimeFormatter.ofPattern(YEAR_FORMAT)), weekOfYear);
+                    startTime = startTime.with(ChronoField.DAY_OF_WEEK, dayOfMonth);
+                    startTime = startTime.plusWeeks(timeInterval);
+                    break;
+                case MONTH:
+                    partitionName = partitionPrefix + startTime.format(DateTimeFormatter.ofPattern(MONTHS_FORMAT));
+                    startTime = startTime.withDayOfMonth(dayOfMonth);
+                    startTime = startTime.plusMonths(timeInterval);
+                    break;
+                case YEAR:
+                    partitionName = partitionPrefix + startTime.format(DateTimeFormatter.ofPattern(YEAR_FORMAT));
+                    startTime = startTime.withDayOfYear(1);
+                    startTime = startTime.plusYears(timeInterval);
+                    break;
+                default:
+                    throw new AnalysisException("Multi build partition does not support time interval type: "
+                            + this.timeUnitType);
+            }
+            if (this.timeUnitType != TimestampArithmeticExpr.TimeUnit.DAY && startTime.isAfter(this.endTime)) {
+                startTime = this.endTime;
+            }
+            PartitionValue upperPartitionValue = new PartitionValue(startTime.format(dateTypeFormat()));
+            PartitionKeyDesc partitionKeyDesc = PartitionKeyDesc.createFixed(
+                    Lists.newArrayList(lowerPartitionValue),
+                    Lists.newArrayList(upperPartitionValue)
+            );
+            singlePartitionDescList.add(
+                new SinglePartitionDesc(
+                    false,
+                    partitionName,
+                    partitionKeyDesc,
+                    properties)
+            );
+
+            countNum++;
+            if (countNum > maxAllowedLimit) {
+                throw new AnalysisException("The number of Multi partitions too much, should not exceed:"
+                        + maxAllowedLimit);
+            }
+        }
+        return singlePartitionDescList;
+    }
+
+    private void timeTrans() throws AnalysisException {
+
+        if (partitionKeyDesc.getLowerValues().size() != 1 || partitionKeyDesc.getUpperValues().size() != 1) {
+            throw new AnalysisException("Multi build partition column size must be one "
+                    + "but START column size is " + partitionKeyDesc.getLowerValues().size()
+                    + ", END column size is " + partitionKeyDesc.getUpperValues().size() + ".");
+        }
+
+        String startString = partitionKeyDesc.getLowerValues().get(0).getStringValue();
+        String endString = partitionKeyDesc.getUpperValues().get(0).getStringValue();
+
+        try {
+            this.startDateTimeFormat = dateFormat(this.timeUnitType, startString);
+            this.endDateTimeFormat = dateFormat(this.timeUnitType, endString);
+            this.startTime = DateTools.formatDateTimeAndFullZero(startString, startDateTimeFormat);
+            this.endTime = DateTools.formatDateTimeAndFullZero(endString, endDateTimeFormat);
+        } catch (Exception e) {
+            throw new AnalysisException("Multi build partition START or END time style is illegal.");
+        }
+
+        if (!this.startTime.isBefore(this.endTime)) {
+            throw new AnalysisException("Multi build partition start time should less than end time.");
+        }
+    }
+
+
+    private void timeIntervalTrans() throws AnalysisException {
+        this.timeInterval = partitionKeyDesc.getTimeInterval();
+        String timeType = partitionKeyDesc.getTimeType();
+        if (timeType == null) {
+            throw new AnalysisException("Unknown time interval type for Multi build partition.");
+        }
+        if (this.timeInterval <= 0) {
+            throw new AnalysisException("Multi partition every clause mush be larger than zero.");

Review Comment:
   I will



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

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

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


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


[GitHub] [doris] morningman commented on a diff in pull request #13772: [feature](planner) add multi partition

Posted by GitBox <gi...@apache.org>.
morningman commented on code in PR #13772:
URL: https://github.com/apache/doris/pull/13772#discussion_r1011250020


##########
fe/fe-core/src/main/cup/sql_parser.cup:
##########
@@ -349,6 +349,7 @@ terminal String
     KW_ENTER,
     KW_ERRORS,
     KW_EVENTS,
+    KW_EVERY,

Review Comment:
   maybe we can reuse keyword `INTERVAL`? 
   And in MySQL, there are many stmt like `INTERVAL 1 DAY`, and I found in PG:
   https://www.postgresql.org/docs/current/ddl-partitioning.html
   
   we can refer as:
   ```
   PARTITION BY RANGE(date)
   FROM ("2022-10-11") TO ("2022-11-10") INTERVAL 1 DAY/MONTH/WEEK/YEAR
   
   PARTITION BY RANGE(int_col)
   FROM ("10") TO ("100") INTERVAL 10
   ```
   
   
   



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

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

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


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


[GitHub] [doris] catpineapple commented on a diff in pull request #13772: [feature](planner) add multi partition

Posted by GitBox <gi...@apache.org>.
catpineapple commented on code in PR #13772:
URL: https://github.com/apache/doris/pull/13772#discussion_r1011282923


##########
fe/fe-core/src/main/cup/sql_parser.cup:
##########
@@ -349,6 +349,7 @@ terminal String
     KW_ENTER,
     KW_ERRORS,
     KW_EVENTS,
+    KW_EVERY,

Review Comment:
   Reusing keywords is a good idea, I'll give it a try



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

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

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


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


[GitHub] [doris] catpineapple commented on pull request #13772: [feature](planner) add multi partition

Posted by GitBox <gi...@apache.org>.
catpineapple commented on PR #13772:
URL: https://github.com/apache/doris/pull/13772#issuecomment-1305299819

   > And please add some regression tests for this feature.
   
   


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

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

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


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


[GitHub] [doris] github-actions[bot] commented on pull request #13772: [feature](planner) add multi partition

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #13772:
URL: https://github.com/apache/doris/pull/13772#issuecomment-1311179786

   PR approved by anyone and no changes requested.


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

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

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


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


[GitHub] [doris] zy-kkk commented on pull request #13772: [feature](planner) add multi partition

Posted by GitBox <gi...@apache.org>.
zy-kkk commented on PR #13772:
URL: https://github.com/apache/doris/pull/13772#issuecomment-1294891251

   This is a cool new feature, and I think you should add a detailed documentation on how to use it


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

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

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


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


[GitHub] [doris] morningman commented on a diff in pull request #13772: [feature](planner) add multi partition

Posted by GitBox <gi...@apache.org>.
morningman commented on code in PR #13772:
URL: https://github.com/apache/doris/pull/13772#discussion_r1014645125


##########
fe/fe-core/src/main/java/org/apache/doris/analysis/MultiPartitionDesc.java:
##########
@@ -0,0 +1,316 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.analysis;
+
+import org.apache.doris.analysis.TimestampArithmeticExpr.TimeUnit;
+import org.apache.doris.catalog.DynamicPartitionProperty;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.Config;
+import org.apache.doris.common.DdlException;
+import org.apache.doris.common.util.DynamicPartitionUtil;
+import org.apache.doris.planner.DateTools;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+
+import java.time.DayOfWeek;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoField;
+import java.time.temporal.WeekFields;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+// to describe the key list partition's information in create table stmt
+public class MultiPartitionDesc implements AllPartitionDesc {
+    public static final String HOURS_FORMAT = "yyyyMMddHH";
+    public static final String HOUR_FORMAT = "yyyy-MM-dd HH";
+    public static final String DATES_FORMAT = "yyyyMMdd";
+    public static final String DATE_FORMAT = "yyyy-MM-dd";
+    public static final String MONTHS_FORMAT = "yyyyMM";
+    public static final String MONTH_FORMAT = "yyyy-MM";
+    public static final String YEAR_FORMAT = "yyyy";
+    public static final String DATETIME_FORMAT = "yyyy-MM-dd HH:mm:ss";
+
+
+
+    private final String partitionPrefix = "p_";
+    private LocalDateTime startTime;
+    private LocalDateTime endTime;
+
+    private DateTimeFormatter startDateTimeFormat;
+    private DateTimeFormatter endDateTimeFormat;
+
+
+    private Long timeInterval;
+    private final PartitionKeyDesc partitionKeyDesc;
+    private TimestampArithmeticExpr.TimeUnit timeUnitType;
+    private final Map<String, String> properties;
+    private final List<SinglePartitionDesc> singlePartitionDescList = Lists.newArrayList();
+
+    private final ImmutableSet<TimestampArithmeticExpr.TimeUnit> timeUnitTypeMultiPartition = ImmutableSet.of(
+            TimestampArithmeticExpr.TimeUnit.HOUR,
+            TimestampArithmeticExpr.TimeUnit.DAY,
+            TimestampArithmeticExpr.TimeUnit.WEEK,
+            TimestampArithmeticExpr.TimeUnit.MONTH,
+            TimestampArithmeticExpr.TimeUnit.YEAR
+    );
+
+    private final Integer maxAllowedLimit = Config.max_multi_partition_num;
+
+    public MultiPartitionDesc(PartitionKeyDesc partitionKeyDesc,
+                          Map<String, String> properties) throws AnalysisException {
+        this.partitionKeyDesc = partitionKeyDesc;
+        this.properties = properties;
+        this.timeIntervalTrans();
+        this.timeTrans();
+    }
+
+    public List<SinglePartitionDesc> getSinglePartitionDescList() throws AnalysisException {
+        if (singlePartitionDescList.size() == 0) {
+            buildMultiPartitionToSinglePartitionDescs();
+        }
+        return singlePartitionDescList;
+    }
+
+    private List<SinglePartitionDesc> buildMultiPartitionToSinglePartitionDescs() throws AnalysisException {
+        String partitionName;
+        long countNum = 0;
+        int dayOfWeek = 1;
+        int dayOfMonth = 1;
+        String partitionPrefix = this.partitionPrefix;
+        LocalDateTime startTime = this.startTime;
+        if (properties != null) {
+            if (properties.containsKey(DynamicPartitionProperty.START_DAY_OF_WEEK)) {
+                String dayOfWeekStr = properties.get(DynamicPartitionProperty.START_DAY_OF_WEEK);
+                try {
+                    DynamicPartitionUtil.checkStartDayOfWeek(dayOfWeekStr);
+                } catch (DdlException e) {
+                    throw new AnalysisException(e.getMessage());
+                }
+                dayOfWeek = Integer.parseInt(dayOfWeekStr);
+            }
+            if (properties.containsKey(DynamicPartitionProperty.START_DAY_OF_MONTH)) {
+                String dayOfMonthStr = properties.get(DynamicPartitionProperty.START_DAY_OF_MONTH);
+                try {
+                    DynamicPartitionUtil.checkStartDayOfMonth(dayOfMonthStr);

Review Comment:
   If there are some methods in `DynamicPartitionUtil` that can be used in this class,
   then I think these methods should be moved to a common class, may be `TimeUtils.java`



##########
fe/fe-core/src/main/java/org/apache/doris/analysis/MultiPartitionDesc.java:
##########
@@ -0,0 +1,316 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.analysis;
+
+import org.apache.doris.analysis.TimestampArithmeticExpr.TimeUnit;
+import org.apache.doris.catalog.DynamicPartitionProperty;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.Config;
+import org.apache.doris.common.DdlException;
+import org.apache.doris.common.util.DynamicPartitionUtil;
+import org.apache.doris.planner.DateTools;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+
+import java.time.DayOfWeek;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoField;
+import java.time.temporal.WeekFields;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+// to describe the key list partition's information in create table stmt
+public class MultiPartitionDesc implements AllPartitionDesc {
+    public static final String HOURS_FORMAT = "yyyyMMddHH";
+    public static final String HOUR_FORMAT = "yyyy-MM-dd HH";
+    public static final String DATES_FORMAT = "yyyyMMdd";
+    public static final String DATE_FORMAT = "yyyy-MM-dd";
+    public static final String MONTHS_FORMAT = "yyyyMM";
+    public static final String MONTH_FORMAT = "yyyy-MM";
+    public static final String YEAR_FORMAT = "yyyy";
+    public static final String DATETIME_FORMAT = "yyyy-MM-dd HH:mm:ss";
+
+
+
+    private final String partitionPrefix = "p_";
+    private LocalDateTime startTime;
+    private LocalDateTime endTime;
+
+    private DateTimeFormatter startDateTimeFormat;
+    private DateTimeFormatter endDateTimeFormat;
+
+
+    private Long timeInterval;
+    private final PartitionKeyDesc partitionKeyDesc;
+    private TimestampArithmeticExpr.TimeUnit timeUnitType;
+    private final Map<String, String> properties;
+    private final List<SinglePartitionDesc> singlePartitionDescList = Lists.newArrayList();
+
+    private final ImmutableSet<TimestampArithmeticExpr.TimeUnit> timeUnitTypeMultiPartition = ImmutableSet.of(
+            TimestampArithmeticExpr.TimeUnit.HOUR,
+            TimestampArithmeticExpr.TimeUnit.DAY,
+            TimestampArithmeticExpr.TimeUnit.WEEK,
+            TimestampArithmeticExpr.TimeUnit.MONTH,
+            TimestampArithmeticExpr.TimeUnit.YEAR
+    );
+
+    private final Integer maxAllowedLimit = Config.max_multi_partition_num;
+
+    public MultiPartitionDesc(PartitionKeyDesc partitionKeyDesc,
+                          Map<String, String> properties) throws AnalysisException {
+        this.partitionKeyDesc = partitionKeyDesc;
+        this.properties = properties;
+        this.timeIntervalTrans();
+        this.timeTrans();
+    }
+
+    public List<SinglePartitionDesc> getSinglePartitionDescList() throws AnalysisException {
+        if (singlePartitionDescList.size() == 0) {
+            buildMultiPartitionToSinglePartitionDescs();
+        }
+        return singlePartitionDescList;
+    }
+
+    private List<SinglePartitionDesc> buildMultiPartitionToSinglePartitionDescs() throws AnalysisException {
+        String partitionName;
+        long countNum = 0;
+        int dayOfWeek = 1;
+        int dayOfMonth = 1;
+        String partitionPrefix = this.partitionPrefix;
+        LocalDateTime startTime = this.startTime;
+        if (properties != null) {
+            if (properties.containsKey(DynamicPartitionProperty.START_DAY_OF_WEEK)) {
+                String dayOfWeekStr = properties.get(DynamicPartitionProperty.START_DAY_OF_WEEK);
+                try {
+                    DynamicPartitionUtil.checkStartDayOfWeek(dayOfWeekStr);
+                } catch (DdlException e) {
+                    throw new AnalysisException(e.getMessage());
+                }
+                dayOfWeek = Integer.parseInt(dayOfWeekStr);
+            }
+            if (properties.containsKey(DynamicPartitionProperty.START_DAY_OF_MONTH)) {
+                String dayOfMonthStr = properties.get(DynamicPartitionProperty.START_DAY_OF_MONTH);
+                try {
+                    DynamicPartitionUtil.checkStartDayOfMonth(dayOfMonthStr);
+                } catch (DdlException e) {
+                    throw new AnalysisException(e.getMessage());
+                }
+                dayOfMonth = Integer.parseInt(dayOfMonthStr);
+            }
+
+            if (properties.containsKey(DynamicPartitionProperty.CREATE_HISTORY_PARTITION)) {
+                properties.put(DynamicPartitionProperty.CREATE_HISTORY_PARTITION, "false");
+            }
+            if (properties.containsKey(DynamicPartitionProperty.PREFIX)) {
+                partitionPrefix = properties.get(DynamicPartitionProperty.PREFIX);
+                try {
+                    DynamicPartitionUtil.checkPrefix(partitionPrefix);
+                } catch (DdlException e) {
+                    throw new AnalysisException(e.getMessage());
+                }
+            }
+        }
+        WeekFields weekFields = WeekFields.of(DayOfWeek.of(dayOfWeek), 1);
+        while (startTime.isBefore(this.endTime)) {
+            System.out.println("---" + startTime.format(dateTypeFormat()));
+            PartitionValue lowerPartitionValue = new PartitionValue(startTime.format(dateTypeFormat()));
+            switch (this.timeUnitType) {
+                case HOUR:
+                    partitionName = partitionPrefix + startTime.format(DateTimeFormatter.ofPattern(HOURS_FORMAT));
+                    startTime = startTime.plusHours(timeInterval);
+                    break;
+                case DAY:
+                    partitionName = partitionPrefix + startTime.format(DateTimeFormatter.ofPattern(DATES_FORMAT));
+                    startTime = startTime.plusDays(timeInterval);
+                    break;
+                case WEEK:
+                    LocalDate localDate = LocalDate.of(startTime.getYear(), startTime.getMonthValue(),
+                            startTime.getDayOfMonth());
+                    int weekOfYear = localDate.get(weekFields.weekOfYear());
+                    partitionName = String.format("%s%s_%02d", partitionPrefix,
+                            startTime.format(DateTimeFormatter.ofPattern(YEAR_FORMAT)), weekOfYear);
+                    startTime = startTime.with(ChronoField.DAY_OF_WEEK, dayOfMonth);
+                    startTime = startTime.plusWeeks(timeInterval);
+                    break;
+                case MONTH:
+                    partitionName = partitionPrefix + startTime.format(DateTimeFormatter.ofPattern(MONTHS_FORMAT));
+                    startTime = startTime.withDayOfMonth(dayOfMonth);
+                    startTime = startTime.plusMonths(timeInterval);
+                    break;
+                case YEAR:
+                    partitionName = partitionPrefix + startTime.format(DateTimeFormatter.ofPattern(YEAR_FORMAT));
+                    startTime = startTime.withDayOfYear(1);
+                    startTime = startTime.plusYears(timeInterval);
+                    break;
+                default:
+                    throw new AnalysisException("Multi build partition does not support time interval type: "
+                            + this.timeUnitType);
+            }
+            if (this.timeUnitType != TimestampArithmeticExpr.TimeUnit.DAY && startTime.isAfter(this.endTime)) {
+                startTime = this.endTime;
+            }
+            PartitionValue upperPartitionValue = new PartitionValue(startTime.format(dateTypeFormat()));
+            PartitionKeyDesc partitionKeyDesc = PartitionKeyDesc.createFixed(
+                    Lists.newArrayList(lowerPartitionValue),
+                    Lists.newArrayList(upperPartitionValue)
+            );
+            singlePartitionDescList.add(
+                new SinglePartitionDesc(
+                    false,
+                    partitionName,
+                    partitionKeyDesc,
+                    properties)
+            );
+
+            countNum++;
+            if (countNum > maxAllowedLimit) {
+                throw new AnalysisException("The number of Multi partitions too much, should not exceed:"
+                        + maxAllowedLimit);
+            }
+        }
+        return singlePartitionDescList;
+    }
+
+    private void timeTrans() throws AnalysisException {
+
+        if (partitionKeyDesc.getLowerValues().size() != 1 || partitionKeyDesc.getUpperValues().size() != 1) {
+            throw new AnalysisException("Multi build partition column size must be one "
+                    + "but START column size is " + partitionKeyDesc.getLowerValues().size()
+                    + ", END column size is " + partitionKeyDesc.getUpperValues().size() + ".");
+        }
+
+        String startString = partitionKeyDesc.getLowerValues().get(0).getStringValue();
+        String endString = partitionKeyDesc.getUpperValues().get(0).getStringValue();
+
+        try {
+            this.startDateTimeFormat = dateFormat(this.timeUnitType, startString);
+            this.endDateTimeFormat = dateFormat(this.timeUnitType, endString);
+            this.startTime = DateTools.formatDateTimeAndFullZero(startString, startDateTimeFormat);
+            this.endTime = DateTools.formatDateTimeAndFullZero(endString, endDateTimeFormat);
+        } catch (Exception e) {
+            throw new AnalysisException("Multi build partition START or END time style is illegal.");
+        }
+
+        if (!this.startTime.isBefore(this.endTime)) {
+            throw new AnalysisException("Multi build partition start time should less than end time.");
+        }
+    }
+
+
+    private void timeIntervalTrans() throws AnalysisException {
+        this.timeInterval = partitionKeyDesc.getTimeInterval();
+        String timeType = partitionKeyDesc.getTimeType();
+        if (timeType == null) {
+            throw new AnalysisException("Unknown time interval type for Multi build partition.");
+        }
+        if (this.timeInterval <= 0) {
+            throw new AnalysisException("Multi partition every clause mush be larger than zero.");

Review Comment:
   ```suggestion
               throw new AnalysisException("Multi partition time interval mush be larger than zero.");
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/planner/DateTools.java:
##########
@@ -0,0 +1,48 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.planner;
+
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoField;
+import java.time.temporal.TemporalAccessor;
+
+public class DateTools {
+    // format string DateTime  And Full Zero for hour,minute,second
+    public static LocalDateTime formatDateTimeAndFullZero(String datetime, DateTimeFormatter formatter) {

Review Comment:
   You can put this to `TimeUtils.java`



##########
fe/fe-core/src/main/java/org/apache/doris/analysis/MultiPartitionDesc.java:
##########
@@ -0,0 +1,316 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.analysis;
+
+import org.apache.doris.analysis.TimestampArithmeticExpr.TimeUnit;
+import org.apache.doris.catalog.DynamicPartitionProperty;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.Config;
+import org.apache.doris.common.DdlException;
+import org.apache.doris.common.util.DynamicPartitionUtil;
+import org.apache.doris.planner.DateTools;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+
+import java.time.DayOfWeek;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoField;
+import java.time.temporal.WeekFields;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+// to describe the key list partition's information in create table stmt
+public class MultiPartitionDesc implements AllPartitionDesc {
+    public static final String HOURS_FORMAT = "yyyyMMddHH";
+    public static final String HOUR_FORMAT = "yyyy-MM-dd HH";
+    public static final String DATES_FORMAT = "yyyyMMdd";
+    public static final String DATE_FORMAT = "yyyy-MM-dd";
+    public static final String MONTHS_FORMAT = "yyyyMM";
+    public static final String MONTH_FORMAT = "yyyy-MM";
+    public static final String YEAR_FORMAT = "yyyy";
+    public static final String DATETIME_FORMAT = "yyyy-MM-dd HH:mm:ss";
+
+
+
+    private final String partitionPrefix = "p_";
+    private LocalDateTime startTime;
+    private LocalDateTime endTime;
+
+    private DateTimeFormatter startDateTimeFormat;
+    private DateTimeFormatter endDateTimeFormat;
+
+
+    private Long timeInterval;
+    private final PartitionKeyDesc partitionKeyDesc;
+    private TimestampArithmeticExpr.TimeUnit timeUnitType;
+    private final Map<String, String> properties;
+    private final List<SinglePartitionDesc> singlePartitionDescList = Lists.newArrayList();
+
+    private final ImmutableSet<TimestampArithmeticExpr.TimeUnit> timeUnitTypeMultiPartition = ImmutableSet.of(
+            TimestampArithmeticExpr.TimeUnit.HOUR,
+            TimestampArithmeticExpr.TimeUnit.DAY,
+            TimestampArithmeticExpr.TimeUnit.WEEK,
+            TimestampArithmeticExpr.TimeUnit.MONTH,
+            TimestampArithmeticExpr.TimeUnit.YEAR
+    );
+
+    private final Integer maxAllowedLimit = Config.max_multi_partition_num;
+
+    public MultiPartitionDesc(PartitionKeyDesc partitionKeyDesc,
+                          Map<String, String> properties) throws AnalysisException {
+        this.partitionKeyDesc = partitionKeyDesc;
+        this.properties = properties;
+        this.timeIntervalTrans();
+        this.timeTrans();
+    }
+
+    public List<SinglePartitionDesc> getSinglePartitionDescList() throws AnalysisException {
+        if (singlePartitionDescList.size() == 0) {
+            buildMultiPartitionToSinglePartitionDescs();
+        }
+        return singlePartitionDescList;
+    }
+
+    private List<SinglePartitionDesc> buildMultiPartitionToSinglePartitionDescs() throws AnalysisException {
+        String partitionName;
+        long countNum = 0;
+        int dayOfWeek = 1;
+        int dayOfMonth = 1;
+        String partitionPrefix = this.partitionPrefix;
+        LocalDateTime startTime = this.startTime;
+        if (properties != null) {
+            if (properties.containsKey(DynamicPartitionProperty.START_DAY_OF_WEEK)) {
+                String dayOfWeekStr = properties.get(DynamicPartitionProperty.START_DAY_OF_WEEK);
+                try {
+                    DynamicPartitionUtil.checkStartDayOfWeek(dayOfWeekStr);
+                } catch (DdlException e) {
+                    throw new AnalysisException(e.getMessage());
+                }
+                dayOfWeek = Integer.parseInt(dayOfWeekStr);
+            }
+            if (properties.containsKey(DynamicPartitionProperty.START_DAY_OF_MONTH)) {
+                String dayOfMonthStr = properties.get(DynamicPartitionProperty.START_DAY_OF_MONTH);
+                try {
+                    DynamicPartitionUtil.checkStartDayOfMonth(dayOfMonthStr);
+                } catch (DdlException e) {
+                    throw new AnalysisException(e.getMessage());
+                }
+                dayOfMonth = Integer.parseInt(dayOfMonthStr);
+            }
+
+            if (properties.containsKey(DynamicPartitionProperty.CREATE_HISTORY_PARTITION)) {
+                properties.put(DynamicPartitionProperty.CREATE_HISTORY_PARTITION, "false");
+            }
+            if (properties.containsKey(DynamicPartitionProperty.PREFIX)) {
+                partitionPrefix = properties.get(DynamicPartitionProperty.PREFIX);
+                try {
+                    DynamicPartitionUtil.checkPrefix(partitionPrefix);
+                } catch (DdlException e) {
+                    throw new AnalysisException(e.getMessage());
+                }
+            }
+        }
+        WeekFields weekFields = WeekFields.of(DayOfWeek.of(dayOfWeek), 1);
+        while (startTime.isBefore(this.endTime)) {
+            System.out.println("---" + startTime.format(dateTypeFormat()));
+            PartitionValue lowerPartitionValue = new PartitionValue(startTime.format(dateTypeFormat()));
+            switch (this.timeUnitType) {
+                case HOUR:
+                    partitionName = partitionPrefix + startTime.format(DateTimeFormatter.ofPattern(HOURS_FORMAT));
+                    startTime = startTime.plusHours(timeInterval);
+                    break;
+                case DAY:
+                    partitionName = partitionPrefix + startTime.format(DateTimeFormatter.ofPattern(DATES_FORMAT));
+                    startTime = startTime.plusDays(timeInterval);
+                    break;
+                case WEEK:
+                    LocalDate localDate = LocalDate.of(startTime.getYear(), startTime.getMonthValue(),
+                            startTime.getDayOfMonth());
+                    int weekOfYear = localDate.get(weekFields.weekOfYear());
+                    partitionName = String.format("%s%s_%02d", partitionPrefix,
+                            startTime.format(DateTimeFormatter.ofPattern(YEAR_FORMAT)), weekOfYear);
+                    startTime = startTime.with(ChronoField.DAY_OF_WEEK, dayOfMonth);
+                    startTime = startTime.plusWeeks(timeInterval);
+                    break;
+                case MONTH:
+                    partitionName = partitionPrefix + startTime.format(DateTimeFormatter.ofPattern(MONTHS_FORMAT));
+                    startTime = startTime.withDayOfMonth(dayOfMonth);
+                    startTime = startTime.plusMonths(timeInterval);
+                    break;
+                case YEAR:
+                    partitionName = partitionPrefix + startTime.format(DateTimeFormatter.ofPattern(YEAR_FORMAT));
+                    startTime = startTime.withDayOfYear(1);
+                    startTime = startTime.plusYears(timeInterval);
+                    break;
+                default:
+                    throw new AnalysisException("Multi build partition does not support time interval type: "
+                            + this.timeUnitType);
+            }
+            if (this.timeUnitType != TimestampArithmeticExpr.TimeUnit.DAY && startTime.isAfter(this.endTime)) {
+                startTime = this.endTime;
+            }
+            PartitionValue upperPartitionValue = new PartitionValue(startTime.format(dateTypeFormat()));
+            PartitionKeyDesc partitionKeyDesc = PartitionKeyDesc.createFixed(
+                    Lists.newArrayList(lowerPartitionValue),
+                    Lists.newArrayList(upperPartitionValue)
+            );
+            singlePartitionDescList.add(
+                new SinglePartitionDesc(
+                    false,
+                    partitionName,
+                    partitionKeyDesc,
+                    properties)
+            );
+
+            countNum++;
+            if (countNum > maxAllowedLimit) {
+                throw new AnalysisException("The number of Multi partitions too much, should not exceed:"
+                        + maxAllowedLimit);
+            }
+        }
+        return singlePartitionDescList;
+    }
+
+    private void timeTrans() throws AnalysisException {
+
+        if (partitionKeyDesc.getLowerValues().size() != 1 || partitionKeyDesc.getUpperValues().size() != 1) {
+            throw new AnalysisException("Multi build partition column size must be one "
+                    + "but START column size is " + partitionKeyDesc.getLowerValues().size()
+                    + ", END column size is " + partitionKeyDesc.getUpperValues().size() + ".");
+        }
+
+        String startString = partitionKeyDesc.getLowerValues().get(0).getStringValue();
+        String endString = partitionKeyDesc.getUpperValues().get(0).getStringValue();
+
+        try {
+            this.startDateTimeFormat = dateFormat(this.timeUnitType, startString);
+            this.endDateTimeFormat = dateFormat(this.timeUnitType, endString);
+            this.startTime = DateTools.formatDateTimeAndFullZero(startString, startDateTimeFormat);
+            this.endTime = DateTools.formatDateTimeAndFullZero(endString, endDateTimeFormat);
+        } catch (Exception e) {
+            throw new AnalysisException("Multi build partition START or END time style is illegal.");
+        }
+
+        if (!this.startTime.isBefore(this.endTime)) {
+            throw new AnalysisException("Multi build partition start time should less than end time.");
+        }
+    }
+
+
+    private void timeIntervalTrans() throws AnalysisException {
+        this.timeInterval = partitionKeyDesc.getTimeInterval();
+        String timeType = partitionKeyDesc.getTimeType();
+        if (timeType == null) {
+            throw new AnalysisException("Unknown time interval type for Multi build partition.");
+        }
+        if (this.timeInterval <= 0) {
+            throw new AnalysisException("Multi partition every clause mush be larger than zero.");
+        }
+        try {
+            this.timeUnitType = TimestampArithmeticExpr.TimeUnit.valueOf(timeType);
+        } catch (Exception e) {
+            throw new AnalysisException("Multi build partition got an unknow time interval type: "
+                    + timeType);
+        }
+        if (!timeUnitTypeMultiPartition.contains(this.timeUnitType)) {
+            throw new AnalysisException("Multi build partition does not support time interval type: "
+                    + this.timeUnitType);
+        }
+    }
+
+    private static DateTimeFormatter dateFormat(TimestampArithmeticExpr.TimeUnit timeUnitType,
+            String dateTimeStr) throws AnalysisException {
+        DateTimeFormatter res;
+        switch (timeUnitType) {
+            case HOUR:
+                if (dateTimeStr.length() == 10) {
+                    res = DateTimeFormatter.ofPattern(HOURS_FORMAT);
+                } else if (dateTimeStr.length() == 13) {
+                    res = DateTimeFormatter.ofPattern(HOUR_FORMAT);
+                } else if (dateTimeStr.length() == 19) {
+                    res = DateTimeFormatter.ofPattern(DATETIME_FORMAT);
+                } else {
+                    throw new AnalysisException("can not probe datetime(hour) format:" + dateTimeStr);
+                }
+                break;
+            case DAY: case WEEK:
+                if (dateTimeStr.length() == 8) {
+                    res = DateTimeFormatter.ofPattern(DATES_FORMAT);
+                } else if (dateTimeStr.length() == 10) {
+                    res = DateTimeFormatter.ofPattern(DATE_FORMAT);
+                } else if (dateTimeStr.length() == 19) {
+                    res = DateTimeFormatter.ofPattern(DATETIME_FORMAT);
+                } else {
+                    throw new AnalysisException("can not probe datetime(day or week) format:" + dateTimeStr);
+                }
+                break;
+            case MONTH:
+                if (dateTimeStr.length() == 6) {
+                    res = DateTimeFormatter.ofPattern(MONTHS_FORMAT);
+                } else if (dateTimeStr.length() == 7) {
+                    res = DateTimeFormatter.ofPattern(MONTH_FORMAT);
+                } else if (dateTimeStr.length() == 10) {
+                    res = DateTimeFormatter.ofPattern(DATE_FORMAT);
+                } else if (dateTimeStr.length() == 19) {
+                    res = DateTimeFormatter.ofPattern(DATETIME_FORMAT);
+                } else {
+                    throw new AnalysisException("can not probe datetime(month) format:" + dateTimeStr);
+                }
+                break;
+            case YEAR:
+                if (dateTimeStr.length() == 4) {
+                    res = DateTimeFormatter.ofPattern(YEAR_FORMAT);
+                } else if (dateTimeStr.length() == 8) {
+                    res = DateTimeFormatter.ofPattern(DATES_FORMAT);
+                } else if (dateTimeStr.length() == 10) {
+                    res = DateTimeFormatter.ofPattern(DATE_FORMAT);
+                } else if (dateTimeStr.length() == 19) {
+                    res = DateTimeFormatter.ofPattern(DATETIME_FORMAT);
+                } else {
+                    throw new AnalysisException("can not probe datetime(year) format:" + dateTimeStr);
+                }
+                break;
+            default:
+                throw new AnalysisException("Multi build partition does not support time interval type: "
+                        + timeUnitType);
+        }
+        return res;
+    }
+
+    private DateTimeFormatter dateTypeFormat() {
+        return DateTimeFormatter.ofPattern(this.timeUnitType.equals(TimeUnit.HOUR) ? DATETIME_FORMAT : DATE_FORMAT);
+    }
+
+
+    private List<AllPartitionDesc> aaa(int ii) throws AnalysisException {

Review Comment:
   what's this?



##########
fe/fe-core/src/main/cup/sql_parser.cup:
##########
@@ -2927,6 +2938,22 @@ single_partition_desc ::=
     :}
     ;
 
+multi_partition_desc ::=
+    fixed_multi_partition_key_desc:desc
+        opt_key_value_map:properties
+    {:
+        RESULT = new MultiPartitionDesc(desc, properties);
+    :}
+    ;
+
+fixed_multi_partition_key_desc ::=
+    // FROM (lower) TO (upper) INTERVAL time_interval time_type
+    KW_FROM LPAREN partition_key_list:lower RPAREN KW_TO LPAREN partition_key_list:upper RPAREN KW_INTERVAL INTEGER_LITERAL:time_interval ident:time_type

Review Comment:
   use `time_unit` instead of `ident:time_type`



##########
fe/fe-core/src/main/java/org/apache/doris/analysis/MultiPartitionDesc.java:
##########
@@ -0,0 +1,316 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.analysis;
+
+import org.apache.doris.analysis.TimestampArithmeticExpr.TimeUnit;
+import org.apache.doris.catalog.DynamicPartitionProperty;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.Config;
+import org.apache.doris.common.DdlException;
+import org.apache.doris.common.util.DynamicPartitionUtil;
+import org.apache.doris.planner.DateTools;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+
+import java.time.DayOfWeek;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoField;
+import java.time.temporal.WeekFields;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+// to describe the key list partition's information in create table stmt
+public class MultiPartitionDesc implements AllPartitionDesc {
+    public static final String HOURS_FORMAT = "yyyyMMddHH";
+    public static final String HOUR_FORMAT = "yyyy-MM-dd HH";
+    public static final String DATES_FORMAT = "yyyyMMdd";
+    public static final String DATE_FORMAT = "yyyy-MM-dd";
+    public static final String MONTHS_FORMAT = "yyyyMM";
+    public static final String MONTH_FORMAT = "yyyy-MM";
+    public static final String YEAR_FORMAT = "yyyy";
+    public static final String DATETIME_FORMAT = "yyyy-MM-dd HH:mm:ss";
+
+
+
+    private final String partitionPrefix = "p_";
+    private LocalDateTime startTime;
+    private LocalDateTime endTime;
+
+    private DateTimeFormatter startDateTimeFormat;
+    private DateTimeFormatter endDateTimeFormat;
+
+
+    private Long timeInterval;
+    private final PartitionKeyDesc partitionKeyDesc;
+    private TimestampArithmeticExpr.TimeUnit timeUnitType;
+    private final Map<String, String> properties;
+    private final List<SinglePartitionDesc> singlePartitionDescList = Lists.newArrayList();
+
+    private final ImmutableSet<TimestampArithmeticExpr.TimeUnit> timeUnitTypeMultiPartition = ImmutableSet.of(
+            TimestampArithmeticExpr.TimeUnit.HOUR,
+            TimestampArithmeticExpr.TimeUnit.DAY,
+            TimestampArithmeticExpr.TimeUnit.WEEK,
+            TimestampArithmeticExpr.TimeUnit.MONTH,
+            TimestampArithmeticExpr.TimeUnit.YEAR
+    );
+
+    private final Integer maxAllowedLimit = Config.max_multi_partition_num;
+
+    public MultiPartitionDesc(PartitionKeyDesc partitionKeyDesc,
+                          Map<String, String> properties) throws AnalysisException {
+        this.partitionKeyDesc = partitionKeyDesc;
+        this.properties = properties;
+        this.timeIntervalTrans();
+        this.timeTrans();
+    }
+
+    public List<SinglePartitionDesc> getSinglePartitionDescList() throws AnalysisException {
+        if (singlePartitionDescList.size() == 0) {
+            buildMultiPartitionToSinglePartitionDescs();
+        }
+        return singlePartitionDescList;
+    }
+
+    private List<SinglePartitionDesc> buildMultiPartitionToSinglePartitionDescs() throws AnalysisException {
+        String partitionName;
+        long countNum = 0;
+        int dayOfWeek = 1;
+        int dayOfMonth = 1;
+        String partitionPrefix = this.partitionPrefix;
+        LocalDateTime startTime = this.startTime;
+        if (properties != null) {
+            if (properties.containsKey(DynamicPartitionProperty.START_DAY_OF_WEEK)) {
+                String dayOfWeekStr = properties.get(DynamicPartitionProperty.START_DAY_OF_WEEK);
+                try {
+                    DynamicPartitionUtil.checkStartDayOfWeek(dayOfWeekStr);
+                } catch (DdlException e) {
+                    throw new AnalysisException(e.getMessage());
+                }
+                dayOfWeek = Integer.parseInt(dayOfWeekStr);
+            }
+            if (properties.containsKey(DynamicPartitionProperty.START_DAY_OF_MONTH)) {
+                String dayOfMonthStr = properties.get(DynamicPartitionProperty.START_DAY_OF_MONTH);
+                try {
+                    DynamicPartitionUtil.checkStartDayOfMonth(dayOfMonthStr);
+                } catch (DdlException e) {
+                    throw new AnalysisException(e.getMessage());
+                }
+                dayOfMonth = Integer.parseInt(dayOfMonthStr);
+            }
+
+            if (properties.containsKey(DynamicPartitionProperty.CREATE_HISTORY_PARTITION)) {
+                properties.put(DynamicPartitionProperty.CREATE_HISTORY_PARTITION, "false");
+            }
+            if (properties.containsKey(DynamicPartitionProperty.PREFIX)) {
+                partitionPrefix = properties.get(DynamicPartitionProperty.PREFIX);
+                try {
+                    DynamicPartitionUtil.checkPrefix(partitionPrefix);
+                } catch (DdlException e) {
+                    throw new AnalysisException(e.getMessage());
+                }
+            }
+        }
+        WeekFields weekFields = WeekFields.of(DayOfWeek.of(dayOfWeek), 1);
+        while (startTime.isBefore(this.endTime)) {
+            System.out.println("---" + startTime.format(dateTypeFormat()));
+            PartitionValue lowerPartitionValue = new PartitionValue(startTime.format(dateTypeFormat()));
+            switch (this.timeUnitType) {
+                case HOUR:
+                    partitionName = partitionPrefix + startTime.format(DateTimeFormatter.ofPattern(HOURS_FORMAT));
+                    startTime = startTime.plusHours(timeInterval);
+                    break;
+                case DAY:
+                    partitionName = partitionPrefix + startTime.format(DateTimeFormatter.ofPattern(DATES_FORMAT));
+                    startTime = startTime.plusDays(timeInterval);
+                    break;
+                case WEEK:
+                    LocalDate localDate = LocalDate.of(startTime.getYear(), startTime.getMonthValue(),
+                            startTime.getDayOfMonth());
+                    int weekOfYear = localDate.get(weekFields.weekOfYear());
+                    partitionName = String.format("%s%s_%02d", partitionPrefix,
+                            startTime.format(DateTimeFormatter.ofPattern(YEAR_FORMAT)), weekOfYear);
+                    startTime = startTime.with(ChronoField.DAY_OF_WEEK, dayOfMonth);
+                    startTime = startTime.plusWeeks(timeInterval);
+                    break;
+                case MONTH:
+                    partitionName = partitionPrefix + startTime.format(DateTimeFormatter.ofPattern(MONTHS_FORMAT));
+                    startTime = startTime.withDayOfMonth(dayOfMonth);
+                    startTime = startTime.plusMonths(timeInterval);
+                    break;
+                case YEAR:
+                    partitionName = partitionPrefix + startTime.format(DateTimeFormatter.ofPattern(YEAR_FORMAT));
+                    startTime = startTime.withDayOfYear(1);
+                    startTime = startTime.plusYears(timeInterval);
+                    break;
+                default:
+                    throw new AnalysisException("Multi build partition does not support time interval type: "
+                            + this.timeUnitType);
+            }
+            if (this.timeUnitType != TimestampArithmeticExpr.TimeUnit.DAY && startTime.isAfter(this.endTime)) {
+                startTime = this.endTime;
+            }
+            PartitionValue upperPartitionValue = new PartitionValue(startTime.format(dateTypeFormat()));
+            PartitionKeyDesc partitionKeyDesc = PartitionKeyDesc.createFixed(
+                    Lists.newArrayList(lowerPartitionValue),
+                    Lists.newArrayList(upperPartitionValue)
+            );
+            singlePartitionDescList.add(
+                new SinglePartitionDesc(
+                    false,
+                    partitionName,
+                    partitionKeyDesc,
+                    properties)
+            );
+
+            countNum++;
+            if (countNum > maxAllowedLimit) {
+                throw new AnalysisException("The number of Multi partitions too much, should not exceed:"
+                        + maxAllowedLimit);
+            }
+        }
+        return singlePartitionDescList;
+    }
+
+    private void timeTrans() throws AnalysisException {
+
+        if (partitionKeyDesc.getLowerValues().size() != 1 || partitionKeyDesc.getUpperValues().size() != 1) {
+            throw new AnalysisException("Multi build partition column size must be one "

Review Comment:
   ```suggestion
               throw new AnalysisException("partition column number in multi partition clause must be one "
   ```



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

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

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


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


[GitHub] [doris] github-actions[bot] commented on pull request #13772: [feature](planner) add multi partition

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #13772:
URL: https://github.com/apache/doris/pull/13772#issuecomment-1311179759

   PR approved by at least one committer and no changes requested.


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

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

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


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


[GitHub] [doris] morningman commented on a diff in pull request #13772: [feature](planner) add multi partition

Posted by GitBox <gi...@apache.org>.
morningman commented on code in PR #13772:
URL: https://github.com/apache/doris/pull/13772#discussion_r1011254752


##########
fe/fe-core/src/main/cup/sql_parser.cup:
##########
@@ -349,6 +349,7 @@ terminal String
     KW_ENTER,
     KW_ERRORS,
     KW_EVENTS,
+    KW_EVERY,

Review Comment:
   And I see the keyword `KW_MULTI`, but i didn't see examples.
   Could you please given more example about the new syntax.



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

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

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


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


[GitHub] [doris] morningman commented on pull request #13772: [feature](planner) add multi partition

Posted by GitBox <gi...@apache.org>.
morningman commented on PR #13772:
URL: https://github.com/apache/doris/pull/13772#issuecomment-1304560609

   And please add some regression tests for this feature.


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

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

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


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


[GitHub] [doris] catpineapple commented on a diff in pull request #13772: [feature](planner) add multi partition

Posted by GitBox <gi...@apache.org>.
catpineapple commented on code in PR #13772:
URL: https://github.com/apache/doris/pull/13772#discussion_r1014976484


##########
fe/fe-core/src/main/java/org/apache/doris/analysis/MultiPartitionDesc.java:
##########
@@ -0,0 +1,316 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.analysis;
+
+import org.apache.doris.analysis.TimestampArithmeticExpr.TimeUnit;
+import org.apache.doris.catalog.DynamicPartitionProperty;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.Config;
+import org.apache.doris.common.DdlException;
+import org.apache.doris.common.util.DynamicPartitionUtil;
+import org.apache.doris.planner.DateTools;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+
+import java.time.DayOfWeek;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoField;
+import java.time.temporal.WeekFields;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+// to describe the key list partition's information in create table stmt
+public class MultiPartitionDesc implements AllPartitionDesc {
+    public static final String HOURS_FORMAT = "yyyyMMddHH";
+    public static final String HOUR_FORMAT = "yyyy-MM-dd HH";
+    public static final String DATES_FORMAT = "yyyyMMdd";
+    public static final String DATE_FORMAT = "yyyy-MM-dd";
+    public static final String MONTHS_FORMAT = "yyyyMM";
+    public static final String MONTH_FORMAT = "yyyy-MM";
+    public static final String YEAR_FORMAT = "yyyy";
+    public static final String DATETIME_FORMAT = "yyyy-MM-dd HH:mm:ss";
+
+
+
+    private final String partitionPrefix = "p_";
+    private LocalDateTime startTime;
+    private LocalDateTime endTime;
+
+    private DateTimeFormatter startDateTimeFormat;
+    private DateTimeFormatter endDateTimeFormat;
+
+
+    private Long timeInterval;
+    private final PartitionKeyDesc partitionKeyDesc;
+    private TimestampArithmeticExpr.TimeUnit timeUnitType;
+    private final Map<String, String> properties;
+    private final List<SinglePartitionDesc> singlePartitionDescList = Lists.newArrayList();
+
+    private final ImmutableSet<TimestampArithmeticExpr.TimeUnit> timeUnitTypeMultiPartition = ImmutableSet.of(
+            TimestampArithmeticExpr.TimeUnit.HOUR,
+            TimestampArithmeticExpr.TimeUnit.DAY,
+            TimestampArithmeticExpr.TimeUnit.WEEK,
+            TimestampArithmeticExpr.TimeUnit.MONTH,
+            TimestampArithmeticExpr.TimeUnit.YEAR
+    );
+
+    private final Integer maxAllowedLimit = Config.max_multi_partition_num;
+
+    public MultiPartitionDesc(PartitionKeyDesc partitionKeyDesc,
+                          Map<String, String> properties) throws AnalysisException {
+        this.partitionKeyDesc = partitionKeyDesc;
+        this.properties = properties;
+        this.timeIntervalTrans();
+        this.timeTrans();
+    }
+
+    public List<SinglePartitionDesc> getSinglePartitionDescList() throws AnalysisException {
+        if (singlePartitionDescList.size() == 0) {
+            buildMultiPartitionToSinglePartitionDescs();
+        }
+        return singlePartitionDescList;
+    }
+
+    private List<SinglePartitionDesc> buildMultiPartitionToSinglePartitionDescs() throws AnalysisException {
+        String partitionName;
+        long countNum = 0;
+        int dayOfWeek = 1;
+        int dayOfMonth = 1;
+        String partitionPrefix = this.partitionPrefix;
+        LocalDateTime startTime = this.startTime;
+        if (properties != null) {
+            if (properties.containsKey(DynamicPartitionProperty.START_DAY_OF_WEEK)) {
+                String dayOfWeekStr = properties.get(DynamicPartitionProperty.START_DAY_OF_WEEK);
+                try {
+                    DynamicPartitionUtil.checkStartDayOfWeek(dayOfWeekStr);
+                } catch (DdlException e) {
+                    throw new AnalysisException(e.getMessage());
+                }
+                dayOfWeek = Integer.parseInt(dayOfWeekStr);
+            }
+            if (properties.containsKey(DynamicPartitionProperty.START_DAY_OF_MONTH)) {
+                String dayOfMonthStr = properties.get(DynamicPartitionProperty.START_DAY_OF_MONTH);
+                try {
+                    DynamicPartitionUtil.checkStartDayOfMonth(dayOfMonthStr);
+                } catch (DdlException e) {
+                    throw new AnalysisException(e.getMessage());
+                }
+                dayOfMonth = Integer.parseInt(dayOfMonthStr);
+            }
+
+            if (properties.containsKey(DynamicPartitionProperty.CREATE_HISTORY_PARTITION)) {
+                properties.put(DynamicPartitionProperty.CREATE_HISTORY_PARTITION, "false");
+            }
+            if (properties.containsKey(DynamicPartitionProperty.PREFIX)) {
+                partitionPrefix = properties.get(DynamicPartitionProperty.PREFIX);
+                try {
+                    DynamicPartitionUtil.checkPrefix(partitionPrefix);
+                } catch (DdlException e) {
+                    throw new AnalysisException(e.getMessage());
+                }
+            }
+        }
+        WeekFields weekFields = WeekFields.of(DayOfWeek.of(dayOfWeek), 1);
+        while (startTime.isBefore(this.endTime)) {
+            System.out.println("---" + startTime.format(dateTypeFormat()));
+            PartitionValue lowerPartitionValue = new PartitionValue(startTime.format(dateTypeFormat()));
+            switch (this.timeUnitType) {
+                case HOUR:
+                    partitionName = partitionPrefix + startTime.format(DateTimeFormatter.ofPattern(HOURS_FORMAT));
+                    startTime = startTime.plusHours(timeInterval);
+                    break;
+                case DAY:
+                    partitionName = partitionPrefix + startTime.format(DateTimeFormatter.ofPattern(DATES_FORMAT));
+                    startTime = startTime.plusDays(timeInterval);
+                    break;
+                case WEEK:
+                    LocalDate localDate = LocalDate.of(startTime.getYear(), startTime.getMonthValue(),
+                            startTime.getDayOfMonth());
+                    int weekOfYear = localDate.get(weekFields.weekOfYear());
+                    partitionName = String.format("%s%s_%02d", partitionPrefix,
+                            startTime.format(DateTimeFormatter.ofPattern(YEAR_FORMAT)), weekOfYear);
+                    startTime = startTime.with(ChronoField.DAY_OF_WEEK, dayOfMonth);
+                    startTime = startTime.plusWeeks(timeInterval);
+                    break;
+                case MONTH:
+                    partitionName = partitionPrefix + startTime.format(DateTimeFormatter.ofPattern(MONTHS_FORMAT));
+                    startTime = startTime.withDayOfMonth(dayOfMonth);
+                    startTime = startTime.plusMonths(timeInterval);
+                    break;
+                case YEAR:
+                    partitionName = partitionPrefix + startTime.format(DateTimeFormatter.ofPattern(YEAR_FORMAT));
+                    startTime = startTime.withDayOfYear(1);
+                    startTime = startTime.plusYears(timeInterval);
+                    break;
+                default:
+                    throw new AnalysisException("Multi build partition does not support time interval type: "
+                            + this.timeUnitType);
+            }
+            if (this.timeUnitType != TimestampArithmeticExpr.TimeUnit.DAY && startTime.isAfter(this.endTime)) {
+                startTime = this.endTime;
+            }
+            PartitionValue upperPartitionValue = new PartitionValue(startTime.format(dateTypeFormat()));
+            PartitionKeyDesc partitionKeyDesc = PartitionKeyDesc.createFixed(
+                    Lists.newArrayList(lowerPartitionValue),
+                    Lists.newArrayList(upperPartitionValue)
+            );
+            singlePartitionDescList.add(
+                new SinglePartitionDesc(
+                    false,
+                    partitionName,
+                    partitionKeyDesc,
+                    properties)
+            );
+
+            countNum++;
+            if (countNum > maxAllowedLimit) {
+                throw new AnalysisException("The number of Multi partitions too much, should not exceed:"
+                        + maxAllowedLimit);
+            }
+        }
+        return singlePartitionDescList;
+    }
+
+    private void timeTrans() throws AnalysisException {
+
+        if (partitionKeyDesc.getLowerValues().size() != 1 || partitionKeyDesc.getUpperValues().size() != 1) {
+            throw new AnalysisException("Multi build partition column size must be one "
+                    + "but START column size is " + partitionKeyDesc.getLowerValues().size()
+                    + ", END column size is " + partitionKeyDesc.getUpperValues().size() + ".");
+        }
+
+        String startString = partitionKeyDesc.getLowerValues().get(0).getStringValue();
+        String endString = partitionKeyDesc.getUpperValues().get(0).getStringValue();
+
+        try {
+            this.startDateTimeFormat = dateFormat(this.timeUnitType, startString);
+            this.endDateTimeFormat = dateFormat(this.timeUnitType, endString);
+            this.startTime = DateTools.formatDateTimeAndFullZero(startString, startDateTimeFormat);
+            this.endTime = DateTools.formatDateTimeAndFullZero(endString, endDateTimeFormat);
+        } catch (Exception e) {
+            throw new AnalysisException("Multi build partition START or END time style is illegal.");
+        }
+
+        if (!this.startTime.isBefore(this.endTime)) {
+            throw new AnalysisException("Multi build partition start time should less than end time.");
+        }
+    }
+
+
+    private void timeIntervalTrans() throws AnalysisException {
+        this.timeInterval = partitionKeyDesc.getTimeInterval();
+        String timeType = partitionKeyDesc.getTimeType();
+        if (timeType == null) {
+            throw new AnalysisException("Unknown time interval type for Multi build partition.");
+        }
+        if (this.timeInterval <= 0) {
+            throw new AnalysisException("Multi partition every clause mush be larger than zero.");
+        }
+        try {
+            this.timeUnitType = TimestampArithmeticExpr.TimeUnit.valueOf(timeType);
+        } catch (Exception e) {
+            throw new AnalysisException("Multi build partition got an unknow time interval type: "
+                    + timeType);
+        }
+        if (!timeUnitTypeMultiPartition.contains(this.timeUnitType)) {
+            throw new AnalysisException("Multi build partition does not support time interval type: "
+                    + this.timeUnitType);
+        }
+    }
+
+    private static DateTimeFormatter dateFormat(TimestampArithmeticExpr.TimeUnit timeUnitType,
+            String dateTimeStr) throws AnalysisException {
+        DateTimeFormatter res;
+        switch (timeUnitType) {
+            case HOUR:
+                if (dateTimeStr.length() == 10) {
+                    res = DateTimeFormatter.ofPattern(HOURS_FORMAT);
+                } else if (dateTimeStr.length() == 13) {
+                    res = DateTimeFormatter.ofPattern(HOUR_FORMAT);
+                } else if (dateTimeStr.length() == 19) {
+                    res = DateTimeFormatter.ofPattern(DATETIME_FORMAT);
+                } else {
+                    throw new AnalysisException("can not probe datetime(hour) format:" + dateTimeStr);
+                }
+                break;
+            case DAY: case WEEK:
+                if (dateTimeStr.length() == 8) {
+                    res = DateTimeFormatter.ofPattern(DATES_FORMAT);
+                } else if (dateTimeStr.length() == 10) {
+                    res = DateTimeFormatter.ofPattern(DATE_FORMAT);
+                } else if (dateTimeStr.length() == 19) {
+                    res = DateTimeFormatter.ofPattern(DATETIME_FORMAT);
+                } else {
+                    throw new AnalysisException("can not probe datetime(day or week) format:" + dateTimeStr);
+                }
+                break;
+            case MONTH:
+                if (dateTimeStr.length() == 6) {
+                    res = DateTimeFormatter.ofPattern(MONTHS_FORMAT);
+                } else if (dateTimeStr.length() == 7) {
+                    res = DateTimeFormatter.ofPattern(MONTH_FORMAT);
+                } else if (dateTimeStr.length() == 10) {
+                    res = DateTimeFormatter.ofPattern(DATE_FORMAT);
+                } else if (dateTimeStr.length() == 19) {
+                    res = DateTimeFormatter.ofPattern(DATETIME_FORMAT);
+                } else {
+                    throw new AnalysisException("can not probe datetime(month) format:" + dateTimeStr);
+                }
+                break;
+            case YEAR:
+                if (dateTimeStr.length() == 4) {
+                    res = DateTimeFormatter.ofPattern(YEAR_FORMAT);
+                } else if (dateTimeStr.length() == 8) {
+                    res = DateTimeFormatter.ofPattern(DATES_FORMAT);
+                } else if (dateTimeStr.length() == 10) {
+                    res = DateTimeFormatter.ofPattern(DATE_FORMAT);
+                } else if (dateTimeStr.length() == 19) {
+                    res = DateTimeFormatter.ofPattern(DATETIME_FORMAT);
+                } else {
+                    throw new AnalysisException("can not probe datetime(year) format:" + dateTimeStr);
+                }
+                break;
+            default:
+                throw new AnalysisException("Multi build partition does not support time interval type: "
+                        + timeUnitType);
+        }
+        return res;
+    }
+
+    private DateTimeFormatter dateTypeFormat() {
+        return DateTimeFormatter.ofPattern(this.timeUnitType.equals(TimeUnit.HOUR) ? DATETIME_FORMAT : DATE_FORMAT);
+    }
+
+
+    private List<AllPartitionDesc> aaa(int ii) throws AnalysisException {

Review Comment:
   sorry ,my fault,forgot to delete before submitting.



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

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

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


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


[GitHub] [doris] catpineapple commented on a diff in pull request #13772: [feature](planner) add multi partition

Posted by GitBox <gi...@apache.org>.
catpineapple commented on code in PR #13772:
URL: https://github.com/apache/doris/pull/13772#discussion_r1014970740


##########
fe/fe-core/src/main/java/org/apache/doris/analysis/MultiPartitionDesc.java:
##########
@@ -0,0 +1,316 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package org.apache.doris.analysis;
+
+import org.apache.doris.analysis.TimestampArithmeticExpr.TimeUnit;
+import org.apache.doris.catalog.DynamicPartitionProperty;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.Config;
+import org.apache.doris.common.DdlException;
+import org.apache.doris.common.util.DynamicPartitionUtil;
+import org.apache.doris.planner.DateTools;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+
+import java.time.DayOfWeek;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
+import java.time.temporal.ChronoField;
+import java.time.temporal.WeekFields;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+// to describe the key list partition's information in create table stmt
+public class MultiPartitionDesc implements AllPartitionDesc {
+    public static final String HOURS_FORMAT = "yyyyMMddHH";
+    public static final String HOUR_FORMAT = "yyyy-MM-dd HH";
+    public static final String DATES_FORMAT = "yyyyMMdd";
+    public static final String DATE_FORMAT = "yyyy-MM-dd";
+    public static final String MONTHS_FORMAT = "yyyyMM";
+    public static final String MONTH_FORMAT = "yyyy-MM";
+    public static final String YEAR_FORMAT = "yyyy";
+    public static final String DATETIME_FORMAT = "yyyy-MM-dd HH:mm:ss";
+
+
+
+    private final String partitionPrefix = "p_";
+    private LocalDateTime startTime;
+    private LocalDateTime endTime;
+
+    private DateTimeFormatter startDateTimeFormat;
+    private DateTimeFormatter endDateTimeFormat;
+
+
+    private Long timeInterval;
+    private final PartitionKeyDesc partitionKeyDesc;
+    private TimestampArithmeticExpr.TimeUnit timeUnitType;
+    private final Map<String, String> properties;
+    private final List<SinglePartitionDesc> singlePartitionDescList = Lists.newArrayList();
+
+    private final ImmutableSet<TimestampArithmeticExpr.TimeUnit> timeUnitTypeMultiPartition = ImmutableSet.of(
+            TimestampArithmeticExpr.TimeUnit.HOUR,
+            TimestampArithmeticExpr.TimeUnit.DAY,
+            TimestampArithmeticExpr.TimeUnit.WEEK,
+            TimestampArithmeticExpr.TimeUnit.MONTH,
+            TimestampArithmeticExpr.TimeUnit.YEAR
+    );
+
+    private final Integer maxAllowedLimit = Config.max_multi_partition_num;
+
+    public MultiPartitionDesc(PartitionKeyDesc partitionKeyDesc,
+                          Map<String, String> properties) throws AnalysisException {
+        this.partitionKeyDesc = partitionKeyDesc;
+        this.properties = properties;
+        this.timeIntervalTrans();
+        this.timeTrans();
+    }
+
+    public List<SinglePartitionDesc> getSinglePartitionDescList() throws AnalysisException {
+        if (singlePartitionDescList.size() == 0) {
+            buildMultiPartitionToSinglePartitionDescs();
+        }
+        return singlePartitionDescList;
+    }
+
+    private List<SinglePartitionDesc> buildMultiPartitionToSinglePartitionDescs() throws AnalysisException {
+        String partitionName;
+        long countNum = 0;
+        int dayOfWeek = 1;
+        int dayOfMonth = 1;
+        String partitionPrefix = this.partitionPrefix;
+        LocalDateTime startTime = this.startTime;
+        if (properties != null) {
+            if (properties.containsKey(DynamicPartitionProperty.START_DAY_OF_WEEK)) {
+                String dayOfWeekStr = properties.get(DynamicPartitionProperty.START_DAY_OF_WEEK);
+                try {
+                    DynamicPartitionUtil.checkStartDayOfWeek(dayOfWeekStr);
+                } catch (DdlException e) {
+                    throw new AnalysisException(e.getMessage());
+                }
+                dayOfWeek = Integer.parseInt(dayOfWeekStr);
+            }
+            if (properties.containsKey(DynamicPartitionProperty.START_DAY_OF_MONTH)) {
+                String dayOfMonthStr = properties.get(DynamicPartitionProperty.START_DAY_OF_MONTH);
+                try {
+                    DynamicPartitionUtil.checkStartDayOfMonth(dayOfMonthStr);

Review Comment:
   This piece of code is to solve the conflict between dynamic partitioning and multi partitioning at one create table sql. It is reasonable to refer to the params/methods  of dynamic partitions for judgment. These params/methods are still belong to dynamic partitions essentially, so I think they should  not be moved.



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

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

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


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


[GitHub] [doris] catpineapple closed pull request #13772: [feature](planner) add multi partition

Posted by GitBox <gi...@apache.org>.
catpineapple closed pull request #13772: [feature](planner) add multi partition 
URL: https://github.com/apache/doris/pull/13772


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

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

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


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