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

[doris] branch branch-1.2-lts updated (b536cdf5e9 -> e873398511)

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

morningman pushed a change to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git


    from b536cdf5e9 [chore](third-party) Bump the version of hadoop_libs (#20368)
     new ccbd791cf6 [Bug](materialized-view) forbid create mv/rollup on mow table (#20001)
     new 6c92931ead [security] Don't print password in BaseController (#18862)
     new 48ed4421b4 [fix](fe)ordering exprs should be substituted in the same way as select part (#20091)
     new c82875aec1 [fix](dynamic_partition) fix dynamic partition not work when drop and  recover olap table (#19031)
     new dc981cc55e [Bug](function) fix equals implements not judge order by elements of function call expr (#20083)
     new edf982e26b [BUG]storage_min_left_capacity_bytes default value has integer overflow #19943
     new 93698651b3 [Fix](dynamic-partition) Try to avoid setting a zero-bucket-size partition. (#20177)
     new 7b36e719b5 [fix](dynamic_partition) fix dead lock when modify dynamic partition property for olap table (#20390)
     new e873398511 [chore](macOS) Fix JAVA_OPTS in start_be.sh (#19267)

The 9 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 bin/start_be.sh                                    | 10 ++++-
 conf/asan_suppr.conf                               |  2 +-
 .../main/java/org/apache/doris/common/Config.java  |  2 +-
 .../doris/alter/MaterializedViewHandler.java       |  6 +++
 .../apache/doris/analysis/FunctionCallExpr.java    | 22 ++++++++--
 .../java/org/apache/doris/analysis/SelectStmt.java |  4 ++
 .../apache/doris/catalog/CatalogRecycleBin.java    |  2 +
 .../doris/clone/DynamicPartitionScheduler.java     | 11 ++++-
 .../doris/common/util/DynamicPartitionUtil.java    |  6 ---
 .../apache/doris/datasource/InternalCatalog.java   |  8 ++++
 .../doris/httpv2/controller/BaseController.java    |  2 +-
 .../java/org/apache/doris/alter/AlterTest.java     |  2 +-
 .../doris/catalog/DynamicPartitionTableTest.java   |  2 +
 .../java/org/apache/doris/catalog/RecoverTest.java | 47 ++++++++++++++++++++++
 .../doris/nereids/rules/mv/SelectMvIndexTest.java  |  2 +-
 .../planner/MaterializedViewFunctionTest.java      |  2 +-
 .../correctness_p0/test_grouping_with_alias.out    |  3 ++
 .../query_p0/group_concat/test_group_concat.out    |  4 ++
 .../correctness_p0/test_grouping_with_alias.groovy |  4 +-
 .../test_mv_useless/mow_invalid.groovy}            | 36 ++++++++---------
 .../query_p0/group_concat/test_group_concat.groovy |  4 ++
 .../test_uniq_mv_schema_change.groovy              |  4 +-
 .../test_uniq_rollup_schema_change.groovy          |  2 +-
 23 files changed, 145 insertions(+), 42 deletions(-)
 copy regression-test/suites/{query_p0/limit/OffsetInSubqueryWithJoin.groovy => materialized_view_p0/test_mv_useless/mow_invalid.groovy} (58%)


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


[doris] 09/09: [chore](macOS) Fix JAVA_OPTS in start_be.sh (#19267)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit e87339851134918dd31940feea9094e79e2b05c1
Author: Adonis Ling <ad...@gmail.com>
AuthorDate: Mon May 8 14:01:10 2023 +0800

    [chore](macOS) Fix JAVA_OPTS in start_be.sh (#19267)
    
    We should set -XX:-MaxFDLimit on macOS if we enable java support for BE otherwise BE may fail to start up.
---
 bin/start_be.sh | 10 +++++++++-
 1 file changed, 9 insertions(+), 1 deletion(-)

diff --git a/bin/start_be.sh b/bin/start_be.sh
index cb37a54cf8..b36c1569a4 100755
--- a/bin/start_be.sh
+++ b/bin/start_be.sh
@@ -286,7 +286,15 @@ else
 fi
 
 if [[ "${MACHINE_OS}" == "Darwin" ]]; then
-    final_java_opt="${final_java_opt} -XX:-MaxFDLimit"
+    max_fd_limit='-XX:-MaxFDLimit'
+
+    if ! echo "${final_java_opt}" | grep "${max_fd_limit/-/\-}" >/dev/null; then
+        final_java_opt="${final_java_opt} ${max_fd_limit}"
+    fi
+
+    if [[ -n "${JAVA_OPTS}" ]] && ! echo "${JAVA_OPTS}" | grep "${max_fd_limit/-/\-}" >/dev/null; then
+        JAVA_OPTS="${JAVA_OPTS} ${max_fd_limit}"
+    fi
 fi
 
 # set LIBHDFS_OPTS for hadoop libhdfs


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


[doris] 06/09: [BUG]storage_min_left_capacity_bytes default value has integer overflow #19943

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit edf982e26b4a3f654010afb6fa0bd4d4f6b3ae10
Author: Long Zhao <29...@qq.com>
AuthorDate: Mon May 29 19:50:31 2023 +0800

    [BUG]storage_min_left_capacity_bytes default value has integer overflow #19943
---
 fe/fe-common/src/main/java/org/apache/doris/common/Config.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java
index 6d561df1ce..796c5eca0f 100644
--- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java
+++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java
@@ -1008,7 +1008,7 @@ public class Config extends ConfigBase {
     @ConfField(mutable = true, masterOnly = true)
     public static int storage_high_watermark_usage_percent = 85;
     @ConfField(mutable = true, masterOnly = true)
-    public static long storage_min_left_capacity_bytes = 2 * 1024 * 1024 * 1024; // 2G
+    public static long storage_min_left_capacity_bytes = 2 * 1024 * 1024 * 1024L; // 2G
 
     /**
      * If capacity of disk reach the 'storage_flood_stage_usage_percent' and 'storage_flood_stage_left_capacity_bytes',


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


[doris] 03/09: [fix](fe)ordering exprs should be substituted in the same way as select part (#20091)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 48ed4421b4103061e0bb648f4e083c1d633cbaf3
Author: starocean999 <40...@users.noreply.github.com>
AuthorDate: Sat May 27 21:00:57 2023 +0800

    [fix](fe)ordering exprs should be substituted in the same way as select part (#20091)
---
 fe/fe-core/src/main/java/org/apache/doris/analysis/SelectStmt.java    | 4 ++++
 regression-test/data/correctness_p0/test_grouping_with_alias.out      | 3 +++
 regression-test/suites/correctness_p0/test_grouping_with_alias.groovy | 4 +++-
 3 files changed, 10 insertions(+), 1 deletion(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/SelectStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/SelectStmt.java
index 7fe993ae9a..acae7147aa 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SelectStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/SelectStmt.java
@@ -1162,6 +1162,10 @@ public class SelectStmt extends QueryStmt {
             havingClauseAfterAnaylzed =
                     havingClauseAfterAnaylzed.substitute(countAllMap, analyzer, false);
         }
+        if (sortInfo != null) {
+            // the ordering exprs must substitute in the same way as resultExprs
+            sortInfo.substituteOrderingExprs(countAllMap, analyzer);
+        }
         aggExprs.clear();
         TreeNode.collect(substitutedAggs, Expr.isAggregatePredicate(), aggExprs);
 
diff --git a/regression-test/data/correctness_p0/test_grouping_with_alias.out b/regression-test/data/correctness_p0/test_grouping_with_alias.out
index 850b4daf7b..11c59c5041 100644
--- a/regression-test/data/correctness_p0/test_grouping_with_alias.out
+++ b/regression-test/data/correctness_p0/test_grouping_with_alias.out
@@ -7,3 +7,6 @@ all	2
 -- !select2 --
 1.0	1
 
+-- !select3 --
+1.0	1
+
diff --git a/regression-test/suites/correctness_p0/test_grouping_with_alias.groovy b/regression-test/suites/correctness_p0/test_grouping_with_alias.groovy
index 26f6c5eeab..45232edc45 100644
--- a/regression-test/suites/correctness_p0/test_grouping_with_alias.groovy
+++ b/regression-test/suites/correctness_p0/test_grouping_with_alias.groovy
@@ -54,7 +54,9 @@
     """
     sql """insert into cf_member values(2, 2, '2'); """
 
-    qt_select2 """select floor(id-1.0), count(*) from cf_member cm group by floor(id-1.0);"""
+    qt_select2 """select floor(id-1.0), count(*) from cf_member cm group by floor(id-1.0) order by floor(id-1.0);"""
+
+    qt_select3 """select floor(id-1.0), count(*) from cf_member cm group by 1 order by 1;"""
 
     sql """DROP TABLE IF EXISTS `cf_member`; """
  } 
\ No newline at end of file


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


[doris] 05/09: [Bug](function) fix equals implements not judge order by elements of function call expr (#20083)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit dc981cc55e9422e003455a526e95eb86fc4507ac
Author: Pxl <px...@qq.com>
AuthorDate: Mon May 29 19:03:05 2023 +0800

    [Bug](function) fix equals implements not judge order by elements of function call expr (#20083)
    
    fix equals implements not judge order by elements of function call expr
---
 .../apache/doris/analysis/FunctionCallExpr.java    | 22 ++++++++++++++++++----
 .../query_p0/group_concat/test_group_concat.out    |  4 ++++
 .../query_p0/group_concat/test_group_concat.groovy |  4 ++++
 3 files changed, 26 insertions(+), 4 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java
index e35eec84d2..9e1cf21648 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/FunctionCallExpr.java
@@ -486,6 +486,14 @@ public class FunctionCallExpr extends Expr {
             return false;
         }
         FunctionCallExpr o = (FunctionCallExpr) obj;
+        if (orderByElements.size() != o.orderByElements.size()) {
+            return false;
+        }
+        for (int i = 0; i < orderByElements.size(); i++) {
+            if (!orderByElements.get(i).equals(o.orderByElements.get(i))) {
+                return false;
+            }
+        }
         return /*opcode == o.opcode && aggOp == o.aggOp &&*/ fnName.equals(o.fnName)
                 && fnParams.isDistinct() == o.fnParams.isDistinct()
                 && fnParams.isStar() == o.fnParams.isStar();
@@ -520,12 +528,18 @@ public class FunctionCallExpr extends Expr {
                     || fnName.getFunction().equalsIgnoreCase("sm4_encrypt"))) {
                 result.add("\'***\'");
             } else if (orderByElements.size() > 0 && i == len - orderByElements.size()) {
-                result.add("ORDER BY " + children.get(i).toSql());
-            } else {
-                result.add(children.get(i).toSql());
+                sb.append("ORDER BY ");
+            }
+            sb.append(children.get(i).toSql());
+            if (orderByElements.size() > 0 && i >= len - orderByElements.size()) {
+                if (orderByElements.get(i - len + orderByElements.size()).getIsAsc()) {
+                    sb.append(" ASC");
+                } else {
+                    sb.append(" DESC");
+                }
             }
         }
-        sb.append(Joiner.on(", ").join(result)).append(")");
+        sb.append(")");
         return sb.toString();
     }
 
diff --git a/regression-test/data/query_p0/group_concat/test_group_concat.out b/regression-test/data/query_p0/group_concat/test_group_concat.out
index 0e6aab79a8..0bc76e1146 100644
--- a/regression-test/data/query_p0/group_concat/test_group_concat.out
+++ b/regression-test/data/query_p0/group_concat/test_group_concat.out
@@ -48,3 +48,7 @@ false
 -- !select_group_concat_order_by_desc3 --
 1	3, 21, 2, 11, 1
 2	23, 222, 22, 211, 21
+
+-- !select_group_concat_order_by --
+1,11,2,21,21,211,22,222,23,3	3,23,222,22,211,21,21,2,11,1
+
diff --git a/regression-test/suites/query_p0/group_concat/test_group_concat.groovy b/regression-test/suites/query_p0/group_concat/test_group_concat.groovy
index a5781f39e1..f2d5dd3504 100644
--- a/regression-test/suites/query_p0/group_concat/test_group_concat.groovy
+++ b/regression-test/suites/query_p0/group_concat/test_group_concat.groovy
@@ -62,4 +62,8 @@ suite("test_group_concat") {
     qt_select_group_concat_order_by_desc3 """
                 SELECT b1, group_concat(cast(abs(b3) as varchar) order by abs(b2) desc, b3 desc) FROM table_group_concat  group by b1 order by b1
               """
+    qt_select_group_concat_order_by """
+                select group_concat(b3,',' order by b3 asc),group_concat(b3,',' order by b3 desc) from table_group_concat;
+    """
+
 }


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


[doris] 07/09: [Fix](dynamic-partition) Try to avoid setting a zero-bucket-size partition. (#20177)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 93698651b3843cc0ef77928025f4b7902dec9296
Author: Xiangyu Wang <du...@gmail.com>
AuthorDate: Wed May 31 13:09:03 2023 +0800

    [Fix](dynamic-partition) Try to avoid setting a zero-bucket-size partition. (#20177)
    
    A fallback to avoid BE crash problem when partition's bucket size is 0, but not resolved.
---
 .../src/main/java/org/apache/doris/datasource/InternalCatalog.java  | 6 ++++++
 1 file changed, 6 insertions(+)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java
index 3e61fe8127..4b62de26ef 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java
@@ -100,6 +100,7 @@ import org.apache.doris.catalog.PartitionInfo;
 import org.apache.doris.catalog.PartitionItem;
 import org.apache.doris.catalog.PartitionType;
 import org.apache.doris.catalog.PrimitiveType;
+import org.apache.doris.catalog.RandomDistributionInfo;
 import org.apache.doris.catalog.RangePartitionItem;
 import org.apache.doris.catalog.Replica;
 import org.apache.doris.catalog.Replica.ReplicaState;
@@ -1393,6 +1394,11 @@ public class InternalCatalog implements CatalogIf<Database> {
                     if (hashDistributionInfo.getBucketNum() <= 0) {
                         throw new DdlException("Cannot assign hash distribution buckets less than 1");
                     }
+                } else if (distributionInfo.getType() == DistributionInfoType.RANDOM) {
+                    RandomDistributionInfo randomDistributionInfo = (RandomDistributionInfo) distributionInfo;
+                    if (randomDistributionInfo.getBucketNum() <= 0) {
+                        throw new DdlException("Cannot assign random distribution buckets less than 1");
+                    }
                 }
             } else {
                 // make sure partition-dristribution-info is deep copied from default-distribution-info


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


[doris] 08/09: [fix](dynamic_partition) fix dead lock when modify dynamic partition property for olap table (#20390)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 7b36e719b53fe42e5eb6820f3f88e3c3f4790c9f
Author: caiconghui <55...@users.noreply.github.com>
AuthorDate: Sat Jun 3 08:25:20 2023 +0800

    [fix](dynamic_partition) fix dead lock when modify dynamic partition property for olap table (#20390)
    
    Co-authored-by: caiconghui1 <ca...@jd.com>
---
 .../java/org/apache/doris/common/util/DynamicPartitionUtil.java     | 6 ------
 .../src/main/java/org/apache/doris/datasource/InternalCatalog.java  | 2 ++
 .../java/org/apache/doris/catalog/DynamicPartitionTableTest.java    | 2 ++
 3 files changed, 4 insertions(+), 6 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/DynamicPartitionUtil.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/DynamicPartitionUtil.java
index 56dc3232ea..4660e4eec6 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/util/DynamicPartitionUtil.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/DynamicPartitionUtil.java
@@ -472,12 +472,6 @@ public class DynamicPartitionUtil {
         if (olapTable.getTableProperty() != null
                 && olapTable.getTableProperty().getDynamicPartitionProperty() != null) {
             if (olapTable.getTableProperty().getDynamicPartitionProperty().getEnable()) {
-                if (!isReplay) {
-                    // execute create partition first time only in master of FE, So no need execute
-                    // when it's replay
-                    Env.getCurrentEnv().getDynamicPartitionScheduler()
-                            .executeDynamicPartitionFirstTime(dbId, olapTable.getId());
-                }
                 Env.getCurrentEnv().getDynamicPartitionScheduler()
                         .registerDynamicPartitionTable(dbId, olapTable.getId());
             } else {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java
index 4b62de26ef..c25f2f1044 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java
@@ -2226,6 +2226,8 @@ public class InternalCatalog implements CatalogIf<Database> {
                 LOG.info("successfully create table[{};{}]", tableName, tableId);
                 // register or remove table from DynamicPartition after table created
                 DynamicPartitionUtil.registerOrRemoveDynamicPartitionTable(db.getId(), olapTable, false);
+                Env.getCurrentEnv().getDynamicPartitionScheduler()
+                            .executeDynamicPartitionFirstTime(db.getId(), olapTable.getId());
                 Env.getCurrentEnv().getDynamicPartitionScheduler()
                         .createOrUpdateRuntimeInfo(tableId, DynamicPartitionScheduler.LAST_UPDATE_TIME,
                                 TimeUtils.getCurrentFormatTime());
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/DynamicPartitionTableTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/DynamicPartitionTableTest.java
index 84270649fb..9c8a2aa297 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/DynamicPartitionTableTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/DynamicPartitionTableTest.java
@@ -699,11 +699,13 @@ public class DynamicPartitionTableTest {
         // history_partition_num too large, but because start is -10, so modify ok
         String alter3 = "alter table test.dynamic_partition3 set ('dynamic_partition.history_partition_num' = '1000')";
         ExceptionChecker.expectThrowsNoException(() -> alterTable(alter3));
+        Env.getCurrentEnv().getDynamicPartitionScheduler().executeDynamicPartitionFirstTime(db.getId(), tbl.getId());
         Assert.assertEquals(14, tbl.getPartitionNames().size());
 
         // set start and history_partition_num properly.
         String alter4 = "alter table test.dynamic_partition3 set ('dynamic_partition.history_partition_num' = '100', 'dynamic_partition.start' = '-20')";
         ExceptionChecker.expectThrowsNoException(() -> alterTable(alter4));
+        Env.getCurrentEnv().getDynamicPartitionScheduler().executeDynamicPartitionFirstTime(db.getId(), tbl.getId());
         Assert.assertEquals(24, tbl.getPartitionNames().size());
     }
 


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


[doris] 04/09: [fix](dynamic_partition) fix dynamic partition not work when drop and recover olap table (#19031)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit c82875aec1c5162aa98d8a237875769ed79bf929
Author: caiconghui <55...@users.noreply.github.com>
AuthorDate: Mon May 29 13:02:10 2023 +0800

    [fix](dynamic_partition) fix dynamic partition not work when drop and  recover olap table (#19031)
    
    when olap table is dynamic partition enable, if drop and recover olap table, the table should be added to DynamicPartitionScheduler again
    
    ---------
    
    Co-authored-by: caiconghui1 <ca...@jd.com>
---
 .../apache/doris/catalog/CatalogRecycleBin.java    |  2 +
 .../doris/clone/DynamicPartitionScheduler.java     | 11 ++++-
 .../java/org/apache/doris/catalog/RecoverTest.java | 47 ++++++++++++++++++++++
 3 files changed, 58 insertions(+), 2 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/CatalogRecycleBin.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/CatalogRecycleBin.java
index 1193300a1b..0af262fe79 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/CatalogRecycleBin.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/CatalogRecycleBin.java
@@ -24,6 +24,7 @@ import org.apache.doris.common.DdlException;
 import org.apache.doris.common.FeMetaVersion;
 import org.apache.doris.common.io.Text;
 import org.apache.doris.common.io.Writable;
+import org.apache.doris.common.util.DynamicPartitionUtil;
 import org.apache.doris.common.util.MasterDaemon;
 import org.apache.doris.common.util.RangeUtils;
 import org.apache.doris.common.util.TimeUtils;
@@ -696,6 +697,7 @@ public class CatalogRecycleBin extends MasterDaemon implements Writable {
                 RecoverInfo recoverInfo = new RecoverInfo(db.getId(), table.getId(), -1L, "", newTableName, "");
                 Env.getCurrentEnv().getEditLog().logRecoverTable(recoverInfo);
             }
+            DynamicPartitionUtil.registerOrRemoveDynamicPartitionTable(db.getId(), (OlapTable) table, isReplay);
         } finally {
             table.writeUnlock();
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java
index 7c20455742..eac10bb2d0 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java
@@ -109,6 +109,11 @@ public class DynamicPartitionScheduler extends MasterDaemon {
         dynamicPartitionTableInfo.add(Pair.of(dbId, tableId));
     }
 
+    // only for test
+    public boolean containsDynamicPartitionTable(Long dbId, Long tableId) {
+        return dynamicPartitionTableInfo.contains(Pair.of(dbId, tableId));
+    }
+
     public void removeDynamicPartitionTable(Long dbId, Long tableId) {
         dynamicPartitionTableInfo.remove(Pair.of(dbId, tableId));
     }
@@ -273,8 +278,10 @@ public class DynamicPartitionScheduler extends MasterDaemon {
                 } catch (Exception e) {
                     isPartitionExists = true;
                     if (addPartitionKeyRange.equals(partitionItem.getItems())) {
-                        LOG.info("partition range {} exist in table {}, clear fail msg",
-                                addPartitionKeyRange, olapTable.getName());
+                        if (LOG.isDebugEnabled()) {
+                            LOG.debug("partition range {} exist in table {}, clear fail msg",
+                                    addPartitionKeyRange, olapTable.getName());
+                        }
                         clearCreatePartitionFailedMsg(olapTable.getId());
                     } else {
                         recordCreatePartitionFailedMsg(db.getFullName(), olapTable.getName(),
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/RecoverTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/RecoverTest.java
index cf096da8b2..df4ad65747 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/RecoverTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/RecoverTest.java
@@ -135,6 +135,10 @@ public class RecoverTest {
                 .flatMap(db -> db.getTable(tblName)).isPresent();
     }
 
+    private static boolean checkTableInDynamicScheduler(Long dbId, Long tableId) {
+        return Env.getCurrentEnv().getDynamicPartitionScheduler().containsDynamicPartitionTable(dbId, tableId);
+    }
+
     private static boolean checkPartitionExist(String dbName, String tblName, String partName) {
         return Env.getCurrentInternalCatalog()
                 .getDb(ClusterNamespace.getFullName(SystemInfoService.DEFAULT_CLUSTER, dbName))
@@ -389,4 +393,47 @@ public class RecoverTest {
         recoverPartition("test2", "table2", "p1", -1);
         Assert.assertTrue(checkPartitionExist("test2", "table2", "p1"));
     }
+
+
+    @Test
+    public void testDynamicTableRecover() throws Exception {
+        createDb("test3");
+        createTable("CREATE TABLE test3.`table3` (\n"
+                + "  `event_date` datetime(3) NOT NULL COMMENT \"\",\n"
+                + "  `app_name` varchar(64) NOT NULL COMMENT \"\",\n"
+                + "  `package_name` varchar(64) NOT NULL COMMENT \"\",\n"
+                + "  `age` varchar(32) NOT NULL COMMENT \"\",\n"
+                + "  `gender` varchar(32) NOT NULL COMMENT \"\",\n"
+                + "  `level` varchar(64) NOT NULL COMMENT \"\",\n"
+                + "  `city` varchar(64) NOT NULL COMMENT \"\",\n"
+                + "  `model` varchar(64) NOT NULL COMMENT \"\",\n"
+                + "  `brand` varchar(64) NOT NULL COMMENT \"\",\n"
+                + "  `hours` varchar(16) NOT NULL COMMENT \"\",\n"
+                + "  `use_num` int(11) SUM NOT NULL COMMENT \"\",\n"
+                + "  `use_time` double SUM NOT NULL COMMENT \"\",\n"
+                + "  `start_times` bigint(20) SUM NOT NULL COMMENT \"\"\n"
+                + ") ENGINE=OLAP\n"
+                + "AGGREGATE KEY(`event_date`, `app_name`, `package_name`, `age`, `gender`, `level`, `city`, \n"
+                + "  `model`, `brand`, `hours`) COMMENT \"OLAP\"\n"
+                + "PARTITION BY RANGE(`event_date`)\n"
+                + "(PARTITION p1 VALUES [('2020-02-27 00:00:00'), ('2020-03-02 00:00:00')),\n"
+                + "PARTITION p2 VALUES [('2020-03-02 00:00:00'), ('2020-03-07 00:00:00')))\n"
+                + "DISTRIBUTED BY HASH(`event_date`, `app_name`, `package_name`, `age`, `gender`, `level`, `city`, \n"
+                + " `model`, `brand`, `hours`) BUCKETS 1 PROPERTIES (\n"
+                + "\"replication_num\" = \"1\",\n"
+                + "\"dynamic_partition.enable\" = \"true\",\n"
+                + "\"dynamic_partition.time_unit\" = \"DAY\",\n"
+                + "\"dynamic_partition.end\" = \"3\",\n"
+                + "\"dynamic_partition.prefix\" = \"p\",\n"
+                + "\"dynamic_partition.buckets\" = \"1\",\n"
+                + "\"dynamic_partition.replication_num\" = \"1\",\n"
+                + "\"dynamic_partition.create_history_partition\"=\"true\",\n"
+                + "\"dynamic_partition.start\" = \"-3\"\n"
+                + ");\n");
+        Long dbId = getDbId("test3");
+        Long tableId = getTableId("test3", "table3");
+        dropTable("test3", "table3");
+        recoverTable("test3", "table3", -1);
+        Assert.assertTrue(checkTableInDynamicScheduler(dbId, tableId));
+    }
 }


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


[doris] 01/09: [Bug](materialized-view) forbid create mv/rollup on mow table (#20001)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit ccbd791cf667a6bbc1462111cc4ebf78322e3381
Author: Pxl <px...@qq.com>
AuthorDate: Thu May 25 15:30:12 2023 +0800

    [Bug](materialized-view) forbid create mv/rollup on mow table (#20001)
    
    forbid create mv/rollup on mow table
---
 conf/asan_suppr.conf                               |  2 +-
 .../doris/alter/MaterializedViewHandler.java       |  6 ++++
 .../java/org/apache/doris/alter/AlterTest.java     |  2 +-
 .../doris/nereids/rules/mv/SelectMvIndexTest.java  |  2 +-
 .../planner/MaterializedViewFunctionTest.java      |  2 +-
 .../test_mv_useless/mow_invalid.groovy             | 40 ++++++++++++++++++++++
 .../test_uniq_mv_schema_change.groovy              |  4 +--
 .../test_uniq_rollup_schema_change.groovy          |  2 +-
 8 files changed, 53 insertions(+), 7 deletions(-)

diff --git a/conf/asan_suppr.conf b/conf/asan_suppr.conf
index 254e6901ac..fe294d58e8 100644
--- a/conf/asan_suppr.conf
+++ b/conf/asan_suppr.conf
@@ -16,4 +16,4 @@
 # under the License.
 
 # This is a known leak.
-leak:brpc
\ No newline at end of file
+leak:brpc
diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java
index 355b34f1e8..d5ce9c7a22 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java
@@ -444,6 +444,9 @@ public class MaterializedViewHandler extends AlterHandler {
         if (olapTable.hasMaterializedIndex(addMVClause.getMVName())) {
             throw new DdlException("Materialized view[" + addMVClause.getMVName() + "] already exists");
         }
+        if (olapTable.getEnableUniqueKeyMergeOnWrite()) {
+            throw new DdlException("MergeOnWrite table can't create materialized view.");
+        }
         // check if mv columns are valid
         // a. Aggregate or Unique table:
         //     1. For aggregate table, mv columns with aggregate function should be same as base schema
@@ -569,6 +572,9 @@ public class MaterializedViewHandler extends AlterHandler {
     public List<Column> checkAndPrepareMaterializedView(AddRollupClause addRollupClause, OlapTable olapTable,
             long baseIndexId, boolean changeStorageFormat)
             throws DdlException {
+        if (olapTable.getEnableUniqueKeyMergeOnWrite()) {
+            throw new DdlException("MergeOnWrite table can't create materialized view.");
+        }
         String rollupIndexName = addRollupClause.getRollupName();
         List<String> rollupColumnNames = addRollupClause.getColumnNames();
         if (changeStorageFormat) {
diff --git a/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java b/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java
index 553dfc6b6a..a5c820a2e5 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java
@@ -124,7 +124,7 @@ public class AlterTest {
                         + "PARTITION BY RANGE(k1)\n" + "(\n"
                         + "    PARTITION p1 values less than('2020-02-01 00:00:00'),\n"
                         + "    PARTITION p2 values less than('2020-03-01 00:00:00')\n" + ")\n"
-                        + "DISTRIBUTED BY HASH(k2) BUCKETS 3\n" + "PROPERTIES('replication_num' = '1');");
+                        + "DISTRIBUTED BY HASH(k2) BUCKETS 3\n" + "PROPERTIES('replication_num' = '1','enable_unique_key_merge_on_write' = 'false');");
 
         createTable("create external table test.odbc_table\n" + "(  `k1` bigint(20) COMMENT \"\",\n"
                 + "  `k2` datetime COMMENT \"\",\n" + "  `k3` varchar(20) COMMENT \"\",\n"
diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/mv/SelectMvIndexTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/mv/SelectMvIndexTest.java
index a1bf9dc70e..a2631fa0bc 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/mv/SelectMvIndexTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/mv/SelectMvIndexTest.java
@@ -758,7 +758,7 @@ public class SelectMvIndexTest extends BaseMaterializedIndexSelectTest implement
     @Test
     public void testUniqueTableInQuery() throws Exception {
         String uniqueTable = "CREATE TABLE " + TEST_TABLE_NAME + " (k1 int, k2 int, v1 int) UNIQUE KEY (k1, k2) "
-                + "DISTRIBUTED BY HASH(k1) BUCKETS 3 PROPERTIES ('replication_num' = '1');";
+                + "DISTRIBUTED BY HASH(k1) BUCKETS 3 PROPERTIES ('replication_num' = '1','enable_unique_key_merge_on_write' = 'false');";
         createTable(uniqueTable);
         String createK1MV = "create materialized view only_k1 as select k2 from " + TEST_TABLE_NAME + " group by "
                 + "k2;";
diff --git a/fe/fe-core/src/test/java/org/apache/doris/planner/MaterializedViewFunctionTest.java b/fe/fe-core/src/test/java/org/apache/doris/planner/MaterializedViewFunctionTest.java
index fc7fc32823..9db1b85920 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/planner/MaterializedViewFunctionTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/planner/MaterializedViewFunctionTest.java
@@ -660,7 +660,7 @@ public class MaterializedViewFunctionTest {
     @Test
     public void testUniqueTableInQuery() throws Exception {
         String uniqueTable = "CREATE TABLE " + TEST_TABLE_NAME + " (k1 int, k2 int, v1 int) UNIQUE KEY (k1, k2) "
-                + "DISTRIBUTED BY HASH(k1) BUCKETS 3 PROPERTIES ('replication_num' = '1');";
+                + "DISTRIBUTED BY HASH(k1) BUCKETS 3 PROPERTIES ('replication_num' = '1', 'enable_unique_key_merge_on_write' = 'false');";
         dorisAssert.withTable(uniqueTable);
         String createK1K2MV = "create materialized view only_k1 as select k2, k1 from " + TEST_TABLE_NAME + " group by "
                 + "k2, k1;";
diff --git a/regression-test/suites/materialized_view_p0/test_mv_useless/mow_invalid.groovy b/regression-test/suites/materialized_view_p0/test_mv_useless/mow_invalid.groovy
new file mode 100644
index 0000000000..febdc259a7
--- /dev/null
+++ b/regression-test/suites/materialized_view_p0/test_mv_useless/mow_invalid.groovy
@@ -0,0 +1,40 @@
+// 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.
+
+suite ("mow_invalid") {
+    sql """ drop table if exists u_table; """
+
+    sql """
+            create table u_table (
+                k1 int null,
+                k2 int not null,
+                k3 bigint null,
+                k4 varchar(100) null
+            )
+            unique key (k1,k2,k3)
+            distributed BY hash(k1) buckets 3
+            properties(
+                "replication_num" = "1",
+                "enable_unique_key_merge_on_write" = "true"
+            );
+        """
+
+    test {
+        sql "create materialized view k123p as select k1,k2+k3 from u_table;"
+        exception "errCode = 2,"
+    }
+}
diff --git a/regression-test/suites/schema_change_p0/test_uniq_mv_schema_change.groovy b/regression-test/suites/schema_change_p0/test_uniq_mv_schema_change.groovy
index cad59ed449..e2c2a20893 100644
--- a/regression-test/suites/schema_change_p0/test_uniq_mv_schema_change.groovy
+++ b/regression-test/suites/schema_change_p0/test_uniq_mv_schema_change.groovy
@@ -72,7 +72,7 @@ suite ("test_uniq_mv_schema_change") {
                 `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间")
             UNIQUE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`)
             BUCKETS 1
-            PROPERTIES ( "replication_num" = "1", "light_schema_change" = "true");
+            PROPERTIES ( "replication_num" = "1", "light_schema_change" = "false", 'enable_unique_key_merge_on_write' = 'false');
         """
 
     //add materialized view
@@ -223,4 +223,4 @@ suite ("test_uniq_mv_schema_change") {
     } finally {
         //try_sql("DROP TABLE IF EXISTS ${tableName}")
     }
-}
\ No newline at end of file
+}
diff --git a/regression-test/suites/schema_change_p0/test_uniq_rollup_schema_change.groovy b/regression-test/suites/schema_change_p0/test_uniq_rollup_schema_change.groovy
index ceef82590c..b547b4b049 100644
--- a/regression-test/suites/schema_change_p0/test_uniq_rollup_schema_change.groovy
+++ b/regression-test/suites/schema_change_p0/test_uniq_rollup_schema_change.groovy
@@ -234,4 +234,4 @@ suite ("test_uniq_rollup_schema_change") {
     } finally {
         //try_sql("DROP TABLE IF EXISTS ${tableName}")
     }
-}
\ No newline at end of file
+}


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


[doris] 02/09: [security] Don't print password in BaseController (#18862)

Posted by mo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-1.2-lts
in repository https://gitbox.apache.org/repos/asf/doris.git

commit 6c92931eadffc293d75a445dadbe500c31a10ae3
Author: ZhangJian He <sh...@gmail.com>
AuthorDate: Sun May 28 22:49:18 2023 +0800

    [security] Don't print password in BaseController (#18862)
---
 .../main/java/org/apache/doris/httpv2/controller/BaseController.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/controller/BaseController.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/controller/BaseController.java
index daeced3051..6a5792d929 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/controller/BaseController.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/controller/BaseController.java
@@ -185,7 +185,7 @@ public class BaseController {
         public String toString() {
             StringBuilder sb = new StringBuilder();
             sb.append("user: ").append(fullUserName).append(", remote ip: ").append(remoteIp);
-            sb.append(", password: ").append(password).append(", cluster: ").append(cluster);
+            sb.append(", password: ").append("********").append(", cluster: ").append(cluster);
             return sb.toString();
         }
     }


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