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/30 15:44:47 UTC

[doris] branch branch-2.0-alpha updated (e341114166 -> e6fecabebb)

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

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


    from e341114166 [enhancement](test) split large data of p2 cases (#19186)
     new 0a06f7372f [bugfix](low cardinality) cached code is wrong will result wrong query result when many null pages (#19221)
     new 8cfb970f67 [fix](schema_change) remove shadow prefix of schema for tablesink (#18822)
     new e6fecabebb [vectorized](function) add some check about result type in array map #19228

The 3 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/common/config.h                             |   1 +
 be/src/olap/comparison_predicate.h                 |  20 +++-
 be/src/olap/schema_change.cpp                      |  12 ++-
 be/src/olap/tablet.h                               |   2 +
 be/src/vec/columns/column_dictionary.h             |   4 +-
 be/src/vec/exprs/lambda_function/lambda_function.h |   2 +-
 .../lambda_function/varray_filter_function.cpp     |   2 +-
 .../exprs/lambda_function/varray_map_function.cpp  |  24 ++++-
 .../apache/doris/alter/SchemaChangeHandler.java    |   1 +
 .../main/java/org/apache/doris/catalog/Column.java |   4 +
 .../org/apache/doris/planner/OlapTableSink.java    |   2 +-
 gensrc/script/doris_builtins_functions.py          |   2 +-
 .../data/nereids_p0/test_dict_with_null.out        | 112 +++++++++++++++++++++
 .../data/query_p0/test_dict_with_null.out          | 112 +++++++++++++++++++++
 .../suites/nereids_p0/test_dict_with_null.groovy   |   5 +-
 .../suites/query_p0/test_dict_with_null.groovy     |   5 +-
 .../test_agg_keys_schema_change_decimalv3.groovy   |   1 +
 17 files changed, 299 insertions(+), 12 deletions(-)
 create mode 100644 regression-test/data/nereids_p0/test_dict_with_null.out
 create mode 100644 regression-test/data/query_p0/test_dict_with_null.out


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


[doris] 01/03: [bugfix](low cardinality) cached code is wrong will result wrong query result when many null pages (#19221)

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

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

commit 0a06f7372fe0e1ea9281b6a1e95570f7f6c74338
Author: yiguolei <67...@qq.com>
AuthorDate: Sat Apr 29 21:28:41 2023 +0800

    [bugfix](low cardinality) cached code is wrong will result wrong query result when many null pages (#19221)
    
    Sometimes the dict is not initialized when run comparison predicate here, for example, the full page is null, then the reader will skip read, so that the dictionary is not inited. The cached code is wrong during this case, because the following page maybe not null, and the dict should have items in the future.
    This will result the dict string column query return wrong result, if there are many null values in the column.
    I also add some regression test for dict column's equal query, larger than query, less than query.
    
    ---------
    
    Co-authored-by: yiguolei <yi...@gmail.com>
---
 be/src/common/config.h                             |   1 +
 be/src/olap/comparison_predicate.h                 |  20 +++-
 be/src/vec/columns/column_dictionary.h             |   4 +-
 .../data/nereids_p0/test_dict_with_null.out        | 112 +++++++++++++++++++++
 .../data/query_p0/test_dict_with_null.out          | 112 +++++++++++++++++++++
 .../suites/nereids_p0/test_dict_with_null.groovy   |   5 +-
 .../suites/query_p0/test_dict_with_null.groovy     |   5 +-
 7 files changed, 255 insertions(+), 4 deletions(-)

diff --git a/be/src/common/config.h b/be/src/common/config.h
index 65ce764be1..62651e9e51 100644
--- a/be/src/common/config.h
+++ b/be/src/common/config.h
@@ -284,6 +284,7 @@ CONF_Bool(disable_storage_page_cache, "false");
 CONF_Bool(disable_storage_row_cache, "true");
 
 CONF_Bool(enable_low_cardinality_optimize, "true");
+CONF_Bool(enable_low_cardinality_cache_code, "true");
 
 // be policy
 // whether check compaction checksum
diff --git a/be/src/olap/comparison_predicate.h b/be/src/olap/comparison_predicate.h
index 0371b27282..1a7fa8baa7 100644
--- a/be/src/olap/comparison_predicate.h
+++ b/be/src/olap/comparison_predicate.h
@@ -564,8 +564,26 @@ private:
             const vectorized::ColumnDictI32& column) const {
         /// if _cache_code_enabled is false, always find the code from dict.
         if (UNLIKELY(!_cache_code_enabled || _cached_code == _InvalidateCodeValue)) {
-            _cached_code = _is_range() ? column.find_code_by_bound(_value, _is_greater(), _is_eq())
+            int32_t code = _is_range() ? column.find_code_by_bound(_value, _is_greater(), _is_eq())
                                        : column.find_code(_value);
+
+            // Protect the invalid code logic, to avoid data error.
+            if (code == _InvalidateCodeValue) {
+                LOG(FATAL) << "column dictionary should not return the code " << code
+                           << ", because it is assumed as an invalid code in comparison predicate";
+            }
+            // Sometimes the dict is not initialized when run comparison predicate here, for example,
+            // the full page is null, then the reader will skip read, so that the dictionary is not
+            // inited. The cached code is wrong during this case, because the following page maybe not
+            // null, and the dict should have items in the future.
+            //
+            // Cached code may have problems, so that add a config here, if not opened, then
+            // we will return the code and not cache it.
+            if (column.is_dict_empty() || !config::enable_low_cardinality_cache_code) {
+                return code;
+            }
+            // If the dict is not empty, then the dict is inited and we could cache the value.
+            _cached_code = code;
         }
         return _cached_code;
     }
diff --git a/be/src/vec/columns/column_dictionary.h b/be/src/vec/columns/column_dictionary.h
index 8a2c055d13..0d3b4ee8de 100644
--- a/be/src/vec/columns/column_dictionary.h
+++ b/be/src/vec/columns/column_dictionary.h
@@ -306,6 +306,8 @@ public:
 
     bool is_dict_sorted() const { return _dict_sorted; }
 
+    bool is_dict_empty() const { return _dict.empty(); }
+
     bool is_dict_code_converted() const { return _dict_code_converted; }
 
     MutableColumnPtr convert_to_predicate_column_if_dictionary() override {
@@ -507,7 +509,7 @@ public:
 
         size_t byte_size() { return _dict_data->size() * sizeof((*_dict_data)[0]); }
 
-        bool empty() { return _dict_data->empty(); }
+        bool empty() const { return _dict_data->empty(); }
 
         size_t avg_str_len() { return empty() ? 0 : _total_str_len / _dict_data->size(); }
 
diff --git a/regression-test/data/nereids_p0/test_dict_with_null.out b/regression-test/data/nereids_p0/test_dict_with_null.out
new file mode 100644
index 0000000000..7ce45abf43
--- /dev/null
+++ b/regression-test/data/nereids_p0/test_dict_with_null.out
@@ -0,0 +1,112 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !sql1 --
+101	abc
+
+-- !sql2 --
+101	abc
+
+-- !sql3 --
+101	abc
+
+-- !sql4 --
+100	\N
+99	\N
+98	\N
+97	\N
+96	\N
+95	\N
+94	\N
+93	\N
+92	\N
+91	\N
+90	\N
+89	\N
+88	\N
+87	\N
+86	\N
+85	\N
+84	\N
+83	\N
+82	\N
+81	\N
+80	\N
+79	\N
+78	\N
+77	\N
+76	\N
+75	\N
+74	\N
+73	\N
+72	\N
+71	\N
+70	\N
+69	\N
+68	\N
+67	\N
+66	\N
+65	\N
+64	\N
+63	\N
+62	\N
+61	\N
+60	\N
+59	\N
+58	\N
+57	\N
+56	\N
+55	\N
+54	\N
+53	\N
+52	\N
+51	\N
+50	\N
+49	\N
+48	\N
+47	\N
+46	\N
+45	\N
+44	\N
+43	\N
+42	\N
+41	\N
+40	\N
+39	\N
+38	\N
+37	\N
+36	\N
+35	\N
+34	\N
+33	\N
+32	\N
+31	\N
+30	\N
+29	\N
+28	\N
+27	\N
+26	\N
+25	\N
+24	\N
+23	\N
+22	\N
+21	\N
+20	\N
+19	\N
+18	\N
+17	\N
+16	\N
+15	\N
+14	\N
+13	\N
+12	\N
+11	\N
+10	\N
+9	\N
+8	\N
+7	\N
+6	\N
+5	\N
+4	\N
+3	\N
+2	\N
+1	\N
+
diff --git a/regression-test/data/query_p0/test_dict_with_null.out b/regression-test/data/query_p0/test_dict_with_null.out
new file mode 100644
index 0000000000..7ce45abf43
--- /dev/null
+++ b/regression-test/data/query_p0/test_dict_with_null.out
@@ -0,0 +1,112 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !sql1 --
+101	abc
+
+-- !sql2 --
+101	abc
+
+-- !sql3 --
+101	abc
+
+-- !sql4 --
+100	\N
+99	\N
+98	\N
+97	\N
+96	\N
+95	\N
+94	\N
+93	\N
+92	\N
+91	\N
+90	\N
+89	\N
+88	\N
+87	\N
+86	\N
+85	\N
+84	\N
+83	\N
+82	\N
+81	\N
+80	\N
+79	\N
+78	\N
+77	\N
+76	\N
+75	\N
+74	\N
+73	\N
+72	\N
+71	\N
+70	\N
+69	\N
+68	\N
+67	\N
+66	\N
+65	\N
+64	\N
+63	\N
+62	\N
+61	\N
+60	\N
+59	\N
+58	\N
+57	\N
+56	\N
+55	\N
+54	\N
+53	\N
+52	\N
+51	\N
+50	\N
+49	\N
+48	\N
+47	\N
+46	\N
+45	\N
+44	\N
+43	\N
+42	\N
+41	\N
+40	\N
+39	\N
+38	\N
+37	\N
+36	\N
+35	\N
+34	\N
+33	\N
+32	\N
+31	\N
+30	\N
+29	\N
+28	\N
+27	\N
+26	\N
+25	\N
+24	\N
+23	\N
+22	\N
+21	\N
+20	\N
+19	\N
+18	\N
+17	\N
+16	\N
+15	\N
+14	\N
+13	\N
+12	\N
+11	\N
+10	\N
+9	\N
+8	\N
+7	\N
+6	\N
+5	\N
+4	\N
+3	\N
+2	\N
+1	\N
+
diff --git a/regression-test/suites/nereids_p0/test_dict_with_null.groovy b/regression-test/suites/nereids_p0/test_dict_with_null.groovy
index becceafdde..a0858c2122 100644
--- a/regression-test/suites/nereids_p0/test_dict_with_null.groovy
+++ b/regression-test/suites/nereids_p0/test_dict_with_null.groovy
@@ -44,5 +44,8 @@ suite("dict_with_null", "query") {
     insert_sql += ", (101, 'abc')"
 
     sql insert_sql
-    sql "select * from test_dict_with_null where c_string > '0'"
+    qt_sql1 "select * from test_dict_with_null where c_string > '0'"
+    qt_sql2 "select * from test_dict_with_null where c_string < 'dfg'"
+    qt_sql3 "select * from test_dict_with_null where c_string = 'abc'"
+    qt_sql4 "select * from test_dict_with_null where c_string is null order by c_int desc"
 }
\ No newline at end of file
diff --git a/regression-test/suites/query_p0/test_dict_with_null.groovy b/regression-test/suites/query_p0/test_dict_with_null.groovy
index a5c84444ae..b3738bb68a 100644
--- a/regression-test/suites/query_p0/test_dict_with_null.groovy
+++ b/regression-test/suites/query_p0/test_dict_with_null.groovy
@@ -42,5 +42,8 @@ suite("dict_with_null", "query") {
     insert_sql += ", (101, 'abc')"
 
     sql insert_sql
-    sql "select * from test_dict_with_null where c_string > '0'"
+    qt_sql1 "select * from test_dict_with_null where c_string > '0'"
+    qt_sql2 "select * from test_dict_with_null where c_string < 'dfg'"
+    qt_sql3 "select * from test_dict_with_null where c_string = 'abc'"
+    qt_sql4 "select * from test_dict_with_null where c_string is null order by c_int desc"
 }
\ No newline at end of file


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


[doris] 03/03: [vectorized](function) add some check about result type in array map #19228

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

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

commit e6fecabebbaceb1276dfcfddedb57b2140f9d769
Author: Kang <kx...@gmail.com>
AuthorDate: Sun Apr 30 23:32:21 2023 +0800

    [vectorized](function) add some check about result type in array map #19228
---
 be/src/vec/exprs/lambda_function/lambda_function.h |  2 +-
 .../lambda_function/varray_filter_function.cpp     |  2 +-
 .../exprs/lambda_function/varray_map_function.cpp  | 24 +++++++++++++++++++++-
 gensrc/script/doris_builtins_functions.py          |  2 +-
 4 files changed, 26 insertions(+), 4 deletions(-)

diff --git a/be/src/vec/exprs/lambda_function/lambda_function.h b/be/src/vec/exprs/lambda_function/lambda_function.h
index a1eb173725..a7f2fb8f69 100644
--- a/be/src/vec/exprs/lambda_function/lambda_function.h
+++ b/be/src/vec/exprs/lambda_function/lambda_function.h
@@ -32,7 +32,7 @@ public:
     virtual std::string get_name() const = 0;
 
     virtual doris::Status execute(VExprContext* context, doris::vectorized::Block* block,
-                                  int* result_column_id, DataTypePtr result_type,
+                                  int* result_column_id, const DataTypePtr& result_type,
                                   const std::vector<VExpr*>& children) = 0;
 };
 
diff --git a/be/src/vec/exprs/lambda_function/varray_filter_function.cpp b/be/src/vec/exprs/lambda_function/varray_filter_function.cpp
index 8d4acdf8f8..636ee709c2 100644
--- a/be/src/vec/exprs/lambda_function/varray_filter_function.cpp
+++ b/be/src/vec/exprs/lambda_function/varray_filter_function.cpp
@@ -42,7 +42,7 @@ public:
     std::string get_name() const override { return name; }
 
     doris::Status execute(VExprContext* context, doris::vectorized::Block* block,
-                          int* result_column_id, DataTypePtr result_type,
+                          int* result_column_id, const DataTypePtr& result_type,
                           const std::vector<VExpr*>& children) override {
         ///* array_filter(array, array<boolean>) *///
 
diff --git a/be/src/vec/exprs/lambda_function/varray_map_function.cpp b/be/src/vec/exprs/lambda_function/varray_map_function.cpp
index 5fac25ca3f..e2fedd0cb6 100644
--- a/be/src/vec/exprs/lambda_function/varray_map_function.cpp
+++ b/be/src/vec/exprs/lambda_function/varray_map_function.cpp
@@ -40,7 +40,7 @@ public:
     std::string get_name() const override { return name; }
 
     doris::Status execute(VExprContext* context, doris::vectorized::Block* block,
-                          int* result_column_id, DataTypePtr result_type,
+                          int* result_column_id, const DataTypePtr& result_type,
                           const std::vector<VExpr*>& children) override {
         ///* array_map(lambda,arg1,arg2,.....) *///
 
@@ -118,6 +118,12 @@ public:
                                                "R" + array_column_type_name.name};
             lambda_block.insert(std::move(data_column));
         }
+        //check nullable(array(nullable(nested)))
+        DCHECK(result_type->is_nullable() &&
+               is_array(((DataTypeNullable*)result_type.get())->get_nested_type()))
+                << "array_map result type is error, now must be nullable(array): "
+                << result_type->get_name()
+                << " ,and block structure is: " << block->dump_structure();
 
         //3. child[0]->execute(new_block)
         RETURN_IF_ERROR(children[0]->execute(context, &lambda_block, result_column_id));
@@ -136,6 +142,7 @@ public:
                           result_type, res_name};
 
         } else {
+            // deal with eg: select array_map(x -> x is null, [null, 1, 2]);
             // need to create the nested column null map for column array
             auto nested_null_map = ColumnUInt8::create(res_col->size(), 0);
             result_arr = {ColumnNullable::create(
@@ -147,6 +154,21 @@ public:
         }
         block->insert(std::move(result_arr));
         *result_column_id = block->columns() - 1;
+        //check nullable(nested)
+        DCHECK((assert_cast<const DataTypeArray*>(
+                        (((DataTypeNullable*)result_type.get())->get_nested_type().get())))
+                       ->get_nested_type()
+                       ->equals(*make_nullable(res_type)))
+                << " array_map function FE given result type is: " << result_type->get_name()
+                << " get nested is: "
+                << (assert_cast<const DataTypeArray*>(
+                            (((DataTypeNullable*)result_type.get())->get_nested_type().get())))
+                           ->get_nested_type()
+                           ->get_name()
+                << " and now actual nested type after calculate " << res_type->get_name()
+                << " ,and block structure is: " << block->dump_structure()
+                << " ,and lambda_block structure is: " << lambda_block.dump_structure();
+
         return Status::OK();
     }
 };
diff --git a/gensrc/script/doris_builtins_functions.py b/gensrc/script/doris_builtins_functions.py
index 90e8a87397..409326f1f6 100644
--- a/gensrc/script/doris_builtins_functions.py
+++ b/gensrc/script/doris_builtins_functions.py
@@ -619,7 +619,7 @@ visible_functions = [
     [['array_popfront'], 'ARRAY_DECIMAL128', ['ARRAY_DECIMAL128'], ''],
     [['array_popfront'], 'ARRAY_VARCHAR', ['ARRAY_VARCHAR'], ''],
     [['array_popfront'], 'ARRAY_STRING', ['ARRAY_STRING'], ''],
-    [['array_map'], 'ARRAY',   ['LAMBDA_FUNCTION', 'ARRAY', '...'], ''],
+    [['array_map'], 'ARRAY', ['LAMBDA_FUNCTION', 'ARRAY<K>', '...'], '', ['K']],
     [['array_filter'], 'ARRAY_BOOLEAN',['ARRAY_BOOLEAN', 'ARRAY_BOOLEAN'], ''],
     [['array_filter'], 'ARRAY_TINYINT',['ARRAY_TINYINT', 'ARRAY_BOOLEAN'], ''],
     [['array_filter'], 'ARRAY_SMALLINT',['ARRAY_SMALLINT', 'ARRAY_BOOLEAN'], ''],


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


[doris] 02/03: [fix](schema_change) remove shadow prefix of schema for tablesink (#18822)

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

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

commit 8cfb970f67475ad92740eb7f90f6254274148085
Author: Yongqiang YANG <98...@users.noreply.github.com>
AuthorDate: Sun Apr 30 22:46:36 2023 +0800

    [fix](schema_change) remove shadow prefix of schema for tablesink (#18822)
    
    LSC updates tablet's schema in writing. Be optimized adding columns via linked schema change and
    it distinguishes adding by comparing column name. e.g. if new column's name is not found in old schema,
    then it is a newly-add column.
    
    When a table is under schema-changing, it adds __doris_shadow_ prefix in name of columns in shadow index.
    Then  writes during schema-changing would bring schema with __doris_shadow_ to be.
    If schema change request arrives at be after writes, then be do it as a add-column schema change due to
    __doris_shadow_ is not in base tablet.
---
 be/src/olap/schema_change.cpp                                | 12 +++++++++---
 be/src/olap/tablet.h                                         |  2 ++
 .../java/org/apache/doris/alter/SchemaChangeHandler.java     |  1 +
 .../src/main/java/org/apache/doris/catalog/Column.java       |  4 ++++
 .../main/java/org/apache/doris/planner/OlapTableSink.java    |  2 +-
 .../decimalv3/test_agg_keys_schema_change_decimalv3.groovy   |  1 +
 6 files changed, 18 insertions(+), 4 deletions(-)

diff --git a/be/src/olap/schema_change.cpp b/be/src/olap/schema_change.cpp
index 1fee5f744a..066dd72ab9 100644
--- a/be/src/olap/schema_change.cpp
+++ b/be/src/olap/schema_change.cpp
@@ -1673,7 +1673,7 @@ Status SchemaChangeHandler::_parse_request(const SchemaChangeParams& sc_params,
     for (int i = 0, new_schema_size = new_tablet->tablet_schema()->num_columns();
          i < new_schema_size; ++i) {
         const TabletColumn& new_column = new_tablet->tablet_schema()->column(i);
-        const string& column_name = new_column.name();
+        const std::string& column_name = new_column.name();
         ColumnMapping* column_mapping = changer->get_mutable_column_mapping(i);
         column_mapping->new_column = &new_column;
 
@@ -1698,6 +1698,11 @@ Status SchemaChangeHandler::_parse_request(const SchemaChangeParams& sc_params,
             continue;
         }
 
+        if (column_name.find("__doris_shadow_") == 0) {
+            // Should delete in the future, just a protection for bug.
+            LOG(INFO) << "a shadow column is encountered " << column_name;
+            return Status::InternalError("failed due to operate on shadow column");
+        }
         // Newly added column go here
         column_mapping->ref_column = -1;
 
@@ -1707,8 +1712,9 @@ Status SchemaChangeHandler::_parse_request(const SchemaChangeParams& sc_params,
         RETURN_IF_ERROR(
                 _init_column_mapping(column_mapping, new_column, new_column.default_value()));
 
-        VLOG_TRACE << "A column with default value will be added after schema changing. "
-                   << "column=" << column_name << ", default_value=" << new_column.default_value();
+        LOG(INFO) << "A column with default value will be added after schema changing. "
+                  << "column=" << column_name << ", default_value=" << new_column.default_value()
+                  << " to table " << new_tablet->get_table_id();
     }
 
     if (materialized_function_map.count(WHERE_SIGN)) {
diff --git a/be/src/olap/tablet.h b/be/src/olap/tablet.h
index faa5a5c8e0..db653e6cc2 100644
--- a/be/src/olap/tablet.h
+++ b/be/src/olap/tablet.h
@@ -438,6 +438,8 @@ public:
         return config::max_tablet_io_errors > 0 && _io_error_times >= config::max_tablet_io_errors;
     }
 
+    int64_t get_table_id() { return _tablet_meta->table_id(); }
+
 private:
     Status _init_once_action();
     void _print_missed_versions(const std::vector<Version>& missed_versions) const;
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 c9ce361e3c..c594a1981b 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
@@ -698,6 +698,7 @@ public class SchemaChangeHandler extends AlterHandler {
              */
             modColumn.setName(SHADOW_NAME_PREFIX + modColumn.getName());
         }
+        LOG.info("modify column {} ", modColumn);
         return lightSchemaChange;
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java
index d6a5b1b576..83427f1579 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java
@@ -270,6 +270,10 @@ public class Column implements Writable, GsonPostProcessable {
         return this.name;
     }
 
+    public String getNonShadowName() {
+        return removeNamePrefix(name);
+    }
+
     public String getNameWithoutMvPrefix() {
         return CreateMaterializedViewStmt.mvColumnBreaker(name);
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/OlapTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/planner/OlapTableSink.java
index 0d0df38d4a..bad033fdb9 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/OlapTableSink.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/OlapTableSink.java
@@ -212,7 +212,7 @@ public class OlapTableSink extends DataSink {
             List<String> columns = Lists.newArrayList();
             List<TColumn> columnsDesc = Lists.newArrayList();
             List<TOlapTableIndex> indexDesc = Lists.newArrayList();
-            columns.addAll(indexMeta.getSchema().stream().map(Column::getName).collect(Collectors.toList()));
+            columns.addAll(indexMeta.getSchema().stream().map(Column::getNonShadowName).collect(Collectors.toList()));
             for (Column column : indexMeta.getSchema()) {
                 TColumn tColumn = column.toThrift();
                 column.setIndexFlag(tColumn, table);
diff --git a/regression-test/suites/schema_change_p0/decimalv3/test_agg_keys_schema_change_decimalv3.groovy b/regression-test/suites/schema_change_p0/decimalv3/test_agg_keys_schema_change_decimalv3.groovy
index 321b1cd1a2..45f5be6701 100644
--- a/regression-test/suites/schema_change_p0/decimalv3/test_agg_keys_schema_change_decimalv3.groovy
+++ b/regression-test/suites/schema_change_p0/decimalv3/test_agg_keys_schema_change_decimalv3.groovy
@@ -21,6 +21,7 @@ suite("test_agg_keys_schema_change_decimalv3") {
     def tbName = "test_agg_keys_schema_change_decimalv3"
     def getJobState = { tableName ->
          def jobStateResult = sql """  SHOW ALTER TABLE COLUMN WHERE IndexName='${tableName}' ORDER BY createtime DESC LIMIT 1 """
+         logger.info(jobStateResult.toString());
          return jobStateResult[0][9]
     }
 


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