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/05/16 16:39:12 UTC

[incubator-doris] 02/17: [fix](function) fix last_value get wrong result when have order by clause (#9247)

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 3684be8a52ba51daa36fc1ddbf4cf4586bd4bbc7
Author: zhangstar333 <87...@users.noreply.github.com>
AuthorDate: Sun May 15 23:56:01 2022 +0800

    [fix](function) fix last_value get wrong result when have order by clause (#9247)
---
 .../aggregate_function_window.h                    |   6 +-
 be/test/vec/function/function_bitmap_test.cpp      | 135 +++++++++------------
 .../org/apache/doris/analysis/AnalyticExpr.java    |   5 +-
 .../data/correctness/test_last_value_window.out    |   8 ++
 .../correctness/test_last_value_window.groovy      |  51 ++++++++
 5 files changed, 119 insertions(+), 86 deletions(-)

diff --git a/be/src/vec/aggregate_functions/aggregate_function_window.h b/be/src/vec/aggregate_functions/aggregate_function_window.h
index 133efe7ea0..23c40df7eb 100644
--- a/be/src/vec/aggregate_functions/aggregate_function_window.h
+++ b/be/src/vec/aggregate_functions/aggregate_function_window.h
@@ -180,7 +180,7 @@ private:
     std::string _copied_value;
 };
 
-template <typename T, bool is_nullable, bool is_string, typename StoreType = Value>
+template <typename T, bool result_is_nullable, bool is_string, typename StoreType = Value>
 struct LeadAndLagData {
 public:
     bool has_init() const { return _is_init; }
@@ -193,7 +193,7 @@ public:
     }
 
     void insert_result_into(IColumn& to) const {
-        if constexpr (is_nullable) {
+        if constexpr (result_is_nullable) {
             if (_data_value.is_null()) {
                 auto& col = assert_cast<ColumnNullable&>(to);
                 col.insert_default();
@@ -220,7 +220,7 @@ public:
     }
 
     void set_value(const IColumn** columns, int64_t pos) {
-        if constexpr (is_nullable) {
+        if (is_column_nullable(*columns[0])) {
             const auto* nullable_column = check_and_get_column<ColumnNullable>(columns[0]);
             if (nullable_column && nullable_column->is_null_at(pos)) {
                 _data_value.set_null(true);
diff --git a/be/test/vec/function/function_bitmap_test.cpp b/be/test/vec/function/function_bitmap_test.cpp
index 2341c9152e..4e8282f216 100644
--- a/be/test/vec/function/function_bitmap_test.cpp
+++ b/be/test/vec/function/function_bitmap_test.cpp
@@ -26,157 +26,135 @@ TEST(function_bitmap_test, function_bitmap_min_test) {
     std::string func_name = "bitmap_min";
     InputTypeSet input_types = {TypeIndex::BitMap};
 
-    auto bitmap1 = new BitmapValue(1);
-    auto bitmap2 = new BitmapValue(std::vector<uint64_t>({1, 9999999}));
-    auto empty_bitmap = new BitmapValue();
-    DataSet data_set = {{{bitmap1}, (int64_t)1},
-                        {{bitmap2}, (int64_t)1},
-                        {{empty_bitmap}, Null()},
+    BitmapValue bitmap1(1);
+    BitmapValue bitmap2({1, 9999999});
+    BitmapValue empty_bitmap;
+    DataSet data_set = {{{&bitmap1}, (int64_t)1},
+                        {{&bitmap2}, (int64_t)1},
+                        {{&empty_bitmap}, Null()},
                         {{Null()}, Null()}};
 
     check_function<DataTypeInt64, true>(func_name, input_types, data_set);
-    delete bitmap1;
-    delete bitmap2;
-    delete empty_bitmap;
 }
 TEST(function_bitmap_test, function_bitmap_max_test) {
     std::string func_name = "bitmap_max";
     InputTypeSet input_types = {TypeIndex::BitMap};
 
-    auto bitmap1 = new BitmapValue(1);
-    auto bitmap2 = new BitmapValue(std::vector<uint64_t>({1, 9999999}));
-    auto empty_bitmap = new BitmapValue();
-    DataSet data_set = {{{bitmap1}, (int64_t)1},
-                        {{bitmap2}, (int64_t)9999999},
-                        {{empty_bitmap}, Null()},
+    BitmapValue bitmap1(1);
+    BitmapValue bitmap2({1, 9999999});
+    BitmapValue empty_bitmap;
+    DataSet data_set = {{{&bitmap1}, (int64_t)1},
+                        {{&bitmap2}, (int64_t)9999999},
+                        {{&empty_bitmap}, Null()},
                         {{Null()}, Null()}};
 
     check_function<DataTypeInt64, true>(func_name, input_types, data_set);
-    delete bitmap1;
-    delete bitmap2;
-    delete empty_bitmap;
 }
 
 TEST(function_bitmap_test, function_bitmap_to_string_test) {
     std::string func_name = "bitmap_to_string";
     InputTypeSet input_types = {TypeIndex::BitMap};
 
-    auto bitmap1 = new BitmapValue(1);
-    auto bitmap2 = new BitmapValue(std::vector<uint64_t>({1, 9999999}));
-    auto empty_bitmap = new BitmapValue();
-    DataSet data_set = {{{bitmap1}, std::string("1")},
-                        {{bitmap2}, std::string("1,9999999")},
-                        {{empty_bitmap}, std::string("")},
+    BitmapValue bitmap1(1);
+    BitmapValue bitmap2({1, 9999999});
+    BitmapValue empty_bitmap;
+    DataSet data_set = {{{&bitmap1}, std::string("1")},
+                        {{&bitmap2}, std::string("1,9999999")},
+                        {{&empty_bitmap}, std::string("")},
                         {{Null()}, Null()}};
 
     check_function<DataTypeString, true>(func_name, input_types, data_set);
-    delete bitmap1;
-    delete bitmap2;
-    delete empty_bitmap;
 }
 
 TEST(function_bitmap_test, function_bitmap_and_count) {
     std::string func_name = "bitmap_and_count";
     InputTypeSet input_types = {TypeIndex::BitMap, TypeIndex::BitMap};
-    auto bitmap1 = new BitmapValue(std::vector<uint64_t>({1, 2, 3}));
-    auto bitmap2 = new BitmapValue(std::vector<uint64_t>({3, 4, 5}));
-    auto empty_bitmap = new BitmapValue();
-    DataSet data_set = {{{bitmap1, empty_bitmap}, (int64_t)0},
-                        {{bitmap1, bitmap1}, (int64_t)3},
-                        {{bitmap1, bitmap2}, (int64_t)1}};
+    BitmapValue bitmap1({1, 2, 3});
+    BitmapValue bitmap2({3, 4, 5});
+    BitmapValue empty_bitmap;
+    DataSet data_set = {{{&bitmap1, &empty_bitmap}, (int64_t)0},
+                        {{&bitmap1, &bitmap1}, (int64_t)3},
+                        {{&bitmap1, &bitmap2}, (int64_t)1}};
 
     check_function<DataTypeInt64, true>(func_name, input_types, data_set);
-    delete bitmap1;
-    delete bitmap2;
-    delete empty_bitmap;
 
     {
         InputTypeSet input_types = {TypeIndex::BitMap, TypeIndex::BitMap, TypeIndex::BitMap};
         BitmapValue bitmap1({33, 1, 2019});
         BitmapValue bitmap2({0, 33, std::numeric_limits<uint64_t>::min()});
         BitmapValue bitmap3({33, 5, std::numeric_limits<uint64_t>::max()});
-        auto empty_bitmap = new BitmapValue(); //test empty
+        BitmapValue empty_bitmap; //test empty
 
-        DataSet data_set = {{{&bitmap1, &bitmap2, empty_bitmap}, (int64_t)0},
+        DataSet data_set = {{{&bitmap1, &bitmap2, &empty_bitmap}, (int64_t)0},
                             {{&bitmap1, &bitmap2, &bitmap3}, (int64_t)1}, //33
                             {{&bitmap1, &bitmap2, Null()}, Null()},
                             {{&bitmap1, &bitmap3, &bitmap3}, (int64_t)1}}; //33
 
         check_function<DataTypeInt64, true>(func_name, input_types, data_set);
-        delete empty_bitmap;
     }
 }
 
 TEST(function_bitmap_test, function_bitmap_or_count) {
     std::string func_name = "bitmap_or_count";
     InputTypeSet input_types = {TypeIndex::BitMap, TypeIndex::BitMap};
-    auto bitmap1 = new BitmapValue(std::vector<uint64_t>({1, 2, 3}));
-    auto bitmap2 = new BitmapValue(std::vector<uint64_t>({1, 2, 3, 4}));
-    auto bitmap3 = new BitmapValue(std::vector<uint64_t>({2, 3}));
-    auto empty_bitmap = new BitmapValue();
-    DataSet data_set = {{{bitmap1, empty_bitmap}, (int64_t)3},
-                        {{bitmap2, bitmap3}, (int64_t)4},
-                        {{bitmap1, bitmap3}, (int64_t)3}};
+
+    BitmapValue bitmap1({1, 2, 3});
+    BitmapValue bitmap2({1, 2, 3, 4});
+    BitmapValue bitmap3({2, 3});
+    BitmapValue empty_bitmap;
+    DataSet data_set = {{{&bitmap1, &empty_bitmap}, (int64_t)3},
+                        {{&bitmap2, &bitmap3}, (int64_t)4},
+                        {{&bitmap1, &bitmap3}, (int64_t)3}};
 
     check_function<DataTypeInt64, true>(func_name, input_types, data_set);
-    delete bitmap1;
-    delete bitmap2;
-    delete bitmap3;
-    delete empty_bitmap;
 
     {
         InputTypeSet input_types = {TypeIndex::BitMap, TypeIndex::BitMap, TypeIndex::BitMap};
         BitmapValue bitmap1({1024, 1, 2019});
         BitmapValue bitmap2({0, 33, std::numeric_limits<uint64_t>::min()});
         BitmapValue bitmap3({33, 5, std::numeric_limits<uint64_t>::max()}); //18446744073709551615
-        auto empty_bitmap = new BitmapValue();                              //test empty
+        BitmapValue empty_bitmap;                                           //test empty
 
-        DataSet data_set = {{{&bitmap1, &bitmap2, empty_bitmap}, (int64_t)5}, //0,1,33,1024,2019
+        DataSet data_set = {{{&bitmap1, &bitmap2, &empty_bitmap}, (int64_t)5}, //0,1,33,1024,2019
                             {{&bitmap1, &bitmap2, &bitmap3},
                              (int64_t)7}, //0,1,5,33,1024,2019,18446744073709551615
-                            {{&bitmap1, empty_bitmap, Null()}, Null()},
+                            {{&bitmap1, &empty_bitmap, Null()}, Null()},
                             {{&bitmap1, &bitmap3, &bitmap3},
                              (int64_t)6}}; //1,5,33,1024,2019,18446744073709551615
 
         check_function<DataTypeInt64, true>(func_name, input_types, data_set);
-        delete empty_bitmap;
     }
 }
 
 TEST(function_bitmap_test, function_bitmap_xor_count) {
     std::string func_name = "bitmap_xor_count";
     InputTypeSet input_types = {TypeIndex::BitMap, TypeIndex::BitMap};
-    auto bitmap1 = new BitmapValue(std::vector<uint64_t>({1, 2, 3}));
-    auto bitmap2 = new BitmapValue(std::vector<uint64_t>({1, 2, 3, 4}));
-    auto bitmap3 = new BitmapValue(std::vector<uint64_t>({2, 3}));
-    auto bitmap4 = new BitmapValue(std::vector<uint64_t>({1, 2, 6}));
-    auto empty_bitmap = new BitmapValue();
-    DataSet data_set = {{{bitmap1, empty_bitmap}, (int64_t)3},
-                        {{bitmap2, bitmap3}, (int64_t)2},
-                        {{bitmap1, bitmap4}, (int64_t)2}};
+
+    BitmapValue bitmap1({1, 2, 3});
+    BitmapValue bitmap2({1, 2, 3, 4});
+    BitmapValue bitmap3({2, 3});
+    BitmapValue bitmap4({1, 2, 6});
+    BitmapValue empty_bitmap;
+    DataSet data_set = {{{&bitmap1, &empty_bitmap}, (int64_t)3},
+                        {{&bitmap2, &bitmap3}, (int64_t)2},
+                        {{&bitmap1, &bitmap4}, (int64_t)2}};
 
     check_function<DataTypeInt64, true>(func_name, input_types, data_set);
-    delete bitmap1;
-    delete bitmap2;
-    delete bitmap3;
-    delete bitmap4;
-    delete empty_bitmap;
 
     {
         InputTypeSet input_types = {TypeIndex::BitMap, TypeIndex::BitMap, TypeIndex::BitMap};
         BitmapValue bitmap1({1024, 1, 2019});
         BitmapValue bitmap2({0, 33, std::numeric_limits<uint64_t>::min()});
         BitmapValue bitmap3({33, 5, std::numeric_limits<uint64_t>::max()});
-        auto empty_bitmap = new BitmapValue(); //test empty
+        BitmapValue empty_bitmap; //test empty
 
         DataSet data_set = {
-                {{&bitmap1, &bitmap2, empty_bitmap}, (int64_t)5}, //0,1,33,1024,2019
+                {{&bitmap1, &bitmap2, &empty_bitmap}, (int64_t)5}, //0,1,33,1024,2019
                 {{&bitmap1, &bitmap2, &bitmap3}, (int64_t)6}, //0,1,5,1024,2019,18446744073709551615
-                {{&bitmap1, empty_bitmap, Null()}, Null()},
+                {{&bitmap1, &empty_bitmap, Null()}, Null()},
                 {{&bitmap1, &bitmap3, &bitmap3}, (int64_t)3}}; //1,1024,2019
 
         check_function<DataTypeInt64, true>(func_name, input_types, data_set);
-        delete empty_bitmap;
     }
 }
 
@@ -186,15 +164,14 @@ TEST(function_bitmap_test, function_bitmap_and_not_count) {
     BitmapValue bitmap1({1, 2, 3});
     BitmapValue bitmap2({3, 4, std::numeric_limits<uint64_t>::min()});
     BitmapValue bitmap3({33, 5, std::numeric_limits<uint64_t>::max()});
-    auto empty_bitmap = new BitmapValue();
+    BitmapValue empty_bitmap;
 
-    DataSet data_set = {{{&bitmap1, empty_bitmap}, (int64_t)3}, //1,2,3
+    DataSet data_set = {{{&bitmap1, &empty_bitmap}, (int64_t)3}, //1,2,3
                         {{&bitmap2, Null()}, Null()},
                         {{&bitmap2, &bitmap3}, (int64_t)3},  //0,3,4
                         {{&bitmap1, &bitmap2}, (int64_t)2}}; //1,2
 
     check_function<DataTypeInt64, true>(func_name, input_types, data_set);
-    delete empty_bitmap;
 }
 TEST(function_bitmap_test, function_bitmap_has_all) {
     std::string func_name = "bitmap_has_all";
@@ -207,18 +184,16 @@ TEST(function_bitmap_test, function_bitmap_has_all) {
     BitmapValue bitmap3 = BitmapValue({0, 1, 2});
     BitmapValue bitmap4 = BitmapValue({0, 1, 2, std::numeric_limits<uint64_t>::max()});
     BitmapValue bitmap5 = BitmapValue({0, 1, 2});
-    auto empty_bitmap1 = new BitmapValue();
-    auto empty_bitmap2 = new BitmapValue();
+    BitmapValue empty_bitmap1;
+    BitmapValue empty_bitmap2;
 
     DataSet data_set = {{{&bitmap1, &bitmap2}, uint8(true)},
-                        {{empty_bitmap1, empty_bitmap2}, uint8(true)},
+                        {{&empty_bitmap1, &empty_bitmap2}, uint8(true)},
                         {{&bitmap3, &bitmap4}, uint8(false)},
                         {{&bitmap4, &bitmap5}, uint8(true)},
-                        {{Null(), empty_bitmap1}, Null()}};
+                        {{Null(), &empty_bitmap1}, Null()}};
 
     check_function<DataTypeUInt8, true>(func_name, input_types, data_set);
-    delete empty_bitmap1;
-    delete empty_bitmap2;
 }
 
 } // namespace doris::vectorized
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AnalyticExpr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AnalyticExpr.java
index 7eae49924f..6c6d16807b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AnalyticExpr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AnalyticExpr.java
@@ -707,9 +707,8 @@ public class AnalyticExpr extends Expr {
             resetWindow = true;
         }
 
-       // Change first_value/last_value RANGE windows to ROWS 
-       if ((analyticFnName.getFunction().equalsIgnoreCase(FIRSTVALUE)
-                || analyticFnName.getFunction().equalsIgnoreCase(LASTVALUE))
+        // Change first_value RANGE windows to ROWS
+        if ((analyticFnName.getFunction().equalsIgnoreCase(FIRSTVALUE))
                 && window != null
                 && window.getType() == AnalyticWindow.Type.RANGE) {
             window = new AnalyticWindow(AnalyticWindow.Type.ROWS, window.getLeftBoundary(),
diff --git a/regression-test/data/correctness/test_last_value_window.out b/regression-test/data/correctness/test_last_value_window.out
new file mode 100644
index 0000000000..2b56500191
--- /dev/null
+++ b/regression-test/data/correctness/test_last_value_window.out
@@ -0,0 +1,8 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !select_default --
+21	04-21-11	1	1
+22	04-22-10-21	0	1
+22	04-22-10-21	1	1
+23	04-23-10	1	1
+24	02-24-10-21	1	1
+
diff --git a/regression-test/suites/correctness/test_last_value_window.groovy b/regression-test/suites/correctness/test_last_value_window.groovy
new file mode 100644
index 0000000000..274cdec46d
--- /dev/null
+++ b/regression-test/suites/correctness/test_last_value_window.groovy
@@ -0,0 +1,51 @@
+// 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_last_value_window") {
+    def tableName = "state"
+
+
+    sql """ DROP TABLE IF EXISTS ${tableName} """
+    sql """
+            CREATE TABLE ${tableName} (
+            `myday` INT,
+            `time` VARCHAR(40) NOT NULL,
+            `state` INT
+            ) ENGINE=OLAP
+            DUPLICATE KEY(`myday`,time,state)
+            COMMENT "OLAP"
+            DISTRIBUTED BY HASH(`myday`) BUCKETS 2
+            PROPERTIES (
+            "replication_num" = "1",
+            "in_memory" = "false",
+            "storage_format" = "V2"
+            );
+    """
+
+    sql """ INSERT INTO ${tableName} VALUES 
+            (21,"04-21-11",1),
+            (22,"04-22-10-21",0),
+            (22,"04-22-10-21",1),
+            (23,"04-23-10",1),
+            (24,"02-24-10-21",1); """
+
+    // not_vectorized
+    sql """ set enable_vectorized_engine = false; """
+
+    qt_select_default """ select *,last_value(state) over(partition by myday order by time) from ${tableName}; """
+
+}
\ No newline at end of file


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