You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by ca...@apache.org on 2022/10/21 08:07:37 UTC

[doris] branch master updated: [Bugfix](MV) Fix insert negative value to table with bitmap_union MV will cause count distinct result incorrect (#13507)

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

cambyzju pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/doris.git


The following commit(s) were added to refs/heads/master by this push:
     new 3e92f742bf [Bugfix](MV) Fix insert negative value to table with bitmap_union MV will cause count distinct result incorrect (#13507)
3e92f742bf is described below

commit 3e92f742bfd7b0c293a6279cd597535eb26d27a5
Author: Zhengguo Yang <ya...@gmail.com>
AuthorDate: Fri Oct 21 16:07:31 2022 +0800

    [Bugfix](MV) Fix insert negative value to table with bitmap_union MV will cause count distinct result incorrect (#13507)
---
 be/src/exprs/bitmap_function.cpp                   | 24 +++++++
 be/src/exprs/bitmap_function.h                     |  1 +
 be/src/olap/schema_change.cpp                      | 35 ++++++----
 be/src/vec/functions/function_bitmap.cpp           | 13 ++--
 gensrc/script/doris_builtins_functions.py          |  2 +-
 .../rollup/test_materialized_view_bitmap.groovy    | 77 ++++++++++++++++++++++
 6 files changed, 133 insertions(+), 19 deletions(-)

diff --git a/be/src/exprs/bitmap_function.cpp b/be/src/exprs/bitmap_function.cpp
index 834d283a29..1d4fc4df47 100644
--- a/be/src/exprs/bitmap_function.cpp
+++ b/be/src/exprs/bitmap_function.cpp
@@ -173,6 +173,30 @@ StringVal BitmapFunctions::to_bitmap(doris_udf::FunctionContext* ctx,
     return serialize(ctx, &bitmap);
 }
 
+StringVal BitmapFunctions::to_bitmap_with_check(doris_udf::FunctionContext* ctx,
+                                                const doris_udf::StringVal& src) {
+    BitmapValue bitmap;
+
+    if (!src.is_null) {
+        StringParser::ParseResult parse_result = StringParser::PARSE_SUCCESS;
+        uint64_t int_value = StringParser::string_to_unsigned_int<uint64_t>(
+                reinterpret_cast<char*>(src.ptr), src.len, &parse_result);
+        if (parse_result == StringParser::PARSE_SUCCESS) {
+            bitmap.add(int_value);
+        } else {
+            std::stringstream ss;
+            ss << "The input: " << src.to_string()
+               << " is not valid, to_bitmap only support bigint value from 0 to "
+                  "18446744073709551615 currently, cannot load negative values to column with"
+                  " to_bitmap MV on it.";
+            ctx->set_error(ss.str().c_str());
+            return StringVal::null();
+        }
+    }
+
+    return serialize(ctx, &bitmap);
+}
+
 StringVal BitmapFunctions::bitmap_hash(doris_udf::FunctionContext* ctx,
                                        const doris_udf::StringVal& src) {
     BitmapValue bitmap;
diff --git a/be/src/exprs/bitmap_function.h b/be/src/exprs/bitmap_function.h
index 15e164eef1..7de2363d3a 100644
--- a/be/src/exprs/bitmap_function.h
+++ b/be/src/exprs/bitmap_function.h
@@ -68,6 +68,7 @@ public:
 
     static StringVal bitmap_serialize(FunctionContext* ctx, const StringVal& src);
     static StringVal to_bitmap(FunctionContext* ctx, const StringVal& src);
+    static StringVal to_bitmap_with_check(FunctionContext* ctx, const StringVal& src);
     static StringVal bitmap_hash(FunctionContext* ctx, const StringVal& src);
     static StringVal bitmap_hash64(FunctionContext* ctx, const StringVal& src);
     static StringVal bitmap_or(FunctionContext* ctx, const StringVal& src, const StringVal& dst);
diff --git a/be/src/olap/schema_change.cpp b/be/src/olap/schema_change.cpp
index 3295035f4b..45a8e4268b 100644
--- a/be/src/olap/schema_change.cpp
+++ b/be/src/olap/schema_change.cpp
@@ -437,9 +437,11 @@ bool to_bitmap(RowCursor* read_helper, RowCursor* write_helper, const TabletColu
         switch (ref_column.type()) {
         case OLAP_FIELD_TYPE_TINYINT:
             if (*(int8_t*)src < 0) {
-                LOG(WARNING) << "The input: " << *(int8_t*)src
-                             << " is not valid, to_bitmap only support bigint value from 0 to "
-                                "18446744073709551615 currently";
+                LOG(WARNING)
+                        << "The input: " << *(int8_t*)src
+                        << " is not valid, to_bitmap only support bigint value from 0 to "
+                           "18446744073709551615 currently, cannot create MV with to_bitmap on "
+                           "column with negative values.";
                 return false;
             }
             origin_value = *(int8_t*)src;
@@ -449,9 +451,11 @@ bool to_bitmap(RowCursor* read_helper, RowCursor* write_helper, const TabletColu
             break;
         case OLAP_FIELD_TYPE_SMALLINT:
             if (*(int16_t*)src < 0) {
-                LOG(WARNING) << "The input: " << *(int16_t*)src
-                             << " is not valid, to_bitmap only support bigint value from 0 to "
-                                "18446744073709551615 currently";
+                LOG(WARNING)
+                        << "The input: " << *(int16_t*)src
+                        << " is not valid, to_bitmap only support bigint value from 0 to "
+                           "18446744073709551615 currently, cannot create MV with to_bitmap on "
+                           "column with negative values.";
                 return false;
             }
             origin_value = *(int16_t*)src;
@@ -461,9 +465,11 @@ bool to_bitmap(RowCursor* read_helper, RowCursor* write_helper, const TabletColu
             break;
         case OLAP_FIELD_TYPE_INT:
             if (*(int32_t*)src < 0) {
-                LOG(WARNING) << "The input: " << *(int32_t*)src
-                             << " is not valid, to_bitmap only support bigint value from 0 to "
-                                "18446744073709551615 currently";
+                LOG(WARNING)
+                        << "The input: " << *(int32_t*)src
+                        << " is not valid, to_bitmap only support bigint value from 0 to "
+                           "18446744073709551615 currently, cannot create MV with to_bitmap on "
+                           "column with negative values.";
                 return false;
             }
             origin_value = *(int32_t*)src;
@@ -473,9 +479,11 @@ bool to_bitmap(RowCursor* read_helper, RowCursor* write_helper, const TabletColu
             break;
         case OLAP_FIELD_TYPE_BIGINT:
             if (*(int64_t*)src < 0) {
-                LOG(WARNING) << "The input: " << *(int64_t*)src
-                             << " is not valid, to_bitmap only support bigint value from 0 to "
-                                "18446744073709551615 currently";
+                LOG(WARNING)
+                        << "The input: " << *(int64_t*)src
+                        << " is not valid, to_bitmap only support bigint value from 0 to "
+                           "18446744073709551615 currently, cannot create MV with to_bitmap on "
+                           "column with negative values.";
                 return false;
             }
             origin_value = *(int64_t*)src;
@@ -1747,7 +1755,8 @@ Status SchemaChangeHandler::process_alter_tablet_v2(const TAlterTabletReqV2& req
 
 std::shared_mutex SchemaChangeHandler::_mutex;
 std::unordered_set<int64_t> SchemaChangeHandler::_tablet_ids_in_converting;
-std::set<std::string> SchemaChangeHandler::_supported_functions = {"hll_hash", "to_bitmap"};
+std::set<std::string> SchemaChangeHandler::_supported_functions = {"hll_hash", "to_bitmap",
+                                                                   "to_bitmap_with_check"};
 
 // In the past schema change and rollup will create new tablet  and will wait for txns starting before the task to finished
 // It will cost a lot of time to wait and the task is very difficult to understand.
diff --git a/be/src/vec/functions/function_bitmap.cpp b/be/src/vec/functions/function_bitmap.cpp
index 61e0f99f8c..3db2c1033a 100644
--- a/be/src/vec/functions/function_bitmap.cpp
+++ b/be/src/vec/functions/function_bitmap.cpp
@@ -108,11 +108,14 @@ struct ToBitmapWithCheck {
                 if (LIKELY(parse_result == StringParser::PARSE_SUCCESS)) {
                     res_data[i].add(int_value);
                 } else {
-                    LOG(WARNING) << "The input: " << raw_str
-                                 << " is not valid, to_bitmap only support bigint value from 0 to "
-                                    "18446744073709551615 currently";
-                    return Status::InternalError(
-                            "bitmap value must be in [0, 18446744073709551615)");
+                    std::stringstream ss;
+                    ss << "The input: " << std::string(raw_str, str_size)
+                       << " is not valid, to_bitmap only support bigint value from 0 to "
+                          "18446744073709551615 currently, cannot create MV with to_bitmap on "
+                          "column with negative values or cannot load negative values to column "
+                          "with to_bitmap MV on it.";
+                    LOG(WARNING) << ss.str();
+                    return Status::InternalError(ss.str());
                 }
             }
         }
diff --git a/gensrc/script/doris_builtins_functions.py b/gensrc/script/doris_builtins_functions.py
index 224d14fc70..68308b64e0 100755
--- a/gensrc/script/doris_builtins_functions.py
+++ b/gensrc/script/doris_builtins_functions.py
@@ -2432,7 +2432,7 @@ visible_functions = [
         '_ZN5doris15BitmapFunctions9to_bitmapEPN9doris_udf15FunctionContextERKNS1_9StringValE',
         '', '', 'vec', 'ALWAYS_NOT_NULLABLE'],
     [['to_bitmap_with_check'], 'BITMAP', ['VARCHAR'],
-        '_ZN5doris15BitmapFunctions9to_bitmapEPN9doris_udf15FunctionContextERKNS1_9StringValE',
+        '_ZN5doris15BitmapFunctions20to_bitmap_with_checkEPN9doris_udf15FunctionContextERKNS1_9StringValE',
         '', '', 'vec', 'ALWAYS_NOT_NULLABLE'],
     [['bitmap_hash'], 'BITMAP', ['VARCHAR'],
         '_ZN5doris15BitmapFunctions11bitmap_hashEPN9doris_udf15FunctionContextERKNS1_9StringValE',
diff --git a/regression-test/suites/rollup/test_materialized_view_bitmap.groovy b/regression-test/suites/rollup/test_materialized_view_bitmap.groovy
new file mode 100644
index 0000000000..56a1bc1a7c
--- /dev/null
+++ b/regression-test/suites/rollup/test_materialized_view_bitmap.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("test_materialized_view_bitmap", "rollup") {
+    def tbName1 = "test_materialized_view_bitmap"
+
+    def getJobState = { tableName ->
+        def jobStateResult = sql """  SHOW ALTER TABLE MATERIALIZED VIEW WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1; """
+        return jobStateResult[0][8]
+    }
+    sql "DROP TABLE IF EXISTS ${tbName1}"
+    sql """
+            CREATE TABLE ${tbName1}(
+                k1 BOOLEAN NOT NULL,
+                k2 TINYINT NOT NULL,
+                k3 SMALLINT NOT NULL
+            ) 
+            DISTRIBUTED BY HASH(k1) properties("replication_num" = "1");
+        """
+
+    sql "CREATE MATERIALIZED VIEW test_neg as select k1,bitmap_union(to_bitmap(k2)), bitmap_union(to_bitmap(k3)) FROM ${tbName1} GROUP BY k1;"
+    max_try_secs = 60
+    while (max_try_secs--) {
+        String res = getJobState(tbName1)
+        if (res == "FINISHED") {
+            break
+        } else {
+            Thread.sleep(2000)
+            if (max_try_secs < 1) {
+                println "test timeout," + "state:" + res
+                assertEquals("FINISHED",res)
+            }
+        }
+    }
+
+    sql "set enable_vectorized_engine=false"
+    explain {
+        sql "insert into ${tbName1} values(1,1,1);"
+        contains "to_bitmap_with_check"
+    }
+    sql "set enable_vectorized_engine=true"
+    explain {
+        sql "insert into ${tbName1} values(1,1,1);"
+        contains "to_bitmap_with_check"
+    }
+    sql "insert into ${tbName1} values(1,1,1);"
+    sql "set enable_vectorized_engine=false"
+    sql "insert into ${tbName1} values(0,1,1);"
+    sql "set enable_vectorized_engine=true"
+
+    test {
+        sql "insert into ${tbName1} values(1,-1,-1);"
+        // check exception message contains
+        exception "The input: -1 is not valid, to_bitmap only support bigint value from 0 to 18446744073709551615 currently"
+    }
+    sql "set enable_vectorized_engine=false"
+    test {
+        sql "insert into ${tbName1} values(1,-1,-1);"
+        // check exception message contains
+        exception "The input: -1 is not valid, to_bitmap only support bigint value from 0 to 18446744073709551615 currently"
+    }
+
+    sql "DROP TABLE ${tbName1} FORCE;"
+}


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