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 2022/06/02 16:17:19 UTC

[incubator-doris] branch dev-1.0.1 updated (8bdc7195a9 -> ad0216f823)

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

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


    from 8bdc7195a9 [hotfix] fix unused import from ##9618
     new 3182672279 [fix](Function) fix to_bitmap to return always not nullable (#9859)
     new 9a7d70b2f0 [fix] fix invalid SQL rewrite for field in materialized view (#9877)
     new c4ef128d55 [Vectorized][Function] fix bitmap_intersect get wrong result (#9907)
     new ad0216f823 [fix](hive) fix bug of invalid user info in external table's scan node (#9908)

The 4 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:
 .../aggregate_function_bitmap.h                    | 13 +++++--
 be/src/vec/functions/function_bitmap.cpp           | 45 ++++++++++++++--------
 .../org/apache/doris/planner/BrokerScanNode.java   |  2 +-
 .../doris/rewrite/mvrewrite/CountFieldToSum.java   | 16 ++++----
 .../planner/MaterializedViewFunctionTest.java      | 10 +++++
 ..._variance_agg.out => test_bitmap_intersect.out} |  5 +--
 ...nce_agg.groovy => test_bitmap_intersect.groovy} | 26 ++++++-------
 7 files changed, 74 insertions(+), 43 deletions(-)
 copy regression-test/data/correctness/{test_select_variance_agg.out => test_bitmap_intersect.out} (71%)
 copy regression-test/suites/correctness/{test_select_variance_agg.groovy => test_bitmap_intersect.groovy} (55%)


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


[incubator-doris] 01/04: [fix](Function) fix to_bitmap to return always not nullable (#9859)

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

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

commit 318267227961c1d347e77e8ab07c9b85b99dc3b0
Author: zhangstar333 <87...@users.noreply.github.com>
AuthorDate: Thu Jun 2 10:37:45 2022 +0800

    [fix](Function) fix to_bitmap to return always not nullable (#9859)
---
 be/src/vec/functions/function_bitmap.cpp | 45 +++++++++++++++++++++-----------
 1 file changed, 30 insertions(+), 15 deletions(-)

diff --git a/be/src/vec/functions/function_bitmap.cpp b/be/src/vec/functions/function_bitmap.cpp
index 0b80c88b5d..69b5bd6737 100644
--- a/be/src/vec/functions/function_bitmap.cpp
+++ b/be/src/vec/functions/function_bitmap.cpp
@@ -39,24 +39,39 @@ struct BitmapEmpty {
 
 struct ToBitmap {
     static constexpr auto name = "to_bitmap";
+    using ReturnType = DataTypeBitMap;
 
-    static Status vector(const ColumnString::Chars& data, const ColumnString::Offsets& offsets,
-                         std::vector<BitmapValue>& res, NullMap& null_map) {
-        auto size = offsets.size();
-        res.reserve(size);
-        for (size_t i = 0; i < size; ++i) {
-            res.emplace_back();
+    static void vector(const ColumnString::Chars& data, const ColumnString::Offsets& offsets,
+                       MutableColumnPtr& col_res) {
+        execute<false>(data, offsets, nullptr, col_res);
+    }
 
-            const char* raw_str = reinterpret_cast<const char*>(&data[offsets[i - 1]]);
-            size_t str_size = offsets[i] - offsets[i - 1] - 1;
-            StringParser::ParseResult parse_result = StringParser::PARSE_SUCCESS;
-            uint64_t int_value = StringParser::string_to_unsigned_int<uint64_t>(raw_str, str_size,
-                                                                                &parse_result);
-            if (LIKELY(parse_result == StringParser::PARSE_SUCCESS)) {
-                res.back().add(int_value);
+    static void vector_nullable(const ColumnString::Chars& data,
+                                const ColumnString::Offsets& offsets, const NullMap& nullmap,
+                                MutableColumnPtr& col_res) {
+        execute<true>(data, offsets, &nullmap, col_res);
+    }
+    template <bool arg_is_nullable>
+    static void execute(const ColumnString::Chars& data, const ColumnString::Offsets& offsets,
+                        const NullMap* nullmap, MutableColumnPtr& col_res) {
+        auto* res_column = reinterpret_cast<ColumnBitmap*>(col_res.get());
+        auto& res_data = res_column->get_data();
+        size_t size = offsets.size();
+
+        for (size_t i = 0; i < size; ++i) {
+            if (arg_is_nullable && ((*nullmap)[i])) {
+                continue;
+            } else {
+                const char* raw_str = reinterpret_cast<const char*>(&data[offsets[i - 1]]);
+                size_t str_size = offsets[i] - offsets[i - 1] - 1;
+                StringParser::ParseResult parse_result = StringParser::PARSE_SUCCESS;
+                uint64_t int_value = StringParser::string_to_unsigned_int<uint64_t>(
+                        raw_str, str_size, &parse_result);
+                if (LIKELY(parse_result == StringParser::PARSE_SUCCESS)) {
+                    res_data[i].add(int_value);
+                }
             }
         }
-        return Status::OK();
     }
 };
 
@@ -494,7 +509,7 @@ public:
 };
 
 using FunctionBitmapEmpty = FunctionConst<BitmapEmpty, false>;
-using FunctionToBitmap = FunctionBitmapAlwaysNull<ToBitmap>;
+using FunctionToBitmap = FunctionAlwaysNotNullable<ToBitmap>;
 using FunctionBitmapFromString = FunctionBitmapAlwaysNull<BitmapFromString>;
 using FunctionBitmapHash = FunctionAlwaysNotNullable<BitmapHash>;
 


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


[incubator-doris] 03/04: [Vectorized][Function] fix bitmap_intersect get wrong result (#9907)

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

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

commit c4ef128d559b8a5cf989684dd95a57acf95f83b5
Author: zhangstar333 <87...@users.noreply.github.com>
AuthorDate: Wed Jun 1 23:51:52 2022 +0800

    [Vectorized][Function] fix bitmap_intersect get wrong result (#9907)
---
 .../aggregate_function_bitmap.h                    | 13 +++++--
 .../data/correctness/test_bitmap_intersect.out     |  4 +++
 .../correctness/test_bitmap_intersect.groovy       | 41 ++++++++++++++++++++++
 3 files changed, 55 insertions(+), 3 deletions(-)

diff --git a/be/src/vec/aggregate_functions/aggregate_function_bitmap.h b/be/src/vec/aggregate_functions/aggregate_function_bitmap.h
index 939421656e..f4a237dbb9 100644
--- a/be/src/vec/aggregate_functions/aggregate_function_bitmap.h
+++ b/be/src/vec/aggregate_functions/aggregate_function_bitmap.h
@@ -38,7 +38,7 @@ struct AggregateFunctionBitmapUnionOp {
 
     static void add(BitmapValue& res, const BitmapValue& data, bool& is_first) { res |= data; }
 
-    static void merge(BitmapValue& res, const BitmapValue& data) { res |= data; }
+    static void merge(BitmapValue& res, const BitmapValue& data, bool& is_first) { res |= data; }
 };
 
 struct AggregateFunctionBitmapIntersectOp {
@@ -53,7 +53,14 @@ struct AggregateFunctionBitmapIntersectOp {
         }
     }
 
-    static void merge(BitmapValue& res, const BitmapValue& data) { res &= data; }
+    static void merge(BitmapValue& res, const BitmapValue& data, bool& is_first) {
+        if (UNLIKELY(is_first)) {
+            res = data;
+            is_first = false;
+        } else {
+            res &= data;
+        }
+    }
 };
 
 template <typename Op>
@@ -66,7 +73,7 @@ struct AggregateFunctionBitmapData {
         Op::add(value, data, is_first);
     }
 
-    void merge(const BitmapValue& data) { Op::merge(value, data); }
+    void merge(const BitmapValue& data) { Op::merge(value, data, is_first); }
 
     void write(BufferWritable& buf) const { DataTypeBitMap::serialize_as_stream(value, buf); }
 
diff --git a/regression-test/data/correctness/test_bitmap_intersect.out b/regression-test/data/correctness/test_bitmap_intersect.out
new file mode 100644
index 0000000000..eb88af08bf
--- /dev/null
+++ b/regression-test/data/correctness/test_bitmap_intersect.out
@@ -0,0 +1,4 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !select_default --
+1,2
+
diff --git a/regression-test/suites/correctness/test_bitmap_intersect.groovy b/regression-test/suites/correctness/test_bitmap_intersect.groovy
new file mode 100644
index 0000000000..63fd337411
--- /dev/null
+++ b/regression-test/suites/correctness/test_bitmap_intersect.groovy
@@ -0,0 +1,41 @@
+// 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("test_bitmap_intersect") {
+     def tableName = "test_bitmap"
+
+
+     sql """ DROP TABLE IF EXISTS ${tableName} """
+     sql """
+        create table ${tableName} (tag varchar(20),user_ids bitmap bitmap_union) aggregate key (tag) 
+        distributed by hash (tag) PROPERTIES("replication_num" = "1"); 
+     """
+
+     sql "   insert into ${tableName} values('A', to_bitmap(1)); "
+     sql "   insert into ${tableName} values('A', to_bitmap(2)); "
+     sql "   insert into ${tableName} values('A', to_bitmap(3)); "
+     sql "   insert into ${tableName} values('B', to_bitmap(1)); "
+     sql "   insert into ${tableName} values('B', to_bitmap(2)); "
+    
+     // test_vectorized
+     sql """ set enable_vectorized_engine = true; """
+
+     qt_select_default """ 
+     select bitmap_to_string(bitmap_intersect(user_ids)) from ( select tag, bitmap_union(user_ids) user_ids 
+     from ${tableName} group by tag ) t;  """
+
+ }
\ 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


[incubator-doris] 02/04: [fix] fix invalid SQL rewrite for field in materialized view (#9877)

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

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

commit 9a7d70b2f097d0c29ba85dca18f72018503cf71a
Author: Kikyou1997 <33...@users.noreply.github.com>
AuthorDate: Thu Jun 2 23:43:13 2022 +0800

    [fix] fix invalid SQL rewrite for field in materialized view (#9877)
---
 .../apache/doris/rewrite/mvrewrite/CountFieldToSum.java  | 16 +++++++++-------
 .../doris/planner/MaterializedViewFunctionTest.java      | 10 ++++++++++
 2 files changed, 19 insertions(+), 7 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/rewrite/mvrewrite/CountFieldToSum.java b/fe/fe-core/src/main/java/org/apache/doris/rewrite/mvrewrite/CountFieldToSum.java
index ec3ce7e670..0b06cd52eb 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/rewrite/mvrewrite/CountFieldToSum.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/rewrite/mvrewrite/CountFieldToSum.java
@@ -22,7 +22,6 @@ import org.apache.doris.analysis.CreateMaterializedViewStmt;
 import org.apache.doris.analysis.Expr;
 import org.apache.doris.analysis.FunctionCallExpr;
 import org.apache.doris.analysis.SlotRef;
-import org.apache.doris.analysis.TableName;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.FunctionSet;
 import org.apache.doris.catalog.OlapTable;
@@ -81,15 +80,18 @@ public class CountFieldToSum implements ExprRewriteRule {
         }
 
         // rewrite expr
-        return rewriteExpr(fnChild0, mvColumn, analyzer);
+        return rewriteExpr(mvColumn, analyzer);
     }
 
-    private Expr rewriteExpr(SlotRef queryColumnSlotRef, Column mvColumn, Analyzer analyzer) {
+    private Expr rewriteExpr(Column mvColumn, Analyzer analyzer) {
         Preconditions.checkNotNull(mvColumn);
-        Preconditions.checkNotNull(queryColumnSlotRef);
-        TableName tableName = queryColumnSlotRef.getTableName();
-        Preconditions.checkNotNull(tableName);
-        SlotRef mvSlotRef = new SlotRef(tableName, mvColumn.getName());
+        // Notice that we shouldn't set table name field of mvSlotRef here, for we will analyze the new mvSlotRef
+        // later, if the table name was set here, the Analyzer::registerColumnRef would invoke
+        // Analyzer::resolveColumnRef(TableName, String) which only try to find the column from the tupleByAlias,
+        // as at the most time the alias is not equal with the origin table name, so it would cause the unexpected
+        // exception to Unknown column, because we can't find an alias which named as origin table name that has
+        // required column.
+        SlotRef mvSlotRef = new SlotRef(null, mvColumn.getName());
         List<Expr> newFnParams = Lists.newArrayList();
         newFnParams.add(mvSlotRef);
         FunctionCallExpr result = new FunctionCallExpr("sum", newFnParams);
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 a980ff9a67..1e34a47b84 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
@@ -850,4 +850,14 @@ public class MaterializedViewFunctionTest {
         dorisAssert.query(query).explainContains("mv");
         dorisAssert.dropTable("agg_table", true);
     }
+
+    @Test
+    public void testSelectMVWithTableAlias() throws Exception {
+        String createUserTagMVSql = "create materialized view " + USER_TAG_MV_NAME + " as select user_id, "
+                + "count(tag_id) from " + USER_TAG_TABLE_NAME + " group by user_id;";
+        dorisAssert.withMaterializedView(createUserTagMVSql);
+        String query = "select count(tag_id) from " + USER_TAG_TABLE_NAME + " t ;";
+        String mvColumnName = CreateMaterializedViewStmt.mvColumnBuilder(FunctionSet.COUNT, "tag_id");
+        dorisAssert.query(query).explainContains(USER_TAG_MV_NAME, mvColumnName);
+    }
 }


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


[incubator-doris] 04/04: [fix](hive) fix bug of invalid user info in external table's scan node (#9908)

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

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

commit ad0216f8236112e80996208f4bfccc873d9bc379
Author: slothever <18...@users.noreply.github.com>
AuthorDate: Thu Jun 2 10:41:40 2022 +0800

    [fix](hive) fix bug of invalid user info in external table's scan node (#9908)
    
    Fix the hive external table scan node null exception
    Now hive external table query will fail when use local user@ip
---
 fe/fe-core/src/main/java/org/apache/doris/planner/BrokerScanNode.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/BrokerScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/BrokerScanNode.java
index 3a99bc0203..417b91e145 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/BrokerScanNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/BrokerScanNode.java
@@ -151,7 +151,7 @@ public class BrokerScanNode extends LoadScanNode {
         this.fileStatusesList = fileStatusesList;
         this.filesAdded = filesAdded;
         if (ConnectContext.get() != null) {
-            this.userIdentity = ConnectContext.get().getUserIdentity();
+            this.userIdentity = ConnectContext.get().getCurrentUserIdentity();
         }
     }
 


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