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/04/09 02:01:03 UTC

[doris] branch branch-1.2-lts updated (8df05d6d81 -> 19f1f37433)

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 8df05d6d81 [Enhencement](like) fallback to re2 if hyperscan failed pick #18350 (#18468)
     new 0dac4b49fe [fix](ssl)refactor some SSL info logs to debug logs (#18234)
     new a2ea72eb6e [improvement](dynamic partition) Fix dynamic partition no bucket (#18300)
     new c0231b1abf [improvement](test) print exception when streamload fails (#18315)
     new ff89306e0d [fix](quit) be can not quit cleanly due to deadlock (#17971)
     new b5c7e39cb6 [fix](regression-test) print real and expect rows when fail in exception (#17949)
     new cb98497472 [Bug](decimal) Fix string to decimal (#18282)
     new 085cbf5d8e [improve](clickhouse catalog) Add  `"` wrap select column for the sql query clickhouse jdbc (#18352)
     new 5b4d158466 [Bug](DECIMALV3) fix wrong decimal scale returned by function `round` (#18375)
     new d4d994d844 [fix](planner) decimalv2 castTo decimalv2 should change type directly (#18297)
     new 7356904a50 [fix](bdbje) handle `RollbackException` in `BDBJEJournal.open` (#18471)
     new 9766cbd115 [fix](profile) fix show load query profile (#18487)
     new 19f1f37433 [Optimize](mutlti-catalog) Opt zlib performance by adding '-O3' cflags. (#18488)

The 12 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:
 be/src/olap/data_dir.cpp                           |  9 +-
 be/src/olap/data_dir.h                             |  2 +-
 be/src/util/string_parser.hpp                      |  8 +-
 be/src/vec/functions/round.h                       |  2 +-
 docs/en/docs/admin-manual/query-profile.md         |  6 ++
 .../Show-Statements/SHOW-LOAD-PROFILE.md           | 96 ++++++++++++---------
 docs/zh-CN/docs/admin-manual/query-profile.md      |  6 +-
 .../Show-Statements/SHOW-LOAD-PROFILE.md           | 97 +++++++++++++---------
 .../apache/doris/alter/SchemaChangeHandler.java    |  2 +-
 .../main/java/org/apache/doris/analysis/Expr.java  |  7 ++
 .../apache/doris/analysis/ShowLoadProfileStmt.java | 19 ++++-
 .../doris/analysis/ShowQueryProfileStmt.java       |  7 +-
 .../java/org/apache/doris/catalog/OdbcTable.java   |  7 ++
 .../doris/common/util/DynamicPartitionUtil.java    | 12 ++-
 .../apache/doris/journal/bdbje/BDBJEJournal.java   |  4 +
 .../java/org/apache/doris/mysql/MysqlProto.java    |  6 +-
 .../java/org/apache/doris/qe/ShowExecutor.java     | 17 +++-
 .../doris/catalog/DynamicPartitionTableTest.java   |  6 +-
 .../aggregate/select_distinct.out}                 |  4 +-
 .../sql_functions/math_functions/test_round.out    |  3 +
 .../regression/action/StreamLoadAction.groovy      |  5 +-
 .../doris/regression/util/OutputUtils.groovy       |  1 +
 .../suites/autobucket/test_autobucket.groovy       |  3 +-
 ...vy => test_autobucket_dynamic_partition.groovy} | 31 ++++---
 .../test_dynamic_partition.groovy                  | 73 ++++++++++------
 .../query_p0/aggregate/select_distinct.groovy      | 77 +++++++++++++++++
 .../sql_functions/math_functions/test_round.groovy | 40 ++++++++-
 thirdparty/build-thirdparty.sh                     |  2 +-
 28 files changed, 398 insertions(+), 154 deletions(-)
 copy regression-test/data/{correctness/sql/test_ifnull_function2.out => query_p0/aggregate/select_distinct.out} (69%)
 copy regression-test/suites/autobucket/{test_autobucket.groovy => test_autobucket_dynamic_partition.groovy} (52%)
 create mode 100644 regression-test/suites/query_p0/aggregate/select_distinct.groovy


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


[doris] 02/12: [improvement](dynamic partition) Fix dynamic partition no bucket (#18300)

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 a2ea72eb6ea2750d5a7cc36f799f6910ef752f7d
Author: Jack Drogon <ja...@gmail.com>
AuthorDate: Sun Apr 2 15:51:21 2023 +0800

    [improvement](dynamic partition) Fix dynamic partition no bucket (#18300)
    
    
    Signed-off-by: Jack Drogon <ja...@gmail.com>
---
 .../apache/doris/alter/SchemaChangeHandler.java    |  2 +-
 .../doris/common/util/DynamicPartitionUtil.java    | 12 +++-
 .../doris/catalog/DynamicPartitionTableTest.java   |  6 +-
 .../suites/autobucket/test_autobucket.groovy       |  3 +-
 ...vy => test_autobucket_dynamic_partition.groovy} | 31 +++++----
 .../test_dynamic_partition.groovy                  | 73 ++++++++++++++--------
 6 files changed, 81 insertions(+), 46 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java
index dafd366d25..0ba0a70ae5 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeHandler.java
@@ -1695,7 +1695,7 @@ public class SchemaChangeHandler extends AlterHandler {
                         if (!olapTable.dynamicPartitionExists()) {
                             try {
                                 DynamicPartitionUtil.checkInputDynamicPartitionProperties(properties,
-                                        olapTable.getPartitionInfo());
+                                        olapTable);
                             } catch (DdlException e) {
                                 // This table is not a dynamic partition table
                                 // and didn't supply all dynamic partition properties
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 b51aeaf9bc..b36932f691 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
@@ -20,6 +20,7 @@ package org.apache.doris.common.util;
 import org.apache.doris.analysis.TimestampArithmeticExpr.TimeUnit;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.Database;
+import org.apache.doris.catalog.DistributionInfo;
 import org.apache.doris.catalog.DynamicPartitionProperty;
 import org.apache.doris.catalog.Env;
 import org.apache.doris.catalog.OlapTable;
@@ -398,10 +399,12 @@ public class DynamicPartitionUtil {
     // Check if all requried properties has been set.
     // And also check all optional properties, if not set, set them to default value.
     public static boolean checkInputDynamicPartitionProperties(Map<String, String> properties,
-            PartitionInfo partitionInfo) throws DdlException {
+            OlapTable olapTable) throws DdlException {
         if (properties == null || properties.isEmpty()) {
             return false;
         }
+
+        PartitionInfo partitionInfo = olapTable.getPartitionInfo();
         if (partitionInfo.getType() != PartitionType.RANGE || partitionInfo.isMultiColumnPartition()) {
             throw new DdlException("Dynamic partition only support single-column range partition");
         }
@@ -442,7 +445,9 @@ public class DynamicPartitionUtil {
                 throw new DdlException("Must assign dynamic_partition.end properties");
             }
             if (Strings.isNullOrEmpty(buckets)) {
-                throw new DdlException("Must assign dynamic_partition.buckets properties");
+                DistributionInfo distributionInfo = olapTable.getDefaultDistributionInfo();
+                buckets = String.valueOf(distributionInfo.getBucketNum());
+                properties.put(DynamicPartitionProperty.BUCKETS, buckets);
             }
             if (Strings.isNullOrEmpty(timeZone)) {
                 properties.put(DynamicPartitionProperty.TIME_ZONE, TimeUtils.getSystemTimeZone().getID());
@@ -505,6 +510,7 @@ public class DynamicPartitionUtil {
             properties.remove(DynamicPartitionProperty.BUCKETS);
             analyzedProperties.put(DynamicPartitionProperty.BUCKETS, bucketsValue);
         }
+
         if (properties.containsKey(DynamicPartitionProperty.ENABLE)) {
             String enableValue = properties.get(DynamicPartitionProperty.ENABLE);
             checkEnable(enableValue);
@@ -674,7 +680,7 @@ public class DynamicPartitionUtil {
      */
     public static void checkAndSetDynamicPartitionProperty(OlapTable olapTable, Map<String, String> properties,
             Database db) throws UserException {
-        if (DynamicPartitionUtil.checkInputDynamicPartitionProperties(properties, olapTable.getPartitionInfo())) {
+        if (DynamicPartitionUtil.checkInputDynamicPartitionProperties(properties, olapTable)) {
             Map<String, String> dynamicPartitionProperties =
                     DynamicPartitionUtil.analyzeDynamicPartition(properties, olapTable, db);
             TableProperty tableProperty = olapTable.getTableProperty();
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 3a522ad3a6..ff6d1f687d 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
@@ -258,7 +258,7 @@ public class DynamicPartitionTableTest {
 
     @Test
     public void testMissBuckets() throws Exception {
-        String createOlapTblStmt = "CREATE TABLE test.`dynamic_partition_buckets` (\n"
+        String createOlapTblStmt = "CREATE TABLE test.`dynamic_partition_miss_buckets` (\n"
                 + "  `k1` date NULL COMMENT \"\",\n"
                 + "  `k2` int NULL COMMENT \"\",\n"
                 + "  `k3` smallint NULL COMMENT \"\",\n"
@@ -282,14 +282,12 @@ public class DynamicPartitionTableTest {
                 + "\"dynamic_partition.time_unit\" = \"day\",\n"
                 + "\"dynamic_partition.prefix\" = \"p\"\n"
                 + ");";
-        expectedException.expect(DdlException.class);
-        expectedException.expectMessage("errCode = 2, detailMessage = Must assign dynamic_partition.buckets properties");
         createTable(createOlapTblStmt);
     }
 
     @Test
     public void testNotAllowed() throws Exception {
-        String createOlapTblStmt = "CREATE TABLE test.`dynamic_partition_buckets` (\n"
+        String createOlapTblStmt = "CREATE TABLE test.`dynamic_partition_not_allowed` (\n"
                 + "  `k1` date NULL COMMENT \"\",\n"
                 + "  `k2` int NULL COMMENT \"\",\n"
                 + "  `k3` smallint NULL COMMENT \"\",\n"
diff --git a/regression-test/suites/autobucket/test_autobucket.groovy b/regression-test/suites/autobucket/test_autobucket.groovy
index 29945e0f9a..ab0ae99658 100644
--- a/regression-test/suites/autobucket/test_autobucket.groovy
+++ b/regression-test/suites/autobucket/test_autobucket.groovy
@@ -25,7 +25,7 @@ suite("test_autobucket") {
         COMMENT 'OLAP'
         DISTRIBUTED BY HASH(`user_id`) BUCKETS AUTO
         PROPERTIES (
-        "replication_allocation" = "tag.location.default: 1"
+          "replication_allocation" = "tag.location.default: 1"
         )
         """
 
@@ -35,6 +35,7 @@ suite("test_autobucket") {
     result = sql "show partitions from autobucket_test"
     logger.info("${result}")
     // XXX: buckets at pos(8), next maybe impl by sql meta
+    // 10 is the default buckets without partition size
     assertEquals(Integer.valueOf(result.get(0).get(8)), 10)
 
     sql "drop table if exists autobucket_test"
diff --git a/regression-test/suites/autobucket/test_autobucket.groovy b/regression-test/suites/autobucket/test_autobucket_dynamic_partition.groovy
similarity index 52%
copy from regression-test/suites/autobucket/test_autobucket.groovy
copy to regression-test/suites/autobucket/test_autobucket_dynamic_partition.groovy
index 29945e0f9a..80679cb07d 100644
--- a/regression-test/suites/autobucket/test_autobucket.groovy
+++ b/regression-test/suites/autobucket/test_autobucket_dynamic_partition.groovy
@@ -15,27 +15,34 @@
 // specific language governing permissions and limitations
 // under the License.
 
-suite("test_autobucket") {
-    sql "drop table if exists autobucket_test"
+suite("test_autobucket_dynamic_partition") {
+    sql "drop table if exists test_autobucket_dynamic_partition"
     result = sql """
-        CREATE TABLE `autobucket_test` (
-          `user_id` largeint(40) NOT NULL
-        ) ENGINE=OLAP
-        DUPLICATE KEY(`user_id`)
-        COMMENT 'OLAP'
-        DISTRIBUTED BY HASH(`user_id`) BUCKETS AUTO
+        CREATE TABLE
+        test_autobucket_dynamic_partition (k1 DATETIME)
+        PARTITION BY
+        RANGE (k1) () DISTRIBUTED BY HASH (k1) BUCKETS AUTO
         PROPERTIES (
-        "replication_allocation" = "tag.location.default: 1"
+            "dynamic_partition.enable" = "true",
+            "dynamic_partition.time_unit" = "WEEK",
+            "dynamic_partition.start" = "-2",
+            "dynamic_partition.end" = "2",
+            "dynamic_partition.prefix" = "p",
+            "replication_allocation" = "tag.location.default: 1"
         )
         """
 
-    result = sql "show create table autobucket_test"
+    result = sql "show create table test_autobucket_dynamic_partition"
     assertTrue(result.toString().containsIgnoreCase("BUCKETS AUTO"))
 
-    result = sql "show partitions from autobucket_test"
+    result = sql "show partitions from test_autobucket_dynamic_partition"
     logger.info("${result}")
     // XXX: buckets at pos(8), next maybe impl by sql meta
+    // 10 is the default buckets without partition size
+    assertEquals(result.size(), 3)
     assertEquals(Integer.valueOf(result.get(0).get(8)), 10)
+    assertEquals(Integer.valueOf(result.get(1).get(8)), 10)
+    assertEquals(Integer.valueOf(result.get(2).get(8)), 10)
 
-    sql "drop table if exists autobucket_test"
+    sql "drop table if exists test_autobucket_dynamic_partition"
 }
diff --git a/regression-test/suites/partition_p0/dynamic_partition/test_dynamic_partition.groovy b/regression-test/suites/partition_p0/dynamic_partition/test_dynamic_partition.groovy
index f80de41841..16246bf629 100644
--- a/regression-test/suites/partition_p0/dynamic_partition/test_dynamic_partition.groovy
+++ b/regression-test/suites/partition_p0/dynamic_partition/test_dynamic_partition.groovy
@@ -14,44 +14,68 @@
 // KIND, either express or implied.  See the License for the
 // specific language governing permissions and limitations
 // under the License.
-
 suite("test_dynamic_partition") {
     // todo: test dynamic partition
     sql "drop table if exists dy_par"
     sql """
-        CREATE TABLE IF NOT EXISTS dy_par ( k1 date NOT NULL, k2 varchar(20) NOT NULL, k3 int sum NOT NULL ) 
-        AGGREGATE KEY(k1,k2) 
-        PARTITION BY RANGE(k1) ( ) 
-        DISTRIBUTED BY HASH(k1) BUCKETS 3 
-        PROPERTIES (  
-            "dynamic_partition.enable"="true", 
-            "dynamic_partition.end"="3", 
-            "dynamic_partition.buckets"="10", 
-            "dynamic_partition.start"="-3", 
-            "dynamic_partition.prefix"="p", 
-            "dynamic_partition.time_unit"="DAY", 
+        CREATE TABLE IF NOT EXISTS dy_par ( k1 date NOT NULL, k2 varchar(20) NOT NULL, k3 int sum NOT NULL )
+        AGGREGATE KEY(k1,k2)
+        PARTITION BY RANGE(k1) ( )
+        DISTRIBUTED BY HASH(k1) BUCKETS 3
+        PROPERTIES (
+            "dynamic_partition.enable"="true",
+            "dynamic_partition.end"="3",
+            "dynamic_partition.buckets"="10",
+            "dynamic_partition.start"="-3",
+            "dynamic_partition.prefix"="p",
+            "dynamic_partition.time_unit"="DAY",
             "dynamic_partition.create_history_partition"="true",
             "dynamic_partition.replication_allocation" = "tag.location.default: 1")
         """
     List<List<Object>> result  = sql "show tables like 'dy_par'"
     logger.info("${result}")
     assertEquals(result.size(), 1)
+    result = sql "show partitions from dy_par"
+    // XXX: buckets at pos(8), next maybe impl by sql meta
+    assertEquals(Integer.valueOf(result.get(0).get(8)), 10)
     sql "drop table dy_par"
-    //
+    sql "drop table if exists dy_par_bucket_set_by_distribution"
+    sql """
+        CREATE TABLE IF NOT EXISTS dy_par_bucket_set_by_distribution
+        ( k1 date NOT NULL, k2 varchar(20) NOT NULL, k3 int sum NOT NULL )
+        AGGREGATE KEY(k1,k2)
+        PARTITION BY RANGE(k1) ( )
+        DISTRIBUTED BY HASH(k1) BUCKETS 3
+        PROPERTIES (
+            "dynamic_partition.enable"="true",
+            "dynamic_partition.end"="3",
+            "dynamic_partition.start"="-3",
+            "dynamic_partition.prefix"="p",
+            "dynamic_partition.time_unit"="DAY",
+            "dynamic_partition.create_history_partition"="true",
+            "dynamic_partition.replication_allocation" = "tag.location.default: 1")
+        """
+    result  = sql "show tables like 'dy_par_bucket_set_by_distribution'"
+    logger.info("${result}")
+    assertEquals(result.size(), 1)
+    result = sql "show partitions from dy_par_bucket_set_by_distribution"
+    // XXX: buckets at pos(8), next maybe impl by sql meta
+    assertEquals(Integer.valueOf(result.get(0).get(8)), 3)
+    sql "drop table dy_par_bucket_set_by_distribution"
     sql "drop table if exists dy_par_bad"
     test {
         sql """
-        CREATE TABLE IF NOT EXISTS dy_par_bad ( k1 date NOT NULL, k2 varchar(20) NOT NULL, k3 int sum NOT NULL ) 
-        AGGREGATE KEY(k1,k2) 
-        PARTITION BY RANGE(k1) ( ) 
-        DISTRIBUTED BY HASH(k1) BUCKETS 3 
-        PROPERTIES (  
-            "dynamic_partition.enable"="true", 
-            "dynamic_partition.end"="3", 
-            "dynamic_partition.buckets"="10", 
-            "dynamic_partition.start"="-3", 
-            "dynamic_partition.prefix"="p", 
-            "dynamic_partition.time_unit"="DAY", 
+        CREATE TABLE IF NOT EXISTS dy_par_bad ( k1 date NOT NULL, k2 varchar(20) NOT NULL, k3 int sum NOT NULL )
+        AGGREGATE KEY(k1,k2)
+        PARTITION BY RANGE(k1) ( )
+        DISTRIBUTED BY HASH(k1) BUCKETS 3
+        PROPERTIES (
+            "dynamic_partition.enable"="true",
+            "dynamic_partition.end"="3",
+            "dynamic_partition.buckets"="10",
+            "dynamic_partition.start"="-3",
+            "dynamic_partition.prefix"="p",
+            "dynamic_partition.time_unit"="DAY",
             "dynamic_partition.create_history_partition"="true",
             "dynamic_partition.replication_allocation" = "tag.location.not_exist_tag: 1")
         """
@@ -59,7 +83,6 @@ suite("test_dynamic_partition") {
         exception "errCode = 2,"
     }
     sql "drop table if exists dy_par_bad"
-
     sql """
         CREATE TABLE IF NOT EXISTS dy_par ( k1 datev2 NOT NULL, k2 varchar(20) NOT NULL, k3 int sum NOT NULL )
         AGGREGATE KEY(k1,k2)


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


[doris] 10/12: [fix](bdbje) handle `RollbackException` in `BDBJEJournal.open` (#18471)

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 7356904a5000f1d9fef83f9fc7c27dabc3653ce2
Author: Lei Zhang <27...@users.noreply.github.com>
AuthorDate: Sat Apr 8 15:31:24 2023 +0800

    [fix](bdbje) handle `RollbackException` in `BDBJEJournal.open` (#18471)
    
    reference:
    
    handle bdb rollbackexception #6582
    [fix](bdbje) fix handle bdb RollbackException incorrectly #17483
---
 .../src/main/java/org/apache/doris/journal/bdbje/BDBJEJournal.java    | 4 ++++
 1 file changed, 4 insertions(+)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/journal/bdbje/BDBJEJournal.java b/fe/fe-core/src/main/java/org/apache/doris/journal/bdbje/BDBJEJournal.java
index d6bfb0ee9f..5376fd9547 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/journal/bdbje/BDBJEJournal.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/journal/bdbje/BDBJEJournal.java
@@ -341,6 +341,10 @@ public class BDBJEJournal implements Journal { // CHECKSTYLE IGNORE THIS LINE: B
                 break;
             } catch (InsufficientLogException insufficientLogEx) {
                 reSetupBdbEnvironment(insufficientLogEx);
+            } catch (RollbackException rollbackEx) {
+                LOG.warn("catch rollback log exception. will reopen the ReplicatedEnvironment.", rollbackEx);
+                bdbEnvironment.closeReplicatedEnvironment();
+                bdbEnvironment.openReplicatedEnvironment(new File(environmentPath));
             }
         }
     }


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


[doris] 12/12: [Optimize](mutlti-catalog) Opt zlib performance by adding '-O3' cflags. (#18488)

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 19f1f37433855f477b81b24834a43cd29d3c213e
Author: Qi Chen <ka...@gmail.com>
AuthorDate: Sun Apr 9 08:52:48 2023 +0800

    [Optimize](mutlti-catalog) Opt zlib performance by adding '-O3' cflags. (#18488)
    
    Opt zlib performance by adding '-O3' cflags.
    From orc' reader test, it has 2.5x performance boost.
---
 thirdparty/build-thirdparty.sh | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/thirdparty/build-thirdparty.sh b/thirdparty/build-thirdparty.sh
index 289856cc29..9b9642fff2 100755
--- a/thirdparty/build-thirdparty.sh
+++ b/thirdparty/build-thirdparty.sh
@@ -557,7 +557,7 @@ build_zlib() {
     check_if_source_exist "${ZLIB_SOURCE}"
     cd "${TP_SOURCE_DIR}/${ZLIB_SOURCE}"
 
-    CFLAGS="-fPIC" \
+    CFLAGS="-O3 -fPIC" \
         CPPFLAGS="-I${TP_INCLUDE_DIR}" \
         LDFLAGS="-L${TP_LIB_DIR}" \
         ./configure --prefix="${TP_INSTALL_DIR}" --static


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


[doris] 11/12: [fix](profile) fix show load query profile (#18487)

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 9766cbd115a5d9202e89e2c555d1a571339e46eb
Author: Mingyu Chen <mo...@163.com>
AuthorDate: Sun Apr 9 08:41:18 2023 +0800

    [fix](profile) fix show load query profile (#18487)
    
    Sometimes, `show load profile` will only show part of the insert opertion's profile.
    This is because we assume that for all load operation(including insert), there is only one fragment in the plan.
    But actually, there will be more than 1 fragment in plan. eg:
    
    `insert into tbl1 select * from tbl1 limit 1` will have 2 fragments.
    
    This PR mainly changes:
    
    1. modify the `show load profile`
       Before:  `show load profile "/queryid/taskid/instanceid";`
       After: `show load profile "/queryid/taskid/fragmentid/instanceid";`
    
    2. Modify the display of `ReadColumns` in OlapScanNode
        Because for wide table, the line of `ReadColumns` may be too long for show in profile.
        So I wrap it and each line contains at most 10 columns names.
    
    3. Fix tvf not working with pipeline engine, follow up #18376
---
 docs/en/docs/admin-manual/query-profile.md         |  6 ++
 .../Show-Statements/SHOW-LOAD-PROFILE.md           | 96 ++++++++++++---------
 docs/zh-CN/docs/admin-manual/query-profile.md      |  6 +-
 .../Show-Statements/SHOW-LOAD-PROFILE.md           | 97 +++++++++++++---------
 .../apache/doris/analysis/ShowLoadProfileStmt.java | 19 ++++-
 .../doris/analysis/ShowQueryProfileStmt.java       |  7 +-
 .../java/org/apache/doris/qe/ShowExecutor.java     | 17 +++-
 7 files changed, 157 insertions(+), 91 deletions(-)

diff --git a/docs/en/docs/admin-manual/query-profile.md b/docs/en/docs/admin-manual/query-profile.md
index 03e6ebd595..ac5f631e01 100644
--- a/docs/en/docs/admin-manual/query-profile.md
+++ b/docs/en/docs/admin-manual/query-profile.md
@@ -28,6 +28,11 @@ under the License.
 
 This document focuses on introducing the **Running Profile** which recorded runtime status of Doris in query execution. Using these statistical information, we can understand the execution of frgment to become a expert of Doris's **debugging and tuning**.
 
+You can also refer to following statements to view profile in command line:
+
+- [SHOW QUERY PROFILE](../sql-manual/sql-reference/Show-Statements/SHOW-QUERY-PROFILE.md)
+- [SHOW LOAD PROFILE](../sql-manual/sql-reference/Show-Statements/SHOW-LOAD-PROFILE.md)
+
 ## Noun Interpretation
 
 * **FE**: Frontend, frontend node of Doris. Responsible for metadata management and request access.
@@ -39,6 +44,7 @@ This document focuses on introducing the **Running Profile** which recorded runt
 ## Basic concepts
 
 FE splits the query plan into fragments and distributes them to BE for task execution. BE records the statistics of **Running State** when executing fragment. BE print the outputs statistics of fragment execution into the log. FE can also collect these statistics recorded by each fragment and print the results on FE's web page.
+
 ## Specific operation
 
 Turn on the report switch on FE through MySQL command
diff --git a/docs/en/docs/sql-manual/sql-reference/Show-Statements/SHOW-LOAD-PROFILE.md b/docs/en/docs/sql-manual/sql-reference/Show-Statements/SHOW-LOAD-PROFILE.md
index d17e1cb0ce..0225f1a543 100644
--- a/docs/en/docs/sql-manual/sql-reference/Show-Statements/SHOW-LOAD-PROFILE.md
+++ b/docs/en/docs/sql-manual/sql-reference/Show-Statements/SHOW-LOAD-PROFILE.md
@@ -50,6 +50,12 @@ grammar:
 show load profile "/";
 
 show load profile "/[queryId]"
+
+show load profile "/[queryId]/[TaskId]"
+
+show load profile "/[queryId]/[TaskId]/[FragmentId]/"
+
+show load profile "/[queryId]/[TaskId]/[FragmentId]/[InstanceId]"
 ````
 
 This command will list all currently saved import profiles. Each line corresponds to one import. where the QueryId column is the ID of the import job. This ID can also be viewed through the SHOW LOAD statement. We can select the QueryId corresponding to the Profile we want to see to see the specific situation
@@ -109,10 +115,56 @@ WaitAndFetchResultTime: N/A
    +-----------------------------------+------------+
    ````
    
-3. View the Instance overview of the specified subtask
+3. View the plan tree of the specified subtask
+
+   ```sql
+   show load profile "/980014623046410a-af5d36f23381017f/980014623046410a-af5d36f23381017f";
+
+                         ┌───────────────────────┐
+                         │[-1: OlapTableSink]    │
+                         │Fragment: 0            │
+                         │MaxActiveTime: 86.541ms│
+                         └───────────────────────┘
+                                     │
+                                     │
+                           ┌───────────────────┐
+                           │[1: VEXCHANGE_NODE]│
+                           │Fragment: 0        │
+                           └───────────────────┘
+           ┌─────────────────────────┴───────┐
+           │                                 │
+    ┌─────────────┐              ┌───────────────────────┐
+    │[MemoryUsage]│              │[1: VDataStreamSender] │
+    │Fragment: 0  │              │Fragment: 1            │
+    └─────────────┘              │MaxActiveTime: 34.882ms│
+                                 └───────────────────────┘
+                                             │
+                                             │
+                               ┌───────────────────────────┐
+                               │[0: VNewOlapScanNode(tbl1)]│
+                               │Fragment: 1                │
+                               └───────────────────────────┘
+                           ┌─────────────────┴───────┐
+                           │                         │
+                    ┌─────────────┐            ┌───────────┐
+                    │[MemoryUsage]│            │[VScanner] │
+                    │Fragment: 1  │            │Fragment: 1│
+                    └─────────────┘            └───────────┘
+                                             ┌───────┴─────────┐
+                                             │                 │
+                                    ┌─────────────────┐ ┌─────────────┐
+                                    │[SegmentIterator]│ │[MemoryUsage]│
+                                    │Fragment: 1      │ │Fragment: 1  │
+                                    └─────────────────┘ └─────────────┘
+
+   ```sql
+
+   This will show the plan tree and fragment id on it
+
+4. View the Instance overview of the specified subtask
 
    ```sql
-   mysql> show load profile "/980014623046410a-af5d36f23381017f/980014623046410a-af5d36f23381017f/980014623046410a-88e260f0c43031f5"\G
+   mysql> show load profile "/980014623046410a-af5d36f23381017f/980014623046410a-af5d36f23381017f/1"\G
    +-----------------------------------+------------------+------------+
    | Instances                         | Host             | ActiveTime |
    +-----------------------------------+------------------+------------+
@@ -126,84 +178,48 @@ WaitAndFetchResultTime: N/A
 4. Continue to view the detailed Profile of each operator on a specific Instance
 
    ```sql
-   mysql> show load profile "/10441/980014623046410a-88e260f0c43031f1/980014623046410a-88e260f0c43031f5"\G
+   mysql> show load profile "/980014623046410a-af5d36f23381017f/980014623046410a-af5d36f23381017f/1/980014623046410a-88e260f0c43031f5"\G
    
    *************************** 1. row ***************************
    
    Instance:
    
          ┌-----------------------------------------┐
-   
          │[-1: OlapTableSink]                      │
-   
          │(Active: 2m17s, non-child: 70.91)        │
-   
          │  - Counters:                            │
-   
          │      - CloseWaitTime: 1m53s             │
-   
          │      - ConvertBatchTime: 0ns            │
-   
          │      - MaxAddBatchExecTime: 1m46s       │
-   
          │      - NonBlockingSendTime: 3m11s       │
-   
          │      - NumberBatchAdded: 782            │
-   
          │      - NumberNodeChannels: 1            │
-   
          │      - OpenTime: 743.822us              │
-   
          │      - RowsFiltered: 0                  │
-   
          │      - RowsRead: 1.599729M (1599729)    │
-   
          │      - RowsReturned: 1.599729M (1599729)│
-   
          │      - SendDataTime: 11s761ms           │
-   
          │      - TotalAddBatchExecTime: 1m46s     │
-   
          │      - ValidateDataTime: 9s802ms        │
-   
          └-----------------------------------------┘
-   
                               │
-   
    ┌-----------------------------------------------------┐
-   
    │[0: BROKER_SCAN_NODE]                                │
-   
    │(Active: 56s537ms, non-child: 29.06)                 │
-   
    │  - Counters:                                        │
-   
    │      - BytesDecompressed: 0.00                      │
-   
    │      - BytesRead: 5.77 GB                           │
-   
    │      - DecompressTime: 0ns                          │
-   
    │      - FileReadTime: 34s263ms                       │
-   
    │      - MaterializeTupleTime(*): 45s54ms             │
-   
    │      - NumDiskAccess: 0                             │
-   
    │      - PeakMemoryUsage: 33.03 MB                    │
-   
    │      - RowsRead: 1.599729M (1599729)                │
-   
    │      - RowsReturned: 1.599729M (1599729)            │
-   
-   │      - RowsReturnedRate: 28.295K sec               │
-   
+   │      - RowsReturnedRate: 28.295K sec                │
    │      - TotalRawReadTime(*): 1m20s                   │
-   
    │      - TotalReadThroughput: 30.39858627319336 MB/sec│
-   
    │      - WaitScannerTime: 56s528ms                    │
-   
    └-----------------------------------------------------┘
    ````
 
diff --git a/docs/zh-CN/docs/admin-manual/query-profile.md b/docs/zh-CN/docs/admin-manual/query-profile.md
index a2d765a4e5..c305a6a07e 100644
--- a/docs/zh-CN/docs/admin-manual/query-profile.md
+++ b/docs/zh-CN/docs/admin-manual/query-profile.md
@@ -30,6 +30,10 @@ under the License.
 
 本文档主要介绍Doris在查询执行的统计结果。利用这些统计的信息,可以更好的帮助我们了解Doris的执行情况,并有针对性的进行相应**Debug与调优工作**。
 
+也可以参考如下语法在命令行中查看导入和查询的 Profile:
+
+- [SHOW QUERY PROFILE](../sql-manual/sql-reference/Show-Statements/SHOW-QUERY-PROFILE.md)
+- [SHOW LOAD PROFILE](../sql-manual/sql-reference/Show-Statements/SHOW-LOAD-PROFILE.md)
 
 ## 名词解释
 
@@ -273,4 +277,4 @@ OLAP_SCAN_NODE (id=0):(Active: 1.2ms, % non-child: 0.00%)
  - PeakReservation: Reservation的峰值
  - PeakUnpinnedBytes: unpin的内存数据量
  - PeakUsedReservation: Reservation的内存使用量
- - ReservationLimit: BufferPool的Reservation的限制量
\ No newline at end of file
+ - ReservationLimit: BufferPool的Reservation的限制量
diff --git a/docs/zh-CN/docs/sql-manual/sql-reference/Show-Statements/SHOW-LOAD-PROFILE.md b/docs/zh-CN/docs/sql-manual/sql-reference/Show-Statements/SHOW-LOAD-PROFILE.md
index f53f195b67..a63c182b11 100644
--- a/docs/zh-CN/docs/sql-manual/sql-reference/Show-Statements/SHOW-LOAD-PROFILE.md
+++ b/docs/zh-CN/docs/sql-manual/sql-reference/Show-Statements/SHOW-LOAD-PROFILE.md
@@ -50,6 +50,12 @@ SET [GLOBAL] enable_profile=true;
 show load profile "/";
 
 show load profile "/[queryId]"
+
+show load profile "/[queryId]/[TaskId]"
+
+show load profile "/[queryId]/[TaskId]/[FragmentId]/"
+
+show load profile "/[queryId]/[TaskId]/[FragmentId]/[InstanceId]"
 ```
 
 这个命令会列出当前保存的所有导入 Profile。每行对应一个导入。其中 QueryId 列为导入作业的 ID。这个 ID 也可以通过 SHOW LOAD 语句查看拿到。我们可以选择我们想看的 Profile 对应的 QueryId,查看具体情况
@@ -108,11 +114,56 @@ WaitAndFetchResultTime: N/A
    | 980014623046410a-af5d36f23381017f | 3m14s      |
    +-----------------------------------+------------+
    ```
+3. 查看子任务的执行树:
 
-3. 查看指定子任务的 Instance 概况
+   ```sql
+   show load profile "/980014623046410a-af5d36f23381017f/980014623046410a-af5d36f23381017f";
+
+                         ┌───────────────────────┐
+                         │[-1: OlapTableSink]    │
+                         │Fragment: 0            │
+                         │MaxActiveTime: 86.541ms│
+                         └───────────────────────┘
+                                     │
+                                     │
+                           ┌───────────────────┐
+                           │[1: VEXCHANGE_NODE]│
+                           │Fragment: 0        │
+                           └───────────────────┘
+           ┌─────────────────────────┴───────┐
+           │                                 │
+    ┌─────────────┐              ┌───────────────────────┐
+    │[MemoryUsage]│              │[1: VDataStreamSender] │
+    │Fragment: 0  │              │Fragment: 1            │
+    └─────────────┘              │MaxActiveTime: 34.882ms│
+                                 └───────────────────────┘
+                                             │
+                                             │
+                               ┌───────────────────────────┐
+                               │[0: VNewOlapScanNode(tbl1)]│
+                               │Fragment: 1                │
+                               └───────────────────────────┘
+                           ┌─────────────────┴───────┐
+                           │                         │
+                    ┌─────────────┐            ┌───────────┐
+                    │[MemoryUsage]│            │[VScanner] │
+                    │Fragment: 1  │            │Fragment: 1│
+                    └─────────────┘            └───────────┘
+                                             ┌───────┴─────────┐
+                                             │                 │
+                                    ┌─────────────────┐ ┌─────────────┐
+                                    │[SegmentIterator]│ │[MemoryUsage]│
+                                    │Fragment: 1      │ │Fragment: 1  │
+                                    └─────────────────┘ └─────────────┘
 
    ```sql
-   mysql> show load profile "/980014623046410a-af5d36f23381017f/980014623046410a-af5d36f23381017f";
+
+   这一层会显示子任务的查询树,其中标注了 Fragment id。
+
+4. 查看指定Fragment 的 Instance 概况
+
+   ```sql
+   mysql> show load profile "/980014623046410a-af5d36f23381017f/980014623046410a-af5d36f23381017f/1";
    +-----------------------------------+------------------+------------+
    | Instances                         | Host             | ActiveTime |
    +-----------------------------------+------------------+------------+
@@ -123,87 +174,51 @@ WaitAndFetchResultTime: N/A
    +-----------------------------------+------------------+------------+
    ```
 
-4. 继续查看某一个具体的 Instance 上各个算子的详细 Profile
+5. 继续查看某一个具体的 Instance 上各个算子的详细 Profile
 
    ```sql
-   mysql> show load profile "/980014623046410a-af5d36f23381017f/980014623046410a-af5d36f23381017f/980014623046410a-88e260f0c43031f5"\G
+   mysql> show load profile "/980014623046410a-af5d36f23381017f/980014623046410a-af5d36f23381017f/1/980014623046410a-88e260f0c43031f5"\G
    
    *************************** 1. row ***************************
    
    Instance:
    
          ┌-----------------------------------------┐
-   
          │[-1: OlapTableSink]                      │
-   
          │(Active: 2m17s, non-child: 70.91)        │
-   
          │  - Counters:                            │
-   
          │      - CloseWaitTime: 1m53s             │
-   
          │      - ConvertBatchTime: 0ns            │
-   
          │      - MaxAddBatchExecTime: 1m46s       │
-   
          │      - NonBlockingSendTime: 3m11s       │
-   
          │      - NumberBatchAdded: 782            │
-   
          │      - NumberNodeChannels: 1            │
-   
          │      - OpenTime: 743.822us              │
-   
          │      - RowsFiltered: 0                  │
-   
          │      - RowsRead: 1.599729M (1599729)    │
-   
          │      - RowsReturned: 1.599729M (1599729)│
-   
          │      - SendDataTime: 11s761ms           │
-   
          │      - TotalAddBatchExecTime: 1m46s     │
-   
          │      - ValidateDataTime: 9s802ms        │
-   
          └-----------------------------------------┘
-   
                               │
-   
    ┌-----------------------------------------------------┐
-   
    │[0: BROKER_SCAN_NODE]                                │
-   
    │(Active: 56s537ms, non-child: 29.06)                 │
-   
    │  - Counters:                                        │
-   
    │      - BytesDecompressed: 0.00                      │
-   
    │      - BytesRead: 5.77 GB                           │
-   
    │      - DecompressTime: 0ns                          │
-   
    │      - FileReadTime: 34s263ms                       │
-   
    │      - MaterializeTupleTime(*): 45s54ms             │
-   
    │      - NumDiskAccess: 0                             │
-   
    │      - PeakMemoryUsage: 33.03 MB                    │
-   
    │      - RowsRead: 1.599729M (1599729)                │
-   
    │      - RowsReturned: 1.599729M (1599729)            │
-   
-   │      - RowsReturnedRate: 28.295K sec               │
-   
+   │      - RowsReturnedRate: 28.295K sec                │
    │      - TotalRawReadTime(*): 1m20s                   │
-   
    │      - TotalReadThroughput: 30.39858627319336 MB/sec│
-   
    │      - WaitScannerTime: 56s528ms                    │
-   
    └-----------------------------------------------------┘
    ```
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowLoadProfileStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowLoadProfileStmt.java
index 5175348674..91cbfc3c4e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowLoadProfileStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowLoadProfileStmt.java
@@ -40,6 +40,7 @@ public class ShowLoadProfileStmt extends ShowStmt {
     public enum PathType {
         QUERY_IDS,
         TASK_IDS,
+        FRAGMENTS,
         INSTANCES,
         SINGLE_INSTANCE
     }
@@ -49,6 +50,7 @@ public class ShowLoadProfileStmt extends ShowStmt {
 
     private String jobId = "";
     private String taskId = "";
+    private String fragmentId = "";
     private String instanceId = "";
 
     public ShowLoadProfileStmt(String idPath) {
@@ -67,6 +69,10 @@ public class ShowLoadProfileStmt extends ShowStmt {
         return taskId;
     }
 
+    public String getFragmentId() {
+        return fragmentId;
+    }
+
     public String getInstanceId() {
         return instanceId;
     }
@@ -85,8 +91,8 @@ public class ShowLoadProfileStmt extends ShowStmt {
         }
         pathType = PathType.QUERY_IDS;
         String[] parts = idPath.split("/");
-        if (parts.length > 4) {
-            throw new AnalysisException("Path must in format '/jobId/taskId/instanceId'");
+        if (parts.length > 5) {
+            throw new AnalysisException("Path must in format '/jobId/taskId/fragmentId/instanceId'");
         }
 
         for (int i = 0; i < parts.length; i++) {
@@ -100,9 +106,13 @@ public class ShowLoadProfileStmt extends ShowStmt {
                     break;
                 case 2:
                     taskId = parts[i];
-                    pathType = PathType.INSTANCES;
+                    pathType = PathType.FRAGMENTS;
                     break;
                 case 3:
+                    fragmentId = parts[i];
+                    pathType = PathType.INSTANCES;
+                    break;
+                case 4:
                     instanceId = parts[i];
                     pathType = PathType.SINGLE_INSTANCE;
                     break;
@@ -130,6 +140,8 @@ public class ShowLoadProfileStmt extends ShowStmt {
                 return ShowQueryProfileStmt.META_DATA_QUERY_IDS;
             case TASK_IDS:
                 return META_DATA_TASK_IDS;
+            case FRAGMENTS:
+                return ShowQueryProfileStmt.META_DATA_FRAGMENTS;
             case INSTANCES:
                 return ShowQueryProfileStmt.META_DATA_INSTANCES;
             case SINGLE_INSTANCE:
@@ -139,3 +151,4 @@ public class ShowLoadProfileStmt extends ShowStmt {
         }
     }
 }
+
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowQueryProfileStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowQueryProfileStmt.java
index b36cd30159..f595069825 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowQueryProfileStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowQueryProfileStmt.java
@@ -63,7 +63,7 @@ public class ShowQueryProfileStmt extends ShowStmt {
 
     public enum PathType {
         QUERY_IDS,
-        FRAGMETNS,
+        FRAGMENTS,
         INSTANCES,
         SINGLE_INSTANCE
     }
@@ -120,7 +120,7 @@ public class ShowQueryProfileStmt extends ShowStmt {
                     continue;
                 case 1:
                     queryId = parts[i];
-                    pathType = PathType.FRAGMETNS;
+                    pathType = PathType.FRAGMENTS;
                     break;
                 case 2:
                     fragmentId = parts[i];
@@ -152,7 +152,7 @@ public class ShowQueryProfileStmt extends ShowStmt {
         switch (pathType) {
             case QUERY_IDS:
                 return META_DATA_QUERY_IDS;
-            case FRAGMETNS:
+            case FRAGMENTS:
                 return META_DATA_FRAGMENTS;
             case INSTANCES:
                 return META_DATA_INSTANCES;
@@ -163,3 +163,4 @@ public class ShowQueryProfileStmt extends ShowStmt {
         }
     }
 }
+
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java
index c93807ef2e..a6bd0ee5ef 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java
@@ -1990,7 +1990,7 @@ public class ShowExecutor {
             case QUERY_IDS:
                 rows = ProfileManager.getInstance().getQueryWithType(ProfileManager.ProfileType.QUERY);
                 break;
-            case FRAGMETNS: {
+            case FRAGMENTS: {
                 ProfileTreeNode treeRoot = ProfileManager.getInstance().getFragmentProfileTree(showStmt.getQueryId(),
                         showStmt.getQueryId());
                 if (treeRoot == null) {
@@ -2049,12 +2049,22 @@ public class ShowExecutor {
                 rows = ProfileManager.getInstance().getLoadJobTaskList(showStmt.getJobId());
                 break;
             }
+            case FRAGMENTS: {
+                ProfileTreeNode treeRoot = ProfileManager.getInstance().getFragmentProfileTree(showStmt.getJobId(),
+                        showStmt.getJobId());
+                if (treeRoot == null) {
+                    throw new AnalysisException("Failed to get fragment tree for load: " + showStmt.getJobId());
+                }
+                List<String> row = Lists.newArrayList(ProfileTreePrinter.printFragmentTree(treeRoot));
+                rows.add(row);
+                break;
+            }
             case INSTANCES: {
                 // For load profile, there should be only one fragment in each execution profile
                 // And the fragment id is 0.
                 List<Triple<String, String, Long>> instanceList
                         = ProfileManager.getInstance().getFragmentInstanceList(showStmt.getJobId(),
-                        showStmt.getTaskId(), "0");
+                        showStmt.getTaskId(), ((ShowLoadProfileStmt) stmt).getFragmentId());
                 if (instanceList == null) {
                     throw new AnalysisException("Failed to get instance list for task: " + showStmt.getTaskId());
                 }
@@ -2069,7 +2079,7 @@ public class ShowExecutor {
                 // For load profile, there should be only one fragment in each execution profile.
                 // And the fragment id is 0.
                 ProfileTreeNode treeRoot = ProfileManager.getInstance().getInstanceProfileTree(showStmt.getJobId(),
-                        showStmt.getTaskId(), "0", showStmt.getInstanceId());
+                        showStmt.getTaskId(), showStmt.getFragmentId(), showStmt.getInstanceId());
                 if (treeRoot == null) {
                     throw new AnalysisException("Failed to get instance tree for instance: "
                             + showStmt.getInstanceId());
@@ -2487,3 +2497,4 @@ public class ShowExecutor {
         resultSet = new ShowResultSet(showStmt.getMetaData(), results);
     }
 }
+


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


[doris] 09/12: [fix](planner) decimalv2 castTo decimalv2 should change type directly (#18297)

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 d4d994d844f895f6944829e19c8ddf5754c10a5a
Author: morrySnow <10...@users.noreply.github.com>
AuthorDate: Thu Apr 6 13:51:50 2023 +0800

    [fix](planner) decimalv2 castTo decimalv2 should change type directly (#18297)
---
 .../main/java/org/apache/doris/analysis/Expr.java  |  7 ++
 .../data/query_p0/aggregate/select_distinct.out    |  4 ++
 .../query_p0/aggregate/select_distinct.groovy      | 77 ++++++++++++++++++++++
 3 files changed, 88 insertions(+)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/Expr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/Expr.java
index a94a1ee21d..4c76d1bade 100755
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/Expr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/Expr.java
@@ -1392,6 +1392,13 @@ public abstract class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
                 && (this.type.isStringType() || this.type.isHllType())) {
             return this;
         }
+
+        if (targetType.getPrimitiveType() == PrimitiveType.DECIMALV2
+                && this.type.getPrimitiveType() == PrimitiveType.DECIMALV2) {
+            this.type = targetType;
+            return this;
+        }
+
         // Preconditions.checkState(PrimitiveType.isImplicitCast(type, targetType),
         // "cast %s to %s", this.type, targetType);
         // TODO(zc): use implicit cast
diff --git a/regression-test/data/query_p0/aggregate/select_distinct.out b/regression-test/data/query_p0/aggregate/select_distinct.out
new file mode 100644
index 0000000000..8c6c3f37f1
--- /dev/null
+++ b/regression-test/data/query_p0/aggregate/select_distinct.out
@@ -0,0 +1,4 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !distinct_decimal_cast --
+1	5.300000000
+
diff --git a/regression-test/suites/query_p0/aggregate/select_distinct.groovy b/regression-test/suites/query_p0/aggregate/select_distinct.groovy
new file mode 100644
index 0000000000..6456158bda
--- /dev/null
+++ b/regression-test/suites/query_p0/aggregate/select_distinct.groovy
@@ -0,0 +1,77 @@
+// 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("select_distinct") {
+    sql """DROP TABLE IF EXISTS decimal_a;"""
+    sql """DROP TABLE IF EXISTS decimal_b;"""
+    sql """DROP TABLE IF EXISTS decimal_c;"""
+
+    sql """
+    CREATE TABLE IF NOT EXISTS `decimal_a` (
+      `id` int(11) NOT NULL
+    ) ENGINE=OLAP
+    DUPLICATE KEY(`id`)
+    DISTRIBUTED BY HASH(`id`) BUCKETS 1
+    PROPERTIES (
+      "replication_allocation" = "tag.location.default: 1"
+    );
+    """
+
+    sql """
+    CREATE TABLE IF NOT EXISTS `decimal_b` (
+      `id` int(11) NOT NULL,
+      `age` decimal(11, 3) NULL
+    ) ENGINE=OLAP
+    DUPLICATE KEY(`id`)
+    DISTRIBUTED BY HASH(`id`) BUCKETS 1
+    PROPERTIES (
+      "replication_allocation" = "tag.location.default: 1"
+    );
+    """
+
+    sql """
+    CREATE TABLE IF NOT EXISTS `decimal_c` (
+      `id` int(11) NULL
+    ) ENGINE=OLAP
+    DUPLICATE KEY(`id`)
+    DISTRIBUTED BY HASH(`id`) BUCKETS 64
+    PROPERTIES (
+      "replication_allocation" = "tag.location.default: 1"
+    );
+    """
+
+    sql """insert into decimal_a values(1);"""
+    sql """insert into decimal_b values (1, 5.3);"""
+    sql """insert into decimal_c values(1);"""
+
+    qt_distinct_decimal_cast """
+    select distinct
+      decimal_a.id,
+      case
+        when decimal_b.age >= 0 then decimal_b.age
+        when decimal_b.age >= 0 then floor(decimal_b.age/365)
+      end
+    from
+      decimal_a
+      inner join decimal_b on decimal_a.id =decimal_b.id
+      left join decimal_c on decimal_a.id=decimal_c.id;
+    """
+
+    sql """DROP TABLE IF EXISTS decimal_a;"""
+    sql """DROP TABLE IF EXISTS decimal_b;"""
+    sql """DROP TABLE IF EXISTS decimal_c;"""
+}


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


[doris] 08/12: [Bug](DECIMALV3) fix wrong decimal scale returned by function `round` (#18375)

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 5b4d158466a91a3bd9690bac4d84b66ca405d02d
Author: Gabriel <ga...@gmail.com>
AuthorDate: Thu Apr 6 14:44:21 2023 +0800

    [Bug](DECIMALV3) fix wrong decimal scale returned by function `round` (#18375)
---
 be/src/vec/functions/round.h                       |  2 +-
 .../sql_functions/math_functions/test_round.out    |  3 ++
 .../sql_functions/math_functions/test_round.groovy | 40 +++++++++++++++++++++-
 3 files changed, 43 insertions(+), 2 deletions(-)

diff --git a/be/src/vec/functions/round.h b/be/src/vec/functions/round.h
index 456d6264a2..6d24166736 100644
--- a/be/src/vec/functions/round.h
+++ b/be/src/vec/functions/round.h
@@ -457,7 +457,7 @@ struct Dispatcher {
             const auto* const decimal_col = check_and_get_column<ColumnDecimal<T>>(col_general);
             const auto& vec_src = decimal_col->get_data();
 
-            auto col_res = ColumnDecimal<T>::create(vec_src.size(), decimal_col->get_scale());
+            auto col_res = ColumnDecimal<T>::create(vec_src.size(), scale_arg);
             auto& vec_res = col_res->get_data();
 
             if (!vec_res.empty()) {
diff --git a/regression-test/data/query_p0/sql_functions/math_functions/test_round.out b/regression-test/data/query_p0/sql_functions/math_functions/test_round.out
index 6c65a1982f..6363a18165 100644
--- a/regression-test/data/query_p0/sql_functions/math_functions/test_round.out
+++ b/regression-test/data/query_p0/sql_functions/math_functions/test_round.out
@@ -43,3 +43,6 @@
 -- !select --
 10	10	10
 
+-- !query --
+111	001	15.0700	0.2300
+
diff --git a/regression-test/suites/query_p0/sql_functions/math_functions/test_round.groovy b/regression-test/suites/query_p0/sql_functions/math_functions/test_round.groovy
index 7dad51e383..fa00278981 100644
--- a/regression-test/suites/query_p0/sql_functions/math_functions/test_round.groovy
+++ b/regression-test/suites/query_p0/sql_functions/math_functions/test_round.groovy
@@ -48,5 +48,43 @@ suite("test_round") {
     qt_select """ SELECT ceil(col1, -1), ceil(col2, -1), ceil(col3, -1) FROM `${tableName}`; """
     qt_select """ SELECT truncate(col1, -1), truncate(col2, -1), truncate(col3, -1) FROM `${tableName}`; """
 
-    sql """ DROP TABLE IF EXISTS ${tableName} """
+    def tableName1 = "test_round1"
+    sql """ CREATE TABLE `${tableName1}` (
+          `TENANT_ID` varchar(50) NOT NULL,
+          `PUBONLN_PRC` decimalv3(18, 4) NULL,
+          `PRODENTP_CODE` varchar(50) NULL,
+          `ORD_SUMAMT` decimalv3(18, 4) NULL,
+          `PURC_CNT` decimalv3(12, 2) NULL
+        ) ENGINE=OLAP
+        UNIQUE KEY(`TENANT_ID`)
+        DISTRIBUTED BY HASH(`TENANT_ID`) BUCKETS 16
+        PROPERTIES (
+        "replication_allocation" = "tag.location.default: 1",
+        "in_memory" = "false",
+        "storage_format" = "V2",
+        "enable_unique_key_merge_on_write" = "true",
+        "disable_auto_compaction" = "false"
+        ); """
+
+    def tableName2 = "test_round2"
+    sql """ CREATE TABLE `${tableName2}` (
+          `tenant_id` varchar(50) NOT NULL COMMENT '租户ID',
+          `prodentp_code` varchar(50) NULL COMMENT '生产企业代码',
+          `delv_amt` decimalv3(18, 4) NULL DEFAULT "0" COMMENT '配送金额',
+          `ord_sumamt` decimalv3(18, 4) NULL COMMENT '订单总金额'
+        ) ENGINE=OLAP
+        UNIQUE KEY(`tenant_id`, `prodentp_code`)
+        COMMENT '订单明细配送统计'
+        DISTRIBUTED BY HASH(`prodentp_code`) BUCKETS 16
+        PROPERTIES (
+        "replication_allocation" = "tag.location.default: 1",
+        "in_memory" = "false",
+        "storage_format" = "V2",
+        "enable_unique_key_merge_on_write" = "true",
+        "disable_auto_compaction" = "false"
+        );                """
+
+    sql """ insert into ${tableName1} values ('111', 1.2432, '001', 0.2341, 12.1234123); """
+    sql """ insert into ${tableName2} select  TENANT_ID,PRODENTP_CODE,ROUND((MAX(PURC_CNT)*MAX(PUBONLN_PRC)),2) delv_amt,ROUND(SUM(ORD_SUMAMT),2) from ${tableName1} GROUP BY TENANT_ID,PRODENTP_CODE; """
+    qt_query """ select * from ${tableName2} """
 }


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


[doris] 06/12: [Bug](decimal) Fix string to decimal (#18282)

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 cb9849747210eda7c668df3cf23e3e6736e8ab33
Author: Gabriel <ga...@gmail.com>
AuthorDate: Mon Apr 3 15:30:48 2023 +0800

    [Bug](decimal) Fix string to decimal (#18282)
---
 be/src/util/string_parser.hpp | 8 ++------
 1 file changed, 2 insertions(+), 6 deletions(-)

diff --git a/be/src/util/string_parser.hpp b/be/src/util/string_parser.hpp
index 02006b7c7d..4d203a43d6 100644
--- a/be/src/util/string_parser.hpp
+++ b/be/src/util/string_parser.hpp
@@ -646,7 +646,6 @@ inline T StringParser::string_to_decimal(const char* s, int len, int type_precis
     }
 
     // Find the number of truncated digits before adjusting the precision for an exponent.
-    int truncated_digit_count = precision - type_precision;
     if (exponent > scale) {
         // Ex: 0.1e3 (which at this point would have precision == 1 and scale == 1), the
         //     scale must be set to 0 and the value set to 100 which means a precision of 3.
@@ -678,9 +677,6 @@ inline T StringParser::string_to_decimal(const char* s, int len, int type_precis
     } else if (UNLIKELY(scale > type_scale)) {
         *result = StringParser::PARSE_UNDERFLOW;
         int shift = scale - type_scale;
-        if (UNLIKELY(truncated_digit_count > 0)) {
-            shift -= truncated_digit_count;
-        }
         if (shift > 0) {
             T divisor;
             if constexpr (std::is_same_v<T, vectorized::Int128I>) {
@@ -688,14 +684,14 @@ inline T StringParser::string_to_decimal(const char* s, int len, int type_precis
             } else {
                 divisor = get_scale_multiplier<T>(shift);
             }
-            if (LIKELY(divisor >= 0)) {
+            if (LIKELY(divisor > 0)) {
                 T remainder = value % divisor;
                 value /= divisor;
                 if ((remainder > 0 ? T(remainder) : T(-remainder)) >= (divisor >> 1)) {
                     value += 1;
                 }
             } else {
-                DCHECK(divisor == -1); // //DCHECK_EQ doesn't work with __int128.
+                DCHECK(divisor == -1 || divisor == 0); // //DCHECK_EQ doesn't work with __int128.
                 value = 0;
             }
         }


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


[doris] 03/12: [improvement](test) print exception when streamload fails (#18315)

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 c0231b1abfbf0d2b1ae5511ef003454be56eac9a
Author: Yongqiang YANG <98...@users.noreply.github.com>
AuthorDate: Mon Apr 3 08:56:54 2023 +0800

    [improvement](test) print exception when streamload fails (#18315)
---
 .../org/apache/doris/regression/action/StreamLoadAction.groovy       | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)

diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/StreamLoadAction.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/StreamLoadAction.groovy
index 18a54d1e15..cfbf961ac2 100644
--- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/StreamLoadAction.groovy
+++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/action/StreamLoadAction.groovy
@@ -148,7 +148,9 @@ class StreamLoadAction implements SuiteAction {
             ex = t
         }
         long endTime = System.currentTimeMillis()
-        log.info("Stream load elapsed ${endTime - startTime} ms".toString())
+
+        log.info("Stream load elapsed ${endTime - startTime} ms, response: ${responseText}".toString() +
+                 ex.toString())
         checkResult(responseText, ex, startTime, endTime)
     }
 
@@ -193,6 +195,7 @@ class StreamLoadAction implements SuiteAction {
         int code = resp.getStatusLine().getStatusCode()
         if (code != HttpStatus.SC_OK) {
             String streamBody = EntityUtils.toString(resp.getEntity())
+            log.info("Fail to download data ${url}, code: ${code}, body:\n${streamBody}")
             throw new IllegalStateException("Get http stream failed, status code is ${code}, body:\n${streamBody}")
         }
 


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


[doris] 07/12: [improve](clickhouse catalog) Add `"` wrap select column for the sql query clickhouse jdbc (#18352)

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 085cbf5d8ea58549cffd0b0e25a0087d97a10783
Author: yongkang.zhong <zh...@qq.com>
AuthorDate: Tue Apr 4 10:19:24 2023 +0800

    [improve](clickhouse catalog) Add  `"` wrap select column for the sql query clickhouse jdbc (#18352)
---
 fe/fe-core/src/main/java/org/apache/doris/catalog/OdbcTable.java | 7 +++++++
 1 file changed, 7 insertions(+)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OdbcTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/OdbcTable.java
index fb0cf1d3a3..c10bfacaa0 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OdbcTable.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OdbcTable.java
@@ -97,6 +97,11 @@ public class OdbcTable extends Table {
         return list.stream().map(s -> "\"" + s.toUpperCase() + "\"").collect(Collectors.joining("."));
     }
 
+    private static String clickhouseProperName(String name) {
+        List<String> list = Arrays.asList(name.split("\\."));
+        return list.stream().map(s -> "\"" + s + "\"").collect(Collectors.joining("."));
+    }
+
     private static String saphanaProperName(String name) {
         List<String> list = Arrays.asList(name.split("\\."));
         return list.stream().map(s -> "\"" + s.toUpperCase() + "\"").collect(Collectors.joining("."));
@@ -112,6 +117,8 @@ public class OdbcTable extends Table {
                 return psqlProperName(name);
             case ORACLE:
                 return oracleProperName(name);
+            case CLICKHOUSE:
+                return clickhouseProperName(name);
             case SAP_HANA:
                 return saphanaProperName(name);
             default:


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


[doris] 05/12: [fix](regression-test) print real and expect rows when fail in exception (#17949)

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 b5c7e39cb65b78a05e90ad413d444cfc76ad2a16
Author: Yongqiang YANG <98...@users.noreply.github.com>
AuthorDate: Tue Mar 21 08:52:04 2023 +0800

    [fix](regression-test) print real and expect rows when fail in exception (#17949)
---
 .../src/main/groovy/org/apache/doris/regression/util/OutputUtils.groovy  | 1 +
 1 file changed, 1 insertion(+)

diff --git a/regression-test/framework/src/main/groovy/org/apache/doris/regression/util/OutputUtils.groovy b/regression-test/framework/src/main/groovy/org/apache/doris/regression/util/OutputUtils.groovy
index 957f3fb570..1da3e018b2 100644
--- a/regression-test/framework/src/main/groovy/org/apache/doris/regression/util/OutputUtils.groovy
+++ b/regression-test/framework/src/main/groovy/org/apache/doris/regression/util/OutputUtils.groovy
@@ -132,6 +132,7 @@ class OutputUtils {
 
                     def res = checkCell(info, line, expectCell, realCell, dataType)
                     if(res != null) {
+                        res += "line ${line} mismatch\nExpectRow: ${expectRaw}\nRealRow: ${realRaw}";
                         return res
                     }
                 }


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


[doris] 04/12: [fix](quit) be can not quit cleanly due to deadlock (#17971)

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 ff89306e0d83fd7140d430a82def759068a265a7
Author: Yongqiang YANG <98...@users.noreply.github.com>
AuthorDate: Tue Mar 21 12:52:48 2023 +0800

    [fix](quit) be can not quit cleanly due to deadlock (#17971)
---
 be/src/olap/data_dir.cpp | 9 ++++++++-
 be/src/olap/data_dir.h   | 2 +-
 2 files changed, 9 insertions(+), 2 deletions(-)

diff --git a/be/src/olap/data_dir.cpp b/be/src/olap/data_dir.cpp
index 10056b1917..a47aab55d3 100644
--- a/be/src/olap/data_dir.cpp
+++ b/be/src/olap/data_dir.cpp
@@ -119,8 +119,8 @@ Status DataDir::init() {
 }
 
 void DataDir::stop_bg_worker() {
-    std::unique_lock<std::mutex> lck(_check_path_mutex);
     _stop_bg_worker = true;
+    std::unique_lock<std::mutex> lck(_check_path_mutex);
     _check_path_cv.notify_one();
 }
 
@@ -668,6 +668,10 @@ void DataDir::perform_path_scan() {
             continue;
         }
         for (const auto& tablet_id : tablet_ids) {
+            if (_stop_bg_worker) {
+                break;
+            }
+
             auto tablet_id_path = fmt::format("{}/{}", shard_path, tablet_id);
             std::set<std::string> schema_hashes;
             ret = FileUtils::list_dirs_files(tablet_id_path, &schema_hashes, nullptr,
@@ -680,6 +684,9 @@ void DataDir::perform_path_scan() {
 
             for (const auto& schema_hash : schema_hashes) {
                 int32_t interval_ms = config::path_scan_step_interval_ms;
+                if (_stop_bg_worker) {
+                    break;
+                }
                 if (interval_ms > 0) {
                     std::this_thread::sleep_for(std::chrono::milliseconds(interval_ms));
                 }
diff --git a/be/src/olap/data_dir.h b/be/src/olap/data_dir.h
index 20b23c1405..895c1088c4 100644
--- a/be/src/olap/data_dir.h
+++ b/be/src/olap/data_dir.h
@@ -164,7 +164,7 @@ private:
     bool _check_pending_ids(const std::string& id);
 
 private:
-    bool _stop_bg_worker = false;
+    std::atomic<bool> _stop_bg_worker = false;
 
     std::string _path;
     size_t _path_hash;


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


[doris] 01/12: [fix](ssl)refactor some SSL info logs to debug logs (#18234)

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 0dac4b49fe4ca5d6826d4a7c87975eae9f8d4626
Author: abmdocrt <Yu...@gmail.com>
AuthorDate: Fri Mar 31 08:41:02 2023 +0800

    [fix](ssl)refactor some SSL info logs to debug logs (#18234)
---
 fe/fe-core/src/main/java/org/apache/doris/mysql/MysqlProto.java | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/mysql/MysqlProto.java b/fe/fe-core/src/main/java/org/apache/doris/mysql/MysqlProto.java
index ba1920bd9d..8970e56a59 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/mysql/MysqlProto.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/mysql/MysqlProto.java
@@ -182,10 +182,10 @@ public class MysqlProto {
         ByteBuffer handshakeResponse;
 
         if (capability.isClientUseSsl()) {
-            LOG.info("client is using ssl connection.");
+            LOG.debug("client is using ssl connection.");
             // During development, we set SSL mode to true by default.
             if (SERVER_USE_SSL) {
-                LOG.info("server is also using ssl connection. Will use ssl mode for data exchange.");
+                LOG.debug("server is also using ssl connection. Will use ssl mode for data exchange.");
                 MysqlSslContext mysqlSslContext = context.getMysqlSslContext();
                 mysqlSslContext.init();
                 channel.initSslBuffer();
@@ -218,7 +218,7 @@ public class MysqlProto {
 
                 // Set channel mode to ssl mode to handle socket packet in ssl format.
                 channel.setSslMode(true);
-                LOG.info("switch to ssl mode.");
+                LOG.debug("switch to ssl mode.");
                 handshakeResponse = channel.fetchOnePacket();
                 capability = new MysqlCapability(MysqlProto.readLowestInt4(handshakeResponse));
                 if (!capability.isClientUseSsl()) {


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