You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by da...@apache.org on 2022/07/12 11:41:12 UTC

[doris] branch master updated: [Feature] Lightweight schema change of add/drop column (#10136)

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

dataroaring 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 486cf0ebd4 [Feature] Lightweight schema change of add/drop column (#10136)
486cf0ebd4 is described below

commit 486cf0ebd4f5c216ecbe7deeaabba978f1dac54c
Author: Lightman <31...@users.noreply.github.com>
AuthorDate: Tue Jul 12 19:41:06 2022 +0800

    [Feature] Lightweight schema change of add/drop column (#10136)
    
    * [Schema Change] support fast add/drop column  (#49)
    
    * [feature](schema-change) support fast schema change. coauthor: yixiutt
    
    * [schema change] Using columns desc from fe to read data. coauthor: Lchangliang
    
    * [feature](schema change) schema change optimize for add/drop columns.
    
    1.add uniqueId field for class column.
    2.schema change for add/drop columns directly update schema meta
    
    Co-authored-by: yixiutt <yi...@selectdb.com>
    Co-authored-by: SWJTU-ZhangLei <10...@qq.com>
    
    [Feature](schema change) fix write and add regression test (#69)
    
    Co-authored-by: yixiutt <yi...@selectdb.com>
    
    [schema change] be ssupport that delete use newest schema
    
    add delete regression test
    
    fix regression case (#107)
    
    tmp
    
    [feature](schema change) light schema change exclude rollup and agg/uniq/dup key type.
    
    [feature](schema change) fe olapTable maxUniqueId write in disk.
    
    [feature](schema change) add rpc iface for sc add column.
    
    [feature](schema change) add columnsDesc to TPushReq for ligtht sc.
    
    resolve the deadlock when schema change (#124)
    
    fix columns from fe don't has bitmap_index flag (#134)
    
    add update/delete case
    
    construct MATERIALIZED schema from origin schema when insert
    
    fix not vectorized compaction coredump
    
    use segment cache
    
    choose newest schema by schema version when compaction (#182)
    
    [bugfix](schema change) fix ligth schema change problem.
    
    [feature](schema change) light schema change add alter job. (#1)
    
    fix be ut
    
    [bug] (schema change) unique drop key column should not light schema
    change
    
    [feature](schema change) add schema change regression-test.
    
    fix regression test
    
    [bugfix](schema change) fix multi alter clauses for light schema change. (#2)
    
    [bugfix](schema change) fix multi clauses calculate column unique id (#3)
    
    modify PushTask process (#217)
    
    [Bugfix](schema change) fix jobId replay cause bdbje exception.
    
    [bug](schema change) fix max col unique id repeatitive. (#232)
    
    [optimize](schema change) modify pendingMaxColUniqueId generate rule.
    
    fix compaction error
    * fix be ut
    
    * fix snapshot load core
    
    fix unique_id error (#278)
    
    [refact](fe) remove redundant code for light schema change. (#4)
    
    [refact](fe) remove redundant code for light schema change. (#4)
    
    format fe core
    
    format be core
    
    fix be ut
    
    modify fe meta version
    
    fix rebase error
    
    flush schema into rowset_meta in old table
    
    [refactor](schema change) refact fe light schema change. (#5)
    
    delete the change of schemahash and support get max version schema
    
    * modify for review
    
    * fix be ut
    
    * fix schema change test
---
 be/src/exec/olap_scanner.cpp                       |  28 +-
 be/src/exec/olap_scanner.h                         |   2 +
 be/src/exec/tablet_info.cpp                        |  13 +
 be/src/exec/tablet_info.h                          |   2 +
 be/src/olap/base_tablet.cpp                        |  11 +
 be/src/olap/base_tablet.h                          |   4 +-
 be/src/olap/collect_iterator.cpp                   |   8 +-
 be/src/olap/compaction.cpp                         |  19 +-
 be/src/olap/compaction.h                           |   2 +-
 be/src/olap/data_dir.cpp                           |  14 +
 be/src/olap/delta_writer.cpp                       |  27 +-
 be/src/olap/delta_writer.h                         |  12 +-
 be/src/olap/merger.cpp                             |  11 +-
 be/src/olap/merger.h                               |   2 +
 be/src/olap/push_handler.cpp                       |  76 +--
 be/src/olap/push_handler.h                         |  14 +-
 be/src/olap/reader.cpp                             |  56 +-
 be/src/olap/reader.h                               |   4 +
 be/src/olap/rowset/beta_rowset.cpp                 |   5 +-
 be/src/olap/rowset/beta_rowset.h                   |   3 +-
 be/src/olap/rowset/beta_rowset_reader.cpp          |   2 +-
 be/src/olap/rowset/beta_rowset_writer.cpp          |   2 +
 be/src/olap/rowset/rowset.cpp                      |   7 +-
 be/src/olap/rowset/rowset.h                        |   1 +
 be/src/olap/rowset/rowset_meta.h                   |  14 +
 be/src/olap/rowset/segment_v2/segment.cpp          |  12 +-
 be/src/olap/rowset/segment_v2/segment.h            |   4 +-
 be/src/olap/rowset/segment_v2/segment_iterator.cpp |   2 +-
 be/src/olap/schema_change.cpp                      |  86 +--
 be/src/olap/schema_change.h                        |   6 +-
 be/src/olap/segment_loader.cpp                     |  10 +-
 be/src/olap/segment_loader.h                       |  13 +-
 be/src/olap/snapshot_manager.cpp                   |   3 +-
 be/src/olap/tablet.cpp                             |  29 +-
 be/src/olap/tablet.h                               |  10 +-
 be/src/olap/tablet_meta.cpp                        |  18 +-
 be/src/olap/tablet_meta.h                          |   8 +-
 be/src/olap/tablet_schema.cpp                      | 140 ++++-
 be/src/olap/tablet_schema.h                        |  22 +-
 be/src/runtime/descriptors.cpp                     |   2 +
 be/src/runtime/descriptors.h                       |   4 +
 be/src/runtime/tablets_channel.cpp                 |   2 +
 be/src/vec/exec/volap_scanner.cpp                  |  29 +-
 be/src/vec/exec/volap_scanner.h                    |   2 +
 be/src/vec/olap/block_reader.cpp                   |   2 +-
 be/src/vec/olap/vcollect_iterator.cpp              |   2 +-
 be/src/vec/olap/vcollect_iterator.h                |   2 +-
 be/test/olap/rowset/beta_rowset_test.cpp           |   7 +-
 be/test/olap/test_data/header_without_inc_rs.txt   |  12 +-
 .../org/apache/doris/common/FeMetaVersion.java     |   4 +-
 .../java/org/apache/doris/alter/AlterJobV2.java    |   8 +
 .../java/org/apache/doris/alter/RollupJobV2.java   |   2 +-
 .../apache/doris/alter/SchemaChangeHandler.java    | 576 ++++++++++++++++-----
 .../org/apache/doris/alter/SchemaChangeJobV2.java  |  14 +-
 .../java/org/apache/doris/analysis/ColumnDef.java  |   2 +-
 .../org/apache/doris/analysis/SlotDescriptor.java  |  36 +-
 .../java/org/apache/doris/analysis/SlotRef.java    |   2 +-
 .../java/org/apache/doris/catalog/Catalog.java     | 236 ++++-----
 .../main/java/org/apache/doris/catalog/Column.java |  34 +-
 .../doris/catalog/MaterializedIndexMeta.java       |  12 +
 .../java/org/apache/doris/catalog/OlapTable.java   |  22 +
 .../main/java/org/apache/doris/catalog/Table.java  |   2 +-
 .../doris/datasource/InternalDataSource.java       | 121 ++---
 .../org/apache/doris/journal/JournalEntity.java    |   6 +
 .../java/org/apache/doris/load/DeleteHandler.java  |   9 +-
 .../java/org/apache/doris/load/LoadChecker.java    |   8 +-
 .../org/apache/doris/load/loadv2/SparkLoadJob.java | 120 +++--
 .../java/org/apache/doris/persist/EditLog.java     |  52 +-
 .../org/apache/doris/persist/OperationType.java    |   3 +
 .../doris/persist/TableAddOrDropColumnsInfo.java   | 115 ++++
 .../org/apache/doris/planner/OlapScanNode.java     | 138 +++--
 .../org/apache/doris/planner/OlapTableSink.java    |  33 +-
 .../apache/doris/service/FrontendServiceImpl.java  |  59 +--
 .../org/apache/doris/task/AlterReplicaTask.java    |  15 +-
 .../main/java/org/apache/doris/task/PushTask.java  |  32 +-
 .../doris/alter/SchemaChangeHandlerTest.java       | 326 +++++++++++-
 .../persist/TableAddOrDropColumnsInfoTest.java     |  97 ++++
 gensrc/proto/descriptors.proto                     |   2 +
 gensrc/proto/olap_file.proto                       |   4 +
 gensrc/thrift/AgentService.thrift                  |  17 +-
 gensrc/thrift/Descriptors.thrift                   |  17 +
 gensrc/thrift/PlanNodes.thrift                     |   1 +
 .../test_compaction_schema_change.out              |  11 +
 .../schema_change/test_delete_schema_change.out    |  44 ++
 .../schema_change/test_partition_schema_change.out |  33 ++
 .../schema_change/test_update_schema_change.out    |  64 +++
 .../test_agg_keys_schema_change.groovy             | 275 ++++++++++
 .../schema_change/test_agg_mv_schema_change.groovy | 258 +++++++++
 .../test_agg_rollup_schema_change.groovy           | 256 +++++++++
 .../test_agg_vals_schema_change.groovy             | 272 ++++++++++
 .../schema_change/test_delete_schema_change.sql    |  44 ++
 .../test_dup_keys_schema_change.groovy             | 273 ++++++++++
 .../schema_change/test_dup_mv_schema_change.groovy | 288 +++++++++++
 .../test_dup_rollup_schema_change.groovy           | 287 ++++++++++
 .../test_dup_vals_schema_change.groovy             | 259 +++++++++
 .../schema_change/test_partition_schema_change.sql |  44 ++
 .../test_uniq_keys_schema_change.groovy            | 270 ++++++++++
 .../test_uniq_mv_schema_change.groovy              | 274 ++++++++++
 .../test_uniq_rollup_schema_change.groovy          | 286 ++++++++++
 .../test_uniq_vals_schema_change.groovy            | 261 ++++++++++
 .../schema_change/test_update_schema_change.sql    |  50 ++
 101 files changed, 5683 insertions(+), 757 deletions(-)

diff --git a/be/src/exec/olap_scanner.cpp b/be/src/exec/olap_scanner.cpp
index 29436ecf05..877b76fb8d 100644
--- a/be/src/exec/olap_scanner.cpp
+++ b/be/src/exec/olap_scanner.cpp
@@ -23,7 +23,9 @@
 #include "exprs/expr_context.h"
 #include "gen_cpp/PaloInternalService_types.h"
 #include "olap/decimal12.h"
+#include "olap/field.h"
 #include "olap/storage_engine.h"
+#include "olap/tablet_schema.h"
 #include "olap/uint24.h"
 #include "olap_scan_node.h"
 #include "olap_utils.h"
@@ -86,6 +88,14 @@ Status OlapScanner::prepare(
             LOG(WARNING) << ss.str();
             return Status::InternalError(ss.str());
         }
+        _tablet_schema = _tablet->tablet_schema();
+        if (!_parent->_olap_scan_node.columns_desc.empty() &&
+            _parent->_olap_scan_node.columns_desc[0].col_unique_id >= 0) {
+            _tablet_schema.clear_columns();
+            for (const auto& column_desc : _parent->_olap_scan_node.columns_desc) {
+                _tablet_schema.append_column(TabletColumn(column_desc));
+            }
+        }
         {
             std::shared_lock rdlock(_tablet->get_header_lock());
             const RowsetSharedPtr rowset = _tablet->rowset_with_max_version();
@@ -170,6 +180,7 @@ Status OlapScanner::_init_tablet_reader_params(
     RETURN_IF_ERROR(_init_return_columns(!_tablet_reader_params.direct_mode));
 
     _tablet_reader_params.tablet = _tablet;
+    _tablet_reader_params.tablet_schema = &_tablet_schema;
     _tablet_reader_params.reader_type = READER_QUERY;
     _tablet_reader_params.aggregation = _aggregation;
     _tablet_reader_params.version = Version(0, _version);
@@ -210,7 +221,7 @@ Status OlapScanner::_init_tablet_reader_params(
             _tablet_reader_params.return_columns.push_back(i);
         }
         for (auto index : _return_columns) {
-            if (_tablet->tablet_schema().column(index).is_key()) {
+            if (_tablet_schema.column(index).is_key()) {
                 continue;
             } else {
                 _tablet_reader_params.return_columns.push_back(index);
@@ -219,13 +230,12 @@ Status OlapScanner::_init_tablet_reader_params(
     }
 
     // use _tablet_reader_params.return_columns, because reader use this to merge sort
-    Status res =
-            _read_row_cursor.init(_tablet->tablet_schema(), _tablet_reader_params.return_columns);
+    Status res = _read_row_cursor.init(_tablet_schema, _tablet_reader_params.return_columns);
     if (!res.ok()) {
         LOG(WARNING) << "fail to init row cursor.res = " << res;
         return Status::InternalError("failed to initialize storage read row cursor");
     }
-    _read_row_cursor.allocate_memory_for_string_type(_tablet->tablet_schema());
+    _read_row_cursor.allocate_memory_for_string_type(_tablet_schema);
 
     // If a agg node is this scan node direct parent
     // we will not call agg object finalize method in scan node,
@@ -244,7 +254,9 @@ Status OlapScanner::_init_return_columns(bool need_seq_col) {
         if (!slot->is_materialized()) {
             continue;
         }
-        int32_t index = _tablet->field_index(slot->col_name());
+        int32_t index = slot->col_unique_id() >= 0
+                                ? _tablet_schema.field_index(slot->col_unique_id())
+                                : _tablet_schema.field_index(slot->col_name());
         if (index < 0) {
             std::stringstream ss;
             ss << "field name is invalid. field=" << slot->col_name();
@@ -252,7 +264,7 @@ Status OlapScanner::_init_return_columns(bool need_seq_col) {
             return Status::InternalError(ss.str());
         }
         _return_columns.push_back(index);
-        if (slot->is_nullable() && !_tablet->tablet_schema().column(index).is_nullable())
+        if (slot->is_nullable() && !_tablet_schema.column(index).is_nullable())
             _tablet_columns_convert_to_null_set.emplace(index);
         _query_slots.push_back(slot);
     }
@@ -261,13 +273,13 @@ Status OlapScanner::_init_return_columns(bool need_seq_col) {
     if (_tablet->tablet_schema().has_sequence_col() && need_seq_col) {
         bool has_replace_col = false;
         for (auto col : _return_columns) {
-            if (_tablet->tablet_schema().column(col).aggregation() ==
+            if (_tablet_schema.column(col).aggregation() ==
                 FieldAggregationMethod::OLAP_FIELD_AGGREGATION_REPLACE) {
                 has_replace_col = true;
                 break;
             }
         }
-        if (auto sequence_col_idx = _tablet->tablet_schema().sequence_col_idx();
+        if (auto sequence_col_idx = _tablet_schema.sequence_col_idx();
             has_replace_col && std::find(_return_columns.begin(), _return_columns.end(),
                                          sequence_col_idx) == _return_columns.end()) {
             _return_columns.push_back(sequence_col_idx);
diff --git a/be/src/exec/olap_scanner.h b/be/src/exec/olap_scanner.h
index 65086eaceb..a128026217 100644
--- a/be/src/exec/olap_scanner.h
+++ b/be/src/exec/olap_scanner.h
@@ -145,6 +145,8 @@ protected:
     MonotonicStopWatch _watcher;
 
     std::shared_ptr<MemTracker> _mem_tracker;
+
+    TabletSchema _tablet_schema;
 };
 
 } // namespace doris
diff --git a/be/src/exec/tablet_info.cpp b/be/src/exec/tablet_info.cpp
index 71999bb3bb..097b667c3e 100644
--- a/be/src/exec/tablet_info.cpp
+++ b/be/src/exec/tablet_info.cpp
@@ -31,6 +31,9 @@ void OlapTableIndexSchema::to_protobuf(POlapTableIndexSchema* pindex) const {
     for (auto slot : slots) {
         pindex->add_columns(slot->col_name());
     }
+    for (auto column : columns) {
+        column->to_schema_pb(pindex->add_columns_desc());
+    }
 }
 
 Status OlapTableSchemaParam::init(const POlapTableSchemaParam& pschema) {
@@ -57,6 +60,11 @@ Status OlapTableSchemaParam::init(const POlapTableSchemaParam& pschema) {
             }
             index->slots.emplace_back(it->second);
         }
+        for (auto& pcolumn_desc : p_index.columns_desc()) {
+            TabletColumn* tc = _obj_pool.add(new TabletColumn());
+            tc->init_from_pb(pcolumn_desc);
+            index->columns.emplace_back(tc);
+        }
         _indexes.emplace_back(index);
     }
 
@@ -90,6 +98,11 @@ Status OlapTableSchemaParam::init(const TOlapTableSchemaParam& tschema) {
             }
             index->slots.emplace_back(it->second);
         }
+        for (auto& tcolumn_desc : t_index.columns_desc) {
+            TabletColumn* tc = _obj_pool.add(new TabletColumn());
+            tc->init_from_thrift(tcolumn_desc);
+            index->columns.emplace_back(tc);
+        }
         _indexes.emplace_back(index);
     }
 
diff --git a/be/src/exec/tablet_info.h b/be/src/exec/tablet_info.h
index 40279a70fe..f287fcfa0d 100644
--- a/be/src/exec/tablet_info.h
+++ b/be/src/exec/tablet_info.h
@@ -27,6 +27,7 @@
 #include "common/status.h"
 #include "gen_cpp/Descriptors_types.h"
 #include "gen_cpp/descriptors.pb.h"
+#include "olap/tablet_schema.h"
 #include "runtime/descriptors.h"
 #include "runtime/raw_value.h"
 #include "runtime/tuple.h"
@@ -41,6 +42,7 @@ struct OlapTableIndexSchema {
     int64_t index_id;
     std::vector<SlotDescriptor*> slots;
     int32_t schema_hash;
+    std::vector<TabletColumn*> columns;
 
     void to_protobuf(POlapTableIndexSchema* pindex) const;
 };
diff --git a/be/src/olap/base_tablet.cpp b/be/src/olap/base_tablet.cpp
index 00b7dc30ae..cb2e7561f2 100644
--- a/be/src/olap/base_tablet.cpp
+++ b/be/src/olap/base_tablet.cpp
@@ -72,4 +72,15 @@ void BaseTablet::_gen_tablet_path() {
     }
 }
 
+bool BaseTablet::set_tablet_schema_into_rowset_meta() {
+    bool flag = false;
+    for (RowsetMetaSharedPtr rowset_meta : _tablet_meta->all_mutable_rs_metas()) {
+        if (!rowset_meta->get_rowset_pb().has_tablet_schema()) {
+            rowset_meta->set_tablet_schema(&_schema);
+            flag = true;
+        }
+    }
+    return flag;
+}
+
 } /* namespace doris */
diff --git a/be/src/olap/base_tablet.h b/be/src/olap/base_tablet.h
index 69a092b84a..99c7d1cac3 100644
--- a/be/src/olap/base_tablet.h
+++ b/be/src/olap/base_tablet.h
@@ -66,7 +66,9 @@ public:
     }
 
     // properties encapsulated in TabletSchema
-    const TabletSchema& tablet_schema() const;
+    virtual const TabletSchema& tablet_schema() const;
+
+    bool set_tablet_schema_into_rowset_meta();
 
 protected:
     void _gen_tablet_path();
diff --git a/be/src/olap/collect_iterator.cpp b/be/src/olap/collect_iterator.cpp
index cf7b079004..de163eb3b0 100644
--- a/be/src/olap/collect_iterator.cpp
+++ b/be/src/olap/collect_iterator.cpp
@@ -55,9 +55,9 @@ Status CollectIterator::add_child(RowsetReaderSharedPtr rs_reader) {
 // then merged with the base rowset.
 void CollectIterator::build_heap(const std::vector<RowsetReaderSharedPtr>& rs_readers) {
     DCHECK(rs_readers.size() == _children.size());
-    _reverse = _reader->_tablet->tablet_schema().keys_type() == KeysType::UNIQUE_KEYS;
-    SortType sort_type = _reader->_tablet->tablet_schema().sort_type();
-    int sort_col_num = _reader->_tablet->tablet_schema().sort_col_num();
+    _reverse = _reader->_tablet_schema->keys_type() == KeysType::UNIQUE_KEYS;
+    SortType sort_type = _reader->_tablet_schema->sort_type();
+    int sort_col_num = _reader->_tablet_schema->sort_col_num();
     if (_children.empty()) {
         _inner_iter.reset(nullptr);
         return;
@@ -200,7 +200,7 @@ CollectIterator::Level0Iterator::Level0Iterator(RowsetReaderSharedPtr rs_reader,
 CollectIterator::Level0Iterator::~Level0Iterator() = default;
 
 Status CollectIterator::Level0Iterator::init() {
-    RETURN_NOT_OK_LOG(_row_cursor.init(_reader->_tablet->tablet_schema(), _reader->_seek_columns),
+    RETURN_NOT_OK_LOG(_row_cursor.init(*_reader->_tablet_schema, _reader->_seek_columns),
                       "failed to init row cursor");
     return (this->*_refresh_current_row)();
 }
diff --git a/be/src/olap/compaction.cpp b/be/src/olap/compaction.cpp
index a54329fb8a..e06e5a9b00 100644
--- a/be/src/olap/compaction.cpp
+++ b/be/src/olap/compaction.cpp
@@ -17,7 +17,10 @@
 
 #include "olap/compaction.h"
 
+#include "common/status.h"
 #include "gutil/strings/substitute.h"
+#include "olap/rowset/rowset_meta.h"
+#include "olap/tablet.h"
 #include "util/time.h"
 #include "util/trace.h"
 
@@ -141,8 +144,10 @@ Status Compaction::do_compaction_impl(int64_t permits) {
 
     LOG(INFO) << "start " << merge_type << compaction_name() << ". tablet=" << _tablet->full_name()
               << ", output_version=" << _output_version << ", permits: " << permits;
+    // get cur schema if rowset schema exist, rowset schema must be newer than tablet schema
+    const TabletSchema cur_tablet_schema = _tablet->tablet_schema();
 
-    RETURN_NOT_OK(construct_output_rowset_writer());
+    RETURN_NOT_OK(construct_output_rowset_writer(&cur_tablet_schema));
     RETURN_NOT_OK(construct_input_rowset_readers());
     TRACE("prepare finished");
 
@@ -152,11 +157,11 @@ Status Compaction::do_compaction_impl(int64_t permits) {
     Status res;
 
     if (use_vectorized_compaction) {
-        res = Merger::vmerge_rowsets(_tablet, compaction_type(), _input_rs_readers,
-                                     _output_rs_writer.get(), &stats);
+        res = Merger::vmerge_rowsets(_tablet, compaction_type(), &cur_tablet_schema,
+                                     _input_rs_readers, _output_rs_writer.get(), &stats);
     } else {
-        res = Merger::merge_rowsets(_tablet, compaction_type(), _input_rs_readers,
-                                    _output_rs_writer.get(), &stats);
+        res = Merger::merge_rowsets(_tablet, compaction_type(), &cur_tablet_schema,
+                                    _input_rs_readers, _output_rs_writer.get(), &stats);
     }
 
     if (!res.ok()) {
@@ -219,8 +224,8 @@ Status Compaction::do_compaction_impl(int64_t permits) {
     return Status::OK();
 }
 
-Status Compaction::construct_output_rowset_writer() {
-    return _tablet->create_rowset_writer(_output_version, VISIBLE, NONOVERLAPPING,
+Status Compaction::construct_output_rowset_writer(const TabletSchema* schema) {
+    return _tablet->create_rowset_writer(_output_version, VISIBLE, NONOVERLAPPING, schema,
                                          _oldest_write_timestamp, _newest_write_timestamp,
                                          &_output_rs_writer);
 }
diff --git a/be/src/olap/compaction.h b/be/src/olap/compaction.h
index bd76ded20a..648ff78935 100644
--- a/be/src/olap/compaction.h
+++ b/be/src/olap/compaction.h
@@ -67,7 +67,7 @@ protected:
     Status modify_rowsets();
     void gc_output_rowset();
 
-    Status construct_output_rowset_writer();
+    Status construct_output_rowset_writer(const TabletSchema* schema);
     Status construct_input_rowset_readers();
 
     Status check_version_continuity(const std::vector<RowsetSharedPtr>& rowsets);
diff --git a/be/src/olap/data_dir.cpp b/be/src/olap/data_dir.cpp
index 55530420b3..d63d792acf 100644
--- a/be/src/olap/data_dir.cpp
+++ b/be/src/olap/data_dir.cpp
@@ -485,6 +485,11 @@ Status DataDir::load() {
                           << " schema hash: " << rowset_meta->tablet_schema_hash()
                           << " for txn: " << rowset_meta->txn_id();
             }
+            if (!rowset_meta->get_rowset_pb().has_tablet_schema()) {
+                rowset_meta->set_tablet_schema(&tablet->tablet_schema());
+                RowsetMetaManager::save(_meta, rowset_meta->tablet_uid(), rowset_meta->rowset_id(),
+                                        rowset_meta->get_rowset_pb());
+            }
         } else if (rowset_meta->rowset_state() == RowsetStatePB::VISIBLE &&
                    rowset_meta->tablet_uid() == tablet->tablet_uid()) {
             Status publish_status = tablet->add_rowset(rowset);
@@ -506,6 +511,15 @@ Status DataDir::load() {
             ++invalid_rowset_counter;
         }
     }
+
+    for (int64_t tablet_id : tablet_ids) {
+        TabletSharedPtr tablet = _tablet_manager->get_tablet(tablet_id);
+        if (tablet && tablet->set_tablet_schema_into_rowset_meta()) {
+            TabletMetaManager::save(this, tablet->tablet_id(), tablet->schema_hash(),
+                                    tablet->tablet_meta());
+        }
+    }
+
     // At startup, we only count these invalid rowset, but do not actually delete it.
     // The actual delete operation is in StorageEngine::_clean_unused_rowset_metas,
     // which is cleaned up uniformly by the background cleanup thread.
diff --git a/be/src/olap/delta_writer.cpp b/be/src/olap/delta_writer.cpp
index b088ef50cc..5a1a5c5c06 100644
--- a/be/src/olap/delta_writer.cpp
+++ b/be/src/olap/delta_writer.cpp
@@ -40,7 +40,7 @@ DeltaWriter::DeltaWriter(WriteRequest* req, StorageEngine* storage_engine, bool
           _tablet(nullptr),
           _cur_rowset(nullptr),
           _rowset_writer(nullptr),
-          _tablet_schema(nullptr),
+          _tablet_schema(new TabletSchema),
           _delta_written_success(false),
           _storage_engine(storage_engine),
           _is_vec(is_vec) {}
@@ -121,10 +121,11 @@ Status DeltaWriter::init() {
         RETURN_NOT_OK(_storage_engine->txn_manager()->prepare_txn(_req.partition_id, _tablet,
                                                                   _req.txn_id, _req.load_id));
     }
+    // build tablet schema in request level
+    _build_current_tablet_schema(_req.index_id, _req.ptable_schema_param, _tablet->tablet_schema());
 
     RETURN_NOT_OK(_tablet->create_rowset_writer(_req.txn_id, _req.load_id, PREPARED, OVERLAPPING,
-                                                &_rowset_writer));
-    _tablet_schema = &(_tablet->tablet_schema());
+                                                _tablet_schema.get(), &_rowset_writer));
     _schema.reset(new Schema(*_tablet_schema));
     _reset_mem_table();
 
@@ -172,7 +173,6 @@ Status DeltaWriter::write(const RowBatch* row_batch, const std::vector<int>& row
     if (_is_cancelled) {
         return Status::OLAPInternalError(OLAP_ERR_ALREADY_CANCELLED);
     }
-
     for (const auto& row_idx : row_idxs) {
         _mem_table->insert(row_batch->get_row(row_idx)->get_tuple(0));
     }
@@ -266,9 +266,9 @@ Status DeltaWriter::wait_flush() {
 }
 
 void DeltaWriter::_reset_mem_table() {
-    _mem_table.reset(new MemTable(_tablet->tablet_id(), _schema.get(), _tablet_schema, _req.slots,
-                                  _req.tuple_desc, _tablet->keys_type(), _rowset_writer.get(),
-                                  _mem_tracker, _is_vec));
+    _mem_table.reset(new MemTable(_tablet->tablet_id(), _schema.get(), _tablet_schema.get(),
+                                  _req.slots, _req.tuple_desc, _tablet->keys_type(),
+                                  _rowset_writer.get(), _mem_tracker, _is_vec));
 }
 
 Status DeltaWriter::close() {
@@ -367,4 +367,17 @@ int64_t DeltaWriter::partition_id() const {
     return _req.partition_id;
 }
 
+void DeltaWriter::_build_current_tablet_schema(int64_t index_id,
+                                               const POlapTableSchemaParam& ptable_schema_param,
+                                               const TabletSchema& ori_tablet_schema) {
+    *_tablet_schema = ori_tablet_schema;
+    //new tablet schame if new table
+    if (ptable_schema_param.indexes_size() > 0 &&
+        ptable_schema_param.indexes(0).columns_desc_size() != 0 &&
+        ptable_schema_param.indexes(0).columns_desc(0).unique_id() >= 0) {
+        _tablet_schema->build_current_tablet_schema(index_id, ptable_schema_param,
+                                                    ori_tablet_schema);
+    }
+}
+
 } // namespace doris
diff --git a/be/src/olap/delta_writer.h b/be/src/olap/delta_writer.h
index 1e2e9b6ed0..432f0a0b76 100644
--- a/be/src/olap/delta_writer.h
+++ b/be/src/olap/delta_writer.h
@@ -47,6 +47,8 @@ struct WriteRequest {
     // slots are in order of tablet's schema
     const std::vector<SlotDescriptor*>* slots;
     bool is_high_priority = false;
+    POlapTableSchemaParam ptable_schema_param;
+    int64_t index_id;
 };
 
 // Writer for a particular (load, index, tablet).
@@ -107,6 +109,10 @@ private:
 
     void _reset_mem_table();
 
+    void _build_current_tablet_schema(int64_t index_id,
+                                      const POlapTableSchemaParam& table_schema_param,
+                                      const TabletSchema& ori_tablet_schema);
+
     bool _is_init = false;
     bool _is_cancelled = false;
     WriteRequest _req;
@@ -116,7 +122,11 @@ private:
     // TODO: Recheck the lifetime of _mem_table, Look should use unique_ptr
     std::shared_ptr<MemTable> _mem_table;
     std::unique_ptr<Schema> _schema;
-    const TabletSchema* _tablet_schema;
+    //const TabletSchema* _tablet_schema;
+    // tablet schema owned by delta writer, all write will use this tablet schema
+    // it's build from tablet_schema(stored when create tablet) and OlapTableSchema
+    // every request will have it's own tablet schema so simple schema change can work
+    std::unique_ptr<TabletSchema> _tablet_schema;
     bool _delta_written_success;
 
     StorageEngine* _storage_engine;
diff --git a/be/src/olap/merger.cpp b/be/src/olap/merger.cpp
index c9fec74b25..c9cba8ca1d 100644
--- a/be/src/olap/merger.cpp
+++ b/be/src/olap/merger.cpp
@@ -30,6 +30,7 @@
 namespace doris {
 
 Status Merger::merge_rowsets(TabletSharedPtr tablet, ReaderType reader_type,
+                             const TabletSchema* cur_tablet_schema,
                              const std::vector<RowsetReaderSharedPtr>& src_rowset_readers,
                              RowsetWriter* dst_rowset_writer, Merger::Statistics* stats_output) {
     TRACE_COUNTER_SCOPE_LATENCY_US("merge_rowsets_latency_us");
@@ -40,13 +41,15 @@ Status Merger::merge_rowsets(TabletSharedPtr tablet, ReaderType reader_type,
     reader_params.reader_type = reader_type;
     reader_params.rs_readers = src_rowset_readers;
     reader_params.version = dst_rowset_writer->version();
+
+    reader_params.tablet_schema = cur_tablet_schema;
     RETURN_NOT_OK(reader.init(reader_params));
 
     RowCursor row_cursor;
     RETURN_NOT_OK_LOG(
-            row_cursor.init(tablet->tablet_schema()),
+            row_cursor.init(*cur_tablet_schema),
             "failed to init row cursor when merging rowsets of tablet " + tablet->full_name());
-    row_cursor.allocate_memory_for_string_type(tablet->tablet_schema());
+    row_cursor.allocate_memory_for_string_type(*cur_tablet_schema);
 
     std::unique_ptr<MemPool> mem_pool(new MemPool("Merger:merge_rowsets"));
 
@@ -88,6 +91,7 @@ Status Merger::merge_rowsets(TabletSharedPtr tablet, ReaderType reader_type,
 }
 
 Status Merger::vmerge_rowsets(TabletSharedPtr tablet, ReaderType reader_type,
+                              const TabletSchema* cur_tablet_schema,
                               const std::vector<RowsetReaderSharedPtr>& src_rowset_readers,
                               RowsetWriter* dst_rowset_writer, Statistics* stats_output) {
     TRACE_COUNTER_SCOPE_LATENCY_US("merge_rowsets_latency_us");
@@ -98,8 +102,9 @@ Status Merger::vmerge_rowsets(TabletSharedPtr tablet, ReaderType reader_type,
     reader_params.reader_type = reader_type;
     reader_params.rs_readers = src_rowset_readers;
     reader_params.version = dst_rowset_writer->version();
+    reader_params.tablet_schema = cur_tablet_schema;
 
-    const auto& schema = tablet->tablet_schema();
+    const auto& schema = *cur_tablet_schema;
     reader_params.return_columns.resize(schema.num_columns());
     std::iota(reader_params.return_columns.begin(), reader_params.return_columns.end(), 0);
     reader_params.origin_return_columns = &reader_params.return_columns;
diff --git a/be/src/olap/merger.h b/be/src/olap/merger.h
index 5f258779fb..4d37e6ccd6 100644
--- a/be/src/olap/merger.h
+++ b/be/src/olap/merger.h
@@ -36,10 +36,12 @@ public:
     // return OLAP_SUCCESS and set statistics into `*stats_output`.
     // return others on error
     static Status merge_rowsets(TabletSharedPtr tablet, ReaderType reader_type,
+                                const TabletSchema* cur_tablet_schema,
                                 const std::vector<RowsetReaderSharedPtr>& src_rowset_readers,
                                 RowsetWriter* dst_rowset_writer, Statistics* stats_output);
 
     static Status vmerge_rowsets(TabletSharedPtr tablet, ReaderType reader_type,
+                                 const TabletSchema* cur_tablet_schema,
                                  const std::vector<RowsetReaderSharedPtr>& src_rowset_readers,
                                  RowsetWriter* dst_rowset_writer, Statistics* stats_output);
 };
diff --git a/be/src/olap/push_handler.cpp b/be/src/olap/push_handler.cpp
index 5b3d1b2cbc..64e936739e 100644
--- a/be/src/olap/push_handler.cpp
+++ b/be/src/olap/push_handler.cpp
@@ -116,12 +116,16 @@ Status PushHandler::_do_streaming_ingestion(TabletSharedPtr tablet, const TPushR
 
             DeletePredicatePB del_pred;
             DeleteConditionHandler del_cond_handler;
-            {
-                std::shared_lock rdlock(tablet_var.tablet->get_header_lock());
-                res = del_cond_handler.generate_delete_predicate(
-                        tablet_var.tablet->tablet_schema(), request.delete_conditions, &del_pred);
-                del_preds.push(del_pred);
+            auto tablet_schema = tablet_var.tablet->tablet_schema();
+            if (!request.columns_desc.empty() && request.columns_desc[0].col_unique_id >= 0) {
+                tablet_schema.clear_columns();
+                for (const auto& column_desc : request.columns_desc) {
+                    tablet_schema.append_column(TabletColumn(column_desc));
+                }
             }
+            res = del_cond_handler.generate_delete_predicate(tablet_schema,
+                                                             request.delete_conditions, &del_pred);
+            del_preds.push(del_pred);
             if (!res.ok()) {
                 LOG(WARNING) << "fail to generate delete condition. res=" << res
                              << ", tablet=" << tablet_var.tablet->full_name();
@@ -139,14 +143,24 @@ Status PushHandler::_do_streaming_ingestion(TabletSharedPtr tablet, const TPushR
         return Status::OLAPInternalError(OLAP_ERR_TOO_MANY_VERSION);
     }
 
-    // write
+    auto tablet_schema = tablet_vars->at(0).tablet->tablet_schema();
+    if (!request.columns_desc.empty() && request.columns_desc[0].col_unique_id >= 0) {
+        tablet_schema.clear_columns();
+        for (const auto& column_desc : request.columns_desc) {
+            tablet_schema.append_column(TabletColumn(column_desc));
+        }
+    }
+
+    // writes
     if (push_type == PUSH_NORMAL_V2) {
         res = _convert_v2(tablet_vars->at(0).tablet, tablet_vars->at(1).tablet,
-                          &(tablet_vars->at(0).rowset_to_add), &(tablet_vars->at(1).rowset_to_add));
+                          &(tablet_vars->at(0).rowset_to_add), &(tablet_vars->at(1).rowset_to_add),
+                          &tablet_schema);
 
     } else {
         res = _convert(tablet_vars->at(0).tablet, tablet_vars->at(1).tablet,
-                       &(tablet_vars->at(0).rowset_to_add), &(tablet_vars->at(1).rowset_to_add));
+                       &(tablet_vars->at(0).rowset_to_add), &(tablet_vars->at(1).rowset_to_add),
+                       &tablet_schema);
     }
     if (!res.ok()) {
         LOG(WARNING) << "fail to convert tmp file when realtime push. res=" << res
@@ -205,7 +219,8 @@ void PushHandler::_get_tablet_infos(const std::vector<TabletVars>& tablet_vars,
 }
 
 Status PushHandler::_convert_v2(TabletSharedPtr cur_tablet, TabletSharedPtr new_tablet,
-                                RowsetSharedPtr* cur_rowset, RowsetSharedPtr* new_rowset) {
+                                RowsetSharedPtr* cur_rowset, RowsetSharedPtr* new_rowset,
+                                const TabletSchema* tablet_schema) {
     Status res = Status::OK();
     uint32_t num_rows = 0;
     PUniqueId load_id;
@@ -217,13 +232,13 @@ Status PushHandler::_convert_v2(TabletSharedPtr cur_tablet, TabletSharedPtr new_
 
         // 1. init RowsetBuilder of cur_tablet for current push
         VLOG_NOTICE << "init rowset builder. tablet=" << cur_tablet->full_name()
-                    << ", block_row_size=" << cur_tablet->num_rows_per_row_block();
+                    << ", block_row_size=" << tablet_schema->num_rows_per_row_block();
         // although the spark load output files are fully sorted,
         // but it depends on thirparty implementation, so we conservatively
         // set this value to OVERLAP_UNKNOWN
         std::unique_ptr<RowsetWriter> rowset_writer;
         res = cur_tablet->create_rowset_writer(_request.transaction_id, load_id, PREPARED,
-                                               OVERLAP_UNKNOWN, &rowset_writer);
+                                               OVERLAP_UNKNOWN, tablet_schema, &rowset_writer);
         if (!res.ok()) {
             LOG(WARNING) << "failed to init rowset writer, tablet=" << cur_tablet->full_name()
                          << ", txn_id=" << _request.transaction_id << ", res=" << res;
@@ -245,7 +260,7 @@ Status PushHandler::_convert_v2(TabletSharedPtr cur_tablet, TabletSharedPtr new_
             }
 
             // init schema
-            std::unique_ptr<Schema> schema(new (std::nothrow) Schema(cur_tablet->tablet_schema()));
+            std::unique_ptr<Schema> schema(new (std::nothrow) Schema(*tablet_schema));
             if (schema == nullptr) {
                 LOG(WARNING) << "fail to create schema. tablet=" << cur_tablet->full_name();
                 res = Status::OLAPInternalError(OLAP_ERR_MALLOC_ERROR);
@@ -313,8 +328,8 @@ Status PushHandler::_convert_v2(TabletSharedPtr cur_tablet, TabletSharedPtr new_
         // 5. Convert data for schema change tables
         VLOG_TRACE << "load to related tables of schema_change if possible.";
         if (new_tablet != nullptr) {
-            res = SchemaChangeHandler::schema_version_convert(cur_tablet, new_tablet, cur_rowset,
-                                                              new_rowset, *_desc_tbl);
+            res = SchemaChangeHandler::schema_version_convert(
+                    cur_tablet, new_tablet, cur_rowset, new_rowset, *_desc_tbl, tablet_schema);
             if (!res.ok()) {
                 LOG(WARNING) << "failed to change schema version for delta."
                              << "[res=" << res << " new_tablet='" << new_tablet->full_name()
@@ -329,7 +344,8 @@ Status PushHandler::_convert_v2(TabletSharedPtr cur_tablet, TabletSharedPtr new_
 }
 
 Status PushHandler::_convert(TabletSharedPtr cur_tablet, TabletSharedPtr new_tablet,
-                             RowsetSharedPtr* cur_rowset, RowsetSharedPtr* new_rowset) {
+                             RowsetSharedPtr* cur_rowset, RowsetSharedPtr* new_rowset,
+                             const TabletSchema* tablet_schema) {
     Status res = Status::OK();
     RowCursor row;
     BinaryFile raw_file;
@@ -376,7 +392,7 @@ Status PushHandler::_convert(TabletSharedPtr cur_tablet, TabletSharedPtr new_tab
             }
 
             // init BinaryReader
-            if (!(res = reader->init(cur_tablet, &raw_file))) {
+            if (!(res = reader->init(tablet_schema, &raw_file))) {
                 LOG(WARNING) << "fail to init reader. res=" << res
                              << ", tablet=" << cur_tablet->full_name()
                              << ", file=" << _request.http_file_path;
@@ -388,7 +404,7 @@ Status PushHandler::_convert(TabletSharedPtr cur_tablet, TabletSharedPtr new_tab
         // 2. init RowsetBuilder of cur_tablet for current push
         std::unique_ptr<RowsetWriter> rowset_writer;
         res = cur_tablet->create_rowset_writer(_request.transaction_id, load_id, PREPARED,
-                                               OVERLAP_UNKNOWN, &rowset_writer);
+                                               OVERLAP_UNKNOWN, tablet_schema, &rowset_writer);
         if (!res.ok()) {
             LOG(WARNING) << "failed to init rowset writer, tablet=" << cur_tablet->full_name()
                          << ", txn_id=" << _request.transaction_id << ", res=" << res;
@@ -400,7 +416,7 @@ Status PushHandler::_convert(TabletSharedPtr cur_tablet, TabletSharedPtr new_tab
                     << ", block_row_size=" << cur_tablet->num_rows_per_row_block();
 
         // 4. Init RowCursor
-        if (!(res = row.init(cur_tablet->tablet_schema()))) {
+        if (!(res = row.init(*tablet_schema))) {
             LOG(WARNING) << "fail to init rowcursor. res=" << res;
             break;
         }
@@ -453,8 +469,8 @@ Status PushHandler::_convert(TabletSharedPtr cur_tablet, TabletSharedPtr new_tab
         // 7. Convert data for schema change tables
         VLOG_TRACE << "load to related tables of schema_change if possible.";
         if (new_tablet != nullptr) {
-            res = SchemaChangeHandler::schema_version_convert(cur_tablet, new_tablet, cur_rowset,
-                                                              new_rowset, *_desc_tbl);
+            res = SchemaChangeHandler::schema_version_convert(
+                    cur_tablet, new_tablet, cur_rowset, new_rowset, *_desc_tbl, tablet_schema);
             if (!res.ok()) {
                 LOG(WARNING) << "failed to change schema version for delta."
                              << "[res=" << res << " new_tablet='" << new_tablet->full_name()
@@ -500,13 +516,13 @@ IBinaryReader* IBinaryReader::create(bool need_decompress) {
 
 BinaryReader::BinaryReader() : _row_buf(nullptr), _row_buf_size(0) {}
 
-Status BinaryReader::init(TabletSharedPtr tablet, BinaryFile* file) {
+Status BinaryReader::init(const TabletSchema* tablet_schema, BinaryFile* file) {
     Status res = Status::OK();
 
     do {
         _file = file;
         _content_len = _file->file_length() - _file->header_size();
-        _row_buf_size = tablet->row_size();
+        _row_buf_size = _tablet_schema->row_size();
 
         _row_buf = new (std::nothrow) char[_row_buf_size];
         if (_row_buf == nullptr) {
@@ -521,7 +537,7 @@ Status BinaryReader::init(TabletSharedPtr tablet, BinaryFile* file) {
             break;
         }
 
-        _tablet = tablet;
+        _tablet_schema = tablet_schema;
         _ready = true;
     } while (false);
 
@@ -545,10 +561,10 @@ Status BinaryReader::next(RowCursor* row) {
         return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
     }
 
-    const TabletSchema& schema = _tablet->tablet_schema();
+    const TabletSchema& schema = *_tablet_schema;
     size_t offset = 0;
     size_t field_size = 0;
-    size_t num_null_bytes = (_tablet->num_null_columns() + 7) / 8;
+    size_t num_null_bytes = (schema.num_null_columns() + 7) / 8;
 
     if (!(res = _file->read(_row_buf + offset, num_null_bytes))) {
         LOG(WARNING) << "read file for one row fail. res=" << res;
@@ -642,7 +658,7 @@ LzoBinaryReader::LzoBinaryReader()
           _row_num(0),
           _next_row_start(0) {}
 
-Status LzoBinaryReader::init(TabletSharedPtr tablet, BinaryFile* file) {
+Status LzoBinaryReader::init(const TabletSchema* tablet_schema, BinaryFile* file) {
     Status res = Status::OK();
 
     do {
@@ -663,7 +679,7 @@ Status LzoBinaryReader::init(TabletSharedPtr tablet, BinaryFile* file) {
             break;
         }
 
-        _tablet = tablet;
+        _tablet_schema = tablet_schema;
         _ready = true;
     } while (false);
 
@@ -696,10 +712,10 @@ Status LzoBinaryReader::next(RowCursor* row) {
         }
     }
 
-    const TabletSchema& schema = _tablet->tablet_schema();
+    const TabletSchema& schema = *_tablet_schema;
     size_t offset = 0;
     size_t field_size = 0;
-    size_t num_null_bytes = (_tablet->num_null_columns() + 7) / 8;
+    size_t num_null_bytes = (schema.num_null_columns() + 7) / 8;
 
     size_t p = 0;
     for (size_t i = 0; i < schema.num_columns(); ++i) {
@@ -787,7 +803,7 @@ Status LzoBinaryReader::_next_block() {
         SAFE_DELETE_ARRAY(_row_buf);
 
         _max_row_num = _row_num;
-        _max_row_buf_size = _max_row_num * _tablet->row_size();
+        _max_row_buf_size = _max_row_num * _tablet_schema->row_size();
         _row_buf = new (std::nothrow) char[_max_row_buf_size];
         if (_row_buf == nullptr) {
             LOG(WARNING) << "fail to malloc rows buf. size=" << _max_row_buf_size;
diff --git a/be/src/olap/push_handler.h b/be/src/olap/push_handler.h
index a290eb01c7..70c134c983 100644
--- a/be/src/olap/push_handler.h
+++ b/be/src/olap/push_handler.h
@@ -60,11 +60,13 @@ public:
 
 private:
     Status _convert_v2(TabletSharedPtr cur_tablet, TabletSharedPtr new_tablet_vec,
-                       RowsetSharedPtr* cur_rowset, RowsetSharedPtr* new_rowset);
+                       RowsetSharedPtr* cur_rowset, RowsetSharedPtr* new_rowset,
+                       const TabletSchema* tablet_schema);
     // Convert local data file to internal formatted delta,
     // return new delta's SegmentGroup
     Status _convert(TabletSharedPtr cur_tablet, TabletSharedPtr new_tablet_vec,
-                    RowsetSharedPtr* cur_rowset, RowsetSharedPtr* new_rowset);
+                    RowsetSharedPtr* cur_rowset, RowsetSharedPtr* new_rowset,
+                    const TabletSchema* tablet_schema);
 
     // Only for debug
     std::string _debug_version_list(const Versions& versions) const;
@@ -112,7 +114,7 @@ public:
     static IBinaryReader* create(bool need_decompress);
     virtual ~IBinaryReader() = default;
 
-    virtual Status init(TabletSharedPtr tablet, BinaryFile* file) = 0;
+    virtual Status init(const TabletSchema* tablet_schema, BinaryFile* file) = 0;
     virtual Status finalize() = 0;
 
     virtual Status next(RowCursor* row) = 0;
@@ -131,7 +133,7 @@ protected:
               _ready(false) {}
 
     BinaryFile* _file;
-    TabletSharedPtr _tablet;
+    const TabletSchema* _tablet_schema;
     size_t _content_len;
     size_t _curr;
     uint32_t _adler_checksum;
@@ -144,7 +146,7 @@ public:
     explicit BinaryReader();
     ~BinaryReader() override { finalize(); }
 
-    Status init(TabletSharedPtr tablet, BinaryFile* file) override;
+    Status init(const TabletSchema* tablet_schema, BinaryFile* file) override;
     Status finalize() override;
 
     Status next(RowCursor* row) override;
@@ -161,7 +163,7 @@ public:
     explicit LzoBinaryReader();
     ~LzoBinaryReader() override { finalize(); }
 
-    Status init(TabletSharedPtr tablet, BinaryFile* file) override;
+    Status init(const TabletSchema* tablet_schema, BinaryFile* file) override;
     Status finalize() override;
 
     Status next(RowCursor* row) override;
diff --git a/be/src/olap/reader.cpp b/be/src/olap/reader.cpp
index 3d8a3f398f..a3286f4b33 100644
--- a/be/src/olap/reader.cpp
+++ b/be/src/olap/reader.cpp
@@ -188,7 +188,7 @@ Status TabletReader::_capture_rs_readers(const ReaderParams& read_params,
 
     bool need_ordered_result = true;
     if (read_params.reader_type == READER_QUERY) {
-        if (_tablet->tablet_schema().keys_type() == DUP_KEYS) {
+        if (_tablet_schema->keys_type() == DUP_KEYS) {
             // duplicated keys are allowed, no need to merge sort keys in rowset
             need_ordered_result = false;
         }
@@ -200,7 +200,7 @@ Status TabletReader::_capture_rs_readers(const ReaderParams& read_params,
     }
 
     _reader_context.reader_type = read_params.reader_type;
-    _reader_context.tablet_schema = &_tablet->tablet_schema();
+    _reader_context.tablet_schema = _tablet_schema;
     _reader_context.need_ordered_result = need_ordered_result;
     _reader_context.return_columns = &_return_columns;
     _reader_context.seek_columns = &_seek_columns;
@@ -235,6 +235,7 @@ Status TabletReader::_init_params(const ReaderParams& read_params) {
     _need_agg_finalize = read_params.need_agg_finalize;
     _reader_type = read_params.reader_type;
     _tablet = read_params.tablet;
+    _tablet_schema = read_params.tablet_schema;
 
     _init_conditions_param(read_params);
     _init_load_bf_columns(read_params);
@@ -259,8 +260,8 @@ Status TabletReader::_init_params(const ReaderParams& read_params) {
 
     _init_seek_columns();
 
-    if (_tablet->tablet_schema().has_sequence_col()) {
-        auto sequence_col_idx = _tablet->tablet_schema().sequence_col_idx();
+    if (_tablet_schema->has_sequence_col()) {
+        auto sequence_col_idx = _tablet_schema->sequence_col_idx();
         DCHECK_NE(sequence_col_idx, -1);
         for (auto col : _return_columns) {
             // query has sequence col
@@ -292,16 +293,16 @@ Status TabletReader::_init_return_columns(const ReaderParams& read_params) {
             }
         }
         for (auto id : read_params.return_columns) {
-            if (_tablet->tablet_schema().column(id).is_key()) {
+            if (_tablet_schema->column(id).is_key()) {
                 _key_cids.push_back(id);
             } else {
                 _value_cids.push_back(id);
             }
         }
     } else if (read_params.return_columns.empty()) {
-        for (size_t i = 0; i < _tablet->tablet_schema().num_columns(); ++i) {
+        for (size_t i = 0; i < _tablet_schema->num_columns(); ++i) {
             _return_columns.push_back(i);
-            if (_tablet->tablet_schema().column(i).is_key()) {
+            if (_tablet_schema->column(i).is_key()) {
                 _key_cids.push_back(i);
             } else {
                 _value_cids.push_back(i);
@@ -314,7 +315,7 @@ Status TabletReader::_init_return_columns(const ReaderParams& read_params) {
                !read_params.return_columns.empty()) {
         _return_columns = read_params.return_columns;
         for (auto id : read_params.return_columns) {
-            if (_tablet->tablet_schema().column(id).is_key()) {
+            if (_tablet_schema->column(id).is_key()) {
                 _key_cids.push_back(id);
             } else {
                 _value_cids.push_back(id);
@@ -323,7 +324,7 @@ Status TabletReader::_init_return_columns(const ReaderParams& read_params) {
     } else if (read_params.reader_type == READER_CHECKSUM) {
         _return_columns = read_params.return_columns;
         for (auto id : read_params.return_columns) {
-            if (_tablet->tablet_schema().column(id).is_key()) {
+            if (_tablet_schema->column(id).is_key()) {
                 _key_cids.push_back(id);
             } else {
                 _value_cids.push_back(id);
@@ -353,7 +354,7 @@ void TabletReader::_init_seek_columns() {
         max_key_column_count = std::max(max_key_column_count, key.field_count());
     }
 
-    for (size_t i = 0; i < _tablet->tablet_schema().num_columns(); i++) {
+    for (size_t i = 0; i < _tablet_schema->num_columns(); i++) {
         if (i < max_key_column_count || column_set.find(i) != column_set.end()) {
             _seek_columns.push_back(i);
         }
@@ -373,19 +374,18 @@ Status TabletReader::_init_keys_param(const ReaderParams& read_params) {
     std::vector<RowCursor>(start_key_size).swap(_keys_param.start_keys);
 
     size_t scan_key_size = read_params.start_key.front().size();
-    if (scan_key_size > _tablet->tablet_schema().num_columns()) {
+    if (scan_key_size > _tablet_schema->num_columns()) {
         LOG(WARNING)
                 << "Input param are invalid. Column count is bigger than num_columns of schema. "
                 << "column_count=" << scan_key_size
-                << ", schema.num_columns=" << _tablet->tablet_schema().num_columns();
+                << ", schema.num_columns=" << _tablet_schema->num_columns();
         return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
     }
 
     std::vector<uint32_t> columns(scan_key_size);
     std::iota(columns.begin(), columns.end(), 0);
 
-    std::shared_ptr<Schema> schema =
-            std::make_shared<Schema>(_tablet->tablet_schema().columns(), columns);
+    std::shared_ptr<Schema> schema = std::make_shared<Schema>(_tablet_schema->columns(), columns);
 
     for (size_t i = 0; i < start_key_size; ++i) {
         if (read_params.start_key[i].size() != scan_key_size) {
@@ -396,12 +396,11 @@ Status TabletReader::_init_keys_param(const ReaderParams& read_params) {
         }
 
         Status res = _keys_param.start_keys[i].init_scan_key(
-                _tablet->tablet_schema(), read_params.start_key[i].values(), schema);
+                *_tablet_schema, read_params.start_key[i].values(), schema);
         if (!res.ok()) {
             LOG(WARNING) << "fail to init row cursor. res = " << res;
             return res;
         }
-
         res = _keys_param.start_keys[i].from_tuple(read_params.start_key[i]);
         if (!res.ok()) {
             LOG(WARNING) << "fail to init row cursor from Keys. res=" << res << "key_index=" << i;
@@ -419,7 +418,7 @@ Status TabletReader::_init_keys_param(const ReaderParams& read_params) {
             return Status::OLAPInternalError(OLAP_ERR_INPUT_PARAMETER_ERROR);
         }
 
-        Status res = _keys_param.end_keys[i].init_scan_key(_tablet->tablet_schema(),
+        Status res = _keys_param.end_keys[i].init_scan_key(*_tablet_schema,
                                                            read_params.end_key[i].values(), schema);
         if (!res.ok()) {
             LOG(WARNING) << "fail to init row cursor. res = " << res;
@@ -439,13 +438,12 @@ Status TabletReader::_init_keys_param(const ReaderParams& read_params) {
 }
 
 void TabletReader::_init_conditions_param(const ReaderParams& read_params) {
-    _conditions.set_tablet_schema(&_tablet->tablet_schema());
-    _all_conditions.set_tablet_schema(&_tablet->tablet_schema());
+    _conditions.set_tablet_schema(_tablet_schema);
+    _all_conditions.set_tablet_schema(_tablet_schema);
     for (const auto& condition : read_params.conditions) {
         ColumnPredicate* predicate = _parse_to_predicate(condition);
         if (predicate != nullptr) {
-            if (_tablet->tablet_schema()
-                        .column(_tablet->field_index(condition.column_name))
+            if (_tablet_schema->column(_tablet_schema->field_index(condition.column_name))
                         .aggregation() != FieldAggregationMethod::OLAP_FIELD_AGGREGATION_NONE) {
                 _value_col_predicates.push_back(predicate);
             } else {
@@ -576,11 +574,11 @@ COMPARISON_PREDICATE_CONDITION_VALUE(ge, GreaterEqualPredicate)
 
 ColumnPredicate* TabletReader::_parse_to_predicate(
         const std::pair<std::string, std::shared_ptr<IBloomFilterFuncBase>>& bloom_filter) {
-    int32_t index = _tablet->field_index(bloom_filter.first);
+    int32_t index = _tablet_schema->field_index(bloom_filter.first);
     if (index < 0) {
         return nullptr;
     }
-    const TabletColumn& column = _tablet->tablet_schema().column(index);
+    const TabletColumn& column = _tablet_schema->column(index);
     return BloomFilterColumnPredicateFactory::create_column_predicate(index, bloom_filter.second,
                                                                       column.type());
 }
@@ -588,12 +586,12 @@ ColumnPredicate* TabletReader::_parse_to_predicate(
 ColumnPredicate* TabletReader::_parse_to_predicate(const TCondition& condition,
                                                    bool opposite) const {
     // TODO: not equal and not in predicate is not pushed down
-    int32_t index = _tablet->field_index(condition.column_name);
+    int32_t index = _tablet_schema->field_index(condition.column_name);
     if (index < 0) {
         return nullptr;
     }
 
-    const TabletColumn& column = _tablet->tablet_schema().column(index);
+    const TabletColumn& column = _tablet_schema->column(index);
     ColumnPredicate* predicate = nullptr;
 
     if ((condition.condition_op == "*=" || condition.condition_op == "!*=" ||
@@ -795,7 +793,7 @@ void TabletReader::_init_load_bf_columns(const ReaderParams& read_params, Condit
                                          std::set<uint32_t>* load_bf_columns) {
     // add all columns with condition to load_bf_columns
     for (const auto& cond_column : conditions->columns()) {
-        if (!_tablet->tablet_schema().column(cond_column.first).is_bf_column()) {
+        if (!_tablet_schema->column(cond_column.first).is_bf_column()) {
             continue;
         }
         for (const auto& cond : cond_column.second->conds()) {
@@ -807,7 +805,7 @@ void TabletReader::_init_load_bf_columns(const ReaderParams& read_params, Condit
     }
 
     // remove columns which have same value between start_key and end_key
-    int min_scan_key_len = _tablet->tablet_schema().num_columns();
+    int min_scan_key_len = _tablet_schema->num_columns();
     for (const auto& start_key : read_params.start_key) {
         min_scan_key_len = std::min(min_scan_key_len, static_cast<int>(start_key.size()));
     }
@@ -838,7 +836,7 @@ void TabletReader::_init_load_bf_columns(const ReaderParams& read_params, Condit
     if (max_equal_index == -1) {
         return;
     }
-    FieldType type = _tablet->tablet_schema().column(max_equal_index).type();
+    FieldType type = _tablet_schema->column(max_equal_index).type();
     if ((type != OLAP_FIELD_TYPE_VARCHAR && type != OLAP_FIELD_TYPE_STRING) ||
         max_equal_index + 1 > _tablet->num_short_key_columns()) {
         load_bf_columns->erase(max_equal_index);
@@ -858,7 +856,7 @@ Status TabletReader::_init_delete_condition(const ReaderParams& read_params) {
     }
 
     auto delete_init = [&]() -> Status {
-        return _delete_handler.init(_tablet->tablet_schema(), _tablet->delete_predicates(),
+        return _delete_handler.init(*_tablet_schema, _tablet->delete_predicates(),
                                     read_params.version.second, this);
     };
 
diff --git a/be/src/olap/reader.h b/be/src/olap/reader.h
index c0fb42ebf8..2ca738f8fd 100644
--- a/be/src/olap/reader.h
+++ b/be/src/olap/reader.h
@@ -27,6 +27,7 @@
 #include "olap/row_cursor.h"
 #include "olap/rowset/rowset_reader.h"
 #include "olap/tablet.h"
+#include "olap/tablet_schema.h"
 #include "util/runtime_profile.h"
 
 namespace doris {
@@ -57,6 +58,7 @@ public:
     // mainly include tablet, data version and fetch range.
     struct ReaderParams {
         TabletSharedPtr tablet;
+        const TabletSchema* tablet_schema;
         ReaderType reader_type = READER_QUERY;
         bool direct_mode = false;
         bool aggregation = false;
@@ -171,6 +173,7 @@ protected:
                                std::set<uint32_t>* load_bf_columns);
 
     TabletSharedPtr tablet() { return _tablet; }
+    const TabletSchema& tablet_schema() { return *_tablet_schema; }
 
     std::unique_ptr<MemPool> _predicate_mem_pool;
     std::set<uint32_t> _load_bf_columns;
@@ -183,6 +186,7 @@ protected:
 
     TabletSharedPtr _tablet;
     RowsetReaderContext _reader_context;
+    const TabletSchema* _tablet_schema;
     KeysParam _keys_param;
     std::vector<bool> _is_lower_keys_included;
     std::vector<bool> _is_upper_keys_included;
diff --git a/be/src/olap/rowset/beta_rowset.cpp b/be/src/olap/rowset/beta_rowset.cpp
index d8ef290cda..14fdeee317 100644
--- a/be/src/olap/rowset/beta_rowset.cpp
+++ b/be/src/olap/rowset/beta_rowset.cpp
@@ -69,7 +69,8 @@ Status BetaRowset::do_load(bool /*use_cache*/) {
     return Status::OK();
 }
 
-Status BetaRowset::load_segments(std::vector<segment_v2::SegmentSharedPtr>* segments) {
+Status BetaRowset::load_segments(std::vector<segment_v2::SegmentSharedPtr>* segments,
+                                 const TabletSchema* read_tablet_schema) {
     auto fs = _rowset_meta->fs();
     if (!fs) {
         return Status::OLAPInternalError(OLAP_ERR_INIT_FAILED);
@@ -77,7 +78,7 @@ Status BetaRowset::load_segments(std::vector<segment_v2::SegmentSharedPtr>* segm
     for (int seg_id = 0; seg_id < num_segments(); ++seg_id) {
         auto seg_path = segment_file_path(seg_id);
         std::shared_ptr<segment_v2::Segment> segment;
-        auto s = segment_v2::Segment::open(fs, seg_path, seg_id, _schema, &segment);
+        auto s = segment_v2::Segment::open(fs, seg_path, seg_id, read_tablet_schema, &segment);
         if (!s.ok()) {
             LOG(WARNING) << "failed to open segment. " << seg_path << " under rowset "
                          << unique_id() << " : " << s.to_string();
diff --git a/be/src/olap/rowset/beta_rowset.h b/be/src/olap/rowset/beta_rowset.h
index 9ccaa7b2b9..e1e963f6ba 100644
--- a/be/src/olap/rowset/beta_rowset.h
+++ b/be/src/olap/rowset/beta_rowset.h
@@ -70,7 +70,8 @@ public:
 
     bool check_file_exist() override;
 
-    Status load_segments(std::vector<segment_v2::SegmentSharedPtr>* segments);
+    Status load_segments(std::vector<segment_v2::SegmentSharedPtr>* segments,
+                         const TabletSchema* read_tablet_schema);
 
 protected:
     BetaRowset(const TabletSchema* schema, const std::string& tablet_path,
diff --git a/be/src/olap/rowset/beta_rowset_reader.cpp b/be/src/olap/rowset/beta_rowset_reader.cpp
index b5ff00e077..5dd7c12483 100644
--- a/be/src/olap/rowset/beta_rowset_reader.cpp
+++ b/be/src/olap/rowset/beta_rowset_reader.cpp
@@ -86,7 +86,7 @@ Status BetaRowsetReader::init(RowsetReaderContext* read_context) {
 
     // load segments
     RETURN_NOT_OK(SegmentLoader::instance()->load_segments(
-            _rowset, &_segment_cache_handle,
+            _rowset, &_segment_cache_handle, read_context->tablet_schema,
             read_context->reader_type == ReaderType::READER_QUERY));
 
     // create iterator for each segment
diff --git a/be/src/olap/rowset/beta_rowset_writer.cpp b/be/src/olap/rowset/beta_rowset_writer.cpp
index 439327ef79..87a4ba8b4c 100644
--- a/be/src/olap/rowset/beta_rowset_writer.cpp
+++ b/be/src/olap/rowset/beta_rowset_writer.cpp
@@ -88,6 +88,8 @@ Status BetaRowsetWriter::init(const RowsetWriterContext& rowset_writer_context)
         _rowset_meta->set_newest_write_timestamp(_context.newest_write_timestamp);
     }
     _rowset_meta->set_tablet_uid(_context.tablet_uid);
+    _rowset_meta->set_tablet_schema(_context.tablet_schema);
+
     return Status::OK();
 }
 
diff --git a/be/src/olap/rowset/rowset.cpp b/be/src/olap/rowset/rowset.cpp
index 55fe81ffd3..009ce832bf 100644
--- a/be/src/olap/rowset/rowset.cpp
+++ b/be/src/olap/rowset/rowset.cpp
@@ -23,10 +23,7 @@ namespace doris {
 
 Rowset::Rowset(const TabletSchema* schema, const std::string& tablet_path,
                RowsetMetaSharedPtr rowset_meta)
-        : _schema(schema),
-          _tablet_path(tablet_path),
-          _rowset_meta(std::move(rowset_meta)),
-          _refs_by_reader(0) {
+        : _tablet_path(tablet_path), _rowset_meta(std::move(rowset_meta)), _refs_by_reader(0) {
     _is_pending = !_rowset_meta->has_version();
     if (_is_pending) {
         _is_cumulative = false;
@@ -34,6 +31,8 @@ Rowset::Rowset(const TabletSchema* schema, const std::string& tablet_path,
         Version version = _rowset_meta->version();
         _is_cumulative = version.first != version.second;
     }
+    // build schema from RowsetMeta.tablet_schema or Tablet.tablet_schema
+    _schema = _rowset_meta->tablet_schema() != nullptr ? _rowset_meta->tablet_schema() : schema;
 }
 
 Status Rowset::load(bool use_cache) {
diff --git a/be/src/olap/rowset/rowset.h b/be/src/olap/rowset/rowset.h
index ec2a39652b..3488ed2b54 100644
--- a/be/src/olap/rowset/rowset.h
+++ b/be/src/olap/rowset/rowset.h
@@ -138,6 +138,7 @@ public:
 
     // publish rowset to make it visible to read
     void make_visible(Version version);
+    const TabletSchema* tablet_schema() { return _schema; }
 
     // helper class to access RowsetMeta
     int64_t start_version() const { return rowset_meta()->version().first; }
diff --git a/be/src/olap/rowset/rowset_meta.h b/be/src/olap/rowset/rowset_meta.h
index c91fe0469d..d8f2c98249 100644
--- a/be/src/olap/rowset/rowset_meta.h
+++ b/be/src/olap/rowset/rowset_meta.h
@@ -31,6 +31,7 @@
 #include "json2pb/json_to_pb.h"
 #include "json2pb/pb_to_json.h"
 #include "olap/olap_common.h"
+#include "olap/tablet_schema.h"
 
 namespace doris {
 
@@ -326,6 +327,14 @@ public:
     int64_t oldest_write_timestamp() const { return _rowset_meta_pb.oldest_write_timestamp(); }
 
     int64_t newest_write_timestamp() const { return _rowset_meta_pb.newest_write_timestamp(); }
+    void set_tablet_schema(const TabletSchema* tablet_schema) {
+        TabletSchemaPB* ts_pb = _rowset_meta_pb.mutable_tablet_schema();
+        tablet_schema->to_schema_pb(ts_pb);
+        CHECK(_schema == nullptr);
+        _schema = std::make_shared<TabletSchema>(*tablet_schema);
+    }
+
+    const TabletSchema* tablet_schema() { return _schema.get(); }
 
 private:
     friend class AlphaRowsetMeta;
@@ -369,6 +378,10 @@ private:
             }
             set_num_segments(num_segments);
         }
+        if (_rowset_meta_pb.has_tablet_schema()) {
+            _schema = std::make_shared<TabletSchema>();
+            _schema->init_from_pb(_rowset_meta_pb.tablet_schema());
+        }
     }
 
     friend bool operator==(const RowsetMeta& a, const RowsetMeta& b) {
@@ -384,6 +397,7 @@ private:
 
 private:
     RowsetMetaPB _rowset_meta_pb;
+    std::shared_ptr<TabletSchema> _schema = nullptr;
     RowsetId _rowset_id;
     io::FileSystemPtr _fs;
     bool _is_removed_from_rowset_meta = false;
diff --git a/be/src/olap/rowset/segment_v2/segment.cpp b/be/src/olap/rowset/segment_v2/segment.cpp
index fcf1d58da6..e18a000905 100644
--- a/be/src/olap/rowset/segment_v2/segment.cpp
+++ b/be/src/olap/rowset/segment_v2/segment.cpp
@@ -47,7 +47,7 @@ Status Segment::open(io::FileSystem* fs, const std::string& path, uint32_t segme
 
 Segment::Segment(io::FileSystem* fs, const std::string& path, uint32_t segment_id,
                  const TabletSchema* tablet_schema)
-        : _fs(fs), _path(path), _segment_id(segment_id), _tablet_schema(tablet_schema) {
+        : _fs(fs), _path(path), _segment_id(segment_id), _tablet_schema(*tablet_schema) {
 #ifndef BE_TEST
     _mem_tracker = StorageEngine::instance()->tablet_mem_tracker();
 #else
@@ -177,16 +177,16 @@ Status Segment::_create_column_readers() {
         _column_id_to_footer_ordinal.emplace(column_pb.unique_id(), ordinal);
     }
 
-    _column_readers.resize(_tablet_schema->columns().size());
-    for (uint32_t ordinal = 0; ordinal < _tablet_schema->num_columns(); ++ordinal) {
-        auto& column = _tablet_schema->columns()[ordinal];
+    _column_readers.resize(_tablet_schema.columns().size());
+    for (uint32_t ordinal = 0; ordinal < _tablet_schema.num_columns(); ++ordinal) {
+        auto& column = _tablet_schema.columns()[ordinal];
         auto iter = _column_id_to_footer_ordinal.find(column.unique_id());
         if (iter == _column_id_to_footer_ordinal.end()) {
             continue;
         }
 
         ColumnReaderOptions opts;
-        opts.kept_in_memory = _tablet_schema->is_in_memory();
+        opts.kept_in_memory = _tablet_schema.is_in_memory();
         std::unique_ptr<ColumnReader> reader;
         RETURN_IF_ERROR(ColumnReader::create(opts, _footer.columns(iter->second),
                                              _footer.num_rows(), _fs, _path, &reader));
@@ -197,7 +197,7 @@ Status Segment::_create_column_readers() {
 
 Status Segment::new_column_iterator(uint32_t cid, ColumnIterator** iter) {
     if (_column_readers[cid] == nullptr) {
-        const TabletColumn& tablet_column = _tablet_schema->column(cid);
+        const TabletColumn& tablet_column = _tablet_schema.column(cid);
         if (!tablet_column.has_default_value() && !tablet_column.is_nullable()) {
             return Status::InternalError("invalid nonexistent column without default value.");
         }
diff --git a/be/src/olap/rowset/segment_v2/segment.h b/be/src/olap/rowset/segment_v2/segment.h
index 11925e35e9..2c9cb83df1 100644
--- a/be/src/olap/rowset/segment_v2/segment.h
+++ b/be/src/olap/rowset/segment_v2/segment.h
@@ -76,7 +76,7 @@ public:
 
     Status new_bitmap_index_iterator(uint32_t cid, BitmapIndexIterator** iter);
 
-    size_t num_short_keys() const { return _tablet_schema->num_short_key_columns(); }
+    size_t num_short_keys() const { return _tablet_schema.num_short_key_columns(); }
 
     uint32_t num_rows_per_block() const {
         DCHECK(_load_index_once.has_called() && _load_index_once.stored_result().ok());
@@ -121,7 +121,7 @@ private:
     std::string _path;
 
     uint32_t _segment_id;
-    const TabletSchema* _tablet_schema;
+    TabletSchema _tablet_schema;
 
     // This mem tracker is only for tracking memory use by segment meta data such as footer or index page.
     // The memory consumed by querying is tracked in segment iterator.
diff --git a/be/src/olap/rowset/segment_v2/segment_iterator.cpp b/be/src/olap/rowset/segment_v2/segment_iterator.cpp
index 0e81609005..cedc0a9c6f 100644
--- a/be/src/olap/rowset/segment_v2/segment_iterator.cpp
+++ b/be/src/olap/rowset/segment_v2/segment_iterator.cpp
@@ -143,7 +143,7 @@ Status SegmentIterator::_init(bool is_vec) {
     RETURN_IF_ERROR(_init_return_column_iterators());
     RETURN_IF_ERROR(_init_bitmap_index_iterators());
     // z-order can not use prefix index
-    if (_segment->_tablet_schema->sort_type() != SortType::ZORDER) {
+    if (_segment->_tablet_schema.sort_type() != SortType::ZORDER) {
         RETURN_IF_ERROR(_get_row_ranges_by_keys());
     }
     RETURN_IF_ERROR(_get_row_ranges_by_column_conditions());
diff --git a/be/src/olap/schema_change.cpp b/be/src/olap/schema_change.cpp
index 7f282d0517..ba0609e03e 100644
--- a/be/src/olap/schema_change.cpp
+++ b/be/src/olap/schema_change.cpp
@@ -1632,8 +1632,8 @@ bool SchemaChangeWithSorting::_internal_sorting(
 
     std::unique_ptr<RowsetWriter> rowset_writer;
     if (!new_tablet->create_rowset_writer(version, VISIBLE, segments_overlap,
-                                          oldest_write_timestamp, newest_write_timestamp,
-                                          &rowset_writer)) {
+                                          &new_tablet->tablet_schema(), oldest_write_timestamp,
+                                          newest_write_timestamp, &rowset_writer)) {
         return false;
     }
 
@@ -1658,9 +1658,9 @@ Status VSchemaChangeWithSorting::_internal_sorting(
     MultiBlockMerger merger(new_tablet);
 
     std::unique_ptr<RowsetWriter> rowset_writer;
-    RETURN_IF_ERROR(new_tablet->create_rowset_writer(version, VISIBLE, segments_overlap,
-                                                     oldest_write_timestamp, newest_write_timestamp,
-                                                     &rowset_writer));
+    RETURN_IF_ERROR(new_tablet->create_rowset_writer(
+            version, VISIBLE, segments_overlap, &new_tablet->tablet_schema(),
+            oldest_write_timestamp, newest_write_timestamp, &rowset_writer));
 
     Defer defer {[&]() {
         new_tablet->data_dir()->remove_pending_ids(ROWSET_ID_PREFIX +
@@ -1687,10 +1687,16 @@ bool SchemaChangeWithSorting::_external_sorting(vector<RowsetSharedPtr>& src_row
         }
         rs_readers.push_back(rs_reader);
     }
+    // get cur schema if rowset schema exist, rowset schema must be newer than tablet schema
+    auto max_version_rowset = src_rowsets.back();
+    const TabletSchema* cur_tablet_schema = max_version_rowset->rowset_meta()->tablet_schema();
+    if (cur_tablet_schema == nullptr) {
+        cur_tablet_schema = &(new_tablet->tablet_schema());
+    }
 
     Merger::Statistics stats;
-    auto res = Merger::merge_rowsets(new_tablet, READER_ALTER_TABLE, rs_readers, rowset_writer,
-                                     &stats);
+    auto res = Merger::merge_rowsets(new_tablet, READER_ALTER_TABLE, cur_tablet_schema, rs_readers,
+                                     rowset_writer, &stats);
     if (!res) {
         LOG(WARNING) << "failed to merge rowsets. tablet=" << new_tablet->full_name()
                      << ", version=" << rowset_writer->version().first << "-"
@@ -1713,8 +1719,9 @@ Status VSchemaChangeWithSorting::_external_sorting(vector<RowsetSharedPtr>& src_
     }
 
     Merger::Statistics stats;
-    RETURN_IF_ERROR(Merger::vmerge_rowsets(new_tablet, READER_ALTER_TABLE, rs_readers,
-                                           rowset_writer, &stats));
+    RETURN_IF_ERROR(Merger::vmerge_rowsets(new_tablet, READER_ALTER_TABLE,
+                                           &new_tablet->tablet_schema(), rs_readers, rowset_writer,
+                                           &stats));
 
     _add_merged_rows(stats.merged_rows);
     _add_filtered_rows(stats.filtered_rows);
@@ -1801,6 +1808,13 @@ Status SchemaChangeHandler::_do_process_alter_tablet_v2(const TAlterTabletReqV2&
     // delete handlers for new tablet
     DeleteHandler delete_handler;
     std::vector<ColumnId> return_columns;
+    auto base_tablet_schema = base_tablet->tablet_schema();
+    if (!request.columns.empty() && request.columns[0].col_unique_id >= 0) {
+        base_tablet_schema.clear_columns();
+        for (const auto& column : request.columns) {
+            base_tablet_schema.append_column(TabletColumn(column));
+        }
+    }
 
     // begin to find deltas to convert from base tablet to new tablet so that
     // obtain base tablet and new tablet's push lock and header write lock to prevent loading data
@@ -1811,7 +1825,7 @@ Status SchemaChangeHandler::_do_process_alter_tablet_v2(const TAlterTabletReqV2&
         std::lock_guard<std::shared_mutex> new_tablet_wlock(new_tablet->get_header_lock());
         // check if the tablet has alter task
         // if it has alter task, it means it is under old alter process
-        size_t num_cols = base_tablet->tablet_schema().num_columns();
+        size_t num_cols = base_tablet_schema.num_columns();
         return_columns.resize(num_cols);
         for (int i = 0; i < num_cols; ++i) {
             return_columns[i] = i;
@@ -1821,7 +1835,7 @@ Status SchemaChangeHandler::_do_process_alter_tablet_v2(const TAlterTabletReqV2&
         // with rs_readers
         RowsetReaderContext reader_context;
         reader_context.reader_type = READER_ALTER_TABLE;
-        reader_context.tablet_schema = &base_tablet->tablet_schema();
+        reader_context.tablet_schema = &base_tablet_schema;
         reader_context.need_ordered_result = true;
         reader_context.delete_handler = &delete_handler;
         reader_context.return_columns = &return_columns;
@@ -1907,7 +1921,8 @@ Status SchemaChangeHandler::_do_process_alter_tablet_v2(const TAlterTabletReqV2&
             reader_params.tablet = base_tablet;
             reader_params.reader_type = READER_ALTER_TABLE;
             reader_params.rs_readers = rs_readers;
-            const auto& schema = base_tablet->tablet_schema();
+            reader_params.tablet_schema = &base_tablet_schema;
+            const auto& schema = base_tablet_schema;
             reader_params.return_columns.resize(schema.num_columns());
             std::iota(reader_params.return_columns.begin(), reader_params.return_columns.end(), 0);
             reader_params.origin_return_columns = &reader_params.return_columns;
@@ -1915,8 +1930,8 @@ Status SchemaChangeHandler::_do_process_alter_tablet_v2(const TAlterTabletReqV2&
             // BlockReader::init will call base_tablet->get_header_lock(), but this lock we already get at outer layer, so we just call TabletReader::init
             RETURN_NOT_OK(reader.TabletReader::init(reader_params));
 
-            res = delete_handler.init(base_tablet->tablet_schema(),
-                                      base_tablet->delete_predicates(), end_version, &reader);
+            res = delete_handler.init(base_tablet_schema, base_tablet->delete_predicates(),
+                                      end_version, &reader);
             if (!res) {
                 LOG(WARNING) << "init delete handler failed. base_tablet="
                              << base_tablet->full_name() << ", end_version=" << end_version;
@@ -1947,6 +1962,7 @@ Status SchemaChangeHandler::_do_process_alter_tablet_v2(const TAlterTabletReqV2&
         sc_params.new_tablet = new_tablet;
         sc_params.ref_rowset_readers = rs_readers;
         sc_params.delete_handler = &delete_handler;
+        sc_params.base_tablet_schema = &base_tablet_schema;
         if (request.__isset.materialized_view_params) {
             for (auto item : request.materialized_view_params) {
                 AlterMaterializedViewParam mv_param;
@@ -2025,7 +2041,8 @@ Status SchemaChangeHandler::schema_version_convert(TabletSharedPtr base_tablet,
                                                    TabletSharedPtr new_tablet,
                                                    RowsetSharedPtr* base_rowset,
                                                    RowsetSharedPtr* new_rowset,
-                                                   DescriptorTbl desc_tbl) {
+                                                   DescriptorTbl desc_tbl,
+                                                   const TabletSchema* base_schema_change) {
     Status res = Status::OK();
     LOG(INFO) << "begin to convert delta version for schema changing. "
               << "base_tablet=" << base_tablet->full_name()
@@ -2039,7 +2056,7 @@ Status SchemaChangeHandler::schema_version_convert(TabletSharedPtr base_tablet,
 
     const std::unordered_map<std::string, AlterMaterializedViewParam> materialized_function_map;
     if (res = _parse_request(base_tablet, new_tablet, &rb_changer, &sc_sorting, &sc_directly,
-                             materialized_function_map, desc_tbl);
+                             materialized_function_map, desc_tbl, base_schema_change);
         !res) {
         LOG(WARNING) << "failed to parse the request. res=" << res;
         return res;
@@ -2053,7 +2070,7 @@ Status SchemaChangeHandler::schema_version_convert(TabletSharedPtr base_tablet,
     // c. Convert data
     DeleteHandler delete_handler;
     std::vector<ColumnId> return_columns;
-    size_t num_cols = base_tablet->tablet_schema().num_columns();
+    size_t num_cols = base_schema_change->num_columns();
     return_columns.resize(num_cols);
     for (int i = 0; i < num_cols; ++i) {
         return_columns[i] = i;
@@ -2061,7 +2078,7 @@ Status SchemaChangeHandler::schema_version_convert(TabletSharedPtr base_tablet,
 
     RowsetReaderContext reader_context;
     reader_context.reader_type = READER_ALTER_TABLE;
-    reader_context.tablet_schema = &base_tablet->tablet_schema();
+    reader_context.tablet_schema = base_schema_change;
     reader_context.need_ordered_result = true;
     reader_context.delete_handler = &delete_handler;
     reader_context.return_columns = &return_columns;
@@ -2077,9 +2094,10 @@ Status SchemaChangeHandler::schema_version_convert(TabletSharedPtr base_tablet,
     load_id.set_hi((*base_rowset)->load_id().hi());
     load_id.set_lo((*base_rowset)->load_id().lo());
     std::unique_ptr<RowsetWriter> rowset_writer;
-    RETURN_NOT_OK(new_tablet->create_rowset_writer(
-            (*base_rowset)->txn_id(), load_id, PREPARED,
-            (*base_rowset)->rowset_meta()->segments_overlap(), &rowset_writer));
+    RETURN_NOT_OK(
+            new_tablet->create_rowset_writer((*base_rowset)->txn_id(), load_id, PREPARED,
+                                             (*base_rowset)->rowset_meta()->segments_overlap(),
+                                             reader_context.tablet_schema, &rowset_writer));
 
     auto schema_version_convert_error = [&]() -> Status {
         if (*new_rowset != nullptr) {
@@ -2160,9 +2178,9 @@ Status SchemaChangeHandler::_convert_historical_rowsets(const SchemaChangeParams
     bool sc_directly = false;
 
     // a.Parse the Alter request and convert it into an internal representation
-    Status res =
-            _parse_request(sc_params.base_tablet, sc_params.new_tablet, &rb_changer, &sc_sorting,
-                           &sc_directly, sc_params.materialized_params_map, *sc_params.desc_tbl);
+    Status res = _parse_request(sc_params.base_tablet, sc_params.new_tablet, &rb_changer,
+                                &sc_sorting, &sc_directly, sc_params.materialized_params_map,
+                                *sc_params.desc_tbl, sc_params.base_tablet_schema);
 
     auto process_alter_exit = [&]() -> Status {
         {
@@ -2204,8 +2222,8 @@ Status SchemaChangeHandler::_convert_historical_rowsets(const SchemaChangeParams
         Status status = new_tablet->create_rowset_writer(
                 rs_reader->version(), VISIBLE,
                 rs_reader->rowset()->rowset_meta()->segments_overlap(),
-                rs_reader->oldest_write_timestamp(), rs_reader->newest_write_timestamp(),
-                &rowset_writer);
+                &new_tablet->tablet_schema(), rs_reader->oldest_write_timestamp(),
+                rs_reader->newest_write_timestamp(), &rowset_writer);
         if (!status.ok()) {
             res = Status::OLAPInternalError(OLAP_ERR_ROWSET_BUILDER_INIT);
             return process_alter_exit();
@@ -2267,7 +2285,7 @@ Status SchemaChangeHandler::_parse_request(
         bool* sc_sorting, bool* sc_directly,
         const std::unordered_map<std::string, AlterMaterializedViewParam>&
                 materialized_function_map,
-        DescriptorTbl desc_tbl) {
+        DescriptorTbl desc_tbl, const TabletSchema* base_tablet_schema) {
     // set column mapping
     for (int i = 0, new_schema_size = new_tablet->tablet_schema().num_columns();
          i < new_schema_size; ++i) {
@@ -2276,7 +2294,7 @@ Status SchemaChangeHandler::_parse_request(
         ColumnMapping* column_mapping = rb_changer->get_mutable_column_mapping(i);
 
         if (new_column.has_reference_column()) {
-            int32_t column_index = base_tablet->field_index(new_column.referenced_column());
+            int32_t column_index = base_tablet_schema->field_index(new_column.referenced_column());
 
             if (column_index < 0) {
                 LOG(WARNING) << "referenced column was missing. "
@@ -2295,7 +2313,7 @@ Status SchemaChangeHandler::_parse_request(
             auto mvParam = materialized_function_map.find(column_name)->second;
             column_mapping->materialized_function = mvParam.mv_expr;
             column_mapping->expr = mvParam.expr;
-            int32_t column_index = base_tablet->field_index(mvParam.origin_column_name);
+            int32_t column_index = base_tablet_schema->field_index(mvParam.origin_column_name);
             if (column_index >= 0) {
                 column_mapping->ref_column = column_index;
                 continue;
@@ -2307,7 +2325,7 @@ Status SchemaChangeHandler::_parse_request(
             }
         }
 
-        int32_t column_index = base_tablet->field_index(column_name);
+        int32_t column_index = base_tablet_schema->field_index(column_name);
         if (column_index >= 0) {
             column_mapping->ref_column = column_index;
             continue;
@@ -2316,7 +2334,7 @@ Status SchemaChangeHandler::_parse_request(
         // Newly added column go here
         column_mapping->ref_column = -1;
 
-        if (i < base_tablet->num_short_key_columns()) {
+        if (i < base_tablet_schema->num_short_key_columns()) {
             *sc_directly = true;
         }
         RETURN_IF_ERROR(
@@ -2345,7 +2363,7 @@ Status SchemaChangeHandler::_parse_request(
         }
     }
 
-    const TabletSchema& ref_tablet_schema = base_tablet->tablet_schema();
+    const TabletSchema& ref_tablet_schema = *base_tablet_schema;
     const TabletSchema& new_tablet_schema = new_tablet->tablet_schema();
     if (ref_tablet_schema.keys_type() != new_tablet_schema.keys_type()) {
         // only when base table is dup and mv is agg
@@ -2362,7 +2380,7 @@ Status SchemaChangeHandler::_parse_request(
     //      old keys:    A   B   C   D
     //      new keys:    A   B
     if (new_tablet_schema.keys_type() != KeysType::DUP_KEYS &&
-        new_tablet->num_key_columns() < base_tablet->num_key_columns()) {
+        new_tablet->num_key_columns() < base_tablet_schema->num_key_columns()) {
         // this is a table with aggregate key type, and num of key columns in new schema
         // is less, which means the data in new tablet should be more aggregated.
         // so we use sorting schema change to sort and merge the data.
@@ -2370,7 +2388,7 @@ Status SchemaChangeHandler::_parse_request(
         return Status::OK();
     }
 
-    if (base_tablet->num_short_key_columns() != new_tablet->num_short_key_columns()) {
+    if (base_tablet_schema->num_short_key_columns() != new_tablet->num_short_key_columns()) {
         // the number of short_keys changed, can't do linked schema change
         *sc_directly = true;
         return Status::OK();
diff --git a/be/src/olap/schema_change.h b/be/src/olap/schema_change.h
index 4e186cae15..42909d459a 100644
--- a/be/src/olap/schema_change.h
+++ b/be/src/olap/schema_change.h
@@ -251,7 +251,8 @@ class SchemaChangeHandler {
 public:
     static Status schema_version_convert(TabletSharedPtr base_tablet, TabletSharedPtr new_tablet,
                                          RowsetSharedPtr* base_rowset, RowsetSharedPtr* new_rowset,
-                                         DescriptorTbl desc_tbl);
+                                         DescriptorTbl desc_tbl,
+                                         const TabletSchema* base_schema_change);
 
     // schema change v2, it will not set alter task in base tablet
     static Status process_alter_tablet_v2(const TAlterTabletReqV2& request);
@@ -305,6 +306,7 @@ private:
         AlterTabletType alter_tablet_type;
         TabletSharedPtr base_tablet;
         TabletSharedPtr new_tablet;
+        TabletSchema* base_tablet_schema = nullptr;
         std::vector<RowsetReaderSharedPtr> ref_rowset_readers;
         DeleteHandler* delete_handler = nullptr;
         std::unordered_map<std::string, AlterMaterializedViewParam> materialized_params_map;
@@ -323,7 +325,7 @@ private:
                                  RowBlockChanger* rb_changer, bool* sc_sorting, bool* sc_directly,
                                  const std::unordered_map<std::string, AlterMaterializedViewParam>&
                                          materialized_function_map,
-                                 DescriptorTbl desc_tbl);
+                                 DescriptorTbl desc_tbl, const TabletSchema* base_tablet_schema);
 
     // Initialization Settings for creating a default value
     static Status _init_column_mapping(ColumnMapping* column_mapping,
diff --git a/be/src/olap/segment_loader.cpp b/be/src/olap/segment_loader.cpp
index a105336f6a..6741fcf5a3 100644
--- a/be/src/olap/segment_loader.cpp
+++ b/be/src/olap/segment_loader.cpp
@@ -18,6 +18,7 @@
 #include "olap/segment_loader.h"
 
 #include "olap/rowset/rowset.h"
+#include "olap/tablet_schema.h"
 #include "util/stopwatch.hpp"
 
 namespace doris {
@@ -58,16 +59,17 @@ void SegmentLoader::_insert(const SegmentLoader::CacheKey& key, SegmentLoader::C
 }
 
 Status SegmentLoader::load_segments(const BetaRowsetSharedPtr& rowset,
-                                    SegmentCacheHandle* cache_handle, bool use_cache) {
-    SegmentLoader::CacheKey cache_key(rowset->rowset_id());
-    if (_lookup(cache_key, cache_handle)) {
+                                    SegmentCacheHandle* cache_handle,
+                                    const TabletSchema* read_tablet_schema, bool use_cache) {
+    SegmentLoader::CacheKey cache_key(rowset->rowset_id(), *read_tablet_schema);
+    if (use_cache && _lookup(cache_key, cache_handle)) {
         cache_handle->owned = false;
         return Status::OK();
     }
     cache_handle->owned = !use_cache;
 
     std::vector<segment_v2::SegmentSharedPtr> segments;
-    RETURN_NOT_OK(rowset->load_segments(&segments));
+    RETURN_NOT_OK(rowset->load_segments(&segments, read_tablet_schema));
 
     if (use_cache) {
         // memory of SegmentLoader::CacheValue will be handled by SegmentLoader
diff --git a/be/src/olap/segment_loader.h b/be/src/olap/segment_loader.h
index 8541ae6ed3..2932314285 100644
--- a/be/src/olap/segment_loader.h
+++ b/be/src/olap/segment_loader.h
@@ -25,6 +25,7 @@
 #include "olap/lru_cache.h"
 #include "olap/olap_common.h" // for rowset id
 #include "olap/rowset/beta_rowset.h"
+#include "olap/tablet_schema.h"
 #include "util/time.h"
 
 namespace doris {
@@ -48,11 +49,17 @@ class SegmentLoader {
 public:
     // The cache key or segment lru cache
     struct CacheKey {
-        CacheKey(RowsetId rowset_id_) : rowset_id(rowset_id_) {}
+        CacheKey(RowsetId rowset_id_, const TabletSchema& tablet_schema)
+                : rowset_id(rowset_id_), tablet_schema(tablet_schema) {}
         RowsetId rowset_id;
+        TabletSchema tablet_schema;
 
         // Encode to a flat binary which can be used as LRUCache's key
-        std::string encode() const { return rowset_id.to_string(); }
+        std::string encode() const {
+            TabletSchemaPB tablet_schema_pb;
+            tablet_schema.to_schema_pb(&tablet_schema_pb);
+            return rowset_id.to_string() + tablet_schema_pb.SerializeAsString();
+        }
     };
 
     // The cache value of segment lru cache.
@@ -82,7 +89,7 @@ public:
     // Load segments of "rowset", return the "cache_handle" which contains segments.
     // If use_cache is true, it will be loaded from _cache.
     Status load_segments(const BetaRowsetSharedPtr& rowset, SegmentCacheHandle* cache_handle,
-                         bool use_cache = false);
+                         const TabletSchema* read_tablet_schema, bool use_cache = false);
 
     // Try to prune the segment cache if expired.
     Status prune();
diff --git a/be/src/olap/snapshot_manager.cpp b/be/src/olap/snapshot_manager.cpp
index 3765008a40..436769bc3a 100644
--- a/be/src/olap/snapshot_manager.cpp
+++ b/be/src/olap/snapshot_manager.cpp
@@ -232,7 +232,8 @@ Status SnapshotManager::_rename_rowset_id(const RowsetMetaPB& rs_meta_pb,
     context.tablet_schema_hash = org_rowset_meta->tablet_schema_hash();
     context.rowset_type = org_rowset_meta->rowset_type();
     context.tablet_path = new_tablet_path;
-    context.tablet_schema = &tablet_schema;
+    context.tablet_schema =
+            org_rowset_meta->tablet_schema() ? org_rowset_meta->tablet_schema() : &tablet_schema;
     context.rowset_state = org_rowset_meta->rowset_state();
     context.version = org_rowset_meta->version();
     context.oldest_write_timestamp = org_rowset_meta->oldest_write_timestamp();
diff --git a/be/src/olap/tablet.cpp b/be/src/olap/tablet.cpp
index 49c07adc0c..17188991da 100644
--- a/be/src/olap/tablet.cpp
+++ b/be/src/olap/tablet.cpp
@@ -375,6 +375,15 @@ const RowsetSharedPtr Tablet::rowset_with_max_version() const {
     return iter->second;
 }
 
+const RowsetMetaSharedPtr Tablet::rowset_meta_with_max_schema_version(
+        const std::vector<RowsetMetaSharedPtr>& rowset_metas) {
+    return *std::max_element(rowset_metas.begin(), rowset_metas.end(),
+                             [](const RowsetMetaSharedPtr& a, const RowsetMetaSharedPtr& b) {
+                                 return a->tablet_schema()->schema_version() <
+                                        b->tablet_schema()->schema_version();
+                             });
+}
+
 RowsetSharedPtr Tablet::_rowset_with_largest_size() {
     RowsetSharedPtr largest_rowset = nullptr;
     for (auto& it : _rs_version_map) {
@@ -1553,7 +1562,9 @@ Status Tablet::create_initial_rowset(const int64_t req_version) {
     do {
         // there is no data in init rowset, so overlapping info is unknown.
         std::unique_ptr<RowsetWriter> rs_writer;
-        res = create_rowset_writer(version, VISIBLE, OVERLAP_UNKNOWN, -1, -1, &rs_writer);
+        res = create_rowset_writer(version, VISIBLE, OVERLAP_UNKNOWN,
+                                   &_tablet_meta->tablet_schema(), -1, -1, &rs_writer);
+
         if (!res.ok()) {
             LOG(WARNING) << "failed to init rowset writer for tablet " << full_name();
             break;
@@ -1584,6 +1595,7 @@ Status Tablet::create_initial_rowset(const int64_t req_version) {
 
 Status Tablet::create_rowset_writer(const Version& version, const RowsetStatePB& rowset_state,
                                     const SegmentsOverlapPB& overlap,
+                                    const doris::TabletSchema* tablet_schema,
                                     int64_t oldest_write_timestamp, int64_t newest_write_timestamp,
                                     std::unique_ptr<RowsetWriter>* rowset_writer) {
     RowsetWriterContext context;
@@ -1592,6 +1604,7 @@ Status Tablet::create_rowset_writer(const Version& version, const RowsetStatePB&
     context.segments_overlap = overlap;
     context.oldest_write_timestamp = oldest_write_timestamp;
     context.newest_write_timestamp = newest_write_timestamp;
+    context.tablet_schema = tablet_schema;
     _init_context_common_fields(context);
     return RowsetFactory::create_rowset_writer(context, rowset_writer);
 }
@@ -1599,6 +1612,7 @@ Status Tablet::create_rowset_writer(const Version& version, const RowsetStatePB&
 Status Tablet::create_rowset_writer(const int64_t& txn_id, const PUniqueId& load_id,
                                     const RowsetStatePB& rowset_state,
                                     const SegmentsOverlapPB& overlap,
+                                    const doris::TabletSchema* tablet_schema,
                                     std::unique_ptr<RowsetWriter>* rowset_writer) {
     RowsetWriterContext context;
     context.txn_id = txn_id;
@@ -1607,6 +1621,7 @@ Status Tablet::create_rowset_writer(const int64_t& txn_id, const PUniqueId& load
     context.segments_overlap = overlap;
     context.oldest_write_timestamp = -1;
     context.newest_write_timestamp = -1;
+    context.tablet_schema = tablet_schema;
     _init_context_common_fields(context);
     return RowsetFactory::create_rowset_writer(context, rowset_writer);
 }
@@ -1625,7 +1640,6 @@ void Tablet::_init_context_common_fields(RowsetWriterContext& context) {
         context.rowset_type = StorageEngine::instance()->default_rowset_type();
     }
     context.tablet_path = tablet_path();
-    context.tablet_schema = &(tablet_schema());
     context.data_dir = data_dir();
 }
 
@@ -1797,4 +1811,15 @@ void Tablet::remove_all_remote_rowsets() {
     }
 }
 
+const TabletSchema& Tablet::tablet_schema() const {
+    std::shared_lock wrlock(_meta_lock);
+    _tablet_meta->all_rs_metas();
+    const RowsetMetaSharedPtr rowset_meta =
+            rowset_meta_with_max_schema_version(_tablet_meta->all_rs_metas());
+    if (rowset_meta->tablet_schema() == nullptr) {
+        return _schema;
+    }
+    return *rowset_meta->tablet_schema();
+}
+
 } // namespace doris
diff --git a/be/src/olap/tablet.h b/be/src/olap/tablet.h
index 3da956cb63..83f5e3d2e2 100644
--- a/be/src/olap/tablet.h
+++ b/be/src/olap/tablet.h
@@ -120,6 +120,9 @@ public:
 
     const RowsetSharedPtr rowset_with_max_version() const;
 
+    static const RowsetMetaSharedPtr rowset_meta_with_max_schema_version(
+            const std::vector<RowsetMetaSharedPtr>& rowset_metas);
+
     Status add_inc_rowset(const RowsetSharedPtr& rowset);
     /// Delete stale rowset by timing. This delete policy uses now() minutes
     /// config::tablet_rowset_expired_stale_sweep_time_sec to compute the deadline of expired rowset
@@ -274,13 +277,16 @@ public:
         return _tablet_meta->all_beta();
     }
 
+    const TabletSchema& tablet_schema() const override;
+
     Status create_rowset_writer(const Version& version, const RowsetStatePB& rowset_state,
-                                const SegmentsOverlapPB& overlap, int64_t oldest_write_timestamp,
-                                int64_t newest_write_timestamp,
+                                const SegmentsOverlapPB& overlap, const TabletSchema* tablet_schema,
+                                int64_t oldest_write_timestamp, int64_t newest_write_timestamp,
                                 std::unique_ptr<RowsetWriter>* rowset_writer);
 
     Status create_rowset_writer(const int64_t& txn_id, const PUniqueId& load_id,
                                 const RowsetStatePB& rowset_state, const SegmentsOverlapPB& overlap,
+                                const TabletSchema* tablet_schema,
                                 std::unique_ptr<RowsetWriter>* rowset_writer);
 
     Status create_rowset(RowsetMetaSharedPtr rowset_meta, RowsetSharedPtr* rowset);
diff --git a/be/src/olap/tablet_meta.cpp b/be/src/olap/tablet_meta.cpp
index 3a0490c370..5c1be52076 100644
--- a/be/src/olap/tablet_meta.cpp
+++ b/be/src/olap/tablet_meta.cpp
@@ -139,8 +139,13 @@ TabletMeta::TabletMeta(int64_t table_id, int64_t partition_id, int64_t tablet_id
     bool has_bf_columns = false;
     for (TColumn tcolumn : tablet_schema.columns) {
         ColumnPB* column = schema->add_column();
-        uint32_t unique_id = col_ordinal_to_unique_id.at(col_ordinal++);
-        _init_column_from_tcolumn(unique_id, tcolumn, column);
+        uint32_t unique_id = -1;
+        if (tcolumn.col_unique_id >= 0) {
+            unique_id = tcolumn.col_unique_id;
+        } else {
+            unique_id = col_ordinal_to_unique_id.at(col_ordinal++);
+        }
+        init_column_from_tcolumn(unique_id, tcolumn, column);
 
         if (column->is_key()) {
             ++key_count;
@@ -200,11 +205,12 @@ TabletMeta::TabletMeta(const TabletMeta& b)
           _storage_medium(b._storage_medium),
           _cooldown_resource(b._cooldown_resource) {};
 
-void TabletMeta::_init_column_from_tcolumn(uint32_t unique_id, const TColumn& tcolumn,
-                                           ColumnPB* column) {
+void TabletMeta::init_column_from_tcolumn(uint32_t unique_id, const TColumn& tcolumn,
+                                          ColumnPB* column) {
     column->set_unique_id(unique_id);
+    column->set_col_unique_id(tcolumn.col_unique_id);
     column->set_name(tcolumn.column_name);
-    column->set_has_bitmap_index(false);
+    column->set_has_bitmap_index(tcolumn.has_bitmap_index);
     string data_type;
     EnumToString(TPrimitiveType, tcolumn.column_type.type, data_type);
     column->set_type(data_type);
@@ -243,7 +249,7 @@ void TabletMeta::_init_column_from_tcolumn(uint32_t unique_id, const TColumn& tc
     }
     if (tcolumn.column_type.type == TPrimitiveType::ARRAY) {
         ColumnPB* children_column = column->add_children_columns();
-        _init_column_from_tcolumn(0, tcolumn.children_column[0], children_column);
+        init_column_from_tcolumn(0, tcolumn.children_column[0], children_column);
     }
 }
 
diff --git a/be/src/olap/tablet_meta.h b/be/src/olap/tablet_meta.h
index 6e558d996a..26163a15e4 100644
--- a/be/src/olap/tablet_meta.h
+++ b/be/src/olap/tablet_meta.h
@@ -149,6 +149,7 @@ public:
     TabletSchema* mutable_tablet_schema();
 
     const std::vector<RowsetMetaSharedPtr>& all_rs_metas() const;
+    std::vector<RowsetMetaSharedPtr>& all_mutable_rs_metas();
     Status add_rs_meta(const RowsetMetaSharedPtr& rs_meta);
     void delete_rs_meta_by_version(const Version& version,
                                    std::vector<RowsetMetaSharedPtr>* deleted_rs_metas);
@@ -199,12 +200,13 @@ public:
                     << _cooldown_resource << " to " << resource;
         _cooldown_resource = std::move(resource);
     }
+    static void init_column_from_tcolumn(uint32_t unique_id, const TColumn& tcolumn,
+                                         ColumnPB* column);
 
     DeleteBitmap& delete_bitmap() { return *_delete_bitmap; }
 
 private:
     Status _save_meta(DataDir* data_dir);
-    void _init_column_from_tcolumn(uint32_t unique_id, const TColumn& tcolumn, ColumnPB* column);
 
     // _del_pred_array is ignored to compare.
     friend bool operator==(const TabletMeta& a, const TabletMeta& b);
@@ -473,6 +475,10 @@ inline const std::vector<RowsetMetaSharedPtr>& TabletMeta::all_rs_metas() const
     return _rs_metas;
 }
 
+inline std::vector<RowsetMetaSharedPtr>& TabletMeta::all_mutable_rs_metas() {
+    return _rs_metas;
+}
+
 inline const std::vector<RowsetMetaSharedPtr>& TabletMeta::all_stale_rs_metas() const {
     return _stale_rs_metas;
 }
diff --git a/be/src/olap/tablet_schema.cpp b/be/src/olap/tablet_schema.cpp
index cda1370b7f..a8659fc1e3 100644
--- a/be/src/olap/tablet_schema.cpp
+++ b/be/src/olap/tablet_schema.cpp
@@ -17,6 +17,7 @@
 
 #include "olap/tablet_schema.h"
 
+#include "gen_cpp/descriptors.pb.h"
 #include "tablet_meta.h"
 #include "vec/aggregate_functions/aggregate_function_reader.h"
 #include "vec/aggregate_functions/aggregate_function_simple_factory.h"
@@ -307,8 +308,24 @@ TabletColumn::TabletColumn(FieldAggregationMethod agg, FieldType filed_type, boo
     _length = length;
 }
 
+TabletColumn::TabletColumn(const ColumnPB& column) {
+    init_from_pb(column);
+}
+
+TabletColumn::TabletColumn(const TColumn& column) {
+    init_from_thrift(column);
+}
+
+void TabletColumn::init_from_thrift(const TColumn& tcolumn) {
+    _unique_id = tcolumn.col_unique_id;
+    ColumnPB column_pb;
+    TabletMeta::init_column_from_tcolumn(_unique_id, tcolumn, &column_pb);
+    init_from_pb(column_pb);
+}
+
 void TabletColumn::init_from_pb(const ColumnPB& column) {
     _unique_id = column.unique_id();
+    _col_unique_id = column.col_unique_id();
     _col_name = column.name();
     _type = TabletColumn::get_field_type_by_string(column.type());
     _is_key = column.is_key();
@@ -358,9 +375,10 @@ void TabletColumn::init_from_pb(const ColumnPB& column) {
     }
 }
 
-void TabletColumn::to_schema_pb(ColumnPB* column) {
+void TabletColumn::to_schema_pb(ColumnPB* column) const {
     column->set_unique_id(_unique_id);
     column->set_name(_col_name);
+    column->set_col_unique_id(_col_unique_id);
     column->set_type(get_string_by_field_type(_type));
     column->set_is_key(_is_key);
     column->set_is_nullable(_is_nullable);
@@ -423,6 +441,28 @@ vectorized::AggregateFunctionPtr TabletColumn::get_aggregate_function(
             agg_name, argument_types, {}, argument_types.back()->is_nullable());
 }
 
+void TabletSchema::append_column(TabletColumn column) {
+    if (column.is_key()) {
+        _num_key_columns++;
+    }
+    if (column.is_nullable()) {
+        _num_null_columns++;
+    }
+    _field_name_to_index[column.name()] = _num_columns;
+    _field_id_to_index[column.col_unique_id()] = _num_columns;
+    _cols.push_back(std::move(column));
+    _num_columns++;
+}
+
+void TabletSchema::clear_columns() {
+    _field_name_to_index.clear();
+    _field_id_to_index.clear();
+    _num_columns = 0;
+    _num_null_columns = 0;
+    _num_key_columns = 0;
+    _cols.clear();
+}
+
 void TabletSchema::init_from_pb(const TabletSchemaPB& schema) {
     _keys_type = schema.keys_type();
     _num_columns = 0;
@@ -440,6 +480,9 @@ void TabletSchema::init_from_pb(const TabletSchemaPB& schema) {
             _num_null_columns++;
         }
         _field_name_to_index[column.name()] = _num_columns;
+        if (column.col_unique_id() >= 0) {
+            _field_id_to_index[column.col_unique_id()] = _num_columns;
+        }
         _cols.emplace_back(std::move(column));
         _num_columns++;
     }
@@ -460,27 +503,87 @@ void TabletSchema::init_from_pb(const TabletSchemaPB& schema) {
     _sort_type = schema.sort_type();
     _sort_col_num = schema.sort_col_num();
     _compression_type = schema.compression_type();
+    _schema_version = schema.schema_version();
 }
 
-void TabletSchema::to_schema_pb(TabletSchemaPB* tablet_meta_pb) {
-    tablet_meta_pb->set_keys_type(_keys_type);
+void TabletSchema::build_current_tablet_schema(int64_t index_id,
+                                               const POlapTableSchemaParam& ptable_schema_param,
+                                               const TabletSchema& ori_tablet_schema) {
+    // copy from ori_tablet_schema
+    _keys_type = ori_tablet_schema.keys_type();
+    _num_short_key_columns = ori_tablet_schema.num_short_key_columns();
+    _num_rows_per_row_block = ori_tablet_schema.num_rows_per_row_block();
+    _compress_kind = ori_tablet_schema.compress_kind();
+
+    // todo(yixiu): unique_id
+    _next_column_unique_id = ori_tablet_schema.next_column_unique_id();
+    _is_in_memory = ori_tablet_schema.is_in_memory();
+    _delete_sign_idx = ori_tablet_schema.delete_sign_idx();
+    _sequence_col_idx = ori_tablet_schema.sequence_col_idx();
+    _sort_type = ori_tablet_schema.sort_type();
+    _sort_col_num = ori_tablet_schema.sort_col_num();
+
+    // copy from table_schema_param
+    _num_columns = 0;
+    _num_key_columns = 0;
+    _num_null_columns = 0;
+    bool has_bf_columns = false;
+    _cols.clear();
+    _field_name_to_index.clear();
+    _field_id_to_index.clear();
+
+    for (const POlapTableIndexSchema& index : ptable_schema_param.indexes()) {
+        if (index.id() == index_id) {
+            for (auto& pcolumn : index.columns_desc()) {
+                TabletColumn column;
+                column.init_from_pb(pcolumn);
+                if (column.is_key()) {
+                    _num_key_columns++;
+                }
+                if (column.is_nullable()) {
+                    _num_null_columns++;
+                }
+                if (column.is_bf_column()) {
+                    has_bf_columns = true;
+                }
+                _field_name_to_index[column.name()] = _num_columns;
+                _field_id_to_index[column.col_unique_id()] = _num_columns;
+                _cols.emplace_back(std::move(column));
+                _num_columns++;
+            }
+            break;
+        }
+    }
+    if (has_bf_columns) {
+        _has_bf_fpp = true;
+        _bf_fpp = ori_tablet_schema.bloom_filter_fpp();
+    } else {
+        _has_bf_fpp = false;
+        _bf_fpp = BLOOM_FILTER_DEFAULT_FPP;
+    }
+    _schema_version = ptable_schema_param.version();
+}
+
+void TabletSchema::to_schema_pb(TabletSchemaPB* tablet_schema_pb) const {
+    tablet_schema_pb->set_keys_type(_keys_type);
     for (auto& col : _cols) {
-        ColumnPB* column = tablet_meta_pb->add_column();
+        ColumnPB* column = tablet_schema_pb->add_column();
         col.to_schema_pb(column);
     }
-    tablet_meta_pb->set_num_short_key_columns(_num_short_key_columns);
-    tablet_meta_pb->set_num_rows_per_row_block(_num_rows_per_row_block);
-    tablet_meta_pb->set_compress_kind(_compress_kind);
+    tablet_schema_pb->set_num_short_key_columns(_num_short_key_columns);
+    tablet_schema_pb->set_num_rows_per_row_block(_num_rows_per_row_block);
+    tablet_schema_pb->set_compress_kind(_compress_kind);
     if (_has_bf_fpp) {
-        tablet_meta_pb->set_bf_fpp(_bf_fpp);
-    }
-    tablet_meta_pb->set_next_column_unique_id(_next_column_unique_id);
-    tablet_meta_pb->set_is_in_memory(_is_in_memory);
-    tablet_meta_pb->set_delete_sign_idx(_delete_sign_idx);
-    tablet_meta_pb->set_sequence_col_idx(_sequence_col_idx);
-    tablet_meta_pb->set_sort_type(_sort_type);
-    tablet_meta_pb->set_sort_col_num(_sort_col_num);
-    tablet_meta_pb->set_compression_type(_compression_type);
+        tablet_schema_pb->set_bf_fpp(_bf_fpp);
+    }
+    tablet_schema_pb->set_next_column_unique_id(_next_column_unique_id);
+    tablet_schema_pb->set_is_in_memory(_is_in_memory);
+    tablet_schema_pb->set_delete_sign_idx(_delete_sign_idx);
+    tablet_schema_pb->set_sequence_col_idx(_sequence_col_idx);
+    tablet_schema_pb->set_sort_type(_sort_type);
+    tablet_schema_pb->set_sort_col_num(_sort_col_num);
+    tablet_schema_pb->set_schema_version(_schema_version);
+    tablet_schema_pb->set_compression_type(_compression_type);
 }
 
 uint32_t TabletSchema::mem_size() const {
@@ -511,6 +614,11 @@ int32_t TabletSchema::field_index(const std::string& field_name) const {
     return (found == _field_name_to_index.end()) ? -1 : found->second;
 }
 
+int32_t TabletSchema::field_index(int32_t col_unique_id) const {
+    const auto& found = _field_id_to_index.find(col_unique_id);
+    return (found == _field_id_to_index.end()) ? -1 : found->second;
+}
+
 const std::vector<TabletColumn>& TabletSchema::columns() const {
     return _cols;
 }
diff --git a/be/src/olap/tablet_schema.h b/be/src/olap/tablet_schema.h
index f0a5b72157..1ae27b8cb3 100644
--- a/be/src/olap/tablet_schema.h
+++ b/be/src/olap/tablet_schema.h
@@ -31,18 +31,24 @@ namespace vectorized {
 class Block;
 }
 
+class POlapTableSchemaParam;
+
 class TabletColumn {
 public:
     TabletColumn();
+    TabletColumn(const ColumnPB& column);
+    TabletColumn(const TColumn& column);
     TabletColumn(FieldAggregationMethod agg, FieldType type);
     TabletColumn(FieldAggregationMethod agg, FieldType filed_type, bool is_nullable);
     TabletColumn(FieldAggregationMethod agg, FieldType filed_type, bool is_nullable,
                  int32_t unique_id, size_t length);
     void init_from_pb(const ColumnPB& column);
-    void to_schema_pb(ColumnPB* column);
+    void init_from_thrift(const TColumn& column);
+    void to_schema_pb(ColumnPB* column) const;
     uint32_t mem_size() const;
 
     int32_t unique_id() const { return _unique_id; }
+    int32_t col_unique_id() const { return _col_unique_id; }
     std::string name() const { return _col_name; }
     void set_name(std::string col_name) { _col_name = col_name; }
     FieldType type() const { return _type; }
@@ -114,6 +120,7 @@ private:
     TabletColumn* _parent = nullptr;
     std::vector<TabletColumn> _sub_columns;
     uint32_t _sub_column_count = 0;
+    int32_t _col_unique_id = -1;
 };
 
 bool operator==(const TabletColumn& a, const TabletColumn& b);
@@ -126,11 +133,13 @@ public:
     // void create_from_pb(const TabletSchemaPB& schema, TabletSchema* tablet_schema).
     TabletSchema() = default;
     void init_from_pb(const TabletSchemaPB& schema);
-    void to_schema_pb(TabletSchemaPB* tablet_meta_pb);
+    void to_schema_pb(TabletSchemaPB* tablet_meta_pb) const;
+    void append_column(TabletColumn column);
     uint32_t mem_size() const;
 
     size_t row_size() const;
     int32_t field_index(const std::string& field_name) const;
+    int32_t field_index(int32_t col_unique_id) const;
     const TabletColumn& column(size_t ordinal) const;
     const std::vector<TabletColumn>& columns() const;
     size_t num_columns() const { return _num_columns; }
@@ -143,6 +152,7 @@ public:
     size_t sort_col_num() const { return _sort_col_num; }
     CompressKind compress_kind() const { return _compress_kind; }
     size_t next_column_unique_id() const { return _next_column_unique_id; }
+    bool has_bf_fpp() const { return _has_bf_fpp; }
     double bloom_filter_fpp() const { return _bf_fpp; }
     bool is_in_memory() const { return _is_in_memory; }
     void set_is_in_memory(bool is_in_memory) { _is_in_memory = is_in_memory; }
@@ -152,11 +162,17 @@ public:
     int32_t sequence_col_idx() const { return _sequence_col_idx; }
     segment_v2::CompressionTypePB compression_type() const { return _compression_type; }
 
+    int32_t schema_version() const { return _schema_version; }
+    void clear_columns();
     vectorized::Block create_block(
             const std::vector<uint32_t>& return_columns,
             const std::unordered_set<uint32_t>* tablet_columns_need_convert_null = nullptr) const;
     vectorized::Block create_block() const;
 
+    void build_current_tablet_schema(int64_t index_id,
+                                     const POlapTableSchemaParam& ptable_schema_param,
+                                     const TabletSchema& out_tablet_schema);
+
 private:
     // Only for unit test.
     void init_field_index_for_test();
@@ -170,6 +186,7 @@ private:
     size_t _sort_col_num = 0;
     std::vector<TabletColumn> _cols;
     std::unordered_map<std::string, int32_t> _field_name_to_index;
+    std::unordered_map<int32_t, int32_t> _field_id_to_index;
     size_t _num_columns = 0;
     size_t _num_key_columns = 0;
     size_t _num_null_columns = 0;
@@ -184,6 +201,7 @@ private:
     bool _is_in_memory = false;
     int32_t _delete_sign_idx = -1;
     int32_t _sequence_col_idx = -1;
+    int32_t _schema_version = -1;
 };
 
 bool operator==(const TabletSchema& a, const TabletSchema& b);
diff --git a/be/src/runtime/descriptors.cpp b/be/src/runtime/descriptors.cpp
index 4b81fc2cfe..e035068ef2 100644
--- a/be/src/runtime/descriptors.cpp
+++ b/be/src/runtime/descriptors.cpp
@@ -55,6 +55,7 @@ SlotDescriptor::SlotDescriptor(const TSlotDescriptor& tdesc)
           _tuple_offset(tdesc.byteOffset),
           _null_indicator_offset(tdesc.nullIndicatorByte, tdesc.nullIndicatorBit),
           _col_name(tdesc.colName),
+          _col_unique_id(tdesc.col_unique_id),
           _slot_idx(tdesc.slotIdx),
           _slot_size(_type.get_slot_size()),
           _field_idx(-1),
@@ -68,6 +69,7 @@ SlotDescriptor::SlotDescriptor(const PSlotDescriptor& pdesc)
           _tuple_offset(pdesc.byte_offset()),
           _null_indicator_offset(pdesc.null_indicator_byte(), pdesc.null_indicator_bit()),
           _col_name(pdesc.col_name()),
+          _col_unique_id(-1),
           _slot_idx(pdesc.slot_idx()),
           _slot_size(_type.get_slot_size()),
           _field_idx(-1),
diff --git a/be/src/runtime/descriptors.h b/be/src/runtime/descriptors.h
index ee18f8a450..85f5bea18a 100644
--- a/be/src/runtime/descriptors.h
+++ b/be/src/runtime/descriptors.h
@@ -113,6 +113,8 @@ public:
 
     doris::vectorized::DataTypePtr get_data_type_ptr() const;
 
+    int32_t col_unique_id() const { return _col_unique_id; }
+
 private:
     friend class DescriptorTbl;
     friend class TupleDescriptor;
@@ -127,6 +129,8 @@ private:
     const NullIndicatorOffset _null_indicator_offset;
     const std::string _col_name;
 
+    const int32_t _col_unique_id;
+
     // the idx of the slot in the tuple descriptor (0-based).
     // this is provided by the FE
     const int _slot_idx;
diff --git a/be/src/runtime/tablets_channel.cpp b/be/src/runtime/tablets_channel.cpp
index 9733ff0bc7..2ee5ea2dcd 100644
--- a/be/src/runtime/tablets_channel.cpp
+++ b/be/src/runtime/tablets_channel.cpp
@@ -225,6 +225,7 @@ Status TabletsChannel::_open_all_writers(const PTabletWriterOpenRequest& request
     }
     for (auto& tablet : request.tablets()) {
         WriteRequest wrequest;
+        wrequest.index_id = request.index_id();
         wrequest.tablet_id = tablet.tablet_id();
         wrequest.schema_hash = schema_hash;
         wrequest.write_type = WriteType::LOAD;
@@ -234,6 +235,7 @@ Status TabletsChannel::_open_all_writers(const PTabletWriterOpenRequest& request
         wrequest.tuple_desc = _tuple_desc;
         wrequest.slots = index_slots;
         wrequest.is_high_priority = _is_high_priority;
+        wrequest.ptable_schema_param = request.schema();
 
         DeltaWriter* writer = nullptr;
         auto st = DeltaWriter::open(&wrequest, &writer, _is_vec);
diff --git a/be/src/vec/exec/volap_scanner.cpp b/be/src/vec/exec/volap_scanner.cpp
index 2b3093ce8b..cc063a5978 100644
--- a/be/src/vec/exec/volap_scanner.cpp
+++ b/be/src/vec/exec/volap_scanner.cpp
@@ -74,6 +74,18 @@ Status VOlapScanner::prepare(
             LOG(WARNING) << ss.str();
             return Status::InternalError(ss.str());
         }
+        _tablet_schema = _tablet->tablet_schema();
+        if (!_parent->_olap_scan_node.columns_desc.empty() &&
+            _parent->_olap_scan_node.columns_desc[0].col_unique_id >= 0) {
+            // Originally scanner get TabletSchema from tablet object in BE.
+            // To support lightweight schema change for adding / dropping columns,
+            // tabletschema is bounded to rowset and tablet's schema maybe outdated,
+            //  so we have to use schema from a query plan witch FE puts it in query plans.
+            _tablet_schema.clear_columns();
+            for (const auto& column_desc : _parent->_olap_scan_node.columns_desc) {
+                _tablet_schema.append_column(TabletColumn(column_desc));
+            }
+        }
         {
             std::shared_lock rdlock(_tablet->get_header_lock());
             const RowsetSharedPtr rowset = _tablet->rowset_with_max_version();
@@ -156,6 +168,7 @@ Status VOlapScanner::_init_tablet_reader_params(
     RETURN_IF_ERROR(_init_return_columns(!_tablet_reader_params.direct_mode));
 
     _tablet_reader_params.tablet = _tablet;
+    _tablet_reader_params.tablet_schema = &_tablet_schema;
     _tablet_reader_params.reader_type = READER_QUERY;
     _tablet_reader_params.aggregation = _aggregation;
     _tablet_reader_params.version = Version(0, _version);
@@ -192,11 +205,11 @@ Status VOlapScanner::_init_tablet_reader_params(
         _tablet_reader_params.return_columns = _return_columns;
     } else {
         // we need to fetch all key columns to do the right aggregation on storage engine side.
-        for (size_t i = 0; i < _tablet->num_key_columns(); ++i) {
+        for (size_t i = 0; i < _tablet_schema.num_key_columns(); ++i) {
             _tablet_reader_params.return_columns.push_back(i);
         }
         for (auto index : _return_columns) {
-            if (_tablet->tablet_schema().column(index).is_key()) {
+            if (_tablet_schema.column(index).is_key()) {
                 continue;
             } else {
                 _tablet_reader_params.return_columns.push_back(index);
@@ -221,7 +234,9 @@ Status VOlapScanner::_init_return_columns(bool need_seq_col) {
         if (!slot->is_materialized()) {
             continue;
         }
-        int32_t index = _tablet->field_index(slot->col_name());
+        int32_t index = slot->col_unique_id() >= 0
+                                ? _tablet_schema.field_index(slot->col_unique_id())
+                                : _tablet_schema.field_index(slot->col_name());
         if (index < 0) {
             std::stringstream ss;
             ss << "field name is invalid. field=" << slot->col_name();
@@ -229,21 +244,21 @@ Status VOlapScanner::_init_return_columns(bool need_seq_col) {
             return Status::InternalError(ss.str());
         }
         _return_columns.push_back(index);
-        if (slot->is_nullable() && !_tablet->tablet_schema().column(index).is_nullable())
+        if (slot->is_nullable() && !_tablet_schema.column(index).is_nullable())
             _tablet_columns_convert_to_null_set.emplace(index);
     }
 
     // expand the sequence column
-    if (_tablet->tablet_schema().has_sequence_col() && need_seq_col) {
+    if (_tablet_schema.has_sequence_col() && need_seq_col) {
         bool has_replace_col = false;
         for (auto col : _return_columns) {
-            if (_tablet->tablet_schema().column(col).aggregation() ==
+            if (_tablet_schema.column(col).aggregation() ==
                 FieldAggregationMethod::OLAP_FIELD_AGGREGATION_REPLACE) {
                 has_replace_col = true;
                 break;
             }
         }
-        if (auto sequence_col_idx = _tablet->tablet_schema().sequence_col_idx();
+        if (auto sequence_col_idx = _tablet_schema.sequence_col_idx();
             has_replace_col && std::find(_return_columns.begin(), _return_columns.end(),
                                          sequence_col_idx) == _return_columns.end()) {
             _return_columns.push_back(sequence_col_idx);
diff --git a/be/src/vec/exec/volap_scanner.h b/be/src/vec/exec/volap_scanner.h
index d4c40c13c1..6fd3ee89d9 100644
--- a/be/src/vec/exec/volap_scanner.h
+++ b/be/src/vec/exec/volap_scanner.h
@@ -143,6 +143,8 @@ private:
 
     VExprContext* _vconjunct_ctx = nullptr;
     bool _need_to_close = false;
+
+    TabletSchema _tablet_schema;
 };
 
 } // namespace vectorized
diff --git a/be/src/vec/olap/block_reader.cpp b/be/src/vec/olap/block_reader.cpp
index 681b6d4e55..ab187367f9 100644
--- a/be/src/vec/olap/block_reader.cpp
+++ b/be/src/vec/olap/block_reader.cpp
@@ -79,7 +79,7 @@ void BlockReader::_init_agg_state(const ReaderParams& read_params) {
     _stored_has_null_tag.resize(_stored_data_columns.size());
     _stored_has_string_tag.resize(_stored_data_columns.size());
 
-    auto& tablet_schema = tablet()->tablet_schema();
+    auto& tablet_schema = *_tablet_schema;
     for (auto idx : _agg_columns_idx) {
         AggregateFunctionPtr function =
                 tablet_schema
diff --git a/be/src/vec/olap/vcollect_iterator.cpp b/be/src/vec/olap/vcollect_iterator.cpp
index 2b68dc7a38..81eb20f3a2 100644
--- a/be/src/vec/olap/vcollect_iterator.cpp
+++ b/be/src/vec/olap/vcollect_iterator.cpp
@@ -55,7 +55,7 @@ Status VCollectIterator::add_child(RowsetReaderSharedPtr rs_reader) {
 // then merged with the base rowset.
 Status VCollectIterator::build_heap(std::vector<RowsetReaderSharedPtr>& rs_readers) {
     DCHECK(rs_readers.size() == _children.size());
-    _skip_same = _reader->_tablet->tablet_schema().keys_type() == KeysType::UNIQUE_KEYS;
+    _skip_same = _reader->_tablet_schema->keys_type() == KeysType::UNIQUE_KEYS;
     if (_children.empty()) {
         _inner_iter.reset(nullptr);
         return Status::OK();
diff --git a/be/src/vec/olap/vcollect_iterator.h b/be/src/vec/olap/vcollect_iterator.h
index 6cae36dcc3..4c8002d885 100644
--- a/be/src/vec/olap/vcollect_iterator.h
+++ b/be/src/vec/olap/vcollect_iterator.h
@@ -73,7 +73,7 @@ private:
     // then merged with other rowset readers.
     class LevelIterator {
     public:
-        LevelIterator(TabletReader* reader) : _schema(reader->tablet()->tablet_schema()) {};
+        LevelIterator(TabletReader* reader) : _schema(reader->tablet_schema()) {};
 
         virtual Status init() = 0;
 
diff --git a/be/test/olap/rowset/beta_rowset_test.cpp b/be/test/olap/rowset/beta_rowset_test.cpp
index a71b2497d5..01e994fb84 100644
--- a/be/test/olap/rowset/beta_rowset_test.cpp
+++ b/be/test/olap/rowset/beta_rowset_test.cpp
@@ -433,6 +433,7 @@ TEST_F(BetaRowsetTest, ReadTest) {
     Aws::SDKOptions aws_options = Aws::SDKOptions {};
     Aws::InitAPI(aws_options);
 
+    TabletSchema dummy_schema;
     // failed to head object
     {
         Aws::Auth::AWSCredentials aws_cred("ak", "sk");
@@ -446,7 +447,7 @@ TEST_F(BetaRowsetTest, ReadTest) {
         rowset.rowset_meta()->set_fs(fs);
 
         std::vector<segment_v2::SegmentSharedPtr> segments;
-        Status st = rowset.load_segments(&segments);
+        Status st = rowset.load_segments(&segments, &dummy_schema);
         ASSERT_FALSE(st.ok());
     }
 
@@ -461,7 +462,7 @@ TEST_F(BetaRowsetTest, ReadTest) {
         rowset.rowset_meta()->set_fs(fs);
 
         std::vector<segment_v2::SegmentSharedPtr> segments;
-        Status st = rowset.load_segments(&segments);
+        Status st = rowset.load_segments(&segments, &dummy_schema);
         ASSERT_FALSE(st.ok());
     }
 
@@ -476,7 +477,7 @@ TEST_F(BetaRowsetTest, ReadTest) {
         rowset.rowset_meta()->set_fs(fs);
 
         std::vector<segment_v2::SegmentSharedPtr> segments;
-        Status st = rowset.load_segments(&segments);
+        Status st = rowset.load_segments(&segments, &dummy_schema);
         ASSERT_FALSE(st.ok());
     }
 
diff --git a/be/test/olap/test_data/header_without_inc_rs.txt b/be/test/olap/test_data/header_without_inc_rs.txt
index 61fc42e419..040b37b244 100644
--- a/be/test/olap/test_data/header_without_inc_rs.txt
+++ b/be/test/olap/test_data/header_without_inc_rs.txt
@@ -19,7 +19,8 @@
                 "is_nullable": false,
                 "length": 8,
                 "index_length": 8,
-                "visible": true
+                "visible": true,
+                "col_unique_id": -1
             },
             {
                 "unique_id": 1,
@@ -31,7 +32,8 @@
                 "default_value": "MA==",
                 "length": 16387,
                 "index_length": 16,
-                "visible": true
+                "visible": true,
+                "col_unique_id": -1
             },
             {
                 "unique_id": 2,
@@ -42,7 +44,8 @@
                 "is_nullable": false,
                 "length": 4,
                 "index_length": 4,
-                "visible": true
+                "visible": true,
+                "col_unique_id": -1
             }
         ],
         "num_short_key_columns": 1,
@@ -54,7 +57,8 @@
         "sequence_col_idx": -1,
         "sort_type": "LEXICAL",
         "sort_col_num": 0,
-        "compression_type": "LZ4F"
+        "compression_type": "LZ4F",
+        "schema_version": 0
     },
     "rs_metas": [
         {
diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/FeMetaVersion.java b/fe/fe-common/src/main/java/org/apache/doris/common/FeMetaVersion.java
index 1a5d6da4ce..7ecc5eb1ff 100644
--- a/fe/fe-common/src/main/java/org/apache/doris/common/FeMetaVersion.java
+++ b/fe/fe-common/src/main/java/org/apache/doris/common/FeMetaVersion.java
@@ -42,8 +42,10 @@ public final class FeMetaVersion {
     public static final int VERSION_110 = 110;
     // add catalog PrivTable in PaloAuth to support unified privilege management
     public static final int VERSION_111 = 111;
+    // add row policy and add maxColUniqueId for olapTable
+    public static final int VERSION_112 = 112;
     // note: when increment meta version, should assign the latest version to VERSION_CURRENT
-    public static final int VERSION_CURRENT = VERSION_111;
+    public static final int VERSION_CURRENT = VERSION_112;
 
     // all logs meta version should >= the minimum version, so that we could remove many if clause, for example
     // if (FE_METAVERSION < VERSION_94) ...
diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java
index 89dce3e169..9cfe420f34 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/AlterJobV2.java
@@ -111,6 +111,10 @@ public abstract class AlterJobV2 implements Writable {
         return jobState;
     }
 
+    public void setJobState(JobState jobState) {
+        this.jobState = jobState;
+    }
+
     public JobType getType() {
         return type;
     }
@@ -143,6 +147,10 @@ public abstract class AlterJobV2 implements Writable {
         return finishedTimeMs;
     }
 
+    public void setFinishedTimeMs(long finishedTimeMs) {
+        this.finishedTimeMs = finishedTimeMs;
+    }
+
     /**
      * The keyword 'synchronized' only protects 2 methods:
      * run() and cancel()
diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java
index 4fe0c9854e..bcd73e37be 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java
@@ -383,7 +383,7 @@ public class RollupJobV2 extends AlterJobV2 implements GsonPostProcessable {
                         AlterReplicaTask rollupTask = new AlterReplicaTask(rollupReplica.getBackendId(), dbId, tableId,
                                 partitionId, rollupIndexId, baseIndexId, rollupTabletId, baseTabletId,
                                 rollupReplica.getId(), rollupSchemaHash, baseSchemaHash, visibleVersion, jobId,
-                                JobType.ROLLUP, defineExprs, descTable);
+                                JobType.ROLLUP, defineExprs, descTable, null);
                         rollupBatchTask.addTask(rollupTask);
                     }
                 }
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 574c908803..c5b5151882 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
@@ -53,6 +53,7 @@ import org.apache.doris.catalog.Replica;
 import org.apache.doris.catalog.Replica.ReplicaState;
 import org.apache.doris.catalog.ReplicaAllocation;
 import org.apache.doris.catalog.Table;
+import org.apache.doris.catalog.TableIf.TableType;
 import org.apache.doris.catalog.Tablet;
 import org.apache.doris.catalog.TabletMeta;
 import org.apache.doris.common.AnalysisException;
@@ -70,6 +71,7 @@ import org.apache.doris.common.util.PropertyAnalyzer;
 import org.apache.doris.common.util.Util;
 import org.apache.doris.mysql.privilege.PrivPredicate;
 import org.apache.doris.persist.RemoveAlterJobV2OperationLog;
+import org.apache.doris.persist.TableAddOrDropColumnsInfo;
 import org.apache.doris.qe.ConnectContext;
 import org.apache.doris.task.AgentBatchTask;
 import org.apache.doris.task.AgentTaskExecutor;
@@ -101,6 +103,7 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
+import java.util.function.IntSupplier;
 
 public class SchemaChangeHandler extends AlterHandler {
     private static final Logger LOG = LogManager.getLogger(SchemaChangeHandler.class);
@@ -125,8 +128,16 @@ public class SchemaChangeHandler extends AlterHandler {
         super("schema change", Config.default_schema_change_scheduler_interval_millisecond);
     }
 
-    private void processAddColumn(AddColumnClause alterClause, OlapTable olapTable,
-                                  Map<Long, LinkedList<Column>> indexSchemaMap) throws DdlException {
+    /**
+     * @param alterClause
+     * @param olapTable
+     * @param indexSchemaMap
+     * @param colUniqueIdSupplier for multi add columns clause, we need stash middle state of maxColUniqueId
+     * @return true: can light schema change, false: cannot light schema change
+     * @throws DdlException
+     */
+    private boolean processAddColumn(AddColumnClause alterClause, OlapTable olapTable,
+            Map<Long, LinkedList<Column>> indexSchemaMap, IntSupplier colUniqueIdSupplier) throws DdlException {
         Column column = alterClause.getColumn();
         ColumnPosition columnPos = alterClause.getColPos();
         String targetIndexName = alterClause.getRollupName();
@@ -142,12 +153,18 @@ public class SchemaChangeHandler extends AlterHandler {
         }
 
         Set<String> newColNameSet = Sets.newHashSet(column.getName());
-        addColumnInternal(olapTable, column, columnPos, targetIndexId, baseIndexId,
-                indexSchemaMap, newColNameSet);
+
+        //only new table generate ColUniqueId, exist table do not.
+        if (olapTable.getMaxColUniqueId() > Column.COLUMN_UNIQUE_ID_INIT_VALUE) {
+            column.setUniqueId(colUniqueIdSupplier.getAsInt());
+        }
+
+        return addColumnInternal(olapTable, column, columnPos, targetIndexId, baseIndexId, indexSchemaMap,
+                newColNameSet, false);
     }
 
-    private void processAddColumn(AddColumnClause alterClause,
-            Table externalTable, List<Column> newSchema) throws DdlException {
+    private void processAddColumn(AddColumnClause alterClause, Table externalTable, List<Column> newSchema)
+            throws DdlException {
         Column column = alterClause.getColumn();
         ColumnPosition columnPos = alterClause.getColPos();
         Set<String> newColNameSet = Sets.newHashSet(column.getName());
@@ -155,8 +172,8 @@ public class SchemaChangeHandler extends AlterHandler {
         addColumnInternal(column, columnPos, newSchema, newColNameSet);
     }
 
-    private void processAddColumns(AddColumnsClause alterClause,
-            Table externalTable, List<Column> newSchema) throws DdlException {
+    private void processAddColumns(AddColumnsClause alterClause, Table externalTable, List<Column> newSchema)
+            throws DdlException {
         List<Column> columns = alterClause.getColumns();
         Set<String> newColNameSet = Sets.newHashSet();
         for (Column column : alterClause.getColumns()) {
@@ -168,8 +185,18 @@ public class SchemaChangeHandler extends AlterHandler {
         }
     }
 
-    private void processAddColumns(AddColumnsClause alterClause, OlapTable olapTable,
-                                   Map<Long, LinkedList<Column>> indexSchemaMap) throws DdlException {
+    /**
+     * @param alterClause
+     * @param olapTable
+     * @param indexSchemaMap
+     * @param ignoreSameColumn
+     * @param colUniqueIdSupplier for multi add columns clause, we need stash middle state of maxColUniqueId
+     * @return true: can light schema change, false: cannot light schema change
+     * @throws DdlException
+     */
+    public boolean processAddColumns(AddColumnsClause alterClause, OlapTable olapTable,
+            Map<Long, LinkedList<Column>> indexSchemaMap, boolean ignoreSameColumn, IntSupplier colUniqueIdSupplier)
+            throws DdlException {
         List<Column> columns = alterClause.getColumns();
         String targetIndexName = alterClause.getRollupName();
         checkIndexExists(olapTable, targetIndexName);
@@ -188,14 +215,26 @@ public class SchemaChangeHandler extends AlterHandler {
             targetIndexId = olapTable.getIndexIdByName(targetIndexName);
         }
 
+        //for new table calculate column unique id
+        if (olapTable.getMaxColUniqueId() > Column.COLUMN_UNIQUE_ID_INIT_VALUE) {
+            for (Column column : columns) {
+                column.setUniqueId(colUniqueIdSupplier.getAsInt());
+            }
+        }
+
+        boolean ligthSchemaChange = true;
         for (Column column : columns) {
-            addColumnInternal(olapTable, column, null, targetIndexId, baseIndexId,
-                    indexSchemaMap, newColNameSet);
+            boolean result = addColumnInternal(olapTable, column, null, targetIndexId, baseIndexId, indexSchemaMap,
+                    newColNameSet, ignoreSameColumn);
+            if (!result) {
+                ligthSchemaChange = false;
+            }
         }
+        return ligthSchemaChange;
     }
 
-    private void processDropColumn(DropColumnClause alterClause,
-            Table externalTable, List<Column> newSchema) throws DdlException {
+    private void processDropColumn(DropColumnClause alterClause, Table externalTable, List<Column> newSchema)
+            throws DdlException {
         String dropColName = alterClause.getColName();
 
         // find column in base index and remove it
@@ -208,8 +247,9 @@ public class SchemaChangeHandler extends AlterHandler {
                     baseIter.remove();
                     found = true;
                 } else {
-                    throw new DdlException("Do not allow remove last column of table: " + externalTable.getName()
-                            + " column: " + dropColName);
+                    throw new DdlException(
+                            "Do not allow remove last column of table: " + externalTable.getName() + " column: "
+                                    + dropColName);
                 }
                 break;
             }
@@ -220,8 +260,23 @@ public class SchemaChangeHandler extends AlterHandler {
         }
     }
 
-    private void processDropColumn(DropColumnClause alterClause, OlapTable olapTable,
+    /**
+     * @param alterClause
+     * @param olapTable
+     * @param indexSchemaMap
+     * @param indexes
+     * @return true: can light schema change, false: cannot
+     * @throws DdlException
+     */
+    private boolean processDropColumn(DropColumnClause alterClause, OlapTable olapTable,
             Map<Long, LinkedList<Column>> indexSchemaMap, List<Index> indexes) throws DdlException {
+
+        boolean ligthSchemaChange = false;
+        if (olapTable.getMaxColUniqueId() > Column.COLUMN_UNIQUE_ID_INIT_VALUE) {
+            //assume can light schema change.
+            ligthSchemaChange = true;
+        }
+
         String dropColName = alterClause.getColName();
         String targetIndexName = alterClause.getRollupName();
         checkIndexExists(olapTable, targetIndexName);
@@ -241,6 +296,7 @@ public class SchemaChangeHandler extends AlterHandler {
             boolean isKey = false;
             for (Column column : baseSchema) {
                 if (column.isKey() && column.getName().equalsIgnoreCase(dropColName)) {
+                    ligthSchemaChange = false;
                     isKey = true;
                     break;
                 }
@@ -260,6 +316,7 @@ public class SchemaChangeHandler extends AlterHandler {
                 for (Column column : baseSchema) {
                     if (column.isKey() && column.getName().equalsIgnoreCase(dropColName)) {
                         isKey = true;
+                        ligthSchemaChange = false;
                     } else if (AggregateType.REPLACE == column.getAggregationType()
                             || AggregateType.REPLACE_IF_NOT_NULL == column.getAggregationType()) {
                         hasReplaceColumn = true;
@@ -279,6 +336,7 @@ public class SchemaChangeHandler extends AlterHandler {
                 for (Column column : targetIndexSchema) {
                     if (column.isKey() && column.getName().equalsIgnoreCase(dropColName)) {
                         isKey = true;
+                        ligthSchemaChange = false;
                     } else if (AggregateType.REPLACE == column.getAggregationType()
                             || AggregateType.REPLACE_IF_NOT_NULL == column.getAggregationType()) {
                         hasReplaceColumn = true;
@@ -289,6 +347,15 @@ public class SchemaChangeHandler extends AlterHandler {
                             "Can not drop key column when rollup has value column with REPLACE aggregation method");
                 }
             }
+        } else if (KeysType.DUP_KEYS == olapTable.getKeysType()) {
+            long baseIndexId = olapTable.getBaseIndexId();
+            List<Column> baseSchema = indexSchemaMap.get(baseIndexId);
+            for (Column column : baseSchema) {
+                if (column.isKey() && column.getName().equalsIgnoreCase(dropColName)) {
+                    ligthSchemaChange = false;
+                    break;
+                }
+            }
         }
 
         Iterator<Index> it = indexes.iterator();
@@ -332,12 +399,14 @@ public class SchemaChangeHandler extends AlterHandler {
                 while (iter.hasNext()) {
                     Column column = iter.next();
                     if (column.getName().equalsIgnoreCase(dropColName)) {
+                        ligthSchemaChange = false;
                         iter.remove();
                         break;
                     }
                 }
             } // end for index names
         } else {
+            ligthSchemaChange = false;
             // if specify rollup index, only drop column from specified rollup index
             long targetIndexId = olapTable.getIndexIdByName(targetIndexName);
             // find column
@@ -356,11 +425,12 @@ public class SchemaChangeHandler extends AlterHandler {
                 throw new DdlException("Column does not exists: " + dropColName);
             }
         }
+        return ligthSchemaChange;
     }
 
     // User can modify column type and column position
-    private void processModifyColumn(ModifyColumnClause alterClause,
-            Table externalTable, List<Column> newSchema) throws DdlException {
+    private void processModifyColumn(ModifyColumnClause alterClause, Table externalTable, List<Column> newSchema)
+            throws DdlException {
         Column modColumn = alterClause.getColumn();
         ColumnPosition columnPos = alterClause.getColPos();
 
@@ -427,7 +497,7 @@ public class SchemaChangeHandler extends AlterHandler {
 
     // User can modify column type and column position
     private void processModifyColumn(ModifyColumnClause alterClause, OlapTable olapTable,
-                                     Map<Long, LinkedList<Column>> indexSchemaMap) throws DdlException {
+            Map<Long, LinkedList<Column>> indexSchemaMap) throws DdlException {
         Column modColumn = alterClause.getColumn();
         if (KeysType.AGG_KEYS == olapTable.getKeysType()) {
             if (modColumn.isKey() && null != modColumn.getAggregationType()) {
@@ -438,16 +508,17 @@ public class SchemaChangeHandler extends AlterHandler {
             }
         } else if (KeysType.UNIQUE_KEYS == olapTable.getKeysType()) {
             if (null != modColumn.getAggregationType()) {
-                throw new DdlException("Can not assign aggregation method"
-                        + " on column in Unique data model table: " + modColumn.getName());
+                throw new DdlException("Can not assign aggregation method" + " on column in Unique data model table: "
+                        + modColumn.getName());
             }
             if (!modColumn.isKey()) {
                 modColumn.setAggregationType(AggregateType.REPLACE, true);
             }
         } else {
             if (null != modColumn.getAggregationType()) {
-                throw new DdlException("Can not assign aggregation method"
-                        + " on column in Duplicate data model table: " + modColumn.getName());
+                throw new DdlException(
+                        "Can not assign aggregation method" + " on column in Duplicate data model table: "
+                                + modColumn.getName());
             }
             if (!modColumn.isKey()) {
                 modColumn.setAggregationType(AggregateType.NONE, true);
@@ -518,6 +589,7 @@ public class SchemaChangeHandler extends AlterHandler {
         Column oriColumn = schemaForFinding.get(modColIndex);
         // retain old column name
         modColumn.setName(oriColumn.getName());
+        modColumn.setUniqueId(oriColumn.getUniqueId());
 
         // handle the move operation in 'indexForFindingColumn' if has
         if (hasColPos) {
@@ -614,8 +686,8 @@ public class SchemaChangeHandler extends AlterHandler {
         }
     }
 
-    private void processReorderColumn(ReorderColumnsClause alterClause,
-            Table externalTable, List<Column> newSchema) throws DdlException {
+    private void processReorderColumn(ReorderColumnsClause alterClause, Table externalTable, List<Column> newSchema)
+            throws DdlException {
         List<String> orderedColNames = alterClause.getColumnsByPos();
 
         newSchema.clear();
@@ -648,7 +720,7 @@ public class SchemaChangeHandler extends AlterHandler {
     }
 
     private void processReorderColumn(ReorderColumnsClause alterClause, OlapTable olapTable,
-                                      Map<Long, LinkedList<Column>> indexSchemaMap) throws DdlException {
+            Map<Long, LinkedList<Column>> indexSchemaMap) throws DdlException {
         List<String> orderedColNames = alterClause.getColumnsByPos();
         String targetIndexName = alterClause.getRollupName();
         checkIndexExists(olapTable, targetIndexName);
@@ -663,7 +735,7 @@ public class SchemaChangeHandler extends AlterHandler {
         long targetIndexId = olapTable.getIndexIdByName(targetIndexName);
 
         LinkedList<Column> newSchema = new LinkedList<Column>();
-        LinkedList<Column> targetIndexSchema = indexSchemaMap.get(targetIndexId);
+        List<Column> targetIndexSchema = indexSchemaMap.get(targetIndexId);
 
         // check and create new ordered column list
         Set<String> colNameSet = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER);
@@ -703,9 +775,8 @@ public class SchemaChangeHandler extends AlterHandler {
      * Add 'newColumn' to specified index.
      * Modified schema will be saved in 'indexSchemaMap'
      */
-    private void addColumnInternal(Column newColumn, ColumnPosition columnPos,
-                                   List<Column> modIndexSchema,
-                                   Set<String> newColNameSet) throws DdlException {
+    private void addColumnInternal(Column newColumn, ColumnPosition columnPos, List<Column> modIndexSchema,
+            Set<String> newColNameSet) throws DdlException {
         String newColName = newColumn.getName();
         int posIndex = -1;
         boolean hasPos = (columnPos != null && !columnPos.isFirst());
@@ -755,16 +826,33 @@ public class SchemaChangeHandler extends AlterHandler {
         }
     }
 
-    /*
-     * Add 'newColumn' to specified index.
-     * Modified schema will be saved in 'indexSchemaMap'
+    /**
+     * @param olapTable
+     * @param newColumn Add 'newColumn' to specified index.
+     * @param columnPos
+     * @param targetIndexId
+     * @param baseIndexId
+     * @param indexSchemaMap Modified schema will be saved in 'indexSchemaMap'
+     * @param newColNameSet
+     * @param ignoreSameColumn
+     * @return true: can light schema change, false: cannot
+     * @throws DdlException
      */
-    private void addColumnInternal(OlapTable olapTable, Column newColumn, ColumnPosition columnPos,
-                                   long targetIndexId, long baseIndexId,
-                                   Map<Long, LinkedList<Column>> indexSchemaMap,
-                                   Set<String> newColNameSet) throws DdlException {
+    private boolean addColumnInternal(OlapTable olapTable, Column newColumn, ColumnPosition columnPos,
+            long targetIndexId, long baseIndexId, Map<Long, LinkedList<Column>> indexSchemaMap,
+            Set<String> newColNameSet, boolean ignoreSameColumn) throws DdlException {
 
+        //only new table generate ColUniqueId, exist table do not.
+        boolean ligthSchemaChange = olapTable.getMaxColUniqueId() > Column.COLUMN_UNIQUE_ID_INIT_VALUE;
         String newColName = newColumn.getName();
+
+        //make sure olapTable has locked
+        if (newColumn.getUniqueId() > Integer.MAX_VALUE) {
+            throw new DdlException("schema change add column times overflow: " + newColName);
+        }
+        LOG.debug("table: {}, newColumn: {}, uniqueId: {}", olapTable.getName(), newColumn.getName(),
+                newColumn.getUniqueId());
+
         // check the validation of aggregation method on column.
         // also fill the default aggregation method if not specified.
         if (KeysType.AGG_KEYS == olapTable.getKeysType()) {
@@ -772,30 +860,31 @@ public class SchemaChangeHandler extends AlterHandler {
                 throw new DdlException("Can not assign aggregation method on key column: " + newColName);
             } else if (null == newColumn.getAggregationType()) {
                 newColumn.setIsKey(true);
-            } else if (newColumn.getAggregationType() == AggregateType.SUM
-                    && newColumn.getDefaultValue() != null && !newColumn.getDefaultValue().equals("0")) {
-                throw new DdlException("The default value of '"
-                        + newColName + "' with SUM aggregation function must be zero");
+            } else if (newColumn.getAggregationType() == AggregateType.SUM && newColumn.getDefaultValue() != null
+                    && !newColumn.getDefaultValue().equals("0")) {
+                throw new DdlException(
+                        "The default value of '" + newColName + "' with SUM aggregation function must be zero");
             } else if (olapTable.getDefaultDistributionInfo() instanceof RandomDistributionInfo) {
                 if (newColumn.getAggregationType() == AggregateType.REPLACE
                         || newColumn.getAggregationType() == AggregateType.REPLACE_IF_NOT_NULL) {
-                    throw new DdlException("Can not add value column with aggregation type "
-                                + newColumn.getAggregationType() + " for olap table with random distribution : "
-                                + newColName);
+                    throw new DdlException(
+                            "Can not add value column with aggregation type " + newColumn.getAggregationType()
+                                    + " for olap table with random distribution : " + newColName);
                 }
             }
         } else if (KeysType.UNIQUE_KEYS == olapTable.getKeysType()) {
             if (newColumn.getAggregationType() != null) {
-                throw new DdlException("Can not assign aggregation method"
-                        + " on column in Unique data model table: " + newColName);
+                throw new DdlException(
+                        "Can not assign aggregation method" + " on column in Unique data model table: " + newColName);
             }
             if (!newColumn.isKey()) {
                 newColumn.setAggregationType(AggregateType.REPLACE, true);
             }
         } else {
             if (newColumn.getAggregationType() != null) {
-                throw new DdlException("Can not assign aggregation method"
-                        + " on column in Duplicate data model table: " + newColName);
+                throw new DdlException(
+                        "Can not assign aggregation method" + " on column in Duplicate data model table: "
+                                + newColName);
             }
             if (!newColumn.isKey()) {
                 if (targetIndexId != -1L
@@ -816,13 +905,20 @@ public class SchemaChangeHandler extends AlterHandler {
             throw new DdlException("BITMAP_UNION must be used in AGG_KEYS");
         }
 
+        //type key column do not allow light schema change.
+        if (newColumn.isKey()) {
+            ligthSchemaChange = false;
+        }
+
         // check if the new column already exist in base schema.
         // do not support adding new column which already exist in base schema.
         List<Column> baseSchema = olapTable.getBaseSchema(true);
         boolean found = false;
+        Column foundColumn = null;
         for (Column column : baseSchema) {
             if (column.getName().equalsIgnoreCase(newColName)) {
                 found = true;
+                foundColumn = column;
                 break;
             }
         }
@@ -832,7 +928,11 @@ public class SchemaChangeHandler extends AlterHandler {
             } else if (newColName.equalsIgnoreCase(Column.SEQUENCE_COL)) {
                 throw new DdlException("Can not enable sequence column support, already supported sequence column.");
             } else {
-                throw new DdlException("Can not add column which already exists in base table: " + newColName);
+                if (ignoreSameColumn && newColumn.equals(foundColumn)) {
+                    //for add columns rpc, allow add same type column.
+                } else {
+                    throw new DdlException("Can not add column which already exists in base table: " + newColName);
+                }
             }
         }
 
@@ -864,9 +964,10 @@ public class SchemaChangeHandler extends AlterHandler {
                 modIndexSchema = indexSchemaMap.get(baseIndexId);
                 checkAndAddColumn(modIndexSchema, newColumn, columnPos, newColNameSet, true);
                 if (targetIndexId == -1L) {
-                    return;
+                    return ligthSchemaChange;
                 }
                 // 2. add to rollup
+                ligthSchemaChange = false;
                 modIndexSchema = indexSchemaMap.get(targetIndexId);
                 checkAndAddColumn(modIndexSchema, newColumn, columnPos, newColNameSet, false);
             }
@@ -876,9 +977,10 @@ public class SchemaChangeHandler extends AlterHandler {
                 List<Column> modIndexSchema = indexSchemaMap.get(baseIndexId);
                 checkAndAddColumn(modIndexSchema, newColumn, columnPos, newColNameSet, true);
                 // no specified target index. return
-                return;
+                return ligthSchemaChange;
             } else {
                 // add to rollup index
+                ligthSchemaChange = false;
                 List<Column> modIndexSchema = indexSchemaMap.get(targetIndexId);
                 checkAndAddColumn(modIndexSchema, newColumn, columnPos, newColNameSet, false);
 
@@ -902,13 +1004,15 @@ public class SchemaChangeHandler extends AlterHandler {
 
             if (targetIndexId == -1L) {
                 // no specified target index. return
-                return;
+                return ligthSchemaChange;
             }
 
+            ligthSchemaChange = false;
             // 2. add to rollup index
             modIndexSchema = indexSchemaMap.get(targetIndexId);
             checkAndAddColumn(modIndexSchema, newColumn, columnPos, newColNameSet, false);
         }
+        return ligthSchemaChange;
     }
 
     /*
@@ -921,7 +1025,7 @@ public class SchemaChangeHandler extends AlterHandler {
      * So that k1 will be added to base index 'twice', and we just ignore this repeat adding.
      */
     private void checkAndAddColumn(List<Column> modIndexSchema, Column newColumn, ColumnPosition columnPos,
-                                   Set<String> newColNameSet, boolean isBaseIndex) throws DdlException {
+            Set<String> newColNameSet, boolean isBaseIndex) throws DdlException {
         int posIndex = -1;
         int lastVisibleIdx = -1;
         String newColName = newColumn.getName();
@@ -987,8 +1091,8 @@ public class SchemaChangeHandler extends AlterHandler {
 
     private void checkIndexExists(OlapTable olapTable, String targetIndexName) throws DdlException {
         if (targetIndexName != null && !olapTable.hasMaterializedIndex(targetIndexName)) {
-            throw new DdlException("Index[" + targetIndexName + "] does not exist in table[" + olapTable.getName()
-                    + "]");
+            throw new DdlException(
+                    "Index[" + targetIndexName + "] does not exist in table[" + olapTable.getName() + "]");
         }
     }
 
@@ -1002,7 +1106,7 @@ public class SchemaChangeHandler extends AlterHandler {
     }
 
     private void createJob(long dbId, OlapTable olapTable, Map<Long, LinkedList<Column>> indexSchemaMap,
-                           Map<String, String> propertyMap, List<Index> indexes) throws UserException {
+            Map<String, String> propertyMap, List<Index> indexes) throws UserException {
         if (olapTable.getState() == OlapTableState.ROLLUP) {
             throw new DdlException("Table[" + olapTable.getName() + "]'s is doing ROLLUP job");
         }
@@ -1022,8 +1126,8 @@ public class SchemaChangeHandler extends AlterHandler {
                 if (key.endsWith(PropertyAnalyzer.PROPERTIES_SHORT_KEY)) {
                     // short key
                     String[] keyArray = key.split("#");
-                    if (keyArray.length != 2 || keyArray[0].isEmpty()
-                            || !keyArray[1].equals(PropertyAnalyzer.PROPERTIES_SHORT_KEY)) {
+                    if (keyArray.length != 2 || keyArray[0].isEmpty() || !keyArray[1].equals(
+                            PropertyAnalyzer.PROPERTIES_SHORT_KEY)) {
                         throw new DdlException("Invalid alter table property: " + key);
                     }
 
@@ -1117,8 +1221,8 @@ public class SchemaChangeHandler extends AlterHandler {
         // create job
         Catalog catalog = Catalog.getCurrentCatalog();
         long jobId = catalog.getNextId();
-        SchemaChangeJobV2 schemaChangeJob = new SchemaChangeJobV2(jobId, dbId,
-                olapTable.getId(), olapTable.getName(), timeoutSecond * 1000);
+        SchemaChangeJobV2 schemaChangeJob = new SchemaChangeJobV2(jobId, dbId, olapTable.getId(), olapTable.getName(),
+                timeoutSecond * 1000);
         schemaChangeJob.setBloomFilterInfo(hasBfChange, bfColumns, bfFpp);
         schemaChangeJob.setAlterIndexInfo(hasIndexChange, indexes);
 
@@ -1209,8 +1313,9 @@ public class SchemaChangeHandler extends AlterHandler {
             boolean hasKey = false;
             for (Column column : alterSchema) {
                 if (column.isKey() && meetValue) {
-                    throw new DdlException("Invalid column order. value should be after key. index["
-                            + olapTable.getIndexNameById(alterIndexId) + "]");
+                    throw new DdlException(
+                            "Invalid column order. value should be after key. index[" + olapTable.getIndexNameById(
+                                    alterIndexId) + "]");
                 }
                 if (!column.isKey()) {
                     meetValue = true;
@@ -1244,9 +1349,9 @@ public class SchemaChangeHandler extends AlterHandler {
                         if (alterColumn.nameEquals(partitionCol.getName(), true)) {
                             // 2.1 partition column cannot be modified
                             if (needAlterColumns.contains(alterColumn) && !alterColumn.equals(partitionCol)) {
-                                throw new DdlException("Can not modify partition column["
-                                        + partitionCol.getName() + "]. index["
-                                        + olapTable.getIndexNameById(alterIndexId) + "]");
+                                throw new DdlException(
+                                        "Can not modify partition column[" + partitionCol.getName() + "]. index["
+                                                + olapTable.getIndexNameById(alterIndexId) + "]");
                             }
                             found = true;
                             break;
@@ -1255,8 +1360,9 @@ public class SchemaChangeHandler extends AlterHandler {
 
                     if (!found && alterIndexId == olapTable.getBaseIndexId()) {
                         // 2.1 partition column cannot be deleted.
-                        throw new DdlException("Partition column[" + partitionCol.getName()
-                                + "] cannot be dropped. index[" + olapTable.getIndexNameById(alterIndexId) + "]");
+                        throw new DdlException(
+                                "Partition column[" + partitionCol.getName() + "] cannot be dropped. index["
+                                        + olapTable.getIndexNameById(alterIndexId) + "]");
                         // ATTN. partition columns' order also need remaining unchanged.
                         // for now, we only allow one partition column, so no need to check order.
                     }
@@ -1273,9 +1379,9 @@ public class SchemaChangeHandler extends AlterHandler {
                         if (alterColumn.nameEquals(distributionCol.getName(), true)) {
                             // 3.1 distribution column cannot be modified
                             if (needAlterColumns.contains(alterColumn) && !alterColumn.equals(distributionCol)) {
-                                throw new DdlException("Can not modify distribution column["
-                                        + distributionCol.getName() + "]. index["
-                                        + olapTable.getIndexNameById(alterIndexId) + "]");
+                                throw new DdlException(
+                                        "Can not modify distribution column[" + distributionCol.getName() + "]. index["
+                                                + olapTable.getIndexNameById(alterIndexId) + "]");
                             }
                             found = true;
                             break;
@@ -1284,8 +1390,9 @@ public class SchemaChangeHandler extends AlterHandler {
 
                     if (!found && alterIndexId == olapTable.getBaseIndexId()) {
                         // 2.2 distribution column cannot be deleted.
-                        throw new DdlException("Distribution column[" + distributionCol.getName()
-                                + "] cannot be dropped. index[" + olapTable.getIndexNameById(alterIndexId) + "]");
+                        throw new DdlException(
+                                "Distribution column[" + distributionCol.getName() + "] cannot be dropped. index["
+                                        + olapTable.getIndexNameById(alterIndexId) + "]");
                     }
                 } // end for distributionCols
             }
@@ -1340,8 +1447,8 @@ public class SchemaChangeHandler extends AlterHandler {
                 // index state is SHADOW
                 MaterializedIndex shadowIndex = new MaterializedIndex(shadowIndexId, IndexState.SHADOW);
                 MaterializedIndex originIndex = partition.getIndex(originIndexId);
-                TabletMeta shadowTabletMeta = new TabletMeta(dbId, tableId, partitionId,
-                        shadowIndexId, newSchemaHash, medium);
+                TabletMeta shadowTabletMeta = new TabletMeta(dbId, tableId, partitionId, shadowIndexId, newSchemaHash,
+                        medium);
                 ReplicaAllocation replicaAlloc = olapTable.getPartitionInfo().getReplicaAllocation(partitionId);
                 Short totalReplicaNum = replicaAlloc.getTotalReplicaNum();
                 for (Tablet originTablet : originIndex.getTablets()) {
@@ -1398,8 +1505,8 @@ public class SchemaChangeHandler extends AlterHandler {
 
                 schemaChangeJob.addPartitionShadowIndex(partitionId, shadowIndexId, shadowIndex);
             } // end for partition
-            schemaChangeJob.addIndexSchema(shadowIndexId, originIndexId, newIndexName,
-                    newSchemaVersion, newSchemaHash, newShortKeyColumnCount, entry.getValue());
+            schemaChangeJob.addIndexSchema(shadowIndexId, originIndexId, newIndexName, newSchemaVersion, newSchemaHash,
+                    newShortKeyColumnCount, entry.getValue());
         } // end for index
 
         // set table state
@@ -1425,25 +1532,24 @@ public class SchemaChangeHandler extends AlterHandler {
     }
 
     private void runAlterJobV2() {
-        runnableSchemaChangeJobV2.values().forEach(
-                alterJobsV2 -> {
-                    if (!alterJobsV2.isDone() && !activeSchemaChangeJobsV2.containsKey(alterJobsV2.getJobId())
-                            && activeSchemaChangeJobsV2.size() < MAX_ACTIVE_SCHEMA_CHANGE_JOB_V2_SIZE) {
-                        if (FeConstants.runningUnitTest) {
-                            alterJobsV2.run();
-                        } else {
-                            schemaChangeThreadPool.submit(() -> {
-                                if (activeSchemaChangeJobsV2.putIfAbsent(alterJobsV2.getJobId(), alterJobsV2) == null) {
-                                    try {
-                                        alterJobsV2.run();
-                                    } finally {
-                                        activeSchemaChangeJobsV2.remove(alterJobsV2.getJobId());
-                                    }
-                                }
-                            });
+        runnableSchemaChangeJobV2.values().forEach(alterJobsV2 -> {
+            if (!alterJobsV2.isDone() && !activeSchemaChangeJobsV2.containsKey(alterJobsV2.getJobId())
+                    && activeSchemaChangeJobsV2.size() < MAX_ACTIVE_SCHEMA_CHANGE_JOB_V2_SIZE) {
+                if (FeConstants.runningUnitTest) {
+                    alterJobsV2.run();
+                } else {
+                    schemaChangeThreadPool.submit(() -> {
+                        if (activeSchemaChangeJobsV2.putIfAbsent(alterJobsV2.getJobId(), alterJobsV2) == null) {
+                            try {
+                                alterJobsV2.run();
+                            } finally {
+                                activeSchemaChangeJobsV2.remove(alterJobsV2.getJobId());
+                            }
                         }
-                    }
-                });
+                    });
+                }
+            }
+        });
     }
 
     @Override
@@ -1465,8 +1571,8 @@ public class SchemaChangeHandler extends AlterHandler {
                 continue;
             }
             if (ctx != null) {
-                if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(
-                        ctx, db.getFullName(), alterJob.getTableName(), PrivPredicate.ALTER)) {
+                if (!Catalog.getCurrentCatalog().getAuth()
+                        .checkTblPriv(ctx, db.getFullName(), alterJob.getTableName(), PrivPredicate.ALTER)) {
                     continue;
                 }
             }
@@ -1483,6 +1589,19 @@ public class SchemaChangeHandler extends AlterHandler {
             throws UserException {
         olapTable.writeLockOrDdlException();
         try {
+            //alterClauses can or cannot light schema change
+            boolean ligthSchemaChange = true;
+            //for multi add colmuns clauses
+            IntSupplier colUniqueIdSupplier = new IntSupplier() {
+                public int pendingMaxColUniqueId = olapTable.getMaxColUniqueId();
+
+                @Override
+                public int getAsInt() {
+                    pendingMaxColUniqueId++;
+                    return pendingMaxColUniqueId;
+                }
+            };
+
             // index id -> index schema
             Map<Long, LinkedList<Column>> indexSchemaMap = new HashMap<>();
             for (Map.Entry<Long, List<Column>> entry : olapTable.getIndexIdToSchema(true).entrySet()) {
@@ -1509,8 +1628,8 @@ public class SchemaChangeHandler extends AlterHandler {
                     } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_DISTRIBUTION_TYPE)) {
                         String distributionType = properties.get(PropertyAnalyzer.PROPERTIES_DISTRIBUTION_TYPE);
                         if (!distributionType.equalsIgnoreCase("random")) {
-                            throw new DdlException("Only support modifying distribution type of table from"
-                                + " hash to random");
+                            throw new DdlException(
+                                    "Only support modifying distribution type of table from" + " hash to random");
                         }
                         Catalog.getCurrentCatalog().convertDistributionType(db, olapTable);
                         return;
@@ -1523,14 +1642,13 @@ public class SchemaChangeHandler extends AlterHandler {
                     } else if (DynamicPartitionUtil.checkDynamicPartitionPropertiesExist(properties)) {
                         if (!olapTable.dynamicPartitionExists()) {
                             try {
-                                DynamicPartitionUtil.checkInputDynamicPartitionProperties(
-                                        properties, olapTable.getPartitionInfo());
+                                DynamicPartitionUtil.checkInputDynamicPartitionProperties(properties,
+                                        olapTable.getPartitionInfo());
                             } catch (DdlException e) {
                                 // This table is not a dynamic partition table
                                 // and didn't supply all dynamic partition properties
-                                throw new DdlException("Table " + db.getFullName() + "."
-                                        + olapTable.getName() + " is not a dynamic partition table."
-                                        + " Use command `HELP ALTER TABLE` "
+                                throw new DdlException("Table " + db.getFullName() + "." + olapTable.getName()
+                                        + " is not a dynamic partition table." + " Use command `HELP ALTER TABLE` "
                                         + "to see how to change a normal table to a dynamic partition table.");
                             }
                         }
@@ -1538,8 +1656,8 @@ public class SchemaChangeHandler extends AlterHandler {
                         return;
                     } else if (properties.containsKey(
                             "default." + PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION)) {
-                        Preconditions.checkNotNull(properties.get("default."
-                                + PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION));
+                        Preconditions.checkNotNull(
+                                properties.get("default." + PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION));
                         Catalog.getCurrentCatalog().modifyTableDefaultReplicaAllocation(db, olapTable, properties);
                         return;
                     } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION)) {
@@ -1559,36 +1677,63 @@ public class SchemaChangeHandler extends AlterHandler {
 
                 if (alterClause instanceof AddColumnClause) {
                     // add column
-                    processAddColumn((AddColumnClause) alterClause, olapTable, indexSchemaMap);
+                    boolean clauseCanLigthSchemaChange = processAddColumn((AddColumnClause) alterClause, olapTable,
+                            indexSchemaMap, colUniqueIdSupplier);
+                    if (clauseCanLigthSchemaChange == false) {
+                        ligthSchemaChange = false;
+                    }
                 } else if (alterClause instanceof AddColumnsClause) {
                     // add columns
-                    processAddColumns((AddColumnsClause) alterClause, olapTable, indexSchemaMap);
+                    boolean clauseCanLigthSchemaChange = processAddColumns((AddColumnsClause) alterClause, olapTable,
+                            indexSchemaMap, false, colUniqueIdSupplier);
+                    if (clauseCanLigthSchemaChange == false) {
+                        ligthSchemaChange = false;
+                    }
                 } else if (alterClause instanceof DropColumnClause) {
                     // drop column and drop indexes on this column
-                    processDropColumn((DropColumnClause) alterClause, olapTable, indexSchemaMap, newIndexes);
+                    boolean clauseCanLigthSchemaChange = processDropColumn((DropColumnClause) alterClause, olapTable,
+                            indexSchemaMap, newIndexes);
+                    if (clauseCanLigthSchemaChange == false) {
+                        ligthSchemaChange = false;
+                    }
                 } else if (alterClause instanceof ModifyColumnClause) {
                     // modify column
                     processModifyColumn((ModifyColumnClause) alterClause, olapTable, indexSchemaMap);
+                    ligthSchemaChange = false;
                 } else if (alterClause instanceof ReorderColumnsClause) {
                     // reorder column
                     processReorderColumn((ReorderColumnsClause) alterClause, olapTable, indexSchemaMap);
+                    ligthSchemaChange = false;
                 } else if (alterClause instanceof ModifyTablePropertiesClause) {
                     // modify table properties
                     // do nothing, properties are already in propertyMap
+                    ligthSchemaChange = false;
                 } else if (alterClause instanceof CreateIndexClause) {
                     if (processAddIndex((CreateIndexClause) alterClause, olapTable, newIndexes)) {
                         return;
                     }
+                    ligthSchemaChange = false;
                 } else if (alterClause instanceof DropIndexClause) {
                     if (processDropIndex((DropIndexClause) alterClause, olapTable, newIndexes)) {
                         return;
                     }
+                    ligthSchemaChange = false;
                 } else {
                     Preconditions.checkState(false);
                 }
             } // end for alter clauses
 
-            createJob(db.getId(), olapTable, indexSchemaMap, propertyMap, newIndexes);
+            LOG.debug("processAddColumns, table: {}({}), ligthSchemaChange: {}", olapTable.getName(), olapTable.getId(),
+                    ligthSchemaChange);
+
+            if (ligthSchemaChange) {
+                long jobId = Catalog.getCurrentCatalog().getNextId();
+                //for schema change add/drop value column optimize, direct modify table meta.
+                modifyTableAddOrDropColumns(db, olapTable, indexSchemaMap, newIndexes, jobId, false);
+                return;
+            } else {
+                createJob(db.getId(), olapTable, indexSchemaMap, propertyMap, newIndexes);
+            }
         } finally {
             olapTable.writeUnlock();
         }
@@ -1690,9 +1835,8 @@ public class SchemaChangeHandler extends AlterHandler {
     /**
      * Update some specified partitions' in-memory property of table
      */
-    public void updatePartitionsInMemoryMeta(Database db,
-            String tableName, List<String> partitionNames, Map<String, String> properties)
-            throws DdlException, MetaNotFoundException {
+    public void updatePartitionsInMemoryMeta(Database db, String tableName, List<String> partitionNames,
+            Map<String, String> properties) throws DdlException, MetaNotFoundException {
         OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableName, Table.TableType.OLAP);
         boolean isInMemory = Boolean.parseBoolean(properties.get(PropertyAnalyzer.PROPERTIES_INMEMORY));
         if (isInMemory == olapTable.isInMemory()) {
@@ -1714,10 +1858,8 @@ public class SchemaChangeHandler extends AlterHandler {
      * Update one specified partition's in-memory property by partition name of table
      * This operation may return partial successfully, with a exception to inform user to retry
      */
-    public void updatePartitionInMemoryMeta(Database db,
-                                            String tableName,
-                                            String partitionName,
-                                            boolean isInMemory) throws UserException {
+    public void updatePartitionInMemoryMeta(Database db, String tableName, String partitionName, boolean isInMemory)
+            throws UserException {
         // be id -> <tablet id,schemaHash>
         Map<Long, Set<Pair<Long, Integer>>> beIdToTabletIdWithHash = Maps.newHashMap();
         OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableName, Table.TableType.OLAP);
@@ -1733,8 +1875,8 @@ public class SchemaChangeHandler extends AlterHandler {
                 int schemaHash = olapTable.getSchemaHashByIndexId(index.getId());
                 for (Tablet tablet : index.getTablets()) {
                     for (Replica replica : tablet.getReplicas()) {
-                        Set<Pair<Long, Integer>> tabletIdWithHash =
-                                beIdToTabletIdWithHash.computeIfAbsent(replica.getBackendId(), k -> Sets.newHashSet());
+                        Set<Pair<Long, Integer>> tabletIdWithHash = beIdToTabletIdWithHash.computeIfAbsent(
+                                replica.getBackendId(), k -> Sets.newHashSet());
                         tabletIdWithHash.add(new Pair<>(tablet.getId(), schemaHash));
                     }
                 }
@@ -1748,16 +1890,16 @@ public class SchemaChangeHandler extends AlterHandler {
         AgentBatchTask batchTask = new AgentBatchTask();
         for (Map.Entry<Long, Set<Pair<Long, Integer>>> kv : beIdToTabletIdWithHash.entrySet()) {
             countDownLatch.addMark(kv.getKey(), kv.getValue());
-            UpdateTabletMetaInfoTask task = new UpdateTabletMetaInfoTask(kv.getKey(), kv.getValue(),
-                    isInMemory, countDownLatch);
+            UpdateTabletMetaInfoTask task = new UpdateTabletMetaInfoTask(kv.getKey(), kv.getValue(), isInMemory,
+                    countDownLatch);
             batchTask.addTask(task);
         }
         if (!FeConstants.runningUnitTest) {
             // send all tasks and wait them finished
             AgentTaskQueue.addBatchTask(batchTask);
             AgentTaskExecutor.submit(batchTask);
-            LOG.info("send update tablet meta task for table {}, partitions {}, number: {}",
-                    tableName, partitionName, batchTask.getTaskNum());
+            LOG.info("send update tablet meta task for table {}, partitions {}, number: {}", tableName, partitionName,
+                    batchTask.getTaskNum());
 
             // estimate timeout
             long timeout = Config.tablet_create_timeout_second * 1000L * totalTaskNum;
@@ -1779,8 +1921,8 @@ public class SchemaChangeHandler extends AlterHandler {
                 } else {
                     List<Map.Entry<Long, Set<Pair<Long, Integer>>>> unfinishedMarks = countDownLatch.getLeftMarks();
                     // only show at most 3 results
-                    List<Map.Entry<Long, Set<Pair<Long, Integer>>>> subList
-                            = unfinishedMarks.subList(0, Math.min(unfinishedMarks.size(), 3));
+                    List<Map.Entry<Long, Set<Pair<Long, Integer>>>> subList = unfinishedMarks.subList(0,
+                            Math.min(unfinishedMarks.size(), 3));
                     if (!subList.isEmpty()) {
                         errMsg += " Unfinished mark: " + Joiner.on(", ").join(subList);
                     }
@@ -1815,11 +1957,11 @@ public class SchemaChangeHandler extends AlterHandler {
             // find from new alter jobs first
             List<AlterJobV2> schemaChangeJobV2List = getUnfinishedAlterJobV2ByTableId(olapTable.getId());
             // current schemaChangeJob job doesn't support batch operation,so just need to get one job
-            schemaChangeJobV2 = schemaChangeJobV2List.size() == 0
-                    ? null : Iterables.getOnlyElement(schemaChangeJobV2List);
+            schemaChangeJobV2 = schemaChangeJobV2List.size() == 0 ? null
+                    : Iterables.getOnlyElement(schemaChangeJobV2List);
             if (schemaChangeJobV2 == null) {
-                throw new DdlException("Table[" + tableName + "] is under schema change state"
-                        + " but could not find related job");
+                throw new DdlException(
+                        "Table[" + tableName + "] is under schema change state" + " but could not find related job");
             }
         } finally {
             olapTable.writeUnlock();
@@ -1851,8 +1993,8 @@ public class SchemaChangeHandler extends AlterHandler {
         for (Index existedIdx : existedIndexes) {
             if (existedIdx.getIndexName().equalsIgnoreCase(indexDef.getIndexName())) {
                 if (indexDef.isSetIfNotExists()) {
-                    LOG.info("create index[{}] which already exists on table[{}]",
-                            indexDef.getIndexName(), olapTable.getName());
+                    LOG.info("create index[{}] which already exists on table[{}]", indexDef.getIndexName(),
+                            olapTable.getName());
                     return true;
                 }
                 throw new DdlException("index `" + indexDef.getIndexName() + "` already exist.");
@@ -1882,8 +2024,8 @@ public class SchemaChangeHandler extends AlterHandler {
      * Returns true if the index does not exist, there is no need to create the job to drop the index.
      * Otherwise return false, there is need to create a job to drop the index.
      */
-    private boolean processDropIndex(DropIndexClause alterClause, OlapTable olapTable,
-            List<Index> indexes) throws DdlException {
+    private boolean processDropIndex(DropIndexClause alterClause, OlapTable olapTable, List<Index> indexes)
+            throws DdlException {
         String indexName = alterClause.getIndexName();
         List<Index> existedIndexes = olapTable.getIndexes();
         Index found = null;
@@ -1943,4 +2085,172 @@ public class SchemaChangeHandler extends AlterHandler {
         }
         super.replayAlterJobV2(alterJob);
     }
+
+    // the invoker should keep table's write lock
+    public void modifyTableAddOrDropColumns(Database db, OlapTable olapTable,
+            Map<Long, LinkedList<Column>> indexSchemaMap,
+            List<Index> indexes, long jobId, boolean isReplay) throws DdlException {
+
+        LOG.debug("indexSchemaMap:{}, indexes:{}", indexSchemaMap, indexes);
+        if (olapTable.getState() == OlapTableState.ROLLUP) {
+            throw new DdlException("Table[" + olapTable.getName() + "]'s is doing ROLLUP job");
+        }
+
+        // for now table's state can only be NORMAL
+        Preconditions.checkState(olapTable.getState() == OlapTableState.NORMAL, olapTable.getState().name());
+
+        // for bitmapIndex
+        boolean hasIndexChange = false;
+        Set<Index> newSet = new HashSet<>(indexes);
+        Set<Index> oriSet = new HashSet<>(olapTable.getIndexes());
+        if (!newSet.equals(oriSet)) {
+            hasIndexChange = true;
+        }
+
+        // begin checking each table
+        // ATTN: DO NOT change any meta in this loop
+        Map<Long, List<Column>> changedIndexIdToSchema = Maps.newHashMap();
+        for (Long alterIndexId : indexSchemaMap.keySet()) {
+            // Must get all columns including invisible columns.
+            // Because in alter process, all columns must be considered.
+            List<Column> alterSchema = indexSchemaMap.get(alterIndexId);
+
+            LOG.debug("index[{}] is changed. start checking...", alterIndexId);
+            // 1. check order: a) has key; b) value after key
+            boolean meetValue = false;
+            boolean hasKey = false;
+            for (Column column : alterSchema) {
+                if (column.isKey() && meetValue) {
+                    throw new DdlException(
+                            "Invalid column order. value should be after key. index[" + olapTable.getIndexNameById(
+                                    alterIndexId) + "]");
+                }
+                if (!column.isKey()) {
+                    meetValue = true;
+                } else {
+                    hasKey = true;
+                }
+            }
+            if (!hasKey) {
+                throw new DdlException("No key column left. index[" + olapTable.getIndexNameById(alterIndexId) + "]");
+            }
+
+            // 2. check partition key
+            PartitionInfo partitionInfo = olapTable.getPartitionInfo();
+            if (partitionInfo.getType() == PartitionType.RANGE || partitionInfo.getType() == PartitionType.LIST) {
+                List<Column> partitionColumns = partitionInfo.getPartitionColumns();
+                for (Column partitionCol : partitionColumns) {
+                    boolean found = false;
+                    for (Column alterColumn : alterSchema) {
+                        if (alterColumn.nameEquals(partitionCol.getName(), true)) {
+                            found = true;
+                            break;
+                        }
+                    } // end for alterColumns
+
+                    if (!found && alterIndexId == olapTable.getBaseIndexId()) {
+                        // 2.1 partition column cannot be deleted.
+                        throw new DdlException(
+                                "Partition column[" + partitionCol.getName() + "] cannot be dropped. index["
+                                        + olapTable.getIndexNameById(alterIndexId) + "]");
+                    }
+                } // end for partitionColumns
+            }
+
+            // 3. check distribution key:
+            DistributionInfo distributionInfo = olapTable.getDefaultDistributionInfo();
+            if (distributionInfo.getType() == DistributionInfoType.HASH) {
+                List<Column> distributionColumns = ((HashDistributionInfo) distributionInfo).getDistributionColumns();
+                for (Column distributionCol : distributionColumns) {
+                    boolean found = false;
+                    for (Column alterColumn : alterSchema) {
+                        if (alterColumn.nameEquals(distributionCol.getName(), true)) {
+                            found = true;
+                            break;
+                        }
+                    } // end for alterColumns
+                    if (!found && alterIndexId == olapTable.getBaseIndexId()) {
+                        // 2.2 distribution column cannot be deleted.
+                        throw new DdlException(
+                                "Distribution column[" + distributionCol.getName() + "] cannot be dropped. index["
+                                        + olapTable.getIndexNameById(alterIndexId) + "]");
+                    }
+                } // end for distributionCols
+            }
+
+            // 5. store the changed columns for edit log
+            changedIndexIdToSchema.put(alterIndexId, alterSchema);
+
+            LOG.debug("schema change[{}-{}-{}] check pass.", db.getId(), olapTable.getId(), alterIndexId);
+        } // end for indices
+
+        if (changedIndexIdToSchema.isEmpty() && !hasIndexChange) {
+            throw new DdlException("Nothing is changed. please check your alter stmt.");
+        }
+
+        //update base index schema
+        long baseIndexId = olapTable.getBaseIndexId();
+        List<Long> indexIds = new ArrayList<Long>();
+        indexIds.add(baseIndexId);
+        indexIds.addAll(olapTable.getIndexIdListExceptBaseIndex());
+        for (int i = 0; i < indexIds.size(); i++) {
+            List<Column> indexSchema = indexSchemaMap.get(indexIds.get(i));
+            MaterializedIndexMeta currentIndexMeta = olapTable.getIndexMetaByIndexId(indexIds.get(i));
+            currentIndexMeta.setSchema(indexSchema);
+
+            int currentSchemaVersion = currentIndexMeta.getSchemaVersion();
+            int newSchemaVersion = currentSchemaVersion + 1;
+            currentIndexMeta.setSchemaVersion(newSchemaVersion);
+        }
+        olapTable.setIndexes(indexes);
+        olapTable.rebuildFullSchema();
+
+        //update max column unique id
+        int maxColUniqueId = olapTable.getMaxColUniqueId();
+        for (Column column : indexSchemaMap.get(olapTable.getBaseIndexId())) {
+            if (column.getUniqueId() > maxColUniqueId) {
+                maxColUniqueId = column.getUniqueId();
+            }
+        }
+        olapTable.setMaxColUniqueId(maxColUniqueId);
+
+        if (!isReplay) {
+            TableAddOrDropColumnsInfo info = new TableAddOrDropColumnsInfo(db.getId(), olapTable.getId(),
+                    indexSchemaMap, indexes, jobId);
+            LOG.debug("logModifyTableAddOrDropColumns info:{}", info);
+            Catalog.getCurrentCatalog().getEditLog().logModifyTableAddOrDropColumns(info);
+        }
+
+        //for compatibility, we need create a finished state schema change job v2
+
+        SchemaChangeJobV2 schemaChangeJob = new SchemaChangeJobV2(jobId, db.getId(), olapTable.getId(),
+                olapTable.getName(), 1000);
+        schemaChangeJob.setJobState(AlterJobV2.JobState.FINISHED);
+        schemaChangeJob.setFinishedTimeMs(System.currentTimeMillis());
+        this.addAlterJobV2(schemaChangeJob);
+
+        LOG.info("finished modify table's add or drop columns. table: {}, is replay: {}", olapTable.getName(),
+                isReplay);
+    }
+
+    public void replayModifyTableAddOrDropColumns(TableAddOrDropColumnsInfo info) throws MetaNotFoundException {
+        LOG.debug("info:{}", info);
+        long dbId = info.getDbId();
+        long tableId = info.getTableId();
+        Map<Long, LinkedList<Column>> indexSchemaMap = info.getIndexSchemaMap();
+        List<Index> indexes = info.getIndexes();
+        long jobId = info.getJobId();
+
+        Database db =  Catalog.getCurrentCatalog().getInternalDataSource().getDbOrMetaException(dbId);
+        OlapTable olapTable = (OlapTable) db.getTableOrMetaException(tableId, TableType.OLAP);
+        olapTable.writeLock();
+        try {
+            modifyTableAddOrDropColumns(db, olapTable, indexSchemaMap, indexes, jobId, true);
+        } catch (DdlException e) {
+            // should not happen
+            LOG.warn("failed to replay modify table add or drop columns", e);
+        } finally {
+            olapTable.writeUnlock();
+        }
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java
index 1c112d6d28..44ff3cdfae 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java
@@ -435,7 +435,7 @@ public class SchemaChangeJobV2 extends AlterJobV2 {
                     long originIdxId = indexIdMap.get(shadowIdxId);
                     int shadowSchemaHash = indexSchemaVersionAndHashMap.get(shadowIdxId).schemaHash;
                     int originSchemaHash = tbl.getSchemaHashByIndexId(indexIdMap.get(shadowIdxId));
-
+                    List<Column> originSchemaColumns = tbl.getSchemaByIndexId(originIdxId);
                     for (Tablet shadowTablet : shadowIdx.getTablets()) {
                         long shadowTabletId = shadowTablet.getId();
                         long originTabletId = partitionIndexTabletMap.get(partitionId, shadowIdxId).get(shadowTabletId);
@@ -444,7 +444,7 @@ public class SchemaChangeJobV2 extends AlterJobV2 {
                             AlterReplicaTask rollupTask = new AlterReplicaTask(shadowReplica.getBackendId(), dbId,
                                     tableId, partitionId, shadowIdxId, originIdxId, shadowTabletId, originTabletId,
                                     shadowReplica.getId(), shadowSchemaHash, originSchemaHash, visibleVersion, jobId,
-                                    JobType.SCHEMA_CHANGE, defineExprs, descTable);
+                                    JobType.SCHEMA_CHANGE, defineExprs, descTable, originSchemaColumns);
                             schemaChangeBatchTask.addTask(rollupTask);
                         }
                     }
@@ -641,6 +641,16 @@ public class SchemaChangeJobV2 extends AlterJobV2 {
             tbl.setStorageFormat(storageFormat);
         }
 
+        //update max column unique id
+        int maxColUniqueId = tbl.getMaxColUniqueId();
+        for (Column column : tbl.getFullSchema()) {
+            if (column.getUniqueId() > maxColUniqueId) {
+                maxColUniqueId = column.getUniqueId();
+            }
+        }
+        tbl.setMaxColUniqueId(maxColUniqueId);
+        LOG.debug("fullSchema:{}, maxColUniqueId:{}", tbl.getFullSchema(), maxColUniqueId);
+
         tbl.setState(OlapTableState.NORMAL);
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ColumnDef.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ColumnDef.java
index b1b471c603..75b3290030 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ColumnDef.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ColumnDef.java
@@ -400,7 +400,7 @@ public class ColumnDef {
 
     public Column toColumn() {
         return new Column(name, typeDef.getType(), isKey, aggregateType, isAllowNull, defaultValue.value, comment,
-                visible, defaultValue.defaultValueExprDef);
+                visible, defaultValue.defaultValueExprDef, Column.COLUMN_UNIQUE_ID_INIT_VALUE);
     }
 
     @Override
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/SlotDescriptor.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/SlotDescriptor.java
index 4f041dfe01..8c405011ae 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SlotDescriptor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/SlotDescriptor.java
@@ -29,11 +29,14 @@ import org.apache.doris.thrift.TSlotDescriptor;
 import com.google.common.base.MoreObjects;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 
 import java.util.Collections;
 import java.util.List;
 
 public class SlotDescriptor {
+    private static final Logger LOG = LogManager.getLogger(SlotDescriptor.class);
     private final SlotId id;
     private final TupleDescriptor parent;
     private Type type;
@@ -278,27 +281,26 @@ public class SlotDescriptor {
 
     // TODO
     public TSlotDescriptor toThrift() {
-        if (originType != null) {
-            return new TSlotDescriptor(id.asInt(), parent.getId().asInt(), originType.toThrift(), -1,
-                    byteOffset, nullIndicatorByte,
-                    nullIndicatorBit, ((column != null) ? column.getName() : ""), slotIdx, isMaterialized);
-        } else {
-            return new TSlotDescriptor(id.asInt(), parent.getId().asInt(), type.toThrift(), -1,
-                    byteOffset, nullIndicatorByte,
-                    nullIndicatorBit, ((column != null) ? column.getName() : ""), slotIdx, isMaterialized);
+
+        TSlotDescriptor tSlotDescriptor = new TSlotDescriptor(id.asInt(), parent.getId().asInt(),
+                (originType != null ? originType.toThrift() : type.toThrift()), -1, byteOffset, nullIndicatorByte,
+                nullIndicatorBit, ((column != null) ? column.getName() : ""), slotIdx, isMaterialized);
+
+        if (column != null) {
+            LOG.debug("column name:{}, column unique id:{}", column.getName(), column.getUniqueId());
+            tSlotDescriptor.setColUniqueId(column.getUniqueId());
         }
+        return tSlotDescriptor;
     }
 
     public String debugString() {
         String colStr = (column == null ? "null" : column.getName());
         String typeStr = (type == null ? "null" : type.toString());
         String parentTupleId = (parent == null) ? "null" : parent.getId().toString();
-        return MoreObjects.toStringHelper(this).add("id", id.asInt()).add("parent", parentTupleId)
-                .add("col", colStr).add("type", typeStr).add("materialized", isMaterialized)
-                .add("byteSize", byteSize).add("byteOffset", byteOffset)
-                .add("nullIndicatorByte", nullIndicatorByte)
-                .add("nullIndicatorBit", nullIndicatorBit)
-                .add("slotIdx", slotIdx).toString();
+        return MoreObjects.toStringHelper(this).add("id", id.asInt()).add("parent", parentTupleId).add("col", colStr)
+                .add("type", typeStr).add("materialized", isMaterialized).add("byteSize", byteSize)
+                .add("byteOffset", byteOffset).add("nullIndicatorByte", nullIndicatorByte)
+                .add("nullIndicatorBit", nullIndicatorBit).add("slotIdx", slotIdx).toString();
     }
 
     @Override
@@ -311,10 +313,8 @@ public class SlotDescriptor {
         String colStr = (column == null ? "null" : column.getName());
         String typeStr = (type == null ? "null" : type.toString());
         String parentTupleId = (parent == null) ? "null" : parent.getId().toString();
-        builder.append(prefix).append("SlotDescriptor{")
-            .append("id=").append(id)
-            .append(", col=").append(colStr)
-            .append(", type=").append(typeStr).append("}\n");
+        builder.append(prefix).append("SlotDescriptor{").append("id=").append(id).append(", col=").append(colStr)
+                .append(", type=").append(typeStr).append("}\n");
 
         prefix += "  ";
         builder.append(prefix).append("parent=").append(parentTupleId).append("\n");
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/SlotRef.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/SlotRef.java
index 6a0976404b..0bd5470ed0 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SlotRef.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/SlotRef.java
@@ -71,7 +71,7 @@ public class SlotRef extends Expr {
     public SlotRef(SlotDescriptor desc) {
         super();
         this.tblName = null;
-        this.col = null;
+        this.col = desc.getColumn() != null ? desc.getColumn().getName() : null;
         this.desc = desc;
         this.type = desc.getType();
         // TODO(zc): label is meaningful
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java
index 4020ffa72f..add27b1430 100755
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Catalog.java
@@ -71,6 +71,7 @@ import org.apache.doris.analysis.ReplacePartitionClause;
 import org.apache.doris.analysis.RestoreStmt;
 import org.apache.doris.analysis.RollupRenameClause;
 import org.apache.doris.analysis.ShowAlterStmt.AlterType;
+import org.apache.doris.analysis.TableName;
 import org.apache.doris.analysis.TableRenameClause;
 import org.apache.doris.analysis.TruncateTableStmt;
 import org.apache.doris.analysis.UninstallPluginStmt;
@@ -251,6 +252,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Optional;
 import java.util.Set;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentHashMap;
@@ -567,21 +569,19 @@ public class Catalog {
         this.metaContext.setThreadLocalInfo();
 
         this.stat = new TabletSchedulerStat();
-        this.tabletScheduler = new TabletScheduler(this, systemInfo,
-                tabletInvertedIndex, stat, Config.tablet_rebalancer_type);
+        this.tabletScheduler = new TabletScheduler(this, systemInfo, tabletInvertedIndex, stat,
+                Config.tablet_rebalancer_type);
         this.tabletChecker = new TabletChecker(this, systemInfo, tabletScheduler, stat);
 
         // The pendingLoadTaskScheduler's queue size should not less than Config.desired_max_waiting_jobs.
         // So that we can guarantee that all submitted load jobs can be scheduled without being starved.
         this.pendingLoadTaskScheduler = new MasterTaskExecutor("pending_load_task_scheduler",
-                Config.async_pending_load_task_pool_size,
-                Config.desired_max_waiting_jobs, !isCheckpointCatalog);
+                Config.async_pending_load_task_pool_size, Config.desired_max_waiting_jobs, !isCheckpointCatalog);
         // The loadingLoadTaskScheduler's queue size is unlimited, so that it can receive all loading tasks
         // created after pending tasks finish. And don't worry about the high concurrency, because the
         // concurrency is limited by Config.desired_max_waiting_jobs and Config.async_loading_load_task_pool_size.
         this.loadingLoadTaskScheduler = new MasterTaskExecutor("loading_load_task_scheduler",
-                Config.async_loading_load_task_pool_size,
-                Integer.MAX_VALUE, !isCheckpointCatalog);
+                Config.async_loading_load_task_pool_size, Integer.MAX_VALUE, !isCheckpointCatalog);
 
         this.loadJobScheduler = new LoadJobScheduler();
         this.loadManager = new LoadManager(loadJobScheduler);
@@ -792,8 +792,8 @@ public class Catalog {
         // 1. check and create dirs and files
         File meta = new File(metaDir);
         if (!meta.exists()) {
-            LOG.warn("Doris' meta dir {} does not exist."
-                    + " You need to create it before starting FE", meta.getAbsolutePath());
+            LOG.warn("Doris' meta dir {} does not exist." + " You need to create it before starting FE",
+                    meta.getAbsolutePath());
             throw new Exception(meta.getAbsolutePath() + " does not exist, will exit");
         }
 
@@ -866,8 +866,7 @@ public class Catalog {
             }
 
             // check file integrity, if has.
-            if ((roleFile.exists() && !versionFile.exists())
-                    || (!roleFile.exists() && versionFile.exists())) {
+            if ((roleFile.exists() && !versionFile.exists()) || (!roleFile.exists() && versionFile.exists())) {
                 throw new IOException("role file and version file must both exist or both not exist. "
                         + "please specific one helper node to recover. will exit.");
             }
@@ -914,12 +913,12 @@ public class Catalog {
                     // But is metadata_failure_recovery is true,
                     // we will not check it because this may be a FE migration.
                     String[] split = nodeName.split("_");
-                    if (Config.metadata_failure_recovery.equals("false")
-                            && !selfNode.first.equalsIgnoreCase(split[0])) {
-                        throw new IOException("the self host " + selfNode.first
-                                + " does not equal to the host in ROLE"
-                                + " file " + split[0] + ". You need to set 'priority_networks' config"
-                                + " in fe.conf to match the host " + split[0]);
+                    if (Config.metadata_failure_recovery.equals("false") && !selfNode.first.equalsIgnoreCase(
+                            split[0])) {
+                        throw new IOException(
+                                "the self host " + selfNode.first + " does not equal to the host in ROLE" + " file "
+                                        + split[0] + ". You need to set 'priority_networks' config"
+                                        + " in fe.conf to match the host " + split[0]);
                     }
                 }
             }
@@ -988,8 +987,8 @@ public class Catalog {
             if (!versionFile.exists()) {
                 // If the version file doesn't exist, download it from helper node
                 if (!getVersionFileFromHelper(rightHelperNode)) {
-                    throw new IOException("fail to download version file from "
-                            + rightHelperNode.first + " will exit.");
+                    throw new IOException(
+                            "fail to download version file from " + rightHelperNode.first + " will exit.");
                 }
 
                 // NOTE: cluster_id will be init when Storage object is constructed,
@@ -1028,8 +1027,8 @@ public class Catalog {
                         Preconditions.checkNotNull(token);
                         Preconditions.checkNotNull(remoteToken);
                         if (!token.equals(remoteToken)) {
-                            throw new IOException("token is not equal with helper node "
-                                    + rightHelperNode.first + ". will exit.");
+                            throw new IOException(
+                                    "token is not equal with helper node " + rightHelperNode.first + ". will exit.");
                         }
                     }
                 } catch (Exception e) {
@@ -1051,8 +1050,7 @@ public class Catalog {
         }
 
         Preconditions.checkState(helperNodes.size() == 1);
-        LOG.info("finished to get cluster id: {}, role: {} and node name: {}",
-                clusterId, role.name(), nodeName);
+        LOG.info("finished to get cluster id: {}, role: {} and node name: {}", clusterId, role.name(), nodeName);
     }
 
     public static String genFeNodeName(String host, int port, boolean isOldStyle) {
@@ -1072,13 +1070,13 @@ public class Catalog {
         Pair<String, Integer> rightHelperNode = null;
         for (Pair<String, Integer> helperNode : helperNodes) {
             try {
-                URL url = new URL("http://" + helperNode.first + ":" + Config.http_port
-                        + "/role?host=" + selfNode.first + "&port=" + selfNode.second);
+                URL url = new URL("http://" + helperNode.first + ":" + Config.http_port + "/role?host=" + selfNode.first
+                        + "&port=" + selfNode.second);
                 HttpURLConnection conn = null;
                 conn = (HttpURLConnection) url.openConnection();
                 if (conn.getResponseCode() != 200) {
-                    LOG.warn("failed to get fe node type from helper node: {}. response code: {}",
-                            helperNode, conn.getResponseCode());
+                    LOG.warn("failed to get fe node type from helper node: {}. response code: {}", helperNode,
+                            conn.getResponseCode());
                     continue;
                 }
 
@@ -1166,9 +1164,8 @@ public class Catalog {
                          * In this case, some errors have caused users to be troubled.
                          * So here directly exit the program and inform the user to avoid unnecessary trouble.
                          */
-                        throw new AnalysisException(
-                                "Do not specify the helper node to FE itself. "
-                                        + "Please specify it to the existing running Master or Follower FE");
+                        throw new AnalysisException("Do not specify the helper node to FE itself. "
+                                + "Please specify it to the existing running Master or Follower FE");
                     }
                     helperNodes.add(helperHostPort);
                 }
@@ -1188,8 +1185,7 @@ public class Catalog {
         // 1. check if this is the first time to start up
         File roleFile = new File(this.imageDir, Storage.ROLE_FILE);
         File versionFile = new File(this.imageDir, Storage.VERSION_FILE);
-        if ((roleFile.exists() && !versionFile.exists())
-                || (!roleFile.exists() && versionFile.exists())) {
+        if ((roleFile.exists() && !versionFile.exists()) || (!roleFile.exists() && versionFile.exists())) {
             throw new Exception("role file and version file must both exist or both not exist. "
                     + "please specific one helper node to recover. will exit.");
         }
@@ -1459,7 +1455,8 @@ public class Catalog {
         if (Config.lower_case_table_names != GlobalVariable.lowerCaseTableNames) {
             LOG.error("The configuration of \'lower_case_table_names\' does not support modification, "
                             + "the expected value is {}, but the actual value is {}",
-                    GlobalVariable.lowerCaseTableNames, Config.lower_case_table_names);
+                    GlobalVariable.lowerCaseTableNames,
+                    Config.lower_case_table_names);
             System.exit(-1);
         }
         LOG.info("lower_case_table_names is {}", GlobalVariable.lowerCaseTableNames);
@@ -1513,8 +1510,7 @@ public class Catalog {
             StorageInfo info = getStorageInfo(infoUrl);
             long version = info.getImageSeq();
             if (version > localImageVersion) {
-                String url = "http://" + helperNode.first + ":" + Config.http_port
-                        + "/image?version=" + version;
+                String url = "http://" + helperNode.first + ":" + Config.http_port + "/image?version=" + version;
                 String filename = Storage.IMAGE + "." + version;
                 File dir = new File(this.imageDir);
                 MetaHelper.getRemoteFile(url, HTTP_TIMEOUT_SECOND * 1000, MetaHelper.getOutputStream(filename, dir));
@@ -1554,8 +1550,8 @@ public class Catalog {
             connection.setReadTimeout(HTTP_TIMEOUT_SECOND * 1000);
 
             String response;
-            try (BufferedReader bufferedReader
-                    = new BufferedReader(new InputStreamReader(connection.getInputStream()))) {
+            try (BufferedReader bufferedReader = new BufferedReader(
+                    new InputStreamReader(connection.getInputStream()))) {
                 String line;
                 StringBuilder sb = new StringBuilder();
                 while ((line = bufferedReader.readLine()) != null) {
@@ -2030,8 +2026,8 @@ public class Catalog {
 
     public long saveExportJob(CountingDataOutputStream dos, long checksum) throws IOException {
         long curTime = System.currentTimeMillis();
-        List<ExportJob> jobs = exportMgr.getJobs().stream()
-                .filter(t -> !t.isExpired(curTime)).collect(Collectors.toList());
+        List<ExportJob> jobs = exportMgr.getJobs().stream().filter(t -> !t.isExpired(curTime))
+                .collect(Collectors.toList());
         int size = jobs.size();
         checksum ^= size;
         dos.writeInt(size);
@@ -2617,11 +2613,11 @@ public class Catalog {
      * 4. set table id and base index id
      * 5. set bloom filter columns
      * 6. set and build TableProperty includes:
-     *     6.1. dynamicProperty
-     *     6.2. replicationNum
-     *     6.3. inMemory
-     *     6.4. storageFormat
-     *     6.5. compressionType
+     * 6.1. dynamicProperty
+     * 6.2. replicationNum
+     * 6.3. inMemory
+     * 6.4. storageFormat
+     * 6.5. compressionType
      * 7. set index meta
      * 8. check colocation properties
      * 9. create tablet in BE
@@ -2692,10 +2688,8 @@ public class Catalog {
 
         // 1.2 other table type
         sb.append("CREATE ");
-        if (table.getType() == TableType.ODBC
-                || table.getType() == TableType.MYSQL
-                || table.getType() == TableType.ELASTICSEARCH
-                || table.getType() == TableType.BROKER
+        if (table.getType() == TableType.ODBC || table.getType() == TableType.MYSQL
+                || table.getType() == TableType.ELASTICSEARCH || table.getType() == TableType.BROKER
                 || table.getType() == TableType.HIVE) {
             sb.append("EXTERNAL ");
         }
@@ -2834,8 +2828,7 @@ public class Catalog {
                     Preconditions.checkState(partitionId.size() == 1);
                     partition = olapTable.getPartition(partitionId.get(0));
                 }
-                sb.append(partition.getVisibleVersion())
-                        .append("\"");
+                sb.append(partition.getVisibleVersion()).append("\"");
             }
 
             // colocateTable
@@ -2897,8 +2890,8 @@ public class Catalog {
                 sb.append("\"password\" = \"").append(hidePassword ? "" : mysqlTable.getPasswd()).append("\",\n");
                 sb.append("\"charset\" = \"").append(mysqlTable.getCharset()).append("\",\n");
             } else {
-                sb.append("\"odbc_catalog_resource\" = \"")
-                        .append(mysqlTable.getOdbcCatalogResourceName()).append("\",\n");
+                sb.append("\"odbc_catalog_resource\" = \"").append(mysqlTable.getOdbcCatalogResourceName())
+                        .append("\",\n");
             }
             sb.append("\"database\" = \"").append(mysqlTable.getMysqlDatabaseName()).append("\",\n");
             sb.append("\"table\" = \"").append(mysqlTable.getMysqlTableName()).append("\"\n");
@@ -2918,8 +2911,8 @@ public class Catalog {
                 sb.append("\"driver\" = \"").append(odbcTable.getOdbcDriver()).append("\",\n");
                 sb.append("\"odbc_type\" = \"").append(odbcTable.getOdbcTableTypeName()).append("\",\n");
             } else {
-                sb.append("\"odbc_catalog_resource\" = \"")
-                        .append(odbcTable.getOdbcCatalogResourceName()).append("\",\n");
+                sb.append("\"odbc_catalog_resource\" = \"").append(odbcTable.getOdbcCatalogResourceName())
+                        .append("\",\n");
             }
             sb.append("\"database\" = \"").append(odbcTable.getOdbcDatabaseName()).append("\",\n");
             sb.append("\"table\" = \"").append(odbcTable.getOdbcTableName()).append("\"\n");
@@ -3038,8 +3031,7 @@ public class Catalog {
                         sb.append(")");
                     }
                     sb.append("(\"version_info\" = \"");
-                    sb.append(partition.getVisibleVersion())
-                            .append("\"");
+                    sb.append(partition.getVisibleVersion()).append("\"");
                     sb.append(");");
                     addPartitionStmt.add(sb.toString());
                 }
@@ -3160,8 +3152,7 @@ public class Catalog {
                 LOG.error(frontend.toString() + " does not exist.");
                 return;
             }
-            if (removedFe.getRole() == FrontendNodeType.FOLLOWER
-                    || removedFe.getRole() == FrontendNodeType.REPLICA) {
+            if (removedFe.getRole() == FrontendNodeType.FOLLOWER || removedFe.getRole() == FrontendNodeType.REPLICA) {
                 helperNodes.remove(Pair.create(removedFe.getHost(), removedFe.getEditLogPort()));
             }
 
@@ -3217,8 +3208,8 @@ public class Catalog {
                     for (Partition partition : olapTable.getAllPartitions()) {
                         long partitionId = partition.getId();
                         DataProperty dataProperty = partitionInfo.getDataProperty(partition.getId());
-                        Preconditions.checkNotNull(dataProperty, partition.getName()
-                                + ", pId:" + partitionId + ", db: " + dbId + ", tbl: " + tableId);
+                        Preconditions.checkNotNull(dataProperty,
+                                partition.getName() + ", pId:" + partitionId + ", db: " + dbId + ", tbl: " + tableId);
                         if (dataProperty.getStorageMedium() == TStorageMedium.SSD
                                 && dataProperty.getCooldownTimeMs() < currentTimeMs) {
                             // expire. change to HDD.
@@ -3257,8 +3248,8 @@ public class Catalog {
                 // use try lock to avoid blocking a long time.
                 // if block too long, backend report rpc will timeout.
                 if (!olapTable.tryWriteLockIfExist(Table.TRY_LOCK_TIMEOUT_MS, TimeUnit.MILLISECONDS)) {
-                    LOG.warn("try get table {} writelock but failed"
-                            + " when checking backend storage medium", table.getName());
+                    LOG.warn("try get table {} writelock but failed" + " when checking backend storage medium",
+                            table.getName());
                     continue;
                 }
                 Preconditions.checkState(olapTable.isWriteLockHeldByCurrentThread());
@@ -3278,16 +3269,13 @@ public class Catalog {
                             DataProperty hddProperty = new DataProperty(TStorageMedium.HDD);
                             partitionInfo.setDataProperty(partition.getId(), hddProperty);
                             storageMediumMap.put(partitionId, TStorageMedium.HDD);
-                            LOG.debug("partition[{}-{}-{}] storage medium changed from SSD to HDD",
-                                    dbId, tableId, partitionId);
+                            LOG.debug("partition[{}-{}-{}] storage medium changed from SSD to HDD", dbId, tableId,
+                                    partitionId);
 
                             // log
-                            ModifyPartitionInfo info =
-                                    new ModifyPartitionInfo(db.getId(), olapTable.getId(),
-                                            partition.getId(),
-                                            hddProperty,
-                                            ReplicaAllocation.NOT_SET,
-                                            partitionInfo.getIsInMemory(partition.getId()));
+                            ModifyPartitionInfo info = new ModifyPartitionInfo(db.getId(), olapTable.getId(),
+                                    partition.getId(), hddProperty, ReplicaAllocation.NOT_SET,
+                                    partitionInfo.getIsInMemory(partition.getId()));
                             editLog.logModifyPartition(info);
                         }
                     } // end for partitions
@@ -3721,15 +3709,15 @@ public class Catalog {
                         if (bucketsNum == -1) {
                             bucketsNum = partition.getDistributionInfo().getBucketNum();
                         } else if (bucketsNum != partition.getDistributionInfo().getBucketNum()) {
-                            throw new DdlException("Partitions in table " + table.getName()
-                                    + " have different buckets number");
+                            throw new DdlException(
+                                    "Partitions in table " + table.getName() + " have different buckets number");
                         }
 
                         if (replicaAlloc == null) {
                             replicaAlloc = partitionInfo.getReplicaAllocation(partition.getId());
                         } else if (!replicaAlloc.equals(partitionInfo.getReplicaAllocation(partition.getId()))) {
-                            throw new DdlException("Partitions in table " + table.getName()
-                                    + " have different replica allocation.");
+                            throw new DdlException(
+                                    "Partitions in table " + table.getName() + " have different replica allocation.");
                         }
                     }
                 }
@@ -3778,8 +3766,7 @@ public class Catalog {
             TablePropertyInfo info = new TablePropertyInfo(table.getId(), groupId, properties);
             editLog.logModifyTableColocate(info);
         }
-        LOG.info("finished modify table's colocation property. table: {}, is replay: {}",
-                table.getName(), isReplay);
+        LOG.info("finished modify table's colocation property. table: {}, is replay: {}", table.getName(), isReplay);
     }
 
     public void replayModifyTableColocate(TablePropertyInfo info) throws MetaNotFoundException {
@@ -3871,8 +3858,8 @@ public class Catalog {
 
             if (table.getPartitionInfo().getType() != PartitionType.RANGE
                     && table.getPartitionInfo().getType() != PartitionType.LIST) {
-                throw new DdlException("Table[" + table.getName() + "] is single partitioned. "
-                        + "no need to rename partition name.");
+                throw new DdlException(
+                        "Table[" + table.getName() + "] is single partitioned. " + "no need to rename partition name.");
             }
 
             String partitionName = renameClause.getPartitionName();
@@ -3940,17 +3927,17 @@ public class Catalog {
             // Merge the new properties with origin properties, and then analyze them
             Map<String, String> origDynamicProperties = tableProperty.getOriginDynamicPartitionProperty();
             origDynamicProperties.putAll(properties);
-            Map<String, String> analyzedDynamicPartition = DynamicPartitionUtil
-                    .analyzeDynamicPartition(origDynamicProperties, table.getPartitionInfo());
+            Map<String, String> analyzedDynamicPartition = DynamicPartitionUtil.analyzeDynamicPartition(
+                    origDynamicProperties, table.getPartitionInfo());
             tableProperty.modifyTableProperties(analyzedDynamicPartition);
             tableProperty.buildDynamicProperty();
         }
 
         DynamicPartitionUtil.registerOrRemoveDynamicPartitionTable(db.getId(), table, false);
-        dynamicPartitionScheduler.createOrUpdateRuntimeInfo(
-                table.getId(), DynamicPartitionScheduler.LAST_UPDATE_TIME, TimeUtils.getCurrentFormatTime());
-        ModifyTablePropertyOperationLog info
-                = new ModifyTablePropertyOperationLog(db.getId(), table.getId(), logProperties);
+        dynamicPartitionScheduler.createOrUpdateRuntimeInfo(table.getId(), DynamicPartitionScheduler.LAST_UPDATE_TIME,
+                TimeUtils.getCurrentFormatTime());
+        ModifyTablePropertyOperationLog info = new ModifyTablePropertyOperationLog(db.getId(), table.getId(),
+                logProperties);
         editLog.logDynamicPartition(info);
     }
 
@@ -3964,22 +3951,24 @@ public class Catalog {
 
     /**
      * Set replication number for unpartitioned table.
+     *
      * @param db
      * @param table
      * @param properties
      * @throws DdlException
      */
     // The caller need to hold the table write lock
-    public void modifyTableReplicaAllocation(Database db, OlapTable table,
-            Map<String, String> properties) throws UserException {
+    public void modifyTableReplicaAllocation(Database db, OlapTable table, Map<String, String> properties)
+            throws UserException {
         Preconditions.checkArgument(table.isWriteLockHeldByCurrentThread());
         String defaultReplicationNumName = "default." + PropertyAnalyzer.PROPERTIES_REPLICATION_NUM;
         PartitionInfo partitionInfo = table.getPartitionInfo();
         if (partitionInfo.getType() == PartitionType.RANGE || partitionInfo.getType() == PartitionType.LIST) {
-            throw new DdlException("This is a partitioned table, you should specify partitions"
-                    + " with MODIFY PARTITION clause."
-                    + " If you want to set default replication number, please use '" + defaultReplicationNumName
-                    + "' instead of '" + PropertyAnalyzer.PROPERTIES_REPLICATION_NUM + "' to escape misleading.");
+            throw new DdlException(
+                    "This is a partitioned table, you should specify partitions" + " with MODIFY PARTITION clause."
+                            + " If you want to set default replication number, please use '" + defaultReplicationNumName
+                            + "' instead of '" + PropertyAnalyzer.PROPERTIES_REPLICATION_NUM
+                            + "' to escape misleading.");
         }
         String partitionName = table.getName();
         Partition partition = table.getPartition(partitionName);
@@ -3997,13 +3986,14 @@ public class Catalog {
         ModifyPartitionInfo info = new ModifyPartitionInfo(db.getId(), table.getId(), partition.getId(),
                 newDataProperty, replicaAlloc, isInMemory);
         editLog.logModifyPartition(info);
-        LOG.debug("modify partition[{}-{}-{}] replica allocation to {}", db.getId(), table.getId(),
-                partition.getName(), replicaAlloc.toCreateStmt());
+        LOG.debug("modify partition[{}-{}-{}] replica allocation to {}", db.getId(), table.getId(), partition.getName(),
+                replicaAlloc.toCreateStmt());
     }
 
     /**
      * Set default replication allocation for a specified table.
      * You can see the default replication allocation by executing Show Create Table stmt.
+     *
      * @param db
      * @param table
      * @param properties
@@ -4022,8 +4012,8 @@ public class Catalog {
         tableProperty.buildReplicaAllocation();
 
         // log
-        ModifyTablePropertyOperationLog info
-                = new ModifyTablePropertyOperationLog(db.getId(), table.getId(), properties);
+        ModifyTablePropertyOperationLog info = new ModifyTablePropertyOperationLog(db.getId(), table.getId(),
+                properties);
         editLog.logModifyReplicationNum(info);
         LOG.debug("modify table[{}] replication num to {}", table.getName(),
                 properties.get(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM));
@@ -4045,13 +4035,13 @@ public class Catalog {
             table.getPartitionInfo().setIsInMemory(partition.getId(), tableProperty.isInMemory());
         }
 
-        ModifyTablePropertyOperationLog info
-                = new ModifyTablePropertyOperationLog(db.getId(), table.getId(), properties);
+        ModifyTablePropertyOperationLog info = new ModifyTablePropertyOperationLog(db.getId(), table.getId(),
+                properties);
         editLog.logModifyInMemory(info);
     }
 
-    public void replayModifyTableProperty(short opCode,
-            ModifyTablePropertyOperationLog info) throws MetaNotFoundException {
+    public void replayModifyTableProperty(short opCode, ModifyTablePropertyOperationLog info)
+            throws MetaNotFoundException {
         long dbId = info.getDbId();
         long tableId = info.getTableId();
         Map<String, String> properties = info.getProperties();
@@ -4099,17 +4089,18 @@ public class Catalog {
                 DistributionInfo distributionInfo = distributionDesc.toDistributionInfo(baseSchema);
                 // for now. we only support modify distribution's bucket num
                 if (distributionInfo.getType() != defaultDistributionInfo.getType()) {
-                    throw new DdlException("Cannot change distribution type when modify"
-                            + " default distribution bucket num");
+                    throw new DdlException(
+                            "Cannot change distribution type when modify" + " default distribution bucket num");
                 }
                 if (distributionInfo.getType() == DistributionInfoType.HASH) {
                     HashDistributionInfo hashDistributionInfo = (HashDistributionInfo) distributionInfo;
                     List<Column> newDistriCols = hashDistributionInfo.getDistributionColumns();
-                    List<Column> defaultDistriCols = ((HashDistributionInfo) defaultDistributionInfo)
-                            .getDistributionColumns();
+                    List<Column> defaultDistriCols
+                            = ((HashDistributionInfo) defaultDistributionInfo).getDistributionColumns();
                     if (!newDistriCols.equals(defaultDistriCols)) {
-                        throw new DdlException("Cannot assign hash distribution with different distribution cols. "
-                                + "default is: " + defaultDistriCols);
+                        throw new DdlException(
+                                "Cannot assign hash distribution with different distribution cols. " + "default is: "
+                                        + defaultDistriCols);
                     }
                 }
 
@@ -4163,8 +4154,8 @@ public class Catalog {
     // Switch catalog of this sesseion.
     public void changeCatalog(ConnectContext ctx, String catalogName) throws DdlException {
         if (dataSourceMgr.getCatalogNullable(catalogName) == null) {
-            throw new DdlException(ErrorCode.ERR_UNKNOWN_CATALOG.formatErrorMsg(
-                    catalogName), ErrorCode.ERR_UNKNOWN_CATALOG);
+            throw new DdlException(ErrorCode.ERR_UNKNOWN_CATALOG.formatErrorMsg(catalogName),
+                    ErrorCode.ERR_UNKNOWN_CATALOG);
         }
         ctx.changeDefaultCatalog(catalogName);
     }
@@ -4596,8 +4587,8 @@ public class Catalog {
                 Replica replica = tablet.getReplicaById(info.getReplicaId());
                 if (replica != null) {
                     replica.setBad(true);
-                    LOG.debug("get replica {} of tablet {} on backend {} to bad when replaying",
-                            info.getReplicaId(), info.getTabletId(), info.getBackendId());
+                    LOG.debug("get replica {} of tablet {} on backend {} to bad when replaying", info.getReplicaId(),
+                            info.getTabletId(), info.getBackendId());
                 }
             } finally {
                 olapTable.writeUnlock();
@@ -4674,8 +4665,8 @@ public class Catalog {
         ReplacePartitionOperationLog info = new ReplacePartitionOperationLog(db.getId(), olapTable.getId(),
                 partitionNames, tempPartitionNames, isStrictRange, useTempPartitionName);
         editLog.logReplaceTempPartition(info);
-        LOG.info("finished to replace partitions {} with temp partitions {} from table: {}",
-                clause.getPartitionNames(), clause.getTempPartitionNames(), olapTable.getName());
+        LOG.info("finished to replace partitions {} with temp partitions {} from table: {}", clause.getPartitionNames(),
+                clause.getTempPartitionNames(), olapTable.getName());
     }
 
     public void replayReplaceTempPartition(ReplacePartitionOperationLog replaceTempPartitionLog)
@@ -4687,8 +4678,7 @@ public class Catalog {
         olapTable.writeLock();
         try {
             olapTable.replaceTempPartitions(replaceTempPartitionLog.getPartitions(),
-                    replaceTempPartitionLog.getTempPartitions(),
-                    replaceTempPartitionLog.isStrictRange(),
+                    replaceTempPartitionLog.getTempPartitions(), replaceTempPartitionLog.isStrictRange(),
                     replaceTempPartitionLog.useTempPartitionName());
         } catch (DdlException e) {
             throw new MetaNotFoundException(e);
@@ -4764,8 +4754,8 @@ public class Catalog {
         setReplicaStatusInternal(log.getTabletId(), log.getBackendId(), log.getReplicaStatus(), true);
     }
 
-    private void setReplicaStatusInternal(long tabletId, long backendId,
-            ReplicaStatus status, boolean isReplay) throws MetaNotFoundException {
+    private void setReplicaStatusInternal(long tabletId, long backendId, ReplicaStatus status, boolean isReplay)
+            throws MetaNotFoundException {
         try {
             TabletMeta meta = tabletInvertedIndex.getTabletMeta(tabletId);
             if (meta == null) {
@@ -4782,12 +4772,12 @@ public class Catalog {
                 if (status == ReplicaStatus.BAD || status == ReplicaStatus.OK) {
                     if (replica.setBad(status == ReplicaStatus.BAD)) {
                         if (!isReplay) {
-                            SetReplicaStatusOperationLog log
-                                    = new SetReplicaStatusOperationLog(backendId, tabletId, status);
+                            SetReplicaStatusOperationLog log = new SetReplicaStatusOperationLog(backendId, tabletId,
+                                    status);
                             getEditLog().logSetReplicaStatus(log);
                         }
-                        LOG.info("set replica {} of tablet {} on backend {} as {}. is replay: {}",
-                                replica.getId(), tabletId, backendId, status, isReplay);
+                        LOG.info("set replica {} of tablet {} on backend {} as {}. is replay: {}", replica.getId(),
+                                tabletId, backendId, status, isReplay);
                     }
                 }
             } finally {
@@ -4944,4 +4934,16 @@ public class Catalog {
         }
         return count;
     }
+
+    public TableName getTableNameByTableId(Long tableId) {
+        for (String dbName : getInternalDataSource().getDbNames()) {
+            DatabaseIf db = getInternalDataSource().getDbNullable(dbName);
+            Optional<Table> table = db.getTable(tableId);
+            if (table.isPresent()) {
+                return new TableName(InternalDataSource.INTERNAL_DS_NAME,
+                                    db.getFullName(), table.get().getName());
+            }
+        }
+        return null;
+    }
 }
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 3badbcab4b..25cd47f628 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
@@ -20,6 +20,7 @@ package org.apache.doris.catalog;
 import org.apache.doris.alter.SchemaChangeHandler;
 import org.apache.doris.analysis.DefaultValueExprDef;
 import org.apache.doris.analysis.Expr;
+import org.apache.doris.analysis.IndexDef;
 import org.apache.doris.analysis.SlotRef;
 import org.apache.doris.analysis.StringLiteral;
 import org.apache.doris.common.AnalysisException;
@@ -55,6 +56,7 @@ public class Column implements Writable {
     public static final String DELETE_SIGN = "__DORIS_DELETE_SIGN__";
     public static final String SEQUENCE_COL = "__DORIS_SEQUENCE_COL__";
     private static final String COLUMN_ARRAY_CHILDREN = "item";
+    public static final int COLUMN_UNIQUE_ID_INIT_VALUE = -1;
 
     @SerializedName(value = "name")
     private String name;
@@ -93,6 +95,9 @@ public class Column implements Writable {
     @SerializedName(value = "defaultValueExprDef")
     private DefaultValueExprDef defaultValueExprDef; // used for default value
 
+    @SerializedName(value = "uniqueId")
+    private int uniqueId;
+
     public Column() {
         this.name = "";
         this.type = Type.NULL;
@@ -102,6 +107,7 @@ public class Column implements Writable {
         this.visible = true;
         this.defineExpr = null;
         this.children = new ArrayList<>(Type.MAX_NESTING_DEPTH);
+        this.uniqueId = -1;
     }
 
     public Column(String name, PrimitiveType dataType) {
@@ -123,11 +129,13 @@ public class Column implements Writable {
 
     public Column(String name, Type type, boolean isKey, AggregateType aggregateType, boolean isAllowNull,
             String defaultValue, String comment) {
-        this(name, type, isKey, aggregateType, isAllowNull, defaultValue, comment, true, null);
+        this(name, type, isKey, aggregateType, isAllowNull, defaultValue, comment, true, null,
+                COLUMN_UNIQUE_ID_INIT_VALUE);
     }
 
     public Column(String name, Type type, boolean isKey, AggregateType aggregateType, boolean isAllowNull,
-            String defaultValue, String comment, boolean visible, DefaultValueExprDef defaultValueExprDef) {
+            String defaultValue, String comment, boolean visible, DefaultValueExprDef defaultValueExprDef,
+            int colUniqueId) {
         this.name = name;
         if (this.name == null) {
             this.name = "";
@@ -149,6 +157,7 @@ public class Column implements Writable {
         this.visible = visible;
         this.children = new ArrayList<>(Type.MAX_NESTING_DEPTH);
         createChildrenColumn(this.type, this);
+        this.uniqueId = colUniqueId;
     }
 
     public Column(Column column) {
@@ -164,6 +173,7 @@ public class Column implements Writable {
         this.stats = column.getStats();
         this.visible = column.visible;
         this.children = column.getChildren();
+        this.uniqueId = column.getUniqueId();
     }
 
     public void createChildrenColumn(Type type, Column column) {
@@ -366,6 +376,7 @@ public class Column implements Writable {
         tColumn.setVisible(visible);
         toChildrenThrift(this, tColumn);
 
+        tColumn.setColUniqueId(uniqueId);
         // ATTN:
         // Currently, this `toThrift()` method is only used from CreateReplicaTask.
         // And CreateReplicaTask does not need `defineExpr` field.
@@ -688,4 +699,23 @@ public class Column implements Writable {
         sb.append(defaultValue == null ? "" : defaultValue);
         return sb.toString();
     }
+
+    public void setUniqueId(int colUniqueId) {
+        this.uniqueId = colUniqueId;
+    }
+
+    public int getUniqueId() {
+        return this.uniqueId;
+    }
+
+    public void setIndexFlag(TColumn tColumn, List<Index> indexes) {
+        for (Index index : indexes) {
+            if (index.getIndexType() == IndexDef.IndexType.BITMAP) {
+                List<String> columns = index.getColumns();
+                if (tColumn.getColumnName().equals(columns.get(0))) {
+                    tColumn.setHasBitmapIndex(true);
+                }
+            }
+        }
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java
index ede7c32a17..8e600c5991 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/MaterializedIndexMeta.java
@@ -104,6 +104,14 @@ public class MaterializedIndexMeta implements Writable, GsonPostProcessable {
         }
     }
 
+    public void setSchema(List<Column> newSchema) {
+        this.schema = newSchema;
+    }
+
+    public void setSchemaHash(int newSchemaHash) {
+        this.schemaHash = newSchemaHash;
+    }
+
     public int getSchemaHash() {
         return schemaHash;
     }
@@ -112,6 +120,10 @@ public class MaterializedIndexMeta implements Writable, GsonPostProcessable {
         return shortKeyColumnCount;
     }
 
+    public void setSchemaVersion(int newSchemaVersion) {
+        this.schemaVersion = newSchemaVersion;
+    }
+
     public int getSchemaVersion() {
         return schemaVersion;
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java
index 80ab60064d..9e23ebe374 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java
@@ -38,6 +38,7 @@ import org.apache.doris.clone.TabletScheduler;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.common.FeConstants;
+import org.apache.doris.common.FeMetaVersion;
 import org.apache.doris.common.Pair;
 import org.apache.doris.common.UserException;
 import org.apache.doris.common.io.DeepCopy;
@@ -141,6 +142,8 @@ public class OlapTable extends Table {
 
     private TableProperty tableProperty;
 
+    private int maxColUniqueId = Column.COLUMN_UNIQUE_ID_INIT_VALUE;
+
     public OlapTable() {
         // for persist
         super(TableType.OLAP);
@@ -191,6 +194,20 @@ public class OlapTable extends Table {
         return this.tableProperty;
     }
 
+    //take care: only use at create olap table.
+    public int incAndGetMaxColUniqueId() {
+        this.maxColUniqueId++;
+        return this.maxColUniqueId;
+    }
+
+    public int getMaxColUniqueId() {
+        return this.maxColUniqueId;
+    }
+
+    public void setMaxColUniqueId(int maxColUniqueId) {
+        this.maxColUniqueId = maxColUniqueId;
+    }
+
     public boolean dynamicPartitionExists() {
         return tableProperty != null
                 && tableProperty.getDynamicPartitionProperty() != null
@@ -1144,6 +1161,7 @@ public class OlapTable extends Table {
         }
 
         tempPartitions.write(out);
+        out.writeInt(maxColUniqueId);
     }
 
     @Override
@@ -1235,6 +1253,10 @@ public class OlapTable extends Table {
             }
         }
         tempPartitions.unsetPartitionInfo();
+
+        if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_112) {
+            maxColUniqueId = in.readInt();
+        }
         // In the present, the fullSchema could be rebuilt by schema change while the properties is changed by MV.
         // After that, some properties of fullSchema and nameToColumn may be not same as properties of base columns.
         // So, here we need to rebuild the fullSchema to ensure the correctness of the properties.
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java
index 124c453261..887ae74c59 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Table.java
@@ -81,7 +81,7 @@ public class Table extends MetaObject implements Writable, TableIf {
      * Schema change (c3 to bigint)
      * When OlapTable is changing schema, the fullSchema is (c1 int, c2 int, c3 int, SHADOW_NAME_PRFIX_c3 bigint)
      * The fullSchema of OlapTable is mainly used by Scanner of Load job.
-     *
+     * <p>
      * If you want to get the mv columns, you should call getIndexToSchema in Subclass OlapTable.
      */
     protected List<Column> fullSchema;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalDataSource.java b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalDataSource.java
index 332fa85078..634bca7079 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalDataSource.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalDataSource.java
@@ -340,8 +340,8 @@ public class InternalDataSource implements DataSourceIf<Database> {
                 Collection<Partition> allPartitions = olapTable.getAllPartitions();
                 for (Partition partition : allPartitions) {
                     long partitionId = partition.getId();
-                    TStorageMedium medium =
-                            olapTable.getPartitionInfo().getDataProperty(partitionId).getStorageMedium();
+                    TStorageMedium medium = olapTable.getPartitionInfo().getDataProperty(partitionId)
+                            .getStorageMedium();
                     for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.ALL)) {
                         long indexId = index.getId();
                         int schemaHash = olapTable.getSchemaHashByIndexId(indexId);
@@ -475,8 +475,8 @@ public class InternalDataSource implements DataSourceIf<Database> {
                     fullNameToDb.remove(db.getAttachDb());
                     db.setDbState(DbState.NORMAL);
                     info.setUpdateDbState(DbState.NORMAL);
-                    final Cluster cluster =
-                            nameToCluster.get(ClusterNamespace.getClusterNameFromFullName(db.getAttachDb()));
+                    final Cluster cluster = nameToCluster.get(
+                            ClusterNamespace.getClusterNameFromFullName(db.getAttachDb()));
                     final BaseParam param = new BaseParam();
                     param.addStringParam(db.getAttachDb());
                     param.addLongParam(db.getId());
@@ -845,8 +845,7 @@ public class InternalDataSource implements DataSourceIf<Database> {
                     OlapTable olapTable = (OlapTable) table;
                     if ((olapTable.getState() != OlapTableState.NORMAL)) {
                         throw new DdlException("The table [" + tableName + "]'s state is " + olapTable.getState()
-                                + ", cannot be dropped."
-                                + " please cancel the operation on olap table firstly."
+                                + ", cannot be dropped." + " please cancel the operation on olap table firstly."
                                 + " If you want to forcibly drop(cannot be recovered),"
                                 + " please use \"DROP table FORCE\".");
                     }
@@ -925,10 +924,9 @@ public class InternalDataSource implements DataSourceIf<Database> {
             schemaHash = olapTable.getSchemaHashByIndexId(info.getIndexId());
         }
 
-        Replica replica =
-                new Replica(info.getReplicaId(), info.getBackendId(), info.getVersion(), schemaHash, info.getDataSize(),
-                        info.getRowCount(), ReplicaState.NORMAL, info.getLastFailedVersion(),
-                        info.getLastSuccessVersion());
+        Replica replica = new Replica(info.getReplicaId(), info.getBackendId(), info.getVersion(), schemaHash,
+                info.getDataSize(), info.getRowCount(), ReplicaState.NORMAL, info.getLastFailedVersion(),
+                info.getLastSuccessVersion());
         tablet.addReplica(replica);
     }
 
@@ -1091,8 +1089,8 @@ public class InternalDataSource implements DataSourceIf<Database> {
             } finally {
                 table.readUnlock();
             }
-            CreateTableStmt parsedCreateTableStmt =
-                    (CreateTableStmt) SqlParserUtils.parseAndAnalyzeStmt(createTableStmt.get(0), ConnectContext.get());
+            CreateTableStmt parsedCreateTableStmt = (CreateTableStmt) SqlParserUtils.parseAndAnalyzeStmt(
+                    createTableStmt.get(0), ConnectContext.get());
             parsedCreateTableStmt.setTableName(stmt.getTableName());
             parsedCreateTableStmt.setIfNotExists(stmt.isIfNotExists());
             createTable(parsedCreateTableStmt);
@@ -1178,8 +1176,8 @@ public class InternalDataSource implements DataSourceIf<Database> {
                 long tableId = table.getId();
                 for (Partition partition : olapTable.getAllPartitions()) {
                     long partitionId = partition.getId();
-                    TStorageMedium medium =
-                            olapTable.getPartitionInfo().getDataProperty(partitionId).getStorageMedium();
+                    TStorageMedium medium = olapTable.getPartitionInfo().getDataProperty(partitionId)
+                            .getStorageMedium();
                     for (MaterializedIndex mIndex : partition.getMaterializedIndices(IndexExtState.ALL)) {
                         long indexId = mIndex.getId();
                         int schemaHash = olapTable.getSchemaHashByIndexId(indexId);
@@ -1263,8 +1261,8 @@ public class InternalDataSource implements DataSourceIf<Database> {
                 if (distributionInfo.getType() == DistributionInfoType.HASH) {
                     HashDistributionInfo hashDistributionInfo = (HashDistributionInfo) distributionInfo;
                     List<Column> newDistriCols = hashDistributionInfo.getDistributionColumns();
-                    List<Column> defaultDistriCols =
-                            ((HashDistributionInfo) defaultDistributionInfo).getDistributionColumns();
+                    List<Column> defaultDistriCols
+                            = ((HashDistributionInfo) defaultDistributionInfo).getDistributionColumns();
                     if (!newDistriCols.equals(defaultDistriCols)) {
                         throw new DdlException(
                                 "Cannot assign hash distribution with different distribution cols. " + "default is: "
@@ -1437,9 +1435,8 @@ public class InternalDataSource implements DataSourceIf<Database> {
                 for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.ALL)) {
                     long indexId = index.getId();
                     int schemaHash = olapTable.getSchemaHashByIndexId(indexId);
-                    TabletMeta tabletMeta =
-                            new TabletMeta(info.getDbId(), info.getTableId(), partition.getId(), index.getId(),
-                                    schemaHash, info.getDataProperty().getStorageMedium());
+                    TabletMeta tabletMeta = new TabletMeta(info.getDbId(), info.getTableId(), partition.getId(),
+                            index.getId(), schemaHash, info.getDataProperty().getStorageMedium());
                     for (Tablet tablet : index.getTablets()) {
                         long tabletId = tablet.getId();
                         invertedIndex.addTablet(tabletId, tabletMeta);
@@ -1599,11 +1596,10 @@ public class InternalDataSource implements DataSourceIf<Database> {
                     long backendId = replica.getBackendId();
                     long replicaId = replica.getId();
                     countDownLatch.addMark(backendId, tabletId);
-                    CreateReplicaTask task =
-                            new CreateReplicaTask(backendId, dbId, tableId, partitionId, indexId, tabletId, replicaId,
-                                    shortKeyColumnCount, schemaHash, version, keysType, storageType, storageMedium,
-                                    schema, bfColumns, bfFpp, countDownLatch, indexes, isInMemory, tabletType,
-                                    dataSortInfo, compressionType);
+                    CreateReplicaTask task = new CreateReplicaTask(backendId, dbId, tableId, partitionId, indexId,
+                            tabletId, replicaId, shortKeyColumnCount, schemaHash, version, keysType, storageType,
+                            storageMedium, schema, bfColumns, bfFpp, countDownLatch, indexes, isInMemory, tabletType,
+                            dataSortInfo, compressionType);
                     task.setStorageFormat(storageFormat);
                     batchTask.addTask(task);
                     // add to AgentTaskQueue for handling finish report.
@@ -1700,9 +1696,15 @@ public class InternalDataSource implements DataSourceIf<Database> {
 
         // create table
         long tableId = Catalog.getCurrentCatalog().getNextId();
-        OlapTable olapTable =
-                new OlapTable(tableId, tableName, baseSchema, keysType, partitionInfo, defaultDistributionInfo,
-                        indexes);
+        OlapTable olapTable = new OlapTable(tableId, tableName, baseSchema, keysType, partitionInfo,
+                defaultDistributionInfo, indexes);
+
+        for (Column column : baseSchema) {
+            column.setUniqueId(olapTable.incAndGetMaxColUniqueId());
+            LOG.debug("table: {}, newColumn: {}, uniqueId: {}", olapTable.getName(), column.getName(),
+                    column.getUniqueId());
+        }
+
         olapTable.setComment(stmt.getComment());
 
         // set base index id
@@ -1732,8 +1734,8 @@ public class InternalDataSource implements DataSourceIf<Database> {
         olapTable.setCompressionType(compressionType);
 
         // check data sort properties
-        DataSortInfo dataSortInfo =
-                PropertyAnalyzer.analyzeDataSortInfo(properties, keysType, keysDesc.keysColumnSize(), storageFormat);
+        DataSortInfo dataSortInfo = PropertyAnalyzer.analyzeDataSortInfo(properties, keysType,
+                keysDesc.keysColumnSize(), storageFormat);
         olapTable.setDataSortInfo(dataSortInfo);
 
         // analyze bloom filter columns
@@ -1765,8 +1767,8 @@ public class InternalDataSource implements DataSourceIf<Database> {
         olapTable.setReplicationAllocation(replicaAlloc);
 
         // set in memory
-        boolean isInMemory =
-                PropertyAnalyzer.analyzeBooleanProp(properties, PropertyAnalyzer.PROPERTIES_INMEMORY, false);
+        boolean isInMemory = PropertyAnalyzer.analyzeBooleanProp(properties, PropertyAnalyzer.PROPERTIES_INMEMORY,
+                false);
         olapTable.setIsInMemory(isInMemory);
 
         // set remote storage
@@ -1789,8 +1791,8 @@ public class InternalDataSource implements DataSourceIf<Database> {
             long partitionId = partitionNameToId.get(tableName);
             DataProperty dataProperty = null;
             try {
-                dataProperty =
-                        PropertyAnalyzer.analyzeDataProperty(stmt.getProperties(), DataProperty.DEFAULT_DATA_PROPERTY);
+                dataProperty = PropertyAnalyzer.analyzeDataProperty(stmt.getProperties(),
+                        DataProperty.DEFAULT_DATA_PROPERTY);
             } catch (AnalysisException e) {
                 throw new DdlException(e.getMessage());
             }
@@ -1858,8 +1860,8 @@ public class InternalDataSource implements DataSourceIf<Database> {
             // set rollup index meta to olap table
             List<Column> rollupColumns = Catalog.getCurrentCatalog().getMaterializedViewHandler()
                     .checkAndPrepareMaterializedView(addRollupClause, olapTable, baseRollupIndex, false);
-            short rollupShortKeyColumnCount =
-                    Catalog.calcShortKeyColumnCount(rollupColumns, alterClause.getProperties());
+            short rollupShortKeyColumnCount = Catalog.calcShortKeyColumnCount(rollupColumns,
+                    alterClause.getProperties());
             int rollupSchemaHash = Util.generateSchemaHash();
             long rollupIndexId = Catalog.getCurrentCatalog().getNextId();
             olapTable.setIndexMeta(rollupIndexId, addRollupClause.getRollupName(), rollupColumns, schemaVersion,
@@ -1991,10 +1993,10 @@ public class InternalDataSource implements DataSourceIf<Database> {
                 // we have added these index to memory, only need to persist here
                 if (Catalog.getCurrentColocateIndex().isColocateTable(tableId)) {
                     GroupId groupId = Catalog.getCurrentColocateIndex().getGroup(tableId);
-                    Map<Tag, List<List<Long>>> backendsPerBucketSeq =
-                            Catalog.getCurrentColocateIndex().getBackendsPerBucketSeq(groupId);
-                    ColocatePersistInfo info =
-                            ColocatePersistInfo.createForAddTable(groupId, tableId, backendsPerBucketSeq);
+                    Map<Tag, List<List<Long>>> backendsPerBucketSeq = Catalog.getCurrentColocateIndex()
+                            .getBackendsPerBucketSeq(groupId);
+                    ColocatePersistInfo info = ColocatePersistInfo.createForAddTable(groupId, tableId,
+                            backendsPerBucketSeq);
                     Catalog.getCurrentCatalog().getEditLog().logColocateAddTable(info);
                 }
                 LOG.info("successfully create table[{};{}]", tableName, tableId);
@@ -2111,8 +2113,8 @@ public class InternalDataSource implements DataSourceIf<Database> {
         HiveTable hiveTable = new HiveTable(tableId, tableName, columns, stmt.getProperties());
         hiveTable.setComment(stmt.getComment());
         // check hive table whether exists in hive database
-        HiveMetaStoreClient hiveMetaStoreClient =
-                HiveMetaStoreClientHelper.getClient(hiveTable.getHiveProperties().get(HiveTable.HIVE_METASTORE_URIS));
+        HiveMetaStoreClient hiveMetaStoreClient = HiveMetaStoreClientHelper.getClient(
+                hiveTable.getHiveProperties().get(HiveTable.HIVE_METASTORE_URIS));
         if (!HiveMetaStoreClientHelper.tableExists(hiveMetaStoreClient, hiveTable.getHiveDb(),
                 hiveTable.getHiveTable())) {
             throw new DdlException(String.format("Table [%s] dose not exist in Hive.", hiveTable.getHiveDbTable()));
@@ -2140,9 +2142,8 @@ public class InternalDataSource implements DataSourceIf<Database> {
             throw new DdlException(
                     String.format("Table [%s] dose not exist in Hive Metastore.", hudiTable.getHmsTableIdentifer()));
         }
-        org.apache.hadoop.hive.metastore.api.Table hiveTable =
-                HiveMetaStoreClientHelper.getTable(hudiTable.getHmsDatabaseName(), hudiTable.getHmsTableName(),
-                        metastoreUris);
+        org.apache.hadoop.hive.metastore.api.Table hiveTable = HiveMetaStoreClientHelper.getTable(
+                hudiTable.getHmsDatabaseName(), hudiTable.getHmsTableName(), metastoreUris);
         if (!HudiUtils.isHudiTable(hiveTable)) {
             throw new DdlException(String.format("Table [%s] is not a hudi table.", hudiTable.getHmsTableIdentifer()));
         }
@@ -2229,8 +2230,8 @@ public class InternalDataSource implements DataSourceIf<Database> {
             for (List<Long> backendIds : chosenBackendIds.values()) {
                 for (long backendId : backendIds) {
                     long replicaId = Catalog.getCurrentCatalog().getNextId();
-                    Replica replica =
-                            new Replica(replicaId, backendId, replicaState, version, tabletMeta.getOldSchemaHash());
+                    Replica replica = new Replica(replicaId, backendId, replicaState, version,
+                            tabletMeta.getOldSchemaHash());
                     tablet.addReplica(replica);
                     totalReplicaNum++;
                 }
@@ -2403,8 +2404,8 @@ public class InternalDataSource implements DataSourceIf<Database> {
             truncateTableInternal(olapTable, newPartitions, truncateEntireTable);
 
             // write edit log
-            TruncateTableInfo info =
-                    new TruncateTableInfo(db.getId(), olapTable.getId(), newPartitions, truncateEntireTable);
+            TruncateTableInfo info = new TruncateTableInfo(db.getId(), olapTable.getId(), newPartitions,
+                    truncateEntireTable);
             Catalog.getCurrentCatalog().getEditLog().logTruncateTable(info);
         } finally {
             olapTable.writeUnlock();
@@ -2449,13 +2450,13 @@ public class InternalDataSource implements DataSourceIf<Database> {
                 TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex();
                 for (Partition partition : info.getPartitions()) {
                     long partitionId = partition.getId();
-                    TStorageMedium medium =
-                            olapTable.getPartitionInfo().getDataProperty(partitionId).getStorageMedium();
+                    TStorageMedium medium = olapTable.getPartitionInfo().getDataProperty(partitionId)
+                            .getStorageMedium();
                     for (MaterializedIndex mIndex : partition.getMaterializedIndices(IndexExtState.ALL)) {
                         long indexId = mIndex.getId();
                         int schemaHash = olapTable.getSchemaHashByIndexId(indexId);
-                        TabletMeta tabletMeta =
-                                new TabletMeta(db.getId(), olapTable.getId(), partitionId, indexId, schemaHash, medium);
+                        TabletMeta tabletMeta = new TabletMeta(db.getId(), olapTable.getId(), partitionId, indexId,
+                                schemaHash, medium);
                         for (Tablet tablet : mIndex.getTablets()) {
                             long tabletId = tablet.getId();
                             invertedIndex.addTablet(tabletId, tabletMeta);
@@ -2508,8 +2509,8 @@ public class InternalDataSource implements DataSourceIf<Database> {
             if (nameToCluster.containsKey(clusterName)) {
                 ErrorReport.reportDdlException(ErrorCode.ERR_CLUSTER_HAS_EXIST, clusterName);
             } else {
-                List<Long> backendList =
-                        Catalog.getCurrentSystemInfo().createCluster(clusterName, stmt.getInstanceNum());
+                List<Long> backendList = Catalog.getCurrentSystemInfo()
+                        .createCluster(clusterName, stmt.getInstanceNum());
                 // 1: BE returned is less than requested, throws DdlException.
                 // 2: BE returned is more than or equal to 0, succeeds.
                 if (backendList != null || stmt.getInstanceNum() == 0) {
@@ -2967,8 +2968,8 @@ public class InternalDataSource implements DataSourceIf<Database> {
                 if (db.getDbState() == DbState.MOVE) {
                     int tabletTotal = 0;
                     int tabletQuorum = 0;
-                    final Set<Long> beIds =
-                            Sets.newHashSet(Catalog.getCurrentSystemInfo().getClusterBackendIds(db.getClusterName()));
+                    final Set<Long> beIds = Sets.newHashSet(
+                            Catalog.getCurrentSystemInfo().getClusterBackendIds(db.getClusterName()));
                     final Set<String> tableNames = db.getTableNamesWithLock();
                     for (String tableName : tableNames) {
 
@@ -2981,8 +2982,8 @@ public class InternalDataSource implements DataSourceIf<Database> {
                         olapTable.readLock();
                         try {
                             for (Partition partition : olapTable.getPartitions()) {
-                                ReplicaAllocation replicaAlloc =
-                                        olapTable.getPartitionInfo().getReplicaAllocation(partition.getId());
+                                ReplicaAllocation replicaAlloc = olapTable.getPartitionInfo()
+                                        .getReplicaAllocation(partition.getId());
                                 short totalReplicaNum = replicaAlloc.getTotalReplicaNum();
                                 for (MaterializedIndex materializedIndex : partition.getMaterializedIndices(
                                         IndexExtState.ALL)) {
@@ -3071,8 +3072,8 @@ public class InternalDataSource implements DataSourceIf<Database> {
 
     public void initDefaultCluster() {
         final List<Long> backendList = Lists.newArrayList();
-        final List<Backend> defaultClusterBackends =
-                Catalog.getCurrentSystemInfo().getClusterBackends(SystemInfoService.DEFAULT_CLUSTER);
+        final List<Backend> defaultClusterBackends = Catalog.getCurrentSystemInfo()
+                .getClusterBackends(SystemInfoService.DEFAULT_CLUSTER);
         for (Backend backend : defaultClusterBackends) {
             backendList.add(backend.getId());
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java b/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java
index 78d200b9f7..9803f83cc3 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java
@@ -87,6 +87,7 @@ import org.apache.doris.persist.ReplaceTableOperationLog;
 import org.apache.doris.persist.ReplicaPersistInfo;
 import org.apache.doris.persist.RoutineLoadOperation;
 import org.apache.doris.persist.SetReplicaStatusOperationLog;
+import org.apache.doris.persist.TableAddOrDropColumnsInfo;
 import org.apache.doris.persist.TableInfo;
 import org.apache.doris.persist.TablePropertyInfo;
 import org.apache.doris.persist.TruncateTableInfo;
@@ -662,6 +663,11 @@ public class JournalEntity implements Writable {
                 isRead = true;
                 break;
             }
+            case OperationType.OP_MODIFY_TABLE_ADD_OR_DROP_COLUMNS: {
+                data = TableAddOrDropColumnsInfo.read(in);
+                isRead = true;
+                break;
+            }
             default: {
                 IOException e = new IOException();
                 LOG.error("UNKNOWN Operation Type {}", opCode, e);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java b/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java
index 649e030a50..0ca0793475 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/DeleteHandler.java
@@ -65,6 +65,7 @@ import org.apache.doris.task.AgentBatchTask;
 import org.apache.doris.task.AgentTaskExecutor;
 import org.apache.doris.task.AgentTaskQueue;
 import org.apache.doris.task.PushTask;
+import org.apache.doris.thrift.TColumn;
 import org.apache.doris.thrift.TPriority;
 import org.apache.doris.thrift.TPushType;
 import org.apache.doris.thrift.TTaskType;
@@ -238,6 +239,11 @@ public class DeleteHandler implements Writable {
                         long indexId = index.getId();
                         int schemaHash = olapTable.getSchemaHashByIndexId(indexId);
 
+                        List<TColumn> columnsDesc = new ArrayList<TColumn>();
+                        for (Column column : olapTable.getSchemaByIndexId(indexId)) {
+                            columnsDesc.add(column.toThrift());
+                        }
+
                         for (Tablet tablet : index.getTablets()) {
                             long tabletId = tablet.getId();
 
@@ -260,7 +266,8 @@ public class DeleteHandler implements Writable {
                                         TTaskType.REALTIME_PUSH,
                                         transactionId,
                                         Catalog.getCurrentGlobalTransactionMgr()
-                                                .getTransactionIDGenerator().getNextTransactionId());
+                                                .getTransactionIDGenerator().getNextTransactionId(),
+                                        columnsDesc);
                                 pushTask.setIsSchemaChanging(false);
                                 pushTask.setCountDownLatch(countDownLatch);
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java b/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java
index 0faf356029..954db1d971 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java
@@ -18,6 +18,7 @@
 package org.apache.doris.load;
 
 import org.apache.doris.catalog.Catalog;
+import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.Database;
 import org.apache.doris.catalog.MaterializedIndex;
 import org.apache.doris.catalog.MaterializedIndex.IndexExtState;
@@ -41,6 +42,7 @@ import org.apache.doris.task.HadoopLoadPendingTask;
 import org.apache.doris.task.MasterTask;
 import org.apache.doris.task.MasterTaskExecutor;
 import org.apache.doris.task.PushTask;
+import org.apache.doris.thrift.TColumn;
 import org.apache.doris.thrift.TPriority;
 import org.apache.doris.thrift.TPushType;
 import org.apache.doris.thrift.TTaskType;
@@ -434,6 +436,10 @@ public class LoadChecker extends MasterDaemon {
                         }
 
                         int schemaHash = tableLoadInfo.getIndexSchemaHash(indexId);
+                        List<TColumn> columnsDesc = new ArrayList<TColumn>();
+                        for (Column column : table.getSchemaByIndexId(indexId)) {
+                            columnsDesc.add(column.toThrift());
+                        }
                         short quorumNum = (short) (replicationNum / 2 + 1);
                         for (Tablet tablet : index.getTablets()) {
                             long tabletId = tablet.getId();
@@ -471,7 +477,7 @@ public class LoadChecker extends MasterDaemon {
                                             job.getId(), type, job.getConditions(), needDecompress, job.getPriority(),
                                             TTaskType.REALTIME_PUSH, job.getTransactionId(),
                                             Catalog.getCurrentGlobalTransactionMgr()
-                                                    .getTransactionIDGenerator().getNextTransactionId());
+                                                    .getTransactionIDGenerator().getNextTransactionId(), columnsDesc);
                                     pushTask.setIsSchemaChanging(autoLoadToTwoTablet);
                                     if (AgentTaskQueue.addTask(pushTask)) {
                                         batchTask.addTask(pushTask);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java
index b9ec64ea7f..59a03c5753 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java
@@ -73,6 +73,7 @@ import org.apache.doris.task.PushTask;
 import org.apache.doris.thrift.TBrokerRangeDesc;
 import org.apache.doris.thrift.TBrokerScanRange;
 import org.apache.doris.thrift.TBrokerScanRangeParams;
+import org.apache.doris.thrift.TColumn;
 import org.apache.doris.thrift.TDescriptorTable;
 import org.apache.doris.thrift.TFileFormatType;
 import org.apache.doris.thrift.TFileType;
@@ -101,6 +102,7 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.File;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -109,7 +111,7 @@ import java.util.Set;
  * There are 4 steps in SparkLoadJob:
  * Step1: SparkLoadPendingTask will be created by unprotectedExecuteJob method and submit spark etl job.
  * Step2: LoadEtlChecker will check spark etl job status periodically
- *        and send push tasks to be when spark etl job is finished.
+ * and send push tasks to be when spark etl job is finished.
  * Step3: LoadLoadingChecker will check loading status periodically and commit transaction when push tasks are finished.
  * Step4: PublishVersionDaemon will send publish version tasks to be and finish transaction.
  */
@@ -149,9 +151,8 @@ public class SparkLoadJob extends BulkLoadJob {
         super(EtlJobType.SPARK);
     }
 
-    public SparkLoadJob(long dbId, String label, ResourceDesc resourceDesc,
-                        OriginStatement originStmt, UserIdentity userInfo)
-            throws MetaNotFoundException {
+    public SparkLoadJob(long dbId, String label, ResourceDesc resourceDesc, OriginStatement originStmt,
+            UserIdentity userInfo) throws MetaNotFoundException {
         super(EtlJobType.SPARK, dbId, label, originStmt, userInfo);
         this.resourceDesc = resourceDesc;
     }
@@ -166,6 +167,7 @@ public class SparkLoadJob extends BulkLoadJob {
 
     /**
      * merge system conf with load stmt
+     *
      * @throws DdlException
      */
     private void setResourceInfo() throws DdlException {
@@ -210,8 +212,8 @@ public class SparkLoadJob extends BulkLoadJob {
         }
 
         // create pending task
-        LoadTask task = new SparkLoadPendingTask(this, fileGroupAggInfo.getAggKeyToFileGroups(),
-                                                 sparkResource, brokerDesc);
+        LoadTask task = new SparkLoadPendingTask(this, fileGroupAggInfo.getAggKeyToFileGroups(), sparkResource,
+                brokerDesc);
         task.init();
         idToTasks.put(task.getSignature(), task);
         Catalog.getCurrentCatalog().getPendingLoadTaskScheduler().submit(task);
@@ -229,19 +231,16 @@ public class SparkLoadJob extends BulkLoadJob {
         try {
             // check if job has been cancelled
             if (isTxnDone()) {
-                LOG.warn(new LogBuilder(LogKey.LOAD_JOB, id)
-                                 .add("state", state)
-                                 .add("error_msg", "this task will be ignored when job is: " + state)
-                                 .build());
+                LOG.warn(new LogBuilder(LogKey.LOAD_JOB, id).add("state", state)
+                        .add("error_msg", "this task will be ignored when job is: " + state).build());
                 return;
             }
 
             if (finishedTaskIds.contains(attachment.getTaskId())) {
-                LOG.warn(new LogBuilder(LogKey.LOAD_JOB, id)
-                                 .add("task_id", attachment.getTaskId())
-                                 .add("error_msg", "this is a duplicated callback of pending task "
-                                         + "when broker already has loading task")
-                                 .build());
+                LOG.warn(new LogBuilder(LogKey.LOAD_JOB, id).add("task_id", attachment.getTaskId()).add("error_msg",
+                                "this is a duplicated callback of pending task "
+                                + "when broker already has loading task")
+                        .build());
                 return;
             }
 
@@ -286,7 +285,7 @@ public class SparkLoadJob extends BulkLoadJob {
      * 1. RUNNING, update etl job progress
      * 2. CANCELLED, cancel load job
      * 3. FINISHED, get the etl output file paths, update job state to LOADING and log job update info
-     *
+     * <p>
      * Send push tasks if job state changed to LOADING
      */
     public void updateEtlStatus() throws Exception {
@@ -296,8 +295,8 @@ public class SparkLoadJob extends BulkLoadJob {
 
         // get etl status
         SparkEtlJobHandler handler = new SparkEtlJobHandler();
-        EtlStatus status = handler.getEtlJobStatus(sparkLoadAppHandle,
-                appId, id, etlOutputPath, sparkResource, brokerDesc);
+        EtlStatus status = handler.getEtlJobStatus(sparkLoadAppHandle, appId, id, etlOutputPath, sparkResource,
+                brokerDesc);
         writeLock();
         try {
             switch (status.getState()) {
@@ -364,8 +363,8 @@ public class SparkLoadJob extends BulkLoadJob {
         unprotectedPrepareLoadingInfos();
     }
 
-    private void unprotectedUpdateToLoadingState(EtlStatus etlStatus,
-            Map<String, Long> filePathToSize) throws LoadException {
+    private void unprotectedUpdateToLoadingState(EtlStatus etlStatus, Map<String, Long> filePathToSize)
+            throws LoadException {
         try {
             for (Map.Entry<String, Long> entry : filePathToSize.entrySet()) {
                 String filePath = entry.getKey();
@@ -428,8 +427,8 @@ public class SparkLoadJob extends BulkLoadJob {
         AgentBatchTask batchTask = new AgentBatchTask();
         boolean hasLoadPartitions = false;
         Set<Long> totalTablets = Sets.newHashSet();
-        List<? extends TableIf> tableList =
-                db.getTablesOnIdOrderOrThrowException(Lists.newArrayList(tableToLoadPartitions.keySet()));
+        List<? extends TableIf> tableList = db.getTablesOnIdOrderOrThrowException(
+                Lists.newArrayList(tableToLoadPartitions.keySet()));
         MetaLockUtils.readLockTables(tableList);
         try {
             writeLock();
@@ -454,20 +453,26 @@ public class SparkLoadJob extends BulkLoadJob {
                         }
 
                         hasLoadPartitions = true;
-                        int quorumReplicaNum = olapTable.getPartitionInfo()
-                                .getReplicaAllocation(partitionId).getTotalReplicaNum() / 2 + 1;
+                        int quorumReplicaNum =
+                                olapTable.getPartitionInfo().getReplicaAllocation(partitionId).getTotalReplicaNum() / 2
+                                        + 1;
 
                         List<MaterializedIndex> indexes = partition.getMaterializedIndices(IndexExtState.ALL);
                         for (MaterializedIndex index : indexes) {
                             long indexId = index.getId();
                             int schemaHash = indexToSchemaHash.get(indexId);
 
+                            List<TColumn> columnsDesc = new ArrayList<TColumn>();
+                            for (Column column : olapTable.getSchemaByIndexId(indexId)) {
+                                columnsDesc.add(column.toThrift());
+                            }
+
                             int bucket = 0;
                             for (Tablet tablet : index.getTablets()) {
                                 long tabletId = tablet.getId();
                                 totalTablets.add(tabletId);
                                 String tabletMetaStr = String.format("%d.%d.%d.%d.%d", olapTable.getId(), partitionId,
-                                                                     indexId, bucket++, schemaHash);
+                                        indexId, bucket++, schemaHash);
                                 Set<Long> tabletAllReplicas = Sets.newHashSet();
                                 Set<Long> tabletFinishedReplicas = Sets.newHashSet();
                                 for (Replica replica : tablet.getReplicas()) {
@@ -482,8 +487,8 @@ public class SparkLoadJob extends BulkLoadJob {
                                         PushBrokerReaderParams params = getPushBrokerReaderParams(olapTable, indexId);
                                         // deep copy TBrokerScanRange because filePath and fileSize will be updated
                                         // in different tablet push task
-                                        TBrokerScanRange tBrokerScanRange
-                                                = new TBrokerScanRange(params.tBrokerScanRange);
+                                        TBrokerScanRange tBrokerScanRange = new TBrokerScanRange(
+                                                params.tBrokerScanRange);
                                         // update filePath fileSize
                                         TBrokerRangeDesc tBrokerRangeDesc = tBrokerScanRange.getRanges().get(0);
                                         tBrokerRangeDesc.setPath("");
@@ -497,22 +502,21 @@ public class SparkLoadJob extends BulkLoadJob {
                                         // update broker address
                                         Backend backend = Catalog.getCurrentCatalog().getCurrentSystemInfo()
                                                 .getBackend(backendId);
-                                        FsBroker fsBroker = Catalog.getCurrentCatalog().getBrokerMgr().getBroker(
-                                                brokerDesc.getName(), backend.getHost());
-                                        tBrokerScanRange.getBrokerAddresses().add(
-                                                new TNetworkAddress(fsBroker.ip, fsBroker.port));
+                                        FsBroker fsBroker = Catalog.getCurrentCatalog().getBrokerMgr()
+                                                .getBroker(brokerDesc.getName(), backend.getHost());
+                                        tBrokerScanRange.getBrokerAddresses()
+                                                .add(new TNetworkAddress(fsBroker.ip, fsBroker.port));
 
                                         LOG.debug("push task for replica {}, broker {}:{},"
                                                         + " backendId {}, filePath {}, fileSize {}",
-                                                replicaId, fsBroker.ip, fsBroker.port, backendId,
-                                                tBrokerRangeDesc.path,
+                                                replicaId, fsBroker.ip,
+                                                fsBroker.port, backendId, tBrokerRangeDesc.path,
                                                 tBrokerRangeDesc.file_size);
 
-                                        PushTask pushTask = new PushTask(
-                                                backendId, dbId, olapTable.getId(), partitionId, indexId, tabletId,
-                                                replicaId, schemaHash, 0, id, TPushType.LOAD_V2,
-                                                TPriority.NORMAL, transactionId, taskSignature,
-                                                tBrokerScanRange, params.tDescriptorTable);
+                                        PushTask pushTask = new PushTask(backendId, dbId, olapTable.getId(),
+                                                partitionId, indexId, tabletId, replicaId, schemaHash, 0, id,
+                                                TPushType.LOAD_V2, TPriority.NORMAL, transactionId, taskSignature,
+                                                tBrokerScanRange, params.tDescriptorTable, columnsDesc);
                                         if (AgentTaskQueue.addTask(pushTask)) {
                                             batchTask.addTask(pushTask);
                                             if (!tabletToSentReplicaPushTask.containsKey(tabletId)) {
@@ -548,11 +552,8 @@ public class SparkLoadJob extends BulkLoadJob {
                 }
 
                 if (!hasLoadPartitions) {
-                    String errMsg = new LogBuilder(LogKey.LOAD_JOB, id)
-                            .add("database_id", dbId)
-                            .add("label", label)
-                            .add("error_msg", "all partitions have no load data")
-                            .build();
+                    String errMsg = new LogBuilder(LogKey.LOAD_JOB, id).add("database_id", dbId).add("label", label)
+                            .add("error_msg", "all partitions have no load data").build();
                     throw new LoadException(errMsg);
                 }
 
@@ -632,19 +633,16 @@ public class SparkLoadJob extends BulkLoadJob {
     }
 
     private void tryCommitJob() throws UserException {
-        LOG.info(new LogBuilder(LogKey.LOAD_JOB, id)
-                         .add("txn_id", transactionId)
-                         .add("msg", "Load job try to commit txn")
-                         .build());
+        LOG.info(new LogBuilder(LogKey.LOAD_JOB, id).add("txn_id", transactionId)
+                .add("msg", "Load job try to commit txn").build());
         Database db = getDb();
         List<Table> tableList = db.getTablesOnIdOrderOrThrowException(
                 Lists.newArrayList(tableToLoadPartitions.keySet()));
         MetaLockUtils.writeLockTablesOrMetaException(tableList);
         try {
-            Catalog.getCurrentGlobalTransactionMgr().commitTransaction(
-                    dbId, tableList, transactionId, commitInfos,
-                    new LoadJobFinalOperation(id, loadingStatus, progress, loadStartTimestamp,
-                                              finishTimestamp, state, failMsg));
+            Catalog.getCurrentGlobalTransactionMgr().commitTransaction(dbId, tableList, transactionId, commitInfos,
+                    new LoadJobFinalOperation(id, loadingStatus, progress, loadStartTimestamp, finishTimestamp, state,
+                            failMsg));
         } catch (TabletQuorumFailedException e) {
             // retry in next loop
         } finally {
@@ -785,9 +783,8 @@ public class SparkLoadJob extends BulkLoadJob {
      * log load job update info when job state changed to etl or loading
      */
     private void unprotectedLogUpdateStateInfo() {
-        SparkLoadJobStateUpdateInfo info = new SparkLoadJobStateUpdateInfo(
-                id, state, transactionId, sparkLoadAppHandle, etlStartTimestamp, appId, etlOutputPath,
-                loadStartTimestamp, tabletMetaToFileInfo);
+        SparkLoadJobStateUpdateInfo info = new SparkLoadJobStateUpdateInfo(id, state, transactionId, sparkLoadAppHandle,
+                etlStartTimestamp, appId, etlOutputPath, loadStartTimestamp, tabletMetaToFileInfo);
         Catalog.getCurrentCatalog().getEditLog().logUpdateLoadJob(info);
     }
 
@@ -809,8 +806,8 @@ public class SparkLoadJob extends BulkLoadJob {
                 unprotectedPrepareLoadingInfos();
                 break;
             default:
-                LOG.warn("replay update load job state info failed. error: wrong state. job id: {}, state: {}",
-                         id, state);
+                LOG.warn("replay update load job state info failed. error: wrong state. job id: {}, state: {}", id,
+                        state);
                 break;
         }
     }
@@ -830,10 +827,9 @@ public class SparkLoadJob extends BulkLoadJob {
         @SerializedName(value = "tabletMetaToFileInfo")
         private Map<String, Pair<String, Long>> tabletMetaToFileInfo;
 
-        public SparkLoadJobStateUpdateInfo(
-                long jobId, JobState state, long transactionId, SparkLoadAppHandle sparkLoadAppHandle,
-                long etlStartTimestamp, String appId, String etlOutputPath, long loadStartTimestamp,
-                Map<String, Pair<String, Long>> tabletMetaToFileInfo) {
+        public SparkLoadJobStateUpdateInfo(long jobId, JobState state, long transactionId,
+                SparkLoadAppHandle sparkLoadAppHandle, long etlStartTimestamp, String appId, String etlOutputPath,
+                long loadStartTimestamp, Map<String, Pair<String, Long>> tabletMetaToFileInfo) {
             super(jobId, state, transactionId, loadStartTimestamp);
             this.sparkLoadAppHandle = sparkLoadAppHandle;
             this.etlStartTimestamp = etlStartTimestamp;
@@ -867,7 +863,7 @@ public class SparkLoadJob extends BulkLoadJob {
      * Params for be push broker reader
      * 1. TBrokerScanRange: file path and size, broker address, tranform expr
      * 2. TDescriptorTable: src and dest SlotDescriptors, src and dest tupleDescriptors
-     *
+     * <p>
      * These params are sent to Be through push task
      */
     private static class PushBrokerReaderParams {
@@ -896,7 +892,7 @@ public class SparkLoadJob extends BulkLoadJob {
         }
 
         private void initTBrokerScanRange(DescriptorTable descTable, TupleDescriptor destTupleDesc,
-                                          List<Column> columns, BrokerDesc brokerDesc) throws AnalysisException {
+                List<Column> columns, BrokerDesc brokerDesc) throws AnalysisException {
             // scan range params
             TBrokerScanRangeParams params = new TBrokerScanRangeParams();
             params.setStrictMode(false);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java
index 3e0b531970..03bc694ae3 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java
@@ -180,47 +180,45 @@ public class EditLog {
                 }
                 case OperationType.OP_CREATE_TABLE: {
                     CreateTableInfo info = (CreateTableInfo) journal.getData();
-                    LOG.info("Begin to unprotect create table. db = "
-                            + info.getDbName() + " table = " + info.getTable().getId());
+                    LOG.info("Begin to unprotect create table. db = " + info.getDbName() + " table = " + info.getTable()
+                            .getId());
                     catalog.replayCreateTable(info.getDbName(), info.getTable());
                     break;
                 }
                 case OperationType.OP_ALTER_EXTERNAL_TABLE_SCHEMA: {
                     RefreshExternalTableInfo info = (RefreshExternalTableInfo) journal.getData();
-                    LOG.info("Begin to unprotect alter external table schema. db = "
-                            + info.getDbName() + " table = " + info.getTableName());
+                    LOG.info("Begin to unprotect alter external table schema. db = " + info.getDbName() + " table = "
+                            + info.getTableName());
                     catalog.replayAlterExternalTableSchema(info.getDbName(), info.getTableName(), info.getNewSchema());
                     break;
                 }
                 case OperationType.OP_DROP_TABLE: {
                     DropInfo info = (DropInfo) journal.getData();
-                    Database db =
-                            Catalog.getCurrentInternalCatalog().getDbOrMetaException(info.getDbId());
-                    LOG.info("Begin to unprotect drop table. db = "
-                            + db.getFullName() + " table = " + info.getTableId());
+                    Database db = Catalog.getCurrentInternalCatalog().getDbOrMetaException(info.getDbId());
+                    LOG.info("Begin to unprotect drop table. db = " + db.getFullName() + " table = "
+                            + info.getTableId());
                     catalog.replayDropTable(db, info.getTableId(), info.isForceDrop());
                     break;
                 }
                 case OperationType.OP_ADD_PARTITION: {
                     PartitionPersistInfo info = (PartitionPersistInfo) journal.getData();
-                    LOG.info("Begin to unprotect add partition. db = " + info.getDbId()
-                            + " table = " + info.getTableId()
-                            + " partitionName = " + info.getPartition().getName());
+                    LOG.info(
+                            "Begin to unprotect add partition. db = " + info.getDbId() + " table = " + info.getTableId()
+                                    + " partitionName = " + info.getPartition().getName());
                     catalog.replayAddPartition(info);
                     break;
                 }
                 case OperationType.OP_DROP_PARTITION: {
                     DropPartitionInfo info = (DropPartitionInfo) journal.getData();
-                    LOG.info("Begin to unprotect drop partition. db = " + info.getDbId()
-                            + " table = " + info.getTableId()
-                            + " partitionName = " + info.getPartitionName());
+                    LOG.info("Begin to unprotect drop partition. db = " + info.getDbId() + " table = "
+                            + info.getTableId() + " partitionName = " + info.getPartitionName());
                     catalog.replayDropPartition(info);
                     break;
                 }
                 case OperationType.OP_MODIFY_PARTITION: {
                     ModifyPartitionInfo info = (ModifyPartitionInfo) journal.getData();
-                    LOG.info("Begin to unprotect modify partition. db = " + info.getDbId()
-                            + " table = " + info.getTableId() + " partitionId = " + info.getPartitionId());
+                    LOG.info("Begin to unprotect modify partition. db = " + info.getDbId() + " table = "
+                            + info.getTableId() + " partitionId = " + info.getPartitionId());
                     catalog.getAlterInstance().replayModifyPartition(info);
                     break;
                 }
@@ -285,8 +283,9 @@ public class EditLog {
                 case OperationType.OP_BATCH_DROP_ROLLUP: {
                     BatchDropInfo batchDropInfo = (BatchDropInfo) journal.getData();
                     for (long indexId : batchDropInfo.getIndexIdSet()) {
-                        catalog.getMaterializedViewHandler().replayDropRollup(new DropInfo(batchDropInfo.getDbId(),
-                                batchDropInfo.getTableId(), indexId, false), catalog);
+                        catalog.getMaterializedViewHandler().replayDropRollup(
+                                new DropInfo(batchDropInfo.getDbId(), batchDropInfo.getTableId(), indexId, false),
+                                catalog);
                     }
                     break;
                 }
@@ -846,6 +845,11 @@ public class EditLog {
                     catalog.getDataSourceMgr().replayAlterCatalogProps(log);
                     break;
                 }
+                case OperationType.OP_MODIFY_TABLE_ADD_OR_DROP_COLUMNS: {
+                    final TableAddOrDropColumnsInfo info = (TableAddOrDropColumnsInfo) journal.getData();
+                    catalog.getSchemaChangeHandler().replayModifyTableAddOrDropColumns(info);
+                    break;
+                }
                 default: {
                     IOException e = new IOException();
                     LOG.error("UNKNOWN Operation Type {}", opCode, e);
@@ -935,13 +939,13 @@ public class EditLog {
         }
 
         if (LOG.isDebugEnabled()) {
-            LOG.debug("nextId = {}, numTransactions = {}, totalTimeTransactions = {}, op = {}",
-                    txId, numTransactions, totalTimeTransactions, op);
+            LOG.debug("nextId = {}, numTransactions = {}, totalTimeTransactions = {}, op = {}", txId, numTransactions,
+                    totalTimeTransactions, op);
         }
 
         if (txId >= Config.edit_log_roll_num) {
-            LOG.info("txId {} is equal to or larger than edit_log_roll_num {}, will roll edit.",
-                    txId, Config.edit_log_roll_num);
+            LOG.info("txId {} is equal to or larger than edit_log_roll_num {}, will roll edit.", txId,
+                    Config.edit_log_roll_num);
             rollEditLog();
             txId = 0;
         }
@@ -1472,4 +1476,8 @@ public class EditLog {
     public Journal getJournal() {
         return this.journal;
     }
+
+    public void logModifyTableAddOrDropColumns(TableAddOrDropColumnsInfo info) {
+        logEdit(OperationType.OP_MODIFY_TABLE_ADD_OR_DROP_COLUMNS, info);
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java b/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java
index 79d3cf63a0..047714d0f0 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java
@@ -69,6 +69,9 @@ public class OperationType {
     public static final short OP_MODIFY_COMMENT = 126;
     public static final short OP_MODIFY_TABLE_ENGINE = 127;
 
+    //schema change for add and drop columns
+    public static final short OP_MODIFY_TABLE_ADD_OR_DROP_COLUMNS = 128;
+
     // 30~39 130~139 230~239 ...
     // load job for only hadoop load
     public static final short OP_LOAD_START = 30;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/TableAddOrDropColumnsInfo.java b/fe/fe-core/src/main/java/org/apache/doris/persist/TableAddOrDropColumnsInfo.java
new file mode 100644
index 0000000000..6e644f6a3b
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/persist/TableAddOrDropColumnsInfo.java
@@ -0,0 +1,115 @@
+// 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.
+
+package org.apache.doris.persist;
+
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.Index;
+import org.apache.doris.common.io.Text;
+import org.apache.doris.common.io.Writable;
+import org.apache.doris.persist.gson.GsonUtils;
+
+import com.google.gson.annotations.SerializedName;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * PersistInfo for Table properties
+ */
+public class TableAddOrDropColumnsInfo implements Writable {
+    @SerializedName(value = "dbId")
+    private long dbId;
+    @SerializedName(value = "tableId")
+    private long tableId;
+    @SerializedName(value = "indexSchemaMap")
+    private Map<Long, LinkedList<Column>> indexSchemaMap;
+    @SerializedName(value = "indexes")
+    private List<Index> indexes;
+    @SerializedName(value = "jobId")
+    private long jobId;
+
+    public TableAddOrDropColumnsInfo(long dbId, long tableId,
+            Map<Long, LinkedList<Column>> indexSchemaMap, List<Index> indexes, long jobId) {
+        this.dbId = dbId;
+        this.tableId = tableId;
+        this.indexSchemaMap = indexSchemaMap;
+        this.indexes = indexes;
+        this.jobId = jobId;
+    }
+
+    public long getDbId() {
+        return dbId;
+    }
+
+    public long getTableId() {
+        return tableId;
+    }
+
+    public Map<Long, LinkedList<Column>> getIndexSchemaMap() {
+        return indexSchemaMap;
+    }
+
+    public List<Index> getIndexes() {
+        return indexes;
+    }
+
+    public long getJobId() {
+        return jobId;
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+        Text.writeString(out, GsonUtils.GSON.toJson(this));
+    }
+
+    public static TableAddOrDropColumnsInfo read(DataInput in) throws IOException {
+        return GsonUtils.GSON.fromJson(Text.readString(in), TableAddOrDropColumnsInfo.class);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (obj == this) {
+            return true;
+        }
+
+        if (!(obj instanceof TableAddOrDropColumnsInfo)) {
+            return false;
+        }
+
+        TableAddOrDropColumnsInfo info = (TableAddOrDropColumnsInfo) obj;
+
+        return (dbId == info.dbId && tableId == tableId
+                && indexSchemaMap.equals(info.indexSchemaMap) && indexes.equals(info.indexes)
+                && jobId == info.jobId);
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder sb = new StringBuilder();
+        sb.append(" dbId: ").append(dbId);
+        sb.append(" tableId: ").append(tableId);
+        sb.append(" indexSchemaMap: ").append(indexSchemaMap);
+        sb.append(" indexes: ").append(indexes);
+        sb.append(" jobId: ").append(jobId);
+        return sb.toString();
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java
index 7df0cc38de..17f1898437 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java
@@ -56,6 +56,7 @@ import org.apache.doris.resource.Tag;
 import org.apache.doris.statistics.StatisticalType;
 import org.apache.doris.statistics.StatsRecursiveDerive;
 import org.apache.doris.system.Backend;
+import org.apache.doris.thrift.TColumn;
 import org.apache.doris.thrift.TExplainLevel;
 import org.apache.doris.thrift.TNetworkAddress;
 import org.apache.doris.thrift.TOlapScanNode;
@@ -95,8 +96,10 @@ public class OlapScanNode extends ScanNode {
 
     private List<TScanRangeLocations> result = new ArrayList<>();
     /*
-     * When the field value is ON, the storage engine can return the data directly without pre-aggregation.
-     * When the field value is OFF, the storage engine needs to aggregate the data before returning to scan node.
+     * When the field value is ON, the storage engine can return the data directly
+     * without pre-aggregation.
+     * When the field value is OFF, the storage engine needs to aggregate the data
+     * before returning to scan node.
      * For example:
      * Aggregate table: k1, k2, v1 sum
      * Field value is ON
@@ -109,20 +112,26 @@ public class OlapScanNode extends ScanNode {
      * This aggregation info is null.
      * Query2: select k1, min(v1) from table group by k1
      * This aggregation function in query is min which different from the schema.
-     * So the data stored in storage engine need to be merged firstly before returning to scan node.
+     * So the data stored in storage engine need to be merged firstly before
+     * returning to scan node.
      *
      * There are currently two places to modify this variable:
      * 1. The turnOffPreAgg() method of SingleNodePlanner.
-     *      This method will only be called on the left deepest OlapScanNode the plan tree,
-     *      while other nodes are false by default (because the Aggregation operation is executed after Join,
-     *      we cannot judge whether other OlapScanNodes can close the pre-aggregation).
-     *      So even the Duplicate key table, if it is not the left deepest node, it will remain false too.
+     * This method will only be called on the left deepest OlapScanNode the plan
+     * tree,
+     * while other nodes are false by default (because the Aggregation operation is
+     * executed after Join,
+     * we cannot judge whether other OlapScanNodes can close the pre-aggregation).
+     * So even the Duplicate key table, if it is not the left deepest node, it will
+     * remain false too.
      *
-     * 2. After MaterializedViewSelector selects the materialized view, the updateScanRangeInfoByNewMVSelector()\
-     *    method of OlapScanNode may be called to update this variable.
-     *      This call will be executed on all ScanNodes in the plan tree. In this step,
-     *      for the DuplicateKey table, the variable will be set to true.
-     *      See comment of "isPreAggregation" variable in MaterializedViewSelector for details.
+     * 2. After MaterializedViewSelector selects the materialized view, the
+     * updateScanRangeInfoByNewMVSelector()\
+     * method of OlapScanNode may be called to update this variable.
+     * This call will be executed on all ScanNodes in the plan tree. In this step,
+     * for the DuplicateKey table, the variable will be set to true.
+     * See comment of "isPreAggregation" variable in MaterializedViewSelector for
+     * details.
      */
     private boolean isPreAggregation = false;
     private String reasonOfPreAggregation = null;
@@ -142,7 +151,8 @@ public class OlapScanNode extends ScanNode {
     private HashSet<Long> scanBackendIds = new HashSet<>();
 
     private Map<Long, Integer> tabletId2BucketSeq = Maps.newHashMap();
-    // a bucket seq may map to many tablets, and each tablet has a TScanRangeLocations.
+    // a bucket seq may map to many tablets, and each tablet has a
+    // TScanRangeLocations.
     public ArrayListMultimap<Integer, TScanRangeLocations> bucketSeq2locations = ArrayListMultimap.create();
 
     // Constructs node to scan given data files of table 'tbl'.
@@ -203,36 +213,45 @@ public class OlapScanNode extends ScanNode {
     }
 
     /**
-     * The function is used to directly select the index id of the base table as the selectedIndexId.
-     * It makes sure that the olap scan node must scan the base data rather than scan the materialized view data.
+     * The function is used to directly select the index id of the base table as the
+     * selectedIndexId.
+     * It makes sure that the olap scan node must scan the base data rather than
+     * scan the materialized view data.
      *
      * This function is mainly used to update stmt.
      * Update stmt also needs to scan data like normal queries.
      * But its syntax is different from ordinary queries,
-     *   so planner cannot use the logic of query to automatically match the best index id.
-     * So, here it need to manually specify the index id to scan the base table directly.
+     * so planner cannot use the logic of query to automatically match the best
+     * index id.
+     * So, here it need to manually specify the index id to scan the base table
+     * directly.
      */
     public void useBaseIndexId() {
         this.selectedIndexId = olapTable.getBaseIndexId();
     }
 
     /**
-     * This method is mainly used to update scan range info in OlapScanNode by the new materialized selector.
+     * This method is mainly used to update scan range info in OlapScanNode by the
+     * new materialized selector.
      * Situation1:
-     * If the new scan range is same as the old scan range which determined by the old materialized selector,
+     * If the new scan range is same as the old scan range which determined by the
+     * old materialized selector,
      * the scan range will not be changed.
      * <p>
      * Situation2: Scan range is difference. The type of table is duplicated.
      * The new scan range is used directly.
-     * The reason is that the old selector does not support SPJ<->SPJG, so the result of old one must be incorrect.
+     * The reason is that the old selector does not support SPJ<->SPJG, so the
+     * result of old one must be incorrect.
      * <p>
      * Situation3: Scan range is difference. The type of table is aggregated.
      * The new scan range is different from the old one.
      * If the test_materialized_view is set to true, an error will be reported.
      * The query will be cancelled.
      * <p>
-     * Situation4: Scan range is difference. The type of table is aggregated. `test_materialized_view` is set to false.
-     * The result of the old version selector will be selected. Print the warning log
+     * Situation4: Scan range is difference. The type of table is aggregated.
+     * `test_materialized_view` is set to false.
+     * The result of the old version selector will be selected. Print the warning
+     * log
      *
      * @param selectedIndexId
      * @param isPreAggregation
@@ -255,8 +274,7 @@ public class OlapScanNode extends ScanNode {
         String scanRangeInfo = stringBuilder.toString();
         String situation;
         boolean update;
-        CHECK:
-        { // CHECKSTYLE IGNORE THIS LINE
+        CHECK: { // CHECKSTYLE IGNORE THIS LINE
             if (olapTable.getKeysType() == KeysType.DUP_KEYS) {
                 situation = "The key type of table is duplicate.";
                 update = true;
@@ -296,12 +314,14 @@ public class OlapScanNode extends ScanNode {
 
     /**
      * In some situation, the column type between base and mv is different.
-     * If mv selector selects the mv index, the type of column should be changed to the type of mv column.
+     * If mv selector selects the mv index, the type of column should be changed to
+     * the type of mv column.
      * For example:
      * base table: k1 int, k2 int
      * mv table: k1 int, k2 bigint sum
      * The type of `k2` column between base and mv is different.
-     * When mv selector selects the mv table to scan, the type of column should be changed to bigint in here.
+     * When mv selector selects the mv table to scan, the type of column should be
+     * changed to bigint in here.
      * Currently, only `SUM` aggregate type could match this changed.
      */
     private void updateColumnType() {
@@ -346,9 +366,12 @@ public class OlapScanNode extends ScanNode {
 
         /**
          * Compute InAccurate cardinality before mv selector and tablet pruning.
-         * - Accurate statistical information relies on the selector of materialized views and bucket reduction.
-         * - However, Those both processes occur after the reorder algorithm is completed.
-         * - When Join reorder is turned on, the cardinality must be calculated before the reorder algorithm.
+         * - Accurate statistical information relies on the selector of materialized
+         * views and bucket reduction.
+         * - However, Those both processes occur after the reorder algorithm is
+         * completed.
+         * - When Join reorder is turned on, the cardinality must be calculated before
+         * the reorder algorithm.
          * - So only an inaccurate cardinality can be calculated here.
          */
         mockRowCountInStatistic();
@@ -388,7 +411,8 @@ public class OlapScanNode extends ScanNode {
     public void finalize(Analyzer analyzer) throws UserException {
         LOG.debug("OlapScanNode get scan range locations. Tuple: {}", desc);
         /**
-         * If JoinReorder is turned on, it will be calculated init(), and this value is not accurate.
+         * If JoinReorder is turned on, it will be calculated init(), and this value is
+         * not accurate.
          * In the following logic, cardinality will be accurately calculated again.
          * So here we need to reset the value of cardinality.
          */
@@ -400,7 +424,8 @@ public class OlapScanNode extends ScanNode {
         } catch (AnalysisException e) {
             throw new UserException(e.getMessage());
         }
-        // Relatively accurate cardinality according to ScanRange in getScanRangeLocations
+        // Relatively accurate cardinality according to ScanRange in
+        // getScanRangeLocations
         computeStats(analyzer);
         computeNumNodes();
     }
@@ -418,7 +443,8 @@ public class OlapScanNode extends ScanNode {
             avgRowSize = totalBytes / (float) cardinality * COMPRESSION_RATIO;
             capCardinalityAtLimit();
         }
-        // when node scan has no data, cardinality should be 0 instead of a invalid value after computeStats()
+        // when node scan has no data, cardinality should be 0 instead of a invalid
+        // value after computeStats()
         cardinality = cardinality == -1 ? 0 : cardinality;
 
         // update statsDeriveResult for real statistics
@@ -433,7 +459,8 @@ public class OlapScanNode extends ScanNode {
         if (cardinality > 0) {
             numNodes = scanBackendIds.size();
         }
-        // even current node scan has no data,at least on backend will be assigned when the fragment actually execute
+        // even current node scan has no data,at least on backend will be assigned when
+        // the fragment actually execute
         numNodes = numNodes <= 0 ? 1 : numNodes;
     }
 
@@ -470,10 +497,10 @@ public class OlapScanNode extends ScanNode {
         } else if (partitionInfo.getType() == PartitionType.LIST) {
             if (analyzer.partitionPruneV2Enabled()) {
                 partitionPruner = new ListPartitionPrunerV2(keyItemMap, partitionInfo.getPartitionColumns(),
-                    columnNameToRange);
+                        columnNameToRange);
             } else {
                 partitionPruner = new ListPartitionPruner(keyItemMap,
-                    partitionInfo.getPartitionColumns(), columnFilters);
+                        partitionInfo.getPartitionColumns(), columnFilters);
             }
         }
         return partitionPruner.prune();
@@ -502,7 +529,7 @@ public class OlapScanNode extends ScanNode {
     }
 
     private void addScanRangeLocations(Partition partition,
-                                       List<Tablet> tablets) throws UserException {
+            List<Tablet> tablets) throws UserException {
         long visibleVersion = partition.getVisibleVersion();
         String visibleVersionStr = String.valueOf(visibleVersion);
 
@@ -613,8 +640,8 @@ public class OlapScanNode extends ScanNode {
             }
         } else {
             selectedPartitionIds = selectedPartitionIds.stream()
-                  .filter(id -> olapTable.getPartition(id).hasData())
-                  .collect(Collectors.toList());
+                    .filter(id -> olapTable.getPartition(id).hasData())
+                    .collect(Collectors.toList());
         }
         selectedPartitionNum = selectedPartitionIds.size();
 
@@ -633,9 +660,10 @@ public class OlapScanNode extends ScanNode {
         // Step2: select best rollup
         long start = System.currentTimeMillis();
         if (olapTable.getKeysType() == KeysType.DUP_KEYS) {
-            //This function is compatible with the INDEX selection logic of ROLLUP,
-            //so the Duplicate table here returns base index directly
-            //and the selection logic of materialized view is selected in "MaterializedViewSelector"
+            // This function is compatible with the INDEX selection logic of ROLLUP,
+            // so the Duplicate table here returns base index directly
+            // and the selection logic of materialized view is selected in
+            // "MaterializedViewSelector"
             selectedIndexId = olapTable.getBaseIndexId();
             LOG.debug("The best index will be selected later in mv selector");
             return;
@@ -757,18 +785,23 @@ public class OlapScanNode extends ScanNode {
     protected void toThrift(TPlanNode msg) {
         List<String> keyColumnNames = new ArrayList<String>();
         List<TPrimitiveType> keyColumnTypes = new ArrayList<TPrimitiveType>();
+        List<TColumn> columnsDesc = new ArrayList<TColumn>();
+
         if (selectedIndexId != -1) {
-            for (Column col : olapTable.getSchemaByIndexId(selectedIndexId)) {
-                if (!col.isKey()) {
-                    break;
+            for (Column col : olapTable.getSchemaByIndexId(selectedIndexId, true)) {
+                TColumn tColumn = col.toThrift();
+                col.setIndexFlag(tColumn, olapTable.getIndexes());
+                columnsDesc.add(tColumn);
+                if ((Util.showHiddenColumns() || (!Util.showHiddenColumns() && col.isVisible())) && col.isKey()) {
+                    keyColumnNames.add(col.getName());
+                    keyColumnTypes.add(col.getDataType().toThrift());
                 }
-                keyColumnNames.add(col.getName());
-                keyColumnTypes.add(col.getDataType().toThrift());
             }
         }
+
         msg.node_type = TPlanNodeType.OLAP_SCAN_NODE;
-        msg.olap_scan_node =
-                new TOlapScanNode(desc.getId().asInt(), keyColumnNames, keyColumnTypes, isPreAggregation);
+        msg.olap_scan_node = new TOlapScanNode(desc.getId().asInt(), keyColumnNames, keyColumnTypes, isPreAggregation,
+                columnsDesc);
         if (null != sortColumn) {
             msg.olap_scan_node.setSortColumn(sortColumn);
         }
@@ -833,7 +866,6 @@ public class OlapScanNode extends ScanNode {
         return desc.getId();
     }
 
-
     private boolean isEquivalenceExpr(Expr expr) {
         if (expr instanceof InPredicate) {
             return true;
@@ -914,10 +946,10 @@ public class OlapScanNode extends ScanNode {
     }
 
     /*
-    Although sometimes the scan range only involves one instance,
-        the data distribution cannot be set to UNPARTITIONED here.
-    The reason is that @coordinator will not set the scan range for the fragment,
-        when data partition of fragment is UNPARTITIONED.
+     * Although sometimes the scan range only involves one instance,
+     * the data distribution cannot be set to UNPARTITIONED here.
+     * The reason is that @coordinator will not set the scan range for the fragment,
+     * when data partition of fragment is UNPARTITIONED.
      */
     public DataPartition constructInputPartitionByDistributionInfo() throws UserException {
         ColocateTableIndex colocateTableIndex = Catalog.getCurrentColocateIndex();
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 3b462f6d20..1c69425ec8 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
@@ -45,6 +45,7 @@ import org.apache.doris.common.Status;
 import org.apache.doris.common.UserException;
 import org.apache.doris.system.Backend;
 import org.apache.doris.system.SystemInfoService;
+import org.apache.doris.thrift.TColumn;
 import org.apache.doris.thrift.TDataSink;
 import org.apache.doris.thrift.TDataSinkType;
 import org.apache.doris.thrift.TExplainLevel;
@@ -93,8 +94,8 @@ public class OlapTableSink extends DataSink {
         this.partitionIds = partitionIds;
     }
 
-    public void init(TUniqueId loadId, long txnId, long dbId, long loadChannelTimeoutS,
-                     int sendBatchParallelism, boolean loadToSingleTablet) throws AnalysisException {
+    public void init(TUniqueId loadId, long txnId, long dbId, long loadChannelTimeoutS, int sendBatchParallelism,
+            boolean loadToSingleTablet) throws AnalysisException {
         TOlapTableSink tSink = new TOlapTableSink();
         tSink.setLoadId(loadId);
         tSink.setTxnId(txnId);
@@ -102,8 +103,8 @@ public class OlapTableSink extends DataSink {
         tSink.setLoadChannelTimeoutS(loadChannelTimeoutS);
         tSink.setSendBatchParallelism(sendBatchParallelism);
         if (loadToSingleTablet && !(dstTable.getDefaultDistributionInfo() instanceof RandomDistributionInfo)) {
-            throw new AnalysisException("if load_to_single_tablet set to true,"
-                    + " the olap table must be with random distribution");
+            throw new AnalysisException(
+                    "if load_to_single_tablet set to true," + " the olap table must be with random distribution");
         }
         tSink.setLoadToSingleTablet(loadToSingleTablet);
         tDataSink = new TDataSink(TDataSinkType.OLAP_TABLE_SINK);
@@ -177,7 +178,7 @@ public class OlapTableSink extends DataSink {
         TOlapTableSchemaParam schemaParam = new TOlapTableSchemaParam();
         schemaParam.setDbId(dbId);
         schemaParam.setTableId(table.getId());
-        schemaParam.setVersion(0);
+        schemaParam.setVersion(table.getIndexMetaByIndexId(table.getBaseIndexId()).getSchemaVersion());
 
         schemaParam.tuple_desc = tupleDescriptor.toThrift();
         for (SlotDescriptor slotDesc : tupleDescriptor.getSlots()) {
@@ -187,9 +188,15 @@ public class OlapTableSink extends DataSink {
         for (Map.Entry<Long, MaterializedIndexMeta> pair : table.getIndexIdToMeta().entrySet()) {
             MaterializedIndexMeta indexMeta = pair.getValue();
             List<String> columns = Lists.newArrayList();
+            List<TColumn> columnsDesc = Lists.newArrayList();
             columns.addAll(indexMeta.getSchema().stream().map(Column::getName).collect(Collectors.toList()));
+            for (Column column : indexMeta.getSchema()) {
+                TColumn tColumn = column.toThrift();
+                column.setIndexFlag(tColumn, table.getIndexes());
+                columnsDesc.add(tColumn);
+            }
             TOlapTableIndexSchema indexSchema = new TOlapTableIndexSchema(pair.getKey(), columns,
-                    indexMeta.getSchemaHash());
+                    indexMeta.getSchemaHash(), columnsDesc);
             schemaParam.addToIndexes(indexSchema);
         }
         return schemaParam;
@@ -276,8 +283,7 @@ public class OlapTableSink extends DataSink {
                     tPartition.setNumBuckets(index.getTablets().size());
                 }
                 partitionParam.addToPartitions(tPartition);
-                partitionParam.setDistributedColumns(
-                        getDistColumns(partition.getDistributionInfo()));
+                partitionParam.setDistributedColumns(getDistColumns(partition.getDistributionInfo()));
                 break;
             }
             default: {
@@ -328,13 +334,12 @@ public class OlapTableSink extends DataSink {
                 for (Tablet tablet : index.getTablets()) {
                     Multimap<Long, Long> bePathsMap = tablet.getNormalReplicaBackendPathMap();
                     if (bePathsMap.keySet().size() < quorum) {
-                        throw new UserException(InternalErrorCode.REPLICA_FEW_ERR, "tablet " + tablet.getId()
-                                + " has few replicas: " + bePathsMap.keySet().size()
-                                + ", alive backends: [" + StringUtils.join(bePathsMap.keySet(), ",")
-                                + "]");
+                        throw new UserException(InternalErrorCode.REPLICA_FEW_ERR,
+                                "tablet " + tablet.getId() + " has few replicas: " + bePathsMap.keySet().size()
+                                        + ", alive backends: [" + StringUtils.join(bePathsMap.keySet(), ",") + "]");
                     }
-                    locationParam.addToTablets(new TTabletLocation(tablet.getId(),
-                            Lists.newArrayList(bePathsMap.keySet())));
+                    locationParam.addToTablets(
+                            new TTabletLocation(tablet.getId(), Lists.newArrayList(bePathsMap.keySet())));
                     allBePathsMap.putAll(bePathsMap);
                 }
             }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java
index 8f62a135ec..e5eed8c250 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java
@@ -211,8 +211,8 @@ public class FrontendServiceImpl implements FrontendService.Iface {
         if (db != null) {
             for (String tableName : db.getTableNamesWithLock()) {
                 LOG.debug("get table: {}, wait to check", tableName);
-                if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(currentUser, params.db,
-                        tableName, PrivPredicate.SHOW)) {
+                if (!Catalog.getCurrentCatalog().getAuth()
+                        .checkTblPriv(currentUser, params.db, tableName, PrivPredicate.SHOW)) {
                     continue;
                 }
 
@@ -270,14 +270,14 @@ public class FrontendServiceImpl implements FrontendService.Iface {
                     }
                 }
                 for (TableIf table : tables) {
-                    if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(currentUser, params.db,
-                            table.getName(), PrivPredicate.SHOW)) {
+                    if (!Catalog.getCurrentCatalog().getAuth()
+                            .checkTblPriv(currentUser, params.db, table.getName(), PrivPredicate.SHOW)) {
                         continue;
                     }
                     table.readLock();
                     try {
-                        if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(currentUser, params.db,
-                                table.getName(), PrivPredicate.SHOW)) {
+                        if (!Catalog.getCurrentCatalog().getAuth()
+                                .checkTblPriv(currentUser, params.db, table.getName(), PrivPredicate.SHOW)) {
                             continue;
                         }
 
@@ -386,8 +386,8 @@ public class FrontendServiceImpl implements FrontendService.Iface {
         } else {
             currentUser = UserIdentity.createAnalyzedUserIdentWithIp(params.user, params.user_ip);
         }
-        if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(currentUser, params.db,
-                params.getTableName(), PrivPredicate.SHOW)) {
+        if (!Catalog.getCurrentCatalog().getAuth()
+                .checkTblPriv(currentUser, params.db, params.getTableName(), PrivPredicate.SHOW)) {
             return result;
         }
 
@@ -504,7 +504,7 @@ public class FrontendServiceImpl implements FrontendService.Iface {
     }
 
     private void checkPasswordAndPrivs(String cluster, String user, String passwd, String db, String tbl,
-                                       String clientIp, PrivPredicate predicate) throws AuthenticationException {
+            String clientIp, PrivPredicate predicate) throws AuthenticationException {
 
         final String fullUserName = ClusterNamespace.getFullName(cluster, user);
         final String fullDbName = ClusterNamespace.getFullName(cluster, db);
@@ -533,8 +533,8 @@ public class FrontendServiceImpl implements FrontendService.Iface {
             result.setTxnId(tmpRes.getTxnId()).setDbId(tmpRes.getDbId());
         } catch (DuplicatedRequestException e) {
             // this is a duplicate request, just return previous txn id
-            LOG.warn("duplicate request for stream load. request id: {}, txn: {}",
-                    e.getDuplicatedRequestId(), e.getTxnId());
+            LOG.warn("duplicate request for stream load. request id: {}, txn: {}", e.getDuplicatedRequestId(),
+                    e.getTxnId());
             result.setTxnId(e.getTxnId());
         } catch (LabelAlreadyUsedException e) {
             status.setStatusCode(TStatusCode.LABEL_ALREADY_EXISTS);
@@ -560,8 +560,8 @@ public class FrontendServiceImpl implements FrontendService.Iface {
         }
 
         if (Strings.isNullOrEmpty(request.getAuthCodeUuid())) {
-            checkPasswordAndPrivs(cluster, request.getUser(), request.getPasswd(), request.getDb(),
-                    request.getTbl(), request.getUserIp(), PrivPredicate.LOAD);
+            checkPasswordAndPrivs(cluster, request.getUser(), request.getPasswd(), request.getDb(), request.getTbl(),
+                    request.getUserIp(), PrivPredicate.LOAD);
         }
 
         // check label
@@ -584,10 +584,10 @@ public class FrontendServiceImpl implements FrontendService.Iface {
         // begin
         long timeoutSecond = request.isSetTimeout() ? request.getTimeout() : Config.stream_load_default_timeout_second;
         MetricRepo.COUNTER_LOAD_ADD.increase(1L);
-        long txnId = Catalog.getCurrentGlobalTransactionMgr().beginTransaction(
-                db.getId(), Lists.newArrayList(table.getId()), request.getLabel(), request.getRequestId(),
-                new TxnCoordinator(TxnSourceType.BE, clientIp),
-                TransactionState.LoadJobSourceType.BACKEND_STREAMING, -1, timeoutSecond);
+        long txnId = Catalog.getCurrentGlobalTransactionMgr()
+                .beginTransaction(db.getId(), Lists.newArrayList(table.getId()), request.getLabel(),
+                        request.getRequestId(), new TxnCoordinator(TxnSourceType.BE, clientIp),
+                        TransactionState.LoadJobSourceType.BACKEND_STREAMING, -1, timeoutSecond);
         if (!Strings.isNullOrEmpty(request.getAuthCodeUuid())) {
             Catalog.getCurrentGlobalTransactionMgr().getTransactionState(db.getId(), txnId)
                     .setAuthCode(request.getAuthCodeUuid());
@@ -631,8 +631,8 @@ public class FrontendServiceImpl implements FrontendService.Iface {
         } else if (request.isSetAuthCodeUuid()) {
             checkAuthCodeUuid(request.getDb(), request.getTxnId(), request.getAuthCodeUuid());
         } else {
-            checkPasswordAndPrivs(cluster, request.getUser(), request.getPasswd(), request.getDb(),
-                    request.getTbl(), request.getUserIp(), PrivPredicate.LOAD);
+            checkPasswordAndPrivs(cluster, request.getUser(), request.getPasswd(), request.getDb(), request.getTbl(),
+                    request.getUserIp(), PrivPredicate.LOAD);
         }
 
         // get database
@@ -713,8 +713,8 @@ public class FrontendServiceImpl implements FrontendService.Iface {
         List<Table> tableList = database.getTablesOnIdOrderOrThrowException(tableIdList);
         for (Table table : tableList) {
             // check auth
-            checkPasswordAndPrivs(cluster, request.getUser(), request.getPasswd(), request.getDb(),
-                    table.getName(), request.getUserIp(), PrivPredicate.LOAD);
+            checkPasswordAndPrivs(cluster, request.getUser(), request.getPasswd(), request.getDb(), table.getName(),
+                    request.getUserIp(), PrivPredicate.LOAD);
         }
 
         String txnOperation = request.getOperation().trim();
@@ -767,8 +767,8 @@ public class FrontendServiceImpl implements FrontendService.Iface {
         } else if (request.isSetAuthCodeUuid()) {
             checkAuthCodeUuid(request.getDb(), request.getTxnId(), request.getAuthCodeUuid());
         } else {
-            checkPasswordAndPrivs(cluster, request.getUser(), request.getPasswd(), request.getDb(),
-                    request.getTbl(), request.getUserIp(), PrivPredicate.LOAD);
+            checkPasswordAndPrivs(cluster, request.getUser(), request.getPasswd(), request.getDb(), request.getTbl(),
+                    request.getUserIp(), PrivPredicate.LOAD);
         }
 
         // get database
@@ -835,8 +835,8 @@ public class FrontendServiceImpl implements FrontendService.Iface {
         } else if (request.isSetAuthCodeUuid()) {
             checkAuthCodeUuid(request.getDb(), request.getTxnId(), request.getAuthCodeUuid());
         } else {
-            checkPasswordAndPrivs(cluster, request.getUser(), request.getPasswd(), request.getDb(),
-                    request.getTbl(), request.getUserIp(), PrivPredicate.LOAD);
+            checkPasswordAndPrivs(cluster, request.getUser(), request.getPasswd(), request.getDb(), request.getTbl(),
+                    request.getUserIp(), PrivPredicate.LOAD);
         }
         String dbName = ClusterNamespace.getFullName(cluster, request.getDb());
         Database db;
@@ -896,8 +896,8 @@ public class FrontendServiceImpl implements FrontendService.Iface {
         long timeoutMs = request.isSetThriftRpcTimeoutMs() ? request.getThriftRpcTimeoutMs() : 5000;
         Table table = db.getTableOrMetaException(request.getTbl(), TableType.OLAP);
         if (!table.tryReadLock(timeoutMs, TimeUnit.MILLISECONDS)) {
-            throw new UserException("get table read lock timeout, database="
-                    + fullDbName + ",table=" + table.getName());
+            throw new UserException(
+                    "get table read lock timeout, database=" + fullDbName + ",table=" + table.getName());
         }
         try {
             StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request);
@@ -918,8 +918,9 @@ public class FrontendServiceImpl implements FrontendService.Iface {
 
     @Override
     public TStatus snapshotLoaderReport(TSnapshotLoaderReportRequest request) throws TException {
-        if (Catalog.getCurrentCatalog().getBackupHandler().report(request.getTaskType(), request.getJobId(),
-                request.getTaskId(), request.getFinishedNum(), request.getTotalNum())) {
+        if (Catalog.getCurrentCatalog().getBackupHandler()
+                .report(request.getTaskType(), request.getJobId(), request.getTaskId(), request.getFinishedNum(),
+                        request.getTotalNum())) {
             return new TStatus(TStatusCode.OK);
         }
         return new TStatus(TStatusCode.CANCELLED);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/AlterReplicaTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/AlterReplicaTask.java
index 4235ce0ded..d9ac6f0bb4 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/task/AlterReplicaTask.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/task/AlterReplicaTask.java
@@ -21,12 +21,15 @@ import org.apache.doris.alter.AlterJobV2;
 import org.apache.doris.analysis.DescriptorTable;
 import org.apache.doris.analysis.Expr;
 import org.apache.doris.analysis.SlotRef;
+import org.apache.doris.catalog.Column;
 import org.apache.doris.thrift.TAlterMaterializedViewParam;
 import org.apache.doris.thrift.TAlterTabletReqV2;
+import org.apache.doris.thrift.TColumn;
 import org.apache.doris.thrift.TTaskType;
 
 import com.google.common.collect.Lists;
 
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
@@ -48,6 +51,7 @@ public class AlterReplicaTask extends AgentTask {
 
     private Map<String, Expr> defineExprs;
     private DescriptorTable descTable;
+    private List<Column> baseSchemaColumns;
 
     /**
      * AlterReplicaTask constructor.
@@ -56,7 +60,7 @@ public class AlterReplicaTask extends AgentTask {
     public AlterReplicaTask(long backendId, long dbId, long tableId, long partitionId, long rollupIndexId,
             long baseIndexId, long rollupTabletId, long baseTabletId, long newReplicaId, int newSchemaHash,
             int baseSchemaHash, long version, long jobId, AlterJobV2.JobType jobType, Map<String, Expr> defineExprs,
-            DescriptorTable descTable) {
+            DescriptorTable descTable, List<Column> baseSchemaColumns) {
         super(null, backendId, TTaskType.ALTER, dbId, tableId, partitionId, rollupIndexId, rollupTabletId);
 
         this.baseTabletId = baseTabletId;
@@ -71,6 +75,7 @@ public class AlterReplicaTask extends AgentTask {
         this.jobType = jobType;
         this.defineExprs = defineExprs;
         this.descTable = descTable;
+        this.baseSchemaColumns = baseSchemaColumns;
     }
 
     public long getBaseTabletId() {
@@ -115,6 +120,14 @@ public class AlterReplicaTask extends AgentTask {
             }
         }
         req.setDescTbl(descTable.toThrift());
+
+        if (baseSchemaColumns != null) {
+            List<TColumn> columns = new ArrayList<TColumn>();
+            for (Column column : baseSchemaColumns) {
+                columns.add(column.toThrift());
+            }
+            req.setColumns(columns);
+        }
         return req;
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/PushTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/PushTask.java
index 1c32c9680b..fb61fe356d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/task/PushTask.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/task/PushTask.java
@@ -26,6 +26,7 @@ import org.apache.doris.analysis.Predicate;
 import org.apache.doris.analysis.SlotRef;
 import org.apache.doris.common.MarkedCountDownLatch;
 import org.apache.doris.thrift.TBrokerScanRange;
+import org.apache.doris.thrift.TColumn;
 import org.apache.doris.thrift.TCondition;
 import org.apache.doris.thrift.TDescriptorTable;
 import org.apache.doris.thrift.TPriority;
@@ -69,11 +70,13 @@ public class PushTask extends AgentTask {
     private TBrokerScanRange tBrokerScanRange;
     private TDescriptorTable tDescriptorTable;
 
-    public PushTask(TResourceInfo resourceInfo, long backendId, long dbId, long tableId, long partitionId,
-                    long indexId, long tabletId, long replicaId, int schemaHash, long version,
-                    String filePath, long fileSize, int timeoutSecond, long loadJobId, TPushType pushType,
-                    List<Predicate> conditions, boolean needDecompress, TPriority priority, TTaskType taskType,
-                    long transactionId, long signature) {
+    // for light schema change
+    private List<TColumn> columnsDesc = null;
+
+    public PushTask(TResourceInfo resourceInfo, long backendId, long dbId, long tableId, long partitionId, long indexId,
+            long tabletId, long replicaId, int schemaHash, long version, String filePath, long fileSize,
+            int timeoutSecond, long loadJobId, TPushType pushType, List<Predicate> conditions, boolean needDecompress,
+            TPriority priority, TTaskType taskType, long transactionId, long signature, List<TColumn> columnsDesc) {
         super(resourceInfo, backendId, taskType, dbId, tableId, partitionId, indexId, tabletId, signature);
         this.replicaId = replicaId;
         this.schemaHash = schemaHash;
@@ -92,17 +95,17 @@ public class PushTask extends AgentTask {
         this.transactionId = transactionId;
         this.tBrokerScanRange = null;
         this.tDescriptorTable = null;
+        this.columnsDesc = columnsDesc;
     }
 
     // for load v2 (SparkLoadJob)
     public PushTask(long backendId, long dbId, long tableId, long partitionId, long indexId, long tabletId,
-                    long replicaId, int schemaHash, int timeoutSecond, long loadJobId, TPushType pushType,
-                    TPriority priority, long transactionId, long signature,
-                    TBrokerScanRange tBrokerScanRange, TDescriptorTable tDescriptorTable) {
-        this(null, backendId, dbId, tableId, partitionId, indexId,
-                tabletId, replicaId, schemaHash, -1, null,
-                0, timeoutSecond, loadJobId, pushType, null, false,
-                priority, TTaskType.REALTIME_PUSH, transactionId, signature);
+            long replicaId, int schemaHash, int timeoutSecond, long loadJobId, TPushType pushType, TPriority priority,
+            long transactionId, long signature, TBrokerScanRange tBrokerScanRange, TDescriptorTable tDescriptorTable,
+            List<TColumn> columnsDesc) {
+        this(null, backendId, dbId, tableId, partitionId, indexId, tabletId, replicaId, schemaHash, -1, null, 0,
+                timeoutSecond, loadJobId, pushType, null, false, priority, TTaskType.REALTIME_PUSH, transactionId,
+                signature, columnsDesc);
         this.tBrokerScanRange = tBrokerScanRange;
         this.tDescriptorTable = tDescriptorTable;
     }
@@ -171,6 +174,7 @@ public class PushTask extends AgentTask {
                 LOG.warn("unknown push type. type: " + pushType.name());
                 break;
         }
+        request.setColumnsDesc(columnsDesc);
 
         return request;
     }
@@ -182,8 +186,8 @@ public class PushTask extends AgentTask {
     public void countDownLatch(long backendId, long tabletId) {
         if (this.latch != null) {
             if (latch.markedCountDown(backendId, tabletId)) {
-                LOG.debug("pushTask current latch count: {}. backend: {}, tablet:{}",
-                         latch.getCount(), backendId, tabletId);
+                LOG.debug("pushTask current latch count: {}. backend: {}, tablet:{}", latch.getCount(), backendId,
+                        tabletId);
             }
         }
     }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeHandlerTest.java b/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeHandlerTest.java
index 48973f1871..9fcc04dc98 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeHandlerTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/alter/SchemaChangeHandlerTest.java
@@ -17,24 +17,341 @@
 
 package org.apache.doris.alter;
 
+import org.apache.doris.analysis.AlterTableStmt;
 import org.apache.doris.analysis.ColumnPosition;
+import org.apache.doris.catalog.Catalog;
 import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.Database;
 import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.MaterializedIndexMeta;
 import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.common.FeConstants;
 import org.apache.doris.common.jmockit.Deencapsulation;
+import org.apache.doris.utframe.TestWithFeService;
 
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import mockit.Expectations;
 import mockit.Injectable;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
 import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
 
-public class SchemaChangeHandlerTest {
+import java.util.Map;
+
+public class SchemaChangeHandlerTest extends TestWithFeService {
+    private static final Logger LOG = LogManager.getLogger(SchemaChangeHandlerTest.class);
+    private int jobSize = 0;
+
+    @Override
+    protected void runBeforeAll() throws Exception {
+        FeConstants.default_scheduler_interval_millisecond = 10;
+        //create database db1
+        createDatabase("test");
+
+        //create tables
+        String createAggTblStmtStr = "CREATE TABLE IF NOT EXISTS test.sc_agg (\n" + "user_id LARGEINT NOT NULL,\n"
+                + "date DATE NOT NULL,\n" + "city VARCHAR(20),\n" + "age SMALLINT,\n" + "sex TINYINT,\n"
+                + "last_visit_date DATETIME REPLACE DEFAULT '1970-01-01 00:00:00',\n" + "cost BIGINT SUM DEFAULT '0',\n"
+                + "max_dwell_time INT MAX DEFAULT '0',\n" + "min_dwell_time INT MIN DEFAULT '99999')\n"
+                + "AGGREGATE KEY(user_id, date, city, age, sex)\n" + "DISTRIBUTED BY HASH(user_id) BUCKETS 1\n"
+                + "PROPERTIES ('replication_num' = '1');";
+        createTable(createAggTblStmtStr);
+
+        String createUniqTblStmtStr = "CREATE TABLE IF NOT EXISTS test.sc_uniq (\n" + "user_id LARGEINT NOT NULL,\n"
+                + "username VARCHAR(50) NOT NULL,\n" + "city VARCHAR(20),\n" + "age SMALLINT,\n" + "sex TINYINT,\n"
+                + "phone LARGEINT,\n" + "address VARCHAR(500),\n" + "register_time DATETIME)\n"
+                + "UNIQUE  KEY(user_id, username)\n" + "DISTRIBUTED BY HASH(user_id) BUCKETS 1\n"
+                + "PROPERTIES ('replication_num' = '1');";
+        createTable(createUniqTblStmtStr);
+
+        String createDupTblStmtStr = "CREATE TABLE IF NOT EXISTS test.sc_dup (\n" + "timestamp DATETIME,\n"
+                + "type INT,\n" + "error_code INT,\n" + "error_msg VARCHAR(1024),\n" + "op_id BIGINT,\n"
+                + "op_time DATETIME)\n" + "DUPLICATE  KEY(timestamp, type)\n" + "DISTRIBUTED BY HASH(type) BUCKETS 1\n"
+                + "PROPERTIES ('replication_num' = '1');";
+        createTable(createDupTblStmtStr);
+    }
+
+    private void waitAlterJobDone(Map<Long, AlterJobV2> alterJobs) throws Exception {
+        for (AlterJobV2 alterJobV2 : alterJobs.values()) {
+            while (!alterJobV2.getJobState().isFinalState()) {
+                LOG.info("alter job {} is running. state: {}", alterJobV2.getJobId(), alterJobV2.getJobState());
+                Thread.sleep(1000);
+            }
+            LOG.info("alter job {} is done. state: {}", alterJobV2.getJobId(), alterJobV2.getJobState());
+            Assert.assertEquals(AlterJobV2.JobState.FINISHED, alterJobV2.getJobState());
+
+            Database db = Catalog.getCurrentInternalCatalog().getDbOrMetaException(alterJobV2.getDbId());
+            OlapTable tbl = (OlapTable) db.getTableOrMetaException(alterJobV2.getTableId(), Table.TableType.OLAP);
+            while (tbl.getState() != OlapTable.OlapTableState.NORMAL) {
+                Thread.sleep(1000);
+            }
+        }
+    }
+
+    @Test
+    public void testAggAddOrDropColumn() throws Exception {
+        LOG.info("dbName: {}", Catalog.getCurrentInternalCatalog().getDbNames());
+
+        Database db = Catalog.getCurrentInternalCatalog().getDbOrMetaException("default_cluster:test");
+        OlapTable tbl = (OlapTable) db.getTableOrMetaException("sc_agg", Table.TableType.OLAP);
+        tbl.readLock();
+        try {
+            Assertions.assertNotNull(tbl);
+            System.out.println(tbl.getName());
+            Assertions.assertEquals("Doris", tbl.getEngine());
+            Assertions.assertEquals(9, tbl.getBaseSchema().size());
+        } finally {
+            tbl.readUnlock();
+        }
+
+        //process agg add value column schema change
+        String addValColStmtStr = "alter table test.sc_agg add column new_v1 int MAX default '0'";
+        AlterTableStmt addValColStmt = (AlterTableStmt) parseAndAnalyzeStmt(addValColStmtStr);
+        Catalog.getCurrentCatalog().getAlterInstance().processAlterTable(addValColStmt);
+        jobSize++;
+        //check alter job, do not create job
+        Map<Long, AlterJobV2> alterJobs = Catalog.getCurrentCatalog().getSchemaChangeHandler().getAlterJobsV2();
+        Assertions.assertEquals(jobSize, alterJobs.size());
+
+        tbl.readLock();
+        try {
+            Assertions.assertEquals(10, tbl.getBaseSchema().size());
+            String baseIndexName = tbl.getIndexNameById(tbl.getBaseIndexId());
+            Assertions.assertEquals(baseIndexName, tbl.getName());
+            MaterializedIndexMeta indexMeta = tbl.getIndexMetaByIndexId(tbl.getBaseIndexId());
+            Assertions.assertNotNull(indexMeta);
+        } finally {
+            tbl.readUnlock();
+        }
+
+        //process agg add  key column schema change
+        String addKeyColStmtStr = "alter table test.sc_agg add column new_k1 int default '1'";
+        AlterTableStmt addKeyColStmt = (AlterTableStmt) parseAndAnalyzeStmt(addKeyColStmtStr);
+        Catalog.getCurrentCatalog().getAlterInstance().processAlterTable(addKeyColStmt);
+
+        //check alter job
+        jobSize++;
+        Assertions.assertEquals(jobSize, alterJobs.size());
+        waitAlterJobDone(alterJobs);
+
+        tbl.readLock();
+        try {
+            Assertions.assertEquals(11, tbl.getBaseSchema().size());
+            String baseIndexName = tbl.getIndexNameById(tbl.getBaseIndexId());
+            Assertions.assertEquals(baseIndexName, tbl.getName());
+            MaterializedIndexMeta indexMeta = tbl.getIndexMetaByIndexId(tbl.getBaseIndexId());
+            Assertions.assertNotNull(indexMeta);
+        } finally {
+            tbl.readUnlock();
+        }
+
+        //process agg drop value column schema change
+        String dropValColStmtStr = "alter table test.sc_agg drop column new_v1";
+        AlterTableStmt dropValColStmt = (AlterTableStmt) parseAndAnalyzeStmt(dropValColStmtStr);
+        Catalog.getCurrentCatalog().getAlterInstance().processAlterTable(dropValColStmt);
+        jobSize++;
+        //check alter job, do not create job
+        LOG.info("alterJobs:{}", alterJobs);
+        Assertions.assertEquals(jobSize, alterJobs.size());
+
+        tbl.readLock();
+        try {
+            Assertions.assertEquals(10, tbl.getBaseSchema().size());
+            String baseIndexName = tbl.getIndexNameById(tbl.getBaseIndexId());
+            Assertions.assertEquals(baseIndexName, tbl.getName());
+            MaterializedIndexMeta indexMeta = tbl.getIndexMetaByIndexId(tbl.getBaseIndexId());
+            Assertions.assertNotNull(indexMeta);
+        } finally {
+            tbl.readUnlock();
+        }
+
+        try {
+            //process agg drop key column with replace schema change, expect exception.
+            String dropKeyColStmtStr = "alter table test.sc_agg drop column new_k1";
+            AlterTableStmt dropKeyColStmt = (AlterTableStmt) parseAndAnalyzeStmt(dropKeyColStmtStr);
+            Catalog.getCurrentCatalog().getAlterInstance().processAlterTable(dropKeyColStmt);
+            Assert.fail();
+        } catch (Exception e) {
+            LOG.info(e.getMessage());
+        }
+
+        LOG.info("getIndexIdToSchema 1: {}", tbl.getIndexIdToSchema(true));
+
+        String addRollUpStmtStr = "alter table test.sc_agg add rollup agg_rollup(user_id, max_dwell_time);";
+        AlterTableStmt addRollUpStmt = (AlterTableStmt) parseAndAnalyzeStmt(addRollUpStmtStr);
+        Catalog.getCurrentCatalog().getAlterInstance().processAlterTable(addRollUpStmt);
+        // 2. check alter job
+        Map<Long, AlterJobV2> materializedViewAlterJobs = Catalog.getCurrentCatalog().getMaterializedViewHandler()
+                .getAlterJobsV2();
+        waitAlterJobDone(materializedViewAlterJobs);
+        Assertions.assertEquals(1, materializedViewAlterJobs.size());
+
+        LOG.info("getIndexIdToSchema 2: {}", tbl.getIndexIdToSchema(true));
+
+        //process agg drop value column with rollup schema change
+        String dropRollUpValColStmtStr = "alter table test.sc_agg drop column max_dwell_time";
+        AlterTableStmt dropRollUpValColStmt = (AlterTableStmt) parseAndAnalyzeStmt(dropRollUpValColStmtStr);
+        Catalog.getCurrentCatalog().getAlterInstance().processAlterTable(dropRollUpValColStmt);
+        jobSize++;
+        //check alter job, need create job
+        LOG.info("alterJobs:{}", alterJobs);
+        Assertions.assertEquals(jobSize, alterJobs.size());
+        waitAlterJobDone(materializedViewAlterJobs);
+
+        tbl.readLock();
+        try {
+            Assertions.assertEquals(9, tbl.getBaseSchema().size());
+            String baseIndexName = tbl.getIndexNameById(tbl.getBaseIndexId());
+            Assertions.assertEquals(baseIndexName, tbl.getName());
+            MaterializedIndexMeta indexMeta = tbl.getIndexMetaByIndexId(tbl.getBaseIndexId());
+            Assertions.assertNotNull(indexMeta);
+        } finally {
+            tbl.readUnlock();
+        }
+
+        //process agg add mul value column schema change
+        String addMultiValColStmtStr
+                = "alter table test.sc_agg add column new_v2 int MAX default '0', add column new_v3 int MAX default '1';";
+        AlterTableStmt addMultiValColStmt = (AlterTableStmt) parseAndAnalyzeStmt(addMultiValColStmtStr);
+        Catalog.getCurrentCatalog().getAlterInstance().processAlterTable(addMultiValColStmt);
+        jobSize++;
+        //check alter job, do not create job
+        Assertions.assertEquals(jobSize, alterJobs.size());
+
+        tbl.readLock();
+        try {
+            Assertions.assertEquals(11, tbl.getBaseSchema().size());
+            String baseIndexName = tbl.getIndexNameById(tbl.getBaseIndexId());
+            Assertions.assertEquals(baseIndexName, tbl.getName());
+            MaterializedIndexMeta indexMeta = tbl.getIndexMetaByIndexId(tbl.getBaseIndexId());
+            Assertions.assertNotNull(indexMeta);
+            Assertions.assertEquals(12, tbl.getMaxColUniqueId());
+        } finally {
+            tbl.readUnlock();
+        }
+    }
+
+    @Test
+    public void testUniqAddOrDropColumn() throws Exception {
+
+        LOG.info("dbName: {}", Catalog.getCurrentInternalCatalog().getDbNames());
+
+        Database db = Catalog.getCurrentInternalCatalog().getDbOrMetaException("default_cluster:test");
+        OlapTable tbl = (OlapTable) db.getTableOrMetaException("sc_uniq", Table.TableType.OLAP);
+        tbl.readLock();
+        try {
+            Assertions.assertNotNull(tbl);
+            System.out.println(tbl.getName());
+            Assertions.assertEquals("Doris", tbl.getEngine());
+            Assertions.assertEquals(8, tbl.getBaseSchema().size());
+        } finally {
+            tbl.readUnlock();
+        }
+
+        //process uniq add value column schema change
+        String addValColStmtStr = "alter table test.sc_uniq add column new_v1 int default '0'";
+        AlterTableStmt addValColStmt = (AlterTableStmt) parseAndAnalyzeStmt(addValColStmtStr);
+        Catalog.getCurrentCatalog().getAlterInstance().processAlterTable(addValColStmt);
+        jobSize++;
+        //check alter job, do not create job
+        Map<Long, AlterJobV2> alterJobs = Catalog.getCurrentCatalog().getSchemaChangeHandler().getAlterJobsV2();
+        LOG.info("alterJobs:{}", alterJobs);
+        Assertions.assertEquals(jobSize, alterJobs.size());
+
+        tbl.readLock();
+        try {
+            Assertions.assertEquals(9, tbl.getBaseSchema().size());
+            String baseIndexName = tbl.getIndexNameById(tbl.getBaseIndexId());
+            Assertions.assertEquals(baseIndexName, tbl.getName());
+            MaterializedIndexMeta indexMeta = tbl.getIndexMetaByIndexId(tbl.getBaseIndexId());
+            Assertions.assertNotNull(indexMeta);
+        } finally {
+            tbl.readUnlock();
+        }
+
+        //process uniq drop val column schema change
+        String dropValColStmtStr = "alter table test.sc_uniq drop column new_v1";
+        AlterTableStmt dropValColStm = (AlterTableStmt) parseAndAnalyzeStmt(dropValColStmtStr);
+        Catalog.getCurrentCatalog().getAlterInstance().processAlterTable(dropValColStm);
+        jobSize++;
+        //check alter job
+        Assertions.assertEquals(jobSize, alterJobs.size());
+        tbl.readLock();
+        try {
+            Assertions.assertEquals(8, tbl.getBaseSchema().size());
+            String baseIndexName = tbl.getIndexNameById(tbl.getBaseIndexId());
+            Assertions.assertEquals(baseIndexName, tbl.getName());
+            MaterializedIndexMeta indexMeta = tbl.getIndexMetaByIndexId(tbl.getBaseIndexId());
+            Assertions.assertNotNull(indexMeta);
+        } finally {
+            tbl.readUnlock();
+        }
+    }
+
+    @Test
+    public void testDupAddOrDropColumn() throws Exception {
+
+        LOG.info("dbName: {}", Catalog.getCurrentInternalCatalog().getDbNames());
+
+        Database db = Catalog.getCurrentInternalCatalog().getDbOrMetaException("default_cluster:test");
+        OlapTable tbl = (OlapTable) db.getTableOrMetaException("sc_dup", Table.TableType.OLAP);
+        tbl.readLock();
+        try {
+            Assertions.assertNotNull(tbl);
+            System.out.println(tbl.getName());
+            Assertions.assertEquals("Doris", tbl.getEngine());
+            Assertions.assertEquals(6, tbl.getBaseSchema().size());
+        } finally {
+            tbl.readUnlock();
+        }
+
+        //process uniq add value column schema change
+        String addValColStmtStr = "alter table test.sc_dup add column new_v1 int default '0'";
+        AlterTableStmt addValColStmt = (AlterTableStmt) parseAndAnalyzeStmt(addValColStmtStr);
+        Catalog.getCurrentCatalog().getAlterInstance().processAlterTable(addValColStmt);
+        jobSize++;
+        //check alter job, do not create job
+        Map<Long, AlterJobV2> alterJobs = Catalog.getCurrentCatalog().getSchemaChangeHandler().getAlterJobsV2();
+        LOG.info("alterJobs:{}", alterJobs);
+        Assertions.assertEquals(jobSize, alterJobs.size());
+
+        tbl.readLock();
+        try {
+            Assertions.assertEquals(7, tbl.getBaseSchema().size());
+            String baseIndexName = tbl.getIndexNameById(tbl.getBaseIndexId());
+            Assertions.assertEquals(baseIndexName, tbl.getName());
+            MaterializedIndexMeta indexMeta = tbl.getIndexMetaByIndexId(tbl.getBaseIndexId());
+            Assertions.assertNotNull(indexMeta);
+        } finally {
+            tbl.readUnlock();
+        }
+
+        //process uniq drop val column schema change
+        String dropValColStmtStr = "alter table test.sc_dup drop column new_v1";
+        AlterTableStmt dropValColStm = (AlterTableStmt) parseAndAnalyzeStmt(dropValColStmtStr);
+        Catalog.getCurrentCatalog().getAlterInstance().processAlterTable(dropValColStm);
+        jobSize++;
+        //check alter job
+        Assertions.assertEquals(jobSize, alterJobs.size());
+        tbl.readLock();
+        try {
+            Assertions.assertEquals(6, tbl.getBaseSchema().size());
+            String baseIndexName = tbl.getIndexNameById(tbl.getBaseIndexId());
+            Assertions.assertEquals(baseIndexName, tbl.getName());
+            MaterializedIndexMeta indexMeta = tbl.getIndexMetaByIndexId(tbl.getBaseIndexId());
+            Assertions.assertNotNull(indexMeta);
+        } finally {
+            tbl.readUnlock();
+        }
+    }
 
     @Test
     public void testAddValueColumnOnAggMV(@Injectable OlapTable olapTable, @Injectable Column newColumn,
-                                          @Injectable ColumnPosition columnPosition) {
+            @Injectable ColumnPosition columnPosition) {
         SchemaChangeHandler schemaChangeHandler = new SchemaChangeHandler();
         new Expectations() {
             {
@@ -51,8 +368,7 @@ public class SchemaChangeHandlerTest {
 
         try {
             Deencapsulation.invoke(schemaChangeHandler, "addColumnInternal", olapTable, newColumn, columnPosition,
-                                   new Long(2), new Long(1),
-                                   Maps.newHashMap(), Sets.newHashSet());
+                    new Long(2), new Long(1), Maps.newHashMap(), Sets.newHashSet(), false);
             Assert.fail();
         } catch (Exception e) {
             System.out.println(e.getMessage());
diff --git a/fe/fe-core/src/test/java/org/apache/doris/persist/TableAddOrDropColumnsInfoTest.java b/fe/fe-core/src/test/java/org/apache/doris/persist/TableAddOrDropColumnsInfoTest.java
new file mode 100644
index 0000000000..a97dd27bf9
--- /dev/null
+++ b/fe/fe-core/src/test/java/org/apache/doris/persist/TableAddOrDropColumnsInfoTest.java
@@ -0,0 +1,97 @@
+// 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.
+
+package org.apache.doris.persist;
+
+import org.apache.doris.analysis.IndexDef;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.Index;
+import org.apache.doris.catalog.PrimitiveType;
+import org.apache.doris.catalog.ScalarType;
+import org.apache.doris.common.io.Text;
+import org.apache.doris.persist.gson.GsonUtils;
+
+import com.google.common.collect.Lists;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+public class TableAddOrDropColumnsInfoTest {
+    private static String fileName = "./TableAddOrDropColumnsInfoTest";
+
+    @Test
+    public void testSerialization() throws IOException {
+        // 1. Write objects to file
+        File file = new File(fileName);
+        file.createNewFile();
+
+        DataOutputStream out = new DataOutputStream(new FileOutputStream(file));
+
+        long dbId = 12345678;
+        long tableId = 87654321;
+        long jobId = 23456781;
+        LinkedList<Column> fullSchema = new LinkedList<>();
+        fullSchema.add(new Column("testCol1", ScalarType.createType(PrimitiveType.INT)));
+        fullSchema.add(new Column("testCol2", ScalarType.createType(PrimitiveType.VARCHAR)));
+        fullSchema.add(new Column("testCol3", ScalarType.createType(PrimitiveType.DATE)));
+        fullSchema.add(new Column("testCol4", ScalarType.createType(PrimitiveType.DATETIME)));
+
+        Map<Long, LinkedList<Column>> indexSchemaMap = new HashMap<>();
+        indexSchemaMap.put(tableId, fullSchema);
+
+        List<Index> indexes = Lists.newArrayList(
+                new Index("index", Lists.newArrayList("testCol1"), IndexDef.IndexType.BITMAP, "xxxxxx"));
+
+        TableAddOrDropColumnsInfo tableAddOrDropColumnsInfo1 = new TableAddOrDropColumnsInfo(dbId, tableId,
+                indexSchemaMap, indexes, jobId);
+
+        String c1Json = GsonUtils.GSON.toJson(tableAddOrDropColumnsInfo1);
+        Text.writeString(out, c1Json);
+        out.flush();
+        out.close();
+
+        // 2. Read objects from file
+        DataInputStream in = new DataInputStream(new FileInputStream(file));
+
+        String readJson = Text.readString(in);
+        TableAddOrDropColumnsInfo tableAddOrDropColumnsInfo2 = GsonUtils.GSON.fromJson(readJson,
+                TableAddOrDropColumnsInfo.class);
+
+        Assert.assertEquals(tableAddOrDropColumnsInfo1.getDbId(), tableAddOrDropColumnsInfo2.getDbId());
+        Assert.assertEquals(tableAddOrDropColumnsInfo1.getTableId(), tableAddOrDropColumnsInfo2.getTableId());
+        Assert.assertEquals(tableAddOrDropColumnsInfo1.getIndexSchemaMap(),
+                tableAddOrDropColumnsInfo2.getIndexSchemaMap());
+
+    }
+
+    @After
+    public void tearDown() {
+        File file = new File(fileName);
+        file.delete();
+    }
+}
diff --git a/gensrc/proto/descriptors.proto b/gensrc/proto/descriptors.proto
index 3dab3f0bee..acde58bbfa 100644
--- a/gensrc/proto/descriptors.proto
+++ b/gensrc/proto/descriptors.proto
@@ -21,6 +21,7 @@ package doris;
 option java_package = "org.apache.doris.proto";
 
 import "types.proto";
+import "olap_file.proto";
 
 message PSlotDescriptor {
     required int32 id = 1;
@@ -47,6 +48,7 @@ message POlapTableIndexSchema {
     required int64 id = 1;
     repeated string columns = 2;
     required int32 schema_hash = 3;
+    repeated ColumnPB columns_desc = 4;
 };
 
 message POlapTableSchemaParam {
diff --git a/gensrc/proto/olap_file.proto b/gensrc/proto/olap_file.proto
index a9ba857c7e..e7a3893fc1 100644
--- a/gensrc/proto/olap_file.proto
+++ b/gensrc/proto/olap_file.proto
@@ -107,6 +107,8 @@ message RowsetMetaPB {
     // the encoded segment min/max key of segments in this rowset,
     // only used in unique key data model with primary_key_index support.
     repeated KeyBoundsPB segments_key_bounds = 27;
+    // tablet meta pb, for compaction
+    optional TabletSchemaPB tablet_schema = 28;
     // spare field id for future use
     optional AlphaRowsetExtraMetaPB alpha_rowset_extra_meta_pb = 50;
     // to indicate whether the data between the segments overlap
@@ -190,6 +192,7 @@ message ColumnPB {
     optional bool visible = 16 [default=true];
     repeated ColumnPB children_columns = 17;
     repeated string children_column_names = 18;
+    optional int32 col_unique_id = 19 [default=-1]; // default is important
 }
 
 enum SortType {
@@ -211,6 +214,7 @@ message TabletSchemaPB {
     optional SortType sort_type = 11;
     optional int32 sort_col_num = 12;
     optional segment_v2.CompressionTypePB compression_type = 13 [default=LZ4F];
+    optional int32 schema_version = 14;
 }
 
 enum TabletStatePB {
diff --git a/gensrc/thrift/AgentService.thrift b/gensrc/thrift/AgentService.thrift
index 4e420de9df..78c1fe0c3b 100644
--- a/gensrc/thrift/AgentService.thrift
+++ b/gensrc/thrift/AgentService.thrift
@@ -25,25 +25,12 @@ include "PlanNodes.thrift"
 include "Descriptors.thrift"
 include "Exprs.thrift"
 
-struct TColumn {
-    1: required string column_name
-    2: required Types.TColumnType column_type
-    3: optional Types.TAggregationType aggregation_type
-    4: optional bool is_key
-    5: optional bool is_allow_null
-    6: optional string default_value
-    7: optional bool is_bloom_filter_column
-    8: optional Exprs.TExpr define_expr
-    9: optional bool visible = true
-    10: optional list<TColumn> children_column
-}
-
 struct TTabletSchema {
     1: required i16 short_key_column_count
     2: required Types.TSchemaHash schema_hash
     3: required Types.TKeysType keys_type
     4: required Types.TStorageType storage_type
-    5: required list<TColumn> columns
+    5: required list<Descriptors.TColumn> columns
     6: optional double bloom_filter_fpp
     7: optional list<Descriptors.TOlapTableIndex> indexes
     8: optional bool is_in_memory
@@ -168,6 +155,7 @@ struct TAlterTabletReqV2 {
     7: optional list<TAlterMaterializedViewParam> materialized_view_params
     8: optional TAlterTabletType alter_tablet_type = TAlterTabletType.SCHEMA_CHANGE
     9: optional Descriptors.TDescriptorTable desc_tbl
+    10: optional list<Descriptors.TColumn> columns
 }
 
 struct TAlterMaterializedViewParam {
@@ -209,6 +197,7 @@ struct TPushReq {
     // 14 and 15 are used by spark load
     14: optional PlanNodes.TBrokerScanRange broker_scan_range
     15: optional Descriptors.TDescriptorTable desc_tbl
+    16: optional list<Descriptors.TColumn> columns_desc
 }
 
 struct TCloneReq {
diff --git a/gensrc/thrift/Descriptors.thrift b/gensrc/thrift/Descriptors.thrift
index 187df7e0b3..787a6e606e 100644
--- a/gensrc/thrift/Descriptors.thrift
+++ b/gensrc/thrift/Descriptors.thrift
@@ -21,6 +21,21 @@ namespace java org.apache.doris.thrift
 include "Types.thrift"
 include "Exprs.thrift"
 
+struct TColumn {
+    1: required string column_name
+    2: required Types.TColumnType column_type
+    3: optional Types.TAggregationType aggregation_type
+    4: optional bool is_key
+    5: optional bool is_allow_null
+    6: optional string default_value
+    7: optional bool is_bloom_filter_column
+    8: optional Exprs.TExpr define_expr
+    9: optional bool visible = true
+    10: optional list<TColumn> children_column
+    11: optional i32 col_unique_id  = -1
+    12: optional bool has_bitmap_index = false
+}
+
 struct TSlotDescriptor {
   1: required Types.TSlotId id
   2: required Types.TTupleId parent
@@ -32,6 +47,7 @@ struct TSlotDescriptor {
   8: required string colName;
   9: required i32 slotIdx
   10: required bool isMaterialized
+  11: optional i32 col_unique_id = -1
 }
 
 struct TTupleDescriptor {
@@ -158,6 +174,7 @@ struct TOlapTableIndexSchema {
     1: required i64 id
     2: required list<string> columns
     3: required i32 schema_hash
+    4: required list<TColumn> columns_desc
 }
 
 struct TOlapTableSchemaParam {
diff --git a/gensrc/thrift/PlanNodes.thrift b/gensrc/thrift/PlanNodes.thrift
index dec3ca2f64..2469ebd4e2 100644
--- a/gensrc/thrift/PlanNodes.thrift
+++ b/gensrc/thrift/PlanNodes.thrift
@@ -419,6 +419,7 @@ struct TOlapScanNode {
   5: optional string sort_column
   6: optional Types.TKeysType keyType
   7: optional string table_name
+  8: required list<Descriptors.TColumn> columns_desc
 }
 
 struct TEqJoinCondition {
diff --git a/regression-test/data/schema_change/test_compaction_schema_change.out b/regression-test/data/schema_change/test_compaction_schema_change.out
new file mode 100644
index 0000000000..c7f14cd658
--- /dev/null
+++ b/regression-test/data/schema_change/test_compaction_schema_change.out
@@ -0,0 +1,11 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !compaction --
+100
+
+-- !compaction --
+100
+
+-- !compaction --
+1
+2
+
diff --git a/regression-test/data/schema_change/test_delete_schema_change.out b/regression-test/data/schema_change/test_delete_schema_change.out
new file mode 100644
index 0000000000..ff8a11901d
--- /dev/null
+++ b/regression-test/data/schema_change/test_delete_schema_change.out
@@ -0,0 +1,44 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !test_delete_schema_change --
+0
+
+-- !test_delete_schema_change_2 --
+0
+
+-- !test_delete_schema_change_3 --
+1
+
+-- !test_delete_schema_change_4 --
+1
+
+-- !test_delete_schema_change_5 --
+1	2017-10-01	Beijing	10	1	2020-01-01T00:00	2020-01-01T00:00	2020-01-01T00:00	1	30	20
+2	2017-10-01	Beijing	10	1	2020-01-02T00:00	2020-01-02T00:00	2020-01-02T00:00	1	31	21
+
+-- !test_delete_schema_change_6 --
+0
+
+-- !test_delete_schema_change_7 --
+2	2017-10-01	Beijing	10	1	2020-01-02T00:00	2020-01-02T00:00	2020-01-02T00:00	1	31	21	1
+1	2017-10-01	Beijing	10	1	2020-01-01T00:00	2020-01-01T00:00	2020-01-01T00:00	1	30	20	1
+
+-- !test_delete_schema_change_8 --
+1
+
+-- !test_delete_schema_change_9 --
+1
+
+-- !test_delete_schema_change_10 --
+1
+
+-- !test_delete_schema_change_11 --
+1
+
+-- !test_delete_schema_change_12 --
+0
+
+-- !test_delete_schema_change_13 --
+3	2017-10-01	Beijing	10	1	2020-01-03T00:00	2020-01-03T00:00	2020-01-03T00:00	1	32	20	2
+2	2017-10-01	Beijing	10	1	2020-01-03T00:00	2020-01-03T00:00	2020-01-03T00:00	1	32	20	2
+1	2017-10-01	Beijing	10	1	2020-01-02T00:00	2020-01-02T00:00	2020-01-02T00:00	1	31	19	2
+
diff --git a/regression-test/data/schema_change/test_partition_schema_change.out b/regression-test/data/schema_change/test_partition_schema_change.out
new file mode 100644
index 0000000000..5c568d5ecd
--- /dev/null
+++ b/regression-test/data/schema_change/test_partition_schema_change.out
@@ -0,0 +1,33 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !test_partition_schema_change --
+0
+
+-- !test_partition_schema_change_2 --
+0
+
+-- !test_partition_schema_change_3 --
+1
+
+-- !test_partition_schema_change_4 --
+1
+
+-- !test_partition_schema_change_5 --
+1
+
+-- !test_partition_schema_change_6 --
+1	2017-01-02	Beijing	10	1	2017-01-02T00:00	1	30	20
+1	2017-02-02	Beijing	10	1	2017-02-02T00:00	1	30	20
+1	2017-03-02	Beijing	10	1	2017-03-02T00:00	1	30	20
+
+-- !test_partition_schema_change_7 --
+0
+
+-- !test_partition_schema_change_8 --
+1
+
+-- !test_partition_schema_change_9 --
+1	2017-01-02	Beijing	10	1	2017-01-02T00:00	1	30	20	1
+1	2017-02-02	Beijing	10	1	2017-02-02T00:00	1	30	20	1
+2	2017-02-03	Beijing	10	1	2017-02-02T00:00	1	30	20	2
+1	2017-03-02	Beijing	10	1	2017-03-02T00:00	1	30	20	1
+
diff --git a/regression-test/data/schema_change/test_update_schema_change.out b/regression-test/data/schema_change/test_update_schema_change.out
new file mode 100644
index 0000000000..aeacfd28c6
--- /dev/null
+++ b/regression-test/data/schema_change/test_update_schema_change.out
@@ -0,0 +1,64 @@
+-- This file is automatically generated. You should know what you did if you want to edit this
+-- !test_update_schema_change --
+0
+
+-- !test_update_schema_change_2 --
+0
+
+-- !test_update_schema_change_3 --
+1
+
+-- !test_update_schema_change_4 --
+1
+
+-- !test_update_schema_change_5 --
+1	2017-10-01	Beijing	10	1	2020-01-01T00:00	2020-01-01T00:00	2020-01-01T00:00	1	30	20
+2	2017-10-01	Beijing	10	1	2020-01-02T00:00	2020-01-02T00:00	2020-01-02T00:00	1	31	21
+
+-- !test_update_schema_change_6 --
+0
+
+-- !test_update_schema_change_7 --
+2	2017-10-01	Beijing	10	1	2020-01-02T00:00	2020-01-02T00:00	2020-01-02T00:00	1	31	21	1
+1	2017-10-01	Beijing	10	1	2020-01-01T00:00	2020-01-01T00:00	2020-01-01T00:00	1	30	20	1
+
+-- !test_update_schema_change_8 --
+1
+
+-- !test_update_schema_change_9 --
+1	2017-10-01	Beijing	10	1	2020-01-01T00:00	2020-01-01T00:00	2020-01-01T00:00	1	30	20	2
+2	2017-10-01	Beijing	10	1	2020-01-02T00:00	2020-01-02T00:00	2020-01-02T00:00	1	31	21	1
+
+-- !test_update_schema_change_10 --
+1
+
+-- !test_update_schema_change_11 --
+1
+
+-- !test_update_schema_change_12 --
+2
+
+-- !test_update_schema_change_13 --
+5	2017-10-01	Beijing	10	1	2020-01-02T00:00	2020-01-02T00:00	2020-01-02T00:00	1	31	21	20
+3	2017-10-01	Beijing	10	1	2020-01-01T00:00	2020-01-01T00:00	2020-01-01T00:00	1	30	20	20
+2	2017-10-01	Beijing	10	1	2020-01-02T00:00	2020-01-02T00:00	2020-01-02T00:00	1	31	21	1
+1	2017-10-01	Beijing	10	1	2020-01-01T00:00	2020-01-01T00:00	2020-01-01T00:00	1	30	20	20
+
+-- !test_update_schema_change_14 --
+0
+
+-- !test_update_schema_change_15 --
+5	2017-10-01	Beijing	10	1	2020-01-02T00:00	2020-01-02T00:00	2020-01-02T00:00	1	31	21
+3	2017-10-01	Beijing	10	1	2020-01-01T00:00	2020-01-01T00:00	2020-01-01T00:00	1	30	20
+2	2017-10-01	Beijing	10	1	2020-01-02T00:00	2020-01-02T00:00	2020-01-02T00:00	1	31	21
+1	2017-10-01	Beijing	10	1	2020-01-01T00:00	2020-01-01T00:00	2020-01-01T00:00	1	30	20
+
+-- !test_update_schema_change_16 --
+1
+
+-- !test_update_schema_change_17 --
+5	2017-10-01	Beijing	10	1	2020-01-02T00:00	2020-01-02T00:00	2020-01-02T00:00	20	31	21
+3	2017-10-01	Beijing	10	1	2020-01-01T00:00	2020-01-01T00:00	2020-01-01T00:00	1	30	20
+2	2017-10-01	Beijing	10	1	2020-01-02T00:00	2020-01-02T00:00	2020-01-02T00:00	1	31	21
+1	2017-10-01	Beijing	10	1	2020-01-01T00:00	2020-01-01T00:00	2020-01-01T00:00	1	30	20
+
diff --git a/regression-test/suites/schema_change/test_agg_keys_schema_change.groovy b/regression-test/suites/schema_change/test_agg_keys_schema_change.groovy
new file mode 100644
index 0000000000..13594474f7
--- /dev/null
+++ b/regression-test/suites/schema_change/test_agg_keys_schema_change.groovy
@@ -0,0 +1,275 @@
+// 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.
+
+import org.codehaus.groovy.runtime.IOGroovyMethods
+
+suite ("test_agg_keys_schema_change") {
+    def tableName = "schema_change_agg_keys_regression_test"
+
+    try {
+
+        String[][] backends = sql """ show backends; """
+        assertTrue(backends.size() > 0)
+        String backend_id;
+        def backendId_to_backendIP = [:]
+        def backendId_to_backendHttpPort = [:]
+        for (String[] backend in backends) {
+            backendId_to_backendIP.put(backend[0], backend[2])
+            backendId_to_backendHttpPort.put(backend[0], backend[5])
+        }
+
+        backend_id = backendId_to_backendIP.keySet()[0]
+        StringBuilder showConfigCommand = new StringBuilder();
+        showConfigCommand.append("curl -X GET http://")
+        showConfigCommand.append(backendId_to_backendIP.get(backend_id))
+        showConfigCommand.append(":")
+        showConfigCommand.append(backendId_to_backendHttpPort.get(backend_id))
+        showConfigCommand.append("/api/show_config")
+        logger.info(showConfigCommand.toString())
+        def process = showConfigCommand.toString().execute()
+        int code = process.waitFor()
+        String err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+        String out = process.getText()
+        logger.info("Show config: code=" + code + ", out=" + out + ", err=" + err)
+        assertEquals(code, 0)
+        def configList = parseJson(out.trim())
+        assert configList instanceof List
+
+        boolean disableAutoCompaction = true
+        for (Object ele in (List) configList) {
+            assert ele instanceof List<String>
+            if (((List<String>) ele)[0] == "disable_auto_compaction") {
+                disableAutoCompaction = Boolean.parseBoolean(((List<String>) ele)[2])
+            }
+        }
+
+        sql """ DROP TABLE IF EXISTS ${tableName} """
+        sql """
+                CREATE TABLE ${tableName} (
+                    `user_id` LARGEINT NOT NULL COMMENT "用户id",
+                    `date` DATE NOT NULL COMMENT "数据灌入日期时间",
+                    `city` VARCHAR(20) COMMENT "用户所在城市",
+                    `age` SMALLINT COMMENT "用户年龄",
+                    `sex` TINYINT COMMENT "用户性别",
+
+                    `cost` BIGINT SUM DEFAULT "0" COMMENT "用户总消费",
+                    `max_dwell_time` INT MAX DEFAULT "0" COMMENT "用户最大停留时间",
+                    `min_dwell_time` INT MIN DEFAULT "99999" COMMENT "用户最小停留时间",
+                    `hll_col` HLL HLL_UNION NOT NULL COMMENT "HLL列",
+                    `bitmap_col` Bitmap BITMAP_UNION NOT NULL COMMENT "bitmap列")
+                AGGREGATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`)
+                PROPERTIES ( "replication_num" = "1" );
+            """
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (1, '2017-10-01', 'Beijing', 10, 1, 1, 30, 20, hll_hash(1), to_bitmap(1))
+            """
+        sql """ INSERT INTO ${tableName} VALUES
+                (1, '2017-10-01', 'Beijing', 10, 1, 1, 31, 19, hll_hash(2), to_bitmap(2))
+            """
+        sql """ INSERT INTO ${tableName} VALUES
+                (2, '2017-10-01', 'Beijing', 10, 1, 1, 31, 21, hll_hash(2), to_bitmap(2))
+            """
+        sql """ INSERT INTO ${tableName} VALUES
+                (2, '2017-10-01', 'Beijing', 10, 1, 1, 32, 20, hll_hash(3), to_bitmap(3))
+            """
+
+        def result = "null";
+        result = sql """ select * from ${tableName} order by user_id"""
+        assertTrue(result.size() == 2)
+        assertTrue(result[0].size() == 10)
+        assertTrue(result[0][5] == 2, "user id 1 cost should be 2")
+        assertTrue(result[1][5] == 2, "user id 2 cost should be 2")
+
+        // add key column case 1, not light schema change
+        sql """
+            ALTER table ${tableName} ADD COLUMN new_key_column INT default "2" 
+            """
+
+        result = "null"
+        while (!result.contains("FINISHED")){
+            result = sql "SHOW ALTER TABLE COLUMN WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;"
+            result = result.toString()
+            logger.info("result: ${result}")
+            if(result.contains("CANCELLED")){
+                break
+            }
+            Thread.sleep(1000)
+        }
+
+        sql """ INSERT INTO ${tableName} (`user_id`,`date`,`city`,`age`,`sex`,`cost`,`max_dwell_time`,`min_dwell_time`, `hll_col`, `bitmap_col`)
+                VALUES
+                (3, '2017-10-01', 'Beijing', 10, 1, 100, 32, 20, hll_hash(4), to_bitmap(4))
+            """
+        result = "null"
+        result = sql """SELECT * FROM ${tableName} WHERE user_id = 3"""
+
+        assertTrue(result.size() == 1)
+        assertTrue(result[0][5] == 2, "new key column default value should be 2")
+        assertTrue(result[0].size() == 11)
+
+        // add key column case 2
+        sql """ INSERT INTO ${tableName} VALUES
+                (3, '2017-10-01', 'Beijing', 10, 1, 3, 110, 32, 20, hll_hash(4), to_bitmap(4))
+            """
+        result = "null"
+        result = sql """ SELECT * FROM ${tableName} WHERE user_id = 3 """
+
+        assertTrue(result.size() == 2)
+        assertTrue(result[0].size() == 11)
+
+        result = "null"
+        result = sql """ select count(*) from ${tableName} """
+        logger.info("result.size:" + result.size() + " result[0].size:" + result[0].size + " " + result[0][0])
+        assertTrue(result.size() == 1)
+        assertTrue(result[0].size() == 1)
+        assertTrue(result[0][0] == 4, "total count is 4")
+
+        // drop key column, not light schema change
+        sql """
+            ALTER TABLE ${tableName} DROP COLUMN new_key_column
+            """
+        result = "null"
+        while (!result.contains("FINISHED")){
+            result = sql "SHOW ALTER TABLE COLUMN WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;"
+            result = result.toString()
+            logger.info("result: ${result}")
+            if(result.contains("CANCELLED")){
+                break
+            }
+            Thread.sleep(1000)
+        }
+
+        result = sql """ select * from ${tableName} where user_id = 3 """
+        assertTrue(result.size() == 1)
+        assertTrue(result[0].size() == 10)
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, 1, 1, 32, 20, hll_hash(5), to_bitmap(5))
+            """
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, 1, 1, 32, 20, hll_hash(5), to_bitmap(5))
+            """
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, 1, 1, 32, 20, hll_hash(5), to_bitmap(5))
+            """
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, 1, 1, 32, 20, hll_hash(5), to_bitmap(5))
+            """
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, 1, 1, 32, 20, hll_hash(5), to_bitmap(5))
+            """
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, 1, 1, 32, 20, hll_hash(5), to_bitmap(5))
+            """
+
+        Thread.sleep(30 * 1000)
+        // compaction
+        String[][] tablets = sql """ show tablets from ${tableName}; """
+        for (String[] tablet in tablets) {
+                String tablet_id = tablet[0]
+                backend_id = tablet[2]
+                logger.info("run compaction:" + tablet_id)
+                StringBuilder sb = new StringBuilder();
+                sb.append("curl -X POST http://")
+                sb.append(backendId_to_backendIP.get(backend_id))
+                sb.append(":")
+                sb.append(backendId_to_backendHttpPort.get(backend_id))
+                sb.append("/api/compaction/run?tablet_id=")
+                sb.append(tablet_id)
+                sb.append("&compact_type=cumulative")
+
+                String command = sb.toString()
+                process = command.execute()
+                code = process.waitFor()
+                err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+                out = process.getText()
+                logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err)
+        }
+
+        // wait for all compactions done
+        for (String[] tablet in tablets) {
+                boolean running = true
+                do {
+                    Thread.sleep(1000)
+                    String tablet_id = tablet[0]
+                    backend_id = tablet[2]
+                    StringBuilder sb = new StringBuilder();
+                    sb.append("curl -X GET http://")
+                    sb.append(backendId_to_backendIP.get(backend_id))
+                    sb.append(":")
+                    sb.append(backendId_to_backendHttpPort.get(backend_id))
+                    sb.append("/api/compaction/run_status?tablet_id=")
+                    sb.append(tablet_id)
+
+                    String command = sb.toString()
+                    process = command.execute()
+                    code = process.waitFor()
+                    err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+                    out = process.getText()
+                    logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err)
+                    assertEquals(code, 0)
+                    def compactionStatus = parseJson(out.trim())
+                    assertEquals("success", compactionStatus.status.toLowerCase())
+                    running = compactionStatus.run_status
+                } while (running)
+        }
+         
+        result = sql """ select count(*) from ${tableName} """
+        assertTrue(result.size() == 1)
+        assertTrue(result[0][0] == 4)
+
+        result = sql """  SELECT * FROM ${tableName} WHERE user_id=2 """
+        assertTrue(result.size() == 1)
+        assertTrue(result[0].size() == 10)
+
+        int rowCount = 0
+        for (String[] tablet in tablets) {
+                String tablet_id = tablet[0]
+                backend_id = tablet[2]
+                StringBuilder sb = new StringBuilder();
+                sb.append("curl -X GET http://")
+                sb.append(backendId_to_backendIP.get(backend_id))
+                sb.append(":")
+                sb.append(backendId_to_backendHttpPort.get(backend_id))
+                sb.append("/api/compaction/show?tablet_id=")
+                sb.append(tablet_id)
+                String command = sb.toString()
+                // wait for cleaning stale_rowsets
+                process = command.execute()
+                code = process.waitFor()
+                err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+                out = process.getText()
+                logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err)
+                assertEquals(code, 0)
+                def tabletJson = parseJson(out.trim())
+                assert tabletJson.rowsets instanceof List
+            for (String rowset in (List<String>) tabletJson.rowsets) {
+                rowCount += Integer.parseInt(rowset.split(" ")[1])
+            }
+        }
+        logger.info("size:" + rowCount)
+        assertTrue(rowCount <= 8)
+    } finally {
+        //try_sql("DROP TABLE IF EXISTS ${tableName}")
+    }
+}
\ No newline at end of file
diff --git a/regression-test/suites/schema_change/test_agg_mv_schema_change.groovy b/regression-test/suites/schema_change/test_agg_mv_schema_change.groovy
new file mode 100644
index 0000000000..d13c66ebdf
--- /dev/null
+++ b/regression-test/suites/schema_change/test_agg_mv_schema_change.groovy
@@ -0,0 +1,258 @@
+      
+// 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.
+
+import org.codehaus.groovy.runtime.IOGroovyMethods
+
+suite ("test_agg_mv_schema_change") {
+    def tableName = "schema_change_agg_mv_regression_test"
+
+    try {
+        String[][] backends = sql """ show backends; """
+        assertTrue(backends.size() > 0)
+        String backend_id;
+        def backendId_to_backendIP = [:]
+        def backendId_to_backendHttpPort = [:]
+        for (String[] backend in backends) {
+            backendId_to_backendIP.put(backend[0], backend[2])
+            backendId_to_backendHttpPort.put(backend[0], backend[5])
+        }
+
+        backend_id = backendId_to_backendIP.keySet()[0]
+        StringBuilder showConfigCommand = new StringBuilder();
+        showConfigCommand.append("curl -X GET http://")
+        showConfigCommand.append(backendId_to_backendIP.get(backend_id))
+        showConfigCommand.append(":")
+        showConfigCommand.append(backendId_to_backendHttpPort.get(backend_id))
+        showConfigCommand.append("/api/show_config")
+        logger.info(showConfigCommand.toString())
+        def process = showConfigCommand.toString().execute()
+        int code = process.waitFor()
+        String err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+        String out = process.getText()
+        logger.info("Show config: code=" + code + ", out=" + out + ", err=" + err)
+        assertEquals(code, 0)
+        def configList = parseJson(out.trim())
+        assert configList instanceof List
+
+        boolean disableAutoCompaction = true
+        for (Object ele in (List) configList) {
+            assert ele instanceof List<String>
+            if (((List<String>) ele)[0] == "disable_auto_compaction") {
+                disableAutoCompaction = Boolean.parseBoolean(((List<String>) ele)[2])
+            }
+        }
+
+        sql """ DROP TABLE IF EXISTS ${tableName} """
+        sql """
+                CREATE TABLE ${tableName} (
+                    `user_id` LARGEINT NOT NULL COMMENT "用户id",
+                    `date` DATE NOT NULL COMMENT "数据灌入日期时间",
+                    `city` VARCHAR(20) COMMENT "用户所在城市",
+                    `age` SMALLINT COMMENT "用户年龄",
+                    `sex` TINYINT COMMENT "用户性别",
+
+                    `cost` BIGINT SUM DEFAULT "0" COMMENT "用户总消费",
+                    `max_dwell_time` INT MAX DEFAULT "0" COMMENT "用户最大停留时间",
+                    `min_dwell_time` INT MIN DEFAULT "99999" COMMENT "用户最小停留时间",
+                    `hll_col` HLL HLL_UNION NOT NULL COMMENT "HLL列",
+                    `bitmap_col` Bitmap BITMAP_UNION NOT NULL COMMENT "bitmap列")
+                AGGREGATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`)
+                PROPERTIES ( "replication_num" = "1" );
+            """
+
+        //add materialized view
+        def result = "null"
+        def mvName = "mv1"
+        sql "create materialized view ${mvName} as select user_id, date, city, age, sex, sum(cost) from ${tableName} group by user_id, date, city, age, sex, cost;"
+        while (!result.contains("FINISHED")){
+            result = sql "SHOW ALTER TABLE MATERIALIZED VIEW WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;"
+            result = result.toString()
+            logger.info("result: ${result}")
+            if(result.contains("CANCELLED")){
+                break
+            }
+            Thread.sleep(1000)
+        }
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (1, '2017-10-01', 'Beijing', 10, 1, 1, 30, 20, hll_hash(1), to_bitmap(1))
+            """
+        sql """ INSERT INTO ${tableName} VALUES
+                (1, '2017-10-01', 'Beijing', 10, 1, 1, 31, 19, hll_hash(2), to_bitmap(2))
+            """
+        sql """ INSERT INTO ${tableName} VALUES
+                (2, '2017-10-01', 'Beijing', 10, 1, 1, 31, 21, hll_hash(2), to_bitmap(2))
+            """
+        sql """ INSERT INTO ${tableName} VALUES
+                (2, '2017-10-01', 'Beijing', 10, 1, 1, 32, 20, hll_hash(3), to_bitmap(3))
+            """
+
+        result = "null";
+        result = sql """ select * from ${tableName} """
+        assertTrue(result.size() == 2)
+        assertTrue(result[0].size() == 10)
+        assertTrue(result[0][5] == 2, "user id 1 cost should be 2")
+        assertTrue(result[1][5] == 2, "user id 2 cost should be 2")
+        assertTrue(result[0].size() == 10)
+
+        // drop value column with mv, not light schema change
+        sql """
+            ALTER TABLE ${tableName} DROP COLUMN cost
+            """
+
+        result = "null"
+        while (!result.contains("FINISHED")){
+            result = sql "SHOW ALTER TABLE COLUMN WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;"
+            result = result.toString()
+            logger.info("result: ${result}")
+            if(result.contains("CANCELLED")) {
+                log.info("rollup job is cancelled, result: ${result}".toString())
+                break
+            }
+            Thread.sleep(1000)
+        }
+
+        sql """ INSERT INTO ${tableName} (`user_id`, `date`, `city`, `age`, `sex`, `max_dwell_time`,`min_dwell_time`, `hll_col`, `bitmap_col`)
+                VALUES
+                (3, '2017-10-01', 'Beijing', 10, 1, 32, 20, hll_hash(4), to_bitmap(4))
+            """
+        result = "null"
+        result = sql """ SELECT * FROM ${tableName} WHERE user_id = 3 """
+
+        assertTrue(result.size() == 1)
+        assertTrue(result[0].size() == 9)
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, 1, 32, 20, hll_hash(5), to_bitmap(5))
+            """
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, 1, 32, 20, hll_hash(5), to_bitmap(5))
+            """
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, 1, 32, 20, hll_hash(5), to_bitmap(5))
+            """
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, 1, 32, 20, hll_hash(5), to_bitmap(5))
+            """
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, 1, 32, 20, hll_hash(5), to_bitmap(5))
+            """
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, 1, 32, 20, hll_hash(5), to_bitmap(5))
+            """
+
+        Thread.sleep(30 * 1000)
+        // compaction
+        String[][] tablets = sql """ show tablets from ${tableName}; """
+        for (String[] tablet in tablets) {
+                String tablet_id = tablet[0]
+                backend_id = tablet[2]
+                logger.info("run compaction:" + tablet_id)
+                StringBuilder sb = new StringBuilder();
+                sb.append("curl -X POST http://")
+                sb.append(backendId_to_backendIP.get(backend_id))
+                sb.append(":")
+                sb.append(backendId_to_backendHttpPort.get(backend_id))
+                sb.append("/api/compaction/run?tablet_id=")
+                sb.append(tablet_id)
+                sb.append("&compact_type=cumulative")
+
+                String command = sb.toString()
+                process = command.execute()
+                code = process.waitFor()
+                err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+                out = process.getText()
+                logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err)
+                //assertEquals(code, 0)
+        }
+
+        // wait for all compactions done
+        for (String[] tablet in tablets) {
+                boolean running = true
+                do {
+                    Thread.sleep(1000)
+                    String tablet_id = tablet[0]
+                    backend_id = tablet[2]
+                    StringBuilder sb = new StringBuilder();
+                    sb.append("curl -X GET http://")
+                    sb.append(backendId_to_backendIP.get(backend_id))
+                    sb.append(":")
+                    sb.append(backendId_to_backendHttpPort.get(backend_id))
+                    sb.append("/api/compaction/run_status?tablet_id=")
+                    sb.append(tablet_id)
+
+                    String command = sb.toString()
+                    process = command.execute()
+                    code = process.waitFor()
+                    err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+                    out = process.getText()
+                    logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err)
+                    assertEquals(code, 0)
+                    def compactionStatus = parseJson(out.trim())
+                    assertEquals("success", compactionStatus.status.toLowerCase())
+                    running = compactionStatus.run_status
+                } while (running)
+        }
+         
+        result = sql """ select count(*) from ${tableName} """
+        assertTrue(result.size() == 1)
+        assertTrue(result[0][0] == 4)
+
+        result = sql """  SELECT * FROM ${tableName} WHERE user_id=2 """
+        assertTrue(result.size() == 1)
+        assertTrue(result[0].size() == 9)
+
+        int rowCount = 0
+        for (String[] tablet in tablets) {
+                String tablet_id = tablet[0]
+                backend_id = tablet[2]
+                StringBuilder sb = new StringBuilder();
+                sb.append("curl -X GET http://")
+                sb.append(backendId_to_backendIP.get(backend_id))
+                sb.append(":")
+                sb.append(backendId_to_backendHttpPort.get(backend_id))
+                sb.append("/api/compaction/show?tablet_id=")
+                sb.append(tablet_id)
+                String command = sb.toString()
+                // wait for cleaning stale_rowsets
+                process = command.execute()
+                code = process.waitFor()
+                err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+                out = process.getText()
+                logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err)
+                assertEquals(code, 0)
+                def tabletJson = parseJson(out.trim())
+                assert tabletJson.rowsets instanceof List
+            for (String rowset in (List<String>) tabletJson.rowsets) {
+                rowCount += Integer.parseInt(rowset.split(" ")[1])
+            }
+        }
+        logger.info("size:" + rowCount)
+        assertTrue(rowCount <= 14)
+        } finally {
+            //try_sql("DROP TABLE IF EXISTS ${tableName}")
+        }
+
+}
+
+    
\ No newline at end of file
diff --git a/regression-test/suites/schema_change/test_agg_rollup_schema_change.groovy b/regression-test/suites/schema_change/test_agg_rollup_schema_change.groovy
new file mode 100644
index 0000000000..c77cb5e946
--- /dev/null
+++ b/regression-test/suites/schema_change/test_agg_rollup_schema_change.groovy
@@ -0,0 +1,256 @@
+      
+// 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.
+
+import org.codehaus.groovy.runtime.IOGroovyMethods
+
+suite ("test_agg_rollup_schema_change") {
+    def tableName = "schema_change_agg_rollup_regression_test"
+
+    try {
+
+        String[][] backends = sql """ show backends; """
+        assertTrue(backends.size() > 0)
+        String backend_id;
+        def backendId_to_backendIP = [:]
+        def backendId_to_backendHttpPort = [:]
+        for (String[] backend in backends) {
+            backendId_to_backendIP.put(backend[0], backend[2])
+            backendId_to_backendHttpPort.put(backend[0], backend[5])
+        }
+
+        backend_id = backendId_to_backendIP.keySet()[0]
+        StringBuilder showConfigCommand = new StringBuilder();
+        showConfigCommand.append("curl -X GET http://")
+        showConfigCommand.append(backendId_to_backendIP.get(backend_id))
+        showConfigCommand.append(":")
+        showConfigCommand.append(backendId_to_backendHttpPort.get(backend_id))
+        showConfigCommand.append("/api/show_config")
+        logger.info(showConfigCommand.toString())
+        def process = showConfigCommand.toString().execute()
+        int code = process.waitFor()
+        String err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+        String out = process.getText()
+        logger.info("Show config: code=" + code + ", out=" + out + ", err=" + err)
+        assertEquals(code, 0)
+        def configList = parseJson(out.trim())
+        assert configList instanceof List
+
+        boolean disableAutoCompaction = true
+        for (Object ele in (List) configList) {
+            assert ele instanceof List<String>
+            if (((List<String>) ele)[0] == "disable_auto_compaction") {
+                disableAutoCompaction = Boolean.parseBoolean(((List<String>) ele)[2])
+            }
+        }
+
+        sql """ DROP TABLE IF EXISTS ${tableName} """
+        sql """
+                CREATE TABLE ${tableName} (
+                    `user_id` LARGEINT NOT NULL COMMENT "用户id",
+                    `date` DATE NOT NULL COMMENT "数据灌入日期时间",
+                    `city` VARCHAR(20) COMMENT "用户所在城市",
+                    `age` SMALLINT COMMENT "用户年龄",
+                    `sex` TINYINT COMMENT "用户性别",
+
+                    `cost` BIGINT SUM DEFAULT "0" COMMENT "用户总消费",
+                    `max_dwell_time` INT MAX DEFAULT "0" COMMENT "用户最大停留时间",
+                    `min_dwell_time` INT MIN DEFAULT "99999" COMMENT "用户最小停留时间",
+                    `hll_col` HLL HLL_UNION NOT NULL COMMENT "HLL列",
+                    `bitmap_col` Bitmap BITMAP_UNION NOT NULL COMMENT "bitmap列")
+                AGGREGATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`)
+                PROPERTIES ( "replication_num" = "1" );
+            """
+
+        //add rollup
+        def result = "null"
+        def rollupName = "rollup_cost"
+        sql "ALTER TABLE ${tableName} ADD ROLLUP ${rollupName}(`user_id`,`date`,`city`,`age`,`sex`, cost);"
+        while (!result.contains("FINISHED")){
+            result = sql "SHOW ALTER TABLE ROLLUP WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;"
+            result = result.toString()
+            logger.info("result: ${result}")
+            if(result.contains("CANCELLED")){
+                break
+            }
+            Thread.sleep(1000)
+        }
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (1, '2017-10-01', 'Beijing', 10, 1, 1, 30, 20, hll_hash(1), to_bitmap(1))
+            """
+        sql """ INSERT INTO ${tableName} VALUES
+                (1, '2017-10-01', 'Beijing', 10, 1, 1, 31, 19, hll_hash(2), to_bitmap(2))
+            """
+        sql """ INSERT INTO ${tableName} VALUES
+                (2, '2017-10-01', 'Beijing', 10, 1, 1, 31, 21, hll_hash(2), to_bitmap(2))
+            """
+        sql """ INSERT INTO ${tableName} VALUES
+                (2, '2017-10-01', 'Beijing', 10, 1, 1, 32, 20, hll_hash(3), to_bitmap(3))
+            """
+
+        result = "null";
+        result = sql """ select * from ${tableName} """
+        assertTrue(result.size() == 2)
+        assertTrue(result[0].size() == 10)
+        assertTrue(result[0][5] == 2, "user id 1 cost should be 2")
+        assertTrue(result[1][5] == 2, "user id 2 cost should be 2")
+        assertTrue(result[0].size() == 10)
+
+        // drop value column with rollup, not light schema change
+        sql """
+            ALTER TABLE ${tableName} DROP COLUMN cost
+            """
+
+        result = "null"
+        while (!result.contains("FINISHED")){
+            result = sql "SHOW ALTER TABLE COLUMN WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;"
+            result = result.toString()
+            logger.info("result: ${result}")
+            if(result.contains("CANCELLED")) {
+                log.info("rollup job is cancelled, result: ${result}".toString())
+                break
+            }
+            Thread.sleep(1000)
+        }
+
+        sql """ INSERT INTO ${tableName} (`user_id`, `date`, `city`, `age`, `sex`, `max_dwell_time`,`min_dwell_time`, `hll_col`, `bitmap_col`)
+                VALUES
+                (3, '2017-10-01', 'Beijing', 10, 1, 32, 20, hll_hash(4), to_bitmap(4))
+            """
+        result = "null"
+        result = sql """ SELECT * FROM ${tableName} WHERE user_id = 3 """
+
+        assertTrue(result.size() == 1)
+        assertTrue(result[0].size() == 9)
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, 1, 32, 20, hll_hash(5), to_bitmap(5))
+            """
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, 1, 32, 20, hll_hash(5), to_bitmap(5))
+            """
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, 1, 32, 20, hll_hash(5), to_bitmap(5))
+            """
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, 1, 32, 20, hll_hash(5), to_bitmap(5))
+            """
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, 1, 32, 20, hll_hash(5), to_bitmap(5))
+            """
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, 1, 32, 20, hll_hash(5), to_bitmap(5))
+            """
+
+        Thread.sleep(30 * 1000)
+        // compaction
+        String[][] tablets = sql """ show tablets from ${tableName}; """
+        for (String[] tablet in tablets) {
+                String tablet_id = tablet[0]
+                backend_id = tablet[2]
+                logger.info("run compaction:" + tablet_id)
+                StringBuilder sb = new StringBuilder();
+                sb.append("curl -X POST http://")
+                sb.append(backendId_to_backendIP.get(backend_id))
+                sb.append(":")
+                sb.append(backendId_to_backendHttpPort.get(backend_id))
+                sb.append("/api/compaction/run?tablet_id=")
+                sb.append(tablet_id)
+                sb.append("&compact_type=cumulative")
+
+                String command = sb.toString()
+                process = command.execute()
+                code = process.waitFor()
+                err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+                out = process.getText()
+                logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err)
+                //assertEquals(code, 0)
+        }
+
+        // wait for all compactions done
+        for (String[] tablet in tablets) {
+                boolean running = true
+                do {
+                    Thread.sleep(1000)
+                    String tablet_id = tablet[0]
+                    backend_id = tablet[2]
+                    StringBuilder sb = new StringBuilder();
+                    sb.append("curl -X GET http://")
+                    sb.append(backendId_to_backendIP.get(backend_id))
+                    sb.append(":")
+                    sb.append(backendId_to_backendHttpPort.get(backend_id))
+                    sb.append("/api/compaction/run_status?tablet_id=")
+                    sb.append(tablet_id)
+
+                    String command = sb.toString()
+                    process = command.execute()
+                    code = process.waitFor()
+                    err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+                    out = process.getText()
+                    logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err)
+                    assertEquals(code, 0)
+                    def compactionStatus = parseJson(out.trim())
+                    assertEquals("success", compactionStatus.status.toLowerCase())
+                    running = compactionStatus.run_status
+                } while (running)
+        }
+         
+        result = sql """ select count(*) from ${tableName} """
+        assertTrue(result.size() == 1)
+        assertTrue(result[0][0] == 4)
+
+        result = sql """  SELECT * FROM ${tableName} WHERE user_id=2 """
+        assertTrue(result.size() == 1)
+        assertTrue(result[0].size() == 9)
+
+        int rowCount = 0
+        for (String[] tablet in tablets) {
+                String tablet_id = tablet[0]
+                backend_id = tablet[2]
+                StringBuilder sb = new StringBuilder();
+                sb.append("curl -X GET http://")
+                sb.append(backendId_to_backendIP.get(backend_id))
+                sb.append(":")
+                sb.append(backendId_to_backendHttpPort.get(backend_id))
+                sb.append("/api/compaction/show?tablet_id=")
+                sb.append(tablet_id)
+                String command = sb.toString()
+                // wait for cleaning stale_rowsets
+                process = command.execute()
+                code = process.waitFor()
+                err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+                out = process.getText()
+                logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err)
+                assertEquals(code, 0)
+                def tabletJson = parseJson(out.trim())
+                assert tabletJson.rowsets instanceof List
+            for (String rowset in (List<String>) tabletJson.rowsets) {
+                rowCount += Integer.parseInt(rowset.split(" ")[1])
+            }
+        }
+        logger.info("size:" + rowCount)
+        assertTrue(rowCount <= 12)
+    } finally {
+            //try_sql("DROP TABLE IF EXISTS ${tableName}")
+    }
+}
diff --git a/regression-test/suites/schema_change/test_agg_vals_schema_change.groovy b/regression-test/suites/schema_change/test_agg_vals_schema_change.groovy
new file mode 100644
index 0000000000..304f3e4033
--- /dev/null
+++ b/regression-test/suites/schema_change/test_agg_vals_schema_change.groovy
@@ -0,0 +1,272 @@
+// 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.
+
+import org.codehaus.groovy.runtime.IOGroovyMethods
+
+suite ("test_agg_vals_schema_change") {
+    def tableName = "schema_change_agg_vals_regression_test"
+
+    try {
+
+        String[][] backends = sql """ show backends; """
+        assertTrue(backends.size() > 0)
+        String backend_id;
+        def backendId_to_backendIP = [:]
+        def backendId_to_backendHttpPort = [:]
+        for (String[] backend in backends) {
+            backendId_to_backendIP.put(backend[0], backend[2])
+            backendId_to_backendHttpPort.put(backend[0], backend[5])
+        }
+
+        backend_id = backendId_to_backendIP.keySet()[0]
+        StringBuilder showConfigCommand = new StringBuilder();
+        showConfigCommand.append("curl -X GET http://")
+        showConfigCommand.append(backendId_to_backendIP.get(backend_id))
+        showConfigCommand.append(":")
+        showConfigCommand.append(backendId_to_backendHttpPort.get(backend_id))
+        showConfigCommand.append("/api/show_config")
+        logger.info(showConfigCommand.toString())
+        def process = showConfigCommand.toString().execute()
+        int code = process.waitFor()
+        String err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+        String out = process.getText()
+        logger.info("Show config: code=" + code + ", out=" + out + ", err=" + err)
+        assertEquals(code, 0)
+        def configList = parseJson(out.trim())
+        assert configList instanceof List
+
+        boolean disableAutoCompaction = true
+        for (Object ele in (List) configList) {
+            assert ele instanceof List<String>
+            if (((List<String>) ele)[0] == "disable_auto_compaction") {
+                disableAutoCompaction = Boolean.parseBoolean(((List<String>) ele)[2])
+            }
+        }
+
+    sql """ DROP TABLE IF EXISTS ${tableName} """
+
+    sql """
+            CREATE TABLE ${tableName} (
+                `user_id` LARGEINT NOT NULL COMMENT "用户id",
+                `date` DATE NOT NULL COMMENT "数据灌入日期时间",
+                `city` VARCHAR(20) COMMENT "用户所在城市",
+                `age` SMALLINT COMMENT "用户年龄",
+                `sex` TINYINT COMMENT "用户性别",
+                `last_visit_date` DATETIME REPLACE DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间",
+                `last_update_date` DATETIME REPLACE DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次更新时间",
+                `last_visit_date_not_null` DATETIME REPLACE NOT NULL DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间",
+                `cost` BIGINT SUM DEFAULT "0" COMMENT "用户总消费",
+                `max_dwell_time` INT MAX DEFAULT "0" COMMENT "用户最大停留时间",
+                `min_dwell_time` INT MIN DEFAULT "99999" COMMENT "用户最小停留时间",
+                `hll_col` HLL HLL_UNION NOT NULL COMMENT "HLL列",
+                `bitmap_col` Bitmap BITMAP_UNION NOT NULL COMMENT "bitmap列")
+            AGGREGATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`)
+            PROPERTIES ( "replication_num" = "1" );
+        """
+
+    sql """ INSERT INTO ${tableName} VALUES
+             (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-01', '2020-01-01', '2020-01-01', 1, 30, 20, hll_hash(1), to_bitmap(1))
+        """
+
+    sql """ INSERT INTO ${tableName} VALUES
+             (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 19, hll_hash(2), to_bitmap(2))
+        """
+
+    sql """ INSERT INTO ${tableName} VALUES
+             (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 21, hll_hash(2), to_bitmap(2))
+        """
+
+    sql """ INSERT INTO ${tableName} VALUES
+             (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, hll_hash(3), to_bitmap(3))
+        """
+    def result1 = sql """
+                       select * from ${tableName}
+                    """
+    assertTrue(result1.size() == 2)
+    assertTrue(result1[0].size() == 13)
+    assertTrue(result1[0][8] == 2, "user id 1 cost should be 2")
+    assertTrue(result1[1][8] == 2, "user id 2 cost should be 2")
+
+    // add column
+    sql """
+        ALTER table ${tableName} ADD COLUMN new_column INT MAX default "1" 
+        """
+
+    def result2 = sql """ SELECT * FROM ${tableName} WHERE user_id=2 """
+    assertTrue(result1[0][8] == 2, "user id 2 cost should be 2")
+
+    sql """ INSERT INTO ${tableName} VALUES
+             (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, hll_hash(4), to_bitmap(4), 2)
+        """
+    result2 = sql """ SELECT * FROM ${tableName} WHERE user_id=2 """
+    assertTrue(result2[0][8] == 3, "user id 2 cost should be 3")
+
+
+    sql """ INSERT INTO ${tableName} (`user_id`,`date`,`city`,`age`,`sex`,`last_visit_date`,`last_update_date`,
+                                      `last_visit_date_not_null`,`cost`,`max_dwell_time`,`min_dwell_time`, `hll_col`, `bitmap_col`)
+            VALUES
+             (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, hll_hash(4), to_bitmap(4))
+        """
+
+    result2 = sql """ SELECT * FROM ${tableName} WHERE user_id=3 """
+
+    assertTrue(result2.size() == 1)
+    assertTrue(result2[0].size() == 14)
+    assertTrue(result2[0][13] == 1, "new add column default value should be 1")
+
+    sql """ INSERT INTO ${tableName} VALUES
+             (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, hll_hash(4), to_bitmap(4), 2)
+        """
+    def result3 = sql """ SELECT * FROM ${tableName} WHERE user_id = 3 """
+
+    assertTrue(result3.size() == 1)
+    assertTrue(result3[0].size() == 14)
+    assertTrue(result3[0][13] == 2, "new add column value is set to 2")
+
+    def result4 = sql """ select count(*) from ${tableName} """
+    logger.info("result4.size:"+result4.size() + " result4[0].size:" + result4[0].size + " " + result4[0][0])
+    assertTrue(result4.size() == 1)
+    assertTrue(result4[0].size() == 1)
+    assertTrue(result4[0][0] == 3, "total count is 3")
+
+    // drop column
+    sql """
+          ALTER TABLE ${tableName} DROP COLUMN last_visit_date
+          """
+    def result5 = sql """ select * from ${tableName} where user_id = 3 """
+    assertTrue(result5.size() == 1)
+    assertTrue(result5[0].size() == 13)
+
+    sql """ INSERT INTO ${tableName} VALUES
+             (4, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, hll_hash(4), to_bitmap(4), 2)
+        """
+
+    def result6 = sql """ select * from ${tableName} where user_id = 4 """
+    assertTrue(result6.size() == 1)
+    assertTrue(result6[0].size() == 13)
+
+    sql """ INSERT INTO ${tableName} VALUES
+             (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, hll_hash(5), to_bitmap(5), 2)
+        """
+    sql """ INSERT INTO ${tableName} VALUES
+             (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, hll_hash(5), to_bitmap(5), 2)
+        """
+    sql """ INSERT INTO ${tableName} VALUES
+             (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, hll_hash(5), to_bitmap(5), 2)
+        """
+    sql """ INSERT INTO ${tableName} VALUES
+             (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, hll_hash(5), to_bitmap(5), 2)
+        """
+    sql """ INSERT INTO ${tableName} VALUES
+             (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, hll_hash(5), to_bitmap(5), 2)
+        """
+    sql """ INSERT INTO ${tableName} VALUES
+             (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, hll_hash(5), to_bitmap(5), 2)
+        """
+
+    Thread.sleep(30 * 1000)
+    // compaction
+    String[][] tablets = sql """ show tablets from ${tableName}; """
+    for (String[] tablet in tablets) {
+            String tablet_id = tablet[0]
+            backend_id = tablet[2]
+            logger.info("run compaction:" + tablet_id)
+            StringBuilder sb = new StringBuilder();
+            sb.append("curl -X POST http://")
+            sb.append(backendId_to_backendIP.get(backend_id))
+            sb.append(":")
+            sb.append(backendId_to_backendHttpPort.get(backend_id))
+            sb.append("/api/compaction/run?tablet_id=")
+            sb.append(tablet_id)
+            sb.append("&compact_type=cumulative")
+
+            String command = sb.toString()
+            process = command.execute()
+            code = process.waitFor()
+            err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+            out = process.getText()
+            logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err)
+            //assertEquals(code, 0)
+    }
+
+    // wait for all compactions done
+    for (String[] tablet in tablets) {
+            boolean running = true
+            do {
+                Thread.sleep(1000)
+                String tablet_id = tablet[0]
+                backend_id = tablet[2]
+                StringBuilder sb = new StringBuilder();
+                sb.append("curl -X GET http://")
+                sb.append(backendId_to_backendIP.get(backend_id))
+                sb.append(":")
+                sb.append(backendId_to_backendHttpPort.get(backend_id))
+                sb.append("/api/compaction/run_status?tablet_id=")
+                sb.append(tablet_id)
+
+                String command = sb.toString()
+                process = command.execute()
+                code = process.waitFor()
+                err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+                out = process.getText()
+                logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err)
+                assertEquals(code, 0)
+                def compactionStatus = parseJson(out.trim())
+                assertEquals("success", compactionStatus.status.toLowerCase())
+                running = compactionStatus.run_status
+            } while (running)
+    }
+    def result7 = sql """ select count(*) from ${tableName} """
+    assertTrue(result7.size() == 1)
+    assertTrue(result7[0][0] == 5)
+
+    def result8 = sql """  SELECT * FROM ${tableName} WHERE user_id=2 """
+    assertTrue(result8.size() == 1)
+    assertTrue(result8[0].size() == 13)
+
+    int rowCount = 0
+    for (String[] tablet in tablets) {
+            String tablet_id = tablet[0]
+            backend_id = tablet[2]
+            StringBuilder sb = new StringBuilder();
+            sb.append("curl -X GET http://")
+            sb.append(backendId_to_backendIP.get(backend_id))
+            sb.append(":")
+            sb.append(backendId_to_backendHttpPort.get(backend_id))
+            sb.append("/api/compaction/show?tablet_id=")
+            sb.append(tablet_id)
+            String command = sb.toString()
+            // wait for cleaning stale_rowsets
+            process = command.execute()
+            code = process.waitFor()
+            err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+            out = process.getText()
+            logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err)
+            assertEquals(code, 0)
+            def tabletJson = parseJson(out.trim())
+            assert tabletJson.rowsets instanceof List
+        for (String rowset in (List<String>) tabletJson.rowsets) {
+            rowCount += Integer.parseInt(rowset.split(" ")[1])
+        }
+    }
+    logger.info("size:" + rowCount)
+    assertTrue(rowCount <= 8)
+    } finally {
+        //try_sql("DROP TABLE IF EXISTS ${tableName}")
+    }
+
+}
diff --git a/regression-test/suites/schema_change/test_delete_schema_change.sql b/regression-test/suites/schema_change/test_delete_schema_change.sql
new file mode 100644
index 0000000000..e3a92a38f5
--- /dev/null
+++ b/regression-test/suites/schema_change/test_delete_schema_change.sql
@@ -0,0 +1,44 @@
+DROP TABLE IF EXISTS schema_change_delete_regression_test;
+
+CREATE TABLE schema_change_delete_regression_test (
+                `user_id` LARGEINT NOT NULL COMMENT "用户id",
+                `date` DATE NOT NULL COMMENT "数据灌入日期时间",
+                `city` VARCHAR(20) COMMENT "用户所在城市",
+                `age` SMALLINT COMMENT "用户年龄",
+                `sex` TINYINT COMMENT "用户性别",
+                `last_visit_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间",
+                `last_update_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次更新时间",
+                `last_visit_date_not_null` DATETIME NOT NULL DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间",
+                `cost` BIGINT DEFAULT "0" COMMENT "用户总消费",
+                `max_dwell_time` INT DEFAULT "0" COMMENT "用户最大停留时间",
+                `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间")
+            DUPLICATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`)
+            PROPERTIES ( "replication_num" = "1" );
+
+INSERT INTO schema_change_delete_regression_test VALUES
+             (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-01', '2020-01-01', '2020-01-01', 1, 30, 20);
+
+INSERT INTO schema_change_delete_regression_test VALUES
+             (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 21);
+
+SELECT * FROM schema_change_delete_regression_test order by user_id ASC, last_visit_date;
+
+ALTER table schema_change_delete_regression_test ADD COLUMN new_column INT default "1";
+
+SELECT * FROM schema_change_delete_regression_test order by user_id DESC, last_visit_date;
+
+INSERT INTO schema_change_delete_regression_test VALUES
+             (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 19, 2);
+
+INSERT INTO schema_change_delete_regression_test VALUES
+             (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2);
+
+INSERT INTO schema_change_delete_regression_test VALUES
+             (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 1);
+
+INSERT INTO schema_change_delete_regression_test VALUES
+             (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2);
+             
+DELETE FROM schema_change_delete_regression_test where new_column = 1;
+
+SELECT * FROM schema_change_delete_regression_test order by user_id DESC, last_visit_date;
\ No newline at end of file
diff --git a/regression-test/suites/schema_change/test_dup_keys_schema_change.groovy b/regression-test/suites/schema_change/test_dup_keys_schema_change.groovy
new file mode 100644
index 0000000000..6711429d5a
--- /dev/null
+++ b/regression-test/suites/schema_change/test_dup_keys_schema_change.groovy
@@ -0,0 +1,273 @@
+// 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.
+
+import org.codehaus.groovy.runtime.IOGroovyMethods
+
+suite ("test_dup_keys_schema_change") {
+    def tableName = "schema_change_dup_keys_regression_test"
+
+    try {
+        String[][] backends = sql """ show backends; """
+        assertTrue(backends.size() > 0)
+        String backend_id;
+        def backendId_to_backendIP = [:]
+        def backendId_to_backendHttpPort = [:]
+        for (String[] backend in backends) {
+            backendId_to_backendIP.put(backend[0], backend[2])
+            backendId_to_backendHttpPort.put(backend[0], backend[5])
+        }
+
+        backend_id = backendId_to_backendIP.keySet()[0]
+        StringBuilder showConfigCommand = new StringBuilder();
+        showConfigCommand.append("curl -X GET http://")
+        showConfigCommand.append(backendId_to_backendIP.get(backend_id))
+        showConfigCommand.append(":")
+        showConfigCommand.append(backendId_to_backendHttpPort.get(backend_id))
+        showConfigCommand.append("/api/show_config")
+        logger.info(showConfigCommand.toString())
+        def process = showConfigCommand.toString().execute()
+        int code = process.waitFor()
+        String err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+        String out = process.getText()
+        logger.info("Show config: code=" + code + ", out=" + out + ", err=" + err)
+        assertEquals(code, 0)
+        def configList = parseJson(out.trim())
+        assert configList instanceof List
+
+        boolean disableAutoCompaction = true
+        for (Object ele in (List) configList) {
+            assert ele instanceof List<String>
+            if (((List<String>) ele)[0] == "disable_auto_compaction") {
+                disableAutoCompaction = Boolean.parseBoolean(((List<String>) ele)[2])
+            }
+        }
+
+        sql """ DROP TABLE IF EXISTS ${tableName} """
+
+        sql """
+                CREATE TABLE ${tableName} (
+                    `user_id` LARGEINT NOT NULL COMMENT "用户id",
+                    `date` DATE NOT NULL COMMENT "数据灌入日期时间",
+                    `city` VARCHAR(20) COMMENT "用户所在城市",
+                    `age` SMALLINT COMMENT "用户年龄",
+                    `sex` TINYINT COMMENT "用户性别",
+                    `last_visit_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间",
+                    `last_update_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次更新时间",
+                    `last_visit_date_not_null` DATETIME NOT NULL DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间",
+                    `cost` BIGINT DEFAULT "0" COMMENT "用户总消费",
+                    `max_dwell_time` INT DEFAULT "0" COMMENT "用户最大停留时间",
+                    `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间")
+                DUPLICATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`)
+                PROPERTIES ( "replication_num" = "1" );
+            """
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-01', '2020-01-01', '2020-01-01', 1, 30, 20)
+            """
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 19)
+            """
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 21)
+            """
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20)
+            """
+        def result = sql """
+                        select count(*) from ${tableName}
+                        """
+        assertTrue(result.size() == 1)
+        assertTrue(result[0].size() == 1)
+        assertTrue(result[0][0] == 4, "total columns should be 4 rows")
+
+        // add column
+        sql """
+            ALTER table ${tableName} ADD COLUMN new_column INT default "1" 
+            """
+
+        sql """ SELECT * FROM ${tableName} WHERE user_id=2 """
+
+        sql """ INSERT INTO ${tableName} (`user_id`,`date`,`city`,`age`,`sex`,`last_visit_date`,`last_update_date`,
+                                        `last_visit_date_not_null`,`cost`,`max_dwell_time`,`min_dwell_time`)
+                VALUES
+                (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20)
+            """
+
+        result = sql """ SELECT * FROM ${tableName} WHERE user_id=3 """
+
+        assertTrue(result.size() == 1)
+        assertTrue(result[0].size() == 12)
+        assertTrue(result[0][11] == 1, "new add column default value should be 1")
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+        result = sql """ SELECT * FROM ${tableName} WHERE user_id = 3 """
+
+        assertTrue(result.size() == 2)
+        assertTrue(result[0].size() == 12)
+        assertTrue(result[1][11] == 2, "new add column value is set to 2")
+
+        result = sql """ select count(*) from ${tableName} """
+        logger.info("result.size:" + result.size() + " result[0].size:" + result[0].size + " " + result[0][0])
+        assertTrue(result.size() == 1)
+        assertTrue(result[0].size() == 1)
+        assertTrue(result[0][0] == 6, "total count is 6")
+
+        // drop column
+        sql """
+            ALTER TABLE ${tableName} DROP COLUMN sex
+            """
+        result = "null"
+        while (!result.contains("FINISHED")){
+            result = sql "SHOW ALTER TABLE COLUMN WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;"
+            result = result.toString()
+            logger.info("result: ${result}")
+            if(result.contains("CANCELLED")) {
+                log.info("rollup job is cancelled, result: ${result}".toString())
+                break
+            }
+            Thread.sleep(1000)
+        }
+        result = sql """ select * from ${tableName} where user_id = 3 """
+        assertTrue(result.size() == 2)
+        assertTrue(result[0].size() == 11)
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (4, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+
+        result = sql """ select * from ${tableName} where user_id = 4 """
+        assertTrue(result.size() == 1)
+        assertTrue(result[0].size() == 11)
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+
+        Thread.sleep(30 * 1000)
+        // compaction
+        String[][] tablets = sql """ show tablets from ${tableName}; """
+        for (String[] tablet in tablets) {
+                String tablet_id = tablet[0]
+                backend_id = tablet[2]
+                logger.info("run compaction:" + tablet_id)
+                StringBuilder sb = new StringBuilder();
+                sb.append("curl -X POST http://")
+                sb.append(backendId_to_backendIP.get(backend_id))
+                sb.append(":")
+                sb.append(backendId_to_backendHttpPort.get(backend_id))
+                sb.append("/api/compaction/run?tablet_id=")
+                sb.append(tablet_id)
+                sb.append("&compact_type=cumulative")
+
+                String command = sb.toString()
+                process = command.execute()
+                code = process.waitFor()
+                err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+                out = process.getText()
+                logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err)
+                //assertEquals(code, 0)
+        }
+
+        // wait for all compactions done
+        for (String[] tablet in tablets) {
+                boolean running = true
+                do {
+                    Thread.sleep(1000)
+                    String tablet_id = tablet[0]
+                    backend_id = tablet[2]
+                    StringBuilder sb = new StringBuilder();
+                    sb.append("curl -X GET http://")
+                    sb.append(backendId_to_backendIP.get(backend_id))
+                    sb.append(":")
+                    sb.append(backendId_to_backendHttpPort.get(backend_id))
+                    sb.append("/api/compaction/run_status?tablet_id=")
+                    sb.append(tablet_id)
+
+                    String command = sb.toString()
+                    process = command.execute()
+                    code = process.waitFor()
+                    err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+                    out = process.getText()
+                    logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err)
+                    assertEquals(code, 0)
+                    def compactionStatus = parseJson(out.trim())
+                    assertEquals("success", compactionStatus.status.toLowerCase())
+                    running = compactionStatus.run_status
+                } while (running)
+        }
+
+        result = sql """ select count(*) from ${tableName} """
+        assertTrue(result.size() == 1)
+        assertTrue(result[0][0] == 13)
+
+
+        result = sql """  SELECT * FROM ${tableName} WHERE user_id=2 """
+        assertTrue(result.size() == 2)
+        assertTrue(result[0].size() == 11)
+
+        int rowCount = 0
+        for (String[] tablet in tablets) {
+                String tablet_id = tablet[0]
+                backend_id = tablet[2]
+                StringBuilder sb = new StringBuilder();
+                sb.append("curl -X GET http://")
+                sb.append(backendId_to_backendIP.get(backend_id))
+                sb.append(":")
+                sb.append(backendId_to_backendHttpPort.get(backend_id))
+                sb.append("/api/compaction/show?tablet_id=")
+                sb.append(tablet_id)
+                String command = sb.toString()
+                // wait for cleaning stale_rowsets
+                process = command.execute()
+                code = process.waitFor()
+                err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+                out = process.getText()
+                logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err)
+                assertEquals(code, 0)
+                def tabletJson = parseJson(out.trim())
+                assert tabletJson.rowsets instanceof List
+            for (String rowset in (List<String>) tabletJson.rowsets) {
+                rowCount += Integer.parseInt(rowset.split(" ")[1])
+            }
+        }
+        logger.info("size:" + rowCount)
+        assertTrue(rowCount < 10)
+    } finally {
+        //try_sql("DROP TABLE IF EXISTS ${tableName}")
+    }
+
+}
diff --git a/regression-test/suites/schema_change/test_dup_mv_schema_change.groovy b/regression-test/suites/schema_change/test_dup_mv_schema_change.groovy
new file mode 100644
index 0000000000..810d75d7b8
--- /dev/null
+++ b/regression-test/suites/schema_change/test_dup_mv_schema_change.groovy
@@ -0,0 +1,288 @@
+// 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.
+
+import org.codehaus.groovy.runtime.IOGroovyMethods
+
+suite ("test_dup_mv_schema_change") {
+    def tableName = "schema_change_dup_mv_regression_test"
+
+    try {
+        String[][] backends = sql """ show backends; """
+        assertTrue(backends.size() > 0)
+        String backend_id;
+        def backendId_to_backendIP = [:]
+        def backendId_to_backendHttpPort = [:]
+        for (String[] backend in backends) {
+            backendId_to_backendIP.put(backend[0], backend[2])
+            backendId_to_backendHttpPort.put(backend[0], backend[5])
+        }
+
+        backend_id = backendId_to_backendIP.keySet()[0]
+        StringBuilder showConfigCommand = new StringBuilder();
+        showConfigCommand.append("curl -X GET http://")
+        showConfigCommand.append(backendId_to_backendIP.get(backend_id))
+        showConfigCommand.append(":")
+        showConfigCommand.append(backendId_to_backendHttpPort.get(backend_id))
+        showConfigCommand.append("/api/show_config")
+        logger.info(showConfigCommand.toString())
+        def process = showConfigCommand.toString().execute()
+        int code = process.waitFor()
+        String err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+        String out = process.getText()
+        logger.info("Show config: code=" + code + ", out=" + out + ", err=" + err)
+        assertEquals(code, 0)
+        def configList = parseJson(out.trim())
+        assert configList instanceof List
+
+        boolean disableAutoCompaction = true
+        for (Object ele in (List) configList) {
+            assert ele instanceof List<String>
+            if (((List<String>) ele)[0] == "disable_auto_compaction") {
+                disableAutoCompaction = Boolean.parseBoolean(((List<String>) ele)[2])
+            }
+        }
+
+        sql """ DROP TABLE IF EXISTS ${tableName} """
+
+        sql """
+                CREATE TABLE ${tableName} (
+                    `user_id` LARGEINT NOT NULL COMMENT "用户id",
+                    `date` DATE NOT NULL COMMENT "数据灌入日期时间",
+                    `city` VARCHAR(20) COMMENT "用户所在城市",
+                    `age` SMALLINT COMMENT "用户年龄",
+                    `sex` TINYINT COMMENT "用户性别",
+                    `last_visit_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间",
+                    `last_update_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次更新时间",
+                    `last_visit_date_not_null` DATETIME NOT NULL DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间",
+                    `cost` BIGINT DEFAULT "0" COMMENT "用户总消费",
+                    `max_dwell_time` INT DEFAULT "0" COMMENT "用户最大停留时间",
+                    `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间")
+                DUPLICATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`)
+                PROPERTIES ( "replication_num" = "1" );
+            """
+
+        //add materialized view
+        def result = "null"
+        def mvName = "mv1"
+        sql "create materialized view ${mvName} as select user_id, date, city, age,sex from ${tableName};"
+        while (!result.contains("FINISHED")){
+            result = sql "SHOW ALTER TABLE MATERIALIZED VIEW WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;"
+            result = result.toString()
+            logger.info("result: ${result}")
+            if(result.contains("CANCELLED")){
+                break
+            }
+            Thread.sleep(1000)
+        }
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-01', '2020-01-01', '2020-01-01', 1, 30, 20)
+            """
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 19)
+            """
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 21)
+            """
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20)
+            """
+        result = sql """
+                        select count(*) from ${tableName}
+                        """
+        assertTrue(result.size() == 1)
+        assertTrue(result[0].size() == 1)
+        assertTrue(result[0][0] == 4, "total columns should be 4 rows")
+
+        // add column
+        sql """
+            ALTER table ${tableName} ADD COLUMN new_column INT default "1" 
+            """
+
+        sql """ SELECT * FROM ${tableName} WHERE user_id=2 """
+
+        sql """ INSERT INTO ${tableName} (`user_id`,`date`,`city`,`age`,`sex`,`last_visit_date`,`last_update_date`,
+                                        `last_visit_date_not_null`,`cost`,`max_dwell_time`,`min_dwell_time`)
+                VALUES
+                (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20)
+            """
+
+        result = sql """ SELECT * FROM ${tableName} WHERE user_id=3 """
+
+        assertTrue(result.size() == 1)
+        assertTrue(result[0].size() == 12)
+        assertTrue(result[0][11] == 1, "new add column default value should be 1")
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+        result = sql """ SELECT * FROM ${tableName} WHERE user_id = 3 """
+
+        assertTrue(result.size() == 2)
+        assertTrue(result[0].size() == 12)
+        assertTrue(result[1][11] == 2, "new add column value is set to 2")
+
+        result = sql """ select count(*) from ${tableName} """
+        logger.info("result.size:" + result.size() + " result[0].size:" + result[0].size + " " + result[0][0])
+        assertTrue(result.size() == 1)
+        assertTrue(result[0].size() == 1)
+        assertTrue(result[0][0] == 6, "total count is 6")
+
+        // drop column
+        sql """
+            ALTER TABLE ${tableName} DROP COLUMN sex
+            """
+        result = "null"
+        while (!result.contains("FINISHED")){
+            result = sql "SHOW ALTER TABLE COLUMN WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;"
+            result = result.toString()
+            logger.info("result: ${result}")
+            if(result.contains("CANCELLED")) {
+                log.info("rollup job is cancelled, result: ${result}".toString())
+                break
+            }
+            Thread.sleep(1000)
+        }
+
+        result = sql """ select * from ${tableName} where user_id = 3 """
+        assertTrue(result.size() == 2)
+        assertTrue(result[0].size() == 11)
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (4, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+
+        result = sql """ select * from ${tableName} where user_id = 4 """
+        assertTrue(result.size() == 1)
+        assertTrue(result[0].size() == 11)
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+
+        Thread.sleep(30 * 1000)
+        // compaction
+        String[][] tablets = sql """ show tablets from ${tableName}; """
+        for (String[] tablet in tablets) {
+                String tablet_id = tablet[0]
+                backend_id = tablet[2]
+                logger.info("run compaction:" + tablet_id)
+                StringBuilder sb = new StringBuilder();
+                sb.append("curl -X POST http://")
+                sb.append(backendId_to_backendIP.get(backend_id))
+                sb.append(":")
+                sb.append(backendId_to_backendHttpPort.get(backend_id))
+                sb.append("/api/compaction/run?tablet_id=")
+                sb.append(tablet_id)
+                sb.append("&compact_type=cumulative")
+
+                String command = sb.toString()
+                process = command.execute()
+                code = process.waitFor()
+                err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+                out = process.getText()
+                logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err)
+                //assertEquals(code, 0)
+        }
+
+        // wait for all compactions done
+        for (String[] tablet in tablets) {
+                boolean running = true
+                do {
+                    Thread.sleep(1000)
+                    String tablet_id = tablet[0]
+                    backend_id = tablet[2]
+                    StringBuilder sb = new StringBuilder();
+                    sb.append("curl -X GET http://")
+                    sb.append(backendId_to_backendIP.get(backend_id))
+                    sb.append(":")
+                    sb.append(backendId_to_backendHttpPort.get(backend_id))
+                    sb.append("/api/compaction/run_status?tablet_id=")
+                    sb.append(tablet_id)
+
+                    String command = sb.toString()
+                    process = command.execute()
+                    code = process.waitFor()
+                    err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+                    out = process.getText()
+                    logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err)
+                    assertEquals(code, 0)
+                    def compactionStatus = parseJson(out.trim())
+                    assertEquals("success", compactionStatus.status.toLowerCase())
+                    running = compactionStatus.run_status
+                } while (running)
+        }
+
+        result = sql """ select count(*) from ${tableName} """
+        assertTrue(result.size() == 1)
+        assertTrue(result[0][0] == 13)
+
+
+        result = sql """  SELECT * FROM ${tableName} WHERE user_id=2 """
+        assertTrue(result.size() == 2)
+        assertTrue(result[0].size() == 11)
+
+        int rowCount = 0
+        for (String[] tablet in tablets) {
+                String tablet_id = tablet[0]
+                backend_id = tablet[2]
+                StringBuilder sb = new StringBuilder();
+                sb.append("curl -X GET http://")
+                sb.append(backendId_to_backendIP.get(backend_id))
+                sb.append(":")
+                sb.append(backendId_to_backendHttpPort.get(backend_id))
+                sb.append("/api/compaction/show?tablet_id=")
+                sb.append(tablet_id)
+                String command = sb.toString()
+                // wait for cleaning stale_rowsets
+                process = command.execute()
+                code = process.waitFor()
+                err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+                out = process.getText()
+                logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err)
+                assertEquals(code, 0)
+                def tabletJson = parseJson(out.trim())
+                assert tabletJson.rowsets instanceof List
+            for (String rowset in (List<String>) tabletJson.rowsets) {
+                rowCount += Integer.parseInt(rowset.split(" ")[1])
+            }
+        }
+        logger.info("size:" + rowCount)
+        assertTrue(rowCount <= 14)
+    } finally {
+        //try_sql("DROP TABLE IF EXISTS ${tableName}")
+    }
+
+}
diff --git a/regression-test/suites/schema_change/test_dup_rollup_schema_change.groovy b/regression-test/suites/schema_change/test_dup_rollup_schema_change.groovy
new file mode 100644
index 0000000000..6c1e8c2e42
--- /dev/null
+++ b/regression-test/suites/schema_change/test_dup_rollup_schema_change.groovy
@@ -0,0 +1,287 @@
+// 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.
+
+import org.codehaus.groovy.runtime.IOGroovyMethods
+
+suite ("test_dup_rollup_schema_change") {
+    def tableName = "schema_change_dup_rollup_regression_test"
+
+    try {
+        String[][] backends = sql """ show backends; """
+        assertTrue(backends.size() > 0)
+        String backend_id;
+        def backendId_to_backendIP = [:]
+        def backendId_to_backendHttpPort = [:]
+        for (String[] backend in backends) {
+            backendId_to_backendIP.put(backend[0], backend[2])
+            backendId_to_backendHttpPort.put(backend[0], backend[5])
+        }
+
+        backend_id = backendId_to_backendIP.keySet()[0]
+        StringBuilder showConfigCommand = new StringBuilder();
+        showConfigCommand.append("curl -X GET http://")
+        showConfigCommand.append(backendId_to_backendIP.get(backend_id))
+        showConfigCommand.append(":")
+        showConfigCommand.append(backendId_to_backendHttpPort.get(backend_id))
+        showConfigCommand.append("/api/show_config")
+        logger.info(showConfigCommand.toString())
+        def process = showConfigCommand.toString().execute()
+        int code = process.waitFor()
+        String err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+        String out = process.getText()
+        logger.info("Show config: code=" + code + ", out=" + out + ", err=" + err)
+        assertEquals(code, 0)
+        def configList = parseJson(out.trim())
+        assert configList instanceof List
+
+        boolean disableAutoCompaction = true
+        for (Object ele in (List) configList) {
+            assert ele instanceof List<String>
+            if (((List<String>) ele)[0] == "disable_auto_compaction") {
+                disableAutoCompaction = Boolean.parseBoolean(((List<String>) ele)[2])
+            }
+        }
+        sql """ DROP TABLE IF EXISTS ${tableName} """
+
+        sql """
+                CREATE TABLE ${tableName} (
+                    `user_id` LARGEINT NOT NULL COMMENT "用户id",
+                    `date` DATE NOT NULL COMMENT "数据灌入日期时间",
+                    `city` VARCHAR(20) COMMENT "用户所在城市",
+                    `age` SMALLINT COMMENT "用户年龄",
+                    `sex` TINYINT COMMENT "用户性别",
+                    `last_visit_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间",
+                    `last_update_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次更新时间",
+                    `last_visit_date_not_null` DATETIME NOT NULL DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间",
+                    `cost` BIGINT DEFAULT "0" COMMENT "用户总消费",
+                    `max_dwell_time` INT DEFAULT "0" COMMENT "用户最大停留时间",
+                    `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间")
+                DUPLICATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`)
+                PROPERTIES ( "replication_num" = "1" );
+            """
+
+        //add rollup
+        def result = "null"
+        def rollupName = "rollup_cost"
+        sql "ALTER TABLE ${tableName} ADD ROLLUP ${rollupName}(`user_id`,`date`,`city`,`age`,`sex`, cost);"
+        while (!result.contains("FINISHED")){
+            result = sql "SHOW ALTER TABLE ROLLUP WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;"
+            result = result.toString()
+            logger.info("result: ${result}")
+            if(result.contains("CANCELLED")){
+                break
+            }
+            Thread.sleep(1000)
+        }
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-01', '2020-01-01', '2020-01-01', 1, 30, 20)
+            """
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 19)
+            """
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 21)
+            """
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20)
+            """
+        result = sql """
+                        select count(*) from ${tableName}
+                        """
+        assertTrue(result.size() == 1)
+        assertTrue(result[0].size() == 1)
+        assertTrue(result[0][0] == 4, "total columns should be 4 rows")
+
+        // add column
+        sql """
+            ALTER table ${tableName} ADD COLUMN new_column INT default "1" 
+            """
+
+        sql """ SELECT * FROM ${tableName} WHERE user_id=2 """
+
+        sql """ INSERT INTO ${tableName} (`user_id`,`date`,`city`,`age`,`sex`,`last_visit_date`,`last_update_date`,
+                                        `last_visit_date_not_null`,`cost`,`max_dwell_time`,`min_dwell_time`)
+                VALUES
+                (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20)
+            """
+
+        result = sql """ SELECT * FROM ${tableName} WHERE user_id=3 """
+
+        assertTrue(result.size() == 1)
+        assertTrue(result[0].size() == 12)
+        assertTrue(result[0][11] == 1, "new add column default value should be 1")
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+        result = sql """ SELECT * FROM ${tableName} WHERE user_id = 3 """
+
+        assertTrue(result.size() == 2)
+        assertTrue(result[0].size() == 12)
+        assertTrue(result[1][11] == 2, "new add column value is set to 2")
+
+        result = sql """ select count(*) from ${tableName} """
+        logger.info("result.size:" + result.size() + " result[0].size:" + result[0].size + " " + result[0][0])
+        assertTrue(result.size() == 1)
+        assertTrue(result[0].size() == 1)
+        assertTrue(result[0][0] == 6, "total count is 6")
+
+        // drop column
+        sql """
+            ALTER TABLE ${tableName} DROP COLUMN sex
+            """
+        result = "null"
+        while (!result.contains("FINISHED")){
+            result = sql "SHOW ALTER TABLE COLUMN WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;"
+            result = result.toString()
+            logger.info("result: ${result}")
+            if(result.contains("CANCELLED")) {
+                log.info("rollup job is cancelled, result: ${result}".toString())
+                break
+            }
+            Thread.sleep(1000)
+        }
+
+        result = sql """ select * from ${tableName} where user_id = 3 """
+        assertTrue(result.size() == 2)
+        assertTrue(result[0].size() == 11)
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (4, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+
+        result = sql """ select * from ${tableName} where user_id = 4 """
+        assertTrue(result.size() == 1)
+        assertTrue(result[0].size() == 11)
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+
+        Thread.sleep(30 * 1000)
+        // compaction
+        String[][] tablets = sql """ show tablets from ${tableName}; """
+        for (String[] tablet in tablets) {
+                String tablet_id = tablet[0]
+                backend_id = tablet[2]
+                logger.info("run compaction:" + tablet_id)
+                StringBuilder sb = new StringBuilder();
+                sb.append("curl -X POST http://")
+                sb.append(backendId_to_backendIP.get(backend_id))
+                sb.append(":")
+                sb.append(backendId_to_backendHttpPort.get(backend_id))
+                sb.append("/api/compaction/run?tablet_id=")
+                sb.append(tablet_id)
+                sb.append("&compact_type=cumulative")
+
+                String command = sb.toString()
+                process = command.execute()
+                code = process.waitFor()
+                err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+                out = process.getText()
+                logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err)
+                //assertEquals(code, 0)
+        }
+
+        // wait for all compactions done
+        for (String[] tablet in tablets) {
+                boolean running = true
+                do {
+                    Thread.sleep(1000)
+                    String tablet_id = tablet[0]
+                    backend_id = tablet[2]
+                    StringBuilder sb = new StringBuilder();
+                    sb.append("curl -X GET http://")
+                    sb.append(backendId_to_backendIP.get(backend_id))
+                    sb.append(":")
+                    sb.append(backendId_to_backendHttpPort.get(backend_id))
+                    sb.append("/api/compaction/run_status?tablet_id=")
+                    sb.append(tablet_id)
+
+                    String command = sb.toString()
+                    process = command.execute()
+                    code = process.waitFor()
+                    err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+                    out = process.getText()
+                    logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err)
+                    assertEquals(code, 0)
+                    def compactionStatus = parseJson(out.trim())
+                    assertEquals("success", compactionStatus.status.toLowerCase())
+                    running = compactionStatus.run_status
+                } while (running)
+        }
+
+        result = sql """ select count(*) from ${tableName} """
+        assertTrue(result.size() == 1)
+        assertTrue(result[0][0] == 13)
+
+
+        result = sql """  SELECT * FROM ${tableName} WHERE user_id=2 """
+        assertTrue(result.size() == 2)
+        assertTrue(result[0].size() == 11)
+
+        int rowCount = 0
+        for (String[] tablet in tablets) {
+                String tablet_id = tablet[0]
+                StringBuilder sb = new StringBuilder();
+                backend_id = tablet[2]
+                sb.append("curl -X GET http://")
+                sb.append(backendId_to_backendIP.get(backend_id))
+                sb.append(":")
+                sb.append(backendId_to_backendHttpPort.get(backend_id))
+                sb.append("/api/compaction/show?tablet_id=")
+                sb.append(tablet_id)
+                String command = sb.toString()
+                // wait for cleaning stale_rowsets
+                process = command.execute()
+                code = process.waitFor()
+                err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+                out = process.getText()
+                logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err)
+                assertEquals(code, 0)
+                def tabletJson = parseJson(out.trim())
+                assert tabletJson.rowsets instanceof List
+            for (String rowset in (List<String>) tabletJson.rowsets) {
+                rowCount += Integer.parseInt(rowset.split(" ")[1])
+            }
+        }
+        logger.info("size:" + rowCount)
+        assertTrue(rowCount <= 14)
+    } finally {
+        //try_sql("DROP TABLE IF EXISTS ${tableName}")
+    }
+
+}
diff --git a/regression-test/suites/schema_change/test_dup_vals_schema_change.groovy b/regression-test/suites/schema_change/test_dup_vals_schema_change.groovy
new file mode 100644
index 0000000000..e10e0724ce
--- /dev/null
+++ b/regression-test/suites/schema_change/test_dup_vals_schema_change.groovy
@@ -0,0 +1,259 @@
+// 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.
+
+import org.codehaus.groovy.runtime.IOGroovyMethods
+
+suite ("test_dup_vals_schema_change") {
+    def tableName = "schema_change_dup_vals_regression_test"
+
+    try {
+        String[][] backends = sql """ show backends; """
+        assertTrue(backends.size() > 0)
+        String backend_id;
+        def backendId_to_backendIP = [:]
+        def backendId_to_backendHttpPort = [:]
+        for (String[] backend in backends) {
+            backendId_to_backendIP.put(backend[0], backend[2])
+            backendId_to_backendHttpPort.put(backend[0], backend[5])
+        }
+
+        backend_id = backendId_to_backendIP.keySet()[0]
+        StringBuilder showConfigCommand = new StringBuilder();
+        showConfigCommand.append("curl -X GET http://")
+        showConfigCommand.append(backendId_to_backendIP.get(backend_id))
+        showConfigCommand.append(":")
+        showConfigCommand.append(backendId_to_backendHttpPort.get(backend_id))
+        showConfigCommand.append("/api/show_config")
+        logger.info(showConfigCommand.toString())
+        def process = showConfigCommand.toString().execute()
+        int code = process.waitFor()
+        String err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+        String out = process.getText()
+        logger.info("Show config: code=" + code + ", out=" + out + ", err=" + err)
+        assertEquals(code, 0)
+        def configList = parseJson(out.trim())
+        assert configList instanceof List
+
+        boolean disableAutoCompaction = true
+        for (Object ele in (List) configList) {
+            assert ele instanceof List<String>
+            if (((List<String>) ele)[0] == "disable_auto_compaction") {
+                disableAutoCompaction = Boolean.parseBoolean(((List<String>) ele)[2])
+            }
+        }
+        sql """ DROP TABLE IF EXISTS ${tableName} """
+
+        sql """
+                CREATE TABLE ${tableName} (
+                    `user_id` LARGEINT NOT NULL COMMENT "用户id",
+                    `date` DATE NOT NULL COMMENT "数据灌入日期时间",
+                    `city` VARCHAR(20) COMMENT "用户所在城市",
+                    `age` SMALLINT COMMENT "用户年龄",
+                    `sex` TINYINT COMMENT "用户性别",
+                    `last_visit_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间",
+                    `last_update_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次更新时间",
+                    `last_visit_date_not_null` DATETIME NOT NULL DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间",
+                    `cost` BIGINT DEFAULT "0" COMMENT "用户总消费",
+                    `max_dwell_time` INT DEFAULT "0" COMMENT "用户最大停留时间",
+                    `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间")
+                DUPLICATE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`)
+                PROPERTIES ( "replication_num" = "1" );
+            """
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-01', '2020-01-01', '2020-01-01', 1, 30, 20)
+            """
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 19)
+            """
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 21)
+            """
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20)
+            """
+        def result1 = sql """
+                        select count(*) from ${tableName}
+                        """
+        assertTrue(result1.size() == 1)
+        assertTrue(result1[0].size() == 1)
+        assertTrue(result1[0][0] == 4, "total columns should be 4 rows")
+
+        // add column
+        sql """
+            ALTER table ${tableName} ADD COLUMN new_column INT default "1" 
+            """
+
+        sql """ SELECT * FROM ${tableName} WHERE user_id=2 """
+
+        sql """ INSERT INTO ${tableName} (`user_id`,`date`,`city`,`age`,`sex`,`last_visit_date`,`last_update_date`,
+                                        `last_visit_date_not_null`,`cost`,`max_dwell_time`,`min_dwell_time`)
+                VALUES
+                (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20)
+            """
+
+        def result2 = sql """ SELECT * FROM ${tableName} WHERE user_id=3 """
+
+        assertTrue(result2.size() == 1)
+        assertTrue(result2[0].size() == 12)
+        assertTrue(result2[0][11] == 1, "new add column default value should be 1")
+
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+        def result3 = sql """ SELECT * FROM ${tableName} WHERE user_id = 3 """
+
+        assertTrue(result3.size() == 2)
+        assertTrue(result3[0].size() == 12)
+        assertTrue(result3[1][11] == 2, "new add column value is set to 2")
+
+        def result4 = sql """ select count(*) from ${tableName} """
+        logger.info("result4.size:"+result4.size() + " result4[0].size:" + result4[0].size + " " + result4[0][0])
+        assertTrue(result4.size() == 1)
+        assertTrue(result4[0].size() == 1)
+        assertTrue(result4[0][0] == 6, "total count is 6")
+
+        // drop column
+        sql """
+            ALTER TABLE ${tableName} DROP COLUMN last_visit_date
+            """
+        def result5 = sql """ select * from ${tableName} where user_id = 3 """
+        assertTrue(result5.size() == 2)
+        assertTrue(result5[0].size() == 11)
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (4, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+
+        def result6 = sql """ select * from ${tableName} where user_id = 4 """
+        assertTrue(result6.size() == 1)
+        assertTrue(result6[0].size() == 11)
+
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+        sql """ INSERT INTO ${tableName} VALUES
+                (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+            """
+
+        Thread.sleep(30 * 1000)
+        // compaction
+        String[][] tablets = sql """ show tablets from ${tableName}; """
+        for (String[] tablet in tablets) {
+                String tablet_id = tablet[0]
+                backend_id = tablet[2]
+                logger.info("run compaction:" + tablet_id)
+                StringBuilder sb = new StringBuilder();
+                sb.append("curl -X POST http://")
+                sb.append(backendId_to_backendIP.get(backend_id))
+                sb.append(":")
+                sb.append(backendId_to_backendHttpPort.get(backend_id))
+                sb.append("/api/compaction/run?tablet_id=")
+                sb.append(tablet_id)
+                sb.append("&compact_type=cumulative")
+
+                String command = sb.toString()
+                process = command.execute()
+                code = process.waitFor()
+                err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+                out = process.getText()
+                logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err)
+                //assertEquals(code, 0)
+        }
+
+        // wait for all compactions done
+        for (String[] tablet in tablets) {
+                boolean running = true
+                do {
+                    Thread.sleep(1000)
+                    String tablet_id = tablet[0]
+                    backend_id = tablet[2]
+                    StringBuilder sb = new StringBuilder();
+                    sb.append("curl -X GET http://")
+                    sb.append(backendId_to_backendIP.get(backend_id))
+                    sb.append(":")
+                    sb.append(backendId_to_backendHttpPort.get(backend_id))
+                    sb.append("/api/compaction/run_status?tablet_id=")
+                    sb.append(tablet_id)
+
+                    String command = sb.toString()
+                    process = command.execute()
+                    code = process.waitFor()
+                    err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+                    out = process.getText()
+                    logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err)
+                    assertEquals(code, 0)
+                    def compactionStatus = parseJson(out.trim())
+                    assertEquals("success", compactionStatus.status.toLowerCase())
+                    running = compactionStatus.run_status
+                } while (running)
+        }
+        def result7 = sql """ select count(*) from ${tableName} """
+        assertTrue(result7.size() == 1)
+        assertTrue(result7[0][0] == 13)
+
+        def result8 = sql """  SELECT * FROM ${tableName} WHERE user_id=2 """
+        assertTrue(result8.size() == 2)
+        assertTrue(result8[0].size() == 11)
+
+        int rowCount = 0
+        for (String[] tablet in tablets) {
+                String tablet_id = tablet[0]
+                backend_id = tablet[2]
+                StringBuilder sb = new StringBuilder();
+                sb.append("curl -X GET http://")
+                sb.append(backendId_to_backendIP.get(backend_id))
+                sb.append(":")
+                sb.append(backendId_to_backendHttpPort.get(backend_id))
+                sb.append("/api/compaction/show?tablet_id=")
+                sb.append(tablet_id)
+                String command = sb.toString()
+                // wait for cleaning stale_rowsets
+                process = command.execute()
+                code = process.waitFor()
+                err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+                out = process.getText()
+                logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err)
+                assertEquals(code, 0)
+                def tabletJson = parseJson(out.trim())
+                assert tabletJson.rowsets instanceof List
+            for (String rowset in (List<String>) tabletJson.rowsets) {
+                rowCount += Integer.parseInt(rowset.split(" ")[1])
+            }
+        }
+        logger.info("size:" + rowCount)
+        assertTrue(rowCount < 10)
+    } finally {
+        //try_sql("DROP TABLE IF EXISTS ${tableName}")
+    }
+}
diff --git a/regression-test/suites/schema_change/test_partition_schema_change.sql b/regression-test/suites/schema_change/test_partition_schema_change.sql
new file mode 100644
index 0000000000..58925efc3b
--- /dev/null
+++ b/regression-test/suites/schema_change/test_partition_schema_change.sql
@@ -0,0 +1,44 @@
+DROP TABLE IF EXISTS example_range_tbl;
+CREATE TABLE example_range_tbl
+(
+    `user_id` LARGEINT NOT NULL COMMENT "用户id",
+    `date` DATE NOT NULL COMMENT "数据灌入日期时间",
+    `city` VARCHAR(20) COMMENT "用户所在城市",
+    `age` SMALLINT COMMENT "用户年龄",
+    `sex` TINYINT COMMENT "用户性别",
+    `last_visit_date` DATETIME REPLACE DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间",
+    `cost` BIGINT SUM DEFAULT "0" COMMENT "用户总消费",
+    `max_dwell_time` INT MAX DEFAULT "0" COMMENT "用户最大停留时间",
+    `min_dwell_time` INT MIN DEFAULT "99999" COMMENT "用户最小停留时间"
+)
+ENGINE=OLAP
+AGGREGATE KEY(`user_id`, `date`, `city`, `age`, `sex`)
+PARTITION BY RANGE(`date`)
+(
+    PARTITION `p201701` VALUES LESS THAN ("2017-02-01"),
+    PARTITION `p201702` VALUES LESS THAN ("2017-03-01"),
+    PARTITION `p201703` VALUES LESS THAN ("2017-04-01")
+)
+DISTRIBUTED BY HASH(`user_id`) BUCKETS 1
+PROPERTIES
+(
+    "replication_num" = "1"
+);
+
+INSERT INTO example_range_tbl VALUES
+    (1, '2017-01-02', 'Beijing', 10, 1, "2017-01-02 00:00:00", 1, 30, 20);
+
+INSERT INTO example_range_tbl VALUES
+    (1, '2017-02-02', 'Beijing', 10, 1, "2017-02-02 00:00:00", 1, 30, 20);
+
+INSERT INTO example_range_tbl VALUES
+    (1, '2017-03-02', 'Beijing', 10, 1, "2017-03-02 00:00:00", 1, 30, 20);
+
+select * from example_range_tbl order by `date`;
+
+ALTER table example_range_tbl ADD COLUMN new_column INT MAX default "1";
+
+INSERT INTO example_range_tbl VALUES
+    (2, '2017-02-03', 'Beijing', 10, 1, "2017-02-02 00:00:00", 1, 30, 20, 2);
+
+select * from example_range_tbl order by `date`;
\ No newline at end of file
diff --git a/regression-test/suites/schema_change/test_uniq_keys_schema_change.groovy b/regression-test/suites/schema_change/test_uniq_keys_schema_change.groovy
new file mode 100644
index 0000000000..568df986f5
--- /dev/null
+++ b/regression-test/suites/schema_change/test_uniq_keys_schema_change.groovy
@@ -0,0 +1,270 @@
+// 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.
+
+import org.codehaus.groovy.runtime.IOGroovyMethods
+
+suite ("test_uniq_keys_schema_change") {
+    def tableName = "schema_change_uniq_keys_regression_test"
+
+    try {
+        String[][] backends = sql """ show backends; """
+        assertTrue(backends.size() > 0)
+        String backend_id;
+        def backendId_to_backendIP = [:]
+        def backendId_to_backendHttpPort = [:]
+        for (String[] backend in backends) {
+            backendId_to_backendIP.put(backend[0], backend[2])
+            backendId_to_backendHttpPort.put(backend[0], backend[5])
+        }
+
+        backend_id = backendId_to_backendIP.keySet()[0]
+        StringBuilder showConfigCommand = new StringBuilder();
+        showConfigCommand.append("curl -X GET http://")
+        showConfigCommand.append(backendId_to_backendIP.get(backend_id))
+        showConfigCommand.append(":")
+        showConfigCommand.append(backendId_to_backendHttpPort.get(backend_id))
+        showConfigCommand.append("/api/show_config")
+        logger.info(showConfigCommand.toString())
+        def process = showConfigCommand.toString().execute()
+        int code = process.waitFor()
+        String err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+        String out = process.getText()
+        logger.info("Show config: code=" + code + ", out=" + out + ", err=" + err)
+        assertEquals(code, 0)
+        def configList = parseJson(out.trim())
+        assert configList instanceof List
+
+        boolean disableAutoCompaction = true
+        for (Object ele in (List) configList) {
+            assert ele instanceof List<String>
+            if (((List<String>) ele)[0] == "disable_auto_compaction") {
+                disableAutoCompaction = Boolean.parseBoolean(((List<String>) ele)[2])
+            }
+        }
+    sql """ DROP TABLE IF EXISTS ${tableName} """
+
+    sql """
+            CREATE TABLE schema_change_uniq_keys_regression_test (
+                `user_id` LARGEINT NOT NULL COMMENT "用户id",
+                `date` DATE NOT NULL COMMENT "数据灌入日期时间",
+                `city` VARCHAR(20) COMMENT "用户所在城市",
+                `age` SMALLINT COMMENT "用户年龄",
+                `sex` TINYINT COMMENT "用户性别",
+                `last_visit_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间",
+                `last_update_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次更新时间",
+                `last_visit_date_not_null` DATETIME NOT NULL DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间",
+                `cost` BIGINT DEFAULT "0" COMMENT "用户总消费",
+                `max_dwell_time` INT DEFAULT "0" COMMENT "用户最大停留时间",
+                `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间")
+            UNIQUE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`)
+            PROPERTIES ( "replication_num" = "1" );
+        """
+
+    sql """ INSERT INTO schema_change_uniq_keys_regression_test VALUES
+             (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-01', '2020-01-01', '2020-01-01', 1, 30, 20)
+        """
+
+    sql """ INSERT INTO schema_change_uniq_keys_regression_test VALUES
+             (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 19)
+        """
+
+    sql """ INSERT INTO schema_change_uniq_keys_regression_test VALUES
+             (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 21)
+        """
+
+    sql """ INSERT INTO schema_change_uniq_keys_regression_test VALUES
+             (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20)
+        """
+    def result = sql """
+                       select count(*) from schema_change_uniq_keys_regression_test
+                    """
+    assertTrue(result.size() == 1)
+    assertTrue(result[0].size() == 1)
+    assertTrue(result[0][0] == 2, "total columns should be 2 rows")
+
+    // add column
+    sql """
+        ALTER table ${tableName} ADD COLUMN new_column INT default "1" 
+        """
+
+    sql """ SELECT * FROM ${tableName} WHERE user_id=2 """
+
+    sql """ INSERT INTO ${tableName} (`user_id`,`date`,`city`,`age`,`sex`,`last_visit_date`,`last_update_date`,
+                                      `last_visit_date_not_null`,`cost`,`max_dwell_time`,`min_dwell_time`)
+            VALUES
+             (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20)
+        """
+
+    result = sql """ SELECT * FROM ${tableName} WHERE user_id=3 """
+
+    assertTrue(result.size() == 1)
+    assertTrue(result[0].size() == 12)
+    assertTrue(result[0][11] == 1, "new add column default value should be 1")
+
+
+    sql """ INSERT INTO ${tableName} VALUES
+             (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+        """
+    result = sql """ SELECT * FROM ${tableName} WHERE user_id = 3 """
+
+    assertTrue(result.size() == 1)
+    assertTrue(result[0].size() == 12)
+    assertTrue(result[0][11] == 2, "new add column value is set to 2")
+
+    result = sql """ select count(*) from ${tableName} """
+    logger.info("result.size:" + result.size() + " result[0].size:" + result[0].size + " " + result[0][0])
+    assertTrue(result.size() == 1)
+    assertTrue(result[0].size() == 1)
+    assertTrue(result[0][0] == 3, "total count is 3")
+
+    // drop column will throws exception 'Can not drop key column in Unique data model table'
+    // sql """
+    //       ALTER TABLE ${tableName} DROP COLUMN sex
+    //       """
+    // result = "null"
+    // while (!result.contains("FINISHED")){
+    //     result = sql "SHOW ALTER TABLE COLUMN WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;"
+    //     result = result.toString()
+    //     logger.info("result: ${result}")
+    //     if(result.contains("CANCELLED")) {
+    //         log.info("rollup job is cancelled, result: ${result}".toString())
+    //         break
+    //     }
+    //     Thread.sleep(1000)
+    // }
+    // result = sql """ select * from ${tableName} where user_id = 3 """
+    // assertTrue(result.size() == 1)
+    // assertTrue(result[0].size() == 11)
+
+    sql """ INSERT INTO ${tableName} VALUES
+             (4, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+        """
+
+    result = sql """ select * from ${tableName} where user_id = 4 """
+    assertTrue(result.size() == 1)
+    assertTrue(result[0].size() == 12)
+
+    sql """ INSERT INTO ${tableName} VALUES
+             (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+        """
+    sql """ INSERT INTO ${tableName} VALUES
+             (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+        """
+    sql """ INSERT INTO ${tableName} VALUES
+             (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+        """
+    sql """ INSERT INTO ${tableName} VALUES
+             (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+        """
+    sql """ INSERT INTO ${tableName} VALUES
+             (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+        """
+    sql """ INSERT INTO ${tableName} VALUES
+             (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+        """
+
+    Thread.sleep(30 * 1000)
+    // compaction
+    String[][] tablets = sql """ show tablets from ${tableName}; """
+    for (String[] tablet in tablets) {
+            String tablet_id = tablet[0]
+            backend_id = tablet[2]
+            logger.info("run compaction:" + tablet_id)
+            StringBuilder sb = new StringBuilder();
+            sb.append("curl -X POST http://")
+            sb.append(backendId_to_backendIP.get(backend_id))
+            sb.append(":")
+            sb.append(backendId_to_backendHttpPort.get(backend_id))
+            sb.append("/api/compaction/run?tablet_id=")
+            sb.append(tablet_id)
+            sb.append("&compact_type=cumulative")
+
+            String command = sb.toString()
+            process = command.execute()
+            code = process.waitFor()
+            err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+            out = process.getText()
+            logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err)
+            //assertEquals(code, 0)
+    }
+
+    // wait for all compactions done
+    for (String[] tablet in tablets) {
+            boolean running = true
+            do {
+                Thread.sleep(1000)
+                String tablet_id = tablet[0]
+                backend_id = tablet[2]
+                StringBuilder sb = new StringBuilder();
+                sb.append("curl -X GET http://")
+                sb.append(backendId_to_backendIP.get(backend_id))
+                sb.append(":")
+                sb.append(backendId_to_backendHttpPort.get(backend_id))
+                sb.append("/api/compaction/run_status?tablet_id=")
+                sb.append(tablet_id)
+
+                String command = sb.toString()
+                process = command.execute()
+                code = process.waitFor()
+                err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+                out = process.getText()
+                logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err)
+                assertEquals(code, 0)
+                def compactionStatus = parseJson(out.trim())
+                assertEquals("success", compactionStatus.status.toLowerCase())
+                running = compactionStatus.run_status
+            } while (running)
+    }
+    result = sql """ select count(*) from ${tableName} """
+    assertTrue(result.size() == 1)
+    assertTrue(result[0][0] == 5)
+
+    result = sql """  SELECT * FROM ${tableName} WHERE user_id=2 """
+    assertTrue(result.size() == 1)
+    assertTrue(result[0].size() == 12)
+
+    int rowCount = 0
+    for (String[] tablet in tablets) {
+            String tablet_id = tablet[0]
+            backend_id = tablet[2]
+            StringBuilder sb = new StringBuilder();
+            sb.append("curl -X GET http://")
+            sb.append(backendId_to_backendIP.get(backend_id))
+            sb.append(":")
+            sb.append(backendId_to_backendHttpPort.get(backend_id))
+            sb.append("/api/compaction/show?tablet_id=")
+            sb.append(tablet_id)
+            String command = sb.toString()
+            // wait for cleaning stale_rowsets
+            process = command.execute()
+            code = process.waitFor()
+            err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+            out = process.getText()
+            logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err)
+            assertEquals(code, 0)
+            def tabletJson = parseJson(out.trim())
+            assert tabletJson.rowsets instanceof List
+        for (String rowset in (List<String>) tabletJson.rowsets) {
+            rowCount += Integer.parseInt(rowset.split(" ")[1])
+        }
+    }
+    logger.info("size:" + rowCount)
+    assertTrue(rowCount <= 10)
+    } finally {
+        //try_sql("DROP TABLE IF EXISTS ${tableName}")
+    }
+}
diff --git a/regression-test/suites/schema_change/test_uniq_mv_schema_change.groovy b/regression-test/suites/schema_change/test_uniq_mv_schema_change.groovy
new file mode 100644
index 0000000000..19070173f3
--- /dev/null
+++ b/regression-test/suites/schema_change/test_uniq_mv_schema_change.groovy
@@ -0,0 +1,274 @@
+// 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.
+
+import org.codehaus.groovy.runtime.IOGroovyMethods
+
+suite ("test_uniq_mv_schema_change") {
+    def tableName = "schema_change_uniq_mv_regression_test"
+
+    try {
+        String[][] backends = sql """ show backends; """
+        assertTrue(backends.size() > 0)
+        String backend_id;
+        def backendId_to_backendIP = [:]
+        def backendId_to_backendHttpPort = [:]
+        for (String[] backend in backends) {
+            backendId_to_backendIP.put(backend[0], backend[2])
+            backendId_to_backendHttpPort.put(backend[0], backend[5])
+        }
+
+        backend_id = backendId_to_backendIP.keySet()[0]
+        StringBuilder showConfigCommand = new StringBuilder();
+        showConfigCommand.append("curl -X GET http://")
+        showConfigCommand.append(backendId_to_backendIP.get(backend_id))
+        showConfigCommand.append(":")
+        showConfigCommand.append(backendId_to_backendHttpPort.get(backend_id))
+        showConfigCommand.append("/api/show_config")
+        logger.info(showConfigCommand.toString())
+        def process = showConfigCommand.toString().execute()
+        int code = process.waitFor()
+        String err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+        String out = process.getText()
+        logger.info("Show config: code=" + code + ", out=" + out + ", err=" + err)
+        assertEquals(code, 0)
+        def configList = parseJson(out.trim())
+        assert configList instanceof List
+
+        boolean disableAutoCompaction = true
+        for (Object ele in (List) configList) {
+            assert ele instanceof List<String>
+            if (((List<String>) ele)[0] == "disable_auto_compaction") {
+                disableAutoCompaction = Boolean.parseBoolean(((List<String>) ele)[2])
+            }
+        }
+    sql """ DROP TABLE IF EXISTS ${tableName} """
+
+    sql """
+            CREATE TABLE ${tableName} (
+                `user_id` LARGEINT NOT NULL COMMENT "用户id",
+                `date` DATE NOT NULL COMMENT "数据灌入日期时间",
+                `city` VARCHAR(20) COMMENT "用户所在城市",
+                `age` SMALLINT COMMENT "用户年龄",
+                `sex` TINYINT COMMENT "用户性别",
+                `last_visit_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间",
+                `last_update_date` DATETIME DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次更新时间",
+                `last_visit_date_not_null` DATETIME NOT NULL DEFAULT "1970-01-01 00:00:00" COMMENT "用户最后一次访问时间",
+                `cost` BIGINT DEFAULT "0" COMMENT "用户总消费",
+                `max_dwell_time` INT DEFAULT "0" COMMENT "用户最大停留时间",
+                `min_dwell_time` INT DEFAULT "99999" COMMENT "用户最小停留时间")
+            UNIQUE KEY(`user_id`, `date`, `city`, `age`, `sex`) DISTRIBUTED BY HASH(`user_id`)
+            PROPERTIES ( "replication_num" = "1" );
+        """
+
+    //add materialized view
+    def result = "null"
+    def mvName = "mv1"
+    sql "create materialized view ${mvName} as select user_id, date, city, age, sex from ${tableName} group by user_id, date, city, age, sex;"
+    while (!result.contains("FINISHED")){
+        result = sql "SHOW ALTER TABLE MATERIALIZED VIEW WHERE TableName='${tableName}' ORDER BY CreateTime DESC LIMIT 1;"
+        result = result.toString()
+        logger.info("result: ${result}")
+        if(result.contains("CANCELLED")){
+            break
+        }
+        Thread.sleep(1000)
+    }
+
+    sql """ INSERT INTO ${tableName} VALUES
+             (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-01', '2020-01-01', '2020-01-01', 1, 30, 20)
+        """
+
+    sql """ INSERT INTO ${tableName} VALUES
+             (1, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 19)
+        """
+
+    sql """ INSERT INTO ${tableName} VALUES
+             (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-02', '2020-01-02', '2020-01-02', 1, 31, 21)
+        """
+
+    sql """ INSERT INTO ${tableName} VALUES
+             (2, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20)
+        """
+    result = sql """
+                       select count(*) from ${tableName}
+                    """
+    assertTrue(result.size() == 1)
+    assertTrue(result[0].size() == 1)
+    assertTrue(result[0][0] == 2, "total columns should be 2 rows")
+
+    // add column
+    sql """
+        ALTER table ${tableName} ADD COLUMN new_column INT default "1" 
+        """
+
+    sql """ SELECT * FROM ${tableName} WHERE user_id=2 """
+
+    sql """ INSERT INTO ${tableName} (`user_id`,`date`,`city`,`age`,`sex`,`last_visit_date`,`last_update_date`,
+                                      `last_visit_date_not_null`,`cost`,`max_dwell_time`,`min_dwell_time`)
+            VALUES
+             (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20)
+        """
+
+    result = sql """ SELECT * FROM ${tableName} WHERE user_id=3 """
+
+    assertTrue(result.size() == 1)
+    assertTrue(result[0].size() == 12)
+    assertTrue(result[0][11] == 1, "new add column default value should be 1")
+
+
+    sql """ INSERT INTO ${tableName} VALUES
+             (3, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 1, 32, 20, 2)
+        """
+    result = sql """ SELECT * FROM ${tableName} WHERE user_id = 3 """
+
+    assertTrue(result.size() == 1)
+    assertTrue(result[0].size() == 12)
+    assertTrue(result[0][11] == 2, "new add column value is set to 2")
+
+    result = sql """ select count(*) from ${tableName} """
+    logger.info("result.size:" + result.size() + " result[0].size:" + result[0].size + " " + result[0][0])
+    assertTrue(result.size() == 1)
+    assertTrue(result[0].size() == 1)
+    assertTrue(result[0][0] == 3, "total count is 3")
+
+    // drop column
+    sql """
+          ALTER TABLE ${tableName} DROP COLUMN cost
+          """
+
+    result = sql """ select * from ${tableName} where user_id = 3 """
+    assertTrue(result.size() == 1)
+    assertTrue(result[0].size() == 11)
+
+    sql """ INSERT INTO ${tableName} VALUES
+             (4, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 32, 20, 2)
+        """
+
+    result = sql """ select * from ${tableName} where user_id = 4 """
+    assertTrue(result.size() == 1)
+    assertTrue(result[0].size() == 11)
+
+    sql """ INSERT INTO ${tableName} VALUES
+             (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 32, 20, 2)
+        """
+    sql """ INSERT INTO ${tableName} VALUES
+             (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 32, 20, 2)
+        """
+    sql """ INSERT INTO ${tableName} VALUES
+             (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 32, 20, 2)
+        """
+    sql """ INSERT INTO ${tableName} VALUES
+             (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 32, 20, 2)
+        """
+    sql """ INSERT INTO ${tableName} VALUES
+             (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 32, 20, 2)
+        """
+    sql """ INSERT INTO ${tableName} VALUES
+             (5, '2017-10-01', 'Beijing', 10, 1, '2020-01-03', '2020-01-03', '2020-01-03', 32, 20, 2)
+        """
+
+    Thread.sleep(30 * 1000)
+    // compaction
+    String[][] tablets = sql """ show tablets from ${tableName}; """
+    for (String[] tablet in tablets) {
+            String tablet_id = tablet[0]
+            backend_id = tablet[2]
+            logger.info("run compaction:" + tablet_id)
+            StringBuilder sb = new StringBuilder();
+            sb.append("curl -X POST http://")
+            sb.append(backendId_to_backendIP.get(backend_id))
+            sb.append(":")
+            sb.append(backendId_to_backendHttpPort.get(backend_id))
+            sb.append("/api/compaction/run?tablet_id=")
+            sb.append(tablet_id)
+            sb.append("&compact_type=cumulative")
+
+            String command = sb.toString()
+            process = command.execute()
+            code = process.waitFor()
+            err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+            out = process.getText()
+            logger.info("Run compaction: code=" + code + ", out=" + out + ", err=" + err)
+            //assertEquals(code, 0)
+    }
+
+    // wait for all compactions done
+    for (String[] tablet in tablets) {
+            boolean running = true
+            do {
+                Thread.sleep(1000)
+                String tablet_id = tablet[0]
+                backend_id = tablet[2]
+                StringBuilder sb = new StringBuilder();
+                sb.append("curl -X GET http://")
+                sb.append(backendId_to_backendIP.get(backend_id))
+                sb.append(":")
+                sb.append(backendId_to_backendHttpPort.get(backend_id))
+                sb.append("/api/compaction/run_status?tablet_id=")
+                sb.append(tablet_id)
+
+                String command = sb.toString()
+                process = command.execute()
+                code = process.waitFor()
+                err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+                out = process.getText()
+                logger.info("Get compaction status: code=" + code + ", out=" + out + ", err=" + err)
+                assertEquals(code, 0)
+                def compactionStatus = parseJson(out.trim())
+                assertEquals("success", compactionStatus.status.toLowerCase())
+                running = compactionStatus.run_status
+            } while (running)
+    }
+    result = sql """ select count(*) from ${tableName} """
+    assertTrue(result.size() == 1)
+    assertTrue(result[0][0] == 5)
+
+    result = sql """  SELECT * FROM ${tableName} WHERE user_id=2 """
+    assertTrue(result.size() == 1)
+    assertTrue(result[0].size() == 11)
+
+    int rowCount = 0
+    for (String[] tablet in tablets) {
+            String tablet_id = tablet[0]
+            backend_id = tablet[2]
+            StringBuilder sb = new StringBuilder();
+            sb.append("curl -X GET http://")
+            sb.append(backendId_to_backendIP.get(backend_id))
+            sb.append(":")
+            sb.append(backendId_to_backendHttpPort.get(backend_id))
+            sb.append("/api/compaction/show?tablet_id=")
+            sb.append(tablet_id)
+            String command = sb.toString()
+            // wait for cleaning stale_rowsets
+            process = command.execute()
+            code = process.waitFor()
+            err = IOGroovyMethods.getText(new BufferedReader(new InputStreamReader(process.getErrorStream())));
+            out = process.getText()
+            logger.info("Show tablets status: code=" + code + ", out=" + out + ", err=" + err)
+            assertEquals(code, 0)
+            def tabletJson = parseJson(out.trim())
+            assert tabletJson.rowsets instanceof List
+        for (String rowset in (List<String>) tabletJson.rowsets) {
+            rowCount += Integer.parseInt(rowset.split(" ")[1])
+        }
+    }
+    logger.info("size:" + rowCount)
+    assertTrue(rowCount <= 14)
+    } finally {
+        //try_sql("DROP TABLE IF EXISTS ${tableName}")
+    }
+}
diff --git a/regression-test/suites/schema_change/test_uniq_rollup_schema_change.groovy b/regression-test/suites/schema_change/test_uniq_rollup_schema_change.groovy
new file mode 100644
index 0000000000..317e099b2c
... 613 lines suppressed ...


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