You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by mo...@apache.org on 2021/04/24 09:42:36 UTC

[incubator-doris] branch master updated: [Feature] Add list partition support (#5529)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new de87f4a  [Feature] Add list partition support (#5529)
de87f4a is described below

commit de87f4ae840c1a527379c046d7f01afb2e2d7dcc
Author: qiye <ji...@gmail.com>
AuthorDate: Sat Apr 24 17:42:27 2021 +0800

    [Feature] Add list partition support (#5529)
    
    Add list partition support
---
 be/src/exec/tablet_info.cpp                        |  61 +-
 be/src/exec/tablet_info.h                          |  15 +-
 be/test/exec/tablet_info_test.cpp                  | 413 +++++++++++++
 .../alter-table/alter-table-temp-partition.md      |  49 +-
 docs/en/administrator-guide/dynamic-partition.md   |   2 +
 .../administrator-guide/load-data/delete-manual.md |   2 +-
 docs/en/getting-started/best-practice.md           |  15 +-
 docs/en/getting-started/data-partition.md          | 179 ++++--
 .../sql-statements/Data Definition/CREATE TABLE.md | 216 +++++--
 .../sql-statements/Data Definition/RESTORE.md      |   2 +-
 .../sql-statements/Data Manipulation/DELETE.md     |   2 +-
 .../alter-table/alter-table-temp-partition.md      |  49 +-
 .../zh-CN/administrator-guide/dynamic-partition.md |   2 +
 .../administrator-guide/load-data/delete-manual.md |   8 +-
 docs/zh-CN/getting-started/best-practice.md        |  16 +-
 docs/zh-CN/getting-started/data-partition.md       | 173 ++++--
 .../sql-statements/Data Definition/CREATE TABLE.md | 235 ++++++--
 .../sql-statements/Data Definition/RESTORE.md      |   2 +-
 .../sql-statements/Data Manipulation/DELETE.md     |   2 +-
 fe/fe-core/src/main/cup/sql_parser.cup             |  98 +++-
 .../apache/doris/alter/SchemaChangeHandler.java    |   6 +-
 .../apache/doris/analysis/AddPartitionClause.java  |   6 +-
 .../org/apache/doris/analysis/BoolLiteral.java     |   8 +
 .../org/apache/doris/analysis/CreateTableStmt.java |   9 +-
 .../apache/doris/analysis/ListPartitionDesc.java   | 110 ++++
 .../org/apache/doris/analysis/LiteralExpr.java     |   2 +-
 .../org/apache/doris/analysis/PartitionDesc.java   |  85 ++-
 .../apache/doris/analysis/PartitionKeyDesc.java    |  64 +-
 .../apache/doris/analysis/RangePartitionDesc.java  | 120 +---
 ...PartitionDesc.java => SinglePartitionDesc.java} |  14 +-
 .../org/apache/doris/analysis/StringLiteral.java   |   9 +-
 .../java/org/apache/doris/backup/RestoreJob.java   |  69 ++-
 .../java/org/apache/doris/catalog/Catalog.java     | 147 +++--
 .../apache/doris/catalog/CatalogRecycleBin.java    |  71 ++-
 .../apache/doris/catalog/ListPartitionInfo.java    | 219 +++++++
 .../apache/doris/catalog/ListPartitionItem.java    | 153 +++++
 .../java/org/apache/doris/catalog/OlapTable.java   | 164 +++---
 .../org/apache/doris/catalog/PartitionInfo.java    | 143 ++++-
 .../org/apache/doris/catalog/PartitionItem.java    |  33 ++
 .../org/apache/doris/catalog/PartitionKey.java     |  43 ++
 .../org/apache/doris/catalog/PartitionType.java    |   7 +-
 .../apache/doris/catalog/RangePartitionInfo.java   | 172 +-----
 .../apache/doris/catalog/RangePartitionItem.java   |  83 +++
 .../doris/clone/DynamicPartitionScheduler.java     |  24 +-
 .../org/apache/doris/common/FeMetaVersion.java     |   4 +-
 .../doris/common/proc/EsPartitionsProcDir.java     |   2 +-
 .../doris/common/proc/PartitionsProcDir.java       |  16 +-
 .../org/apache/doris/common/util/ListUtil.java     |  84 +++
 .../org/apache/doris/common/util/RangeUtils.java   |  38 +-
 .../external/elasticsearch/EsShardPartitions.java  |   8 +-
 .../external/elasticsearch/EsTablePartitions.java  |   7 +-
 .../java/org/apache/doris/load/DeleteHandler.java  |   5 +-
 .../doris/load/loadv2/SparkLoadPendingTask.java    |  12 +-
 .../apache/doris/persist/PartitionPersistInfo.java |  17 +-
 .../org/apache/doris/planner/DataSplitSink.java    |   2 +-
 .../java/org/apache/doris/planner/EsScanNode.java  |   5 +-
 .../apache/doris/planner/ListPartitionPruner.java  | 149 +++--
 .../org/apache/doris/planner/OlapScanNode.java     |  32 +-
 .../org/apache/doris/planner/OlapTableSink.java    |  57 +-
 .../apache/doris/planner/RangePartitionPruner.java |   9 +-
 .../apache/doris/planner/StreamLoadPlanner.java    |  25 +-
 .../java/org/apache/doris/qe/ConnectProcessor.java |  12 +-
 .../org/apache/doris/qe/cache/PartitionRange.java  |   9 +-
 .../apache/doris/task/HadoopLoadPendingTask.java   |   2 +-
 fe/fe-core/src/main/jflex/sql_scanner.flex         |   1 +
 .../doris/analysis/ListPartitionPrunerTest.java    | 170 ++++++
 .../doris/analysis/PartitionKeyDescTest.java       |  28 +-
 .../org/apache/doris/backup/CatalogMocker.java     |   8 +-
 .../org/apache/doris/catalog/CatalogTestUtil.java  |  11 +-
 .../org/apache/doris/catalog/CreateTableTest.java  | 251 ++++++++
 .../doris/catalog/ListPartitionInfoTest.java       | 227 +++++++
 .../org/apache/doris/catalog/PartitionKeyTest.java |  58 +-
 .../doris/catalog/RangePartitionInfoTest.java      | 160 ++---
 .../apache/doris/catalog/TempPartitionTest.java    | 653 ++++++++++++++++++++-
 .../org/apache/doris/common/util/ListUtilTest.java |  64 ++
 .../load/loadv2/SparkLoadPendingTaskTest.java      |  23 +-
 .../apache/doris/planner/OlapTableSinkTest.java    |  35 ++
 gensrc/thrift/Descriptors.thrift                   |   1 +
 gensrc/thrift/Partitions.thrift                    |   3 +
 79 files changed, 4536 insertions(+), 954 deletions(-)

diff --git a/be/src/exec/tablet_info.cpp b/be/src/exec/tablet_info.cpp
index d734a4c..f73e308 100644
--- a/be/src/exec/tablet_info.cpp
+++ b/be/src/exec/tablet_info.cpp
@@ -122,10 +122,22 @@ std::string OlapTableSchemaParam::debug_string() const {
 
 std::string OlapTablePartition::debug_string(TupleDescriptor* tuple_desc) const {
     std::stringstream ss;
+    std::stringstream in_keys_ss;
+    int idx = 0;
+    in_keys_ss << "[";
+    for (auto in_key : in_keys) {
+        if (idx++ > 0) {
+            in_keys_ss << ",";
+        }
+        in_keys_ss << Tuple::to_string(in_key, *tuple_desc);
+    }
+    in_keys_ss << "]";
     ss << "(id=" << id << ",start_key=" << Tuple::to_string(start_key, *tuple_desc)
-       << ",end_key=" << Tuple::to_string(end_key, *tuple_desc) << ",num_buckets=" << num_buckets
+       << ",end_key=" << Tuple::to_string(end_key, *tuple_desc) 
+       << ",in_key=" << in_keys_ss.str()
+       << ",num_buckets=" << num_buckets
        << ",indexes=[";
-    int idx = 0;
+    idx = 0;
     for (auto& index : indexes) {
         if (idx++ > 0) {
             ss << ",";
@@ -211,6 +223,12 @@ Status OlapTablePartitionParam::init() {
         } else if (t_part.__isset.end_keys) {
             RETURN_IF_ERROR(_create_partition_keys(t_part.end_keys, &part->end_key));
         }
+        if (t_part.__isset.in_keys) {
+            part->in_keys.resize(t_part.in_keys.size());
+            for (int j = 0; j < t_part.in_keys.size(); j++) {
+                RETURN_IF_ERROR(_create_partition_keys(t_part.in_keys[j], &part->in_keys[j]));
+            }
+        }
 
         part->num_buckets = t_part.num_buckets;
         auto num_indexes = _schema->indexes().size();
@@ -237,14 +255,27 @@ Status OlapTablePartitionParam::init() {
             }
         }
         _partitions.emplace_back(part);
-        _partitions_map->emplace(part->end_key, part);
+        if (t_part.__isset.in_keys) {
+            for (auto in_key : part->in_keys) {
+                _partitions_map->emplace(in_key, part);
+            }
+        } else {
+            _partitions_map->emplace(part->end_key, part);
+        }
     }
     return Status::OK();
 }
 
 bool OlapTablePartitionParam::find_tablet(Tuple* tuple, const OlapTablePartition** partition,
                                           uint32_t* dist_hashes) const {
-    auto it = _partitions_map->upper_bound(tuple);
+    const TOlapTablePartition& t_part = _t_param.partitions[0];
+    std::map<Tuple*, OlapTablePartition*, OlapTablePartKeyComparator>::iterator it;
+    if (t_part.__isset.in_keys) {
+        it = _partitions_map->find(tuple);
+    } else {
+        it = _partitions_map->upper_bound(tuple);
+        
+    }
     if (it == _partitions_map->end()) {
         return false;
     }
@@ -312,6 +343,28 @@ Status OlapTablePartitionParam::_create_partition_key(const TExprNode& t_expr, T
         memcpy(slot, &val, sizeof(val));
         break;
     }
+    case TExprNodeType::STRING_LITERAL: {
+        int len = t_expr.string_literal.value.size();
+        const char* str_val = t_expr.string_literal.value.c_str();
+
+        // CHAR is a fixed-length string and needs to use the length in the slot definition,
+        // VARVHAR is a variable-length string and needs to use the length of the string itself
+        // padding 0 to CHAR field
+        if (TYPE_CHAR == slot_desc->type().type && len < slot_desc->type().len) {
+            auto new_ptr = (char*)_mem_pool->allocate(slot_desc->type().len);
+            memset(new_ptr, 0, slot_desc->type().len);
+            memcpy(new_ptr, str_val, len);
+
+            str_val = new_ptr;
+            len = slot_desc->type().len;
+        }
+        *reinterpret_cast<StringValue*>(slot) = StringValue(const_cast<char*>(str_val), len);
+        break;
+    }
+    case TExprNodeType::BOOL_LITERAL: {
+        *reinterpret_cast<bool*>(slot) = t_expr.bool_literal.value;
+        break;
+    }
     default: {
         std::stringstream ss;
         ss << "unsupported partition column node type, type=" << t_expr.node_type;
diff --git a/be/src/exec/tablet_info.h b/be/src/exec/tablet_info.h
index 194fd90..59c2962 100644
--- a/be/src/exec/tablet_info.h
+++ b/be/src/exec/tablet_info.h
@@ -94,6 +94,7 @@ struct OlapTablePartition {
     int64_t id = 0;
     Tuple* start_key = nullptr;
     Tuple* end_key = nullptr;
+    std::vector<Tuple*> in_keys;
     int64_t num_buckets = 0;
     std::vector<OlapTableIndexTablets> indexes;
 
@@ -168,11 +169,23 @@ private:
 
     // check if this partition contain this key
     bool _part_contains(OlapTablePartition* part, Tuple* key) const {
-        if (part->start_key == nullptr) {
+        if ((part->start_key == nullptr) && (part->in_keys.size() == 0)) {
             // start_key is nullptr means the lower bound is boundless
             return true;
         }
         OlapTablePartKeyComparator comparator(_partition_slot_descs);
+        const TOlapTablePartition& t_part = _t_param.partitions[0];
+        // when list partition, return true if equals. 
+        if (t_part.__isset.in_keys) {
+            bool ret = false;
+            for (auto in_key : part->in_keys) {
+                ret = !comparator(key, in_key) && !comparator(in_key, key);
+                if (ret) {
+                    break;
+                }
+            }
+            return ret;
+        } 
         return !comparator(key, part->start_key);
     }
 
diff --git a/be/test/exec/tablet_info_test.cpp b/be/test/exec/tablet_info_test.cpp
index 6669fb5..01d325e 100644
--- a/be/test/exec/tablet_info_test.cpp
+++ b/be/test/exec/tablet_info_test.cpp
@@ -224,6 +224,419 @@ TEST_F(OlapTablePartitionParamTest, normal) {
     }
 }
 
+/*
+ *PARTITION BY LIST(`k1`)
+ * (
+ * PARTITION p1 VALUES IN ("1", "2"),
+ * PARTITION p2 VALUES IN ("3"),
+ * PARTITION p3 VALUES IN ("4", "5", "6")
+ * )
+ * 
+*/
+TEST_F(OlapTablePartitionParamTest, single_list_partition) {
+    TDescriptorTable t_desc_tbl;
+    auto t_schema = get_schema(&t_desc_tbl);
+    std::shared_ptr<OlapTableSchemaParam> schema(new OlapTableSchemaParam());
+    auto st = schema->init(t_schema);
+    ASSERT_TRUE(st.ok());
+    LOG(INFO) << schema->debug_string();
+
+    // 1 | 2 | 3 | 4 | 5 | 6
+    std::vector<TExprNode> t_node_1;
+    t_node_1.resize(1);
+    t_node_1[0].node_type = TExprNodeType::INT_LITERAL;
+    t_node_1[0].type = t_desc_tbl.slotDescriptors[1].slotType;
+    t_node_1[0].num_children = 0;
+    t_node_1[0].__isset.int_literal = true;
+    t_node_1[0].int_literal.value = 1;
+
+    std::vector<TExprNode> t_node_2;
+    t_node_2.resize(1);
+    t_node_2[0].node_type = TExprNodeType::INT_LITERAL;
+    t_node_2[0].type = t_desc_tbl.slotDescriptors[1].slotType;
+    t_node_2[0].num_children = 0;
+    t_node_2[0].__isset.int_literal = true;
+    t_node_2[0].int_literal.value = 2;
+
+    std::vector<TExprNode> t_node_3;
+    t_node_3.resize(1);
+    t_node_3[0].node_type = TExprNodeType::INT_LITERAL;
+    t_node_3[0].type = t_desc_tbl.slotDescriptors[1].slotType;
+    t_node_3[0].num_children = 0;
+    t_node_3[0].__isset.int_literal = true;
+    t_node_3[0].int_literal.value = 3;
+
+    std::vector<TExprNode> t_node_4;
+    t_node_4.resize(1);
+    t_node_4[0].node_type = TExprNodeType::INT_LITERAL;
+    t_node_4[0].type = t_desc_tbl.slotDescriptors[1].slotType;
+    t_node_4[0].num_children = 0;
+    t_node_4[0].__isset.int_literal = true;
+    t_node_4[0].int_literal.value = 4;
+
+    std::vector<TExprNode> t_node_5;
+    t_node_5.resize(1);
+    t_node_5[0].node_type = TExprNodeType::INT_LITERAL;
+    t_node_5[0].type = t_desc_tbl.slotDescriptors[1].slotType;
+    t_node_5[0].num_children = 0;
+    t_node_5[0].__isset.int_literal = true;
+    t_node_5[0].int_literal.value = 5;
+
+    std::vector<TExprNode> t_node_6;
+    t_node_6.resize(1);
+    t_node_6[0].node_type = TExprNodeType::INT_LITERAL;
+    t_node_6[0].type = t_desc_tbl.slotDescriptors[1].slotType;
+    t_node_6[0].num_children = 0;
+    t_node_6[0].__isset.int_literal = true;
+    t_node_6[0].int_literal.value = 6;
+
+    TOlapTablePartitionParam t_partition_param;
+    t_partition_param.db_id = 1;
+    t_partition_param.table_id = 2;
+    t_partition_param.version = 0;
+    t_partition_param.__set_partition_column("c2");
+    t_partition_param.__set_distributed_columns({"c1", "c3"});
+    t_partition_param.partitions.resize(3);
+    t_partition_param.partitions[0].id = 10;
+    t_partition_param.partitions[0].__isset.in_keys = true;
+    t_partition_param.partitions[0].in_keys.emplace_back(t_node_1);
+    t_partition_param.partitions[0].in_keys.emplace_back(t_node_2);
+    t_partition_param.partitions[0].num_buckets = 1;
+    t_partition_param.partitions[0].indexes.resize(2);
+    t_partition_param.partitions[0].indexes[0].index_id = 4;
+    t_partition_param.partitions[0].indexes[0].tablets = {21};
+    t_partition_param.partitions[0].indexes[1].index_id = 5;
+    t_partition_param.partitions[0].indexes[1].tablets = {22};
+
+    t_partition_param.partitions[1].id = 11;
+    t_partition_param.partitions[1].__isset.in_keys = true;
+    t_partition_param.partitions[1].in_keys.emplace_back(t_node_3);
+    t_partition_param.partitions[1].num_buckets = 2;
+    t_partition_param.partitions[1].indexes.resize(2);
+    t_partition_param.partitions[1].indexes[0].index_id = 4;
+    t_partition_param.partitions[1].indexes[0].tablets = {31, 32};
+    t_partition_param.partitions[1].indexes[1].index_id = 5;
+    t_partition_param.partitions[1].indexes[1].tablets = {33, 34};
+
+    t_partition_param.partitions[2].id = 12;
+    t_partition_param.partitions[2].__isset.in_keys = true;
+    t_partition_param.partitions[2].in_keys.emplace_back(t_node_4);
+    t_partition_param.partitions[2].in_keys.emplace_back(t_node_5);
+    t_partition_param.partitions[2].in_keys.emplace_back(t_node_6);
+    t_partition_param.partitions[2].num_buckets = 4;
+    t_partition_param.partitions[2].indexes.resize(2);
+    t_partition_param.partitions[2].indexes[0].index_id = 4;
+    t_partition_param.partitions[2].indexes[0].tablets = {41, 42, 43, 44};
+    t_partition_param.partitions[2].indexes[1].index_id = 5;
+    t_partition_param.partitions[2].indexes[1].tablets = {45, 46, 47, 48};
+
+    OlapTablePartitionParam part(schema, t_partition_param);
+    st = part.init();
+    ASSERT_TRUE(st.ok());
+    LOG(INFO) << part.debug_string();
+
+    ObjectPool pool;
+    DescriptorTbl* desc_tbl = nullptr;
+    st = DescriptorTbl::create(&pool, t_desc_tbl, &desc_tbl);
+    ASSERT_TRUE(st.ok());
+    RowDescriptor row_desc(*desc_tbl, {0}, {false});
+    TupleDescriptor* tuple_desc = desc_tbl->get_tuple_descriptor(0);
+    auto tracker = std::make_shared<MemTracker>();
+    RowBatch batch(row_desc, 1024, tracker.get());
+    // 12, 1, "abc"
+    {
+        Tuple* tuple = (Tuple*)batch.tuple_data_pool()->allocate(tuple_desc->byte_size());
+        memset(tuple, 0, tuple_desc->byte_size());
+
+        *reinterpret_cast<int*>(tuple->get_slot(4)) = 12;
+        *reinterpret_cast<int64_t*>(tuple->get_slot(8)) = 1;
+        StringValue* str_val = reinterpret_cast<StringValue*>(tuple->get_slot(16));
+        str_val->ptr = (char*)batch.tuple_data_pool()->allocate(10);
+        str_val->len = 3;
+        memcpy(str_val->ptr, "abc", str_val->len);
+
+        // 1:
+        uint32_t dist_hash = 0;
+        const OlapTablePartition* partition = nullptr;
+        auto found = part.find_tablet(tuple, &partition, &dist_hash);
+        ASSERT_TRUE(found);
+        ASSERT_EQ(10, partition->id);
+    }
+    // 13, 3, "abcd"
+    {
+        Tuple* tuple = (Tuple*)batch.tuple_data_pool()->allocate(tuple_desc->byte_size());
+        memset(tuple, 0, tuple_desc->byte_size());
+
+        *reinterpret_cast<int*>(tuple->get_slot(4)) = 13;
+        *reinterpret_cast<int64_t*>(tuple->get_slot(8)) = 3;
+        StringValue* str_val = reinterpret_cast<StringValue*>(tuple->get_slot(16));
+        str_val->ptr = (char*)batch.tuple_data_pool()->allocate(10);
+        str_val->len = 4;
+        memcpy(str_val->ptr, "abcd", str_val->len);
+
+        // 3:
+        uint32_t dist_hash = 0;
+        const OlapTablePartition* partition = nullptr;
+        auto found = part.find_tablet(tuple, &partition, &dist_hash);
+        ASSERT_TRUE(found);
+        ASSERT_EQ(11, partition->id);
+    }
+    // 14, 50, "abcde"
+    {
+        Tuple* tuple = (Tuple*)batch.tuple_data_pool()->allocate(tuple_desc->byte_size());
+        memset(tuple, 0, tuple_desc->byte_size());
+
+        *reinterpret_cast<int*>(tuple->get_slot(4)) = 14;
+        *reinterpret_cast<int64_t*>(tuple->get_slot(8)) = 50;
+        StringValue* str_val = reinterpret_cast<StringValue*>(tuple->get_slot(16));
+        str_val->ptr = reinterpret_cast<char*>(batch.tuple_data_pool()->allocate(10));
+        str_val->len = 5;
+        memcpy(str_val->ptr, "abcde", str_val->len);
+
+        // 50:
+        uint32_t dist_hash = 0;
+        const OlapTablePartition* partition = nullptr;
+        auto found = part.find_tablet(tuple, &partition, &dist_hash);
+        ASSERT_FALSE(found);
+    }
+
+    // 15, 6, "abcdef"
+    {
+        Tuple* tuple = (Tuple*)batch.tuple_data_pool()->allocate(tuple_desc->byte_size());
+        memset(tuple, 0, tuple_desc->byte_size());
+
+        *reinterpret_cast<int*>(tuple->get_slot(4)) = 15;
+        *reinterpret_cast<int64_t*>(tuple->get_slot(8)) = 6;
+        StringValue* str_val = reinterpret_cast<StringValue*>(tuple->get_slot(16));
+        str_val->ptr = reinterpret_cast<char*>(batch.tuple_data_pool()->allocate(10));
+        str_val->len = 6;
+        memcpy(str_val->ptr, "abcdef", str_val->len);
+
+        // 6:
+        uint32_t dist_hash = 0;
+        const OlapTablePartition* partition = nullptr;
+        auto found = part.find_tablet(tuple, &partition, &dist_hash);
+        ASSERT_TRUE(found);
+        ASSERT_EQ(12, partition->id);
+    }
+}
+
+/* 
+ * multi list partition key test 
+ * 
+ * PARTITION BY LIST(c2, c3)
+ * (
+ *  PARTITION p1 VALUES IN (("1", "beijing"),("1", "shanghai")),
+ *  PARTITION p2 VALUES IN (("1", "tianjin"),("2", "beijing")),
+ *  PARTITION p3 VALUES IN (("2", "shanghai"))
+ * )
+ * 
+ */
+TEST_F(OlapTablePartitionParamTest, multi_list_partition) {
+    TDescriptorTable t_desc_tbl;
+    auto t_schema = get_schema(&t_desc_tbl);
+    std::shared_ptr<OlapTableSchemaParam> schema(new OlapTableSchemaParam());
+    auto st = schema->init(t_schema);
+    ASSERT_TRUE(st.ok());
+    LOG(INFO) << schema->debug_string();
+
+    // 1
+    TExprNode node_1;
+    node_1.node_type = TExprNodeType::INT_LITERAL;
+    node_1.type = t_desc_tbl.slotDescriptors[1].slotType;
+    node_1.num_children = 0;
+    node_1.__isset.int_literal = true;
+    node_1.int_literal.value = 1;
+    // 2
+    TExprNode node_2;
+    node_2.node_type = TExprNodeType::INT_LITERAL;
+    node_2.type = t_desc_tbl.slotDescriptors[1].slotType;
+    node_2.num_children = 0;
+    node_2.__isset.int_literal = true;
+    node_2.int_literal.value = 2;
+    // beijing 
+    TExprNode node_b;
+    node_b.node_type = TExprNodeType::STRING_LITERAL;
+    node_b.type = t_desc_tbl.slotDescriptors[2].slotType;
+    node_b.num_children = 0;
+    node_b.__isset.string_literal = true;
+    node_b.string_literal.value = "beijing";
+    // shanghai 
+    TExprNode node_s;
+    node_s.node_type = TExprNodeType::STRING_LITERAL;
+    node_s.type = t_desc_tbl.slotDescriptors[2].slotType;
+    node_s.num_children = 0;
+    node_s.__isset.string_literal = true;
+    node_s.string_literal.value = "shanghai";
+    // tianjin
+    TExprNode node_t;
+    node_t.node_type = TExprNodeType::STRING_LITERAL;
+    node_t.type = t_desc_tbl.slotDescriptors[2].slotType;
+    node_t.num_children = 0;
+    node_t.__isset.string_literal = true;
+    node_t.string_literal.value = "tianjin";
+
+    // (1, beijing)
+    std::vector<TExprNode> t_nodes_1;
+    t_nodes_1.emplace_back(node_1);
+    t_nodes_1.emplace_back(node_b);
+
+    // (1, shanghai)
+    std::vector<TExprNode> t_nodes_2;
+    t_nodes_2.emplace_back(node_1);
+    t_nodes_2.emplace_back(node_s);
+
+    // (1, tianjin)
+    std::vector<TExprNode> t_nodes_3;
+    t_nodes_3.emplace_back(node_1);
+    t_nodes_3.emplace_back(node_t);
+
+    // (2, beijing)
+    std::vector<TExprNode> t_nodes_4;
+    t_nodes_4.emplace_back(node_2);
+    t_nodes_4.emplace_back(node_b);
+
+    // (2, shanghai)
+    std::vector<TExprNode> t_nodes_5;
+    t_nodes_5.emplace_back(node_2);
+    t_nodes_5.emplace_back(node_s);
+
+    TOlapTablePartitionParam t_partition_param;
+    t_partition_param.db_id = 1;
+    t_partition_param.table_id = 2;
+    t_partition_param.version = 0;
+    t_partition_param.__set_partition_columns({"c2", "c3"});
+    t_partition_param.__set_distributed_columns({"c1"});
+    t_partition_param.partitions.resize(3);
+
+    // (("1", "beijing"),("1", "shanghai"))
+    t_partition_param.partitions[0].id = 10;
+    t_partition_param.partitions[0].__isset.in_keys = true;
+    t_partition_param.partitions[0].in_keys.resize(2);
+    t_partition_param.partitions[0].in_keys[0] = t_nodes_1;
+    t_partition_param.partitions[0].in_keys[1] = t_nodes_2;
+    t_partition_param.partitions[0].num_buckets = 1;
+    t_partition_param.partitions[0].indexes.resize(2);
+    t_partition_param.partitions[0].indexes[0].index_id = 4;
+    t_partition_param.partitions[0].indexes[0].tablets = {21};
+    t_partition_param.partitions[0].indexes[1].index_id = 5;
+    t_partition_param.partitions[0].indexes[1].tablets = {22};
+
+    // (("1", "tianjin"),("2", "beijing"))
+    t_partition_param.partitions[1].id = 11;
+    t_partition_param.partitions[1].__isset.in_keys = true;
+    t_partition_param.partitions[1].in_keys.emplace_back(t_nodes_3);
+    t_partition_param.partitions[1].in_keys.emplace_back(t_nodes_4);
+    t_partition_param.partitions[1].num_buckets = 2;
+    t_partition_param.partitions[1].indexes.resize(2);
+    t_partition_param.partitions[1].indexes[0].index_id = 4;
+    t_partition_param.partitions[1].indexes[0].tablets = {31, 32};
+    t_partition_param.partitions[1].indexes[1].index_id = 5;
+    t_partition_param.partitions[1].indexes[1].tablets = {33, 34};
+
+    // (("2", "shanghai")))
+    t_partition_param.partitions[2].id = 12;
+    t_partition_param.partitions[2].__isset.in_keys = true;
+    t_partition_param.partitions[2].in_keys.emplace_back(t_nodes_5);
+    t_partition_param.partitions[2].num_buckets = 4;
+    t_partition_param.partitions[2].indexes.resize(2);
+    t_partition_param.partitions[2].indexes[0].index_id = 4;
+    t_partition_param.partitions[2].indexes[0].tablets = {41, 42, 43, 44};
+    t_partition_param.partitions[2].indexes[1].index_id = 5;
+    t_partition_param.partitions[2].indexes[1].tablets = {45, 46, 47, 48};
+
+    OlapTablePartitionParam part(schema, t_partition_param);
+    st = part.init();
+    LOG(INFO) << st.get_error_msg();
+    ASSERT_TRUE(st.ok());
+    LOG(INFO) << part.debug_string();
+
+    ObjectPool pool;
+    DescriptorTbl* desc_tbl = nullptr;
+    st = DescriptorTbl::create(&pool, t_desc_tbl, &desc_tbl);
+    ASSERT_TRUE(st.ok());
+    RowDescriptor row_desc(*desc_tbl, {0}, {false});
+    TupleDescriptor* tuple_desc = desc_tbl->get_tuple_descriptor(0);
+    auto tracker = std::make_shared<MemTracker>();
+    RowBatch batch(row_desc, 1024, tracker.get());
+    // 12, 1, "beijing"
+    {
+        Tuple* tuple = (Tuple*)batch.tuple_data_pool()->allocate(tuple_desc->byte_size());
+        memset(tuple, 0, tuple_desc->byte_size());
+
+        *reinterpret_cast<int*>(tuple->get_slot(4)) = 12;
+        *reinterpret_cast<int64_t*>(tuple->get_slot(8)) = 1;
+        StringValue* str_val = reinterpret_cast<StringValue*>(tuple->get_slot(16));
+        str_val->ptr = (char*)batch.tuple_data_pool()->allocate(10);
+        str_val->len = 7;
+        memcpy(str_val->ptr, "beijing", str_val->len);
+
+        // 1, beijing
+        uint32_t dist_hash = 0;
+        const OlapTablePartition* partition = nullptr;
+        auto found = part.find_tablet(tuple, &partition, &dist_hash);
+        ASSERT_TRUE(found);
+        ASSERT_EQ(10, partition->id);
+    }
+    // 13, 2, "shanghai"
+    {
+        Tuple* tuple = (Tuple*)batch.tuple_data_pool()->allocate(tuple_desc->byte_size());
+        memset(tuple, 0, tuple_desc->byte_size());
+
+        *reinterpret_cast<int*>(tuple->get_slot(4)) = 13;
+        *reinterpret_cast<int64_t*>(tuple->get_slot(8)) = 2;
+        StringValue* str_val = reinterpret_cast<StringValue*>(tuple->get_slot(16));
+        str_val->ptr = (char*)batch.tuple_data_pool()->allocate(10);
+        str_val->len = 8;
+        memcpy(str_val->ptr, "shanghai", str_val->len);
+
+        // 2, shanghai
+        uint32_t dist_hash = 0;
+        const OlapTablePartition* partition = nullptr;
+        auto found = part.find_tablet(tuple, &partition, &dist_hash);
+        ASSERT_TRUE(found);
+        ASSERT_EQ(12, partition->id);
+    }
+    // 14, 50, "beijing"
+    {
+        Tuple* tuple = (Tuple*)batch.tuple_data_pool()->allocate(tuple_desc->byte_size());
+        memset(tuple, 0, tuple_desc->byte_size());
+
+        *reinterpret_cast<int*>(tuple->get_slot(4)) = 14;
+        *reinterpret_cast<int64_t*>(tuple->get_slot(8)) = 50;
+        StringValue* str_val = reinterpret_cast<StringValue*>(tuple->get_slot(16));
+        str_val->ptr = reinterpret_cast<char*>(batch.tuple_data_pool()->allocate(10));
+        str_val->len = 7;
+        memcpy(str_val->ptr, "beijing", str_val->len);
+
+        // 50, beijing
+        uint32_t dist_hash = 0;
+        const OlapTablePartition* partition = nullptr;
+        auto found = part.find_tablet(tuple, &partition, &dist_hash);
+        ASSERT_FALSE(found);
+    }
+
+    // 15, 1, "tianjin"
+    {
+        Tuple* tuple = (Tuple*)batch.tuple_data_pool()->allocate(tuple_desc->byte_size());
+        memset(tuple, 0, tuple_desc->byte_size());
+
+        *reinterpret_cast<int*>(tuple->get_slot(4)) = 15;
+        *reinterpret_cast<int64_t*>(tuple->get_slot(8)) = 1;
+        StringValue* str_val = reinterpret_cast<StringValue*>(tuple->get_slot(16));
+        str_val->ptr = reinterpret_cast<char*>(batch.tuple_data_pool()->allocate(10));
+        str_val->len = 7;
+        memcpy(str_val->ptr, "tianjin", str_val->len);
+
+        // 1, tianjin
+        uint32_t dist_hash = 0;
+        const OlapTablePartition* partition = nullptr;
+        auto found = part.find_tablet(tuple, &partition, &dist_hash);
+        ASSERT_TRUE(found);
+        ASSERT_EQ(11, partition->id);
+    }
+}
+
 TEST_F(OlapTablePartitionParamTest, to_protobuf) {
     TDescriptorTable t_desc_tbl;
     auto t_schema = get_schema(&t_desc_tbl);
diff --git a/docs/en/administrator-guide/alter-table/alter-table-temp-partition.md b/docs/en/administrator-guide/alter-table/alter-table-temp-partition.md
index 412df27..94e735d 100644
--- a/docs/en/administrator-guide/alter-table/alter-table-temp-partition.md
+++ b/docs/en/administrator-guide/alter-table/alter-table-temp-partition.md
@@ -52,6 +52,15 @@ ALTER TABLE tbl2 ADD TEMPORARY PARTITION tp1 VALUES [("2020-01-01"), ("2020-02-0
 ALTER TABLE tbl1 ADD TEMPORARY PARTITION tp1 VALUES LESS THAN ("2020-02-01")
 ("in_memory" = "true", "replication_num" = "1")
 DISTRIBUTED BY HASH (k1) BUCKETS 5;
+
+ALTER TABLE tbl3 ADD TEMPORARY PARTITION tp1 VALUES IN ("Beijing", "Shanghai");
+
+ALTER TABLE tbl4 ADD TEMPORARY PARTITION tp1 VALUES IN ((1, "Beijing"), (1, "Shanghai"));
+
+ALTER TABLE tbl3 ADD TEMPORARY PARTITION tp1 VALUES IN ("Beijing", "Shanghai")
+("in_memory" = "true", "replication_num" = "1")
+DISTRIBUTED BY HASH(k1) BUCKETS 5;
+
 ```
 
 See `HELP ALTER TABLE;` for more help and examples.
@@ -97,7 +106,13 @@ The replace operation has two special optional parameters:
 
 1. `strict_range`
 
-    The default is true. When this parameter is true, the range union of all formal partitions to be replaced needs to be the same as the range union of the temporary partitions to be replaced. When set to false, you only need to ensure that the range between the new formal partitions does not overlap after replacement. Here are some examples:
+    The default is true. 
+    
+    For Range partition, When this parameter is true, the range union of all formal partitions to be replaced needs to be the same as the range union of the temporary partitions to be replaced. When set to false, you only need to ensure that the range between the new formal partitions does not overlap after replacement. 
+    
+    For List partition, this parameter is always true, and the enumeration values of all full partitions to be replaced must be identical to the enumeration values of the temporary partitions to be replaced.
+
+    Here are some examples:
 
     * Example 1
 
@@ -131,6 +146,38 @@ The replace operation has two special optional parameters:
 
         The union of ranges is not the same. If `strict_range` is true, you cannot use tp1 and tp2 to replace p1. If false, and the two partition ranges `[10, 30), [40, 50)` and the other formal partitions do not overlap, they can be replaced.
 
+    * Example 3
+
+        Enumerated values of partitions p1, p2 to be replaced (=> union).
+
+        ```
+        (1, 2, 3), (4, 5, 6) => (1, 2, 3, 4, 5, 6)
+        ```
+
+        Replace the enumerated values of partitions tp1, tp2, tp3 (=> union).
+
+        ```
+        (1, 2, 3), (4), (5, 6) => (1, 2, 3, 4, 5, 6)
+        ```
+
+        The enumeration values are the same, you can use tp1, tp2, tp3 to replace p1, p2
+
+    * Example 4
+
+        Enumerated values of partitions p1, p2, p3 to be replaced (=> union).
+
+        ```
+        (("1", "beijing"), ("1", "shanghai")), (("2", "beijing"), ("2", "shanghai")), (("3", "beijing"), ("3", "shanghai")) => (("1", "beijing"), ("3", "shanghai")) "), ("1", "shanghai"), ("2", "beijing"), ("2", "shanghai"), ("3", "beijing"), ("3", "shanghai"))
+        ```
+
+        Replace the enumerated values of partitions tp1, tp2 (=> union).
+
+        ```
+        (("1", "beijing"), ("1", "shanghai")), (("2", "beijing"), ("2", "shanghai"), ("3", "beijing"), ("3", "shanghai")) => (("1", "beijing") , ("1", "shanghai"), ("2", "beijing"), ("2", "shanghai"), ("3", "beijing"), ("3", "shanghai"))
+        ```
+
+        The enumeration values are the same, you can use tp1, tp2 to replace p1, p2, p3
+
 2. `use_temp_partition_name`
 
     The default is false. When this parameter is false, and the number of partitions to be replaced is the same as the number of replacement partitions, the name of the formal partition after the replacement remains unchanged. If true, after replacement, the name of the formal partition is the name of the replacement partition. Here are some examples:
diff --git a/docs/en/administrator-guide/dynamic-partition.md b/docs/en/administrator-guide/dynamic-partition.md
index 1a75491..09f3a27 100644
--- a/docs/en/administrator-guide/dynamic-partition.md
+++ b/docs/en/administrator-guide/dynamic-partition.md
@@ -30,6 +30,8 @@ Dynamic partition is a new feature introduced in Doris version 0.12. It's design
 
 At present, the functions of dynamically adding partitions and dynamically deleting partitions are realized.
 
+Dynamic partitioning is only supported for Range partitions.
+
 ## Noun Interpretation
 
 * FE: Frontend, the front-end node of Doris. Responsible for metadata management and request access.
diff --git a/docs/en/administrator-guide/load-data/delete-manual.md b/docs/en/administrator-guide/load-data/delete-manual.md
index 7324a8e..8ee465b 100644
--- a/docs/en/administrator-guide/load-data/delete-manual.md
+++ b/docs/en/administrator-guide/load-data/delete-manual.md
@@ -67,7 +67,7 @@ Explanation:
 2. The column in the WHERE condition can only be the `key` column.
 3. Cannot delete when the `key` column does not exist in any rollup table.
 4. Each condition in WHERE condition can only be connected by `and`. If you want `or`, you are suggested to write these conditions into two delete statements.
-5. If the specified table is a range partitioned table, `PARTITION` must be specified unless the table is a single partition table,.
+5. If the specified table is a range or list partitioned table, `PARTITION` must be specified unless the table is a single partition table,.
 6. Unlike the insert into command, delete statement cannot specify `label` manually. You can view the concept of `label` in [Insert Into](./insert-into-manual.md)
 
 ## Delete Result
diff --git a/docs/en/getting-started/best-practice.md b/docs/en/getting-started/best-practice.md
index 0d0c76e..930bdb3 100644
--- a/docs/en/getting-started/best-practice.md
+++ b/docs/en/getting-started/best-practice.md
@@ -98,15 +98,22 @@ In the process of using Star Schema, users are advised to use Star Schema to dis
 
 ### 1.3 Partitioning and Bucketing 
 
-Doris supports two-level partitioned storage. The first layer is RANGE partition and the second layer is HASH bucket.
+Doris supports two-level partitioned storage. The first level is partition, which currently supports both RANGE and LIST partition types, and the second layer is HASH bucket.
 
-1.3.1. RANGE Partitioning
+1.3.1. Partitioning
 
-The RANGE partition is used to divide data into different intervals, which can be logically understood as dividing the original table into multiple sub-tables. In business, most users will choose to partition on time, which has the following advantages:
+Partition is used to divide data into different intervals, which can be logically understood as dividing the original table into multiple sub-tables. Data can be easily managed by partition, for example, to delete data more quickly.
+
+1.3.1.1. Range Partitioning
+
+In business, most users will choose to partition on time, which has the following advantages:
 
 * Differentiable heat and cold data
 * Availability of Doris Hierarchical Storage (SSD + SATA)
-* Delete data by partition more quickly
+
+1.3.1.2. List Partitioning
+
+In business,, users can select cities or other enumeration values for partition.
 
 1.3.2. Hash Bucketing
 
diff --git a/docs/en/getting-started/data-partition.md b/docs/en/getting-started/data-partition.md
index a9efe9a..4ab0241 100644
--- a/docs/en/getting-started/data-partition.md
+++ b/docs/en/getting-started/data-partition.md
@@ -57,14 +57,16 @@ See more help with `HELP CREATE TABLE;`.
 This section introduces Doris's approach to building tables with an example.
 
 ```
-CREATE TABLE IF NOT EXISTS example_db.expamle_tbl
+-- Range Partition
+
+CREATE TABLE IF NOT EXISTS example_db.expamle_range_tbl
 (
-    `user_id` LARGEINT NOT NULL COMMENT "user id",
+    `user_id` LARGEINT NOT NULL COMMENT "User id",
     `date` DATE NOT NULL COMMENT "Data fill in date time",
     `timestamp` DATETIME NOT NULL COMMENT "Timestamp of data being poured",
     `city` VARCHAR(20) COMMENT "The city where the user is located",
-    `age` SMALLINT COMMENT "user age",
-    `sex` TINYINT COMMENT "User Gender",
+    `age` SMALLINT COMMENT "User age",
+    `sex` TINYINT COMMENT "User gender",
     `last_visit_date` DATETIME REPLACE DEFAULT "1970-01-01 00:00:00" COMMENT "User last visit time",
     `cost` BIGINT SUM DEFAULT "0" COMMENT "Total user consumption",
     `max_dwell_time` INT MAX DEFAULT "0" COMMENT "User maximum dwell time",
@@ -86,6 +88,38 @@ PROPERTIES
     "storage_cooldown_time" = "2018-01-01 12:00:00"
 );
 
+
+-- List Partition
+
+CREATE TABLE IF NOT EXISTS example_db.expamle_list_tbl
+(
+    `user_id` LARGEINT NOT NULL COMMENT "User id",
+    `date` DATE NOT NULL COMMENT "Data fill in date time",
+    `timestamp` DATETIME NOT NULL COMMENT "Timestamp of data being poured",
+    `city` VARCHAR(20) COMMENT "The city where the user is located",
+    `age` SMALLINT COMMENT "User Age",
+    `sex` TINYINT COMMENT "User gender",
+    `last_visit_date` DATETIME REPLACE DEFAULT "1970-01-01 00:00:00" COMMENT "User last visit time",
+    `cost` BIGINT SUM DEFAULT "0" COMMENT "Total user consumption",
+    `max_dwell_time` INT MAX DEFAULT "0" COMMENT "User maximum dwell time",
+    `min_dwell_time` INT MIN DEFAULT "99999" COMMENT "User minimum dwell time"
+)
+ENGINE=olap
+AGGREGATE KEY(`user_id`, `date`, `timestamp`, `city`, `age`, `sex`)
+PARTITION BY LIST(`city`)
+(
+    PARTITION `p_cn` VALUES IN ("Beijing", "Shanghai", "Hong Kong"),
+    PARTITION `p_usa` VALUES IN ("New York", "San Francisco"),
+    PARTITION `p_jp` VALUES IN ("Tokyo")
+)
+DISTRIBUTED BY HASH(`user_id`) BUCKETS 16
+PROPERTIES
+(
+    "replication_num" = "3",
+    "storage_medium" = "SSD",
+    "storage_cooldown_time" = "2018-01-01 12:00:00"
+);
+
 ```
 
 ### Column Definition
@@ -106,7 +140,7 @@ When defining columns, you can refer to the following suggestions:
 
 ### Partitioning and binning
 
-Doris supports two levels of data partitioning. The first layer is Partition, which only supports the division of Range. The second layer is Bucket (Tablet), which only supports the way Hash is divided.
+Doris supports two levels of data partitioning. The first layer is Partition, which supports Range and List partitioning. The second layer is the Bucket (Tablet), which only supports Hash partitioning.
 
 It is also possible to use only one layer of partitioning. When using a layer partition, only Bucket partitioning is supported.
 
@@ -114,12 +148,15 @@ It is also possible to use only one layer of partitioning. When using a layer pa
 
     * The Partition column can specify one or more columns. The partition class must be a KEY column. The use of multi-column partitions is described later in the **Multi-column partitioning** summary. 
     * Regardless of the type of partition column, double quotes are required when writing partition values.
-    * Partition columns are usually time columns for easy management of old and new data.
     * There is no theoretical limit on the number of partitions.
     * When you do not use Partition to build a table, the system will automatically generate a Partition with the same name as the table name. This Partition is not visible to the user and cannot be modified.
+    
+    #### Range Partition
+
+    * Partition columns are usually time columns for easy management of old and new data.
     * Partition supports only the upper bound by `VALUES LESS THAN (...)`, the system will use the upper bound of the previous partition as the lower bound of the partition, and generate a left closed right open interval. Passing, also supports specifying the upper and lower bounds by `VALUES [...)`, and generating a left closed right open interval.
     * It is easier to understand by specifying `VALUES [...)`. Here is an example of the change in partition range when adding or deleting partitions using the `VALUES LESS THAN (...)` statement:
-        * As the example above, when the table is built, the following 3 partitions are automatically generated:
+        * As in the `example_range_tbl` example above, when the table is built, the following 3 partitions are automatically generated:
             ```
             P201701: [MIN_VALUE, 2017-02-01)
             P201702: [2017-02-01, 2017-03-01)
@@ -176,6 +213,39 @@ It is also possible to use only one layer of partitioning. When using a layer pa
     
     You cannot add partitions with overlapping ranges.
 
+    #### List Partition
+
+    * The partition column supports the `BOOLEAN, TINYINT, SMALLINT, INT, BIGINT, LARGEINT, DATE, DATETIME, CHAR, VARCHAR` data type, and the partition value is an enumeration value. Partitions can be hit only if the data is one of the target partition enumeration values.
+    * Partition supports specifying the number of partitions contained in each partition via `VALUES IN (...) ` to specify the enumeration values contained in each partition.
+    * The following example illustrates how partitions change when adding or deleting partitions.
+      
+        * As in the `example_list_tbl` example above, when the table is built, the following three partitions are automatically created.
+
+            ```
+            p_cn: ("Beijing", "Shanghai", "Hong Kong")
+            p_usa: ("New York", "San Francisco")
+            p_jp: ("Tokyo")
+            ```
+
+        * When we add a partition p_uk VALUES IN ("London"), the result of the partition is as follows
+        
+            ```
+            p_cn: ("Beijing", "Shanghai", "Hong Kong")
+            p_usa: ("New York", "San Francisco")
+            p_jp: ("Tokyo")
+            p_uk: ("London")
+            ```
+        
+        * When we delete the partition p_jp, the result of the partition is as follows.
+
+            ```
+            p_cn: ("Beijing", "Shanghai", "Hong Kong")
+            p_usa: ("New York", "San Francisco")
+            p_uk: ("London")
+            ```
+
+    You cannot add partitions with overlapping ranges.
+
 2. Bucket
 
     * If a Partition is used, the `DISTRIBUTED ...` statement describes the division rules for the data in each partition. If you do not use Partition, it describes the rules for dividing the data of the entire table.
@@ -203,37 +273,70 @@ It is also possible to use only one layer of partitioning. When using a layer pa
 
 Doris supports specifying multiple columns as partition columns, examples are as follows:
 
-```
-PARTITION BY RANGE(`date`, `id`)
-(
-    PARTITION `p201701_1000` VALUES LESS THAN ("2017-02-01", "1000"),
-    PARTITION `p201702_2000` VALUES LESS THAN ("2017-03-01", "2000"),
-    PARTITION `p201703_all` VALUES LESS THAN ("2017-04-01")
-)
-```
-
-In the above example, we specify `date` (DATE type) and `id` (INT type) as partition columns. The resulting partitions in the above example are as follows:
-
-```
-*p201701_1000: [(MIN_VALUE, MIN_VALUE), ("2017-02-01", "1000") )
-*p201702_2000: [("2017-02-01", "1000"), ("2017-03-01", "2000") )
-*p201703_all: [("2017-03-01", "2000"), ("2017-04-01", MIN_VALUE))
-```
-
-Note that the last partition user defaults only the partition value of the `date` column, so the partition value of the `id` column will be filled with `MIN_VALUE` by default. When the user inserts data, the partition column values ​​are compared in order, and the corresponding partition is finally obtained. Examples are as follows:
-
-```
-* Data --> Partition
-* 2017-01-01, 200 --> p201701_1000
-* 2017-01-01, 2000 --> p201701_1000
-* 2017-02-01, 100 --> p201701_1000
-* 2017-02-01, 2000 --> p201702_2000
-* 2017-02-15, 5000 --> p201702_2000
-* 2017-03-01, 2000 --> p201703_all
-* 2017-03-10, 1 --> p201703_all
-* 2017-04-01, 1000 --> Unable to import
-* 2017-05-01, 1000 --> Unable to import
-```
+##### Range Partition
+
+    ```
+    PARTITION BY RANGE(`date`, `id`)
+    (
+        PARTITION `p201701_1000` VALUES LESS THAN ("2017-02-01", "1000"),
+        PARTITION `p201702_2000` VALUES LESS THAN ("2017-03-01", "2000"),
+        PARTITION `p201703_all` VALUES LESS THAN ("2017-04-01")
+    )
+    ```
+
+    In the above example, we specify `date` (DATE type) and `id` (INT type) as partition columns. The resulting partitions in the above example are as follows:
+
+    ```
+    *p201701_1000: [(MIN_VALUE, MIN_VALUE), ("2017-02-01", "1000") )
+    *p201702_2000: [("2017-02-01", "1000"), ("2017-03-01", "2000") )
+    *p201703_all: [("2017-03-01", "2000"), ("2017-04-01", MIN_VALUE))
+    ```
+
+    Note that the last partition user defaults only the partition value of the `date` column, so the partition value of the `id` column will be filled with `MIN_VALUE` by default. When the user inserts data, the partition column values ​​are compared in order, and the corresponding partition is finally obtained. Examples are as follows:
+
+    ```
+    * Data --> Partition
+    * 2017-01-01, 200   --> p201701_1000
+    * 2017-01-01, 2000  --> p201701_1000
+    * 2017-02-01, 100   --> p201701_1000
+    * 2017-02-01, 2000  --> p201702_2000
+    * 2017-02-15, 5000  --> p201702_2000
+    * 2017-03-01, 2000  --> p201703_all
+    * 2017-03-10, 1     --> p201703_all
+    * 2017-04-01, 1000  --> Unable to import
+    * 2017-05-01, 1000  --> Unable to import
+    ```
+
+##### List Partition
+
+    ```
+    PARTITION BY LIST(`id`, `city`)
+    (
+        PARTITION `p1_city` VALUES IN (("1", "Beijing"), ("1", "Shanghai")),
+        PARTITION `p2_city` VALUES IN (("2", "Beijing"), ("2", "Shanghai")),
+        PARTITION `p3_city` VALUES IN (("3", "Beijing"), ("3", "Shanghai"))
+    )
+    ```
+
+    In the above example, we specify `id`(INT type) and `city`(VARCHAR type) as partition columns. The above example ends up with the following partitions.
+
+    ```
+    * p1_city: [("1", "Beijing"), ("1", "Shanghai")]
+    * p2_city: [("2", "Beijing"), ("2", "Shanghai")]
+    * p3_city: [("3", "Beijing"), ("3", "Shanghai")]
+    ```
+
+    When the user inserts data, the partition column values will be compared sequentially in order to finally get the corresponding partition. An example is as follows.
+
+    ```
+    * Data ---> Partition
+    * 1, Beijing  ---> p1_city
+    * 1, Shanghai ---> p1_city
+    * 2, Shanghai ---> p2_city
+    * 3, Beijing  ---> p3_city
+    * 1, Tianjin  ---> Unable to import
+    * 4, Beijing  ---> Unable to import
+    ```
 
 ### PROPERTIES
 
diff --git a/docs/en/sql-reference/sql-statements/Data Definition/CREATE TABLE.md b/docs/en/sql-reference/sql-statements/Data Definition/CREATE TABLE.md
index c03a142..56e22d9 100644
--- a/docs/en/sql-reference/sql-statements/Data Definition/CREATE TABLE.md	
+++ b/docs/en/sql-reference/sql-statements/Data Definition/CREATE TABLE.md	
@@ -170,39 +170,83 @@ Syntax:
     NOTICE:
         Except for AGGREGATE KEY, no need to specify aggregation type for value columns.
 5. partition_desc
-    Partition has two ways to use:
-    1) LESS THAN
-    Syntax:
-
-        ```
-        PARTITION BY RANGE (k1, k2, ...)
-        (
-        PARTITION partition_name1 VALUES LESS THAN MAXVALUE|("value1", "value2", ...),
-        PARTITION partition_name2 VALUES LESS THAN MAXVALUE|("value1", "value2", ...)
-        ...
-        )
-        ```
+    Currently, both RANGE and LIST partitioning methods are supported.
+    5.1 RANGE partition 
+        RANGE Partition has two ways to use:
+        1) LESS THAN
+        Syntax:
+
+            ```
+            PARTITION BY RANGE (k1, k2, ...)
+            (
+            PARTITION partition_name1 VALUES LESS THAN MAXVALUE|("value1", "value2", ...),
+            PARTITION partition_name2 VALUES LESS THAN MAXVALUE|("value1", "value2", ...)
+            ...
+            )
+            ```
+
+        Explain:
+            Use the specified key column and the specified range of values for partitioning.
+            1) Partition name only support [A-z0-9_]
+            2) Partition key column's type should be:
+                TINYINT, SMALLINT, INT, BIGINT, LARGEINT, DATE, DATETIME
+            3) The range is [closed, open). And the lower bound of first partition is MIN VALUE of  specified column type.
+            4) NULL values should be save in partition which includes MIN VALUE.
+            5) Support multi partition columns, the the default partition value is MIN VALUE.
+        2)Fixed Range
+        Syntax:
+            ```
+            PARTITION BY RANGE (k1, k2, k3, ...)
+            (
+            PARTITION partition_name1 VALUES [("k1-lower1", "k2-lower1", "k3-lower1",...),  ("k1-upper1", "k2-upper1", "k3-upper1", ...)),
+            PARTITION partition_name2 VALUES [("k1-lower1-2", "k2-lower1-2", ...), ("k1-upper1-2",  MAXVALUE, ))
+            "k3-upper1-2", ...
+            )
+            ```
+        Explain:
+            1)The Fixed Range is more flexible than the LESS THAN, and the left and right intervals    are completely determined by the user.
+            2)Others are consistent with LESS THAN.
+
+    5.2 LIST partition
+        LIST partition is divided into single column partition and multi-column partition
+        1) Single column partition
+            Syntax.
+
+            ```
+                PARTITION BY LIST(k1)
+                (
+                PARTITION partition_name1 VALUES IN ("value1", "value2", ...) ,
+                PARTITION partition_name2 VALUES IN ("value1", "value2", ...)
+                ...
+                )
+            ```
+
+            Explain:
+                Use the specified key column and the formulated enumeration value for partitioning.
+                1) Partition name only support [A-z0-9_]
+                2) Partition key column's type should be:
+                    BOOLEAN, TINYINT, SMALLINT, INT, BIGINT, LARGEINT, DATE, DATETIME, CHAR, VARCHAR
+                3) Partition is a collection of enumerated values, partition values cannot be duplicated between partitions
+                4) NULL values cannot be imported
+                5) partition values cannot be defaulted, at least one must be specified
+
+        2) Multi-column partition
+            Syntax.
+
+            ```
+                PARTITION BY LIST(k1, k2)
+                (
+                PARTITION partition_name1 VALUES IN (("value1", "value2"), ("value1", "value2"), ...) ,
+                PARTITION partition_name2 VALUES IN (("value1", "value2"), ("value1", "value2"), ...)
+                ...
+                )
+            ```
+
+            Explain:
+                1) the partition of a multi-column partition is a collection of tuple enumeration values
+                2) The number of tuple values per partition must be equal to the number of columns in the partition
+                3) The other partitions are synchronized with the single column partition
 
-    Explain:
-        1) Partition name only support [A-z0-9_]
-        2) Partition key column's type should be:
-            TINYINT, SMALLINT, INT, BIGINT, LARGEINT, DATE, DATETIME
-        3) The range is [closed, open). And the lower bound of first partition is MIN VALUE of  specified column type.
-        4) NULL values should be save in partition which includes MIN VALUE.
-        5) Support multi partition columns, the the default partition value is MIN VALUE.
-    2)Fixed Range
-    Syntax:
-        ```
-        PARTITION BY RANGE (k1, k2, k3, ...)
-        (
-        PARTITION partition_name1 VALUES [("k1-lower1", "k2-lower1", "k3-lower1",...),  ("k1-upper1", "k2-upper1", "k3-upper1", ...)),
-        PARTITION partition_name2 VALUES [("k1-lower1-2", "k2-lower1-2", ...), ("k1-upper1-2",  MAXVALUE, ))
-        "k3-upper1-2", ...
-        )
-        ```
-    Explain:
-        1)The Fixed Range is more flexible than the LESS THAN, and the left and right intervals    are completely determined by the user.
-        2)Others are consistent with LESS THAN.
 6. distribution_desc
     1) Hash
     Syntax:
@@ -246,7 +290,7 @@ Syntax:
         )
         ```
     
-    4) if you want to use the dynamic partitioning feature, specify it in properties
+    4) if you want to use the dynamic partitioning feature, specify it in properties. Note: Dynamic partitioning only supports RANGE partitions
     
         ```
         PROPERTIES (
@@ -269,7 +313,7 @@ Syntax:
        
        Dynamic_partition. Buckets: specifies the number of partition buckets that are automatically created
        ```
-8. rollup_index
+    5)  You can create multiple Rollups in bulk when building a table
     grammar:
     ```
       ROLLUP (rollup_name (column_name1, column_name2, ...)
@@ -277,7 +321,7 @@ Syntax:
                       [PROPERTIES ("key"="value", ...)],...)
     ```
 
-    5) if you want to use the inmemory table feature, specify it in properties
+    6) if you want to use the inmemory table feature, specify it in properties
 
         ```
         PROPERTIES (
@@ -323,7 +367,7 @@ Syntax:
     );
     ```
 
-3. Create an olap table, with range partitioned, distributed by hash.
+3. Create an olap table, with range partitioned, distributed by hash. Records with the same key exist at the same time, set the initial storage medium and cooling time, use default column storage.
 
 1) LESS THAN
 
@@ -381,9 +425,83 @@ Syntax:
     PROPERTIES(
     "storage_medium" = "SSD"
     );
+   
+4. Create an olap table, with list partitioned, distributed by hash. Records with the same key exist at the same time, set the initial storage medium and cooling time, use default column storage.
+
+    1) Single column partition
+
+    ```
+    CREATE TABLE example_db.table_list
+    (
+    k1 INT,
+    k2 VARCHAR(128),
+    k3 SMALLINT,
+    v1 VARCHAR(2048),
+    v2 DATETIME DEFAULT "2014-02-04 15:36:00"
+    )
+    ENGINE=olap
+    DUPLICATE KEY(k1, k2, k3)
+    PARTITION BY LIST (k1)
+    (
+    PARTITION p1 VALUES IN ("1", "2", "3"),
+    PARTITION p2 VALUES IN ("4", "5", "6"),
+    PARTITION p3 VALUES IN ("7", "8", "9")
+    )
+    DISTRIBUTED BY HASH(k2) BUCKETS 32
+    PROPERTIES(
+    "storage_medium" = "SSD", "storage_cooldown_time" = "2022-06-04 00:00:00"
+    );
+    ```
+
+    Explain:
+    This statement will divide the data into 3 partitions as follows.
+
+    ```
+    ("1", "2", "3")
+    ("4", "5", "6")
+    ("7", "8", "9")
+    ```
+
+    Data that does not fall within these partition enumeration values will be filtered as illegal data
+
+    2) Multi-column partition
+
+    ```
+    CREATE TABLE example_db.table_list
+    (
+    k1 INT,
+    k2 VARCHAR(128),
+    k3 SMALLINT,
+    v1 VARCHAR(2048),
+    v2 DATETIME DEFAULT "2014-02-04 15:36:00"
+    )
+    ENGINE=olap
+    DUPLICATE KEY(k1, k2, k3)
+    PARTITION BY LIST (k1, k2)
+    (
+    PARTITION p1 VALUES IN (("1", "beijing"), ("1", "shanghai")),
+    PARTITION p2 VALUES IN (("2", "beijing"), ("2", "shanghai")),
+    PARTITION p3 VALUES IN (("3", "beijing"), ("3", "shanghai"))
+    )
+    DISTRIBUTED BY HASH(k2) BUCKETS 32
+    PROPERTIES(
+    "storage_medium" = "SSD", "storage_cooldown_time" = "2022-06-04 00:00:00"
+    );
+    ```
+
+    Explain:
+    This statement will divide the data into 3 partitions as follows.
+
+    ```
+    (("1", "beijing"), ("1", "shanghai"))
+    (("2", "beijing"), ("2", "shanghai"))
+    (("3", "beijing"), ("3", "shanghai"))
+    ```
+
+    Data that is not within these partition enumeration values will be filtered as illegal data
 
-4. Create a mysql table
-   4.1 Create MySQL table directly from external table information
+5. Create a mysql table
+   5.1 Create MySQL table directly from external table information
 ```
     CREATE EXTERNAL TABLE example_db.table_mysql
     (
@@ -405,7 +523,7 @@ Syntax:
     )
 ```
 
-   4.2 Create MySQL table with external ODBC catalog resource
+   5.2 Create MySQL table with external ODBC catalog resource
 ```
    CREATE EXTERNAL RESOURCE "mysql_resource" 
    PROPERTIES
@@ -435,7 +553,7 @@ Syntax:
     )
 ```
 
-5. Create a broker table, with file on HDFS, line delimit by "|", column separated by "\n"
+6. Create a broker table, with file on HDFS, line delimit by "|", column separated by "\n"
 
     ```
     CREATE EXTERNAL TABLE example_db.table_broker (
@@ -458,7 +576,7 @@ Syntax:
     );
     ```
 
-6. Create table will HLL column
+7. Create table will HLL column
 
     ```
     CREATE TABLE example_db.example_table
@@ -473,7 +591,7 @@ Syntax:
     DISTRIBUTED BY HASH(k1) BUCKETS 32;
     ```
 
-7. Create a table will BITMAP_UNION column
+8. Create a table will BITMAP_UNION column
 
     ```
     CREATE TABLE example_db.example_table
@@ -488,7 +606,7 @@ Syntax:
     DISTRIBUTED BY HASH(k1) BUCKETS 32;
     ```
 
-8. Create 2 colocate join table.
+9. Create 2 colocate join table.
 
     ```
     CREATE TABLE `t1` (
@@ -511,7 +629,7 @@ Syntax:
     );
     ```
 
-9. Create a broker table, with file on BOS.
+10. Create a broker table, with file on BOS.
 
     ```
     CREATE EXTERNAL TABLE example_db.table_broker (
@@ -529,7 +647,7 @@ Syntax:
     );
     ```
 
-10. Create a table with a bitmap index 
+11. Create a table with a bitmap index 
 
     ```
     CREATE TABLE example_db.table_hash
@@ -547,7 +665,7 @@ Syntax:
     PROPERTIES ("storage_type"="column");
     ```
     
-11. Create a dynamic partitioning table (dynamic partitioning needs to be enabled in FE configuration), which creates partitions 3 days in advance every day. For example, if today is' 2020-01-08 ', partitions named 'p20200108', 'p20200109', 'p20200110', 'p20200111' will be created.
+12. Create a dynamic partitioning table (dynamic partitioning needs to be enabled in FE configuration), which creates partitions 3 days in advance every day. For example, if today is' 2020-01-08 ', partitions named 'p20200108', 'p20200109', 'p20200110', 'p20200111' will be created.
 
     ```
     [types: [DATE]; keys: [2020-01-08]; ‥types: [DATE]; keys: [2020-01-09]; )
@@ -582,7 +700,7 @@ Syntax:
         "dynamic_partition.buckets" = "32"
          );
      ```
-12. Create a table with rollup index
+13. Create a table with rollup index
 ```
     CREATE TABLE example_db.rolup_index_table
     (
@@ -602,7 +720,7 @@ Syntax:
     PROPERTIES("replication_num" = "3");
 ```
 
-13. Create a inmemory table:
+14. Create a inmemory table:
 
 ```
     CREATE TABLE example_db.table_hash
@@ -620,7 +738,7 @@ Syntax:
     PROPERTIES ("in_memory"="true");
 ```
 
-14. Create a hive external table
+15. Create a hive external table
 ```
     CREATE TABLE example_db.table_hive
     (
diff --git a/docs/en/sql-reference/sql-statements/Data Definition/RESTORE.md b/docs/en/sql-reference/sql-statements/Data Definition/RESTORE.md
index 0a467d7..7df04c4 100644
--- a/docs/en/sql-reference/sql-statements/Data Definition/RESTORE.md	
+++ b/docs/en/sql-reference/sql-statements/Data Definition/RESTORE.md	
@@ -42,7 +42,7 @@ Explain:
 2. The ON clause identifies the tables and partitions that need to be restored. If no partition is specified, all partitions of the table are restored by default. The specified tables and partitions must already exist in the warehouse backup.
 3. The backup tables in the warehouse can be restored to new tables through AS statements. But the new table name cannot already exist in the database. Partition name cannot be changed.
 4. The backup tables in the warehouse can be restored and replaced with the same-name tables in the database, but the table structure of the two tables must be completely consistent. Table structure includes: table name, column, partition, Rollup and so on.
-5. Partitions of the recovery table can be specified, and the system checks whether the partition Range matches.
+5. Partitions of the recovery table can be specified, and the system checks whether the partition Range or List matches.
 6. PROPERTIES currently supports the following attributes:
 "Backup_timestamp" = "2018-05-04-16-45-08": specifies which version of the time to restore the corresponding backup must be filled in. This information can be obtained through the `SHOW SNAPSHOT ON repo;'statement.
 "Replication_num" = "3": Specifies the number of replicas of the restored table or partition. The default is 3. If an existing table or partition is restored, the number of copies must be the same as the number of copies of an existing table or partition. At the same time, there must be enough hosts to accommodate multiple copies.
diff --git a/docs/en/sql-reference/sql-statements/Data Manipulation/DELETE.md b/docs/en/sql-reference/sql-statements/Data Manipulation/DELETE.md
index a0ef240..be4c606 100644
--- a/docs/en/sql-reference/sql-statements/Data Manipulation/DELETE.md	
+++ b/docs/en/sql-reference/sql-statements/Data Manipulation/DELETE.md	
@@ -40,7 +40,7 @@ under the License.
         2) When the selected key column does not exist in a rollup, delete cannot be performed.
         3) The relationship between conditions can only be "and".
         If you want to achieve the "or" relationship, you need to divide the conditions into two DELETE statements.
-        4) If it is a RANGE partitioned table, you can specify the partition. If not specified, and the session variable delete_without_partition is true, it will be applied to all partitions. If it is a single partition table, you do not need to specify it.
+        4) If it is a partitioned table, you can specify the partition. If not specified, and the session variable delete_without_partition is true, it will be applied to all partitions. If it is a single partition table, you do not need to specify it.
     
     Notice:
         This statement may reduce query efficiency for a period of time after execution.
diff --git a/docs/zh-CN/administrator-guide/alter-table/alter-table-temp-partition.md b/docs/zh-CN/administrator-guide/alter-table/alter-table-temp-partition.md
index a9493bd..b467596 100644
--- a/docs/zh-CN/administrator-guide/alter-table/alter-table-temp-partition.md
+++ b/docs/zh-CN/administrator-guide/alter-table/alter-table-temp-partition.md
@@ -52,6 +52,15 @@ ALTER TABLE tbl2 ADD TEMPORARY PARTITION tp1 VALUES [("2020-01-01"), ("2020-02-0
 ALTER TABLE tbl1 ADD TEMPORARY PARTITION tp1 VALUES LESS THAN("2020-02-01")
 ("in_memory" = "true", "replication_num" = "1")
 DISTRIBUTED BY HASH(k1) BUCKETS 5;
+
+ALTER TABLE tbl3 ADD TEMPORARY PARTITION tp1 VALUES IN ("Beijing", "Shanghai");
+
+ALTER TABLE tbl4 ADD TEMPORARY PARTITION tp1 VALUES IN ((1, "Beijing"), (1, "Shanghai"));
+
+ALTER TABLE tbl3 ADD TEMPORARY PARTITION tp1 VALUES IN ("Beijing", "Shanghai")
+("in_memory" = "true", "replication_num" = "1")
+DISTRIBUTED BY HASH(k1) BUCKETS 5;
+
 ```
 
 通过 `HELP ALTER TABLE;` 查看更多帮助和示例。
@@ -97,7 +106,13 @@ PROPERTIES (
 
 1. `strict_range`
 
-    默认为 true。当该参数为 true 时,表示要被替换的所有正式分区的范围并集需要和替换的临时分区的范围并集完全相同。当置为 false 时,只需要保证替换后,新的正式分区间的范围不重叠即可。下面举例说明:
+    默认为 true。  
+
+    对于 Range 分区,当该参数为 true 时,表示要被替换的所有正式分区的范围并集需要和替换的临时分区的范围并集完全相同。当置为 false 时,只需要保证替换后,新的正式分区间的范围不重叠即可。
+
+    对于 List 分区,该参数恒为 true。要被替换的所有正式分区的枚举值必须和替换的临时分区枚举值完全相同。
+    
+    下面举例说明:
     
     * 示例1
     
@@ -131,6 +146,38 @@ PROPERTIES (
         
         范围并集不相同,如果 `strict_range` 为 true,则不可以使用 tp1 和 tp2 替换 p1。如果为 false,且替换后的两个分区范围 `[10, 30), [40, 50)` 和其他正式分区不重叠,则可以替换。
 
+    * 示例3
+
+        待替换的分区 p1, p2 的枚举值(=> 并集):
+
+        ```
+        (1, 2, 3), (4, 5, 6) => (1, 2, 3, 4, 5, 6)
+        ```
+
+        替换分区 tp1, tp2, tp3 的枚举值(=> 并集):
+
+        ```
+        (1, 2, 3), (4), (5, 6) => (1, 2, 3, 4, 5, 6)
+        ```
+
+        枚举值并集相同,可以使用 tp1,tp2,tp3 替换 p1,p2
+
+    * 示例4
+
+        待替换的分区 p1, p2,p3 的枚举值(=> 并集):
+
+        ```
+        (("1","beijing"), ("1", "shanghai")), (("2","beijing"), ("2", "shanghai")), (("3","beijing"), ("3", "shanghai")) => (("1","beijing"), ("1", "shanghai"), ("2","beijing"), ("2", "shanghai"), ("3","beijing"), ("3", "shanghai"))
+        ```
+
+        替换分区 tp1, tp2 的枚举值(=> 并集):
+
+        ```
+        (("1","beijing"), ("1", "shanghai")), (("2","beijing"), ("2", "shanghai"), ("3","beijing"), ("3", "shanghai")) => (("1","beijing"), ("1", "shanghai"), ("2","beijing"), ("2", "shanghai"), ("3","beijing"), ("3", "shanghai"))
+        ```
+
+        枚举值并集相同,可以使用 tp1,tp2 替换 p1,p2,p3
+
 2. `use_temp_partition_name`
 
     默认为 false。当该参数为 false,并且待替换的分区和替换分区的个数相同时,则替换后的正式分区名称维持不变。如果为 true,则替换后,正式分区的名称为替换分区的名称。下面举例说明:
diff --git a/docs/zh-CN/administrator-guide/dynamic-partition.md b/docs/zh-CN/administrator-guide/dynamic-partition.md
index 2e84276..dfbc085 100644
--- a/docs/zh-CN/administrator-guide/dynamic-partition.md
+++ b/docs/zh-CN/administrator-guide/dynamic-partition.md
@@ -30,6 +30,8 @@ under the License.
 
 目前实现了动态添加分区及动态删除分区的功能。
 
+动态分区只支持 Range 分区。
+
 ## 名词解释
 
 * FE:Frontend,Doris 的前端节点。负责元数据管理和请求接入。
diff --git a/docs/zh-CN/administrator-guide/load-data/delete-manual.md b/docs/zh-CN/administrator-guide/load-data/delete-manual.md
index ac48e87..6bfdeab 100644
--- a/docs/zh-CN/administrator-guide/load-data/delete-manual.md
+++ b/docs/zh-CN/administrator-guide/load-data/delete-manual.md
@@ -64,10 +64,10 @@ DELETE FROM my_table PARTITION p1 WHERE k1 < 3 AND k2 = "abc";
 
 1. `Where`语句中的op的类型可包括`=, >, <, >=, <=, !=, in, not in`。
 2. `Where`语句中的列只能是`key`列
-3.  当选定的`key`列不存在某个rollup表内时,无法进行delete
-4.  条件语句中各个条件只能是`and`关系,如希望达成`or`可将条件分别写入两个delete语句中
-5.  如果指定表为RANGE分区表,则必须指定 `PARTITION`。如果是单分区表,可以不指定。
-6.  不同于Insert into命令,delete不能手动指定`label`,有关label的概念可以查看[Insert Into文档](./insert-into-manual.md)
+3.  当选定的`key`列不存在某个 rollup 表内时,无法进行 delete
+4.  条件语句中各个条件只能是`and`关系,如希望达成`or`可将条件分别写入两个 delete 语句中
+5.  如果指定表为 RANGE 或者 LIST 分区表,则必须指定 `PARTITION`。如果是单分区表,可以不指定。
+6.  不同于 Insert into 命令,delete 不能手动指定`label`,有关 label 的概念可以查看[Insert Into文档](./insert-into-manual.md)
 
 ## 返回结果
 
diff --git a/docs/zh-CN/getting-started/best-practice.md b/docs/zh-CN/getting-started/best-practice.md
index c4e2a4f..a51adfa 100644
--- a/docs/zh-CN/getting-started/best-practice.md
+++ b/docs/zh-CN/getting-started/best-practice.md
@@ -97,15 +97,23 @@ Doris 数据模型上目前分为三类: AGGREGATE KEY, UNIQUE KEY, DUPLICATE KE
  
 ### 1.3 分区和分桶
 
-Doris 支持两级分区存储, 第一层为 RANGE 分区(partition), 第二层为 HASH 分桶(bucket)。
+Doris 支持两级分区存储, 第一层为分区(partition),目前支持 RANGE 分区和 LIST 分区两种类型, 第二层为 HASH 分桶(bucket)。
 
-1.3.1. RANGE分区(partition)
+1.3.1. 分区(partition)
 
-    RANGE分区用于将数据划分成不同区间, 逻辑上可以理解为将原始表划分成了多个子表。业务上,多数用户会选择采用按时间进行partition, 让时间进行partition有以下好处:
+    分区用于将数据划分成不同区间, 逻辑上可以理解为将原始表划分成了多个子表。可以方便的按分区对数据进行管理,例如,删除数据时,更加迅速。
+
+1.3.1.1. RANGE分区
+
+    业务上,多数用户会选择采用按时间进行partition, 让时间进行partition有以下好处:
     
     * 可区分冷热数据
     * 可用上Doris分级存储(SSD + SATA)的功能
-    * 按分区删除数据时,更加迅速
+
+1.3.1.2. LIST分区
+
+    业务上,用户可以选择城市或者其他枚举值进行partition。
+
 
 1.3.2. HASH分桶(bucket)
 
diff --git a/docs/zh-CN/getting-started/data-partition.md b/docs/zh-CN/getting-started/data-partition.md
index bce0ec5..14f6073 100644
--- a/docs/zh-CN/getting-started/data-partition.md
+++ b/docs/zh-CN/getting-started/data-partition.md
@@ -57,7 +57,9 @@ Doris 的建表是一个同步命令,命令返回成功,即表示建表成
 本小节通过一个例子,来介绍 Doris 的建表方式。
 
 ```
-CREATE TABLE IF NOT EXISTS example_db.expamle_tbl
+-- Range Partition
+
+CREATE TABLE IF NOT EXISTS example_db.expamle_range_tbl
 (
     `user_id` LARGEINT NOT NULL COMMENT "用户id",
     `date` DATE NOT NULL COMMENT "数据灌入日期时间",
@@ -86,6 +88,38 @@ PROPERTIES
     "storage_cooldown_time" = "2018-01-01 12:00:00"
 );
 
+
+-- List Partition
+
+CREATE TABLE IF NOT EXISTS example_db.expamle_list_tbl
+(
+    `user_id` LARGEINT NOT NULL COMMENT "用户id",
+    `date` DATE NOT NULL COMMENT "数据灌入日期时间",
+    `timestamp` DATETIME 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`, `timestamp`, `city`, `age`, `sex`)
+PARTITION BY LIST(`city`)
+(
+    PARTITION `p_cn` VALUES IN ("Beijing", "Shanghai", "Hong Kong"),
+    PARTITION `p_usa` VALUES IN ("New York", "San Francisco"),
+    PARTITION `p_jp` VALUES IN ("Tokyo")
+)
+DISTRIBUTED BY HASH(`user_id`) BUCKETS 16
+PROPERTIES
+(
+    "replication_num" = "3",
+    "storage_medium" = "SSD",
+    "storage_cooldown_time" = "2018-01-01 12:00:00"
+);
+
 ``` 
 
 ### 列定义
@@ -106,7 +140,7 @@ AGGREGATE KEY 数据模型中,所有没有指定聚合方式(SUM、REPLACE
 
 ### 分区与分桶
 
-Doris 支持两层的数据划分。第一层是 Partition,仅支持 Range 的划分方式。第二层是 Bucket(Tablet),仅支持 Hash 的划分方式。
+Doris 支持两层的数据划分。第一层是 Partition,支持 Range 和 List 的划分方式。第二层是 Bucket(Tablet),仅支持 Hash 的划分方式。
 
 也可以仅使用一层分区。使用一层分区时,只支持 Bucket 划分。
 
@@ -114,14 +148,17 @@ Doris 支持两层的数据划分。第一层是 Partition,仅支持 Range 的
 
     * Partition 列可以指定一列或多列。分区类必须为 KEY 列。多列分区的使用方式在后面 **多列分区** 小结介绍。
     * 不论分区列是什么类型,在写分区值时,都需要加双引号。
-    * 分区列通常为时间列,以方便的管理新旧数据。
     * 分区数量理论上没有上限。
     * 当不使用 Partition 建表时,系统会自动生成一个和表名同名的,全值范围的 Partition。该 Partition 对用户不可见,并且不可删改。
+  
+    #### Range 分区
+
+    * 分区列通常为时间列,以方便的管理新旧数据。
     * Partition 支持通过 `VALUES LESS THAN (...)` 仅指定上界,系统会将前一个分区的上界作为该分区的下界,生成一个左闭右开的区间。通过,也支持通过 `VALUES [...)` 指定同时指定上下界,生成一个左闭右开的区间。
 
     * 通过 `VALUES [...)` 同时指定上下界比较容易理解。这里举例说明,当使用 `VALUES LESS THAN (...)` 语句进行分区的增删操作时,分区范围的变化情况:
     
-        * 如上示例,当建表完成后,会自动生成如下3个分区:
+        * 如上 `expamle_range_tbl` 示例,当建表完成后,会自动生成如下3个分区:
 
             ```
             p201701: [MIN_VALUE,  2017-02-01)
@@ -180,6 +217,39 @@ Doris 支持两层的数据划分。第一层是 Partition,仅支持 Range 的
     
     不可添加范围重叠的分区。
 
+    #### List 分区
+
+    * 分区列支持 `BOOLEAN, TINYINT, SMALLINT, INT, BIGINT, LARGEINT, DATE, DATETIME, CHAR, VARCHAR` 数据类型,分区值为枚举值。只有当数据为目标分区枚举值其中之一时,才可以命中分区。
+    * Partition 支持通过 `VALUES IN (...)` 来指定每个分区包含的枚举值。
+    * 下面通过示例说明,进行分区的增删操作时,分区的变化。
+      
+        * 如上 `example_list_tbl` 示例,当建表完成后,会自动生成如下3个分区:
+
+            ```
+            p_cn: ("Beijing", "Shanghai", "Hong Kong")
+            p_usa: ("New York", "San Francisco")
+            p_jp: ("Tokyo")
+            ```
+
+        * 当我们增加一个分区 p_uk VALUES IN ("London"),分区结果如下:
+        
+            ```
+            p_cn: ("Beijing", "Shanghai", "Hong Kong")
+            p_usa: ("New York", "San Francisco")
+            p_jp: ("Tokyo")
+            p_uk: ("London")
+            ```
+        
+        * 当我们删除分区 p_jp,分区结果如下:
+
+            ```
+            p_cn: ("Beijing", "Shanghai", "Hong Kong")
+            p_usa: ("New York", "San Francisco")
+            p_uk: ("London")
+            ```
+
+    不可添加范围重叠的分区。
+
 2. Bucket
 
     * 如果使用了 Partition,则 `DISTRIBUTED ...` 语句描述的是数据在**各个分区内**的划分规则。如果不使用 Partition,则描述的是对整个表的数据的划分规则。
@@ -206,37 +276,70 @@ Doris 支持两层的数据划分。第一层是 Partition,仅支持 Range 的
 
 Doris 支持指定多列作为分区列,示例如下:
 
-```
-PARTITION BY RANGE(`date`, `id`)
-(
-    PARTITION `p201701_1000` VALUES LESS THAN ("2017-02-01", "1000"),
-    PARTITION `p201702_2000` VALUES LESS THAN ("2017-03-01", "2000"),
-    PARTITION `p201703_all`  VALUES LESS THAN ("2017-04-01")
-)
-```
-
-在以上示例中,我们指定 `date`(DATE 类型) 和 `id`(INT 类型) 作为分区列。以上示例最终得到的分区如下:
-
-```
-* p201701_1000:    [(MIN_VALUE,  MIN_VALUE), ("2017-02-01", "1000")   )
-* p201702_2000:    [("2017-02-01", "1000"),  ("2017-03-01", "2000")   )
-* p201703_all:     [("2017-03-01", "2000"),  ("2017-04-01", MIN_VALUE)) 
-```
-
-注意,最后一个分区用户缺省只指定了 `date` 列的分区值,所以 `id` 列的分区值会默认填充 `MIN_VALUE`。当用户插入数据时,分区列值会按照顺序依次比较,最终得到对应的分区。举例如下:
-
-```
-* 数据  -->  分区
-* 2017-01-01, 200     --> p201701_1000
-* 2017-01-01, 2000    --> p201701_1000
-* 2017-02-01, 100     --> p201701_1000
-* 2017-02-01, 2000    --> p201702_2000
-* 2017-02-15, 5000    --> p201702_2000
-* 2017-03-01, 2000    --> p201703_all
-* 2017-03-10, 1       --> p201703_all
-* 2017-04-01, 1000    --> 无法导入
-* 2017-05-01, 1000    --> 无法导入
-```
+##### Range 分区
+
+    ```
+    PARTITION BY RANGE(`date`, `id`)
+    (
+        PARTITION `p201701_1000` VALUES LESS THAN ("2017-02-01", "1000"),
+        PARTITION `p201702_2000` VALUES LESS THAN ("2017-03-01", "2000"),
+        PARTITION `p201703_all`  VALUES LESS THAN ("2017-04-01")
+    )
+    ```
+
+    在以上示例中,我们指定 `date`(DATE 类型) 和 `id`(INT 类型) 作为分区列。以上示例最终得到的分区如下:
+
+    ```
+    * p201701_1000:    [(MIN_VALUE,  MIN_VALUE), ("2017-02-01", "1000")   )
+    * p201702_2000:    [("2017-02-01", "1000"),  ("2017-03-01", "2000")   )
+    * p201703_all:     [("2017-03-01", "2000"),  ("2017-04-01", MIN_VALUE)) 
+    ```
+
+    注意,最后一个分区用户缺省只指定了 `date` 列的分区值,所以 `id` 列的分区值会默认填充 `MIN_VALUE`。当用户插入数据时,分区列值会按照顺序依次比较,最终得到对应的分区。举例如下:
+
+    ```
+    * 数据  -->  分区
+    * 2017-01-01, 200     --> p201701_1000
+    * 2017-01-01, 2000    --> p201701_1000
+    * 2017-02-01, 100     --> p201701_1000
+    * 2017-02-01, 2000    --> p201702_2000
+    * 2017-02-15, 5000    --> p201702_2000
+    * 2017-03-01, 2000    --> p201703_all
+    * 2017-03-10, 1       --> p201703_all
+    * 2017-04-01, 1000    --> 无法导入
+    * 2017-05-01, 1000    --> 无法导入
+    ```
+
+##### List 分区
+
+    ```
+    PARTITION BY LIST(`id`, `city`)
+    (
+        PARTITION `p1_city` VALUES IN (("1", "Beijing"), ("1", "Shanghai")),
+        PARTITION `p2_city` VALUES IN (("2", "Beijing"), ("2", "Shanghai")),
+        PARTITION `p3_city` VALUES IN (("3", "Beijing"), ("3", "Shanghai"))
+    )
+    ```
+
+    在以上示例中,我们指定 `id`(INT 类型) 和 `city`(VARCHAR 类型) 作为分区列。以上示例最终得到的分区如下:
+
+    ```
+    * p1_city: [("1", "Beijing"), ("1", "Shanghai")]
+    * p2_city: [("2", "Beijing"), ("2", "Shanghai")]
+    * p3_city: [("3", "Beijing"), ("3", "Shanghai")]
+    ```
+
+    当用户插入数据时,分区列值会按照顺序依次比较,最终得到对应的分区。举例如下:
+
+    ```
+    * 数据  --->  分区
+    * 1, Beijing     ---> p1_city
+    * 1, Shanghai    ---> p1_city
+    * 2, Shanghai    ---> p2_city
+    * 3, Beijing     ---> p3_city
+    * 1, Tianjin     ---> 无法导入
+    * 4, Beijing     ---> 无法导入
+    ```
 
 ### PROPERTIES
 
diff --git a/docs/zh-CN/sql-reference/sql-statements/Data Definition/CREATE TABLE.md b/docs/zh-CN/sql-reference/sql-statements/Data Definition/CREATE TABLE.md
index 7103bc2..28d1e2f 100644
--- a/docs/zh-CN/sql-reference/sql-statements/Data Definition/CREATE TABLE.md	
+++ b/docs/zh-CN/sql-reference/sql-statements/Data Definition/CREATE TABLE.md	
@@ -164,7 +164,7 @@ under the License.
     其中 database 是 hive 表对应的库名字,table 是 hive 表的名字,hive.metastore.uris 是 hive metastore 服务地址。
     注意:目前hive外部表仅用于Spark Load使用,不支持查询。
 
-1. key_desc
+4. key_desc
     语法:
         `key_type(k1[,k2 ...])`
     说明:
@@ -180,48 +180,90 @@ under the License.
     注意:
         除AGGREGATE KEY外,其他key_type在建表时,value列不需要指定聚合类型。
 
-2. partition_desc
-    partition描述有两种使用方式
-    1) LESS THAN
-        语法:
-
-        ```
-            PARTITION BY RANGE (k1, k2, ...)
-            (
-            PARTITION partition_name1 VALUES LESS THAN MAXVALUE|("value1", "value2", ...),
-            PARTITION partition_name2 VALUES LESS THAN MAXVALUE|("value1", "value2", ...)
-            ...
-            )
-        ```
-        
-        说明:
-            使用指定的 key 列和指定的数值范围进行分区。
-            1) 分区名称仅支持字母开头,字母、数字和下划线组成
-            2) 目前仅支持以下类型的列作为 Range 分区列,且只能指定一个分区列
-                TINYINT, SMALLINT, INT, BIGINT, LARGEINT, DATE, DATETIME
-            3) 分区为左闭右开区间,首个分区的左边界为做最小值
-            4) NULL 值只会存放在包含最小值的分区中。当包含最小值的分区被删除后,NULL 值将无法导入。
-            5) 可以指定一列或多列作为分区列。如果分区值缺省,则会默认填充最小值。
-        
-        注意:
-            1) 分区一般用于时间维度的数据管理
-            2) 有数据回溯需求的,可以考虑首个分区为空分区,以便后续增加分区
-
-    2)Fixed Range
-        语法:
-        ```
-            PARTITION BY RANGE (k1, k2, k3, ...)
-            (
-            PARTITION partition_name1 VALUES [("k1-lower1", "k2-lower1", "k3-lower1",...), ("k1-upper1", "k2-upper1", "k3-upper1", ...)),
-            PARTITION partition_name2 VALUES [("k1-lower1-2", "k2-lower1-2", ...), ("k1-upper1-2", MAXVALUE, ))
-            "k3-upper1-2", ...
-            )
-        ```
-        说明:
-            1)Fixed Range比LESS THAN相对灵活些,左右区间完全由用户自己确定
-            2)其他与LESS THAN保持同步
-
-3. distribution_desc
+5. partition_desc
+    目前支持 RANGE 和 LIST 两种分区方式。
+    5.1 RANGE 分区
+        RANGE partition描述有两种使用方式
+        1) LESS THAN
+            语法:
+
+            ```
+                PARTITION BY RANGE (k1, k2, ...)
+                (
+                PARTITION partition_name1 VALUES LESS THAN MAXVALUE|("value1", "value2", ...),
+                PARTITION partition_name2 VALUES LESS THAN MAXVALUE|("value1", "value2", ...)
+                ...
+                )
+            ```
+            
+            说明:
+                使用指定的 key 列和指定的数值范围进行分区。
+                1) 分区名称仅支持字母开头,字母、数字和下划线组成
+                2) 目前仅支持以下类型的列作为 Range 分区列
+                    TINYINT, SMALLINT, INT, BIGINT, LARGEINT, DATE, DATETIME
+                3) 分区为左闭右开区间,首个分区的左边界为做最小值
+                4) NULL 值只会存放在包含最小值的分区中。当包含最小值的分区被删除后,NULL 值将无法导入。
+                5) 可以指定一列或多列作为分区列。如果分区值缺省,则会默认填充最小值。
+            
+            注意:
+                1) 分区一般用于时间维度的数据管理
+                2) 有数据回溯需求的,可以考虑首个分区为空分区,以便后续增加分区
+
+        2)Fixed Range
+            语法:
+            ```
+                PARTITION BY RANGE (k1, k2, k3, ...)
+                (
+                PARTITION partition_name1 VALUES [("k1-lower1", "k2-lower1", "k3-lower1",...), ("k1-upper1", "k2-upper1", "k3-upper1", ...)),
+                PARTITION partition_name2 VALUES [("k1-lower1-2", "k2-lower1-2", ...), ("k1-upper1-2", MAXVALUE, ))
+                "k3-upper1-2", ...
+                )
+            ```
+            说明:
+                1)Fixed Range比LESS THAN相对灵活些,左右区间完全由用户自己确定
+                2)其他与LESS THAN保持同步
+
+    5.2 LIST 分区
+        LIST partition分为单列分区和多列分区
+        1) 单列分区
+            语法:
+
+            ```
+                PARTITION BY LIST(k1)
+                (
+                PARTITION partition_name1 VALUES IN ("value1", "value2", ...),
+                PARTITION partition_name2 VALUES IN ("value1", "value2", ...)
+                ...
+                )
+            ```
+
+            说明:
+                使用指定的 key 列和制定的枚举值进行分区。
+                1) 分区名称仅支持字母开头,字母、数字和下划线组成
+                2) 目前仅支持以下类型的列作为 List 分区列
+                    BOOLEAN, TINYINT, SMALLINT, INT, BIGINT, LARGEINT, DATE, DATETIME, CHAR, VARCHAR
+                3) 分区为枚举值集合,各个分区之间分区值不能重复
+                4) 不可导入 NULL 值
+                5) 分区值不能缺省,必须指定至少一个
+
+        2) 多列分区
+            语法:
+
+            ```
+                PARTITION BY LIST(k1, k2)
+                (
+                PARTITION partition_name1 VALUES IN (("value1", "value2"), ("value1", "value2"), ...),
+                PARTITION partition_name2 VALUES IN (("value1", "value2"), ("value1", "value2"), ...)
+                ...
+                )
+            ```
+
+            说明:
+                1) 多列分区的分区是元组枚举值的集合
+                2) 每个元组值的个数必须与分区列个数相等
+                3) 其他与单列分区保持同步
+
+6. distribution_desc
         1) Hash 分桶
         语法:
             `DISTRIBUTED BY HASH (k1[,k2 ...]) [BUCKETS num]`
@@ -230,7 +272,7 @@ under the License.
 
     建议:建议使用Hash分桶方式
 
-4. PROPERTIES
+7. PROPERTIES
     1) 如果 ENGINE 类型为 olap
            可以在 properties 设置该表数据的初始存储介质、存储到期时间和副本数。
 
@@ -271,7 +313,7 @@ under the License.
            )
 ```
 
-    4) 如果希望使用动态分区特性,需要在properties 中指定
+    4) 如果希望使用动态分区特性,需要在properties 中指定。注意:动态分区只支持 RANGE 分区
 
 ```
       PROPERTIES (
@@ -418,9 +460,84 @@ under the License.
     );
     ```
 
-4. 创建一个 mysql 表
+4. 创建一个 olap 表,使用 List 分区,使用Hash分桶,默认使用列存,
+   相同key的记录同时存在,设置初始存储介质和冷却时间
+
+    1)单列分区
+
+    ```
+    CREATE TABLE example_db.table_list
+    (
+    k1 INT,
+    k2 VARCHAR(128),
+    k3 SMALLINT,
+    v1 VARCHAR(2048),
+    v2 DATETIME DEFAULT "2014-02-04 15:36:00"
+    )
+    ENGINE=olap
+    DUPLICATE KEY(k1, k2, k3)
+    PARTITION BY LIST (k1)
+    (
+    PARTITION p1 VALUES IN ("1", "2", "3"),
+    PARTITION p2 VALUES IN ("4", "5", "6"),
+    PARTITION p3 VALUES IN ("7", "8", "9")
+    )
+    DISTRIBUTED BY HASH(k2) BUCKETS 32
+    PROPERTIES(
+    "storage_medium" = "SSD", "storage_cooldown_time" = "2022-06-04 00:00:00"
+    );
+    ```
+
+    说明:
+    这个语句会将数据划分成如下3个分区:
+
+    ```
+    ("1", "2", "3")
+    ("4", "5", "6")
+    ("7", "8", "9")
+    ```
+
+    不在这些分区枚举值内的数据将视为非法数据被过滤
+
+    2) 多列分区
+
+    ```
+    CREATE TABLE example_db.table_list
+    (
+    k1 INT,
+    k2 VARCHAR(128),
+    k3 SMALLINT,
+    v1 VARCHAR(2048),
+    v2 DATETIME DEFAULT "2014-02-04 15:36:00"
+    )
+    ENGINE=olap
+    DUPLICATE KEY(k1, k2, k3)
+    PARTITION BY LIST (k1, k2)
+    (
+    PARTITION p1 VALUES IN (("1","beijing"), ("1", "shanghai")),
+    PARTITION p2 VALUES IN (("2","beijing"), ("2", "shanghai")),
+    PARTITION p3 VALUES IN (("3","beijing"), ("3", "shanghai"))
+    )
+    DISTRIBUTED BY HASH(k2) BUCKETS 32
+    PROPERTIES(
+    "storage_medium" = "SSD", "storage_cooldown_time" = "2022-06-04 00:00:00"
+    );
+    ```
+
+    说明:
+    这个语句会将数据划分成如下3个分区:
 
-   4.1 直接通过外表信息创建mysql表
+    ```
+    (("1","beijing"), ("1", "shanghai"))
+    (("2","beijing"), ("2", "shanghai"))
+    (("3","beijing"), ("3", "shanghai"))
+    ```
+
+    不在这些分区枚举值内的数据将视为非法数据被过滤
+
+5. 创建一个 mysql 表
+
+   5.1 直接通过外表信息创建mysql表
 ```
     CREATE EXTERNAL TABLE example_db.table_mysql
     (
@@ -442,7 +559,7 @@ under the License.
     )
 ```
 
-   4.2 通过External Catalog Resource创建mysql表
+   5.2 通过External Catalog Resource创建mysql表
 ```
    CREATE EXTERNAL RESOURCE "mysql_resource" 
    PROPERTIES
@@ -472,7 +589,7 @@ under the License.
     )
 ```
 
-5. 创建一个数据文件存储在HDFS上的 broker 外部表, 数据使用 "|" 分割,"\n" 换行
+6. 创建一个数据文件存储在HDFS上的 broker 外部表, 数据使用 "|" 分割,"\n" 换行
 
 ```
     CREATE EXTERNAL TABLE example_db.table_broker (
@@ -495,7 +612,7 @@ under the License.
     )
 ```
 
-6. 创建一张含有HLL列的表
+7. 创建一张含有HLL列的表
 
 ```
     CREATE TABLE example_db.example_table
@@ -511,7 +628,7 @@ under the License.
     PROPERTIES ("storage_type"="column");
 ```
 
-7. 创建一张含有BITMAP_UNION聚合类型的表(v1和v2列的原始数据类型必须是TINYINT,SMALLINT,INT)
+8. 创建一张含有BITMAP_UNION聚合类型的表(v1和v2列的原始数据类型必须是TINYINT,SMALLINT,INT)
 
 ```
     CREATE TABLE example_db.example_table
@@ -527,7 +644,7 @@ under the License.
     PROPERTIES ("storage_type"="column");
 ```
 
-8. 创建两张支持Colocate Join的表t1 和t2
+9. 创建两张支持Colocate Join的表t1 和t2
 
 ```
     CREATE TABLE `t1` (
@@ -551,7 +668,7 @@ under the License.
     );
 ```
 
-9. 创建一个数据文件存储在BOS上的 broker 外部表
+10. 创建一个数据文件存储在BOS上的 broker 外部表
 
 ```
     CREATE EXTERNAL TABLE example_db.table_broker (
@@ -569,7 +686,7 @@ under the License.
     )
 ```
 
-10. 创建一个带有bitmap 索引的表
+11. 创建一个带有bitmap 索引的表
 
 ```
     CREATE TABLE example_db.table_hash
@@ -587,7 +704,7 @@ under the License.
     PROPERTIES ("storage_type"="column");
 ```
 
-11. 创建一个动态分区表(需要在FE配置中开启动态分区功能),该表每天提前创建3天的分区,并删除3天前的分区。例如今天为`2020-01-08`,则会创建分区名为`p20200108`, `p20200109`, `p20200110`, `p20200111`的分区. 分区范围分别为: 
+12. 创建一个动态分区表(需要在FE配置中开启动态分区功能),该表每天提前创建3天的分区,并删除3天前的分区。例如今天为`2020-01-08`,则会创建分区名为`p20200108`, `p20200109`, `p20200110`, `p20200111`的分区. 分区范围分别为: 
 
 ```
 [types: [DATE]; keys: [2020-01-08]; ‥types: [DATE]; keys: [2020-01-09]; )
@@ -624,7 +741,7 @@ under the License.
      );
 ```
 
-12. Create a table with rollup index
+13. Create a table with rollup index
 ```
     CREATE TABLE example_db.rolup_index_table
     (
@@ -643,7 +760,8 @@ under the License.
     )
     PROPERTIES("replication_num" = "3");
     
-13. 创建一个内存表
+14. 创建一个内存表
+
 ```
     CREATE TABLE example_db.table_hash
     (
@@ -660,7 +778,8 @@ under the License.
     PROPERTIES ("in_memory"="true");
 ```
 
-14. 创建一个hive外部表
+15. 创建一个hive外部表
+
 ```
     CREATE TABLE example_db.table_hive
     (
diff --git a/docs/zh-CN/sql-reference/sql-statements/Data Definition/RESTORE.md b/docs/zh-CN/sql-reference/sql-statements/Data Definition/RESTORE.md
index 9871639..ee400e3 100644
--- a/docs/zh-CN/sql-reference/sql-statements/Data Definition/RESTORE.md	
+++ b/docs/zh-CN/sql-reference/sql-statements/Data Definition/RESTORE.md	
@@ -42,7 +42,7 @@ under the License.
         2. ON 子句中标识需要恢复的表和分区。如果不指定分区,则默认恢复该表的所有分区。所指定的表和分区必须已存在于仓库备份中。
         3. 可以通过 AS 语句将仓库中备份的表名恢复为新的表。但新表名不能已存在于数据库中。分区名称不能修改。
         4. 可以将仓库中备份的表恢复替换数据库中已有的同名表,但须保证两张表的表结构完全一致。表结构包括:表名、列、分区、Rollup等等。
-        5. 可以指定恢复表的部分分区,系统会检查分区 Range 是否能够匹配。
+        5. 可以指定恢复表的部分分区,系统会检查分区 Range 或者 List 是否能够匹配。
         6. PROPERTIES 目前支持以下属性:
                 "backup_timestamp" = "2018-05-04-16-45-08":指定了恢复对应备份的哪个时间版本,必填。该信息可以通过 `SHOW SNAPSHOT ON repo;` 语句获得。
                 "replication_num" = "3":指定恢复的表或分区的副本数。默认为3。若恢复已存在的表或分区,则副本数必须和已存在表或分区的副本数相同。同时,必须有足够的 host 容纳多个副本。
diff --git a/docs/zh-CN/sql-reference/sql-statements/Data Manipulation/DELETE.md b/docs/zh-CN/sql-reference/sql-statements/Data Manipulation/DELETE.md
index c1dcbeb..4f7de60 100644
--- a/docs/zh-CN/sql-reference/sql-statements/Data Manipulation/DELETE.md	
+++ b/docs/zh-CN/sql-reference/sql-statements/Data Manipulation/DELETE.md	
@@ -40,7 +40,7 @@ under the License.
         2) 当选定的 key 列不存在于某个 rollup 中时,无法进行 delete。
         3) 条件之间只能是“与”的关系。
            若希望达成“或”的关系,需要将条件分写在两个 DELETE 语句中。
-        4) 如果为RANGE分区表,可以指定分区,如不指定,且会话变量 delete_without_partition 为 true,则会应用到所有分区。如果是单分区表,可以不指定。
+        4) 如果为分区表,可以指定分区,如不指定,且会话变量 delete_without_partition 为 true,则会应用到所有分区。如果是单分区表,可以不指定。
            
     注意:
         该语句可能会降低执行后一段时间内的查询效率。
diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup
index 50151fc..bb2801e 100644
--- a/fe/fe-core/src/main/cup/sql_parser.cup
+++ b/fe/fe-core/src/main/cup/sql_parser.cup
@@ -244,7 +244,7 @@ terminal String KW_ADD, KW_ADMIN, KW_AFTER, KW_AGGREGATE, KW_ALL, KW_ALTER, KW_A
     KW_INNER, KW_INSERT, KW_INT, KW_INTERMEDIATE, KW_INTERSECT, KW_INTERVAL, KW_INTO, KW_IS, KW_ISNULL, KW_ISOLATION,
     KW_JOIN,
     KW_KEY, KW_KEYS, KW_KILL,
-    KW_LABEL, KW_LARGEINT, KW_LAST, KW_LEFT, KW_LESS, KW_LEVEL, KW_LIKE, KW_LIMIT, KW_LINK, KW_LOAD,
+    KW_LABEL, KW_LARGEINT, KW_LAST, KW_LEFT, KW_LESS, KW_LEVEL, KW_LIKE, KW_LIMIT, KW_LINK, KW_LIST, KW_LOAD,
     KW_LOCAL, KW_LOCATION,
     KW_MATERIALIZED, KW_MAX, KW_MAX_VALUE, KW_MERGE, KW_MIN, KW_MINUTE, KW_MINUS, KW_MIGRATE, KW_MIGRATIONS, KW_MODIFY, KW_MONTH,
     KW_NAME, KW_NAMES, KW_NEGATIVE, KW_NO, KW_NOT, KW_NULL, KW_NULLS,
@@ -416,11 +416,15 @@ nonterminal Long opt_field_length;
 nonterminal KeysDesc opt_keys;
 
 nonterminal PartitionKeyDesc partition_key_desc;
+nonterminal PartitionKeyDesc list_partition_key_desc;
 nonterminal PartitionKeyDesc fixed_partition_key_desc;
 nonterminal List<PartitionValue> partition_key_list;
-nonterminal SingleRangePartitionDesc single_range_partition_desc;
-nonterminal List<SingleRangePartitionDesc> opt_single_range_partition_desc_list;
-nonterminal List<SingleRangePartitionDesc> single_range_partition_desc_list;
+nonterminal List<PartitionValue> partition_value_list;
+nonterminal List<PartitionValue> partition_key_item_list;
+nonterminal List<List<PartitionValue>> list_partition_values_list;
+nonterminal SinglePartitionDesc single_partition_desc;
+nonterminal List<SinglePartitionDesc> opt_single_partition_desc_list;
+nonterminal List<SinglePartitionDesc> single_partition_desc_list;
 
 nonterminal List<AccessPrivilege> privilege_list;
 nonterminal List<String> string_list;
@@ -947,7 +951,7 @@ alter_table_clause ::=
     {:
         RESULT = new ModifyTablePropertiesClause(properties);
     :}
-    | KW_ADD opt_tmp:isTempPartition single_range_partition_desc:desc opt_distribution:distribution opt_properties:properties
+    | KW_ADD opt_tmp:isTempPartition single_partition_desc:desc opt_distribution:distribution opt_properties:properties
     {:
         RESULT = new AddPartitionClause(desc, distribution, properties, isTempPartition);
     :}
@@ -1865,10 +1869,16 @@ opt_partition ::=
     :}
     /* Range partition */
     | KW_PARTITION KW_BY KW_RANGE LPAREN ident_list:columns RPAREN
-            LPAREN opt_single_range_partition_desc_list:list RPAREN
+            LPAREN opt_single_partition_desc_list:list RPAREN
     {:
         RESULT = new RangePartitionDesc(columns, list);
     :}
+    /* List partition */
+    | KW_PARTITION KW_BY KW_LIST LPAREN ident_list:columns RPAREN
+            LPAREN opt_single_partition_desc_list:list RPAREN
+    {:
+        RESULT = new ListPartitionDesc(columns, list);
+    :}
     ;
 
 opt_distribution ::=
@@ -1933,39 +1943,45 @@ opt_keys ::=
     :}
     ;
 
-opt_single_range_partition_desc_list ::=
+opt_single_partition_desc_list ::=
     /* Empty */
     {:
         RESULT = null;
     :}
-    | single_range_partition_desc_list:list
+    | single_partition_desc_list:list
     {:
         RESULT = list;
     :}
     ;
 
-single_range_partition_desc_list ::=
-    single_range_partition_desc_list:list COMMA single_range_partition_desc:desc
+single_partition_desc_list ::=
+    single_partition_desc_list:list COMMA single_partition_desc:desc
     {:
         list.add(desc);
         RESULT = list;
     :}
-    | single_range_partition_desc:desc
+    | single_partition_desc:desc
     {:
         RESULT = Lists.newArrayList(desc);
     :}
     ;
 
-single_range_partition_desc ::=
+single_partition_desc ::=
     KW_PARTITION opt_if_not_exists:ifNotExists ident:partName KW_VALUES KW_LESS KW_THAN partition_key_desc:desc
         opt_key_value_map:properties
     {:
-        RESULT = new SingleRangePartitionDesc(ifNotExists, partName, desc, properties);
+        RESULT = new SinglePartitionDesc(ifNotExists, partName, desc, properties);
     :}
     | KW_PARTITION opt_if_not_exists:ifNotExists ident:partName KW_VALUES fixed_partition_key_desc:desc
         opt_key_value_map:properties
     {:
-        RESULT = new SingleRangePartitionDesc(ifNotExists, partName, desc, properties);
+        RESULT = new SinglePartitionDesc(ifNotExists, partName, desc, properties);
+    :}
+    /* list partition */
+    | KW_PARTITION opt_if_not_exists:ifNotExists ident:partName KW_VALUES KW_IN list_partition_key_desc:desc
+        opt_key_value_map:properties
+    {:
+        RESULT = new SinglePartitionDesc(ifNotExists, partName, desc, properties);
     :}
     ;
 
@@ -1976,7 +1992,57 @@ partition_key_desc ::=
     :}
     | LPAREN partition_key_list:keys RPAREN
     {:
-        RESULT = new PartitionKeyDesc(keys);
+        RESULT = PartitionKeyDesc.createLessThan(keys);
+    :}
+    ;
+
+/* list partition PartitionKeyDesc */
+list_partition_key_desc ::=
+    LPAREN list_partition_values_list:keys RPAREN
+    {:
+        RESULT = PartitionKeyDesc.createIn(keys);
+    :}
+    ;
+
+/* List<List<PartitionValue>> inValues */
+list_partition_values_list ::=
+    partition_value_list:item
+    {:
+        ArrayList<List<PartitionValue>> l = new ArrayList();
+        l.add(item);
+        RESULT = l;
+    :}
+    | list_partition_values_list:l COMMA partition_value_list:item
+    {:
+        l.add(item);
+        RESULT = l;
+    :}
+    ;
+
+/* List<PartitionValue> */
+partition_value_list ::=
+    /* single partition key */
+    STRING_LITERAL:item
+    {:
+        RESULT = Lists.newArrayList(new PartitionValue(item));
+    :}
+    /* multi partition keys : (1, "beijing") */
+    | LPAREN partition_key_item_list:l RPAREN
+    {:
+        RESULT = l;
+    :}
+    ;
+
+/* List<PartitionValue> */
+partition_key_item_list ::=
+    STRING_LITERAL:item
+    {:
+        RESULT = Lists.newArrayList(new PartitionValue(item));
+    :}
+    | partition_key_item_list:l COMMA STRING_LITERAL:item
+    {:
+        l.add(new PartitionValue(item));
+        RESULT = l;
     :}
     ;
 
@@ -2010,7 +2076,7 @@ fixed_partition_key_desc ::=
     /* format: [(lower), (upper))*/
     LBRACKET LPAREN partition_key_list:lower RPAREN COMMA LPAREN partition_key_list:upper RPAREN RPAREN
     {:
-        RESULT = new PartitionKeyDesc(lower, upper);
+        RESULT = PartitionKeyDesc.createFixed(lower, upper);
     :}
     ;
 
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 ee9ae4b..3a00b62 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
@@ -49,7 +49,6 @@ import org.apache.doris.catalog.OlapTable.OlapTableState;
 import org.apache.doris.catalog.Partition;
 import org.apache.doris.catalog.PartitionInfo;
 import org.apache.doris.catalog.PartitionType;
-import org.apache.doris.catalog.RangePartitionInfo;
 import org.apache.doris.catalog.Replica;
 import org.apache.doris.catalog.Replica.ReplicaState;
 import org.apache.doris.catalog.Table;
@@ -1215,9 +1214,8 @@ public class SchemaChangeHandler extends AlterHandler {
 
             // 3. check partition key
             PartitionInfo partitionInfo = olapTable.getPartitionInfo();
-            if (partitionInfo.getType() == PartitionType.RANGE) {
-                RangePartitionInfo rangePartitionInfo = (RangePartitionInfo) partitionInfo;
-                List<Column> partitionColumns = rangePartitionInfo.getPartitionColumns();
+            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) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AddPartitionClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AddPartitionClause.java
index 23f8a23..ace4b4c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AddPartitionClause.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AddPartitionClause.java
@@ -25,13 +25,13 @@ import java.util.Map;
 // clause which is used to add partition
 public class AddPartitionClause extends AlterTableClause {
 
-    private SingleRangePartitionDesc partitionDesc;
+    private SinglePartitionDesc partitionDesc;
     private DistributionDesc distributionDesc;
     private Map<String, String> properties;
     // true if this is to add a temporary partition
     private boolean isTempPartition;
 
-    public SingleRangePartitionDesc getSingeRangePartitionDesc() {
+    public SinglePartitionDesc getSingeRangePartitionDesc() {
         return partitionDesc;
     }
 
@@ -43,7 +43,7 @@ public class AddPartitionClause extends AlterTableClause {
         return isTempPartition;
     }
 
-    public AddPartitionClause(SingleRangePartitionDesc partitionDesc,
+    public AddPartitionClause(SinglePartitionDesc partitionDesc,
                               DistributionDesc distributionDesc,
                               Map<String, String> properties,
                               boolean isTempPartition) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/BoolLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/BoolLiteral.java
index 55d7275..5edefdd 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/BoolLiteral.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/BoolLiteral.java
@@ -69,6 +69,11 @@ public class BoolLiteral extends LiteralExpr {
     }
 
     @Override
+    public Object getRealValue() {
+        return getValue();
+    }
+
+    @Override
     public boolean isMinValue() {
         return false;
     }
@@ -78,6 +83,9 @@ public class BoolLiteral extends LiteralExpr {
         if (expr instanceof NullLiteral) {
             return 1;
         }
+        if (expr == MaxLiteral.MAX_VALUE) {
+            return -1;
+        }
         return Long.signum(getLongValue() - expr.getLongValue());
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java
index 7641ac9..d20c75a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableStmt.java
@@ -22,7 +22,6 @@ import org.apache.doris.catalog.Catalog;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.Index;
 import org.apache.doris.catalog.KeysType;
-import org.apache.doris.catalog.PartitionType;
 import org.apache.doris.catalog.PrimitiveType;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.Config;
@@ -382,12 +381,12 @@ public class CreateTableStmt extends DdlStmt {
         if (engineName.equals("olap")) {
             // analyze partition
             if (partitionDesc != null) {
-                if (partitionDesc.getType() != PartitionType.RANGE) {
-                    throw new AnalysisException("Currently only support range partition with engine type olap");
+                if (partitionDesc instanceof ListPartitionDesc || partitionDesc instanceof RangePartitionDesc) {
+                    partitionDesc.analyze(columnDefs, properties);
+                } else {
+                    throw new AnalysisException("Currently only support range and list partition with engine type olap");
                 }
 
-                RangePartitionDesc rangePartitionDesc = (RangePartitionDesc) partitionDesc;
-                rangePartitionDesc.analyze(columnDefs, properties);
             }
 
             // analyze distribution
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ListPartitionDesc.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ListPartitionDesc.java
new file mode 100644
index 0000000..31aec5f
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ListPartitionDesc.java
@@ -0,0 +1,110 @@
+// 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.analysis;
+
+import org.apache.doris.analysis.PartitionKeyDesc.PartitionKeyValueType;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.ListPartitionInfo;
+import org.apache.doris.catalog.PartitionInfo;
+import org.apache.doris.catalog.PartitionType;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.DdlException;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+// to describe the key list partition's information in create table stmt
+public class ListPartitionDesc extends PartitionDesc {
+
+    public ListPartitionDesc(List<String> partitionColNames,
+                             List<SinglePartitionDesc> singlePartitionDescs) {
+        super(partitionColNames, singlePartitionDescs);
+        type = PartitionType.LIST;
+    }
+
+    @Override
+    public void checkPartitionKeyValueType(PartitionKeyDesc partitionKeyDesc) throws AnalysisException {
+        if (partitionKeyDesc.getPartitionType() != PartitionKeyValueType.IN) {
+            throw new AnalysisException("You can only use in values to create list partitions");
+        }
+    }
+
+    @Override
+    public String toSql() {
+        StringBuilder sb = new StringBuilder();
+        sb.append("PARTITION BY LIST(");
+        int idx = 0;
+        for (String column : partitionColNames) {
+            if (idx != 0) {
+                sb.append(", ");
+            }
+            sb.append("`").append(column).append("`");
+            idx++;
+        }
+        sb.append(")\n(\n");
+
+        for (int i = 0; i < singlePartitionDescs.size(); i++) {
+            if (i != 0) {
+                sb.append(",\n");
+            }
+            sb.append(singlePartitionDescs.get(i).toSql());
+        }
+        sb.append("\n)");
+        return sb.toString();
+    }
+
+    @Override
+    public String toString() {
+        return toSql();
+    }
+
+    @Override
+    public PartitionInfo toPartitionInfo(List<Column> schema, Map<String, Long> partitionNameToId, boolean isTemp)
+            throws DdlException {
+        List<Column> partitionColumns = new ArrayList<>();
+
+        // check and get partition column
+        for (String colName : partitionColNames) {
+            boolean find = false;
+            for (Column column : schema) {
+                if (column.getName().equalsIgnoreCase(colName)) {
+                    try {
+                        ListPartitionInfo.checkPartitionColumn(column);
+                    } catch (AnalysisException e) {
+                        throw new DdlException(e.getMessage());
+                    }
+
+                    partitionColumns.add(column);
+                    find = true;
+                    break;
+                }
+            }
+            if (!find) {
+                throw new DdlException("Partition column[" + colName + "] does not found");
+            }
+        }
+
+        ListPartitionInfo listPartitionInfo = new ListPartitionInfo(partitionColumns);
+        for (SinglePartitionDesc desc : singlePartitionDescs) {
+            long partitionId = partitionNameToId.get(desc.getPartitionName());
+            listPartitionInfo.handleNewSinglePartitionDesc(desc, partitionId, isTemp);
+        }
+        return listPartitionInfo;
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/LiteralExpr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/LiteralExpr.java
index 2823480..5bed750 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/LiteralExpr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/LiteralExpr.java
@@ -117,7 +117,7 @@ public abstract class LiteralExpr extends Expr {
      * return real value
      */
     public Object getRealValue() {
-        // implemented: TINYINT/SMALLINT/INT/BIGINT/LARGEINT/DATE/DATETIME
+        // implemented: TINYINT/SMALLINT/INT/BIGINT/LARGEINT/DATE/DATETIME/CHAR/VARCHAR/BOOLEAN
         Preconditions.checkState(false, "should implement this in derived class. " + this.type.toSql());
         return null;
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionDesc.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionDesc.java
index f2b46af..7566cdd 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionDesc.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionDesc.java
@@ -17,23 +17,104 @@
 
 package org.apache.doris.analysis;
 
+import org.apache.doris.catalog.AggregateType;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.PartitionInfo;
 import org.apache.doris.catalog.PartitionType;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.DdlException;
+import org.apache.doris.qe.ConnectContext;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
 
 import org.apache.commons.lang.NotImplementedException;
 
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 public class PartitionDesc {
+    protected List<String> partitionColNames;
+    protected List<SinglePartitionDesc> singlePartitionDescs;
 
     protected PartitionType type;
 
+    public PartitionDesc(List<String> partitionColNames,
+                         List<SinglePartitionDesc> singlePartitionDescs) {
+        this.partitionColNames = partitionColNames;
+        this.singlePartitionDescs = singlePartitionDescs;
+        if (this.singlePartitionDescs == null) {
+            this.singlePartitionDescs = Lists.newArrayList();
+        }
+    }
+
+    public List<SinglePartitionDesc> getSinglePartitionDescs() {
+        return this.singlePartitionDescs;
+    }
+
+    public List<String> getPartitionColNames() {
+        return partitionColNames;
+    }
+
     public void analyze(List<ColumnDef> columnDefs, Map<String, String> otherProperties) throws AnalysisException {
-        throw new NotImplementedException();
+        if (partitionColNames == null || partitionColNames.isEmpty()) {
+            throw new AnalysisException("No partition columns.");
+        }
+
+        Set<String> partColNames = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER);
+        for (String partitionCol : partitionColNames) {
+            if (!partColNames.add(partitionCol)) {
+                throw new AnalysisException("Duplicated partition column " + partitionCol);
+            }
+
+            boolean found = false;
+            for (ColumnDef columnDef : columnDefs) {
+                if (columnDef.getName().equals(partitionCol)) {
+                    if (!columnDef.isKey() && columnDef.getAggregateType() != AggregateType.NONE) {
+                        throw new AnalysisException("The partition column could not be aggregated column");
+                    }
+                    if (columnDef.getType().isFloatingPointType()) {
+                        throw new AnalysisException("Floating point type column can not be partition column");
+                    }
+                    if (!ConnectContext.get().getSessionVariable().isAllowPartitionColumnNullable()
+                            && columnDef.isAllowNull()) {
+                        throw new AnalysisException("The partition column must be NOT NULL");
+                    }
+                    if (this instanceof ListPartitionDesc && columnDef.isAllowNull()) {
+                        throw new AnalysisException("The list partition column must be NOT NULL");
+                    }
+                    found = true;
+                    break;
+                }
+            }
+
+            if (!found) {
+                throw new AnalysisException("Partition column[" + partitionCol + "] does not exist in column list.");
+            }
+        }
+
+        Set<String> nameSet = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER);
+        for (SinglePartitionDesc desc : singlePartitionDescs) {
+            if (!nameSet.add(desc.getPartitionName())) {
+                throw new AnalysisException("Duplicated partition name: " + desc.getPartitionName());
+            }
+            // in create table stmt, we use given properties
+            // copy one. because ProperAnalyzer will remove entry after analyze
+            Map<String, String> givenProperties = null;
+            if (otherProperties != null) {
+                givenProperties = Maps.newHashMap(otherProperties);
+            }
+            // check partitionType
+            checkPartitionKeyValueType(desc.getPartitionKeyDesc());
+            // analyze singlePartitionDesc
+            desc.analyze(columnDefs.size(), givenProperties);
+        }
+    }
+
+    public void checkPartitionKeyValueType(PartitionKeyDesc partitionKeyDesc) throws AnalysisException {
+
     }
 
     public PartitionType getType() {
@@ -44,7 +125,7 @@ public class PartitionDesc {
         throw new NotImplementedException();
     }
 
-    public PartitionInfo toPartitionInfo(List<Column> columns, Map<String, Long> partitionNameToId, boolean isTemp)
+    public PartitionInfo toPartitionInfo(List<Column> schema, Map<String, Long> partitionNameToId, boolean isTemp)
             throws DdlException {
         throw new NotImplementedException();
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionKeyDesc.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionKeyDesc.java
index 7fe7285..726c333 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionKeyDesc.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/PartitionKeyDesc.java
@@ -29,35 +29,45 @@ import java.util.List;
 public class PartitionKeyDesc {
     public static final PartitionKeyDesc MAX_VALUE = new PartitionKeyDesc();
 
-    public enum PartitionRangeType {
+    public enum PartitionKeyValueType {
         INVALID,
         LESS_THAN,
-        FIXED
+        FIXED,
+        IN
     }
 
     private List<PartitionValue> lowerValues;
     private List<PartitionValue> upperValues;
-    private PartitionRangeType partitionType;
+    private List<List<PartitionValue>> inValues;
+    private PartitionKeyValueType partitionKeyValueType;
 
     public static PartitionKeyDesc createMaxKeyDesc() {
         return MAX_VALUE;
     }
 
     private PartitionKeyDesc() {
-        partitionType = PartitionRangeType.LESS_THAN; // LESS_THAN is default type.
+        partitionKeyValueType = PartitionKeyValueType.LESS_THAN; // LESS_THAN is default type.
     }
 
-    // values less than
-    public PartitionKeyDesc(List<PartitionValue> upperValues) {
-        this.upperValues = upperValues;
-        partitionType = PartitionRangeType.LESS_THAN;
+    public static PartitionKeyDesc createLessThan(List<PartitionValue> upperValues) {
+        PartitionKeyDesc desc = new PartitionKeyDesc();
+        desc.upperValues = upperValues;
+        desc.partitionKeyValueType = PartitionKeyValueType.LESS_THAN;
+        return desc;
     }
 
-    // fixed range
-    public PartitionKeyDesc(List<PartitionValue> lowerValues, List<PartitionValue> upperValues) {
-        this.lowerValues = lowerValues;
-        this.upperValues = upperValues;
-        partitionType = PartitionRangeType.FIXED;
+    public static PartitionKeyDesc createIn(List<List<PartitionValue>> inValues) {
+        PartitionKeyDesc desc = new PartitionKeyDesc();
+        desc.inValues = inValues;
+        desc.partitionKeyValueType = PartitionKeyValueType.IN;
+        return desc;
+    }
+
+    public static PartitionKeyDesc createFixed(List<PartitionValue> lowerValues, List<PartitionValue> upperValues) {
+        PartitionKeyDesc desc = new PartitionKeyDesc();
+        desc.lowerValues = lowerValues;
+        desc.upperValues = upperValues;
+        return desc;
     }
 
     public List<PartitionValue> getLowerValues() {
@@ -68,6 +78,10 @@ public class PartitionKeyDesc {
         return upperValues;
     }
 
+    public List<List<PartitionValue>> getInValues() {
+        return inValues;
+    }
+
     public boolean isMax() {
         return this == MAX_VALUE;
     }
@@ -80,13 +94,13 @@ public class PartitionKeyDesc {
         return upperValues != null;
     }
 
-    public PartitionRangeType getPartitionType() {
-        return partitionType;
+    public PartitionKeyValueType getPartitionType() {
+        return partitionKeyValueType;
     }
 
     public void analyze(int partColNum) throws AnalysisException {
         if (!isMax()) {
-            if (upperValues.isEmpty() || upperValues.size() > partColNum) {
+            if ((upperValues != null && (upperValues.isEmpty() || upperValues.size() > partColNum))) {
                 throw new AnalysisException("Partition values number is more than partition column number: " + toSql());
             }
         }
@@ -119,13 +133,27 @@ public class PartitionKeyDesc {
             return "MAXVALUE";
         }
 
-        if (partitionType == PartitionRangeType.LESS_THAN) {
+        if (partitionKeyValueType == PartitionKeyValueType.LESS_THAN) {
             return getPartitionValuesStr(upperValues);
-        } else if (partitionType == PartitionRangeType.FIXED) {
+        } else if (partitionKeyValueType == PartitionKeyValueType.FIXED) {
             StringBuilder sb = new StringBuilder("[");
             sb.append(getPartitionValuesStr(lowerValues)).append(", ").append(getPartitionValuesStr(upperValues));
             sb.append(")");
             return sb.toString();
+        } else if (partitionKeyValueType == PartitionKeyValueType.IN) {
+            StringBuilder sb = new StringBuilder("(");
+            for (int i = 0; i < inValues.size(); i++) {
+                String valueStr = getPartitionValuesStr(inValues.get(i));
+                if (inValues.get(i).size() == 1) {
+                    valueStr = valueStr.substring(1, valueStr.length() - 1);
+                }
+                sb.append(valueStr);
+                if (i < inValues.size() -1) {
+                    sb.append(",");
+                }
+            }
+            sb.append(")");
+            return sb.toString();
         } else {
             return "INVALID";
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/RangePartitionDesc.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/RangePartitionDesc.java
index f0c345c..eb08ad2 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/RangePartitionDesc.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/RangePartitionDesc.java
@@ -17,101 +17,31 @@
 
 package org.apache.doris.analysis;
 
-import org.apache.doris.analysis.PartitionKeyDesc.PartitionRangeType;
-import org.apache.doris.catalog.AggregateType;
+import org.apache.doris.analysis.PartitionKeyDesc.PartitionKeyValueType;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.PartitionInfo;
-import org.apache.doris.catalog.PartitionType;
 import org.apache.doris.catalog.RangePartitionInfo;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.DdlException;
 
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import org.apache.doris.qe.ConnectContext;
-
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 // to describe the key range partition's information in create table stmt
 public class RangePartitionDesc extends PartitionDesc {
-    private List<String> partitionColNames;
-    private List<SingleRangePartitionDesc> singleRangePartitionDescs;
 
     public RangePartitionDesc(List<String> partitionColNames,
-                              List<SingleRangePartitionDesc> singlePartitionDescs) {
-        type = PartitionType.RANGE;
-        this.partitionColNames = partitionColNames;
-        this.singleRangePartitionDescs = singlePartitionDescs;
-        if (singleRangePartitionDescs == null) {
-            singleRangePartitionDescs = Lists.newArrayList();
-        }
-    }
-
-    public List<SingleRangePartitionDesc> getSingleRangePartitionDescs() {
-        return this.singleRangePartitionDescs;
-    }
-
-    public List<String> getPartitionColNames() {
-        return partitionColNames;
+                              List<SinglePartitionDesc> singlePartitionDescs) {
+        super(partitionColNames, singlePartitionDescs);
+        type = org.apache.doris.catalog.PartitionType.RANGE;
     }
 
     @Override
-    public void analyze(List<ColumnDef> columnDefs, Map<String, String> otherProperties) throws AnalysisException {
-        if (partitionColNames == null || partitionColNames.isEmpty()) {
-            throw new AnalysisException("No partition columns.");
-        }
-
-        Set<String> partColNames = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER);
-        for (String partitionCol : partitionColNames) {
-            if (!partColNames.add(partitionCol)) {
-                throw new AnalysisException("Duplicated partition column " + partitionCol);
-            }
-
-            boolean found = false;
-            for (ColumnDef columnDef : columnDefs) {
-                if (columnDef.getName().equals(partitionCol)) {
-                    if (!columnDef.isKey() && columnDef.getAggregateType() != AggregateType.NONE) {
-                        throw new AnalysisException("The partition column could not be aggregated column");
-                    }
-                    if (columnDef.getType().isFloatingPointType()) {
-                        throw new AnalysisException("Floating point type column can not be partition column");
-                    }
-                    if (!ConnectContext.get().getSessionVariable().isAllowPartitionColumnNullable()
-                            && columnDef.isAllowNull()) {
-                        throw new AnalysisException("The partition column must be NOT NULL");
-                    }
-                    found = true;
-                    break;
-                }
-            }
-
-            if (!found) {
-                throw new AnalysisException("Partition column[" + partitionCol + "] does not exist in column list.");
-            }
-        }
-
-        Set<String> nameSet = Sets.newTreeSet(String.CASE_INSENSITIVE_ORDER);
-        PartitionRangeType partitionType = PartitionRangeType.INVALID;
-        for (SingleRangePartitionDesc desc : singleRangePartitionDescs) {
-            if (!nameSet.add(desc.getPartitionName())) {
-                throw new AnalysisException("Duplicated partition name: " + desc.getPartitionName());
-            }
-            // in create table stmt, we use given properties
-            // copy one. because ProperAnalyzer will remove entry after analyze
-            Map<String, String> givenProperties = null;
-            if (otherProperties != null) {
-                givenProperties = Maps.newHashMap(otherProperties);
-            }
-            // check partitionType
-            if (partitionType == PartitionRangeType.INVALID) {
-                partitionType = desc.getPartitionKeyDesc().getPartitionType();
-            } else if (partitionType != desc.getPartitionKeyDesc().getPartitionType()) {
-                throw new AnalysisException("You can only use one of these methods to create partitions");
-            }
-            desc.analyze(columnDefs.size(), givenProperties);
+    public void checkPartitionKeyValueType(PartitionKeyDesc partitionKeyDesc) throws AnalysisException {
+        if (partitionKeyDesc.getPartitionType() != PartitionKeyValueType.FIXED &&
+                partitionKeyDesc.getPartitionType() != PartitionKeyValueType.LESS_THAN) {
+            throw new AnalysisException("You can only use fixed or less than values to create range partitions");
         }
     }
 
@@ -129,11 +59,11 @@ public class RangePartitionDesc extends PartitionDesc {
         }
         sb.append(")\n(\n");
         
-        for (int i = 0; i < singleRangePartitionDescs.size(); i++) {
+        for (int i = 0; i < singlePartitionDescs.size(); i++) {
             if (i != 0) {
                 sb.append(",\n");
             }
-            sb.append(singleRangePartitionDescs.get(i).toSql());
+            sb.append(singlePartitionDescs.get(i).toSql());
         }
         sb.append("\n)");
         return sb.toString();
@@ -147,28 +77,15 @@ public class RangePartitionDesc extends PartitionDesc {
     @Override
     public PartitionInfo toPartitionInfo(List<Column> schema, Map<String, Long> partitionNameToId, boolean isTemp)
             throws DdlException {
-        List<Column> partitionColumns = Lists.newArrayList();
+        List<Column> partitionColumns = new ArrayList<>();
 
         // check and get partition column
         for (String colName : partitionColNames) {
             boolean find = false;
             for (Column column : schema) {
                 if (column.getName().equalsIgnoreCase(colName)) {
-                    if (!column.isKey() && column.getAggregationType() != AggregateType.NONE) {
-                        throw new DdlException("The partition column could not be aggregated column");
-                    }
-
-                    if (column.getType().isFloatingPointType()) {
-                        throw new DdlException("Floating point type column can not be partition column");
-                    }
-
-                    if (!ConnectContext.get().getSessionVariable().isAllowPartitionColumnNullable()
-                            && column.isAllowNull()) {
-                        throw new DdlException("The partition column must be NOT NULL");
-                    }
-
                     try {
-                        RangePartitionInfo.checkRangeColumnType(column);
+                        RangePartitionInfo.checkPartitionColumn(column);
                     } catch (AnalysisException e) {
                         throw new DdlException(e.getMessage());
                     }
@@ -176,6 +93,7 @@ public class RangePartitionDesc extends PartitionDesc {
                     partitionColumns.add(column);
                     find = true;
                     break;
+
                 }
             }
             if (!find) {
@@ -186,17 +104,17 @@ public class RangePartitionDesc extends PartitionDesc {
         /*
          * validate key range
          * eg.
-         * VALUE LESS THEN (10, 100, 1000)
-         * VALUE LESS THEN (50, 500)
-         * VALUE LESS THEN (80)
-         * 
+         * VALUE LESS THAN (10, 100, 1000)
+         * VALUE LESS THAN (50, 500)
+         * VALUE LESS THAN (80)
+         *
          * key range is:
          * ( {MIN, MIN, MIN},     {10,  100, 1000} )
          * [ {10,  100, 1000},    {50,  500, MIN } )
          * [ {50,  500, MIN },    {80,  MIN, MIN } )
          */
         RangePartitionInfo rangePartitionInfo = new RangePartitionInfo(partitionColumns);
-        for (SingleRangePartitionDesc desc : singleRangePartitionDescs) {
+        for (SinglePartitionDesc desc : singlePartitionDescs) {
             long partitionId = partitionNameToId.get(desc.getPartitionName());
             rangePartitionInfo.handleNewSinglePartitionDesc(desc, partitionId, isTemp);
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/SingleRangePartitionDesc.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/SinglePartitionDesc.java
similarity index 92%
rename from fe/fe-core/src/main/java/org/apache/doris/analysis/SingleRangePartitionDesc.java
rename to fe/fe-core/src/main/java/org/apache/doris/analysis/SinglePartitionDesc.java
index 027df42..102d4b8 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SingleRangePartitionDesc.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/SinglePartitionDesc.java
@@ -17,7 +17,7 @@
 
 package org.apache.doris.analysis;
 
-import org.apache.doris.analysis.PartitionKeyDesc.PartitionRangeType;
+import org.apache.doris.analysis.PartitionKeyDesc.PartitionKeyValueType;
 import org.apache.doris.catalog.DataProperty;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.FeConstants;
@@ -33,7 +33,7 @@ import org.apache.doris.thrift.TTabletType;
 
 import java.util.Map;
 
-public class SingleRangePartitionDesc {
+public class SinglePartitionDesc {
     private boolean isAnalyzed;
 
     private boolean ifNotExists;
@@ -48,10 +48,10 @@ public class SingleRangePartitionDesc {
     private TTabletType tabletType = TTabletType.TABLET_TYPE_DISK;
     private Pair<Long, Long> versionInfo;
 
-    public SingleRangePartitionDesc(boolean ifNotExists, String partName, PartitionKeyDesc partitionKeyDesc,
-                                    Map<String, String> properties) {
+    public SinglePartitionDesc(boolean ifNotExists, String partName, PartitionKeyDesc partitionKeyDesc,
+                               Map<String, String> properties) {
         this.ifNotExists = ifNotExists;
-        
+
         this.isAnalyzed = false;
 
         this.partName = partName;
@@ -146,8 +146,8 @@ public class SingleRangePartitionDesc {
     public String toSql() {
         StringBuilder sb = new StringBuilder();
         sb.append("PARTITION ").append(partName);
-        if (partitionKeyDesc.getPartitionType() == PartitionRangeType.LESS_THAN) {
-            sb.append(" VALUES LESS THEN ");
+        if (partitionKeyDesc.getPartitionType() == PartitionKeyValueType.LESS_THAN) {
+            sb.append(" VALUES LESS THAN ");
         } else {
             sb.append(" VALUES ");
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/StringLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/StringLiteral.java
index c7ef065..9f0b60c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/StringLiteral.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/StringLiteral.java
@@ -87,7 +87,9 @@ public class StringLiteral extends LiteralExpr {
         if (expr instanceof NullLiteral) {
             return 1;
         }
-
+        if (expr == MaxLiteral.MAX_VALUE) {
+            return -1;
+        }
         // compare string with utf-8 byte array, same with DM,BE,StorageEngine
         byte[] thisBytes = null;
         byte[] otherBytes = null;
@@ -166,6 +168,11 @@ public class StringLiteral extends LiteralExpr {
         return Double.valueOf(value);
     }
 
+    @Override
+    public String getRealValue() {
+        return getStringValue();
+    }
+
     /**
      * Convert a string literal to a date literal
      *
diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java
index 8c246fd..7f96fe7 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/backup/RestoreJob.java
@@ -37,9 +37,8 @@ import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.catalog.OlapTable.OlapTableState;
 import org.apache.doris.catalog.Partition;
 import org.apache.doris.catalog.PartitionInfo;
-import org.apache.doris.catalog.PartitionKey;
+import org.apache.doris.catalog.PartitionItem;
 import org.apache.doris.catalog.PartitionType;
-import org.apache.doris.catalog.RangePartitionInfo;
 import org.apache.doris.catalog.Replica;
 import org.apache.doris.catalog.Replica.ReplicaState;
 import org.apache.doris.catalog.Resource;
@@ -82,7 +81,6 @@ import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Multimap;
-import com.google.common.collect.Range;
 import com.google.common.collect.Table.Cell;
 
 import java.io.DataInput;
@@ -554,14 +552,11 @@ public class RestoreJob extends AbstractJob {
                             if (localPartition != null) {
                                 // Partition already exist.
                                 PartitionInfo localPartInfo = localOlapTbl.getPartitionInfo();
-                                if (localPartInfo.getType() == PartitionType.RANGE) {
-                                    // If this is a range partition, check range
-                                    RangePartitionInfo localRangePartInfo = (RangePartitionInfo) localPartInfo;
-                                    RangePartitionInfo remoteRangePartInfo
-                                            = (RangePartitionInfo) remoteOlapTbl.getPartitionInfo();
-                                    Range<PartitionKey> localRange = localRangePartInfo.getRange(localPartition.getId());
-                                    Range<PartitionKey> remoteRange = remoteRangePartInfo.getRange(backupPartInfo.id);
-                                    if (localRange.equals(remoteRange)) {
+                                if (localPartInfo.getType() == PartitionType.RANGE
+                                        || localPartInfo.getType() == PartitionType.LIST) {
+                                    PartitionItem localItem = localPartInfo.getItem(localPartition.getId());
+                                    PartitionItem remoteItem = remoteOlapTbl.getPartitionInfo().getItem(backupPartInfo.id);
+                                    if (localItem.equals(remoteItem)) {
                                         // Same partition, same range
                                         if (genFileMappingWhenBackupReplicasEqual(localPartInfo, localPartition, localTbl, backupPartInfo, partitionName, tblInfo)) {
                                             return;
@@ -570,7 +565,7 @@ public class RestoreJob extends AbstractJob {
                                         // Same partition name, different range
                                         status = new Status(ErrCode.COMMON_ERROR, "Partition " + partitionName
                                                 + " in table " + localTbl.getName()
-                                                + " has different range with partition in repository");
+                                                + " has different partition item with partition in repository");
                                         return;
                                     }
                                 } else {
@@ -582,16 +577,13 @@ public class RestoreJob extends AbstractJob {
                             } else {
                                 // partitions does not exist
                                 PartitionInfo localPartitionInfo = localOlapTbl.getPartitionInfo();
-                                if (localPartitionInfo.getType() == PartitionType.RANGE) {
-                                    // Check if the partition range can be added to the table
-                                    RangePartitionInfo localRangePartitionInfo = (RangePartitionInfo) localPartitionInfo;
-                                    RangePartitionInfo remoteRangePartitionInfo
-                                            = (RangePartitionInfo) remoteOlapTbl.getPartitionInfo();
-                                    Range<PartitionKey> remoteRange = remoteRangePartitionInfo.getRange(backupPartInfo.id);
-                                    if (localRangePartitionInfo.getAnyIntersectRange(remoteRange, false) != null) {
+                                if (localPartitionInfo.getType() == PartitionType.RANGE
+                                        || localPartitionInfo.getType() == PartitionType.LIST) {
+                                    PartitionItem remoteItem = remoteOlapTbl.getPartitionInfo().getItem(backupPartInfo.id);
+                                    if (localPartitionInfo.getAnyIntersectItem(remoteItem, false) != null) {
                                         status = new Status(ErrCode.COMMON_ERROR, "Partition " + partitionName
                                                 + " in table " + localTbl.getName()
-                                                + " has conflict range with existing ranges");
+                                                + " has conflict partition item with existing items");
                                         return;
                                     } else {
                                         // this partition can be added to this table, set ids
@@ -754,7 +746,7 @@ public class RestoreJob extends AbstractJob {
         } else {
             ok = true;
         }
-
+            
         if (ok) {
             LOG.debug("finished to create all restored replcias. {}", this);
             // add restored partitions.
@@ -766,16 +758,20 @@ public class RestoreJob extends AbstractJob {
                 try {
                     Partition restoredPart = entry.second;
                     OlapTable remoteTbl = (OlapTable) backupMeta.getTable(entry.first);
-                    RangePartitionInfo localPartitionInfo = (RangePartitionInfo) localTbl.getPartitionInfo();
-                    RangePartitionInfo remotePartitionInfo = (RangePartitionInfo) remoteTbl.getPartitionInfo();
-                    BackupPartitionInfo backupPartitionInfo
-                            = jobInfo.getOlapTableInfo(entry.first).getPartInfo(restoredPart.getName());
-                    long remotePartId = backupPartitionInfo.id;
-                    Range<PartitionKey> remoteRange = remotePartitionInfo.getRange(remotePartId);
-                    DataProperty remoteDataProperty = remotePartitionInfo.getDataProperty(remotePartId);
-                    localPartitionInfo.addPartition(restoredPart.getId(), false, remoteRange,
-                            remoteDataProperty, (short) restoreReplicationNum,
-                            remotePartitionInfo.getIsInMemory(remotePartId));
+                    if (localTbl.getPartitionInfo().getType() == PartitionType.RANGE
+                            || localTbl.getPartitionInfo().getType() == PartitionType.LIST) {
+
+                        PartitionInfo remotePartitionInfo = remoteTbl.getPartitionInfo();
+                        PartitionInfo localPartitionInfo = localTbl.getPartitionInfo();
+                        BackupPartitionInfo backupPartitionInfo
+                                = jobInfo.getOlapTableInfo(entry.first).getPartInfo(restoredPart.getName());
+                        long remotePartId = backupPartitionInfo.id;
+                        PartitionItem remoteItem = remoteTbl.getPartitionInfo().getItem(remotePartId);
+                        DataProperty remoteDataProperty = remotePartitionInfo.getDataProperty(remotePartId);
+                        localPartitionInfo.addPartition(restoredPart.getId(), false, remoteItem,
+                                remoteDataProperty, (short) restoreReplicationNum,
+                                remotePartitionInfo.getIsInMemory(remotePartId));
+                    }
                     localTbl.addPartition(restoredPart);
                 } finally {
                     localTbl.writeUnlock();
@@ -958,7 +954,8 @@ public class RestoreJob extends AbstractJob {
         Partition remotePart = remoteTbl.getPartition(partName);
         Preconditions.checkNotNull(remotePart);
         PartitionInfo localPartitionInfo = localTbl.getPartitionInfo();
-        Preconditions.checkState(localPartitionInfo.getType() == PartitionType.RANGE);
+        Preconditions.checkState(localPartitionInfo.getType() == PartitionType.RANGE
+                                    || localPartitionInfo.getType() == PartitionType.LIST);
 
         // generate new partition id
         long newPartId = catalog.getNextId();
@@ -1074,13 +1071,12 @@ public class RestoreJob extends AbstractJob {
             OlapTable localTbl = (OlapTable) db.getTable(entry.first);
             Partition restorePart = entry.second;
             OlapTable remoteTbl = (OlapTable) backupMeta.getTable(entry.first);
-            RangePartitionInfo localPartitionInfo = (RangePartitionInfo) localTbl.getPartitionInfo();
-            RangePartitionInfo remotePartitionInfo = (RangePartitionInfo) remoteTbl.getPartitionInfo();
+            PartitionInfo localPartitionInfo = localTbl.getPartitionInfo();
+            PartitionInfo remotePartitionInfo = remoteTbl.getPartitionInfo();
             BackupPartitionInfo backupPartitionInfo = jobInfo.getOlapTableInfo(entry.first).getPartInfo(restorePart.getName());
             long remotePartId = backupPartitionInfo.id;
-            Range<PartitionKey> remoteRange = remotePartitionInfo.getRange(remotePartId);
             DataProperty remoteDataProperty = remotePartitionInfo.getDataProperty(remotePartId);
-            localPartitionInfo.addPartition(restorePart.getId(), false, remoteRange,
+            localPartitionInfo.addPartition(restorePart.getId(), false, remotePartitionInfo.getItem(remotePartId),
                     remoteDataProperty, (short) restoreReplicationNum,
                     remotePartitionInfo.getIsInMemory(remotePartId));
             localTbl.addPartition(restorePart);
@@ -1098,6 +1094,7 @@ public class RestoreJob extends AbstractJob {
                 }
             }
         }
+        
 
         // restored tables
         for (Table restoreTbl : restoredTbls) {
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 16f24eb..3ae8116 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
@@ -67,7 +67,6 @@ import org.apache.doris.analysis.LinkDbStmt;
 import org.apache.doris.analysis.MigrateDbStmt;
 import org.apache.doris.analysis.PartitionDesc;
 import org.apache.doris.analysis.PartitionRenameClause;
-import org.apache.doris.analysis.RangePartitionDesc;
 import org.apache.doris.analysis.RecoverDbStmt;
 import org.apache.doris.analysis.RecoverPartitionStmt;
 import org.apache.doris.analysis.RecoverTableStmt;
@@ -75,7 +74,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.SingleRangePartitionDesc;
+import org.apache.doris.analysis.SinglePartitionDesc;
 import org.apache.doris.analysis.TableName;
 import org.apache.doris.analysis.TableRef;
 import org.apache.doris.analysis.TableRenameClause;
@@ -225,7 +224,6 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Multimap;
 import com.google.common.collect.Queues;
-import com.google.common.collect.Range;
 import com.google.common.collect.Sets;
 import com.sleepycat.je.rep.InsufficientLogException;
 import com.sleepycat.je.rep.NetworkRestore;
@@ -3048,7 +3046,7 @@ public class Catalog {
     }
 
     public void addPartition(Database db, String tableName, AddPartitionClause addPartitionClause) throws DdlException {
-        SingleRangePartitionDesc singlePartitionDesc = addPartitionClause.getSingeRangePartitionDesc();
+        SinglePartitionDesc singlePartitionDesc = addPartitionClause.getSingeRangePartitionDesc();
         DistributionDesc distributionDesc = addPartitionClause.getDistributionDesc();
         boolean isTempPartition = addPartitionClause.isTempPartition();
 
@@ -3081,8 +3079,8 @@ public class Catalog {
 
             // check partition type
             PartitionInfo partitionInfo = olapTable.getPartitionInfo();
-            if (partitionInfo.getType() != PartitionType.RANGE) {
-                throw new DdlException("Only support adding partition to range partitioned table");
+            if (partitionInfo.getType() != PartitionType.RANGE && partitionInfo.getType() != PartitionType.LIST) {
+                throw new DdlException("Only support adding partition to range and list partitioned table");
             }
 
             // check partition name
@@ -3105,9 +3103,8 @@ public class Catalog {
                 properties.put(PropertyAnalyzer.PROPERTIES_INMEMORY, olapTable.isInMemory().toString());
             }
 
-            RangePartitionInfo rangePartitionInfo = (RangePartitionInfo) partitionInfo;
-            singlePartitionDesc.analyze(rangePartitionInfo.getPartitionColumns().size(), properties);
-            rangePartitionInfo.checkAndCreateRange(singlePartitionDesc, isTempPartition);
+            singlePartitionDesc.analyze(partitionInfo.getPartitionColumns().size(), properties);
+            partitionInfo.createAndCheckPartitionItem(singlePartitionDesc, isTempPartition);
 
             // get distributionInfo
             List<Column> baseSchema = olapTable.getBaseSchema();
@@ -3235,13 +3232,12 @@ public class Catalog {
 
                 // check partition type
                 PartitionInfo partitionInfo = olapTable.getPartitionInfo();
-                if (partitionInfo.getType() != PartitionType.RANGE) {
-                    throw new DdlException("Only support adding partition to range partitioned table");
+                if (partitionInfo.getType() != PartitionType.RANGE && partitionInfo.getType() != PartitionType.LIST) {
+                    throw new DdlException("Only support adding partition to range and list partitioned table");
                 }
 
                 // update partition info
-                RangePartitionInfo rangePartitionInfo = (RangePartitionInfo) partitionInfo;
-                rangePartitionInfo.handleNewSinglePartitionDesc(singlePartitionDesc, partitionId, isTempPartition);
+                partitionInfo.handleNewSinglePartitionDesc(singlePartitionDesc, partitionId, isTempPartition);
 
                 if (isTempPartition) {
                     olapTable.addTempPartition(partition);
@@ -3250,11 +3246,23 @@ public class Catalog {
                 }
 
                 // log
-                PartitionPersistInfo info = new PartitionPersistInfo(db.getId(), olapTable.getId(), partition,
-                        rangePartitionInfo.getRange(partitionId), dataProperty,
-                        rangePartitionInfo.getReplicationNum(partitionId),
-                        rangePartitionInfo.getIsInMemory(partitionId),
-                        isTempPartition);
+                PartitionPersistInfo info = null;
+                if (partitionInfo.getType() == PartitionType.RANGE) {
+                    info = new PartitionPersistInfo(db.getId(), olapTable.getId(), partition,
+                            partitionInfo.getItem(partitionId).getItems(),
+                            ListPartitionItem.DUMMY_ITEM,
+                            dataProperty,
+                            partitionInfo.getReplicationNum(partitionId),
+                            partitionInfo.getIsInMemory(partitionId),
+                            isTempPartition);
+                } else if (partitionInfo.getType() == PartitionType.LIST) {
+                    info = new PartitionPersistInfo(db.getId(), olapTable.getId(), partition,
+                            RangePartitionItem.DUMMY_ITEM,
+                            partitionInfo.getItem(partitionId), dataProperty,
+                            partitionInfo.getReplicationNum(partitionId),
+                            partitionInfo.getIsInMemory(partitionId),
+                            isTempPartition);
+                }
                 editLog.logAddPartition(info);
 
                 LOG.info("succeed in creating partition[{}], temp: {}", partitionId, isTempPartition);
@@ -3282,9 +3290,15 @@ public class Catalog {
                 olapTable.addPartition(partition);
             }
 
-            ((RangePartitionInfo) partitionInfo).unprotectHandleNewSinglePartitionDesc(partition.getId(),
-                    info.isTempPartition(), info.getRange(), info.getDataProperty(), info.getReplicationNum(),
-                    info.isInMemory());
+            PartitionItem partitionItem = null;
+            if (partitionInfo.getType() == PartitionType.RANGE) {
+                partitionItem = new RangePartitionItem(info.getRange());
+            } else if (partitionInfo.getType() == PartitionType.LIST) {
+                partitionItem = info.getListPartitionItem();
+            }
+
+            partitionInfo.unprotectHandleNewSinglePartitionDesc(partition.getId(), info.isTempPartition(),
+                    partitionItem, info.getDataProperty(), info.getReplicationNum(), info.isInMemory());
 
             if (!isCheckpointThread()) {
                 // add to inverted index
@@ -3328,7 +3342,7 @@ public class Catalog {
         }
 
         PartitionInfo partitionInfo = olapTable.getPartitionInfo();
-        if (partitionInfo.getType() != PartitionType.RANGE) {
+        if (partitionInfo.getType() != PartitionType.RANGE && partitionInfo.getType() != PartitionType.LIST) {
             throw new DdlException("Alter table [" + olapTable.getName() + "] failed. Not a partitioned table");
         }
 
@@ -3528,12 +3542,10 @@ public class Catalog {
         Map<String, Long> partitionNameToId = Maps.newHashMap();
         if (partitionDesc != null) {
             // gen partition id first
-            if (partitionDesc instanceof RangePartitionDesc) {
-                RangePartitionDesc rangeDesc = (RangePartitionDesc) partitionDesc;
-                for (SingleRangePartitionDesc desc : rangeDesc.getSingleRangePartitionDescs()) {
-                    long partitionId = getNextId();
-                    partitionNameToId.put(desc.getPartitionName(), partitionId);
-                }
+            PartitionDesc partDesc = partitionDesc;
+            for (SinglePartitionDesc desc : partDesc.getSinglePartitionDescs()) {
+                long partitionId = getNextId();
+                partitionNameToId.put(desc.getPartitionName(), partitionId);
             }
             partitionInfo = partitionDesc.toPartitionInfo(baseSchema, partitionNameToId, false);
         } else {
@@ -3754,12 +3766,20 @@ public class Catalog {
                         tabletIdSet, olapTable.getCopiedIndexes(),
                         isInMemory, storageFormat, tabletType);
                 olapTable.addPartition(partition);
-            } else if (partitionInfo.getType() == PartitionType.RANGE) {
+            } else if (partitionInfo.getType() == PartitionType.RANGE || partitionInfo.getType() == PartitionType.LIST) {
                 try {
                     // just for remove entries in stmt.getProperties(),
                     // and then check if there still has unknown properties
                     PropertyAnalyzer.analyzeDataProperty(stmt.getProperties(), DataProperty.DEFAULT_DATA_PROPERTY);
-                    DynamicPartitionUtil.checkAndSetDynamicPartitionProperty(olapTable, properties);
+                    if (partitionInfo.getType() == PartitionType.RANGE) {
+                        DynamicPartitionUtil.checkAndSetDynamicPartitionProperty(olapTable, properties);
+
+                    } else if (partitionInfo.getType() == PartitionType.LIST) {
+                        if (DynamicPartitionUtil.checkDynamicPartitionPropertiesExist(properties)) {
+                            throw new DdlException("Only support dynamic partition properties on range partition table");
+
+                        }
+                    }
 
                     if (properties != null && !properties.isEmpty()) {
                         // here, all properties should be checked
@@ -3770,9 +3790,8 @@ public class Catalog {
                 }
 
                 // this is a 2-level partitioned tables
-                RangePartitionInfo rangePartitionInfo = (RangePartitionInfo) partitionInfo;
                 for (Map.Entry<String, Long> entry : partitionNameToId.entrySet()) {
-                    DataProperty dataProperty = rangePartitionInfo.getDataProperty(entry.getValue());
+                    DataProperty dataProperty = partitionInfo.getDataProperty(entry.getValue());
                     Partition partition = createPartitionWithIndices(db.getClusterName(), db.getId(), olapTable.getId(),
                             olapTable.getBaseIndexId(), entry.getValue(), entry.getKey(),
                             olapTable.getIndexIdToMeta(), distributionInfo,
@@ -3781,7 +3800,7 @@ public class Catalog {
                             versionInfo, bfColumns, bfFpp,
                             tabletIdSet, olapTable.getCopiedIndexes(),
                             isInMemory, storageFormat,
-                            rangePartitionInfo.getTabletType(entry.getValue()));
+                            partitionInfo.getTabletType(entry.getValue()));
                     olapTable.addPartition(partition);
                 }
             } else {
@@ -4000,7 +4019,7 @@ public class Catalog {
             if (separatePartition) {
                 partitionId = Lists.newArrayList();
             }
-            if (partitionInfo.getType() == PartitionType.RANGE) {
+            if (partitionInfo.getType() == PartitionType.RANGE || partitionInfo.getType() == PartitionType.LIST) {
                 sb.append("\n").append(partitionInfo.toSql(olapTable, partitionId));
             }
 
@@ -4167,28 +4186,38 @@ public class Catalog {
         createTableStmt.add(sb.toString());
 
         // 2. add partition
-        if (separatePartition && (table instanceof OlapTable)
-                && ((OlapTable) table).getPartitionInfo().getType() == PartitionType.RANGE
-                && ((OlapTable) table).getPartitions().size() > 1) {
-            OlapTable olapTable = (OlapTable) table;
-            RangePartitionInfo partitionInfo = (RangePartitionInfo) olapTable.getPartitionInfo();
-            boolean first = true;
-            for (Map.Entry<Long, Range<PartitionKey>> entry : partitionInfo.getSortedRangeMap(false)) {
-                if (first) {
-                    first = false;
-                    continue;
+        if (separatePartition && (table instanceof OlapTable) && ((OlapTable) table).getPartitions().size() > 1) {
+            if (((OlapTable) table).getPartitionInfo().getType() == PartitionType.RANGE ||
+                    ((OlapTable) table).getPartitionInfo().getType() == PartitionType.LIST) {
+                OlapTable olapTable = (OlapTable) table;
+                PartitionInfo partitionInfo = olapTable.getPartitionInfo();
+                boolean first = true;
+                for (Map.Entry<Long, PartitionItem> entry : partitionInfo.getSortedItemMap(false)) {
+                    if (first) {
+                        first = false;
+                        continue;
+                    }
+                    sb = new StringBuilder();
+                    Partition partition = olapTable.getPartition(entry.getKey());
+                    sb.append("ALTER TABLE ").append(table.getName());
+                    sb.append(" ADD PARTITION ").append(partition.getName()).append(" VALUES ");
+                    if (partitionInfo.getType() == PartitionType.RANGE) {
+                        sb.append("[");
+                        sb.append(((RangePartitionItem) entry.getValue()).getItems().lowerEndpoint().toSql());
+                        sb.append(", ");
+                        sb.append(((RangePartitionItem) entry.getValue()).getItems().upperEndpoint().toSql());
+                        sb.append(")");
+                    } else if (partitionInfo.getType() == PartitionType.LIST) {
+                        sb.append("IN (");
+                        sb.append(((ListPartitionItem) entry.getValue()).toSql());
+                        sb.append(")");
+                    }
+                    sb.append("(\"version_info\" = \"");
+                    sb.append(Joiner.on(",").join(partition.getVisibleVersion(), partition.getVisibleVersionHash()))
+                            .append("\"");
+                    sb.append(");");
+                    addPartitionStmt.add(sb.toString());
                 }
-                sb = new StringBuilder();
-                Partition partition = olapTable.getPartition(entry.getKey());
-                sb.append("ALTER TABLE ").append(table.getName());
-                sb.append(" ADD PARTITION ").append(partition.getName()).append(" VALUES [");
-                sb.append(entry.getValue().lowerEndpoint().toSql());
-                sb.append(", ").append(entry.getValue().upperEndpoint().toSql()).append(")");
-                sb.append("(\"version_info\" = \"");
-                sb.append(Joiner.on(",").join(partition.getVisibleVersion(), partition.getVisibleVersionHash()))
-                        .append("\"");
-                sb.append(");");
-                addPartitionStmt.add(sb.toString());
             }
         }
 
@@ -5128,7 +5157,7 @@ public class Catalog {
                 // user set a new colocate group,
                 // check if all partitions all this table has same buckets num and same replication number
                 PartitionInfo partitionInfo = table.getPartitionInfo();
-                if (partitionInfo.getType() == PartitionType.RANGE) {
+                if (partitionInfo.getType() == PartitionType.RANGE || partitionInfo.getType() == PartitionType.LIST) {
                     int bucketsNum = -1;
                     short replicationNum = -1;
                     for (Partition partition : table.getPartitions()) {
@@ -5279,7 +5308,7 @@ public class Catalog {
                 throw new DdlException("Table[" + table.getName() + "] is under " + table.getState());
             }
 
-            if (table.getPartitionInfo().getType() != PartitionType.RANGE) {
+            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.");
             }
@@ -5370,8 +5399,8 @@ public class Catalog {
         Preconditions.checkArgument(table.isWriteLockHeldByCurrentThread());
         String defaultReplicationNumName = "default."+ PropertyAnalyzer.PROPERTIES_REPLICATION_NUM;
         PartitionInfo partitionInfo = table.getPartitionInfo();
-        if (partitionInfo.getType() == PartitionType.RANGE) {
-            throw new DdlException("This is a range partitioned table, you should specify partitions with MODIFY PARTITION clause." +
+        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.");
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/CatalogRecycleBin.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/CatalogRecycleBin.java
index 596bb2d..bdee431 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/CatalogRecycleBin.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/CatalogRecycleBin.java
@@ -31,15 +31,15 @@ import org.apache.doris.common.util.RangeUtils;
 import org.apache.doris.persist.RecoverInfo;
 import org.apache.doris.thrift.TStorageMedium;
 
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Range;
 import com.google.common.collect.Sets;
 
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
@@ -106,7 +106,8 @@ public class CatalogRecycleBin extends MasterDaemon implements Writable {
     }
 
     public synchronized boolean recyclePartition(long dbId, long tableId, Partition partition,
-                                                 Range<PartitionKey> range, DataProperty dataProperty,
+                                                 Range<PartitionKey> range, PartitionItem listPartitionItem,
+                                                 DataProperty dataProperty,
                                                  short replicationNum,
                                                  boolean isInMemory) {
         if (idToPartition.containsKey(partition.getId())) {
@@ -119,7 +120,7 @@ public class CatalogRecycleBin extends MasterDaemon implements Writable {
 
         // recycle partition
         RecyclePartitionInfo partitionInfo = new RecyclePartitionInfo(dbId, tableId, partition,
-                                                                      range, dataProperty, replicationNum,
+                                                                      range, listPartitionItem, dataProperty, replicationNum,
                                                                       isInMemory);
         idToRecycleTime.put(partition.getId(), System.currentTimeMillis());
         idToPartition.put(partition.getId(), partitionInfo);
@@ -426,22 +427,28 @@ public class CatalogRecycleBin extends MasterDaemon implements Writable {
         if (recoverPartitionInfo == null) {
             throw new DdlException("No partition named " + partitionName + " in table " + table.getName());
         }
-        
-        // check if range is invalid
+
+        PartitionInfo partitionInfo = table.getPartitionInfo();
         Range<PartitionKey> recoverRange = recoverPartitionInfo.getRange();
-        RangePartitionInfo partitionInfo = (RangePartitionInfo) table.getPartitionInfo();
-        if (partitionInfo.getAnyIntersectRange(recoverRange, false) != null) {
-            throw new DdlException("Can not recover partition[" + partitionName + "]. Range conflict.");
+        PartitionItem recoverItem = null;
+        if (partitionInfo.getType() == PartitionType.RANGE) {
+            recoverItem = new RangePartitionItem(recoverRange);
+        } else if (partitionInfo.getType() == PartitionType.LIST) {
+            recoverItem = recoverPartitionInfo.getListPartitionItem();;
+        }
+        // check if partition item is invalid
+        if (partitionInfo.getAnyIntersectItem(recoverItem, false) != null) {
+            throw new DdlException("Can not recover partition[" + partitionName + "]. Partition item conflict.");
         }
 
         // recover partition
         Partition recoverPartition = recoverPartitionInfo.getPartition();
         Preconditions.checkState(recoverPartition.getName().equalsIgnoreCase(partitionName));
         table.addPartition(recoverPartition);
-        
+
         // recover partition info
         long partitionId = recoverPartition.getId();
-        partitionInfo.setRange(partitionId, false, recoverRange);
+        partitionInfo.setItem(partitionId, false, recoverItem);
         partitionInfo.setDataProperty(partitionId, recoverPartitionInfo.getDataProperty());
         partitionInfo.setReplicationNum(partitionId, recoverPartitionInfo.getReplicationNum());
         partitionInfo.setIsInMemory(partitionId, recoverPartitionInfo.isInMemory());
@@ -461,19 +468,25 @@ public class CatalogRecycleBin extends MasterDaemon implements Writable {
         Iterator<Map.Entry<Long, RecyclePartitionInfo>> iterator = idToPartition.entrySet().iterator();
         while (iterator.hasNext()) {
             Map.Entry<Long, RecyclePartitionInfo> entry = iterator.next();
-            RecyclePartitionInfo partitionInfo = entry.getValue();
-            if (partitionInfo.getPartition().getId() != partitionId) {
+            RecyclePartitionInfo recyclePartitionInfo = entry.getValue();
+            if (recyclePartitionInfo.getPartition().getId() != partitionId) {
                 continue;
             }
 
-            Preconditions.checkState(partitionInfo.getTableId() == table.getId());
+            Preconditions.checkState(recyclePartitionInfo.getTableId() == table.getId());
 
-            table.addPartition(partitionInfo.getPartition());
-            RangePartitionInfo rangePartitionInfo = (RangePartitionInfo) table.getPartitionInfo();
-            rangePartitionInfo.setRange(partitionId, false, partitionInfo.getRange());
-            rangePartitionInfo.setDataProperty(partitionId, partitionInfo.getDataProperty());
-            rangePartitionInfo.setReplicationNum(partitionId, partitionInfo.getReplicationNum());
-            rangePartitionInfo.setIsInMemory(partitionId, partitionInfo.isInMemory());
+            table.addPartition(recyclePartitionInfo.getPartition());
+            PartitionInfo partitionInfo = table.getPartitionInfo();
+            PartitionItem recoverItem = null;
+            if (partitionInfo.getType() == PartitionType.RANGE) {
+                recoverItem = new RangePartitionItem(recyclePartitionInfo.getRange());
+            } else if (partitionInfo.getType() == PartitionType.LIST) {
+                recoverItem = recyclePartitionInfo.getListPartitionItem();
+            }
+            partitionInfo.setItem(partitionId, false, recoverItem);
+            partitionInfo.setDataProperty(partitionId, recyclePartitionInfo.getDataProperty());
+            partitionInfo.setReplicationNum(partitionId, recyclePartitionInfo.getReplicationNum());
+            partitionInfo.setIsInMemory(partitionId, recyclePartitionInfo.isInMemory());
 
             iterator.remove();
             idToRecycleTime.remove(partitionId);
@@ -725,6 +738,7 @@ public class CatalogRecycleBin extends MasterDaemon implements Writable {
         private long tableId;
         private Partition partition;
         private Range<PartitionKey> range;
+        private PartitionItem listPartitionItem;
         private DataProperty dataProperty;
         private short replicationNum;
         private boolean isInMemory;
@@ -734,12 +748,14 @@ public class CatalogRecycleBin extends MasterDaemon implements Writable {
         }
 
         public RecyclePartitionInfo(long dbId, long tableId, Partition partition,
-                                    Range<PartitionKey> range, DataProperty dataProperty, short replicationNum,
+                                    Range<PartitionKey> range, PartitionItem listPartitionItem,
+                                    DataProperty dataProperty, short replicationNum,
                                     boolean isInMemory) {
             this.dbId = dbId;
             this.tableId = tableId;
             this.partition = partition;
             this.range = range;
+            this.listPartitionItem = listPartitionItem;
             this.dataProperty = dataProperty;
             this.replicationNum = replicationNum;
             this.isInMemory = isInMemory;
@@ -761,6 +777,10 @@ public class CatalogRecycleBin extends MasterDaemon implements Writable {
             return range;
         }
 
+        public PartitionItem getListPartitionItem() {
+            return listPartitionItem;
+        }
+
         public DataProperty getDataProperty() {
             return dataProperty;
         }
@@ -779,6 +799,7 @@ public class CatalogRecycleBin extends MasterDaemon implements Writable {
             out.writeLong(tableId);
             partition.write(out);
             RangeUtils.writeRange(out, range);
+            listPartitionItem.write(out);
             dataProperty.write(out);
             out.writeShort(replicationNum);
             out.writeBoolean(isInMemory);
@@ -789,6 +810,12 @@ public class CatalogRecycleBin extends MasterDaemon implements Writable {
             tableId = in.readLong();
             partition = Partition.read(in);
             range = RangeUtils.readRange(in);
+            if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_98) {
+                listPartitionItem = ListPartitionItem.read(in);
+            } else {
+                listPartitionItem = ListPartitionItem.DUMMY_ITEM;
+            }
+
             dataProperty = DataProperty.read(in);
             replicationNum = in.readShort();
             if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_72) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/ListPartitionInfo.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/ListPartitionInfo.java
new file mode 100644
index 0000000..76ef0b2
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/ListPartitionInfo.java
@@ -0,0 +1,219 @@
+// 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.catalog;
+
+import org.apache.doris.analysis.PartitionKeyDesc;
+import org.apache.doris.analysis.PartitionValue;
+import org.apache.doris.analysis.SinglePartitionDesc;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.DdlException;
+import org.apache.doris.common.util.ListUtil;
+
+import com.google.common.base.Preconditions;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+public class ListPartitionInfo extends PartitionInfo{
+
+    public ListPartitionInfo() {
+        // for persist
+        super();
+    }
+
+    public ListPartitionInfo(List<Column> partitionColumns) {
+        super(PartitionType.LIST);
+        this.partitionColumns = partitionColumns;
+        this.isMultiColumnPartition = partitionColumns.size() > 1;
+    }
+
+    public static PartitionInfo read(DataInput in) throws IOException {
+        PartitionInfo partitionInfo = new ListPartitionInfo();
+        partitionInfo.readFields(in);
+        return partitionInfo;
+    }
+
+    public PartitionItem createAndCheckPartitionItem(SinglePartitionDesc desc, boolean isTemp) throws DdlException {
+        // get partition key
+        PartitionKeyDesc partitionKeyDesc = desc.getPartitionKeyDesc();
+
+        for (List<PartitionValue> values : partitionKeyDesc.getInValues()) {
+            Preconditions.checkArgument(values.size() == partitionColumns.size(),
+                    "partition key desc list size[" + values.size() + "] is not equal to " +
+                            "partition column size[" + partitionColumns.size() + "]");
+        }
+        List<PartitionKey> partitionKeys = new ArrayList<>();
+        try {
+            for (List<PartitionValue> values : partitionKeyDesc.getInValues()) {
+                PartitionKey partitionKey = PartitionKey.createListPartitionKey(values, partitionColumns);
+                checkNewPartitionKey(partitionKey, partitionKeyDesc, isTemp);
+                if (partitionKeys.contains(partitionKey)) {
+                    throw new AnalysisException("The partition key[" + partitionKeyDesc.toSql() + "] has duplicate item ["
+                            + partitionKey.toSql() + "].");
+                }
+                partitionKeys.add(partitionKey);
+            }
+        } catch (AnalysisException e) {
+            throw new DdlException("Invalid list value format: " + e.getMessage());
+        }
+        return new ListPartitionItem(partitionKeys);
+    }
+
+    private void checkNewPartitionKey(PartitionKey newKey, PartitionKeyDesc keyDesc, boolean isTemp) throws AnalysisException {
+        Map<Long, PartitionItem> id2Item = idToItem;
+        if (isTemp) {
+             id2Item = idToTempItem;
+        }
+        // check new partition key not exists.
+        for (Map.Entry<Long, PartitionItem> entry : id2Item.entrySet()) {
+            if (((ListPartitionItem)entry.getValue()).getItems().contains(newKey)) {
+                StringBuilder sb = new StringBuilder();
+                sb.append("The partition key[").append(newKey.toSql()).append("] in partition item[")
+                        .append(keyDesc.toSql()).append("] is conflict with current partitionKeys[")
+                        .append(((ListPartitionItem) entry.getValue()).toSql()).append("]");
+                throw new AnalysisException(sb.toString());
+            }
+        }
+    }
+
+    @Override
+    public void checkPartitionItemListsMatch(List<PartitionItem> list1, List<PartitionItem> list2) throws DdlException {
+        ListUtil.checkPartitionKeyListsMatch(list1, list2);
+    }
+
+    @Override
+    public void checkPartitionItemListsConflict(List<PartitionItem> list1, List<PartitionItem> list2) throws DdlException {
+        ListUtil.checkListsConflict(list1, list2);
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+        super.write(out);
+        // partition columns
+        out.writeInt(partitionColumns.size());
+        for (Column column : partitionColumns) {
+            column.write(out);
+        }
+
+        out.writeInt(idToItem.size());
+        for (Map.Entry<Long, PartitionItem> entry : idToItem.entrySet()) {
+            out.writeLong(entry.getKey());
+            entry.getValue().write(out);
+        }
+
+        out.writeInt(idToTempItem.size());
+        for (Map.Entry<Long, PartitionItem> entry : idToTempItem.entrySet()) {
+            out.writeLong(entry.getKey());
+            entry.getValue().write(out);
+        }
+
+    }
+
+    public void readFields(DataInput in) throws IOException {
+        super.readFields(in);
+
+        int counter = in.readInt();
+        for (int i = 0; i < counter; i++) {
+            Column column = Column.read(in);
+            partitionColumns.add(column);
+        }
+
+        this.isMultiColumnPartition = partitionColumns.size() > 1;
+
+        counter = in.readInt();
+        for (int i = 0; i < counter; i++) {
+            long partitionId = in.readLong();
+            ListPartitionItem partitionItem = ListPartitionItem.read(in);
+            idToItem.put(partitionId, partitionItem);
+        }
+
+        counter = in.readInt();
+        for (int i = 0; i < counter; i++) {
+            long partitionId = in.readLong();
+            ListPartitionItem partitionItem = ListPartitionItem.read(in);
+            idToTempItem.put(partitionId, partitionItem);
+        }
+    }
+
+    public static void checkPartitionColumn(Column column) throws AnalysisException {
+        PrimitiveType type = column.getDataType();
+        if (!type.isFixedPointType() && !type.isDateType()
+                && !type.isCharFamily() && type != PrimitiveType.BOOLEAN) {
+            throw new AnalysisException("Column[" + column.getName() + "] type[" + type
+                    + "] cannot be a list partition key.");
+        }
+    }
+
+    @Override
+    public String toSql(OlapTable table, List<Long> partitionId) {
+        StringBuilder sb = new StringBuilder();
+        sb.append("PARTITION BY LIST(");
+        int idx = 0;
+        for (Column column : partitionColumns) {
+            if (idx != 0) {
+                sb.append(", ");
+            }
+            sb.append("`").append(column.getName()).append("`");
+            idx++;
+        }
+        sb.append(")\n(");
+
+        // sort list
+        List<Map.Entry<Long, PartitionItem>> entries = new ArrayList<>(this.idToItem.entrySet());
+        Collections.sort(entries, ListUtil.LIST_MAP_ENTRY_COMPARATOR);
+        idx = 0;
+        for (Map.Entry<Long, PartitionItem> entry : entries) {
+            Partition partition = table.getPartition(entry.getKey());
+            String partitionName = partition.getName();
+            List<PartitionKey> partitionKeys = entry.getValue().getItems();
+
+            sb.append("PARTITION ").append(partitionName).append(" VALUES IN ");
+            sb.append("(");
+            int idxInternal = 0;
+            for (PartitionKey partitionKey : partitionKeys) {
+                String partitionKeyStr = partitionKey.toSql();
+                if (!isMultiColumnPartition) {
+                    partitionKeyStr = partitionKeyStr.substring(1, partitionKeyStr.length() - 1);
+                }
+                sb.append(partitionKeyStr);
+                if (partitionKeys.size() > 1 && idxInternal != partitionKeys.size() - 1) {
+                    sb.append(",");
+                }
+                idxInternal++;
+            }
+            sb.append(")");
+
+            if (partitionId != null) {
+                partitionId.add(entry.getKey());
+                break;
+            }
+
+            if (idx != entries.size() - 1) {
+                sb.append(",\n");
+            }
+            idx++;
+        }
+        sb.append(")");
+        return sb.toString();
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/ListPartitionItem.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/ListPartitionItem.java
new file mode 100644
index 0000000..5395d83
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/ListPartitionItem.java
@@ -0,0 +1,153 @@
+// 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.catalog;
+
+import com.clearspring.analytics.util.Lists;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+public class ListPartitionItem extends PartitionItem {
+    private List<PartitionKey> partitionKeys;
+
+    public static ListPartitionItem DUMMY_ITEM = new ListPartitionItem(Lists.newArrayList());
+
+    public ListPartitionItem(List<PartitionKey> partitionKeys) {
+        this.partitionKeys = partitionKeys;
+    }
+
+    public List<PartitionKey> getItems() {
+        return partitionKeys;
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+        out.writeInt(partitionKeys.size());
+        for (PartitionKey partitionKey : partitionKeys) {
+            partitionKey.write(out);
+        }
+    }
+
+    public static ListPartitionItem read(DataInput input) throws IOException {
+        int counter = input.readInt();
+        List<PartitionKey> partitionKeys = new ArrayList<>();
+        for (int i = 0; i < counter; i++) {
+            PartitionKey partitionKey = PartitionKey.read(input);
+            partitionKeys.add(partitionKey);
+        }
+        return new ListPartitionItem(partitionKeys);
+    }
+
+    @Override
+    public int compareTo(PartitionItem other) {
+        int thisKeyLen = this.partitionKeys.size();
+        int otherKeyLen = ((ListPartitionItem) other).getItems().size();
+        int minLen = Math.min(thisKeyLen, otherKeyLen);
+        for (int i = 0; i < minLen; i++) {
+            int ret = this.getItems().get(i).compareTo(((ListPartitionItem) other).getItems().get(i));
+            if (0 != ret) {
+                return ret;
+            }
+        }
+        return Integer.compare(thisKeyLen, otherKeyLen);
+    }
+
+    @Override
+    public PartitionItem getIntersect(PartitionItem newItem) {
+        List<PartitionKey> newKeys = newItem.getItems();
+        for (PartitionKey newKey : newKeys) {
+            if (partitionKeys.contains(newKey)) {
+                return newItem;
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj) {
+            return true;
+        }
+
+        if (!(obj instanceof ListPartitionItem)) {
+            return false;
+        }
+
+        ListPartitionItem partitionItem = (ListPartitionItem) obj;
+        // check keys
+        if (partitionKeys != partitionItem.getItems()) {
+            if (partitionKeys.size() != partitionItem.getItems().size()) {
+                return false;
+            }
+            for (int i = 0; i < partitionKeys.size(); i++) {
+                if (!partitionKeys.get(i).equals(partitionItem.getItems().get(i))) {
+                    return false;
+                }
+            }
+        }
+
+        return true;
+    }
+
+    @Override
+    public int hashCode() {
+        int result = 17;
+        result = 31 * result + partitionKeys.size();
+        for (PartitionKey partitionKey : partitionKeys) {
+            result = 31 * result + partitionKey.hashCode();
+        }
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder builder = new StringBuilder();
+        builder.append("partitionKeys: [");
+        for (PartitionKey partitionKey : partitionKeys) {
+            builder.append(partitionKey.toString());
+        }
+        builder.append("]; ");
+        return builder.toString();
+    }
+
+    public String toSql() {
+        StringBuilder sb = new StringBuilder();
+        int size = partitionKeys.size();
+        if (size > 1) {
+            sb.append("(");
+        }
+
+        int i = 0;
+        for (PartitionKey partitionKey : partitionKeys) {
+            sb.append(partitionKey.toSql());
+            if (i < partitionKeys.size() - 1) {
+                sb.append(",");
+            }
+            i++;
+        }
+
+        if (size > 1) {
+            sb.append(")");
+        }
+
+        return sb.toString();
+    }
+}
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 0e9a2da..72b0d9c 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
@@ -34,6 +34,7 @@ import org.apache.doris.catalog.Replica.ReplicaState;
 import org.apache.doris.catalog.Tablet.TabletStatus;
 import org.apache.doris.clone.TabletSchedCtx;
 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;
@@ -41,7 +42,6 @@ import org.apache.doris.common.Pair;
 import org.apache.doris.common.io.DeepCopy;
 import org.apache.doris.common.io.Text;
 import org.apache.doris.common.util.PropertyAnalyzer;
-import org.apache.doris.common.util.RangeUtils;
 import org.apache.doris.common.util.Util;
 import org.apache.doris.qe.OriginStatement;
 import org.apache.doris.system.SystemInfoService;
@@ -52,16 +52,16 @@ import org.apache.doris.thrift.TStorageType;
 import org.apache.doris.thrift.TTableDescriptor;
 import org.apache.doris.thrift.TTableType;
 
-import org.apache.commons.codec.digest.DigestUtils;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Range;
 import com.google.common.collect.Sets;
 
+import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
@@ -428,19 +428,16 @@ public class OlapTable extends Table {
         }
 
         // reset partition info and idToPartition map
-        if (partitionInfo.getType() == PartitionType.RANGE) {
-            RangePartitionInfo rangePartitionInfo = (RangePartitionInfo) partitionInfo;
+        if (partitionInfo.getType() == PartitionType.RANGE || partitionInfo.getType() == PartitionType.LIST) {
             for (Map.Entry<String, Long> entry : origPartNameToId.entrySet()) {
                 long newPartId = catalog.getNextId();
-                rangePartitionInfo.idToDataProperty.put(newPartId,
-                                                        rangePartitionInfo.idToDataProperty.remove(entry.getValue()));
-                rangePartitionInfo.idToReplicationNum.remove(entry.getValue());
-                rangePartitionInfo.idToReplicationNum.put(newPartId,
-                                                          (short) restoreReplicationNum);
-                rangePartitionInfo.getIdToRange(false).put(newPartId,
-                        rangePartitionInfo.getIdToRange(false).remove(entry.getValue()));
-
-                rangePartitionInfo.idToInMemory.put(newPartId, rangePartitionInfo.idToInMemory.remove(entry.getValue()));
+                partitionInfo.idToDataProperty.put(newPartId,
+                                                        partitionInfo.idToDataProperty.remove(entry.getValue()));
+                partitionInfo.idToReplicationNum.remove(entry.getValue());
+                partitionInfo.idToReplicationNum.put(newPartId, (short) restoreReplicationNum);
+                partitionInfo.getIdToItem(false).put(newPartId,
+                        partitionInfo.getIdToItem(false).remove(entry.getValue()));
+                partitionInfo.idToInMemory.put(newPartId, partitionInfo.idToInMemory.remove(entry.getValue()));
                 idToPartition.put(newPartId, idToPartition.remove(entry.getValue()));
             }
         } else {
@@ -681,22 +678,44 @@ public class OlapTable extends Table {
             idToPartition.remove(partition.getId());
             nameToPartition.remove(partitionName);
 
-            Preconditions.checkState(partitionInfo.getType() == PartitionType.RANGE);
-            RangePartitionInfo rangePartitionInfo = (RangePartitionInfo) partitionInfo;
+            Preconditions.checkState(partitionInfo.getType() == PartitionType.RANGE ||
+                    partitionInfo.getType() == PartitionType.LIST);
 
             if (!isForceDrop) {
                 // recycle partition
-                Catalog.getCurrentRecycleBin().recyclePartition(dbId, id, partition,
-                        rangePartitionInfo.getRange(partition.getId()),
-                        rangePartitionInfo.getDataProperty(partition.getId()),
-                        rangePartitionInfo.getReplicationNum(partition.getId()),
-                        rangePartitionInfo.getIsInMemory(partition.getId()));
+                if (partitionInfo.getType() == PartitionType.RANGE) {
+                    Catalog.getCurrentRecycleBin().recyclePartition(dbId, id, partition,
+                                              partitionInfo.getItem(partition.getId()).getItems(),
+                                              new ListPartitionItem(Lists.newArrayList(new PartitionKey())),
+                                              partitionInfo.getDataProperty(partition.getId()),
+                                              partitionInfo.getReplicationNum(partition.getId()),
+                                              partitionInfo.getIsInMemory(partition.getId()));
+
+                } else if (partitionInfo.getType() == PartitionType.LIST) {
+                    // construct a dummy range
+                    List<Column> dummyColumns = new ArrayList<>();
+                    dummyColumns.add(new Column("dummy", PrimitiveType.INT));
+                    PartitionKey dummyKey = null;
+                    try {
+                        dummyKey = PartitionKey.createInfinityPartitionKey(dummyColumns, false);
+                    } catch (AnalysisException e) {
+                        e.printStackTrace();
+                    }
+                    Range<PartitionKey> dummyRange = Range.open(new PartitionKey(), dummyKey);
+
+                    Catalog.getCurrentRecycleBin().recyclePartition(dbId, id, partition,
+                            dummyRange,
+                            partitionInfo.getItem(partition.getId()),
+                            partitionInfo.getDataProperty(partition.getId()),
+                            partitionInfo.getReplicationNum(partition.getId()),
+                            partitionInfo.getIsInMemory(partition.getId()));
+                }
             } else if (!reserveTablets) {
                 Catalog.getCurrentCatalog().onErasePartition(partition);
             }
 
             // drop partition info
-            rangePartitionInfo.dropPartition(partition.getId());
+            partitionInfo.dropPartition(partition.getId());
         }
         return partition;
     }
@@ -1133,7 +1152,9 @@ public class OlapTable extends Table {
             partitionInfo = SinglePartitionInfo.read(in);
         } else if (partType == PartitionType.RANGE) {
             partitionInfo = RangePartitionInfo.read(in);
-        } else {
+        } else if (partType == PartitionType.LIST) {
+            partitionInfo = ListPartitionInfo.read(in);
+        }else {
             throw new IOException("invalid partition type: " + partType);
         }
 
@@ -1197,9 +1218,9 @@ public class OlapTable extends Table {
             if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_77) {
                 RangePartitionInfo tempRangeInfo = tempPartitions.getPartitionInfo();
                 if (tempRangeInfo != null) {
-                    for (long partitionId : tempRangeInfo.getIdToRange(false).keySet()) {
-                        ((RangePartitionInfo) this.partitionInfo).addPartition(partitionId, true,
-                                tempRangeInfo.getRange(partitionId), tempRangeInfo.getDataProperty(partitionId),
+                    for (long partitionId : tempRangeInfo.getIdToItem(false).keySet()) {
+                        this.partitionInfo.addPartition(partitionId, true,
+                                tempRangeInfo.getItem(partitionId), tempRangeInfo.getDataProperty(partitionId),
                                 tempRangeInfo.getReplicationNum(partitionId), tempRangeInfo.getIsInMemory(partitionId));
                     }
                 }
@@ -1288,11 +1309,11 @@ public class OlapTable extends Table {
         short replicationNum = partitionInfo.getReplicationNum(oldPartition.getId());
         boolean isInMemory = partitionInfo.getIsInMemory(oldPartition.getId());
 
-        if (partitionInfo.getType() == PartitionType.RANGE) {
-            RangePartitionInfo rangePartitionInfo = (RangePartitionInfo) partitionInfo;
-            Range<PartitionKey> range = rangePartitionInfo.getRange(oldPartition.getId());
-            rangePartitionInfo.dropPartition(oldPartition.getId());
-            rangePartitionInfo.addPartition(newPartition.getId(), false, range, dataProperty,
+        if (partitionInfo.getType() == PartitionType.RANGE
+                || partitionInfo.getType() == PartitionType.LIST) {
+            PartitionItem item = partitionInfo.getItem(oldPartition.getId());
+            partitionInfo.dropPartition(oldPartition.getId());
+            partitionInfo.addPartition(newPartition.getId(), false, item, dataProperty,
                     replicationNum, isInMemory);
         } else {
             partitionInfo.dropPartition(oldPartition.getId());
@@ -1533,41 +1554,8 @@ public class OlapTable extends Table {
      */
     public void replaceTempPartitions(List<String> partitionNames, List<String> tempPartitionNames,
             boolean strictRange, boolean useTempPartitionName) throws DdlException {
-        RangePartitionInfo rangeInfo = (RangePartitionInfo) partitionInfo;
-
-        if (strictRange) {
-            // check if range of partitions and temp partitions are exactly same
-            List<Range<PartitionKey>> rangeList = Lists.newArrayList();
-            List<Range<PartitionKey>> tempRangeList = Lists.newArrayList();
-            for (String partName : partitionNames) {
-                Partition partition = nameToPartition.get(partName);
-                Preconditions.checkNotNull(partition);
-                rangeList.add(rangeInfo.getRange(partition.getId()));
-            }
-
-            for (String partName : tempPartitionNames) {
-                Partition partition = tempPartitions.getPartition(partName);
-                Preconditions.checkNotNull(partition);
-                tempRangeList.add(rangeInfo.getRange(partition.getId()));
-            }
-            RangeUtils.checkRangeListsMatch(rangeList, tempRangeList);
-        } else {
-            // check after replacing, whether the range will conflict
-            Set<Long> replacePartitionIds = Sets.newHashSet();
-            for (String partName : partitionNames) {
-                Partition partition = nameToPartition.get(partName);
-                Preconditions.checkNotNull(partition);
-                replacePartitionIds.add(partition.getId());
-            }
-            List<Range<PartitionKey>> replacePartitionRanges = Lists.newArrayList();
-            for (String partName : tempPartitionNames) {
-                Partition partition = tempPartitions.getPartition(partName);
-                Preconditions.checkNotNull(partition);
-                replacePartitionRanges.add(rangeInfo.getRange(partition.getId()));
-            }
-            List<Range<PartitionKey>> sortedRangeList = rangeInfo.getRangeList(replacePartitionIds, false);
-            RangeUtils.checkRangeConflict(sortedRangeList, replacePartitionRanges);
-        }
+        // check partition items
+        checkPartition(partitionNames, tempPartitionNames, strictRange);
         
         // begin to replace
         // 1. drop old partitions
@@ -1584,7 +1572,7 @@ public class OlapTable extends Table {
             // drop
             tempPartitions.dropPartition(partitionName, false);
             // move the range from idToTempRange to idToRange
-            rangeInfo.moveRangeFromTempToFormal(partition.getId());
+            partitionInfo.moveFromTempToFormal(partition.getId());
         }
 
         // change the name so that after replacing, the partition name remain unchanged
@@ -1595,6 +1583,44 @@ public class OlapTable extends Table {
         }
     }
 
+    private void checkPartition(List<String> partitionNames, List<String> tempPartitionNames,
+                                boolean strictRange) throws DdlException {
+        if (strictRange) {
+            List<PartitionItem> list = Lists.newArrayList();
+            List<PartitionItem> tempList = Lists.newArrayList();
+            for (String partName : partitionNames) {
+                Partition partition = nameToPartition.get(partName);
+                Preconditions.checkNotNull(partition);
+                list.add(partitionInfo.getItem(partition.getId()));
+            }
+            for (String partName : tempPartitionNames) {
+                Partition partition = tempPartitions.getPartition(partName);
+                Preconditions.checkNotNull(partition);
+                tempList.add(partitionInfo.getItem(partition.getId()));
+            }
+            partitionInfo.checkPartitionItemListsMatch(list, tempList);
+        } else {
+            // check after replacing, whether the range will conflict
+            Set<Long> replacePartitionIds = Sets.newHashSet();
+            for (String partName : partitionNames) {
+                Partition partition = nameToPartition.get(partName);
+                Preconditions.checkNotNull(partition);
+                replacePartitionIds.add(partition.getId());
+            }
+            // get all items except for partitions in "replacePartitionIds"
+            List<PartitionItem> currentItemList = partitionInfo.getItemList(replacePartitionIds, false);
+
+            List<PartitionItem> replacePartitionItems = Lists.newArrayList();
+            for (String partName : tempPartitionNames) {
+                Partition partition = tempPartitions.getPartition(partName);
+                Preconditions.checkNotNull(partition);
+                replacePartitionItems.add(partitionInfo.getItem(partition.getId()));
+            }
+
+            partitionInfo.checkPartitionItemListsConflict(currentItemList, replacePartitionItems);
+        }
+    }
+
     public void addTempPartition(Partition partition) {
         tempPartitions.addPartition(partition);
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionInfo.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionInfo.java
index 575525b..b64b341 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionInfo.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionInfo.java
@@ -17,23 +17,29 @@
 
 package org.apache.doris.catalog;
 
+import org.apache.doris.analysis.SinglePartitionDesc;
+import org.apache.doris.common.DdlException;
 import org.apache.doris.common.FeMetaVersion;
 import org.apache.doris.common.io.Text;
 import org.apache.doris.common.io.Writable;
+import org.apache.doris.thrift.TStorageMedium;
+import org.apache.doris.thrift.TTabletType;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 
-import org.apache.doris.thrift.TStorageMedium;
-import org.apache.doris.thrift.TTabletType;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 /*
  * Repository of a partition's related infos
@@ -42,6 +48,12 @@ public class PartitionInfo implements Writable {
     private static final Logger LOG = LogManager.getLogger(PartitionInfo.class);
 
     protected PartitionType type;
+    // partition columns for list and range partitions
+    protected List<Column> partitionColumns = Lists.newArrayList();
+    // formal partition id -> partition item
+    protected Map<Long, PartitionItem> idToItem = Maps.newHashMap();
+    // temp partition id -> partition item
+    protected Map<Long, PartitionItem> idToTempItem = Maps.newHashMap();
     // partition id -> data property
     protected Map<Long, DataProperty> idToDataProperty;
     // partition id -> replication num
@@ -71,10 +83,122 @@ public class PartitionInfo implements Writable {
         this.idToTabletType = new HashMap<>();
     }
 
+    public PartitionInfo(PartitionType type, List<Column> partitionColumns) {
+        this(type);
+        this.partitionColumns = partitionColumns;
+        this.isMultiColumnPartition = partitionColumns.size() > 1;
+    }
+
     public PartitionType getType() {
         return type;
     }
 
+    public List<Column> getPartitionColumns(){
+        return partitionColumns;
+    }
+
+    public Map<Long, PartitionItem> getIdToItem(boolean isTemp) {
+        if (isTemp) {
+            return idToTempItem;
+        } else {
+            return idToItem;
+        }
+    }
+
+    public PartitionItem getItem(long partitionId) {
+        PartitionItem item = idToItem.get(partitionId);
+        if (item == null) {
+            item = idToTempItem.get(partitionId);
+        }
+        return item;
+    }
+
+    public void setItem(long partitionId, boolean isTemp, PartitionItem item) {
+        setItemInternal(partitionId, isTemp, item);
+    }
+
+    private void setItemInternal(long partitionId, boolean isTemp, PartitionItem item) {
+        if (isTemp) {
+            idToTempItem.put(partitionId, item);
+        } else {
+            idToItem.put(partitionId, item);
+        }
+    }
+
+    public PartitionItem handleNewSinglePartitionDesc(SinglePartitionDesc desc,
+                                              long partitionId, boolean isTemp) throws DdlException {
+        Preconditions.checkArgument(desc.isAnalyzed());
+        PartitionItem partitionItem = createAndCheckPartitionItem(desc, isTemp);
+        setItemInternal(partitionId, isTemp, partitionItem);
+
+        idToDataProperty.put(partitionId, desc.getPartitionDataProperty());
+        idToReplicationNum.put(partitionId, desc.getReplicationNum());
+        idToInMemory.put(partitionId, desc.isInMemory());
+
+        return partitionItem;
+    }
+
+    public PartitionItem createAndCheckPartitionItem(SinglePartitionDesc desc, boolean isTemp) throws DdlException {
+        return null;
+    }
+
+    public void unprotectHandleNewSinglePartitionDesc(long partitionId, boolean isTemp, PartitionItem partitionItem,
+                                                      DataProperty dataProperty, short replicationNum,
+                                                      boolean isInMemory) {
+        setItemInternal(partitionId, isTemp, partitionItem);
+        idToDataProperty.put(partitionId, dataProperty);
+        idToReplicationNum.put(partitionId, replicationNum);
+        idToInMemory.put(partitionId, isInMemory);
+    }
+
+    public List<Map.Entry<Long, PartitionItem>> getSortedItemMap(boolean isTemp) {
+        Map<Long, PartitionItem> tmpMap = idToItem;
+        if (isTemp) {
+            tmpMap = idToTempItem;
+        }
+        List<Map.Entry<Long, PartitionItem>> sortedList = Lists.newArrayList(tmpMap.entrySet());
+        Collections.sort(sortedList, PartitionItem.ITEM_MAP_ENTRY_COMPARATOR);
+        return sortedList;
+    }
+
+    // get sorted item list, exclude partitions which ids are in 'excludePartitionIds'
+    public List<PartitionItem> getItemList(Set<Long> excludePartitionIds, boolean isTemp) {
+        Map<Long, PartitionItem> tempMap = idToItem;
+        if (isTemp) {
+            tempMap = idToTempItem;
+        }
+        List<PartitionItem> resultList = Lists.newArrayList();
+        for (Map.Entry<Long, PartitionItem> entry : tempMap.entrySet()) {
+            if (!excludePartitionIds.contains(entry.getKey())) {
+                resultList.add(entry.getValue());
+            }
+        }
+        return resultList;
+    }
+
+    // return any item intersect with the newItem.
+    // return null if no item intersect.
+    public PartitionItem getAnyIntersectItem(PartitionItem newItem, boolean isTemp) {
+        Map<Long, PartitionItem> tmpMap = idToItem;
+        if (isTemp) {
+            tmpMap = idToTempItem;
+        }
+        PartitionItem retItem;
+        for (PartitionItem item : tmpMap.values()) {
+            retItem = item.getIntersect(newItem);
+            if (null != retItem) {
+                return retItem;
+            }
+        }
+        return null;
+    }
+
+    public void checkPartitionItemListsMatch(List<PartitionItem> list1, List<PartitionItem> list2) throws DdlException {
+    }
+
+    public void checkPartitionItemListsConflict(List<PartitionItem> list1, List<PartitionItem> list2) throws DdlException {
+    }
+
     public DataProperty getDataProperty(long partitionId) {
         return idToDataProperty.get(partitionId);
     }
@@ -117,6 +241,14 @@ public class PartitionInfo implements Writable {
         idToDataProperty.remove(partitionId);
         idToReplicationNum.remove(partitionId);
         idToInMemory.remove(partitionId);
+        idToItem.remove(partitionId);
+        idToTempItem.remove(partitionId);
+    }
+
+    public void addPartition(long partitionId, boolean isTemp, PartitionItem item, DataProperty dataProperty,
+                                 short replicationNum, boolean isInMemory){
+        addPartition(partitionId, dataProperty, replicationNum, isInMemory);
+        setItemInternal(partitionId, isTemp, item);
     }
 
     public void addPartition(long partitionId, DataProperty dataProperty,
@@ -141,6 +273,13 @@ public class PartitionInfo implements Writable {
         return "";
     }
 
+    public void moveFromTempToFormal(long tempPartitionId) {
+        PartitionItem item = idToTempItem.remove(tempPartitionId);
+        if (item != null) {
+            idToItem.put(tempPartitionId, item);
+        }
+    }
+
     @Override
     public void write(DataOutput out) throws IOException {
         Text.writeString(out, type.name());
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionItem.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionItem.java
new file mode 100644
index 0000000..f71b8c6
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionItem.java
@@ -0,0 +1,33 @@
+// 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.catalog;
+
+import org.apache.doris.common.io.Writable;
+
+import java.util.Comparator;
+import java.util.Map;
+
+public abstract class PartitionItem implements Comparable<PartitionItem>,Writable {
+    public static final Comparator<Map.Entry<Long, PartitionItem>> ITEM_MAP_ENTRY_COMPARATOR =
+            Comparator.comparing(o -> ((ListPartitionItem) o.getValue()).getItems().iterator().next());
+
+    public abstract <T> T getItems();
+
+    public abstract PartitionItem getIntersect(PartitionItem newItem);
+
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionKey.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionKey.java
index 3fd0732..e0aec61 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionKey.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionKey.java
@@ -17,12 +17,14 @@
 
 package org.apache.doris.catalog;
 
+import org.apache.doris.analysis.BoolLiteral;
 import org.apache.doris.analysis.DateLiteral;
 import org.apache.doris.analysis.IntLiteral;
 import org.apache.doris.analysis.LargeIntLiteral;
 import org.apache.doris.analysis.LiteralExpr;
 import org.apache.doris.analysis.MaxLiteral;
 import org.apache.doris.analysis.PartitionValue;
+import org.apache.doris.analysis.StringLiteral;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.io.Text;
 import org.apache.doris.common.io.Writable;
@@ -85,6 +87,40 @@ public class PartitionKey implements Comparable<PartitionKey>, Writable {
         return partitionKey;
     }
 
+    public static PartitionKey createListPartitionKey(List<PartitionValue> values, List<Column> columns)
+            throws AnalysisException {
+        // for multi list partition:
+        //
+        // PARTITION BY LIST(k1, k2)
+        // (
+        //     PARTITION p1 VALUES IN (("1","beijing"), ("1", "shanghai")),
+        //     PARTITION p2 VALUES IN (("2","shanghai"))
+        // )
+        //
+        // for single list partition:
+        //
+        // PARTITION BY LIST(`k1`)
+        // (
+        //     PARTITION p1 VALUES IN ("1", "2", "3", "4", "5"),
+        //     PARTITION p2 VALUES IN ("6", "7", "8", "9", "10"),
+        //     PARTITION p3 VALUES IN ("11", "12", "13", "14", "15"),
+        //     PARTITION p4 VALUES IN ("16", "17", "18", "19", "20"),
+        //     PARTITION p5 VALUES IN ("21", "22", "23", "24", "25"),
+        //     PARTITION p6 VALUES IN ("26")
+        // )
+        //
+        Preconditions.checkArgument(values.size() == columns.size(),
+                "in value size[" + values.size() + "] is not equal to partition column size[" + columns.size() + "].");
+
+        PartitionKey partitionKey = new PartitionKey();
+        for (int i = 0; i < values.size(); i++) {
+            partitionKey.keys.add(values.get(i).getValue(Type.fromPrimitiveType(columns.get(i).getDataType())));
+            partitionKey.types.add(columns.get(i).getDataType());
+        }
+
+        return partitionKey;
+    }
+
     public void pushColumn(LiteralExpr keyValue, PrimitiveType keyType) {
         keys.add(keyValue);
         types.add(keyType);
@@ -271,6 +307,13 @@ public class PartitionKey implements Comparable<PartitionKey>, Writable {
                     case DATETIME:
                         literal = DateLiteral.read(in);
                         break;
+                    case CHAR:
+                    case VARCHAR:
+                        literal = StringLiteral.read(in);
+                        break;
+                    case BOOLEAN:
+                        literal = BoolLiteral.read(in);
+                        break;
                     default:
                         throw new IOException("type[" + type.name() + "] not supported: ");
                 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionType.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionType.java
index f35e200..f148f0f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/PartitionType.java
@@ -21,7 +21,8 @@ import org.apache.doris.thrift.TPartitionType;
 
 public enum PartitionType {
     UNPARTITIONED("UNPARTITIONED"),
-    RANGE("RANGE");
+    RANGE("RANGE"),
+    LIST("LIST");
 
     public String typeString;
 
@@ -35,6 +36,8 @@ public enum PartitionType {
                 return UNPARTITIONED;
             case RANGE_PARTITIONED:
                 return RANGE;
+            case LIST_PARTITIONED:
+                return LIST;
             default:
                 return UNPARTITIONED;
         }
@@ -46,6 +49,8 @@ public enum PartitionType {
                 return TPartitionType.UNPARTITIONED;
             case RANGE:
                 return TPartitionType.RANGE_PARTITIONED;
+            case LIST:
+                return TPartitionType.LIST_PARTITIONED;
             default:
                 return TPartitionType.UNPARTITIONED;
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/RangePartitionInfo.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/RangePartitionInfo.java
index c3ad7b8..71f1c7a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/RangePartitionInfo.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/RangePartitionInfo.java
@@ -18,7 +18,7 @@
 package org.apache.doris.catalog;
 
 import org.apache.doris.analysis.PartitionKeyDesc;
-import org.apache.doris.analysis.SingleRangePartitionDesc;
+import org.apache.doris.analysis.SinglePartitionDesc;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.common.FeMetaVersion;
@@ -26,12 +26,8 @@ import org.apache.doris.common.util.RangeUtils;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
 import com.google.common.collect.Range;
 
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
@@ -39,16 +35,8 @@ import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 public class RangePartitionInfo extends PartitionInfo {
-    private static final Logger LOG = LogManager.getLogger(RangePartitionInfo.class);
-
-    private List<Column> partitionColumns = Lists.newArrayList();
-    // formal partition id -> partition range
-    private Map<Long, Range<PartitionKey>> idToRange = Maps.newHashMap();
-    // temp partition id -> partition range
-    private Map<Long, Range<PartitionKey>> idToTempRange = Maps.newHashMap();
 
     public RangePartitionInfo() {
         // for persist
@@ -61,24 +49,7 @@ public class RangePartitionInfo extends PartitionInfo {
         this.isMultiColumnPartition = partitionColumns.size() > 1;
     }
 
-    public List<Column> getPartitionColumns() {
-        return partitionColumns;
-    }
-
-    @Override
-    public void dropPartition(long partitionId) {
-        super.dropPartition(partitionId);
-        idToRange.remove(partitionId);
-        idToTempRange.remove(partitionId);
-    }
-
-    public void addPartition(long partitionId, boolean isTemp, Range<PartitionKey> range, DataProperty dataProperty,
-                             short replicationNum, boolean isInMemory) {
-        addPartition(partitionId, dataProperty, replicationNum, isInMemory);
-        setRangeInternal(partitionId, isTemp, range);
-    }
-
-    public Range<PartitionKey> checkAndCreateRange(SingleRangePartitionDesc desc, boolean isTemp) throws DdlException {
+    public PartitionItem createAndCheckPartitionItem(SinglePartitionDesc desc, boolean isTemp) throws DdlException {
         Range<PartitionKey> newRange = null;
         PartitionKeyDesc partitionKeyDesc = desc.getPartitionKeyDesc();
         // check range
@@ -89,7 +60,7 @@ public class RangePartitionInfo extends PartitionInfo {
         }
 
         Preconditions.checkNotNull(newRange);
-        return newRange;
+        return new RangePartitionItem(newRange);
     }
 
     // create a new range and check it.
@@ -97,7 +68,7 @@ public class RangePartitionInfo extends PartitionInfo {
             throws AnalysisException, DdlException {
         Range<PartitionKey> newRange = null;
         // generate and sort the existing ranges
-        List<Map.Entry<Long, Range<PartitionKey>>> sortedRanges = getSortedRangeMap(isTemp);
+        List<Map.Entry<Long, PartitionItem>> sortedRanges = getSortedItemMap(isTemp);
 
         // create upper values for new range
         PartitionKey newRangeUpper = null;
@@ -112,8 +83,8 @@ public class RangePartitionInfo extends PartitionInfo {
 
         Range<PartitionKey> lastRange = null;
         Range<PartitionKey> currentRange = null;
-        for (Map.Entry<Long, Range<PartitionKey>> entry : sortedRanges) {
-            currentRange = entry.getValue();
+        for (Map.Entry<Long, PartitionItem> entry : sortedRanges) {
+            currentRange = entry.getValue().getItems();
             // check if equals to upper bound
             PartitionKey upperKey = currentRange.upperEndpoint();
             if (upperKey.compareTo(newRangeUpper) >= 0) {
@@ -156,53 +127,7 @@ public class RangePartitionInfo extends PartitionInfo {
         return newRange;
     }
 
-    public Range<PartitionKey> handleNewSinglePartitionDesc(SingleRangePartitionDesc desc, 
-            long partitionId, boolean isTemp) throws DdlException {
-        Preconditions.checkArgument(desc.isAnalyzed());
-        Range<PartitionKey> range = null;
-        try {
-            range = checkAndCreateRange(desc, isTemp);
-            setRangeInternal(partitionId, isTemp, range);
-        } catch (IllegalArgumentException e) {
-            // Range.closedOpen may throw this if (lower > upper)
-            throw new DdlException("Invalid key range: " + e.getMessage());
-        }
-        idToDataProperty.put(partitionId, desc.getPartitionDataProperty());
-        idToReplicationNum.put(partitionId, desc.getReplicationNum());
-        idToInMemory.put(partitionId, desc.isInMemory());
-        return range;
-    }
-
-    public void unprotectHandleNewSinglePartitionDesc(long partitionId, boolean isTemp, Range<PartitionKey> range,
-                                                      DataProperty dataProperty, short replicationNum,
-                                                      boolean isInMemory) {
-        setRangeInternal(partitionId, isTemp, range);
-        idToDataProperty.put(partitionId, dataProperty);
-        idToReplicationNum.put(partitionId, replicationNum);
-        idToInMemory.put(partitionId, isInMemory);
-    }
-
-    public void setRange(long partitionId, boolean isTemp, Range<PartitionKey> range) {
-        setRangeInternal(partitionId, isTemp, range);
-    }
-
-    public Map<Long, Range<PartitionKey>> getIdToRange(boolean isTemp) {
-        if (isTemp) {
-            return idToTempRange;
-        } else {
-            return idToRange;
-        }
-    }
-
-    public Range<PartitionKey> getRange(long partitionId) {
-        Range<PartitionKey> range = idToRange.get(partitionId);
-        if (range == null) {
-            range = idToTempRange.get(partitionId);
-        }
-        return range;
-    }
-
-    public static void checkRangeColumnType(Column column) throws AnalysisException {
+    public static void checkPartitionColumn(Column column) throws AnalysisException {
         PrimitiveType type = column.getDataType();
         if (!type.isFixedPointType() && !type.isDateType()) {
             throw new AnalysisException("Column[" + column.getName() + "] type[" + type
@@ -210,62 +135,26 @@ public class RangePartitionInfo extends PartitionInfo {
         }
     }
 
-    public List<Map.Entry<Long, Range<PartitionKey>>> getSortedRangeMap(boolean isTemp) {
-        Map<Long, Range<PartitionKey>> tmpMap = idToRange;
+    @Override
+    public List<Map.Entry<Long, PartitionItem>> getSortedItemMap(boolean isTemp) {
+        Map<Long, PartitionItem> tmpMap = idToItem;
         if (isTemp) {
-            tmpMap = idToTempRange;
+            tmpMap = idToTempItem;
         }
-        List<Map.Entry<Long, Range<PartitionKey>>> sortedList = Lists.newArrayList(tmpMap.entrySet());
+        List<Map.Entry<Long, PartitionItem>> sortedList = Lists.newArrayList(tmpMap.entrySet());
         Collections.sort(sortedList, RangeUtils.RANGE_MAP_ENTRY_COMPARATOR);
         return sortedList;
     }
 
-    // get a sorted range list, exclude partitions which ids are in 'excludePartitionIds'
-    public List<Range<PartitionKey>> getRangeList(Set<Long> excludePartitionIds, boolean isTemp) {
-        Map<Long, Range<PartitionKey>> tmpMap = idToRange;
-        if (isTemp) {
-            tmpMap = idToTempRange;
-        }
-        List<Range<PartitionKey>> resultList = Lists.newArrayList();
-        for (Map.Entry<Long, Range<PartitionKey>> entry : tmpMap.entrySet()) {
-            if (!excludePartitionIds.contains(entry.getKey())) {
-                resultList.add(entry.getValue());
-            }
-        }
-        return resultList;
-    }
-
-    // return any range intersect with the newRange.
-    // return null if no range intersect.
-    public Range<PartitionKey> getAnyIntersectRange(Range<PartitionKey> newRange, boolean isTemp) {
-        Map<Long, Range<PartitionKey>> tmpMap = idToRange;
-        if (isTemp) {
-            tmpMap = idToTempRange;
-        }
-        for (Range<PartitionKey> range : tmpMap.values()) {
-            if (range.isConnected(newRange)) {
-                Range<PartitionKey> intersection = range.intersection(newRange);
-                if (!intersection.isEmpty()) {
-                    return range;
-                }
-            }
-        }
-        return null;
-    }
-
-    private void setRangeInternal(long partitionId, boolean isTemp, Range<PartitionKey> range) {
-        if (isTemp) {
-            idToTempRange.put(partitionId, range);
-        } else {
-            idToRange.put(partitionId, range);
-        }
+    @Override
+    public void checkPartitionItemListsMatch(List<PartitionItem> list1, List<PartitionItem> list2)
+            throws DdlException {
+        RangeUtils.checkPartitionItemListsMatch(list1, list2);
     }
 
-    public void moveRangeFromTempToFormal(long tempPartitionId) {
-        Range<PartitionKey> range = idToTempRange.remove(tempPartitionId);
-        if (range != null) {
-            idToRange.put(tempPartitionId, range);
-        }
+    @Override
+    public void checkPartitionItemListsConflict(List<PartitionItem> list1, List<PartitionItem> list2) throws DdlException {
+        RangeUtils.checkRangeConflict(list1, list2);
     }
 
     public static PartitionInfo read(DataInput in) throws IOException {
@@ -284,16 +173,16 @@ public class RangePartitionInfo extends PartitionInfo {
             column.write(out);
         }
 
-        out.writeInt(idToRange.size());
-        for (Map.Entry<Long, Range<PartitionKey>> entry : idToRange.entrySet()) {
+        out.writeInt(idToItem.size());
+        for (Map.Entry<Long, PartitionItem> entry : idToItem.entrySet()) {
             out.writeLong(entry.getKey());
-            RangeUtils.writeRange(out, entry.getValue());
+            entry.getValue().write(out);
         }
 
-        out.writeInt(idToTempRange.size());
-        for (Map.Entry<Long, Range<PartitionKey>> entry : idToTempRange.entrySet()) {
+        out.writeInt(idToTempItem.size());
+        for (Map.Entry<Long, PartitionItem> entry : idToTempItem.entrySet()) {
             out.writeLong(entry.getKey());
-            RangeUtils.writeRange(out, entry.getValue());
+            entry.getValue().write(out);
         }
     }
 
@@ -312,7 +201,7 @@ public class RangePartitionInfo extends PartitionInfo {
         for (int i = 0; i < counter; i++) {
             long partitionId = in.readLong();
             Range<PartitionKey> range = RangeUtils.readRange(in);
-            idToRange.put(partitionId, range);
+            idToItem.put(partitionId, new RangePartitionItem(range));
         }
 
         if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_77) {
@@ -320,7 +209,7 @@ public class RangePartitionInfo extends PartitionInfo {
             for (int i = 0; i < counter; i++) {
                 long partitionId = in.readLong();
                 Range<PartitionKey> range = RangeUtils.readRange(in);
-                idToTempRange.put(partitionId, range);
+                idToTempItem.put(partitionId, new RangePartitionItem(range));
             }
         }
     }
@@ -340,15 +229,14 @@ public class RangePartitionInfo extends PartitionInfo {
         sb.append(")\n(");
 
         // sort range
-        List<Map.Entry<Long, Range<PartitionKey>>> entries =
-                new ArrayList<Map.Entry<Long, Range<PartitionKey>>>(this.idToRange.entrySet());
+        List<Map.Entry<Long, PartitionItem>> entries = new ArrayList<>(this.idToItem.entrySet());
         Collections.sort(entries, RangeUtils.RANGE_MAP_ENTRY_COMPARATOR);
 
         idx = 0;
-        for (Map.Entry<Long, Range<PartitionKey>> entry : entries) {
+        for (Map.Entry<Long, PartitionItem> entry : entries) {
             Partition partition = table.getPartition(entry.getKey());
             String partitionName = partition.getName();
-            Range<PartitionKey> range = entry.getValue();
+            Range<PartitionKey> range = entry.getValue().getItems();
 
             // print all partitions' range is fixed range, even if some of them is created by less than range
             sb.append("PARTITION ").append(partitionName).append(" VALUES [");
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/RangePartitionItem.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/RangePartitionItem.java
new file mode 100644
index 0000000..db3fe21
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/RangePartitionItem.java
@@ -0,0 +1,83 @@
+// 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.catalog;
+
+import org.apache.doris.common.util.RangeUtils;
+
+import com.google.common.collect.Range;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class RangePartitionItem extends PartitionItem {
+    private Range<PartitionKey> partitionKeyRange;
+    public static final Range<PartitionKey> DUMMY_ITEM;
+
+    static {
+        DUMMY_ITEM = Range.closed(new PartitionKey(), new PartitionKey());
+    }
+
+    public RangePartitionItem(Range<PartitionKey> range) {
+        this.partitionKeyRange = range;
+    }
+
+    public Range<PartitionKey> getItems() {
+        return partitionKeyRange;
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+        RangeUtils.writeRange(out, partitionKeyRange);
+    }
+
+    @Override
+    public int compareTo(PartitionItem other) {
+        if (partitionKeyRange.contains(other.getItems())) {
+            return 1;
+        }
+        if (partitionKeyRange.equals(((RangePartitionItem) other).getItems())) {
+            return 0;
+        }
+        return -1;
+    }
+
+    @Override
+    public PartitionItem getIntersect(PartitionItem newItem) {
+        Range<PartitionKey> newRange = newItem.getItems();
+        if (partitionKeyRange.isConnected(newRange)) {
+            Range<PartitionKey> intersection = partitionKeyRange.intersection(newRange);
+            if (!intersection.isEmpty()) {
+                return new RangePartitionItem(intersection);
+            }
+        }
+        return null;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (obj instanceof RangePartitionItem) {
+            return partitionKeyRange.equals(((RangePartitionItem) obj).getItems());
+        }
+        return false;
+    }
+
+    @Override
+    public String toString() {
+        return partitionKeyRange.toString();
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java
index 32b1872..4855554 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/clone/DynamicPartitionScheduler.java
@@ -23,15 +23,17 @@ import org.apache.doris.analysis.DropPartitionClause;
 import org.apache.doris.analysis.HashDistributionDesc;
 import org.apache.doris.analysis.PartitionKeyDesc;
 import org.apache.doris.analysis.PartitionValue;
-import org.apache.doris.analysis.SingleRangePartitionDesc;
+import org.apache.doris.analysis.SinglePartitionDesc;
 import org.apache.doris.catalog.Catalog;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.Database;
 import org.apache.doris.catalog.DynamicPartitionProperty;
 import org.apache.doris.catalog.HashDistributionInfo;
 import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.PartitionItem;
 import org.apache.doris.catalog.PartitionKey;
 import org.apache.doris.catalog.RangePartitionInfo;
+import org.apache.doris.catalog.RangePartitionItem;
 import org.apache.doris.catalog.Table;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.Config;
@@ -159,13 +161,13 @@ public class DynamicPartitionScheduler extends MasterDaemon {
                         db.getFullName(), olapTable.getName());
                 continue;
             }
-            for (Range<PartitionKey> partitionKeyRange : rangePartitionInfo.getIdToRange(false).values()) {
+            for (PartitionItem partitionItem : rangePartitionInfo.getIdToItem(false).values()) {
                 // only support single column partition now
                 try {
-                    RangeUtils.checkRangeIntersect(partitionKeyRange, addPartitionKeyRange);
+                    RangeUtils.checkRangeIntersect(partitionItem.getItems(), addPartitionKeyRange);
                 } catch (DdlException e) {
                     isPartitionExists = true;
-                    if (addPartitionKeyRange.equals(partitionKeyRange)) {
+                    if (addPartitionKeyRange.equals(partitionItem.getItems())) {
                         clearCreatePartitionFailedMsg(olapTable.getName());
                     } else {
                         recordCreatePartitionFailedMsg(db.getFullName(), olapTable.getName(), e.getMessage());
@@ -178,7 +180,7 @@ public class DynamicPartitionScheduler extends MasterDaemon {
             }
 
             // construct partition desc
-            PartitionKeyDesc partitionKeyDesc = new PartitionKeyDesc(Collections.singletonList(lowerValue), Collections.singletonList(upperValue));
+            PartitionKeyDesc partitionKeyDesc = PartitionKeyDesc.createFixed(Collections.singletonList(lowerValue), Collections.singletonList(upperValue));
             HashMap<String, String> partitionProperties = new HashMap<>(1);
             if (dynamicPartitionProperty.getReplicationNum() == DynamicPartitionProperty.NOT_SET_REPLICATION_NUM) {
                 partitionProperties.put("replication_num", String.valueOf(olapTable.getDefaultReplicationNum()));
@@ -187,7 +189,7 @@ public class DynamicPartitionScheduler extends MasterDaemon {
             }
             String partitionName = dynamicPartitionProperty.getPrefix() + DynamicPartitionUtil.getFormattedPartitionName(
                     dynamicPartitionProperty.getTimeZone(), prevBorder, dynamicPartitionProperty.getTimeUnit());
-            SingleRangePartitionDesc rangePartitionDesc = new SingleRangePartitionDesc(true, partitionName,
+            SinglePartitionDesc rangePartitionDesc = new SinglePartitionDesc(true, partitionName,
                     partitionKeyDesc, partitionProperties);
 
             // construct distribution desc
@@ -237,12 +239,12 @@ public class DynamicPartitionScheduler extends MasterDaemon {
         }
         RangePartitionInfo info = (RangePartitionInfo) (olapTable.getPartitionInfo());
 
-        List<Map.Entry<Long, Range<PartitionKey>>> idToRanges = new ArrayList<>(info.getIdToRange(false).entrySet());
-        idToRanges.sort(Comparator.comparing(o -> o.getValue().upperEndpoint()));
-        for (Map.Entry<Long, Range<PartitionKey>> idToRange : idToRanges) {
+        List<Map.Entry<Long, PartitionItem>> idToItems = new ArrayList<>(info.getIdToItem(false).entrySet());
+        idToItems.sort(Comparator.comparing(o -> ((RangePartitionItem) o.getValue()).getItems().upperEndpoint()));
+        for (Map.Entry<Long, PartitionItem> idToItem : idToItems) {
             try {
-                Long checkDropPartitionId = idToRange.getKey();
-                Range<PartitionKey> checkDropPartitionKey = idToRange.getValue();
+                Long checkDropPartitionId = idToItem.getKey();
+                Range<PartitionKey> checkDropPartitionKey = idToItem.getValue().getItems();
                 RangeUtils.checkRangeIntersect(reservePartitionKeyRange, checkDropPartitionKey);
                 if (checkDropPartitionKey.upperEndpoint().compareTo(reservePartitionKeyRange.lowerEndpoint()) <= 0) {
                     String dropPartitionName = olapTable.getPartition(checkDropPartitionId).getName();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/FeMetaVersion.java b/fe/fe-core/src/main/java/org/apache/doris/common/FeMetaVersion.java
index ad480e0..5f4d443 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/FeMetaVersion.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/FeMetaVersion.java
@@ -206,6 +206,8 @@ public final class FeMetaVersion {
     public static final int VERSION_96 = 96;
     // persist orig stmt of export job
     public static final int VERSION_97 = 97;
+    // add list partition
+    public static final int VERSION_98 = 98;
     // note: when increment meta version, should assign the latest version to VERSION_CURRENT
-    public static final int VERSION_CURRENT = VERSION_97;
+    public static final int VERSION_CURRENT = VERSION_98;
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/EsPartitionsProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/EsPartitionsProcDir.java
index 18c3fb8..b08444b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/EsPartitionsProcDir.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/EsPartitionsProcDir.java
@@ -90,7 +90,7 @@ public class EsPartitionsProcDir implements ProcDirInterface {
                     colNames.add(column.getName());
                 }
                 partitionInfo.add(joiner.join(colNames));  // partition key
-                partitionInfo.add(rangePartitionInfo.getRange(esShardPartitions.getPartitionId()).toString()); // range
+                partitionInfo.add(rangePartitionInfo.getItem(esShardPartitions.getPartitionId()).getItems().toString());// range
                 partitionInfo.add("-");  // dis
                 partitionInfo.add(esShardPartitions.getShardRoutings().size());  // shards
                 partitionInfo.add(1);  //  replica num
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/PartitionsProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/PartitionsProcDir.java
index 5bd6227..f56d7c1 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/PartitionsProcDir.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/PartitionsProcDir.java
@@ -33,7 +33,6 @@ import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.catalog.Partition;
 import org.apache.doris.catalog.PartitionInfo;
 import org.apache.doris.catalog.PartitionType;
-import org.apache.doris.catalog.RangePartitionInfo;
 import org.apache.doris.catalog.Table.TableType;
 import org.apache.doris.catalog.Type;
 import org.apache.doris.common.AnalysisException;
@@ -214,9 +213,8 @@ public class PartitionsProcDir implements ProcDirInterface {
 
             // for range partitions, we return partitions in ascending range order by default.
             // this is to be consistent with the behaviour before 0.12
-            if (tblPartitionInfo.getType() == PartitionType.RANGE) {
-                RangePartitionInfo rangePartitionInfo = (RangePartitionInfo) tblPartitionInfo;
-                partitionIds = rangePartitionInfo.getSortedRangeMap(isTempPartition).stream()
+            if (tblPartitionInfo.getType() == PartitionType.RANGE || tblPartitionInfo.getType() == PartitionType.LIST) {
+                partitionIds = tblPartitionInfo.getSortedItemMap(isTempPartition).stream()
                         .map(Map.Entry::getKey).collect(Collectors.toList());
             } else {
                 Collection<Partition> partitions = isTempPartition ? olapTable.getTempPartitions() : olapTable.getPartitions();
@@ -236,15 +234,15 @@ public class PartitionsProcDir implements ProcDirInterface {
                 partitionInfo.add(partition.getVisibleVersionHash());
                 partitionInfo.add(partition.getState());
 
-                if (tblPartitionInfo.getType() == PartitionType.RANGE) {
-                    // partition range info
-                    List<Column> partitionColumns = ((RangePartitionInfo) tblPartitionInfo).getPartitionColumns();
-                    List<String> colNames = new ArrayList<String>();
+                if (tblPartitionInfo.getType() == PartitionType.RANGE
+                        || tblPartitionInfo.getType() == PartitionType.LIST) {
+                    List<Column> partitionColumns = tblPartitionInfo.getPartitionColumns();
+                    List<String> colNames = new ArrayList<>();
                     for (Column column : partitionColumns) {
                         colNames.add(column.getName());
                     }
                     partitionInfo.add(joiner.join(colNames));
-                    partitionInfo.add(((RangePartitionInfo) tblPartitionInfo).getRange(partitionId).toString());
+                    partitionInfo.add(tblPartitionInfo.getItem(partitionId).getItems().toString());
                 } else {
                     partitionInfo.add("");
                     partitionInfo.add("");
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/ListUtil.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/ListUtil.java
index d5dc38c..cfe5a29 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/util/ListUtil.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/ListUtil.java
@@ -17,12 +17,26 @@
 
 package org.apache.doris.common.util;
 
+import org.apache.doris.catalog.ListPartitionItem;
+import org.apache.doris.catalog.PartitionItem;
+import org.apache.doris.catalog.PartitionKey;
+import org.apache.doris.common.DdlException;
+
 import com.google.common.base.Preconditions;
 
 import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
 import java.util.List;
+import java.util.Map;
 
 public class ListUtil {
+
+    public static final Comparator<Map.Entry<Long, PartitionItem>> LIST_MAP_ENTRY_COMPARATOR =
+            Comparator.comparing(o -> ((ListPartitionItem) o.getValue()).getItems().iterator().next());
+    public static final Comparator<PartitionItem> PARTITION_KEY_COMPARATOR =
+            Comparator.comparing(o -> ((ListPartitionItem) o).getItems().iterator().next());
+
     /**
      * split a list to multi expected number sublist
      * for example:
@@ -55,4 +69,74 @@ public class ListUtil {
 
         return result;
     }
+
+    public static void checkPartitionKeyListsMatch(List<PartitionItem> list1, List<PartitionItem> list2) throws DdlException {
+        Collections.sort(list1, PARTITION_KEY_COMPARATOR);
+        Collections.sort(list2, PARTITION_KEY_COMPARATOR);
+
+        int idx1 = 0;
+        int idx2 = 0;
+        List<PartitionKey> keys1 = new ArrayList<>();
+        List<PartitionKey> keys2 = new ArrayList<>();
+        keys1.addAll(list1.get(idx1).getItems());
+        keys2.addAll(list2.get(idx2).getItems());
+
+        while (true) {
+            int size = Math.min(keys1.size(), keys2.size());
+            for (int i = 0; i < size; i++) {
+                int res = keys1.get(i).compareTo(keys2.get(i));
+                if (res != 0) {
+                    throw new DdlException("2 partition key lists are not matched. "
+                            + keys1 + " vs. " + keys2);
+                }
+            }
+
+            int res = keys1.size() - keys2.size();
+            if (res == 0) {
+                ++idx1;
+                ++idx2;
+                if (idx1 == list1.size() || idx2 == list2.size()) {
+                    break;
+                }
+                keys1.clear();
+                keys2.clear();
+                keys1.addAll(list1.get(idx1).getItems());
+                keys2.addAll(list2.get(idx2).getItems());
+            } else if (res > 0) {
+                if (++idx2 == list2.size()) {
+                    break;
+                }
+                keys1.removeAll(keys2);
+                keys2.clear();
+                keys2.addAll(list2.get(idx2).getItems());
+            } else {
+                if (++idx1 == list1.size()) {
+                    break;
+                }
+                keys2.removeAll(keys1);
+                keys1.clear();
+                keys1.addAll(list1.get(idx1).getItems());
+            }
+        }
+
+        if (idx1 < list1.size() || idx2 < list2.size()) {
+            throw new DdlException("2 partition key lists are not matched. "
+                    + list1 + " vs. " + list2);
+        }
+    }
+
+    // Check if items in list2 conflicts with items in list1
+    public static void checkListsConflict(List<PartitionItem> list1, List<PartitionItem> list2) throws DdlException {
+        for (PartitionItem checkItem : list2) {
+            List<PartitionKey> checkKeys = checkItem.getItems();
+            for (PartitionKey checkKey : checkKeys) {
+                for (PartitionItem currentItem : list1) {
+                    if (((ListPartitionItem) currentItem).getItems().contains(checkKey)) {
+                        throw new DdlException("The partition key[" + checkKey.toSql() + "] is overlap with current " +
+                                currentItem.toString());
+                    }
+                }
+            }
+        }
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/RangeUtils.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/RangeUtils.java
index e91b94a..eeef22e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/util/RangeUtils.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/RangeUtils.java
@@ -17,7 +17,9 @@
 
 package org.apache.doris.common.util;
 
+import org.apache.doris.catalog.PartitionItem;
 import org.apache.doris.catalog.PartitionKey;
+import org.apache.doris.catalog.RangePartitionItem;
 import org.apache.doris.common.DdlException;
 
 import com.google.common.collect.BoundType;
@@ -35,11 +37,11 @@ import java.util.Map;
 
 public class RangeUtils {
     
-    public static final Comparator<Map.Entry<Long, Range<PartitionKey>>> RANGE_MAP_ENTRY_COMPARATOR =
-            Comparator.comparing(o -> o.getValue().lowerEndpoint());
+    public static final Comparator<Map.Entry<Long, PartitionItem>> RANGE_MAP_ENTRY_COMPARATOR =
+            Comparator.comparing(o -> (((RangePartitionItem)o.getValue()).getItems()).lowerEndpoint());
 
-    public static final Comparator<Range<PartitionKey>> RANGE_COMPARATOR = 
-            Comparator.comparing(o -> o.lowerEndpoint());
+    public static final Comparator<PartitionItem> RANGE_COMPARATOR =
+            Comparator.comparing(o -> ((RangePartitionItem) o).getItems().lowerEndpoint());
 
     public static void checkRangeIntersect(Range<PartitionKey> range1, Range<PartitionKey> range2) throws DdlException {
         if (range2.isConnected(range1)) {
@@ -74,14 +76,14 @@ public class RangeUtils {
      *      4.2 upper bounds (20 and 20) are equal.
      *  5. Not more next ranges, so 2 lists are equal.
      */
-    public static void checkRangeListsMatch(List<Range<PartitionKey>> list1, List<Range<PartitionKey>> list2) throws DdlException {
+    public static void checkPartitionItemListsMatch(List<PartitionItem> list1, List<PartitionItem> list2) throws DdlException {
         Collections.sort(list1, RangeUtils.RANGE_COMPARATOR);
         Collections.sort(list2, RangeUtils.RANGE_COMPARATOR);
 
         int idx1 = 0;
         int idx2 = 0;
-        Range<PartitionKey> range1 = list1.get(idx1);
-        Range<PartitionKey> range2 = list2.get(idx2);
+        Range<PartitionKey> range1 = list1.get(idx1).getItems();
+        Range<PartitionKey> range2 = list2.get(idx2).getItems();
         while (true) {
             if (range1.lowerEndpoint().compareTo(range2.lowerEndpoint()) != 0) {
                 throw new DdlException("2 range lists are not stricly matched. "
@@ -95,21 +97,21 @@ public class RangeUtils {
                 if (idx1 == list1.size() || idx2 == list2.size()) {
                     break;
                 }
-                range1 = list1.get(idx1);
-                range2 = list2.get(idx2);
+                range1 = list1.get(idx1).getItems();
+                range2 = list2.get(idx2).getItems();
                 continue;
             } else if (res > 0) {
                 if (++idx2 == list2.size()) {
                     break;
                 }
                 range1 = Range.closedOpen(range2.upperEndpoint(), range1.upperEndpoint());
-                range2 = list2.get(idx2);
+                range2 = list2.get(idx2).getItems();
             } else {
                 if (++idx1 == list1.size()) {
                     break;
                 }
                 range2 = Range.closedOpen(range1.upperEndpoint(), range2.upperEndpoint());
-                range1 = list1.get(idx1);
+                range1 = list1.get(idx1).getItems();
             }
         }
 
@@ -192,18 +194,18 @@ public class RangeUtils {
     }
 
     // check if any ranges in "rangesToBeChecked" conflict with ranges in "baseRanges".
-    public static void checkRangeConflict(List<Range<PartitionKey>> baseRanges,
-            List<Range<PartitionKey>> rangesToBeChecked) throws DdlException {
+    public static void checkRangeConflict(List<PartitionItem> baseRanges,
+            List<PartitionItem> rangesToBeChecked) throws DdlException {
 
         RangeMap<PartitionKey, Long> baseRangeMap = TreeRangeMap.create();
         long idx = 0;
-        for (Range<PartitionKey> baseRange : baseRanges) {
-            baseRangeMap.put(baseRange, idx++);
+        for (PartitionItem item : baseRanges) {
+            baseRangeMap.put(item.getItems(), idx++);
         }
 
-        for (Range<PartitionKey> range : rangesToBeChecked) {
-            if (!baseRangeMap.subRangeMap(range).asMapOfRanges().isEmpty()) {
-                throw new DdlException("Range: " + range + " conflicts with existing range");
+        for (PartitionItem item : rangesToBeChecked) {
+            if (!baseRangeMap.subRangeMap(item.getItems()).asMapOfRanges().isEmpty()) {
+                throw new DdlException("Range: " + item.getItems() + " conflicts with existing range");
             }
         }
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/external/elasticsearch/EsShardPartitions.java b/fe/fe-core/src/main/java/org/apache/doris/external/elasticsearch/EsShardPartitions.java
index b81dead..a7ceff2 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/external/elasticsearch/EsShardPartitions.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/external/elasticsearch/EsShardPartitions.java
@@ -17,7 +17,7 @@
 
 package org.apache.doris.external.elasticsearch;
 
-import org.apache.doris.analysis.SingleRangePartitionDesc;
+import org.apache.doris.analysis.SinglePartitionDesc;
 import org.apache.doris.catalog.PartitionKey;
 import org.apache.doris.thrift.TNetworkAddress;
 
@@ -40,7 +40,7 @@ public class EsShardPartitions {
     private final String indexName;
     // shardid -> host1, host2, host3
     private Map<Integer, List<EsShardRouting>> shardRoutings;
-    private SingleRangePartitionDesc partitionDesc;
+    private SinglePartitionDesc partitionDesc;
     private PartitionKey partitionKey;
     private long partitionId = -1;
 
@@ -127,11 +127,11 @@ public class EsShardPartitions {
         return shardRoutings;
     }
 
-    public SingleRangePartitionDesc getPartitionDesc() {
+    public SinglePartitionDesc getPartitionDesc() {
         return partitionDesc;
     }
 
-    public void setPartitionDesc(SingleRangePartitionDesc partitionDesc) {
+    public void setPartitionDesc(SinglePartitionDesc partitionDesc) {
         this.partitionDesc = partitionDesc;
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/external/elasticsearch/EsTablePartitions.java b/fe/fe-core/src/main/java/org/apache/doris/external/elasticsearch/EsTablePartitions.java
index d6bc5e3..43f7013 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/external/elasticsearch/EsTablePartitions.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/external/elasticsearch/EsTablePartitions.java
@@ -20,7 +20,7 @@ package org.apache.doris.external.elasticsearch;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.EsTable;
 import org.apache.doris.catalog.PartitionInfo;
-import org.apache.doris.catalog.PartitionKey;
+import org.apache.doris.catalog.PartitionItem;
 import org.apache.doris.catalog.RangePartitionInfo;
 import org.apache.doris.catalog.SinglePartitionInfo;
 import org.apache.doris.common.DdlException;
@@ -29,7 +29,6 @@ import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
 import com.google.common.collect.Maps;
-import com.google.common.collect.Range;
 
 import java.util.ArrayList;
 import java.util.Comparator;
@@ -96,13 +95,13 @@ public class EsTablePartitions {
             esShardPartitionsList.sort(Comparator.comparing(EsShardPartitions::getPartitionKey));
             long partitionId = 0;
             for (EsShardPartitions esShardPartitions : esShardPartitionsList) {
-                Range<PartitionKey> range = partitionInfo.handleNewSinglePartitionDesc(
+                PartitionItem item = partitionInfo.handleNewSinglePartitionDesc(
                         esShardPartitions.getPartitionDesc(), partitionId, false);
                 esTablePartitions.addPartition(esShardPartitions.getIndexName(), partitionId);
                 esShardPartitions.setPartitionId(partitionId);
                 ++partitionId;
                 LOG.debug("add partition to es table [{}] with range [{}]", esTable.getName(),
-                        range);
+                        item.getItems());
             }
         }
         return esTablePartitions;
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 4542ec2..b583c7b 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
@@ -149,9 +149,10 @@ public class DeleteHandler implements Writable {
                 }
 
                 if (noPartitionSpecified) {
-                    if (olapTable.getPartitionInfo().getType() == PartitionType.RANGE) {
+                    if (olapTable.getPartitionInfo().getType() == PartitionType.RANGE ||
+                            olapTable.getPartitionInfo().getType() == PartitionType.LIST) {
                         if (!ConnectContext.get().getSessionVariable().isDeleteWithoutPartition()) {
-                            throw new DdlException("This is a range partitioned table."
+                            throw new DdlException("This is a range or list partitioned table."
                                     + " You should specify partition in delete stmt, or set delete_without_partition to true");
                         } else {
                             partitionNames.addAll(olapTable.getPartitionNames());
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadPendingTask.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadPendingTask.java
index deb612d..5f4bd82 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadPendingTask.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadPendingTask.java
@@ -33,6 +33,7 @@ import org.apache.doris.catalog.HiveTable;
 import org.apache.doris.catalog.KeysType;
 import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.catalog.Partition;
+import org.apache.doris.catalog.PartitionItem;
 import org.apache.doris.catalog.PartitionKey;
 import org.apache.doris.catalog.PartitionType;
 import org.apache.doris.catalog.PrimitiveType;
@@ -339,7 +340,7 @@ public class SparkLoadPendingTask extends LoadTask {
                 partitionColumnRefs.add(column.getName());
             }
 
-            for (Map.Entry<Long, Range<PartitionKey>> entry : rangePartitionInfo.getSortedRangeMap(false)) {
+            for (Map.Entry<Long, PartitionItem> entry : rangePartitionInfo.getSortedItemMap(false)) {
                 long partitionId = entry.getKey();
                 if (!partitionIds.contains(partitionId)) {
                     continue;
@@ -354,7 +355,7 @@ public class SparkLoadPendingTask extends LoadTask {
                 int bucketNum = partition.getDistributionInfo().getBucketNum();
 
                 // is max partition
-                Range<PartitionKey> range = entry.getValue();
+                Range<PartitionKey> range = entry.getValue().getItems();
                 boolean isMaxPartition = range.upperEndpoint().isMaxValue();
 
                 // start keys
@@ -380,8 +381,7 @@ public class SparkLoadPendingTask extends LoadTask {
 
                 etlPartitions.add(new EtlPartition(partitionId, startKeys, endKeys, isMaxPartition, bucketNum));
             }
-        } else {
-            Preconditions.checkState(type == PartitionType.UNPARTITIONED);
+        } else if (type == PartitionType.UNPARTITIONED) {
             Preconditions.checkState(partitionIds.size() == 1);
 
             for (Long partitionId : partitionIds) {
@@ -394,8 +394,10 @@ public class SparkLoadPendingTask extends LoadTask {
                 int bucketNum = partition.getDistributionInfo().getBucketNum();
 
                 etlPartitions.add(new EtlPartition(partitionId, Lists.newArrayList(), Lists.newArrayList(),
-                                                   true, bucketNum));
+                        true, bucketNum));
             }
+        } else {
+            throw new LoadException("Spark Load does not support list partition yet");
         }
 
         // distribution column refs
diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/PartitionPersistInfo.java b/fe/fe-core/src/main/java/org/apache/doris/persist/PartitionPersistInfo.java
index 5b4fd3f..d482e79 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/persist/PartitionPersistInfo.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/persist/PartitionPersistInfo.java
@@ -19,7 +19,9 @@ package org.apache.doris.persist;
 
 import org.apache.doris.catalog.Catalog;
 import org.apache.doris.catalog.DataProperty;
+import org.apache.doris.catalog.ListPartitionItem;
 import org.apache.doris.catalog.Partition;
+import org.apache.doris.catalog.PartitionItem;
 import org.apache.doris.catalog.PartitionKey;
 import org.apache.doris.common.FeMetaVersion;
 import org.apache.doris.common.io.Writable;
@@ -37,6 +39,7 @@ public class PartitionPersistInfo implements Writable {
     private Partition partition;
 
     private Range<PartitionKey> range;
+    private PartitionItem listPartitionItem;
     private DataProperty dataProperty;
     private short replicationNum;
     private boolean isInMemory = false;
@@ -46,13 +49,14 @@ public class PartitionPersistInfo implements Writable {
     }
 
     public PartitionPersistInfo(long dbId, long tableId, Partition partition, Range<PartitionKey> range,
-                                DataProperty dataProperty, short replicationNum,
+                                PartitionItem listPartitionItem, DataProperty dataProperty, short replicationNum,
                                 boolean isInMemory, boolean isTempPartition) {
         this.dbId = dbId;
         this.tableId = tableId;
         this.partition = partition;
 
         this.range = range;
+        this.listPartitionItem = listPartitionItem;
         this.dataProperty = dataProperty;
 
         this.replicationNum = replicationNum;
@@ -76,6 +80,10 @@ public class PartitionPersistInfo implements Writable {
         return range;
     }
 
+    public PartitionItem getListPartitionItem() {
+        return listPartitionItem;
+    }
+
     public DataProperty getDataProperty() {
         return dataProperty;
     }
@@ -98,6 +106,7 @@ public class PartitionPersistInfo implements Writable {
         partition.write(out);
 
         RangeUtils.writeRange(out, range);
+        listPartitionItem.write(out);
         dataProperty.write(out);
         out.writeShort(replicationNum);
         out.writeBoolean(isInMemory);
@@ -110,6 +119,12 @@ public class PartitionPersistInfo implements Writable {
         partition = Partition.read(in);
 
         range = RangeUtils.readRange(in);
+        if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_98) {
+            listPartitionItem = ListPartitionItem.read(in);
+        } else {
+            listPartitionItem = ListPartitionItem.DUMMY_ITEM;
+        }
+
         dataProperty = DataProperty.read(in);
         replicationNum = in.readShort();
         if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_72) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/DataSplitSink.java b/fe/fe-core/src/main/java/org/apache/doris/planner/DataSplitSink.java
index 5da1f09..217f53d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/DataSplitSink.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/DataSplitSink.java
@@ -334,7 +334,7 @@ public class DataSplitSink extends DataSink {
             Column partitionColumn = rangePartitionInfo.getPartitionColumns().get(0);
             PrimitiveType columnType = partitionColumn.getDataType();
 
-            Range<PartitionKey> range = rangePartitionInfo.getRange(partitionId);
+            Range<PartitionKey> range = rangePartitionInfo.getItem(partitionId).getItems();
             // start key
             EtlPartitionKey startPartitionKey = null;
             PartitionKey startKey = range.lowerEndpoint();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/EsScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/EsScanNode.java
index e662de0..6853459 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/EsScanNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/EsScanNode.java
@@ -23,7 +23,7 @@ import org.apache.doris.analysis.TupleDescriptor;
 import org.apache.doris.catalog.Catalog;
 import org.apache.doris.catalog.EsTable;
 import org.apache.doris.catalog.PartitionInfo;
-import org.apache.doris.catalog.PartitionKey;
+import org.apache.doris.catalog.PartitionItem;
 import org.apache.doris.catalog.RangePartitionInfo;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.UserException;
@@ -48,7 +48,6 @@ import com.google.common.collect.HashMultimap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Multimap;
-import com.google.common.collect.Range;
 import com.google.common.collect.Sets;
 
 import java.util.ArrayList;
@@ -286,7 +285,7 @@ public class EsScanNode extends ScanNode {
         switch (partitionInfo.getType()) {
             case RANGE: {
                 RangePartitionInfo rangePartitionInfo = (RangePartitionInfo) partitionInfo;
-                Map<Long, Range<PartitionKey>> keyRangeById = rangePartitionInfo.getIdToRange(false);
+                Map<Long, PartitionItem> keyRangeById = rangePartitionInfo.getIdToItem(false);
                 partitionPruner = new RangePartitionPruner(keyRangeById, rangePartitionInfo.getPartitionColumns(),
                         columnFilters);
                 return partitionPruner.prune();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/ListPartitionPruner.java b/fe/fe-core/src/main/java/org/apache/doris/planner/ListPartitionPruner.java
index 9c840a8..a9eba85 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/ListPartitionPruner.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/ListPartitionPruner.java
@@ -20,6 +20,7 @@ package org.apache.doris.planner;
 import org.apache.doris.analysis.InPredicate;
 import org.apache.doris.analysis.LiteralExpr;
 import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.PartitionItem;
 import org.apache.doris.catalog.PartitionKey;
 import org.apache.doris.catalog.Type;
 import org.apache.doris.common.AnalysisException;
@@ -28,27 +29,21 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Range;
 import com.google.common.collect.Sets;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-/*
- * (hujie, cmy)
- * ATTN: do not delete it before considering useless for certain
+/**
+ * list partition pruner
  */
-@Deprecated
 public class ListPartitionPruner implements PartitionPruner {
-    private static final Logger LOG = LoggerFactory.getLogger(ListPartitionPruner.class);
 
-    private Map<Long, List<PartitionKey>> partitionListMap;
+    private Map<Long, PartitionItem> partitionListMap;
     private List<Column>                       partitionColumns;
     private Map<String, PartitionColumnFilter> partitionColumnFilters;
 
-    public ListPartitionPruner(Map<Long, List<PartitionKey>> listMap,
+    public ListPartitionPruner(Map<Long, PartitionItem> listMap,
                                List<Column> columns,
                                Map<String, PartitionColumnFilter> filters) {
         partitionListMap = listMap;
@@ -56,31 +51,99 @@ public class ListPartitionPruner implements PartitionPruner {
         partitionColumnFilters = filters;
     }
 
-    private Collection<Long> pruneListMap(
-            Map<Long, List<PartitionKey>> listMap,
-            Range<PartitionKey> range) {
+    private Collection<Long> pruneListMap(Map<Long, PartitionItem> listMap,
+                                          Range<PartitionKey> range,
+                                          int columnId) {
         Set<Long> resultSet = Sets.newHashSet();
-        for (Map.Entry<Long, List<PartitionKey>> entry : listMap.entrySet()) {
-            for (PartitionKey key : entry.getValue()) {
-                if (range.contains(key)) {
+        for (Map.Entry<Long, PartitionItem> entry : listMap.entrySet()) {
+            List<PartitionKey> partitionKeys = entry.getValue().getItems();
+            for (PartitionKey partitionKey : partitionKeys) {
+                LiteralExpr expr = partitionKey.getKeys().get(columnId);
+                if (contain(range, expr, columnId)) {
                     resultSet.add(entry.getKey());
-                    break;
                 }
             }
         }
         return resultSet;
     }
 
+    /**
+     * check literal expr exist in partition range
+     * @param range the partition key range
+     * @param literalExpr expr to be checked
+     * @param columnId expr column index in partition key
+     * @return
+     */
+    private boolean contain(Range<PartitionKey> range, LiteralExpr literalExpr, int columnId) {
+        LiteralExpr lowerExpr = range.lowerEndpoint().getKeys().get(columnId);
+        LiteralExpr upperExpr = range.upperEndpoint().getKeys().get(columnId);
+        BoundType lType = range.lowerBoundType();
+        BoundType uType = range.upperBoundType();
+        int ret1 = PartitionKey.compareLiteralExpr(literalExpr, lowerExpr);
+        int ret2 = PartitionKey.compareLiteralExpr(literalExpr, upperExpr);
+
+        if (lType == BoundType.CLOSED && uType == BoundType.CLOSED) {
+            if (ret1 >= 0 && ret2 <= 0) {
+                return true;
+            }
+        } else if (lType == BoundType.CLOSED && uType == BoundType.OPEN) {
+            if (ret1 >= 0 && ret2 < 0) {
+                return true;
+            }
+        } else if (lType == BoundType.OPEN && uType == BoundType.CLOSED) {
+            if (ret1 > 0 && ret2 <= 0) {
+                return true;
+            }
+        } else if (lType == BoundType.OPEN && uType == BoundType.OPEN) {
+            if (ret1 > 0 && ret2 < 0) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    /**
+     * get min literal expr from partition key list map by partition key column id.
+     * @param columnId
+     * @return
+     */
+    private LiteralExpr getMinLiteral(int columnId) {
+        LiteralExpr minLiteral = null;
+        for (Map.Entry<Long, PartitionItem> entry : partitionListMap.entrySet()) {
+            List<PartitionKey> partitionKeys = entry.getValue().getItems();
+            for (PartitionKey partitionKey : partitionKeys) {
+                minLiteral = getMinExpr(partitionKey.getKeys().get(columnId), minLiteral);
+            }
+        }
+        return minLiteral;
+    }
+
+    private LiteralExpr getMinExpr(LiteralExpr expr, LiteralExpr minLiteral) {
+        if (minLiteral == null) {
+            minLiteral = expr;
+            return minLiteral;
+        }
+        if (expr.compareLiteral(minLiteral) < 0) {
+            minLiteral = expr;
+        }
+        return minLiteral;
+    }
+
     private Collection<Long> prune(
-            Map<Long, List<PartitionKey>> listMap,
+            Map<Long, PartitionItem> listMap,
             int columnId,
             PartitionKey minKey,
             PartitionKey maxKey,
             int complex)
             throws AnalysisException {
+        // if partition item map is empty, no need to prune.
+        if (listMap.size() == 0) {
+            return Lists.newArrayList();
+        }
+
         if (columnId == partitionColumns.size()) {
             try {
-                return pruneListMap(listMap, Range.closed(minKey, maxKey));
+                return pruneListMap(listMap, Range.closed(minKey, maxKey), columnId - 1);
             } catch (IllegalArgumentException e) {
                 return Lists.newArrayList();
             }
@@ -89,13 +152,13 @@ public class ListPartitionPruner implements PartitionPruner {
         PartitionColumnFilter filter = partitionColumnFilters.get(keyColumn.getName());
         // no filter in this column
         if (null == filter) {
-            minKey.pushColumn(LiteralExpr.createInfinity(Type.fromPrimitiveType(keyColumn.getDataType()), false),
-                    keyColumn.getDataType());
+            minKey.pushColumn(getMinLiteral(columnId), keyColumn.getDataType());
             maxKey.pushColumn(LiteralExpr.createInfinity(Type.fromPrimitiveType(keyColumn.getDataType()), true),
-                    keyColumn.getDataType());
+                                keyColumn.getDataType());
             Collection<Long> result = null;
             try {
-                return pruneListMap(listMap, Range.closed(minKey, maxKey));
+                // prune next partition column
+                result = prune(listMap, columnId + 1, minKey, maxKey, complex);
             } catch (IllegalArgumentException e) {
                 result = Lists.newArrayList();
             }
@@ -105,13 +168,18 @@ public class ListPartitionPruner implements PartitionPruner {
         }
         InPredicate inPredicate = filter.getInPredicate();
         if (null == inPredicate || inPredicate.getChildren().size() * complex > 100) {
+            // case: where k1 = 1;
             if (filter.lowerBoundInclusive && filter.upperBoundInclusive 
                     && filter.lowerBound != null && filter.upperBound != null 
                     && 0 == filter.lowerBound.compareLiteral(filter.upperBound)) {
                 minKey.pushColumn(filter.lowerBound, keyColumn.getDataType());
                 maxKey.pushColumn(filter.upperBound, keyColumn.getDataType());
-                Collection<Long> result =
-                        prune(listMap, columnId + 1, minKey, maxKey, complex);
+                // handle like in predicate
+                Collection<Long> result = pruneListMap(listMap, Range.closed(minKey, maxKey), columnId);
+                // prune next partition column
+                if (partitionColumns.size() > 1) {
+                    result.retainAll(prune(listMap, columnId + 1, minKey, maxKey, complex));
+                }
                 minKey.popColumn();
                 maxKey.popColumn();
                 return result;
@@ -124,25 +192,13 @@ public class ListPartitionPruner implements PartitionPruner {
             int lastColumnId = partitionColumns.size() - 1;
             if (filter.lowerBound != null) {
                 minKey.pushColumn(filter.lowerBound, keyColumn.getDataType());
-                if (filter.lowerBoundInclusive && columnId != lastColumnId) {
-                    Column column = partitionColumns.get(columnId + 1);
-                    minKey.pushColumn(LiteralExpr.createInfinity(Type.fromPrimitiveType(column.getDataType()), false),
-                            column.getDataType());
-                    isPushMin = true;
-                }
             } else {
-                minKey.pushColumn(LiteralExpr.createInfinity(Type.fromPrimitiveType(keyColumn.getDataType()), false),
+                minKey.pushColumn(getMinLiteral(columnId),
                         keyColumn.getDataType());
                 isPushMin = true;
             }
             if (filter.upperBound != null) {
                 maxKey.pushColumn(filter.upperBound, keyColumn.getDataType());
-                if (filter.upperBoundInclusive && columnId != lastColumnId) {
-                    Column column = partitionColumns.get(columnId + 1);
-                    maxKey.pushColumn(LiteralExpr.createInfinity(Type.fromPrimitiveType(column.getDataType()), true),
-                            column.getDataType());
-                    isPushMax = true;
-                }
             } else {
                 maxKey.pushColumn(LiteralExpr.createInfinity(Type.fromPrimitiveType(keyColumn.getDataType()), true),
                         keyColumn.getDataType());
@@ -151,7 +207,12 @@ public class ListPartitionPruner implements PartitionPruner {
 
             Collection<Long> result = null;
             try {
-                return pruneListMap(listMap, Range.range(minKey, lowerType, maxKey, upperType));
+                result = pruneListMap(listMap, Range.range(minKey, lowerType, maxKey, upperType), columnId);
+                // prune next partition column
+                if (partitionColumns.size() > 1) {
+                    result.retainAll(prune(listMap, columnId + 1, minKey, maxKey, complex));
+                }
+                return result;
             } catch (IllegalArgumentException e) {
                 result = Lists.newArrayList();
             }
@@ -170,7 +231,13 @@ public class ListPartitionPruner implements PartitionPruner {
             LiteralExpr expr = (LiteralExpr) inPredicate.getChild(i);
             minKey.pushColumn(expr, keyColumn.getDataType());
             maxKey.pushColumn(expr, keyColumn.getDataType());
-            resultSet.addAll(prune(listMap, columnId + 1, minKey, maxKey, complex));
+            Collection<Long> result = pruneListMap(listMap, Range.closed(minKey, maxKey), columnId);
+            // prune next partition column
+            if (partitionColumns.size() > 1) {
+                // Take the intersection
+                result.retainAll(prune(listMap, columnId + 1, minKey, maxKey, complex));
+            }
+            resultSet.addAll(result);
             minKey.popColumn();
             maxKey.popColumn();
         }
@@ -183,5 +250,3 @@ public class ListPartitionPruner implements PartitionPruner {
         return prune(partitionListMap, 0, minKey, maxKey, 1);
     }
 }
-
-/* vim: set expandtab ts=4 sw=4 sts=4 tw=100: */
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 3ec9866..5628e8a 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
@@ -39,9 +39,8 @@ import org.apache.doris.catalog.MaterializedIndexMeta;
 import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.catalog.Partition;
 import org.apache.doris.catalog.PartitionInfo;
-import org.apache.doris.catalog.PartitionKey;
+import org.apache.doris.catalog.PartitionItem;
 import org.apache.doris.catalog.PartitionType;
-import org.apache.doris.catalog.RangePartitionInfo;
 import org.apache.doris.catalog.Replica;
 import org.apache.doris.catalog.Tablet;
 import org.apache.doris.common.AnalysisException;
@@ -71,7 +70,6 @@ import com.google.common.base.Preconditions;
 import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
-import com.google.common.collect.Range;
 
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -333,22 +331,28 @@ public class OlapScanNode extends ScanNode {
         cardinality = cardinality == -1 ? 0 : cardinality;
     }
 
-    private Collection<Long> partitionPrune(RangePartitionInfo partitionInfo, PartitionNames partitionNames) throws AnalysisException {
-        Map<Long, Range<PartitionKey>> keyRangeById = null;
+    private Collection<Long> partitionPrune(PartitionInfo partitionInfo, PartitionNames partitionNames) throws AnalysisException {
+        PartitionPruner partitionPruner = null;
+        Map<Long, PartitionItem> keyItemMap;
         if (partitionNames != null) {
-            keyRangeById = Maps.newHashMap();
+            keyItemMap = Maps.newHashMap();
             for (String partName : partitionNames.getPartitionNames()) {
-                Partition part = olapTable.getPartition(partName, partitionNames.isTemp());
-                if (part == null) {
+                Partition partition = olapTable.getPartition(partName, partitionNames.isTemp());
+                if (partition == null) {
                     ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_SUCH_PARTITION, partName);
                 }
-                keyRangeById.put(part.getId(), partitionInfo.getRange(part.getId()));
+                keyItemMap.put(partition.getId(), partitionInfo.getItem(partition.getId()));
             }
         } else {
-            keyRangeById = partitionInfo.getIdToRange(false);
+            keyItemMap = partitionInfo.getIdToItem(false);
+        }
+        if (partitionInfo.getType() == PartitionType.RANGE) {
+            partitionPruner = new RangePartitionPruner(keyItemMap,
+                    partitionInfo.getPartitionColumns(), columnFilters);
+        } else if (partitionInfo.getType() == PartitionType.LIST) {
+            partitionPruner = new ListPartitionPruner(keyItemMap,
+                    partitionInfo.getPartitionColumns(), columnFilters);
         }
-        PartitionPruner partitionPruner = new RangePartitionPruner(keyRangeById,
-                partitionInfo.getPartitionColumns(), columnFilters);
         return partitionPruner.prune();
     }
 
@@ -464,8 +468,8 @@ public class OlapScanNode extends ScanNode {
         // Step1: compute partition ids
         PartitionNames partitionNames = ((BaseTableRef) desc.getRef()).getPartitionNames();
         PartitionInfo partitionInfo = olapTable.getPartitionInfo();
-        if (partitionInfo.getType() == PartitionType.RANGE) {
-            selectedPartitionIds = partitionPrune((RangePartitionInfo) partitionInfo, partitionNames);
+        if (partitionInfo.getType() == PartitionType.RANGE || partitionInfo.getType() == PartitionType.LIST) {
+            selectedPartitionIds = partitionPrune(partitionInfo, partitionNames);
         } else {
             selectedPartitionIds = null;
         }
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 0d9d4d1..8ca0cca 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
@@ -24,14 +24,17 @@ import org.apache.doris.catalog.Catalog;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.DistributionInfo;
 import org.apache.doris.catalog.HashDistributionInfo;
+import org.apache.doris.catalog.ListPartitionItem;
 import org.apache.doris.catalog.MaterializedIndex;
 import org.apache.doris.catalog.MaterializedIndex.IndexExtState;
 import org.apache.doris.catalog.MaterializedIndexMeta;
 import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.catalog.Partition;
+import org.apache.doris.catalog.PartitionInfo;
+import org.apache.doris.catalog.PartitionItem;
 import org.apache.doris.catalog.PartitionKey;
 import org.apache.doris.catalog.PartitionType;
-import org.apache.doris.catalog.RangePartitionInfo;
+import org.apache.doris.catalog.RangePartitionItem;
 import org.apache.doris.catalog.Tablet;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.DdlException;
@@ -45,6 +48,7 @@ import org.apache.doris.system.SystemInfoService;
 import org.apache.doris.thrift.TDataSink;
 import org.apache.doris.thrift.TDataSinkType;
 import org.apache.doris.thrift.TExplainLevel;
+import org.apache.doris.thrift.TExprNode;
 import org.apache.doris.thrift.TNodeInfo;
 import org.apache.doris.thrift.TOlapTableIndexSchema;
 import org.apache.doris.thrift.TOlapTableIndexTablets;
@@ -67,6 +71,7 @@ import org.apache.commons.collections.CollectionUtils;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.stream.Collectors;
@@ -213,32 +218,22 @@ public class OlapTableSink extends DataSink {
 
         PartitionType partType = table.getPartitionInfo().getType();
         switch (partType) {
+            case LIST:
             case RANGE: {
-                RangePartitionInfo rangePartitionInfo = (RangePartitionInfo) table.getPartitionInfo();
-                for (Column partCol : rangePartitionInfo.getPartitionColumns()) {
+                PartitionInfo partitionInfo = table.getPartitionInfo();
+                for (Column partCol : partitionInfo.getPartitionColumns()) {
                     partitionParam.addToPartitionColumns(partCol.getName());
                 }
 
-                int partColNum = rangePartitionInfo.getPartitionColumns().size();
+                int partColNum = partitionInfo.getPartitionColumns().size();
                 DistributionInfo selectedDistInfo = null;
 
                 for (Long partitionId : partitionIds) {
                     Partition partition = table.getPartition(partitionId);
                     TOlapTablePartition tPartition = new TOlapTablePartition();
                     tPartition.setId(partition.getId());
-                    Range<PartitionKey> range = rangePartitionInfo.getRange(partition.getId());
-                    // set start keys
-                    if (range.hasLowerBound() && !range.lowerEndpoint().isMinValue()) {
-                        for (int i = 0; i < partColNum; i++) {
-                            tPartition.addToStartKeys(range.lowerEndpoint().getKeys().get(i).treeToThrift().getNodes().get(0));
-                        }
-                    }
-                    // set end keys
-                    if (range.hasUpperBound() && !range.upperEndpoint().isMaxValue()) {
-                        for (int i = 0; i < partColNum; i++) {
-                            tPartition.addToEndKeys(range.upperEndpoint().getKeys().get(i).treeToThrift().getNodes().get(0));
-                        }
-                    }
+                    // set partition keys
+                    setPartitionKeys(tPartition, partitionInfo.getItem(partition.getId()), partColNum);
 
                     for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.ALL)) {
                         tPartition.addToIndexes(new TOlapTableIndexTablets(index.getId(), Lists.newArrayList(
@@ -287,6 +282,34 @@ public class OlapTableSink extends DataSink {
         return partitionParam;
     }
 
+    private void setPartitionKeys(TOlapTablePartition tPartition, PartitionItem partitionItem, int partColNum) {
+        if (partitionItem instanceof RangePartitionItem) {
+            Range<PartitionKey> range = partitionItem.getItems();
+            // set start keys
+            if (range.hasLowerBound() && !range.lowerEndpoint().isMinValue()) {
+                for (int i = 0; i < partColNum; i++) {
+                    tPartition.addToStartKeys(range.lowerEndpoint().getKeys().get(i).treeToThrift().getNodes().get(0));
+                }
+            }
+            // set end keys
+            if (range.hasUpperBound() && !range.upperEndpoint().isMaxValue()) {
+                for (int i = 0; i < partColNum; i++) {
+                    tPartition.addToEndKeys(range.upperEndpoint().getKeys().get(i).treeToThrift().getNodes().get(0));
+                }
+            }
+        } else if (partitionItem instanceof ListPartitionItem){
+            List<PartitionKey> partitionKeys = partitionItem.getItems();
+            // set in keys
+            for (PartitionKey partitionKey : partitionKeys) {
+                List<TExprNode> tExprNodes = new ArrayList<>();
+                for (int i = 0; i < partColNum; i++) {
+                    tExprNodes.add(partitionKey.getKeys().get(i).treeToThrift().getNodes().get(0));
+                }
+                tPartition.addToInKeys(tExprNodes);
+            }
+        }
+    }
+
     private TOlapTableLocationParam createLocation(OlapTable table) throws UserException {
         TOlapTableLocationParam locationParam = new TOlapTableLocationParam();
         // BE id -> path hash
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/RangePartitionPruner.java b/fe/fe-core/src/main/java/org/apache/doris/planner/RangePartitionPruner.java
index d91ae08..8568599 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/RangePartitionPruner.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/RangePartitionPruner.java
@@ -21,6 +21,7 @@ import org.apache.doris.analysis.InPredicate;
 import org.apache.doris.analysis.LiteralExpr;
 import org.apache.doris.analysis.NullLiteral;
 import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.PartitionItem;
 import org.apache.doris.catalog.PartitionKey;
 import org.apache.doris.catalog.Type;
 import org.apache.doris.common.AnalysisException;
@@ -43,11 +44,11 @@ import java.util.Set;
 public class RangePartitionPruner implements PartitionPruner {
     private static final Logger LOG = LogManager.getLogger(RangePartitionPruner.class);
 
-    private Map<Long, Range<PartitionKey>> partitionRangeMap;
+    private Map<Long, PartitionItem> partitionRangeMap;
     private List<Column> partitionColumns;
     private Map<String, PartitionColumnFilter> partitionColumnFilters;
 
-    public RangePartitionPruner(Map<Long, Range<PartitionKey>> rangeMap,
+    public RangePartitionPruner(Map<Long, PartitionItem> rangeMap,
                                 List<Column> columns,
                                 Map<String, PartitionColumnFilter> filters) {
         partitionRangeMap = rangeMap;
@@ -186,8 +187,8 @@ public class RangePartitionPruner implements PartitionPruner {
         PartitionKey maxKey = new PartitionKey();
         // Map to RangeMapTree
         RangeMap<PartitionKey, Long> rangeMap = TreeRangeMap.create();
-        for (Map.Entry<Long, Range<PartitionKey>> entry : partitionRangeMap.entrySet()) {
-            rangeMap.put(entry.getValue(), entry.getKey());
+        for (Map.Entry<Long, PartitionItem> entry : partitionRangeMap.entrySet()) {
+            rangeMap.put(entry.getValue().getItems(), entry.getKey());
         }
         return prune(rangeMap, 0, minKey, maxKey, 1);
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/StreamLoadPlanner.java b/fe/fe-core/src/main/java/org/apache/doris/planner/StreamLoadPlanner.java
index 32a1b34..39c0f5d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/planner/StreamLoadPlanner.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/planner/StreamLoadPlanner.java
@@ -30,9 +30,9 @@ import org.apache.doris.catalog.Database;
 import org.apache.doris.catalog.KeysType;
 import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.catalog.Partition;
-import org.apache.doris.catalog.PartitionKey;
+import org.apache.doris.catalog.PartitionInfo;
+import org.apache.doris.catalog.PartitionItem;
 import org.apache.doris.catalog.PartitionType;
-import org.apache.doris.catalog.RangePartitionInfo;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.common.ErrorCode;
@@ -54,7 +54,6 @@ import org.apache.doris.thrift.TUniqueId;
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
-import com.google.common.collect.Range;
 
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -219,11 +218,11 @@ public class StreamLoadPlanner {
             }
         } else {
             List<Expr> conjuncts = scanNode.getConjuncts();
-            if (destTable.getPartitionInfo().getType() == PartitionType.RANGE && !conjuncts.isEmpty()) {
-                RangePartitionInfo rangePartitionInfo = (RangePartitionInfo)(destTable.getPartitionInfo());
-                Map<Long, Range<PartitionKey>> keyRangeById = rangePartitionInfo.getIdToRange(false);
+            if (destTable.getPartitionInfo().getType() != PartitionType.UNPARTITIONED && !conjuncts.isEmpty()) {
+                PartitionInfo partitionInfo = destTable.getPartitionInfo();
+                Map<Long, PartitionItem> itemById = partitionInfo.getIdToItem(false);
                 Map<String, PartitionColumnFilter> columnFilters = Maps.newHashMap();
-                for (Column column : rangePartitionInfo.getPartitionColumns()) {
+                for (Column column : partitionInfo.getPartitionColumns()) {
                     SlotDescriptor slotDesc = tupleDesc.getColumnSlot(column.getName());
                     if (null == slotDesc) {
                         continue;
@@ -234,10 +233,16 @@ public class StreamLoadPlanner {
                     }
                 }
                 if (columnFilters.isEmpty()) {
-                    partitionIds.addAll(keyRangeById.keySet());
+                    partitionIds.addAll(itemById.keySet());
                 } else {
-                    PartitionPruner partitionPruner = new RangePartitionPruner(keyRangeById,
-                            rangePartitionInfo.getPartitionColumns(), columnFilters);
+                    PartitionPruner partitionPruner = null;
+                    if (destTable.getPartitionInfo().getType() == PartitionType.RANGE) {
+                        partitionPruner = new RangePartitionPruner(itemById,
+                                partitionInfo.getPartitionColumns(), columnFilters);
+                    } else if (destTable.getPartitionInfo().getType() == PartitionType.LIST) {
+                        partitionPruner = new ListPartitionPruner(itemById,
+                                partitionInfo.getPartitionColumns(), columnFilters);
+                    }
                     partitionIds.addAll(partitionPruner.prune());
                 }
             } else {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java
index 8a30f1e..f3c41c1 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java
@@ -50,11 +50,11 @@ import org.apache.doris.thrift.TMasterOpRequest;
 import org.apache.doris.thrift.TMasterOpResult;
 import org.apache.doris.thrift.TUniqueId;
 
+import com.google.common.base.Strings;
+
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
-import com.google.common.base.Strings;
-
 import java.io.IOException;
 import java.io.StringReader;
 import java.io.UnsupportedEncodingException;
@@ -242,7 +242,7 @@ public class ConnectProcessor {
     }
 
     // analyze the origin stmt and return multi-statements
-    private List<StatementBase> analyze(String originStmt) throws AnalysisException {
+    private List<StatementBase> analyze(String originStmt) throws AnalysisException, DdlException {
         LOG.debug("the originStmts are: {}", originStmt);
         // Parse statement with parser generated by CUP&FLEX
         SqlScanner input = new SqlScanner(new StringReader(originStmt), ctx.getSessionVariable().getSqlMode());
@@ -251,9 +251,9 @@ public class ConnectProcessor {
             return SqlParserUtils.getMultiStmts(parser);
         } catch (Error e) {
             throw new AnalysisException("Please check your sql, we meet an error when parsing.", e);
-        } catch (AnalysisException e) {
-            LOG.warn("origin_stmt: " + originStmt + "; Analyze error message: " + parser.getErrorMsg(originStmt), e);
+        } catch (AnalysisException | DdlException e) {
             String errorMessage = parser.getErrorMsg(originStmt);
+            LOG.debug("origin stmt: {}; Analyze error message: {}", originStmt, parser.getErrorMsg(originStmt), e);
             if (errorMessage == null) {
                 throw e;
             } else {
@@ -262,7 +262,7 @@ public class ConnectProcessor {
         } catch (Exception e) {
             // TODO(lingbin): we catch 'Exception' to prevent unexpected error,
             // should be removed this try-catch clause future.
-            throw new AnalysisException("Internal Error, maybe syntax error or this is a bug, please contact with Palo RD.");
+            throw new AnalysisException("Internal Error, maybe syntax error or this is a bug");
         }
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/cache/PartitionRange.java b/fe/fe-core/src/main/java/org/apache/doris/qe/cache/PartitionRange.java
index aa09ad8..15a2c8f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/cache/PartitionRange.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/cache/PartitionRange.java
@@ -26,11 +26,13 @@ import org.apache.doris.analysis.Expr;
 import org.apache.doris.analysis.LiteralExpr;
 import org.apache.doris.analysis.IntLiteral;
 import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.PartitionItem;
 import org.apache.doris.catalog.PrimitiveType;
 import org.apache.doris.catalog.RangePartitionInfo;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.Partition;
 import org.apache.doris.catalog.PartitionKey;
+import org.apache.doris.catalog.RangePartitionItem;
 import org.apache.doris.catalog.Type;
 import org.apache.doris.common.Config;
 import org.apache.doris.planner.PartitionColumnFilter;
@@ -38,7 +40,6 @@ import org.apache.doris.planner.PartitionColumnFilter;
 import org.apache.doris.common.AnalysisException;
 
 import com.google.common.collect.Lists;
-import com.google.common.collect.Range;
 
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -479,11 +480,11 @@ public class PartitionRange {
      * PARTITION p20200102 VALUES [("20200102"), ("20200103")) )
      */
     private void getTablePartitionList(OlapTable table) {
-        Map<Long, Range<PartitionKey>> range = rangePartitionInfo.getIdToRange(false);
-        for (Map.Entry<Long, Range<PartitionKey>> entry : range.entrySet()) {
+        Map<Long, PartitionItem> range = rangePartitionInfo.getIdToItem(false);
+        for (Map.Entry<Long, PartitionItem> entry : range.entrySet()) {
             Long partId = entry.getKey();
             for (PartitionSingle single : partitionSingleList) {
-                if (entry.getValue().contains(single.getPartitionKey())) {
+                if (((RangePartitionItem) entry.getValue()).getItems().contains(single.getPartitionKey())) {
                     if (single.getPartitionId() == 0) {
                         single.setPartitionId(partId);
                     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/HadoopLoadPendingTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/HadoopLoadPendingTask.java
index 09140e9..309815a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/task/HadoopLoadPendingTask.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/task/HadoopLoadPendingTask.java
@@ -305,7 +305,7 @@ public class HadoopLoadPendingTask extends LoadPendingTask {
 
                 // begin keys
                 // is max partition
-                Range<PartitionKey> range = rangePartitionInfo.getRange(partitionId);
+                Range<PartitionKey> range = rangePartitionInfo.getItem(partitionId).getItems();
                 boolean isMaxPartition = range.upperEndpoint().isMaxValue();
 
                 // start keys
diff --git a/fe/fe-core/src/main/jflex/sql_scanner.flex b/fe/fe-core/src/main/jflex/sql_scanner.flex
index c3d918c..e4a1e45 100644
--- a/fe/fe-core/src/main/jflex/sql_scanner.flex
+++ b/fe/fe-core/src/main/jflex/sql_scanner.flex
@@ -245,6 +245,7 @@ import org.apache.doris.qe.SqlModeHelper;
         keywordMap.put("like", new Integer(SqlParserSymbols.KW_LIKE));
         keywordMap.put("limit", new Integer(SqlParserSymbols.KW_LIMIT));
         keywordMap.put("link", new Integer(SqlParserSymbols.KW_LINK));
+        keywordMap.put("list", new Integer(SqlParserSymbols.KW_LIST));
         keywordMap.put("load", new Integer(SqlParserSymbols.KW_LOAD));
         keywordMap.put("local", new Integer(SqlParserSymbols.KW_LOCAL));
         keywordMap.put("location", new Integer(SqlParserSymbols.KW_LOCATION));
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/ListPartitionPrunerTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/ListPartitionPrunerTest.java
new file mode 100644
index 0000000..e1cc577
--- /dev/null
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/ListPartitionPrunerTest.java
@@ -0,0 +1,170 @@
+// 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.analysis;
+
+import org.apache.doris.common.Config;
+import org.apache.doris.common.FeConstants;
+import org.apache.doris.utframe.DorisAssert;
+import org.apache.doris.utframe.UtFrameUtils;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.util.UUID;
+
+public class ListPartitionPrunerTest {
+    private static String runningDir = "fe/mocked/DemoTest/" + UUID.randomUUID().toString() + "/";
+    private static DorisAssert dorisAssert;
+
+    @Rule
+    public ExpectedException expectedEx = ExpectedException.none();
+
+    @AfterClass
+    public static void tearDown() throws Exception {
+        UtFrameUtils.cleanDorisFeDir(runningDir);
+    }
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        Config.enable_batch_delete_by_default = true;
+        FeConstants.runningUnitTest = true;
+        UtFrameUtils.createMinDorisCluster(runningDir);
+
+        String createSinglePartColWithSinglePartKey = "create table test.t1\n"
+                + "(k1 int, k2 varchar(128), k3 int, v1 int, v2 int)\n"
+                +"partition by list(k1)\n"
+                + "(\n"
+                + "partition p1 values in (\"1\"),\n"
+                + "partition p2 values in (\"2\")\n"
+                + ")\n"
+                + "distributed by hash(k2) buckets 1\n"
+                + "properties('replication_num' = '1');";
+        String createSinglePartColWithMultiPartKey = "create table test.t2\n"
+                + "(k1 int, k2 varchar(128), k3 int, v1 int, v2 int)\n"
+                +"partition by list(k1)\n"
+                + "(\n"
+                + "partition p1 values in (\"1\", \"3\", \"5\"),\n"
+                + "partition p2 values in (\"2\", \"4\", \"6\"),\n"
+                + "partition p3 values in (\"7\", \"8\")\n"
+                + ")\n"
+                + "distributed by hash(k2) buckets 1\n"
+                + "properties('replication_num' = '1');";
+        String createMultiPartColWithSinglePartKey = "create table test.t3\n"
+                + "(k1 int, k2 varchar(128), k3 int, v1 int, v2 int)\n"
+                +"partition by list(k1, k2)\n"
+                + "(\n"
+                + "partition p1 values in ((\"1\", \"beijing\")),\n"
+                + "partition p2 values in ((\"2\", \"beijing\"))\n"
+                + ")\n"
+                + "distributed by hash(k2) buckets 1\n"
+                + "properties('replication_num' = '1');";
+        String createMultiPartColWithMultiPartKey = "create table test.t4\n"
+                + "(k1 int, k2 varchar(128), k3 int, v1 int, v2 int)\n"
+                +"partition by list(k1, k2)\n"
+                + "(\n"
+                + "partition p1 values in ((\"1\", \"beijing\"), (\"2\", \"shanghai\")),\n"
+                + "partition p2 values in ((\"2\", \"beijing\")),\n"
+                + "partition p3 values in ((\"3\", \"tianjin\"), (\"1\", \"shanghai\"))\n"
+                + ")\n"
+                + "distributed by hash(k2) buckets 1\n"
+                + "properties('replication_num' = '1');";
+        dorisAssert = new DorisAssert();
+        dorisAssert.withDatabase("test").useDatabase("test");
+        dorisAssert.withTable(createSinglePartColWithSinglePartKey)
+                .withTable(createSinglePartColWithMultiPartKey)
+                .withTable(createMultiPartColWithSinglePartKey)
+                .withTable(createMultiPartColWithMultiPartKey);
+    }
+
+    @Test
+    public void testSelectWithPartition() throws Exception {
+        String sql = "select * from t1 partition p1;";
+        dorisAssert.query(sql).explainContains("partitions=1/2");
+
+        sql = "select * from t2 partition (p2, p3);";
+        dorisAssert.query(sql).explainContains("partitions=2/3");
+
+        sql = "select * from t3 partition (p1, p2);";
+        dorisAssert.query(sql).explainContains("partitions=2/2");
+
+        sql = "select * from t4 partition p2;";
+        dorisAssert.query(sql).explainContains("partitions=1/3");
+    }
+
+    @Test
+    public void testPartitionPrune() throws Exception {
+        // single partition column
+        String sql = "select * from t2 where k1 < 7";
+        dorisAssert.query(sql).explainContains("partitions=2/3");
+
+        sql = "select * from t2 where k1 = 1;";
+        dorisAssert.query(sql).explainContains("partitions=1/3");
+
+        sql = "select * from t2 where k1 in (1, 2);";
+        dorisAssert.query(sql).explainContains("partitions=2/3");
+
+        sql = "select * from t2 where k1 >= 6;";
+        dorisAssert.query(sql).explainContains("partitions=2/3");
+
+        sql = "select * from t2 where k1 < 8 and k1 > 6;";
+        dorisAssert.query(sql).explainContains("partitions=1/3");
+
+        sql = "select * from t2 where k2 = \"beijing\";";
+        dorisAssert.query(sql).explainContains("partitions=3/3");
+
+        // multi partition columns
+        sql = "select * from t4 where k1 = 2;";
+        dorisAssert.query(sql).explainContains("partitions=2/3");
+
+        sql = "select * from t4 where k2 = \"tianjin\";";
+        dorisAssert.query(sql).explainContains("partitions=1/3");
+
+        sql = "select * from t4 where k1 = 1 and k2 = \"shanghai\";";
+        dorisAssert.query(sql).explainContains("partitions=2/3");
+
+        sql = "select * from t4 where k1 in (1, 3) and k2 in (\"tianjin\", \"shanghai\");";
+        dorisAssert.query(sql).explainContains("partitions=2/3");
+
+        sql = "select * from t4 where k1 in (1, 3);";
+        dorisAssert.query(sql).explainContains("partitions=2/3");
+
+        sql = "select * from t4 where k2 in (\"tianjin\", \"shanghai\");";
+        dorisAssert.query(sql).explainContains("partitions=2/3");
+
+        sql = "select * from t4 where k1 < 3;";
+        dorisAssert.query(sql).explainContains("partitions=3/3");
+
+        sql = "select * from t4 where k1 > 2;";
+        dorisAssert.query(sql).explainContains("partitions=1/3");
+
+        sql = "select * from t4 where k2 <\"shanghai\";";
+        dorisAssert.query(sql).explainContains("partitions=2/3");
+
+        sql = "select * from t4 where k2 >=\"shanghai\";";
+        dorisAssert.query(sql).explainContains("partitions=2/3");
+
+        sql = "select * from t4 where k1 > 1 and k2 < \"shanghai\";";
+        dorisAssert.query(sql).explainContains("partitions=2/3");
+
+        sql = "select * from t4 where k1 >= 2 and k2 = \"shanghai\";";
+        dorisAssert.query(sql).explainContains("partitions=2/3");
+    }
+
+}
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/PartitionKeyDescTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/PartitionKeyDescTest.java
index b694de8..ffc8eef 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/PartitionKeyDescTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/PartitionKeyDescTest.java
@@ -17,11 +17,15 @@
 
 package org.apache.doris.analysis;
 
+import org.apache.doris.common.DdlException;
+
 import com.google.common.collect.Lists;
+
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+import java.util.ArrayList;
 import java.util.List;
 
 public class PartitionKeyDescTest {
@@ -34,8 +38,8 @@ public class PartitionKeyDescTest {
     }
 
     @Test
-    public void testNormal() {
-        PartitionKeyDesc desc = new PartitionKeyDesc(values);
+    public void testNormal() throws DdlException {
+        PartitionKeyDesc desc = PartitionKeyDesc.createLessThan(values);
 
         Assert.assertEquals(values, desc.getUpperValues());
         Assert.assertEquals("('1', 'abc')", desc.toSql());
@@ -48,4 +52,24 @@ public class PartitionKeyDescTest {
         Assert.assertNull(desc.getUpperValues());
         Assert.assertEquals("MAXVALUE", desc.toSql());
     }
+
+    @Test
+    public void testListMulti() {
+        List<List<PartitionValue>> list = new ArrayList<>();
+        list.add(values);
+        list.add(Lists.newArrayList(new PartitionValue("2"), new PartitionValue("cde")));
+        PartitionKeyDesc desc = PartitionKeyDesc.createIn(list);
+        Assert.assertEquals(list, desc.getInValues());
+        Assert.assertEquals("(('1', 'abc'),('2', 'cde'))", desc.toSql());
+    }
+
+    @Test
+    public void testListSingle() {
+        List<List<PartitionValue>> list = new ArrayList<>();
+        list.add(Lists.newArrayList(new PartitionValue("1")));
+        list.add(Lists.newArrayList(new PartitionValue("2")));
+        PartitionKeyDesc desc = PartitionKeyDesc.createIn(list);
+        Assert.assertEquals(list, desc.getInValues());
+        Assert.assertEquals("('1','2')", desc.toSql());
+    }
 }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/backup/CatalogMocker.java b/fe/fe-core/src/test/java/org/apache/doris/backup/CatalogMocker.java
index 20537be..d184f22 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/backup/CatalogMocker.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/backup/CatalogMocker.java
@@ -33,10 +33,12 @@ import org.apache.doris.catalog.MysqlTable;
 import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.catalog.Partition;
 import org.apache.doris.catalog.PartitionInfo;
+import org.apache.doris.catalog.PartitionItem;
 import org.apache.doris.catalog.PartitionKey;
 import org.apache.doris.catalog.PrimitiveType;
 import org.apache.doris.catalog.RandomDistributionInfo;
 import org.apache.doris.catalog.RangePartitionInfo;
+import org.apache.doris.catalog.RangePartitionItem;
 import org.apache.doris.catalog.Replica;
 import org.apache.doris.catalog.Replica.ReplicaState;
 import org.apache.doris.catalog.ScalarType;
@@ -294,7 +296,8 @@ public class CatalogMocker {
                 PartitionKey.createPartitionKey(Lists.newArrayList(new PartitionValue("10")),
                                                 Lists.newArrayList(TEST_TBL_BASE_SCHEMA.get(0)));
         Range<PartitionKey> rangeP1 = Range.closedOpen(rangeP1Lower, rangeP1Upper);
-        rangePartitionInfo.setRange(TEST_PARTITION1_ID, false, rangeP1);
+        PartitionItem item1 = new RangePartitionItem(rangeP1);
+        rangePartitionInfo.setItem(TEST_PARTITION1_ID, false, item1);
 
         PartitionKey rangeP2Lower =
                 PartitionKey.createPartitionKey(Lists.newArrayList(new PartitionValue("10")),
@@ -303,7 +306,8 @@ public class CatalogMocker {
                 PartitionKey.createPartitionKey(Lists.newArrayList(new PartitionValue("20")),
                                                 Lists.newArrayList(TEST_TBL_BASE_SCHEMA.get(0)));
         Range<PartitionKey> rangeP2 = Range.closedOpen(rangeP2Lower, rangeP2Upper);
-        rangePartitionInfo.setRange(TEST_PARTITION2_ID, false, rangeP2);
+        PartitionItem item2 = new RangePartitionItem(rangeP2);
+        rangePartitionInfo.setItem(TEST_PARTITION1_ID, false, item2);
 
         rangePartitionInfo.setReplicationNum(TEST_PARTITION1_ID, (short) 3);
         rangePartitionInfo.setReplicationNum(TEST_PARTITION2_ID, (short) 3);
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/CatalogTestUtil.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/CatalogTestUtil.java
index eb68f6b..0ec5a1b 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/CatalogTestUtil.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/CatalogTestUtil.java
@@ -19,7 +19,7 @@ package org.apache.doris.catalog;
 
 import org.apache.doris.analysis.PartitionKeyDesc;
 import org.apache.doris.analysis.PartitionValue;
-import org.apache.doris.analysis.SingleRangePartitionDesc;
+import org.apache.doris.analysis.SinglePartitionDesc;
 import org.apache.doris.catalog.MaterializedIndex.IndexExtState;
 import org.apache.doris.catalog.MaterializedIndex.IndexState;
 import org.apache.doris.catalog.Replica.ReplicaState;
@@ -245,13 +245,12 @@ public class CatalogTestUtil {
 
         // table
         List<Column> partitionColumns = Lists.newArrayList();
-        List<SingleRangePartitionDesc> singleRangePartitionDescs = Lists.newArrayList();
+        List<SinglePartitionDesc> singlePartitionDescs = Lists.newArrayList();
         partitionColumns.add(userId);
 
-        singleRangePartitionDescs.add(new SingleRangePartitionDesc(false, "p1",
-                                                                   new PartitionKeyDesc(Lists
-                                                                           .newArrayList(new PartitionValue("100"))),
-                                                                   null));
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p1",
+                PartitionKeyDesc.createLessThan(Lists.newArrayList(new PartitionValue("100"))),
+                null));
 
         RangePartitionInfo partitionInfo = new RangePartitionInfo(partitionColumns);
         Map<String, String> properties = Maps.newHashMap();
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableTest.java
index 0dabc18..4aec3be 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableTest.java
@@ -143,6 +143,59 @@ public class CreateTableTest {
                         + "distributed by hash(k2) buckets 1\n" + "properties('replication_num' = '1',\n"
                         + "'function_column.sequence_type' = 'int');"));
 
+        /**
+         * create table with list partition
+         */
+        // single partition column with single key
+        ExceptionChecker
+                .expectThrowsNoException(() -> createTable("create table test.tbl9\n"
+                        + "(k1 int, k2 varchar(128), k3 int, v1 int, v2 int)\n"
+                        +"partition by list(k1)\n"
+                        + "(\n"
+                        + "partition p1 values in (\"1\"),\n"
+                        + "partition p2 values in (\"2\")\n"
+                        + ")\n"
+                        + "distributed by hash(k2) buckets 1\n"
+                        + "properties('replication_num' = '1');"));
+
+        // single partition column with multi keys
+        ExceptionChecker
+                .expectThrowsNoException(() -> createTable("create table test.tbl10\n"
+                        + "(k1 int, k2 varchar(128), k3 int, v1 int, v2 int)\n"
+                        +"partition by list(k1)\n"
+                        + "(\n"
+                        + "partition p1 values in (\"1\", \"3\", \"5\"),\n"
+                        + "partition p2 values in (\"2\", \"4\", \"6\"),\n"
+                        + "partition p3 values in (\"7\", \"8\")\n"
+                        + ")\n"
+                        + "distributed by hash(k2) buckets 1\n"
+                        + "properties('replication_num' = '1');"));
+
+        // multi partition columns with single key
+        ExceptionChecker
+                .expectThrowsNoException(() -> createTable("create table test.tbl11\n"
+                        + "(k1 int, k2 varchar(128), k3 int, v1 int, v2 int)\n"
+                        +"partition by list(k1, k2)\n"
+                        + "(\n"
+                        + "partition p1 values in ((\"1\", \"beijing\")),\n"
+                        + "partition p2 values in ((\"2\", \"beijing\"))\n"
+                        + ")\n"
+                        + "distributed by hash(k2) buckets 1\n"
+                        + "properties('replication_num' = '1');"));
+
+        // multi partition columns with multi keys
+        ExceptionChecker
+                .expectThrowsNoException(() -> createTable("create table test.tbl12\n"
+                        + "(k1 int, k2 varchar(128), k3 int, v1 int, v2 int)\n"
+                        +"partition by list(k1, k2)\n"
+                        + "(\n"
+                        + "partition p1 values in ((\"1\", \"beijing\"), (\"1\", \"shanghai\")),\n"
+                        + "partition p2 values in ((\"2\", \"beijing\"), (\"2\", \"shanghai\")),\n"
+                        + "partition p3 values in ((\"3\", \"tianjin\"))\n"
+                        + ")\n"
+                        + "distributed by hash(k2) buckets 1\n"
+                        + "properties('replication_num' = '1');"));
+
         Database db = Catalog.getCurrentCatalog().getDb("default_cluster:test");
         OlapTable tbl6 = (OlapTable) db.getTable("tbl6");
         Assert.assertTrue(tbl6.getColumn("k1").isKey());
@@ -222,5 +275,203 @@ public class CreateTableTest {
                                 + "partition by range(k2)\n" + "(partition p1 values less than(\"10\"))\n"
                                 + "distributed by hash(k2) buckets 1\n" + "properties('replication_num' = '1',\n"
                                 + "'function_column.sequence_type' = 'double');"));
+
+        /**
+         * create table with list partition
+         */
+        // single partition column with single key
+        ExceptionChecker
+                .expectThrowsWithMsg(AnalysisException.class, "Syntax error", () -> createTable("create table test.tbl9\n"
+                        + "(k1 int, k2 varchar(128), k3 int, v1 int, v2 int)\n"
+                        +"partition by list(k1)\n"
+                        + "(\n"
+                        + "partition p1 values in (\"1\"),\n"
+                        + "partition p2 values in ()\n"
+                        + ")\n"
+                        + "distributed by hash(k2) buckets 1\n"
+                        + "properties('replication_num' = '1');"));
+
+        // single partition column with multi keys
+        ExceptionChecker
+                .expectThrowsWithMsg(IllegalArgumentException.class, "partition key desc list size[2] is not equal to partition column size[1]",
+                        () -> createTable("create table test.tbl10\n"
+                        + "(k1 int, k2 varchar(128), k3 int, v1 int, v2 int)\n"
+                        +"partition by list(k1)\n"
+                        + "(\n"
+                        + "partition p1 values in (\"1\", \"3\", \"5\"),\n"
+                        + "partition p2 values in (\"2\", \"4\", \"6\"),\n"
+                        + "partition p3 values in ((\"7\", \"8\"))\n"
+                        + ")\n"
+                        + "distributed by hash(k2) buckets 1\n"
+                        + "properties('replication_num' = '1');"));
+
+        // multi partition columns with single key
+        ExceptionChecker
+                .expectThrowsWithMsg(IllegalArgumentException.class, "partition key desc list size[1] is not equal to partition column size[2]",
+                        () -> createTable("create table test.tbl11\n"
+                        + "(k1 int, k2 varchar(128), k3 int, v1 int, v2 int)\n"
+                        +"partition by list(k1, k2)\n"
+                        + "(\n"
+                        + "partition p1 values in ((\"1\", \"beijing\")),\n"
+                        + "partition p2 values in (\"2\", \"beijing\")\n"
+                        + ")\n"
+                        + "distributed by hash(k2) buckets 1\n"
+                        + "properties('replication_num' = '1');"));
+
+        // multi partition columns with multi keys
+        ExceptionChecker
+                .expectThrowsWithMsg(IllegalArgumentException.class, "partition key desc list size[3] is not equal to partition column size[2]",
+                        () -> createTable("create table test.tbl12\n"
+                        + "(k1 int, k2 varchar(128), k3 int, v1 int, v2 int)\n"
+                        +"partition by list(k1, k2)\n"
+                        + "(\n"
+                        + "partition p1 values in ((\"1\", \"beijing\"), (\"1\", \"shanghai\")),\n"
+                        + "partition p2 values in ((\"2\", \"beijing\"), (\"2\", \"shanghai\")),\n"
+                        + "partition p3 values in ((\"3\", \"tianjin\", \"3\"))\n"
+                        + ")\n"
+                        + "distributed by hash(k2) buckets 1\n"
+                        + "properties('replication_num' = '1');"));
+
+        // multi partition columns with multi keys
+        ExceptionChecker
+                .expectThrowsWithMsg(AnalysisException.class, "Syntax error",
+                        () -> createTable("create table test.tbl13\n"
+                        + "(k1 int, k2 varchar(128), k3 int, v1 int, v2 int)\n"
+                        +"partition by list(k1, k2)\n"
+                        + "(\n"
+                        + "partition p1 values in ((\"1\", \"beijing\"), (\"1\", \"shanghai\")),\n"
+                        + "partition p2 values in ((\"2\", \"beijing\"), (\"2\", \"shanghai\")),\n"
+                        + "partition p3 values in ()\n"
+                        + ")\n"
+                        + "distributed by hash(k2) buckets 1\n"
+                        + "properties('replication_num' = '1');"));
+
+        /**
+         * create table with both list and range partition
+         */
+        // list contain less than
+        ExceptionChecker
+                .expectThrowsWithMsg(AnalysisException.class, "You can only use in values to create list partitions",
+                        () -> createTable("CREATE TABLE test.tbl14 (\n" +
+                                "    k1 int, k2 varchar(128), k3 int, v1 int, v2 int\n" +
+                                ")\n" +
+                                "PARTITION BY LIST(k1)\n" +
+                                "(\n" +
+                                "    PARTITION p1 VALUES less than (\"1\"),\n" +
+                                "    PARTITION p2 VALUES less than (\"2\"),\n" +
+                                "    partition p3 values less than (\"5\")\n" +
+                                ")DISTRIBUTED BY HASH(k2) BUCKETS 10\n" +
+                                "PROPERTIES(\"replication_num\" = \"1\");"));
+
+        // range contain in
+        ExceptionChecker
+                .expectThrowsWithMsg(AnalysisException.class, "You can only use fixed or less than values to create range partitions",
+                        () -> createTable("CREATE TABLE test.tbl15 (\n" +
+                                "    k1 int, k2 varchar(128), k3 int, v1 int, v2 int\n" +
+                                ")\n" +
+                                "PARTITION BY range(k1)\n" +
+                                "(\n" +
+                                "    PARTITION p1 VALUES in (\"1\"),\n" +
+                                "    PARTITION p2 VALUES in (\"2\"),\n" +
+                                "    partition p3 values in (\"5\")\n" +
+                                ")DISTRIBUTED BY HASH(k2) BUCKETS 10\n" +
+                                "PROPERTIES(\"replication_num\" = \"1\");"));
+
+        // list contain both
+        ExceptionChecker
+                .expectThrowsWithMsg(AnalysisException.class, "You can only use in values to create list partitions",
+                        () -> createTable("CREATE TABLE test.tbl15 (\n" +
+                                "    k1 int, k2 varchar(128), k3 int, v1 int, v2 int\n" +
+                                ")\n" +
+                                "PARTITION BY LIST(k1)\n" +
+                                "(\n" +
+                                "    PARTITION p1 VALUES in (\"1\"),\n" +
+                                "    PARTITION p2 VALUES in (\"2\"),\n" +
+                                "    partition p3 values less than (\"5\")\n" +
+                                ")DISTRIBUTED BY HASH(k2) BUCKETS 10\n" +
+                                "PROPERTIES(\"replication_num\" = \"1\");"));
+
+        // range contain both
+        ExceptionChecker
+                .expectThrowsWithMsg(AnalysisException.class, "You can only use fixed or less than values to create range partitions",
+                        () -> createTable("CREATE TABLE test.tbl16 (\n" +
+                                "    k1 int, k2 varchar(128), k3 int, v1 int, v2 int\n" +
+                                ")\n" +
+                                "PARTITION BY RANGE(k1)\n" +
+                                "(\n" +
+                                "    PARTITION p1 VALUES less than (\"1\"),\n" +
+                                "    PARTITION p2 VALUES less than (\"2\"),\n" +
+                                "    partition p3 values in (\"5\")\n" +
+                                ")DISTRIBUTED BY HASH(k2) BUCKETS 10\n" +
+                                "PROPERTIES(\"replication_num\" = \"1\");"));
+
+        // range: partition content != partition key type
+        ExceptionChecker
+                .expectThrowsWithMsg(DdlException.class, "Invalid number format: beijing",
+                        () -> createTable("CREATE TABLE test.tbl17 (\n" +
+                                "    k1 int, k2 varchar(128), k3 int, v1 int, v2 int\n" +
+                                ")\n" +
+                                "PARTITION BY range(k1)\n" +
+                                "(\n" +
+                                "    PARTITION p1 VALUES less than (\"beijing\"),\n" +
+                                "    PARTITION p2 VALUES less than (\"shanghai\"),\n" +
+                                "    partition p3 values less than (\"tianjin\")\n" +
+                                ")DISTRIBUTED BY HASH(k2) BUCKETS 10\n" +
+                                "PROPERTIES(\"replication_num\" = \"1\");"));
+
+        // list: partition content != partition key type
+        ExceptionChecker
+                .expectThrowsWithMsg(DdlException.class, "Invalid number format: beijing",
+                        () -> createTable("CREATE TABLE test.tbl18 (\n" +
+                                "    k1 int, k2 varchar(128), k3 int, v1 int, v2 int\n" +
+                                ")\n" +
+                                "PARTITION BY list(k1)\n" +
+                                "(\n" +
+                                "    PARTITION p1 VALUES in (\"beijing\"),\n" +
+                                "    PARTITION p2 VALUES in (\"shanghai\"),\n" +
+                                "    partition p3 values in (\"tianjin\")\n" +
+                                ")DISTRIBUTED BY HASH(k2) BUCKETS 10\n" +
+                                "PROPERTIES(\"replication_num\" = \"1\");"));
+
+        /**
+         * dynamic partition table
+         */
+        // list partition with dynamic properties
+        ExceptionChecker
+                .expectThrowsWithMsg(DdlException.class, "Only support dynamic partition properties on range partition table",
+                        () -> createTable("CREATE TABLE test.tbl19\n" +
+                                "(\n" +
+                                "    k1 DATE\n" +
+                                ")\n" +
+                                "PARTITION BY LIST(k1) ()\n" +
+                                "DISTRIBUTED BY HASH(k1)\n" +
+                                "PROPERTIES\n" +
+                                "(\n" +
+                                "    \"dynamic_partition.enable\" = \"true\",\n" +
+                                "    \"dynamic_partition.time_unit\" = \"MONTH\",\n" +
+                                "    \"dynamic_partition.end\" = \"2\",\n" +
+                                "    \"dynamic_partition.prefix\" = \"p\",\n" +
+                                "    \"dynamic_partition.buckets\" = \"8\",\n" +
+                                "    \"dynamic_partition.start_day_of_month\" = \"3\"\n" +
+                                ");\n"));
+
+        // no partition table with dynamic properties
+        ExceptionChecker
+                .expectThrowsWithMsg(DdlException.class, "Only support dynamic partition properties on range partition table",
+                        () -> createTable("CREATE TABLE test.tbl20\n" +
+                                "(\n" +
+                                "    k1 DATE\n" +
+                                ")\n" +
+                                "DISTRIBUTED BY HASH(k1)\n" +
+                                "PROPERTIES\n" +
+                                "(\n" +
+                                "    \"dynamic_partition.enable\" = \"true\",\n" +
+                                "    \"dynamic_partition.time_unit\" = \"MONTH\",\n" +
+                                "    \"dynamic_partition.end\" = \"2\",\n" +
+                                "    \"dynamic_partition.prefix\" = \"p\",\n" +
+                                "    \"dynamic_partition.buckets\" = \"8\",\n" +
+                                "    \"dynamic_partition.start_day_of_month\" = \"3\"\n" +
+                                ");"));
+
     }
 }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/ListPartitionInfoTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/ListPartitionInfoTest.java
new file mode 100644
index 0000000..ec481a7
--- /dev/null
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/ListPartitionInfoTest.java
@@ -0,0 +1,227 @@
+// 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.catalog;
+
+import org.apache.doris.analysis.PartitionKeyDesc;
+import org.apache.doris.analysis.PartitionValue;
+import org.apache.doris.analysis.SinglePartitionDesc;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.DdlException;
+
+import com.google.common.collect.Lists;
+
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+
+public class ListPartitionInfoTest {
+    private List<Column> partitionColumns;
+    private ListPartitionInfo partitionInfo;
+
+    private List<SinglePartitionDesc> singlePartitionDescs;
+
+    @Before
+    public void setUp() {
+        partitionColumns = new LinkedList<>();
+        singlePartitionDescs = new LinkedList<>();
+    }
+
+    @Test
+    public void testTinyInt() throws AnalysisException, DdlException {
+        Column k1 = new Column("k1", new ScalarType(PrimitiveType.TINYINT), true, null, "", "");
+        partitionColumns.add(k1);
+
+        List<List<PartitionValue>> inValues = new ArrayList<>();
+        inValues.add(Lists.newArrayList(new PartitionValue("-128")));
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p1",
+                PartitionKeyDesc.createIn(inValues), null));
+
+        partitionInfo = new ListPartitionInfo(partitionColumns);
+        PartitionItem partitionItem = null;
+        for (SinglePartitionDesc singlePartitionDesc : singlePartitionDescs) {
+            singlePartitionDesc.analyze(1, null);
+            partitionItem = partitionInfo.handleNewSinglePartitionDesc(singlePartitionDesc, 20000L, false);
+        }
+        Assert.assertEquals("-128", ((ListPartitionItem) partitionItem).getItems().get(0).getKeys().get(0).getStringValue());
+
+    }
+
+    @Test
+    public void testSmallInt() throws AnalysisException, DdlException {
+        Column k1 = new Column("k1", new ScalarType(PrimitiveType.SMALLINT), true, null, "", "");
+        partitionColumns.add(k1);
+
+        List<List<PartitionValue>> inValues = new ArrayList<>();
+        inValues.add(Lists.newArrayList(new PartitionValue("-32768")));
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p1",
+                PartitionKeyDesc.createIn(inValues), null));
+
+        partitionInfo = new ListPartitionInfo(partitionColumns);
+        PartitionItem partitionItem = null;
+        for (SinglePartitionDesc singlePartitionDesc : singlePartitionDescs) {
+            singlePartitionDesc.analyze(1, null);
+            partitionItem = partitionInfo.handleNewSinglePartitionDesc(singlePartitionDesc, 20000L, false);
+        }
+        Assert.assertEquals("-32768", ((ListPartitionItem) partitionItem).getItems().get(0).getKeys().get(0).getStringValue());
+    }
+
+    @Test
+    public void testInt() throws DdlException, AnalysisException {
+        Column k1 = new Column("k1", new ScalarType(PrimitiveType.INT), true, null, "", "");
+        partitionColumns.add(k1);
+
+        List<List<PartitionValue>> inValues = new ArrayList<>();
+        inValues.add(Lists.newArrayList(new PartitionValue("-2147483648")));
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p1",
+                PartitionKeyDesc.createIn(inValues), null));
+
+        partitionInfo = new ListPartitionInfo(partitionColumns);
+        PartitionItem partitionItem = null;
+        for (SinglePartitionDesc singlePartitionDesc : singlePartitionDescs) {
+            singlePartitionDesc.analyze(1, null);
+            partitionItem = partitionInfo.handleNewSinglePartitionDesc(singlePartitionDesc, 20000L, false);
+        }
+        Assert.assertEquals("-2147483648", ((ListPartitionItem) partitionItem).getItems().get(0).getKeys().get(0).getStringValue());
+    }
+
+    @Test
+    public void testBigInt() throws AnalysisException, DdlException {
+        Column k1 = new Column("k1", new ScalarType(PrimitiveType.BIGINT), true, null, "", "");
+        partitionColumns.add(k1);
+
+        List<List<PartitionValue>> inValues = new ArrayList<>();
+        inValues.add(Lists.newArrayList(new PartitionValue("-9223372036854775808")));
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p1",
+                PartitionKeyDesc.createIn(inValues), null));
+
+        partitionInfo = new ListPartitionInfo(partitionColumns);
+        PartitionItem partitionItem = null;
+        for (SinglePartitionDesc singlePartitionDesc : singlePartitionDescs) {
+            singlePartitionDesc.analyze(1, null);
+            partitionItem = partitionInfo.handleNewSinglePartitionDesc(singlePartitionDesc, 20000L, false);
+        }
+        Assert.assertEquals("-9223372036854775808", ((ListPartitionItem) partitionItem).getItems().get(0).getKeys().get(0).getStringValue());
+    }
+
+    @Test
+    public void testLargeInt() throws AnalysisException, DdlException {
+        Column k1 = new Column("k1", new ScalarType(PrimitiveType.LARGEINT), true, null, "", "");
+        partitionColumns.add(k1);
+
+        List<List<PartitionValue>> inValues = new ArrayList<>();
+        inValues.add(Lists.newArrayList(new PartitionValue("-170141183460469231731687303715884105728")));
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p1",
+                PartitionKeyDesc.createIn(inValues), null));
+
+        partitionInfo = new ListPartitionInfo(partitionColumns);
+        PartitionItem partitionItem = null;
+        for (SinglePartitionDesc singlePartitionDesc : singlePartitionDescs) {
+            singlePartitionDesc.analyze(1, null);
+            partitionItem = partitionInfo.handleNewSinglePartitionDesc(singlePartitionDesc, 20000L, false);
+        }
+        Assert.assertEquals("-170141183460469231731687303715884105728", ((ListPartitionItem) partitionItem).getItems().get(0).getKeys().get(0).getStringValue());
+    }
+
+    @Test
+    public void testString() throws AnalysisException, DdlException {
+        Column k1 = new Column("k1", new ScalarType(PrimitiveType.CHAR), true, null, "", "");
+        partitionColumns.add(k1);
+
+        List<List<PartitionValue>> inValues = new ArrayList<>();
+        inValues.add(Lists.newArrayList(new PartitionValue("Beijing")));
+        inValues.add(Lists.newArrayList(new PartitionValue("Shanghai")));
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p1",
+                PartitionKeyDesc.createIn(inValues), null));
+
+        partitionInfo = new ListPartitionInfo(partitionColumns);
+        PartitionItem partitionItem = null;
+        for (SinglePartitionDesc singlePartitionDesc : singlePartitionDescs) {
+            singlePartitionDesc.analyze(1, null);
+            partitionItem = partitionInfo.handleNewSinglePartitionDesc(singlePartitionDesc, 20000L, false);
+        }
+        Assert.assertEquals("Beijing", ((ListPartitionItem) partitionItem).getItems().get(0).getKeys().get(0).getStringValue());
+        Assert.assertEquals("Shanghai", ((ListPartitionItem) partitionItem).getItems().get(1).getKeys().get(0).getStringValue());
+    }
+
+    @Test
+    public void testBoolean() throws AnalysisException, DdlException {
+        Column k1 = new Column("k1", new ScalarType(PrimitiveType.BOOLEAN), true, null, "", "");
+        partitionColumns.add(k1);
+
+        List<List<PartitionValue>> inValues = new ArrayList<>();
+        inValues.add(Lists.newArrayList(new PartitionValue("true")));
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p1",
+                PartitionKeyDesc.createIn(inValues), null));
+
+        partitionInfo = new ListPartitionInfo(partitionColumns);
+        PartitionItem partitionItem = null;
+        for (SinglePartitionDesc singlePartitionDesc : singlePartitionDescs) {
+            singlePartitionDesc.analyze(1, null);
+            partitionItem = partitionInfo.handleNewSinglePartitionDesc(singlePartitionDesc, 20000L, false);
+        }
+        Assert.assertEquals(true, ((ListPartitionItem) partitionItem).getItems().get(0).getKeys().get(0).getRealValue());
+    }
+
+    @Test(expected = DdlException.class)
+    public void testDuplicateKey() throws AnalysisException, DdlException {
+        Column k1 = new Column("k1", new ScalarType(PrimitiveType.VARCHAR), true, null, "", "");
+        partitionColumns.add(k1);
+
+        List<List<PartitionValue>> inValues = new ArrayList<>();
+        inValues.add(Lists.newArrayList(new PartitionValue("beijing")));
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p1",
+                PartitionKeyDesc.createIn(inValues), null));
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p2",
+                PartitionKeyDesc.createIn(inValues), null));
+
+
+        partitionInfo = new ListPartitionInfo(partitionColumns);
+        for (SinglePartitionDesc singlePartitionDesc : singlePartitionDescs) {
+            singlePartitionDesc.analyze(1, null);
+            partitionInfo.handleNewSinglePartitionDesc(singlePartitionDesc, 20000L, false);
+        }
+    }
+
+    @Test
+    public void testMultiPartitionKeys() throws AnalysisException, DdlException {
+        Column k1 = new Column("k1", new ScalarType(PrimitiveType.VARCHAR), true, null, "", "");
+        Column k2 = new Column("k2", new ScalarType(PrimitiveType.INT), true, null, "", "");
+        partitionColumns.add(k1);
+        partitionColumns.add(k2);
+
+        List<List<PartitionValue>> inValues = new ArrayList<>();
+        inValues.add(Lists.newArrayList(new PartitionValue("beijing"), new PartitionValue("100")));
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p1",
+                PartitionKeyDesc.createIn(inValues), null));
+
+
+        partitionInfo = new ListPartitionInfo(partitionColumns);
+        PartitionItem partitionItem = null;
+        for (SinglePartitionDesc singlePartitionDesc : singlePartitionDescs) {
+            singlePartitionDesc.analyze(2, null);
+            partitionItem = partitionInfo.handleNewSinglePartitionDesc(singlePartitionDesc, 20000L, false);
+        }
+
+        Assert.assertEquals("beijing", ((ListPartitionItem) partitionItem).getItems().get(0).getKeys().get(0).getRealValue());
+        Assert.assertEquals(100, ((ListPartitionItem) partitionItem).getItems().get(0).getKeys().get(1).getLongValue());
+    }
+}
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/PartitionKeyTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/PartitionKeyTest.java
index 5c0142e..e6eb4ec 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/PartitionKeyTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/PartitionKeyTest.java
@@ -45,6 +45,9 @@ public class PartitionKeyTest {
     private static Column largeInt;
     private static Column date;
     private static Column datetime;
+    private static Column charString;
+    private static Column varchar;
+    private static Column bool;
     
     private Catalog catalog;
 
@@ -60,6 +63,9 @@ public class PartitionKeyTest {
         largeInt = new Column("largeint", PrimitiveType.LARGEINT);
         date = new Column("date", PrimitiveType.DATE);
         datetime = new Column("datetime", PrimitiveType.DATETIME);
+        charString = new Column("char", PrimitiveType.CHAR);
+        varchar = new Column("varchar", PrimitiveType.VARCHAR);
+        bool = new Column("bool", PrimitiveType.BOOLEAN);
 
         allColumns = Arrays.asList(tinyInt, smallInt, int32, bigInt, largeInt, date, datetime);
     }
@@ -146,6 +152,48 @@ public class PartitionKeyTest {
                 allColumns);
         pk2 = PartitionKey.createInfinityPartitionKey(allColumns, false);
         Assert.assertTrue(!pk1.equals(pk2) && pk1.compareTo(pk2) == 1);
+
+        // case11
+        pk1 = PartitionKey.createPartitionKey(Arrays.asList(new PartitionValue("beijing"), new PartitionValue("shanghai")),
+                Arrays.asList(charString, varchar));
+        pk2 = PartitionKey.createPartitionKey(Arrays.asList(new PartitionValue("beijing"), new PartitionValue("shanghai")),
+                Arrays.asList(charString, varchar));
+        Assert.assertTrue(pk1.equals(pk2) && pk1.compareTo(pk2) == 0);
+
+        // case12
+        pk1 = PartitionKey.createPartitionKey(Arrays.asList(new PartitionValue("beijing"), new PartitionValue("shanghai")),
+                Arrays.asList(charString, varchar));
+        pk2 = PartitionKey.createPartitionKey(Arrays.asList(new PartitionValue("shijiazhuang"), new PartitionValue("tianjin")),
+                Arrays.asList(charString, varchar));
+        Assert.assertTrue(!pk1.equals(pk2) && pk1.compareTo(pk2) == -1);
+
+        // case13
+        pk1 = PartitionKey.createPartitionKey(Arrays.asList(new PartitionValue("beijing"), new PartitionValue("shanghai")),
+                Arrays.asList(charString, varchar));
+        pk2 = PartitionKey.createPartitionKey(Arrays.asList(new PartitionValue("beijing"), new PartitionValue("tianjin")),
+                Arrays.asList(charString, varchar));
+        Assert.assertTrue(!pk1.equals(pk2) && pk1.compareTo(pk2) == -1);
+
+        // case14
+        pk1 = PartitionKey.createPartitionKey(Arrays.asList(new PartitionValue("true")),
+                Arrays.asList(bool));
+        pk2 = PartitionKey.createPartitionKey(Arrays.asList(new PartitionValue("false")),
+                Arrays.asList(bool));
+        Assert.assertTrue(!pk1.equals(pk2) && pk1.compareTo(pk2) == 1);
+
+        // case15
+        pk1 = PartitionKey.createPartitionKey(Arrays.asList(new PartitionValue("true")),
+                Arrays.asList(bool));
+        pk2 = PartitionKey.createPartitionKey(Arrays.asList(new PartitionValue("true")),
+                Arrays.asList(bool));
+        Assert.assertTrue(pk1.equals(pk2) && pk1.compareTo(pk2) == 0);
+
+        // case16
+        pk1 = PartitionKey.createPartitionKey(Arrays.asList(new PartitionValue("false")),
+                Arrays.asList(bool));
+        pk2 = PartitionKey.createPartitionKey(Arrays.asList(new PartitionValue("false")),
+                Arrays.asList(bool));
+        Assert.assertTrue(pk1.equals(pk2) && pk1.compareTo(pk2) == 0);
     }
 
     @Test
@@ -175,7 +223,15 @@ public class PartitionKeyTest {
         columns.add(new Column("column10", ScalarType.createType(PrimitiveType.DATE), true, null, "", ""));
         keys.add(new PartitionValue("2014-12-27 11:12:13"));
         columns.add(new Column("column11", ScalarType.createType(PrimitiveType.DATETIME), true, null, "", ""));
-  
+        keys.add(new PartitionValue("beijing"));
+        columns.add(new Column("column12", ScalarType.createType(PrimitiveType.VARCHAR), true, null, "", ""));
+        keys.add(new PartitionValue("shanghai"));
+        columns.add(new Column("column13", ScalarType.createType(PrimitiveType.CHAR), true, null, "", ""));
+        keys.add(new PartitionValue("true"));
+        columns.add(new Column("column14", ScalarType.createType(PrimitiveType.BOOLEAN), true, null, "", ""));
+        keys.add(new PartitionValue("false"));
+        columns.add(new Column("column15", ScalarType.createType(PrimitiveType.BOOLEAN), true, null, "", ""));
+
         PartitionKey key = PartitionKey.createPartitionKey(keys, columns);
         key.write(dos);
 
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/RangePartitionInfoTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/RangePartitionInfoTest.java
index 3a75802..2f52178 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/RangePartitionInfoTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/RangePartitionInfoTest.java
@@ -18,9 +18,9 @@
 package org.apache.doris.catalog;
 
 import org.apache.doris.analysis.PartitionKeyDesc;
-import org.apache.doris.analysis.PartitionKeyDesc.PartitionRangeType;
+import org.apache.doris.analysis.PartitionKeyDesc.PartitionKeyValueType;
 import org.apache.doris.analysis.PartitionValue;
-import org.apache.doris.analysis.SingleRangePartitionDesc;
+import org.apache.doris.analysis.SinglePartitionDesc;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.DdlException;
 
@@ -38,12 +38,12 @@ public class RangePartitionInfoTest {
     private List<Column> partitionColumns;
     private RangePartitionInfo partitionInfo;
 
-    private List<SingleRangePartitionDesc> singleRangePartitionDescs;
+    private List<SinglePartitionDesc> singlePartitionDescs;
 
     @Before
     public void setUp() {
         partitionColumns = new LinkedList<Column>();
-        singleRangePartitionDescs = new LinkedList<SingleRangePartitionDesc>();
+        singlePartitionDescs = new LinkedList<SinglePartitionDesc>();
     }
 
     @Test(expected = DdlException.class)
@@ -51,15 +51,15 @@ public class RangePartitionInfoTest {
         Column k1 = new Column("k1", new ScalarType(PrimitiveType.TINYINT), true, null, "", "");
         partitionColumns.add(k1);
 
-        singleRangePartitionDescs.add(new SingleRangePartitionDesc(false, "p1",
-                               new PartitionKeyDesc(Lists .newArrayList(new PartitionValue("-128"))),
-                               null));
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p1",
+                PartitionKeyDesc.createLessThan(Lists.newArrayList(new PartitionValue("-128"))),
+                null));
 
 
         partitionInfo = new RangePartitionInfo(partitionColumns);
-        for (SingleRangePartitionDesc singleRangePartitionDesc : singleRangePartitionDescs) {
-            singleRangePartitionDesc.analyze(1, null);
-            partitionInfo.handleNewSinglePartitionDesc(singleRangePartitionDesc, 20000L, false);
+        for (SinglePartitionDesc singlePartitionDesc : singlePartitionDescs) {
+            singlePartitionDesc.analyze(1, null);
+            partitionInfo.handleNewSinglePartitionDesc(singlePartitionDesc, 20000L, false);
         }
     }
 
@@ -68,14 +68,14 @@ public class RangePartitionInfoTest {
         Column k1 = new Column("k1", new ScalarType(PrimitiveType.SMALLINT), true, null, "", "");
         partitionColumns.add(k1);
 
-        singleRangePartitionDescs.add(new SingleRangePartitionDesc(false, "p1",
-                               new PartitionKeyDesc(Lists.newArrayList(new PartitionValue("-32768"))),
-                               null));
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p1",
+                PartitionKeyDesc.createLessThan(Lists.newArrayList(new PartitionValue("-32768"))),
+                null));
 
         partitionInfo = new RangePartitionInfo(partitionColumns);
-        for (SingleRangePartitionDesc singleRangePartitionDesc : singleRangePartitionDescs) {
-            singleRangePartitionDesc.analyze(1, null);
-            partitionInfo.handleNewSinglePartitionDesc(singleRangePartitionDesc, 20000L, false);
+        for (SinglePartitionDesc singlePartitionDesc : singlePartitionDescs) {
+            singlePartitionDesc.analyze(1, null);
+            partitionInfo.handleNewSinglePartitionDesc(singlePartitionDesc, 20000L, false);
         }
     }
 
@@ -84,14 +84,14 @@ public class RangePartitionInfoTest {
         Column k1 = new Column("k1", new ScalarType(PrimitiveType.INT), true, null, "", "");
         partitionColumns.add(k1);
 
-        singleRangePartitionDescs.add(new SingleRangePartitionDesc(false, "p1",
-                           new PartitionKeyDesc(Lists.newArrayList(new PartitionValue("-2147483648"))),
-                           null));
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p1",
+                PartitionKeyDesc.createLessThan(Lists.newArrayList(new PartitionValue("-2147483648"))),
+                null));
 
         partitionInfo = new RangePartitionInfo(partitionColumns);
-        for (SingleRangePartitionDesc singleRangePartitionDesc : singleRangePartitionDescs) {
-            singleRangePartitionDesc.analyze(1, null);
-            partitionInfo.handleNewSinglePartitionDesc(singleRangePartitionDesc, 20000L, false);
+        for (SinglePartitionDesc singlePartitionDesc : singlePartitionDescs) {
+            singlePartitionDesc.analyze(1, null);
+            partitionInfo.handleNewSinglePartitionDesc(singlePartitionDesc, 20000L, false);
         }
     }
 
@@ -100,20 +100,20 @@ public class RangePartitionInfoTest {
         Column k1 = new Column("k1", new ScalarType(PrimitiveType.BIGINT), true, null, "", "");
         partitionColumns.add(k1);
 
-        singleRangePartitionDescs.add(new SingleRangePartitionDesc(false, "p1", new PartitionKeyDesc(Lists
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p1", PartitionKeyDesc.createLessThan(Lists
                 .newArrayList(new PartitionValue("-9223372036854775808"))), null));
-        singleRangePartitionDescs.add(new SingleRangePartitionDesc(false, "p2", new PartitionKeyDesc(Lists
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p2", PartitionKeyDesc.createLessThan(Lists
                 .newArrayList(new PartitionValue("-9223372036854775806"))), null));
-        singleRangePartitionDescs.add(new SingleRangePartitionDesc(false, "p3", new PartitionKeyDesc(Lists
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p3", PartitionKeyDesc.createLessThan(Lists
                 .newArrayList(new PartitionValue("0"))), null));
-        singleRangePartitionDescs.add(new SingleRangePartitionDesc(false, "p4", new PartitionKeyDesc(Lists
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p4", PartitionKeyDesc.createLessThan(Lists
                 .newArrayList(new PartitionValue("9223372036854775806"))), null));
 
         partitionInfo = new RangePartitionInfo(partitionColumns);
 
-        for (SingleRangePartitionDesc singleRangePartitionDesc : singleRangePartitionDescs) {
-            singleRangePartitionDesc.analyze(1, null);
-            partitionInfo.handleNewSinglePartitionDesc(singleRangePartitionDesc, 20000L, false);
+        for (SinglePartitionDesc singlePartitionDesc : singlePartitionDescs) {
+            singlePartitionDesc.analyze(1, null);
+            partitionInfo.handleNewSinglePartitionDesc(singlePartitionDesc, 20000L, false);
         }
     }
 
@@ -122,20 +122,20 @@ public class RangePartitionInfoTest {
         Column k1 = new Column("k1", new ScalarType(PrimitiveType.BIGINT), true, null, "", "");
         partitionColumns.add(k1);
 
-        singleRangePartitionDescs.add(new SingleRangePartitionDesc(false, "p1", new PartitionKeyDesc(Lists
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p1", PartitionKeyDesc.createLessThan(Lists
                 .newArrayList(new PartitionValue("-9223372036854775806"))), null));
-        singleRangePartitionDescs.add(new SingleRangePartitionDesc(false, "p2", new PartitionKeyDesc(Lists
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p2", PartitionKeyDesc.createLessThan(Lists
                 .newArrayList(new PartitionValue("-9223372036854775805"))), null));
-        singleRangePartitionDescs.add(new SingleRangePartitionDesc(false, "p3", new PartitionKeyDesc(Lists
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p3", PartitionKeyDesc.createLessThan(Lists
                 .newArrayList(new PartitionValue("0"))), null));
-        singleRangePartitionDescs.add(new SingleRangePartitionDesc(false, "p4", new PartitionKeyDesc(Lists
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p4", PartitionKeyDesc.createLessThan(Lists
                 .newArrayList(new PartitionValue("9223372036854775806"))), null));
 
         partitionInfo = new RangePartitionInfo(partitionColumns);
 
-        for (SingleRangePartitionDesc singleRangePartitionDesc : singleRangePartitionDescs) {
-            singleRangePartitionDesc.analyze(1, null);
-            partitionInfo.handleNewSinglePartitionDesc(singleRangePartitionDesc, 20000L, false);
+        for (SinglePartitionDesc singlePartitionDesc : singlePartitionDescs) {
+            singlePartitionDesc.analyze(1, null);
+            partitionInfo.handleNewSinglePartitionDesc(singlePartitionDesc, 20000L, false);
         }
     }
 
@@ -157,29 +157,29 @@ public class RangePartitionInfoTest {
         partitionColumns.add(k2);
 
         //add RangePartitionDescs
-        PartitionKeyDesc p1 = new PartitionKeyDesc(
+        PartitionKeyDesc p1 = PartitionKeyDesc.createFixed(
                 Lists.newArrayList(new PartitionValue("20190101"), new PartitionValue("100")),
                 Lists.newArrayList(new PartitionValue("20190101"), new PartitionValue("200")));
-        PartitionKeyDesc p2 = new PartitionKeyDesc(
+        PartitionKeyDesc p2 = PartitionKeyDesc.createFixed(
                 Lists.newArrayList(new PartitionValue("20190105"), new PartitionValue("10")),
                 Lists.newArrayList(new PartitionValue("20190107"), new PartitionValue("10")));
-        PartitionKeyDesc p3 = new PartitionKeyDesc(
+        PartitionKeyDesc p3 = PartitionKeyDesc.createFixed(
                 Lists.newArrayList(new PartitionValue("20181231"), new PartitionValue("10")),
                 Lists.newArrayList(new PartitionValue("20190101"), new PartitionValue("100")));
-        PartitionKeyDesc p4 = new PartitionKeyDesc(
+        PartitionKeyDesc p4 = PartitionKeyDesc.createFixed(
                 Lists.newArrayList(new PartitionValue("20190105"), new PartitionValue("100")),
                 Lists.newArrayList(new PartitionValue("20190120"), new PartitionValue("10000000000")));
 
-        singleRangePartitionDescs.add(new SingleRangePartitionDesc(false, "p1", p1, null));
-        singleRangePartitionDescs.add(new SingleRangePartitionDesc(false, "p2", p2, null));
-        singleRangePartitionDescs.add(new SingleRangePartitionDesc(false, "p3", p3, null));
-        singleRangePartitionDescs.add(new SingleRangePartitionDesc(false, "p4", p4, null));
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p1", p1, null));
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p2", p2, null));
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p3", p3, null));
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p4", p4, null));
 
         partitionInfo = new RangePartitionInfo(partitionColumns);
 
-        for (SingleRangePartitionDesc singleRangePartitionDesc : singleRangePartitionDescs) {
-            singleRangePartitionDesc.analyze(columns, null);
-            partitionInfo.handleNewSinglePartitionDesc(singleRangePartitionDesc, 20000L, false);
+        for (SinglePartitionDesc singlePartitionDesc : singlePartitionDescs) {
+            singlePartitionDesc.analyze(columns, null);
+            partitionInfo.handleNewSinglePartitionDesc(singlePartitionDesc, 20000L, false);
         }
     }
 
@@ -202,28 +202,28 @@ public class RangePartitionInfoTest {
         partitionColumns.add(k3);
 
         //add RangePartitionDescs
-        PartitionKeyDesc p1 = new PartitionKeyDesc(
+        PartitionKeyDesc p1 = PartitionKeyDesc.createLessThan(
                 Lists.newArrayList(new PartitionValue("2019-02-01"), new PartitionValue("100"), new PartitionValue("200")));
-        PartitionKeyDesc p2 = new PartitionKeyDesc(
+        PartitionKeyDesc p2 = PartitionKeyDesc.createFixed(
                 Lists.newArrayList(new PartitionValue("2020-02-01"), new PartitionValue("100"), new PartitionValue("200")),
                 Lists.newArrayList(new PartitionValue("10000000000")));
-        PartitionKeyDesc p3 = new PartitionKeyDesc(
+        PartitionKeyDesc p3 = PartitionKeyDesc.createLessThan(
                 Lists.newArrayList(new PartitionValue("2021-02-01")));
 
-        singleRangePartitionDescs.add(new SingleRangePartitionDesc(false, "p1", p1, null));
-        singleRangePartitionDescs.add(new SingleRangePartitionDesc(false, "p2", p2, null));
-        singleRangePartitionDescs.add(new SingleRangePartitionDesc(false, "p3", p3, null));
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p1", p1, null));
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p2", p2, null));
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p3", p3, null));
         partitionInfo = new RangePartitionInfo(partitionColumns);
-        PartitionRangeType partitionType = PartitionRangeType.INVALID;
-        for (SingleRangePartitionDesc singleRangePartitionDesc : singleRangePartitionDescs) {
+        PartitionKeyValueType partitionKeyValueType = PartitionKeyValueType.INVALID;
+        for (SinglePartitionDesc singlePartitionDesc : singlePartitionDescs) {
             // check partitionType
-            if (partitionType == PartitionRangeType.INVALID) {
-                partitionType = singleRangePartitionDesc.getPartitionKeyDesc().getPartitionType();
-            } else if (partitionType != singleRangePartitionDesc.getPartitionKeyDesc().getPartitionType()) {
+            if (partitionKeyValueType == PartitionKeyValueType.INVALID) {
+                partitionKeyValueType = singlePartitionDesc.getPartitionKeyDesc().getPartitionType();
+            } else if (partitionKeyValueType != singlePartitionDesc.getPartitionKeyDesc().getPartitionType()) {
                 throw new AnalysisException("You can only use one of these methods to create partitions");
             }
-            singleRangePartitionDesc.analyze(partitionColumns.size(), null);
-            partitionInfo.handleNewSinglePartitionDesc(singleRangePartitionDesc, 20000L, false);
+            singlePartitionDesc.analyze(partitionColumns.size(), null);
+            partitionInfo.handleNewSinglePartitionDesc(singlePartitionDesc, 20000L, false);
         }
     }
 
@@ -242,16 +242,16 @@ public class RangePartitionInfoTest {
         partitionColumns.add(k2);
 
         //add RangePartitionDescs
-        PartitionKeyDesc p1 = new PartitionKeyDesc(new ArrayList<>(),
+        PartitionKeyDesc p1 = PartitionKeyDesc.createFixed(new ArrayList<>(),
                 Lists.newArrayList(new PartitionValue("20190101"), new PartitionValue("200")));
 
-        singleRangePartitionDescs.add(new SingleRangePartitionDesc(false, "p1", p1, null));
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p1", p1, null));
 
         partitionInfo = new RangePartitionInfo(partitionColumns);
 
-        for (SingleRangePartitionDesc singleRangePartitionDesc : singleRangePartitionDescs) {
-            singleRangePartitionDesc.analyze(columns, null);
-            partitionInfo.handleNewSinglePartitionDesc(singleRangePartitionDesc, 20000L, false);
+        for (SinglePartitionDesc singlePartitionDesc : singlePartitionDescs) {
+            singlePartitionDesc.analyze(columns, null);
+            partitionInfo.handleNewSinglePartitionDesc(singlePartitionDesc, 20000L, false);
         }
     }
 
@@ -271,17 +271,17 @@ public class RangePartitionInfoTest {
         partitionColumns.add(k2);
 
         //add RangePartitionDescs
-        PartitionKeyDesc p1 = new PartitionKeyDesc(
+        PartitionKeyDesc p1 = PartitionKeyDesc.createFixed(
                 Lists.newArrayList(new PartitionValue("20190101"), new PartitionValue("200")),
                 new ArrayList<>());
 
-        singleRangePartitionDescs.add(new SingleRangePartitionDesc(false, "p1", p1, null));
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p1", p1, null));
 
         partitionInfo = new RangePartitionInfo(partitionColumns);
 
-        for (SingleRangePartitionDesc singleRangePartitionDesc : singleRangePartitionDescs) {
-            singleRangePartitionDesc.analyze(columns, null);
-            partitionInfo.handleNewSinglePartitionDesc(singleRangePartitionDesc, 20000L, false);
+        for (SinglePartitionDesc singlePartitionDesc : singlePartitionDescs) {
+            singlePartitionDesc.analyze(columns, null);
+            partitionInfo.handleNewSinglePartitionDesc(singlePartitionDesc, 20000L, false);
         }
     }
 
@@ -300,17 +300,17 @@ public class RangePartitionInfoTest {
         partitionColumns.add(k2);
 
         //add RangePartitionDescs
-        PartitionKeyDesc p1 = new PartitionKeyDesc(
+        PartitionKeyDesc p1 = PartitionKeyDesc.createFixed(
                 Lists.newArrayList(new PartitionValue("20190101"), new PartitionValue("100")),
                 Lists.newArrayList(new PartitionValue("20190201")));
 
-        singleRangePartitionDescs.add(new SingleRangePartitionDesc(false, "p1", p1, null));
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p1", p1, null));
 
         partitionInfo = new RangePartitionInfo(partitionColumns);
 
-        for (SingleRangePartitionDesc singleRangePartitionDesc : singleRangePartitionDescs) {
-            singleRangePartitionDesc.analyze(columns, null);
-            partitionInfo.handleNewSinglePartitionDesc(singleRangePartitionDesc, 20000L, false);
+        for (SinglePartitionDesc singlePartitionDesc : singlePartitionDescs) {
+            singlePartitionDesc.analyze(columns, null);
+            partitionInfo.handleNewSinglePartitionDesc(singlePartitionDesc, 20000L, false);
         }
     }
 
@@ -330,17 +330,17 @@ public class RangePartitionInfoTest {
         partitionColumns.add(k2);
 
         //add RangePartitionDescs
-        PartitionKeyDesc p1 = new PartitionKeyDesc(
+        PartitionKeyDesc p1 = PartitionKeyDesc.createFixed(
                 Lists.newArrayList(new PartitionValue("20190101"), new PartitionValue("100")),
                 Lists.newArrayList(new PartitionValue("20190101"), new PartitionValue("100")));
 
-        singleRangePartitionDescs.add(new SingleRangePartitionDesc(false, "p1", p1, null));
+        singlePartitionDescs.add(new SinglePartitionDesc(false, "p1", p1, null));
 
         partitionInfo = new RangePartitionInfo(partitionColumns);
 
-        for (SingleRangePartitionDesc singleRangePartitionDesc : singleRangePartitionDescs) {
-            singleRangePartitionDesc.analyze(columns, null);
-            partitionInfo.handleNewSinglePartitionDesc(singleRangePartitionDesc, 20000L, false);
+        for (SinglePartitionDesc singlePartitionDesc : singlePartitionDescs) {
+            singlePartitionDesc.analyze(columns, null);
+            partitionInfo.handleNewSinglePartitionDesc(singlePartitionDesc, 20000L, false);
         }
     }
 
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/TempPartitionTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/TempPartitionTest.java
index b39b3c0..88fce47 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/TempPartitionTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/TempPartitionTest.java
@@ -68,9 +68,9 @@ public class TempPartitionTest {
 
     @BeforeClass
     public static void setup() throws Exception {
+        FeConstants.default_scheduler_interval_millisecond = 100;
         UtFrameUtils.createMinDorisCluster(runningDir);
         ctx = UtFrameUtils.createDefaultCtx();
-        FeConstants.default_scheduler_interval_millisecond = 100;
     }
 
     @AfterClass
@@ -558,6 +558,657 @@ public class TempPartitionTest {
         checkShowPartitionsResultNum("db3.tbl3", false, 2);
         checkShowPartitionsResultNum("db3.tbl3", true, 0);
     }
+
+    @Test
+    public void testForListPartitionTable() throws Exception {
+        MetaContext metaContext = new MetaContext();
+        metaContext.setMetaVersion(FeMetaVersion.VERSION_CURRENT);
+        metaContext.setThreadLocalInfo();
+
+        // create database db4
+        String createDbStmtStr = "create database db4;";
+        CreateDbStmt createDbStmt = (CreateDbStmt) UtFrameUtils.parseAndAnalyzeStmt(createDbStmtStr, ctx);
+        Catalog.getCurrentCatalog().createDb(createDbStmt);
+        System.out.println(Catalog.getCurrentCatalog().getDbNames());
+
+        // create table tbl4
+        String createTblStmtStr1 = "create table db4.tbl4 (k1 int, k2 int)\n" +
+                "partition by list(k1)\n" +
+                "(\n" +
+                "partition p1 values in ('1', '2', '3'),\n" +
+                "partition p2 values in ('4', '5', '6'),\n" +
+                "partition p3 values in ('7', '8', '9')\n" +
+                ")\n" +
+                "distributed by hash(k2) buckets 1\n" +
+                "properties('replication_num' = '1');";
+        CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseAndAnalyzeStmt(createTblStmtStr1, ctx);
+        Catalog.getCurrentCatalog().createTable(createTableStmt);
+
+        Database db4 = Catalog.getCurrentCatalog().getDb("default_cluster:db4");
+        OlapTable tbl4 = (OlapTable) db4.getTable("tbl4");
+
+        testSerializeOlapTable(tbl4);
+
+        Map<String, Long> originPartitionTabletIds = Maps.newHashMap();
+        getPartitionNameToTabletIdMap("db4.tbl4", false, originPartitionTabletIds);
+        Assert.assertEquals(3, originPartitionTabletIds.keySet().size());
+
+        // show temp partition
+        checkShowPartitionsResultNum("db4.tbl4", true, 0);
+        checkShowPartitionsResultNum("db4.tbl4", false, 3);
+
+        // add temp partition with duplicate name
+        String stmtStr = "alter table db4.tbl4 add temporary partition p1 values in ('1', '2', '3');";
+        alterTable(stmtStr, true);
+
+        // add temp partition
+        stmtStr = "alter table db4.tbl4 add temporary partition tp1 values in ('1', '2', '3');";
+        alterTable(stmtStr, false);
+
+        stmtStr = "alter table db4.tbl4 add temporary partition tp2 values in ('1', '2', '3');";
+        alterTable(stmtStr, true);
+
+        stmtStr = "alter table db4.tbl4 add temporary partition tp1 values in ('4', '5', '6');";
+        alterTable(stmtStr, true);
+
+        stmtStr = "alter table db4.tbl4 add temporary partition tp2 values in ('4', '5', '6');";
+        alterTable(stmtStr, false);
+
+        stmtStr = "alter table db4.tbl4 add temporary partition tp3 values in ('6', '7', '8', '9');";
+        alterTable(stmtStr, true);
+
+        stmtStr = "alter table db4.tbl4 add temporary partition tp3 values in ('7', '8', '9');";
+        alterTable(stmtStr, false);
+
+        Map<String, Long> tempPartitionTabletIds = Maps.newHashMap();
+        getPartitionNameToTabletIdMap("db4.tbl4", true, tempPartitionTabletIds);
+        Assert.assertEquals(3, tempPartitionTabletIds.keySet().size());
+
+        System.out.println("partition tablets: " + originPartitionTabletIds);
+        System.out.println("temp partition tablets: " + tempPartitionTabletIds);
+
+        testSerializeOlapTable(tbl4);
+
+        // drop non exist temp partition
+        stmtStr = "alter table db4.tbl4 drop temporary partition tp4;";
+        alterTable(stmtStr, true);
+
+        stmtStr = "alter table db4.tbl4 drop temporary partition if exists tp4;";
+        alterTable(stmtStr, false);
+
+        stmtStr = "alter table db4.tbl4 drop temporary partition tp3;";
+        alterTable(stmtStr, false);
+
+        Map<String, Long> originPartitionTabletIds2 = Maps.newHashMap();
+        getPartitionNameToTabletIdMap("db4.tbl4", false, originPartitionTabletIds2);
+        Assert.assertEquals(originPartitionTabletIds2, originPartitionTabletIds);
+
+        Map<String, Long> tempPartitionTabletIds2 = Maps.newHashMap();
+        getPartitionNameToTabletIdMap("db4.tbl4", true, tempPartitionTabletIds2);
+        Assert.assertEquals(2, tempPartitionTabletIds2.keySet().size());
+        Assert.assertTrue(!tempPartitionTabletIds2.containsKey("tp3"));
+
+        checkShowPartitionsResultNum("db4.tbl4", true, 2);
+        checkShowPartitionsResultNum("db4.tbl4", false, 3);
+
+        stmtStr = "alter table db4.tbl4 add temporary partition tp3 values in ('7', '8', '9');";
+        alterTable(stmtStr, false);
+        checkShowPartitionsResultNum("db4.tbl4", true, 3);
+
+        stmtStr = "alter table db4.tbl4 drop partition p1;";
+        alterTable(stmtStr, false);
+        checkShowPartitionsResultNum("db4.tbl4", true, 3);
+        checkShowPartitionsResultNum("db4.tbl4", false, 2);
+
+        originPartitionTabletIds2 = Maps.newHashMap();
+        getPartitionNameToTabletIdMap("db4.tbl4", false, originPartitionTabletIds2);
+        Assert.assertEquals(2, originPartitionTabletIds2.size());
+        Assert.assertTrue(!originPartitionTabletIds2.containsKey("p1"));
+
+        String recoverStr = "recover partition p1 from db4.tbl4;";
+        RecoverPartitionStmt recoverStmt = (RecoverPartitionStmt) UtFrameUtils.parseAndAnalyzeStmt(recoverStr, ctx);
+        Catalog.getCurrentCatalog().recoverPartition(recoverStmt);
+        checkShowPartitionsResultNum("db4.tbl4", true, 3);
+        checkShowPartitionsResultNum("db4.tbl4", false, 3);
+
+        originPartitionTabletIds2 = Maps.newHashMap();
+        getPartitionNameToTabletIdMap("db4.tbl4", false, originPartitionTabletIds2);
+        Assert.assertEquals(originPartitionTabletIds2, originPartitionTabletIds);
+
+        tempPartitionTabletIds2 = Maps.newHashMap();
+        getPartitionNameToTabletIdMap("db4.tbl4", true, tempPartitionTabletIds2);
+        Assert.assertEquals(3, tempPartitionTabletIds2.keySet().size());
+
+        // Here, we should have 3 partitions p1,p2,p3, and 3 temp partitions tp1,tp2,tp3
+        System.out.println("we have partition tablets: " + originPartitionTabletIds2);
+        System.out.println("we have temp partition tablets: " + tempPartitionTabletIds2);
+
+        stmtStr = "alter table db4.tbl4 replace partition(p1, p2) with temporary partition(tp2, tp3);";
+        alterTable(stmtStr, true);
+
+        stmtStr = "alter table db4.tbl4 replace partition(p1, p2) with temporary partition(tp1, tp2) properties('invalid' = 'invalid');";
+        alterTable(stmtStr, true);
+
+        stmtStr = "alter table db4.tbl4 replace partition(p1, p2) with temporary partition(tp2, tp3);";
+        alterTable(stmtStr, true);
+
+        stmtStr = "alter table db4.tbl4 replace partition(p1, p2) with temporary partition(tp1, tp2) properties('use_temp_partition_name' = 'true');";
+        alterTable(stmtStr, false);
+        checkShowPartitionsResultNum("db4.tbl4", true, 1); // tp3
+        checkShowPartitionsResultNum("db4.tbl4", false, 3);// tp1, tp2, p3
+
+        checkTabletExists(tempPartitionTabletIds2.values(), true);
+        checkTabletExists(Lists.newArrayList(originPartitionTabletIds2.get("p3")), true);
+        checkTabletExists(Lists.newArrayList(originPartitionTabletIds2.get("p1"), originPartitionTabletIds2.get("p2")), false);
+
+        String truncateStr = "truncate table db4.tbl4 partition (p3);";
+        TruncateTableStmt truncateTableStmt = (TruncateTableStmt)UtFrameUtils.parseAndAnalyzeStmt(truncateStr, ctx);
+        Catalog.getCurrentCatalog().truncateTable(truncateTableStmt);
+        checkShowPartitionsResultNum("db4.tbl4", true, 1);
+        checkShowPartitionsResultNum("db4.tbl4", false, 3);
+        checkPartitionExist(tbl4, "tp1", false, true);
+        checkPartitionExist(tbl4, "tp2", false, true);
+        checkPartitionExist(tbl4, "p3", false, true);
+        checkPartitionExist(tbl4, "tp3", true, true);
+
+        stmtStr = "alter table db4.tbl4 drop partition p3;";
+        alterTable(stmtStr, false);
+        stmtStr = "alter table db4.tbl4 add partition p31 values in ('7', '8');";
+        alterTable(stmtStr, false);
+        stmtStr = "alter table db4.tbl4 add partition p32 values in ('9');";
+        alterTable(stmtStr, false);
+
+        // for now, we have 4 partitions: tp1, tp2, p31, p32, 1 temp partition: tp3
+        checkShowPartitionsResultNum("db4.tbl4", false, 4);
+        checkShowPartitionsResultNum("db4.tbl4", true, 1);
+
+        stmtStr = "alter table db4.tbl4 replace partition(p31) with temporary partition(tp3);";
+        alterTable(stmtStr, true);
+        stmtStr = "alter table db4.tbl4 replace partition(p31, p32) with temporary partition(tp3);";
+        alterTable(stmtStr, false);
+        checkShowPartitionsResultNum("db4.tbl4", false, 3);
+        checkShowPartitionsResultNum("db4.tbl4", true, 0);
+        checkPartitionExist(tbl4, "tp1", false, true);
+        checkPartitionExist(tbl4, "tp2", false, true);
+        checkPartitionExist(tbl4, "tp3", false, true);
+
+        stmtStr = "alter table db4.tbl4 add temporary partition p1 values in ('1', '2', '3');";
+        alterTable(stmtStr, false);
+        stmtStr = "alter table db4.tbl4 add temporary partition p2 values in ('4', '5', '6');";
+        alterTable(stmtStr, false);
+        stmtStr = "alter table db4.tbl4 add temporary partition p3 values in ('7', '8', '9');";
+        alterTable(stmtStr, false);
+        stmtStr = "alter table db4.tbl4 replace partition(tp1, tp2) with temporary partition(p1, p2);";
+        alterTable(stmtStr, false);
+        checkPartitionExist(tbl4, "tp1", false, true);
+        checkPartitionExist(tbl4, "tp2", false, true);
+        checkPartitionExist(tbl4, "tp3", false, true);
+        checkPartitionExist(tbl4, "p1", true, false);
+        checkPartitionExist(tbl4, "p2", true, false);
+        checkPartitionExist(tbl4, "p3", true, true);
+
+        stmtStr = "alter table db4.tbl4 replace partition(tp3) with temporary partition(p3) properties('use_temp_partition_name' = 'true');";
+        alterTable(stmtStr, false);
+        checkPartitionExist(tbl4, "tp1", false, true);
+        checkPartitionExist(tbl4, "tp2", false, true);
+        checkPartitionExist(tbl4, "p3", false, true);
+        checkPartitionExist(tbl4, "p1", true, false);
+        checkPartitionExist(tbl4, "p2", true, false);
+        checkPartitionExist(tbl4, "p3", true, false);
+        checkShowPartitionsResultNum("db4.tbl4", false, 3);
+        checkShowPartitionsResultNum("db4.tbl4", true, 0);
+
+        stmtStr = "alter table db4.tbl4 add temporary partition tp1 values in ('1', '2', '3');"; // name conflict
+        alterTable(stmtStr, true);
+        stmtStr = "alter table db4.tbl4 rename partition p3 tp3;";
+        alterTable(stmtStr, false);
+        stmtStr = "alter table db4.tbl4 add temporary partition p1 values in ('1', '2', '3');";
+        alterTable(stmtStr, false);
+
+        originPartitionTabletIds2 = Maps.newHashMap();
+        getPartitionNameToTabletIdMap("db4.tbl4", false, originPartitionTabletIds2);
+        Assert.assertEquals(3, originPartitionTabletIds2.size());
+
+        tempPartitionTabletIds2 = Maps.newHashMap();
+        getPartitionNameToTabletIdMap("db4.tbl4", true, tempPartitionTabletIds2);
+        Assert.assertEquals(1, tempPartitionTabletIds2.keySet().size());
+
+        // for now , we have 3 partitions: tp1, tp2, tp3, 1 temp partition: p1
+        System.out.println("we have partition tablets: " + originPartitionTabletIds2);
+        System.out.println("we have temp partition tablets: " + tempPartitionTabletIds2);
+
+        stmtStr = "alter table db4.tbl4 add rollup r1(k1);";
+        alterTable(stmtStr, true);
+
+        // truncate table will delete temporary partitions
+        truncateStr = "truncate table db4.tbl4";
+        truncateTableStmt = (TruncateTableStmt) UtFrameUtils.parseAndAnalyzeStmt(truncateStr, ctx);
+        Catalog.getCurrentCatalog().truncateTable(truncateTableStmt);
+        checkShowPartitionsResultNum("db4.tbl4", false, 3);
+        checkShowPartitionsResultNum("db4.tbl4", true, 0);
+
+        stmtStr = "alter table db4.tbl4 add rollup r1(k1);";
+        alterTable(stmtStr, false);
+
+        stmtStr = "alter table db4.tbl4 add temporary partition p2 values in ('1', '2', '3', '4', '5', '6');";
+        alterTable(stmtStr, true);
+
+        // wait rollup finish
+        Map<Long, AlterJobV2> alterJobs = Catalog.getCurrentCatalog().getRollupHandler().getAlterJobsV2();
+        for (AlterJobV2 alterJobV2 : alterJobs.values()) {
+            while (!alterJobV2.getJobState().isFinalState()) {
+                System.out.println(
+                        "alter job " + alterJobV2.getDbId() + " is running. state: " + alterJobV2.getJobState());
+                Thread.sleep(5000);
+            }
+            System.out.println("alter job " + alterJobV2.getDbId() + " is done. state: " + alterJobV2.getJobState());
+            Assert.assertEquals(AlterJobV2.JobState.FINISHED, alterJobV2.getJobState());
+        }
+        // waiting table state to normal
+        Thread.sleep(500);
+
+        stmtStr = "alter table db4.tbl4 add temporary partition p2 values in ('1', '2', '3', '4', '5', '6');";
+        alterTable(stmtStr, false);
+
+        TempPartitions tempPartitions = Deencapsulation.getField(tbl4, "tempPartitions");
+        testSerializeTempPartitions(tempPartitions);
+
+        stmtStr = "alter table db4.tbl4 replace partition (tp1, tp2) with temporary partition (p2);";
+        alterTable(stmtStr, false);
+        checkShowPartitionsResultNum("db4.tbl4", false, 2);
+        checkShowPartitionsResultNum("db4.tbl4", true, 0);
+        checkPartitionExist(tbl4, "p2", false, true);
+        checkPartitionExist(tbl4, "tp3", false, true);
+        checkPartitionExist(tbl4, "tp1", false, false);
+        checkPartitionExist(tbl4, "tp2", false, false);
+        checkPartitionExist(tbl4, "p2", true, false);
+
+        checkTablet("db4.tbl4", "p2", false, 2);
+        checkTablet("db4.tbl4", "tp3", false, 2);
+
+        // for now, we have 2 partitions: p2, tp3, ('1', '2', '3', '4', '5', '6'), ('7', '8', '9'). 0 temp partition.
+        stmtStr = "alter table db4.tbl4 add temporary partition tp4 values in ('1', '2', '3', '4', '5', '6') ('in_memory' = 'true') distributed by hash(k1) buckets 3";
+        alterTable(stmtStr, true);
+        stmtStr = "alter table db4.tbl4 add temporary partition tp4 values in ('1', '2', '3', '4', '5', '6') ('in_memory' = 'true', 'replication_num' = '2') distributed by hash(k2) buckets 3";
+        alterTable(stmtStr, true);
+        stmtStr = "alter table db4.tbl4 add temporary partition tp4 values in ('1', '2', '3', '4', '5', '6') ('in_memory' = 'true', 'replication_num' = '1') distributed by hash(k2) buckets 3";
+        alterTable(stmtStr, false);
+
+        Partition p2 = tbl4.getPartition("p2");
+        Assert.assertNotNull(p2);
+        Assert.assertFalse(tbl4.getPartitionInfo().getIsInMemory(p2.getId()));
+        Assert.assertEquals(1, p2.getDistributionInfo().getBucketNum());
+
+        stmtStr = "alter table db4.tbl4 replace partition (p2) with temporary partition (tp4)";
+        alterTable(stmtStr, false);
+
+        // for now, we have 2 partitions: p2, tp3, ('1', '2', '3', '4', '5', '6'), ('7', '8', '9'). 0 temp partition. and p2 bucket is 3, 'in_memory' is true.
+        p2 = tbl4.getPartition("p2");
+        Assert.assertNotNull(p2);
+        Assert.assertTrue(tbl4.getPartitionInfo().getIsInMemory(p2.getId()));
+        Assert.assertEquals(3, p2.getDistributionInfo().getBucketNum());
+
+        stmtStr = "alter table db4.tbl4 add temporary partition tp1 values in ('1', '2', '3');";
+        alterTable(stmtStr, false);
+        stmtStr = "alter table db4.tbl4 add temporary partition tp2 values in ('4', '5', '6');";
+        alterTable(stmtStr, false);
+
+        checkShowPartitionsResultNum("db4.tbl4", false, 2);
+        checkShowPartitionsResultNum("db4.tbl4", true, 2);
+        checkPartitionExist(tbl4, "p2", false, true);
+        checkPartitionExist(tbl4, "tp3", false, true);
+        checkPartitionExist(tbl4, "tp1", true, true);
+        checkPartitionExist(tbl4, "tp2", true, true);
+
+        stmtStr = "alter table db4.tbl4 replace partition (p2) with temporary partition (tp1, tp2)";
+        alterTable(stmtStr, false);
+
+        checkShowPartitionsResultNum("db4.tbl4", false, 3);
+        checkShowPartitionsResultNum("db4.tbl4", true, 0);
+        checkPartitionExist(tbl4, "tp1", false, true);
+        checkPartitionExist(tbl4, "tp2", false, true);
+        checkPartitionExist(tbl4, "tp3", false, true);
+        checkPartitionExist(tbl4, "tp1", true, false);
+        checkPartitionExist(tbl4, "tp2", true, false);
+
+        // for now, tbl4 has 3 formal partition:
+        // tp1 (1, 2, 3)
+        // tp2 (4, 5, 6)
+        // tp3 (7, 8, 9)
+        // Test strict range
+        stmtStr = "alter table db4.tbl4 add temporary partition p2 values in ('4', '5')";
+        alterTable(stmtStr, false);
+        stmtStr = "alter table db4.tbl4 add temporary partition p31 values in ('7', '8')";
+        alterTable(stmtStr, false);
+        stmtStr = "alter table db4.tbl4 add temporary partition p32 values in ('9')";
+        alterTable(stmtStr, false);
+
+        stmtStr = "alter table db4.tbl4 replace partition (tp2) with temporary partition (p2) properties('strict_range' = 'true');";
+        alterTable(stmtStr, true);
+        stmtStr = "alter table db4.tbl4 replace partition (tp2) with temporary partition (p2) properties('strict_range' = 'false', 'use_temp_partition_name' = 'true');";
+        alterTable(stmtStr, false);
+        stmtStr = "alter table db4.tbl4 replace partition (tp3) with temporary partition (p31, p32) properties('strict_range' = 'true', 'use_temp_partition_name' = 'true');";
+        alterTable(stmtStr, false);
+
+        stmtStr = "alter table db4.tbl4 add temporary partition p4 values in ('1', '2', '3', '4')";
+        alterTable(stmtStr, false);
+
+        stmtStr = "alter table db4.tbl4 replace partition (tp1) with temporary partition (p4) properties('strict_range' = 'false');";
+        alterTable(stmtStr, true);
+    }
+    @Test
+    public void testForMultiListPartitionTable() throws Exception {
+        MetaContext metaContext = new MetaContext();
+        metaContext.setMetaVersion(FeMetaVersion.VERSION_CURRENT);
+        metaContext.setThreadLocalInfo();
+
+        // create database db5
+        String createDbStmtStr = "create database db5;";
+        CreateDbStmt createDbStmt = (CreateDbStmt) UtFrameUtils.parseAndAnalyzeStmt(createDbStmtStr, ctx);
+        Catalog.getCurrentCatalog().createDb(createDbStmt);
+        System.out.println(Catalog.getCurrentCatalog().getDbNames());
+
+        // create table tbl5
+        String createTblStmtStr1 = "create table db5.tbl5 (k1 int, k2 varchar)\n" +
+                "partition by list(k1, k2)\n" +
+                "(\n" +
+                "partition p1 values in ((\"1\",\"beijing\"), (\"1\", \"shanghai\")),\n" +
+                "partition p2 values in ((\"2\",\"beijing\"), (\"2\", \"shanghai\")),\n" +
+                "partition p3 values in ((\"3\",\"beijing\"), (\"3\", \"shanghai\"))\n" +
+                ")\n" +
+                "distributed by hash(k2) buckets 1\n" +
+                "properties('replication_num' = '1');";
+        CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseAndAnalyzeStmt(createTblStmtStr1, ctx);
+        Catalog.getCurrentCatalog().createTable(createTableStmt);
+
+        Database db5 = Catalog.getCurrentCatalog().getDb("default_cluster:db5");
+        OlapTable tbl5 = (OlapTable) db5.getTable("tbl5");
+
+        testSerializeOlapTable(tbl5);
+
+        Map<String, Long> originPartitionTabletIds = Maps.newHashMap();
+        getPartitionNameToTabletIdMap("db5.tbl5", false, originPartitionTabletIds);
+        Assert.assertEquals(3, originPartitionTabletIds.keySet().size());
+
+        // show temp partition
+        checkShowPartitionsResultNum("db5.tbl5", true, 0);
+        checkShowPartitionsResultNum("db5.tbl5", false, 3);
+
+        // add temp partition with duplicate name
+        String stmtStr = "alter table db5.tbl5 add temporary partition p1 values in ((\"1\",\"beijing\"), (\"1\", \"shanghai\"));";
+        alterTable(stmtStr, true);
+
+        // add temp partition
+        stmtStr = "alter table db5.tbl5 add temporary partition tp1 values in ((\"1\",\"beijing\"), (\"1\", \"shanghai\"));";
+        alterTable(stmtStr, false);
+
+        stmtStr = "alter table db5.tbl5 add temporary partition tp2 values in ((\"1\",\"beijing\"), (\"1\", \"shanghai\");";
+        alterTable(stmtStr, true);
+
+        stmtStr = "alter table db5.tbl5 add temporary partition tp1 values in ((\"2\",\"beijing\"), (\"2\", \"shanghai\"));";
+        alterTable(stmtStr, true);
+
+        stmtStr = "alter table db5.tbl5 add temporary partition tp2 values in ((\"2\",\"beijing\"), (\"2\", \"shanghai\"));";
+        alterTable(stmtStr, false);
+
+        stmtStr = "alter table db5.tbl5 add temporary partition tp3 values in ((\"2\",\"beijing\"), (\"3\",\"beijing\"), (\"3\", \"shanghai\"));";
+        alterTable(stmtStr, true);
+
+        stmtStr = "alter table db5.tbl5 add temporary partition tp3 values in ((\"3\",\"beijing\"), (\"3\", \"shanghai\"));";
+        alterTable(stmtStr, false);
+
+        Map<String, Long> tempPartitionTabletIds = Maps.newHashMap();
+        getPartitionNameToTabletIdMap("db5.tbl5", true, tempPartitionTabletIds);
+        Assert.assertEquals(3, tempPartitionTabletIds.keySet().size());
+
+        System.out.println("partition tablets: " + originPartitionTabletIds);
+        System.out.println("temp partition tablets: " + tempPartitionTabletIds);
+
+        testSerializeOlapTable(tbl5);
+
+        // drop non exist temp partition
+        stmtStr = "alter table db5.tbl5 drop temporary partition tp4;";
+        alterTable(stmtStr, true);
+
+        stmtStr = "alter table db5.tbl5 drop temporary partition if exists tp4;";
+        alterTable(stmtStr, false);
+
+        stmtStr = "alter table db5.tbl5 drop temporary partition tp3;";
+        alterTable(stmtStr, false);
+
+        Map<String, Long> originPartitionTabletIds2 = Maps.newHashMap();
+        getPartitionNameToTabletIdMap("db5.tbl5", false, originPartitionTabletIds2);
+        Assert.assertEquals(originPartitionTabletIds2, originPartitionTabletIds);
+
+        Map<String, Long> tempPartitionTabletIds2 = Maps.newHashMap();
+        getPartitionNameToTabletIdMap("db5.tbl5", true, tempPartitionTabletIds2);
+        Assert.assertEquals(2, tempPartitionTabletIds2.keySet().size());
+        Assert.assertTrue(!tempPartitionTabletIds2.containsKey("tp3"));
+
+        checkShowPartitionsResultNum("db5.tbl5", true, 2);
+        checkShowPartitionsResultNum("db5.tbl5", false, 3);
+
+        stmtStr = "alter table db5.tbl5 add temporary partition tp3 values in ((\"3\",\"beijing\"), (\"3\", \"shanghai\"));";
+        alterTable(stmtStr, false);
+        checkShowPartitionsResultNum("db5.tbl5", true, 3);
+
+        stmtStr = "alter table db5.tbl5 drop partition p1;";
+        alterTable(stmtStr, false);
+        checkShowPartitionsResultNum("db5.tbl5", true, 3);
+        checkShowPartitionsResultNum("db5.tbl5", false, 2);
+
+        originPartitionTabletIds2 = Maps.newHashMap();
+        getPartitionNameToTabletIdMap("db5.tbl5", false, originPartitionTabletIds2);
+        Assert.assertEquals(2, originPartitionTabletIds2.size());
+        Assert.assertTrue(!originPartitionTabletIds2.containsKey("p1"));
+
+        String recoverStr = "recover partition p1 from db5.tbl5;";
+        RecoverPartitionStmt recoverStmt = (RecoverPartitionStmt) UtFrameUtils.parseAndAnalyzeStmt(recoverStr, ctx);
+        Catalog.getCurrentCatalog().recoverPartition(recoverStmt);
+        checkShowPartitionsResultNum("db5.tbl5", true, 3);
+        checkShowPartitionsResultNum("db5.tbl5", false, 3);
+
+        originPartitionTabletIds2 = Maps.newHashMap();
+        getPartitionNameToTabletIdMap("db5.tbl5", false, originPartitionTabletIds2);
+        Assert.assertEquals(originPartitionTabletIds2, originPartitionTabletIds);
+
+        tempPartitionTabletIds2 = Maps.newHashMap();
+        getPartitionNameToTabletIdMap("db5.tbl5", true, tempPartitionTabletIds2);
+        Assert.assertEquals(3, tempPartitionTabletIds2.keySet().size());
+
+        // Here, we should have 3 partitions p1,p2,p3, and 3 temp partitions tp1,tp2,tp3
+        System.out.println("we have partition tablets: " + originPartitionTabletIds2);
+        System.out.println("we have temp partition tablets: " + tempPartitionTabletIds2);
+
+        stmtStr = "alter table db5.tbl5 replace partition(p1, p2) with temporary partition(tp2, tp3);";
+        alterTable(stmtStr, true);
+
+        stmtStr = "alter table db5.tbl5 replace partition(p1, p2) with temporary partition(tp1, tp2) properties('invalid' = 'invalid');";
+        alterTable(stmtStr, true);
+
+        stmtStr = "alter table db5.tbl5 replace partition(p1, p2) with temporary partition(tp2, tp3);";
+        alterTable(stmtStr, true);
+
+        stmtStr = "alter table db5.tbl5 replace partition(p1, p2) with temporary partition(tp1, tp2) properties('use_temp_partition_name' = 'true');";
+        alterTable(stmtStr, false);
+        checkShowPartitionsResultNum("db5.tbl5", true, 1); // tp3
+        checkShowPartitionsResultNum("db5.tbl5", false, 3);// tp1, tp2, p3
+
+        checkTabletExists(tempPartitionTabletIds2.values(), true);
+        checkTabletExists(Lists.newArrayList(originPartitionTabletIds2.get("p3")), true);
+        checkTabletExists(Lists.newArrayList(originPartitionTabletIds2.get("p1"), originPartitionTabletIds2.get("p2")), false);
+
+        String truncateStr = "truncate table db5.tbl5 partition (p3);";
+        TruncateTableStmt truncateTableStmt = (TruncateTableStmt)UtFrameUtils.parseAndAnalyzeStmt(truncateStr, ctx);
+        Catalog.getCurrentCatalog().truncateTable(truncateTableStmt);
+        checkShowPartitionsResultNum("db5.tbl5", true, 1);
+        checkShowPartitionsResultNum("db5.tbl5", false, 3);
+        checkPartitionExist(tbl5, "tp1", false, true);
+        checkPartitionExist(tbl5, "tp2", false, true);
+        checkPartitionExist(tbl5, "p3", false, true);
+        checkPartitionExist(tbl5, "tp3", true, true);
+
+        stmtStr = "alter table db5.tbl5 drop partition p3;";
+        alterTable(stmtStr, false);
+        stmtStr = "alter table db5.tbl5 add partition p31 values in ((\"3\",\"beijing\"));";
+        alterTable(stmtStr, false);
+        stmtStr = "alter table db5.tbl5 add partition p32 values in ((\"3\", \"shanghai\"));";
+        alterTable(stmtStr, false);
+
+        // for now, we have 4 partitions: tp1, tp2, p31, p32, 1 temp partition: tp3
+        checkShowPartitionsResultNum("db5.tbl5", false, 4);
+        checkShowPartitionsResultNum("db5.tbl5", true, 1);
+
+        stmtStr = "alter table db5.tbl5 replace partition(p31) with temporary partition(tp3);";
+        alterTable(stmtStr, true);
+        stmtStr = "alter table db5.tbl5 replace partition(p31, p32) with temporary partition(tp3);";
+        alterTable(stmtStr, false);
+        checkShowPartitionsResultNum("db5.tbl5", false, 3);
+        checkShowPartitionsResultNum("db5.tbl5", true, 0);
+        checkPartitionExist(tbl5, "tp1", false, true);
+        checkPartitionExist(tbl5, "tp2", false, true);
+        checkPartitionExist(tbl5, "tp3", false, true);
+
+        stmtStr = "alter table db5.tbl5 add temporary partition p1 values in ((\"1\",\"beijing\"), (\"1\", \"shanghai\"));";
+        alterTable(stmtStr, false);
+        stmtStr = "alter table db5.tbl5 add temporary partition p2 values in ((\"2\",\"beijing\"), (\"2\", \"shanghai\"));";
+        alterTable(stmtStr, false);
+        stmtStr = "alter table db5.tbl5 add temporary partition p3 values in ((\"3\",\"beijing\"), (\"3\", \"shanghai\"));";
+        alterTable(stmtStr, false);
+        stmtStr = "alter table db5.tbl5 replace partition(tp1, tp2) with temporary partition(p1, p2);";
+        alterTable(stmtStr, false);
+        checkPartitionExist(tbl5, "tp1", false, true);
+        checkPartitionExist(tbl5, "tp2", false, true);
+        checkPartitionExist(tbl5, "tp3", false, true);
+        checkPartitionExist(tbl5, "p1", true, false);
+        checkPartitionExist(tbl5, "p2", true, false);
+        checkPartitionExist(tbl5, "p3", true, true);
+
+        stmtStr = "alter table db5.tbl5 replace partition(tp3) with temporary partition(p3) properties('use_temp_partition_name' = 'true');";
+        alterTable(stmtStr, false);
+        checkPartitionExist(tbl5, "tp1", false, true);
+        checkPartitionExist(tbl5, "tp2", false, true);
+        checkPartitionExist(tbl5, "p3", false, true);
+        checkPartitionExist(tbl5, "p1", true, false);
+        checkPartitionExist(tbl5, "p2", true, false);
+        checkPartitionExist(tbl5, "p3", true, false);
+        checkShowPartitionsResultNum("db5.tbl5", false, 3);
+        checkShowPartitionsResultNum("db5.tbl5", true, 0);
+
+        stmtStr = "alter table db5.tbl5 add temporary partition tp1 values in ((\"1\",\"beijing\"), (\"1\", \"shanghai\"));"; // name conflict
+        alterTable(stmtStr, true);
+        stmtStr = "alter table db5.tbl5 rename partition p3 tp3;";
+        alterTable(stmtStr, false);
+        stmtStr = "alter table db5.tbl5 add temporary partition p1 values in ((\"1\",\"beijing\"), (\"1\", \"shanghai\"));";
+        alterTable(stmtStr, false);
+
+        originPartitionTabletIds2 = Maps.newHashMap();
+        getPartitionNameToTabletIdMap("db5.tbl5", false, originPartitionTabletIds2);
+        Assert.assertEquals(3, originPartitionTabletIds2.size());
+
+        tempPartitionTabletIds2 = Maps.newHashMap();
+        getPartitionNameToTabletIdMap("db5.tbl5", true, tempPartitionTabletIds2);
+        Assert.assertEquals(1, tempPartitionTabletIds2.keySet().size());
+
+        // for now , we have 3 partitions: tp1, tp2, tp3, 1 temp partition: p1
+        System.out.println("we have partition tablets: " + originPartitionTabletIds2);
+        System.out.println("we have temp partition tablets: " + tempPartitionTabletIds2);
+
+        stmtStr = "alter table db5.tbl5 add rollup r1(k1);";
+        alterTable(stmtStr, true);
+
+        // truncate table will delete temporary partitions
+        truncateStr = "truncate table db5.tbl5";
+        truncateTableStmt = (TruncateTableStmt) UtFrameUtils.parseAndAnalyzeStmt(truncateStr, ctx);
+        Catalog.getCurrentCatalog().truncateTable(truncateTableStmt);
+        checkShowPartitionsResultNum("db5.tbl5", false, 3);
+        checkShowPartitionsResultNum("db5.tbl5", true, 0);
+
+        stmtStr = "alter table db5.tbl5 add rollup r1(k1);";
+        alterTable(stmtStr, false);
+
+        stmtStr = "alter table db5.tbl5 add temporary partition p2 values in ((\"1\",\"beijing\"), (\"1\", \"shanghai\"), (\"2\",\"beijing\"), (\"2\", \"shanghai\"));";
+        alterTable(stmtStr, true);
+
+        // wait rollup finish
+        Map<Long, AlterJobV2> alterJobs = Catalog.getCurrentCatalog().getRollupHandler().getAlterJobsV2();
+        for (AlterJobV2 alterJobV2 : alterJobs.values()) {
+            while (!alterJobV2.getJobState().isFinalState()) {
+                System.out.println(
+                        "alter job " + alterJobV2.getDbId() + " is running. state: " + alterJobV2.getJobState());
+                Thread.sleep(5000);
+            }
+            System.out.println("alter job " + alterJobV2.getDbId() + " is done. state: " + alterJobV2.getJobState());
+            Assert.assertEquals(AlterJobV2.JobState.FINISHED, alterJobV2.getJobState());
+        }
+        // waiting table state to normal
+        Thread.sleep(500);
+
+        stmtStr = "alter table db5.tbl5 add temporary partition p2 values in ((\"1\",\"beijing\"), (\"1\", \"shanghai\"), (\"2\",\"beijing\"), (\"2\", \"shanghai\"));";
+        alterTable(stmtStr, false);
+
+        TempPartitions tempPartitions = Deencapsulation.getField(tbl5, "tempPartitions");
+        testSerializeTempPartitions(tempPartitions);
+
+        stmtStr = "alter table db5.tbl5 replace partition (tp1, tp2) with temporary partition (p2);";
+        alterTable(stmtStr, false);
+        checkShowPartitionsResultNum("db5.tbl5", false, 2);
+        checkShowPartitionsResultNum("db5.tbl5", true, 0);
+        checkPartitionExist(tbl5, "p2", false, true);
+        checkPartitionExist(tbl5, "tp3", false, true);
+        checkPartitionExist(tbl5, "tp1", false, false);
+        checkPartitionExist(tbl5, "tp2", false, false);
+        checkPartitionExist(tbl5, "p2", true, false);
+
+        checkTablet("db5.tbl5", "p2", false, 2);
+        checkTablet("db5.tbl5", "tp3", false, 2);
+
+        // for now, we have 2 partitions: p2, tp3, (("1","beijing"), ("1", "shanghai"), ("2","beijing"), ("2", "shanghai")), ('7', '8', '9'). 0 temp partition.
+        stmtStr = "alter table db5.tbl5 add temporary partition tp4 values in ((\"1\",\"beijing\"), (\"1\", \"shanghai\"), (\"2\",\"beijing\"), (\"2\", \"shanghai\")) ('in_memory' = 'true') distributed by hash(k1) buckets 3";
+        alterTable(stmtStr, true);
+        stmtStr = "alter table db5.tbl5 add temporary partition tp4 values in ((\"1\",\"beijing\"), (\"1\", \"shanghai\"), (\"2\",\"beijing\"), (\"2\", \"shanghai\")) ('in_memory' = 'true', 'replication_num' = '2') distributed by hash(k2) buckets 3";
+        alterTable(stmtStr, true);
+        stmtStr = "alter table db5.tbl5 add temporary partition tp4 values in ((\"1\",\"beijing\"), (\"1\", \"shanghai\"), (\"2\",\"beijing\"), (\"2\", \"shanghai\")) ('in_memory' = 'true', 'replication_num' = '1') distributed by hash(k2) buckets 3";
+        alterTable(stmtStr, false);
+
+        Partition p2 = tbl5.getPartition("p2");
+        Assert.assertNotNull(p2);
+        Assert.assertFalse(tbl5.getPartitionInfo().getIsInMemory(p2.getId()));
+        Assert.assertEquals(1, p2.getDistributionInfo().getBucketNum());
+
+        stmtStr = "alter table db5.tbl5 replace partition (p2) with temporary partition (tp4)";
+        alterTable(stmtStr, false);
+
+        // for now, we have 2 partitions: p2, tp3, (("1","beijing"), ("1", "shanghai"), ("2","beijing"), ("2", "shanghai")), ('7', '8', '9'). 0 temp partition. and p2 bucket is 3, 'in_memory' is true.
+        p2 = tbl5.getPartition("p2");
+        Assert.assertNotNull(p2);
+        Assert.assertTrue(tbl5.getPartitionInfo().getIsInMemory(p2.getId()));
+        Assert.assertEquals(3, p2.getDistributionInfo().getBucketNum());
+
+        stmtStr = "alter table db5.tbl5 add temporary partition tp1 values in ((\"1\",\"beijing\"), (\"1\", \"shanghai\"));";
+        alterTable(stmtStr, false);
+        stmtStr = "alter table db5.tbl5 add temporary partition tp2 values in ((\"2\",\"beijing\"), (\"2\", \"shanghai\"));";
+        alterTable(stmtStr, false);
+
+        checkShowPartitionsResultNum("db5.tbl5", false, 2);
+        checkShowPartitionsResultNum("db5.tbl5", true, 2);
+        checkPartitionExist(tbl5, "p2", false, true);
+        checkPartitionExist(tbl5, "tp3", false, true);
+        checkPartitionExist(tbl5, "tp1", true, true);
+        checkPartitionExist(tbl5, "tp2", true, true);
+
+        stmtStr = "alter table db5.tbl5 replace partition (p2) with temporary partition (tp1, tp2)";
+        alterTable(stmtStr, false);
+
+        checkShowPartitionsResultNum("db5.tbl5", false, 3);
+        checkShowPartitionsResultNum("db5.tbl5", true, 0);
+        checkPartitionExist(tbl5, "tp1", false, true);
+        checkPartitionExist(tbl5, "tp2", false, true);
+        checkPartitionExist(tbl5, "tp3", false, true);
+        checkPartitionExist(tbl5, "tp1", true, false);
+        checkPartitionExist(tbl5, "tp2", true, false);
+
+    }
     
     private void testSerializeOlapTable(OlapTable tbl) throws IOException, AnalysisException {
         // 1. Write objects to file
diff --git a/fe/fe-core/src/test/java/org/apache/doris/common/util/ListUtilTest.java b/fe/fe-core/src/test/java/org/apache/doris/common/util/ListUtilTest.java
index 7585818..342fc78 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/common/util/ListUtilTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/common/util/ListUtilTest.java
@@ -18,15 +18,52 @@
 package org.apache.doris.common.util;
 
 import com.google.common.collect.Lists;
+import org.apache.doris.analysis.PartitionValue;
+import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.ListPartitionItem;
+import org.apache.doris.catalog.PartitionItem;
+import org.apache.doris.catalog.PartitionKey;
+import org.apache.doris.catalog.PrimitiveType;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.DdlException;
 import org.junit.Assert;
+import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
 
+import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 
 public class ListUtilTest {
 
+    private static PartitionKey pk1;
+    private static PartitionKey pk2;
+    private static PartitionKey pk3;
+    private static List<PartitionKey> listA = new ArrayList<>();
+    private static List<PartitionKey> listB = new ArrayList<>();
+    private static List<PartitionKey> listC = new ArrayList<>();
+
+    @BeforeClass
+    public static void setUp() throws AnalysisException {
+        Column charString = new Column("char", PrimitiveType.CHAR);
+        Column varchar = new Column("varchar", PrimitiveType.VARCHAR);
+
+        pk1 = PartitionKey.createPartitionKey(Arrays.asList(new PartitionValue("beijing")), Arrays.asList(charString));
+        pk2 = PartitionKey.createPartitionKey(Arrays.asList(new PartitionValue("shanghai")), Arrays.asList(varchar));
+        pk3 = PartitionKey.createPartitionKey(Arrays.asList(new PartitionValue("tianjin")), Arrays.asList(varchar));
+
+        listA.add(pk1);
+
+        listB.add(pk1);
+        listB.add(pk2);
+
+        listC.add(pk1);
+        listC.add(pk2);
+        listC.add(pk3);
+    }
+
     @Rule
     public ExpectedException expectedEx = ExpectedException.none();
 
@@ -98,4 +135,31 @@ public class ListUtilTest {
 
         ListUtil.splitBySize(lists, expectSize);
     }
+
+    @Test
+    public void testListsMatchNormal() throws DdlException {
+
+        List<PartitionItem> list1 = Arrays.asList(new ListPartitionItem(listA), new ListPartitionItem(listB));
+        List<PartitionItem> list2 = Arrays.asList(new ListPartitionItem(listA), new ListPartitionItem(listB));
+
+        ListUtil.checkPartitionKeyListsMatch(list1, list2);
+
+    }
+
+    @Test(expected = DdlException.class)
+    public void testListsMatchSameSize() throws DdlException {
+        List<PartitionItem> list1 = Arrays.asList(new ListPartitionItem(listA), new ListPartitionItem(listB));
+        List<PartitionItem> list2 = Arrays.asList(new ListPartitionItem(listA), new ListPartitionItem(listC));
+
+        ListUtil.checkPartitionKeyListsMatch(list1, list2);
+    }
+
+    @Test(expected = DdlException.class)
+    public void testListMatchDiffSize() throws DdlException {
+        List<PartitionItem> list1 = Arrays.asList(new ListPartitionItem(listA), new ListPartitionItem(listB));
+        List<PartitionItem> list2 = Arrays.asList(new ListPartitionItem(listA), new ListPartitionItem(listB),
+                new ListPartitionItem(listC));
+
+        ListUtil.checkPartitionKeyListsMatch(list1, list2);
+    }
 }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLoadPendingTaskTest.java b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLoadPendingTaskTest.java
index a487c18..491773b 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLoadPendingTaskTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/load/loadv2/SparkLoadPendingTaskTest.java
@@ -17,17 +17,11 @@
 
 package org.apache.doris.load.loadv2;
 
-import mockit.Expectations;
-import mockit.Injectable;
-import mockit.Mock;
-import mockit.MockUp;
-import mockit.Mocked;
-
 import org.apache.doris.analysis.BrokerDesc;
 import org.apache.doris.analysis.DataDescription;
 import org.apache.doris.analysis.PartitionKeyDesc;
 import org.apache.doris.analysis.PartitionValue;
-import org.apache.doris.analysis.SingleRangePartitionDesc;
+import org.apache.doris.analysis.SinglePartitionDesc;
 import org.apache.doris.catalog.AggregateType;
 import org.apache.doris.catalog.Catalog;
 import org.apache.doris.catalog.Column;
@@ -55,6 +49,7 @@ import org.apache.doris.load.loadv2.etl.EtlJobConfig.EtlIndex;
 import org.apache.doris.load.loadv2.etl.EtlJobConfig.EtlPartition;
 import org.apache.doris.load.loadv2.etl.EtlJobConfig.EtlPartitionInfo;
 import org.apache.doris.load.loadv2.etl.EtlJobConfig.EtlTable;
+
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
@@ -64,6 +59,12 @@ import org.junit.Test;
 import java.util.List;
 import java.util.Map;
 
+import mockit.Expectations;
+import mockit.Injectable;
+import mockit.Mock;
+import mockit.MockUp;
+import mockit.Mocked;
+
 public class SparkLoadPendingTaskTest {
 
     @Test
@@ -231,12 +232,12 @@ public class SparkLoadPendingTaskTest {
         int partitionColumnIndex = 0;
         List<Partition> partitions = Lists.newArrayList(partition1, partition2);
         RangePartitionInfo partitionInfo = new RangePartitionInfo(Lists.newArrayList(columns.get(partitionColumnIndex)));
-        PartitionKeyDesc partitionKeyDesc1 = new PartitionKeyDesc(Lists.newArrayList(new PartitionValue("10")));
-        SingleRangePartitionDesc partitionDesc1 = new SingleRangePartitionDesc(false, "p1", partitionKeyDesc1, null);
+        PartitionKeyDesc partitionKeyDesc1 = PartitionKeyDesc.createLessThan(Lists.newArrayList(new PartitionValue("10")));
+        SinglePartitionDesc partitionDesc1 = new SinglePartitionDesc(false, "p1", partitionKeyDesc1, null);
         partitionDesc1.analyze(1, null);
         partitionInfo.handleNewSinglePartitionDesc(partitionDesc1, partition1Id, false);
-        PartitionKeyDesc partitionKeyDesc2 = new PartitionKeyDesc(Lists.newArrayList(new PartitionValue("20")));
-        SingleRangePartitionDesc partitionDesc2 = new SingleRangePartitionDesc(false, "p2", partitionKeyDesc2, null);
+        PartitionKeyDesc partitionKeyDesc2 = PartitionKeyDesc.createLessThan(Lists.newArrayList(new PartitionValue("20")));
+        SinglePartitionDesc partitionDesc2 = new SinglePartitionDesc(false, "p2", partitionKeyDesc2, null);
         partitionDesc2.analyze(1, null);
         partitionInfo.handleNewSinglePartitionDesc(partitionDesc2, partition2Id, false);
 
diff --git a/fe/fe-core/src/test/java/org/apache/doris/planner/OlapTableSinkTest.java b/fe/fe-core/src/test/java/org/apache/doris/planner/OlapTableSinkTest.java
index 57964ac..0e4e764 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/planner/OlapTableSinkTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/planner/OlapTableSinkTest.java
@@ -23,6 +23,7 @@ import org.apache.doris.analysis.SlotDescriptor;
 import org.apache.doris.analysis.TupleDescriptor;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.HashDistributionInfo;
+import org.apache.doris.catalog.ListPartitionInfo;
 import org.apache.doris.catalog.MaterializedIndex;
 import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.catalog.Partition;
@@ -157,4 +158,38 @@ public class OlapTableSinkTest {
         LOG.info("sink is {}", sink.toThrift());
         LOG.info("{}", sink.getExplainString("", TExplainLevel.NORMAL));
     }
+
+    @Test
+    public void testListPartition(
+            @Injectable ListPartitionInfo partInfo,
+            @Injectable MaterializedIndex index) throws UserException {
+        TupleDescriptor tuple = getTuple();
+
+        HashDistributionInfo distInfo = new HashDistributionInfo(
+                2, Lists.newArrayList(new Column("k1", PrimitiveType.BIGINT)));
+
+        Column partKey = new Column("k2", PrimitiveType.VARCHAR);
+        PartitionKey key = PartitionKey.createPartitionKey(Lists.newArrayList(new PartitionValue("123")), Lists.newArrayList(partKey));
+        Partition p1 = new Partition(1, "p1", index, distInfo);
+        Partition p2 = new Partition(2, "p2", index, distInfo);
+
+        new Expectations() {{
+            dstTable.getId(); result = 1;
+            dstTable.getPartitionInfo(); result = partInfo;
+            partInfo.getType(); result = PartitionType.LIST;
+            partInfo.getPartitionColumns(); result = Lists.newArrayList(partKey);
+            dstTable.getPartitions(); result = Lists.newArrayList(p1, p2);
+            dstTable.getPartition(p1.getId()); result = p1;
+        }};
+
+        OlapTableSink sink = new OlapTableSink(dstTable, tuple, Lists.newArrayList(p1.getId()));
+        sink.init(new TUniqueId(1, 2), 3, 4, 1000);
+        try {
+            sink.complete();
+        } catch (UserException e) {
+
+        }
+        LOG.info("sink is {}", sink.toThrift());
+        LOG.info("{}", sink.getExplainString("", TExplainLevel.NORMAL));
+    }
 }
diff --git a/gensrc/thrift/Descriptors.thrift b/gensrc/thrift/Descriptors.thrift
index ba5de78..4224e86 100644
--- a/gensrc/thrift/Descriptors.thrift
+++ b/gensrc/thrift/Descriptors.thrift
@@ -133,6 +133,7 @@ struct TOlapTablePartition {
 
     6: optional list<Exprs.TExprNode> start_keys
     7: optional list<Exprs.TExprNode> end_keys
+    8: optional list<list<Exprs.TExprNode>> in_keys
 }
 
 struct TOlapTablePartitionParam {
diff --git a/gensrc/thrift/Partitions.thrift b/gensrc/thrift/Partitions.thrift
index 9a3305b..8eecbb4 100644
--- a/gensrc/thrift/Partitions.thrift
+++ b/gensrc/thrift/Partitions.thrift
@@ -34,6 +34,9 @@ enum TPartitionType {
   // ordered partition on a list of exprs
   // (partition bounds don't overlap)
   RANGE_PARTITIONED,
+  
+  // partition on a list of exprs
+  LIST_PARTITIONED,
 
   // unordered partition on a set of exprs
   // (only use in bucket shuffle join)

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