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/09/04 02:59:52 UTC

[incubator-doris] branch master updated: [Feature][ResourceTag] Support Resource Tag (#6203)

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 7db8841  [Feature][ResourceTag] Support Resource Tag (#6203)
7db8841 is described below

commit 7db8841ae264012f37bb929b0f7ba6ffefe703c8
Author: Mingyu Chen <mo...@gmail.com>
AuthorDate: Sat Sep 4 10:59:35 2021 +0800

    [Feature][ResourceTag] Support Resource Tag (#6203)
    
    #5902
    This CL mainly changes:
    
    1. Support setting tags for BE nodes:
    
        ```
        alter system add backend "1272:9050, 1212:9050" properties("tag.location": "zoneA");
        alter system modify backend "1272:9050, 1212:9050" set ("tag.location": "zoneB");
        ```
        And for compatibility, all BE nodes will be set a "default" tag when upgrading: `"tag.location": "default"`.
    
    2. Create a new class `ReplicaAllocation` to replace the previous `replication_num`.
    
        `ReplicaAllocation` represents the allocation of the replicas of a tablet. It contains a map from
        Tag to number of replicas.
        For example, if user set a table's replication num to 3, it will be converted to a ReplicaAllocation
        like: `"tag.location.default" : "3"`, which means the tablet will have 3 replicas and all of them will be
        allocated in BE nodes with tag "default";
    
    3. Support create table with replication allocation:
    
        ```
        CREATE TABLE example_db.table_hash
        (
        k1 TINYINT
        )
        DISTRIBUTED BY HASH(k1) BUCKETS 32
        PROPERTIES (
            "replication_allocation"="tag.location.zone1:1, tag.location.zone2:2"
        );
        ```
    
        Also support set replica allocation for dynamic tables, and modify replica allocation at runtime.
    
        For compatibility, user can still set "replication_num" = "3", and it will be automatically converted to:
        ` "replication_allocation"="tag.location.default:3"`
    
    4. Support tablet repair and balance based on Tag
    
        1. For tablets of non-colocate table, most of the logic is the same as before,
           but when selecting the destination node for clone, the tag of the node will be considered.
           If the required tag does not exist, it cannot be repaired.
           Similarly, under the condition of ensuring that the replicas are complete, the tablet will be
           reallocated according to the tag or the replicas will be balanced.
    
           Balancing is performed separately within each resource group.
    
        2. For tablets of colocate table, the backends sequence of buckets will be splitted by tag.
           For example, if replica allocation is "tag.location.zone1:1, tag.location.zone2:2",
    
           And zone1 has 2 BE: A, B; zone2 has 3 BE: C, D, F
    
           there will be 2 backend sequences: one is for zone1, and the other is for zone2.
           And one posible seqeunces will be:
    
    
           zone1: [A]   [B]   [A]   [B]
           zone2: [C, D][D, F][F, C][C, D]
    
    5. Support setting tags for user and restrict execution node with tags:
    
    
        ```
        set property for 'cmy' 'resource_tags.location' : 'zone1, zone2';
        ```
    
        After setting, the user 'cmy' can only query data stored on backends with tag zone1 and zone2,
        And query can only be executed on backends with tag zone1 and zone2
    
    
        For compatibility, after upgrading, the property `resource_tags.location` will be empty,
        so that user can still query data stored on any backends.
    
    6. Modify the Unit test frame of FE so that we can created multi backends with different mocked IP in unit test.
    
        This help us to easily test some distributed cases like query, tablet repair and balance
    
    The document will be added in another PR.
    
    Also fix a bug described in #6194
---
 .travis.yml                                        |   2 +-
 fe/fe-core/src/main/cup/sql_parser.cup             |  16 +-
 .../main/java/org/apache/doris/alter/Alter.java    |  22 +-
 .../doris/alter/MaterializedViewHandler.java       |   2 +-
 .../java/org/apache/doris/alter/RollupJob.java     |   2 +-
 .../java/org/apache/doris/alter/RollupJobV2.java   |   3 +-
 .../apache/doris/alter/SchemaChangeHandler.java    |  16 +-
 .../org/apache/doris/alter/SchemaChangeJob.java    |   4 +-
 .../org/apache/doris/alter/SchemaChangeJobV2.java  |   2 +-
 .../java/org/apache/doris/alter/SystemHandler.java |   7 +-
 .../apache/doris/analysis/AddBackendClause.java    |  33 +-
 .../doris/analysis/AlterDatabasePropertyStmt.java  |  53 +++
 .../org/apache/doris/analysis/AlterSystemStmt.java |   3 +-
 .../java/org/apache/doris/analysis/Analyzer.java   |   3 +-
 .../org/apache/doris/analysis/BackendClause.java   |   1 -
 .../doris/analysis/CreateTableAsSelectStmt.java    |   3 +-
 ...BackendClause.java => ModifyBackendClause.java} |  57 +--
 .../doris/analysis/ModifyPartitionClause.java      |  10 +-
 .../analysis/ModifyTablePropertiesClause.java      |  14 +-
 .../org/apache/doris/analysis/RestoreStmt.java     |  21 +-
 .../doris/analysis/ShowDynamicPartitionStmt.java   |   1 +
 .../org/apache/doris/analysis/ShowRestoreStmt.java |   2 +-
 .../apache/doris/analysis/SinglePartitionDesc.java |  24 +-
 .../org/apache/doris/backup/BackupHandler.java     |   2 +-
 .../java/org/apache/doris/backup/Repository.java   |  31 +-
 .../java/org/apache/doris/backup/RestoreJob.java   |  66 +--
 .../java/org/apache/doris/catalog/Catalog.java     | 178 ++++---
 .../apache/doris/catalog/CatalogRecycleBin.java    |  28 +-
 .../apache/doris/catalog/ColocateGroupSchema.java  |  38 +-
 .../apache/doris/catalog/ColocateTableIndex.java   | 312 ++++++-------
 .../java/org/apache/doris/catalog/Database.java    |  10 +-
 .../org/apache/doris/catalog/DatabaseProperty.java |  62 +++
 .../doris/catalog/DynamicPartitionProperty.java    |  30 +-
 .../org/apache/doris/catalog/MetadataViewer.java   |   2 +-
 .../java/org/apache/doris/catalog/OlapTable.java   | 135 ++++--
 .../org/apache/doris/catalog/PartitionInfo.java    |  66 +--
 .../apache/doris/catalog/ReplicaAllocation.java    | 132 ++++++
 .../org/apache/doris/catalog/TableProperty.java    |  69 ++-
 .../main/java/org/apache/doris/catalog/Tablet.java |  54 ++-
 .../apache/doris/clone/BackendLoadStatistic.java   |  12 +-
 .../org/apache/doris/clone/BeLoadRebalancer.java   |   9 +-
 .../apache/doris/clone/ClusterLoadStatistic.java   |  30 +-
 .../clone/ColocateTableCheckerAndBalancer.java     | 143 ++++--
 .../doris/clone/DynamicPartitionScheduler.java     |  10 +-
 .../java/org/apache/doris/clone/MovesCacheMap.java |  56 ++-
 .../apache/doris/clone/PartitionRebalancer.java    |  33 +-
 .../java/org/apache/doris/clone/Rebalancer.java    |  16 +-
 .../java/org/apache/doris/clone/TabletChecker.java |   8 +-
 .../org/apache/doris/clone/TabletSchedCtx.java     |  38 +-
 .../org/apache/doris/clone/TabletScheduler.java    | 193 +++++---
 .../apache/doris/clone/TabletSchedulerStat.java    |   2 +
 .../java/org/apache/doris/common/ErrorCode.java    |  38 +-
 .../java/org/apache/doris/common/FeConstants.java  |   2 +
 .../org/apache/doris/common/FeMetaVersion.java     |   4 +-
 .../apache/doris/common/proc/BackendsProcDir.java  |   4 +-
 .../doris/common/proc/ClusterBalanceProcDir.java   |   2 +-
 ...StatByMedium.java => ClusterLoadStatByTag.java} |  37 +-
 ...ium.java => ClusterLoadStatByTagAndMedium.java} |  13 +-
 .../common/proc/ClusterLoadStatisticProcDir.java   |  13 +-
 .../proc/ColocationGroupBackendSeqsProcNode.java   |  34 +-
 .../doris/common/proc/ColocationGroupProcDir.java  |   7 +-
 .../doris/common/proc/FrontendsProcNode.java       |  12 +-
 .../doris/common/proc/PartitionsProcDir.java       |  11 +-
 .../apache/doris/common/proc/StatisticProcDir.java |   5 +-
 .../doris/common/util/DynamicPartitionUtil.java    |  17 +-
 .../apache/doris/common/util/PropertyAnalyzer.java |  94 +++-
 .../doris/consistency/CheckConsistencyJob.java     |   6 +-
 .../doris/consistency/ConsistencyChecker.java      |   2 +-
 .../org/apache/doris/deploy/DeployManager.java     |   3 +-
 .../doris/http/meta/ColocateMetaService.java       |  11 +-
 .../org/apache/doris/http/rest/LoadAction.java     |   3 +-
 .../doris/httpv2/meta/ColocateMetaService.java     |  24 +-
 .../org/apache/doris/httpv2/rest/LoadAction.java   |   7 +-
 .../apache/doris/httpv2/util/LoadSubmitter.java    |  10 +-
 .../org/apache/doris/journal/JournalEntity.java    |   6 +-
 .../java/org/apache/doris/load/DeleteHandler.java  |   3 +-
 .../java/org/apache/doris/load/LoadChecker.java    |   8 +-
 .../org/apache/doris/load/loadv2/SparkLoadJob.java |   8 +-
 .../org/apache/doris/master/ReportHandler.java     |   9 +-
 .../mysql/privilege/CommonUserProperties.java      |  19 +-
 .../org/apache/doris/mysql/privilege/PaloAuth.java |  17 +-
 .../apache/doris/mysql/privilege/UserProperty.java |  58 ++-
 .../doris/mysql/privilege/UserPropertyMgr.java     |  12 +-
 .../mysql/privilege/UserResourceProperty.java      |  31 ++
 .../apache/doris/persist/ColocatePersistInfo.java  |  60 ++-
 .../java/org/apache/doris/persist/EditLog.java     |   9 +
 .../apache/doris/persist/ModifyPartitionInfo.java  |  54 ++-
 .../org/apache/doris/persist/OperationType.java    |   1 +
 .../apache/doris/persist/PartitionPersistInfo.java |  22 +-
 .../org/apache/doris/planner/OlapScanNode.java     |  23 +-
 .../org/apache/doris/planner/OlapTableSink.java    |   6 +-
 .../java/org/apache/doris/qe/ConnectContext.java   |  31 +-
 .../java/org/apache/doris/qe/ConnectProcessor.java |   4 +-
 .../main/java/org/apache/doris/qe/Coordinator.java |  33 +-
 .../main/java/org/apache/doris/qe/DdlExecutor.java |   5 +-
 .../apache/doris/qe/InsertStreamTxnExecutor.java   |   4 +-
 .../java/org/apache/doris/qe/MultiLoadMgr.java     |   6 +-
 .../java/org/apache/doris/qe/ShowExecutor.java     |  18 +-
 .../java/org/apache/doris/qe/SimpleScheduler.java  |  12 +
 .../java/org/apache/doris/qe/StmtExecutor.java     |   2 +-
 .../main/java/org/apache/doris/resource/Tag.java   |  20 +-
 .../main/java/org/apache/doris/system/Backend.java |  19 +-
 .../java/org/apache/doris/system/HeartbeatMgr.java |  31 +-
 .../org/apache/doris/system/SystemInfoService.java | 126 ++++-
 .../java/org/apache/doris/task/AgentBatchTask.java |   4 +-
 .../doris/transaction/DatabaseTransactionMgr.java  |   4 +-
 .../org/apache/doris/alter/AlterJobV2Test.java     |   2 +-
 .../java/org/apache/doris/alter/AlterTest.java     |  34 +-
 .../org/apache/doris/alter/BatchRollupJobTest.java |   2 +-
 .../doris/analysis/AdminSetConfigStmtTest.java     |   2 +-
 .../doris/analysis/AdminShowReplicaTest.java       |   2 +-
 .../org/apache/doris/analysis/AggregateTest.java   |   2 +-
 .../org/apache/doris/analysis/InsertStmtTest.java  |   2 +-
 .../doris/analysis/ListPartitionPrunerTest.java    |   2 +-
 .../org/apache/doris/analysis/QueryStmtTest.java   |   6 +-
 .../org/apache/doris/analysis/SelectStmtTest.java  |   2 +-
 .../org/apache/doris/analysis/SetVariableTest.java |   2 +-
 .../apache/doris/analysis/ShowViewStmtTest.java    |   3 +-
 .../apache/doris/analysis/StmtRewriterTest.java    |   2 +-
 .../analysis/TableNameComparedLowercaseTest.java   |   2 +-
 .../analysis/TableNameStoredLowercaseTest.java     |   2 +-
 .../org/apache/doris/backup/CatalogMocker.java     |  13 +-
 .../org/apache/doris/backup/RepositoryTest.java    |  13 +-
 .../org/apache/doris/backup/RestoreJobTest.java    |  14 +-
 .../doris/blockrule/SqlBlockRuleMgrTest.java       |   2 +-
 .../org/apache/doris/catalog/AdminStmtTest.java    |   2 +-
 .../apache/doris/catalog/CatalogOperationTest.java |   2 +-
 .../org/apache/doris/catalog/CatalogTestUtil.java  |   2 +-
 .../doris/catalog/ColocateTableIndexTest.java      |  34 ++
 .../apache/doris/catalog/ColocateTableTest.java    |  27 +-
 .../apache/doris/catalog/CreateEncryptKeyTest.java |   3 +-
 .../apache/doris/catalog/CreateFunctionTest.java   |   2 +-
 .../apache/doris/catalog/CreateTableLikeTest.java  |   8 +-
 .../org/apache/doris/catalog/CreateTableTest.java  |   8 +-
 .../org/apache/doris/catalog/CreateViewTest.java   |   2 +-
 .../java/org/apache/doris/catalog/DropDbTest.java  |   4 +-
 .../apache/doris/catalog/DropPartitionTest.java    |   2 +-
 .../org/apache/doris/catalog/DropTableTest.java    |   3 +-
 .../doris/catalog/DynamicPartitionTableTest.java   |   6 +-
 .../apache/doris/catalog/ModifyBackendTagTest.java | 187 ++++++++
 .../java/org/apache/doris/catalog/RecoverTest.java |   2 +-
 .../doris/catalog/ReplicaAllocationTest.java       | 159 +++++++
 .../apache/doris/catalog/TablePropertyTest.java    |   8 +
 .../java/org/apache/doris/catalog/TabletTest.java  |  13 +-
 .../apache/doris/catalog/TempPartitionTest.java    |   2 +-
 .../org/apache/doris/catalog/UserPropertyTest.java |   3 +-
 .../doris/clone/ClusterLoadStatisticsTest.java     |   3 +-
 .../clone/ColocateTableCheckerAndBalancerTest.java | 138 ++++--
 .../java/org/apache/doris/clone/RebalanceTest.java |  36 +-
 .../doris/clone/TabletRepairAndBalanceTest.java    | 515 +++++++++++++++++++++
 .../org/apache/doris/clone/TabletSchedCtxTest.java |   8 +-
 .../doris/cluster/SystemInfoServiceTest.java       |   7 +-
 .../org/apache/doris/common/ExceptionChecker.java  |   1 +
 .../org/apache/doris/common/util/UnitTestUtil.java |   5 +-
 .../org/apache/doris/http/DorisHttpTestCase.java   |   5 +-
 .../apache/doris/load/loadv2/SparkLoadJobTest.java |   9 +-
 .../doris/load/sync/canal/CanalSyncDataTest.java   |   5 +-
 .../persist/BatchModifyPartitionsInfoTest.java     |   4 +-
 .../org/apache/doris/planner/ColocatePlanTest.java |   9 +-
 .../doris/planner/DistributedPlannerTest.java      |   2 +-
 .../planner/MaterializedViewFunctionTest.java      |   2 +-
 .../apache/doris/planner/OlapTableSinkTest.java    |   3 +-
 .../java/org/apache/doris/planner/PlannerTest.java |   2 +-
 .../org/apache/doris/planner/QueryPlanTest.java    |   6 +-
 .../apache/doris/planner/ResourceTagQueryTest.java | 293 ++++++++++++
 .../org/apache/doris/plugin/PluginMgrTest.java     |  14 +-
 .../apache/doris/qe/AuditEventProcessorTest.java   |   7 +-
 .../java/org/apache/doris/qe/MultiLoadMgrTest.java |  13 +-
 .../java/org/apache/doris/qe/VariableMgrTest.java  |   6 +-
 .../ExtractCommonFactorsRuleFunctionTest.java      |   7 +-
 .../org/apache/doris/utframe/AnotherDemoTest.java  |   3 +-
 .../{DemoTest.java => DemoMultiBackendsTest.java}  |  97 +++-
 .../java/org/apache/doris/utframe/DemoTest.java    |   2 +-
 .../apache/doris/utframe/MockedBackendFactory.java |  38 +-
 .../org/apache/doris/utframe/UtFrameUtils.java     |  28 +-
 175 files changed, 3743 insertions(+), 1203 deletions(-)

diff --git a/.travis.yml b/.travis.yml
index 40a7b28..334615b 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -35,7 +35,7 @@ before_script:
   - rm -rf site-repo
   
 script:
-  - npm run build
+  - travis_wait 30 npm run build
 
 after_success:
   - if [ "$TRAVIS_EVENT_TYPE" != "push" ]; then exit 0; fi
diff --git a/fe/fe-core/src/main/cup/sql_parser.cup b/fe/fe-core/src/main/cup/sql_parser.cup
index 711f32a..f4e573d 100644
--- a/fe/fe-core/src/main/cup/sql_parser.cup
+++ b/fe/fe-core/src/main/cup/sql_parser.cup
@@ -823,6 +823,10 @@ alter_stmt ::=
     {:
         RESULT = new AlterDatabaseRename(dbName, newDbName);
     :}
+    | KW_ALTER KW_DATABASE ident:dbName KW_SET KW_PROPERTIES LPAREN key_value_map:map RPAREN
+    {:
+        RESULT = new AlterDatabasePropertyStmt(dbName, map);
+    :}
     | KW_ALTER KW_ROUTINE KW_LOAD KW_FOR job_label:jobLabel opt_properties:jobProperties
       opt_datasource_properties:datasourceProperties
     {:
@@ -1084,13 +1088,13 @@ opt_enable_feature_properties ::=
     ;
 
 alter_system_clause ::=
-    KW_ADD KW_BACKEND string_list:hostPorts
+    KW_ADD KW_BACKEND string_list:hostPorts opt_properties:properties
     {:
-        RESULT = new AddBackendClause(hostPorts, false);
+        RESULT = new AddBackendClause(hostPorts, false, properties);
     :}
     | KW_ADD KW_FREE KW_BACKEND string_list:hostPorts
     {:
-        RESULT = new AddBackendClause(hostPorts, true);
+        RESULT = new AddBackendClause(hostPorts, true, Maps.newHashMap());
     :}
     | KW_ADD KW_BACKEND KW_TO ident:clusterName string_list:hostPorts
     {:
@@ -1141,7 +1145,11 @@ alter_system_clause ::=
     | KW_SET KW_LOAD KW_ERRORS KW_HUB opt_properties:properties
     {:
         RESULT = new AlterLoadErrorUrlClause(properties);
-    :} 
+    :}
+	| KW_MODIFY KW_BACKEND string_list:hostPorts KW_SET LPAREN key_value_map:properties RPAREN
+	{:
+		RESULT = new ModifyBackendClause(hostPorts, properties);
+	:}
     ;
 
 // Sync Stmt
diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java
index fe3537a..bf1a0c8 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/Alter.java
@@ -45,6 +45,7 @@ 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.ReplicaAllocation;
 import org.apache.doris.catalog.Table;
 import org.apache.doris.catalog.Table.TableType;
 import org.apache.doris.catalog.View;
@@ -566,11 +567,10 @@ public class Alter {
 
         // get value from properties here
         // 1. data property
-        DataProperty newDataProperty =
-                PropertyAnalyzer.analyzeDataProperty(properties, null);
-        // 2. replication num
-        short newReplicationNum =
-                PropertyAnalyzer.analyzeReplicationNum(properties, (short) -1);
+        DataProperty newDataProperty = PropertyAnalyzer.analyzeDataProperty(properties, null);
+        // 2. replica allocation
+        ReplicaAllocation replicaAlloc = PropertyAnalyzer.analyzeReplicaAllocation(properties, "");
+        Catalog.getCurrentSystemInfo().checkReplicaAllocation(db.getClusterName(), replicaAlloc);
         // 3. in memory
         boolean newInMemory = PropertyAnalyzer.analyzeBooleanProp(properties,
                 PropertyAnalyzer.PROPERTIES_INMEMORY, false);
@@ -586,9 +586,9 @@ public class Alter {
             if (newDataProperty != null) {
                 partitionInfo.setDataProperty(partition.getId(), newDataProperty);
             }
-            // 2. replication num
-            if (newReplicationNum != (short) -1) {
-                partitionInfo.setReplicationNum(partition.getId(), newReplicationNum);
+            // 2. replica allocation
+            if (!replicaAlloc.isNotSet()) {
+                partitionInfo.setReplicaAllocation(partition.getId(), replicaAlloc);
             }
             // 3. in memory
             boolean oldInMemory = partitionInfo.getIsInMemory(partition.getId());
@@ -600,7 +600,7 @@ public class Alter {
                 partitionInfo.setTabletType(partition.getId(), tTabletType);
             }
             ModifyPartitionInfo info = new ModifyPartitionInfo(db.getId(), olapTable.getId(), partition.getId(),
-                    newDataProperty, newReplicationNum, hasInMemory ? newInMemory : oldInMemory);
+                    newDataProperty, replicaAlloc, hasInMemory ? newInMemory : oldInMemory);
             modifyPartitionInfos.add(info);
         }
 
@@ -618,8 +618,8 @@ public class Alter {
             if (info.getDataProperty() != null) {
                 partitionInfo.setDataProperty(info.getPartitionId(), info.getDataProperty());
             }
-            if (info.getReplicationNum() != (short) -1) {
-                partitionInfo.setReplicationNum(info.getPartitionId(), info.getReplicationNum());
+            if (!info.getReplicaAlloc().isNotSet()) {
+                partitionInfo.setReplicaAllocation(info.getPartitionId(), info.getReplicaAlloc());
             }
             partitionInfo.setIsInMemory(info.getPartitionId(), info.isInMemory());
         } finally {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java
index f1d231a..66c1c53 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/MaterializedViewHandler.java
@@ -366,7 +366,7 @@ public class MaterializedViewHandler extends AlterHandler {
             MaterializedIndex mvIndex = new MaterializedIndex(mvIndexId, IndexState.SHADOW);
             MaterializedIndex baseIndex = partition.getIndex(baseIndexId);
             TabletMeta mvTabletMeta = new TabletMeta(dbId, tableId, partitionId, mvIndexId, mvSchemaHash, medium);
-            short replicationNum = olapTable.getPartitionInfo().getReplicationNum(partitionId);
+            short replicationNum = olapTable.getPartitionInfo().getReplicaAllocation(partitionId).getTotalReplicaNum();
             for (Tablet baseTablet : baseIndex.getTablets()) {
                 long baseTabletId = baseTablet.getId();
                 long mvTabletId = catalog.getNextId();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJob.java b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJob.java
index 7195d58..9b83815 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJob.java
@@ -655,7 +655,7 @@ public class RollupJob extends AlterJob {
                         continue;
                     }
 
-                    short expectReplicationNum = olapTable.getPartitionInfo().getReplicationNum(partition.getId());
+                    short expectReplicationNum = olapTable.getPartitionInfo().getReplicaAllocation(partition.getId()).getTotalReplicaNum();
                     MaterializedIndex rollupIndex = entry.getValue();
                     for (Tablet rollupTablet : rollupIndex.getTablets()) {
                         // yiguolei: the rollup tablet only contains the replica that is healthy at rollup time
diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java
index ad410b7..28fbd79 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/RollupJobV2.java
@@ -439,7 +439,8 @@ public class RollupJobV2 extends AlterJobV2 implements GsonPostProcessable {
 
                 long visiableVersion = partition.getVisibleVersion();
                 long visiableVersionHash = partition.getVisibleVersionHash();
-                short expectReplicationNum = tbl.getPartitionInfo().getReplicationNum(partition.getId());
+                short expectReplicationNum = tbl.getPartitionInfo().getReplicaAllocation(partitionId).getTotalReplicaNum();
+
 
                 MaterializedIndex rollupIndex = entry.getValue();
                 for (Tablet rollupTablet : rollupIndex.getTablets()) {
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 b54847e..8db63fd 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
@@ -51,6 +51,7 @@ import org.apache.doris.catalog.PartitionInfo;
 import org.apache.doris.catalog.PartitionType;
 import org.apache.doris.catalog.Replica;
 import org.apache.doris.catalog.Replica.ReplicaState;
+import org.apache.doris.catalog.ReplicaAllocation;
 import org.apache.doris.catalog.Table;
 import org.apache.doris.catalog.Tablet;
 import org.apache.doris.catalog.TabletMeta;
@@ -1333,7 +1334,8 @@ public class SchemaChangeHandler extends AlterHandler {
                 MaterializedIndex shadowIndex = new MaterializedIndex(shadowIndexId, IndexState.SHADOW);
                 MaterializedIndex originIndex = partition.getIndex(originIndexId);
                 TabletMeta shadowTabletMeta = new TabletMeta(dbId, tableId, partitionId, shadowIndexId, newSchemaHash, medium);
-                short replicationNum = olapTable.getPartitionInfo().getReplicationNum(partitionId);
+                ReplicaAllocation replicaAlloc = olapTable.getPartitionInfo().getReplicaAllocation(partitionId);
+                Short totalReplicaNum = replicaAlloc.getTotalReplicaNum();
                 for (Tablet originTablet : originIndex.getTablets()) {
                     long originTabletId = originTablet.getId();
                     long shadowTabletId = catalog.getNextId();
@@ -1366,7 +1368,7 @@ public class SchemaChangeHandler extends AlterHandler {
                         healthyReplicaNum++;
                     }
 
-                    if (healthyReplicaNum < replicationNum / 2 + 1) {
+                    if (healthyReplicaNum < totalReplicaNum / 2 + 1) {
                         /*
                          * TODO(cmy): This is a bad design.
                          * Because in the schema change job, we will only send tasks to the shadow replicas that have been created,
@@ -1673,12 +1675,12 @@ public class SchemaChangeHandler extends AlterHandler {
                         }
                         Catalog.getCurrentCatalog().modifyTableDynamicPartition(db, olapTable, properties);
                         return;
-                    } else if (properties.containsKey("default." + PropertyAnalyzer.PROPERTIES_REPLICATION_NUM)) {
-                        Preconditions.checkNotNull(properties.get(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM));
-                        Catalog.getCurrentCatalog().modifyTableDefaultReplicationNum(db, olapTable, properties);
+                    } else if (properties.containsKey("default." + PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION)) {
+                        Preconditions.checkNotNull(properties.get("default." + PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION));
+                        Catalog.getCurrentCatalog().modifyTableDefaultReplicaAllocation(db, olapTable, properties);
                         return;
-                    } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM)) {
-                        Catalog.getCurrentCatalog().modifyTableReplicationNum(db, olapTable, properties);
+                    } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION)) {
+                        Catalog.getCurrentCatalog().modifyTableReplicaAllocation(db, olapTable, properties);
                         return;
                     }
                 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java
index 9bde44b..f781d65 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJob.java
@@ -402,7 +402,7 @@ public class SchemaChangeJob extends AlterJob {
                 List<AgentTask> tasks = new LinkedList<AgentTask>();
                 for (Partition partition : olapTable.getPartitions()) {
                     long partitionId = partition.getId();
-                    short replicationNum = olapTable.getPartitionInfo().getReplicationNum(partitionId);
+                    short replicationNum = olapTable.getPartitionInfo().getReplicaAllocation(partitionId).getTotalReplicaNum();
                     for (Long indexId : this.changedIndexIdToSchema.keySet()) {
                         MaterializedIndex alterIndex = partition.getIndex(indexId);
                         if (alterIndex == null) {
@@ -679,7 +679,7 @@ public class SchemaChangeJob extends AlterJob {
                 boolean hasUnfinishedPartition = false;
                 for (Partition partition : olapTable.getPartitions()) {
                     long partitionId = partition.getId();
-                    short expectReplicationNum = olapTable.getPartitionInfo().getReplicationNum(partition.getId());
+                    short expectReplicationNum = olapTable.getPartitionInfo().getReplicaAllocation(partition.getId()).getTotalReplicaNum();
                     boolean hasUnfinishedIndex = false;
                     for (long indexId : this.changedIndexIdToSchema.keySet()) {
                         MaterializedIndex materializedIndex = partition.getIndex(indexId);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java
index b5bd9a7..1ad1493 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SchemaChangeJobV2.java
@@ -475,7 +475,7 @@ public class SchemaChangeJobV2 extends AlterJobV2 {
 
                 long visiableVersion = partition.getVisibleVersion();
                 long visiableVersionHash = partition.getVisibleVersionHash();
-                short expectReplicationNum = tbl.getPartitionInfo().getReplicationNum(partition.getId());
+                short expectReplicationNum = tbl.getPartitionInfo().getReplicaAllocation(partition.getId()).getTotalReplicaNum();
 
                 Map<Long, MaterializedIndex> shadowIndexMap = partitionIndexMap.row(partitionId);
                 for (Map.Entry<Long, MaterializedIndex> entry : shadowIndexMap.entrySet()) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/alter/SystemHandler.java b/fe/fe-core/src/main/java/org/apache/doris/alter/SystemHandler.java
index 33ec411..f8a59bc 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/alter/SystemHandler.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/alter/SystemHandler.java
@@ -29,6 +29,7 @@ import org.apache.doris.analysis.DecommissionBackendClause;
 import org.apache.doris.analysis.DropBackendClause;
 import org.apache.doris.analysis.DropFollowerClause;
 import org.apache.doris.analysis.DropObserverClause;
+import org.apache.doris.analysis.ModifyBackendClause;
 import org.apache.doris.analysis.ModifyBrokerClause;
 import org.apache.doris.catalog.Catalog;
 import org.apache.doris.catalog.Database;
@@ -143,8 +144,8 @@ public class SystemHandler extends AlterHandler {
                     && Catalog.getCurrentCatalog().getCluster(destClusterName) == null) {
                 throw new DdlException("Cluster: " + destClusterName + " does not exist.");
             }
-            Catalog.getCurrentSystemInfo().addBackends(addBackendClause.getHostPortPairs(), 
-                addBackendClause.isFree(), addBackendClause.getDestCluster());
+            Catalog.getCurrentSystemInfo().addBackends(addBackendClause.getHostPortPairs(),
+                    addBackendClause.isFree(), addBackendClause.getDestCluster(), addBackendClause.getTag());
         } else if (alterClause instanceof DropBackendClause) {
             // drop backend
             DropBackendClause dropBackendClause = (DropBackendClause) alterClause;
@@ -188,6 +189,8 @@ public class SystemHandler extends AlterHandler {
         } else if (alterClause instanceof AlterLoadErrorUrlClause) {
             AlterLoadErrorUrlClause clause = (AlterLoadErrorUrlClause) alterClause;
             Catalog.getCurrentCatalog().getLoadInstance().setLoadErrorHubInfo(clause.getProperties());
+        } else if (alterClause instanceof ModifyBackendClause) {
+            Catalog.getCurrentSystemInfo().modifyBackends(((ModifyBackendClause) alterClause));
         } else {
             Preconditions.checkState(false, alterClause.getClass());
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AddBackendClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AddBackendClause.java
index 642732f..302bc99 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AddBackendClause.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AddBackendClause.java
@@ -17,27 +17,38 @@
 
 package org.apache.doris.analysis;
 
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.util.PropertyAnalyzer;
+import org.apache.doris.resource.Tag;
+
 import com.google.common.base.Strings;
+import com.google.common.collect.Maps;
 
 import java.util.List;
+import java.util.Map;
 
 public class AddBackendClause extends BackendClause {
-
     // be in free state is not owned by any cluster
     protected boolean isFree;
     // cluster that backend will be added to 
     protected String destCluster;
+    protected Map<String, String> properties = Maps.newHashMap();
+    private Tag tag;
 
     public AddBackendClause(List<String> hostPorts) {
         super(hostPorts);
         this.isFree = true;
         this.destCluster = "";
     }
-   
-    public AddBackendClause(List<String> hostPorts, boolean isFree) {
+
+    public AddBackendClause(List<String> hostPorts, boolean isFree, Map<String, String> properties) {
         super(hostPorts);
         this.isFree = isFree;
         this.destCluster = "";
+        this.properties = properties;
+        if (this.properties == null) {
+            this.properties = Maps.newHashMap();
+        }
     }
 
     public AddBackendClause(List<String> hostPorts, String destCluster) {
@@ -46,6 +57,21 @@ public class AddBackendClause extends BackendClause {
         this.destCluster = destCluster;
     }
 
+    public Tag getTag() {
+        return tag;
+    }
+
+    @Override
+    public void analyze(Analyzer analyzer) throws AnalysisException {
+        super.analyze(analyzer);
+        tag = PropertyAnalyzer.analyzeBackendTagProperties(properties);
+    }
+
+    @Override
+    public Map<String, String> getProperties() {
+        return properties;
+    }
+
     @Override
     public String toSql() {
         StringBuilder sb = new StringBuilder();
@@ -77,3 +103,4 @@ public class AddBackendClause extends BackendClause {
     }
 
 }
+
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterDatabasePropertyStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterDatabasePropertyStmt.java
new file mode 100644
index 0000000..09ea28c
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterDatabasePropertyStmt.java
@@ -0,0 +1,53 @@
+// 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.UserException;
+import org.apache.doris.common.util.PrintableMap;
+
+import java.util.Map;
+
+public class AlterDatabasePropertyStmt extends DdlStmt {
+    private String dbName;
+    private Map<String, String> properties;
+
+    public AlterDatabasePropertyStmt(String dbName, Map<String, String> properties) {
+        this.dbName = dbName;
+        this.properties = properties;
+    }
+
+    public String getDbName() {
+        return dbName;
+    }
+
+    public Map<String, String> getProperties() {
+        return properties;
+    }
+
+    @Override
+    public void analyze(Analyzer analyzer) throws UserException {
+        super.analyze(analyzer);
+        // TODO: add some property check
+    }
+
+    @Override
+    public String toSql() {
+        return "ALTER DATABASE " + dbName + " SET PROPERTIES ("
+                + new PrintableMap<String, String>(properties, "=", true, false, ",") + ")";
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterSystemStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterSystemStmt.java
index 8e2175b..cf16640 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterSystemStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/AlterSystemStmt.java
@@ -54,7 +54,8 @@ public class AlterSystemStmt extends DdlStmt {
                 || (alterClause instanceof AddFollowerClause)
                 || (alterClause instanceof DropFollowerClause)
                 || (alterClause instanceof ModifyBrokerClause)
-                || (alterClause instanceof AlterLoadErrorUrlClause));
+                || (alterClause instanceof AlterLoadErrorUrlClause)
+                || (alterClause instanceof ModifyBackendClause));
 
         alterClause.analyze(analyzer);
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java
index 3c5f7e3..2a1fa33 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/Analyzer.java
@@ -40,10 +40,10 @@ import org.apache.doris.rewrite.ExprRewriteRule;
 import org.apache.doris.rewrite.ExprRewriter;
 import org.apache.doris.rewrite.ExtractCommonFactorsRule;
 import org.apache.doris.rewrite.FoldConstantsRule;
+import org.apache.doris.rewrite.NormalizeBinaryPredicatesRule;
 import org.apache.doris.rewrite.RewriteAliasFunctionRule;
 import org.apache.doris.rewrite.RewriteEncryptKeyRule;
 import org.apache.doris.rewrite.RewriteFromUnixTimeRule;
-import org.apache.doris.rewrite.NormalizeBinaryPredicatesRule;
 import org.apache.doris.rewrite.SimplifyInvalidDateBinaryPredicatesDateRule;
 import org.apache.doris.rewrite.mvrewrite.CountDistinctToBitmap;
 import org.apache.doris.rewrite.mvrewrite.CountDistinctToBitmapOrHLLRule;
@@ -463,6 +463,7 @@ public class Analyzer {
             // aliasMap_.put(alias, result);
             tupleByAlias.put(alias, result);
         }
+
         tableRefMap_.put(result.getId(), ref);
 
         return result;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/BackendClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/BackendClause.java
index c09ef49..342b154 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/BackendClause.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/BackendClause.java
@@ -32,7 +32,6 @@ import java.util.Map;
 
 public class BackendClause extends AlterClause {
     protected List<String> hostPorts;
-
     protected List<Pair<String, Integer>> hostPortPairs;
 
     protected BackendClause(List<String> hostPorts) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableAsSelectStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableAsSelectStmt.java
index a5769ffb..9f2c188 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableAsSelectStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/CreateTableAsSelectStmt.java
@@ -19,7 +19,6 @@ package org.apache.doris.analysis;
 
 import org.apache.doris.catalog.Table;
 import org.apache.doris.common.AnalysisException;
-import org.apache.doris.common.DdlException;
 import org.apache.doris.common.ErrorCode;
 import org.apache.doris.common.ErrorReport;
 import org.apache.doris.common.UserException;
@@ -93,7 +92,7 @@ public class CreateTableAsSelectStmt extends StatementBase {
         // Create table
         try {
             analyzer.getCatalog().createTable(createTableStmt);
-        } catch (DdlException e) {
+        } catch (UserException e) {
             throw new AnalysisException(e.getMessage());
         }
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/AddBackendClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyBackendClause.java
similarity index 54%
copy from fe/fe-core/src/main/java/org/apache/doris/analysis/AddBackendClause.java
copy to fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyBackendClause.java
index 642732f..f6d6262 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/AddBackendClause.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyBackendClause.java
@@ -17,48 +17,38 @@
 
 package org.apache.doris.analysis;
 
-import com.google.common.base.Strings;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.util.PropertyAnalyzer;
+import org.apache.doris.resource.Tag;
 
-import java.util.List;
+import com.google.common.collect.Maps;
 
-public class AddBackendClause extends BackendClause {
+import java.util.List;
+import java.util.Map;
 
-    // be in free state is not owned by any cluster
-    protected boolean isFree;
-    // cluster that backend will be added to 
-    protected String destCluster;
+public class ModifyBackendClause extends BackendClause {
+    protected Map<String, String> properties = Maps.newHashMap();
+    private Tag tag;
 
-    public AddBackendClause(List<String> hostPorts) {
+    public ModifyBackendClause(List<String> hostPorts, Map<String, String> properties) {
         super(hostPorts);
-        this.isFree = true;
-        this.destCluster = "";
+        this.properties = properties;
     }
-   
-    public AddBackendClause(List<String> hostPorts, boolean isFree) {
-        super(hostPorts);
-        this.isFree = isFree;
-        this.destCluster = "";
+
+    @Override
+    public void analyze(Analyzer analyzer) throws AnalysisException {
+        super.analyze(analyzer);
+        tag = PropertyAnalyzer.analyzeBackendTagProperties(properties);
     }
 
-    public AddBackendClause(List<String> hostPorts, String destCluster) {
-        super(hostPorts);
-        this.isFree = false;
-        this.destCluster = destCluster;
+    public Tag getTag() {
+        return tag;
     }
 
     @Override
     public String toSql() {
         StringBuilder sb = new StringBuilder();
-        sb.append("ADD ");
-        if (isFree) {
-            sb.append("FREE ");
-        }
-        sb.append("BACKEND ");
-
-        if (!Strings.isNullOrEmpty(destCluster)) {
-            sb.append("to").append(destCluster);
-        }
-
+        sb.append("MODIFY BACKEND ");
         for (int i = 0; i < hostPorts.size(); i++) {
             sb.append("\"").append(hostPorts.get(i)).append("\"");
             if (i != hostPorts.size() - 1) {
@@ -67,13 +57,4 @@ public class AddBackendClause extends BackendClause {
         }
         return sb.toString();
     }
-
-    public boolean isFree() {
-        return this.isFree;
-    } 
-
-    public String getDestCluster() {
-        return destCluster;
-    }
-
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyPartitionClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyPartitionClause.java
index cba71ae..5dd3c31 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyPartitionClause.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyPartitionClause.java
@@ -20,9 +20,9 @@ package org.apache.doris.analysis;
 import org.apache.doris.alter.AlterOpType;
 import org.apache.doris.catalog.DataProperty;
 import org.apache.doris.common.AnalysisException;
-import org.apache.doris.common.FeConstants;
 import org.apache.doris.common.util.PrintableMap;
 import org.apache.doris.common.util.PropertyAnalyzer;
+
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
@@ -93,7 +93,7 @@ public class ModifyPartitionClause extends AlterTableClause {
     }
 
     // Check the following properties' legality before modifying partition.
-    // 1. replication_num
+    // 1. replication_num or replica_allocation
     // 2. storage_medium && storage_cooldown_time
     // 3. in_memory
     // 4. tablet type
@@ -103,10 +103,8 @@ public class ModifyPartitionClause extends AlterTableClause {
         newDataProperty = PropertyAnalyzer.analyzeDataProperty(properties, DataProperty.DEFAULT_DATA_PROPERTY);
         Preconditions.checkNotNull(newDataProperty);
 
-        // 2. replication num
-        short newReplicationNum = (short) -1;
-        newReplicationNum = PropertyAnalyzer.analyzeReplicationNum(properties, FeConstants.default_replication_num);
-        Preconditions.checkState(newReplicationNum != (short) -1);
+        // 2. replica allocation
+        PropertyAnalyzer.analyzeReplicaAllocation(properties, "");
 
         // 3. in memory
         PropertyAnalyzer.analyzeBooleanProp(properties, PropertyAnalyzer.PROPERTIES_INMEMORY, false);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyTablePropertiesClause.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyTablePropertiesClause.java
index 4ad057e..782ad3d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyTablePropertiesClause.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ModifyTablePropertiesClause.java
@@ -18,6 +18,7 @@
 package org.apache.doris.analysis;
 
 import org.apache.doris.alter.AlterOpType;
+import org.apache.doris.catalog.ReplicaAllocation;
 import org.apache.doris.catalog.TableProperty;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.util.DynamicPartitionUtil;
@@ -74,11 +75,14 @@ public class ModifyTablePropertiesClause extends AlterTableClause {
             }
         } else if (DynamicPartitionUtil.checkDynamicPartitionPropertiesExist(properties)) {
             // do nothing, dynamic properties will be analyzed in SchemaChangeHandler.process
-        } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM)) {
-            PropertyAnalyzer.analyzeReplicationNum(properties, false);
-        } else if (properties.containsKey("default." + PropertyAnalyzer.PROPERTIES_REPLICATION_NUM)) {
-            short defaultReplicationNum = PropertyAnalyzer.analyzeReplicationNum(properties, true);
-            properties.put(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM, Short.toString(defaultReplicationNum));
+        } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM)
+                || properties.containsKey(PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION)) {
+            ReplicaAllocation replicaAlloc = PropertyAnalyzer.analyzeReplicaAllocation(properties, "");
+            properties.put(PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION, replicaAlloc.toCreateStmt());
+        } else if (properties.containsKey("default." + PropertyAnalyzer.PROPERTIES_REPLICATION_NUM)
+                || properties.containsKey("default." + PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION)) {
+            ReplicaAllocation replicaAlloc = PropertyAnalyzer.analyzeReplicaAllocation(properties, "default");
+            properties.put("default." + PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION, replicaAlloc.toCreateStmt());
         } else if (properties.containsKey(PropertyAnalyzer.PROPERTIES_INMEMORY)) {
             this.needTableStable = false;
             this.opType = AlterOpType.MODIFY_TABLE_PROPERTY_SYNC;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/RestoreStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/RestoreStmt.java
index a397d44..6a0bfe6 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/RestoreStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/RestoreStmt.java
@@ -17,12 +17,13 @@
 
 package org.apache.doris.analysis;
 
+import org.apache.doris.catalog.ReplicaAllocation;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.ErrorCode;
 import org.apache.doris.common.ErrorReport;
-import org.apache.doris.common.FeConstants;
 import org.apache.doris.common.UserException;
 import org.apache.doris.common.util.PrintableMap;
+import org.apache.doris.common.util.PropertyAnalyzer;
 
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
@@ -37,7 +38,7 @@ public class RestoreStmt extends AbstractBackupStmt {
     private final static String PROP_META_VERSION = "meta_version";
 
     private boolean allowLoad = false;
-    private int replicationNum = FeConstants.default_replication_num;
+    private ReplicaAllocation replicaAlloc = ReplicaAllocation.DEFAULT_ALLOCATION;
     private String backupTimestamp = null;
     private int metaVersion = -1;
 
@@ -50,8 +51,8 @@ public class RestoreStmt extends AbstractBackupStmt {
         return allowLoad;
     }
 
-    public int getReplicationNum() {
-        return replicationNum;
+    public ReplicaAllocation getReplicaAlloc() {
+        return replicaAlloc;
     }
 
     public String getBackupTimestamp() {
@@ -101,16 +102,10 @@ public class RestoreStmt extends AbstractBackupStmt {
         }
 
         // replication num
-        if (copiedProperties.containsKey(PROP_REPLICATION_NUM)) {
-            try {
-                replicationNum = Integer.valueOf(copiedProperties.get(PROP_REPLICATION_NUM));
-            } catch (NumberFormatException e) {
-                ErrorReport.reportAnalysisException(ErrorCode.ERR_COMMON_ERROR,
-                        "Invalid replication num format: " + copiedProperties.get(PROP_REPLICATION_NUM));
-            }
-            copiedProperties.remove(PROP_REPLICATION_NUM);
+        this.replicaAlloc = PropertyAnalyzer.analyzeReplicaAllocation(copiedProperties, "");
+        if (this.replicaAlloc.isNotSet()) {
+            this.replicaAlloc = ReplicaAllocation.DEFAULT_ALLOCATION;
         }
-
         // backup timestamp
         if (copiedProperties.containsKey(PROP_BACKUP_TIMESTAMP)) {
             backupTimestamp = copiedProperties.get(PROP_BACKUP_TIMESTAMP);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDynamicPartitionStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDynamicPartitionStmt.java
index cecf94c..ab8994a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDynamicPartitionStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowDynamicPartitionStmt.java
@@ -39,6 +39,7 @@ public class ShowDynamicPartitionStmt extends ShowStmt {
                     .addColumn(new Column("Prefix", ScalarType.createVarchar(20)))
                     .addColumn(new Column("Buckets", ScalarType.createVarchar(20)))
                     .addColumn(new Column("ReplicationNum", ScalarType.createVarchar(20)))
+                    .addColumn(new Column("ReplicaAllocation", ScalarType.createVarchar(128)))
                     .addColumn(new Column("StartOf", ScalarType.createVarchar(20)))
                     .addColumn(new Column("LastUpdateTime", ScalarType.createVarchar(20)))
                     .addColumn(new Column("LastSchedulerTime", ScalarType.createVarchar(20)))
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowRestoreStmt.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowRestoreStmt.java
index b47a0a3..22baa5b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowRestoreStmt.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/ShowRestoreStmt.java
@@ -39,7 +39,7 @@ import java.util.function.Predicate;
 public class ShowRestoreStmt extends ShowStmt {
     public static final ImmutableList<String> TITLE_NAMES = new ImmutableList.Builder<String>()
             .add("JobId").add("Label").add("Timestamp").add("DbName").add("State")
-            .add("AllowLoad").add("ReplicationNum")
+            .add("AllowLoad").add("ReplicationNum").add("ReplicaAllocation")
             .add("RestoreObjs").add("CreateTime").add("MetaPreparedTime").add("SnapshotFinishedTime")
             .add("DownloadFinishedTime").add("FinishedTime").add("UnfinishedTasks").add("Progress")
             .add("TaskErrMsg").add("Status").add("Timeout")
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/SinglePartitionDesc.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/SinglePartitionDesc.java
index 102d4b8..b332718 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SinglePartitionDesc.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/SinglePartitionDesc.java
@@ -19,17 +19,17 @@ package org.apache.doris.analysis;
 
 import org.apache.doris.analysis.PartitionKeyDesc.PartitionKeyValueType;
 import org.apache.doris.catalog.DataProperty;
+import org.apache.doris.catalog.ReplicaAllocation;
 import org.apache.doris.common.AnalysisException;
-import org.apache.doris.common.FeConstants;
 import org.apache.doris.common.FeNameFormat;
 import org.apache.doris.common.Pair;
 import org.apache.doris.common.util.PrintableMap;
 import org.apache.doris.common.util.PropertyAnalyzer;
+import org.apache.doris.thrift.TTabletType;
 
 import com.google.common.base.Joiner;
 import com.google.common.base.Joiner.MapJoiner;
 import com.google.common.base.Preconditions;
-import org.apache.doris.thrift.TTabletType;
 
 import java.util.Map;
 
@@ -43,7 +43,7 @@ public class SinglePartitionDesc {
     private Map<String, String> properties;
 
     private DataProperty partitionDataProperty;
-    private Short replicationNum;
+    private ReplicaAllocation replicaAlloc;
     private boolean isInMemory = false;
     private TTabletType tabletType = TTabletType.TABLET_TYPE_DISK;
     private Pair<Long, Long> versionInfo;
@@ -59,7 +59,7 @@ public class SinglePartitionDesc {
         this.properties = properties;
 
         this.partitionDataProperty = DataProperty.DEFAULT_DATA_PROPERTY;
-        this.replicationNum = FeConstants.default_replication_num;
+        this.replicaAlloc = ReplicaAllocation.DEFAULT_ALLOCATION;
     }
 
     public boolean isSetIfNotExists() {
@@ -78,15 +78,17 @@ public class SinglePartitionDesc {
         return partitionDataProperty;
     }
 
-    public short getReplicationNum() {
-        return replicationNum;
+    public ReplicaAllocation getReplicaAlloc() {
+        return replicaAlloc;
     }
 
     public boolean isInMemory() {
         return isInMemory;
     }
 
-    public TTabletType getTabletType() { return tabletType; }
+    public TTabletType getTabletType() {
+        return tabletType;
+    }
 
     public Pair<Long, Long> getVersionInfo() {
         return versionInfo;
@@ -111,13 +113,13 @@ public class SinglePartitionDesc {
 
         // analyze data property
         partitionDataProperty = PropertyAnalyzer.analyzeDataProperty(properties,
-                                                                     DataProperty.DEFAULT_DATA_PROPERTY);
+                DataProperty.DEFAULT_DATA_PROPERTY);
         Preconditions.checkNotNull(partitionDataProperty);
 
         // analyze replication num
-        replicationNum = PropertyAnalyzer.analyzeReplicationNum(properties, FeConstants.default_replication_num);
-        if (replicationNum == null) {
-            throw new AnalysisException("Invalid replication number: " + replicationNum);
+        replicaAlloc = PropertyAnalyzer.analyzeReplicaAllocation(properties, "");
+        if (replicaAlloc.isNotSet()) {
+            replicaAlloc = ReplicaAllocation.DEFAULT_ALLOCATION;
         }
 
         // analyze version info
diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java
index 0571059..a5b6cd9 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/backup/BackupHandler.java
@@ -399,7 +399,7 @@ public class BackupHandler extends MasterDaemon implements Writable {
 
         // Create a restore job
         RestoreJob restoreJob = new RestoreJob(stmt.getLabel(), stmt.getBackupTimestamp(),
-                db.getId(), db.getFullName(), jobInfo, stmt.allowLoad(), stmt.getReplicationNum(),
+                db.getId(), db.getFullName(), jobInfo, stmt.allowLoad(), stmt.getReplicaAlloc(),
                 stmt.getTimeoutMs(), stmt.getMetaVersion(), catalog, repository.getId());
         catalog.getEditLog().logRestoreJob(restoreJob);
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/backup/Repository.java b/fe/fe-core/src/main/java/org/apache/doris/backup/Repository.java
index 1197063..3a2623d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/backup/Repository.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/backup/Repository.java
@@ -304,9 +304,10 @@ public class Repository implements Writable {
                 DIR_SNAPSHOT_CONTENT,
                 childPath);
         try {
-            return new URI(path).normalize().toString();
-        } catch (URISyntaxException e) {
-            LOG.warn("failed to normalize path: {}", path, e);
+            URI uri = new URI(path);
+            return uri.normalize().toString();
+        } catch (Exception e) {
+            LOG.warn("Invalid path: " + path, e);
             return null;
         }
     }
@@ -314,17 +315,23 @@ public class Repository implements Writable {
     // Check if this repo is available.
     // If failed to connect this repo, set errMsg and return false.
     public boolean ping() {
-        String checkPath = Paths.get(location, joinPrefix(PREFIX_REPO, name)).toString();
-        Status st = storage.checkPathExist(checkPath);
-        if (!st.ok()) {
-            errMsg = TimeUtils.longToTimeString(System.currentTimeMillis()) + ": " + st.getErrMsg();
+        String path = location + "/" + joinPrefix(PREFIX_REPO, name);
+        try {
+            URI checkUri = new URI(path);
+            Status st = storage.checkPathExist(checkUri.normalize().toString());
+            if (!st.ok()) {
+                errMsg = TimeUtils.longToTimeString(System.currentTimeMillis()) + ": " + st.getErrMsg();
+                return false;
+            }
+            // clear err msg
+            errMsg = null;
+
+            return true;
+        } catch (URISyntaxException e) {
+            errMsg = TimeUtils.longToTimeString(System.currentTimeMillis())
+                    + ": Invalid path. " + path + ", error: " + e.getMessage();
             return false;
         }
-
-        // clear err msg
-        errMsg = null;
-
-        return true;
     }
 
     // Visit the repository, and list all existing snapshot names
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 ab902d5..8e92395 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
@@ -41,6 +41,7 @@ import org.apache.doris.catalog.PartitionItem;
 import org.apache.doris.catalog.PartitionType;
 import org.apache.doris.catalog.Replica;
 import org.apache.doris.catalog.Replica.ReplicaState;
+import org.apache.doris.catalog.ReplicaAllocation;
 import org.apache.doris.catalog.Resource;
 import org.apache.doris.catalog.ResourceMgr;
 import org.apache.doris.catalog.Table;
@@ -56,6 +57,7 @@ import org.apache.doris.common.MetaNotFoundException;
 import org.apache.doris.common.Pair;
 import org.apache.doris.common.io.Text;
 import org.apache.doris.common.util.TimeUtils;
+import org.apache.doris.resource.Tag;
 import org.apache.doris.task.AgentBatchTask;
 import org.apache.doris.task.AgentTask;
 import org.apache.doris.task.AgentTaskExecutor;
@@ -126,7 +128,7 @@ public class RestoreJob extends AbstractJob {
     private long snapshotFinishedTime = -1;
     private long downloadFinishedTime = -1;
 
-    private int restoreReplicationNum;
+    private ReplicaAllocation replicaAlloc;
 
     // this 2 members is to save all newly restored objs
     // tbl name -> part
@@ -158,13 +160,13 @@ public class RestoreJob extends AbstractJob {
     }
 
     public RestoreJob(String label, String backupTs, long dbId, String dbName, BackupJobInfo jobInfo,
-            boolean allowLoad, int restoreReplicationNum, long timeoutMs, int metaVersion,
-            Catalog catalog, long repoId) {
+                      boolean allowLoad, ReplicaAllocation replicaAlloc, long timeoutMs, int metaVersion,
+                      Catalog catalog, long repoId) {
         super(JobType.RESTORE, label, dbId, dbName, timeoutMs, catalog, repoId);
         this.backupTimestamp = backupTs;
         this.jobInfo = jobInfo;
         this.allowLoad = allowLoad;
-        this.restoreReplicationNum = restoreReplicationNum;
+        this.replicaAlloc = replicaAlloc;
         this.state = RestoreJobState.PENDING;
         this.metaVersion = metaVersion;
     }
@@ -590,7 +592,7 @@ public class RestoreJob extends AbstractJob {
                                         Partition restorePart = resetPartitionForRestore(localOlapTbl, remoteOlapTbl,
                                                 partitionName,
                                                 db.getClusterName(),
-                                                restoreReplicationNum);
+                                                replicaAlloc);
                                         if (restorePart == null) {
                                             return;
                                         }
@@ -619,7 +621,7 @@ public class RestoreJob extends AbstractJob {
                     }
 
                     // reset all ids in this table
-                    Status st = remoteOlapTbl.resetIdsForRestore(catalog, db, restoreReplicationNum);
+                    Status st = remoteOlapTbl.resetIdsForRestore(catalog, db, replicaAlloc);
                     if (!st.ok()) {
                         status = st;
                         return;
@@ -769,7 +771,7 @@ public class RestoreJob extends AbstractJob {
                         PartitionItem remoteItem = remoteTbl.getPartitionInfo().getItem(remotePartId);
                         DataProperty remoteDataProperty = remotePartitionInfo.getDataProperty(remotePartId);
                         localPartitionInfo.addPartition(restoredPart.getId(), false, remoteItem,
-                                remoteDataProperty, (short) restoreReplicationNum,
+                                remoteDataProperty, replicaAlloc,
                                 remotePartitionInfo.getIsInMemory(remotePartId));
                     }
                     localTbl.addPartition(restoredPart);
@@ -899,12 +901,14 @@ public class RestoreJob extends AbstractJob {
 
     private boolean genFileMappingWhenBackupReplicasEqual(PartitionInfo localPartInfo, Partition localPartition, Table localTbl,
                                                           BackupPartitionInfo backupPartInfo, String partitionName, BackupOlapTableInfo tblInfo) {
-        if (localPartInfo.getReplicationNum(localPartition.getId()) != restoreReplicationNum) {
+        short restoreReplicaNum = replicaAlloc.getTotalReplicaNum();
+        short localReplicaNum = localPartInfo.getReplicaAllocation(localPartition.getId()).getTotalReplicaNum();
+        if (localReplicaNum != restoreReplicaNum) {
             status = new Status(ErrCode.COMMON_ERROR, "Partition " + partitionName
                     + " in table " + localTbl.getName()
                     + " has different replication num '"
-                    + localPartInfo.getReplicationNum(localPartition.getId())
-                    + "' with partition in repository, which is " + restoreReplicationNum);
+                    + localReplicaNum
+                    + "' with partition in repository, which is " + restoreReplicaNum);
             return true;
         }
 
@@ -949,13 +953,13 @@ public class RestoreJob extends AbstractJob {
     // reset remote partition.
     // reset all id in remote partition, but DO NOT modify any exist catalog objects.
     private Partition resetPartitionForRestore(OlapTable localTbl, OlapTable remoteTbl, String partName,
-            String clusterName, int restoreReplicationNum) {
+                                               String clusterName, ReplicaAllocation replicaAlloc) {
         Preconditions.checkState(localTbl.getPartition(partName) == null);
         Partition remotePart = remoteTbl.getPartition(partName);
         Preconditions.checkNotNull(remotePart);
         PartitionInfo localPartitionInfo = localTbl.getPartitionInfo();
         Preconditions.checkState(localPartitionInfo.getType() == PartitionType.RANGE
-                                    || localPartitionInfo.getType() == PartitionType.LIST);
+                || localPartitionInfo.getType() == PartitionType.LIST);
 
         // generate new partition id
         long newPartId = catalog.getNextId();
@@ -993,20 +997,20 @@ public class RestoreJob extends AbstractJob {
                 remoteIdx.addTablet(newTablet, null /* tablet meta */, true /* is restore */);
 
                 // replicas
-                List<Long> beIds = Catalog.getCurrentSystemInfo().seqChooseBackendIds(restoreReplicationNum, true,
-                                                                                      true, clusterName);
-                if (beIds == null) {
-                    status = new Status(ErrCode.COMMON_ERROR,
-                            "failed to get enough backends for creating replica of tablet "
-                                    + newTabletId + ". need: " + restoreReplicationNum);
+                try {
+                    Map<Tag, List<Long>> beIds = Catalog.getCurrentSystemInfo().chooseBackendIdByFilters(replicaAlloc, clusterName, null);
+                    for (Map.Entry<Tag, List<Long>> entry : beIds.entrySet()) {
+                        for (Long beId : entry.getValue()) {
+                            long newReplicaId = catalog.getNextId();
+                            Replica newReplica = new Replica(newReplicaId, beId, ReplicaState.NORMAL,
+                                    visibleVersion, visibleVersionHash, schemaHash);
+                            newTablet.addReplica(newReplica, true /* is restore */);
+                        }
+                    }
+                } catch (DdlException e) {
+                    status = new Status(ErrCode.COMMON_ERROR, e.getMessage());
                     return null;
                 }
-                for (Long beId : beIds) {
-                    long newReplicaId = catalog.getNextId();
-                    Replica newReplica = new Replica(newReplicaId, beId, ReplicaState.NORMAL,
-                            visibleVersion, visibleVersionHash, schemaHash);
-                    newTablet.addReplica(newReplica, true /* is restore */);
-                }
             }
         }
         return remotePart;
@@ -1083,7 +1087,7 @@ public class RestoreJob extends AbstractJob {
             long remotePartId = backupPartitionInfo.id;
             DataProperty remoteDataProperty = remotePartitionInfo.getDataProperty(remotePartId);
             localPartitionInfo.addPartition(restorePart.getId(), false, remotePartitionInfo.getItem(remotePartId),
-                    remoteDataProperty, (short) restoreReplicationNum,
+                    remoteDataProperty, replicaAlloc,
                     remotePartitionInfo.getIsInMemory(remotePartId));
             localTbl.addPartition(restorePart);
 
@@ -1465,7 +1469,8 @@ public class RestoreJob extends AbstractJob {
         info.add(dbName);
         info.add(state.name());
         info.add(String.valueOf(allowLoad));
-        info.add(String.valueOf(restoreReplicationNum));
+        info.add(String.valueOf(replicaAlloc.getTotalReplicaNum()));
+        info.add(replicaAlloc.toCreateStmt());
         info.add(getRestoreObjs());
         info.add(TimeUtils.longToTimeString(createTime));
         info.add(TimeUtils.longToTimeString(metaPreparedTime));
@@ -1661,7 +1666,7 @@ public class RestoreJob extends AbstractJob {
         out.writeLong(snapshotFinishedTime);
         out.writeLong(downloadFinishedTime);
 
-        out.writeInt(restoreReplicationNum);
+        replicaAlloc.write(out);
 
         out.writeInt(restoredPartitions.size());
         for (Pair<String, Partition> entry : restoredPartitions) {
@@ -1729,7 +1734,12 @@ public class RestoreJob extends AbstractJob {
         snapshotFinishedTime = in.readLong();
         downloadFinishedTime = in.readLong();
 
-        restoreReplicationNum = in.readInt();
+        if (Catalog.getCurrentCatalogJournalVersion() < FeMetaVersion.VERSION_105) {
+            int restoreReplicationNum = in.readInt();
+            replicaAlloc = new ReplicaAllocation((short) restoreReplicationNum);
+        } else {
+            replicaAlloc = ReplicaAllocation.read(in);
+        }
 
         int size = in.readInt();
         for (int i = 0; i < size; i++) {
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 a5d102a..c6ecb27 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
@@ -210,6 +210,7 @@ import org.apache.doris.qe.GlobalVariable;
 import org.apache.doris.qe.JournalObservable;
 import org.apache.doris.qe.SessionVariable;
 import org.apache.doris.qe.VariableMgr;
+import org.apache.doris.resource.Tag;
 import org.apache.doris.service.FrontendOptions;
 import org.apache.doris.statistics.StatisticsManager;
 import org.apache.doris.system.Backend;
@@ -2993,7 +2994,7 @@ public class Catalog {
      * 10. add this table to FE's meta
      * 11. add this table to ColocateGroup if necessary
      */
-    public void createTable(CreateTableStmt stmt) throws DdlException {
+    public void createTable(CreateTableStmt stmt) throws UserException {
         String engineName = stmt.getEngineName();
         String dbName = stmt.getDbName();
         String tableName = stmt.getTableName();
@@ -3121,9 +3122,10 @@ public class Catalog {
 
             Map<String, String> properties = singlePartitionDesc.getProperties();
             // partition properties should inherit table properties
-            Short replicationNum = olapTable.getDefaultReplicationNum();
-            if (!properties.containsKey(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM)) {
-                properties.put(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM, replicationNum.toString());
+            ReplicaAllocation replicaAlloc = olapTable.getDefaultReplicaAllocation();
+            if (!properties.containsKey(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM)
+                    && !properties.containsKey(PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION)) {
+                properties.put(PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION, replicaAlloc.toCreateStmt());
             }
             if (!properties.containsKey(PropertyAnalyzer.PROPERTIES_INMEMORY)) {
                 properties.put(PropertyAnalyzer.PROPERTIES_INMEMORY, olapTable.isInMemory().toString());
@@ -3166,7 +3168,7 @@ public class Catalog {
                 ColocateGroupSchema groupSchema = colocateTableIndex.getGroupSchema(fullGroupName);
                 Preconditions.checkNotNull(groupSchema);
                 groupSchema.checkDistribution(distributionInfo);
-                groupSchema.checkReplicationNum(singlePartitionDesc.getReplicationNum());
+                groupSchema.checkReplicaAllocation(singlePartitionDesc.getReplicaAlloc());
             }
 
             indexIdToMeta = olapTable.getCopiedIndexIdToMeta();
@@ -3195,7 +3197,7 @@ public class Catalog {
                     indexIdToMeta,
                     distributionInfo,
                     dataProperty.getStorageMedium(),
-                    singlePartitionDesc.getReplicationNum(),
+                    singlePartitionDesc.getReplicaAlloc(),
                     singlePartitionDesc.getVersionInfo(),
                     bfColumns, olapTable.getBfFpp(),
                     tabletIdSet, olapTable.getCopiedIndexes(),
@@ -3271,14 +3273,14 @@ public class Catalog {
                             partitionInfo.getItem(partitionId).getItems(),
                             ListPartitionItem.DUMMY_ITEM,
                             dataProperty,
-                            partitionInfo.getReplicationNum(partitionId),
+                            partitionInfo.getReplicaAllocation(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.getReplicaAllocation(partitionId),
                             partitionInfo.getIsInMemory(partitionId),
                             isTempPartition);
                 }
@@ -3317,7 +3319,7 @@ public class Catalog {
             }
 
             partitionInfo.unprotectHandleNewSinglePartitionDesc(partition.getId(), info.isTempPartition(),
-                    partitionItem, info.getDataProperty(), info.getReplicationNum(), info.isInMemory());
+                    partitionItem, info.getDataProperty(), info.getReplicaAlloc(), info.isInMemory());
 
             if (!isCheckpointThread()) {
                 // add to inverted index
@@ -3424,7 +3426,7 @@ public class Catalog {
                                                  Map<Long, MaterializedIndexMeta> indexIdToMeta,
                                                  DistributionInfo distributionInfo,
                                                  TStorageMedium storageMedium,
-                                                 short replicationNum,
+                                                 ReplicaAllocation replicaAlloc,
                                                  Pair<Long, Long> versionInfo,
                                                  Set<String> bfColumns,
                                                  double bfFpp,
@@ -3461,6 +3463,7 @@ public class Catalog {
         long version = partition.getVisibleVersion();
         long versionHash = partition.getVisibleVersionHash();
 
+        short totalReplicaNum = replicaAlloc.getTotalReplicaNum();
         for (Map.Entry<Long, MaterializedIndex> entry : indexMap.entrySet()) {
             long indexId = entry.getKey();
             MaterializedIndex index = entry.getValue();
@@ -3470,7 +3473,7 @@ public class Catalog {
             int schemaHash = indexMeta.getSchemaHash();
             TabletMeta tabletMeta = new TabletMeta(dbId, tableId, partitionId, indexId, schemaHash, storageMedium);
             createTablets(clusterName, index, ReplicaState.NORMAL, distributionInfo, version, versionHash,
-                    replicationNum, tabletMeta, tabletIdSet);
+                    replicaAlloc, tabletMeta, tabletIdSet);
 
             boolean ok = false;
             String errMsg = null;
@@ -3480,7 +3483,7 @@ public class Catalog {
             TStorageType storageType = indexMeta.getStorageType();
             List<Column> schema = indexMeta.getSchema();
             KeysType keysType = indexMeta.getKeysType();
-            int totalTaskNum = index.getTablets().size() * replicationNum;
+            int totalTaskNum = index.getTablets().size() * totalReplicaNum;
             MarkedCountDownLatch<Long, Long> countDownLatch = new MarkedCountDownLatch<Long, Long>(totalTaskNum);
             AgentBatchTask batchTask = new AgentBatchTask();
             for (Tablet tablet : index.getTablets()) {
@@ -3546,7 +3549,7 @@ public class Catalog {
     }
 
     // Create olap table and related base index synchronously.
-    private void createOlapTable(Database db, CreateTableStmt stmt) throws DdlException {
+    private void createOlapTable(Database db, CreateTableStmt stmt) throws UserException {
         String tableName = stmt.getTableName();
         LOG.debug("begin create olap table: {}", tableName);
 
@@ -3628,23 +3631,18 @@ public class Catalog {
             throw new DdlException(e.getMessage());
         }
 
-        // analyze replication_num
-        short replicationNum = FeConstants.default_replication_num;
-        try {
-            boolean isReplicationNumSet = properties != null && properties.containsKey(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM);
-            replicationNum = PropertyAnalyzer.analyzeReplicationNum(properties, replicationNum);
-            if (isReplicationNumSet) {
-                olapTable.setReplicationNum(replicationNum);
-            }
-        } catch (AnalysisException e) {
-            throw new DdlException(e.getMessage());
+        // analyze replica allocation
+        ReplicaAllocation replicaAlloc = PropertyAnalyzer.analyzeReplicaAllocation(properties, "");
+        if (replicaAlloc.isNotSet()) {
+            replicaAlloc = ReplicaAllocation.DEFAULT_ALLOCATION;
         }
+        olapTable.setReplicationAllocation(replicaAlloc);
 
         // set in memory
         boolean isInMemory = PropertyAnalyzer.analyzeBooleanProp(properties, PropertyAnalyzer.PROPERTIES_INMEMORY, false);
         olapTable.setIsInMemory(isInMemory);
 
-        TTabletType tabletType = TTabletType.TABLET_TYPE_DISK;
+        TTabletType tabletType;
         try {
             tabletType = PropertyAnalyzer.analyzeTabletType(properties);
         } catch (AnalysisException e) {
@@ -3666,7 +3664,7 @@ public class Catalog {
             }
             Preconditions.checkNotNull(dataProperty);
             partitionInfo.setDataProperty(partitionId, dataProperty);
-            partitionInfo.setReplicationNum(partitionId, replicationNum);
+            partitionInfo.setReplicaAllocation(partitionId, replicaAlloc);
             partitionInfo.setIsInMemory(partitionId, isInMemory);
             partitionInfo.setTabletType(partitionId, tabletType);
         }
@@ -3778,7 +3776,7 @@ public class Catalog {
                         olapTable.getIndexIdToMeta(),
                         distributionInfo,
                         partitionInfo.getDataProperty(partitionId).getStorageMedium(),
-                        partitionInfo.getReplicationNum(partitionId),
+                        partitionInfo.getReplicaAllocation(partitionId),
                         versionInfo, bfColumns, bfFpp,
                         tabletIdSet, olapTable.getCopiedIndexes(),
                         isInMemory, storageFormat, tabletType);
@@ -3813,7 +3811,7 @@ public class Catalog {
                             olapTable.getBaseIndexId(), entry.getValue(), entry.getKey(),
                             olapTable.getIndexIdToMeta(), distributionInfo,
                             dataProperty.getStorageMedium(),
-                            partitionInfo.getReplicationNum(entry.getValue()),
+                            partitionInfo.getReplicaAllocation(entry.getValue()),
                             versionInfo, bfColumns, bfFpp,
                             tabletIdSet, olapTable.getCopiedIndexes(),
                             isInMemory, storageFormat,
@@ -3843,7 +3841,7 @@ public class Catalog {
                 // we have added these index to memory, only need to persist here
                 if (getColocateTableIndex().isColocateTable(tableId)) {
                     GroupId groupId = getColocateTableIndex().getGroup(tableId);
-                    List<List<Long>> backendsPerBucketSeq = getColocateTableIndex().getBackendsPerBucketSeq(groupId);
+                    Map<Tag, List<List<Long>>> backendsPerBucketSeq = getColocateTableIndex().getBackendsPerBucketSeq(groupId);
                     ColocatePersistInfo info = ColocatePersistInfo.createForAddTable(groupId, tableId, backendsPerBucketSeq);
                     editLog.logColocateAddTable(info);
                 }
@@ -3883,7 +3881,6 @@ public class Catalog {
 
     private void createOdbcTable(Database db, CreateTableStmt stmt) throws DdlException {
         String tableName = stmt.getTableName();
-
         List<Column> columns = stmt.getColumns();
 
         long tableId = Catalog.getCurrentCatalog().getNextId();
@@ -4104,9 +4101,9 @@ public class Catalog {
             sb.append("\nPROPERTIES (\n");
 
             // replicationNum
-            Short replicationNum = olapTable.getDefaultReplicationNum();
-            sb.append("\"").append(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM).append("\" = \"");
-            sb.append(replicationNum).append("\"");
+            ReplicaAllocation replicaAlloc = olapTable.getDefaultReplicaAllocation();
+            sb.append("\"").append(PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION).append("\" = \"");
+            sb.append(replicaAlloc.toCreateStmt()).append("\"");
 
             // bloom filter
             Set<String> bfColumnNames = olapTable.getCopiedBfColumns();
@@ -4138,7 +4135,7 @@ public class Catalog {
 
             // dynamic partition
             if (olapTable.dynamicPartitionExists()) {
-                sb.append(olapTable.getTableProperty().getDynamicPartitionProperty().getProperties(replicationNum));
+                sb.append(olapTable.getTableProperty().getDynamicPartitionProperty().getProperties(replicaAlloc));
             }
 
             // in memory
@@ -4366,14 +4363,12 @@ public class Catalog {
     }
 
     private void createTablets(String clusterName, MaterializedIndex index, ReplicaState replicaState,
-                               DistributionInfo distributionInfo, long version, long versionHash, short replicationNum,
+                               DistributionInfo distributionInfo, long version, long versionHash, ReplicaAllocation replicaAlloc,
                                TabletMeta tabletMeta, Set<Long> tabletIdSet) throws DdlException {
-        Preconditions.checkArgument(replicationNum > 0);
-
         DistributionInfoType distributionInfoType = distributionInfo.getType();
         if (distributionInfoType == DistributionInfoType.HASH) {
             ColocateTableIndex colocateIndex = Catalog.getCurrentColocateIndex();
-            List<List<Long>> backendsPerBucketSeq = null;
+            Map<Tag, List<List<Long>>> backendsPerBucketSeq = null;
             GroupId groupId = null;
             if (colocateIndex.isColocateTable(tabletMeta.getTableId())) {
                 // if this is a colocate table, try to get backend seqs from colocation index.
@@ -4386,7 +4381,7 @@ public class Catalog {
             // otherwise, backends should be chosen from backendsPerBucketSeq;
             boolean chooseBackendsArbitrary = backendsPerBucketSeq == null || backendsPerBucketSeq.isEmpty();
             if (chooseBackendsArbitrary) {
-                backendsPerBucketSeq = Lists.newArrayList();
+                backendsPerBucketSeq = Maps.newHashMap();
             }
             for (int i = 0; i < distributionInfo.getBucketNum(); ++i) {
                 // create a new tablet with random chosen backends
@@ -4397,29 +4392,42 @@ public class Catalog {
                 tabletIdSet.add(tablet.getId());
 
                 // get BackendIds
-                List<Long> chosenBackendIds;
+                Map<Tag, List<Long>> chosenBackendIds;
                 if (chooseBackendsArbitrary) {
                     // This is the first colocate table in the group, or just a normal table,
                     // randomly choose backends
                     if (!Config.disable_storage_medium_check) {
-                        chosenBackendIds = chosenBackendIdBySeq(replicationNum, clusterName, tabletMeta.getStorageMedium());
+                        chosenBackendIds = getCurrentSystemInfo().chooseBackendIdByFilters(replicaAlloc, clusterName,
+                                tabletMeta.getStorageMedium());
                     } else {
-                        chosenBackendIds = chosenBackendIdBySeq(replicationNum, clusterName);
+                        chosenBackendIds = getCurrentSystemInfo().chooseBackendIdByFilters(replicaAlloc, clusterName, null);
+                    }
+
+                    for (Map.Entry<Tag, List<Long>> entry : chosenBackendIds.entrySet()) {
+                        backendsPerBucketSeq.putIfAbsent(entry.getKey(), Lists.newArrayList());
+                        backendsPerBucketSeq.get(entry.getKey()).add(entry.getValue());
                     }
-                    backendsPerBucketSeq.add(chosenBackendIds);
                 } else {
                     // get backends from existing backend sequence
-                    chosenBackendIds = backendsPerBucketSeq.get(i);
+                    chosenBackendIds = Maps.newHashMap();
+                    for (Map.Entry<Tag, List<List<Long>>> entry : backendsPerBucketSeq.entrySet()) {
+                        chosenBackendIds.put(entry.getKey(), entry.getValue().get(i));
+                    }
                 }
 
                 // create replicas
-                for (long backendId : chosenBackendIds) {
-                    long replicaId = getNextId();
-                    Replica replica = new Replica(replicaId, backendId, replicaState, version, versionHash,
-                            tabletMeta.getOldSchemaHash());
-                    tablet.addReplica(replica);
+                short totalReplicaNum = (short) 0;
+                for (List<Long> backendIds : chosenBackendIds.values()) {
+                    for (long backendId : backendIds) {
+                        long replicaId = getNextId();
+                        Replica replica = new Replica(replicaId, backendId, replicaState, version, versionHash,
+                                tabletMeta.getOldSchemaHash());
+                        tablet.addReplica(replica);
+                        totalReplicaNum++;
+                    }
                 }
-                Preconditions.checkState(chosenBackendIds.size() == replicationNum, chosenBackendIds.size() + " vs. " + replicationNum);
+                Preconditions.checkState(totalReplicaNum == replicaAlloc.getTotalReplicaNum(),
+						totalReplicaNum + " vs. " + replicaAlloc.getTotalReplicaNum());
             }
 
             if (groupId != null && chooseBackendsArbitrary) {
@@ -4433,24 +4441,6 @@ public class Catalog {
         }
     }
 
-    // create replicas for tablet with random chosen backends
-    private List<Long> chosenBackendIdBySeq(int replicationNum, String clusterName, TStorageMedium storageMedium) throws DdlException {
-        List<Long> chosenBackendIds = Catalog.getCurrentSystemInfo().seqChooseBackendIdsByStorageMedium(replicationNum,
-                true, true, clusterName, storageMedium);
-        if (chosenBackendIds == null) {
-            throw new DdlException("Failed to find enough host with storage medium is " + storageMedium + " in all backends. need: " + replicationNum);
-        }
-        return chosenBackendIds;
-    }
-
-    private List<Long> chosenBackendIdBySeq(int replicationNum, String clusterName) throws DdlException {
-        List<Long> chosenBackendIds = Catalog.getCurrentSystemInfo().seqChooseBackendIds(replicationNum, true, true, clusterName);
-        if (chosenBackendIds == null) {
-            throw new DdlException("Failed to find enough host in all backends. need: " + replicationNum);
-        }
-        return chosenBackendIds;
-    }
-
     // Drop table
     public void dropTable(DropTableStmt stmt) throws DdlException {
         String dbName = stmt.getDbName();
@@ -4884,7 +4874,7 @@ public class Catalog {
                                     new ModifyPartitionInfo(db.getId(), olapTable.getId(),
                                             partition.getId(),
                                             DataProperty.DEFAULT_DATA_PROPERTY,
-                                            (short) -1,
+                                            ReplicaAllocation.NOT_SET,
                                             partitionInfo.getIsInMemory(partition.getId()));
                             editLog.logModifyPartition(info);
                         }
@@ -5291,7 +5281,7 @@ public class Catalog {
                 PartitionInfo partitionInfo = table.getPartitionInfo();
                 if (partitionInfo.getType() == PartitionType.RANGE || partitionInfo.getType() == PartitionType.LIST) {
                     int bucketsNum = -1;
-                    short replicationNum = -1;
+                    ReplicaAllocation replicaAlloc = null;
                     for (Partition partition : table.getPartitions()) {
                         if (bucketsNum == -1) {
                             bucketsNum = partition.getDistributionInfo().getBucketNum();
@@ -5299,10 +5289,10 @@ public class Catalog {
                             throw new DdlException("Partitions in table " + table.getName() + " have different buckets number");
                         }
 
-                        if (replicationNum == -1) {
-                            replicationNum = partitionInfo.getReplicationNum(partition.getId());
-                        } else if (replicationNum != partitionInfo.getReplicationNum(partition.getId())) {
-                            throw new DdlException("Partitions in table " + table.getName() + " have different replication number");
+                        if (replicaAlloc == null) {
+                            replicaAlloc = partitionInfo.getReplicaAllocation(partition.getId());
+                        } else if (!replicaAlloc.equals(partitionInfo.getReplicaAllocation(partition.getId()))) {
+                            throw new DdlException("Partitions in table " + table.getName() + " have different replica allocation.");
                         }
                     }
                 }
@@ -5311,7 +5301,7 @@ public class Catalog {
                 groupSchema.checkColocateSchema(table);
             }
 
-            List<List<Long>> backendsPerBucketSeq = null;
+            Map<Tag, List<List<Long>>> backendsPerBucketSeq = null;
             if (groupSchema == null) {
                 // assign to a newly created group, set backends sequence.
                 // we arbitrarily choose a tablet backends sequence from this table,
@@ -5501,7 +5491,7 @@ public class Catalog {
     }
 
     public void modifyTableDynamicPartition(Database db, OlapTable table, Map<String, String> properties)
-            throws DdlException {
+            throws UserException {
         Map<String, String> logProperties = new HashMap<>(properties);
         TableProperty tableProperty = table.getTableProperty();
         if (tableProperty == null) {
@@ -5530,8 +5520,8 @@ public class Catalog {
      * @param properties
      * @throws DdlException
      */
-    // The caller need to hold the table's write lock
-    public void modifyTableReplicationNum(Database db, OlapTable table, Map<String, String> properties) throws DdlException {
+    // The caller need to hold the table write lock
+    public void modifyTableReplicaAllocation(Database db, OlapTable table, Map<String, String> properties) throws UserException {
         Preconditions.checkArgument(table.isWriteLockHeldByCurrentThread());
         String defaultReplicationNumName = "default." + PropertyAnalyzer.PROPERTIES_REPLICATION_NUM;
         PartitionInfo partitionInfo = table.getPartitionInfo();
@@ -5546,35 +5536,40 @@ public class Catalog {
             throw new DdlException("Partition does not exist. name: " + partitionName);
         }
 
-        short replicationNum = Short.valueOf(properties.get(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM));
+        ReplicaAllocation replicaAlloc = PropertyAnalyzer.analyzeReplicaAllocation(properties, "");
+        Catalog.getCurrentSystemInfo().checkReplicaAllocation(db.getClusterName(), replicaAlloc);
+        Preconditions.checkState(!replicaAlloc.isNotSet());
         boolean isInMemory = partitionInfo.getIsInMemory(partition.getId());
         DataProperty newDataProperty = partitionInfo.getDataProperty(partition.getId());
-        partitionInfo.setReplicationNum(partition.getId(), replicationNum);
+        partitionInfo.setReplicaAllocation(partition.getId(), replicaAlloc);
         // log
         ModifyPartitionInfo info = new ModifyPartitionInfo(db.getId(), table.getId(), partition.getId(),
-                newDataProperty, replicationNum, isInMemory);
+                newDataProperty, replicaAlloc, isInMemory);
         editLog.logModifyPartition(info);
-        LOG.debug("modify partition[{}-{}-{}] replication num to {}", db.getId(), table.getId(), partition.getName(),
-                replicationNum);
+        LOG.debug("modify partition[{}-{}-{}] replica allocation to {}", db.getId(), table.getId(),
+                partition.getName(), replicaAlloc.toCreateStmt());
     }
 
     /**
-     * Set default replication number for a specified table.
-     * You can see the default replication number by Show Create Table stmt.
+     * Set default replication allocation for a specified table.
+     * You can see the default replication allocation by executing Show Create Table stmt.
      * @param db
      * @param table
      * @param properties
      */
-    // The caller need to hold the table's write lock
-    public void modifyTableDefaultReplicationNum(Database db, OlapTable table, Map<String, String> properties) {
+    // The caller need to hold the table write lock
+    public void modifyTableDefaultReplicaAllocation(Database db, OlapTable table, Map<String, String> properties) {
         Preconditions.checkArgument(table.isWriteLockHeldByCurrentThread());
+
         TableProperty tableProperty = table.getTableProperty();
         if (tableProperty == null) {
             tableProperty = new TableProperty(properties);
+            table.setTableProperty(tableProperty);
         } else {
             tableProperty.modifyTableProperties(properties);
         }
-        tableProperty.buildReplicationNum();
+        tableProperty.buildReplicaAllocation();
+
         // log
         ModifyTablePropertyOperationLog info = new ModifyTablePropertyOperationLog(db.getId(), table.getId(), properties);
         editLog.logModifyReplicationNum(info);
@@ -6315,15 +6310,16 @@ public class Catalog {
                         olapTable.readLock();
                         try {
                             for (Partition partition : olapTable.getPartitions()) {
-                                final short replicationNum = olapTable.getPartitionInfo()
-                                        .getReplicationNum(partition.getId());
+                                ReplicaAllocation replicaAlloc = olapTable.getPartitionInfo()
+                                        .getReplicaAllocation(partition.getId());
+                                short totalReplicaNum = replicaAlloc.getTotalReplicaNum();
                                 for (MaterializedIndex materializedIndex : partition.getMaterializedIndices(IndexExtState.ALL)) {
                                     if (materializedIndex.getState() != IndexState.NORMAL) {
                                         continue;
                                     }
                                     for (Tablet tablet : materializedIndex.getTablets()) {
                                         int replicaNum = 0;
-                                        int quorum = replicationNum / 2 + 1;
+                                        int quorum = totalReplicaNum / 2 + 1;
                                         for (Replica replica : tablet.getReplicas()) {
                                             if (replica.getState() != ReplicaState.CLONE
                                                     && beIds.contains(replica.getBackendId())) {
@@ -6631,7 +6627,7 @@ public class Catalog {
                         copiedTbl.getIndexIdToMeta(),
                         copiedTbl.getDefaultDistributionInfo(),
                         copiedTbl.getPartitionInfo().getDataProperty(oldPartitionId).getStorageMedium(),
-                        copiedTbl.getPartitionInfo().getReplicationNum(oldPartitionId),
+                        copiedTbl.getPartitionInfo().getReplicaAllocation(oldPartitionId),
                         null /* version info */,
                         copiedTbl.getCopiedBfColumns(),
                         copiedTbl.getBfFpp(),
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 333a89e..5d5e903 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
@@ -108,7 +108,7 @@ public class CatalogRecycleBin extends MasterDaemon implements Writable {
     public synchronized boolean recyclePartition(long dbId, long tableId, Partition partition,
                                                  Range<PartitionKey> range, PartitionItem listPartitionItem,
                                                  DataProperty dataProperty,
-                                                 short replicationNum,
+                                                 ReplicaAllocation replicaAlloc,
                                                  boolean isInMemory) {
         if (idToPartition.containsKey(partition.getId())) {
             LOG.error("partition[{}] already in recycle bin.", partition.getId());
@@ -120,8 +120,7 @@ public class CatalogRecycleBin extends MasterDaemon implements Writable {
 
         // recycle partition
         RecyclePartitionInfo partitionInfo = new RecyclePartitionInfo(dbId, tableId, partition,
-                                                                      range, listPartitionItem, dataProperty, replicationNum,
-                                                                      isInMemory);
+                range, listPartitionItem, dataProperty, replicaAlloc, isInMemory);
         idToRecycleTime.put(partition.getId(), System.currentTimeMillis());
         idToPartition.put(partition.getId(), partitionInfo);
         LOG.info("recycle partition[{}-{}]", partition.getId(), partition.getName());
@@ -450,7 +449,7 @@ public class CatalogRecycleBin extends MasterDaemon implements Writable {
         long partitionId = recoverPartition.getId();
         partitionInfo.setItem(partitionId, false, recoverItem);
         partitionInfo.setDataProperty(partitionId, recoverPartitionInfo.getDataProperty());
-        partitionInfo.setReplicationNum(partitionId, recoverPartitionInfo.getReplicationNum());
+        partitionInfo.setReplicaAllocation(partitionId, recoverPartitionInfo.getReplicaAlloc());
         partitionInfo.setIsInMemory(partitionId, recoverPartitionInfo.isInMemory());
 
         // remove from recycle bin
@@ -485,7 +484,7 @@ public class CatalogRecycleBin extends MasterDaemon implements Writable {
             }
             partitionInfo.setItem(partitionId, false, recoverItem);
             partitionInfo.setDataProperty(partitionId, recyclePartitionInfo.getDataProperty());
-            partitionInfo.setReplicationNum(partitionId, recyclePartitionInfo.getReplicationNum());
+            partitionInfo.setReplicaAllocation(partitionId, recyclePartitionInfo.getReplicaAlloc());
             partitionInfo.setIsInMemory(partitionId, recyclePartitionInfo.isInMemory());
 
             iterator.remove();
@@ -740,7 +739,7 @@ public class CatalogRecycleBin extends MasterDaemon implements Writable {
         private Range<PartitionKey> range;
         private PartitionItem listPartitionItem;
         private DataProperty dataProperty;
-        private short replicationNum;
+        private ReplicaAllocation replicaAlloc;
         private boolean isInMemory;
 
         public RecyclePartitionInfo() {
@@ -749,7 +748,7 @@ public class CatalogRecycleBin extends MasterDaemon implements Writable {
 
         public RecyclePartitionInfo(long dbId, long tableId, Partition partition,
                                     Range<PartitionKey> range, PartitionItem listPartitionItem,
-                                    DataProperty dataProperty, short replicationNum,
+                                    DataProperty dataProperty, ReplicaAllocation replicaAlloc,
                                     boolean isInMemory) {
             this.dbId = dbId;
             this.tableId = tableId;
@@ -757,7 +756,7 @@ public class CatalogRecycleBin extends MasterDaemon implements Writable {
             this.range = range;
             this.listPartitionItem = listPartitionItem;
             this.dataProperty = dataProperty;
-            this.replicationNum = replicationNum;
+            this.replicaAlloc = replicaAlloc;
             this.isInMemory = isInMemory;
         }
 
@@ -785,8 +784,8 @@ public class CatalogRecycleBin extends MasterDaemon implements Writable {
             return dataProperty;
         }
 
-        public short getReplicationNum() {
-            return replicationNum;
+        public ReplicaAllocation getReplicaAlloc() {
+            return replicaAlloc;
         }
 
         public boolean isInMemory() {
@@ -801,7 +800,7 @@ public class CatalogRecycleBin extends MasterDaemon implements Writable {
             RangeUtils.writeRange(out, range);
             listPartitionItem.write(out);
             dataProperty.write(out);
-            out.writeShort(replicationNum);
+            replicaAlloc.write(out);
             out.writeBoolean(isInMemory);
         }
 
@@ -817,7 +816,12 @@ public class CatalogRecycleBin extends MasterDaemon implements Writable {
             }
 
             dataProperty = DataProperty.read(in);
-            replicationNum = in.readShort();
+            if (Catalog.getCurrentCatalogJournalVersion() < FeMetaVersion.VERSION_105) {
+                short replicationNum = in.readShort();
+                replicaAlloc = new ReplicaAllocation(replicationNum);
+            } else {
+                replicaAlloc = ReplicaAllocation.read(in);
+            }
             if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_72) {
                 isInMemory = in.readBoolean();
             }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/ColocateGroupSchema.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/ColocateGroupSchema.java
index 0ea7b15..ea2d8ca 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/ColocateGroupSchema.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/ColocateGroupSchema.java
@@ -21,6 +21,7 @@ import org.apache.doris.catalog.ColocateTableIndex.GroupId;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.common.ErrorCode;
 import org.apache.doris.common.ErrorReport;
+import org.apache.doris.common.FeMetaVersion;
 import org.apache.doris.common.io.Writable;
 
 import com.google.common.collect.Lists;
@@ -38,17 +39,17 @@ public class ColocateGroupSchema implements Writable {
     private GroupId groupId;
     private List<Type> distributionColTypes = Lists.newArrayList();
     private int bucketsNum;
-    private short replicationNum;
+    private ReplicaAllocation replicaAlloc;
 
     private ColocateGroupSchema() {
 
     }
 
-    public ColocateGroupSchema(GroupId groupId, List<Column> distributionCols, int bucketsNum, short replicationNum) {
+    public ColocateGroupSchema(GroupId groupId, List<Column> distributionCols, int bucketsNum, ReplicaAllocation replicaAlloc) {
         this.groupId = groupId;
         this.distributionColTypes = distributionCols.stream().map(c -> c.getType()).collect(Collectors.toList());
         this.bucketsNum = bucketsNum;
-        this.replicationNum = replicationNum;
+        this.replicaAlloc = replicaAlloc;
     }
 
     public GroupId getGroupId() {
@@ -59,8 +60,8 @@ public class ColocateGroupSchema implements Writable {
         return bucketsNum;
     }
 
-    public short getReplicationNum() {
-        return replicationNum;
+    public ReplicaAllocation getReplicaAlloc() {
+        return replicaAlloc;
     }
 
     public List<Type> getDistributionColTypes() {
@@ -69,7 +70,7 @@ public class ColocateGroupSchema implements Writable {
 
     public void checkColocateSchema(OlapTable tbl) throws DdlException {
         checkDistribution(tbl.getDefaultDistributionInfo());
-        checkReplicationNum(tbl.getPartitionInfo());
+        checkReplicaAllocation(tbl.getPartitionInfo());
     }
 
     public void checkDistribution(DistributionInfo distributionInfo) throws DdlException {
@@ -95,17 +96,19 @@ public class ColocateGroupSchema implements Writable {
         }
     }
 
-    public void checkReplicationNum(PartitionInfo partitionInfo) throws DdlException {
-        for (Short repNum : partitionInfo.idToReplicationNum.values()) {
-            if (repNum != replicationNum) {
-                ErrorReport.reportDdlException(ErrorCode.ERR_COLOCATE_TABLE_MUST_HAS_SAME_REPLICATION_NUM, replicationNum);
+    public void checkReplicaAllocation(PartitionInfo partitionInfo) throws DdlException {
+        for (ReplicaAllocation replicaAlloc : partitionInfo.idToReplicaAllocation.values()) {
+            if (!replicaAlloc.equals(this.replicaAlloc)) {
+                ErrorReport.reportDdlException(ErrorCode.ERR_COLOCATE_TABLE_MUST_HAS_SAME_REPLICATION_ALLOCATION,
+                        this.replicaAlloc);
             }
         }
     }
 
-    public void checkReplicationNum(short repNum) throws DdlException {
-        if (repNum != replicationNum) {
-            ErrorReport.reportDdlException(ErrorCode.ERR_COLOCATE_TABLE_MUST_HAS_SAME_REPLICATION_NUM, replicationNum);
+    public void checkReplicaAllocation(ReplicaAllocation replicaAlloc) throws DdlException {
+        if (!replicaAlloc.equals(this.replicaAlloc)) {
+            ErrorReport.reportDdlException(ErrorCode.ERR_COLOCATE_TABLE_MUST_HAS_SAME_REPLICATION_ALLOCATION,
+                    this.replicaAlloc);
         }
     }
 
@@ -123,7 +126,7 @@ public class ColocateGroupSchema implements Writable {
             ColumnType.write(out, type);
         }
         out.writeInt(bucketsNum);
-        out.writeShort(replicationNum);
+        this.replicaAlloc.write(out);
     }
 
     public void readFields(DataInput in) throws IOException {
@@ -133,6 +136,11 @@ public class ColocateGroupSchema implements Writable {
             distributionColTypes.add(ColumnType.read(in));
         }
         bucketsNum = in.readInt();
-        replicationNum = in.readShort();
+        if (Catalog.getCurrentCatalogJournalVersion() < FeMetaVersion.VERSION_105) {
+            short replicationNum = in.readShort();
+            this.replicaAlloc = new ReplicaAllocation(replicationNum);
+        } else {
+            this.replicaAlloc = ReplicaAllocation.read(in);
+        }
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/ColocateTableIndex.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/ColocateTableIndex.java
index 68c552d..0aca4ac 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/ColocateTableIndex.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/ColocateTableIndex.java
@@ -22,15 +22,20 @@ import org.apache.doris.common.MetaNotFoundException;
 import org.apache.doris.common.io.Text;
 import org.apache.doris.common.io.Writable;
 import org.apache.doris.persist.ColocatePersistInfo;
+import org.apache.doris.persist.gson.GsonUtils;
+import org.apache.doris.resource.Tag;
 
 import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.HashBasedTable;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Multimap;
 import com.google.common.collect.Sets;
+import com.google.common.collect.Table;
+import com.google.gson.annotations.SerializedName;
 
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -38,7 +43,6 @@ import org.apache.logging.log4j.Logger;
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
-import java.util.ArrayList;
 import java.util.Collection;
 import java.util.HashSet;
 import java.util.List;
@@ -55,7 +59,9 @@ public class ColocateTableIndex implements Writable {
     private static final Logger LOG = LogManager.getLogger(ColocateTableIndex.class);
 
     public static class GroupId implements Writable {
+        @SerializedName(value = "dbId")
         public Long dbId;
+        @SerializedName(value = "grpId")
         public Long grpId;
 
         private GroupId() {
@@ -67,18 +73,23 @@ public class ColocateTableIndex implements Writable {
         }
 
         public static GroupId read(DataInput in) throws IOException {
-            GroupId groupId = new GroupId();
-            groupId.readFields(in);
-            return groupId;
+            if (Catalog.getCurrentCatalogJournalVersion() < FeMetaVersion.VERSION_105) {
+                GroupId groupId = new GroupId();
+                groupId.readFields(in);
+                return groupId;
+            } else {
+                String json = Text.readString(in);
+                return GsonUtils.GSON.fromJson(json, GroupId.class);
+            }
         }
 
         @Override
         public void write(DataOutput out) throws IOException {
-            out.writeLong(dbId);
-            out.writeLong(grpId);
+            Text.writeString(out, GsonUtils.GSON.toJson(this));
         }
 
-        public void readFields(DataInput in) throws IOException {
+        @Deprecated
+        private void readFields(DataInput in) throws IOException {
             dbId = in.readLong();
             grpId = in.readLong();
         }
@@ -115,9 +126,11 @@ public class ColocateTableIndex implements Writable {
     // group id -> group schema
     private Map<GroupId, ColocateGroupSchema> group2Schema = Maps.newHashMap();
     // group_id -> bucketSeq -> backend ids
-    private Map<GroupId, List<List<Long>>> group2BackendsPerBucketSeq = Maps.newHashMap();
+    private Table<GroupId, Tag, List<List<Long>>> group2BackendsPerBucketSeq = HashBasedTable.create();
     // the colocate group is unstable
     private Set<GroupId> unstableGroups = Sets.newHashSet();
+    // save some error msg of the group for show. no need to persist
+    private Map<GroupId, String> group2ErrMsgs = Maps.newHashMap();
 
     private transient ReentrantReadWriteLock lock = new ReentrantReadWriteLock();
 
@@ -161,7 +174,7 @@ public class ColocateTableIndex implements Writable {
                 HashDistributionInfo distributionInfo = (HashDistributionInfo) tbl.getDefaultDistributionInfo();
                 ColocateGroupSchema groupSchema = new ColocateGroupSchema(groupId,
                         distributionInfo.getDistributionColumns(), distributionInfo.getBucketNum(),
-                        tbl.getDefaultReplicationNum());
+                        tbl.getDefaultReplicaAllocation());
                 groupName2Id.put(fullGroupName, groupId);
                 group2Schema.put(groupId, groupSchema);
             }
@@ -173,10 +186,21 @@ public class ColocateTableIndex implements Writable {
         }
     }
 
-    public void addBackendsPerBucketSeq(GroupId groupId, List<List<Long>> backendsPerBucketSeq) {
+    public void addBackendsPerBucketSeq(GroupId groupId, Map<Tag, List<List<Long>>> backendsPerBucketSeq) {
+        writeLock();
+        try {
+            for (Map.Entry<Tag, List<List<Long>>> entry : backendsPerBucketSeq.entrySet()) {
+                group2BackendsPerBucketSeq.put(groupId, entry.getKey(), entry.getValue());
+            }
+        } finally {
+            writeUnlock();
+        }
+    }
+
+    public void addBackendsPerBucketSeqByTag(GroupId groupId, Tag tag, List<List<Long>> backendsPerBucketSeq) {
         writeLock();
         try {
-            group2BackendsPerBucketSeq.put(groupId, backendsPerBucketSeq);
+            group2BackendsPerBucketSeq.put(groupId, tag, backendsPerBucketSeq);
         } finally {
             writeUnlock();
         }
@@ -229,7 +253,7 @@ public class ColocateTableIndex implements Writable {
             group2Tables.remove(groupId, tableId);
             if (!group2Tables.containsKey(groupId)) {
                 // all tables of this group are removed, remove the group
-                group2BackendsPerBucketSeq.remove(groupId);
+                group2BackendsPerBucketSeq.rowMap().remove(groupId);
                 group2Schema.remove(groupId);
                 unstableGroups.remove(groupId);
                 String fullGroupName = null;
@@ -317,11 +341,11 @@ public class ColocateTableIndex implements Writable {
         }
     }
 
-    public Set<Long> getBackendsByGroup(GroupId groupId) {
+    public Set<Long> getBackendsByGroup(GroupId groupId, Tag tag) {
         readLock();
         try {
             Set<Long> allBackends = new HashSet<>();
-            List<List<Long>> backendsPerBucketSeq = group2BackendsPerBucketSeq.get(groupId);
+            List<List<Long>> backendsPerBucketSeq = group2BackendsPerBucketSeq.get(groupId, tag);
             // if create colocate table with empty partition or create colocate table
             // with dynamic_partition will cause backendsPerBucketSeq == null
             if (backendsPerBucketSeq != null) {
@@ -347,10 +371,23 @@ public class ColocateTableIndex implements Writable {
         }
     }
 
-    public List<List<Long>> getBackendsPerBucketSeq(GroupId groupId) {
+    public Map<Tag, List<List<Long>>> getBackendsPerBucketSeq(GroupId groupId) {
+        readLock();
+        try {
+            Map<Tag, List<List<Long>>> backendsPerBucketSeq = group2BackendsPerBucketSeq.row(groupId);
+            if (backendsPerBucketSeq == null) {
+                return Maps.newHashMap();
+            }
+            return backendsPerBucketSeq;
+        } finally {
+            readUnlock();
+        }
+    }
+
+    public List<List<Long>> getBackendsPerBucketSeqByTag(GroupId groupId, Tag tag) {
         readLock();
         try {
-            List<List<Long>> backendsPerBucketSeq = group2BackendsPerBucketSeq.get(groupId);
+            List<List<Long>> backendsPerBucketSeq = group2BackendsPerBucketSeq.get(groupId, tag);
             if (backendsPerBucketSeq == null) {
                 return Lists.newArrayList();
             }
@@ -360,18 +397,50 @@ public class ColocateTableIndex implements Writable {
         }
     }
 
-    public List<Set<Long>> getBackendsPerBucketSeqSet(GroupId groupId) {
+    // Get all backend ids except for the given tag
+    public Set<Long> getBackendIdsExceptForTag(GroupId groupId, Tag tag) {
+        Set<Long> beIds = Sets.newHashSet();
         readLock();
         try {
-            List<List<Long>> backendsPerBucketSeq = group2BackendsPerBucketSeq.get(groupId);
+            Map<Tag, List<List<Long>>> backendsPerBucketSeq = group2BackendsPerBucketSeq.row(groupId);
             if (backendsPerBucketSeq == null) {
+                return beIds;
+            }
+
+            for (Map.Entry<Tag, List<List<Long>>> entry : backendsPerBucketSeq.entrySet()) {
+                if (entry.getKey().equals(tag)) {
+                    continue;
+                }
+                for (List<Long> list : entry.getValue()) {
+                    beIds.addAll(list);
+                }
+            }
+            return beIds;
+        } finally {
+            readUnlock();
+        }
+    }
+
+
+    public List<Set<Long>> getBackendsPerBucketSeqSet(GroupId groupId) {
+        readLock();
+        try {
+            Map<Tag, List<List<Long>>> backendsPerBucketSeqMap = group2BackendsPerBucketSeq.row(groupId);
+            if (backendsPerBucketSeqMap == null) {
                 return Lists.newArrayList();
             }
-            List<Set<Long>> sets = Lists.newArrayList();
-            for (List<Long> backends : backendsPerBucketSeq) {
-                sets.add(Sets.newHashSet(backends));
+            List<Set<Long>> list = Lists.newArrayList();
+
+            // Merge backend ids of all tags
+            for (Map.Entry<Tag, List<List<Long>>> backendsPerBucketSeq : backendsPerBucketSeqMap.entrySet()) {
+                for (int i = 0; i < backendsPerBucketSeq.getValue().size(); ++i) {
+                    if (list.size() == i) {
+                        list.add(Sets.newHashSet());
+                    }
+                    list.get(i).addAll(backendsPerBucketSeq.getValue().get(i));
+                }
             }
-            return sets;
+            return list;
         } finally {
             readUnlock();
         }
@@ -380,15 +449,21 @@ public class ColocateTableIndex implements Writable {
     public Set<Long> getTabletBackendsByGroup(GroupId groupId, int tabletOrderIdx) {
         readLock();
         try {
-            List<List<Long>> backendsPerBucketSeq = group2BackendsPerBucketSeq.get(groupId);
-            if (backendsPerBucketSeq == null) {
+            Map<Tag, List<List<Long>>> backendsPerBucketSeqMap = group2BackendsPerBucketSeq.row(groupId);
+            if (backendsPerBucketSeqMap == null) {
                 return Sets.newHashSet();
             }
-            if (tabletOrderIdx >= backendsPerBucketSeq.size()) {
-                return Sets.newHashSet();
+
+            // Merge backend ids of all tags
+            Set<Long> beIds = Sets.newHashSet();
+            for (Map.Entry<Tag, List<List<Long>>> backendsPerBucketSeq : backendsPerBucketSeqMap.entrySet()) {
+                if (tabletOrderIdx >= backendsPerBucketSeq.getValue().size()) {
+                    return Sets.newHashSet();
+                }
+                beIds.addAll(backendsPerBucketSeq.getValue().get(tabletOrderIdx));
             }
 
-            return Sets.newHashSet(backendsPerBucketSeq.get(tabletOrderIdx));
+            return beIds;
         } finally {
             readUnlock();
         }
@@ -444,13 +519,13 @@ public class ColocateTableIndex implements Writable {
 
     public void replayAddTableToGroup(ColocatePersistInfo info) throws MetaNotFoundException {
         Database db = Catalog.getCurrentCatalog().getDbOrMetaException(info.getGroupId().dbId);
-        OlapTable tbl = db.getTableOrMetaException(info.getTableId(), Table.TableType.OLAP);
+        OlapTable tbl = db.getTableOrMetaException(info.getTableId(), org.apache.doris.catalog.Table.TableType.OLAP);
         writeLock();
         try {
-            if (!group2BackendsPerBucketSeq.containsKey(info.getGroupId())) {
-                group2BackendsPerBucketSeq.put(info.getGroupId(), info.getBackendsPerBucketSeq());
+            Map<Tag, List<List<Long>>> map = info.getBackendsPerBucketSeq();
+            for (Map.Entry<Tag, List<List<Long>>> entry : map.entrySet()) {
+                group2BackendsPerBucketSeq.put(info.getGroupId(), entry.getKey(), entry.getValue());
             }
-
             addTableToGroup(info.getGroupId().dbId, tbl, tbl.getColocateGroup(), info.getGroupId());
         } finally {
             writeUnlock();
@@ -499,11 +574,12 @@ public class ColocateTableIndex implements Writable {
                 info.add(Joiner.on(", ").join(group2Tables.get(groupId)));
                 ColocateGroupSchema groupSchema = group2Schema.get(groupId);
                 info.add(String.valueOf(groupSchema.getBucketsNum()));
-                info.add(String.valueOf(groupSchema.getReplicationNum()));
+                info.add(String.valueOf(groupSchema.getReplicaAlloc().toCreateStmt()));
                 List<String> cols = groupSchema.getDistributionColTypes().stream().map(
                         e -> e.toSql()).collect(Collectors.toList());
                 info.add(Joiner.on(", ").join(cols));
                 info.add(String.valueOf(!unstableGroups.contains(groupId)));
+                info.add(Strings.nullToEmpty(group2ErrMsgs.get(groupId)));
                 infos.add(info);
             }
         } finally {
@@ -528,12 +604,16 @@ public class ColocateTableIndex implements Writable {
             groupSchema.write(out); // group schema
 
             // backend seq
-            List<List<Long>> backendsPerBucketSeq = group2BackendsPerBucketSeq.get(entry.getValue());
+            Map<Tag, List<List<Long>>> backendsPerBucketSeq = group2BackendsPerBucketSeq.row(entry.getValue());
             out.writeInt(backendsPerBucketSeq.size());
-            for (List<Long> bucket2BEs : backendsPerBucketSeq) {
-                out.writeInt(bucket2BEs.size());
-                for (Long be : bucket2BEs) {
-                    out.writeLong(be);
+            for (Map.Entry<Tag, List<List<Long>>> tag2Bucket2BEs : backendsPerBucketSeq.entrySet()) {
+                tag2Bucket2BEs.getKey().write(out);
+                out.writeInt(tag2Bucket2BEs.getValue().size());
+                for (List<Long> beIds : tag2Bucket2BEs.getValue()) {
+                    out.writeInt(beIds.size());
+                    for (Long be : beIds) {
+                        out.writeLong(be);
+                    }
                 }
             }
         }
@@ -548,60 +628,8 @@ public class ColocateTableIndex implements Writable {
     public void readFields(DataInput in) throws IOException {
         int size = in.readInt();
         if (Catalog.getCurrentCatalogJournalVersion() < FeMetaVersion.VERSION_55) {
-            Multimap<Long, Long> tmpGroup2Tables = ArrayListMultimap.create();
-            Map<Long, Long> tmpTable2Group = Maps.newHashMap();
-            Map<Long, Long> tmpGroup2Db = Maps.newHashMap();
-            Map<Long, List<List<Long>>> tmpGroup2BackendsPerBucketSeq = Maps.newHashMap();
-            Set<Long> tmpBalancingGroups = Sets.newHashSet();
-
-            for (int i = 0; i < size; i++) {
-                long group = in.readLong();
-                int tableSize = in.readInt();
-                List<Long> tables = new ArrayList<>();
-                for (int j = 0; j < tableSize; j++) {
-                    tables.add(in.readLong());
-                }
-                tmpGroup2Tables.putAll(group, tables);
-            }
-
-            size = in.readInt();
-            for (int i = 0; i < size; i++) {
-                long table = in.readLong();
-                long group = in.readLong();
-                tmpTable2Group.put(table, group);
-            }
-
-            size = in.readInt();
-            for (int i = 0; i < size; i++) {
-                long group = in.readLong();
-                long db = in.readLong();
-                tmpGroup2Db.put(group, db);
-            }
-
-            size = in.readInt();
-            for (int i = 0; i < size; i++) {
-                long group = in.readLong();
-                List<List<Long>> bucketBeLists = new ArrayList<>();
-                int bucketBeListsSize = in.readInt();
-                for (int j = 0; j < bucketBeListsSize; j++) {
-                    int beListSize = in.readInt();
-                    List<Long> beLists = new ArrayList<>();
-                    for (int k = 0; k < beListSize; k++) {
-                        beLists.add(in.readLong());
-                    }
-                    bucketBeLists.add(beLists);
-                }
-                tmpGroup2BackendsPerBucketSeq.put(group, bucketBeLists);
-            }
-
-            size = in.readInt();
-            for (int i = 0; i < size; i++) {
-                long group = in.readLong();
-                tmpBalancingGroups.add(group);
-            }
-
-            convertedToNewMembers(tmpGroup2Tables, tmpTable2Group, tmpGroup2Db, tmpGroup2BackendsPerBucketSeq,
-                    tmpBalancingGroups);
+            throw new IOException("This is a very old metadata with version: "
+                    + Catalog.getCurrentCatalogJournalVersion() + ", can not be read");
         } else {
             for (int i = 0; i < size; i++) {
                 String fullGrpName = Text.readString(in);
@@ -616,18 +644,37 @@ public class ColocateTableIndex implements Writable {
                 ColocateGroupSchema groupSchema = ColocateGroupSchema.read(in);
                 group2Schema.put(grpId, groupSchema);
 
-                List<List<Long>> backendsPerBucketSeq = Lists.newArrayList();
-                int beSize = in.readInt();
-                for (int j = 0; j < beSize; j++) {
-                    int seqSize = in.readInt();
-                    List<Long> seq = Lists.newArrayList();
-                    for (int k = 0; k < seqSize; k++) {
-                        long beId = in.readLong();
-                        seq.add(beId);
+                // backends seqs
+                if (Catalog.getCurrentCatalogJournalVersion() < FeMetaVersion.VERSION_105) {
+                    List<List<Long>> bucketsSeq = Lists.newArrayList();
+                    int beSize = in.readInt();
+                    for (int j = 0; j < beSize; j++) {
+                        int seqSize = in.readInt();
+                        List<Long> seq = Lists.newArrayList();
+                        for (int k = 0; k < seqSize; k++) {
+                            long beId = in.readLong();
+                            seq.add(beId);
+                        }
+                        bucketsSeq.add(seq);
+                    }
+                    group2BackendsPerBucketSeq.put(grpId, Tag.DEFAULT_BACKEND_TAG, bucketsSeq);
+                } else {
+                    int tagSize = in.readInt();
+                    for (int j = 0; j < tagSize; j++) {
+                        Tag tag = Tag.read(in);
+                        int bucketSize = in.readInt();
+                        List<List<Long>> bucketsSeq = Lists.newArrayList();
+                        for (int k = 0; k < bucketSize; k++) {
+                            List<Long> beIds = Lists.newArrayList();
+                            int beSize = in.readInt();
+                            for (int l = 0; l < beSize; l++) {
+                                beIds.add(in.readLong());
+                            }
+                            bucketsSeq.add(beIds);
+                        }
+                        group2BackendsPerBucketSeq.put(grpId, tag, bucketsSeq);
                     }
-                    backendsPerBucketSeq.add(seq);
                 }
-                group2BackendsPerBucketSeq.put(grpId, backendsPerBucketSeq);
             }
 
             size = in.readInt();
@@ -637,71 +684,12 @@ public class ColocateTableIndex implements Writable {
         }
     }
 
-    private void convertedToNewMembers(Multimap<Long, Long> tmpGroup2Tables, Map<Long, Long> tmpTable2Group,
-            Map<Long, Long> tmpGroup2Db, Map<Long, List<List<Long>>> tmpGroup2BackendsPerBucketSeq,
-            Set<Long> tmpBalancingGroups) {
-
-        LOG.debug("debug: tmpGroup2Tables {}", tmpGroup2Tables);
-        LOG.debug("debug: tmpTable2Group {}", tmpTable2Group);
-        LOG.debug("debug: tmpGroup2Db {}", tmpGroup2Db);
-        LOG.debug("debug: tmpGroup2BackendsPerBucketSeq {}", tmpGroup2BackendsPerBucketSeq);
-        LOG.debug("debug: tmpBalancingGroups {}", tmpBalancingGroups);
-
-        for (Map.Entry<Long, Long> entry : tmpGroup2Db.entrySet()) {
-            GroupId groupId = new GroupId(entry.getValue(), entry.getKey());
-            Database db = Catalog.getCurrentCatalog().getDbNullable(groupId.dbId);
-            if (db == null) {
-                continue;
-            }
-            Collection<Long> tableIds = tmpGroup2Tables.get(groupId.grpId);
-
-            for (Long tblId : tableIds) {
-                OlapTable tbl = (OlapTable) db.getTableNullable(tblId);
-                if (tbl == null) {
-                    continue;
-                }
-                tbl.readLock();
-                try {
-                    if (tblId.equals(groupId.grpId)) {
-                        // this is a parent table, use its name as group name
-                        groupName2Id.put(groupId.dbId + "_" + tbl.getName(), groupId);
-
-                        ColocateGroupSchema groupSchema = new ColocateGroupSchema(groupId,
-                                ((HashDistributionInfo)tbl.getDefaultDistributionInfo()).getDistributionColumns(),
-                                tbl.getDefaultDistributionInfo().getBucketNum(),
-                                tbl.getPartitionInfo().idToReplicationNum.values().stream().findFirst().get());
-                        group2Schema.put(groupId, groupSchema);
-                        group2BackendsPerBucketSeq.put(groupId, tmpGroup2BackendsPerBucketSeq.get(groupId.grpId));
-                    }
-                } finally {
-                    tbl.readUnlock();
-                }
-
-                group2Tables.put(groupId, tblId);
-                table2Group.put(tblId, groupId);
-            }
-        }
-    }
-
-    public void setBackendsSetByIdxForGroup(GroupId groupId, int tabletOrderIdx, Set<Long> newBackends) {
-        writeLock();
-        try {
-            List<List<Long>> backends = group2BackendsPerBucketSeq.get(groupId);
-            if (backends == null) {
-                return;
-            }
-            Preconditions.checkState(tabletOrderIdx < backends.size(), tabletOrderIdx + " vs. " + backends.size());
-            backends.set(tabletOrderIdx, Lists.newArrayList(newBackends));
-            ColocatePersistInfo info = ColocatePersistInfo.createForBackendsPerBucketSeq(groupId, backends);
-            Catalog.getCurrentCatalog().getEditLog().logColocateBackendsPerBucketSeq(info);
-        } finally {
-            writeUnlock();
-        }
+    public void setErrMsgForGroup(GroupId groupId, String message) {
+        group2ErrMsgs.put(groupId, message);
     }
 
     // just for ut
     public Map<Long, GroupId> getTable2Group() {
         return table2Group;
     }
-
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java
index 7c4db1e..0f15ff9 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Database.java
@@ -107,6 +107,8 @@ public class Database extends MetaObject implements Writable {
     private String attachDbName;
     private DbState dbState;
 
+    private DatabaseProperty dbProperties = new DatabaseProperty();
+
     public Database() {
         this(0, null);
     }
@@ -300,6 +302,7 @@ public class Database extends MetaObject implements Writable {
         return nameToTable.containsKey(tableName);
     }
 
+    // return pair <success?, table exist?>
     public Pair<Boolean, Boolean> createTableWithLock(Table table, boolean isReplay, boolean setIfNotExist) {
         boolean result = true;
         // if a table is already exists, then edit log won't be executed
@@ -534,7 +537,7 @@ public class Database extends MetaObject implements Writable {
                 table.readLock();
                 try {
                     for (Partition partition : olapTable.getAllPartitions()) {
-                        short replicationNum = olapTable.getPartitionInfo().getReplicationNum(partition.getId());
+                        short replicationNum = olapTable.getPartitionInfo().getReplicaAllocation(partition.getId()).getTotalReplicaNum();
                         if (ret < replicationNum) {
                             ret = replicationNum;
                         }
@@ -596,6 +599,7 @@ public class Database extends MetaObject implements Writable {
         dbEncryptKey.write(out);
 
         out.writeLong(replicaQuotaSize);
+        dbProperties.write(out);
     }
 
     @Override
@@ -652,6 +656,10 @@ public class Database extends MetaObject implements Writable {
         } else {
             replicaQuotaSize = FeConstants.default_db_replica_quota_size;
         }
+
+        if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_105) {
+            dbProperties = DatabaseProperty.read(in);
+        }
     }
 
     @Override
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/DatabaseProperty.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/DatabaseProperty.java
new file mode 100644
index 0000000..27ef78e
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/DatabaseProperty.java
@@ -0,0 +1,62 @@
+// 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.Text;
+import org.apache.doris.common.io.Writable;
+import org.apache.doris.persist.gson.GsonUtils;
+
+import com.google.common.collect.Maps;
+import com.google.gson.annotations.SerializedName;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.Map;
+
+public class DatabaseProperty implements Writable {
+
+    @SerializedName(value = "properties")
+    private Map<String, String> properties = Maps.newHashMap();
+
+    public DatabaseProperty() {
+
+    }
+
+    public void put(String key, String val) {
+        properties.put(key, val);
+    }
+
+    public String get(String key) {
+        return properties.get(key);
+    }
+
+    public String getOrDefault(String key, String defaultVal) {
+        return properties.getOrDefault(key, defaultVal);
+    }
+
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+        Text.writeString(out, GsonUtils.GSON.toJson(this));
+    }
+
+    public static DatabaseProperty read(DataInput in) throws IOException {
+        return GsonUtils.GSON.fromJson(Text.readString(in), DatabaseProperty.class);
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/DynamicPartitionProperty.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/DynamicPartitionProperty.java
index 42e62ab..ecbe7f2 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/DynamicPartitionProperty.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/DynamicPartitionProperty.java
@@ -18,8 +18,10 @@
 package org.apache.doris.catalog;
 
 import org.apache.doris.analysis.TimestampArithmeticExpr.TimeUnit;
+import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.FeConstants;
 import org.apache.doris.common.util.DynamicPartitionUtil.StartOfDate;
+import org.apache.doris.common.util.PropertyAnalyzer;
 import org.apache.doris.common.util.TimeUtils;
 
 import java.util.Map;
@@ -37,6 +39,7 @@ public class DynamicPartitionProperty {
     public static final String START_DAY_OF_MONTH = "dynamic_partition.start_day_of_month";
     public static final String TIME_ZONE = "dynamic_partition.time_zone";
     public static final String REPLICATION_NUM = "dynamic_partition.replication_num";
+    public static final String REPLICATION_ALLOCATION = "dynamic_partition.replication_allocation";
     public static final String CREATE_HISTORY_PARTITION = "dynamic_partition.create_history_partition";
     public static final String HISTORY_PARTITION_NUM = "dynamic_partition.history_partition_num";
     public static final String HOT_PARTITION_NUM = "dynamic_partition.hot_partition_num";
@@ -57,7 +60,8 @@ public class DynamicPartitionProperty {
     private StartOfDate startOfWeek;
     private StartOfDate startOfMonth;
     private TimeZone tz = TimeUtils.getSystemTimeZone();
-    private int replicationNum;
+    // if NOT_SET, it will use table's default replica allocation
+    private ReplicaAllocation replicaAlloc;
     private boolean createHistoryPartition = false;
     private int historyPartitionNum;
     // This property are used to describe the number of partitions that need to be reserved on the high-speed storage.
@@ -75,7 +79,7 @@ public class DynamicPartitionProperty {
             this.end = Integer.parseInt(properties.get(END));
             this.prefix = properties.get(PREFIX);
             this.buckets = Integer.parseInt(properties.get(BUCKETS));
-            this.replicationNum = Integer.parseInt(properties.getOrDefault(REPLICATION_NUM, String.valueOf(NOT_SET_REPLICATION_NUM)));
+            this.replicaAlloc = analyzeReplicaAllocation(properties);
             this.createHistoryPartition = Boolean.parseBoolean(properties.get(CREATE_HISTORY_PARTITION));
             this.historyPartitionNum = Integer.parseInt(properties.getOrDefault(HISTORY_PARTITION_NUM, String.valueOf(NOT_SET_HISTORY_PARTITION_NUM)));
             this.hotPartitionNum = Integer.parseInt(properties.getOrDefault(HOT_PARTITION_NUM, "0"));
@@ -85,6 +89,15 @@ public class DynamicPartitionProperty {
         }
     }
 
+    private ReplicaAllocation analyzeReplicaAllocation(Map<String, String> properties) {
+        try {
+            return PropertyAnalyzer.analyzeReplicaAllocation(properties, "dynamic_partition");
+        } catch (AnalysisException e) {
+            // should not happen
+            return ReplicaAllocation.NOT_SET;
+        }
+    }
+
     private void createStartOfs(Map<String, String> properties) {
         if (properties.containsKey(START_DAY_OF_WEEK)) {
             startOfWeek = new StartOfDate(-1, -1, Integer.valueOf(properties.get(START_DAY_OF_WEEK)));
@@ -163,25 +176,22 @@ public class DynamicPartitionProperty {
         return tz;
     }
 
-    public int getReplicationNum() {
-        return replicationNum;
+    public ReplicaAllocation getReplicaAllocation() {
+        return replicaAlloc;
     }
 
     /**
      * use table replication_num as dynamic_partition.replication_num default value
      */
-    public String getProperties(int tableReplicationNum) {
-        int useReplicationNum = replicationNum;
-        if (useReplicationNum == NOT_SET_REPLICATION_NUM) {
-            useReplicationNum = tableReplicationNum;
-        }
+    public String getProperties(ReplicaAllocation tableReplicaAlloc) {
+        ReplicaAllocation tmpAlloc = this.replicaAlloc.isNotSet() ? tableReplicaAlloc : this.replicaAlloc;
         String res = ",\n\"" + ENABLE + "\" = \"" + enable + "\"" +
                 ",\n\"" + TIME_UNIT + "\" = \"" + timeUnit + "\"" +
                 ",\n\"" + TIME_ZONE + "\" = \"" + tz.getID() + "\"" +
                 ",\n\"" + START + "\" = \"" + start + "\"" +
                 ",\n\"" + END + "\" = \"" + end + "\"" +
                 ",\n\"" + PREFIX + "\" = \"" + prefix + "\"" +
-                ",\n\"" + REPLICATION_NUM + "\" = \"" + useReplicationNum + "\"" +
+                ",\n\"" + REPLICATION_ALLOCATION + "\" = \"" + tmpAlloc.toCreateStmt() + "\"" +
                 ",\n\"" + BUCKETS + "\" = \"" + buckets + "\"" +
                 ",\n\"" + CREATE_HISTORY_PARTITION + "\" = \"" + createHistoryPartition + "\"" +
                 ",\n\"" + HISTORY_PARTITION_NUM + "\" = \"" + historyPartitionNum + "\"" +
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/MetadataViewer.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/MetadataViewer.java
index 8734264..9acce81 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/MetadataViewer.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/MetadataViewer.java
@@ -71,7 +71,7 @@ public class MetadataViewer {
             for (String partName : partitions) {
                 Partition partition = olapTable.getPartition(partName);
                 long visibleVersion = partition.getVisibleVersion();
-                short replicationNum = olapTable.getPartitionInfo().getReplicationNum(partition.getId());
+                short replicationNum = olapTable.getPartitionInfo().getReplicaAllocation(partition.getId()).getTotalReplicaNum();
 
                 for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.VISIBLE)) {
                     int schemaHash = olapTable.getSchemaHashByIndexId(index.getId());
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 d409acc..1af2097 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
@@ -39,11 +39,14 @@ import org.apache.doris.common.DdlException;
 import org.apache.doris.common.FeConstants;
 import org.apache.doris.common.FeMetaVersion;
 import org.apache.doris.common.Pair;
+import org.apache.doris.common.UserException;
 import org.apache.doris.common.io.DeepCopy;
 import org.apache.doris.common.io.Text;
 import org.apache.doris.common.util.PropertyAnalyzer;
 import org.apache.doris.common.util.Util;
 import org.apache.doris.qe.OriginStatement;
+import org.apache.doris.resource.Tag;
+import org.apache.doris.system.Backend;
 import org.apache.doris.system.SystemInfoService;
 import org.apache.doris.thrift.TOlapTable;
 import org.apache.doris.thrift.TStorageFormat;
@@ -411,7 +414,7 @@ public class OlapTable extends Table {
         }
     }
 
-    public Status resetIdsForRestore(Catalog catalog, Database db, int restoreReplicationNum) {
+    public Status resetIdsForRestore(Catalog catalog, Database db, ReplicaAllocation restoreReplicaAlloc) {
         // table id
         id = catalog.getNextId();
 
@@ -442,14 +445,14 @@ public class OlapTable extends Table {
         if (partitionInfo.getType() == PartitionType.RANGE || partitionInfo.getType() == PartitionType.LIST) {
             for (Map.Entry<String, Long> entry : origPartNameToId.entrySet()) {
                 long newPartId = catalog.getNextId();
-                partitionInfo.resetPartitionIdForRestore(newPartId, entry.getValue(), (short) restoreReplicationNum, false);
+                partitionInfo.resetPartitionIdForRestore(newPartId, entry.getValue(), restoreReplicaAlloc, false);
                 idToPartition.put(newPartId, idToPartition.remove(entry.getValue()));
             }
         } else {
             // Single partitioned
             long newPartId = catalog.getNextId();
             for (Map.Entry<String, Long> entry : origPartNameToId.entrySet()) {
-                partitionInfo.resetPartitionIdForRestore(newPartId, entry.getValue(), (short) restoreReplicationNum, true);
+                partitionInfo.resetPartitionIdForRestore(newPartId, entry.getValue(), restoreReplicaAlloc, true);
                 idToPartition.put(newPartId, idToPartition.remove(entry.getValue()));
             }
         }
@@ -457,6 +460,8 @@ public class OlapTable extends Table {
         // for each partition, reset rollup index map
         for (Map.Entry<Long, Partition> entry : idToPartition.entrySet()) {
             Partition partition = entry.getValue();
+            // entry.getKey() is the new partition id, use it to get the restore specified replica allocation
+            ReplicaAllocation replicaAlloc = partitionInfo.getReplicaAllocation(entry.getKey());
             for (Map.Entry<Long, String> entry2 : origIdxIdToName.entrySet()) {
                 MaterializedIndex idx = partition.getIndex(entry2.getKey());
                 long newIdxId = indexNameToId.get(entry2.getValue());
@@ -477,19 +482,19 @@ public class OlapTable extends Table {
                     idx.addTablet(newTablet, null /* tablet meta */, true /* is restore */);
 
                     // replicas
-                    List<Long> beIds = Catalog.getCurrentSystemInfo().seqChooseBackendIds(partitionInfo.getReplicationNum(entry.getKey()),
-                                                                                          true, true,
-                                                                                          db.getClusterName());
-                    if (beIds == null) {
-                        return new Status(ErrCode.COMMON_ERROR, "failed to find "
-                                + partitionInfo.getReplicationNum(entry.getKey())
-                                + " different hosts to create table: " + name);
-                    }
-                    for (Long beId : beIds) {
-                        long newReplicaId = catalog.getNextId();
-                        Replica replica = new Replica(newReplicaId, beId, ReplicaState.NORMAL,
-                                partition.getVisibleVersion(), partition.getVisibleVersionHash(), schemaHash);
-                        newTablet.addReplica(replica, true /* is restore */);
+                    try {
+                        Map<Tag, List<Long>> tag2beIds = Catalog.getCurrentSystemInfo().chooseBackendIdByFilters(
+                                replicaAlloc, db.getClusterName(), null);
+                        for (Map.Entry<Tag, List<Long>> entry3 : tag2beIds.entrySet()) {
+                            for (Long beId : entry3.getValue()) {
+                                long newReplicaId = catalog.getNextId();
+                                Replica replica = new Replica(newReplicaId, beId, ReplicaState.NORMAL,
+                                        partition.getVisibleVersion(), partition.getVisibleVersionHash(), schemaHash);
+                                newTablet.addReplica(replica, true /* is restore */);
+                            }
+                        }
+                    } catch (DdlException e) {
+                        return new Status(ErrCode.COMMON_ERROR, e.getMessage());
                     }
                 }
             }
@@ -687,11 +692,11 @@ public class OlapTable extends Table {
                 // recycle partition
                 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()));
+                            partitionInfo.getItem(partition.getId()).getItems(),
+                            new ListPartitionItem(Lists.newArrayList(new PartitionKey())),
+                            partitionInfo.getDataProperty(partition.getId()),
+                            partitionInfo.getReplicaAllocation(partition.getId()),
+                            partitionInfo.getIsInMemory(partition.getId()));
 
                 } else if (partitionInfo.getType() == PartitionType.LIST) {
                     // construct a dummy range
@@ -709,7 +714,7 @@ public class OlapTable extends Table {
                             dummyRange,
                             partitionInfo.getItem(partition.getId()),
                             partitionInfo.getDataProperty(partition.getId()),
-                            partitionInfo.getReplicationNum(partition.getId()),
+                            partitionInfo.getReplicaAllocation(partition.getId()),
                             partitionInfo.getIsInMemory(partition.getId()));
                 }
             } else if (!reserveTablets) {
@@ -1252,7 +1257,7 @@ public class OlapTable extends Table {
                     for (long partitionId : tempRangeInfo.getIdToItem(false).keySet()) {
                         this.partitionInfo.addPartition(partitionId, true,
                                 tempRangeInfo.getItem(partitionId), tempRangeInfo.getDataProperty(partitionId),
-                                tempRangeInfo.getReplicationNum(partitionId), tempRangeInfo.getIsInMemory(partitionId));
+                                tempRangeInfo.getReplicaAllocation(partitionId), tempRangeInfo.getIsInMemory(partitionId));
                     }
                 }
                 tempPartitions.unsetPartitionInfo();
@@ -1337,7 +1342,7 @@ public class OlapTable extends Table {
         nameToPartition.put(newPartition.getName(), newPartition);
 
         DataProperty dataProperty = partitionInfo.getDataProperty(oldPartition.getId());
-        short replicationNum = partitionInfo.getReplicationNum(oldPartition.getId());
+        ReplicaAllocation replicaAlloc = partitionInfo.getReplicaAllocation(oldPartition.getId());
         boolean isInMemory = partitionInfo.getIsInMemory(oldPartition.getId());
 
         if (partitionInfo.getType() == PartitionType.RANGE
@@ -1345,10 +1350,10 @@ public class OlapTable extends Table {
             PartitionItem item = partitionInfo.getItem(oldPartition.getId());
             partitionInfo.dropPartition(oldPartition.getId());
             partitionInfo.addPartition(newPartition.getId(), false, item, dataProperty,
-                    replicationNum, isInMemory);
+                    replicaAlloc, isInMemory);
         } else {
             partitionInfo.dropPartition(oldPartition.getId());
-            partitionInfo.addPartition(newPartition.getId(), dataProperty, replicationNum, isInMemory);
+            partitionInfo.addPartition(newPartition.getId(), dataProperty, replicaAlloc, isInMemory);
         }
 
         return oldPartition;
@@ -1393,7 +1398,7 @@ public class OlapTable extends Table {
         for (Partition partition : idToPartition.values()) {
             long visibleVersion = partition.getVisibleVersion();
             long visibleVersionHash = partition.getVisibleVersionHash();
-            short replicationNum = partitionInfo.getReplicationNum(partition.getId());
+            ReplicaAllocation replicaAlloc = partitionInfo.getReplicaAllocation(partition.getId());
             for (MaterializedIndex mIndex : partition.getMaterializedIndices(IndexExtState.ALL)) {
                 for (Tablet tablet : mIndex.getTablets()) {
                     if (tabletScheduler.containsTablet(tablet.getId())) {
@@ -1403,7 +1408,7 @@ public class OlapTable extends Table {
                     }
 
                     Pair<TabletStatus, TabletSchedCtx.Priority> statusPair = tablet.getHealthStatusWithPriority(
-                            infoService, clusterName, visibleVersion, visibleVersionHash, replicationNum,
+                            infoService, clusterName, visibleVersion, visibleVersionHash, replicaAlloc,
                             aliveBeIdsInCluster);
                     if (statusPair.first != TabletStatus.HEALTHY) {
                         LOG.info("table {} is not stable because tablet {} status is {}. replicas: {}",
@@ -1417,20 +1422,45 @@ public class OlapTable extends Table {
     }
 
     // arbitrarily choose a partition, and get the buckets backends sequence from base index.
-    public List<List<Long>> getArbitraryTabletBucketsSeq() throws DdlException {
-        List<List<Long>> backendsPerBucketSeq = Lists.newArrayList();
+    public Map<Tag, List<List<Long>>> getArbitraryTabletBucketsSeq() throws DdlException {
+        SystemInfoService infoService = Catalog.getCurrentSystemInfo();
+        Map<Tag, List<List<Long>>> backendsPerBucketSeq = Maps.newHashMap();
         for (Partition partition : idToPartition.values()) {
-            short replicationNum = partitionInfo.getReplicationNum(partition.getId());
+            ReplicaAllocation replicaAlloc = partitionInfo.getReplicaAllocation(partition.getId());
+            short totalReplicaNum = replicaAlloc.getTotalReplicaNum();
             MaterializedIndex baseIdx = partition.getBaseIndex();
             for (Long tabletId : baseIdx.getTabletIdsInOrder()) {
                 Tablet tablet = baseIdx.getTablet(tabletId);
                 List<Long> replicaBackendIds = tablet.getNormalReplicaBackendIds();
-                if (replicaBackendIds.size() < replicationNum) {
+                if (replicaBackendIds.size() != totalReplicaNum) {
                     // this should not happen, but in case, throw an exception to terminate this process
                     throw new DdlException("Normal replica number of tablet " + tabletId + " is: "
-                            + replicaBackendIds.size() + ", which is less than expected: " + replicationNum);
+                            + replicaBackendIds.size() + ", but expected: " + totalReplicaNum);
+                }
+
+                // check tag
+                Map<Tag, Short> currentReplicaAlloc = Maps.newHashMap();
+                Map<Tag, List<Long>> tag2beIds = Maps.newHashMap();
+                for (long beId : replicaBackendIds) {
+                    Backend be = infoService.getBackend(beId);
+                    if (be == null) {
+                        continue;
+                    }
+                    short num = currentReplicaAlloc.getOrDefault(be.getTag(), (short) 0);
+                    currentReplicaAlloc.put(be.getTag(), (short) (num + 1));
+                    List<Long> beIds = tag2beIds.getOrDefault(be.getTag(), Lists.newArrayList());
+                    beIds.add(beId);
+                    tag2beIds.put(be.getTag(), beIds);
+                }
+                if (!currentReplicaAlloc.equals(replicaAlloc.getAllocMap())) {
+                    throw new DdlException("The relica allocation is " + currentReplicaAlloc.toString()
+                            + ", but expected: " + replicaAlloc.toCreateStmt());
+                }
+
+                for (Map.Entry<Tag, List<Long>> entry : tag2beIds.entrySet()) {
+                    backendsPerBucketSeq.putIfAbsent(entry.getKey(), Lists.newArrayList());
+                    backendsPerBucketSeq.get(entry.getKey()).add(entry.getValue());
                 }
-                backendsPerBucketSeq.add(replicaBackendIds.subList(0, replicationNum));
             }
             break;
         }
@@ -1507,19 +1537,18 @@ public class OlapTable extends Table {
         return hasChanged;
     }
 
-    public void setReplicationNum(Short replicationNum) {
+    public void setReplicationAllocation(ReplicaAllocation replicaAlloc) {
         if (tableProperty == null) {
             tableProperty = new TableProperty(new HashMap<>());
         }
-        tableProperty.modifyTableProperties(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM, replicationNum.toString());
-        tableProperty.buildReplicationNum();
+        tableProperty.setReplicaAlloc(replicaAlloc);
     }
 
-    public Short getDefaultReplicationNum() {
+    public ReplicaAllocation getDefaultReplicaAllocation() {
         if (tableProperty != null) {
-            return tableProperty.getReplicationNum();
+            return tableProperty.getReplicaAllocation();
         }
-        return FeConstants.default_replication_num;
+        return ReplicaAllocation.DEFAULT_ALLOCATION;
     }
 
     public Boolean isInMemory() {
@@ -1717,4 +1746,30 @@ public class OlapTable extends Table {
         }
         return false;
     }
+
+    // for ut
+    public void checkReplicaAllocation() throws UserException {
+        SystemInfoService infoService = Catalog.getCurrentSystemInfo();
+        for (Partition partition : getPartitions()) {
+            ReplicaAllocation replicaAlloc = getPartitionInfo().getReplicaAllocation(partition.getId());
+            Map<Tag, Short> allocMap = replicaAlloc.getAllocMap();
+            for (MaterializedIndex mIndex : partition.getMaterializedIndices(IndexExtState.VISIBLE)) {
+                for (Tablet tablet : mIndex.getTablets()) {
+                    Map<Tag, Short> curMap = Maps.newHashMap();
+                    for (Replica replica : tablet.getReplicas()) {
+                        Backend be = infoService.getBackend(replica.getBackendId());
+                        if (be == null) {
+                            continue;
+                        }
+                        short num = curMap.getOrDefault(be.getTag(), (short) 0);
+                        curMap.put(be.getTag(), (short) (num + 1));
+                    }
+                    if (!curMap.equals(allocMap)) {
+                        throw new UserException("replica allocation of tablet " + tablet.getId() + " is not expected"
+                                + ", expected: " + allocMap.toString() + ", actual: " + curMap.toString());
+                    }
+                }
+            }
+        }
+    }
 }
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 9451910..0790468 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
@@ -56,8 +56,8 @@ public class PartitionInfo implements Writable {
     protected Map<Long, PartitionItem> idToTempItem = Maps.newHashMap();
     // partition id -> data property
     protected Map<Long, DataProperty> idToDataProperty;
-    // partition id -> replication num
-    protected Map<Long, Short> idToReplicationNum;
+    // partition id -> replication allocation
+    protected Map<Long, ReplicaAllocation> idToReplicaAllocation;
     // true if the partition has multi partition columns
     protected boolean isMultiColumnPartition = false;
 
@@ -70,7 +70,7 @@ public class PartitionInfo implements Writable {
 
     public PartitionInfo() {
         this.idToDataProperty = new HashMap<>();
-        this.idToReplicationNum = new HashMap<>();
+        this.idToReplicaAllocation = new HashMap<>();
         this.idToInMemory = new HashMap<>();
         this.idToTabletType = new HashMap<>();
     }
@@ -78,7 +78,7 @@ public class PartitionInfo implements Writable {
     public PartitionInfo(PartitionType type) {
         this.type = type;
         this.idToDataProperty = new HashMap<>();
-        this.idToReplicationNum = new HashMap<>();
+        this.idToReplicaAllocation = new HashMap<>();
         this.idToInMemory = new HashMap<>();
         this.idToTabletType = new HashMap<>();
     }
@@ -132,7 +132,7 @@ public class PartitionInfo implements Writable {
         setItemInternal(partitionId, isTemp, partitionItem);
 
         idToDataProperty.put(partitionId, desc.getPartitionDataProperty());
-        idToReplicationNum.put(partitionId, desc.getReplicationNum());
+        idToReplicaAllocation.put(partitionId, desc.getReplicaAlloc());
         idToInMemory.put(partitionId, desc.isInMemory());
 
         return partitionItem;
@@ -143,11 +143,11 @@ public class PartitionInfo implements Writable {
     }
 
     public void unprotectHandleNewSinglePartitionDesc(long partitionId, boolean isTemp, PartitionItem partitionItem,
-                                                      DataProperty dataProperty, short replicationNum,
+                                                      DataProperty dataProperty, ReplicaAllocation replicaAlloc,
                                                       boolean isInMemory) {
         setItemInternal(partitionId, isTemp, partitionItem);
         idToDataProperty.put(partitionId, dataProperty);
-        idToReplicationNum.put(partitionId, replicationNum);
+        idToReplicaAllocation.put(partitionId, replicaAlloc);
         idToInMemory.put(partitionId, isInMemory);
     }
 
@@ -209,15 +209,16 @@ public class PartitionInfo implements Writable {
         idToDataProperty.put(partitionId, newDataProperty);
     }
 
-    public short getReplicationNum(long partitionId) {
-        if (!idToReplicationNum.containsKey(partitionId)) {
-            LOG.debug("failed to get replica num for partition: {}", partitionId);
+    public ReplicaAllocation getReplicaAllocation(long partitionId) {
+        if (!idToReplicaAllocation.containsKey(partitionId)) {
+            LOG.debug("failed to get replica allocation for partition: {}", partitionId);
+            return ReplicaAllocation.DEFAULT_ALLOCATION;
         }
-        return idToReplicationNum.get(partitionId);
+        return idToReplicaAllocation.get(partitionId);
     }
 
-    public void setReplicationNum(long partitionId, short replicationNum) {
-        idToReplicationNum.put(partitionId, replicationNum);
+    public void setReplicaAllocation(long partitionId, ReplicaAllocation replicaAlloc) {
+        this.idToReplicaAllocation.put(partitionId, replicaAlloc);
     }
 
     public boolean getIsInMemory(long partitionId) {
@@ -241,23 +242,23 @@ public class PartitionInfo implements Writable {
 
     public void dropPartition(long partitionId) {
         idToDataProperty.remove(partitionId);
-        idToReplicationNum.remove(partitionId);
+        idToReplicaAllocation.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);
+                             ReplicaAllocation replicaAlloc, boolean isInMemory) {
+        addPartition(partitionId, dataProperty, replicaAlloc, isInMemory);
         setItemInternal(partitionId, isTemp, item);
     }
 
     public void addPartition(long partitionId, DataProperty dataProperty,
-                             short replicationNum,
+                             ReplicaAllocation replicaAlloc,
                              boolean isInMemory) {
         idToDataProperty.put(partitionId, dataProperty);
-        idToReplicationNum.put(partitionId, replicationNum);
+        idToReplicaAllocation.put(partitionId, replicaAlloc);
         idToInMemory.put(partitionId, isInMemory);
     }
 
@@ -282,10 +283,11 @@ public class PartitionInfo implements Writable {
         }
     }
 
-    public void resetPartitionIdForRestore(long newPartitionId, long oldPartitionId, short restoreReplicationNum, boolean isSinglePartitioned) {
+    public void resetPartitionIdForRestore(long newPartitionId, long oldPartitionId, ReplicaAllocation restoreReplicaAlloc,
+                                           boolean isSinglePartitioned) {
         idToDataProperty.put(newPartitionId, idToDataProperty.remove(oldPartitionId));
-        idToReplicationNum.remove(oldPartitionId);
-        idToReplicationNum.put(newPartitionId, restoreReplicationNum);
+        idToReplicaAllocation.remove(oldPartitionId);
+        idToReplicaAllocation.put(newPartitionId, restoreReplicaAlloc);
         if (!isSinglePartitioned) {
             idToItem.put(newPartitionId, idToItem.remove(oldPartitionId));
         }
@@ -296,8 +298,8 @@ public class PartitionInfo implements Writable {
     public void write(DataOutput out) throws IOException {
         Text.writeString(out, type.name());
 
-        Preconditions.checkState(idToDataProperty.size() == idToReplicationNum.size());
-        Preconditions.checkState(idToInMemory.keySet().equals(idToReplicationNum.keySet()));
+        Preconditions.checkState(idToDataProperty.size() == idToReplicaAllocation.size());
+        Preconditions.checkState(idToInMemory.keySet().equals(idToReplicaAllocation.keySet()));
         out.writeInt(idToDataProperty.size());
         for (Map.Entry<Long, DataProperty> entry : idToDataProperty.entrySet()) {
             out.writeLong(entry.getKey());
@@ -308,7 +310,7 @@ public class PartitionInfo implements Writable {
                 entry.getValue().write(out);
             }
 
-            out.writeShort(idToReplicationNum.get(entry.getKey()));
+            idToReplicaAllocation.get(entry.getKey()).write(out);
             out.writeBoolean(idToInMemory.get(entry.getKey()));
         }
     }
@@ -326,8 +328,15 @@ public class PartitionInfo implements Writable {
                 idToDataProperty.put(partitionId, DataProperty.read(in));
             }
 
-            short replicationNum = in.readShort();
-            idToReplicationNum.put(partitionId, replicationNum);
+            if (Catalog.getCurrentCatalogJournalVersion() < FeMetaVersion.VERSION_105) {
+                short replicationNum = in.readShort();
+                ReplicaAllocation replicaAlloc = new ReplicaAllocation(replicationNum);
+                idToReplicaAllocation.put(partitionId, replicaAlloc);
+            } else {
+                ReplicaAllocation replicaAlloc = ReplicaAllocation.read(in);
+                idToReplicaAllocation.put(partitionId, replicaAlloc);
+            }
+
             if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_72) {
                 idToInMemory.put(partitionId, in.readBoolean());
             } else {
@@ -350,11 +359,12 @@ public class PartitionInfo implements Writable {
                 buff.append(false);
             }
             buff.append("; ");
-            buff.append("data_property: ").append(entry.getValue().toString()).append("; ");;
-            buff.append("replica number: ").append(idToReplicationNum.get(entry.getKey())).append("; ");;
+            buff.append("data_property: ").append(entry.getValue().toString()).append("; ");
+            buff.append("replica number: ").append(idToReplicaAllocation.get(entry.getKey())).append("; ");
             buff.append("in memory: ").append(idToInMemory.get(entry.getKey()));
         }
 
         return buff.toString();
     }
+
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/ReplicaAllocation.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/ReplicaAllocation.java
new file mode 100644
index 0000000..6b02270
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/ReplicaAllocation.java
@@ -0,0 +1,132 @@
+// 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.Text;
+import org.apache.doris.common.io.Writable;
+import org.apache.doris.common.util.PropertyAnalyzer;
+import org.apache.doris.persist.gson.GsonUtils;
+import org.apache.doris.resource.Tag;
+
+import com.clearspring.analytics.util.Lists;
+import com.google.common.base.Joiner;
+import com.google.common.collect.Maps;
+import com.google.gson.annotations.SerializedName;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+// ReplicaAllocation is used to describe the distribution of replicas of a tablet.
+// By default, 3 replicas of a tablet are distributed on 3 BE nodes with Tag "default".
+public class ReplicaAllocation implements Writable {
+
+    public static final ReplicaAllocation DEFAULT_ALLOCATION;
+    // represent that replica allocation is not set.
+    public static final ReplicaAllocation NOT_SET;
+
+    static {
+        DEFAULT_ALLOCATION = new ReplicaAllocation((short) 3);
+        NOT_SET = new ReplicaAllocation();
+    }
+
+    @SerializedName(value = "allocMap")
+    private Map<Tag, Short> allocMap = Maps.newHashMap();
+
+    public ReplicaAllocation() {
+
+    }
+
+    // For convert the old replica number to replica allocation
+    public ReplicaAllocation(short replicaNum) {
+        allocMap.put(Tag.DEFAULT_BACKEND_TAG, replicaNum);
+    }
+
+    public ReplicaAllocation(Map<Tag, Short> allocMap) {
+        this.allocMap = allocMap;
+    }
+
+    public void put(Tag tag, Short num) {
+        this.allocMap.put(tag, num);
+    }
+
+    public Map<Tag, Short> getAllocMap() {
+        return allocMap;
+    }
+
+    public short getTotalReplicaNum() {
+        short num = 0;
+        for (Short s : allocMap.values()) {
+            num += s;
+        }
+        return num;
+    }
+
+    public boolean isEmpty() {
+        return allocMap.isEmpty();
+    }
+
+    public boolean isNotSet() {
+        return this.equals(NOT_SET);
+    }
+
+    public Short getReplicaNumByTag(Tag tag) {
+        return allocMap.getOrDefault(tag, (short) 0);
+    }
+
+    public static ReplicaAllocation read(DataInput in) throws IOException {
+        String json = Text.readString(in);
+        return GsonUtils.GSON.fromJson(json, ReplicaAllocation.class);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        ReplicaAllocation that = (ReplicaAllocation) o;
+        return that.allocMap.equals(this.allocMap);
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hash(allocMap);
+    }
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+        Text.writeString(out, GsonUtils.GSON.toJson(this));
+    }
+
+    @Override
+    public String toString() {
+        return toCreateStmt();
+    }
+
+    // For show create table stmt. like:
+    // "tag.location.zone1: 2, tag.location.zone2: 1"
+    public String toCreateStmt() {
+        List<String> tags = Lists.newArrayList();
+        for (Map.Entry<Tag, Short> entry : allocMap.entrySet()) {
+            tags.add(PropertyAnalyzer.TAG_LOCATION + "." + entry.getKey().value + ": " + entry.getValue());
+        }
+        return Joiner.on(", ").join(tags);
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java
index 8f1b2c9..dcbaeba 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableProperty.java
@@ -17,9 +17,10 @@
 
 package org.apache.doris.catalog;
 
+import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.DdlException;
-import org.apache.doris.common.FeConstants;
+import org.apache.doris.common.FeMetaVersion;
 import org.apache.doris.common.io.Text;
 import org.apache.doris.common.io.Writable;
 import org.apache.doris.common.util.PropertyAnalyzer;
@@ -27,9 +28,13 @@ import org.apache.doris.persist.OperationType;
 import org.apache.doris.persist.gson.GsonUtils;
 import org.apache.doris.thrift.TStorageFormat;
 
+import com.google.common.base.Strings;
 import com.google.common.collect.Maps;
 import com.google.gson.annotations.SerializedName;
 
+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,18 +44,19 @@ import java.util.Map;
 /**  TableProperty contains additional information about OlapTable
  *  TableProperty includes properties to persistent the additional information
  *  Different properties is recognized by prefix such as dynamic_partition
- *  If there is different type properties is added.Write a method such as buildDynamicProperty to build it.
+ *  If there is different type properties is added, write a method such as buildDynamicProperty to build it.
  */
 public class TableProperty implements Writable {
+    private static final Logger LOG = LogManager.getLogger(TableProperty.class);
+
     public static final String DYNAMIC_PARTITION_PROPERTY_PREFIX = "dynamic_partition";
 
     @SerializedName(value = "properties")
     private Map<String, String> properties;
 
+    // the follower variables are built from "properties"
     private DynamicPartitionProperty dynamicPartitionProperty = new DynamicPartitionProperty(Maps.newHashMap());
-    // table's default replication num
-    private Short replicationNum = FeConstants.default_replication_num;
-
+    private ReplicaAllocation replicaAlloc = ReplicaAllocation.DEFAULT_ALLOCATION;
     private boolean isInMemory = false;
 
     /*
@@ -82,7 +88,7 @@ public class TableProperty implements Writable {
                 executeBuildDynamicProperty();
                 break;
             case OperationType.OP_MODIFY_REPLICATION_NUM:
-                buildReplicationNum();
+                buildReplicaAllocation();
                 break;
             case OperationType.OP_MODIFY_IN_MEMORY:
                 buildInMemory();
@@ -116,12 +122,6 @@ public class TableProperty implements Writable {
         return this;
     }
 
-    public TableProperty buildReplicationNum() {
-        replicationNum = Short.parseShort(properties.getOrDefault(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM,
-                String.valueOf(FeConstants.default_replication_num)));
-        return this;
-    }
-
     public TableProperty buildInMemory() {
         isInMemory = Boolean.parseBoolean(properties.getOrDefault(PropertyAnalyzer.PROPERTIES_INMEMORY, "false"));
         return this;
@@ -137,6 +137,17 @@ public class TableProperty implements Writable {
         properties.putAll(modifyProperties);
     }
 
+    public void setReplicaAlloc(ReplicaAllocation replicaAlloc) {
+        this.replicaAlloc = replicaAlloc;
+        // set it to "properties" so that this info can be persisted
+        properties.put("default." + PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION,
+                replicaAlloc.toCreateStmt());
+    }
+
+    public ReplicaAllocation getReplicaAllocation() {
+        return replicaAlloc;
+    }
+
     public void modifyTableProperties(String key, String value) {
         properties.put(key, value);
     }
@@ -159,10 +170,6 @@ public class TableProperty implements Writable {
         return origProp;
     }
 
-    public Short getReplicationNum() {
-        return replicationNum;
-    }
-
     public boolean isInMemory() {
         return isInMemory;
     }
@@ -171,16 +178,42 @@ public class TableProperty implements Writable {
         return storageFormat;
     }
 
+    public void buildReplicaAllocation() {
+        try {
+            // Must copy the properties because "analyzeReplicaAllocation" with remove the property
+            // from the properties.
+            Map<String, String> copiedProperties = Maps.newHashMap(properties);
+            this.replicaAlloc = PropertyAnalyzer.analyzeReplicaAllocation(copiedProperties, "default");
+        } catch (AnalysisException e) {
+            // should not happen
+            LOG.error("should not happen when build replica allocation", e);
+            this.replicaAlloc = ReplicaAllocation.DEFAULT_ALLOCATION;
+        }
+    }
+
     @Override
     public void write(DataOutput out) throws IOException {
         Text.writeString(out, GsonUtils.GSON.toJson(this));
     }
 
     public static TableProperty read(DataInput in) throws IOException {
-        return GsonUtils.GSON.fromJson(Text.readString(in), TableProperty.class)
+        TableProperty tableProperty = GsonUtils.GSON.fromJson(Text.readString(in), TableProperty.class)
                 .executeBuildDynamicProperty()
-                .buildReplicationNum()
                 .buildInMemory()
                 .buildStorageFormat();
+        if (Catalog.getCurrentCatalogJournalVersion() < FeMetaVersion.VERSION_105) {
+            // get replica num from property map and create replica allocation
+            String repNum = tableProperty.properties.remove(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM);
+            if (!Strings.isNullOrEmpty(repNum)) {
+                ReplicaAllocation replicaAlloc = new ReplicaAllocation(Short.valueOf(repNum));
+                tableProperty.properties.put("default." + PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION,
+                        replicaAlloc.toCreateStmt());
+            } else {
+                tableProperty.properties.put("default." + PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION,
+                        ReplicaAllocation.DEFAULT_ALLOCATION.toCreateStmt());
+            }
+        }
+        tableProperty.buildReplicaAllocation();
+        return tableProperty;
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Tablet.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Tablet.java
index eba74ad..fa795ed 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Tablet.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Tablet.java
@@ -23,24 +23,27 @@ import org.apache.doris.clone.TabletSchedCtx.Priority;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.Pair;
 import org.apache.doris.common.io.Writable;
+import org.apache.doris.resource.Tag;
 import org.apache.doris.system.Backend;
 import org.apache.doris.system.SystemInfoService;
 
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
 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.Sets;
 import com.google.gson.annotations.SerializedName;
 
-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.ArrayList;
 import java.util.Iterator;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.stream.Collectors;
 import java.util.stream.LongStream;
@@ -58,6 +61,7 @@ public class Tablet extends MetaObject implements Writable {
         REPLICA_RELOCATING, // replica is healthy, but is under relocating (eg. BE is decommission).
         REDUNDANT, // too much replicas.
         REPLICA_MISSING_IN_CLUSTER, // not enough healthy replicas in correct cluster.
+        REPLICA_MISSING_FOR_TAG, // not enough healthy replicas in backend with specified tag.
         FORCE_REDUNDANT, // some replica is missing or bad, but there is no other backends for repair,
                          // at least one replica has to be deleted first to make room for new replica.
         COLOCATE_MISMATCH, // replicas do not all locate in right colocate backends set.
@@ -392,21 +396,26 @@ public class Tablet extends MetaObject implements Writable {
      * A replica is healthy only if
      * 1. the backend is available
      * 2. replica version is caught up, and last failed version is -1
-     *
+     * <p>
      * A tablet is healthy only if
      * 1. healthy replica num is equal to replicationNum
-     * 2. all healthy replicas are in right cluster
+     * 2. all healthy replicas are in right cluster and tag
      */
     public Pair<TabletStatus, TabletSchedCtx.Priority> getHealthStatusWithPriority(
             SystemInfoService systemInfoService, String clusterName,
-            long visibleVersion, long visibleVersionHash, int replicationNum,
+            long visibleVersion, long visibleVersionHash, ReplicaAllocation replicaAlloc,
             List<Long> aliveBeIdsInCluster) {
 
+
+        Map<Tag, Short> allocMap = replicaAlloc.getAllocMap();
+        Map<Tag, Short> currentAllocMap = Maps.newHashMap();
+
+        short replicationNum = replicaAlloc.getTotalReplicaNum();
         int alive = 0;
         int aliveAndVersionComplete = 0;
         int stable = 0;
         int availableInCluster = 0;
-        
+
         Replica needFurtherRepairReplica = null;
         Set<String> hosts = Sets.newHashSet();
         for (Replica replica : replicas) {
@@ -443,6 +452,9 @@ public class Tablet extends MetaObject implements Writable {
             if (replica.needFurtherRepair() && needFurtherRepairReplica == null) {
                 needFurtherRepairReplica = replica;
             }
+
+            short curNum = currentAllocMap.getOrDefault(backend.getTag(), (short) 0);
+            currentAllocMap.put(backend.getTag(), (short) (curNum + 1));
         }
 
         // 1. alive replicas are not enough
@@ -504,7 +516,17 @@ public class Tablet extends MetaObject implements Writable {
         // 4. healthy replicas in cluster are not enough
         if (availableInCluster < replicationNum) {
             return Pair.create(TabletStatus.REPLICA_MISSING_IN_CLUSTER, TabletSchedCtx.Priority.LOW);
-        } else if (replicas.size() > replicationNum) {
+        }
+
+        // 5. got enough healthy replicas, check tag
+        for (Map.Entry<Tag, Short> alloc : allocMap.entrySet()) {
+            if (!currentAllocMap.containsKey(alloc.getKey())
+                    || currentAllocMap.get(alloc.getKey()) < alloc.getValue()) {
+                return Pair.create(TabletStatus.REPLICA_MISSING_FOR_TAG, TabletSchedCtx.Priority.NORMAL);
+            }
+        }
+
+        if (replicas.size() > replicationNum) {
             if (needFurtherRepairReplica != null) {
                 return Pair.create(TabletStatus.NEED_FURTHER_REPAIR, TabletSchedCtx.Priority.HIGH);
             }
@@ -512,7 +534,7 @@ public class Tablet extends MetaObject implements Writable {
             return Pair.create(TabletStatus.REDUNDANT, TabletSchedCtx.Priority.VERY_HIGH);
         }
 
-        // 5. healthy
+        // 6. healthy
         return Pair.create(TabletStatus.HEALTHY, TabletSchedCtx.Priority.NORMAL);
     }
 
@@ -527,19 +549,21 @@ public class Tablet extends MetaObject implements Writable {
      *      
      *      backends set:       1,2,3
      *      tablet replicas:    1,2,4,5
-     *      
+     *
      * 2. Version incomplete:
      *      backend matched, but some replica(in backends set)'s version is incomplete
-     *      
+     *
      * 3. Redundant:
      *      backends set:       1,2,3
      *      tablet replicas:    1,2,3,4
-     *      
+     *
      * No need to check if backend is available. We consider all backends in 'backendsSet' are available,
      * If not, unavailable backends will be relocated by CalocateTableBalancer first.
      */
-    public TabletStatus getColocateHealthStatus(long visibleVersion, int replicationNum, Set<Long> backendsSet) {
-
+    public TabletStatus getColocateHealthStatus(long visibleVersion, ReplicaAllocation replicaAlloc, Set<Long> backendsSet) {
+        // Here we don't need to care about tag. Because the replicas of the colocate table has been confirmed
+        // in ColocateTableCheckerAndBalancer.
+        Short totalReplicaNum = replicaAlloc.getTotalReplicaNum();
         // 1. check if replicas' backends are mismatch
         Set<Long> replicaBackendIds = getBackendIds();
         if (!replicaBackendIds.containsAll(backendsSet)) {
@@ -560,7 +584,7 @@ public class Tablet extends MetaObject implements Writable {
         }
 
         // 3. check redundant
-        if (replicas.size() > replicationNum) {
+        if (replicas.size() > totalReplicaNum) {
             return TabletStatus.COLOCATE_REDUNDANT;
         }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/BackendLoadStatistic.java b/fe/fe-core/src/main/java/org/apache/doris/clone/BackendLoadStatistic.java
index f4d0bbb..2210a3f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/clone/BackendLoadStatistic.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/clone/BackendLoadStatistic.java
@@ -23,6 +23,7 @@ import org.apache.doris.catalog.TabletInvertedIndex;
 import org.apache.doris.clone.BalanceStatus.ErrCode;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.util.DebugUtil;
+import org.apache.doris.resource.Tag;
 import org.apache.doris.system.Backend;
 import org.apache.doris.system.SystemInfoService;
 import org.apache.doris.thrift.TStorageMedium;
@@ -87,6 +88,8 @@ public class BackendLoadStatistic {
 
     private boolean isAvailable;
 
+    private Tag tag;
+
     public static class LoadScore {
         public double replicaNumCoefficient = 0.5;
         public double capacityCoefficient = 0.5;
@@ -102,10 +105,11 @@ public class BackendLoadStatistic {
     private Map<TStorageMedium, Classification> clazzMap = Maps.newHashMap();
     private List<RootPathLoadStatistic> pathStatistics = Lists.newArrayList();
 
-    public BackendLoadStatistic(long beId, String clusterName, SystemInfoService infoService,
-            TabletInvertedIndex invertedIndex) {
+    public BackendLoadStatistic(long beId, String clusterName, Tag tag, SystemInfoService infoService,
+                                TabletInvertedIndex invertedIndex) {
         this.beId = beId;
         this.clusterName = clusterName;
+        this.tag = tag;
         this.infoService = infoService;
         this.invertedIndex = invertedIndex;
     }
@@ -118,6 +122,10 @@ public class BackendLoadStatistic {
         return clusterName;
     }
 
+    public Tag getTag() {
+        return tag;
+    }
+
     public boolean isAvailable() {
         return isAvailable;
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/BeLoadRebalancer.java b/fe/fe-core/src/main/java/org/apache/doris/clone/BeLoadRebalancer.java
index 9e26978..fb99b88 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/clone/BeLoadRebalancer.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/clone/BeLoadRebalancer.java
@@ -71,7 +71,8 @@ public class BeLoadRebalancer extends Rebalancer {
      */
     @Override
     protected List<TabletSchedCtx> selectAlternativeTabletsForCluster(
-            String clusterName, ClusterLoadStatistic clusterStat, TStorageMedium medium) {
+            ClusterLoadStatistic clusterStat, TStorageMedium medium) {
+        String clusterName = clusterStat.getClusterName();
         List<TabletSchedCtx> alternativeTablets = Lists.newArrayList();
 
         // get classification of backends
@@ -162,7 +163,9 @@ public class BeLoadRebalancer extends Rebalancer {
 
                     TabletSchedCtx tabletCtx = new TabletSchedCtx(TabletSchedCtx.Type.BALANCE, clusterName,
                             tabletMeta.getDbId(), tabletMeta.getTableId(), tabletMeta.getPartitionId(),
-                            tabletMeta.getIndexId(), tabletId, System.currentTimeMillis());
+                            tabletMeta.getIndexId(), tabletId, null /* replica alloc is not used for balance*/,
+                            System.currentTimeMillis());
+                    tabletCtx.setTag(clusterStat.getTag());
                     // balance task's priority is always LOW
                     tabletCtx.setOrigPriority(Priority.LOW);
 
@@ -197,7 +200,7 @@ public class BeLoadRebalancer extends Rebalancer {
      */
     @Override
     public void completeSchedCtx(TabletSchedCtx tabletCtx, Map<Long, PathSlot> backendsWorkingSlots) throws SchedException {
-        ClusterLoadStatistic clusterStat = statisticMap.get(tabletCtx.getCluster());
+        ClusterLoadStatistic clusterStat = statisticMap.get(tabletCtx.getCluster(), tabletCtx.getTag());
         if (clusterStat == null) {
             throw new SchedException(Status.UNRECOVERABLE, "cluster does not exist");
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/ClusterLoadStatistic.java b/fe/fe-core/src/main/java/org/apache/doris/clone/ClusterLoadStatistic.java
index e2b6cdc..6dd18c1 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/clone/ClusterLoadStatistic.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/clone/ClusterLoadStatistic.java
@@ -17,18 +17,20 @@
 
 package org.apache.doris.clone;
 
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.TreeMultimap;
 import org.apache.doris.catalog.TabletInvertedIndex;
 import org.apache.doris.clone.BackendLoadStatistic.Classification;
 import org.apache.doris.clone.BackendLoadStatistic.LoadScore;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.util.DebugUtil;
+import org.apache.doris.resource.Tag;
 import org.apache.doris.system.Backend;
 import org.apache.doris.system.SystemInfoService;
 import org.apache.doris.thrift.TStorageMedium;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.TreeMultimap;
+
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -39,7 +41,7 @@ import java.util.Optional;
 import java.util.stream.Collectors;
 
 /*
- * Load statistics of a cluster
+ * Load statistics of a cluster with specified tag
  */
 public class ClusterLoadStatistic {
     private static final Logger LOG = LogManager.getLogger(ClusterLoadStatistic.class);
@@ -48,6 +50,7 @@ public class ClusterLoadStatistic {
     private TabletInvertedIndex invertedIndex;
 
     private String clusterName;
+    private Tag tag;
 
     private Map<TStorageMedium, Long> totalCapacityMap = Maps.newHashMap();
     private Map<TStorageMedium, Long> totalUsedCapacityMap = Maps.newHashMap();
@@ -61,18 +64,27 @@ public class ClusterLoadStatistic {
     private Map<TStorageMedium, TreeMultimap<Long, Long>> beByTotalReplicaCountMaps = Maps.newHashMap();
     private Map<TStorageMedium, TreeMultimap<Long, TabletInvertedIndex.PartitionBalanceInfo>> skewMaps = Maps.newHashMap();
 
-    public ClusterLoadStatistic(String clusterName, SystemInfoService infoService,
+    public ClusterLoadStatistic(String clusterName, Tag tag, SystemInfoService infoService,
                                 TabletInvertedIndex invertedIndex) {
         this.clusterName = clusterName;
+        this.tag = tag;
         this.infoService = infoService;
         this.invertedIndex = invertedIndex;
     }
 
+    public String getClusterName() {
+        return clusterName;
+    }
+
+    public Tag getTag() {
+        return tag;
+    }
+
     public void init() {
-        ImmutableMap<Long, Backend> backends = infoService.getBackendsInCluster(clusterName);
-        for (Backend backend : backends.values()) {
+        List<Backend> backends = infoService.getBackendsByTagInCluster(clusterName, tag);
+        for (Backend backend : backends) {
             BackendLoadStatistic beStatistic = new BackendLoadStatistic(backend.getId(),
-                    backend.getOwnerClusterName(), infoService, invertedIndex);
+                    backend.getOwnerClusterName(), backend.getTag(), infoService, invertedIndex);
             try {
                 beStatistic.init();
             } catch (LoadBalanceException e) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/ColocateTableCheckerAndBalancer.java b/fe/fe-core/src/main/java/org/apache/doris/clone/ColocateTableCheckerAndBalancer.java
index 00bc1a1..faf8828 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/clone/ColocateTableCheckerAndBalancer.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/clone/ColocateTableCheckerAndBalancer.java
@@ -26,22 +26,28 @@ import org.apache.doris.catalog.MaterializedIndex;
 import org.apache.doris.catalog.MaterializedIndex.IndexExtState;
 import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.catalog.Partition;
+import org.apache.doris.catalog.ReplicaAllocation;
 import org.apache.doris.catalog.Tablet;
 import org.apache.doris.catalog.Tablet.TabletStatus;
 import org.apache.doris.clone.TabletSchedCtx.Priority;
 import org.apache.doris.clone.TabletScheduler.AddResult;
 import org.apache.doris.common.Config;
+import org.apache.doris.common.DdlException;
+import org.apache.doris.common.FeConstants;
 import org.apache.doris.common.util.MasterDaemon;
 import org.apache.doris.persist.ColocatePersistInfo;
+import org.apache.doris.resource.Tag;
 import org.apache.doris.system.Backend;
 import org.apache.doris.system.SystemInfoService;
 
+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.Sets;
-
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
+import com.google.common.collect.Table;
 
 import java.util.List;
 import java.util.Map;
@@ -56,8 +62,6 @@ import java.util.stream.IntStream;
 public class ColocateTableCheckerAndBalancer extends MasterDaemon {
     private static final Logger LOG = LogManager.getLogger(ColocateTableCheckerAndBalancer.class);
 
-    private static final long CHECK_INTERVAL_MS = 20 * 1000L; // 20 second
-
     private ColocateTableCheckerAndBalancer(long intervalMs) {
         super("colocate group clone checker", intervalMs);
     }
@@ -67,9 +71,10 @@ public class ColocateTableCheckerAndBalancer extends MasterDaemon {
         if (INSTANCE == null) {
             synchronized (ColocateTableCheckerAndBalancer.class) {
                 if (INSTANCE == null) {
-                    INSTANCE = new ColocateTableCheckerAndBalancer(CHECK_INTERVAL_MS);
+                    INSTANCE = new ColocateTableCheckerAndBalancer(FeConstants.tablet_checker_interval_ms);
                 }
-            } }
+            }
+        }
         return INSTANCE;
     }
 
@@ -145,27 +150,48 @@ public class ColocateTableCheckerAndBalancer extends MasterDaemon {
                 continue;
             }
 
-            Map<String, ClusterLoadStatistic> statisticMap = catalog.getTabletScheduler().getStatisticMap();
+            Table<String, Tag, ClusterLoadStatistic> statisticMap = catalog.getTabletScheduler().getStatisticMap();
             if (statisticMap == null) {
                 continue;
             }
-            ClusterLoadStatistic statistic = statisticMap.get(db.getClusterName());
-            if (statistic == null) {
-                continue;
-            }
-            List<List<Long>> backendsPerBucketSeq = colocateIndex.getBackendsPerBucketSeq(groupId);
-            if (backendsPerBucketSeq.isEmpty()) {
-                continue;
+
+            ColocateGroupSchema groupSchema = colocateIndex.getGroupSchema(groupId);
+            ReplicaAllocation replicaAlloc = groupSchema.getReplicaAlloc();
+            try {
+                Catalog.getCurrentSystemInfo().checkReplicaAllocation(db.getClusterName(), replicaAlloc);
+            } catch (DdlException e) {
+                colocateIndex.setErrMsgForGroup(groupId, e.getMessage());
             }
+            Map<Tag, Short> allocMap = replicaAlloc.getAllocMap();
 
-            Set<Long> unavailableBeIdsInGroup = getUnavailableBeIdsInGroup(infoService, colocateIndex, groupId);
-            List<Long> availableBeIds = getAvailableBeIds(db.getClusterName(), infoService);
-            List<List<Long>> balancedBackendsPerBucketSeq = Lists.newArrayList();
-            if (relocateAndBalance(groupId, unavailableBeIdsInGroup, availableBeIds, colocateIndex, infoService, statistic, balancedBackendsPerBucketSeq)) {
-                colocateIndex.addBackendsPerBucketSeq(groupId, balancedBackendsPerBucketSeq);
-                ColocatePersistInfo info = ColocatePersistInfo.createForBackendsPerBucketSeq(groupId, balancedBackendsPerBucketSeq);
-                catalog.getEditLog().logColocateBackendsPerBucketSeq(info);
-                LOG.info("balance group {}. now backends per bucket sequence is: {}", groupId, balancedBackendsPerBucketSeq);
+            for (Map.Entry<Tag, Short> entry : allocMap.entrySet()) {
+                Tag tag = entry.getKey();
+                ClusterLoadStatistic statistic = statisticMap.get(db.getClusterName(), tag);
+                if (statistic == null) {
+                    continue;
+                }
+                List<List<Long>> backendsPerBucketSeq = colocateIndex.getBackendsPerBucketSeqByTag(groupId, tag);
+                if (backendsPerBucketSeq.isEmpty()) {
+                    continue;
+                }
+
+                // get all unavailable backends in the backend bucket sequence of this group
+                Set<Long> unavailableBeIdsInGroup = getUnavailableBeIdsInGroup(infoService, colocateIndex, groupId, tag);
+                // get all available backends for this group
+                Set<Long> beIdsInOtherTag = colocateIndex.getBackendIdsExceptForTag(groupId, tag);
+                List<Long> availableBeIds = getAvailableBeIds(db.getClusterName(), tag, beIdsInOtherTag, infoService);
+                // try relocate or balance this group for specified tag
+                List<List<Long>> balancedBackendsPerBucketSeq = Lists.newArrayList();
+                if (relocateAndBalance(groupId, tag, unavailableBeIdsInGroup, availableBeIds, colocateIndex,
+                        infoService, statistic, balancedBackendsPerBucketSeq)) {
+                    colocateIndex.addBackendsPerBucketSeqByTag(groupId, tag, balancedBackendsPerBucketSeq);
+                    Map<Tag, List<List<Long>>> balancedBackendsPerBucketSeqMap = Maps.newHashMap();
+                    balancedBackendsPerBucketSeqMap.put(tag, balancedBackendsPerBucketSeq);
+                    ColocatePersistInfo info = ColocatePersistInfo.createForBackendsPerBucketSeq(groupId, balancedBackendsPerBucketSeqMap);
+                    catalog.getEditLog().logColocateBackendsPerBucketSeq(info);
+                    LOG.info("balance group {}. now backends per bucket sequence for tag {} is: {}",
+                            groupId, tag, balancedBackendsPerBucketSeq);
+                }
             }
         }
     }
@@ -203,9 +229,9 @@ public class ColocateTableCheckerAndBalancer extends MasterDaemon {
                 olapTable.readLock();
                 try {
                     for (Partition partition : olapTable.getPartitions()) {
-                        short replicationNum = olapTable.getPartitionInfo().getReplicationNum(partition.getId());
+                        ReplicaAllocation replicaAlloc = olapTable.getPartitionInfo().getReplicaAllocation(partition.getId());
+                        short replicationNum = replicaAlloc.getTotalReplicaNum();
                         long visibleVersion = partition.getVisibleVersion();
-                        long visibleVersionHash = partition.getVisibleVersionHash();
                         // Here we only get VISIBLE indexes. All other indexes are not queryable.
                         // So it does not matter if tablets of other indexes are not matched.
                         for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.VISIBLE)) {
@@ -216,7 +242,7 @@ public class ColocateTableCheckerAndBalancer extends MasterDaemon {
                                 Set<Long> bucketsSeq = backendBucketsSeq.get(idx);
                                 Preconditions.checkState(bucketsSeq.size() == replicationNum, bucketsSeq.size() + " vs. " + replicationNum);
                                 Tablet tablet = index.getTablet(tabletId);
-                                TabletStatus st = tablet.getColocateHealthStatus(visibleVersion, replicationNum, bucketsSeq);
+                                TabletStatus st = tablet.getColocateHealthStatus(visibleVersion, replicaAlloc, bucketsSeq);
                                 if (st != TabletStatus.HEALTHY) {
                                     isGroupStable = false;
                                     LOG.debug("get unhealthy tablet {} in colocate table. status: {}", tablet.getId(), st);
@@ -224,6 +250,7 @@ public class ColocateTableCheckerAndBalancer extends MasterDaemon {
                                     TabletSchedCtx tabletCtx = new TabletSchedCtx(
                                             TabletSchedCtx.Type.REPAIR, db.getClusterName(),
                                             db.getId(), tableId, partition.getId(), index.getId(), tablet.getId(),
+                                            olapTable.getPartitionInfo().getReplicaAllocation(partition.getId()),
                                             System.currentTimeMillis());
                                     // the tablet status will be set again when being scheduled
                                     tabletCtx.setTabletStatus(st);
@@ -258,28 +285,40 @@ public class ColocateTableCheckerAndBalancer extends MasterDaemon {
     }
 
     /*
-     * The balance logic is as follow:
-     * 
+     * Each balance is performed for a single resource group in a colocate group.
+     * For example, if the replica allocation of a colocate group is {TagA: 2, TagB: 1},
+     * So the backend bucket seq may be like:
+     *
+     *       0  1  2  3
+     * TagA  A  B  C  A
+     * TagA  B  C  A  B
+     * TagB  D  D  D  D
+     *
+     * First, we will hanlde resource group of TagA, then TagB.
+     *
+     * For a single resource group, the balance logic is as follow
+     * (Suppose there is only one resource group with 3 replicas):
+     *
      * All backends: A,B,C,D,E,F,G,H,I,J
-     * 
+     *
      * One group's buckets sequence:
-     * 
+     *
      * Buckets sequence:    0  1  2  3
      * Backend set:         A  A  A  A
      *                      B  D  F  H
      *                      C  E  G  I
-     *           
+     *
      * Then each backend has different replica num:
-     * 
+     *
      * Backends:    A B C D E F G H I J
      * Replica num: 4 1 1 1 1 1 1 1 1 0
-     * 
+     *
      * The goal of balance is to evenly distribute replicas on all backends. For this example, we want the
      * following result (one possible result):
-     * 
+     *
      * Backends:    A B C D E F G H I J
      * Replica num: 2 2 1 1 1 1 1 1 1 1
-     * 
+     *
      * Algorithm:
      * 0. Generate the flat list of backends per bucket sequence:
      *      A B C A D E A F G A H I
@@ -302,19 +341,20 @@ public class ColocateTableCheckerAndBalancer extends MasterDaemon {
      *  Return true if backends per bucket sequence change and new sequence is saved in balancedBackendsPerBucketSeq.
      *  Return false if nothing changed.
      */
-    private boolean relocateAndBalance(GroupId groupId, Set<Long> unavailableBeIds, List<Long> availableBeIds,
+    private boolean relocateAndBalance(GroupId groupId, Tag tag, Set<Long> unavailableBeIds, List<Long> availableBeIds,
                                        ColocateTableIndex colocateIndex, SystemInfoService infoService,
                                        ClusterLoadStatistic statistic, List<List<Long>> balancedBackendsPerBucketSeq) {
         ColocateGroupSchema groupSchema = colocateIndex.getGroupSchema(groupId);
-        int replicationNum = groupSchema.getReplicationNum();
-        List<List<Long>> backendsPerBucketSeq = Lists.newArrayList(colocateIndex.getBackendsPerBucketSeq(groupId));
+        short replicaNum = groupSchema.getReplicaAlloc().getReplicaNumByTag(tag);
+        List<List<Long>> backendsPerBucketSeq = Lists.newArrayList(colocateIndex.getBackendsPerBucketSeqByTag(groupId, tag));
         // [[A,B,C],[B,C,D]] -> [A,B,C,B,C,D]
         List<Long> flatBackendsPerBucketSeq = backendsPerBucketSeq.stream().flatMap(List::stream).collect(Collectors.toList());
 
         boolean isChanged = false;
-        OUT: while (true) {
+        OUT:
+        while (true) {
             // update backends and hosts at each round
-            backendsPerBucketSeq = Lists.partition(flatBackendsPerBucketSeq, replicationNum);
+            backendsPerBucketSeq = Lists.partition(flatBackendsPerBucketSeq, replicaNum);
             List<List<String>> hostsPerBucketSeq = getHostsPerBucketSeq(backendsPerBucketSeq, infoService);
             if (hostsPerBucketSeq == null) {
                 // error happens, change nothing
@@ -328,7 +368,7 @@ public class ColocateTableCheckerAndBalancer extends MasterDaemon {
             // first choose the unavailable be as src be
             for (Long beId : unavailableBeIds) {
                 seqIndexes = getBeSeqIndexes(flatBackendsPerBucketSeq, beId);
-                if (seqIndexes.size() > 0) {
+                if (!seqIndexes.isEmpty()) {
                     srcBeId = beId;
                     hasUnavailableBe = true;
                     break;
@@ -343,7 +383,7 @@ public class ColocateTableCheckerAndBalancer extends MasterDaemon {
                 break;
             }
 
-            if (seqIndexes == null || seqIndexes.size() <= 0) {
+            if (seqIndexes == null || seqIndexes.isEmpty()) {
                 // choose max bucketId num be as src be
                 Preconditions.checkState(backendsPerBucketSeq.size() > 0);
                 srcBeId = backendWithReplicaNum.get(0).getKey();
@@ -375,7 +415,7 @@ public class ColocateTableCheckerAndBalancer extends MasterDaemon {
                 for (int seqIndex : seqIndexes) {
                     // the bucket index.
                     // eg: 0 / 3 = 0, so that the bucket index of the 4th backend id in flatBackendsPerBucketSeq is 0.
-                    int bucketIndex = seqIndex / replicationNum;
+                    int bucketIndex = seqIndex / replicaNum;
                     List<Long> backendsSet = backendsPerBucketSeq.get(bucketIndex);
                     List<String> hostsSet = hostsPerBucketSeq.get(bucketIndex);
                     // the replicas of a tablet can not locate in same Backend or same host
@@ -411,7 +451,7 @@ public class ColocateTableCheckerAndBalancer extends MasterDaemon {
         }
 
         if (isChanged) {
-            balancedBackendsPerBucketSeq.addAll(Lists.partition(flatBackendsPerBucketSeq, replicationNum));
+            balancedBackendsPerBucketSeq.addAll(Lists.partition(flatBackendsPerBucketSeq, replicaNum));
         }
         return isChanged;
     }
@@ -491,24 +531,25 @@ public class ColocateTableCheckerAndBalancer extends MasterDaemon {
                 idx -> flatBackendsPerBucketSeq.get(idx).equals(beId)).collect(Collectors.toList());
     }
 
-    private Set<Long> getUnavailableBeIdsInGroup(SystemInfoService infoService, ColocateTableIndex colocateIndex, GroupId groupId) {
-        Set<Long> backends = colocateIndex.getBackendsByGroup(groupId);
+    private Set<Long> getUnavailableBeIdsInGroup(SystemInfoService infoService, ColocateTableIndex colocateIndex,
+                                                 GroupId groupId, Tag tag) {
+        Set<Long> backends = colocateIndex.getBackendsByGroup(groupId, tag);
         Set<Long> unavailableBeIds = Sets.newHashSet();
         for (Long backendId : backends) {
-            if (!checkBackendAvailable(backendId, infoService)) {
+            if (!checkBackendAvailable(backendId, tag, Sets.newHashSet(), infoService)) {
                 unavailableBeIds.add(backendId);
             }
         }
         return unavailableBeIds;
     }
 
-    private List<Long> getAvailableBeIds(String cluster, SystemInfoService infoService) {
+    private List<Long> getAvailableBeIds(String cluster, Tag tag, Set<Long> excludedBeIds, SystemInfoService infoService) {
         // get all backends to allBackendIds, and check be availability using checkBackendAvailable
         // backend stopped for a short period of time is still considered available
         List<Long> allBackendIds = infoService.getClusterBackendIds(cluster, false);
         List<Long> availableBeIds = Lists.newArrayList();
         for (Long backendId : allBackendIds) {
-            if (checkBackendAvailable(backendId, infoService)) {
+            if (checkBackendAvailable(backendId, tag, excludedBeIds, infoService)) {
                 availableBeIds.add(backendId);
             }
         }
@@ -519,11 +560,13 @@ public class ColocateTableCheckerAndBalancer extends MasterDaemon {
      * check backend available
      * backend stopped for a short period of time is still considered available
      */
-    private boolean checkBackendAvailable(Long backendId, SystemInfoService infoService) {
+    private boolean checkBackendAvailable(Long backendId, Tag tag, Set<Long> excludedBeIds, SystemInfoService infoService) {
         long currTime = System.currentTimeMillis();
         Backend be = infoService.getBackend(backendId);
         if (be == null) {
             return false;
+        } else if (!be.getTag().equals(tag) || excludedBeIds.contains(be.getId())) {
+            return false;
         } else if (!be.isAvailable()) {
             // 1. BE is dead for a long time
             // 2. BE is under decommission
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 e75624f..62905a7 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
@@ -201,11 +201,13 @@ public class DynamicPartitionScheduler extends MasterDaemon {
 
             // construct partition desc
             PartitionKeyDesc partitionKeyDesc = PartitionKeyDesc.createFixed(Collections.singletonList(lowerValue), Collections.singletonList(upperValue));
-            HashMap<String, String> partitionProperties = Maps.newHashMap();
-            if (dynamicPartitionProperty.getReplicationNum() == DynamicPartitionProperty.NOT_SET_REPLICATION_NUM) {
-                partitionProperties.put("replication_num", String.valueOf(olapTable.getDefaultReplicationNum()));
+            HashMap<String, String> partitionProperties = new HashMap<>(1);
+            if (dynamicPartitionProperty.getReplicaAllocation().isNotSet()) {
+                partitionProperties.put(PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION,
+                        olapTable.getDefaultReplicaAllocation().toCreateStmt());
             } else {
-                partitionProperties.put("replication_num", String.valueOf(dynamicPartitionProperty.getReplicationNum()));
+                partitionProperties.put(PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION,
+                        dynamicPartitionProperty.getReplicaAllocation().toCreateStmt());
             }
 
             if (hotPartitionNum > 0) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/MovesCacheMap.java b/fe/fe-core/src/main/java/org/apache/doris/clone/MovesCacheMap.java
index ff8a2d9..ad4c8d8 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/clone/MovesCacheMap.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/clone/MovesCacheMap.java
@@ -17,11 +17,16 @@
 
 package org.apache.doris.clone;
 
+import org.apache.doris.common.Pair;
+import org.apache.doris.resource.Tag;
+import org.apache.doris.thrift.TStorageMedium;
+
 import com.google.common.cache.Cache;
 import com.google.common.cache.CacheBuilder;
+import com.google.common.collect.HashBasedTable;
 import com.google.common.collect.Maps;
-import org.apache.doris.common.Pair;
-import org.apache.doris.thrift.TStorageMedium;
+import com.google.common.collect.Table;
+
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -41,8 +46,8 @@ import java.util.stream.Collectors;
 public class MovesCacheMap {
     private static final Logger LOG = LogManager.getLogger(MovesCacheMap.class);
 
-    // cluster -> medium -> MovesCache
-    private final Map<String, Map<TStorageMedium, MovesCache>> cacheMap = Maps.newHashMap();
+    // cluster -> tag -> medium -> MovesCache
+    private final Table<String, Tag, Map<TStorageMedium, MovesCache>> cacheMap = HashBasedTable.create();
     private long lastExpireConfig = -1L;
 
     // TabletId -> Pair<Move, ToDeleteReplicaId>, 'ToDeleteReplicaId == -1' means this move haven't been scheduled successfully.
@@ -59,31 +64,51 @@ public class MovesCacheMap {
     }
 
     // Cyclical update the cache mapping, cuz the cluster may be deleted, we should delete the corresponding cache too.
-    public void updateMapping(Map<String, ClusterLoadStatistic> statisticMap, long expireAfterAccessSecond) {
+    public void updateMapping(Table<String, Tag, ClusterLoadStatistic> statisticMap, long expireAfterAccessSecond) {
         if (expireAfterAccessSecond > 0 && lastExpireConfig != expireAfterAccessSecond) {
-            LOG.debug("Reset expireAfterAccess, last {}s, now {}s. Moves will be cleared.", lastExpireConfig, expireAfterAccessSecond);
+            LOG.debug("Reset expireAfterAccess, last {} s, now {} s. Moves will be cleared.", lastExpireConfig, expireAfterAccessSecond);
             cacheMap.clear();
             lastExpireConfig = expireAfterAccessSecond;
         }
 
-        cacheMap.keySet().stream().filter(k -> !statisticMap.containsKey(k)).forEach(cacheMap::remove);
+        cacheMap.cellSet().stream().filter(c -> !statisticMap.contains(c.getRowKey(), c.getColumnKey())).forEach(
+                c -> cacheMap.remove(c.getRowKey(), c.getColumnKey()));
 
-        List<String> toAdd = statisticMap.keySet().stream().filter(k -> !cacheMap.containsKey(k)).collect(Collectors.toList());
-        for (String cluster : toAdd) {
-            Map<TStorageMedium, MovesCache> cacheMap = Maps.newHashMap();
-            Arrays.stream(TStorageMedium.values()).forEach(m -> cacheMap.put(m, new MovesCache(expireAfterAccessSecond, TimeUnit.SECONDS)));
-            this.cacheMap.put(cluster, cacheMap);
+        List<Table.Cell<String, Tag, ClusterLoadStatistic>> toAdd = statisticMap.cellSet().stream()
+                .filter(c -> !cacheMap.contains(c.getRowKey(), c.getColumnKey()))
+                .collect(Collectors.toList());
+        for (Table.Cell<String, Tag, ClusterLoadStatistic> cell : toAdd) {
+            Map<TStorageMedium, MovesCache> newCacheMap = Maps.newHashMap();
+            Arrays.stream(TStorageMedium.values()).forEach(m -> newCacheMap.put(m, new MovesCache(expireAfterAccessSecond, TimeUnit.SECONDS)));
+            this.cacheMap.put(cell.getRowKey(), cell.getColumnKey(), newCacheMap);
         }
     }
 
-    public MovesCache getCache(String clusterName, TStorageMedium medium) {
-        Map<TStorageMedium, MovesCache> clusterMoves = cacheMap.get(clusterName);
+    public MovesCache getCache(String clusterName, Tag tag, TStorageMedium medium) {
+        Map<TStorageMedium, MovesCache> clusterMoves = cacheMap.get(clusterName, tag);
         if (clusterMoves != null) {
             return clusterMoves.get(medium);
         }
         return null;
     }
 
+    // For given tablet ctx, find it in cacheMap
+    public Pair<PartitionRebalancer.TabletMove, Long> getTabletMove(TabletSchedCtx tabletCtx) {
+        Map<Tag, Map<TStorageMedium, MovesCache>> tagMap = cacheMap.row(tabletCtx.getCluster());
+        if (tagMap == null) {
+            return null;
+        }
+        for (Map<TStorageMedium, MovesCache> mediumMap : tagMap.values()) {
+            MovesCache cache = mediumMap.get(tabletCtx.getStorageMedium());
+            if (cache == null) {
+                continue;
+            }
+            return cache.get().getIfPresent(tabletCtx.getTabletId());
+        }
+        return null;
+    }
+
+
     // For each MovesCache, performs any pending maintenance operations needed by the cache.
     public void maintain() {
         cacheMap.values().forEach(maps -> maps.values().forEach(map -> map.get().cleanUp()));
@@ -96,7 +121,8 @@ public class MovesCacheMap {
     @Override
     public String toString() {
         StringJoiner sj = new StringJoiner("\n", "MovesInProgress detail:\n", "");
-        cacheMap.forEach((key, value) -> value.forEach((k, v) -> sj.add("(" + key + "-" + k + ": " + v.get().asMap() + ")")));
+        cacheMap.cellSet().forEach(c -> c.getValue().forEach((k, v)
+                -> sj.add("(" + c.getRowKey() + "-" + c.getColumnKey() + "-" + k + ": " + v.get().asMap() + ")")));
         return sj.toString();
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/PartitionRebalancer.java b/fe/fe-core/src/main/java/org/apache/doris/clone/PartitionRebalancer.java
index d4cd812..d4678b7 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/clone/PartitionRebalancer.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/clone/PartitionRebalancer.java
@@ -17,18 +17,22 @@
 
 package org.apache.doris.clone;
 
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Ordering;
-import com.google.common.collect.TreeMultimap;
 import org.apache.doris.catalog.Replica;
 import org.apache.doris.catalog.TabletInvertedIndex;
 import org.apache.doris.catalog.TabletMeta;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.Pair;
+import org.apache.doris.resource.Tag;
 import org.apache.doris.system.SystemInfoService;
 import org.apache.doris.thrift.TStorageMedium;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Ordering;
+import com.google.common.collect.Table;
+import com.google.common.collect.TreeMultimap;
+
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -67,8 +71,9 @@ public class PartitionRebalancer extends Rebalancer {
 
     @Override
     protected List<TabletSchedCtx> selectAlternativeTabletsForCluster(
-            String clusterName, ClusterLoadStatistic clusterStat, TStorageMedium medium) {
-        MovesCacheMap.MovesCache movesInProgress = movesCacheMap.getCache(clusterName, medium);
+            ClusterLoadStatistic clusterStat, TStorageMedium medium) {
+        String clusterName = clusterStat.getClusterName();
+        MovesCacheMap.MovesCache movesInProgress = movesCacheMap.getCache(clusterName, clusterStat.getTag(), medium);
         Preconditions.checkNotNull(movesInProgress, "clusterStat is got from statisticMap, movesCacheMap should have the same entry");
 
         // Iterating through Cache.asMap().values() does not reset access time for the entries you retrieve.
@@ -143,7 +148,9 @@ public class PartitionRebalancer extends Rebalancer {
             TabletMeta tabletMeta = tabletCandidates.get(pickedTabletId);
             TabletSchedCtx tabletCtx = new TabletSchedCtx(TabletSchedCtx.Type.BALANCE, clusterName,
                     tabletMeta.getDbId(), tabletMeta.getTableId(), tabletMeta.getPartitionId(),
-                    tabletMeta.getIndexId(), pickedTabletId, System.currentTimeMillis());
+                    tabletMeta.getIndexId(), pickedTabletId, null /* replica alloc is not used for balance*/,
+                    System.currentTimeMillis());
+            tabletCtx.setTag(clusterStat.getTag());
             // Balance task's priority is always LOW
             tabletCtx.setOrigPriority(TabletSchedCtx.Priority.LOW);
             alternativeTablets.add(tabletCtx);
@@ -218,7 +225,7 @@ public class PartitionRebalancer extends Rebalancer {
     @Override
     protected void completeSchedCtx(TabletSchedCtx tabletCtx, Map<Long, TabletScheduler.PathSlot> backendsWorkingSlots)
             throws SchedException {
-        MovesCacheMap.MovesCache movesInProgress = movesCacheMap.getCache(tabletCtx.getCluster(), tabletCtx.getStorageMedium());
+        MovesCacheMap.MovesCache movesInProgress = movesCacheMap.getCache(tabletCtx.getCluster(), tabletCtx.getTag(), tabletCtx.getStorageMedium());
         Preconditions.checkNotNull(movesInProgress, "clusterStat is got from statisticMap, movesInProgressMap should have the same entry");
 
         try {
@@ -240,7 +247,7 @@ public class PartitionRebalancer extends Rebalancer {
             }
 
             // Choose a path in destination
-            ClusterLoadStatistic clusterStat = statisticMap.get(tabletCtx.getCluster());
+            ClusterLoadStatistic clusterStat = statisticMap.get(tabletCtx.getCluster(), tabletCtx.getTag());
             Preconditions.checkNotNull(clusterStat, "cluster does not exist: " + tabletCtx.getCluster());
             BackendLoadStatistic beStat = clusterStat.getBackendLoadStatistic(move.toBe);
             Preconditions.checkNotNull(beStat);
@@ -280,11 +287,9 @@ public class PartitionRebalancer extends Rebalancer {
 
     @Override
     public Long getToDeleteReplicaId(TabletSchedCtx tabletCtx) {
-        MovesCacheMap.MovesCache movesInProgress = movesCacheMap.getCache(tabletCtx.getCluster(), tabletCtx.getStorageMedium());
-
         // We don't invalidate the cached move here, cuz the redundant repair progress is just started.
         // The move should be invalidated by TTL or Algo.CheckMoveCompleted()
-        Pair<TabletMove, Long> pair = movesInProgress.get().getIfPresent(tabletCtx.getTabletId());
+        Pair<TabletMove, Long> pair = movesCacheMap.getTabletMove(tabletCtx);
         if (pair != null) {
             Preconditions.checkState(pair.second != -1L);
             return pair.second;
@@ -294,7 +299,7 @@ public class PartitionRebalancer extends Rebalancer {
     }
 
     @Override
-    public void updateLoadStatistic(Map<String, ClusterLoadStatistic> statisticMap) {
+    public void updateLoadStatistic(Table<String, Tag, ClusterLoadStatistic> statisticMap) {
         super.updateLoadStatistic(statisticMap);
         movesCacheMap.updateMapping(statisticMap, Config.partition_rebalance_move_expire_after_access);
         // Perform cache maintenance
diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/Rebalancer.java b/fe/fe-core/src/main/java/org/apache/doris/clone/Rebalancer.java
index f854ef0..1fca406 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/clone/Rebalancer.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/clone/Rebalancer.java
@@ -19,13 +19,15 @@ package org.apache.doris.clone;
 
 import org.apache.doris.catalog.TabletInvertedIndex;
 import org.apache.doris.clone.TabletScheduler.PathSlot;
+import org.apache.doris.resource.Tag;
 import org.apache.doris.system.SystemInfoService;
 import org.apache.doris.task.AgentBatchTask;
 import org.apache.doris.thrift.TStorageMedium;
 
+import com.google.common.collect.HashBasedTable;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Table;
 
-import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
@@ -45,7 +47,7 @@ import java.util.Map;
 public abstract class Rebalancer {
     // When Rebalancer init, the statisticMap is usually empty. So it's no need to be an arg.
     // Only use updateLoadStatistic() to load stats.
-    protected Map<String, ClusterLoadStatistic> statisticMap = new HashMap<>();
+    protected Table<String, Tag, ClusterLoadStatistic> statisticMap = HashBasedTable.create();
     protected TabletInvertedIndex invertedIndex;
     protected SystemInfoService infoService;
 
@@ -56,10 +58,9 @@ public abstract class Rebalancer {
 
     public List<TabletSchedCtx> selectAlternativeTablets() {
         List<TabletSchedCtx> alternativeTablets = Lists.newArrayList();
-        for (Map.Entry<String, ClusterLoadStatistic> entry : statisticMap.entrySet()) {
+        for (Table.Cell<String, Tag, ClusterLoadStatistic> entry : statisticMap.cellSet()) {
             for (TStorageMedium medium : TStorageMedium.values()) {
-                alternativeTablets.addAll(selectAlternativeTabletsForCluster(entry.getKey(),
-                        entry.getValue(), medium));
+                alternativeTablets.addAll(selectAlternativeTabletsForCluster(entry.getValue(), medium));
             }
         }
         return alternativeTablets;
@@ -68,8 +69,7 @@ public abstract class Rebalancer {
     // The returned TabletSchedCtx should have the tablet id at least. {srcReplica, destBe} can be complete here or
     // later(when createBalanceTask called).
     protected abstract List<TabletSchedCtx> selectAlternativeTabletsForCluster(
-            String clusterName, ClusterLoadStatistic clusterStat, TStorageMedium medium);
-
+            ClusterLoadStatistic clusterStat, TStorageMedium medium);
 
     public void createBalanceTask(TabletSchedCtx tabletCtx, Map<Long, PathSlot> backendsWorkingSlots,
                                   AgentBatchTask batchTask) throws SchedException {
@@ -90,7 +90,7 @@ public abstract class Rebalancer {
         return -1L;
     }
 
-    public void updateLoadStatistic(Map<String, ClusterLoadStatistic> statisticMap) {
+    public void updateLoadStatistic(Table<String, Tag, ClusterLoadStatistic> statisticMap) {
         this.statisticMap = statisticMap;
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletChecker.java b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletChecker.java
index f6bd08d..10631b1 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletChecker.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletChecker.java
@@ -32,6 +32,7 @@ import org.apache.doris.catalog.Tablet.TabletStatus;
 import org.apache.doris.clone.TabletScheduler.AddResult;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.DdlException;
+import org.apache.doris.common.FeConstants;
 import org.apache.doris.common.Pair;
 import org.apache.doris.common.util.MasterDaemon;
 import org.apache.doris.metric.GaugeMetric;
@@ -64,8 +65,6 @@ import java.util.stream.Collectors;
 public class TabletChecker extends MasterDaemon {
     private static final Logger LOG = LogManager.getLogger(TabletChecker.class);
 
-    private static final long CHECK_INTERVAL_MS = 20 * 1000L; // 20 second
-
     private Catalog catalog;
     private SystemInfoService infoService;
     private TabletScheduler tabletScheduler;
@@ -127,7 +126,7 @@ public class TabletChecker extends MasterDaemon {
 
     public TabletChecker(Catalog catalog, SystemInfoService infoService, TabletScheduler tabletScheduler,
             TabletSchedulerStat stat) {
-        super("tablet checker", CHECK_INTERVAL_MS);
+        super("tablet checker", FeConstants.tablet_checker_interval_ms);
         this.catalog = catalog;
         this.infoService = infoService;
         this.tabletScheduler = tabletScheduler;
@@ -358,7 +357,7 @@ public class TabletChecker extends MasterDaemon {
                         db.getClusterName(),
                         partition.getVisibleVersion(),
                         partition.getVisibleVersionHash(),
-                        tbl.getPartitionInfo().getReplicationNum(partition.getId()),
+                        tbl.getPartitionInfo().getReplicaAllocation(partition.getId()),
                         aliveBeIdsInCluster);
 
                 if (statusWithPrio.first == TabletStatus.HEALTHY) {
@@ -387,6 +386,7 @@ public class TabletChecker extends MasterDaemon {
                         db.getClusterName(),
                         db.getId(), tbl.getId(),
                         partition.getId(), idx.getId(), tablet.getId(),
+                        tbl.getPartitionInfo().getReplicaAllocation(partition.getId()),
                         System.currentTimeMillis());
                 // the tablet status will be set again when being scheduled
                 tabletCtx.setTabletStatus(statusWithPrio.first);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletSchedCtx.java b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletSchedCtx.java
index d1ffbc1..cbd31ab 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletSchedCtx.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletSchedCtx.java
@@ -24,6 +24,7 @@ import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.catalog.Partition;
 import org.apache.doris.catalog.Replica;
 import org.apache.doris.catalog.Replica.ReplicaState;
+import org.apache.doris.catalog.ReplicaAllocation;
 import org.apache.doris.catalog.Table;
 import org.apache.doris.catalog.Tablet;
 import org.apache.doris.catalog.Tablet.TabletStatus;
@@ -34,6 +35,7 @@ import org.apache.doris.common.FeConstants;
 import org.apache.doris.common.Pair;
 import org.apache.doris.common.util.TimeUtils;
 import org.apache.doris.persist.ReplicaPersistInfo;
+import org.apache.doris.resource.Tag;
 import org.apache.doris.system.Backend;
 import org.apache.doris.system.SystemInfoService;
 import org.apache.doris.task.AgentTaskQueue;
@@ -45,13 +47,13 @@ import org.apache.doris.thrift.TStorageMedium;
 import org.apache.doris.thrift.TTabletInfo;
 import org.apache.doris.thrift.TTaskType;
 
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 import com.google.common.collect.Lists;
 
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -202,9 +204,14 @@ public class TabletSchedCtx implements Comparable<TabletSchedCtx> {
     private int tabletOrderIdx = -1;
 
     private SystemInfoService infoService;
-    
+
+    // replicaAlloc is only set for REPAIR task
+    private ReplicaAllocation replicaAlloc;
+    // tag is only set for BALANCE task, used to identify which resource group this Balance job is in
+    private Tag tag;
+
     public TabletSchedCtx(Type type, String cluster, long dbId, long tblId, long partId,
-            long idxId, long tabletId, long createTime) {
+                          long idxId, long tabletId, ReplicaAllocation replicaAlloc, long createTime) {
         this.type = type;
         this.cluster = cluster;
         this.dbId = dbId;
@@ -215,8 +222,21 @@ public class TabletSchedCtx implements Comparable<TabletSchedCtx> {
         this.createTime = createTime;
         this.infoService = Catalog.getCurrentSystemInfo();
         this.state = State.PENDING;
+        this.replicaAlloc = replicaAlloc;
     }
-    
+
+    public ReplicaAllocation getReplicaAlloc() {
+        return replicaAlloc;
+    }
+
+    public void setTag(Tag tag) {
+        this.tag = tag;
+    }
+
+    public Tag getTag() {
+        return tag;
+    }
+
     public void setType(Type type) {
         this.type = type;
     }
@@ -720,7 +740,7 @@ public class TabletSchedCtx implements Comparable<TabletSchedCtx> {
         // we create a new replica with state CLONE
         if (tabletStatus == TabletStatus.REPLICA_MISSING || tabletStatus == TabletStatus.REPLICA_MISSING_IN_CLUSTER
                 || tabletStatus == TabletStatus.REPLICA_RELOCATING || type == Type.BALANCE
-                || tabletStatus == TabletStatus.COLOCATE_MISMATCH) {
+                || tabletStatus == TabletStatus.COLOCATE_MISMATCH || tabletStatus == TabletStatus.REPLICA_MISSING_FOR_TAG) {
             Replica cloneReplica = new Replica(
                     Catalog.getCurrentCatalog().getNextId(), destBackendId,
                     -1 /* version */, 0 /* version hash */, schemaHash,
@@ -822,9 +842,9 @@ public class TabletSchedCtx implements Comparable<TabletSchedCtx> {
             }
 
             List<Long> aliveBeIdsInCluster = infoService.getClusterBackendIds(db.getClusterName(), true);
-            short replicationNum = olapTable.getPartitionInfo().getReplicationNum(partitionId);
+            ReplicaAllocation replicaAlloc = olapTable.getPartitionInfo().getReplicaAllocation(partitionId);
             Pair<TabletStatus, TabletSchedCtx.Priority> pair = tablet.getHealthStatusWithPriority(
-                    infoService, db.getClusterName(), visibleVersion, visibleVersionHash, replicationNum,
+                    infoService, db.getClusterName(), visibleVersion, visibleVersionHash, replicaAlloc,
                     aliveBeIdsInCluster);
             if (pair.first == TabletStatus.HEALTHY) {
                 throw new SchedException(Status.FINISHED, "tablet is healthy");
diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletScheduler.java
index 56aee10..2a85868 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletScheduler.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletScheduler.java
@@ -38,9 +38,11 @@ import org.apache.doris.clone.TabletSchedCtx.Priority;
 import org.apache.doris.clone.TabletSchedCtx.Type;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.Config;
+import org.apache.doris.common.FeConstants;
 import org.apache.doris.common.Pair;
 import org.apache.doris.common.util.MasterDaemon;
 import org.apache.doris.persist.ReplicaPersistInfo;
+import org.apache.doris.resource.Tag;
 import org.apache.doris.system.Backend;
 import org.apache.doris.system.SystemInfoService;
 import org.apache.doris.task.AgentBatchTask;
@@ -51,15 +53,17 @@ import org.apache.doris.task.CloneTask;
 import org.apache.doris.task.DropReplicaTask;
 import org.apache.doris.thrift.TFinishTaskRequest;
 
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
 import com.google.common.base.Preconditions;
 import com.google.common.collect.EvictingQueue;
+import com.google.common.collect.HashBasedTable;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
-
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
+import com.google.common.collect.Table;
 
 import java.util.Collection;
 import java.util.List;
@@ -117,8 +121,8 @@ public class TabletScheduler extends MasterDaemon {
 
     // be id -> #working slots
     private Map<Long, PathSlot> backendsWorkingSlots = Maps.newConcurrentMap();
-    // cluster name -> load statistic
-    private Map<String, ClusterLoadStatistic> statisticMap = Maps.newConcurrentMap();
+    // cluster name -> Tag -> load statistic
+    private Table<String, Tag, ClusterLoadStatistic> statisticMap = HashBasedTable.create();
     private long lastStatUpdateTime = 0;
 
     private long lastSlotAdjustTime = 0;
@@ -303,20 +307,23 @@ public class TabletScheduler extends MasterDaemon {
      * because we already limit the total number of running clone jobs in cluster by 'backend slots'
      */
     private void updateClusterLoadStatistic() {
-        Map<String, ClusterLoadStatistic> newStatisticMap = Maps.newConcurrentMap();
+        Table<String, Tag, ClusterLoadStatistic> newStatisticMap = HashBasedTable.create();
         Set<String> clusterNames = infoService.getClusterNames();
         for (String clusterName : clusterNames) {
-            ClusterLoadStatistic clusterLoadStatistic = new ClusterLoadStatistic(clusterName,
-                    infoService, invertedIndex);
-            clusterLoadStatistic.init();
-            newStatisticMap.put(clusterName, clusterLoadStatistic);
-            LOG.info("update cluster {} load statistic:\n{}", clusterName, clusterLoadStatistic.getBrief());
+            Set<Tag> tags = infoService.getTagsByCluster(clusterName);
+            for (Tag tag : tags) {
+                ClusterLoadStatistic clusterLoadStatistic = new ClusterLoadStatistic(clusterName, tag,
+                        infoService, invertedIndex);
+                clusterLoadStatistic.init();
+                newStatisticMap.put(clusterName, tag, clusterLoadStatistic);
+                LOG.info("update cluster {} load statistic:\n{}", clusterName, clusterLoadStatistic.getBrief());
+            }
         }
 
         this.statisticMap = newStatisticMap;
     }
 
-    public Map<String, ClusterLoadStatistic> getStatisticMap() {
+    public Table<String, Tag, ClusterLoadStatistic> getStatisticMap() {
         return statisticMap;
     }
 
@@ -489,7 +496,7 @@ public class TabletScheduler extends MasterDaemon {
                 Set<Long> backendsSet = colocateTableIndex.getTabletBackendsByGroup(groupId, tabletOrderIdx);
                 TabletStatus st = tablet.getColocateHealthStatus(
                         partition.getVisibleVersion(),
-                        tbl.getPartitionInfo().getReplicationNum(partition.getId()),
+                        tbl.getPartitionInfo().getReplicaAllocation(partition.getId()),
                         backendsSet);
                 statusPair = Pair.create(st, Priority.HIGH);
                 tabletCtx.setColocateGroupBackendIds(backendsSet);
@@ -499,7 +506,7 @@ public class TabletScheduler extends MasterDaemon {
                         infoService, tabletCtx.getCluster(),
                         partition.getVisibleVersion(),
                         partition.getVisibleVersionHash(),
-                        tbl.getPartitionInfo().getReplicationNum(partition.getId()),
+                        tbl.getPartitionInfo().getReplicaAllocation(partition.getId()),
                         aliveBeIdsInCluster);
             }
 
@@ -526,12 +533,8 @@ public class TabletScheduler extends MasterDaemon {
             } else if (statusPair.first != TabletStatus.HEALTHY
                     && tabletCtx.getType() == TabletSchedCtx.Type.BALANCE) {
                 // we select an unhealthy tablet to do balance, which is not right.
-                // so here we change it to a REPAIR task, and also reset its priority
-                tabletCtx.releaseResource(this);
-                tabletCtx.setType(TabletSchedCtx.Type.REPAIR);
-                tabletCtx.setOrigPriority(statusPair.second);
-                tabletCtx.setLastSchedTime(currentTime);
-                tabletCtx.setLastVisitedTime(currentTime);
+                // so here we stop this task.
+                throw new SchedException(Status.UNRECOVERABLE, "tablet is unhealthy when doing balance");
             }
 
             // we do not concern priority here.
@@ -552,25 +555,28 @@ public class TabletScheduler extends MasterDaemon {
             throws SchedException {
         if (tabletCtx.getType() == Type.REPAIR) {
             switch (status) {
-            case REPLICA_MISSING:
-                handleReplicaMissing(tabletCtx, batchTask);
-                break;
-            case VERSION_INCOMPLETE:
-            case NEED_FURTHER_REPAIR: // same as version incomplete, it prefer to the dest replica which need further repair
-                handleReplicaVersionIncomplete(tabletCtx, batchTask);
-                break;
-            case REPLICA_RELOCATING:
-                handleReplicaRelocating(tabletCtx, batchTask);
-                break;
-            case REDUNDANT:
-                handleRedundantReplica(tabletCtx, false);
-                break;
-            case FORCE_REDUNDANT:
-                handleRedundantReplica(tabletCtx, true);
-                break;
+                case REPLICA_MISSING:
+                    handleReplicaMissing(tabletCtx, batchTask);
+                    break;
+                case VERSION_INCOMPLETE:
+                case NEED_FURTHER_REPAIR: // same as version incomplete, it prefer to the dest replica which need further repair
+                    handleReplicaVersionIncomplete(tabletCtx, batchTask);
+                    break;
+                case REPLICA_RELOCATING:
+                    handleReplicaRelocating(tabletCtx, batchTask);
+                    break;
+                case REDUNDANT:
+                    handleRedundantReplica(tabletCtx, false);
+                    break;
+                case FORCE_REDUNDANT:
+                    handleRedundantReplica(tabletCtx, true);
+                    break;
                 case REPLICA_MISSING_IN_CLUSTER:
                     handleReplicaClusterMigration(tabletCtx, batchTask);
                     break;
+                case REPLICA_MISSING_FOR_TAG:
+                    handleReplicaMissingForTag(tabletCtx, batchTask);
+                    break;
                 case COLOCATE_MISMATCH:
                     handleColocateMismatch(tabletCtx, batchTask);
                     break;
@@ -594,9 +600,10 @@ public class TabletScheduler extends MasterDaemon {
      * 1. find an available path in a backend as destination:
      *      1. backend need to be alive.
      *      2. backend of existing replicas should be excluded. (should not be on same host either)
-     *      3. backend has available slot for clone.
-     *      4. replica can fit in the path (consider the threshold of disk capacity and usage percent).
-     *      5. try to find a path with lowest load score.
+     *      3. backend with proper tag.
+     *      4. backend has available slot for clone.
+     *      5. replica can fit in the path (consider the threshold of disk capacity and usage percent).
+     *      6. try to find a path with lowest load score.
      * 2. find an appropriate source replica:
      *      1. source replica should be healthy
      *      2. backend of source replica has available slot for clone.
@@ -605,8 +612,10 @@ public class TabletScheduler extends MasterDaemon {
      */
     private void handleReplicaMissing(TabletSchedCtx tabletCtx, AgentBatchTask batchTask) throws SchedException {
         stat.counterReplicaMissingErr.incrementAndGet();
+        // find proper tag
+        Tag tag = chooseProperTag(tabletCtx, true);
         // find an available dest backend and path
-        RootPathLoadStatistic destPath = chooseAvailableDestPath(tabletCtx, false /* not for colocate */);
+        RootPathLoadStatistic destPath = chooseAvailableDestPath(tabletCtx, tag, false /* not for colocate */);
         Preconditions.checkNotNull(destPath);
         tabletCtx.setDest(destPath.getBeId(), destPath.getPathHash());
 
@@ -617,6 +626,36 @@ public class TabletScheduler extends MasterDaemon {
         batchTask.addTask(tabletCtx.createCloneReplicaAndTask());
     }
 
+    // In dealing with the case of missing replicas, we need to select a tag with missing replicas
+    // according to the distribution of replicas.
+    // If no replica of the tag is missing, an exception is thrown.
+    // And for deleting redundant replica, also find out a tag which has redundant replica.
+    private Tag chooseProperTag(TabletSchedCtx tabletCtx, boolean forMissingReplica) throws SchedException {
+        Tablet tablet = tabletCtx.getTablet();
+        List<Replica> replicas = tablet.getReplicas();
+        Map<Tag, Short> allocMap = tabletCtx.getReplicaAlloc().getAllocMap();
+        Map<Tag, Short> currentAllocMap = Maps.newHashMap();
+        for (Replica replica : replicas) {
+            Backend be = infoService.getBackend(replica.getBackendId());
+            if (be != null) {
+                Short num = currentAllocMap.getOrDefault(be.getTag(), (short) 0);
+                currentAllocMap.put(be.getTag(), (short) (num + 1));
+            }
+        }
+
+        for (Map.Entry<Tag, Short> entry : allocMap.entrySet()) {
+            short curNum = currentAllocMap.getOrDefault(entry.getKey(), (short) 0);
+            if (forMissingReplica && curNum < entry.getValue()) {
+                return entry.getKey();
+            }
+            if (!forMissingReplica && curNum > entry.getValue()) {
+                return entry.getKey();
+            }
+        }
+
+        throw new SchedException(Status.UNRECOVERABLE, "no proper tag is chose for tablet " + tablet.getId());
+    }
+
     /**
      * Replica version is incomplete, which means this replica is missing some version,
      * and need to be cloned from a healthy replica, in-place.
@@ -628,10 +667,6 @@ public class TabletScheduler extends MasterDaemon {
     private void handleReplicaVersionIncomplete(TabletSchedCtx tabletCtx, AgentBatchTask batchTask)
             throws SchedException {
         stat.counterReplicaVersionMissingErr.incrementAndGet();
-        ClusterLoadStatistic statistic = statisticMap.get(tabletCtx.getCluster());
-        if (statistic == null) {
-            throw new SchedException(Status.UNRECOVERABLE, "cluster does not exist");
-        }
 
         try {
             tabletCtx.chooseDestReplicaForVersionIncomplete(backendsWorkingSlots);
@@ -799,11 +834,6 @@ public class TabletScheduler extends MasterDaemon {
     }
 
     private boolean deleteReplicaOnSameHost(TabletSchedCtx tabletCtx, boolean force) throws SchedException {
-        ClusterLoadStatistic statistic = statisticMap.get(tabletCtx.getCluster());
-        if (statistic == null) {
-            return false;
-        }
-
         // collect replicas of this tablet.
         // host -> (replicas on same host)
         Map<String, List<Replica>> hostToReplicas = Maps.newHashMap();
@@ -826,6 +856,11 @@ public class TabletScheduler extends MasterDaemon {
             if (replicas.size() > 1) {
                 // delete one replica from replicas on same host.
                 // better to choose high load backend
+                Tag tag = chooseProperTag(tabletCtx, false);
+                ClusterLoadStatistic statistic = statisticMap.get(tabletCtx.getCluster(), tag);
+                if (statistic == null) {
+                    return false;
+                }
                 return deleteFromHighLoadBackend(tabletCtx, replicas, force, statistic);
             }
         }
@@ -862,7 +897,8 @@ public class TabletScheduler extends MasterDaemon {
     }
 
     private boolean deleteReplicaOnHighLoadBackend(TabletSchedCtx tabletCtx, boolean force) throws SchedException {
-        ClusterLoadStatistic statistic = statisticMap.get(tabletCtx.getCluster());
+        Tag tag = chooseProperTag(tabletCtx, false);
+        ClusterLoadStatistic statistic = statisticMap.get(tabletCtx.getCluster(), tag);
         if (statistic == null) {
             return false;
         }
@@ -934,7 +970,7 @@ public class TabletScheduler extends MasterDaemon {
          * 2. Wait for any txns before the watermark txn id to be finished. If all are finished, which means this replica is
          *      safe to be deleted.
          */
-        if (!force && replica.getState().canLoad() && replica.getWatermarkTxnId() == -1) {
+        if (!force && replica.getState().canLoad() && replica.getWatermarkTxnId() == -1 && !FeConstants.runningUnitTest) {
             long nextTxnId = Catalog.getCurrentGlobalTransactionMgr().getTransactionIDGenerator().getNextTransactionId();
             replica.setWatermarkTxnId(nextTxnId);
             replica.setState(ReplicaState.DECOMMISSION);
@@ -1003,6 +1039,16 @@ public class TabletScheduler extends MasterDaemon {
     }
 
     /**
+     * Missing for tag, which means some of replicas of this tablet are allocated in wrong backend with specified tag.
+     * Treat it as replica missing, and in handleReplicaMissing(), it will find a property backend to create new replica.
+     */
+    private void handleReplicaMissingForTag(TabletSchedCtx tabletCtx, AgentBatchTask batchTask)
+            throws SchedException {
+        stat.counterReplicaMissingForTagErr.incrementAndGet();
+        handleReplicaMissing(tabletCtx, batchTask);
+    }
+
+    /**
      * Replicas of colocate table's tablet does not locate on right backends set.
      *      backends set:       1,2,3
      *      tablet replicas:    1,2,5
@@ -1018,7 +1064,7 @@ public class TabletScheduler extends MasterDaemon {
 
         stat.counterReplicaColocateMismatch.incrementAndGet();
         // find an available dest backend and path
-        RootPathLoadStatistic destPath = chooseAvailableDestPath(tabletCtx, true /* for colocate */);
+        RootPathLoadStatistic destPath = chooseAvailableDestPath(tabletCtx, null, true /* for colocate */);
         Preconditions.checkNotNull(destPath);
         tabletCtx.setDest(destPath.getBeId(), destPath.getPathHash());
 
@@ -1051,7 +1097,7 @@ public class TabletScheduler extends MasterDaemon {
             addTablet(tabletCtx, false);
         }
     }
-
+ 
     /**
      * Try to create a balance task for a tablet.
      */
@@ -1061,13 +1107,34 @@ public class TabletScheduler extends MasterDaemon {
     }
 
     // choose a path on a backend which is fit for the tablet
-    private RootPathLoadStatistic chooseAvailableDestPath(TabletSchedCtx tabletCtx, boolean forColocate)
+    // if forColocate is false, the tag must be set.
+    private RootPathLoadStatistic chooseAvailableDestPath(TabletSchedCtx tabletCtx, Tag tag, boolean forColocate)
             throws SchedException {
-        ClusterLoadStatistic statistic = statisticMap.get(tabletCtx.getCluster());
-        if (statistic == null) {
-            throw new SchedException(Status.UNRECOVERABLE, "cluster does not exist");
+        List<BackendLoadStatistic> beStatistics;
+        if (tag != null) {
+            Preconditions.checkState(!forColocate);
+            ClusterLoadStatistic statistic = statisticMap.get(tabletCtx.getCluster(), tag);
+            if (statistic == null) {
+                throw new SchedException(Status.UNRECOVERABLE, "cluster does not exist");
+            }
+            beStatistics = statistic.getSortedBeLoadStats(null /* sorted ignore medium */);
+        } else {
+            // for colocate task, get BackendLoadStatistic by colocateBackendIds
+            Preconditions.checkState(forColocate);
+            Preconditions.checkState(tabletCtx.getColocateBackendsSet() != null);
+            Set<Long> colocateBackendIds = tabletCtx.getColocateBackendsSet();
+
+            beStatistics = Lists.newArrayList();
+            Map<Tag, ClusterLoadStatistic> map = statisticMap.row(tabletCtx.getCluster());
+            for (ClusterLoadStatistic clusterStatistic : map.values()) {
+                for (long beId : colocateBackendIds) {
+                    BackendLoadStatistic backendLoadStatistic = clusterStatistic.getBackendLoadStatistic(beId);
+                    if (backendLoadStatistic != null) {
+                        beStatistics.add(backendLoadStatistic);
+                    }
+                }
+            }
         }
-        List<BackendLoadStatistic> beStatistics = statistic.getSortedBeLoadStats(null /* sorted ignore medium */);
 
         // get all available paths which this tablet can fit in.
         // beStatistics is sorted by mix load score in ascend order, so select from first to last.
@@ -1076,15 +1143,23 @@ public class TabletScheduler extends MasterDaemon {
             if (!bes.isAvailable()) {
                 continue;
             }
+
             // exclude host which already has replica of this tablet
             if (tabletCtx.containsBE(bes.getBeId())) {
                 continue;
             }
 
-            if (forColocate && !tabletCtx.getColocateBackendsSet().contains(bes.getBeId())) {
+            // If this for colocate table, only choose backend in colocate backend set.
+            // Else, check the tag.
+            if (forColocate) {
+                if (!tabletCtx.getColocateBackendsSet().contains(bes.getBeId())) {
+                    continue;
+                }
+            } else if (!bes.getTag().equals(tag)) {
                 continue;
             }
 
+
             List<RootPathLoadStatistic> resultPaths = Lists.newArrayList();
             BalanceStatus st = bes.isFit(tabletCtx.getTabletSize(), tabletCtx.getStorageMedium(),
                     resultPaths, tabletCtx.getTabletStatus() != TabletStatus.REPLICA_RELOCATING
diff --git a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletSchedulerStat.java b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletSchedulerStat.java
index dbe1179..f733b05 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/clone/TabletSchedulerStat.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/clone/TabletSchedulerStat.java
@@ -104,6 +104,8 @@ public class TabletSchedulerStat {
     public AtomicLong counterReplicaRedundantErr = new AtomicLong(0L);
     @StatField("num of replica missing in cluster error")
     public AtomicLong counterReplicaMissingInClusterErr = new AtomicLong(0L);
+    @StatField("num of replica missing for tag error")
+    public AtomicLong counterReplicaMissingForTagErr = new AtomicLong(0L);
     @StatField("num of balance scheduled")
     public AtomicLong counterBalanceSchedule = new AtomicLong(0L);
     @StatField("num of colocate replica mismatch")
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/ErrorCode.java b/fe/fe-core/src/main/java/org/apache/doris/common/ErrorCode.java
index 56254e5..d6497dd 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/ErrorCode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/ErrorCode.java
@@ -189,38 +189,38 @@ public enum ErrorCode {
             "Cluster '%s' has backends in decommission"),
     ERR_WRONG_CLUSTER_NAME(5062, new byte[] { '4', '2', '0', '0', '0' },
             "Incorrect cluster name '%s'(name 'default_cluster' is a reserved name)"),
-    ERR_WRONG_NAME_FORMAT(5063, new byte[] { '4', '2', '0', '0', '0' },
+    ERR_WRONG_NAME_FORMAT(5063, new byte[]{'4', '2', '0', '0', '0'},
             "Incorrect %s name '%s'"),
-    ERR_COMMON_ERROR(5064, new byte[] { '4', '2', '0', '0', '0' },
+    ERR_COMMON_ERROR(5064, new byte[]{'4', '2', '0', '0', '0'},
             "%s"),
-    ERR_COLOCATE_FEATURE_DISABLED(5063, new byte[] { '4', '2', '0', '0', '0' },
+    ERR_COLOCATE_FEATURE_DISABLED(5063, new byte[]{'4', '2', '0', '0', '0'},
             "Colocate feature is disabled by Admin"),
-    ERR_COLOCATE_TABLE_NOT_EXIST(5063, new byte[] { '4', '2', '0', '0', '0' },
+    ERR_COLOCATE_TABLE_NOT_EXIST(5063, new byte[]{'4', '2', '0', '0', '0'},
             "Colocate table '%s' does not exist"),
-    ERR_COLOCATE_TABLE_MUST_BE_OLAP_TABLE(5063, new byte[] { '4', '2', '0', '0', '0' },
+    ERR_COLOCATE_TABLE_MUST_BE_OLAP_TABLE(5063, new byte[]{'4', '2', '0', '0', '0'},
             "Colocate table '%s' must be OLAP table"),
-    ERR_COLOCATE_TABLE_MUST_HAS_SAME_REPLICATION_NUM(5063, new byte[] { '4', '2', '0', '0', '0' },
-            "Colocate tables must have same replication num: %s"),
-    ERR_COLOCATE_TABLE_MUST_HAS_SAME_BUCKET_NUM(5063, new byte[] { '4', '2', '0', '0', '0' },
+    ERR_COLOCATE_TABLE_MUST_HAS_SAME_REPLICATION_ALLOCATION(5063, new byte[]{'4', '2', '0', '0', '0'},
+            "Colocate tables must have same replication allocation: %s"),
+    ERR_COLOCATE_TABLE_MUST_HAS_SAME_BUCKET_NUM(5063, new byte[]{'4', '2', '0', '0', '0'},
             "Colocate tables must have same bucket num: %s"),
-    ERR_COLOCATE_TABLE_MUST_HAS_SAME_DISTRIBUTION_COLUMN_SIZE(5063, new byte[] { '4', '2', '0', '0', '0' },
+    ERR_COLOCATE_TABLE_MUST_HAS_SAME_DISTRIBUTION_COLUMN_SIZE(5063, new byte[]{'4', '2', '0', '0', '0'},
             "Colocate tables distribution columns size must be same : %s"),
-    ERR_COLOCATE_TABLE_MUST_HAS_SAME_DISTRIBUTION_COLUMN_TYPE(5063, new byte[] { '4', '2', '0', '0', '0' },
-            "Colocate tables distribution columns must have the same data type: %s should be %s"), 
-    ERR_COLOCATE_NOT_COLOCATE_TABLE(5064, new byte[] { '4', '2', '0', '0', '0' },
+    ERR_COLOCATE_TABLE_MUST_HAS_SAME_DISTRIBUTION_COLUMN_TYPE(5063, new byte[]{'4', '2', '0', '0', '0'},
+            "Colocate tables distribution columns must have the same data type: %s should be %s"),
+    ERR_COLOCATE_NOT_COLOCATE_TABLE(5064, new byte[]{'4', '2', '0', '0', '0'},
             "Table %s is not a colocated table"),
-    ERR_INVALID_OPERATION(5065, new byte[] { '4', '2', '0', '0', '0' }, "Operation %s is invalid"),
-    ERROR_DYNAMIC_PARTITION_TIME_UNIT(5065, new byte[] {'4', '2', '0', '0', '0'},
+    ERR_INVALID_OPERATION(5065, new byte[]{'4', '2', '0', '0', '0'}, "Operation %s is invalid"),
+    ERROR_DYNAMIC_PARTITION_TIME_UNIT(5065, new byte[]{'4', '2', '0', '0', '0'},
             "Unsupported time unit %s. Expect HOUR/DAY/WEEK/MONTH."),
-    ERROR_DYNAMIC_PARTITION_START_ZERO(5066, new byte[] {'4', '2', '0', '0', '0'},
+    ERROR_DYNAMIC_PARTITION_START_ZERO(5066, new byte[]{'4', '2', '0', '0', '0'},
             "Dynamic partition start must less than 0"),
-    ERROR_DYNAMIC_PARTITION_START_FORMAT(5066, new byte[] {'4', '2', '0', '0', '0'},
+    ERROR_DYNAMIC_PARTITION_START_FORMAT(5066, new byte[]{'4', '2', '0', '0', '0'},
             "Invalid dynamic partition start %s"),
-    ERROR_DYNAMIC_PARTITION_END_ZERO(5066, new byte[] {'4', '2', '0', '0', '0'},
+    ERROR_DYNAMIC_PARTITION_END_ZERO(5066, new byte[]{'4', '2', '0', '0', '0'},
             "Dynamic partition end must greater than 0"),
-    ERROR_DYNAMIC_PARTITION_END_FORMAT(5066, new byte[] {'4', '2', '0', '0', '0'},
+    ERROR_DYNAMIC_PARTITION_END_FORMAT(5066, new byte[]{'4', '2', '0', '0', '0'},
             "Invalid dynamic partition end %s"),
-    ERROR_DYNAMIC_PARTITION_END_EMPTY(5066, new byte[] {'4', '2', '0', '0', '0'},
+    ERROR_DYNAMIC_PARTITION_END_EMPTY(5066, new byte[]{'4', '2', '0', '0', '0'},
             "Dynamic partition end is empty"),
     ERROR_DYNAMIC_PARTITION_BUCKETS_ZERO(5067, new byte[] {'4', '2', '0', '0', '0'},
             "Dynamic partition buckets must greater than 0"),
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/FeConstants.java b/fe/fe-core/src/main/java/org/apache/doris/common/FeConstants.java
index b997939..28a0872 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/FeConstants.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/FeConstants.java
@@ -59,4 +59,6 @@ public class FeConstants {
 
     // use \N to indicate NULL
     public static String null_string = "\\N";
+
+    public static long tablet_checker_interval_ms = 20 * 1000L;
 }
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 0aec044..1046a29 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
@@ -220,6 +220,8 @@ public final class FeMetaVersion {
     public static final int VERSION_103 = 103;
     // add sql block rule to deny specified sql
     public static final int VERSION_104 = 104;
+    // change replica to replica allocation
+    public static final int VERSION_105 = 105;
     // note: when increment meta version, should assign the latest version to VERSION_CURRENT
-    public static final int VERSION_CURRENT = VERSION_104;
+    public static final int VERSION_CURRENT = VERSION_105;
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/BackendsProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/BackendsProcDir.java
index 4dff354..c97ae13 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/BackendsProcDir.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/BackendsProcDir.java
@@ -17,7 +17,6 @@
 
 package org.apache.doris.common.proc;
 
-import com.google.gson.Gson;
 import org.apache.doris.alter.DecommissionBackendJob.DecommissionType;
 import org.apache.doris.catalog.Catalog;
 import org.apache.doris.cluster.Cluster;
@@ -35,6 +34,7 @@ import com.google.common.base.Stopwatch;
 import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
+import com.google.gson.Gson;
 
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -164,6 +164,8 @@ public class BackendsProcDir implements ProcDirInterface {
             }
             backendInfo.add(String.format("%.2f", used) + " %");
             backendInfo.add(String.format("%.2f", backend.getMaxDiskUsedPct() * 100) + " %");
+            // tag
+            backendInfo.add(backend.getTag().toString());
 
             backendInfo.add(backend.getHeartbeatErrMsg());
             backendInfo.add(backend.getVersion());
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/ClusterBalanceProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/ClusterBalanceProcDir.java
index 6af20d4..77ebb68 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/ClusterBalanceProcDir.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/ClusterBalanceProcDir.java
@@ -50,7 +50,7 @@ public class ClusterBalanceProcDir implements ProcDirInterface {
     @Override
     public ProcNodeInterface lookup(String name) throws AnalysisException {
         if (name.equals(CLUSTER_LOAD)) {
-            return new ClusterLoadStatByMedium();
+            return new ClusterLoadStatByTag();
         } else if (name.equals(WORKING_SLOTS)) {
             return new SchedulerWorkingSlotsProcDir();
         } else if (name.equals(SCHED_STAT)) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/ClusterLoadStatByMedium.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/ClusterLoadStatByTag.java
similarity index 59%
copy from fe/fe-core/src/main/java/org/apache/doris/common/proc/ClusterLoadStatByMedium.java
copy to fe/fe-core/src/main/java/org/apache/doris/common/proc/ClusterLoadStatByTag.java
index 4afc69c..095143c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/ClusterLoadStatByMedium.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/ClusterLoadStatByTag.java
@@ -17,22 +17,42 @@
 
 package org.apache.doris.common.proc;
 
+import org.apache.doris.catalog.Catalog;
 import org.apache.doris.common.AnalysisException;
-import org.apache.doris.thrift.TStorageMedium;
+import org.apache.doris.resource.Tag;
+import org.apache.doris.system.Backend;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
 
-public class ClusterLoadStatByMedium implements ProcDirInterface {
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+// SHOW PROC "/cluster_balance/cluster_load_stat"
+public class ClusterLoadStatByTag implements ProcDirInterface {
     public static final ImmutableList<String> TITLE_NAMES = new ImmutableList.Builder<String>().add(
             "StorageMedium").build();
 
+    private Map<String, Tag> tagMap = Maps.newHashMap();
+
     @Override
     public ProcResult fetchResult() throws AnalysisException {
         BaseProcResult result = new BaseProcResult();
         result.setNames(TITLE_NAMES);
-        for (TStorageMedium medium : TStorageMedium.values()) {
-            result.addRow(Lists.newArrayList(medium.name()));
+        List<Long> beIds = Catalog.getCurrentSystemInfo().getBackendIds(false);
+        Set<Tag> tags = Sets.newHashSet();
+        for (long beId : beIds) {
+            Backend be = Catalog.getCurrentSystemInfo().getBackend(beId);
+            if (be != null) {
+                tags.add(be.getTag());
+            }
+        }
+        for (Tag tag : tags) {
+            result.addRow(Lists.newArrayList(tag.toKey()));
+            tagMap.put(tag.toKey(), tag);
         }
         return result;
     }
@@ -44,12 +64,11 @@ public class ClusterLoadStatByMedium implements ProcDirInterface {
 
     @Override
     public ProcNodeInterface lookup(String name) throws AnalysisException {
-        for (TStorageMedium medium : TStorageMedium.values()) {
-            if (name.equalsIgnoreCase(medium.name())) {
-                return new ClusterLoadStatisticProcDir(medium);
-            }
+        Tag tag = tagMap.get(name);
+        if (tag == null) {
+            throw new AnalysisException("No such tag: " + name);
         }
-        throw new AnalysisException("no such storage medium: " + name);
+        return new ClusterLoadStatByTagAndMedium(tag);
     }
 
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/ClusterLoadStatByMedium.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/ClusterLoadStatByTagAndMedium.java
similarity index 84%
rename from fe/fe-core/src/main/java/org/apache/doris/common/proc/ClusterLoadStatByMedium.java
rename to fe/fe-core/src/main/java/org/apache/doris/common/proc/ClusterLoadStatByTagAndMedium.java
index 4afc69c..1bee9af 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/ClusterLoadStatByMedium.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/ClusterLoadStatByTagAndMedium.java
@@ -18,15 +18,23 @@
 package org.apache.doris.common.proc;
 
 import org.apache.doris.common.AnalysisException;
+import org.apache.doris.resource.Tag;
 import org.apache.doris.thrift.TStorageMedium;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
-public class ClusterLoadStatByMedium implements ProcDirInterface {
+// SHOW PROC "/cluster_balance/cluster_load_stat/location_default/"
+public class ClusterLoadStatByTagAndMedium implements ProcDirInterface {
     public static final ImmutableList<String> TITLE_NAMES = new ImmutableList.Builder<String>().add(
             "StorageMedium").build();
 
+    private Tag tag;
+
+    public ClusterLoadStatByTagAndMedium(Tag tag) {
+        this.tag = tag;
+    }
+
     @Override
     public ProcResult fetchResult() throws AnalysisException {
         BaseProcResult result = new BaseProcResult();
@@ -46,10 +54,9 @@ public class ClusterLoadStatByMedium implements ProcDirInterface {
     public ProcNodeInterface lookup(String name) throws AnalysisException {
         for (TStorageMedium medium : TStorageMedium.values()) {
             if (name.equalsIgnoreCase(medium.name())) {
-                return new ClusterLoadStatisticProcDir(medium);
+                return new ClusterLoadStatisticProcDir(tag, medium);
             }
         }
         throw new AnalysisException("no such storage medium: " + name);
     }
-
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/ClusterLoadStatisticProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/ClusterLoadStatisticProcDir.java
index df0c182..3e40e92 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/ClusterLoadStatisticProcDir.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/ClusterLoadStatisticProcDir.java
@@ -20,15 +20,17 @@ package org.apache.doris.common.proc;
 import org.apache.doris.catalog.Catalog;
 import org.apache.doris.clone.ClusterLoadStatistic;
 import org.apache.doris.common.AnalysisException;
+import org.apache.doris.resource.Tag;
 import org.apache.doris.system.Backend;
 import org.apache.doris.thrift.TStorageMedium;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Table;
 
 import java.util.List;
 import java.util.Map;
 
-// show proc "/cluster_balance/cluster_load_stat";
+// show proc "/cluster_balance/cluster_load_stat/location_default/HDD";
 public class ClusterLoadStatisticProcDir implements ProcDirInterface {
     public static final ImmutableList<String> TITLE_NAMES = new ImmutableList.Builder<String>()
             .add("BeId").add("Cluster").add("Available").add("UsedCapacity").add("Capacity")
@@ -36,10 +38,12 @@ public class ClusterLoadStatisticProcDir implements ProcDirInterface {
             .add("Class")
             .build();
 
-    private Map<String, ClusterLoadStatistic> statMap;
+    private Table<String, Tag, ClusterLoadStatistic> statMap;
+    private Tag tag;
     private TStorageMedium medium;
 
-    public ClusterLoadStatisticProcDir(TStorageMedium medium) {
+    public ClusterLoadStatisticProcDir(Tag tag, TStorageMedium medium) {
+        this.tag = tag;
         this.medium = medium;
     }
 
@@ -49,8 +53,9 @@ public class ClusterLoadStatisticProcDir implements ProcDirInterface {
         result.setNames(TITLE_NAMES);
 
         statMap = Catalog.getCurrentCatalog().getTabletScheduler().getStatisticMap();
+        Map<String, ClusterLoadStatistic> map = statMap.column(tag);
 
-        statMap.values().forEach(t -> {
+        map.values().forEach(t -> {
             List<List<String>> statistics = t.getClusterStatistic(medium);
             statistics.forEach(result::addRow);
         });
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/ColocationGroupBackendSeqsProcNode.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/ColocationGroupBackendSeqsProcNode.java
index 3e3d5e4..2a164cb 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/ColocationGroupBackendSeqsProcNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/ColocationGroupBackendSeqsProcNode.java
@@ -18,36 +18,46 @@
 package org.apache.doris.common.proc;
 
 import org.apache.doris.common.AnalysisException;
+import org.apache.doris.resource.Tag;
 
 import com.google.common.base.Joiner;
-import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
 import java.util.List;
+import java.util.Map;
 
 /*
  * show proc "/colocation_group/group_name";
  */
 public class ColocationGroupBackendSeqsProcNode implements ProcNodeInterface {
-    public static final ImmutableList<String> TITLE_NAMES = new ImmutableList.Builder<String>()
-            .add("BucketIndex").add("BackendIds").build();
+    private Map<Tag, List<List<Long>>> backendsSeq;
 
-    private List<List<Long>> backendsSeq;
-
-    public ColocationGroupBackendSeqsProcNode(List<List<Long>> backendsSeq) {
+    public ColocationGroupBackendSeqsProcNode(Map<Tag, List<List<Long>>> backendsSeq) {
         this.backendsSeq = backendsSeq;
     }
 
     @Override
     public ProcResult fetchResult() throws AnalysisException {
         BaseProcResult result = new BaseProcResult();
-        result.setNames(TITLE_NAMES);
-        
-        int index = 0;
-        for (List<Long> seqs : backendsSeq) {
+        List<String> titleNames = Lists.newArrayList();
+        titleNames.add("BucketIndex");
+        int bucketNum = 0;
+        for (Tag tag : backendsSeq.keySet()) {
+            titleNames.add(tag.toString());
+            if (bucketNum == 0) {
+                bucketNum = backendsSeq.get(tag).size();
+            } else if (bucketNum != backendsSeq.get(tag).size()) {
+                throw new AnalysisException("Invalid bucket number: " + bucketNum + " vs. " + backendsSeq.get(tag).size());
+            }
+        }
+        result.setNames(titleNames);
+        for (int i = 0; i < bucketNum; i++) {
             List<String> info = Lists.newArrayList();
-            info.add(String.valueOf(index++));
-            info.add(Joiner.on(", ").join(seqs));
+            info.add(String.valueOf(i)); // bucket index
+            for (Tag tag : backendsSeq.keySet()) {
+                List<List<Long>> bucketBackends = backendsSeq.get(tag);
+                info.add(Joiner.on(", ").join(bucketBackends.get(i)));
+            }
             result.addRow(info);
         }
         return result;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/ColocationGroupProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/ColocationGroupProcDir.java
index 58b54bd..8936e66 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/ColocationGroupProcDir.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/ColocationGroupProcDir.java
@@ -21,10 +21,12 @@ import org.apache.doris.catalog.Catalog;
 import org.apache.doris.catalog.ColocateTableIndex;
 import org.apache.doris.catalog.ColocateTableIndex.GroupId;
 import org.apache.doris.common.AnalysisException;
+import org.apache.doris.resource.Tag;
 
 import com.google.common.collect.ImmutableList;
 
 import java.util.List;
+import java.util.Map;
 
 /*
  * show proc "/colocation_group";
@@ -32,7 +34,8 @@ import java.util.List;
 public class ColocationGroupProcDir implements ProcDirInterface {
     public static final ImmutableList<String> TITLE_NAMES = new ImmutableList.Builder<String>()
             .add("GroupId").add("GroupName").add("TableIds")
-            .add("BucketsNum").add("ReplicationNum").add("DistCols").add("IsStable").build();
+            .add("BucketsNum").add("ReplicaAllocation").add("DistCols").add("IsStable")
+            .add("ErrorMsg").build();
 
     @Override
     public boolean register(String name, ProcNodeInterface node) {
@@ -57,7 +60,7 @@ public class ColocationGroupProcDir implements ProcDirInterface {
 
         GroupId groupId = new GroupId(dbId, grpId);
         ColocateTableIndex index = Catalog.getCurrentColocateIndex();
-        List<List<Long>> beSeqs = index.getBackendsPerBucketSeq(groupId);
+        Map<Tag, List<List<Long>>> beSeqs = index.getBackendsPerBucketSeq(groupId);
         return new ColocationGroupBackendSeqsProcNode(beSeqs);
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/FrontendsProcNode.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/FrontendsProcNode.java
index 6bad1aa..c8804f9 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/FrontendsProcNode.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/FrontendsProcNode.java
@@ -45,6 +45,7 @@ public class FrontendsProcNode implements ProcNodeInterface {
             .add("Name").add("IP").add("HostName").add("EditLogPort").add("HttpPort").add("QueryPort").add("RpcPort")
             .add("Role").add("IsMaster").add("ClusterId").add("Join").add("Alive")
             .add("ReplayedJournalId").add("LastHeartbeat").add("IsHelper").add("ErrMsg").add("Version")
+            .add("CurrentConnected")
             .build();
     
     public static final int HOSTNAME_INDEX = 2;
@@ -82,13 +83,15 @@ public class FrontendsProcNode implements ProcNodeInterface {
             // this may happen when majority of FOLLOWERS are down and no MASTER right now.
             LOG.warn("failed to get leader: {}", e.getMessage());
         }
-        
+
         // get all node which are joined in bdb group
         List<InetSocketAddress> allFe = catalog.getHaProtocol().getElectableNodes(true /* include leader */);
         allFe.addAll(catalog.getHaProtocol().getObserverNodes());
         List<Pair<String, Integer>> allFeHosts = convertToHostPortPair(allFe);
         List<Pair<String, Integer>> helperNodes = catalog.getHelperNodes();
-        
+
+        Pair<String, Integer> selfNode = Catalog.getCurrentCatalog().getSelfNode();
+
         for (Frontend fe : catalog.getFrontends(null /* all */)) {
 
             List<String> info = new ArrayList<String>();
@@ -121,12 +124,11 @@ public class FrontendsProcNode implements ProcNodeInterface {
                 info.add(Long.toString(fe.getReplayedJournalId()));
             }
             info.add(TimeUtils.longToTimeString(fe.getLastUpdateTime()));
-            
             info.add(String.valueOf(isHelperNode(helperNodes, fe)));
-
             info.add(fe.getHeartbeatErrMsg());
-
             info.add(fe.getVersion());
+            // To indicate which FE we currently connected
+            info.add(fe.getHost().equals(selfNode.first) ? "Yes" : "No");
 
             infos.add(info);
         }
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 f9a12e2..5f8b5d4 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
@@ -67,9 +67,7 @@ public class PartitionsProcDir implements ProcDirInterface {
             .add("VisibleVersion").add("VisibleVersionTime").add("VisibleVersionHash")
             .add("State").add("PartitionKey").add("Range").add("DistributionKey")
             .add("Buckets").add("ReplicationNum").add("StorageMedium").add("CooldownTime")
-            .add("LastConsistencyCheckTime")
-            .add("DataSize")
-            .add("IsInMemory")
+            .add("LastConsistencyCheckTime").add("DataSize").add("IsInMemory").add("ReplicaAllocation")
             .build();
 
     private Database db;
@@ -266,9 +264,8 @@ public class PartitionsProcDir implements ProcDirInterface {
                 }
 
                 partitionInfo.add(distributionInfo.getBucketNum());
-
-                short replicationNum = tblPartitionInfo.getReplicationNum(partitionId);
-                partitionInfo.add(String.valueOf(replicationNum));
+                // replica num
+                partitionInfo.add(tblPartitionInfo.getReplicaAllocation(partitionId).getTotalReplicaNum());
 
                 DataProperty dataProperty = tblPartitionInfo.getDataProperty(partitionId);
                 partitionInfo.add(dataProperty.getStorageMedium().name());
@@ -282,6 +279,8 @@ public class PartitionsProcDir implements ProcDirInterface {
                         + sizePair.second;
                 partitionInfo.add(readableSize);
                 partitionInfo.add(tblPartitionInfo.getIsInMemory(partitionId));
+                // replica allocation
+                partitionInfo.add(tblPartitionInfo.getReplicaAllocation(partitionId).toCreateStmt());
 
                 partitionInfos.add(partitionInfo);
             }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/proc/StatisticProcDir.java b/fe/fe-core/src/main/java/org/apache/doris/common/proc/StatisticProcDir.java
index 6f051bc..1246ad7 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/proc/StatisticProcDir.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/proc/StatisticProcDir.java
@@ -23,6 +23,7 @@ import org.apache.doris.catalog.MaterializedIndex;
 import org.apache.doris.catalog.MaterializedIndex.IndexExtState;
 import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.catalog.Partition;
+import org.apache.doris.catalog.ReplicaAllocation;
 import org.apache.doris.catalog.Table;
 import org.apache.doris.catalog.Table.TableType;
 import org.apache.doris.catalog.Tablet;
@@ -130,7 +131,7 @@ public class StatisticProcDir implements ProcDirInterface {
                     table.readLock();
                     try {
                         for (Partition partition : olapTable.getAllPartitions()) {
-                            short replicationNum = olapTable.getPartitionInfo().getReplicationNum(partition.getId());
+                            ReplicaAllocation replicaAlloc = olapTable.getPartitionInfo().getReplicaAllocation(partition.getId());
                             ++dbPartitionNum;
                             for (MaterializedIndex materializedIndex : partition.getMaterializedIndices(IndexExtState.VISIBLE)) {
                                 ++dbIndexNum;
@@ -141,7 +142,7 @@ public class StatisticProcDir implements ProcDirInterface {
                                     Pair<TabletStatus, Priority> res = tablet.getHealthStatusWithPriority(
                                             infoService, db.getClusterName(),
                                             partition.getVisibleVersion(), partition.getVisibleVersionHash(),
-                                            replicationNum, aliveBeIdsInCluster);
+                                            replicaAlloc, aliveBeIdsInCluster);
 
                                     // here we treat REDUNDANT as HEALTHY, for user friendly.
                                     if (res.first != TabletStatus.HEALTHY && res.first != TabletStatus.REDUNDANT
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/DynamicPartitionUtil.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/DynamicPartitionUtil.java
index c142e88..09dc859 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/util/DynamicPartitionUtil.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/DynamicPartitionUtil.java
@@ -27,6 +27,7 @@ import org.apache.doris.catalog.PartitionInfo;
 import org.apache.doris.catalog.PartitionType;
 import org.apache.doris.catalog.PrimitiveType;
 import org.apache.doris.catalog.RangePartitionInfo;
+import org.apache.doris.catalog.ReplicaAllocation;
 import org.apache.doris.catalog.Table;
 import org.apache.doris.catalog.TableProperty;
 import org.apache.doris.common.AnalysisException;
@@ -36,6 +37,7 @@ import org.apache.doris.common.ErrorCode;
 import org.apache.doris.common.ErrorReport;
 import org.apache.doris.common.FeConstants;
 import org.apache.doris.common.FeNameFormat;
+import org.apache.doris.common.UserException;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
@@ -311,7 +313,8 @@ public class DynamicPartitionUtil {
     }
 
     // Analyze all properties to check their validation
-    public static Map<String, String> analyzeDynamicPartition(Map<String, String> properties, PartitionInfo partitionInfo) throws DdlException {
+    public static Map<String, String> analyzeDynamicPartition(Map<String, String> properties, PartitionInfo partitionInfo)
+            throws UserException {
         // properties should not be empty, check properties before call this function
         Map<String, String> analyzedProperties = new HashMap<>();
         if (properties.containsKey(DynamicPartitionProperty.TIME_UNIT)) {
@@ -419,11 +422,19 @@ public class DynamicPartitionUtil {
             properties.remove(DynamicPartitionProperty.TIME_ZONE);
             analyzedProperties.put(DynamicPartitionProperty.TIME_ZONE, val);
         }
+
         if (properties.containsKey(DynamicPartitionProperty.REPLICATION_NUM)) {
             String val = properties.get(DynamicPartitionProperty.REPLICATION_NUM);
             checkReplicationNum(val);
             properties.remove(DynamicPartitionProperty.REPLICATION_NUM);
-            analyzedProperties.put(DynamicPartitionProperty.REPLICATION_NUM, val);
+            analyzedProperties.put(DynamicPartitionProperty.REPLICATION_ALLOCATION,
+                    new ReplicaAllocation(Short.valueOf(val)).toCreateStmt());
+        }
+
+        if (properties.containsKey(DynamicPartitionProperty.REPLICATION_ALLOCATION)) {
+            ReplicaAllocation replicaAlloc = PropertyAnalyzer.analyzeReplicaAllocation(properties, "dynamic_partition");
+            properties.remove(DynamicPartitionProperty.REPLICATION_ALLOCATION);
+            analyzedProperties.put(DynamicPartitionProperty.REPLICATION_ALLOCATION, replicaAlloc.toCreateStmt());
         }
 
         if (properties.containsKey(DynamicPartitionProperty.HOT_PARTITION_NUM)) {
@@ -464,7 +475,7 @@ public class DynamicPartitionUtil {
      * properties should be checked before call this method
      */
     public static void checkAndSetDynamicPartitionProperty(OlapTable olapTable, Map<String, String> properties)
-            throws DdlException {
+            throws UserException {
         if (DynamicPartitionUtil.checkInputDynamicPartitionProperties(properties, olapTable.getPartitionInfo())) {
             Map<String, String> dynamicPartitionProperties =
                     DynamicPartitionUtil.analyzeDynamicPartition(properties, olapTable.getPartitionInfo());
diff --git a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java
index 5812eb7..4673b16 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/common/util/PropertyAnalyzer.java
@@ -23,11 +23,13 @@ import org.apache.doris.catalog.DataProperty;
 import org.apache.doris.catalog.KeysType;
 import org.apache.doris.catalog.Partition;
 import org.apache.doris.catalog.PrimitiveType;
+import org.apache.doris.catalog.ReplicaAllocation;
 import org.apache.doris.catalog.ScalarType;
 import org.apache.doris.catalog.Type;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.Pair;
+import org.apache.doris.resource.Tag;
 import org.apache.doris.thrift.TStorageFormat;
 import org.apache.doris.thrift.TStorageMedium;
 import org.apache.doris.thrift.TStorageType;
@@ -50,6 +52,7 @@ public class PropertyAnalyzer {
 
     public static final String PROPERTIES_SHORT_KEY = "short_key";
     public static final String PROPERTIES_REPLICATION_NUM = "replication_num";
+    public static final String PROPERTIES_REPLICATION_ALLOCATION = "replication_allocation";
     public static final String PROPERTIES_STORAGE_TYPE = "storage_type";
     public static final String PROPERTIES_STORAGE_MEDIUM = "storage_medium";
     public static final String PROPERTIES_STORAGE_COLDOWN_TIME = "storage_cooldown_time";
@@ -93,6 +96,8 @@ public class PropertyAnalyzer {
 
     public static final String PROPERTIES_SWAP_TABLE = "swap";
 
+    public static final String TAG_LOCATION = "tag.location";
+
     public static DataProperty analyzeDataProperty(Map<String, String> properties, DataProperty oldDataProperty)
             throws AnalysisException {
         if (properties == null) {
@@ -173,13 +178,14 @@ public class PropertyAnalyzer {
 
         return shortKeyColumnCount;
     }
-    
-    public static Short analyzeReplicationNum(Map<String, String> properties, short oldReplicationNum)
+
+    private static Short analyzeReplicationNum(Map<String, String> properties, String prefix, short oldReplicationNum)
             throws AnalysisException {
         Short replicationNum = oldReplicationNum;
-        if (properties != null && properties.containsKey(PROPERTIES_REPLICATION_NUM)) {
+        String propKey = Strings.isNullOrEmpty(prefix) ? PROPERTIES_REPLICATION_NUM : prefix + "." + PROPERTIES_REPLICATION_NUM;
+        if (properties != null && properties.containsKey(propKey)) {
             try {
-                replicationNum = Short.valueOf(properties.get(PROPERTIES_REPLICATION_NUM));
+                replicationNum = Short.valueOf(properties.get(propKey));
             } catch (Exception e) {
                 throw new AnalysisException(e.getMessage());
             }
@@ -188,21 +194,7 @@ public class PropertyAnalyzer {
                 throw new AnalysisException("Replication num should larger than 0. (suggested 3)");
             }
 
-            properties.remove(PROPERTIES_REPLICATION_NUM);
-        }
-        return replicationNum;
-    }
-
-    public static Short analyzeReplicationNum(Map<String, String> properties, boolean isDefault) throws AnalysisException {
-        String key = "default.";
-        if (isDefault) {
-            key += PropertyAnalyzer.PROPERTIES_REPLICATION_NUM;
-        } else {
-            key = PropertyAnalyzer.PROPERTIES_REPLICATION_NUM;
-        }
-        short replicationNum = Short.valueOf(properties.get(key));
-        if (replicationNum <= 0) {
-            throw new AnalysisException("Replication num should larger than 0. (suggested 3)");
+            properties.remove(propKey);
         }
         return replicationNum;
     }
@@ -456,9 +448,71 @@ public class PropertyAnalyzer {
             throw new AnalysisException("sequence column only support UNIQUE_KEYS");
         }
         PrimitiveType type = PrimitiveType.valueOf(typeStr.toUpperCase());
-        if (!type.isFixedPointType() && !type.isDateType())  {
+        if (!type.isFixedPointType() && !type.isDateType()) {
             throw new AnalysisException("sequence type only support integer types and date types");
         }
         return ScalarType.createType(type);
     }
+
+    public static Tag analyzeBackendTagProperties(Map<String, String> properties) throws AnalysisException {
+        if (properties.containsKey(TAG_LOCATION)) {
+            String tagVal = properties.remove(TAG_LOCATION);
+            return Tag.create(Tag.TYPE_LOCATION, tagVal);
+        }
+        return Tag.DEFAULT_BACKEND_TAG;
+    }
+
+    // There are 2 kinds of replication property:
+    // 1. "replication_num" = "3"
+    // 2. "replication_allocation" = "tag.location.zone1: 2, tag.location.zone2: 1"
+    // These 2 kinds of property will all be converted to a ReplicaAllocation and return.
+    // Return ReplicaAllocation.NOT_SET if no replica property is set.
+    //
+    // prefix is for property key such as "dynamic_partition.replication_num", which prefix is "dynamic_partition"
+    public static ReplicaAllocation analyzeReplicaAllocation(Map<String, String> properties, String prefix)
+            throws AnalysisException {
+        if (properties == null || properties.isEmpty()) {
+            return ReplicaAllocation.NOT_SET;
+        }
+        // if give "replication_num" property, return with default backend tag
+        Short replicaNum = analyzeReplicationNum(properties, prefix, (short) 0);
+        if (replicaNum > 0) {
+            return new ReplicaAllocation(replicaNum);
+        }
+
+        String propKey = Strings.isNullOrEmpty(prefix) ? PROPERTIES_REPLICATION_ALLOCATION
+                : prefix + "." + PROPERTIES_REPLICATION_ALLOCATION;
+        // if not set, return default replication allocation
+        if (!properties.containsKey(propKey)) {
+            return ReplicaAllocation.NOT_SET;
+        }
+
+        // analyze user specified replication allocation
+        // format is as: "tag.location.zone1: 2, tag.location.zone2: 1"
+        ReplicaAllocation replicaAlloc = new ReplicaAllocation();
+        String allocationVal = properties.remove(propKey);
+        allocationVal = allocationVal.replaceAll(" ", "");
+        String[] locations = allocationVal.split(",");
+        for (String location : locations) {
+            String[] parts = location.split(":");
+            if (parts.length != 2) {
+                throw new AnalysisException("Invalid replication allocation property: " + location);
+            }
+            if (!parts[0].startsWith(TAG_LOCATION)) {
+                throw new AnalysisException("Invalid replication allocation tag property: " + location);
+            }
+            String locationVal = parts[0].substring(TAG_LOCATION.length() + 1); // +1 to skip dot.
+            if (Strings.isNullOrEmpty(locationVal)) {
+                throw new AnalysisException("Invalid replication allocation location tag property: " + location);
+            }
+
+            replicaAlloc.put(Tag.create(Tag.TYPE_LOCATION, locationVal), Short.valueOf(parts[1]));
+        }
+
+        if (replicaAlloc.isEmpty()) {
+            throw new AnalysisException("Not specified replica allocation property");
+        }
+        return replicaAlloc;
+    }
+
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/consistency/CheckConsistencyJob.java b/fe/fe-core/src/main/java/org/apache/doris/consistency/CheckConsistencyJob.java
index a10e7e4..d0cbb49 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/consistency/CheckConsistencyJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/consistency/CheckConsistencyJob.java
@@ -147,8 +147,8 @@ public class CheckConsistencyJob {
             }
 
             // check partition's replication num. if 1 replication. skip
-            short replicationNum = olapTable.getPartitionInfo().getReplicationNum(partition.getId());
-            if (replicationNum == (short) 1) {
+            short replicaNum = olapTable.getPartitionInfo().getReplicaAllocation(partition.getId()).getTotalReplicaNum();
+            if (replicaNum == (short) 1) {
                 LOG.debug("partition[{}]'s replication num is 1. skip consistency check", partition.getId());
                 return false;
             }
@@ -199,7 +199,7 @@ public class CheckConsistencyJob {
                 ++sentTaskReplicaNum;
             }
 
-            if (sentTaskReplicaNum < replicationNum / 2 + 1) {
+            if (sentTaskReplicaNum < replicaNum / 2 + 1) {
                 LOG.info("tablet[{}] does not have enough replica to check.", tabletId);
             } else {
                 if (maxDataSize > 0) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/consistency/ConsistencyChecker.java b/fe/fe-core/src/main/java/org/apache/doris/consistency/ConsistencyChecker.java
index acab74d..492a17d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/consistency/ConsistencyChecker.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/consistency/ConsistencyChecker.java
@@ -280,7 +280,7 @@ public class ConsistencyChecker extends MasterDaemon {
                                 new PriorityQueue<>(Math.max(table.getAllPartitions().size(), 1), COMPARATOR);
                         for (Partition partition : table.getPartitions()) {
                             // check partition's replication num. if 1 replication. skip
-                            if (table.getPartitionInfo().getReplicationNum(partition.getId()) == (short) 1) {
+                            if (table.getPartitionInfo().getReplicaAllocation(partition.getId()).getTotalReplicaNum() == (short) 1) {
                                 LOG.debug("partition[{}]'s replication num is 1. ignore", partition.getId());
                                 continue;
                             }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/deploy/DeployManager.java b/fe/fe-core/src/main/java/org/apache/doris/deploy/DeployManager.java
index 4f2df31..8927484 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/deploy/DeployManager.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/deploy/DeployManager.java
@@ -22,6 +22,7 @@ import org.apache.doris.catalog.FsBroker;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.common.Pair;
+import org.apache.doris.common.UserException;
 import org.apache.doris.common.util.MasterDaemon;
 import org.apache.doris.ha.FrontendNodeType;
 import org.apache.doris.system.Backend;
@@ -591,7 +592,7 @@ public class DeployManager extends MasterDaemon {
                         default:
                             break;
                     }
-                } catch (DdlException e) {
+                } catch (UserException e) {
                     LOG.error("Failed to add {} node: {}:{}", nodeType, remoteIp, remotePort, e);
                     return true;
                 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/http/meta/ColocateMetaService.java b/fe/fe-core/src/main/java/org/apache/doris/http/meta/ColocateMetaService.java
index 63c54ee..777586e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/http/meta/ColocateMetaService.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/http/meta/ColocateMetaService.java
@@ -30,7 +30,6 @@ import org.apache.doris.http.rest.RestBaseAction;
 import org.apache.doris.http.rest.RestBaseResult;
 import org.apache.doris.http.rest.RestResult;
 import org.apache.doris.mysql.privilege.PrivPredicate;
-import org.apache.doris.persist.ColocatePersistInfo;
 import org.apache.doris.qe.ConnectContext;
 
 import com.google.common.base.Preconditions;
@@ -196,9 +195,9 @@ public class ColocateMetaService {
             List<Long> clusterBackendIds = Catalog.getCurrentSystemInfo().getClusterBackendIds(clusterName, true);
             //check the Backend id
             for (List<Long> backendIds : backendsPerBucketSeq) {
-                if (backendIds.size() != groupSchema.getReplicationNum()) {
+                if (backendIds.size() != groupSchema.getReplicaAlloc().getTotalReplicaNum()) {
                     throw new DdlException("Invalid backend num per bucket. expected: "
-                            + groupSchema.getReplicationNum() + ", actual: " + backendIds.size());
+                            + groupSchema.getReplicaAlloc().getTotalReplicaNum() + ", actual: " + backendIds.size());
                 }
                 for (Long beId : backendIds) {
                     if (!clusterBackendIds.contains(beId)) {
@@ -216,10 +215,14 @@ public class ColocateMetaService {
             sendResult(request, response);
         }
 
-        private void updateBackendPerBucketSeq(GroupId groupId, List<List<Long>> backendsPerBucketSeq) {
+        private void updateBackendPerBucketSeq(GroupId groupId, List<List<Long>> backendsPerBucketSeq)
+                throws DdlException {
+            throw new DdlException("Currently not support");
+            /*
             colocateIndex.addBackendsPerBucketSeq(groupId, backendsPerBucketSeq);
             ColocatePersistInfo info2 = ColocatePersistInfo.createForBackendsPerBucketSeq(groupId, backendsPerBucketSeq);
             Catalog.getCurrentCatalog().getEditLog().logColocateBackendsPerBucketSeq(info2);
+            */
         }
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/http/rest/LoadAction.java b/fe/fe-core/src/main/java/org/apache/doris/http/rest/LoadAction.java
index 33a3910..9981007 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/http/rest/LoadAction.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/http/rest/LoadAction.java
@@ -116,7 +116,8 @@ public class LoadAction extends RestBaseAction {
             redirectAddr = execEnv.getMultiLoadMgr().redirectAddr(fullDbName, label);
         } else {
             // Choose a backend sequentially.
-            List<Long> backendIds = Catalog.getCurrentSystemInfo().seqChooseBackendIds(1, true, false, clusterName);
+            List<Long> backendIds = Catalog.getCurrentSystemInfo().seqChooseBackendIdsByStorageMediumAndTag(
+                    1, true, false, clusterName, null, null);
             if (backendIds == null) {
                 throw new DdlException("No backend alive.");
             }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/meta/ColocateMetaService.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/meta/ColocateMetaService.java
index c467215..b9c75ff 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/meta/ColocateMetaService.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/meta/ColocateMetaService.java
@@ -25,9 +25,12 @@ import org.apache.doris.common.DdlException;
 import org.apache.doris.httpv2.entity.ResponseEntityBuilder;
 import org.apache.doris.httpv2.rest.RestBaseController;
 import org.apache.doris.mysql.privilege.PrivPredicate;
-import org.apache.doris.persist.ColocatePersistInfo;
 import org.apache.doris.qe.ConnectContext;
 
+import com.google.common.base.Preconditions;
+import com.google.gson.Gson;
+import com.google.gson.reflect.TypeToken;
+
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 import org.springframework.web.bind.annotation.RequestBody;
@@ -36,14 +39,11 @@ import org.springframework.web.bind.annotation.RequestMethod;
 import org.springframework.web.bind.annotation.RestController;
 import org.springframework.web.servlet.view.RedirectView;
 
-import com.google.common.base.Preconditions;
-import com.google.gson.Gson;
-import com.google.gson.reflect.TypeToken;
+import java.lang.reflect.Type;
+import java.util.List;
 
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
-import java.lang.reflect.Type;
-import java.util.List;
 
 /*
  * the colocate meta define in {@link ColocateTableIndex}
@@ -138,9 +138,9 @@ public class ColocateMetaService extends RestBaseController {
         List<Long> clusterBackendIds = Catalog.getCurrentSystemInfo().getClusterBackendIds(clusterName, true);
         //check the Backend id
         for (List<Long> backendIds : backendsPerBucketSeq) {
-            if (backendIds.size() != groupSchema.getReplicationNum()) {
+            if (backendIds.size() != groupSchema.getReplicaAlloc().getTotalReplicaNum()) {
                 return ResponseEntityBuilder.okWithCommonError("Invalid backend num per bucket. expected: "
-                        + groupSchema.getReplicationNum() + ", actual: " + backendIds.size());
+                        + groupSchema.getReplicaAlloc().getTotalReplicaNum() + ", actual: " + backendIds.size());
             }
             for (Long beId : backendIds) {
                 if (!clusterBackendIds.contains(beId)) {
@@ -158,11 +158,13 @@ public class ColocateMetaService extends RestBaseController {
         return ResponseEntityBuilder.ok();
     }
 
-    private void updateBackendPerBucketSeq(GroupId groupId, List<List<Long>> backendsPerBucketSeq) {
+    private void updateBackendPerBucketSeq(GroupId groupId, List<List<Long>> backendsPerBucketSeq)
+            throws DdlException {
+        throw new DdlException("Currently not support");
+        /*
         colocateIndex.addBackendsPerBucketSeq(groupId, backendsPerBucketSeq);
         ColocatePersistInfo info2 = ColocatePersistInfo.createForBackendsPerBucketSeq(groupId, backendsPerBucketSeq);
         Catalog.getCurrentCatalog().getEditLog().logColocateBackendsPerBucketSeq(info2);
+         */
     }
-
-
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/LoadAction.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/LoadAction.java
index 6a618bd..194e244 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/LoadAction.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/rest/LoadAction.java
@@ -17,8 +17,6 @@
 
 package org.apache.doris.httpv2.rest;
 
-import io.netty.handler.codec.http.HttpHeaderNames;
-
 import org.apache.doris.catalog.Catalog;
 import org.apache.doris.cluster.ClusterNamespace;
 import org.apache.doris.common.DdlException;
@@ -44,6 +42,8 @@ import java.util.List;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
+import io.netty.handler.codec.http.HttpHeaderNames;
+
 @RestController
 public class LoadAction extends RestBaseController {
 
@@ -128,7 +128,8 @@ public class LoadAction extends RestBaseController {
                 }
             } else {
                 // Choose a backend sequentially.
-                List<Long> backendIds = Catalog.getCurrentSystemInfo().seqChooseBackendIds(1, true, false, clusterName);
+                List<Long> backendIds = Catalog.getCurrentSystemInfo().seqChooseBackendIdsByStorageMediumAndTag(
+                        1, true, false, clusterName, null, null);
                 if (backendIds == null) {
                     return new RestBaseResult("No backend alive.");
                 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/httpv2/util/LoadSubmitter.java b/fe/fe-core/src/main/java/org/apache/doris/httpv2/util/LoadSubmitter.java
index 1cac7e3..62338d0 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/httpv2/util/LoadSubmitter.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/httpv2/util/LoadSubmitter.java
@@ -25,13 +25,13 @@ import org.apache.doris.httpv2.rest.UploadAction;
 import org.apache.doris.system.Backend;
 import org.apache.doris.system.SystemInfoService;
 
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
 import com.google.common.base.Strings;
 import com.google.gson.Gson;
 import com.google.gson.reflect.TypeToken;
 
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
 import java.io.BufferedInputStream;
 import java.io.BufferedOutputStream;
 import java.io.BufferedReader;
@@ -137,8 +137,8 @@ public class LoadSubmitter {
         }
 
         private Backend selectOneBackend() throws DdlException {
-            List<Long> backendIds = Catalog.getCurrentSystemInfo().seqChooseBackendIds(
-                    1, true, false, SystemInfoService.DEFAULT_CLUSTER);
+            List<Long> backendIds = Catalog.getCurrentSystemInfo().seqChooseBackendIdsByStorageMediumAndTag(
+                    1, true, false, SystemInfoService.DEFAULT_CLUSTER, null, null);
             if (backendIds == null) {
                 throw new DdlException("No alive backend");
             }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java b/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java
index 4bdb054..05fd9b1 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/journal/JournalEntity.java
@@ -203,8 +203,7 @@ public class JournalEntity implements Writable {
                 break;
             }
             case OperationType.OP_MODIFY_PARTITION: {
-                data = new ModifyPartitionInfo();
-                ((ModifyPartitionInfo) data).readFields(in);
+                data = ModifyPartitionInfo.read(in);
                 isRead = true;
                 break;
             }
@@ -470,8 +469,7 @@ public class JournalEntity implements Writable {
             case OperationType.OP_COLOCATE_BACKENDS_PER_BUCKETSEQ:
             case OperationType.OP_COLOCATE_MARK_UNSTABLE:
             case OperationType.OP_COLOCATE_MARK_STABLE: {
-                data = new ColocatePersistInfo();
-                ((ColocatePersistInfo) data).readFields(in);
+                data = ColocatePersistInfo.read(in);
                 isRead = true;
                 break;
             }
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 d23b604..8b127eb 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
@@ -186,7 +186,8 @@ public class DeleteHandler implements Writable {
                         throw new DdlException("Partition does not exist. name: " + partName);
                     }
                     partitions.add(partition);
-                    partitionReplicaNum.put(partition.getId(), olapTable.getPartitionInfo().getReplicationNum(partition.getId()));
+                    partitionReplicaNum.put(partition.getId(),
+                            olapTable.getPartitionInfo().getReplicaAllocation(partition.getId()).getTotalReplicaNum());
                 }
 
                 List<String> deleteConditions = Lists.newArrayList();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java b/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java
index bf804e6..32f36f6 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/LoadChecker.java
@@ -51,12 +51,12 @@ import org.apache.doris.transaction.TabletQuorumFailedException;
 import org.apache.doris.transaction.TransactionState;
 import org.apache.doris.transaction.TransactionStatus;
 
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.List;
@@ -396,7 +396,7 @@ public class LoadChecker extends MasterDaemon {
                         return null;
                     }
                     
-                    short replicationNum = table.getPartitionInfo().getReplicationNum(partition.getId());
+                    short replicationNum = table.getPartitionInfo().getReplicaAllocation(partition.getId()).getTotalReplicaNum();
                     // check all indices (base + roll up (not include ROLLUP state index))
                     List<MaterializedIndex> indices = partition.getMaterializedIndices(IndexExtState.ALL);
                     for (MaterializedIndex index : indices) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java
index 30fceb5..cb38cbf 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/load/loadv2/SparkLoadJob.java
@@ -88,9 +88,6 @@ import org.apache.doris.transaction.TransactionState.LoadJobSourceType;
 import org.apache.doris.transaction.TransactionState.TxnCoordinator;
 import org.apache.doris.transaction.TransactionState.TxnSourceType;
 
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 import com.google.common.collect.Lists;
@@ -98,6 +95,9 @@ import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 import com.google.gson.annotations.SerializedName;
 
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.File;
@@ -445,7 +445,7 @@ public class SparkLoadJob extends BulkLoadJob {
                         }
 
                         hasLoadPartitions = true;
-                        int quorumReplicaNum = olapTable.getPartitionInfo().getReplicationNum(partitionId) / 2 + 1;
+                        int quorumReplicaNum = olapTable.getPartitionInfo().getReplicaAllocation(partitionId).getTotalReplicaNum() / 2 + 1;
 
                         List<MaterializedIndex> indexes = partition.getMaterializedIndices(IndexExtState.ALL);
                         for (MaterializedIndex index : indexes) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java
index 867c634..df0e599 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/master/ReportHandler.java
@@ -27,6 +27,7 @@ import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.catalog.Partition;
 import org.apache.doris.catalog.Replica;
 import org.apache.doris.catalog.Replica.ReplicaState;
+import org.apache.doris.catalog.ReplicaAllocation;
 import org.apache.doris.catalog.Table;
 import org.apache.doris.catalog.Tablet;
 import org.apache.doris.catalog.Tablet.TabletStatus;
@@ -555,7 +556,7 @@ public class ReportHandler extends Daemon {
                         continue;
                     }
 
-                    short replicationNum = olapTable.getPartitionInfo().getReplicationNum(partition.getId());
+                    short replicationNum = olapTable.getPartitionInfo().getReplicaAllocation(partition.getId()).getTotalReplicaNum();
 
                     long indexId = tabletMeta.getIndexId();
                     MaterializedIndex index = partition.getIndex(indexId);
@@ -944,7 +945,7 @@ public class ReportHandler extends Daemon {
             if (partition == null) {
                 throw new MetaNotFoundException("partition[" + partitionId + "] does not exist");
             }
-            short replicationNum = olapTable.getPartitionInfo().getReplicationNum(partition.getId());
+            ReplicaAllocation replicaAlloc = olapTable.getPartitionInfo().getReplicaAllocation(partition.getId());
 
             MaterializedIndex materializedIndex = partition.getIndex(indexId);
             if (materializedIndex == null) {
@@ -980,7 +981,7 @@ public class ReportHandler extends Daemon {
             List<Long> aliveBeIdsInCluster = infoService.getClusterBackendIds(db.getClusterName(), true);
             Pair<TabletStatus, TabletSchedCtx.Priority> status = tablet.getHealthStatusWithPriority(infoService,
                     db.getClusterName(), visibleVersion, visibleVersionHash,
-                    replicationNum, aliveBeIdsInCluster);
+                    replicaAlloc, aliveBeIdsInCluster);
 
             if (status.first == TabletStatus.VERSION_INCOMPLETE || status.first == TabletStatus.REPLICA_MISSING
                     || status.first == TabletStatus.UNRECOVERABLE) {
@@ -1033,7 +1034,7 @@ public class ReportHandler extends Daemon {
                     }
                 }
                 throw new MetaNotFoundException(
-                        "replica is enough[" + tablet.getReplicas().size() + "-" + replicationNum + "]");
+                        "replica is enough[" + tablet.getReplicas().size() + "-" + replicaAlloc.toCreateStmt() + "]");
             }
         } finally {
             olapTable.writeUnlock();
diff --git a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/CommonUserProperties.java b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/CommonUserProperties.java
index 9206e6a..6064a47 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/CommonUserProperties.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/CommonUserProperties.java
@@ -20,25 +20,34 @@ package org.apache.doris.mysql.privilege;
 import org.apache.doris.common.io.Text;
 import org.apache.doris.common.io.Writable;
 import org.apache.doris.persist.gson.GsonUtils;
+import org.apache.doris.resource.Tag;
 
 import com.google.gson.annotations.SerializedName;
 
+import org.glassfish.jersey.internal.guava.Sets;
+
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
+import java.util.Set;
 
 /**
  * Used in
  */
 public class CommonUserProperties implements Writable {
+    // The max connections allowed for a user on one FE
     @SerializedName("maxConn")
     private long maxConn = 100;
+    // The maximum total number of query instances that the user is allowed to send from this FE
     @SerializedName("maxQueryInstances")
     private long maxQueryInstances = -1;
     @SerializedName("sqlBlockRules")
     private String sqlBlockRules = "";
     @SerializedName("cpuResourceLimit")
     private int cpuResourceLimit = -1;
+    // The tag of the resource that the user is allowed to use
+    @SerializedName("resourceTags")
+    private Set<Tag> resourceTags = Sets.newHashSet();
 
     private String[] sqlBlockRulesSplit = {};
 
@@ -84,6 +93,14 @@ public class CommonUserProperties implements Writable {
         this.cpuResourceLimit = cpuResourceLimit;
     }
 
+    public void setResourceTags(Set<Tag> resourceTags) {
+        this.resourceTags = resourceTags;
+    }
+
+    public Set<Tag> getResourceTags() {
+        return resourceTags;
+    }
+
     public static CommonUserProperties read(DataInput in) throws IOException {
         String json = Text.readString(in);
         CommonUserProperties commonUserProperties = GsonUtils.GSON.fromJson(json, CommonUserProperties.class);
@@ -97,4 +114,4 @@ public class CommonUserProperties implements Writable {
         String json = GsonUtils.GSON.toJson(this);
         Text.writeString(out, json);
     }
-}
\ No newline at end of file
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/PaloAuth.java b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/PaloAuth.java
index 80cd5b6..d6cf835 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/PaloAuth.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/PaloAuth.java
@@ -40,6 +40,7 @@ import org.apache.doris.common.FeConstants;
 import org.apache.doris.common.FeMetaVersion;
 import org.apache.doris.common.LdapConfig;
 import org.apache.doris.common.Pair;
+import org.apache.doris.common.UserException;
 import org.apache.doris.common.io.Writable;
 import org.apache.doris.ldap.LdapClient;
 import org.apache.doris.ldap.LdapPrivsChecker;
@@ -47,6 +48,7 @@ import org.apache.doris.load.DppConfig;
 import org.apache.doris.persist.LdapInfo;
 import org.apache.doris.persist.PrivInfo;
 import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.resource.Tag;
 import org.apache.doris.thrift.TFetchResourceResult;
 import org.apache.doris.thrift.TPrivilegeStatus;
 
@@ -1092,17 +1094,17 @@ public class PaloAuth implements Writable {
     }
 
     // update user property
-    public void updateUserProperty(SetUserPropertyStmt stmt) throws DdlException {
+    public void updateUserProperty(SetUserPropertyStmt stmt) throws UserException {
         List<Pair<String, String>> properties = stmt.getPropertyPairList();
         updateUserPropertyInternal(stmt.getUser(), properties, false /* is replay */);
     }
 
-    public void replayUpdateUserProperty(UserPropertyInfo propInfo) throws DdlException {
+    public void replayUpdateUserProperty(UserPropertyInfo propInfo) throws UserException {
         updateUserPropertyInternal(propInfo.getUser(), propInfo.getProperties(), true /* is replay */);
     }
 
     public void updateUserPropertyInternal(String user, List<Pair<String, String>> properties, boolean isReplay)
-            throws DdlException {
+            throws UserException {
         writeLock();
         try {
             propertyMgr.updateUserProperty(user, properties);
@@ -1152,6 +1154,15 @@ public class PaloAuth implements Writable {
         }
     }
 
+    public Set<Tag> getResourceTags(String qualifiedUser) {
+        readLock();
+        try {
+            return propertyMgr.getResourceTags(qualifiedUser);
+        } finally {
+            readUnlock();
+        }
+    }
+
     public void getAllDomains(Set<String> allDomains) {
         readLock();
         try {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/UserProperty.java b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/UserProperty.java
index 689b872..9182cfd 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/UserProperty.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/UserProperty.java
@@ -23,13 +23,16 @@ import org.apache.doris.catalog.Catalog;
 import org.apache.doris.catalog.ResourceGroup;
 import org.apache.doris.catalog.ResourceType;
 import org.apache.doris.cluster.ClusterNamespace;
+import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.common.FeMetaVersion;
 import org.apache.doris.common.LoadException;
 import org.apache.doris.common.Pair;
+import org.apache.doris.common.UserException;
 import org.apache.doris.common.io.Text;
 import org.apache.doris.common.io.Writable;
 import org.apache.doris.load.DppConfig;
+import org.apache.doris.resource.Tag;
 import org.apache.doris.system.SystemInfoService;
 
 import org.apache.commons.lang.StringUtils;
@@ -56,15 +59,18 @@ import java.util.regex.Pattern;
  * This user is just qualified by cluster name, not host which it connected from.
  */
 public class UserProperty implements Writable {
-
+    // advanced properties
     private static final String PROP_MAX_USER_CONNECTIONS = "max_user_connections";
+    private static final String PROP_MAX_QUERY_INSTANCES = "max_query_instances";
+    private static final String PROP_RESOURCE_TAGS = "resource_tags";
     private static final String PROP_RESOURCE = "resource";
+    private static final String PROP_SQL_BLOCK_RULES = "sql_block_rules";
+    private static final String PROP_CPU_RESOURCE_LIMIT = "cpu_resource_limit";
+    // advanced properties end
+
     private static final String PROP_LOAD_CLUSTER = "load_cluster";
     private static final String PROP_QUOTA = "quota";
     private static final String PROP_DEFAULT_LOAD_CLUSTER = "default_load_cluster";
-    private static final String PROP_SQL_BLOCK_RULES = "sql_block_rules";
-    private static final String PROP_MAX_QUERY_INSTANCES = "max_query_instances";
-    private static final String PROP_CPU_RESOURCE_LIMIT = "cpu_resource_limit";
 
     // for system user
     public static final Set<Pattern> ADVANCED_PROPERTIES = Sets.newHashSet();
@@ -88,6 +94,13 @@ public class UserProperty implements Writable {
      */
     private WhiteList whiteList = new WhiteList();
 
+    public static final Set<Tag> INVALID_RESOURCE_TAGS;
+
+    static {
+        INVALID_RESOURCE_TAGS = Sets.newHashSet();
+        INVALID_RESOURCE_TAGS.add(Tag.INVALID_TAG);
+    }
+
     static {
         ADVANCED_PROPERTIES.add(Pattern.compile("^" + PROP_MAX_USER_CONNECTIONS + "$", Pattern.CASE_INSENSITIVE));
         ADVANCED_PROPERTIES.add(Pattern.compile("^" + PROP_RESOURCE + ".", Pattern.CASE_INSENSITIVE));
@@ -96,6 +109,7 @@ public class UserProperty implements Writable {
         ADVANCED_PROPERTIES.add(Pattern.compile("^" + PROP_MAX_QUERY_INSTANCES + "$", Pattern.CASE_INSENSITIVE));
         ADVANCED_PROPERTIES.add(Pattern.compile("^" + PROP_SQL_BLOCK_RULES + "$", Pattern.CASE_INSENSITIVE));
         ADVANCED_PROPERTIES.add(Pattern.compile("^" + PROP_CPU_RESOURCE_LIMIT + "$", Pattern.CASE_INSENSITIVE));
+        ADVANCED_PROPERTIES.add(Pattern.compile("^" + PROP_RESOURCE_TAGS + "$", Pattern.CASE_INSENSITIVE));
 
         COMMON_PROPERTIES.add(Pattern.compile("^" + PROP_QUOTA + ".", Pattern.CASE_INSENSITIVE));
         COMMON_PROPERTIES.add(Pattern.compile("^" + PROP_DEFAULT_LOAD_CLUSTER + "$", Pattern.CASE_INSENSITIVE));
@@ -134,6 +148,10 @@ public class UserProperty implements Writable {
         return whiteList;
     }
 
+    public Set<Tag> getCopiedResourceTags() {
+        return Sets.newHashSet(this.commonProperties.getResourceTags());
+    }
+
     public void setPasswordForDomain(String domain, byte[] password, boolean errOnExist) throws DdlException {
         if (errOnExist && whiteList.containsDomain(domain)) {
             throw new DdlException("Domain " + domain + " of user " + qualifiedUser + " already exists");
@@ -148,12 +166,13 @@ public class UserProperty implements Writable {
         whiteList.removeDomain(domain);
     }
 
-    public void update(List<Pair<String, String>> properties) throws DdlException {
+    public void update(List<Pair<String, String>> properties) throws UserException {
         // copy
         long newMaxConn = this.commonProperties.getMaxConn();
         long newMaxQueryInstances = this.commonProperties.getMaxQueryInstances();
         String sqlBlockRules = this.commonProperties.getSqlBlockRules();
         int cpuResourceLimit = this.commonProperties.getCpuResourceLimit();
+        Set<Tag> resourceTags = this.commonProperties.getResourceTags();
 
         UserResource newResource = resource.getCopiedUserResource();
         String newDefaultLoadCluster = defaultLoadCluster;
@@ -262,6 +281,18 @@ public class UserProperty implements Writable {
                 }
 
                 cpuResourceLimit = limit;
+            } else if (keyArr[0].equalsIgnoreCase(PROP_RESOURCE_TAGS)) {
+                if (keyArr.length != 2) {
+                    throw new DdlException(PROP_RESOURCE_TAGS + " format error");
+                }
+                if (!keyArr[1].equals(Tag.TYPE_LOCATION)) {
+                    throw new DdlException("Only support location tag now");
+                }
+                try {
+                    resourceTags = parseLocationResoureTags(value);
+                } catch (NumberFormatException e) {
+                    throw new DdlException(PROP_RESOURCE_TAGS + " parse failed: " + e.getMessage());
+                }
             } else {
                 throw new DdlException("Unknown user property(" + key + ")");
             }
@@ -272,7 +303,7 @@ public class UserProperty implements Writable {
         this.commonProperties.setMaxQueryInstances(newMaxQueryInstances);
         this.commonProperties.setSqlBlockRules(sqlBlockRules);
         this.commonProperties.setCpuResourceLimit(cpuResourceLimit);
-
+        this.commonProperties.setResourceTags(resourceTags);
         resource = newResource;
         if (newDppConfigs.containsKey(newDefaultLoadCluster)) {
             defaultLoadCluster = newDefaultLoadCluster;
@@ -282,6 +313,16 @@ public class UserProperty implements Writable {
         clusterToDppConfig = newDppConfigs;
     }
 
+    private Set<Tag> parseLocationResoureTags(String value) throws AnalysisException {
+        Set<Tag> tags = Sets.newHashSet();
+        String[] parts = value.replaceAll(" ", "").split(",");
+        for (String part : parts) {
+            Tag tag = Tag.create(Tag.TYPE_LOCATION, part);
+            tags.add(tag);
+        }
+        return tags;
+    }
+
     private void updateLoadCluster(String[] keyArr, String value, Map<String, DppConfig> newDppConfigs)
             throws DdlException {
         if (keyArr.length == 1 && Strings.isNullOrEmpty(value)) {
@@ -366,6 +407,9 @@ public class UserProperty implements Writable {
         // cpu resource limit
         result.add(Lists.newArrayList(PROP_CPU_RESOURCE_LIMIT, String.valueOf(commonProperties.getCpuResourceLimit())));
 
+        // resource tag
+        result.add(Lists.newArrayList(PROP_RESOURCE_TAGS, Joiner.on(", ").join(commonProperties.getResourceTags())));
+
         // resource
         ResourceGroup group = resource.getResource();
         for (Map.Entry<ResourceType, Integer> entry : group.getQuotaMap().entrySet()) {
@@ -441,7 +485,7 @@ public class UserProperty implements Writable {
         userProperty.readFields(in);
         return userProperty;
     }
-    
+
     @Override
     public void write(DataOutput out) throws IOException {
         // user name
diff --git a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/UserPropertyMgr.java b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/UserPropertyMgr.java
index 1f53dd7..18551d9 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/UserPropertyMgr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/UserPropertyMgr.java
@@ -22,8 +22,10 @@ import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.common.Pair;
+import org.apache.doris.common.UserException;
 import org.apache.doris.common.io.Writable;
 import org.apache.doris.load.DppConfig;
+import org.apache.doris.resource.Tag;
 import org.apache.doris.thrift.TAgentServiceVersion;
 import org.apache.doris.thrift.TFetchResourceResult;
 
@@ -106,7 +108,7 @@ public class UserPropertyMgr implements Writable {
         }
     }
 
-    public void updateUserProperty(String user, List<Pair<String, String>> properties) throws DdlException {
+    public void updateUserProperty(String user, List<Pair<String, String>> properties) throws UserException {
         UserProperty property = propertyMap.get(user);
         if (property == null) {
             throw new DdlException("Unknown user(" + user + ")");
@@ -131,6 +133,14 @@ public class UserPropertyMgr implements Writable {
         return existProperty.getMaxQueryInstances();
     }
 
+    public Set<Tag> getResourceTags(String qualifiedUser) {
+        UserProperty existProperty = propertyMap.get(qualifiedUser);
+        if (existProperty == null) {
+            return UserProperty.INVALID_RESOURCE_TAGS;
+        }
+        return existProperty.getCopiedResourceTags();
+    }
+
     public int getPropertyMapSize() {
         return propertyMap.size();
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/UserResourceProperty.java b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/UserResourceProperty.java
new file mode 100644
index 0000000..e4b609d
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/mysql/privilege/UserResourceProperty.java
@@ -0,0 +1,31 @@
+// 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.mysql.privilege;
+
+import org.apache.doris.common.io.Writable;
+
+import java.io.DataOutput;
+import java.io.IOException;
+
+public class UserResourceProperty implements Writable {
+
+    @Override
+    public void write(DataOutput out) throws IOException {
+
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/ColocatePersistInfo.java b/fe/fe-core/src/main/java/org/apache/doris/persist/ColocatePersistInfo.java
index b25b3fb..cdfbc94 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/persist/ColocatePersistInfo.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/persist/ColocatePersistInfo.java
@@ -20,50 +20,59 @@ package org.apache.doris.persist;
 import org.apache.doris.catalog.Catalog;
 import org.apache.doris.catalog.ColocateTableIndex.GroupId;
 import org.apache.doris.common.FeMetaVersion;
+import org.apache.doris.common.io.Text;
 import org.apache.doris.common.io.Writable;
+import org.apache.doris.persist.gson.GsonUtils;
+import org.apache.doris.resource.Tag;
 
-import com.google.common.collect.Lists;
+import com.clearspring.analytics.util.Lists;
+import com.google.common.collect.Maps;
+import com.google.gson.annotations.SerializedName;
 
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 
 /**
  * PersistInfo for ColocateTableIndex
  */
 public class ColocatePersistInfo implements Writable {
+    @SerializedName(value = "groupId")
     private GroupId groupId;
+    @SerializedName(value = "tableId")
     private long tableId;
-    private List<List<Long>> backendsPerBucketSeq = Lists.newArrayList();
+    @SerializedName(value = "backendsPerBucketSeq")
+    private Map<Tag, List<List<Long>>> backendsPerBucketSeq = Maps.newHashMap();
 
     public ColocatePersistInfo() {
 
     }
 
-    public static ColocatePersistInfo createForAddTable(GroupId groupId, long tableId, List<List<Long>> backendsPerBucketSeq) {
+    public static ColocatePersistInfo createForAddTable(GroupId groupId, long tableId, Map<Tag, List<List<Long>>> backendsPerBucketSeq) {
         return new ColocatePersistInfo(groupId, tableId, backendsPerBucketSeq);
     }
 
     public static ColocatePersistInfo createForBackendsPerBucketSeq(GroupId groupId,
-            List<List<Long>> backendsPerBucketSeq) {
+                                                                    Map<Tag, List<List<Long>>> backendsPerBucketSeq) {
         return new ColocatePersistInfo(groupId, -1L, backendsPerBucketSeq);
     }
 
     public static ColocatePersistInfo createForMarkUnstable(GroupId groupId) {
-        return new ColocatePersistInfo(groupId, -1L, new ArrayList<>());
+        return new ColocatePersistInfo(groupId, -1L, Maps.newHashMap());
     }
 
     public static ColocatePersistInfo createForMarkStable(GroupId groupId) {
-        return new ColocatePersistInfo(groupId, -1L, new ArrayList<>());
+        return new ColocatePersistInfo(groupId, -1L, Maps.newHashMap());
     }
 
     public static ColocatePersistInfo createForRemoveTable(long tableId) {
-        return new ColocatePersistInfo(new GroupId(-1, -1), tableId, new ArrayList<>());
+        return new ColocatePersistInfo(new GroupId(-1, -1), tableId, Maps.newHashMap());
     }
 
-    private ColocatePersistInfo(GroupId groupId, long tableId, List<List<Long>> backendsPerBucketSeq) {
+    private ColocatePersistInfo(GroupId groupId, long tableId, Map<Tag, List<List<Long>>> backendsPerBucketSeq) {
         this.groupId = groupId;
         this.tableId = tableId;
         this.backendsPerBucketSeq = backendsPerBucketSeq;
@@ -77,27 +86,28 @@ public class ColocatePersistInfo implements Writable {
         return groupId;
     }
 
-    public List<List<Long>> getBackendsPerBucketSeq() {
+    public Map<Tag, List<List<Long>>> getBackendsPerBucketSeq() {
         return backendsPerBucketSeq;
     }
 
+    public static ColocatePersistInfo read(DataInput in) throws IOException {
+        if (Catalog.getCurrentCatalogJournalVersion() < FeMetaVersion.VERSION_105) {
+            ColocatePersistInfo info = new ColocatePersistInfo();
+            info.readFields(in);
+            return info;
+        } else {
+            String json = Text.readString(in);
+            return GsonUtils.GSON.fromJson(json, ColocatePersistInfo.class);
+        }
+    }
+
     @Override
     public void write(DataOutput out) throws IOException {
-        out.writeLong(tableId);
-        groupId.write(out);
-        // out.writeLong(groupId);
-        // out.writeLong(dbId);
-        int size = backendsPerBucketSeq.size();
-        out.writeInt(size);
-        for (List<Long> beList : backendsPerBucketSeq) {
-            out.writeInt(beList.size());
-            for (Long be : beList) {
-                out.writeLong(be);
-            }
-        }
+        Text.writeString(out, GsonUtils.GSON.toJson(this));
     }
 
-    public void readFields(DataInput in) throws IOException {
+    @Deprecated
+    private void readFields(DataInput in) throws IOException {
         tableId = in.readLong();
         if (Catalog.getCurrentCatalogJournalVersion() < FeMetaVersion.VERSION_55) {
             long grpId = in.readLong();
@@ -108,14 +118,16 @@ public class ColocatePersistInfo implements Writable {
         }
 
         int size = in.readInt();
-        backendsPerBucketSeq = new ArrayList<>();
+        backendsPerBucketSeq = Maps.newHashMap();
+        List<List<Long>> backendsPerBucketSeqList = Lists.newArrayList();
+        backendsPerBucketSeq.put(Tag.DEFAULT_BACKEND_TAG, backendsPerBucketSeqList);
         for (int i = 0; i < size; i++) {
             int beListSize = in.readInt();
             List<Long> beLists = new ArrayList<>();
             for (int j = 0; j < beListSize; j++) {
                 beLists.add(in.readLong());
             }
-            backendsPerBucketSeq.add(beLists);
+            backendsPerBucketSeqList.add(beLists);
         }
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java
index cc84395..c3f18ec 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/persist/EditLog.java
@@ -426,6 +426,11 @@ public class EditLog {
                     Catalog.getCurrentSystemInfo().replayDropBackend(be);
                     break;
                 }
+                case OperationType.OP_MODIFY_BACKEND: {
+                    Backend be = (Backend) journal.getData();
+                    Catalog.getCurrentSystemInfo().replayModifyBackend(be);
+                    break;
+                }
                 case OperationType.OP_BACKEND_STATE_CHANGE: {
                     Backend be = (Backend) journal.getData();
                     Catalog.getCurrentSystemInfo().updateBackendState(be);
@@ -1112,6 +1117,10 @@ public class EditLog {
         logEdit(OperationType.OP_DROP_BACKEND, be);
     }
 
+    public void logModifyBackend(Backend be) {
+        logEdit(OperationType.OP_MODIFY_BACKEND, be);
+    }
+
     public void logAddFrontend(Frontend fe) {
         logEdit(OperationType.OP_ADD_FRONTEND, fe);
     }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/ModifyPartitionInfo.java b/fe/fe-core/src/main/java/org/apache/doris/persist/ModifyPartitionInfo.java
index 7996906..a7d7872 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/persist/ModifyPartitionInfo.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/persist/ModifyPartitionInfo.java
@@ -19,8 +19,12 @@ package org.apache.doris.persist;
 
 import org.apache.doris.catalog.Catalog;
 import org.apache.doris.catalog.DataProperty;
+import org.apache.doris.catalog.ReplicaAllocation;
 import org.apache.doris.common.FeMetaVersion;
+import org.apache.doris.common.io.Text;
 import org.apache.doris.common.io.Writable;
+import org.apache.doris.persist.gson.GsonUtils;
+
 import com.google.gson.annotations.SerializedName;
 
 import java.io.DataInput;
@@ -37,23 +41,25 @@ public class ModifyPartitionInfo implements Writable {
     private long partitionId;
     @SerializedName(value = "dataProperty")
     private DataProperty dataProperty;
-    @SerializedName(value = "replicationNum")
+    @Deprecated
     private short replicationNum;
     @SerializedName(value = "isInMemory")
     private boolean isInMemory;
+    @SerializedName(value = "replicaAlloc")
+    private ReplicaAllocation replicaAlloc;
 
     public ModifyPartitionInfo() {
         // for persist
     }
 
     public ModifyPartitionInfo(long dbId, long tableId, long partitionId,
-                               DataProperty dataProperty, short replicationNum,
+                               DataProperty dataProperty, ReplicaAllocation replicaAlloc,
                                boolean isInMemory) {
         this.dbId = dbId;
         this.tableId = tableId;
         this.partitionId = partitionId;
         this.dataProperty = dataProperty;
-        this.replicationNum = replicationNum;
+        this.replicaAlloc = replicaAlloc;
         this.isInMemory = isInMemory;
     }
 
@@ -73,8 +79,8 @@ public class ModifyPartitionInfo implements Writable {
         return dataProperty;
     }
 
-    public short getReplicationNum() {
-        return replicationNum;
+    public ReplicaAllocation getReplicaAlloc() {
+        return replicaAlloc;
     }
 
     public boolean isInMemory() {
@@ -82,9 +88,14 @@ public class ModifyPartitionInfo implements Writable {
     }
 
     public static ModifyPartitionInfo read(DataInput in) throws IOException {
-        ModifyPartitionInfo info = new ModifyPartitionInfo();
-        info.readFields(in);
-        return info;
+        if (Catalog.getCurrentCatalogJournalVersion() < FeMetaVersion.VERSION_105) {
+            ModifyPartitionInfo info = new ModifyPartitionInfo();
+            info.readFields(in);
+            return info;
+        } else {
+            String json = Text.readString(in);
+            return GsonUtils.GSON.fromJson(json, ModifyPartitionInfo.class);
+        }
     }
 
     @Override
@@ -97,32 +108,21 @@ public class ModifyPartitionInfo implements Writable {
         }
         ModifyPartitionInfo otherInfo = (ModifyPartitionInfo) other;
         return dbId == otherInfo.getDbId() && tableId == otherInfo.getTableId() &&
-                dataProperty.equals(otherInfo.getDataProperty()) && replicationNum == otherInfo.getReplicationNum()
+                dataProperty.equals(otherInfo.getDataProperty()) && replicaAlloc.equals(otherInfo.replicaAlloc)
                 && isInMemory == otherInfo.isInMemory();
     }
 
     @Override
     public void write(DataOutput out) throws IOException {
-        out.writeLong(dbId);
-        out.writeLong(tableId);
-        out.writeLong(partitionId);
-
-        if (dataProperty == null) {
-            out.writeBoolean(false);
-        } else {
-            out.writeBoolean(true);
-            dataProperty.write(out);
-        }
-
-        out.writeShort(replicationNum);
-        out.writeBoolean(isInMemory);
+        Text.writeString(out, GsonUtils.GSON.toJson(this));
     }
 
-    public void readFields(DataInput in) throws IOException {
+    @Deprecated
+    private void readFields(DataInput in) throws IOException {
         dbId = in.readLong();
         tableId = in.readLong();
         partitionId = in.readLong();
-        
+
         boolean hasDataProperty = in.readBoolean();
         if (hasDataProperty) {
             dataProperty = DataProperty.read(in);
@@ -131,9 +131,13 @@ public class ModifyPartitionInfo implements Writable {
         }
 
         replicationNum = in.readShort();
+        if (replicationNum > 0) {
+            replicaAlloc = new ReplicaAllocation(replicationNum);
+        } else {
+            replicaAlloc = ReplicaAllocation.NOT_SET;
+        }
         if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_72) {
             isInMemory = in.readBoolean();
         }
     }
-
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java b/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java
index 3916a76..7a92808 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/persist/OperationType.java
@@ -133,6 +133,7 @@ public class OperationType {
     public static final short OP_UPDATE_CLUSTER_AND_BACKENDS = 88;
     public static final short OP_CREATE_REPOSITORY = 89;
     public static final short OP_DROP_REPOSITORY = 90;
+    public static final short OP_MODIFY_BACKEND = 91;
 
     //colocate table
     public static final short OP_COLOCATE_ADD_TABLE = 94;
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 d482e79..b3fdbd3 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
@@ -23,6 +23,7 @@ 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.catalog.ReplicaAllocation;
 import org.apache.doris.common.FeMetaVersion;
 import org.apache.doris.common.io.Writable;
 import org.apache.doris.common.util.RangeUtils;
@@ -41,7 +42,7 @@ public class PartitionPersistInfo implements Writable {
     private Range<PartitionKey> range;
     private PartitionItem listPartitionItem;
     private DataProperty dataProperty;
-    private short replicationNum;
+    private ReplicaAllocation replicaAlloc;
     private boolean isInMemory = false;
     private boolean isTempPartition = false;
     
@@ -49,7 +50,7 @@ public class PartitionPersistInfo implements Writable {
     }
 
     public PartitionPersistInfo(long dbId, long tableId, Partition partition, Range<PartitionKey> range,
-                                PartitionItem listPartitionItem, DataProperty dataProperty, short replicationNum,
+                                PartitionItem listPartitionItem, DataProperty dataProperty, ReplicaAllocation replicaAlloc,
                                 boolean isInMemory, boolean isTempPartition) {
         this.dbId = dbId;
         this.tableId = tableId;
@@ -59,7 +60,7 @@ public class PartitionPersistInfo implements Writable {
         this.listPartitionItem = listPartitionItem;
         this.dataProperty = dataProperty;
 
-        this.replicationNum = replicationNum;
+        this.replicaAlloc = replicaAlloc;
         this.isInMemory = isInMemory;
         this.isTempPartition = isTempPartition;
     }
@@ -87,9 +88,9 @@ public class PartitionPersistInfo implements Writable {
     public DataProperty getDataProperty() {
         return dataProperty;
     }
-    
-    public short getReplicationNum() {
-        return replicationNum;
+
+    public ReplicaAllocation getReplicaAlloc() {
+        return replicaAlloc;
     }
 
     public boolean isInMemory() {
@@ -108,7 +109,7 @@ public class PartitionPersistInfo implements Writable {
         RangeUtils.writeRange(out, range);
         listPartitionItem.write(out);
         dataProperty.write(out);
-        out.writeShort(replicationNum);
+        replicaAlloc.write(out);
         out.writeBoolean(isInMemory);
         out.writeBoolean(isTempPartition);
     }
@@ -126,7 +127,12 @@ public class PartitionPersistInfo implements Writable {
         }
 
         dataProperty = DataProperty.read(in);
-        replicationNum = in.readShort();
+        if (Catalog.getCurrentCatalogJournalVersion() < FeMetaVersion.VERSION_105) {
+            this.replicaAlloc = new ReplicaAllocation(in.readShort());
+        } else {
+            this.replicaAlloc = ReplicaAllocation.read(in);
+        }
+
         if (Catalog.getCurrentCatalogJournalVersion() >= FeMetaVersion.VERSION_72) {
             isInMemory = in.readBoolean();
         }
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 cc2ff1e..cd0ac63 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
@@ -51,6 +51,7 @@ import org.apache.doris.common.UserException;
 import org.apache.doris.common.util.Util;
 import org.apache.doris.qe.ConnectContext;
 import org.apache.doris.qe.SessionVariable;
+import org.apache.doris.resource.Tag;
 import org.apache.doris.service.FrontendOptions;
 import org.apache.doris.system.Backend;
 import org.apache.doris.thrift.TExplainLevel;
@@ -73,6 +74,7 @@ import com.google.common.collect.Maps;
 
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
+import org.glassfish.jersey.internal.guava.Sets;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -462,6 +464,12 @@ public class OlapScanNode extends ScanNode {
         String visibleVersionStr = String.valueOf(visibleVersion);
         String visibleVersionHashStr = String.valueOf(partition.getVisibleVersionHash());
 
+        Set<Tag> allowedTags = Sets.newHashSet();
+        boolean needCheckTags = false;
+        if (ConnectContext.get() != null) {
+            allowedTags = ConnectContext.get().getResourceTags();
+            needCheckTags = ConnectContext.get().isResourceTagsSet();
+        }
         for (Tablet tablet : tablets) {
             long tabletId = tablet.getId();
             LOG.debug("{} tabletId={}", (logNum++), tabletId);
@@ -500,11 +508,22 @@ public class OlapScanNode extends ScanNode {
             Collections.shuffle(replicas);
             boolean tabletIsNull = true;
             boolean collectedStat = false;
+            List<String> errs = Lists.newArrayList();
             for (Replica replica : replicas) {
                 Backend backend = Catalog.getCurrentSystemInfo().getBackend(replica.getBackendId());
                 if (backend == null || !backend.isAlive()) {
                     LOG.debug("backend {} not exists or is not alive for replica {}",
                             replica.getBackendId(), replica.getId());
+                    errs.add(replica.getId() + "'s backend " + replica.getBackendId() + " does not exist or not alive");
+                    continue;
+                }
+                if (needCheckTags && !allowedTags.isEmpty() && !allowedTags.contains(backend.getTag())) {
+                    String err = String.format("Replica on backend %d with tag %s, which is not in user's resource tags: %s",
+                            backend.getId(), backend.getTag(), allowedTags);
+                    if (LOG.isDebugEnabled()) {
+                        LOG.debug(err);
+                    }
+                    errs.add(err);
                     continue;
                 }
                 String ip = backend.getHost();
@@ -515,7 +534,7 @@ public class OlapScanNode extends ScanNode {
                 paloRange.addToHosts(new TNetworkAddress(ip, port));
                 tabletIsNull = false;
 
-                //for CBO
+                // for CBO
                 if (!collectedStat && replica.getRowCount() != -1) {
                     cardinality += replica.getRowCount();
                     totalBytes += replica.getDataSize();
@@ -524,7 +543,7 @@ public class OlapScanNode extends ScanNode {
                 scanBackendIds.add(backend.getId());
             }
             if (tabletIsNull) {
-                throw new UserException("tablet: " + tabletId + " have no alive replicas.");
+                throw new UserException(tabletId + " have no queryable replicas. err: " + Joiner.on(", ").join(errs));
             }
             TScanRange scanRange = new TScanRange();
             scanRange.setPaloScanRange(paloRange);
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 420c399..1a9a930 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
@@ -17,7 +17,6 @@
 
 package org.apache.doris.planner;
 
-import org.apache.commons.lang.StringUtils;
 import org.apache.doris.analysis.SlotDescriptor;
 import org.apache.doris.analysis.TupleDescriptor;
 import org.apache.doris.catalog.Catalog;
@@ -67,6 +66,7 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Multimap;
 import com.google.common.collect.Range;
 
+import org.apache.commons.lang.StringUtils;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -130,7 +130,7 @@ public class OlapTableSink extends DataSink {
         tSink.setTupleId(tupleDescriptor.getId().asInt());
         int numReplicas = 1;
         for (Partition partition : dstTable.getPartitions()) {
-            numReplicas = dstTable.getPartitionInfo().getReplicationNum(partition.getId());
+            numReplicas = dstTable.getPartitionInfo().getReplicaAllocation(partition.getId()).getTotalReplicaNum();
             break;
         }
         tSink.setNumReplicas(numReplicas);
@@ -320,7 +320,7 @@ public class OlapTableSink extends DataSink {
         Multimap<Long, Long> allBePathsMap = HashMultimap.create();
         for (Long partitionId : partitionIds) {
             Partition partition = table.getPartition(partitionId);
-            int quorum = table.getPartitionInfo().getReplicationNum(partition.getId()) / 2 + 1;            
+            int quorum = table.getPartitionInfo().getReplicaAllocation(partition.getId()).getTotalReplicaNum() / 2 + 1;
             for (MaterializedIndex index : partition.getMaterializedIndices(IndexExtState.ALL)) {
                 // we should ensure the replica backend is alive
                 // otherwise, there will be a 'unknown node id, id=xxx' error for stream load
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java
index 36b4641..5d5ce3f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ConnectContext.java
@@ -28,17 +28,20 @@ import org.apache.doris.mysql.MysqlCommand;
 import org.apache.doris.mysql.MysqlSerializer;
 import org.apache.doris.mysql.privilege.PaloRole;
 import org.apache.doris.plugin.AuditEvent.AuditEventBuilder;
+import org.apache.doris.resource.Tag;
 import org.apache.doris.thrift.TResourceInfo;
 import org.apache.doris.thrift.TUniqueId;
 import org.apache.doris.transaction.TransactionEntry;
 
 import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
 
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
 import java.nio.channels.SocketChannel;
 import java.util.List;
+import java.util.Set;
 
 // When one client connect in, we create a connect context for it.
 // We store session information here. Meanwhile ConnectScheduler all
@@ -104,7 +107,8 @@ public class ConnectContext {
     protected Catalog catalog;
     protected boolean isSend;
 
-    protected AuditEventBuilder auditEventBuilder = new AuditEventBuilder();;
+    protected AuditEventBuilder auditEventBuilder = new AuditEventBuilder();
+    ;
 
     protected String remoteIP;
 
@@ -115,6 +119,14 @@ public class ConnectContext {
 
     // If set to true, the nondeterministic function will not be rewrote to constant.
     private boolean notEvalNondeterministicFunction = false;
+    // The resource tag is used to limit the node resources that the user can use for query.
+    // The default is empty, that is, unlimited.
+    // This property is obtained from UserProperty when the client connection is created.
+    // Only when the connection is created again, the new resource tags will be retrieved from the UserProperty
+    private Set<Tag> resourceTags = Sets.newHashSet();
+    // If set to true, the resource tags set in resourceTags will be used to limit the query resources.
+    // If set to false, the system will not restrict query resources.
+    private boolean isResourceTagsSet = false;
 
     private String sqlHash;
 
@@ -474,9 +486,22 @@ public class ConnectContext {
         }
         return threadInfo;
     }
- 
+
+    public boolean isResourceTagsSet() {
+        return isResourceTagsSet;
+    }
+
+    public Set<Tag> getResourceTags() {
+        return resourceTags;
+    }
+
+    public void setResourceTags(Set<Tag> resourceTags) {
+        this.resourceTags = resourceTags;
+        this.isResourceTagsSet = !this.resourceTags.isEmpty();
+    }
+
     public class ThreadInfo {
-        public List<String>  toRow(long nowMs) {
+        public List<String> toRow(long nowMs) {
             List<String> row = Lists.newArrayList();
             row.add("" + connectionId);
             row.add(ClusterNamespace.getNameFromFullName(qualifiedUser));
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 a788efc..7d44cc2 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,8 +50,8 @@ import org.apache.doris.thrift.TMasterOpRequest;
 import org.apache.doris.thrift.TMasterOpResult;
 import org.apache.doris.thrift.TUniqueId;
 
-import com.google.common.collect.Lists;
 import com.google.common.base.Strings;
+import com.google.common.collect.Lists;
 
 import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.logging.log4j.LogManager;
@@ -236,7 +236,7 @@ public class ConnectProcessor {
             // Catch all throwable.
             // If reach here, maybe palo bug.
             LOG.warn("Process one query failed because unknown reason: ", e);
-            ctx.getState().setError("Unexpected exception: " + e.getMessage());
+            ctx.getState().setError(e.getClass().getSimpleName() + ", msg: " + e.getMessage());
             if (parsedStmt instanceof KillStmt) {
                 // ignore kill stmt execute err(not monitor it)
                 ctx.getState().setErrType(QueryState.ErrType.ANALYSIS_ERR);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java
index 4f38687..5073a8a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java
@@ -1021,12 +1021,25 @@ public class Coordinator {
 
             if (fragment.getDataPartition() == DataPartition.UNPARTITIONED) {
                 Reference<Long> backendIdRef = new Reference<Long>();
-                TNetworkAddress execHostport = SimpleScheduler.getHost(this.idToBackend, backendIdRef);
+                TNetworkAddress execHostport;
+                if (ConnectContext.get() != null && ConnectContext.get().isResourceTagsSet()
+                        && !addressToBackendID.isEmpty()) {
+                    // In this case, we only use the BE where the replica selected by the tag is located to execute this query.
+                    // Otherwise, except for the scan node, the rest of the execution nodes of the query can be executed on any BE.
+                    // addressToBackendID can be empty when this is a constant select stmt like:
+                    //      SELECT  @@session.auto_increment_increment AS auto_increment_increment;
+                    execHostport = SimpleScheduler.getHostByCurrentBackend(addressToBackendID);
+                } else {
+                    execHostport = SimpleScheduler.getHost(this.idToBackend, backendIdRef);
+                }
                 if (execHostport == null) {
                     LOG.warn("DataPartition UNPARTITIONED, no scanNode Backend");
                     throw new UserException("there is no scanNode Backend");
                 }
-                this.addressToBackendID.put(execHostport, backendIdRef.getRef());
+                if (backendIdRef.getRef() != null) {
+                    // backendIdRef can be null is we call getHostByCurrentBackend() before
+                    this.addressToBackendID.put(execHostport, backendIdRef.getRef());
+                }
                 FInstanceExecParam instanceParam = new FInstanceExecParam(null, execHostport,
                         0, params);
                 params.instanceExecParams.add(instanceParam);
@@ -1138,11 +1151,23 @@ public class Coordinator {
 
             if (params.instanceExecParams.isEmpty()) {
                 Reference<Long> backendIdRef = new Reference<Long>();
-                TNetworkAddress execHostport = SimpleScheduler.getHost(this.idToBackend, backendIdRef);
+                TNetworkAddress execHostport;
+                if (ConnectContext.get() != null && !ConnectContext.get().isResourceTagsSet() && !addressToBackendID.isEmpty()) {
+                    // In this case, we only use the BE where the replica selected by the tag is located to execute this query.
+                    // Otherwise, except for the scan node, the rest of the execution nodes of the query can be executed on any BE.
+                    // addressToBackendID can be empty when this is a constant select stmt like:
+                    //      SELECT  @@session.auto_increment_increment AS auto_increment_increment;
+                    execHostport = SimpleScheduler.getHostByCurrentBackend(addressToBackendID);
+                } else {
+                    execHostport = SimpleScheduler.getHost(this.idToBackend, backendIdRef);
+                }
                 if (execHostport == null) {
                     throw new UserException("there is no scanNode Backend");
                 }
-                this.addressToBackendID.put(execHostport, backendIdRef.getRef());
+                if (backendIdRef.getRef() != null) {
+                    // backendIdRef can be null is we call getHostByCurrentBackend() before
+                    this.addressToBackendID.put(execHostport, backendIdRef.getRef());
+                }
                 FInstanceExecParam instanceParam = new FInstanceExecParam(null, execHostport,
                         0, params);
                 params.instanceExecParams.add(instanceParam);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/DdlExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/DdlExecutor.java
index c4b165c..89536ee 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/DdlExecutor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/DdlExecutor.java
@@ -19,11 +19,13 @@ package org.apache.doris.qe;
 
 import org.apache.doris.analysis.AdminCancelRepairTableStmt;
 import org.apache.doris.analysis.AdminCheckTabletsStmt;
+import org.apache.doris.analysis.AdminCleanTrashStmt;
 import org.apache.doris.analysis.AdminRepairTableStmt;
 import org.apache.doris.analysis.AdminSetConfigStmt;
 import org.apache.doris.analysis.AdminSetReplicaStatusStmt;
 import org.apache.doris.analysis.AlterClusterStmt;
 import org.apache.doris.analysis.AlterColumnStatsStmt;
+import org.apache.doris.analysis.AlterDatabasePropertyStmt;
 import org.apache.doris.analysis.AlterDatabaseQuotaStmt;
 import org.apache.doris.analysis.AlterDatabaseRename;
 import org.apache.doris.analysis.AlterRoutineLoadStmt;
@@ -37,7 +39,6 @@ import org.apache.doris.analysis.CancelAlterSystemStmt;
 import org.apache.doris.analysis.CancelAlterTableStmt;
 import org.apache.doris.analysis.CancelBackupStmt;
 import org.apache.doris.analysis.CancelLoadStmt;
-import org.apache.doris.analysis.AdminCleanTrashStmt;
 import org.apache.doris.analysis.CreateClusterStmt;
 import org.apache.doris.analysis.CreateDataSyncJobStmt;
 import org.apache.doris.analysis.CreateDbStmt;
@@ -278,6 +279,8 @@ public class DdlExecutor {
             catalog.getSqlBlockRuleMgr().alterSqlBlockRule((AlterSqlBlockRuleStmt) ddlStmt);
         } else if (ddlStmt instanceof DropSqlBlockRuleStmt) {
             catalog.getSqlBlockRuleMgr().dropSqlBlockRule((DropSqlBlockRuleStmt) ddlStmt);
+        } else if (ddlStmt instanceof AlterDatabasePropertyStmt) {
+            throw new DdlException("Not implemented yet");
         } else {
             throw new DdlException("Unknown statement.");
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/InsertStreamTxnExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/InsertStreamTxnExecutor.java
index 9106006..85ddd1f 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/InsertStreamTxnExecutor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/InsertStreamTxnExecutor.java
@@ -62,8 +62,8 @@ public class InsertStreamTxnExecutor {
         StreamLoadTask streamLoadTask = StreamLoadTask.fromTStreamLoadPutRequest(request);
         StreamLoadPlanner planner = new StreamLoadPlanner(txnEntry.getDb(), (OlapTable) txnEntry.getTable(), streamLoadTask);
         TExecPlanFragmentParams tRequest = planner.plan(streamLoadTask.getId());
-        List<Long> beIds = Catalog.getCurrentSystemInfo().seqChooseBackendIds(
-                1, true, true, txnEntry.getDb().getClusterName());
+        List<Long> beIds = Catalog.getCurrentSystemInfo().seqChooseBackendIdsByStorageMediumAndTag(
+                1, true, false, txnEntry.getDb().getClusterName(), null, null);
         if (beIds == null || beIds.isEmpty()) {
             throw new UserException("there is no scanNode Backend.");
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/MultiLoadMgr.java b/fe/fe-core/src/main/java/org/apache/doris/qe/MultiLoadMgr.java
index bd78e0d..bb0f994 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/MultiLoadMgr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/MultiLoadMgr.java
@@ -19,13 +19,13 @@ package org.apache.doris.qe;
 
 import org.apache.doris.analysis.Analyzer;
 import org.apache.doris.analysis.BrokerDesc;
-import org.apache.doris.analysis.Separator;
 import org.apache.doris.analysis.DataDescription;
 import org.apache.doris.analysis.Expr;
 import org.apache.doris.analysis.ImportWhereStmt;
 import org.apache.doris.analysis.LabelName;
 import org.apache.doris.analysis.LoadStmt;
 import org.apache.doris.analysis.PartitionNames;
+import org.apache.doris.analysis.Separator;
 import org.apache.doris.analysis.SqlParser;
 import org.apache.doris.analysis.SqlScanner;
 import org.apache.doris.catalog.Catalog;
@@ -91,8 +91,8 @@ public class MultiLoadMgr {
                 throw new LabelAlreadyUsedException(label);
             }
             MultiLoadDesc multiLoadDesc = new MultiLoadDesc(multiLabel, properties);
-            List<Long> backendIds = Catalog.getCurrentSystemInfo().seqChooseBackendIds(1,
-                    true, false, ConnectContext.get().getClusterName());
+            List<Long> backendIds = Catalog.getCurrentSystemInfo().seqChooseBackendIdsByStorageMediumAndTag(1,
+                    true, false, ConnectContext.get().getClusterName(), null, null);
             if (backendIds == null) {
                 throw new DdlException("No backend alive.");
             }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java
index 06d4859..8778104 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/ShowExecutor.java
@@ -102,6 +102,7 @@ import org.apache.doris.catalog.MetadataViewer;
 import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.catalog.Partition;
 import org.apache.doris.catalog.Replica;
+import org.apache.doris.catalog.ReplicaAllocation;
 import org.apache.doris.catalog.ScalarType;
 import org.apache.doris.catalog.Table;
 import org.apache.doris.catalog.Tablet;
@@ -154,16 +155,16 @@ import org.apache.doris.system.SystemInfoService;
 import org.apache.doris.thrift.TUnit;
 import org.apache.doris.transaction.GlobalTransactionMgr;
 
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
-import org.apache.commons.lang3.tuple.Triple;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
-
 import java.io.BufferedReader;
 import java.io.InputStream;
 import java.io.InputStreamReader;
@@ -1812,8 +1813,10 @@ public class ShowExecutor {
                     }
                     DynamicPartitionProperty dynamicPartitionProperty = olapTable.getTableProperty().getDynamicPartitionProperty();
                     String tableName = olapTable.getName();
-                    int replicationNum = dynamicPartitionProperty.getReplicationNum();
-                    replicationNum = (replicationNum == DynamicPartitionProperty.NOT_SET_REPLICATION_NUM) ? olapTable.getDefaultReplicationNum() : replicationNum;
+                    ReplicaAllocation replicaAlloc = dynamicPartitionProperty.getReplicaAllocation();
+                    if (replicaAlloc.isNotSet()) {
+                        replicaAlloc = olapTable.getDefaultReplicaAllocation();
+                    }
                     rows.add(Lists.newArrayList(
                             tableName,
                             String.valueOf(dynamicPartitionProperty.getEnable()),
@@ -1822,7 +1825,8 @@ public class ShowExecutor {
                             String.valueOf(dynamicPartitionProperty.getEnd()),
                             dynamicPartitionProperty.getPrefix(),
                             String.valueOf(dynamicPartitionProperty.getBuckets()),
-                            String.valueOf(replicationNum),
+                            String.valueOf(replicaAlloc.getTotalReplicaNum()),
+                            replicaAlloc.toCreateStmt(),
                             dynamicPartitionProperty.getStartOfInfo(),
                             dynamicPartitionScheduler.getRuntimeInfo(olapTable.getId(), DynamicPartitionScheduler.LAST_UPDATE_TIME),
                             dynamicPartitionScheduler.getRuntimeInfo(olapTable.getId(), DynamicPartitionScheduler.LAST_SCHEDULER_TIME),
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SimpleScheduler.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SimpleScheduler.java
index 3943a13..3b5b8a0 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/SimpleScheduler.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SimpleScheduler.java
@@ -249,4 +249,16 @@ public class SimpleScheduler {
             }
         }
     }
+    
+    public static TNetworkAddress getHostByCurrentBackend(Map<TNetworkAddress, Long> addressToBackendID) {
+        int backendSize = addressToBackendID.size();
+        if (backendSize == 0) {
+            return null;
+        }
+        Long id = nextId.getAndIncrement() % backendSize;
+
+        List<TNetworkAddress> idToBackendId = Lists.newArrayList();
+        idToBackendId.addAll(addressToBackendID.keySet());
+        return idToBackendId.get(id.intValue());
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java
index 742672c..f242f4b 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java
@@ -405,7 +405,7 @@ public class StmtExecutor implements ProfileWriter {
             context.getState().setErrType(QueryState.ErrType.ANALYSIS_ERR);
         } catch (Exception e) {
             LOG.warn("execute Exception", e);
-            context.getState().setError(e.getMessage());
+            context.getState().setError(e.getClass().getSimpleName() + ", msg: " + e.getMessage());
             if (parsedStmt instanceof KillStmt) {
                 // ignore kill stmt execute err(not monitor it)
                 context.getState().setErrType(QueryState.ErrType.ANALYSIS_ERR);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/resource/Tag.java b/fe/fe-core/src/main/java/org/apache/doris/resource/Tag.java
index c01eb9d..0155480 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/resource/Tag.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/resource/Tag.java
@@ -61,6 +61,8 @@ public class Tag implements Writable {
     public static final String VALUE_STORE = "store";
     public static final String VALUE_COMPUTATION = "computation";
     public static final String VALUE_DEFAULT_CLUSTER = "default_cluster";
+    public static final String VALUE_DEFAULT_TAG = "default";
+    public static final String VALUE_INVALID_TAG = "invalid";
 
     public static final ImmutableSet<String> RESERVED_TAG_TYPE = ImmutableSet.of(
             TYPE_ROLE, TYPE_FUNCTION, TYPE_LOCATION);
@@ -69,6 +71,14 @@ public class Tag implements Writable {
             VALUE_DEFAULT_CLUSTER);
     private static final String TAG_REGEX = "^[a-z][a-z0-9_]{0,32}$";
 
+    public static final Tag DEFAULT_BACKEND_TAG;
+    public static final Tag INVALID_TAG;
+
+    static {
+        DEFAULT_BACKEND_TAG = new Tag(TYPE_LOCATION, VALUE_DEFAULT_TAG);
+        INVALID_TAG = new Tag(TYPE_LOCATION, VALUE_INVALID_TAG);
+    }
+
     @SerializedName(value = "type")
     public String type;
     @SerializedName(value = "value")
@@ -86,11 +96,15 @@ public class Tag implements Writable {
         return new Tag(type, value);
     }
 
+    public String toKey() {
+        return type + "_" + value;
+    }
+
     @Override
     public int hashCode() {
         return Objects.hash(type, value);
     }
-    
+
     @Override
     public boolean equals(Object other) {
         if (other == this) return true;
@@ -98,12 +112,12 @@ public class Tag implements Writable {
             return false;
         }
         Tag otherTag = (Tag) other;
-        return type.equalsIgnoreCase(otherTag.type) && value.equalsIgnoreCase(otherTag.value);
+        return type.equals(otherTag.type) && value.equals(otherTag.value);
     }
 
     @Override
     public String toString() {
-        return "{\"" + type.toString() + "\" : \"" + value + "\"}";
+        return "{\"" + type + "\" : \"" + value + "\"}";
     }
 
     @Override
diff --git a/fe/fe-core/src/main/java/org/apache/doris/system/Backend.java b/fe/fe-core/src/main/java/org/apache/doris/system/Backend.java
index 869323c..0c5d66d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/system/Backend.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/system/Backend.java
@@ -21,10 +21,12 @@ import org.apache.doris.alter.DecommissionBackendJob.DecommissionType;
 import org.apache.doris.catalog.Catalog;
 import org.apache.doris.catalog.DiskInfo;
 import org.apache.doris.catalog.DiskInfo.DiskState;
+import org.apache.doris.common.FeConstants;
 import org.apache.doris.common.FeMetaVersion;
 import org.apache.doris.common.io.Text;
 import org.apache.doris.common.io.Writable;
 import org.apache.doris.persist.gson.GsonUtils;
+import org.apache.doris.resource.Tag;
 import org.apache.doris.system.HeartbeatResponse.HbStatus;
 import org.apache.doris.thrift.TDisk;
 import org.apache.doris.thrift.TStorageMedium;
@@ -112,6 +114,8 @@ public class Backend implements Writable {
     // additional backendStatus information for BE, display in JSON format
     @SerializedName("backendStatus")
     private BackendStatus backendStatus = new BackendStatus();
+    @SerializedName("tag")
+    private Tag tag = Tag.DEFAULT_BACKEND_TAG;
 
     public Backend() {
         this.host = "";
@@ -128,7 +132,6 @@ public class Backend implements Writable {
 
         this.ownerClusterName = "";
         this.backendState = BackendState.free.ordinal();
-        
         this.decommissionType = DecommissionType.SystemDecommission.ordinal();
     }
 
@@ -628,17 +631,17 @@ public class Backend implements Writable {
                 this.version = hbResponse.getVersion();
             }
 
-            if (this.bePort != hbResponse.getBePort()) {
+            if (this.bePort != hbResponse.getBePort() && !FeConstants.runningUnitTest) {
                 isChanged = true;
                 this.bePort = hbResponse.getBePort();
             }
 
-            if (this.httpPort != hbResponse.getHttpPort()) {
+            if (this.httpPort != hbResponse.getHttpPort() && !FeConstants.runningUnitTest) {
                 isChanged = true;
                 this.httpPort = hbResponse.getHttpPort();
             }
 
-            if (this.brpcPort != hbResponse.getBrpcPort()) {
+            if (this.brpcPort != hbResponse.getBrpcPort() && !FeConstants.runningUnitTest) {
                 isChanged = true;
                 this.brpcPort = hbResponse.getBrpcPort();
             }
@@ -697,5 +700,13 @@ public class Backend implements Writable {
         // the last time when the stream load status was reported by backend
         public long lastStreamLoadTime = -1;
     }
+
+    public void setTag(Tag tag) {
+        this.tag = tag;
+    }
+
+    public Tag getTag() {
+        return tag;
+    }
 }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/system/HeartbeatMgr.java b/fe/fe-core/src/main/java/org/apache/doris/system/HeartbeatMgr.java
index 32bfd60..ec4ceb3 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/system/HeartbeatMgr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/system/HeartbeatMgr.java
@@ -44,6 +44,7 @@ import org.apache.doris.thrift.THeartbeatResult;
 import org.apache.doris.thrift.TMasterInfo;
 import org.apache.doris.thrift.TNetworkAddress;
 import org.apache.doris.thrift.TPaloBrokerService;
+import org.apache.doris.thrift.TStatus;
 import org.apache.doris.thrift.TStatusCode;
 
 import com.google.common.base.Strings;
@@ -212,17 +213,31 @@ public class HeartbeatMgr extends MasterDaemon {
         public HeartbeatResponse call() {
             long backendId = backend.getId();
             HeartbeatService.Client client = null;
+
             TNetworkAddress beAddr = new TNetworkAddress(backend.getHost(), backend.getHeartbeatPort());
             boolean ok = false;
             try {
-                client = ClientPool.backendHeartbeatPool.borrowObject(beAddr);
-
                 TMasterInfo copiedMasterInfo = new TMasterInfo(masterInfo.get());
                 copiedMasterInfo.setBackendIp(backend.getHost());
                 long flags = heartbeatFlags.getHeartbeatFlags();
                 copiedMasterInfo.setHeartbeatFlags(flags);
                 copiedMasterInfo.setBackendId(backendId);
-                THeartbeatResult result = client.heartbeat(copiedMasterInfo);
+                THeartbeatResult result;
+                if (!FeConstants.runningUnitTest) {
+                    client = ClientPool.backendHeartbeatPool.borrowObject(beAddr);
+                    result = client.heartbeat(copiedMasterInfo);
+                } else {
+                    // Mocked result
+                    TBackendInfo backendInfo = new TBackendInfo();
+                    backendInfo.setBePort(1);
+                    backendInfo.setHttpPort(2);
+                    backendInfo.setBeRpcPort(3);
+                    backendInfo.setBrpcPort(4);
+                    backendInfo.setVersion("test-1234");
+                    result = new THeartbeatResult();
+                    result.setStatus(new TStatus(TStatusCode.OK));
+                    result.setBackendInfo(backendInfo);
+                }
 
                 ok = true;
                 if (result.getStatus().getStatusCode() == TStatusCode.OK) {
@@ -249,10 +264,12 @@ public class HeartbeatMgr extends MasterDaemon {
                 return new BackendHbResponse(backendId,
                         Strings.isNullOrEmpty(e.getMessage()) ? "got exception" : e.getMessage());
             } finally {
-                if (ok) {
-                    ClientPool.backendHeartbeatPool.returnObject(beAddr, client);
-                } else {
-                    ClientPool.backendHeartbeatPool.invalidateObject(beAddr, client);
+                if (client != null) {
+                    if (ok) {
+                        ClientPool.backendHeartbeatPool.returnObject(beAddr, client);
+                    } else {
+                        ClientPool.backendHeartbeatPool.invalidateObject(beAddr, client);
+                    }
                 }
             }
         }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java b/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java
index 5e5fc15..67bcebb 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/system/SystemInfoService.java
@@ -17,9 +17,11 @@
 
 package org.apache.doris.system;
 
+import org.apache.doris.analysis.ModifyBackendClause;
 import org.apache.doris.catalog.Catalog;
 import org.apache.doris.catalog.Database;
 import org.apache.doris.catalog.DiskInfo;
+import org.apache.doris.catalog.ReplicaAllocation;
 import org.apache.doris.cluster.Cluster;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.DdlException;
@@ -27,11 +29,14 @@ import org.apache.doris.common.FeConstants;
 import org.apache.doris.common.FeMetaVersion;
 import org.apache.doris.common.Pair;
 import org.apache.doris.common.Status;
+import org.apache.doris.common.UserException;
 import org.apache.doris.common.io.CountingDataOutputStream;
 import org.apache.doris.metric.MetricRepo;
+import org.apache.doris.resource.Tag;
 import org.apache.doris.system.Backend.BackendState;
 import org.apache.doris.thrift.TStatusCode;
 import org.apache.doris.thrift.TStorageMedium;
+
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 import com.google.common.collect.ImmutableMap;
@@ -59,6 +64,7 @@ import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 public class SystemInfoService {
     private static final Logger LOG = LogManager.getLogger(SystemInfoService.class);
@@ -102,18 +108,18 @@ public class SystemInfoService {
     }
 
     // for deploy manager
-    public void addBackends(List<Pair<String, Integer>> hostPortPairs, boolean isFree) throws DdlException {
-        addBackends(hostPortPairs, isFree, "");
+    public void addBackends(List<Pair<String, Integer>> hostPortPairs, boolean isFree) throws UserException {
+        addBackends(hostPortPairs, isFree, "", Tag.DEFAULT_BACKEND_TAG);
     }
-    
+
     /**
      * @param hostPortPairs : backend's host and port
      * @param isFree : if true the backend is not owned by any cluster
      * @param destCluster : if not null or empty backend will be added to destCluster 
      * @throws DdlException
      */
-    public void addBackends(List<Pair<String, Integer>> hostPortPairs, 
-        boolean isFree, String destCluster) throws DdlException {
+    public void addBackends(List<Pair<String, Integer>> hostPortPairs,
+                            boolean isFree, String destCluster, Tag tag) throws UserException {
         for (Pair<String, Integer> pair : hostPortPairs) {
             // check is already exist
             if (getBackendWithHeartbeatPort(pair.first, pair.second) != null) {
@@ -122,7 +128,7 @@ public class SystemInfoService {
         }
 
         for (Pair<String, Integer> pair : hostPortPairs) {
-            addBackend(pair.first, pair.second, isFree, destCluster);
+            addBackend(pair.first, pair.second, isFree, destCluster, tag);
         }
     }
 
@@ -143,7 +149,8 @@ public class SystemInfoService {
     }
 
     // Final entry of adding backend
-    private void addBackend(String host, int heartbeatPort, boolean isFree, String destCluster) throws DdlException {
+    private void addBackend(String host, int heartbeatPort, boolean isFree, String destCluster,
+                            Tag tag) throws UserException {
         Backend newBackend = new Backend(Catalog.getCurrentCatalog().getNextId(), host, heartbeatPort);
         // update idToBackend
         Map<Long, Backend> copiedBackends = Maps.newHashMap(idToBackendRef);
@@ -158,7 +165,7 @@ public class SystemInfoService {
         idToReportVersionRef = newIdToReportVersion;
 
         if (!Strings.isNullOrEmpty(destCluster)) {
-         // add backend to destCluster
+            // add backend to destCluster
             setBackendOwner(newBackend, destCluster);
         } else if (!isFree) {
             // add backend to DEFAULT_CLUSTER
@@ -167,6 +174,9 @@ public class SystemInfoService {
             // backend is free
         }
 
+        // set tags
+        newBackend.setTag(tag);
+
         // log
         Catalog.getCurrentCatalog().getEditLog().logAddBackend(newBackend);
         LOG.info("finished to add {} ", newBackend);
@@ -726,15 +736,41 @@ public class SystemInfoService {
         return classMap;
     }
 
-    public List<Long> seqChooseBackendIdsByStorageMedium(int backendNum, boolean needAlive, boolean isCreate,
-                                                                      String clusterName, TStorageMedium storageMedium) {
-        final List<Backend> backends = getClusterBackends(clusterName).stream().filter(v -> !v.diskExceedLimitByStorageMedium(storageMedium)).collect(Collectors.toList());
-        return seqChooseBackendIds(backendNum, needAlive, isCreate, clusterName, backends);
+
+    // Find enough backend to allocate replica of a tablet.
+    // filters include: tag, cluster, storage medium
+    public Map<Tag, List<Long>> chooseBackendIdByFilters(ReplicaAllocation replicaAlloc, String clusterName, TStorageMedium storageMedium)
+            throws DdlException {
+        Map<Tag, List<Long>> chosenBackendIds = Maps.newHashMap();
+        Map<Tag, Short> allocMap = replicaAlloc.getAllocMap();
+        short totalReplicaNum = 0;
+        for (Map.Entry<Tag, Short> entry : allocMap.entrySet()) {
+            List<Long> beIds = Catalog.getCurrentSystemInfo().seqChooseBackendIdsByStorageMediumAndTag(entry.getValue(),
+                    true, true, clusterName, storageMedium, entry.getKey());
+            if (beIds == null) {
+                throw new DdlException("Failed to find enough host with storage medium and tag("
+                        + (storageMedium == null ? "NaN" : storageMedium) + "/" + entry.getKey()
+                        + ") in all backends. need: " + entry.getValue());
+            }
+            chosenBackendIds.put(entry.getKey(), beIds);
+            totalReplicaNum += beIds.size();
+        }
+        Preconditions.checkState(totalReplicaNum == replicaAlloc.getTotalReplicaNum());
+        return chosenBackendIds;
     }
 
-    public List<Long> seqChooseBackendIds(int backendNum, boolean needAlive, boolean isCreate,
-                                                       String clusterName) {
-        final List<Backend> backends = getClusterBackends(clusterName).stream().filter(v -> !v.diskExceedLimit()).collect(Collectors.toList());
+    public List<Long> seqChooseBackendIdsByStorageMediumAndTag(int backendNum, boolean needAlive, boolean isCreate,
+                                                               String clusterName, TStorageMedium storageMedium, Tag tag) {
+        Stream<Backend> beStream = getClusterBackends(clusterName).stream();
+        if (storageMedium == null) {
+            beStream = beStream.filter(v -> !v.diskExceedLimit());
+        } else {
+            beStream = beStream.filter(v -> !v.diskExceedLimitByStorageMedium(storageMedium));
+        }
+        if (tag != null) {
+            beStream = beStream.filter(v -> v.getTag().equals(tag));
+        }
+        final List<Backend> backends = beStream.collect(Collectors.toList());
         return seqChooseBackendIds(backendNum, needAlive, isCreate, clusterName, backends);
     }
 
@@ -770,8 +806,12 @@ public class SystemInfoService {
         }
 
         // host -> BE list
+        List<Backend> sourceBackend = srcBackends;
+        if (sourceBackend == null) {
+            sourceBackend = getClusterBackends(clusterName);
+        }
         Map<String, List<Backend>> backendMaps = Maps.newHashMap();
-        for (Backend backend : srcBackends) {
+        for (Backend backend : sourceBackend) {
             if (backendMaps.containsKey(backend.getHost())) {
                 backendMaps.get(backend.getHost()).add(backend);
             } else {
@@ -781,7 +821,6 @@ public class SystemInfoService {
             }
         }
 
-
         // if more than one backend exists in same host, select a backend at random
         List<Backend> backends = Lists.newArrayList();
         for (List<Backend> list : backendMaps.values()) {
@@ -1151,5 +1190,58 @@ public class SystemInfoService {
         pathHashToDishInfoRef = newPathInfos;
         LOG.debug("update path infos: {}", newPathInfos);
     }
+
+    public void modifyBackends(ModifyBackendClause alterClause) throws UserException {
+        List<Pair<String, Integer>> hostPortPairs = alterClause.getHostPortPairs();
+        List<Backend> backends = Lists.newArrayList();
+        for (Pair<String, Integer> pair : hostPortPairs) {
+            Backend be = getBackendWithHeartbeatPort(pair.first, pair.second);
+            if (be == null) {
+                throw new DdlException("backend does not exists[" + pair.first + ":" + pair.second + "]");
+            }
+            backends.add(be);
+        }
+
+        Tag tag = alterClause.getTag();
+        for (Backend be : backends) {
+            if (!be.getTag().equals(tag)) {
+                be.setTag(tag);
+                Catalog.getCurrentCatalog().getEditLog().logModifyBackend(be);
+                LOG.info("finished to modify backend {} ", be);
+            }
+        }
+    }
+
+    public void replayModifyBackend(Backend backend) {
+        Backend memBe = getBackend(backend.getId());
+        memBe.setTag(backend.getTag());
+        LOG.debug("replay modify backend: {}", backend);
+    }
+
+    // Check if there is enough suitable BE for replica allocation
+    public void checkReplicaAllocation(String cluster, ReplicaAllocation replicaAlloc) throws DdlException {
+        List<Backend> backends = getClusterBackends(cluster);
+        for (Map.Entry<Tag, Short> entry : replicaAlloc.getAllocMap().entrySet()) {
+            if (backends.stream().filter(b -> b.getTag().equals(entry.getKey())).count()
+                    < entry.getValue()) {
+                throw new DdlException("Failed to find enough host with tag(" + entry.getKey()
+                        + ") in all backends. need: " + entry.getValue());
+            }
+        }
+    }
+
+    public Set<Tag> getTagsByCluster(String clusterName) {
+        List<Backend> bes = getClusterBackends(clusterName);
+        Set<Tag> tags = Sets.newHashSet();
+        for (Backend be : bes) {
+            tags.add(be.getTag());
+        }
+        return tags;
+    }
+
+    public List<Backend> getBackendsByTagInCluster(String clusterName, Tag tag) {
+        List<Backend> bes = getClusterBackends(clusterName);
+        return bes.stream().filter(b -> b.getTag().equals(tag)).collect(Collectors.toList());
+    }
 }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/task/AgentBatchTask.java b/fe/fe-core/src/main/java/org/apache/doris/task/AgentBatchTask.java
index 36ca398..57c228c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/task/AgentBatchTask.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/task/AgentBatchTask.java
@@ -19,6 +19,7 @@ package org.apache.doris.task;
 
 import org.apache.doris.catalog.Catalog;
 import org.apache.doris.common.ClientPool;
+import org.apache.doris.common.FeConstants;
 import org.apache.doris.system.Backend;
 import org.apache.doris.thrift.BackendService;
 import org.apache.doris.thrift.TAgentServiceVersion;
@@ -157,7 +158,8 @@ public class AgentBatchTask implements Runnable {
                 }
                 List<AgentTask> tasks = this.backendIdToTasks.get(backendId);
                 // create AgentClient
-                address = new TNetworkAddress(backend.getHost(), backend.getBePort());
+                String host = FeConstants.runningUnitTest ? "127.0.0.1" : backend.getHost();
+                address = new TNetworkAddress(host, backend.getBePort());
                 client = ClientPool.backendPool.borrowObject(address);
                 List<TAgentTaskRequest> agentTaskRequests = new LinkedList<TAgentTaskRequest>();
                 for (AgentTask task : tasks) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java b/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java
index ee04b40..cdb73f4 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/transaction/DatabaseTransactionMgr.java
@@ -485,7 +485,7 @@ public class DatabaseTransactionMgr {
                     transactionState.prolongPublishTimeout();
                 }
 
-                int quorumReplicaNum = table.getPartitionInfo().getReplicationNum(partition.getId()) / 2 + 1;
+                int quorumReplicaNum = table.getPartitionInfo().getReplicaAllocation(partition.getId()).getTotalReplicaNum() / 2 + 1;
                 for (MaterializedIndex index : allIndices) {
                     for (Tablet tablet : index.getTablets()) {
                         int successReplicaNum = 0;
@@ -759,7 +759,7 @@ public class DatabaseTransactionMgr {
                         transactionState.setErrorMsg(errMsg);
                         return;
                     }
-                    int quorumReplicaNum = partitionInfo.getReplicationNum(partitionId) / 2 + 1;
+                    int quorumReplicaNum = partitionInfo.getReplicaAllocation(partitionId).getTotalReplicaNum() / 2 + 1;
 
                     List<MaterializedIndex> allIndices;
                     if (transactionState.getLoadedTblIndexes().isEmpty()) {
diff --git a/fe/fe-core/src/test/java/org/apache/doris/alter/AlterJobV2Test.java b/fe/fe-core/src/test/java/org/apache/doris/alter/AlterJobV2Test.java
index 3c524a1..af7ae27 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/alter/AlterJobV2Test.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/alter/AlterJobV2Test.java
@@ -54,7 +54,7 @@ public class AlterJobV2Test {
         FeConstants.default_scheduler_interval_millisecond = 1000;
         FeConstants.runningUnitTest = true;
 
-        UtFrameUtils.createMinDorisCluster(runningDir);
+        UtFrameUtils.createDorisCluster(runningDir);
         Config.enable_alpha_rowset = true;
 
         // create connect context
diff --git a/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java b/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java
index 2a77d2f..a2a0667 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java
@@ -39,14 +39,14 @@ import org.apache.doris.qe.ConnectContext;
 import org.apache.doris.thrift.TStorageMedium;
 import org.apache.doris.utframe.UtFrameUtils;
 
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-
 import java.io.File;
 import java.util.List;
 import java.util.Map;
@@ -65,7 +65,7 @@ public class AlterTest {
         Config.dynamic_partition_enable = true;
         Config.dynamic_partition_check_interval_seconds = 1;
         Config.disable_storage_medium_check = true;
-        UtFrameUtils.createMinDorisCluster(runningDir);
+        UtFrameUtils.createDorisCluster(runningDir);
 
         // create connect context
         connectContext = UtFrameUtils.createDefaultCtx();
@@ -316,28 +316,28 @@ public class AlterTest {
         alterTable(stmt, false);
 
         // set table's default replication num
-        Assert.assertEquals(Short.valueOf("1"), tbl.getDefaultReplicationNum());
+        Assert.assertEquals((short) 1, tbl.getDefaultReplicaAllocation().getTotalReplicaNum());
         stmt = "alter table test.tbl1 set ('default.replication_num' = '3');";
         alterTable(stmt, false);
-        Assert.assertEquals(Short.valueOf("3"), tbl.getDefaultReplicationNum());
+        Assert.assertEquals((short) 3, tbl.getDefaultReplicaAllocation().getTotalReplicaNum());
 
         // set range table's real replication num
         Partition p1 = tbl.getPartition("p1");
-        Assert.assertEquals(Short.valueOf("1"), Short.valueOf(tbl.getPartitionInfo().getReplicationNum(p1.getId())));
+        Assert.assertEquals(Short.valueOf("1"), Short.valueOf(tbl.getPartitionInfo().getReplicaAllocation(p1.getId()).getTotalReplicaNum()));
         stmt = "alter table test.tbl1 set ('replication_num' = '3');";
         alterTable(stmt, true);
-        Assert.assertEquals(Short.valueOf("1"), Short.valueOf(tbl.getPartitionInfo().getReplicationNum(p1.getId())));
+        Assert.assertEquals(Short.valueOf("1"), Short.valueOf(tbl.getPartitionInfo().getReplicaAllocation(p1.getId()).getTotalReplicaNum()));
 
         // set un-partitioned table's real replication num
         OlapTable tbl2 = (OlapTable) db.getTableOrMetaException("tbl2");
         Partition partition = tbl2.getPartition(tbl2.getName());
-        Assert.assertEquals(Short.valueOf("1"), Short.valueOf(tbl2.getPartitionInfo().getReplicationNum(partition.getId())));
+        Assert.assertEquals(Short.valueOf("1"), Short.valueOf(tbl2.getPartitionInfo().getReplicaAllocation(partition.getId()).getTotalReplicaNum()));
         stmt = "alter table test.tbl2 set ('replication_num' = '3');";
-        alterTable(stmt, false);
-        Assert.assertEquals(Short.valueOf("3"), Short.valueOf(tbl2.getPartitionInfo().getReplicationNum(partition.getId())));
+        alterTable(stmt, true);
+        // Assert.assertEquals(Short.valueOf("3"), Short.valueOf(tbl2.getPartitionInfo().getReplicaAllocation(partition.getId()).getTotalReplicaNum()));
 
         Thread.sleep(5000); // sleep to wait dynamic partition scheduler run
-        // add partition without set replication num
+        // add partition without set replication num, and default num is 3.
         stmt = "alter table test.tbl1 add partition p4 values less than('2020-04-10')";
         alterTable(stmt, true);
 
@@ -357,16 +357,16 @@ public class AlterTest {
         Partition p4 = tbl4.getPartition("p4");
 
         // batch update replication_num property
-        String stmt = "alter table test.tbl4 modify partition (p1, p2, p4) set ('replication_num' = '3')";
+        String stmt = "alter table test.tbl4 modify partition (p1, p2, p4) set ('replication_num' = '1')";
         List<Partition> partitionList = Lists.newArrayList(p1, p2, p4);
         for (Partition partition : partitionList) {
-            Assert.assertEquals(Short.valueOf("1"), Short.valueOf(tbl4.getPartitionInfo().getReplicationNum(partition.getId())));
+            Assert.assertEquals(Short.valueOf("1"), Short.valueOf(tbl4.getPartitionInfo().getReplicaAllocation(partition.getId()).getTotalReplicaNum()));
         }
         alterTable(stmt, false);
         for (Partition partition : partitionList) {
-            Assert.assertEquals(Short.valueOf("3"), Short.valueOf(tbl4.getPartitionInfo().getReplicationNum(partition.getId())));
+            Assert.assertEquals(Short.valueOf("1"), Short.valueOf(tbl4.getPartitionInfo().getReplicaAllocation(partition.getId()).getTotalReplicaNum()));
         }
-        Assert.assertEquals(Short.valueOf("1"), Short.valueOf(tbl4.getPartitionInfo().getReplicationNum(p3.getId())));
+        Assert.assertEquals(Short.valueOf("1"), Short.valueOf(tbl4.getPartitionInfo().getReplicaAllocation(p3.getId()).getTotalReplicaNum()));
 
         // batch update in_memory property
         stmt = "alter table test.tbl4 modify partition (p1, p2, p3) set ('in_memory' = 'true')";
@@ -401,7 +401,7 @@ public class AlterTest {
         partitionList = Lists.newArrayList(p1, p2, p3, p4);
         alterTable(stmt, false);
         for (Partition partition : partitionList) {
-            Assert.assertEquals(Short.valueOf("1"), Short.valueOf(tbl4.getPartitionInfo().getReplicationNum(partition.getId())));
+            Assert.assertEquals(Short.valueOf("1"), Short.valueOf(tbl4.getPartitionInfo().getReplicaAllocation(partition.getId()).getTotalReplicaNum()));
         }
     }
 
diff --git a/fe/fe-core/src/test/java/org/apache/doris/alter/BatchRollupJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/alter/BatchRollupJobTest.java
index 2ccd1bc..b120778 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/alter/BatchRollupJobTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/alter/BatchRollupJobTest.java
@@ -49,7 +49,7 @@ public class BatchRollupJobTest {
 
     @BeforeClass
     public static void setup() throws Exception {
-        UtFrameUtils.createMinDorisCluster(runningDir);
+        UtFrameUtils.createDorisCluster(runningDir);
         ctx = UtFrameUtils.createDefaultCtx();
     }
 
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/AdminSetConfigStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/AdminSetConfigStmtTest.java
index 83e0ef3..f6ce740 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/AdminSetConfigStmtTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/AdminSetConfigStmtTest.java
@@ -40,7 +40,7 @@ public class AdminSetConfigStmtTest {
 
     @BeforeClass
     public static void beforeClass() throws Exception {
-        UtFrameUtils.createMinDorisCluster(runningDir);
+        UtFrameUtils.createDorisCluster(runningDir);
 
         // create connect context
         connectContext = UtFrameUtils.createDefaultCtx();
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/AdminShowReplicaTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/AdminShowReplicaTest.java
index c67cf8c..efe5a5d 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/AdminShowReplicaTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/AdminShowReplicaTest.java
@@ -50,7 +50,7 @@ public class AdminShowReplicaTest {
 
     @BeforeClass
     public static void beforeClass() throws Exception {
-        UtFrameUtils.createMinDorisCluster(runningDir);
+        UtFrameUtils.createDorisCluster(runningDir);
 
         // create connect context
         connectContext = UtFrameUtils.createDefaultCtx();
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/AggregateTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/AggregateTest.java
index 753e255..3f0028c 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/AggregateTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/AggregateTest.java
@@ -42,7 +42,7 @@ public class AggregateTest {
     @BeforeClass
     public static void beforeClass() throws Exception{
         FeConstants.runningUnitTest = true;
-        UtFrameUtils.createMinDorisCluster(runningDir);
+        UtFrameUtils.createDorisCluster(runningDir);
         dorisAssert = new DorisAssert();
         dorisAssert.withDatabase(DB_NAME).useDatabase(DB_NAME);
         String createTableSQL = "create table " + DB_NAME + "." + TABLE_NAME + " (empid int, name varchar, " +
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/InsertStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/InsertStmtTest.java
index 095b882..9cab5b4 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/InsertStmtTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/InsertStmtTest.java
@@ -55,7 +55,7 @@ public class InsertStmtTest {
 
     @BeforeClass
     public static void setUp() throws Exception {
-        UtFrameUtils.createMinDorisCluster(runningDir);
+        UtFrameUtils.createDorisCluster(runningDir);
         String createTblStmtStr = "create table db.tbl(kk1 int, kk2 varchar(32), kk3 int, kk4 int) "
                 + "AGGREGATE KEY(kk1, kk2,kk3,kk4) distributed by hash(kk1) buckets 3 properties('replication_num' = '1');";
         dorisAssert = new DorisAssert();
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
index 4c4154f..b93cf41 100644
--- 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
@@ -46,7 +46,7 @@ public class ListPartitionPrunerTest {
     public static void setUp() throws Exception {
         Config.enable_batch_delete_by_default = true;
         FeConstants.runningUnitTest = true;
-        UtFrameUtils.createMinDorisCluster(runningDir);
+        UtFrameUtils.createDorisCluster(runningDir);
 
         String createSinglePartColWithSinglePartKey = "create table test.t1\n"
                 + "(k1 int not null, k2 varchar(128), k3 int, v1 int, v2 int)\n"
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/QueryStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/QueryStmtTest.java
index c405fbb..19d9e53 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/QueryStmtTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/QueryStmtTest.java
@@ -17,7 +17,6 @@
 
 package org.apache.doris.analysis;
 
-import com.google.common.collect.Lists;
 import org.apache.doris.catalog.Type;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.Config;
@@ -27,6 +26,9 @@ import org.apache.doris.rewrite.FoldConstantsRule;
 import org.apache.doris.thrift.TExpr;
 import org.apache.doris.utframe.DorisAssert;
 import org.apache.doris.utframe.UtFrameUtils;
+
+import com.google.common.collect.Lists;
+
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -49,7 +51,7 @@ public class QueryStmtTest {
     @BeforeClass
     public static void setUp() throws Exception {
         Config.enable_batch_delete_by_default = true;
-        UtFrameUtils.createMinDorisCluster(runningDir);
+        UtFrameUtils.createDorisCluster(runningDir);
         String createTblStmtStr = "create table db1.tbl1(k1 varchar(32), k2 varchar(32), k3 varchar(32), k4 int) "
                 + "AGGREGATE KEY(k1, k2,k3,k4) distributed by hash(k1) buckets 3 properties('replication_num' = '1');";
         String createBaseAllStmtStr = "create table db1.baseall(k1 int, k2 varchar(32)) distributed by hash(k1) "
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/SelectStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/SelectStmtTest.java
index 07b8ae1..018adad 100755
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/SelectStmtTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/SelectStmtTest.java
@@ -57,7 +57,7 @@ public class SelectStmtTest {
     public static void setUp() throws Exception {
         Config.enable_batch_delete_by_default = true;
         Config.enable_http_server_v2 = false;
-        UtFrameUtils.createMinDorisCluster(runningDir);
+        UtFrameUtils.createDorisCluster(runningDir);
         String createTblStmtStr = "create table db1.tbl1(k1 varchar(32), k2 varchar(32), k3 varchar(32), k4 int, k5 largeint) "
                 + "AGGREGATE KEY(k1, k2,k3,k4,k5) distributed by hash(k1) buckets 3 properties('replication_num' = '1');";
         String createBaseAllStmtStr = "create table db1.baseall(k1 int, k2 varchar(32)) distributed by hash(k1) "
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/SetVariableTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/SetVariableTest.java
index 4652348..f27c227 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/SetVariableTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/SetVariableTest.java
@@ -38,7 +38,7 @@ public class SetVariableTest {
 
     @BeforeClass
     public static void beforeClass() throws Exception {
-        UtFrameUtils.createMinDorisCluster(runningDir);
+        UtFrameUtils.createDorisCluster(runningDir);
         // create connect context
         connectContext = UtFrameUtils.createDefaultCtx();
     }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowViewStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowViewStmtTest.java
index 479727d..87efc37 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowViewStmtTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/ShowViewStmtTest.java
@@ -7,6 +7,7 @@ import org.apache.doris.qe.ShowExecutor;
 import org.apache.doris.qe.ShowResultSet;
 import org.apache.doris.utframe.DorisAssert;
 import org.apache.doris.utframe.UtFrameUtils;
+
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 
@@ -31,7 +32,7 @@ public class ShowViewStmtTest {
 
     @BeforeClass
     public static void setUp() throws Exception {
-        UtFrameUtils.createMinDorisCluster(runningDir);
+        UtFrameUtils.createDorisCluster(runningDir);
         String testTbl1 = "CREATE TABLE `test1` (\n" +
                 "  `a` int(11) NOT NULL COMMENT \"\",\n" +
                 "  `b` int(11) NOT NULL COMMENT \"\"\n" +
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/StmtRewriterTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/StmtRewriterTest.java
index 039a190..6ee7e44 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/StmtRewriterTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/StmtRewriterTest.java
@@ -42,7 +42,7 @@ public class StmtRewriterTest {
     @BeforeClass
     public static void beforeClass() throws Exception{
         FeConstants.runningUnitTest = true;
-        UtFrameUtils.createMinDorisCluster(runningDir);
+        UtFrameUtils.createDorisCluster(runningDir);
         dorisAssert = new DorisAssert();
         dorisAssert.withDatabase(DB_NAME).useDatabase(DB_NAME);
         String createTableSQL = "create table " + DB_NAME + "." + TABLE_NAME + " (empid int, name varchar, " +
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/TableNameComparedLowercaseTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/TableNameComparedLowercaseTest.java
index bee5637..b8408ee 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/TableNameComparedLowercaseTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/TableNameComparedLowercaseTest.java
@@ -51,7 +51,7 @@ public class TableNameComparedLowercaseTest {
         Config.lower_case_table_names = 2;
         Config.enable_batch_delete_by_default = true;
         Config.enable_http_server_v2 = false;
-        UtFrameUtils.createMinDorisCluster(runningDir);
+        UtFrameUtils.createDorisCluster(runningDir);
         String table1 = "CREATE TABLE db1.TABLE1 (\n" +
                 "  `siteid` int(11) NULL DEFAULT \"10\" COMMENT \"\",\n" +
                 "  `citycode` smallint(6) NULL COMMENT \"\",\n" +
diff --git a/fe/fe-core/src/test/java/org/apache/doris/analysis/TableNameStoredLowercaseTest.java b/fe/fe-core/src/test/java/org/apache/doris/analysis/TableNameStoredLowercaseTest.java
index 3bf64fa..2ad2d08 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/analysis/TableNameStoredLowercaseTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/analysis/TableNameStoredLowercaseTest.java
@@ -50,7 +50,7 @@ public class TableNameStoredLowercaseTest {
         Config.lower_case_table_names = 1;
         Config.enable_batch_delete_by_default = true;
         Config.enable_http_server_v2 = false;
-        UtFrameUtils.createMinDorisCluster(runningDir);
+        UtFrameUtils.createDorisCluster(runningDir);
         String table1 = "CREATE TABLE db1.TABLE1 (\n" +
                 "  `siteid` int(11) NULL DEFAULT \"10\" COMMENT \"\",\n" +
                 "  `citycode` smallint(6) NULL COMMENT \"\",\n" +
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 230bdab..d4cf23d 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
@@ -41,6 +41,7 @@ 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.ReplicaAllocation;
 import org.apache.doris.catalog.ScalarType;
 import org.apache.doris.catalog.SinglePartitionInfo;
 import org.apache.doris.catalog.Tablet;
@@ -238,7 +239,7 @@ public class CatalogMocker {
         Partition partition =
                 new Partition(TEST_SINGLE_PARTITION_ID, TEST_SINGLE_PARTITION_NAME, baseIndex, distributionInfo);
         PartitionInfo partitionInfo = new SinglePartitionInfo();
-        partitionInfo.setReplicationNum(TEST_SINGLE_PARTITION_ID, (short) 3);
+        partitionInfo.setReplicaAllocation(TEST_SINGLE_PARTITION_ID, new ReplicaAllocation((short) 3));
         partitionInfo.setIsInMemory(TEST_SINGLE_PARTITION_ID, false);
         DataProperty dataProperty = new DataProperty(TStorageMedium.HDD);
         partitionInfo.setDataProperty(TEST_SINGLE_PARTITION_ID, dataProperty);
@@ -301,23 +302,23 @@ public class CatalogMocker {
 
         PartitionKey rangeP2Lower =
                 PartitionKey.createPartitionKey(Lists.newArrayList(new PartitionValue("10")),
-                                                Lists.newArrayList(TEST_TBL_BASE_SCHEMA.get(0)));
+                        Lists.newArrayList(TEST_TBL_BASE_SCHEMA.get(0)));
         PartitionKey rangeP2Upper =
                 PartitionKey.createPartitionKey(Lists.newArrayList(new PartitionValue("20")),
-                                                Lists.newArrayList(TEST_TBL_BASE_SCHEMA.get(0)));
+                        Lists.newArrayList(TEST_TBL_BASE_SCHEMA.get(0)));
         Range<PartitionKey> rangeP2 = Range.closedOpen(rangeP2Lower, rangeP2Upper);
         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);
+        rangePartitionInfo.setReplicaAllocation(TEST_PARTITION1_ID, new ReplicaAllocation((short) 3));
+        rangePartitionInfo.setReplicaAllocation(TEST_PARTITION2_ID, new ReplicaAllocation((short) 3));
         DataProperty dataPropertyP1 = new DataProperty(TStorageMedium.HDD);
         DataProperty dataPropertyP2 = new DataProperty(TStorageMedium.HDD);
         rangePartitionInfo.setDataProperty(TEST_PARTITION1_ID, dataPropertyP1);
         rangePartitionInfo.setDataProperty(TEST_PARTITION2_ID, dataPropertyP2);
 
         OlapTable olapTable2 = new OlapTable(TEST_TBL2_ID, TEST_TBL2_NAME, TEST_TBL_BASE_SCHEMA,
-                                             KeysType.AGG_KEYS, rangePartitionInfo, distributionInfo2);
+                KeysType.AGG_KEYS, rangePartitionInfo, distributionInfo2);
         Deencapsulation.setField(olapTable2, "baseIndexId", TEST_TBL2_ID);
 
         Tablet baseTabletP1 = new Tablet(TEST_BASE_TABLET_P1_ID);
diff --git a/fe/fe-core/src/test/java/org/apache/doris/backup/RepositoryTest.java b/fe/fe-core/src/test/java/org/apache/doris/backup/RepositoryTest.java
index 1ed18d7..f750ee6 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/backup/RepositoryTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/backup/RepositoryTest.java
@@ -39,6 +39,8 @@ import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.PrintWriter;
+import java.net.MalformedURLException;
+import java.net.URISyntaxException;
 import java.sql.Timestamp;
 import java.util.List;
 import java.util.Map;
@@ -127,7 +129,7 @@ public class RepositoryTest {
     }
 
     @Test
-    public void testassemnblePath() {
+    public void testassemnblePath() throws MalformedURLException, URISyntaxException {
         repo = new Repository(10000, "repo", false, location, storage);
 
         // job info
@@ -136,7 +138,7 @@ public class RepositoryTest {
         String createTime2 = "2018-04-12-20-46-45";
         Timestamp ts = Timestamp.valueOf(createTime);
         long creastTs = ts.getTime();
-        
+
         // "location/__palo_repository_repo_name/__ss_my_sp1/__info_2018-01-01-08-00-00"
         String expected = location + "/" + Repository.PREFIX_REPO + name + "/" + Repository.PREFIX_SNAPSHOT_DIR
                 + label + "/" + Repository.PREFIX_JOB_INFO + createTime2;
@@ -146,12 +148,17 @@ public class RepositoryTest {
         expected = location + "/" + Repository.PREFIX_REPO + name + "/" + Repository.PREFIX_SNAPSHOT_DIR
                 + label + "/" + Repository.FILE_META_INFO;
         Assert.assertEquals(expected, repo.assembleMetaInfoFilePath(label));
-        
+
         // snapshot path
         // /location/__palo_repository_repo_name/__ss_my_ss1/__ss_content/__db_10001/__tbl_10020/__part_10031/__idx_10032/__10023/__3481721
         expected = location + "/" + Repository.PREFIX_REPO + name + "/" + Repository.PREFIX_SNAPSHOT_DIR
                 + label + "/" + "__ss_content/__db_1/__tbl_2/__part_3/__idx_4/__5/__7";
         Assert.assertEquals(expected, repo.assembleRemoteSnapshotPath(label, info));
+
+        String rootTabletPath = "/__db_10000/__tbl_10001/__part_10002/_idx_10001/__10003";
+        String path = repo.getRepoPath(label, rootTabletPath);
+        Assert.assertEquals("bos://backup-cmy/__palo_repository_repo/__ss_label/__ss_content/__db_10000/__tbl_10001/__part_10002/_idx_10001/__10003",
+                path);
     }
 
     @Test
diff --git a/fe/fe-core/src/test/java/org/apache/doris/backup/RestoreJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/backup/RestoreJobTest.java
index dd70058..da8f479 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/backup/RestoreJobTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/backup/RestoreJobTest.java
@@ -19,8 +19,8 @@ package org.apache.doris.backup;
 
 import org.apache.doris.analysis.StorageBackend;
 import org.apache.doris.backup.BackupJobInfo.BackupIndexInfo;
-import org.apache.doris.backup.BackupJobInfo.BackupPartitionInfo;
 import org.apache.doris.backup.BackupJobInfo.BackupOlapTableInfo;
+import org.apache.doris.backup.BackupJobInfo.BackupPartitionInfo;
 import org.apache.doris.backup.BackupJobInfo.BackupTabletInfo;
 import org.apache.doris.backup.RestoreJob.RestoreJobState;
 import org.apache.doris.catalog.Catalog;
@@ -29,6 +29,7 @@ import org.apache.doris.catalog.MaterializedIndex;
 import org.apache.doris.catalog.MaterializedIndex.IndexExtState;
 import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.catalog.Partition;
+import org.apache.doris.catalog.ReplicaAllocation;
 import org.apache.doris.catalog.Resource;
 import org.apache.doris.catalog.Table;
 import org.apache.doris.catalog.Tablet;
@@ -37,6 +38,7 @@ import org.apache.doris.common.FeConstants;
 import org.apache.doris.common.MarkedCountDownLatch;
 import org.apache.doris.common.jmockit.Deencapsulation;
 import org.apache.doris.persist.EditLog;
+import org.apache.doris.resource.Tag;
 import org.apache.doris.system.SystemInfoService;
 import org.apache.doris.task.AgentTask;
 import org.apache.doris.task.AgentTaskQueue;
@@ -47,6 +49,7 @@ import org.apache.doris.thrift.TBackend;
 import org.apache.doris.thrift.TFinishTaskRequest;
 import org.apache.doris.thrift.TStatus;
 import org.apache.doris.thrift.TStatusCode;
+import org.apache.doris.thrift.TStorageMedium;
 import org.apache.doris.thrift.TTaskType;
 
 import com.google.common.collect.Lists;
@@ -158,11 +161,12 @@ public class RestoreJobTest {
         
         new Expectations() {
             {
-                systemInfoService.seqChooseBackendIds(anyInt, anyBoolean, anyBoolean, anyString);
+                systemInfoService.seqChooseBackendIdsByStorageMediumAndTag(anyInt, anyBoolean, anyBoolean, anyString,
+                        (TStorageMedium) any, (Tag) any);
                 minTimes = 0;
                 result = new Delegate() {
                     public synchronized List<Long> seqChooseBackendIds(int backendNum, boolean needAlive,
-                            boolean isCreate, String clusterName) {
+                                                                       boolean isCreate, String clusterName) {
                         List<Long> beIds = Lists.newArrayList();
                         beIds.add(CatalogMocker.BACKEND1_ID);
                         beIds.add(CatalogMocker.BACKEND2_ID);
@@ -245,9 +249,9 @@ public class RestoreJobTest {
         
         // drop this table, cause we want to try restoring this table
         db.dropTable(expectedRestoreTbl.getName());
-        
+
         job = new RestoreJob(label, "2018-01-01 01:01:01", db.getId(), db.getFullName(),
-                jobInfo, false, 3, 100000, -1, catalog, repo.getId());
+                jobInfo, false, new ReplicaAllocation((short) 3), 100000, -1, catalog, repo.getId());
         
         List<Table> tbls = Lists.newArrayList();
         List<Resource> resources = Lists.newArrayList();
diff --git a/fe/fe-core/src/test/java/org/apache/doris/blockrule/SqlBlockRuleMgrTest.java b/fe/fe-core/src/test/java/org/apache/doris/blockrule/SqlBlockRuleMgrTest.java
index 8fd7af2..fdd977d 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/blockrule/SqlBlockRuleMgrTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/blockrule/SqlBlockRuleMgrTest.java
@@ -44,7 +44,7 @@ public class SqlBlockRuleMgrTest {
     
     @BeforeClass
     public static void beforeClass() throws Exception {
-        UtFrameUtils.createMinDorisCluster(runningDir);
+        UtFrameUtils.createDorisCluster(runningDir);
         
         // create connect context
         connectContext = UtFrameUtils.createDefaultCtx();
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/AdminStmtTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/AdminStmtTest.java
index 8f8b603..aaa567d 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/AdminStmtTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/AdminStmtTest.java
@@ -54,7 +54,7 @@ public class AdminStmtTest {
 
     @BeforeClass
     public static void beforeClass() throws Exception {
-        UtFrameUtils.createMinDorisCluster(runningDir);
+        UtFrameUtils.createDorisCluster(runningDir);
 
         // create connect context
         connectContext = UtFrameUtils.createDefaultCtx();
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/CatalogOperationTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/CatalogOperationTest.java
index 1d54545..e8001d3 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/CatalogOperationTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/CatalogOperationTest.java
@@ -47,7 +47,7 @@ public class CatalogOperationTest {
     @BeforeClass
     public static void beforeClass() throws Exception {
         FeConstants.default_scheduler_interval_millisecond = 1000;
-        UtFrameUtils.createMinDorisCluster(runningDir);
+        UtFrameUtils.createDorisCluster(runningDir);
 
         // create connect context
         connectContext = UtFrameUtils.createDefaultCtx();
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 8e2eb77..b540ec6 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
@@ -215,7 +215,7 @@ public class CatalogTestUtil {
         // table
         PartitionInfo partitionInfo = new SinglePartitionInfo();
         partitionInfo.setDataProperty(partitionId, DataProperty.DEFAULT_DATA_PROPERTY);
-        partitionInfo.setReplicationNum(partitionId, (short) 3);
+        partitionInfo.setReplicaAllocation(partitionId, new ReplicaAllocation((short) 3));
         OlapTable table = new OlapTable(tableId, testTable1, columns, KeysType.AGG_KEYS, partitionInfo,
                 distributionInfo);
         table.addPartition(partition);
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/ColocateTableIndexTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/ColocateTableIndexTest.java
index 1867133..e03024c 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/ColocateTableIndexTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/ColocateTableIndexTest.java
@@ -18,12 +18,19 @@
 package org.apache.doris.catalog;
 
 import org.apache.doris.catalog.ColocateTableIndex.GroupId;
+import org.apache.doris.common.FeMetaVersion;
+import org.apache.doris.meta.MetaContext;
 
 import com.google.common.collect.Maps;
 
 import org.junit.Assert;
 import org.junit.Test;
 
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.CopyOnWriteArraySet;
@@ -46,4 +53,31 @@ public class ColocateTableIndexTest {
         balancingGroups.remove(groupId2);
         Assert.assertTrue(balancingGroups.isEmpty());
     }
+
+    @Test
+    public void testSerialization() throws Exception {
+        MetaContext metaContext = new MetaContext();
+        metaContext.setMetaVersion(FeMetaVersion.VERSION_CURRENT);
+        metaContext.setThreadLocalInfo();
+
+        // 1. Write objects to file
+        File file = new File("./GroupIdTest");
+        file.createNewFile();
+        DataOutputStream dos = new DataOutputStream(new FileOutputStream(file));
+
+        ColocateTableIndex.GroupId groupId = new ColocateTableIndex.GroupId(1, 2);
+        groupId.write(dos);
+        dos.flush();
+        dos.close();
+
+        // 2. Read objects from file
+        DataInputStream dis = new DataInputStream(new FileInputStream(file));
+
+        ColocateTableIndex.GroupId rGroupId = ColocateTableIndex.GroupId.read(dis);
+        Assert.assertTrue(groupId.equals(rGroupId));
+
+        // 3. delete files
+        dis.close();
+        file.delete();
+    }
 }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/ColocateTableTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/ColocateTableTest.java
index a4afb90..1b5d632 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/ColocateTableTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/ColocateTableTest.java
@@ -24,10 +24,9 @@ import org.apache.doris.catalog.ColocateTableIndex.GroupId;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.common.jmockit.Deencapsulation;
 import org.apache.doris.qe.ConnectContext;
-
-import com.google.common.collect.Multimap;
-
+import org.apache.doris.resource.Tag;
 import org.apache.doris.utframe.UtFrameUtils;
+
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -37,8 +36,10 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
 
+import com.google.common.collect.Multimap;
+import com.google.common.collect.Table;
+
 import java.io.File;
-import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.UUID;
@@ -58,7 +59,7 @@ public class ColocateTableTest {
 
     @BeforeClass
     public static void beforeClass() throws Exception {
-        UtFrameUtils.createMinDorisCluster(runningDir);
+        UtFrameUtils.createDorisCluster(runningDir);
         connectContext = UtFrameUtils.createDefaultCtx();
 
     }
@@ -110,7 +111,7 @@ public class ColocateTableTest {
         Assert.assertEquals(1, Deencapsulation.<Multimap<GroupId, Long>>getField(index, "group2Tables").size());
         Assert.assertEquals(1, index.getAllGroupIds().size());
         Assert.assertEquals(1, Deencapsulation.<Map<Long, GroupId>>getField(index, "table2Group").size());
-        Assert.assertEquals(1, Deencapsulation.<Map<GroupId, List<List<Long>>>>getField(index, "group2BackendsPerBucketSeq").size());
+        Assert.assertEquals(1, Deencapsulation.<Table<GroupId, Tag, List<List<Long>>>>getField(index, "group2BackendsPerBucketSeq").size());
         Assert.assertEquals(1, Deencapsulation.<Map<GroupId, ColocateGroupSchema>>getField(index, "group2Schema").size());
         Assert.assertEquals(0, index.getUnstableGroupIds().size());
 
@@ -120,8 +121,8 @@ public class ColocateTableTest {
         Assert.assertEquals(dbId, index.getGroup(tableId).dbId);
 
         GroupId groupId = index.getGroup(tableId);
-        List<Long> backendIds = index.getBackendsPerBucketSeq(groupId).get(0);
-        Assert.assertEquals(1, backendIds.size());
+        Map<Tag, List<List<Long>>> backendIds = index.getBackendsPerBucketSeq(groupId);
+        Assert.assertEquals(1, backendIds.get(Tag.DEFAULT_BACKEND_TAG).get(0).size());
 
         String fullGroupName = dbId + "_" + groupName;
         Assert.assertEquals(tableId, index.getTableIdByGroup(fullGroupName));
@@ -129,7 +130,7 @@ public class ColocateTableTest {
         Assert.assertNotNull(groupSchema);
         Assert.assertEquals(dbId, groupSchema.getGroupId().dbId);
         Assert.assertEquals(1, groupSchema.getBucketsNum());
-        Assert.assertEquals(1, groupSchema.getReplicationNum());
+        Assert.assertEquals((short) 1, groupSchema.getReplicaAlloc().getTotalReplicaNum());
     }
 
     @Test
@@ -166,7 +167,7 @@ public class ColocateTableTest {
         Assert.assertEquals(2, Deencapsulation.<Multimap<GroupId, Long>>getField(index, "group2Tables").size());
         Assert.assertEquals(1, index.getAllGroupIds().size());
         Assert.assertEquals(2, Deencapsulation.<Map<Long, GroupId>>getField(index, "table2Group").size());
-        Assert.assertEquals(1, Deencapsulation.<Map<GroupId, List<List<Long>>>>getField(index, "group2BackendsPerBucketSeq").size());
+        Assert.assertEquals(1, Deencapsulation.<Table<GroupId, Tag, List<List<Long>>>>getField(index, "group2BackendsPerBucketSeq").size());
         Assert.assertEquals(1, Deencapsulation.<Map<GroupId, ColocateGroupSchema>>getField(index, "group2Schema").size());
         Assert.assertEquals(0, index.getUnstableGroupIds().size());
 
@@ -181,7 +182,7 @@ public class ColocateTableTest {
         Assert.assertEquals(1, index.getAllGroupIds().size());
         Assert.assertEquals(1, Deencapsulation.<Map<Long, GroupId>>getField(index, "table2Group").size());
         Assert.assertEquals(1,
-                Deencapsulation.<Map<GroupId, List<List<Long>>>>getField(index, "group2BackendsPerBucketSeq").size());
+                Deencapsulation.<Table<GroupId, Tag, List<List<Long>>>>getField(index, "group2BackendsPerBucketSeq").size());
         Assert.assertEquals(0, index.getUnstableGroupIds().size());
 
         Assert.assertFalse(index.isColocateTable(firstTblId));
@@ -194,7 +195,7 @@ public class ColocateTableTest {
         Assert.assertEquals(0, index.getAllGroupIds().size());
         Assert.assertEquals(0, Deencapsulation.<Map<Long, GroupId>>getField(index, "table2Group").size());
         Assert.assertEquals(0,
-                Deencapsulation.<Map<GroupId, List<List<Long>>>>getField(index, "group2BackendsPerBucketSeq").size());
+                Deencapsulation.<Table<GroupId, Tag, List<List<Long>>>>getField(index, "group2BackendsPerBucketSeq").size());
         Assert.assertEquals(0, index.getUnstableGroupIds().size());
 
         Assert.assertFalse(index.isColocateTable(firstTblId));
@@ -246,7 +247,7 @@ public class ColocateTableTest {
                 ");");
 
         expectedEx.expect(DdlException.class);
-        expectedEx.expectMessage("Colocate tables must have same replication num: 1");
+        expectedEx.expectMessage("Colocate tables must have same replication allocation: tag.location.default: 1");
         createTable("create table " + dbName + "." + tableName2 + " (\n" +
                 " `k1` int NULL COMMENT \"\",\n" +
                 " `k2` varchar(10) NULL COMMENT \"\"\n" +
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateEncryptKeyTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateEncryptKeyTest.java
index 0bfe7fd..88fe7f4 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateEncryptKeyTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateEncryptKeyTest.java
@@ -30,6 +30,7 @@ import org.apache.doris.qe.ConnectContext;
 import org.apache.doris.qe.QueryState;
 import org.apache.doris.qe.StmtExecutor;
 import org.apache.doris.utframe.UtFrameUtils;
+
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -44,7 +45,7 @@ public class CreateEncryptKeyTest {
 
     @BeforeClass
     public static void setup() throws Exception {
-        UtFrameUtils.createMinDorisCluster(runningDir);
+        UtFrameUtils.createDorisCluster(runningDir);
         FeConstants.runningUnitTest = true;
     }
 
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateFunctionTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateFunctionTest.java
index 0e4f90a..d56856d 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateFunctionTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateFunctionTest.java
@@ -51,7 +51,7 @@ public class CreateFunctionTest {
 
     @BeforeClass
     public static void setup() throws Exception {
-        UtFrameUtils.createMinDorisCluster(runningDir);
+        UtFrameUtils.createDorisCluster(runningDir);
         FeConstants.runningUnitTest = true;
     }
 
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableLikeTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableLikeTest.java
index 127dd65..1743b61 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableLikeTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateTableLikeTest.java
@@ -17,16 +17,14 @@
 
 package org.apache.doris.catalog;
 
-import avro.shaded.com.google.common.collect.Lists;
-import org.apache.commons.collections.ListUtils;
 import org.apache.doris.analysis.CreateDbStmt;
 import org.apache.doris.analysis.CreateTableLikeStmt;
 import org.apache.doris.analysis.CreateTableStmt;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.common.ExceptionChecker;
-import org.apache.doris.common.util.ListUtil;
 import org.apache.doris.qe.ConnectContext;
 import org.apache.doris.utframe.UtFrameUtils;
+
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -36,6 +34,8 @@ import java.io.File;
 import java.util.List;
 import java.util.UUID;
 
+import avro.shaded.com.google.common.collect.Lists;
+
 /**
  * @author wangcong
  * @version 1.0
@@ -48,7 +48,7 @@ public class CreateTableLikeTest {
 
     @BeforeClass
     public static void beforeClass() throws Exception {
-        UtFrameUtils.createMinDorisCluster(runningDir);
+        UtFrameUtils.createDorisCluster(runningDir);
 
         // create connect context
         connectContext = UtFrameUtils.createDefaultCtx();
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 71f3b6f..4db3053 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
@@ -45,7 +45,7 @@ public class CreateTableTest {
     @BeforeClass
     public static void beforeClass() throws Exception {
         Config.disable_storage_medium_check = true;
-        UtFrameUtils.createMinDorisCluster(runningDir);
+        UtFrameUtils.createDorisCluster(runningDir);
 
         // create connect context
         connectContext = UtFrameUtils.createDefaultCtx();
@@ -241,7 +241,8 @@ public class CreateTableTest {
                         + "properties('replication_num' = '1', 'short_key' = '4');"));
 
         ExceptionChecker
-                .expectThrowsWithMsg(DdlException.class, "Failed to find enough host in all backends. need: 3",
+                .expectThrowsWithMsg(DdlException.class, "Failed to find enough host with storage medium and " +
+                                "tag(NaN/{\"location\" : \"default\"}) in all backends. need: 3",
                         () -> createTable("create table test.atbl5\n" + "(k1 int, k2 int, k3 int)\n"
                                 + "duplicate key(k1, k2, k3)\n" + "distributed by hash(k1) buckets 1\n"
                                 + "properties('replication_num' = '3');"));
@@ -258,7 +259,8 @@ public class CreateTableTest {
 
         ConfigBase.setMutableConfig("disable_storage_medium_check", "false");
         ExceptionChecker
-                .expectThrowsWithMsg(DdlException.class, "Failed to find enough host with storage medium is SSD in all backends. need: 1",
+                .expectThrowsWithMsg(DdlException.class, "Failed to find enough host with storage medium and " +
+                                "tag(SSD/{\"location\" : \"default\"}) in all backends. need: 1",
                         () -> createTable("create table test.tb7(key1 int, key2 varchar(10)) distributed by hash(key1) \n"
                                 + "buckets 1 properties('replication_num' = '1', 'storage_medium' = 'ssd');"));
 
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateViewTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateViewTest.java
index 141e8e1..bbc21b9 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateViewTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/CreateViewTest.java
@@ -41,7 +41,7 @@ public class CreateViewTest {
 
     @BeforeClass
     public static void beforeClass() throws Exception {
-        UtFrameUtils.createMinDorisCluster(runningDir);
+        UtFrameUtils.createDorisCluster(runningDir);
 
         // create connect context
         connectContext = UtFrameUtils.createDefaultCtx();
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/DropDbTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/DropDbTest.java
index d695fa5..0a97a1d 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/DropDbTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/DropDbTest.java
@@ -25,12 +25,12 @@ import org.apache.doris.common.DdlException;
 import org.apache.doris.common.ExceptionChecker;
 import org.apache.doris.qe.ConnectContext;
 import org.apache.doris.utframe.UtFrameUtils;
+
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
-
 import java.io.File;
 import java.util.List;
 import java.util.UUID;
@@ -42,7 +42,7 @@ public class DropDbTest {
 
     @BeforeClass
     public static void beforeClass() throws Exception {
-        UtFrameUtils.createMinDorisCluster(runningDir);
+        UtFrameUtils.createDorisCluster(runningDir);
 
         // create connect context
         connectContext = UtFrameUtils.createDefaultCtx();
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/DropPartitionTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/DropPartitionTest.java
index 43f18ce..4b7dc55 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/DropPartitionTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/DropPartitionTest.java
@@ -42,7 +42,7 @@ public class DropPartitionTest {
 
     @BeforeClass
     public static void beforeClass() throws Exception {
-        UtFrameUtils.createMinDorisCluster(runningDir);
+        UtFrameUtils.createDorisCluster(runningDir);
 
         // create connect context
         connectContext = UtFrameUtils.createDefaultCtx();
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/DropTableTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/DropTableTest.java
index fc543fe..f96c60c 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/DropTableTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/DropTableTest.java
@@ -25,6 +25,7 @@ import org.apache.doris.common.DdlException;
 import org.apache.doris.common.ExceptionChecker;
 import org.apache.doris.qe.ConnectContext;
 import org.apache.doris.utframe.UtFrameUtils;
+
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
@@ -41,7 +42,7 @@ public class DropTableTest {
 
     @BeforeClass
     public static void beforeClass() throws Exception {
-        UtFrameUtils.createMinDorisCluster(runningDir);
+        UtFrameUtils.createDorisCluster(runningDir);
 
         // create connect context
         connectContext = UtFrameUtils.createDefaultCtx();
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/DynamicPartitionTableTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/DynamicPartitionTableTest.java
index 95852ae..a672456 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/DynamicPartitionTableTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/DynamicPartitionTableTest.java
@@ -63,7 +63,7 @@ public class DynamicPartitionTableTest {
         FeConstants.runningUnitTest = true;
         Config.disable_storage_medium_check = true;
 
-        UtFrameUtils.createMinDorisCluster(runningDir);
+        UtFrameUtils.createDorisCluster(runningDir);
 
         // create connect context
         connectContext = UtFrameUtils.createDefaultCtx();
@@ -118,7 +118,7 @@ public class DynamicPartitionTableTest {
         createTable(createOlapTblStmt);
         Database db = Catalog.getCurrentCatalog().getDbOrAnalysisException("default_cluster:test");
         OlapTable table = (OlapTable) db.getTableOrAnalysisException("dynamic_partition_normal");
-        Assert.assertEquals(table.getTableProperty().getDynamicPartitionProperty().getReplicationNum(), DynamicPartitionProperty.NOT_SET_REPLICATION_NUM);
+        Assert.assertTrue(table.getTableProperty().getDynamicPartitionProperty().getReplicaAllocation().isNotSet());
     }
 
     @Test
@@ -458,7 +458,7 @@ public class DynamicPartitionTableTest {
         createTable(createOlapTblStmt);
         Database db = Catalog.getCurrentCatalog().getDbOrAnalysisException("default_cluster:test");
         OlapTable table = (OlapTable) db.getTableOrAnalysisException(tableName);
-        Assert.assertEquals(table.getTableProperty().getDynamicPartitionProperty().getReplicationNum(), 2);
+        Assert.assertEquals(2, table.getTableProperty().getDynamicPartitionProperty().getReplicaAllocation().getTotalReplicaNum());
     }
 
     @Test
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/ModifyBackendTagTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/ModifyBackendTagTest.java
new file mode 100644
index 0000000..4c6b63d
--- /dev/null
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/ModifyBackendTagTest.java
@@ -0,0 +1,187 @@
+// 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.AlterSystemStmt;
+import org.apache.doris.analysis.AlterTableStmt;
+import org.apache.doris.analysis.CreateDbStmt;
+import org.apache.doris.analysis.CreateTableStmt;
+import org.apache.doris.clone.DynamicPartitionScheduler;
+import org.apache.doris.common.DdlException;
+import org.apache.doris.common.ExceptionChecker;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.qe.DdlExecutor;
+import org.apache.doris.resource.Tag;
+import org.apache.doris.system.Backend;
+import org.apache.doris.system.SystemInfoService;
+import org.apache.doris.utframe.UtFrameUtils;
+
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.File;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+public class ModifyBackendTagTest {
+
+    private static String runningDir = "fe/mocked/ModifyBackendTagTest/" + UUID.randomUUID().toString() + "/";
+    private static ConnectContext connectContext;
+
+    @BeforeClass
+    public static void beforeClass() throws Exception {
+        UtFrameUtils.createDorisCluster(runningDir);
+        // create connect context
+        connectContext = UtFrameUtils.createDefaultCtx();
+        // create database
+        String createDbStmtStr = "create database test;";
+        CreateDbStmt createDbStmt = (CreateDbStmt) UtFrameUtils.parseAndAnalyzeStmt(createDbStmtStr, connectContext);
+        Catalog.getCurrentCatalog().createDb(createDbStmt);
+    }
+
+    @AfterClass
+    public static void tearDown() {
+        File file = new File(runningDir);
+        file.delete();
+    }
+
+    @Test
+    public void testModifyBackend() throws Exception {
+        SystemInfoService infoService = Catalog.getCurrentSystemInfo();
+        List<Backend> backends = infoService.getClusterBackends(SystemInfoService.DEFAULT_CLUSTER);
+        Assert.assertEquals(1, backends.size());
+        String beHostPort = backends.get(0).getHost() + ":" + backends.get(0).getHeartbeatPort();
+
+        // modify backend tag
+        String stmtStr = "alter system modify backend \"" + beHostPort + "\" set ('tag.location' = 'zone1')";
+        AlterSystemStmt stmt = (AlterSystemStmt) UtFrameUtils.parseAndAnalyzeStmt(stmtStr, connectContext);
+        DdlExecutor.execute(Catalog.getCurrentCatalog(), stmt);
+        backends = infoService.getClusterBackends(SystemInfoService.DEFAULT_CLUSTER);
+        Assert.assertEquals(1, backends.size());
+
+        // create table
+        String createStr = "create table test.tbl1(\n" +
+                "k1 int\n" +
+                ") distributed by hash(k1)\n" +
+                "buckets 3 properties(\n" +
+                "\"replication_num\" = \"1\"\n" +
+                ");";
+        CreateTableStmt createStmt = (CreateTableStmt) UtFrameUtils.parseAndAnalyzeStmt(createStr, connectContext);
+        ExceptionChecker.expectThrowsWithMsg(DdlException.class,
+                "Failed to find enough host with storage medium and tag(HDD/{\"location\" : \"default\"}) in all backends. need: 1",
+                () -> DdlExecutor.execute(Catalog.getCurrentCatalog(), createStmt));
+
+        createStr = "create table test.tbl1(\n" +
+                "k1 int\n" +
+                ") distributed by hash(k1)\n" +
+                "buckets 3 properties(\n" +
+                "\"replication_allocation\" = \"tag.location.zone1: 1\"\n" +
+                ");";
+        CreateTableStmt createStmt2 = (CreateTableStmt) UtFrameUtils.parseAndAnalyzeStmt(createStr, connectContext);
+        ExceptionChecker.expectThrowsNoException(() -> DdlExecutor.execute(Catalog.getCurrentCatalog(), createStmt2));
+
+        // create dynamic partition tbl
+        createStr = "create table test.tbl3(\n" +
+                "k1 date, k2 int\n" +
+                ") partition by range(k1)()\n" +
+                "distributed by hash(k1)\n" +
+                "buckets 3 properties(\n" +
+                "    \"dynamic_partition.enable\" = \"true\",\n" +
+                "    \"dynamic_partition.time_unit\" = \"DAY\",\n" +
+                "    \"dynamic_partition.start\" = \"-3\",\n" +
+                "    \"dynamic_partition.end\" = \"3\",\n" +
+                "    \"dynamic_partition.prefix\" = \"p\",\n" +
+                "    \"dynamic_partition.buckets\" = \"1\",\n" +
+                "    \"dynamic_partition.replication_num\" = \"1\"\n" +
+                ");";
+        CreateTableStmt createStmt3 = (CreateTableStmt) UtFrameUtils.parseAndAnalyzeStmt(createStr, connectContext);
+        // although there is no exception throw, but partition create failed, because there is no BE
+        // with "default" tag
+        ExceptionChecker.expectThrowsNoException(() -> DdlExecutor.execute(Catalog.getCurrentCatalog(), createStmt3));
+        Database db = Catalog.getCurrentCatalog().getDbNullable("default_cluster:test");
+        Table tbl3 = db.getTableNullable("tbl3");
+        String err = Catalog.getCurrentCatalog().getDynamicPartitionScheduler().getRuntimeInfo(tbl3.getId(), DynamicPartitionScheduler.CREATE_PARTITION_MSG);
+        Assert.assertTrue(err.contains("Failed to find enough host with storage medium and tag"));
+
+        createStr = "create table test.tbl4(\n" +
+                "k1 date, k2 int\n" +
+                ") partition by range(k1)()\n" +
+                "distributed by hash(k1)\n" +
+                "buckets 3 properties(\n" +
+                "    \"dynamic_partition.enable\" = \"true\",\n" +
+                "    \"dynamic_partition.time_unit\" = \"DAY\",\n" +
+                "    \"dynamic_partition.start\" = \"-3\",\n" +
+                "    \"dynamic_partition.end\" = \"3\",\n" +
+                "    \"dynamic_partition.prefix\" = \"p\",\n" +
+                "    \"dynamic_partition.buckets\" = \"1\",\n" +
+                "    \"dynamic_partition.replication_allocation\" = \"tag.location.zone1:1\"\n" +
+                ");";
+        CreateTableStmt createStmt4 = (CreateTableStmt) UtFrameUtils.parseAndAnalyzeStmt(createStr, connectContext);
+        ExceptionChecker.expectThrowsNoException(() -> DdlExecutor.execute(Catalog.getCurrentCatalog(), createStmt4));
+        DynamicPartitionScheduler scheduler = Catalog.getCurrentCatalog().getDynamicPartitionScheduler();
+        OlapTable tbl = (OlapTable) db.getTableNullable("tbl4");
+        PartitionInfo partitionInfo = tbl.getPartitionInfo();
+        Assert.assertEquals(4, partitionInfo.idToItem.size());
+        ReplicaAllocation replicaAlloc = new ReplicaAllocation();
+        replicaAlloc.put(Tag.create(Tag.TYPE_LOCATION, "zone1"), (short) 1);
+        for (ReplicaAllocation allocation : partitionInfo.idToReplicaAllocation.values()) {
+            Assert.assertEquals(replicaAlloc, allocation);
+        }
+
+        ReplicaAllocation defaultAlloc = tbl.getDefaultReplicaAllocation();
+        Assert.assertEquals(ReplicaAllocation.DEFAULT_ALLOCATION, defaultAlloc);
+        TableProperty tableProperty = tbl.getTableProperty();
+        Map<String, String> tblProperties = tableProperty.getProperties();
+        // if replication_num or replica_allocation is not set, it will be set to the default one
+        Assert.assertTrue(tblProperties.containsKey("default.replication_allocation"));
+        Assert.assertEquals("tag.location.default: 3", tblProperties.get("default.replication_allocation"));
+
+        // modify default replica
+        String alterStr = "alter table test.tbl4 set ('default.replication_allocation' = 'tag.location.zonex:1')";
+        AlterTableStmt alterStmt = (AlterTableStmt) UtFrameUtils.parseAndAnalyzeStmt(alterStr, connectContext);
+        ExceptionChecker.expectThrowsNoException(() -> DdlExecutor.execute(Catalog.getCurrentCatalog(), alterStmt));
+        defaultAlloc = tbl.getDefaultReplicaAllocation();
+        ReplicaAllocation expectedAlloc = new ReplicaAllocation();
+        expectedAlloc.put(Tag.create(Tag.TYPE_LOCATION, "zonex"), (short) 1);
+        Assert.assertEquals(expectedAlloc, defaultAlloc);
+        tblProperties = tableProperty.getProperties();
+        Assert.assertTrue(tblProperties.containsKey("default.replication_allocation"));
+
+        // modify partition replica with wrong zone
+        String partName = tbl.getPartitionNames().stream().findFirst().get();
+        alterStr = "alter table test.tbl4 modify partition " + partName
+                + " set ('replication_allocation' = 'tag.location.zonex:1')";
+        AlterTableStmt alterStmt2 = (AlterTableStmt) UtFrameUtils.parseAndAnalyzeStmt(alterStr, connectContext);
+        ExceptionChecker.expectThrowsWithMsg(DdlException.class,
+                "Failed to find enough host with tag({\"location\" : \"zonex\"}) in all backends. need: 1",
+                () -> DdlExecutor.execute(Catalog.getCurrentCatalog(), alterStmt2));
+        tblProperties = tableProperty.getProperties();
+        Assert.assertTrue(tblProperties.containsKey("default.replication_allocation"));
+
+        alterStr = "alter table test.tbl4 modify partition " + partName
+                + " set ('replication_allocation' = 'tag.location.zone1:1')";
+        AlterTableStmt alterStmt3 = (AlterTableStmt) UtFrameUtils.parseAndAnalyzeStmt(alterStr, connectContext);
+        ExceptionChecker.expectThrowsNoException(() -> DdlExecutor.execute(Catalog.getCurrentCatalog(), alterStmt3));
+        tblProperties = tableProperty.getProperties();
+        Assert.assertTrue(tblProperties.containsKey("default.replication_allocation"));
+    }
+}
+
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/RecoverTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/RecoverTest.java
index a69762d..68408e5 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/RecoverTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/RecoverTest.java
@@ -47,7 +47,7 @@ public class RecoverTest {
 
     @BeforeClass
     public static void beforeClass() throws Exception {
-        UtFrameUtils.createMinDorisCluster(runningDir);
+        UtFrameUtils.createDorisCluster(runningDir);
 
         // create connect context
         connectContext = UtFrameUtils.createDefaultCtx();
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/ReplicaAllocationTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/ReplicaAllocationTest.java
new file mode 100644
index 0000000..85db811
--- /dev/null
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/ReplicaAllocationTest.java
@@ -0,0 +1,159 @@
+// 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.AnalysisException;
+import org.apache.doris.common.ExceptionChecker;
+import org.apache.doris.common.FeConstants;
+import org.apache.doris.common.util.PropertyAnalyzer;
+import org.apache.doris.meta.MetaContext;
+import org.apache.doris.resource.Tag;
+
+import com.google.common.collect.Maps;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.Map;
+
+public class ReplicaAllocationTest {
+
+    @Test
+    public void testNormal() throws AnalysisException {
+        // DEFAULT_ALLOCATION
+        ReplicaAllocation replicaAlloc = ReplicaAllocation.DEFAULT_ALLOCATION;
+        Assert.assertFalse(replicaAlloc.isNotSet());
+        Assert.assertTrue(replicaAlloc.equals(ReplicaAllocation.DEFAULT_ALLOCATION));
+        Assert.assertFalse(replicaAlloc.isEmpty());
+        Assert.assertEquals(3, replicaAlloc.getTotalReplicaNum());
+        Assert.assertEquals("tag.location.default: 3", replicaAlloc.toCreateStmt());
+
+        // NOT SET
+        replicaAlloc = ReplicaAllocation.NOT_SET;
+        Assert.assertTrue(replicaAlloc.isNotSet());
+        Assert.assertFalse(replicaAlloc.equals(ReplicaAllocation.DEFAULT_ALLOCATION));
+        Assert.assertTrue(replicaAlloc.isEmpty());
+        Assert.assertEquals(0, replicaAlloc.getTotalReplicaNum());
+        Assert.assertEquals("", replicaAlloc.toCreateStmt());
+
+        // set replica num
+        replicaAlloc = new ReplicaAllocation((short) 5);
+        Assert.assertFalse(replicaAlloc.isNotSet());
+        Assert.assertFalse(replicaAlloc.equals(ReplicaAllocation.DEFAULT_ALLOCATION));
+        Assert.assertFalse(replicaAlloc.isEmpty());
+        Assert.assertEquals(5, replicaAlloc.getTotalReplicaNum());
+        Assert.assertEquals("tag.location.default: 5", replicaAlloc.toCreateStmt());
+
+        // set replica num with tag
+        replicaAlloc = new ReplicaAllocation();
+        replicaAlloc.put(Tag.create(Tag.TYPE_LOCATION, "zone1"), (short) 3);
+        replicaAlloc.put(Tag.create(Tag.TYPE_LOCATION, "zone2"), (short) 2);
+        Assert.assertFalse(replicaAlloc.isNotSet());
+        Assert.assertFalse(replicaAlloc.isEmpty());
+        Assert.assertEquals(5, replicaAlloc.getTotalReplicaNum());
+        Assert.assertEquals("tag.location.zone2: 2, tag.location.zone1: 3", replicaAlloc.toCreateStmt());
+    }
+
+    @Test
+    public void testPropertyAnalyze() throws AnalysisException {
+        Map<String, String> properties = Maps.newHashMap();
+        properties.put(PropertyAnalyzer.PROPERTIES_REPLICATION_NUM, "3");
+        ReplicaAllocation replicaAlloc = PropertyAnalyzer.analyzeReplicaAllocation(properties, "");
+        Assert.assertEquals(ReplicaAllocation.DEFAULT_ALLOCATION, replicaAlloc);
+        Assert.assertTrue(properties.isEmpty());
+
+        // not set
+        properties = Maps.newHashMap();
+        replicaAlloc = PropertyAnalyzer.analyzeReplicaAllocation(properties, "");
+        Assert.assertEquals(ReplicaAllocation.NOT_SET, replicaAlloc);
+
+        properties = Maps.newHashMap();
+        properties.put("default." + PropertyAnalyzer.PROPERTIES_REPLICATION_NUM, "3");
+        replicaAlloc = PropertyAnalyzer.analyzeReplicaAllocation(properties, "default");
+        Assert.assertEquals(ReplicaAllocation.DEFAULT_ALLOCATION, replicaAlloc);
+        Assert.assertTrue(properties.isEmpty());
+
+        properties = Maps.newHashMap();
+        properties.put(PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION, "tag.location.zone2: 2, tag.location.zone1: 3");
+        replicaAlloc = PropertyAnalyzer.analyzeReplicaAllocation(properties, "");
+        Assert.assertNotEquals(ReplicaAllocation.DEFAULT_ALLOCATION, replicaAlloc);
+        Assert.assertFalse(replicaAlloc.isNotSet());
+        Assert.assertFalse(replicaAlloc.isEmpty());
+        Assert.assertEquals(5, replicaAlloc.getTotalReplicaNum());
+        Assert.assertEquals("tag.location.zone2: 2, tag.location.zone1: 3", replicaAlloc.toCreateStmt());
+        Assert.assertTrue(properties.isEmpty());
+
+        properties = Maps.newHashMap();
+        properties.put("dynamic_partition." + PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION, "tag.location.zone2: 1, tag.location.zone1: 3");
+        replicaAlloc = PropertyAnalyzer.analyzeReplicaAllocation(properties, "dynamic_partition");
+        Assert.assertNotEquals(ReplicaAllocation.DEFAULT_ALLOCATION, replicaAlloc);
+        Assert.assertFalse(replicaAlloc.isNotSet());
+        Assert.assertFalse(replicaAlloc.isEmpty());
+        Assert.assertEquals(4, replicaAlloc.getTotalReplicaNum());
+        Assert.assertEquals("tag.location.zone2: 1, tag.location.zone1: 3", replicaAlloc.toCreateStmt());
+        Assert.assertTrue(properties.isEmpty());
+    }
+
+    @Test
+    public void testAbnormal() {
+        final Map<String, String> properties = Maps.newHashMap();
+        properties.put(PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION, "3");
+        ExceptionChecker.expectThrowsWithMsg(AnalysisException.class, "Invalid replication allocation property: 3",
+                ()->PropertyAnalyzer.analyzeReplicaAllocation(properties, ""));
+
+        properties.clear();
+        properties.put(PropertyAnalyzer.PROPERTIES_REPLICATION_ALLOCATION, "tag.location.12321:1");
+        ExceptionChecker.expectThrowsWithMsg(AnalysisException.class, "Invalid tag format: location:12321",
+                ()->PropertyAnalyzer.analyzeReplicaAllocation(properties, ""));
+    }
+
+    @Test
+    public void testPersist() throws IOException, AnalysisException {
+        MetaContext metaContext = new MetaContext();
+        metaContext.setMetaVersion(FeConstants.meta_version);
+        metaContext.setThreadLocalInfo();
+
+        // 1. Write objects to file
+        File file = new File("./replicaInfo");
+        file.createNewFile();
+        DataOutputStream dos = new DataOutputStream(new FileOutputStream(file));
+
+        ReplicaAllocation replicaAlloc = new ReplicaAllocation();
+        replicaAlloc.put(Tag.create(Tag.TYPE_LOCATION, "zone1"), (short) 3);
+        replicaAlloc.put(Tag.create(Tag.TYPE_LOCATION, "zone2"), (short) 2);
+        replicaAlloc.write(dos);
+
+        dos.flush();
+        dos.close();
+
+        // 2. Read objects from file
+        DataInputStream dis = new DataInputStream(new FileInputStream(file));
+        ReplicaAllocation newAlloc = ReplicaAllocation.read(dis);
+        Assert.assertEquals(replicaAlloc, newAlloc);
+
+        // 3. delete files
+        dis.close();
+        file.delete();
+    }
+}
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/TablePropertyTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/TablePropertyTest.java
index fe14c4c..3490d3f 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/TablePropertyTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/TablePropertyTest.java
@@ -18,6 +18,9 @@
 package org.apache.doris.catalog;
 
 
+import org.apache.doris.common.FeMetaVersion;
+import org.apache.doris.meta.MetaContext;
+
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Test;
@@ -41,6 +44,9 @@ public class TablePropertyTest {
 
     @Test
     public void testNormal() throws IOException {
+        MetaContext metaContext = new MetaContext();
+        metaContext.setMetaVersion(FeMetaVersion.VERSION_CURRENT);
+        metaContext.setThreadLocalInfo();
         // 1. Write objects to file
         File file = new File(fileName);
         file.createNewFile();
@@ -55,6 +61,7 @@ public class TablePropertyTest {
         properties.put(DynamicPartitionProperty.BUCKETS, "30");
         properties.put("otherProperty", "unknownProperty");
         TableProperty tableProperty = new TableProperty(properties);
+        tableProperty.setReplicaAlloc(ReplicaAllocation.DEFAULT_ALLOCATION);
         tableProperty.write(out);
         out.flush();
         out.close();
@@ -71,6 +78,7 @@ public class TablePropertyTest {
         Assert.assertEquals(readDynamicPartitionProperty.getStart(), dynamicPartitionProperty.getStart());
         Assert.assertEquals(readDynamicPartitionProperty.getEnd(), dynamicPartitionProperty.getEnd());
         Assert.assertEquals(readDynamicPartitionProperty.getTimeUnit(), dynamicPartitionProperty.getTimeUnit());
+        Assert.assertEquals(ReplicaAllocation.DEFAULT_ALLOCATION, readTableProperty.getReplicaAllocation());
         in.close();
     }
 }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/TabletTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/TabletTest.java
index 0507b96..f9a1811 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/TabletTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/TabletTest.java
@@ -17,18 +17,13 @@
 
 package org.apache.doris.catalog;
 
-import mockit.Expectations;
-import mockit.Mocked;
-
 import org.apache.doris.catalog.Replica.ReplicaState;
 import org.apache.doris.common.FeConstants;
 import org.apache.doris.common.Pair;
-import org.apache.doris.system.Backend;
 import org.apache.doris.thrift.TStorageMedium;
 
 import com.google.common.collect.Sets;
 
-import org.apache.arrow.flatbuf.Bool;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
@@ -38,8 +33,9 @@ import java.io.DataOutputStream;
 import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
+
+import mockit.Expectations;
+import mockit.Mocked;
 
 public class TabletTest {
 
@@ -179,7 +175,8 @@ public class TabletTest {
             }
             tablet.addReplica(new Replica(replicaId++, pair.first, versionAndSuccessVersion, 0L, 0, 200000L, 3000L, ReplicaState.NORMAL, lastFailVersion, 0, versionAndSuccessVersion, 0));
         }
-        Assert.assertEquals(tablet.getColocateHealthStatus(100L, 3, Sets.newHashSet(1L, 2L, 3L)), exceptedTabletStatus);
+        Assert.assertEquals(tablet.getColocateHealthStatus(100L, new ReplicaAllocation((short) 3),
+                Sets.newHashSet(1L, 2L, 3L)), exceptedTabletStatus);
     }
 
     @Test
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 56a0bde..d3b6cf8 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
@@ -69,7 +69,7 @@ public class TempPartitionTest {
     @BeforeClass
     public static void setup() throws Exception {
         FeConstants.default_scheduler_interval_millisecond = 100;
-        UtFrameUtils.createMinDorisCluster(runningDir);
+        UtFrameUtils.createDorisCluster(runningDir);
         ctx = UtFrameUtils.createDefaultCtx();
     }
 
diff --git a/fe/fe-core/src/test/java/org/apache/doris/catalog/UserPropertyTest.java b/fe/fe-core/src/test/java/org/apache/doris/catalog/UserPropertyTest.java
index 9fbfd5b..90c6bc2 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/catalog/UserPropertyTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/catalog/UserPropertyTest.java
@@ -20,6 +20,7 @@ package org.apache.doris.catalog;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.common.FeConstants;
 import org.apache.doris.common.Pair;
+import org.apache.doris.common.UserException;
 import org.apache.doris.load.DppConfig;
 import org.apache.doris.mysql.privilege.UserProperty;
 
@@ -57,7 +58,7 @@ public class UserPropertyTest {
     }
 
     @Test
-    public void testUpdate() throws DdlException {
+    public void testUpdate() throws UserException {
         List<Pair<String, String>> properties = Lists.newArrayList();
         properties.add(Pair.create("MAX_USER_CONNECTIONS", "100"));
         properties.add(Pair.create("resource.cpu_share", "101"));
diff --git a/fe/fe-core/src/test/java/org/apache/doris/clone/ClusterLoadStatisticsTest.java b/fe/fe-core/src/test/java/org/apache/doris/clone/ClusterLoadStatisticsTest.java
index b1b2567..0bbc550 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/clone/ClusterLoadStatisticsTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/clone/ClusterLoadStatisticsTest.java
@@ -23,6 +23,7 @@ import org.apache.doris.catalog.Replica;
 import org.apache.doris.catalog.Replica.ReplicaState;
 import org.apache.doris.catalog.TabletInvertedIndex;
 import org.apache.doris.catalog.TabletMeta;
+import org.apache.doris.resource.Tag;
 import org.apache.doris.system.Backend;
 import org.apache.doris.system.SystemInfoService;
 import org.apache.doris.thrift.TStorageMedium;
@@ -144,7 +145,7 @@ public class ClusterLoadStatisticsTest {
     @Test
     public void test() {
         ClusterLoadStatistic loadStatistic = new ClusterLoadStatistic(SystemInfoService.DEFAULT_CLUSTER,
-                systemInfoService, invertedIndex);
+                Tag.DEFAULT_BACKEND_TAG, systemInfoService, invertedIndex);
         loadStatistic.init();
         List<List<String>> infos = loadStatistic.getClusterStatistic(TStorageMedium.HDD);
         System.out.println(infos);
diff --git a/fe/fe-core/src/test/java/org/apache/doris/clone/ColocateTableCheckerAndBalancerTest.java b/fe/fe-core/src/test/java/org/apache/doris/clone/ColocateTableCheckerAndBalancerTest.java
index 3b5695a..8b525ea 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/clone/ColocateTableCheckerAndBalancerTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/clone/ColocateTableCheckerAndBalancerTest.java
@@ -17,34 +17,37 @@
 
 package org.apache.doris.clone;
 
-import mockit.Delegate;
-import mockit.Expectations;
-import mockit.Mocked;
-
 import org.apache.doris.catalog.ColocateGroupSchema;
 import org.apache.doris.catalog.ColocateTableIndex;
 import org.apache.doris.catalog.ColocateTableIndex.GroupId;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.PrimitiveType;
+import org.apache.doris.catalog.ReplicaAllocation;
 import org.apache.doris.catalog.TabletInvertedIndex;
+import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.Config;
 import org.apache.doris.common.jmockit.Deencapsulation;
+import org.apache.doris.resource.Tag;
 import org.apache.doris.system.Backend;
 import org.apache.doris.system.SystemInfoService;
 
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import mockit.Delegate;
+import mockit.Expectations;
+import mockit.Mocked;
+
 public class ColocateTableCheckerAndBalancerTest {
     private ColocateTableCheckerAndBalancer balancer = ColocateTableCheckerAndBalancer.getInstance();
 
@@ -89,7 +92,9 @@ public class ColocateTableCheckerAndBalancerTest {
         ColocateTableIndex colocateTableIndex = new ColocateTableIndex();
         int replicationNum = 3;
         List<List<Long>> backendsPerBucketSeq = Lists.partition(flatList, replicationNum);
-        colocateTableIndex.addBackendsPerBucketSeq(groupId, backendsPerBucketSeq);
+        Map<Tag, List<List<Long>>> backendsPerBucketSeqMap = Maps.newHashMap();
+        backendsPerBucketSeqMap.put(Tag.DEFAULT_BACKEND_TAG, backendsPerBucketSeq);
+        colocateTableIndex.addBackendsPerBucketSeq(groupId, backendsPerBucketSeqMap);
         return colocateTableIndex;
     }
 
@@ -134,7 +139,8 @@ public class ColocateTableCheckerAndBalancerTest {
         GroupId groupId = new GroupId(10000, 10001);
         List<Column> distributionCols = Lists.newArrayList();
         distributionCols.add(new Column("k1", PrimitiveType.INT));
-        ColocateGroupSchema groupSchema = new ColocateGroupSchema(groupId, distributionCols, 5, (short) 3);
+        ColocateGroupSchema groupSchema = new ColocateGroupSchema(groupId, distributionCols, 5,
+                ReplicaAllocation.DEFAULT_ALLOCATION);
         Map<GroupId, ColocateGroupSchema> group2Schema = Maps.newHashMap();
         group2Schema.put(groupId, groupSchema);
 
@@ -146,7 +152,8 @@ public class ColocateTableCheckerAndBalancerTest {
 
         List<List<Long>> balancedBackendsPerBucketSeq = Lists.newArrayList();
         List<Long> allAvailBackendIds = Lists.newArrayList(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L);
-        boolean changed = (Boolean) Deencapsulation.invoke(balancer, "relocateAndBalance", groupId, new HashSet<Long>(), allAvailBackendIds,
+        boolean changed = (Boolean) Deencapsulation.invoke(balancer, "relocateAndBalance", groupId,
+                Tag.DEFAULT_BACKEND_TAG, new HashSet<Long>(), allAvailBackendIds,
                 colocateTableIndex, infoService, statistic, balancedBackendsPerBucketSeq);
         List<List<Long>> expected = Lists.partition(
                 Lists.newArrayList(9L, 5L, 3L, 4L, 6L, 8L, 7L, 6L, 1L, 2L, 9L, 4L, 1L, 2L, 3L), 3);
@@ -158,7 +165,8 @@ public class ColocateTableCheckerAndBalancerTest {
                 Lists.newArrayList(9L, 8L, 7L, 8L, 6L, 5L, 9L, 4L, 1L, 2L, 3L, 4L, 1L, 2L, 3L));
         Deencapsulation.setField(colocateTableIndex, "group2Schema", group2Schema);
         balancedBackendsPerBucketSeq.clear();
-        changed = (Boolean) Deencapsulation.invoke(balancer, "relocateAndBalance", groupId, new HashSet<Long>(), allAvailBackendIds,
+        changed = (Boolean) Deencapsulation.invoke(balancer, "relocateAndBalance", groupId,
+                Tag.DEFAULT_BACKEND_TAG, new HashSet<Long>(), allAvailBackendIds,
                 colocateTableIndex, infoService, statistic, balancedBackendsPerBucketSeq);
         System.out.println(balancedBackendsPerBucketSeq);
         Assert.assertFalse(changed);
@@ -205,7 +213,8 @@ public class ColocateTableCheckerAndBalancerTest {
         GroupId groupId = new GroupId(10000, 10001);
         List<Column> distributionCols = Lists.newArrayList();
         distributionCols.add(new Column("k1", PrimitiveType.INT));
-        ColocateGroupSchema groupSchema = new ColocateGroupSchema(groupId, distributionCols, 5, (short) 1);
+        ColocateGroupSchema groupSchema = new ColocateGroupSchema(groupId, distributionCols, 5,
+                new ReplicaAllocation((short) 1));
         Map<GroupId, ColocateGroupSchema> group2Schema = Maps.newHashMap();
         group2Schema.put(groupId, groupSchema);
 
@@ -216,8 +225,8 @@ public class ColocateTableCheckerAndBalancerTest {
 
         List<List<Long>> balancedBackendsPerBucketSeq = Lists.newArrayList();
         List<Long> allAvailBackendIds = Lists.newArrayList(7L);
-        boolean changed = Deencapsulation.invoke(balancer, "relocateAndBalance", groupId, new HashSet<Long>(), allAvailBackendIds,
-                                                 colocateTableIndex, infoService, statistic, balancedBackendsPerBucketSeq);
+        boolean changed = Deencapsulation.invoke(balancer, "relocateAndBalance", groupId, Tag.DEFAULT_BACKEND_TAG,
+                new HashSet<Long>(), allAvailBackendIds, colocateTableIndex, infoService, statistic, balancedBackendsPerBucketSeq);
         Assert.assertFalse(changed);
 
         // 2. all backends are checked but this round is not changed
@@ -228,8 +237,8 @@ public class ColocateTableCheckerAndBalancerTest {
 
         balancedBackendsPerBucketSeq = Lists.newArrayList();
         allAvailBackendIds = Lists.newArrayList(7L, 8L, 9L);
-        changed = Deencapsulation.invoke(balancer, "relocateAndBalance", groupId, new HashSet<Long>(), allAvailBackendIds,
-                                         colocateTableIndex, infoService, statistic, balancedBackendsPerBucketSeq);
+        changed = Deencapsulation.invoke(balancer, "relocateAndBalance", groupId, Tag.DEFAULT_BACKEND_TAG,
+                new HashSet<Long>(), allAvailBackendIds, colocateTableIndex, infoService, statistic, balancedBackendsPerBucketSeq);
         Assert.assertFalse(changed);
     }
 
@@ -249,7 +258,7 @@ public class ColocateTableCheckerAndBalancerTest {
         };
         GroupId groupId = new GroupId(10000, 10001);
         List<Column> distributionCols = Lists.newArrayList();
-        ColocateGroupSchema groupSchema = new ColocateGroupSchema(groupId, distributionCols, 5, (short) 1);
+        ColocateGroupSchema groupSchema = new ColocateGroupSchema(groupId, distributionCols, 5, new ReplicaAllocation((short) 1));
         Map<GroupId, ColocateGroupSchema> group2Schema = Maps.newHashMap();
         group2Schema.put(groupId, groupSchema);
 
@@ -259,8 +268,8 @@ public class ColocateTableCheckerAndBalancerTest {
         List<List<Long>> balancedBackendsPerBucketSeq = Lists.newArrayList();
         Set<Long> unAvailBackendIds = Sets.newHashSet(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L);
         List<Long> availBackendIds = Lists.newArrayList();
-        boolean changed = (Boolean) Deencapsulation.invoke(balancer, "relocateAndBalance", groupId, unAvailBackendIds, availBackendIds,
-                colocateTableIndex, infoService, statistic, balancedBackendsPerBucketSeq);
+        boolean changed = (Boolean) Deencapsulation.invoke(balancer, "relocateAndBalance", groupId, Tag.DEFAULT_BACKEND_TAG,
+                unAvailBackendIds, availBackendIds, colocateTableIndex, infoService, statistic, balancedBackendsPerBucketSeq);
         Assert.assertFalse(changed);
     }
 
@@ -298,7 +307,7 @@ public class ColocateTableCheckerAndBalancerTest {
     public final class FakeBackendLoadStatistic extends BackendLoadStatistic {
         public FakeBackendLoadStatistic(long beId, String clusterName, SystemInfoService infoService,
                                     TabletInvertedIndex invertedIndex) {
-            super(beId, clusterName, infoService, invertedIndex);
+            super(beId, clusterName, Tag.DEFAULT_BACKEND_TAG, infoService, invertedIndex);
         }
 
         @Override
@@ -324,6 +333,7 @@ public class ColocateTableCheckerAndBalancerTest {
                                                @Mocked Backend myBackend5
                                                ) {
         GroupId groupId = new GroupId(10000, 10001);
+        Tag tag = Tag.DEFAULT_BACKEND_TAG;
         Set<Long> allBackendsInGroup = Sets.newHashSet(1L, 2L, 3L, 4L, 5L);
         new Expectations() {
             {
@@ -338,6 +348,9 @@ public class ColocateTableCheckerAndBalancerTest {
                 myBackend2.isAvailable();
                 result = true;
                 minTimes = 0;
+                myBackend2.getTag();
+                result = Tag.DEFAULT_BACKEND_TAG;
+                minTimes = 0;
 
                 // backend3 not available, and dead for a long time
                 infoService.getBackend(3L);
@@ -352,6 +365,9 @@ public class ColocateTableCheckerAndBalancerTest {
                 myBackend3.getLastUpdateMs();
                 result = System.currentTimeMillis() - Config.tablet_repair_delay_factor_second * 1000 * 20;
                 minTimes = 0;
+                myBackend3.getTag();
+                result = Tag.DEFAULT_BACKEND_TAG;
+                minTimes = 0;
 
                 // backend4 not available, and dead for a short time
                 infoService.getBackend(4L);
@@ -366,6 +382,9 @@ public class ColocateTableCheckerAndBalancerTest {
                 myBackend4.getLastUpdateMs();
                 result = System.currentTimeMillis();
                 minTimes = 0;
+                myBackend4.getTag();
+                result = Tag.DEFAULT_BACKEND_TAG;
+                minTimes = 0;
 
                 // backend5 not available, and in decommission
                 infoService.getBackend(5L);
@@ -380,26 +399,32 @@ public class ColocateTableCheckerAndBalancerTest {
                 myBackend5.isDecommissioned();
                 result = true;
                 minTimes = 0;
+                myBackend5.getTag();
+                result = Tag.DEFAULT_BACKEND_TAG;
+                minTimes = 0;
 
-                colocateTableIndex.getBackendsByGroup(groupId);
+                colocateTableIndex.getBackendsByGroup(groupId, tag);
                 result = allBackendsInGroup;
                 minTimes = 0;
             }
         };
 
-        Set<Long> unavailableBeIds = Deencapsulation.invoke(balancer, "getUnavailableBeIdsInGroup", infoService, colocateTableIndex, groupId);
+        Set<Long> unavailableBeIds = Deencapsulation.invoke(balancer, "getUnavailableBeIdsInGroup",
+                infoService, colocateTableIndex, groupId, Tag.DEFAULT_BACKEND_TAG);
         System.out.println(unavailableBeIds);
-        Assert.assertArrayEquals(new long[]{1L, 3L, 5L}, unavailableBeIds.stream().mapToLong(i->i).sorted().toArray());
+        Assert.assertArrayEquals(new long[]{1L, 3L, 5L}, unavailableBeIds.stream().mapToLong(i -> i).sorted().toArray());
     }
 
     @Test
     public void testGetAvailableBeIds(@Mocked SystemInfoService infoService,
-                                             @Mocked Backend myBackend2,
-                                             @Mocked Backend myBackend3,
-                                             @Mocked Backend myBackend4,
-                                             @Mocked Backend myBackend5) {
+                                      @Mocked Backend myBackend2,
+                                      @Mocked Backend myBackend3,
+                                      @Mocked Backend myBackend4,
+                                      @Mocked Backend myBackend5,
+                                      @Mocked Backend myBackend6,
+                                      @Mocked Backend myBackend7) throws AnalysisException {
         List<Long> clusterBackendIds = Lists.newArrayList(1L, 2L, 3L, 4L, 5L);
-        new Expectations(){
+        new Expectations() {
             {
                 infoService.getClusterBackendIds("cluster1", false);
                 result = clusterBackendIds;
@@ -416,6 +441,9 @@ public class ColocateTableCheckerAndBalancerTest {
                 myBackend2.isAvailable();
                 result = true;
                 minTimes = 0;
+                myBackend2.getTag();
+                result = Tag.DEFAULT_BACKEND_TAG;
+                minTimes = 0;
 
                 // backend3 not available, and dead for a long time
                 infoService.getBackend(3L);
@@ -430,6 +458,9 @@ public class ColocateTableCheckerAndBalancerTest {
                 myBackend3.getLastUpdateMs();
                 result = System.currentTimeMillis() - Config.tablet_repair_delay_factor_second * 1000 * 20;
                 minTimes = 0;
+                myBackend3.getTag();
+                result = Tag.DEFAULT_BACKEND_TAG;
+                minTimes = 0;
 
                 // backend4 available, not alive but dead for a short time
                 infoService.getBackend(4L);
@@ -444,6 +475,9 @@ public class ColocateTableCheckerAndBalancerTest {
                 myBackend4.getLastUpdateMs();
                 result = System.currentTimeMillis();
                 minTimes = 0;
+                myBackend4.getTag();
+                result = Tag.DEFAULT_BACKEND_TAG;
+                minTimes = 0;
 
                 // backend5 not available, and in decommission
                 infoService.getBackend(5L);
@@ -458,10 +492,52 @@ public class ColocateTableCheckerAndBalancerTest {
                 myBackend5.isDecommissioned();
                 result = true;
                 minTimes = 0;
+                myBackend5.getTag();
+                result = Tag.DEFAULT_BACKEND_TAG;
+                minTimes = 0;
+
+                // backend6 is available, but with different tag
+                infoService.getBackend(5L);
+                result = myBackend6;
+                minTimes = 0;
+                myBackend6.isAvailable();
+                result = false;
+                minTimes = 0;
+                myBackend6.isAlive();
+                result = true;
+                minTimes = 0;
+                myBackend6.isDecommissioned();
+                result = false;
+                minTimes = 0;
+                myBackend6.getTag();
+                result = Tag.create(Tag.TYPE_LOCATION, "new_loc");
+                minTimes = 0;
+
+                // backend7 is available, but in exclude sets
+                infoService.getBackend(5L);
+                result = myBackend7;
+                minTimes = 0;
+                myBackend7.isAvailable();
+                result = false;
+                minTimes = 0;
+                myBackend7.isAlive();
+                result = true;
+                minTimes = 0;
+                myBackend7.isDecommissioned();
+                result = false;
+                minTimes = 0;
+                myBackend7.getTag();
+                result = Tag.DEFAULT_BACKEND_TAG;
+                minTimes = 0;
+                myBackend7.getId();
+                result = 999L;
+                minTimes = 0;
             }
         };
 
-        List<Long> availableBeIds = Deencapsulation.invoke(balancer, "getAvailableBeIds","cluster1", infoService);
-        Assert.assertArrayEquals(new long[]{2L, 4L}, availableBeIds.stream().mapToLong(i->i).sorted().toArray());
+        List<Long> availableBeIds = Deencapsulation.invoke(balancer, "getAvailableBeIds", "cluster1",
+                Tag.DEFAULT_BACKEND_TAG, Sets.newHashSet(999L), infoService);
+        System.out.println(availableBeIds);
+        Assert.assertArrayEquals(new long[]{2L, 4L}, availableBeIds.stream().mapToLong(i -> i).sorted().toArray());
     }
 }
diff --git a/fe/fe-core/src/test/java/org/apache/doris/clone/RebalanceTest.java b/fe/fe-core/src/test/java/org/apache/doris/clone/RebalanceTest.java
index 537fefc..76cd005 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/clone/RebalanceTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/clone/RebalanceTest.java
@@ -17,11 +17,6 @@
 
 package org.apache.doris.clone;
 
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import mockit.Delegate;
-import mockit.Expectations;
-import mockit.Mocked;
 import org.apache.doris.catalog.Catalog;
 import org.apache.doris.catalog.Column;
 import org.apache.doris.catalog.DataProperty;
@@ -33,6 +28,7 @@ import org.apache.doris.catalog.OlapTable;
 import org.apache.doris.catalog.Partition;
 import org.apache.doris.catalog.RangePartitionInfo;
 import org.apache.doris.catalog.Replica;
+import org.apache.doris.catalog.ReplicaAllocation;
 import org.apache.doris.catalog.Tablet;
 import org.apache.doris.catalog.TabletInvertedIndex;
 import org.apache.doris.common.AnalysisException;
@@ -40,6 +36,7 @@ import org.apache.doris.common.Config;
 import org.apache.doris.common.FeConstants;
 import org.apache.doris.common.Pair;
 import org.apache.doris.common.jmockit.Deencapsulation;
+import org.apache.doris.resource.Tag;
 import org.apache.doris.system.SystemInfoService;
 import org.apache.doris.task.AgentBatchTask;
 import org.apache.doris.task.AgentTask;
@@ -50,6 +47,11 @@ import org.apache.doris.thrift.TStatusCode;
 import org.apache.doris.thrift.TStorageMedium;
 import org.apache.doris.thrift.TStorageType;
 import org.apache.doris.thrift.TTabletInfo;
+
+import com.google.common.collect.HashBasedTable;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Table;
+
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -60,12 +62,14 @@ import org.junit.Test;
 
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.function.Function;
 import java.util.stream.Collectors;
 import java.util.stream.LongStream;
 
+import mockit.Delegate;
+import mockit.Expectations;
+import mockit.Mocked;
 import static com.google.common.collect.MoreCollectors.onlyElement;
 
 public class RebalanceTest {
@@ -81,7 +85,7 @@ public class RebalanceTest {
 
     private final SystemInfoService systemInfoService = new SystemInfoService();
     private final TabletInvertedIndex invertedIndex = new TabletInvertedIndex();
-    private Map<String, ClusterLoadStatistic> statisticMap;
+    private Table<String, Tag, ClusterLoadStatistic> statisticMap;
 
     @Before
     public void setUp() throws Exception {
@@ -162,10 +166,10 @@ public class RebalanceTest {
 
     private void generateStatisticMap() {
         ClusterLoadStatistic loadStatistic = new ClusterLoadStatistic(SystemInfoService.DEFAULT_CLUSTER,
-                systemInfoService, invertedIndex);
+                Tag.DEFAULT_BACKEND_TAG, systemInfoService, invertedIndex);
         loadStatistic.init();
-        statisticMap = Maps.newConcurrentMap();
-        statisticMap.put(SystemInfoService.DEFAULT_CLUSTER, loadStatistic);
+        statisticMap = HashBasedTable.create();
+        statisticMap.put(SystemInfoService.DEFAULT_CLUSTER, Tag.DEFAULT_BACKEND_TAG, loadStatistic);
     }
 
     private void createPartitionsForTable(OlapTable olapTable, MaterializedIndex index, Long partitionCount) {
@@ -174,9 +178,9 @@ public class RebalanceTest {
             long id = 31 + idx;
             Partition partition = new Partition(id, "p" + idx, index, new HashDistributionInfo());
             olapTable.addPartition(partition);
-            olapTable.getPartitionInfo().addPartition(id, new DataProperty(TStorageMedium.HDD), (short) 3, false);
+            olapTable.getPartitionInfo().addPartition(id, new DataProperty(TStorageMedium.HDD),
+                    ReplicaAllocation.DEFAULT_ALLOCATION, false);
         });
-
     }
 
     @Test
@@ -282,9 +286,9 @@ public class RebalanceTest {
         Configurator.setLevel("org.apache.doris.clone.MovesInProgressCache", Level.DEBUG);
         MovesCacheMap m = new MovesCacheMap();
         m.updateMapping(statisticMap, 3);
-        m.getCache(SystemInfoService.DEFAULT_CLUSTER, TStorageMedium.HDD).get().put(1L, new Pair<>(null, -1L));
-        m.getCache(SystemInfoService.DEFAULT_CLUSTER, TStorageMedium.SSD).get().put(2L, new Pair<>(null, -1L));
-        m.getCache(SystemInfoService.DEFAULT_CLUSTER, TStorageMedium.SSD).get().put(3L, new Pair<>(null, -1L));
+        m.getCache(SystemInfoService.DEFAULT_CLUSTER, Tag.DEFAULT_BACKEND_TAG, TStorageMedium.HDD).get().put(1L, new Pair<>(null, -1L));
+        m.getCache(SystemInfoService.DEFAULT_CLUSTER, Tag.DEFAULT_BACKEND_TAG, TStorageMedium.SSD).get().put(2L, new Pair<>(null, -1L));
+        m.getCache(SystemInfoService.DEFAULT_CLUSTER, Tag.DEFAULT_BACKEND_TAG, TStorageMedium.SSD).get().put(3L, new Pair<>(null, -1L));
         // Maintenance won't clean up the entries of cache
         m.maintain();
         Assert.assertEquals(3, m.size());
@@ -293,7 +297,7 @@ public class RebalanceTest {
         m.updateMapping(statisticMap, 1);
         Assert.assertEquals(0, m.size());
 
-        m.getCache(SystemInfoService.DEFAULT_CLUSTER, TStorageMedium.SSD).get().put(3L, new Pair<>(null, -1L));
+        m.getCache(SystemInfoService.DEFAULT_CLUSTER, Tag.DEFAULT_BACKEND_TAG, TStorageMedium.SSD).get().put(3L, new Pair<>(null, -1L));
         try {
             Thread.sleep(1000);
             m.maintain();
diff --git a/fe/fe-core/src/test/java/org/apache/doris/clone/TabletRepairAndBalanceTest.java b/fe/fe-core/src/test/java/org/apache/doris/clone/TabletRepairAndBalanceTest.java
new file mode 100644
index 0000000..5099696
--- /dev/null
+++ b/fe/fe-core/src/test/java/org/apache/doris/clone/TabletRepairAndBalanceTest.java
@@ -0,0 +1,515 @@
+// 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.clone;
+
+import org.apache.doris.analysis.AlterSystemStmt;
+import org.apache.doris.analysis.AlterTableStmt;
+import org.apache.doris.analysis.CreateDbStmt;
+import org.apache.doris.analysis.CreateTableStmt;
+import org.apache.doris.analysis.DropTableStmt;
+import org.apache.doris.catalog.Catalog;
+import org.apache.doris.catalog.ColocateGroupSchema;
+import org.apache.doris.catalog.ColocateTableIndex;
+import org.apache.doris.catalog.Database;
+import org.apache.doris.catalog.DiskInfo;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Partition;
+import org.apache.doris.catalog.PartitionInfo;
+import org.apache.doris.catalog.Replica;
+import org.apache.doris.catalog.ReplicaAllocation;
+import org.apache.doris.catalog.TabletInvertedIndex;
+import org.apache.doris.catalog.TabletMeta;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.common.Config;
+import org.apache.doris.common.DdlException;
+import org.apache.doris.common.ExceptionChecker;
+import org.apache.doris.common.FeConstants;
+import org.apache.doris.common.FeMetaVersion;
+import org.apache.doris.common.UserException;
+import org.apache.doris.meta.MetaContext;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.qe.DdlExecutor;
+import org.apache.doris.resource.Tag;
+import org.apache.doris.system.Backend;
+import org.apache.doris.system.SystemInfoService;
+import org.apache.doris.thrift.TDisk;
+import org.apache.doris.thrift.TStorageMedium;
+import org.apache.doris.utframe.UtFrameUtils;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Table;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
+import java.util.UUID;
+
+public class TabletRepairAndBalanceTest {
+    private static final Logger LOG = LogManager.getLogger(TabletRepairAndBalanceTest.class);
+
+    // use a unique dir so that it won't be conflict with other unit test which
+    // may also start a Mocked Frontend
+    private static String runningDirBase = "fe";
+    private static String runningDir = runningDirBase + "/mocked/TabletRepairAndBalanceTest/" + UUID.randomUUID().toString() + "/";
+    private static ConnectContext connectContext;
+
+    private static Random random = new Random(System.currentTimeMillis());
+
+    private static List<Backend> backends = Lists.newArrayList();
+
+    private static Tag tag1;
+    private static Tag tag2;
+
+    static {
+        try {
+            tag1 = Tag.create(Tag.TYPE_LOCATION, "zone1");
+            tag2 = Tag.create(Tag.TYPE_LOCATION, "zone2");
+        } catch (AnalysisException e) {
+            e.printStackTrace();
+        }
+    }
+
+    @BeforeClass
+    public static void beforeClass() throws Exception {
+        System.out.println(runningDir);
+        FeConstants.runningUnitTest = true;
+        FeConstants.tablet_checker_interval_ms = 1000;
+        Config.tablet_repair_delay_factor_second = 1;
+        // 5 backends:
+        // 127.0.0.1
+        // 127.0.0.2
+        // 127.0.0.3
+        // 127.0.0.4
+        // 127.0.0.5
+        UtFrameUtils.createDorisClusterWithMultiTag(runningDir, 5);
+        connectContext = UtFrameUtils.createDefaultCtx();
+
+        // create database
+        String createDbStmtStr = "create database test;";
+        CreateDbStmt createDbStmt = (CreateDbStmt) UtFrameUtils.parseAndAnalyzeStmt(createDbStmtStr, connectContext);
+        Catalog.getCurrentCatalog().createDb(createDbStmt);
+
+        // must set disk info, or the tablet scheduler won't work
+        backends = Catalog.getCurrentSystemInfo().getClusterBackends(SystemInfoService.DEFAULT_CLUSTER);
+        for (Backend be : backends) {
+            Map<String, TDisk> backendDisks = Maps.newHashMap();
+            TDisk tDisk1 = new TDisk();
+            tDisk1.setRootPath("/home/doris.HDD");
+            tDisk1.setDiskTotalCapacity(2000000000);
+            tDisk1.setDataUsedCapacity(1);
+            tDisk1.setUsed(true);
+            tDisk1.setDiskAvailableCapacity(tDisk1.disk_total_capacity - tDisk1.data_used_capacity);
+            tDisk1.setPathHash(random.nextLong());
+            tDisk1.setStorageMedium(TStorageMedium.HDD);
+            backendDisks.put(tDisk1.getRootPath(), tDisk1);
+
+            TDisk tDisk2 = new TDisk();
+            tDisk2.setRootPath("/home/doris.SSD");
+            tDisk2.setDiskTotalCapacity(2000000000);
+            tDisk2.setDataUsedCapacity(1);
+            tDisk2.setUsed(true);
+            tDisk2.setDiskAvailableCapacity(tDisk2.disk_total_capacity - tDisk2.data_used_capacity);
+            tDisk2.setPathHash(random.nextLong());
+            tDisk2.setStorageMedium(TStorageMedium.SSD);
+            backendDisks.put(tDisk2.getRootPath(), tDisk2);
+
+            be.updateDisks(backendDisks);
+        }
+    }
+
+    @AfterClass
+    public static void TearDown() {
+        UtFrameUtils.cleanDorisFeDir(runningDirBase);
+    }
+
+    private static void createTable(String sql) throws Exception {
+        CreateTableStmt createTableStmt = (CreateTableStmt) UtFrameUtils.parseAndAnalyzeStmt(sql, connectContext);
+        Catalog.getCurrentCatalog().createTable(createTableStmt);
+        // must set replicas' path hash, or the tablet scheduler won't work
+        updateReplicaPathHash();
+    }
+
+    private static void dropTable(String sql) throws Exception {
+        DropTableStmt dropTableStmt = (DropTableStmt) UtFrameUtils.parseAndAnalyzeStmt(sql, connectContext);
+        Catalog.getCurrentCatalog().dropTable(dropTableStmt);
+    }
+
+    private static void updateReplicaPathHash() {
+        Table<Long, Long, Replica> replicaMetaTable = Catalog.getCurrentInvertedIndex().getReplicaMetaTable();
+        for (Table.Cell<Long, Long, Replica> cell : replicaMetaTable.cellSet()) {
+            long beId = cell.getColumnKey();
+            Backend be = Catalog.getCurrentSystemInfo().getBackend(beId);
+            if (be == null) {
+                continue;
+            }
+            Replica replica = cell.getValue();
+            TabletMeta tabletMeta = Catalog.getCurrentInvertedIndex().getTabletMeta(cell.getRowKey());
+            ImmutableMap<String, DiskInfo> diskMap = be.getDisks();
+            for (DiskInfo diskInfo : diskMap.values()) {
+                if (diskInfo.getStorageMedium() == tabletMeta.getStorageMedium()) {
+                    replica.setPathHash(diskInfo.getPathHash());
+                    break;
+                }
+            }
+        }
+    }
+
+    private static void alterTable(String sql) throws Exception {
+        AlterTableStmt alterTableStmt = (AlterTableStmt) UtFrameUtils.parseAndAnalyzeStmt(sql, connectContext);
+        Catalog.getCurrentCatalog().getAlterInstance().processAlterTable(alterTableStmt);
+    }
+
+    @Test
+    public void test() throws Exception {
+        Assert.assertEquals(5, backends.size());
+
+        // set tag for all backends. 0-2 to zone1, 4 and 5 to zone2
+        for (int i = 0; i < backends.size(); ++i) {
+            Backend be = backends.get(i);
+            String tag = "zone1";
+            if (i > 2) {
+                tag = "zone2";
+            }
+            String stmtStr = "alter system modify backend \"" + be.getHost() + ":" + be.getHeartbeatPort()
+                    + "\" set ('tag.location' = '" + tag + "')";
+            AlterSystemStmt stmt = (AlterSystemStmt) UtFrameUtils.parseAndAnalyzeStmt(stmtStr, connectContext);
+            DdlExecutor.execute(Catalog.getCurrentCatalog(), stmt);
+        }
+        Tag zone1 = Tag.create(Tag.TYPE_LOCATION, "zone1");
+        Tag zone2 = Tag.create(Tag.TYPE_LOCATION, "zone2");
+        Assert.assertEquals(zone1, backends.get(0).getTag());
+        Assert.assertEquals(zone1, backends.get(1).getTag());
+        Assert.assertEquals(zone1, backends.get(2).getTag());
+        Assert.assertEquals(zone2, backends.get(3).getTag());
+        Assert.assertEquals(zone2, backends.get(4).getTag());
+
+        // create table
+        // 1. no default tag, create will fail
+        String createStr = "create table test.tbl1\n" +
+                "(k1 date, k2 int)\n" +
+                "partition by range(k1)\n" +
+                "(\n" +
+                " partition p1 values less than(\"2021-06-01\"),\n" +
+                " partition p2 values less than(\"2021-07-01\"),\n" +
+                " partition p3 values less than(\"2021-08-01\")\n" +
+                ")\n" +
+                "distributed by hash(k2) buckets 10;";
+        ExceptionChecker.expectThrows(DdlException.class, () -> createTable(createStr));
+
+        // nodes of zone2 not enough, create will fail
+        String createStr2 = "create table test.tbl1\n" +
+                "(k1 date, k2 int)\n" +
+                "partition by range(k1)\n" +
+                "(\n" +
+                " partition p1 values less than(\"2021-06-01\"),\n" +
+                " partition p2 values less than(\"2021-07-01\"),\n" +
+                " partition p3 values less than(\"2021-08-01\")\n" +
+                ")\n" +
+                "distributed by hash(k2) buckets 10\n" +
+                "properties\n" +
+                "(\n" +
+                "    \"replication_allocation\" = \"tag.location.zone1: 2, tag.location.zone2: 3\"\n" +
+                ")";
+        ExceptionChecker.expectThrows(DdlException.class, () -> createTable(createStr2));
+
+        // normal, create success
+        String createStr3 = "create table test.tbl1\n" +
+                "(k1 date, k2 int)\n" +
+                "partition by range(k1)\n" +
+                "(\n" +
+                " partition p1 values less than(\"2021-06-01\"),\n" +
+                " partition p2 values less than(\"2021-07-01\"),\n" +
+                " partition p3 values less than(\"2021-08-01\")\n" +
+                ")\n" +
+                "distributed by hash(k2) buckets 10\n" +
+                "properties\n" +
+                "(\n" +
+                "    \"replication_allocation\" = \"tag.location.zone1: 2, tag.location.zone2: 1\"\n" +
+                ")";
+        ExceptionChecker.expectThrowsNoException(() -> createTable(createStr3));
+        Database db = Catalog.getCurrentCatalog().getDbNullable("default_cluster:test");
+        OlapTable tbl = (OlapTable) db.getTableNullable("tbl1");
+
+        // alter table's replica allocation failed, tag not enough
+        String alterStr = "alter table test.tbl1 set (\"replication_allocation\" = \"tag.location.zone1: 2, tag.location.zone2: 3\");";
+        ExceptionChecker.expectThrows(DdlException.class, () -> alterTable(alterStr));
+        ReplicaAllocation tblReplicaAlloc = tbl.getDefaultReplicaAllocation();
+        Assert.assertEquals(3, tblReplicaAlloc.getTotalReplicaNum());
+        Assert.assertEquals(Short.valueOf((short) 2), tblReplicaAlloc.getReplicaNumByTag(tag1));
+        Assert.assertEquals(Short.valueOf((short) 1), tblReplicaAlloc.getReplicaNumByTag(tag2));
+
+        // alter partition's replica allocation succeed
+        String alterStr2 = "alter table test.tbl1 modify partition p1 set (\"replication_allocation\" = \"tag.location.zone1: 1, tag.location.zone2: 2\");";
+        ExceptionChecker.expectThrowsNoException(() -> alterTable(alterStr2));
+        Partition p1 = tbl.getPartition("p1");
+        ReplicaAllocation p1ReplicaAlloc = tbl.getPartitionInfo().getReplicaAllocation(p1.getId());
+        Assert.assertEquals(3, p1ReplicaAlloc.getTotalReplicaNum());
+        Assert.assertEquals(Short.valueOf((short) 1), p1ReplicaAlloc.getReplicaNumByTag(tag1));
+        Assert.assertEquals(Short.valueOf((short) 2), p1ReplicaAlloc.getReplicaNumByTag(tag2));
+        ExceptionChecker.expectThrows(UserException.class, () -> tbl.checkReplicaAllocation());
+
+        // check backend get() methods
+        SystemInfoService infoService = Catalog.getCurrentSystemInfo();
+        Set<Tag> tags = infoService.getTagsByCluster(SystemInfoService.DEFAULT_CLUSTER);
+        Assert.assertEquals(2, tags.size());
+
+        // check tablet and replica number
+        TabletInvertedIndex invertedIndex = Catalog.getCurrentInvertedIndex();
+        Table<Long, Long, Replica> replicaMetaTable = invertedIndex.getReplicaMetaTable();
+        Assert.assertEquals(30, replicaMetaTable.rowKeySet().size());
+        Assert.assertEquals(5, replicaMetaTable.columnKeySet().size());
+
+        // wait all replica reallocating to correct backend
+        checkTableReplicaAllocation(tbl);
+        Assert.assertEquals(90, replicaMetaTable.cellSet().size());
+
+        // for now, tbl has 3 partitions:
+        // p1: zone1: 1, zone2: 2
+        // p2: zone1: 2, zone2: 1
+        // p3: zone1: 2, zone2: 1
+        // Backends:
+        // [0, 1, 2]: zone1
+        // [3, 4]:    zone2
+
+        // change backend 2 to zone2
+        // set tag for all backends. 0-2 to zone1, 4 and 5 to zone2
+        // and wait all replica reallocating to correct backend
+        Backend be = backends.get(2);
+        String stmtStr = "alter system modify backend \"" + be.getHost() + ":" + be.getHeartbeatPort()
+                + "\" set ('tag.location' = 'zone2')";
+        AlterSystemStmt stmt = (AlterSystemStmt) UtFrameUtils.parseAndAnalyzeStmt(stmtStr, connectContext);
+        DdlExecutor.execute(Catalog.getCurrentCatalog(), stmt);
+        Assert.assertEquals(tag2, be.getTag());
+        ExceptionChecker.expectThrows(UserException.class, () -> tbl.checkReplicaAllocation());
+        checkTableReplicaAllocation(tbl);
+        Assert.assertEquals(90, replicaMetaTable.cellSet().size());
+
+        // For now, Backends:
+        // [0, 1]:      zone1
+        // [2, 3, 4]:   zone2
+        // begin to test colocation table
+        String createStr4 = "create table test.col_tbl1\n" +
+                "(k1 date, k2 int)\n" +
+                "partition by range(k1)\n" +
+                "(\n" +
+                " partition p1 values less than(\"2021-06-01\"),\n" +
+                " partition p2 values less than(\"2021-07-01\"),\n" +
+                " partition p3 values less than(\"2021-08-01\")\n" +
+                ")\n" +
+                "distributed by hash(k2) buckets 10\n" +
+                "properties\n" +
+                "(\n" +
+                "    \"replication_allocation\" = \"tag.location.zone1: 2, tag.location.zone2: 1\",\n" +
+                "    \"colocate_with\" = \"g1\"\n" +
+                ")";
+        ExceptionChecker.expectThrowsNoException(() -> createTable(createStr4));
+        String createStr5 = "create table test.col_tbl2\n" +
+                "(k1 date, k2 int)\n" +
+                "partition by range(k1)\n" +
+                "(\n" +
+                " partition p1 values less than(\"2021-06-01\"),\n" +
+                " partition p2 values less than(\"2021-07-01\"),\n" +
+                " partition p3 values less than(\"2021-08-01\")\n" +
+                ")\n" +
+                "distributed by hash(k2) buckets 10\n" +
+                "properties\n" +
+                "(\n" +
+                "    \"replication_allocation\" = \"tag.location.zone1: 2, tag.location.zone2: 1\",\n" +
+                "    \"colocate_with\" = \"g1\"\n" +
+                ")";
+        ExceptionChecker.expectThrowsNoException(() -> createTable(createStr5));
+
+        OlapTable colTbl1 = (OlapTable) db.getTableNullable("col_tbl1");
+        OlapTable colTbl2 = (OlapTable) db.getTableNullable("col_tbl2");
+        Assert.assertNotNull(colTbl1);
+        Assert.assertNotNull(colTbl2);
+        ColocateTableIndex colocateTableIndex = Catalog.getCurrentColocateIndex();
+        ColocateTableIndex.GroupId groupId = colocateTableIndex.getGroup(colTbl1.getId());
+        Assert.assertEquals(groupId, colocateTableIndex.getGroup(colTbl2.getId()));
+        ColocateGroupSchema groupSchema = colocateTableIndex.getGroupSchema(groupId);
+        ReplicaAllocation groupReplicaAlloc = groupSchema.getReplicaAlloc();
+        Assert.assertEquals(groupReplicaAlloc, colTbl1.getDefaultReplicaAllocation());
+        Assert.assertEquals(groupReplicaAlloc, colTbl2.getDefaultReplicaAllocation());
+        checkTableReplicaAllocation(colTbl1);
+        checkTableReplicaAllocation(colTbl2);
+
+        // change backend4's tag to zone1:
+        // [0, 1, 4]: zone1
+        // [2, 3]:    zone2
+        be = backends.get(4);
+        stmtStr = "alter system modify backend \"" + be.getHost() + ":" + be.getHeartbeatPort()
+                + "\" set ('tag.location' = 'zone1')";
+        stmt = (AlterSystemStmt) UtFrameUtils.parseAndAnalyzeStmt(stmtStr, connectContext);
+        DdlExecutor.execute(Catalog.getCurrentCatalog(), stmt);
+        Assert.assertEquals(tag1, be.getTag());
+        ExceptionChecker.expectThrows(UserException.class, () -> tbl.checkReplicaAllocation());
+
+        checkTableReplicaAllocation(colTbl1);
+        checkTableReplicaAllocation(colTbl2);
+
+        // for now,
+        // backends' tag:
+        // [0, 1, 4]: zone1
+        // [2, 3]:    zone2
+        //
+        // colocate group(col_tbl1, col_tbl2) replica allocation is: zone1:2, zone2:1
+        // tbl1's replica allocation is:
+        //      p1: zone1:1, zone2:2
+        //      p2,p2: zone1:2, zone2:1
+
+        // change tbl1's default replica allocation to zone1:4, which is allowed
+        String alterStr3 = "alter table test.tbl1 set ('default.replication_allocation' = 'tag.location.zone1:4')";
+        ExceptionChecker.expectThrowsNoException(() -> alterTable(alterStr3));
+
+        // change tbl1's p1's replica allocation to zone1:4, which is forbidden
+        String alterStr4 = "alter table test.tbl1 modify partition p1 set ('replication_allocation' = 'tag.location.zone1:4')";
+        ExceptionChecker.expectThrows(DdlException.class, () -> alterTable(alterStr4));
+
+        // change col_tbl1's default replica allocation to zone2:4, which is allowed
+        String alterStr5 = "alter table test.col_tbl1 set ('default.replication_allocation' = 'tag.location.zone2:4')";
+        ExceptionChecker.expectThrowsNoException(() -> alterTable(alterStr5));
+
+        // Drop all tables
+        String dropStmt1 = "drop table test.tbl1 force";
+        String dropStmt2 = "drop table test.col_tbl1 force";
+        String dropStmt3 = "drop table test.col_tbl2 force";
+        ExceptionChecker.expectThrowsNoException(() -> dropTable(dropStmt1));
+        ExceptionChecker.expectThrowsNoException(() -> dropTable(dropStmt2));
+        ExceptionChecker.expectThrowsNoException(() -> dropTable(dropStmt3));
+        Assert.assertEquals(0, replicaMetaTable.size());
+
+        // set all backends' tag to default
+        for (int i = 0; i < backends.size(); ++i) {
+            Backend backend = backends.get(i);
+            String backendStmt = "alter system modify backend \"" + backend.getHost() + ":" + backend.getHeartbeatPort()
+                    + "\" set ('tag.location' = 'default')";
+            AlterSystemStmt systemStmt = (AlterSystemStmt) UtFrameUtils.parseAndAnalyzeStmt(backendStmt, connectContext);
+            DdlExecutor.execute(Catalog.getCurrentCatalog(), systemStmt);
+        }
+        Assert.assertEquals(Tag.DEFAULT_BACKEND_TAG, backends.get(0).getTag());
+        Assert.assertEquals(Tag.DEFAULT_BACKEND_TAG, backends.get(1).getTag());
+        Assert.assertEquals(Tag.DEFAULT_BACKEND_TAG, backends.get(2).getTag());
+        Assert.assertEquals(Tag.DEFAULT_BACKEND_TAG, backends.get(3).getTag());
+        Assert.assertEquals(Tag.DEFAULT_BACKEND_TAG, backends.get(4).getTag());
+
+        // create table tbl2 with "replication_num" property
+        String createStmt = "create table test.tbl2\n" +
+                "(k1 date, k2 int)\n" +
+                "partition by range(k1)\n" +
+                "(\n" +
+                " partition p1 values less than(\"2021-06-01\"),\n" +
+                " partition p2 values less than(\"2021-07-01\"),\n" +
+                " partition p3 values less than(\"2021-08-01\")\n" +
+                ")\n" +
+                "distributed by hash(k2) buckets 10;";
+        ExceptionChecker.expectThrowsNoException(() -> createTable(createStmt));
+        OlapTable tbl2 = (OlapTable) db.getTableNullable("tbl2");
+        ReplicaAllocation defaultAlloc = new ReplicaAllocation((short) 3);
+        Assert.assertEquals(defaultAlloc, tbl2.getDefaultReplicaAllocation());
+        for (Partition partition : tbl2.getPartitions()) {
+            Assert.assertEquals(defaultAlloc, tbl2.getPartitionInfo().getReplicaAllocation(partition.getId()));
+        }
+
+        // add new partition to tbl2
+        String alterStr6 = "alter table test.tbl2 add partition p4 values less than('2021-09-01')";
+        ExceptionChecker.expectThrowsNoException(() -> alterTable(alterStr6));
+        Assert.assertEquals(4, tbl2.getPartitionNames().size());
+        PartitionInfo partitionInfo = tbl2.getPartitionInfo();
+        Assert.assertEquals(ReplicaAllocation.DEFAULT_ALLOCATION, partitionInfo.getReplicaAllocation(tbl2.getPartition("p4").getId()));
+
+        // change tbl2 to a colocate table
+        String alterStr7 = "alter table test.tbl2 SET (\"colocate_with\"=\"newg\")";
+        ExceptionChecker.expectThrowsNoException(() -> alterTable(alterStr7));
+        ColocateTableIndex.GroupId groupId1 = colocateTableIndex.getGroup(tbl2.getId());
+        Assert.assertEquals(ReplicaAllocation.DEFAULT_ALLOCATION, colocateTableIndex.getGroupSchema(groupId1).getReplicaAlloc());
+
+        // test colocate table index persist
+        ExceptionChecker.expectThrowsNoException(() -> testColocateTableIndexSerialization(colocateTableIndex));
+    }
+
+    private void testColocateTableIndexSerialization(ColocateTableIndex colocateTableIndex) throws IOException {
+        MetaContext metaContext = new MetaContext();
+        metaContext.setMetaVersion(FeMetaVersion.VERSION_CURRENT);
+        metaContext.setThreadLocalInfo();
+
+        // 1. Write objects to file
+        File file = new File("./ColocateTableIndexPersist");
+        file.createNewFile();
+        DataOutputStream dos = new DataOutputStream(new FileOutputStream(file));
+        colocateTableIndex.write(dos);
+        dos.flush();
+        dos.close();
+
+        // 2. Read objects from file
+        DataInputStream dis = new DataInputStream(new FileInputStream(file));
+
+        ColocateTableIndex rColocateTableIndex = new ColocateTableIndex();
+        rColocateTableIndex.readFields(dis);
+
+        Assert.assertEquals(1, colocateTableIndex.getAllGroupIds().size());
+        Set<ColocateTableIndex.GroupId> allGroupIds = colocateTableIndex.getAllGroupIds();
+        for (ColocateTableIndex.GroupId groupId : allGroupIds) {
+            Map<Tag, List<List<Long>>> backendsPerBucketSeq = colocateTableIndex.getBackendsPerBucketSeq(groupId);
+            for (Map.Entry<Tag, List<List<Long>>> entry : backendsPerBucketSeq.entrySet()) {
+                List<List<Long>> seq = entry.getValue();
+                Assert.assertEquals(10, seq.size());
+                for (List<Long> beIds : seq) {
+                    Assert.assertEquals(3, beIds.size());
+                }
+            }
+        }
+
+        // 3. delete files
+        dis.close();
+        file.delete();
+    }
+
+    private void checkTableReplicaAllocation(OlapTable tbl) throws InterruptedException {
+        int maxLoop = 300;
+        while (maxLoop-- > 0) {
+            try {
+                tbl.checkReplicaAllocation();
+                break;
+            } catch (UserException e) {
+                System.out.println(e.getMessage());
+            }
+            Thread.sleep(1000);
+            System.out.println("wait table " + tbl.getId() + " to be stable");
+        }
+        ExceptionChecker.expectThrowsNoException(() -> tbl.checkReplicaAllocation());
+        System.out.println("table " + tbl.getId() + " is stable");
+    }
+}
+
+
diff --git a/fe/fe-core/src/test/java/org/apache/doris/clone/TabletSchedCtxTest.java b/fe/fe-core/src/test/java/org/apache/doris/clone/TabletSchedCtxTest.java
index 5733677..b9d9ed2 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/clone/TabletSchedCtxTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/clone/TabletSchedCtxTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.doris.clone;
 
+import org.apache.doris.catalog.ReplicaAllocation;
 import org.apache.doris.clone.TabletSchedCtx.Priority;
 import org.apache.doris.clone.TabletSchedCtx.Type;
 
@@ -31,18 +32,19 @@ public class TabletSchedCtxTest {
     public void testPriorityCompare() {
         // equal priority, but info3's last visit time is earlier than info2 and info1, so info1 should ranks ahead
         PriorityQueue<TabletSchedCtx> pendingTablets = new PriorityQueue<>();
+        ReplicaAllocation replicaAlloc = ReplicaAllocation.DEFAULT_ALLOCATION;
         TabletSchedCtx ctx1 = new TabletSchedCtx(Type.REPAIR, "default_cluster",
-                1, 2, 3, 4, 1000, System.currentTimeMillis());
+                1, 2, 3, 4, 1000, replicaAlloc, System.currentTimeMillis());
         ctx1.setOrigPriority(Priority.NORMAL);
         ctx1.setLastVisitedTime(2);
 
         TabletSchedCtx ctx2 = new TabletSchedCtx(Type.REPAIR, "default_cluster",
-                1, 2, 3, 4, 1001, System.currentTimeMillis());
+                1, 2, 3, 4, 1001, replicaAlloc, System.currentTimeMillis());
         ctx2.setOrigPriority(Priority.NORMAL);
         ctx2.setLastVisitedTime(3);
 
         TabletSchedCtx ctx3 = new TabletSchedCtx(Type.REPAIR, "default_cluster",
-                1, 2, 3, 4, 1001, System.currentTimeMillis());
+                1, 2, 3, 4, 1001, replicaAlloc, System.currentTimeMillis());
         ctx3.setOrigPriority(Priority.NORMAL);
         ctx3.setLastVisitedTime(1);
 
diff --git a/fe/fe-core/src/test/java/org/apache/doris/cluster/SystemInfoServiceTest.java b/fe/fe-core/src/test/java/org/apache/doris/cluster/SystemInfoServiceTest.java
index 6a24b2a..69ad34b 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/cluster/SystemInfoServiceTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/cluster/SystemInfoServiceTest.java
@@ -27,11 +27,13 @@ import org.apache.doris.catalog.TabletInvertedIndex;
 import org.apache.doris.common.AnalysisException;
 import org.apache.doris.common.DdlException;
 import org.apache.doris.common.FeConstants;
+import org.apache.doris.common.UserException;
 import org.apache.doris.common.io.CountingDataOutputStream;
 import org.apache.doris.persist.EditLog;
... 1137 lines suppressed ...

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