diff --git a/fluss-client/src/main/java/org/apache/fluss/client/admin/Admin.java b/fluss-client/src/main/java/org/apache/fluss/client/admin/Admin.java index 1ef5c30ebd..cbb9b1ac4c 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/admin/Admin.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/admin/Admin.java @@ -22,9 +22,14 @@ import org.apache.fluss.client.metadata.KvSnapshots; import org.apache.fluss.client.metadata.LakeSnapshot; import org.apache.fluss.cluster.ServerNode; +import org.apache.fluss.cluster.rebalance.GoalType; +import org.apache.fluss.cluster.rebalance.RebalancePlanForBucket; +import org.apache.fluss.cluster.rebalance.RebalanceResultForBucket; +import org.apache.fluss.cluster.rebalance.ServerTag; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.cluster.AlterConfig; import org.apache.fluss.config.cluster.ConfigEntry; +import org.apache.fluss.exception.AuthorizationException; import org.apache.fluss.exception.DatabaseAlreadyExistException; import org.apache.fluss.exception.DatabaseNotEmptyException; import org.apache.fluss.exception.DatabaseNotExistException; @@ -35,10 +40,15 @@ import org.apache.fluss.exception.InvalidTableException; import org.apache.fluss.exception.KvSnapshotNotExistException; import org.apache.fluss.exception.LakeTableSnapshotNotExistException; +import org.apache.fluss.exception.NoRebalanceInProgressException; import org.apache.fluss.exception.NonPrimaryKeyTableException; import org.apache.fluss.exception.PartitionAlreadyExistsException; import org.apache.fluss.exception.PartitionNotExistException; +import org.apache.fluss.exception.RebalanceFailureException; import org.apache.fluss.exception.SchemaNotExistException; +import org.apache.fluss.exception.ServerNotExistException; +import org.apache.fluss.exception.ServerTagAlreadyExistException; +import org.apache.fluss.exception.ServerTagNotExistException; import org.apache.fluss.exception.TableAlreadyExistException; import org.apache.fluss.exception.TableNotExistException; import org.apache.fluss.exception.TableNotPartitionedException; @@ -60,6 +70,7 @@ import java.util.Collection; import java.util.List; +import java.util.Map; import java.util.concurrent.CompletableFuture; /** @@ -492,4 +503,90 @@ ListOffsetsResult listOffsets( * @return A CompletableFuture indicating completion of the operation. */ CompletableFuture alterClusterConfigs(Collection configs); + + /** + * Add server tag to the specified tabletServers, one tabletServer can only have one serverTag. + * + *

If one tabletServer failed adding tag, none of the tags will take effect. + * + *

+ * + * @param tabletServers the tabletServers we want to add server tags. + * @param serverTag the server tag to be added. + */ + CompletableFuture addServerTag(List tabletServers, ServerTag serverTag); + + /** + * Remove server tag from the specified tabletServers. + * + *

If one tabletServer failed removing tag, none of the tags will be removed. + * + *

    + *
  • {@link AuthorizationException} If the authenticated user doesn't have reset config + * access to the cluster. + *
  • {@link ServerNotExistException} If the tabletServer in {@code tabletServers} does not + * exist. + *
  • {@link ServerTagNotExistException} If the server tag does not exist when {@code + * overWriteIfExists} is false. + *
+ * + * @param tabletServers the tabletServers we want to remove server tags. + */ + CompletableFuture removeServerTag(List tabletServers, ServerTag serverTag); + + /** + * Based on the provided {@code priorityGoals}, Fluss performs load balancing on the cluster's + * bucket load. + * + *

More details, Fluss collects the cluster's load information and optimizes to perform load + * balancing according to the user-defined {@code priorityGoals}. + * + *

Currently, Fluss only supports one active rebalance task in the cluster. If an uncompleted + * rebalance task exists, an {@link RebalanceFailureException} will be thrown. + * + *

    + *
  • {@link AuthorizationException} If the authenticated user doesn't have reset config + * access to the cluster. + *
  • {@link RebalanceFailureException} If the rebalance failed. Such as there is an ongoing + * execution. + *
+ * + * @param priorityGoals the goals to be optimized. + * @param dryRun Calculate and return the rebalance optimization proposal, but do not execute + * it. + * @return the generated rebalance plan for all the tableBuckets which need to do rebalance. + */ + CompletableFuture> rebalance( + List priorityGoals, boolean dryRun); + + /** + * List the rebalance process. + * + *
    + *
  • {@link AuthorizationException} If the authenticated user doesn't have reset config + * access to the cluster. + *
  • {@link NoRebalanceInProgressException} If there are no rebalance tasks in progress. + *
+ * + * @return the rebalance process for all the tableBuckets doing rebalance. + */ + CompletableFuture> listRebalanceProcess(); + + /** + * Cannel the rebalance task. + * + *
    + *
  • {@link AuthorizationException} If the authenticated user doesn't have reset config + * access to the cluster. + *
  • {@link NoRebalanceInProgressException} If there are no rebalance tasks in progress. + *
+ */ + CompletableFuture cancelRebalance(); } diff --git a/fluss-client/src/main/java/org/apache/fluss/client/admin/FlussAdmin.java b/fluss-client/src/main/java/org/apache/fluss/client/admin/FlussAdmin.java index 3dc4c16c90..8f2767248a 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/admin/FlussAdmin.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/admin/FlussAdmin.java @@ -24,6 +24,10 @@ import org.apache.fluss.client.utils.ClientRpcMessageUtils; import org.apache.fluss.cluster.Cluster; import org.apache.fluss.cluster.ServerNode; +import org.apache.fluss.cluster.rebalance.GoalType; +import org.apache.fluss.cluster.rebalance.RebalancePlanForBucket; +import org.apache.fluss.cluster.rebalance.RebalanceResultForBucket; +import org.apache.fluss.cluster.rebalance.ServerTag; import org.apache.fluss.config.cluster.AlterConfig; import org.apache.fluss.config.cluster.ConfigEntry; import org.apache.fluss.exception.LeaderNotAvailableException; @@ -44,8 +48,10 @@ import org.apache.fluss.rpc.gateway.AdminGateway; import org.apache.fluss.rpc.gateway.AdminReadOnlyGateway; import org.apache.fluss.rpc.gateway.TabletServerGateway; +import org.apache.fluss.rpc.messages.AddServerTagRequest; import org.apache.fluss.rpc.messages.AlterClusterConfigsRequest; import org.apache.fluss.rpc.messages.AlterTableRequest; +import org.apache.fluss.rpc.messages.CancelRebalanceRequest; import org.apache.fluss.rpc.messages.CreateAclsRequest; import org.apache.fluss.rpc.messages.CreateDatabaseRequest; import org.apache.fluss.rpc.messages.CreateTableRequest; @@ -66,12 +72,15 @@ import org.apache.fluss.rpc.messages.ListDatabasesResponse; import org.apache.fluss.rpc.messages.ListOffsetsRequest; import org.apache.fluss.rpc.messages.ListPartitionInfosRequest; +import org.apache.fluss.rpc.messages.ListRebalanceProcessRequest; import org.apache.fluss.rpc.messages.ListTablesRequest; import org.apache.fluss.rpc.messages.ListTablesResponse; import org.apache.fluss.rpc.messages.PbAlterConfig; import org.apache.fluss.rpc.messages.PbListOffsetsRespForBucket; import org.apache.fluss.rpc.messages.PbPartitionSpec; import org.apache.fluss.rpc.messages.PbTablePath; +import org.apache.fluss.rpc.messages.RebalanceRequest; +import org.apache.fluss.rpc.messages.RemoveServerTagRequest; import org.apache.fluss.rpc.messages.TableExistsRequest; import org.apache.fluss.rpc.messages.TableExistsResponse; import org.apache.fluss.rpc.protocol.ApiError; @@ -535,6 +544,41 @@ public CompletableFuture alterClusterConfigs(Collection confi return future; } + @Override + public CompletableFuture addServerTag(List tabletServers, ServerTag serverTag) { + AddServerTagRequest request = new AddServerTagRequest().setServerTag(serverTag.value); + tabletServers.forEach(request::addServerId); + return gateway.addServerTag(request).thenApply(r -> null); + } + + @Override + public CompletableFuture removeServerTag( + List tabletServers, ServerTag serverTag) { + RemoveServerTagRequest request = new RemoveServerTagRequest().setServerTag(serverTag.value); + tabletServers.forEach(request::addServerId); + return gateway.removeServerTag(request).thenApply(r -> null); + } + + @Override + public CompletableFuture> rebalance( + List priorityGoals, boolean dryRun) { + RebalanceRequest request = new RebalanceRequest().setDryRun(dryRun); + priorityGoals.forEach(goal -> request.addGoal(goal.value)); + return gateway.rebalance(request).thenApply(ClientRpcMessageUtils::toRebalancePlan); + } + + @Override + public CompletableFuture> listRebalanceProcess() { + return gateway.listRebalanceProcess(new ListRebalanceProcessRequest()) + .thenApply(ClientRpcMessageUtils::toRebalanceProcess); + } + + @Override + public CompletableFuture cancelRebalance() { + CancelRebalanceRequest request = new CancelRebalanceRequest(); + return gateway.cancelRebalance(request).thenApply(r -> null); + } + @Override public void close() { // nothing to do yet diff --git a/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java b/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java index b98b6fb6ca..b655641736 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/utils/ClientRpcMessageUtils.java @@ -25,6 +25,9 @@ import org.apache.fluss.client.metadata.LakeSnapshot; import org.apache.fluss.client.write.KvWriteBatch; import org.apache.fluss.client.write.ReadyWriteBatch; +import org.apache.fluss.cluster.rebalance.RebalancePlanForBucket; +import org.apache.fluss.cluster.rebalance.RebalanceResultForBucket; +import org.apache.fluss.cluster.rebalance.RebalanceStatusForBucket; import org.apache.fluss.config.cluster.AlterConfigOpType; import org.apache.fluss.config.cluster.ConfigEntry; import org.apache.fluss.fs.FsPath; @@ -44,6 +47,7 @@ import org.apache.fluss.rpc.messages.GetLatestLakeSnapshotResponse; import org.apache.fluss.rpc.messages.ListOffsetsRequest; import org.apache.fluss.rpc.messages.ListPartitionInfosResponse; +import org.apache.fluss.rpc.messages.ListRebalanceProcessResponse; import org.apache.fluss.rpc.messages.LookupRequest; import org.apache.fluss.rpc.messages.MetadataRequest; import org.apache.fluss.rpc.messages.PbAlterConfig; @@ -56,10 +60,17 @@ import org.apache.fluss.rpc.messages.PbPrefixLookupReqForBucket; import org.apache.fluss.rpc.messages.PbProduceLogReqForBucket; import org.apache.fluss.rpc.messages.PbPutKvReqForBucket; +import org.apache.fluss.rpc.messages.PbRebalancePlanForBucket; +import org.apache.fluss.rpc.messages.PbRebalancePlanForPartition; +import org.apache.fluss.rpc.messages.PbRebalancePlanForTable; +import org.apache.fluss.rpc.messages.PbRebalanceProcessForBucket; +import org.apache.fluss.rpc.messages.PbRebalanceProcessForPartition; +import org.apache.fluss.rpc.messages.PbRebalanceProcessForTable; import org.apache.fluss.rpc.messages.PbRemotePathAndLocalFile; import org.apache.fluss.rpc.messages.PrefixLookupRequest; import org.apache.fluss.rpc.messages.ProduceLogRequest; import org.apache.fluss.rpc.messages.PutKvRequest; +import org.apache.fluss.rpc.messages.RebalanceResponse; import javax.annotation.Nullable; @@ -328,6 +339,99 @@ public static DropPartitionRequest makeDropPartitionRequest( return dropPartitionRequest; } + public static Map toRebalancePlan( + RebalanceResponse response) { + Map rebalancePlan = new HashMap<>(); + for (PbRebalancePlanForTable pbTable : response.getPlanForTablesList()) { + long tableId = pbTable.getTableId(); + if (pbTable.getPartitionsPlansCount() == 0) { + // none-partition table. + for (PbRebalancePlanForBucket pbBucket : pbTable.getBucketsPlansList()) { + int bucketId = pbBucket.getBucketId(); + rebalancePlan.put( + new TableBucket(tableId, null, bucketId), + toRebalancePlanForBucket(tableId, null, bucketId, pbBucket)); + } + } else { + // partition table. + for (PbRebalancePlanForPartition pbPartition : pbTable.getPartitionsPlansList()) { + long partitionId = pbPartition.getPartitionId(); + for (PbRebalancePlanForBucket pbBucket : pbPartition.getBucketsPlansList()) { + int bucketId = pbBucket.getBucketId(); + rebalancePlan.put( + new TableBucket(tableId, partitionId, bucketId), + toRebalancePlanForBucket(tableId, partitionId, bucketId, pbBucket)); + } + } + } + } + return rebalancePlan; + } + + private static RebalancePlanForBucket toRebalancePlanForBucket( + long tableId, + @Nullable Long partitionId, + int bucketId, + PbRebalancePlanForBucket pbBucket) { + return new RebalancePlanForBucket( + new TableBucket(tableId, partitionId, bucketId), + pbBucket.getOriginalLeader(), + pbBucket.getNewLeader(), + Arrays.stream(pbBucket.getOriginalReplicas()).boxed().collect(Collectors.toList()), + Arrays.stream(pbBucket.getNewReplicas()).boxed().collect(Collectors.toList())); + } + + public static Map toRebalanceProcess( + ListRebalanceProcessResponse response) { + Map rebalanceProcess = new HashMap<>(); + + for (PbRebalanceProcessForTable pbRebalanceProcessForTable : + response.getProcessForTablesList()) { + long tableId = pbRebalanceProcessForTable.getTableId(); + + for (PbRebalanceProcessForPartition pbRebalanceProcessForPartition : + pbRebalanceProcessForTable.getPartitionsProcessesList()) { + long partitionId = pbRebalanceProcessForPartition.getPartitionId(); + + for (PbRebalanceProcessForBucket pbRebalanceProcessForBucket : + pbRebalanceProcessForPartition.getBucketsProcessesList()) { + int bucketId = pbRebalanceProcessForBucket.getBucketId(); + TableBucket tableBucket = new TableBucket(tableId, partitionId, bucketId); + rebalanceProcess.put( + tableBucket, + toRebalanceResultForBucket(tableBucket, pbRebalanceProcessForBucket)); + } + } + + for (PbRebalanceProcessForBucket pbRebalanceProcessForBucket : + pbRebalanceProcessForTable.getBucketsProcessesList()) { + int bucketId = pbRebalanceProcessForBucket.getBucketId(); + TableBucket tableBucket = new TableBucket(tableId, null, bucketId); + rebalanceProcess.put( + tableBucket, + toRebalanceResultForBucket(tableBucket, pbRebalanceProcessForBucket)); + } + } + + return rebalanceProcess; + } + + private static RebalanceResultForBucket toRebalanceResultForBucket( + TableBucket tableBucket, PbRebalanceProcessForBucket pbRebalanceProcessForBucket) { + return RebalanceResultForBucket.of( + new RebalancePlanForBucket( + tableBucket, + pbRebalanceProcessForBucket.getOriginalReplicas()[0], + pbRebalanceProcessForBucket.getNewReplicas()[0], + Arrays.stream(pbRebalanceProcessForBucket.getOriginalReplicas()) + .boxed() + .collect(Collectors.toList()), + Arrays.stream(pbRebalanceProcessForBucket.getNewReplicas()) + .boxed() + .collect(Collectors.toList())), + RebalanceStatusForBucket.of(pbRebalanceProcessForBucket.getRebalanceStatus())); + } + public static List toPartitionInfos(ListPartitionInfosResponse response) { return response.getPartitionsInfosList().stream() .map( diff --git a/fluss-client/src/test/java/org/apache/fluss/client/admin/FlussAdminITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/admin/FlussAdminITCase.java index 860d63f361..4029da97dc 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/admin/FlussAdminITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/admin/FlussAdminITCase.java @@ -24,6 +24,7 @@ import org.apache.fluss.client.table.Table; import org.apache.fluss.client.table.writer.UpsertWriter; import org.apache.fluss.cluster.ServerNode; +import org.apache.fluss.cluster.rebalance.ServerTag; import org.apache.fluss.config.AutoPartitionTimeUnit; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; @@ -42,6 +43,9 @@ import org.apache.fluss.exception.PartitionAlreadyExistsException; import org.apache.fluss.exception.PartitionNotExistException; import org.apache.fluss.exception.SchemaNotExistException; +import org.apache.fluss.exception.ServerNotExistException; +import org.apache.fluss.exception.ServerTagAlreadyExistException; +import org.apache.fluss.exception.ServerTagNotExistException; import org.apache.fluss.exception.TableNotExistException; import org.apache.fluss.exception.TableNotPartitionedException; import org.apache.fluss.exception.TooManyBucketsException; @@ -64,6 +68,7 @@ import org.apache.fluss.metadata.TablePath; import org.apache.fluss.server.kv.snapshot.CompletedSnapshot; import org.apache.fluss.server.kv.snapshot.KvSnapshotHandle; +import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.types.DataTypes; import org.junit.jupiter.api.BeforeEach; @@ -1282,4 +1287,63 @@ public void testSystemsColumns() throws Exception { + "Please use other names for these columns. " + "The reserved system columns are: __offset, __timestamp, __bucket"); } + + @Test + public void testAddAndRemoveServerTags() throws Exception { + ZooKeeperClient zkClient = FLUSS_CLUSTER_EXTENSION.getZooKeeperClient(); + // 1.add server tag to a none exists server. + assertThatThrownBy( + () -> + admin.addServerTag( + Collections.singletonList(100), + ServerTag.PERMANENT_OFFLINE) + .get()) + .cause() + .isInstanceOf(ServerNotExistException.class) + .hasMessageContaining("Server 100 not exists when trying to add server tag."); + + // 2.add server tag for server 0,1. + admin.addServerTag(Arrays.asList(0, 1), ServerTag.PERMANENT_OFFLINE).get(); + // TODO use api to get serverTags instead of getting from zk directly + assertThat(zkClient.getServerTags()).isPresent(); + assertThat(zkClient.getServerTags().get().getServerTags()) + .containsEntry(0, ServerTag.PERMANENT_OFFLINE) + .containsEntry(1, ServerTag.PERMANENT_OFFLINE); + + // 3.add server tag for server 0,2. error will be thrown and tag for 2 will not be added. + assertThatThrownBy( + () -> + admin.addServerTag(Arrays.asList(0, 2), ServerTag.PERMANENT_OFFLINE) + .get()) + .cause() + .isInstanceOf(ServerTagAlreadyExistException.class) + .hasMessageContaining("Server tag PERMANENT_OFFLINE already exists for server 0."); + + // 4.remove server tag for server 100 + assertThatThrownBy( + () -> + admin.removeServerTag( + Collections.singletonList(100), + ServerTag.PERMANENT_OFFLINE) + .get()) + .cause() + .isInstanceOf(ServerNotExistException.class) + .hasMessageContaining("Server 100 not exists when trying to removing server tag."); + + // 5.remove server tag for server 0,1. + admin.removeServerTag(Arrays.asList(0, 1), ServerTag.PERMANENT_OFFLINE).get(); + assertThat(zkClient.getServerTags()).isPresent(); + assertThat(zkClient.getServerTags().get().getServerTags()).isEmpty(); + + // 6.remove server tag for server 2. error will be thrown and tag for 2 will not be removed. + assertThatThrownBy( + () -> + admin.removeServerTag( + Collections.singletonList(0), + ServerTag.PERMANENT_OFFLINE) + .get()) + .cause() + .isInstanceOf(ServerTagNotExistException.class) + .hasMessageContaining("Server tag PERMANENT_OFFLINE not exists for server 0."); + } } diff --git a/fluss-client/src/test/java/org/apache/fluss/client/admin/RebalanceITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/admin/RebalanceITCase.java new file mode 100644 index 0000000000..0ad6b4d204 --- /dev/null +++ b/fluss-client/src/test/java/org/apache/fluss/client/admin/RebalanceITCase.java @@ -0,0 +1,221 @@ +/* + * 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.fluss.client.admin; + +import org.apache.fluss.client.Connection; +import org.apache.fluss.client.ConnectionFactory; +import org.apache.fluss.cluster.rebalance.GoalType; +import org.apache.fluss.cluster.rebalance.RebalancePlanForBucket; +import org.apache.fluss.cluster.rebalance.RebalanceResultForBucket; +import org.apache.fluss.cluster.rebalance.ServerTag; +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.metadata.DatabaseDescriptor; +import org.apache.fluss.metadata.PartitionSpec; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TableDescriptor; +import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.server.replica.ReplicaManager; +import org.apache.fluss.server.testutils.FlussClusterExtension; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +import java.time.Duration; +import java.util.Arrays; +import java.util.Collections; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.apache.fluss.record.TestData.DATA1_SCHEMA; +import static org.apache.fluss.record.TestData.DATA1_TABLE_DESCRIPTOR; +import static org.apache.fluss.testutils.common.CommonTestUtils.retry; +import static org.assertj.core.api.Assertions.assertThat; + +/** ITCase for rebalance. */ +public class RebalanceITCase { + @RegisterExtension + public static final FlussClusterExtension FLUSS_CLUSTER_EXTENSION = + FlussClusterExtension.builder() + .setNumOfTabletServers(4) + .setClusterConf(initConfig()) + .build(); + + private Connection conn; + private Admin admin; + + @BeforeEach + protected void setup() throws Exception { + conn = ConnectionFactory.createConnection(FLUSS_CLUSTER_EXTENSION.getClientConfig()); + admin = conn.getAdmin(); + } + + @AfterEach + protected void teardown() throws Exception { + if (admin != null) { + admin.close(); + admin = null; + } + + if (conn != null) { + conn.close(); + conn = null; + } + } + + @Test + void testRebalanceForLogTable() throws Exception { + String dbName = "db-balance"; + admin.createDatabase(dbName, DatabaseDescriptor.EMPTY, false).get(); + // create somne none partitioned log table. + for (int i = 0; i < 6; i++) { + long tableId = + createTable( + new TablePath(dbName, "test-rebalance_table-" + i), + DATA1_TABLE_DESCRIPTOR, + false); + FLUSS_CLUSTER_EXTENSION.waitUntilTableReady(tableId); + } + + // create some partitioned table with partition. + TableDescriptor descriptor = + TableDescriptor.builder() + .schema(DATA1_SCHEMA) + .distributedBy(3) + .partitionedBy("b") + .build(); + for (int i = 0; i < 3; i++) { + TablePath tablePath = new TablePath(dbName, "test-rebalance_partitioned_table-" + i); + long tableId = createTable(tablePath, descriptor, false); + for (int j = 0; j < 2; j++) { + PartitionSpec partitionSpec = + new PartitionSpec(Collections.singletonMap("b", String.valueOf(j))); + admin.createPartition(tablePath, partitionSpec, false).get(); + long partitionId = + admin.listPartitionInfos(tablePath, partitionSpec) + .get() + .get(0) + .getPartitionId(); + FLUSS_CLUSTER_EXTENSION.waitUntilTablePartitionReady(tableId, partitionId); + } + } + + // verify before rebalance. As we use unbalance assignment, all replicas will be location on + // servers [0,1,2], all leader will be location on server 0. + for (int i = 0; i < 3; i++) { + ReplicaManager replicaManager = + FLUSS_CLUSTER_EXTENSION.getTabletServerById(i).getReplicaManager(); + assertThat(replicaManager.onlineReplicas().count()).isEqualTo(36); + if (i == 0) { + assertThat(replicaManager.leaderCount()).isEqualTo(36); + } else { + assertThat(replicaManager.leaderCount()).isEqualTo(0); + } + } + ReplicaManager replicaManager3 = + FLUSS_CLUSTER_EXTENSION.getTabletServerById(3).getReplicaManager(); + assertThat(replicaManager3.onlineReplicas().count()).isEqualTo(0); + assertThat(replicaManager3.leaderCount()).isEqualTo(0); + + // trigger rebalance with goal set[ReplicaDistributionGoal, LeaderReplicaDistributionGoal] + Map rebalancePlan = + admin.rebalance( + Arrays.asList( + GoalType.REPLICA_DISTRIBUTION_GOAL, + GoalType.LEADER_REPLICA_DISTRIBUTION_GOAL), + false) + .get(); + + // verify list rebalance process + Map rebalanceProcess = + admin.listRebalanceProcess().get(); + Map rebalanceProcessPlan = + getRebalancePlanFromResult(rebalanceProcess); + assertThat(rebalancePlan).containsExactlyEntriesOf(rebalanceProcessPlan); + + retry( + Duration.ofMinutes(2), + () -> { + assertThat(admin.listRebalanceProcess().get()).isEmpty(); + for (int i = 0; i < 4; i++) { + ReplicaManager replicaManager = + FLUSS_CLUSTER_EXTENSION.getTabletServerById(i).getReplicaManager(); + // average will be 27 + assertThat(replicaManager.onlineReplicas().count()).isBetween(24L, 30L); + long leaderCount = replicaManager.leaderCount(); + // average will be 9 + assertThat(leaderCount).isBetween(7L, 11L); + } + }); + + // add server tag PERMANENT_OFFLINE for server 3, trigger all leader and replica removed + // from server 3. + admin.addServerTag(Collections.singletonList(3), ServerTag.PERMANENT_OFFLINE).get(); + rebalancePlan = + admin.rebalance( + Arrays.asList( + GoalType.REPLICA_DISTRIBUTION_GOAL, + GoalType.LEADER_REPLICA_DISTRIBUTION_GOAL), + false) + .get(); + rebalanceProcess = admin.listRebalanceProcess().get(); + rebalanceProcessPlan = getRebalancePlanFromResult(rebalanceProcess); + assertThat(rebalancePlan).containsExactlyEntriesOf(rebalanceProcessPlan); + retry( + Duration.ofMinutes(2), + () -> { + assertThat(admin.listRebalanceProcess().get()).isEmpty(); + assertThat(replicaManager3.onlineReplicas().count()).isEqualTo(0); + assertThat(replicaManager3.leaderCount()).isEqualTo(0); + for (int i = 0; i < 3; i++) { + ReplicaManager replicaManager = + FLUSS_CLUSTER_EXTENSION.getTabletServerById(i).getReplicaManager(); + // average will be 36 + assertThat(replicaManager.onlineReplicas().count()).isBetween(34L, 38L); + long leaderCount = replicaManager.leaderCount(); + // average will be 12 + assertThat(leaderCount).isBetween(10L, 14L); + } + }); + } + + private static Configuration initConfig() { + Configuration configuration = new Configuration(); + // As we want to test rebalance, we need to set this option to true for generate unbalance + // buckets location in server. + configuration.set(ConfigOptions.SERVER_GENERATE_UNBALANCE_ASSIGNMENT_FOR_TEST, true); + configuration.set(ConfigOptions.DEFAULT_REPLICATION_FACTOR, 3); + configuration.set(ConfigOptions.DEFAULT_BUCKET_NUMBER, 3); + return configuration; + } + + private long createTable( + TablePath tablePath, TableDescriptor tableDescriptor, boolean ignoreIfExists) + throws Exception { + admin.createTable(tablePath, tableDescriptor, ignoreIfExists).get(); + return admin.getTableInfo(tablePath).get().getTableId(); + } + + private Map getRebalancePlanFromResult( + Map rebalanceProcess) { + return rebalanceProcess.entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().planForBucket())); + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/GoalType.java b/fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/GoalType.java new file mode 100644 index 0000000000..130b75cf6b --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/GoalType.java @@ -0,0 +1,65 @@ +/* + * 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.fluss.cluster.rebalance; + +import org.apache.fluss.annotation.PublicEvolving; + +import java.util.Arrays; + +/** + * The type of goal to optimize. + * + * @since 0.8 + */ +@PublicEvolving +public enum GoalType { + /** + * Goal to generate replica movement tasks to ensure that the number of replicas on each + * tabletServer is near balanced. + */ + REPLICA_DISTRIBUTION_GOAL(0), + + /** + * Goal to generate leadership movement and leader replica movement tasks to ensure that the + * number of leader replicas on each tabletServer is near balanced. + */ + LEADER_REPLICA_DISTRIBUTION_GOAL(1), + + /** Goal to move the leaders to the first replica of each tableBuckets. */ + PREFERRED_LEADER_GOAL(2); + + public final int value; + + GoalType(int value) { + this.value = value; + } + + public static GoalType valueOf(int value) { + if (value == REPLICA_DISTRIBUTION_GOAL.value) { + return REPLICA_DISTRIBUTION_GOAL; + } else if (value == LEADER_REPLICA_DISTRIBUTION_GOAL.value) { + return LEADER_REPLICA_DISTRIBUTION_GOAL; + } else if (value == PREFERRED_LEADER_GOAL.value) { + return PREFERRED_LEADER_GOAL; + } else { + throw new IllegalArgumentException( + String.format( + "Value %s must be one of %s", value, Arrays.asList(GoalType.values()))); + } + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/RebalancePlanForBucket.java b/fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/RebalancePlanForBucket.java new file mode 100644 index 0000000000..13ec496be5 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/RebalancePlanForBucket.java @@ -0,0 +1,116 @@ +/* + * 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.fluss.cluster.rebalance; + +import org.apache.fluss.annotation.PublicEvolving; +import org.apache.fluss.metadata.TableBucket; + +import java.util.List; +import java.util.Objects; + +/** + * a generated rebalance plan for a tableBucket. + * + * @since 0.8 + */ +@PublicEvolving +public class RebalancePlanForBucket { + private final TableBucket tableBucket; + private final int originalLeader; + private final int newLeader; + private final List originReplicas; + private final List newReplicas; + + public RebalancePlanForBucket( + TableBucket tableBucket, + int originalLeader, + int newLeader, + List originReplicas, + List newReplicas) { + this.tableBucket = tableBucket; + this.originalLeader = originalLeader; + this.newLeader = newLeader; + this.originReplicas = originReplicas; + this.newReplicas = newReplicas; + } + + public TableBucket getTableBucket() { + return tableBucket; + } + + public int getBucketId() { + return tableBucket.getBucket(); + } + + public Integer getOriginalLeader() { + return originalLeader; + } + + public Integer getNewLeader() { + return newLeader; + } + + public List getOriginReplicas() { + return originReplicas; + } + + public List getNewReplicas() { + return newReplicas; + } + + public boolean isLeaderAction() { + return originalLeader != newLeader; + } + + @Override + public String toString() { + return "RebalancePlanForBucket{" + + "tableBucket=" + + tableBucket + + ", originalLeader=" + + originalLeader + + ", newLeader=" + + newLeader + + ", originReplicas=" + + originReplicas + + ", newReplicas=" + + newReplicas + + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + RebalancePlanForBucket that = (RebalancePlanForBucket) o; + return Objects.equals(tableBucket, that.tableBucket) + && originalLeader == that.originalLeader + && newLeader == that.newLeader + && Objects.equals(originReplicas, that.originReplicas) + && Objects.equals(newReplicas, that.newReplicas); + } + + @Override + public int hashCode() { + return Objects.hash(tableBucket, originalLeader, newLeader, originReplicas, newReplicas); + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/RebalanceResultForBucket.java b/fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/RebalanceResultForBucket.java new file mode 100644 index 0000000000..cc4fbd39c3 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/RebalanceResultForBucket.java @@ -0,0 +1,81 @@ +/* + * 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.fluss.cluster.rebalance; + +import org.apache.fluss.annotation.PublicEvolving; +import org.apache.fluss.metadata.TableBucket; + +import java.util.List; + +/** + * Status of rebalance process for a tabletBucket. + * + * @since 0.8 + */ +@PublicEvolving +public class RebalanceResultForBucket { + private final RebalancePlanForBucket rebalancePlanForBucket; + private RebalanceStatusForBucket rebalanceStatusForBucket; + + private RebalanceResultForBucket( + RebalancePlanForBucket rebalancePlanForBucket, + RebalanceStatusForBucket rebalanceStatusForBucket) { + this.rebalancePlanForBucket = rebalancePlanForBucket; + this.rebalanceStatusForBucket = rebalanceStatusForBucket; + } + + public TableBucket tableBucket() { + return rebalancePlanForBucket.getTableBucket(); + } + + public RebalancePlanForBucket planForBucket() { + return rebalancePlanForBucket; + } + + public List newReplicas() { + return rebalancePlanForBucket.getNewReplicas(); + } + + public List originReplicas() { + return rebalancePlanForBucket.getOriginReplicas(); + } + + public RebalanceResultForBucket setNewStatus(RebalanceStatusForBucket status) { + this.rebalanceStatusForBucket = status; + return this; + } + + public RebalanceStatusForBucket status() { + return rebalanceStatusForBucket; + } + + public static RebalanceResultForBucket of( + RebalancePlanForBucket planForBucket, RebalanceStatusForBucket status) { + return new RebalanceResultForBucket(planForBucket, status); + } + + @Override + public String toString() { + return "RebalanceResultForBucket{" + + "rebalancePlanForBucket=" + + rebalancePlanForBucket + + ", rebalanceStatusForBucket=" + + rebalanceStatusForBucket + + '}'; + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/RebalanceStatusForBucket.java b/fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/RebalanceStatusForBucket.java new file mode 100644 index 0000000000..35f833bfd8 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/RebalanceStatusForBucket.java @@ -0,0 +1,52 @@ +/* + * 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.fluss.cluster.rebalance; + +import org.apache.fluss.annotation.PublicEvolving; + +/** + * Rebalance status for single bucket. + * + * @since 0.8 + */ +@PublicEvolving +public enum RebalanceStatusForBucket { + PENDING(1), + REBALANCING(2), + FAILED(3), + COMPLETED(4); + + private final int code; + + RebalanceStatusForBucket(int code) { + this.code = code; + } + + public int getCode() { + return code; + } + + public static RebalanceStatusForBucket of(int code) { + for (RebalanceStatusForBucket status : RebalanceStatusForBucket.values()) { + if (status.code == code) { + return status; + } + } + return null; + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/ServerTag.java b/fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/ServerTag.java new file mode 100644 index 0000000000..5e20b34f73 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/ServerTag.java @@ -0,0 +1,58 @@ +/* + * 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.fluss.cluster.rebalance; + +import org.apache.fluss.annotation.PublicEvolving; + +import java.util.Arrays; + +/** + * The tag of tabletServer. + * + * @since 0.8 + */ +@PublicEvolving +public enum ServerTag { + /** + * The tabletServer is permanently offline. Such as the host where the tabletServer on is + * upcoming decommissioning. + */ + PERMANENT_OFFLINE(0), + + /** The tabletServer is temporarily offline. Such as the tabletServer is upcoming upgrading. */ + TEMPORARY_OFFLINE(1); + + public final int value; + + ServerTag(int value) { + this.value = value; + } + + public static ServerTag valueOf(int value) { + if (value == PERMANENT_OFFLINE.value) { + return PERMANENT_OFFLINE; + } else if (value == TEMPORARY_OFFLINE.value) { + return TEMPORARY_OFFLINE; + } else { + throw new IllegalArgumentException( + String.format( + "Value %s must be one of %s", + value, Arrays.asList(ServerTag.values()))); + } + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java b/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java index 6b045bb389..0cb5b2ae62 100644 --- a/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java +++ b/fluss-common/src/main/java/org/apache/fluss/config/ConfigOptions.java @@ -481,6 +481,15 @@ public class ConfigOptions { .withDescription( "Defines how long the buffer pool will block when waiting for segments to become available."); + public static final ConfigOption SERVER_GENERATE_UNBALANCE_ASSIGNMENT_FOR_TEST = + key("server.generate-unbalance-assignment-for-test") + .booleanType() + .defaultValue(false) + .withDescription( + "Whether to generate unbalance table or partition assignment. This parameter is only used " + + "for itCase. If set to true, the assignment will always be [0,1,2] as replica factor " + + "set as 3 even if there are tabletServers more than 3."); + // ------------------------------------------------------------------ // ZooKeeper Settings // ------------------------------------------------------------------ diff --git a/fluss-common/src/main/java/org/apache/fluss/exception/NoRebalanceInProgressException.java b/fluss-common/src/main/java/org/apache/fluss/exception/NoRebalanceInProgressException.java new file mode 100644 index 0000000000..8b052a5100 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/exception/NoRebalanceInProgressException.java @@ -0,0 +1,34 @@ +/* + * 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.fluss.exception; + +import org.apache.fluss.annotation.PublicEvolving; + +/** + * Thrown if there are no rebalance tasks in progress when list rebalance process. + * + * @since 0.8 + */ +@PublicEvolving +public class NoRebalanceInProgressException extends ApiException { + private static final long serialVersionUID = 1L; + + public NoRebalanceInProgressException(String message) { + super(message); + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/exception/RebalanceFailureException.java b/fluss-common/src/main/java/org/apache/fluss/exception/RebalanceFailureException.java new file mode 100644 index 0000000000..0dcf260b0c --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/exception/RebalanceFailureException.java @@ -0,0 +1,38 @@ +/* + * 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.fluss.exception; + +import org.apache.fluss.annotation.VisibleForTesting; + +/** + * This exception is thrown if rebalance failed. + * + * @since 0.8 + */ +@VisibleForTesting +public class RebalanceFailureException extends ApiException { + private static final long serialVersionUID = 1L; + + public RebalanceFailureException(String message) { + super(message); + } + + public RebalanceFailureException(String message, Throwable cause) { + super(message, cause); + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/exception/ServerNotExistException.java b/fluss-common/src/main/java/org/apache/fluss/exception/ServerNotExistException.java new file mode 100644 index 0000000000..2bdbe621e8 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/exception/ServerNotExistException.java @@ -0,0 +1,34 @@ +/* + * 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.fluss.exception; + +import org.apache.fluss.annotation.PublicEvolving; + +/** + * Thrown if a server does not exist in the cluster. + * + * @since 0.8 + */ +@PublicEvolving +public class ServerNotExistException extends ApiException { + private static final long serialVersionUID = 1L; + + public ServerNotExistException(String message) { + super(message); + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/exception/ServerTagAlreadyExistException.java b/fluss-common/src/main/java/org/apache/fluss/exception/ServerTagAlreadyExistException.java new file mode 100644 index 0000000000..a3d4259b13 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/exception/ServerTagAlreadyExistException.java @@ -0,0 +1,34 @@ +/* + * 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.fluss.exception; + +import org.apache.fluss.annotation.PublicEvolving; + +/** + * Thrown if a server tag already exists for specify tabletServer in the cluster. + * + * @since 0.8 + */ +@PublicEvolving +public class ServerTagAlreadyExistException extends ApiException { + private static final long serialVersionUID = 1L; + + public ServerTagAlreadyExistException(String message) { + super(message); + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/exception/ServerTagNotExistException.java b/fluss-common/src/main/java/org/apache/fluss/exception/ServerTagNotExistException.java new file mode 100644 index 0000000000..bd62672c72 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/exception/ServerTagNotExistException.java @@ -0,0 +1,35 @@ +/* + * 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.fluss.exception; + +import org.apache.fluss.annotation.PublicEvolving; + +/** + * Thrown if a server tag not exist for specify tabletServer in the cluster. + * + * @since 0.8 + */ +@PublicEvolving +public class ServerTagNotExistException extends ApiException { + + private static final long serialVersionUID = 1L; + + public ServerTagNotExistException(String message) { + super(message); + } +} diff --git a/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/AdminGateway.java b/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/AdminGateway.java index 7b322ff67e..996342dfcb 100644 --- a/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/AdminGateway.java +++ b/fluss-rpc/src/main/java/org/apache/fluss/rpc/gateway/AdminGateway.java @@ -17,10 +17,14 @@ package org.apache.fluss.rpc.gateway; +import org.apache.fluss.rpc.messages.AddServerTagRequest; +import org.apache.fluss.rpc.messages.AddServerTagResponse; import org.apache.fluss.rpc.messages.AlterClusterConfigsRequest; import org.apache.fluss.rpc.messages.AlterClusterConfigsResponse; import org.apache.fluss.rpc.messages.AlterTableRequest; import org.apache.fluss.rpc.messages.AlterTableResponse; +import org.apache.fluss.rpc.messages.CancelRebalanceRequest; +import org.apache.fluss.rpc.messages.CancelRebalanceResponse; import org.apache.fluss.rpc.messages.CreateAclsRequest; import org.apache.fluss.rpc.messages.CreateAclsResponse; import org.apache.fluss.rpc.messages.CreateDatabaseRequest; @@ -37,6 +41,12 @@ import org.apache.fluss.rpc.messages.DropPartitionResponse; import org.apache.fluss.rpc.messages.DropTableRequest; import org.apache.fluss.rpc.messages.DropTableResponse; +import org.apache.fluss.rpc.messages.ListRebalanceProcessRequest; +import org.apache.fluss.rpc.messages.ListRebalanceProcessResponse; +import org.apache.fluss.rpc.messages.RebalanceRequest; +import org.apache.fluss.rpc.messages.RebalanceResponse; +import org.apache.fluss.rpc.messages.RemoveServerTagRequest; +import org.apache.fluss.rpc.messages.RemoveServerTagResponse; import org.apache.fluss.rpc.protocol.ApiKeys; import org.apache.fluss.rpc.protocol.RPC; @@ -120,6 +130,22 @@ public interface AdminGateway extends AdminReadOnlyGateway { CompletableFuture alterClusterConfigs( AlterClusterConfigsRequest request); + @RPC(api = ApiKeys.ADD_SERVER_TAG) + CompletableFuture addServerTag(AddServerTagRequest request); + + @RPC(api = ApiKeys.REMOVE_SERVER_TAG) + CompletableFuture removeServerTag(RemoveServerTagRequest request); + + @RPC(api = ApiKeys.REBALANCE) + CompletableFuture rebalance(RebalanceRequest request); + + @RPC(api = ApiKeys.LIST_REBALANCE_PROCESS) + CompletableFuture listRebalanceProcess( + ListRebalanceProcessRequest request); + + @RPC(api = ApiKeys.CANCEL_REBALANCE) + CompletableFuture cancelRebalance(CancelRebalanceRequest request); + // todo: rename table & alter table } diff --git a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java index 8acec7a36d..97e14d79f0 100644 --- a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java +++ b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/ApiKeys.java @@ -74,7 +74,12 @@ public enum ApiKeys { CONTROLLED_SHUTDOWN(1043, 0, 0, PRIVATE), ALTER_TABLE(1044, 0, 0, PUBLIC), DESCRIBE_CLUSTER_CONFIGS(1045, 0, 0, PUBLIC), - ALTER_CLUSTER_CONFIGS(1046, 0, 0, PUBLIC); + ALTER_CLUSTER_CONFIGS(1046, 0, 0, PUBLIC), + ADD_SERVER_TAG(1047, 0, 0, PUBLIC), + REMOVE_SERVER_TAG(1048, 0, 0, PUBLIC), + REBALANCE(1049, 0, 0, PUBLIC), + LIST_REBALANCE_PROCESS(1050, 0, 0, PUBLIC), + CANCEL_REBALANCE(1051, 0, 0, PUBLIC); private static final Map ID_TO_TYPE = Arrays.stream(ApiKeys.values()) diff --git a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java index 4b49a58475..5ee652cce2 100644 --- a/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java +++ b/fluss-rpc/src/main/java/org/apache/fluss/rpc/protocol/Errors.java @@ -52,6 +52,7 @@ import org.apache.fluss.exception.LogOffsetOutOfRangeException; import org.apache.fluss.exception.LogStorageException; import org.apache.fluss.exception.NetworkException; +import org.apache.fluss.exception.NoRebalanceInProgressException; import org.apache.fluss.exception.NonPrimaryKeyTableException; import org.apache.fluss.exception.NotEnoughReplicasAfterAppendException; import org.apache.fluss.exception.NotEnoughReplicasException; @@ -60,11 +61,15 @@ import org.apache.fluss.exception.OutOfOrderSequenceException; import org.apache.fluss.exception.PartitionAlreadyExistsException; import org.apache.fluss.exception.PartitionNotExistException; +import org.apache.fluss.exception.RebalanceFailureException; import org.apache.fluss.exception.RecordTooLargeException; import org.apache.fluss.exception.RetriableAuthenticationException; import org.apache.fluss.exception.SchemaNotExistException; import org.apache.fluss.exception.SecurityDisabledException; import org.apache.fluss.exception.SecurityTokenException; +import org.apache.fluss.exception.ServerNotExistException; +import org.apache.fluss.exception.ServerTagAlreadyExistException; +import org.apache.fluss.exception.ServerTagNotExistException; import org.apache.fluss.exception.StorageException; import org.apache.fluss.exception.TableAlreadyExistException; import org.apache.fluss.exception.TableNotExistException; @@ -228,7 +233,14 @@ public enum Errors { INVALID_ALTER_TABLE_EXCEPTION( 56, "The alter table is invalid.", InvalidAlterTableException::new), DELETION_DISABLED_EXCEPTION( - 57, "Deletion operations are disabled on this table.", DeletionDisabledException::new); + 57, "Deletion operations are disabled on this table.", DeletionDisabledException::new), + SERVER_NOT_EXIST_EXCEPTION(58, "The server is not exist.", ServerNotExistException::new), + SEVER_TAG_ALREADY_EXIST_EXCEPTION( + 59, "The server tag already exist.", ServerTagAlreadyExistException::new), + SEVER_TAG_NOT_EXIST_EXCEPTION(60, "The server tag not exist.", ServerTagNotExistException::new), + REBALANCE_FAILURE_EXCEPTION(61, "The rebalance task failure.", RebalanceFailureException::new), + NO_REBALANCE_IN_PROGRESS_EXCEPTION( + 62, "No rebalance task in progress.", NoRebalanceInProgressException::new); private static final Logger LOG = LoggerFactory.getLogger(Errors.class); diff --git a/fluss-rpc/src/main/proto/FlussApi.proto b/fluss-rpc/src/main/proto/FlussApi.proto index ca41ebcb02..6994525808 100644 --- a/fluss-rpc/src/main/proto/FlussApi.proto +++ b/fluss-rpc/src/main/proto/FlussApi.proto @@ -569,6 +569,44 @@ message AlterClusterConfigsRequest{ message AlterClusterConfigsResponse{ } +message AddServerTagRequest { + repeated int32 server_ids = 1 [packed = true]; + required int32 server_tag = 2; +} + +message AddServerTagResponse { +} + +message RemoveServerTagRequest { + repeated int32 server_ids = 1 [packed = true]; + required int32 server_tag = 2; +} + +message RemoveServerTagResponse { +} + +message RebalanceRequest { + repeated int32 goals = 1 [packed = true]; + required bool dry_run = 2; +} + +message RebalanceResponse { + repeated PbRebalancePlanForTable plan_for_table = 1; +} + +message ListRebalanceProcessRequest { +} + +message ListRebalanceProcessResponse { + repeated PbRebalanceProcessForTable process_for_table = 1; +} + +message CancelRebalanceRequest { +} + +message CancelRebalanceResponse { +} + // --------------- Inner classes ---------------- @@ -921,3 +959,40 @@ message PbDescribeConfig{ required string config_source = 3; } +message PbRebalancePlanForTable { + required int64 table_id = 1; + repeated PbRebalancePlanForPartition partitions_plan = 2; // for none-partition table, this is empty + repeated PbRebalancePlanForBucket buckets_plan = 3; // for partition table, this is empty + +} + +message PbRebalancePlanForPartition { + required int64 partition_id = 1; + repeated PbRebalancePlanForBucket buckets_plan = 2; +} + +message PbRebalancePlanForBucket { + required int32 bucket_id = 1; + optional int32 original_leader = 2; + optional int32 new_leader = 3; + repeated int32 original_replicas = 4 [packed = true]; + repeated int32 new_replicas = 5 [packed = true]; +} + +message PbRebalanceProcessForTable { + required int64 table_id = 1; + repeated PbRebalanceProcessForPartition partitions_process = 2; + repeated PbRebalanceProcessForBucket buckets_process = 3; +} + +message PbRebalanceProcessForPartition { + required int64 partition_id = 1; + repeated PbRebalanceProcessForBucket buckets_process = 2; +} + +message PbRebalanceProcessForBucket { + required int32 bucket_id = 1; + repeated int32 original_replicas = 2 [packed = true]; + repeated int32 new_replicas = 3 [packed = true]; + required int32 rebalance_status = 4; +} \ No newline at end of file diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorContext.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorContext.java index a5f34aa17f..cd55d55b11 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorContext.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorContext.java @@ -18,6 +18,7 @@ package org.apache.fluss.server.coordinator; import org.apache.fluss.annotation.VisibleForTesting; +import org.apache.fluss.cluster.rebalance.ServerTag; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableBucketReplica; @@ -102,6 +103,9 @@ public class CoordinatorContext { */ private final Map> replicasOnOffline = new HashMap<>(); + /** A mapping from tabletServers to server tag. */ + private final Map serverTags = new HashMap<>(); + private ServerInfo coordinatorServerInfo = null; private int coordinatorEpoch = INITIAL_COORDINATOR_EPOCH; @@ -180,7 +184,7 @@ public Map allTables() { return tablePathById; } - public Set allBuckets() { + public Set getAllBuckets() { Set allBuckets = new HashSet<>(); for (Map.Entry>> tableAssign : tableAssignments.entrySet()) { @@ -635,6 +639,26 @@ public void removePartition(TablePartition tablePartition) { } } + public void initSeverTags(Map initialServerTags) { + serverTags.putAll(initialServerTags); + } + + public void putServerTag(int serverId, ServerTag serverTag) { + serverTags.put(serverId, serverTag); + } + + public Map getServerTags() { + return new HashMap<>(serverTags); + } + + public Optional getServerTag(int serverId) { + return Optional.ofNullable(serverTags.get(serverId)); + } + + public void removeServerTag(int serverId) { + serverTags.remove(serverId); + } + private void clearTablesState() { tableAssignments.clear(); partitionAssignments.clear(); @@ -656,6 +680,7 @@ public void resetContext() { // clear the live tablet servers liveTabletServers.clear(); shuttingDownTabletServers.clear(); + serverTags.clear(); } public int getTotalPartitionCount() { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java index b3fed108b8..50793b8f89 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessor.java @@ -21,6 +21,9 @@ import org.apache.fluss.cluster.Endpoint; import org.apache.fluss.cluster.ServerNode; import org.apache.fluss.cluster.ServerType; +import org.apache.fluss.cluster.rebalance.RebalancePlanForBucket; +import org.apache.fluss.cluster.rebalance.RebalanceStatusForBucket; +import org.apache.fluss.cluster.rebalance.ServerTag; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; import org.apache.fluss.exception.FencedLeaderEpochException; @@ -28,7 +31,11 @@ import org.apache.fluss.exception.IneligibleReplicaException; import org.apache.fluss.exception.InvalidCoordinatorException; import org.apache.fluss.exception.InvalidUpdateVersionException; +import org.apache.fluss.exception.ServerNotExistException; +import org.apache.fluss.exception.ServerTagAlreadyExistException; +import org.apache.fluss.exception.ServerTagNotExistException; import org.apache.fluss.exception.TabletServerNotAvailableException; +import org.apache.fluss.exception.UnknownServerException; import org.apache.fluss.exception.UnknownTableOrBucketException; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.TableBucket; @@ -36,14 +43,17 @@ import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePartition; import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.rpc.messages.AddServerTagResponse; import org.apache.fluss.rpc.messages.AdjustIsrResponse; import org.apache.fluss.rpc.messages.CommitKvSnapshotResponse; import org.apache.fluss.rpc.messages.CommitLakeTableSnapshotResponse; import org.apache.fluss.rpc.messages.CommitRemoteLogManifestResponse; import org.apache.fluss.rpc.messages.ControlledShutdownResponse; import org.apache.fluss.rpc.messages.PbCommitLakeTableSnapshotRespForTable; +import org.apache.fluss.rpc.messages.RemoveServerTagResponse; import org.apache.fluss.rpc.protocol.ApiError; import org.apache.fluss.server.coordinator.event.AccessContextEvent; +import org.apache.fluss.server.coordinator.event.AddServerTagEvent; import org.apache.fluss.server.coordinator.event.AdjustIsrReceivedEvent; import org.apache.fluss.server.coordinator.event.CommitKvSnapshotEvent; import org.apache.fluss.server.coordinator.event.CommitLakeTableSnapshotEvent; @@ -58,12 +68,17 @@ import org.apache.fluss.server.coordinator.event.DropPartitionEvent; import org.apache.fluss.server.coordinator.event.DropTableEvent; import org.apache.fluss.server.coordinator.event.EventProcessor; +import org.apache.fluss.server.coordinator.event.ExecuteRebalanceTaskEvent; import org.apache.fluss.server.coordinator.event.FencedCoordinatorEvent; import org.apache.fluss.server.coordinator.event.NewTabletServerEvent; import org.apache.fluss.server.coordinator.event.NotifyKvSnapshotOffsetEvent; import org.apache.fluss.server.coordinator.event.NotifyLeaderAndIsrResponseReceivedEvent; +import org.apache.fluss.server.coordinator.event.RemoveServerTagEvent; import org.apache.fluss.server.coordinator.event.watcher.TableChangeWatcher; import org.apache.fluss.server.coordinator.event.watcher.TabletServerChangeWatcher; +import org.apache.fluss.server.coordinator.rebalance.RebalanceManager; +import org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElection.ControlledShutdownLeaderElection; +import org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElection.ReassignmentLeaderElection; import org.apache.fluss.server.coordinator.statemachine.ReplicaStateMachine; import org.apache.fluss.server.coordinator.statemachine.TableBucketStateMachine; import org.apache.fluss.server.entity.AdjustIsrResultForBucket; @@ -83,6 +98,7 @@ import org.apache.fluss.server.zk.data.LeaderAndIsr; import org.apache.fluss.server.zk.data.PartitionAssignment; import org.apache.fluss.server.zk.data.RemoteLogManifestHandle; +import org.apache.fluss.server.zk.data.ServerTags; import org.apache.fluss.server.zk.data.TableAssignment; import org.apache.fluss.server.zk.data.TabletServerRegistration; import org.apache.fluss.server.zk.data.ZkData.PartitionIdsZNode; @@ -101,6 +117,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.concurrent.CompletableFuture; @@ -109,7 +126,7 @@ import static org.apache.fluss.server.coordinator.statemachine.BucketState.OfflineBucket; import static org.apache.fluss.server.coordinator.statemachine.BucketState.OnlineBucket; -import static org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElectionStrategy.CONTROLLED_SHUTDOWN_ELECTION; +import static org.apache.fluss.server.coordinator.statemachine.ReplicaState.NewReplica; import static org.apache.fluss.server.coordinator.statemachine.ReplicaState.OfflineReplica; import static org.apache.fluss.server.coordinator.statemachine.ReplicaState.OnlineReplica; import static org.apache.fluss.server.coordinator.statemachine.ReplicaState.ReplicaDeletionStarted; @@ -138,8 +155,9 @@ public class CoordinatorEventProcessor implements EventProcessor { private final TabletServerChangeWatcher tabletServerChangeWatcher; private final CoordinatorMetadataCache serverMetadataCache; private final CoordinatorRequestBatch coordinatorRequestBatch; - private final CoordinatorMetricGroup coordinatorMetricGroup; private final String internalListenerName; + private final CoordinatorMetricGroup coordinatorMetricGroup; + private final RebalanceManager rebalanceManager; private final CompletedSnapshotStoreManager completedSnapshotStoreManager; @@ -201,6 +219,8 @@ public CoordinatorEventProcessor( this.lakeTableTieringManager = lakeTableTieringManager; this.coordinatorMetricGroup = coordinatorMetricGroup; this.internalListenerName = conf.getString(ConfigOptions.INTERNAL_LISTENER_NAME); + this.rebalanceManager = + new RebalanceManager(() -> coordinatorEventManager, zooKeeperClient); this.ioExecutor = ioExecutor; } @@ -208,6 +228,10 @@ public CoordinatorEventManager getCoordinatorEventManager() { return coordinatorEventManager; } + public RebalanceManager getRebalanceManager() { + return rebalanceManager; + } + public void startup() { coordinatorContext.setCoordinatorServerInfo(getCoordinatorServerInfo()); // start watchers first so that we won't miss node in zk; @@ -242,6 +266,7 @@ public void startup() { public void shutdown() { // close the event manager coordinatorEventManager.close(); + rebalanceManager.close(); onShutdown(); } @@ -318,6 +343,11 @@ private void initCoordinatorContext() throws Exception { // init tablet server channels coordinatorChannelManager.startup(internalServerNodes); + // load server tags. + zooKeeperClient + .getServerTags() + .ifPresent(tags -> coordinatorContext.initSeverTags(tags.getServerTags())); + // load all tables long start4loadTables = System.currentTimeMillis(); List autoPartitionTables = new ArrayList<>(); @@ -541,6 +571,18 @@ public void process(CoordinatorEvent event) { completeFromCallable( controlledShutdownEvent.getRespCallback(), () -> tryProcessControlledShutdown(controlledShutdownEvent)); + } else if (event instanceof AddServerTagEvent) { + AddServerTagEvent addServerTagEvent = (AddServerTagEvent) event; + completeFromCallable( + addServerTagEvent.getRespCallback(), + () -> processAddServerTag(addServerTagEvent)); + } else if (event instanceof RemoveServerTagEvent) { + RemoveServerTagEvent removeServerTagEvent = (RemoveServerTagEvent) event; + completeFromCallable( + removeServerTagEvent.getRespCallback(), + () -> processRemoveServerTag(removeServerTagEvent)); + } else if (event instanceof ExecuteRebalanceTaskEvent) { + tryToExecuteRebalanceTask((ExecuteRebalanceTaskEvent) event); } else if (event instanceof AccessContextEvent) { AccessContextEvent accessContextEvent = (AccessContextEvent) event; processAccessContext(accessContextEvent); @@ -923,6 +965,374 @@ private void processDeadTabletServer(DeadTabletServerEvent deadTabletServerEvent updateTabletServerMetadataCache(serverInfos, null, null, bucketsWithOfflineLeader); } + private AddServerTagResponse processAddServerTag(AddServerTagEvent event) { + AddServerTagResponse addServerTagResponse = new AddServerTagResponse(); + List serverIds = event.getServerIds(); + ServerTag serverTag = event.getServerTag(); + + // Verify that dose serverTag exist for input serverIds. If any of them exists, throw + // an error and none of them will be written to coordinatorContext and zk. + Map liveTabletServers = coordinatorContext.getLiveTabletServers(); + for (Integer serverId : serverIds) { + if (!liveTabletServers.containsKey(serverId)) { + throw new ServerNotExistException( + String.format( + "Server %s not exists when trying to add server tag.", serverId)); + } + + if (coordinatorContext.getServerTag(serverId).isPresent()) { + throw new ServerTagAlreadyExistException( + String.format( + "Server tag %s already exists for server %s.", + serverTag, serverId)); + } + } + + // First register to zk, and then update coordinatorContext. + Map serverTags = coordinatorContext.getServerTags(); + for (Integer serverId : serverIds) { + serverTags.put(serverId, serverTag); + } + + try { + zooKeeperClient.registerServerTags(new ServerTags(serverTags)); + } catch (Exception e) { + LOG.error("Error when register server tags to zookeeper.", e); + throw new UnknownServerException("Error when register server tags to zookeeper.", e); + } + + // Then update coordinatorContext. + serverIds.forEach(serverId -> coordinatorContext.putServerTag(serverId, serverTag)); + LOG.info("Server tag {} added for servers {}.", serverTag, serverIds); + + return addServerTagResponse; + } + + private RemoveServerTagResponse processRemoveServerTag(RemoveServerTagEvent event) { + RemoveServerTagResponse removeServerTagResponse = new RemoveServerTagResponse(); + List serverIds = event.getServerIds(); + ServerTag serverTag = event.getServerTag(); + + // Verify that dose serverTag not exist for input serverIds. If any of them not exists, + // throw an error and none of them will be removed form coordinatorContext and zk. + Map liveTabletServers = coordinatorContext.getLiveTabletServers(); + for (Integer serverId : serverIds) { + if (!liveTabletServers.containsKey(serverId)) { + throw new ServerNotExistException( + String.format( + "Server %s not exists when trying to removing server tag.", + serverId)); + } + + if (!coordinatorContext.getServerTag(serverId).isPresent()) { + throw new ServerTagNotExistException( + String.format( + "Server tag %s not exists for server %s.", serverTag, serverId)); + } + } + + // First register to zk, and then update coordinatorContext. + Map serverTags = coordinatorContext.getServerTags(); + for (Integer serverId : serverIds) { + serverTags.remove(serverId); + } + + try { + zooKeeperClient.registerServerTags(new ServerTags(serverTags)); + } catch (Exception e) { + LOG.error("Error when register server tags to zookeeper.", e); + throw new UnknownServerException("Error when register server tags to zookeeper.", e); + } + + // Then update coordinatorContext. + serverIds.forEach(coordinatorContext::removeServerTag); + LOG.info("Server tag {} removed for servers {}.", serverTag, serverIds); + + return removeServerTagResponse; + } + + /** + * This method can be trigger by: + * + *
    + *
  • The rebalanceManager submit a new rebalance task. + *
  • The coordinatorServer restart, and want to do the unfinished rebalance task stored in + * Zookeeper. + *
+ */ + private void tryToExecuteRebalanceTask(ExecuteRebalanceTaskEvent event) { + RebalancePlanForBucket planForBucket = event.getRebalancePlanForBucket(); + Set allBuckets = coordinatorContext.getAllBuckets(); + TableBucket tableBucket = planForBucket.getTableBucket(); + if (!allBuckets.contains(tableBucket)) { + LOG.warn( + "Skipping rebalance task of tableBucket {} since it doesn't exist.", + tableBucket); + rebalanceManager.finishRebalanceTask(tableBucket, RebalanceStatusForBucket.FAILED); + return; + } + + if (coordinatorContext.isTableQueuedForDeletion(tableBucket.getTableId())) { + LOG.warn( + "Skipping rebalance task of tableBucket {} since the respective " + + "tables are being deleted.", + tableBucket); + rebalanceManager.finishRebalanceTask(tableBucket, RebalanceStatusForBucket.FAILED); + return; + } + + List newReplicas = planForBucket.getNewReplicas(); + ReplicaReassignment reassignment = + ReplicaReassignment.build( + coordinatorContext.getAssignment(tableBucket), newReplicas); + + if (planForBucket.isLeaderAction() && !reassignment.isBeingReassigned()) { + // buckets only need to change leader like preferred leader election and leader replica + // rebalance. + LOG.info("trigger leader election for tableBucket {}.", tableBucket); + tableBucketStateMachine.handleStateChange( + Collections.singleton(tableBucket), + OnlineBucket, + new ReassignmentLeaderElection(newReplicas)); + rebalanceManager.finishRebalanceTask(tableBucket, RebalanceStatusForBucket.COMPLETED); + } else { + try { + LOG.info( + "Try to processing bucket reassignment for tableBucket {} with assignment: {}.", + tableBucket, + reassignment); + onBucketReassignment(tableBucket, reassignment); + } catch (Exception e) { + LOG.error("Error when processing bucket reassignment.", e); + rebalanceManager.finishRebalanceTask(tableBucket, RebalanceStatusForBucket.FAILED); + } + } + } + + /** try to finish rebalance tasks after receive notify leader and isr response. */ + private void tryToCompleteRebalanceTask(TableBucket tableBucket) { + RebalancePlanForBucket planForBucket = + rebalanceManager.getRebalancePlanForBucket(tableBucket); + if (planForBucket != null) { + ReplicaReassignment reassignment = + ReplicaReassignment.build( + coordinatorContext.getAssignment(tableBucket), + planForBucket.getNewReplicas()); + try { + if (planForBucket.isLeaderAction() && !reassignment.isBeingReassigned()) { + LeaderAndIsr leaderAndIsr = zooKeeperClient.getLeaderAndIsr(tableBucket).get(); + int currentLeader = leaderAndIsr.leader(); + if (currentLeader == planForBucket.getNewLeader()) { + // leader action finish. + rebalanceManager.finishRebalanceTask( + tableBucket, RebalanceStatusForBucket.COMPLETED); + } + } else { + boolean isReassignmentComplete = + isReassignmentComplete(tableBucket, reassignment); + if (isReassignmentComplete) { + LOG.info( + "Target replicas {} have all caught up with the leader for reassigning bucket {}", + reassignment.getTargetReplicas(), + tableBucket); + onBucketReassignment(tableBucket, reassignment); + } + } + } catch (Exception e) { + LOG.error( + "Failed to complete the reassignment for table bucket {}", tableBucket, e); + rebalanceManager.finishRebalanceTask(tableBucket, RebalanceStatusForBucket.FAILED); + } + } + } + + /** + * Reassigning replicas for a tableBucket goes through a few steps listed in the code. + * + *
    + *
  • RS = current assigned replica set + *
  • ORS = original assigned replica set + *
  • TRS = target replica set + *
  • AR = the replicas we are adding as part of this reassignment + *
  • RR = the replicas we are removing as part of this reassignment + *
+ * + *

A reassignment may have up to two phases, each with its own steps: + * + *

To complete the reassignment, we need to bring the new replicas into sync, so depending on + * the state of the ISR, we will execute one of the following steps. + * + *

Phase A (when TRS != ISR): The reassignment is not yet complete + * + *

    + *
  • A1. Bump the leader epoch for the bucket and send LeaderAndIsr updates to CRS. + *
  • A2. Start new replicas AR by moving replicas in AR to NewReplica state. + *
+ * + *

Phase B (when TRS = ISR): The reassignment is complete + * + *

    + *
  • B1. Move all replicas in AR to OnlineReplica state. + *
  • B2. Set RS = TRS, AR = [], RR = [] in memory. + *
  • B3. Send a LeaderAndIsr request with RS = TRS. This will prevent the leader from adding + * any replica in TRS - ORS back in the isr. If the current leader is not in TRS or isn't + * alive, we move the leader to a new replica in TRS. We may send the LeaderAndIsr to more + * than the TRS replicas due to the way the partition state machine works (it reads + * replicas from ZK) + *
  • B4. Move all replicas in RR to OfflineReplica state. As part of OfflineReplica state + * change, we shrink the isr to remove RR in ZooKeeper and send a LeaderAndIsr ONLY to the + * Leader to notify it of the shrunk isr. After that, we send a StopReplica (delete = + * false) to the replicas in RR. + *
  • B5. Move all replicas in RR to NonExistentReplica state. This will send a StopReplica + * (delete = true) to he replicas in RR to physically delete the replicas on disk. + *
  • B6. Update ZK with RS=TRS, AR=[], RR=[]. + *
  • B7. After electing leader, the replicas and isr information changes. So resend the + * update metadata request to every tabletServer. + *
  • B8. Mark the ongoing rebalance task to finish. + *
+ * + *

In general, there are two goals we want to aim for: + * + *

    + *
  • 1. Every replica present in the replica set of a LeaderAndIsrRequest gets the request + * sent to it + *
  • 2. Replicas that are removed from a bucket's assignment get StopReplica sent to them + *
+ * + *

For example, if ORS = {1,2,3} and TRS = {4,5,6}, the values in the table and leader/isr + * paths in ZK may go through the following transitions. + * + * + * + * + * + * + * + * + * + *
RS AR RR leader isr step
{1,2,3} {} {} 1 {1,2,3} (initial state)
{4,5,6,1,2,3} {4,5,6} {1,2,3} 1 {1,2,3} (step A2)
{4,5,6,1,2,3} {4,5,6} {1,2,3} 1 {1,2,3,4,5,6} (phase B)
{4,5,6,1,2,3} {4,5,6} {1,2,3} 4 {1,2,3,4,5,6} (step B3)
{4,5,6,1,2,3} {4,5,6} {1,2,3} 4 {4,5,6} (step B4)
{4,5,6} {} {} 4 {4,5,6} (step B6)
+ * + *

Note that we have to update RS in ZK with TRS last since it's the only place where we + * store ORS persistently. This way, if the coordinatorServer crashes before that step, we can + * still recover. + */ + private void onBucketReassignment(TableBucket tableBucket, ReplicaReassignment reassignment) + throws Exception { + List addingReplicas = reassignment.addingReplicas; + List removingReplicas = reassignment.removingReplicas; + + if (!isReassignmentComplete(tableBucket, reassignment)) { + // A1. Send LeaderAndIsr request to every replica in ORS + TRS (with the new RS, AR and + // RR). + updateLeaderEpochAndSendRequest(tableBucket, reassignment); + + // A2. Set RS = TRS, AR = [], RR = [] in memory. + coordinatorContext.updateBucketReplicaAssignment(tableBucket, reassignment.replicas); + updateReplicaAssignmentForBucket(tableBucket, reassignment.replicas); + + // A3. replicas in AR -> NewReplica + // send the start replica request to the tabletSevers in the reassigned replicas list + // that are not in the assigned + addingReplicas.forEach( + replica -> + replicaStateMachine.handleStateChanges( + Collections.singleton( + new TableBucketReplica(tableBucket, replica)), + NewReplica)); + } else { + // B1. replicas in AR -> OnlineReplica + addingReplicas.forEach( + replica -> + replicaStateMachine.handleStateChanges( + Collections.singleton( + new TableBucketReplica(tableBucket, replica)), + OnlineReplica)); + List targetReplicas = reassignment.getTargetReplicas(); + // B3. Send LeaderAndIsr request with a potential new leader (if current leader not in + // TRS) and a new RS (using TRS) and same isr to every tabletServer in ORS + TRS or TRS + maybeReassignedBucketLeaderIfRequired(tableBucket, targetReplicas); + // B4. replicas in RR -> Offline (force those replicas out of isr) + // B5. replicas in RR -> NonExistentReplica (force those replicas to be deleted) + stopRemovedReplicasOfReassignedBucket(tableBucket, removingReplicas); + // B6. Set RS = TRS, AR = [], RR = [] in memory. + coordinatorContext.updateBucketReplicaAssignment( + tableBucket, reassignment.getTargetReplicas()); + // B7. Update ZK with RS = TRS, AR = [], RR = []. + updateReplicaAssignmentForBucket(tableBucket, targetReplicas); + // B8. After electing a leader in B3, the replicas and isr information changes, so + // resend the update metadata request to every tabletServer. + updateTabletServerMetadataCache( + new HashSet<>(coordinatorContext.getLiveTabletServers().values()), + null, + null, + Collections.singleton(tableBucket)); + // B9. Mark the ongoing rebalance task to finish. + rebalanceManager.finishRebalanceTask(tableBucket, RebalanceStatusForBucket.COMPLETED); + } + } + + private boolean isReassignmentComplete( + TableBucket tableBucket, ReplicaReassignment reassignment) throws Exception { + LeaderAndIsr leaderAndIsr = zooKeeperClient.getLeaderAndIsr(tableBucket).get(); + List isr = leaderAndIsr.isr(); + List targetReplicas = reassignment.getTargetReplicas(); + return targetReplicas.isEmpty() || new HashSet<>(isr).containsAll(targetReplicas); + } + + private void maybeReassignedBucketLeaderIfRequired( + TableBucket tableBucket, List targetReplicas) { + LeaderAndIsr leaderAndIsr = coordinatorContext.getBucketLeaderAndIsr(tableBucket).get(); + int currentLeader = leaderAndIsr.leader(); + if (currentLeader != targetReplicas.get(0)) { + LOG.info( + "Leader {} for tableBucket {} being reassigned. Re-electing leader to {}", + currentLeader, + tableBucket, + targetReplicas.get(0)); + tableBucketStateMachine.handleStateChange( + Collections.singleton(tableBucket), + OnlineBucket, + new ReassignmentLeaderElection(targetReplicas)); + } + } + + private void stopRemovedReplicasOfReassignedBucket( + TableBucket tableBucket, List removingReplicas) { + Set replicasToBeDeleted = new HashSet<>(); + removingReplicas.forEach( + replica -> replicasToBeDeleted.add(new TableBucketReplica(tableBucket, replica))); + replicaStateMachine.handleStateChanges(replicasToBeDeleted, OfflineReplica); + // send stop replica command to the old replicas. + replicaStateMachine.handleStateChanges(replicasToBeDeleted, ReplicaDeletionStarted); + } + + private void updateReplicaAssignmentForBucket( + TableBucket tableBucket, List targetReplicas) throws Exception { + long tableId = tableBucket.getTableId(); + @Nullable Long partitionId = tableBucket.getPartitionId(); + if (partitionId == null) { + Map> tableAssignment = + coordinatorContext.getTableAssignment(tableId); + tableAssignment.put(tableBucket.getBucket(), targetReplicas); + Map newTableAssignment = new HashMap<>(); + tableAssignment.forEach( + (bucket, replicas) -> + newTableAssignment.put(bucket, new BucketAssignment(replicas))); + zooKeeperClient.updateTableAssignment(tableId, new TableAssignment(newTableAssignment)); + } else { + Map> partitionAssignment = + coordinatorContext.getPartitionAssignment( + new TablePartition(tableId, partitionId)); + partitionAssignment.put(tableBucket.getBucket(), targetReplicas); + Map newPartitionAssignment = new HashMap<>(); + partitionAssignment.forEach( + (bucket, replicas) -> + newPartitionAssignment.put(bucket, new BucketAssignment(replicas))); + zooKeeperClient.updatePartitionAssignment( + partitionId, new PartitionAssignment(tableId, newPartitionAssignment)); + } + } + private List tryProcessAdjustIsr( Map leaderAndIsrList) { // TODO verify leader epoch. @@ -990,6 +1400,9 @@ private List tryProcessAdjustIsr( // update coordinator leader and isr cache. newLeaderAndIsrList.forEach(coordinatorContext::putBucketLeaderAndIsr); + // First, try to judge whether the bucket is in rebalance task when isr change. + newLeaderAndIsrList.keySet().forEach(this::tryToCompleteRebalanceTask); + // TODO update metadata for all alive tablet servers. return result; @@ -1043,6 +1456,13 @@ private void validateLeaderAndIsr(TableBucket tableBucket, LeaderAndIsr newLeade throw new IneligibleReplicaException(errorMsg); } } + + List isr = newLeaderAndIsr.isr(); + Set assignment = new HashSet<>(coordinatorContext.getAssignment(tableBucket)); + if (!assignment.containsAll(isr)) { + throw new FencedLeaderEpochException( + "The request isr in adjust isr request is not in assignment."); + } } } @@ -1243,7 +1663,7 @@ private ControlledShutdownResponse tryProcessControlledShutdown( } tableBucketStateMachine.handleStateChange( - bucketsLedByServer, OnlineBucket, CONTROLLED_SHUTDOWN_ELECTION); + bucketsLedByServer, OnlineBucket, new ControlledShutdownLeaderElection()); // TODO need send stop request to the leader? @@ -1353,8 +1773,107 @@ private void updateTabletServerMetadataCache( coordinatorRequestBatch.sendUpdateMetadataRequest(); } + private void updateLeaderEpochAndSendRequest( + TableBucket tableBucket, ReplicaReassignment reassignment) throws Exception { + Optional leaderAndIsrOpt = zooKeeperClient.getLeaderAndIsr(tableBucket); + if (!leaderAndIsrOpt.isPresent()) { + return; + } + LeaderAndIsr leaderAndIsr = leaderAndIsrOpt.get(); + + String partitionName = null; + if (tableBucket.getPartitionId() != null) { + partitionName = coordinatorContext.getPartitionName(tableBucket.getPartitionId()); + if (partitionName == null) { + LOG.error("Can't find partition name for partition: {}.", tableBucket.getBucket()); + return; + } + } + + coordinatorRequestBatch.newBatch(); + coordinatorRequestBatch.addNotifyLeaderRequestForTabletServers( + new HashSet<>(reassignment.replicas), + PhysicalTablePath.of( + coordinatorContext.getTablePathById(tableBucket.getTableId()), + partitionName), + tableBucket, + reassignment.replicas, + leaderAndIsr); + coordinatorRequestBatch.sendRequestToTabletServers( + coordinatorContext.getCoordinatorEpoch()); + } + @VisibleForTesting CompletedSnapshotStoreManager completedSnapshotStoreManager() { return completedSnapshotStoreManager; } + + private static final class ReplicaReassignment { + private final List replicas; + private final List addingReplicas; + private final List removingReplicas; + + private ReplicaReassignment( + List replicas, + List addingReplicas, + List removingReplicas) { + this.replicas = Collections.unmodifiableList(replicas); + this.addingReplicas = Collections.unmodifiableList(addingReplicas); + this.removingReplicas = Collections.unmodifiableList(removingReplicas); + } + + private static ReplicaReassignment build( + List originReplicas, List targetReplicas) { + // targetReplicas behind originReplicas in full set. + List fullReplicaSet = new ArrayList<>(targetReplicas); + fullReplicaSet.addAll(originReplicas); + fullReplicaSet = fullReplicaSet.stream().distinct().collect(Collectors.toList()); + + List newAddingReplicas = new ArrayList<>(fullReplicaSet); + newAddingReplicas.removeAll(originReplicas); + + List newRemovingReplicas = new ArrayList<>(originReplicas); + newRemovingReplicas.removeAll(targetReplicas); + + return new ReplicaReassignment(fullReplicaSet, newAddingReplicas, newRemovingReplicas); + } + + private List getTargetReplicas() { + List computed = new ArrayList<>(replicas); + computed.removeAll(removingReplicas); + return Collections.unmodifiableList(computed); + } + + private boolean isBeingReassigned() { + return !addingReplicas.isEmpty() || !removingReplicas.isEmpty(); + } + + @Override + public String toString() { + return String.format( + "ReplicaAssignment(replicas=%s, addingReplicas=%s, removingReplicas=%s)", + replicas, addingReplicas, removingReplicas); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + ReplicaReassignment that = (ReplicaReassignment) o; + return Objects.equals(replicas, that.replicas) + && Objects.equals(addingReplicas, that.addingReplicas) + && Objects.equals(removingReplicas, that.removingReplicas); + } + + @Override + public int hashCode() { + return Objects.hash(replicas, addingReplicas, removingReplicas); + } + } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java index 03e0bed924..507140e8d5 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java @@ -33,6 +33,7 @@ import org.apache.fluss.server.ServerBase; import org.apache.fluss.server.authorizer.Authorizer; import org.apache.fluss.server.authorizer.AuthorizerLoader; +import org.apache.fluss.server.coordinator.rebalance.RebalanceManager; import org.apache.fluss.server.metadata.CoordinatorMetadataCache; import org.apache.fluss.server.metadata.ServerMetadataCache; import org.apache.fluss.server.metrics.ServerMetricUtils; @@ -505,6 +506,11 @@ public DynamicConfigManager getDynamicConfigManager() { return dynamicConfigManager; } + @VisibleForTesting + public RebalanceManager getRebalanceManager() { + return coordinatorEventProcessor.getRebalanceManager(); + } + private static void validateConfigs(Configuration conf) { if (conf.get(ConfigOptions.DEFAULT_REPLICATION_FACTOR) < 1) { throw new IllegalConfigurationException( diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorService.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorService.java index 8fe9285bb3..cf706c7371 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorService.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorService.java @@ -20,6 +20,8 @@ import org.apache.fluss.annotation.VisibleForTesting; import org.apache.fluss.cluster.ServerType; import org.apache.fluss.cluster.TabletServerInfo; +import org.apache.fluss.cluster.rebalance.GoalType; +import org.apache.fluss.cluster.rebalance.ServerTag; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; import org.apache.fluss.config.cluster.AlterConfig; @@ -29,6 +31,7 @@ import org.apache.fluss.exception.InvalidDatabaseException; import org.apache.fluss.exception.InvalidTableException; import org.apache.fluss.exception.LakeTableAlreadyExistException; +import org.apache.fluss.exception.RebalanceFailureException; import org.apache.fluss.exception.SecurityDisabledException; import org.apache.fluss.exception.TableAlreadyExistException; import org.apache.fluss.exception.TableNotPartitionedException; @@ -43,12 +46,16 @@ import org.apache.fluss.metadata.TableDescriptor; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.rpc.gateway.CoordinatorGateway; +import org.apache.fluss.rpc.messages.AddServerTagRequest; +import org.apache.fluss.rpc.messages.AddServerTagResponse; import org.apache.fluss.rpc.messages.AdjustIsrRequest; import org.apache.fluss.rpc.messages.AdjustIsrResponse; import org.apache.fluss.rpc.messages.AlterClusterConfigsRequest; import org.apache.fluss.rpc.messages.AlterClusterConfigsResponse; import org.apache.fluss.rpc.messages.AlterTableRequest; import org.apache.fluss.rpc.messages.AlterTableResponse; +import org.apache.fluss.rpc.messages.CancelRebalanceRequest; +import org.apache.fluss.rpc.messages.CancelRebalanceResponse; import org.apache.fluss.rpc.messages.CommitKvSnapshotRequest; import org.apache.fluss.rpc.messages.CommitKvSnapshotResponse; import org.apache.fluss.rpc.messages.CommitLakeTableSnapshotRequest; @@ -75,11 +82,17 @@ import org.apache.fluss.rpc.messages.DropTableResponse; import org.apache.fluss.rpc.messages.LakeTieringHeartbeatRequest; import org.apache.fluss.rpc.messages.LakeTieringHeartbeatResponse; +import org.apache.fluss.rpc.messages.ListRebalanceProcessRequest; +import org.apache.fluss.rpc.messages.ListRebalanceProcessResponse; import org.apache.fluss.rpc.messages.MetadataRequest; import org.apache.fluss.rpc.messages.MetadataResponse; import org.apache.fluss.rpc.messages.PbAlterConfig; import org.apache.fluss.rpc.messages.PbHeartbeatReqForTable; import org.apache.fluss.rpc.messages.PbHeartbeatRespForTable; +import org.apache.fluss.rpc.messages.RebalanceRequest; +import org.apache.fluss.rpc.messages.RebalanceResponse; +import org.apache.fluss.rpc.messages.RemoveServerTagRequest; +import org.apache.fluss.rpc.messages.RemoveServerTagResponse; import org.apache.fluss.rpc.netty.server.Session; import org.apache.fluss.rpc.protocol.ApiError; import org.apache.fluss.security.acl.AclBinding; @@ -92,12 +105,16 @@ import org.apache.fluss.server.authorizer.AclDeleteResult; import org.apache.fluss.server.authorizer.Authorizer; import org.apache.fluss.server.coordinator.event.AccessContextEvent; +import org.apache.fluss.server.coordinator.event.AddServerTagEvent; import org.apache.fluss.server.coordinator.event.AdjustIsrReceivedEvent; import org.apache.fluss.server.coordinator.event.CommitKvSnapshotEvent; import org.apache.fluss.server.coordinator.event.CommitLakeTableSnapshotEvent; import org.apache.fluss.server.coordinator.event.CommitRemoteLogManifestEvent; import org.apache.fluss.server.coordinator.event.ControlledShutdownEvent; import org.apache.fluss.server.coordinator.event.EventManager; +import org.apache.fluss.server.coordinator.event.RemoveServerTagEvent; +import org.apache.fluss.server.coordinator.rebalance.RebalanceManager; +import org.apache.fluss.server.coordinator.rebalance.goal.Goal; import org.apache.fluss.server.entity.CommitKvSnapshotData; import org.apache.fluss.server.entity.LakeTieringTableInfo; import org.apache.fluss.server.entity.TablePropertyChanges; @@ -108,6 +125,7 @@ import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.server.zk.data.BucketAssignment; import org.apache.fluss.server.zk.data.PartitionAssignment; +import org.apache.fluss.server.zk.data.RebalancePlan; import org.apache.fluss.server.zk.data.TableAssignment; import org.apache.fluss.server.zk.data.TableRegistration; import org.apache.fluss.utils.IOUtils; @@ -116,6 +134,9 @@ import javax.annotation.Nullable; import java.io.UncheckedIOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -126,6 +147,7 @@ import static org.apache.fluss.config.FlussConfigUtils.isTableStorageConfig; import static org.apache.fluss.rpc.util.CommonRpcMessageUtils.toAclBindingFilters; import static org.apache.fluss.rpc.util.CommonRpcMessageUtils.toAclBindings; +import static org.apache.fluss.server.coordinator.rebalance.goal.GoalUtils.getGoalByType; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.fromTablePath; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.getAdjustIsrData; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.getCommitLakeTableSnapshotData; @@ -133,6 +155,8 @@ import static org.apache.fluss.server.utils.ServerRpcMessageUtils.getPartitionSpec; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.makeCreateAclsResponse; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.makeDropAclsResponse; +import static org.apache.fluss.server.utils.ServerRpcMessageUtils.makeListRebalanceProcessResponse; +import static org.apache.fluss.server.utils.ServerRpcMessageUtils.makeRebalanceRespose; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.toTableChanges; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.toTablePath; import static org.apache.fluss.server.utils.TableAssignmentUtils.generateAssignment; @@ -141,7 +165,6 @@ /** An RPC Gateway service for coordinator server. */ public final class CoordinatorService extends RpcServiceBase implements CoordinatorGateway { - private final int defaultBucketNumber; private final int defaultReplicationFactor; private final boolean logTableAllowCreation; @@ -152,6 +175,10 @@ public final class CoordinatorService extends RpcServiceBase implements Coordina private final LakeTableTieringManager lakeTableTieringManager; private final LakeCatalogDynamicLoader lakeCatalogDynamicLoader; + private final Supplier rebalanceManagerSupplier; + + // This parameter is only used for testing. + private final boolean generateUnBalanceAssignment; public CoordinatorService( Configuration conf, @@ -175,11 +202,15 @@ public CoordinatorService( this.defaultReplicationFactor = conf.getInt(ConfigOptions.DEFAULT_REPLICATION_FACTOR); this.logTableAllowCreation = conf.getBoolean(ConfigOptions.LOG_TABLE_ALLOW_CREATION); this.kvTableAllowCreation = conf.getBoolean(ConfigOptions.KV_TABLE_ALLOW_CREATION); + this.generateUnBalanceAssignment = + conf.getBoolean(ConfigOptions.SERVER_GENERATE_UNBALANCE_ASSIGNMENT_FOR_TEST); this.eventManagerSupplier = () -> coordinatorEventProcessorSupplier.get().getCoordinatorEventManager(); this.coordinatorEpochSupplier = () -> coordinatorEventProcessorSupplier.get().getCoordinatorEpoch(); this.lakeTableTieringManager = lakeTableTieringManager; + this.rebalanceManagerSupplier = + () -> coordinatorEventProcessorSupplier.get().getRebalanceManager(); this.metadataCache = metadataCache; this.lakeCatalogDynamicLoader = lakeCatalogDynamicLoader; } @@ -278,7 +309,14 @@ public CompletableFuture createTable(CreateTableRequest req // the replication factor must be set now int replicaFactor = tableDescriptor.getReplicationFactor(); TabletServerInfo[] servers = metadataCache.getLiveServers(); - tableAssignment = generateAssignment(bucketCount, replicaFactor, servers); + if (generateUnBalanceAssignment) { + // this branch is only used for testing. + tableAssignment = + new TableAssignment( + generateUnBalanceAssignment(bucketCount, replicaFactor)); + } else { + tableAssignment = generateAssignment(bucketCount, replicaFactor, servers); + } } // TODO: should tolerate if the lake exist but matches our schema. This ensures eventually @@ -471,9 +509,16 @@ public CompletableFuture createPartition( // second, generate the PartitionAssignment. int replicaFactor = table.getTableConfig().getReplicationFactor(); TabletServerInfo[] servers = metadataCache.getLiveServers(); - Map bucketAssignments = - generateAssignment(table.bucketCount, replicaFactor, servers) - .getBucketAssignments(); + Map bucketAssignments; + if (generateUnBalanceAssignment) { + // This branch is only used for testing. + bucketAssignments = generateUnBalanceAssignment(table.bucketCount, replicaFactor); + } else { + bucketAssignments = + generateAssignment(table.bucketCount, replicaFactor, servers) + .getBucketAssignments(); + } + PartitionAssignment partitionAssignment = new PartitionAssignment(table.tableId, bucketAssignments); @@ -520,16 +565,15 @@ public CompletableFuture metadata(MetadataRequest request) { AccessContextEvent metadataResponseAccessContextEvent = new AccessContextEvent<>( - ctx -> { - return processMetadataRequest( - request, - listenerName, - session, - authorizer, - metadataCache, - new CoordinatorMetadataProvider( - zkClient, metadataManager, ctx)); - }); + ctx -> + processMetadataRequest( + request, + listenerName, + session, + authorizer, + metadataCache, + new CoordinatorMetadataProvider( + zkClient, metadataManager, ctx))); eventManagerSupplier.get().put(metadataResponseAccessContextEvent); return metadataResponseAccessContextEvent.getResultFuture(); } @@ -715,6 +759,86 @@ public CompletableFuture alterClusterConfigs( return future; } + @Override + public CompletableFuture addServerTag(AddServerTagRequest request) { + CompletableFuture response = new CompletableFuture<>(); + eventManagerSupplier + .get() + .put( + new AddServerTagEvent( + Arrays.stream(request.getServerIds()) + .boxed() + .collect(Collectors.toList()), + ServerTag.valueOf(request.getServerTag()), + response)); + return response; + } + + @Override + public CompletableFuture removeServerTag( + RemoveServerTagRequest request) { + CompletableFuture response = new CompletableFuture<>(); + eventManagerSupplier + .get() + .put( + new RemoveServerTagEvent( + Arrays.stream(request.getServerIds()) + .boxed() + .collect(Collectors.toList()), + ServerTag.valueOf(request.getServerTag()), + response)); + return response; + } + + @Override + public CompletableFuture rebalance(RebalanceRequest request) { + List goalsByPriority = new ArrayList<>(); + Arrays.stream(request.getGoals()) + .forEach(goal -> goalsByPriority.add(getGoalByType(GoalType.valueOf(goal)))); + boolean isDryRun = request.isDryRun(); + + // 1. generate rebalance plan. + RebalanceManager rebalanceManager = rebalanceManagerSupplier.get(); + RebalancePlan rebalancePlan; + try { + rebalancePlan = rebalanceManager.generateRebalancePlan(goalsByPriority); + } catch (Exception e) { + throw new RebalanceFailureException("Failed to generate rebalance plan.", e); + } + + if (!isDryRun) { + if (rebalanceManager.hasOngoingRebalance()) { + throw new RebalanceFailureException( + "Rebalance task already exists. Please wait for it to finish or cancel it first."); + } + + // 2. execute rebalance plan. + rebalanceManager.registerRebalance(rebalancePlan.getExecutePlan()); + } + return CompletableFuture.completedFuture(makeRebalanceRespose(rebalancePlan)); + } + + @Override + public CompletableFuture listRebalanceProcess( + ListRebalanceProcessRequest request) { + if (authorizer != null) { + authorizer.authorize(currentSession(), OperationType.DESCRIBE, Resource.cluster()); + } + + RebalanceManager rebalanceManager = rebalanceManagerSupplier.get(); + return CompletableFuture.completedFuture( + makeListRebalanceProcessResponse( + rebalanceManager.getOngoingRebalanceTasks(), + rebalanceManager.getFinishedRebalanceTasks())); + } + + @Override + public CompletableFuture cancelRebalance( + CancelRebalanceRequest request) { + rebalanceManagerSupplier.get().cancelRebalance(); + return CompletableFuture.completedFuture(new CancelRebalanceResponse()); + } + @VisibleForTesting public DataLakeFormat getDataLakeFormat() { return lakeCatalogDynamicLoader.getLakeCatalogContainer().getDataLakeFormat(); @@ -757,4 +881,22 @@ private void validateTableCreationPermission( } } } + + private Map generateUnBalanceAssignment( + int nBuckets, int replicationFactor) { + Map assignments = new HashMap<>(); + for (int i = 0; i < nBuckets; i++) { + if (replicationFactor == 1) { + assignments.put(i, new BucketAssignment(Collections.singletonList(0))); + } else if (replicationFactor == 2) { + assignments.put(i, new BucketAssignment(Arrays.asList(0, 1))); + } else if (replicationFactor == 3) { + assignments.put(i, new BucketAssignment(Arrays.asList(0, 1, 2))); + } else { + throw new IllegalArgumentException( + "replicationFactor must be 1, 2 or 3 for unbalance assignment."); + } + } + return assignments; + } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/TableManager.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/TableManager.java index b92a2dad6e..ed62f4e62c 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/TableManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/TableManager.java @@ -34,6 +34,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.ArrayList; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -207,6 +208,7 @@ public void resumeDeletions() { } private void resumeTableDeletions() { + new ArrayList<>(8); Set tablesToBeDeleted = new HashSet<>(coordinatorContext.getTablesToBeDeleted()); Set eligibleTableDeletion = new HashSet<>(); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/AddServerTagEvent.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/AddServerTagEvent.java new file mode 100644 index 0000000000..b6e7af8886 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/AddServerTagEvent.java @@ -0,0 +1,52 @@ +/* + * 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.fluss.server.coordinator.event; + +import org.apache.fluss.cluster.rebalance.ServerTag; +import org.apache.fluss.rpc.messages.AddServerTagResponse; + +import java.util.List; +import java.util.concurrent.CompletableFuture; + +/** An event for add server tag. */ +public class AddServerTagEvent implements CoordinatorEvent { + private final List serverIds; + private final ServerTag serverTag; + private final CompletableFuture respCallback; + + public AddServerTagEvent( + List serverIds, + ServerTag serverTag, + CompletableFuture respCallback) { + this.serverIds = serverIds; + this.serverTag = serverTag; + this.respCallback = respCallback; + } + + public List getServerIds() { + return serverIds; + } + + public ServerTag getServerTag() { + return serverTag; + } + + public CompletableFuture getRespCallback() { + return respCallback; + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/ExecuteRebalanceTaskEvent.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/ExecuteRebalanceTaskEvent.java new file mode 100644 index 0000000000..074a6c7fb5 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/ExecuteRebalanceTaskEvent.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.coordinator.event; + +import org.apache.fluss.cluster.rebalance.RebalancePlanForBucket; + +/** An event of executing rebalance task. */ +public class ExecuteRebalanceTaskEvent implements CoordinatorEvent { + private final RebalancePlanForBucket rebalancePlanForBucket; + + public ExecuteRebalanceTaskEvent(RebalancePlanForBucket rebalancePlanForBucket) { + this.rebalancePlanForBucket = rebalancePlanForBucket; + } + + public RebalancePlanForBucket getRebalancePlanForBucket() { + return rebalancePlanForBucket; + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/RemoveServerTagEvent.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/RemoveServerTagEvent.java new file mode 100644 index 0000000000..ede6fdeb0c --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/RemoveServerTagEvent.java @@ -0,0 +1,52 @@ +/* + * 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.fluss.server.coordinator.event; + +import org.apache.fluss.cluster.rebalance.ServerTag; +import org.apache.fluss.rpc.messages.RemoveServerTagResponse; + +import java.util.List; +import java.util.concurrent.CompletableFuture; + +/** An event for remove server tag. */ +public class RemoveServerTagEvent implements CoordinatorEvent { + private final List serverIds; + private final ServerTag serverTag; + private final CompletableFuture respCallback; + + public RemoveServerTagEvent( + List serverIds, + ServerTag serverTag, + CompletableFuture respCallback) { + this.serverIds = serverIds; + this.serverTag = serverTag; + this.respCallback = respCallback; + } + + public List getServerIds() { + return serverIds; + } + + public ServerTag getServerTag() { + return serverTag; + } + + public CompletableFuture getRespCallback() { + return respCallback; + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/ActionAcceptance.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/ActionAcceptance.java new file mode 100644 index 0000000000..5e815e9c34 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/ActionAcceptance.java @@ -0,0 +1,37 @@ +/* + * 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.fluss.server.coordinator.rebalance; + +/** Flags to indicate if an action is acceptable by the goal(s). */ +public enum ActionAcceptance { + /** Action is acceptable -- i.e. it does not violate goal constraints. */ + ACCEPT, + /** + * Action is rejected in replica-level. But, the destination tabletServer may potentially accept + * actions of the same {@link ActionType} from the source tabletServer specified in the given + * action. + */ + REPLICA_REJECT, + + /** + * Action is rejected in server-level. hence, the destination tabletServer does not accept + * actions of the same {@link ActionType} from the source tabletServer specified in the given + * action. + */ + SERVER_REJECT +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/ActionType.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/ActionType.java new file mode 100644 index 0000000000..a24bf8acb6 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/ActionType.java @@ -0,0 +1,30 @@ +/* + * 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.fluss.server.coordinator.rebalance; + +/** Flags to indicate the type of action. */ +public enum ActionType { + /** Move a replica from a source tabletServer to a destination tabletServer. */ + REPLICA_MOVEMENT, + + /** + * Move leadership of a leader from a source tabletServer to a follower of the same replica + * residing in a destination tabletServer. + */ + LEADERSHIP_MOVEMENT; +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/ReBalancingAction.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/ReBalancingAction.java new file mode 100644 index 0000000000..bd62f82c2a --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/ReBalancingAction.java @@ -0,0 +1,101 @@ +/* + * 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.fluss.server.coordinator.rebalance; + +import org.apache.fluss.metadata.TableBucket; + +/** Represents the load rebalancing operation over a replica for Fluss Load GoalOptimizer. */ +public class ReBalancingAction { + private final TableBucket tableBucket; + private final Integer sourceServerId; + private final Integer destinationServerId; + private final ActionType actionType; + + public ReBalancingAction( + TableBucket tableBucket, + Integer sourceServerId, + Integer destinationServerId, + ActionType actionType) { + this.tableBucket = tableBucket; + this.sourceServerId = sourceServerId; + this.destinationServerId = destinationServerId; + this.actionType = actionType; + } + + public TableBucket getTableBucket() { + return tableBucket; + } + + public Integer getSourceServerId() { + return sourceServerId; + } + + public Integer getDestinationServerId() { + return destinationServerId; + } + + public ActionType getActionType() { + return actionType; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + ReBalancingAction that = (ReBalancingAction) o; + + if (!tableBucket.equals(that.tableBucket)) { + return false; + } + if (!sourceServerId.equals(that.sourceServerId)) { + return false; + } + if (!destinationServerId.equals(that.destinationServerId)) { + return false; + } + return actionType == that.actionType; + } + + @Override + public int hashCode() { + int result = tableBucket.hashCode(); + result = 31 * result + sourceServerId.hashCode(); + result = 31 * result + destinationServerId.hashCode(); + result = 31 * result + actionType.hashCode(); + return result; + } + + @Override + public String toString() { + return "ReBalancingAction{" + + "tableBucket=" + + tableBucket + + ", sourceServerId=" + + sourceServerId + + ", destinationServerId=" + + destinationServerId + + ", actionType=" + + actionType + + '}'; + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/RebalanceManager.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/RebalanceManager.java new file mode 100644 index 0000000000..1cbbf4fd24 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/RebalanceManager.java @@ -0,0 +1,344 @@ +/* + * 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.fluss.server.coordinator.rebalance; + +import org.apache.fluss.annotation.VisibleForTesting; +import org.apache.fluss.cluster.rebalance.RebalancePlanForBucket; +import org.apache.fluss.cluster.rebalance.RebalanceResultForBucket; +import org.apache.fluss.cluster.rebalance.RebalanceStatusForBucket; +import org.apache.fluss.cluster.rebalance.ServerTag; +import org.apache.fluss.exception.RebalanceFailureException; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.server.coordinator.CoordinatorContext; +import org.apache.fluss.server.coordinator.event.AccessContextEvent; +import org.apache.fluss.server.coordinator.event.EventManager; +import org.apache.fluss.server.coordinator.event.ExecuteRebalanceTaskEvent; +import org.apache.fluss.server.coordinator.rebalance.goal.Goal; +import org.apache.fluss.server.coordinator.rebalance.goal.GoalOptimizer; +import org.apache.fluss.server.coordinator.rebalance.model.ClusterModel; +import org.apache.fluss.server.coordinator.rebalance.model.RackModel; +import org.apache.fluss.server.coordinator.rebalance.model.ServerModel; +import org.apache.fluss.server.metadata.ServerInfo; +import org.apache.fluss.server.zk.ZooKeeperClient; +import org.apache.fluss.server.zk.data.LeaderAndIsr; +import org.apache.fluss.server.zk.data.RebalancePlan; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; +import javax.annotation.concurrent.GuardedBy; + +import java.util.ArrayDeque; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Queue; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; +import java.util.function.Supplier; + +import static org.apache.fluss.utils.Preconditions.checkNotNull; +import static org.apache.fluss.utils.concurrent.LockUtils.inLock; + +/** A rebalance manager to generate rebalance plan, and execution rebalance plan. */ +public class RebalanceManager { + + private static final Logger LOG = LoggerFactory.getLogger(RebalanceManager.class); + + private final AtomicBoolean isClosed = new AtomicBoolean(false); + private final Lock lock = new ReentrantLock(); + private final ZooKeeperClient zkClient; + private final Supplier eventManagerSupplier; + + @GuardedBy("lock") + private final Queue ongoingRebalanceTasksQueue = new ArrayDeque<>(); + + /** A mapping from table bucket to rebalance status of pending and running tasks. */ + @GuardedBy("lock") + private final Map ongoingRebalanceTasks = + new HashMap<>(); + + /** A mapping from table bucket to rebalance status of failed or completed tasks. */ + @GuardedBy("lock") + private final Map finishedRebalanceTasks = + new HashMap<>(); + + @GuardedBy("lock") + private final GoalOptimizer goalOptimizer; + + @GuardedBy("lock") + private long registerTime; + + public RebalanceManager(Supplier eventManagerSupplier, ZooKeeperClient zkClient) { + this.eventManagerSupplier = eventManagerSupplier; + this.zkClient = zkClient; + this.goalOptimizer = new GoalOptimizer(); + } + + public void startup() { + LOG.info("Start up rebalance manager."); + registerRebalanceFromZookeeper(); + } + + public Map getOngoingRebalanceTasks() { + checkNotClosed(); + return inLock(lock, () -> ongoingRebalanceTasks); + } + + public Map getFinishedRebalanceTasks() { + checkNotClosed(); + return inLock(lock, () -> finishedRebalanceTasks); + } + + private void registerRebalanceFromZookeeper() { + try { + zkClient.getRebalancePlan() + .ifPresent(rebalancePlan -> registerRebalance(rebalancePlan.getExecutePlan())); + } catch (Exception e) { + LOG.error( + "Failed to get rebalance plan from zookeeper, it will be treated as no" + + "rebalance tasks.", + e); + } + } + + public void registerRebalance(Map rebalancePlan) { + checkNotClosed(); + inLock( + lock, + () -> { + registerTime = System.currentTimeMillis(); + // Register to zookeeper first. + try { + zkClient.registerRebalancePlan(new RebalancePlan(rebalancePlan)); + } catch (Exception e) { + LOG.error("Error when register rebalance plan to zookeeper.", e); + throw new RebalanceFailureException( + "Error when register rebalance plan to zookeeper.", e); + } + + // Then, register to ongoingRebalanceTasks. + rebalancePlan.forEach( + ((tableBucket, rebalancePlanForBucket) -> { + ongoingRebalanceTasksQueue.add(tableBucket); + ongoingRebalanceTasks.put( + tableBucket, + RebalanceResultForBucket.of( + rebalancePlanForBucket, + RebalanceStatusForBucket.PENDING)); + })); + + // Trigger one rebalance task to execute. + processNewRebalanceTask(); + }); + } + + public void finishRebalanceTask( + TableBucket tableBucket, RebalanceStatusForBucket statusForBucket) { + checkNotClosed(); + inLock( + lock, + () -> { + if (ongoingRebalanceTasksQueue.contains(tableBucket)) { + ongoingRebalanceTasksQueue.remove(tableBucket); + RebalanceResultForBucket resultForBucket = + ongoingRebalanceTasks.remove(tableBucket); + checkNotNull(resultForBucket, "RebalanceResultForBucket is null."); + finishedRebalanceTasks.put( + tableBucket, resultForBucket.setNewStatus(statusForBucket)); + LOG.info( + "Rebalance in progress: {} tasks pending, {} completed.", + ongoingRebalanceTasksQueue.size(), + finishedRebalanceTasks.size()); + + if (ongoingRebalanceTasksQueue.isEmpty()) { + // All rebalance tasks are completed. + completeRebalance(); + } else { + // Trigger one rebalance task to execute. + processNewRebalanceTask(); + } + } + }); + } + + public void cancelRebalance() { + checkNotClosed(); + inLock( + lock, + () -> { + try { + zkClient.deleteRebalancePlan(); + } catch (Exception e) { + LOG.error("Error when delete rebalance plan from zookeeper.", e); + } + + ongoingRebalanceTasksQueue.clear(); + ongoingRebalanceTasks.clear(); + finishedRebalanceTasks.clear(); + LOG.info("Cancel rebalance task success."); + }); + } + + public boolean hasOngoingRebalance() { + checkNotClosed(); + return inLock( + lock, () -> !ongoingRebalanceTasks.isEmpty() || !finishedRebalanceTasks.isEmpty()); + } + + public RebalancePlan generateRebalancePlan(List goalsByPriority) throws Exception { + checkNotClosed(); + return inLock( + lock, + () -> { + List rebalancePlanForBuckets; + try { + // Generate the latest cluster model. + ClusterModel clusterModel = getClusterModel(); + + // do optimize. + rebalancePlanForBuckets = + goalOptimizer.doOptimizeOnce(clusterModel, goalsByPriority); + } catch (Exception e) { + LOG.error("Failed to generate rebalance plan.", e); + throw e; + } + + // group by tableId and partitionId to generate rebalance plan. + return buildRebalancePlan(rebalancePlanForBuckets); + }); + } + + public @Nullable RebalancePlanForBucket getRebalancePlanForBucket(TableBucket tableBucket) { + checkNotClosed(); + return inLock( + lock, + () -> { + RebalanceResultForBucket resultForBucket = + ongoingRebalanceTasks.get(tableBucket); + if (resultForBucket != null) { + return resultForBucket.planForBucket(); + } + return null; + }); + } + + private void processNewRebalanceTask() { + TableBucket tableBucket = ongoingRebalanceTasksQueue.peek(); + if (tableBucket != null && ongoingRebalanceTasks.containsKey(tableBucket)) { + RebalanceResultForBucket rebalanceResultForBucket = + ongoingRebalanceTasks + .get(tableBucket) + .setNewStatus(RebalanceStatusForBucket.REBALANCING); + eventManagerSupplier + .get() + .put(new ExecuteRebalanceTaskEvent(rebalanceResultForBucket.planForBucket())); + } + } + + private void completeRebalance() { + checkNotClosed(); + inLock( + lock, + () -> { + try { + zkClient.deleteRebalancePlan(); + } catch (Exception e) { + LOG.error("Error when delete rebalance plan from zookeeper.", e); + } + + ongoingRebalanceTasks.clear(); + finishedRebalanceTasks.clear(); + LOG.info( + "Rebalance complete with {} ms.", + System.currentTimeMillis() - registerTime); + }); + } + + @VisibleForTesting + public ClusterModel getClusterModel() throws Exception { + AccessContextEvent accessContextEvent = + new AccessContextEvent<>(this::buildClusterModel); + eventManagerSupplier.get().put(accessContextEvent); + return accessContextEvent.getResultFuture().get(); + } + + private ClusterModel buildClusterModel(CoordinatorContext coordinatorContext) { + Map liveTabletServers = coordinatorContext.getLiveTabletServers(); + Map serverTags = coordinatorContext.getServerTags(); + + Map serverModelMap = new HashMap<>(); + for (ServerInfo serverInfo : liveTabletServers.values()) { + Integer id = serverInfo.id(); + String rack = serverInfo.rack() == null ? RackModel.DEFAULT_RACK : serverInfo.rack(); + if (serverTags.containsKey(id)) { + serverModelMap.put( + id, new ServerModel(id, rack, !isServerOffline(serverTags.get(id)))); + } else { + serverModelMap.put(id, new ServerModel(id, rack, true)); + } + } + + ClusterModel clusterModel = initialClusterModel(serverModelMap); + + // Try to update the cluster model with the latest bucket states. + Set allBuckets = coordinatorContext.getAllBuckets(); + for (TableBucket tableBucket : allBuckets) { + List assignment = coordinatorContext.getAssignment(tableBucket); + LeaderAndIsr isr = coordinatorContext.getBucketLeaderAndIsr(tableBucket).get(); + int leader = isr.leader(); + for (int i = 0; i < assignment.size(); i++) { + int replica = assignment.get(i); + clusterModel.createReplica(replica, tableBucket, i, leader == replica); + } + } + return clusterModel; + } + + private RebalancePlan buildRebalancePlan(List rebalancePlanForBuckets) { + Map bucketPlan = new HashMap<>(); + for (RebalancePlanForBucket rebalancePlanForBucket : rebalancePlanForBuckets) { + bucketPlan.put(rebalancePlanForBucket.getTableBucket(), rebalancePlanForBucket); + } + return new RebalancePlan(bucketPlan); + } + + private boolean isServerOffline(ServerTag serverTag) { + return serverTag == ServerTag.PERMANENT_OFFLINE || serverTag == ServerTag.TEMPORARY_OFFLINE; + } + + private ClusterModel initialClusterModel(Map serverModelMap) { + SortedSet servers = new TreeSet<>(serverModelMap.values()); + return new ClusterModel(servers); + } + + private void checkNotClosed() { + if (isClosed.get()) { + throw new IllegalStateException("RebalanceManager is already closed."); + } + } + + public void close() { + isClosed.compareAndSet(false, true); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/ReplicaReassignment.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/ReplicaReassignment.java new file mode 100644 index 0000000000..1e4ff17ea2 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/ReplicaReassignment.java @@ -0,0 +1,98 @@ +/* + * 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.fluss.server.coordinator.rebalance; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; + +/** Replica reassignment. */ +public class ReplicaReassignment { + private final List replicas; + private final List addingReplicas; + private final List removingReplicas; + + private ReplicaReassignment( + List replicas, List addingReplicas, List removingReplicas) { + this.replicas = Collections.unmodifiableList(replicas); + this.addingReplicas = Collections.unmodifiableList(addingReplicas); + this.removingReplicas = Collections.unmodifiableList(removingReplicas); + } + + private static ReplicaReassignment build( + List originReplicas, List targetReplicas) { + // targetReplicas behind originReplicas in full set. + List fullReplicaSet = new ArrayList<>(targetReplicas); + fullReplicaSet.addAll(originReplicas); + fullReplicaSet = fullReplicaSet.stream().distinct().collect(Collectors.toList()); + + List newAddingReplicas = new ArrayList<>(fullReplicaSet); + newAddingReplicas.removeAll(originReplicas); + + List newRemovingReplicas = new ArrayList<>(originReplicas); + newRemovingReplicas.removeAll(targetReplicas); + + return new ReplicaReassignment(fullReplicaSet, newAddingReplicas, newRemovingReplicas); + } + + private List getTargetReplicas() { + List computed = new ArrayList<>(replicas); + computed.removeAll(removingReplicas); + return Collections.unmodifiableList(computed); + } + + private List getOriginReplicas() { + List computed = new ArrayList<>(replicas); + computed.removeAll(addingReplicas); + return Collections.unmodifiableList(computed); + } + + private boolean isBeingReassigned() { + return !addingReplicas.isEmpty() || !removingReplicas.isEmpty(); + } + + @Override + public String toString() { + return String.format( + "ReplicaAssignment(replicas=%s, addingReplicas=%s, removingReplicas=%s)", + replicas, addingReplicas, removingReplicas); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + ReplicaReassignment that = (ReplicaReassignment) o; + return Objects.equals(replicas, that.replicas) + && Objects.equals(addingReplicas, that.addingReplicas) + && Objects.equals(removingReplicas, that.removingReplicas); + } + + @Override + public int hashCode() { + return Objects.hash(replicas, addingReplicas, removingReplicas); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/AbstractGoal.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/AbstractGoal.java new file mode 100644 index 0000000000..513d7209db --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/AbstractGoal.java @@ -0,0 +1,195 @@ +/* + * 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.fluss.server.coordinator.rebalance.goal; + +import org.apache.fluss.exception.RebalanceFailureException; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.server.coordinator.rebalance.ActionAcceptance; +import org.apache.fluss.server.coordinator.rebalance.ActionType; +import org.apache.fluss.server.coordinator.rebalance.ReBalancingAction; +import org.apache.fluss.server.coordinator.rebalance.model.ClusterModel; +import org.apache.fluss.server.coordinator.rebalance.model.ClusterModelStats; +import org.apache.fluss.server.coordinator.rebalance.model.ReplicaModel; +import org.apache.fluss.server.coordinator.rebalance.model.ServerModel; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Set; +import java.util.SortedSet; + +import static org.apache.fluss.server.coordinator.rebalance.ActionAcceptance.ACCEPT; +import static org.apache.fluss.server.coordinator.rebalance.ActionType.LEADERSHIP_MOVEMENT; +import static org.apache.fluss.server.coordinator.rebalance.ActionType.REPLICA_MOVEMENT; +import static org.apache.fluss.server.coordinator.rebalance.goal.GoalOptimizerUtils.isProposalAcceptableForOptimizedGoals; +import static org.apache.fluss.server.coordinator.rebalance.goal.GoalUtils.legitMove; + +/** An abstract class for goals. */ +public abstract class AbstractGoal implements Goal { + private static final Logger LOG = LoggerFactory.getLogger(AbstractGoal.class); + protected boolean finished; + protected boolean succeeded; + + public AbstractGoal() { + finished = false; + succeeded = true; + } + + @Override + public void optimize(ClusterModel clusterModel, Set optimizedGoals) { + LOG.debug("Starting Optimizing for goal {}", name()); + // Initialize pre-optimized stats. + ClusterModelStats statsBeforeOptimization = clusterModel.getClusterStats(); + LOG.trace("[PRE - {}] {}", name(), statsBeforeOptimization); + finished = false; + long goalStartTime = System.currentTimeMillis(); + initGoalState(clusterModel); + SortedSet offlineServers = clusterModel.offlineServers(); + + while (!finished) { + for (ServerModel server : serversToBalance(clusterModel)) { + rebalanceForServer(server, clusterModel, optimizedGoals); + } + updateGoalState(clusterModel); + } + + ClusterModelStats statsAfterOptimization = clusterModel.getClusterStats(); + LOG.trace("[POST - {}] {}", name(), statsAfterOptimization); + if (LOG.isDebugEnabled()) { + LOG.debug( + "Finished optimization for {} in {}ms.", + name(), + System.currentTimeMillis() - goalStartTime); + } + LOG.trace("Cluster after optimization is {}", clusterModel); + // The optimization cannot make stats worse unless the cluster has (1) offline servers for + // replica move with replicas. + if (offlineServers.isEmpty()) { + ClusterModelStatsComparator comparator = clusterModelStatsComparator(); + // Throw exception when the stats before optimization is preferred. + if (comparator.compare(statsAfterOptimization, statsBeforeOptimization) < 0) { + // If a goal provides worse stats after optimization, that indicates an + // implementation error with the goal. + throw new IllegalStateException( + String.format( + "Optimization for goal %s failed because the optimized result is worse than before." + + " Reason: %s.", + name(), comparator.explainLastComparison())); + } + } + } + + @Override + public void finish() { + finished = true; + } + + @Override + public String name() { + return this.getClass().getSimpleName(); + } + + /** + * Get sorted tabletServers that the rebalance process will go over to apply balancing actions + * to replicas they contain. + */ + protected SortedSet serversToBalance(ClusterModel clusterModel) { + return clusterModel.servers(); + } + + /** + * Initialize states that this goal requires. E.g. run sanity checks regarding hard goals + * requirements. + */ + protected abstract void initGoalState(ClusterModel clusterModel) + throws RebalanceFailureException; + + /** + * Rebalance the given tabletServers without violating the constraints of the current goal and + * optimized goals. + */ + protected abstract void rebalanceForServer( + ServerModel server, ClusterModel clusterModel, Set optimizedGoals) + throws RebalanceFailureException; + + /** Update goal state after one round of rebalance. */ + protected abstract void updateGoalState(ClusterModel clusterModel) + throws RebalanceFailureException; + + /** + * Check if requirements of this goal are not violated if this action is applied to the given + * cluster state, {@code false} otherwise. + */ + protected abstract boolean selfSatisfied(ClusterModel clusterModel, ReBalancingAction action); + + /** + * Attempt to apply the given balancing action to the given replica in the given cluster. The + * application considers the candidate tabletServers as the potential destination tabletServers + * for replica movement or the location of followers for leadership transfer. If the movement + * attempt succeeds, the function returns the server id of the destination, otherwise the + * function returns null. + */ + protected ServerModel maybeApplyBalancingAction( + ClusterModel clusterModel, + ReplicaModel replica, + Collection candidateServers, + ActionType action, + Set optimizedGoals) { + List eligibleServers = new ArrayList<>(candidateServers); + TableBucket tableBucket = replica.tableBucket(); + for (ServerModel server : eligibleServers) { + ReBalancingAction proposal = + new ReBalancingAction(tableBucket, replica.server().id(), server.id(), action); + // A replica should be moved if: + // 0. The move is legit. + // 1. The goal requirements are not violated if this action is applied to the given + // cluster state. + // 2. The movement is acceptable by the previously optimized goals. + + if (!legitMove(replica, server, clusterModel, action)) { + LOG.trace("Replica move to server is not legit for {}.", proposal); + continue; + } + + if (!selfSatisfied(clusterModel, proposal)) { + LOG.trace("Unable to self-satisfy proposal {}.", proposal); + continue; + } + + ActionAcceptance acceptance = + isProposalAcceptableForOptimizedGoals(optimizedGoals, proposal, clusterModel); + LOG.trace( + "Trying to apply legit and self-satisfied action {}, actionAcceptance = {}", + proposal, + acceptance); + if (acceptance == ACCEPT) { + if (action == LEADERSHIP_MOVEMENT) { + clusterModel.relocateLeadership( + tableBucket, replica.server().id(), server.id()); + } else if (action == REPLICA_MOVEMENT) { + clusterModel.relocateReplica(tableBucket, replica.server().id(), server.id()); + } + return server; + } + } + return null; + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/Goal.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/Goal.java new file mode 100644 index 0000000000..e4ab551c53 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/Goal.java @@ -0,0 +1,119 @@ +/* + * 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.fluss.server.coordinator.rebalance.goal; + +import org.apache.fluss.exception.RebalanceFailureException; +import org.apache.fluss.server.coordinator.rebalance.ActionAcceptance; +import org.apache.fluss.server.coordinator.rebalance.ReBalancingAction; +import org.apache.fluss.server.coordinator.rebalance.model.ClusterModel; +import org.apache.fluss.server.coordinator.rebalance.model.ClusterModelStats; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Serializable; +import java.util.Comparator; +import java.util.Set; + +/** This is the interface of the optimization goals used for rebalance. */ +public interface Goal { + Logger LOG = LoggerFactory.getLogger(Goal.class); + + /** + * Optimize the given cluster model as needed for this goal. + * + *

The method will be given a cluster model. The goal can try to optimize the cluster model + * by performing some admin operations(e.g. move replicas or leader of tableBuckets). + * + *

During the optimization, the implementation should make sure that all the previously + * optimized goals are still satisfied after this method completes its execution. The + * implementation can use {@link #actionAcceptance(ReBalancingAction, ClusterModel)} to check + * whether an admin operation is allowed by a previously optimized goals. + * + *

The implementation of a soft goal should return a boolean indicating whether the goal has + * been met after the optimization or not. + * + *

The implementation of a hard goal should throw an {@link RebalanceFailureException} when + * the goal cannot be met. This will then fail the entire optimization attempt. + */ + void optimize(ClusterModel clusterModel, Set optimizedGoals); + + /** + * Check whether the given action is acceptable by this goal in the given state of the cluster. + * An action is (1) accepted by a goal if it satisfies requirements of the goal, or (2) rejected + * by a goal if it violates its requirements. The return value indicates whether the action is + * accepted or why it is rejected. + */ + ActionAcceptance actionAcceptance(ReBalancingAction action, ClusterModel clusterModel); + + /** + * Get an instance of {@link ClusterModelStatsComparator} for this goal. + * + *

The {@link ClusterModelStatsComparator#compare(ClusterModelStats, ClusterModelStats)} + * method should give a preference between two {@link ClusterModelStats}. + * + *

The returned value must not be null. + * + * @return An instance of {@link ClusterModelStatsComparator} for this goal. + */ + ClusterModelStatsComparator clusterModelStatsComparator(); + + /** + * Signal for finishing the process for rebalance. It is intended to mark the goal optimization + * as finished and perform the memory clean up after the goal optimization. + */ + void finish(); + + /** + * @return {@code true} if this is a hard goal, {@code false} otherwise. + */ + boolean isHardGoal(); + + /** + * @return The name of this goal. Name of a goal provides an identification for the goal in + * human-readable format. + */ + String name(); + + /** + * A comparator that compares two cluster model stats. + * + *

Note: this comparator imposes orderings that are inconsistent with equals. + */ + interface ClusterModelStatsComparator extends Comparator, Serializable { + + /** + * Compare two cluster model stats and determine which stats is preferred. + * + * @param stats1 the first stats + * @param stats2 the second stats + * @return Positive value if stats1 is preferred, 0 if the two stats are equally preferred, + * negative value if stats2 is preferred. + */ + @Override + int compare(ClusterModelStats stats1, ClusterModelStats stats2); + + /** + * This is a method to get the reason for the last comparison. The implementation should at + * least provide a reason when the last comparison returns negative value. + * + * @return A string that explains the result of last comparison. + */ + String explainLastComparison(); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/GoalOptimizer.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/GoalOptimizer.java new file mode 100644 index 0000000000..6a34c0122a --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/GoalOptimizer.java @@ -0,0 +1,80 @@ +/* + * 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.fluss.server.coordinator.rebalance.goal; + +import org.apache.fluss.cluster.rebalance.RebalancePlanForBucket; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.server.coordinator.rebalance.model.ClusterModel; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.apache.fluss.server.coordinator.rebalance.goal.GoalOptimizerUtils.getDiff; +import static org.apache.fluss.server.coordinator.rebalance.goal.GoalOptimizerUtils.hasDiff; + +/** A class for optimizing goals in the given order of priority. */ +public class GoalOptimizer { + private static final Logger LOG = LoggerFactory.getLogger(GoalOptimizer.class); + + public List doOptimizeOnce( + ClusterModel clusterModel, List goalsByPriority) { + LOG.trace("Cluster before optimization is {}", clusterModel); + Map> initReplicaDistribution = + clusterModel.getReplicaDistribution(); + Map initLeaderDistribution = clusterModel.getLeaderDistribution(); + + // Set of balancing proposals that will be applied to the given cluster state to satisfy + // goals (leadership transfer AFTER bucket transfer.) + Set optimizedGoals = new HashSet<>(); + Map> preOptimizedReplicaDistribution = null; + Map preOptimizedLeaderDistribution = null; + for (Goal goal : goalsByPriority) { + preOptimizedReplicaDistribution = + preOptimizedReplicaDistribution == null + ? initReplicaDistribution + : clusterModel.getReplicaDistribution(); + preOptimizedLeaderDistribution = + preOptimizedLeaderDistribution == null + ? initLeaderDistribution + : clusterModel.getLeaderDistribution(); + + // executing the goal optimization. + goal.optimize(clusterModel, optimizedGoals); + optimizedGoals.add(goal); + + boolean hasDiff = + hasDiff( + preOptimizedReplicaDistribution, + preOptimizedLeaderDistribution, + clusterModel); + LOG.info( + "[{}/{}] Generated {} proposals for {}", + optimizedGoals.size(), + goalsByPriority.size(), + hasDiff ? "some" : "no", + goal.name()); + } + + return getDiff(initReplicaDistribution, initLeaderDistribution, clusterModel); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/GoalOptimizerUtils.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/GoalOptimizerUtils.java new file mode 100644 index 0000000000..dadde867b2 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/GoalOptimizerUtils.java @@ -0,0 +1,172 @@ +/* + * 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.fluss.server.coordinator.rebalance.goal; + +import org.apache.fluss.cluster.rebalance.RebalancePlanForBucket; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.server.coordinator.rebalance.ActionAcceptance; +import org.apache.fluss.server.coordinator.rebalance.ReBalancingAction; +import org.apache.fluss.server.coordinator.rebalance.model.BucketModel; +import org.apache.fluss.server.coordinator.rebalance.model.ClusterModel; +import org.apache.fluss.server.coordinator.rebalance.model.ReplicaModel; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import static org.apache.fluss.server.coordinator.rebalance.ActionAcceptance.ACCEPT; +import static org.apache.fluss.utils.Preconditions.checkNotNull; + +/** An util class for {@link GoalOptimizer}. */ +public class GoalOptimizerUtils { + + public static final double EPSILON = 1E-5; + + /** Check whether the given proposal is acceptable for all the given optimized goals. */ + public static ActionAcceptance isProposalAcceptableForOptimizedGoals( + Set optimizedGoals, ReBalancingAction action, ClusterModel cluster) { + for (Goal goal : optimizedGoals) { + ActionAcceptance acceptance = goal.actionAcceptance(action, cluster); + if (acceptance != ACCEPT) { + return acceptance; + } + } + return ACCEPT; + } + + /** + * Compare the given values. + * + *

+     *     1. Return 1 if first
+     *     2. -1 if first
+     *     3. 0 otherwise.
+     * 
+ */ + public static int compare(double d1, double d2, double epsilon) { + if (d2 - d1 > epsilon) { + // Second value is larger than the first value. + return -1; + } + if (d1 - d2 > epsilon) { + // First value is larger than the second value. + return 1; + } + // Given values are approximately equal. + return 0; + } + + /** + * Get whether there is any diff represented by a set of rebalance plan to move from the initial + * to final distribution. + */ + public static boolean hasDiff( + Map> initialReplicaDistribution, + Map initialLeaderDistribution, + ClusterModel optimizedCluster) { + Map> finalReplicaDistribution = + optimizedCluster.getReplicaDistribution(); + sanityCheckReplicaDistribution(initialReplicaDistribution, finalReplicaDistribution); + + boolean hasDiff = false; + for (Map.Entry> entry : initialReplicaDistribution.entrySet()) { + TableBucket tableBucket = entry.getKey(); + List initialReplicas = entry.getValue(); + List finalReplicas = finalReplicaDistribution.get(tableBucket); + + if (!finalReplicas.equals(initialReplicas)) { + hasDiff = true; + break; + } else { + BucketModel bucket = optimizedCluster.bucket(tableBucket); + checkNotNull(bucket, "Bucket is not in the cluster."); + ReplicaModel finalLeaderReplica = bucket.leader(); + checkNotNull(finalLeaderReplica, "Leader replica is not in the bucket."); + Integer finalLeader = finalLeaderReplica.server().id(); + if (!initialLeaderDistribution.get(tableBucket).equals(finalLeader)) { + hasDiff = true; + break; + } + // The bucket has no change. + } + } + return hasDiff; + } + + /** + * Get the diff represented by the set of rebalance plan for bucket to move from initial to + * final distribution. + */ + public static List getDiff( + Map> initialReplicaDistribution, + Map initialLeaderDistribution, + ClusterModel optimizedCluster) { + Map> finalReplicaDistribution = + optimizedCluster.getReplicaDistribution(); + sanityCheckReplicaDistribution(initialReplicaDistribution, finalReplicaDistribution); + + // Generate a set of rebalance plans to represent the diff between initial and final + // distribution. + List diff = new ArrayList<>(); + for (Map.Entry> entry : initialReplicaDistribution.entrySet()) { + TableBucket tableBucket = entry.getKey(); + List initialReplicas = entry.getValue(); + List finalReplicas = finalReplicaDistribution.get(tableBucket); + BucketModel bucket = optimizedCluster.bucket(tableBucket); + checkNotNull(bucket, "Bucket is not in the cluster."); + ReplicaModel finalLeaderReplica = bucket.leader(); + checkNotNull(finalLeaderReplica, "Leader replica is not in the bucket."); + int finalLeader = finalLeaderReplica.server().id(); + // The bucket has no change. + if (finalReplicas.equals(initialReplicas) + && initialLeaderDistribution.get(tableBucket).equals(finalLeader)) { + continue; + } + // We need to adjust the final server list order to ensure the final leader is the first + // replica. + if (finalLeader != finalReplicas.get(0)) { + int leaderPos = finalReplicas.indexOf(finalLeader); + finalReplicas.set(leaderPos, finalReplicas.get(0)); + finalReplicas.set(0, finalLeader); + } + diff.add( + new RebalancePlanForBucket( + tableBucket, + initialLeaderDistribution.get(tableBucket), + finalLeader, + initialReplicas, + finalReplicas)); + } + return diff; + } + + /** + * Sanity check to ensure that initial and final replica distribution have exactly the same + * buckets. + */ + private static void sanityCheckReplicaDistribution( + Map> initialReplicaDistribution, + Map> finalReplicaDistribution) { + // Sanity check to make sure that given distributions contain the same replicas. + if (!initialReplicaDistribution.keySet().equals(finalReplicaDistribution.keySet())) { + throw new IllegalArgumentException( + "Initial and final replica distributions do not contain the same buckets."); + } + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/GoalUtils.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/GoalUtils.java new file mode 100644 index 0000000000..7abc55635e --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/GoalUtils.java @@ -0,0 +1,81 @@ +/* + * 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.fluss.server.coordinator.rebalance.goal; + +import org.apache.fluss.cluster.rebalance.GoalType; +import org.apache.fluss.server.coordinator.rebalance.ActionType; +import org.apache.fluss.server.coordinator.rebalance.model.ClusterModel; +import org.apache.fluss.server.coordinator.rebalance.model.ReplicaModel; +import org.apache.fluss.server.coordinator.rebalance.model.ServerModel; + +import java.util.HashSet; +import java.util.Set; +import java.util.stream.Collectors; + +/** An util class for {@link Goal}. */ +public class GoalUtils { + + public static Goal getGoalByType(GoalType goalType) { + switch (goalType) { + case REPLICA_DISTRIBUTION_GOAL: + return new ReplicaDistributionGoal(); + case LEADER_REPLICA_DISTRIBUTION_GOAL: + return new LeaderReplicaDistributionGoal(); + case PREFERRED_LEADER_GOAL: + return new PreferredLeaderElectionGoal(); + default: + throw new IllegalArgumentException("Unsupported goal type " + goalType); + } + } + + /** + * Check whether the proposed action is legit. An action is legit if it is: + * + *
    + *
  • 1. a replica movement across tabletServers, the dest server does not have a replica of + * the same bucket and is allowed to have a replica from the bucket + *
  • a leadership movement, the replica is a leader and the dest server has a follower of + * the same bucket + *
+ */ + public static boolean legitMove( + ReplicaModel replica, + ServerModel destServer, + ClusterModel cluster, + ActionType actionType) { + switch (actionType) { + case REPLICA_MOVEMENT: + return cluster.bucket(replica.tableBucket()).canAssignReplicaToServer(destServer) + && destServer.replica(replica.tableBucket()) == null; + case LEADERSHIP_MOVEMENT: + return replica.isLeader() && destServer.replica(replica.tableBucket()) != null; + default: + return false; + } + } + + /** + * Retrieve alive servers ids that are not excluded for replica moves. Returns a set to provide + * constant time lookup guaranteed by a HashSet. + */ + public static Set aliveServersNotExcludeForReplicaMove(ClusterModel cluster) { + return cluster.aliveServers().stream() + .map(ServerModel::id) + .collect(Collectors.toCollection(HashSet::new)); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/LeaderReplicaDistributionGoal.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/LeaderReplicaDistributionGoal.java new file mode 100644 index 0000000000..8b5faea263 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/LeaderReplicaDistributionGoal.java @@ -0,0 +1,334 @@ +/* + * 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.fluss.server.coordinator.rebalance.goal; + +import org.apache.fluss.exception.RebalanceFailureException; +import org.apache.fluss.server.coordinator.rebalance.ActionAcceptance; +import org.apache.fluss.server.coordinator.rebalance.ActionType; +import org.apache.fluss.server.coordinator.rebalance.ReBalancingAction; +import org.apache.fluss.server.coordinator.rebalance.model.BucketModel; +import org.apache.fluss.server.coordinator.rebalance.model.ClusterModel; +import org.apache.fluss.server.coordinator.rebalance.model.ClusterModelStats; +import org.apache.fluss.server.coordinator.rebalance.model.ReplicaModel; +import org.apache.fluss.server.coordinator.rebalance.model.ServerModel; +import org.apache.fluss.server.coordinator.rebalance.model.Statistic; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.Comparator; +import java.util.HashSet; +import java.util.List; +import java.util.Objects; +import java.util.PriorityQueue; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; +import java.util.stream.Collectors; + +import static org.apache.fluss.server.coordinator.rebalance.ActionAcceptance.ACCEPT; +import static org.apache.fluss.server.coordinator.rebalance.ActionAcceptance.REPLICA_REJECT; +import static org.apache.fluss.server.coordinator.rebalance.goal.GoalOptimizerUtils.EPSILON; +import static org.apache.fluss.server.coordinator.rebalance.goal.ReplicaDistributionAbstractGoal.ChangeType.ADD; +import static org.apache.fluss.server.coordinator.rebalance.goal.ReplicaDistributionAbstractGoal.ChangeType.REMOVE; +import static org.apache.fluss.utils.Preconditions.checkNotNull; + +/** + * Soft goal to generate leadership movement and leader replica movement proposals to ensure that + * the number of leader replicas on each server is. + * + *
    + *
  • Under: (the average number of leader replicas per server) * (1 + leader replica count + * balance percentage) + *
  • Above: (the average number of leader replicas per server) * Math.max(0, 1 - leader replica + * count balance percentage) + *
+ */ +public class LeaderReplicaDistributionGoal extends ReplicaDistributionAbstractGoal { + + private static final Logger LOG = LoggerFactory.getLogger(LeaderReplicaDistributionGoal.class); + + /** + * The maximum allowed extent of unbalance for leader replica distribution. For example, 1.10 + * means the highest leader replica count of a server should not be 1.10x of average leader + * replica count of all alive tabletServers. + */ + private static final Double LEADER_REPLICA_COUNT_REBALANCE_THRESHOLD = 1.10d; + + @Override + public ActionAcceptance actionAcceptance(ReBalancingAction action, ClusterModel clusterModel) { + ServerModel sourceServer = clusterModel.server(action.getSourceServerId()); + checkNotNull( + sourceServer, "Source server " + action.getSourceServerId() + " is not found."); + ReplicaModel sourceReplica = sourceServer.replica(action.getTableBucket()); + checkNotNull(sourceReplica, "Source replica " + action.getTableBucket() + " is not found."); + ServerModel destServer = clusterModel.server(action.getDestinationServerId()); + switch (action.getActionType()) { + case LEADERSHIP_MOVEMENT: + return isLeaderMovementSatisfiable(sourceServer, destServer); + case REPLICA_MOVEMENT: + if (sourceReplica.isLeader()) { + return isLeaderMovementSatisfiable(sourceServer, destServer); + } + return ACCEPT; + default: + throw new IllegalArgumentException( + "Unsupported action type " + action.getActionType()); + } + } + + @Override + protected void rebalanceForServer( + ServerModel server, ClusterModel clusterModel, Set optimizedGoals) + throws RebalanceFailureException { + LOG.debug( + "Rebalancing server {} [limits] lower: {} upper: {}.", + server.id(), + rebalanceLowerLimit, + rebalanceUpperLimit); + int numLeaderReplicas = server.leaderReplicas().size(); + boolean isExcludedForReplicaMove = isExcludedForReplicaMove(server); + boolean requireLessLeaderReplicas = + numLeaderReplicas > (isExcludedForReplicaMove ? 0 : rebalanceUpperLimit) + || !server.isAlive(); + boolean requireMoreLeaderReplicas = + !isExcludedForReplicaMove + && server.isAlive() + && numLeaderReplicas < rebalanceLowerLimit; + // Update server ids over the balance limit for logging purposes. + if (((requireLessLeaderReplicas + && rebalanceByMovingLeadershipOut(server, clusterModel, optimizedGoals))) + && rebalanceByMovingReplicasOut(server, clusterModel, optimizedGoals)) { + serverIdsAboveRebalanceUpperLimit.add(server.id()); + LOG.debug( + "Failed to sufficiently decrease leader replica count in server {}. Leader replicas: {}.", + server.id(), + server.leaderReplicas().size()); + } else if (requireMoreLeaderReplicas + && rebalanceByMovingLeadershipIn(server, clusterModel, optimizedGoals) + && rebalanceByMovingLeaderReplicasIn(server, clusterModel, optimizedGoals)) { + serverIdsBelowRebalanceLowerLimit.add(server.id()); + LOG.debug( + "Failed to sufficiently increase leader replica count in server {}. Leader replicas: {}.", + server.id(), + server.leaderReplicas().size()); + } + } + + @Override + public ClusterModelStatsComparator clusterModelStatsComparator() { + return new LeaderReplicaDistributionGoalStatsComparator(); + } + + @Override + int numInterestedReplicas(ClusterModel clusterModel) { + return clusterModel.numLeaderReplicas(); + } + + @Override + double balancePercentage() { + return LEADER_REPLICA_COUNT_REBALANCE_THRESHOLD; + } + + private ActionAcceptance isLeaderMovementSatisfiable( + ServerModel sourceServer, ServerModel destServer) { + return (isReplicaCountUnderBalanceUpperLimitAfterChange( + destServer, destServer.leaderReplicas().size(), ADD) + && (isExcludedForReplicaMove(sourceServer) + || isReplicaCountAboveBalanceLowerLimitAfterChange( + sourceServer, + sourceServer.leaderReplicas().size(), + REMOVE))) + ? ACCEPT + : REPLICA_REJECT; + } + + private boolean rebalanceByMovingLeadershipOut( + ServerModel server, ClusterModel cluster, Set optimizedGoals) { + // If the source server is excluded for replica move, set its upper limit to 0. + int balanceUpperLimitForSourceServer = + isExcludedForReplicaMove(server) ? 0 : rebalanceUpperLimit; + int numLeaderReplicas = server.leaderReplicas().size(); + for (ReplicaModel leader : new HashSet<>(server.leaderReplicas())) { + BucketModel bucketModel = cluster.bucket(leader.tableBucket()); + checkNotNull(bucketModel, "Bucket " + leader.tableBucket() + " is not found."); + Set candidateServers = + bucketModel.bucketServers().stream() + .filter(b -> b != server) + .collect(Collectors.toSet()); + ServerModel b = + maybeApplyBalancingAction( + cluster, + leader, + candidateServers, + ActionType.LEADERSHIP_MOVEMENT, + optimizedGoals); + // Only check if we successfully moved something. + if (b != null) { + if (--numLeaderReplicas <= balanceUpperLimitForSourceServer) { + return false; + } + } + } + return true; + } + + private boolean rebalanceByMovingLeadershipIn( + ServerModel server, ClusterModel cluster, Set optimizedGoals) { + int numLeaderReplicas = server.leaderReplicas().size(); + Set candidateServers = Collections.singleton(server); + for (ReplicaModel replica : server.replicas()) { + if (replica.isLeader()) { + continue; + } + + BucketModel bucket = cluster.bucket(replica.tableBucket()); + checkNotNull(bucket, "Bucket " + replica.tableBucket() + " is not found."); + ServerModel b = + maybeApplyBalancingAction( + cluster, + Objects.requireNonNull(bucket.leader()), + candidateServers, + ActionType.LEADERSHIP_MOVEMENT, + optimizedGoals); + // Only check if we successfully moved something. + if (b != null) { + if (++numLeaderReplicas >= rebalanceLowerLimit) { + return false; + } + } + } + return true; + } + + private boolean rebalanceByMovingReplicasOut( + ServerModel server, ClusterModel cluster, Set optimizedGoals) { + // Get the eligible servers. + SortedSet candidateServers; + candidateServers = + new TreeSet<>( + Comparator.comparingInt((ServerModel b) -> b.leaderReplicas().size()) + .thenComparingInt(ServerModel::id)); + candidateServers.addAll( + cluster.aliveServers().stream() + .filter(b -> b.leaderReplicas().size() < rebalanceUpperLimit) + .collect(Collectors.toSet())); + + int balanceUpperLimit = rebalanceUpperLimit; + int numReplicas = server.replicas().size(); + for (ReplicaModel replica : server.replicas()) { + ServerModel b = + maybeApplyBalancingAction( + cluster, + replica, + candidateServers, + ActionType.REPLICA_MOVEMENT, + optimizedGoals); + // Only check if we successfully moved something. + if (b != null) { + if (--numReplicas <= balanceUpperLimit) { + return false; + } + // Remove and reinsert the server so the order is correct. + candidateServers.remove(b); + if (b.leaderReplicas().size() < rebalanceUpperLimit) { + candidateServers.add(b); + } + } + } + return true; + } + + private boolean rebalanceByMovingLeaderReplicasIn( + ServerModel server, ClusterModel clusterModel, Set optimizedGoals) { + PriorityQueue eligibleServers = + new PriorityQueue<>( + (b1, b2) -> { + int result = + Integer.compare( + b2.leaderReplicas().size(), b1.leaderReplicas().size()); + return result == 0 ? Integer.compare(b1.id(), b2.id()) : result; + }); + + for (ServerModel aliveServer : clusterModel.aliveServers()) { + if (aliveServer.leaderReplicas().size() > rebalanceLowerLimit) { + eligibleServers.add(aliveServer); + } + } + List candidateServers = Collections.singletonList(server); + int numLeaderReplicas = server.leaderReplicas().size(); + while (!eligibleServers.isEmpty()) { + ServerModel sourceServer = eligibleServers.poll(); + for (ReplicaModel replica : sourceServer.replicas()) { + ServerModel b = + maybeApplyBalancingAction( + clusterModel, + replica, + candidateServers, + ActionType.REPLICA_MOVEMENT, + optimizedGoals); + // Only need to check status if the action is taken. This will also handle the case + // that the source server has nothing to move in. In that case we will never + // reenqueue that source server. + if (b != null) { + if (++numLeaderReplicas >= rebalanceLowerLimit) { + return false; + } + // If the source server has a lower number of leader replicas than the next + // server in the eligible server queue, we reenqueue the source server and + // switch to the next server. + if (!eligibleServers.isEmpty() + && sourceServer.leaderReplicas().size() + < eligibleServers.peek().leaderReplicas().size()) { + eligibleServers.add(sourceServer); + break; + } + } + } + } + return true; + } + + private class LeaderReplicaDistributionGoalStatsComparator + implements ClusterModelStatsComparator { + private String reasonForLastNegativeResult; + + @Override + public int compare(ClusterModelStats stats1, ClusterModelStats stats2) { + // Standard deviation of number of leader replicas over alive servers in the current + // must be less than the pre-optimized stats. + double stDev1 = stats1.leaderReplicaStats().get(Statistic.ST_DEV).doubleValue(); + double stDev2 = stats2.leaderReplicaStats().get(Statistic.ST_DEV).doubleValue(); + int result = GoalOptimizerUtils.compare(stDev2, stDev1, EPSILON); + if (result < 0) { + reasonForLastNegativeResult = + String.format( + "Violated %s. [Std Deviation of Leader Replica Distribution] post-" + + "optimization:%.3f pre-optimization:%.3f", + name(), stDev1, stDev2); + } + return result; + } + + @Override + public String explainLastComparison() { + return reasonForLastNegativeResult; + } + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/PreferredLeaderElectionGoal.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/PreferredLeaderElectionGoal.java new file mode 100644 index 0000000000..9b6da18939 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/PreferredLeaderElectionGoal.java @@ -0,0 +1,101 @@ +/* + * 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.fluss.server.coordinator.rebalance.goal; + +import org.apache.fluss.server.coordinator.rebalance.ActionAcceptance; +import org.apache.fluss.server.coordinator.rebalance.ReBalancingAction; +import org.apache.fluss.server.coordinator.rebalance.model.BucketModel; +import org.apache.fluss.server.coordinator.rebalance.model.ClusterModel; +import org.apache.fluss.server.coordinator.rebalance.model.ClusterModelStats; +import org.apache.fluss.server.coordinator.rebalance.model.ReplicaModel; +import org.apache.fluss.server.coordinator.rebalance.model.ServerModel; + +import java.util.List; +import java.util.Set; + +import static org.apache.fluss.server.coordinator.rebalance.ActionAcceptance.ACCEPT; +import static org.apache.fluss.utils.Preconditions.checkNotNull; + +/** Soft goal to move the leaders to the first replica of each tableBucket. */ +public class PreferredLeaderElectionGoal implements Goal { + + @Override + public void optimize(ClusterModel clusterModel, Set optimizedGoals) { + for (List buckets : clusterModel.getBucketsByTable().values()) { + for (BucketModel bucket : buckets) { + for (int i = 0; i < bucket.replicas().size(); i++) { + // Only try to transfer the leadership to the first replica of the tabletBucket. + if (i > 0) { + break; + } + ReplicaModel r = bucket.replicas().get(i); + // Iterate over the replicas and ensure that (1) the leader is set to the first + // alive replica, and (2) the leadership is not transferred to a server excluded + // for leadership transfer. + ServerModel leaderCandidate = r.server(); + ReplicaModel originLeader = bucket.leader(); + checkNotNull(originLeader, "Leader replica is null."); + if (leaderCandidate.isAlive()) { + if (!r.isLeader()) { + clusterModel.relocateLeadership( + r.tableBucket(), + originLeader.server().id(), + leaderCandidate.id()); + } + break; + } + } + } + } + } + + @Override + public ActionAcceptance actionAcceptance(ReBalancingAction action, ClusterModel clusterModel) { + return ACCEPT; + } + + @Override + public ClusterModelStatsComparator clusterModelStatsComparator() { + return new ClusterModelStatsComparator() { + @Override + public int compare(ClusterModelStats stats1, ClusterModelStats stats2) { + return 0; + } + + @Override + public String explainLastComparison() { + return String.format("Comparison for the %s is irrelevant.", name()); + } + }; + } + + @Override + public void finish() { + // do nothing. + } + + @Override + public boolean isHardGoal() { + return false; + } + + @Override + public String name() { + return PreferredLeaderElectionGoal.class.getSimpleName(); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/ReplicaDistributionAbstractGoal.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/ReplicaDistributionAbstractGoal.java new file mode 100644 index 0000000000..cbd55305c5 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/ReplicaDistributionAbstractGoal.java @@ -0,0 +1,161 @@ +/* + * 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.fluss.server.coordinator.rebalance.goal; + +import org.apache.fluss.exception.RebalanceFailureException; +import org.apache.fluss.server.coordinator.rebalance.ReBalancingAction; +import org.apache.fluss.server.coordinator.rebalance.model.ClusterModel; +import org.apache.fluss.server.coordinator.rebalance.model.ServerModel; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashSet; +import java.util.Set; + +import static org.apache.fluss.server.coordinator.rebalance.ActionAcceptance.ACCEPT; +import static org.apache.fluss.server.coordinator.rebalance.goal.GoalUtils.aliveServersNotExcludeForReplicaMove; + +/** An abstract class for goals that are based on the distribution of replicas. */ +public abstract class ReplicaDistributionAbstractGoal extends AbstractGoal { + private static final Logger LOG = + LoggerFactory.getLogger(ReplicaDistributionAbstractGoal.class); + private static final double BALANCE_MARGIN = 0.9; + protected final Set serverIdsAboveRebalanceUpperLimit; + protected final Set serverIdsBelowRebalanceLowerLimit; + protected double avgReplicasOnAliveServer; + protected int rebalanceUpperLimit; + protected int rebalanceLowerLimit; + // This is used to identify servers not excluded for replica moves. + protected Set serversAllowedReplicaRemove; + + public ReplicaDistributionAbstractGoal() { + serverIdsAboveRebalanceUpperLimit = new HashSet<>(); + serverIdsBelowRebalanceLowerLimit = new HashSet<>(); + } + + private int rebalanceUpperLimit(double balancePercentage) { + return (int) + Math.ceil( + avgReplicasOnAliveServer + * (1 + adjustedRebalancePercentage(balancePercentage))); + } + + private int rebalanceLowerLimit(double balancePercentage) { + return (int) + Math.floor( + avgReplicasOnAliveServer + * Math.max( + 0, (1 - adjustedRebalancePercentage(balancePercentage)))); + } + + private double adjustedRebalancePercentage(double rebalancePercentage) { + return (rebalancePercentage - 1) * BALANCE_MARGIN; + } + + boolean isReplicaCountUnderBalanceUpperLimitAfterChange( + ServerModel server, int currentReplicaCount, ChangeType changeType) { + int serverBalanceUpperLimit = server.isAlive() ? rebalanceUpperLimit : 0; + + return changeType == ChangeType.ADD + ? currentReplicaCount + 1 <= serverBalanceUpperLimit + : currentReplicaCount - 1 <= serverBalanceUpperLimit; + } + + boolean isReplicaCountAboveBalanceLowerLimitAfterChange( + ServerModel server, int currentReplicaCount, ChangeType changeType) { + int serverBalanceLowerLimit = server.isAlive() ? rebalanceLowerLimit : 0; + + return changeType == ChangeType.ADD + ? currentReplicaCount + 1 >= serverBalanceLowerLimit + : currentReplicaCount - 1 >= serverBalanceLowerLimit; + } + + @Override + public boolean isHardGoal() { + return false; + } + + @Override + protected void initGoalState(ClusterModel clusterModel) throws RebalanceFailureException { + serversAllowedReplicaRemove = aliveServersNotExcludeForReplicaMove(clusterModel); + if (serversAllowedReplicaRemove.isEmpty()) { + throw new RebalanceFailureException( + String.format( + "[%s] All alive tabletServers are excluded from replica moves.", + name())); + } + + // Initialize the average replicas on an alive server. + avgReplicasOnAliveServer = + numInterestedReplicas(clusterModel) / (double) serversAllowedReplicaRemove.size(); + + rebalanceUpperLimit = rebalanceUpperLimit(balancePercentage()); + rebalanceLowerLimit = rebalanceLowerLimit(balancePercentage()); + } + + @Override + protected boolean selfSatisfied(ClusterModel clusterModel, ReBalancingAction action) { + // Check that destination and source would not become unbalanced. + return actionAcceptance(action, clusterModel) == ACCEPT; + } + + @Override + protected void updateGoalState(ClusterModel clusterModel) throws RebalanceFailureException { + if (!serverIdsAboveRebalanceUpperLimit.isEmpty()) { + LOG.debug( + "Replicas count on server ids:{} {} above the balance limit of {} after rebalance.", + serverIdsAboveRebalanceUpperLimit, + (serverIdsAboveRebalanceUpperLimit.size() > 1) ? "are" : "is", + rebalanceUpperLimit); + serverIdsAboveRebalanceUpperLimit.clear(); + succeeded = false; + } + + if (!serverIdsBelowRebalanceLowerLimit.isEmpty()) { + LOG.debug( + "Replicas count on server ids:{} {} below the balance limit of {} after rebalance.", + serverIdsBelowRebalanceLowerLimit, + (serverIdsBelowRebalanceLowerLimit.size() > 1) ? "are" : "is", + rebalanceLowerLimit); + serverIdsBelowRebalanceLowerLimit.clear(); + succeeded = false; + } + + // TODO maybe need check offline server. + + finish(); + } + + abstract int numInterestedReplicas(ClusterModel clusterModel); + + /** + * @return The requested balance threshold. + */ + abstract double balancePercentage(); + + protected boolean isExcludedForReplicaMove(ServerModel server) { + return !serversAllowedReplicaRemove.contains(server.id()); + } + + /** Whether bring replica in or out. */ + protected enum ChangeType { + ADD, + REMOVE + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/ReplicaDistributionGoal.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/ReplicaDistributionGoal.java new file mode 100644 index 0000000000..b70c5ca622 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/ReplicaDistributionGoal.java @@ -0,0 +1,292 @@ +/* + * 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.fluss.server.coordinator.rebalance.goal; + +import org.apache.fluss.exception.RebalanceFailureException; +import org.apache.fluss.server.coordinator.rebalance.ActionAcceptance; +import org.apache.fluss.server.coordinator.rebalance.ActionType; +import org.apache.fluss.server.coordinator.rebalance.ReBalancingAction; +import org.apache.fluss.server.coordinator.rebalance.model.ClusterModel; +import org.apache.fluss.server.coordinator.rebalance.model.ClusterModelStats; +import org.apache.fluss.server.coordinator.rebalance.model.ReplicaModel; +import org.apache.fluss.server.coordinator.rebalance.model.ServerModel; +import org.apache.fluss.server.coordinator.rebalance.model.Statistic; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.PriorityQueue; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; +import java.util.stream.Collectors; + +import static org.apache.fluss.server.coordinator.rebalance.ActionAcceptance.ACCEPT; +import static org.apache.fluss.server.coordinator.rebalance.ActionAcceptance.REPLICA_REJECT; +import static org.apache.fluss.server.coordinator.rebalance.goal.GoalOptimizerUtils.EPSILON; +import static org.apache.fluss.server.coordinator.rebalance.goal.ReplicaDistributionAbstractGoal.ChangeType.ADD; +import static org.apache.fluss.server.coordinator.rebalance.goal.ReplicaDistributionAbstractGoal.ChangeType.REMOVE; +import static org.apache.fluss.utils.Preconditions.checkNotNull; + +/** + * Soft goal to generate replica movement proposals to ensure that the number of replicas on each + * server is. + * + *
    + *
  • Under: (the average number of replicas per server) * (1 + replica count balance percentage) + *
  • Above: (the average number of replicas per server) * Math.max(0, 1 - replica count balance + * percentage) + *
+ */ +public class ReplicaDistributionGoal extends ReplicaDistributionAbstractGoal { + + private static final Logger LOG = LoggerFactory.getLogger(ReplicaDistributionGoal.class); + + // TODO configurable. + /** + * The maximum allowed extent of unbalance for replica leader replica distribution. For example, + * 1.10 means the highest leader replica count of a server should not be 1.10x of average leader + * replica count of all alive tabletServers. + */ + private static final Double REPLICA_COUNT_REBALANCE_THRESHOLD = 1.10d; + + @Override + public ActionAcceptance actionAcceptance(ReBalancingAction action, ClusterModel clusterModel) { + switch (action.getActionType()) { + case LEADERSHIP_MOVEMENT: + return ACCEPT; + case REPLICA_MOVEMENT: + ServerModel sourceServer = clusterModel.server(action.getSourceServerId()); + ServerModel destServer = clusterModel.server(action.getDestinationServerId()); + + checkNotNull( + sourceServer, + "Source server " + action.getSourceServerId() + " is not found."); + checkNotNull( + destServer, + "Destination server " + action.getDestinationServerId() + " is not found."); + + // Check that destination and source would not become unbalanced. + return (isReplicaCountUnderBalanceUpperLimitAfterChange( + destServer, destServer.replicas().size(), ADD)) + && (isExcludedForReplicaMove(sourceServer) + || isReplicaCountAboveBalanceLowerLimitAfterChange( + sourceServer, + sourceServer.replicas().size(), + REMOVE)) + ? ACCEPT + : REPLICA_REJECT; + default: + throw new IllegalArgumentException( + "Unsupported balancing action " + action.getActionType() + " is provided."); + } + } + + @Override + protected void rebalanceForServer( + ServerModel server, ClusterModel clusterModel, Set optimizedGoals) + throws RebalanceFailureException { + LOG.debug( + "Rebalancing server {} [limits] lower: {} upper: {}.", + server.id(), + rebalanceLowerLimit, + rebalanceUpperLimit); + int numReplicas = server.replicas().size(); + boolean isExcludeForReplicaMove = isExcludedForReplicaMove(server); + + boolean requireLessReplicas = + numReplicas > rebalanceUpperLimit || isExcludeForReplicaMove || !server.isAlive(); + boolean requireMoreReplicas = + !isExcludeForReplicaMove && server.isAlive() && numReplicas < rebalanceLowerLimit; + if (!requireMoreReplicas && !requireLessReplicas) { + // return if the server is already within the limit. + return; + } + + if (requireLessReplicas + && rebalanceByMovingReplicasOut(server, clusterModel, optimizedGoals)) { + serverIdsAboveRebalanceUpperLimit.add(server.id()); + LOG.debug( + "Failed to sufficiently decrease replica count in server {} with replica movements. " + + "Replicas number after remove: {}.", + server.id(), + server.replicas().size()); + } + + if (requireMoreReplicas + && rebalanceByMovingReplicasIn(server, clusterModel, optimizedGoals)) { + serverIdsBelowRebalanceLowerLimit.add(server.id()); + LOG.debug( + "Failed to sufficiently increase replica count in server {} with replica movements. " + + "Replicas number after remove: {}.", + server.id(), + server.replicas().size()); + } + + if (!serverIdsAboveRebalanceUpperLimit.contains(server.id()) + && !serverIdsBelowRebalanceLowerLimit.contains(server.id())) { + LOG.debug( + "Successfully balanced replica count for server {} by moving replicas. " + + "Replicas number after remove: {}", + server.id(), + server.replicas().size()); + } + } + + @Override + public ClusterModelStatsComparator clusterModelStatsComparator() { + return new ReplicaDistributionGoalStatsComparator(); + } + + @Override + int numInterestedReplicas(ClusterModel clusterModel) { + return clusterModel.numReplicas(); + } + + @Override + double balancePercentage() { + return REPLICA_COUNT_REBALANCE_THRESHOLD; + } + + private boolean rebalanceByMovingReplicasOut( + ServerModel server, ClusterModel cluster, Set optimizedGoals) { + SortedSet candidateServers = + new TreeSet<>( + Comparator.comparingInt((ServerModel b) -> b.replicas().size()) + .thenComparingInt(ServerModel::id)); + + candidateServers.addAll( + cluster.aliveServers().stream() + .filter(b -> b.replicas().size() < rebalanceUpperLimit) + .collect(Collectors.toSet())); + int balanceUpperLimitForSourceServer = + isExcludedForReplicaMove(server) ? 0 : rebalanceUpperLimit; + + // Now let's do the replica out operation. + // TODO maybe use a sorted replicas set + for (ReplicaModel replica : server.replicas()) { + ServerModel b = + maybeApplyBalancingAction( + cluster, + replica, + candidateServers, + ActionType.REPLICA_MOVEMENT, + optimizedGoals); + // Only check if we successfully moved something. + if (b != null) { + if (server.replicas().size() <= balanceUpperLimitForSourceServer) { + return false; + } + + // Remove and reinsert the server so the order is correct. + candidateServers.remove(b); + if (b.replicas().size() < rebalanceUpperLimit) { + candidateServers.add(b); + } + } + } + + return !server.replicas().isEmpty(); + } + + private boolean rebalanceByMovingReplicasIn( + ServerModel aliveDestServer, ClusterModel cluster, Set optimizedGoals) { + PriorityQueue eligibleServers = + new PriorityQueue<>( + (b1, b2) -> { + // Servers are sorted by (1) all replica count then (2) server id. + int resultByAllReplicas = + Integer.compare(b2.replicas().size(), b1.replicas().size()); + return resultByAllReplicas == 0 + ? Integer.compare(b1.id(), b2.id()) + : resultByAllReplicas; + }); + + // Source server can be offline, alive. + for (ServerModel sourceServer : cluster.servers()) { + if (sourceServer.replicas().size() > rebalanceLowerLimit + || isExcludedForReplicaMove(sourceServer)) { + eligibleServers.add(sourceServer); + } + } + + List candidateServers = Collections.singletonList(aliveDestServer); + while (!eligibleServers.isEmpty()) { + ServerModel sourceServer = eligibleServers.poll(); + // TODO maybe use a sorted replicas set + for (ReplicaModel replica : sourceServer.replicas()) { + ServerModel b = + maybeApplyBalancingAction( + cluster, + replica, + candidateServers, + ActionType.REPLICA_MOVEMENT, + optimizedGoals); + // Only need to check status if the action is taken. This will also handle the case + // that the source server has nothing to move in. In that case we will never + // re-enqueue that source server. + if (b != null) { + if (aliveDestServer.replicas().size() >= rebalanceLowerLimit) { + // Note that the server passed to this method is always alive; hence, there + // is no need to check if it is dead. + return false; + } + + if (!eligibleServers.isEmpty()) { + if (sourceServer.replicas().size() + < eligibleServers.peek().replicas().size()) { + eligibleServers.add(sourceServer); + break; + } + } + } + } + } + return true; + } + + private class ReplicaDistributionGoalStatsComparator implements ClusterModelStatsComparator { + private String reasonForLastNegativeResult; + + @Override + public int compare(ClusterModelStats stats1, ClusterModelStats stats2) { + // Standard deviation of number of replicas over servers not excluded for replica moves + // must be less than the + // pre-optimized stats. + double stDev1 = stats1.replicaStats().get(Statistic.ST_DEV).doubleValue(); + double stDev2 = stats2.replicaStats().get(Statistic.ST_DEV).doubleValue(); + int result = GoalOptimizerUtils.compare(stDev2, stDev1, EPSILON); + if (result < 0) { + reasonForLastNegativeResult = + String.format( + "Violated %s. [Std Deviation of Replica Distribution] post-" + + "optimization:%.3f pre-optimization:%.3f", + name(), stDev1, stDev2); + } + return result; + } + + @Override + public String explainLastComparison() { + return reasonForLastNegativeResult; + } + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/model/BucketModel.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/model/BucketModel.java new file mode 100644 index 0000000000..9aff7b0b80 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/model/BucketModel.java @@ -0,0 +1,131 @@ +/* + * 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.fluss.server.coordinator.rebalance.model; + +import org.apache.fluss.metadata.TableBucket; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** A class that holds the information of the {@link TableBucket} for rebalance. */ +public class BucketModel { + private final TableBucket tableBucket; + private final List replicas; + private @Nullable ReplicaModel leader; + // Set of server which are unable to host replica of this replica (such as: the server are + // offline). + private final Set ineligibleServers; + + public BucketModel(TableBucket tableBucket, Set ineligibleServers) { + this.tableBucket = tableBucket; + this.replicas = new ArrayList<>(); + this.leader = null; + this.ineligibleServers = ineligibleServers; + } + + public TableBucket tableBucket() { + return tableBucket; + } + + public @Nullable ReplicaModel leader() { + return leader; + } + + public List replicas() { + return replicas; + } + + public Set bucketServers() { + Set bucketServers = new HashSet<>(); + replicas.forEach(replica -> bucketServers.add(replica.server())); + return bucketServers; + } + + public boolean canAssignReplicaToServer(ServerModel candidateServer) { + return !ineligibleServers.contains(candidateServer); + } + + public ReplicaModel replica(long serverId) { + for (ReplicaModel replica : replicas) { + if (replica.server().id() == serverId) { + return replica; + } + } + + throw new IllegalArgumentException( + "Requested replica " + serverId + " is not a replica of bucket " + tableBucket); + } + + public void addLeader(ReplicaModel leader, int index) { + if (this.leader != null) { + throw new IllegalArgumentException( + String.format( + "Bucket %s already has a leader replica %s. Cannot add a new leader replica %s.", + tableBucket, this.leader, leader)); + } + + if (!leader.isLeader()) { + throw new IllegalArgumentException( + String.format( + "Inconsistent leadership information. Trying to set %s as the leader for bucket %s while " + + "the replica is not marked as a leader", + leader, tableBucket)); + } + + this.leader = leader; + replicas.add(index, leader); + } + + public void addFollower(ReplicaModel follower, int index) { + if (follower.isLeader()) { + throw new IllegalArgumentException( + String.format( + "Inconsistent leadership information. Trying to set %s as the follower for bucket %s while " + + "the replica is marked as a leader", + follower, tableBucket)); + } + + if (!follower.tableBucket().equals(this.tableBucket)) { + throw new IllegalArgumentException( + String.format( + "Inconsistent table bucket. Trying to add follower replica %s to tableBucket %s", + follower, tableBucket)); + } + + // Add follower to list of followers + replicas.add(index, follower); + } + + void relocateLeadership(ReplicaModel prospectiveLeader) { + int leaderPos = replicas.indexOf(prospectiveLeader); + swapReplicaPositions(0, leaderPos); + leader = prospectiveLeader; + } + + private void swapReplicaPositions(int index1, int index2) { + ReplicaModel replica1 = replicas.get(index1); + ReplicaModel replica2 = replicas.get(index2); + + replicas.set(index2, replica1); + replicas.set(index1, replica2); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/model/ClusterModel.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/model/ClusterModel.java new file mode 100644 index 0000000000..8b27452e04 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/model/ClusterModel.java @@ -0,0 +1,276 @@ +/* + * 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.fluss.server.coordinator.rebalance.model; + +import org.apache.fluss.metadata.TableBucket; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.SortedMap; +import java.util.SortedSet; +import java.util.TreeMap; +import java.util.TreeSet; +import java.util.stream.Collectors; + +/** + * A class that holds the information of the cluster for rebalance.The information including live + * tabletServers, bucket distribution, tabletServer tag etc. + * + *

Currently, the clusterModel can only be created by a rebalance request. It's used as the input + * of the GoalOptimizer to generate the rebalance plan for load rebalance. + */ +public class ClusterModel { + // TODO ClusterModel can be implemented in incremental mode, dynamically modified when there are + // events such as table create, table delete, server offline, etc. Currently designed to read + // coordinatorContext and generate it directly + + private final Map racksById; + private final Map serverIdToRack; + private final Set aliveServers; + private final SortedSet offlineServers; + private final SortedSet servers; + private final Map bucketsByTableBucket; + + public ClusterModel(SortedSet servers) { + this.servers = servers; + this.bucketsByTableBucket = new HashMap<>(); + + this.aliveServers = new HashSet<>(); + this.offlineServers = new TreeSet<>(); + for (ServerModel serverModel : servers) { + if (serverModel.isAlive()) { + aliveServers.add(serverModel); + } else { + offlineServers.add(serverModel); + } + } + + this.racksById = new HashMap<>(); + this.serverIdToRack = new HashMap<>(); + for (ServerModel serverModel : servers) { + RackModel rackModel = racksById.computeIfAbsent(serverModel.rack(), RackModel::new); + rackModel.addServer(serverModel); + serverIdToRack.put(serverModel.id(), rackModel); + } + } + + public SortedSet offlineServers() { + return offlineServers; + } + + public SortedSet servers() { + return servers; + } + + public Set aliveServers() { + return Collections.unmodifiableSet(aliveServers); + } + + public @Nullable BucketModel bucket(TableBucket tableBucket) { + return bucketsByTableBucket.get(tableBucket); + } + + public RackModel rack(String rack) { + return racksById.get(rack); + } + + public @Nullable ServerModel server(int serverId) { + RackModel rack = serverIdToRack.get(serverId); + return rack == null ? null : rack.server(serverId); + } + + /** Populate the analysis stats with this cluster. */ + public ClusterModelStats getClusterStats() { + return (new ClusterModelStats()).populate(this); + } + + public int numReplicas() { + return bucketsByTableBucket.values().stream().mapToInt(p -> p.replicas().size()).sum(); + } + + public int numLeaderReplicas() { + return bucketsByTableBucket.size(); + } + + public SortedMap> getBucketsByTable() { + SortedMap> bucketsByTable = new TreeMap<>(); + for (Long tableId : tables()) { + bucketsByTable.put(tableId, new ArrayList<>()); + } + for (Map.Entry entry : bucketsByTableBucket.entrySet()) { + bucketsByTable.get(entry.getKey().getTableId()).add(entry.getValue()); + } + return bucketsByTable; + } + + public Set tables() { + Set tables = new HashSet<>(); + + for (RackModel rack : racksById.values()) { + tables.addAll(rack.tables()); + } + return tables; + } + + /** + * Get the distribution of replicas in the cluster at the point of call. + * + * @return A map from tableBucket to the list of replicas. the first element is the leader, the + * rest are followers. + */ + public Map> getReplicaDistribution() { + Map> replicaDistribution = new HashMap<>(); + for (Map.Entry entry : bucketsByTableBucket.entrySet()) { + TableBucket tableBucket = entry.getKey(); + BucketModel bucket = entry.getValue(); + List replicaIds = + bucket.replicas().stream() + .map(r -> r.server().id()) + .collect(Collectors.toList()); + replicaDistribution.put(tableBucket, replicaIds); + } + return replicaDistribution; + } + + public Map getLeaderDistribution() { + Map leaderDistribution = new HashMap<>(); + for (Map.Entry entry : bucketsByTableBucket.entrySet()) { + TableBucket tableBucket = entry.getKey(); + BucketModel bucket = entry.getValue(); + leaderDistribution.put(tableBucket, bucket.leader().server().id()); + } + return leaderDistribution; + } + + public void createReplica(int serverId, TableBucket tableBucket, int index, boolean isLeader) { + ServerModel server = server(serverId); + if (server == null) { + throw new IllegalArgumentException("Server is not in the cluster."); + } + + ReplicaModel replica = new ReplicaModel(tableBucket, server, isLeader); + server.putReplica(tableBucket, replica); + + if (!bucketsByTableBucket.containsKey(tableBucket)) { + bucketsByTableBucket.put(tableBucket, new BucketModel(tableBucket, offlineServers())); + } + + BucketModel bucket = bucketsByTableBucket.get(tableBucket); + if (isLeader) { + bucket.addLeader(replica, index); + } else { + bucket.addFollower(replica, index); + } + } + + /** + * Relocate leadership from source server to destination server. + * + *

    + *
  • 1. Removes leadership from source replica. + *
  • 2. Adds this leadership to the destination replica. + *
  • 3. Updates the leader and list of followers of the bucket. + *
+ */ + public boolean relocateLeadership( + TableBucket tableBucket, int sourceServerId, int desServerId) { + // Sanity check to see if the source replica is the leader. + BucketModel bucket = bucketsByTableBucket.get(tableBucket); + ReplicaModel sourceReplica = bucket.replica(sourceServerId); + if (!sourceReplica.isLeader()) { + return false; + } + + // Sanity check to see if the destination replica is a follower. + ReplicaModel desReplica = bucket.replica(desServerId); + if (desReplica.isLeader()) { + throw new IllegalArgumentException( + "Cannot relocate leadership of bucket " + + tableBucket + + " from server " + + sourceServerId + + " to server " + + desServerId + + " because the destination replica is a leader."); + } + + ServerModel sourceServer = server(sourceServerId); + if (sourceServer == null) { + throw new IllegalArgumentException("Source server is not in the cluster."); + } + sourceServer.makeFollower(tableBucket); + + ServerModel destServer = server(desServerId); + if (destServer == null) { + throw new IllegalArgumentException("Destination server is not in the cluster."); + } + destServer.makeLeader(tableBucket); + + // Update the leader and list of followers of the bucket. + bucket.relocateLeadership(desReplica); + return true; + } + + /** + * Relocate replica from source server to destination server. + * + *
    + *
  • 1. Removes the replica from source server. + *
  • 2. Set the server of the removed replica as the dest server + *
  • 3. Add this replica to the dest server. + *
+ */ + public void relocateReplica(TableBucket tableBucket, int sourceServerId, int destServerId) { + // Removes the replica from the source server. + ReplicaModel replica = removeReplica(sourceServerId, tableBucket); + if (replica == null) { + throw new IllegalArgumentException("Replica is not in the cluster."); + } + + // Updates the tabletServer of the removed replicas with dest server. + replica.setServer(server(destServerId)); + + // Add this replica back to destination rack and server. + String rack = replica.server().rack(); + rack(rack).addReplica(replica); + } + + private @Nullable ReplicaModel removeReplica(int serverId, TableBucket tableBucket) { + for (RackModel rack : racksById.values()) { + ReplicaModel removedReplica = rack.removeReplica(serverId, tableBucket); + if (removedReplica != null) { + return removedReplica; + } + } + return null; + } + + @Override + public String toString() { + return String.format( + "ClusterModel[serverCount=%s,bucketCount=%s,aliveServerCount=%s]", + servers.size(), bucketsByTableBucket.size(), aliveServers.size()); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/model/ClusterModelStats.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/model/ClusterModelStats.java new file mode 100644 index 0000000000..16bd9f29b0 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/model/ClusterModelStats.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.coordinator.rebalance.model; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.SortedSet; +import java.util.function.Function; + +/** A class that holds the statistics of the cluster for rebalance. */ +public class ClusterModelStats { + private final Map replicaStats; + private final Map leaderReplicaStats; + private int numServers; + private int numReplicasInCluster; + + public ClusterModelStats() { + replicaStats = new HashMap<>(); + leaderReplicaStats = new HashMap<>(); + + numServers = 0; + numReplicasInCluster = 0; + } + + ClusterModelStats populate(ClusterModel clusterModel) { + final SortedSet servers = clusterModel.servers(); + final Set aliveServers = clusterModel.aliveServers(); + this.numServers = servers.size(); + numForReplicas(clusterModel, servers, aliveServers); + numForLeaderReplicas(servers, aliveServers); + return this; + } + + /** Generate statistics for replicas in the given cluster. */ + private void numForReplicas( + ClusterModel clusterModel, + SortedSet servers, + Set aliveServers) { + populateReplicaStats( + serverModel -> serverModel.replicas().size(), replicaStats, servers, aliveServers); + numReplicasInCluster = clusterModel.numReplicas(); + } + + /** Generate statistics for leader replicas in the given cluster. */ + private void numForLeaderReplicas( + SortedSet servers, Set aliveServers) { + populateReplicaStats( + serverModel -> serverModel.leaderReplicas().size(), + leaderReplicaStats, + servers, + aliveServers); + } + + private void populateReplicaStats( + Function numInterestedReplicasFunc, + Map interestedReplicaStats, + SortedSet servers, + Set aliveServers) { + // Average, minimum, and maximum number of replicas of interest in servers. + int maxInterestedReplicasInServer = 0; + int minInterestedReplicasInServer = Integer.MAX_VALUE; + int numInterestedReplicasInCluster = 0; + for (ServerModel server : servers) { + int numInterestedReplicasInServer = numInterestedReplicasFunc.apply(server); + numInterestedReplicasInCluster += numInterestedReplicasInServer; + maxInterestedReplicasInServer = + Math.max(maxInterestedReplicasInServer, numInterestedReplicasInServer); + minInterestedReplicasInServer = + Math.min(minInterestedReplicasInServer, numInterestedReplicasInServer); + } + double avgInterestedReplicas = + ((double) numInterestedReplicasInCluster) / aliveServers.size(); + + // Standard deviation of replicas of interest in alive servers. + double variance = 0.0; + for (ServerModel broker : aliveServers) { + variance += + (Math.pow( + (double) numInterestedReplicasFunc.apply(broker) + - avgInterestedReplicas, + 2) + / aliveServers.size()); + } + + interestedReplicaStats.put(Statistic.AVG, avgInterestedReplicas); + interestedReplicaStats.put(Statistic.MAX, maxInterestedReplicasInServer); + interestedReplicaStats.put(Statistic.MIN, minInterestedReplicasInServer); + interestedReplicaStats.put(Statistic.ST_DEV, Math.sqrt(variance)); + } + + public Map replicaStats() { + return Collections.unmodifiableMap(replicaStats); + } + + public Map leaderReplicaStats() { + return Collections.unmodifiableMap(leaderReplicaStats); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/model/RackModel.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/model/RackModel.java new file mode 100644 index 0000000000..fdf9cbad80 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/model/RackModel.java @@ -0,0 +1,84 @@ +/* + * 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.fluss.server.coordinator.rebalance.model; + +import org.apache.fluss.metadata.TableBucket; + +import javax.annotation.Nullable; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +/** + * A class that holds the information of the rack, including its liveness tabletServers and + * replicas. A rack object is created as part of a cluster structure. + */ +public class RackModel { + public static final String DEFAULT_RACK = "default_rack"; + + private final String rack; + private final Map servers; + + public RackModel(String rack) { + this.rack = rack; + this.servers = new HashMap<>(); + } + + @Nullable + ReplicaModel removeReplica(int serverId, TableBucket tableBucket) { + ServerModel server = servers.get(serverId); + if (server != null) { + return server.removeReplica(tableBucket); + } + + return null; + } + + void addReplica(ReplicaModel replica) { + replica.server().putReplica(replica.tableBucket(), replica); + } + + public String rack() { + return rack; + } + + @Nullable + ServerModel server(int serverId) { + return servers.get(serverId); + } + + public void addServer(ServerModel server) { + servers.put(server.id(), server); + } + + public Set tables() { + Set tables = new HashSet<>(); + + for (ServerModel server : servers.values()) { + tables.addAll(server.tables()); + } + return tables; + } + + @Override + public String toString() { + return String.format("RackModel[rack=%s,servers=%s]", rack, servers.size()); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/model/ReplicaModel.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/model/ReplicaModel.java new file mode 100644 index 0000000000..e67d9bd733 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/model/ReplicaModel.java @@ -0,0 +1,99 @@ +/* + * 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.fluss.server.coordinator.rebalance.model; + +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.server.replica.Replica; + +import java.util.Objects; + +/** A class that holds the information of the {@link Replica} for rebalance. */ +public class ReplicaModel { + private final TableBucket tableBucket; + private final ServerModel originalServer; + private ServerModel server; + private boolean isLeader; + + public ReplicaModel(TableBucket tableBucket, ServerModel server, boolean isLeader) { + this.tableBucket = tableBucket; + this.server = server; + this.isLeader = isLeader; + this.originalServer = server; + } + + public TableBucket tableBucket() { + return tableBucket; + } + + public ServerModel originalServer() { + return originalServer; + } + + public ServerModel server() { + return server; + } + + public int serverId() { + return server.id(); + } + + public boolean isLeader() { + return isLeader; + } + + public void makeFollower() { + setLeadership(false); + } + + public void makeLeader() { + setLeadership(true); + } + + void setLeadership(boolean leader) { + isLeader = leader; + } + + public void setServer(ServerModel server) { + this.server = server; + } + + @Override + public String toString() { + return String.format( + "ReplicaModel[TableBucket=%s,isLeader=%s,rack=%s,server=%s,originalServer=%s]", + tableBucket, isLeader, server.rack(), server.id(), originalServer.id()); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ReplicaModel that = (ReplicaModel) o; + return Objects.equals(tableBucket, that.tableBucket) + && originalServer.id() == that.originalServer.id(); + } + + @Override + public int hashCode() { + return Objects.hash(tableBucket, originalServer.id()); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/model/ServerModel.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/model/ServerModel.java new file mode 100644 index 0000000000..a57bc85b30 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/model/ServerModel.java @@ -0,0 +1,200 @@ +/* + * 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.fluss.server.coordinator.rebalance.model; + +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TablePartition; + +import javax.annotation.Nullable; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +/** A class that holds the information of the tabletServer for rebalance. */ +public class ServerModel implements Comparable { + + private final int serverId; + private final boolean isAlive; + private final String rack; + private final Set replicas; + private final Set leaderReplicas; + /** A map for tracking (tableId) -> (BucketId -> replica) for none-partitioned table. */ + private final Map> tableReplicas; + + /** A map for tracking (tableId, partitionId) -> (BucketId -> replica) for partitioned table. */ + private final Map> tablePartitionReplicas; + + public ServerModel(int serverId, String rack, boolean isAlive) { + this.serverId = serverId; + this.rack = rack; + this.isAlive = isAlive; + this.replicas = new HashSet<>(); + this.leaderReplicas = new HashSet<>(); + this.tableReplicas = new HashMap<>(); + this.tablePartitionReplicas = new HashMap<>(); + } + + public int id() { + return serverId; + } + + public String rack() { + return rack; + } + + public boolean isAlive() { + return isAlive; + } + + public Set replicas() { + return new HashSet<>(replicas); + } + + public Set leaderReplicas() { + return new HashSet<>(leaderReplicas); + } + + public Set tables() { + Set tables = new HashSet<>(tableReplicas.keySet()); + tablePartitionReplicas.keySet().forEach(t -> tables.add(t.getTableId())); + return tables; + } + + public void makeFollower(TableBucket tableBucket) { + ReplicaModel replica = replica(tableBucket); + if (replica != null) { + replica.makeFollower(); + leaderReplicas.remove(replica); + } + } + + public void makeLeader(TableBucket tableBucket) { + ReplicaModel replica = replica(tableBucket); + if (replica != null) { + replica.makeLeader(); + leaderReplicas.add(replica); + } + } + + public void putReplica(TableBucket tableBucket, ReplicaModel replica) { + replicas.add(replica); + replica.setServer(this); + if (tableBucket.getPartitionId() != null) { + TablePartition tablePartition = + new TablePartition(tableBucket.getTableId(), tableBucket.getPartitionId()); + tablePartitionReplicas + .computeIfAbsent(tablePartition, k -> new HashMap<>()) + .put(tableBucket.getBucket(), replica); + } else { + tableReplicas + .computeIfAbsent(tableBucket.getTableId(), k -> new HashMap<>()) + .put(tableBucket.getBucket(), replica); + } + + if (replica.isLeader()) { + leaderReplicas.add(replica); + } + } + + public @Nullable ReplicaModel replica(TableBucket tableBucket) { + if (tableBucket.getPartitionId() == null) { + Map replicas = tableReplicas.get(tableBucket.getTableId()); + if (replicas == null) { + return null; + } + + return replicas.get(tableBucket.getBucket()); + } else { + TablePartition tablePartition = + new TablePartition(tableBucket.getTableId(), tableBucket.getPartitionId()); + Map replicas = tablePartitionReplicas.get(tablePartition); + if (replicas == null) { + return null; + } + return replicas.get(tableBucket.getBucket()); + } + } + + public @Nullable ReplicaModel removeReplica(TableBucket tableBucket) { + ReplicaModel removedReplica = replica(tableBucket); + if (removedReplica != null) { + replicas.remove(removedReplica); + + if (tableBucket.getPartitionId() != null) { + TablePartition tablePartition = + new TablePartition(tableBucket.getTableId(), tableBucket.getPartitionId()); + Map tablePartitionReplicas = + this.tablePartitionReplicas.get(tablePartition); + if (tablePartitionReplicas != null) { + tablePartitionReplicas.remove(tableBucket.getBucket()); + + if (tablePartitionReplicas.isEmpty()) { + this.tablePartitionReplicas.remove(tablePartition); + } + } + } else { + Map tableReplicas = + this.tableReplicas.get(tableBucket.getTableId()); + if (tableReplicas != null) { + tableReplicas.remove(tableBucket.getBucket()); + + if (tableReplicas.isEmpty()) { + this.tableReplicas.remove(tableBucket.getTableId()); + } + } + } + + if (removedReplica.isLeader()) { + leaderReplicas.remove(removedReplica); + } + } + + return removedReplica; + } + + @Override + public int compareTo(ServerModel o) { + return Integer.compare(serverId, o.id()); + } + + @Override + public String toString() { + return String.format( + "ServerModel[id=%s,rack=%s,isAlive=%s,replicaCount=%s]", + serverId, rack, isAlive, replicas.size()); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ServerModel that = (ServerModel) o; + return serverId == that.serverId; + } + + @Override + public int hashCode() { + return serverId; + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaLeaderElectionStrategy.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/model/Statistic.java similarity index 79% rename from fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaLeaderElectionStrategy.java rename to fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/model/Statistic.java index faff47a42d..bf12b8b281 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaLeaderElectionStrategy.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/model/Statistic.java @@ -15,10 +15,12 @@ * limitations under the License. */ -package org.apache.fluss.server.coordinator.statemachine; +package org.apache.fluss.server.coordinator.rebalance.model; -/** The strategies to elect the replica leader. */ -public enum ReplicaLeaderElectionStrategy { - DEFAULT_ELECTION, - CONTROLLED_SHUTDOWN_ELECTION +/** An enum for the statistic. */ +public enum Statistic { + AVG, + MAX, + MIN, + ST_DEV } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaLeaderElection.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaLeaderElection.java new file mode 100644 index 0000000000..492f9ae0b2 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaLeaderElection.java @@ -0,0 +1,124 @@ +/* + * 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.fluss.server.coordinator.statemachine; + +import org.apache.fluss.server.coordinator.statemachine.TableBucketStateMachine.ElectionResult; +import org.apache.fluss.server.zk.data.LeaderAndIsr; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +/** The strategies to elect the replica leader. */ +public abstract class ReplicaLeaderElection { + + /** The default replica leader election. */ + public static class DefaultLeaderElection extends ReplicaLeaderElection { + /** + * Default replica leader election, like electing leader while leader offline. + * + * @param assignments the assignments + * @param aliveReplicas the alive replicas + * @param leaderAndIsr the original leaderAndIsr + * @return the election result + */ + public Optional leaderElection( + List assignments, List aliveReplicas, LeaderAndIsr leaderAndIsr) { + // currently, we always use the first replica in assignment, which also in aliveReplicas + // and + // isr as the leader replica. + List isr = leaderAndIsr.isr(); + for (int assignment : assignments) { + if (aliveReplicas.contains(assignment) && isr.contains(assignment)) { + return Optional.of( + new TableBucketStateMachine.ElectionResult( + aliveReplicas, leaderAndIsr.newLeaderAndIsr(assignment, isr))); + } + } + + return Optional.empty(); + } + } + + /** The controlled shutdown replica leader election. */ + public static class ControlledShutdownLeaderElection extends ReplicaLeaderElection { + /** + * Controlled shutdown replica leader election. + * + * @param assignments the assignments + * @param aliveReplicas the alive replicas + * @param leaderAndIsr the original leaderAndIsr + * @param shutdownTabletServers the shutdown tabletServers + * @return the election result + */ + public Optional leaderElection( + List assignments, + List aliveReplicas, + LeaderAndIsr leaderAndIsr, + Set shutdownTabletServers) { + List originIsr = leaderAndIsr.isr(); + Set isrSet = new HashSet<>(originIsr); + for (Integer id : assignments) { + if (aliveReplicas.contains(id) + && isrSet.contains(id) + && !shutdownTabletServers.contains(id)) { + Set newAliveReplicas = new HashSet<>(aliveReplicas); + newAliveReplicas.removeAll(shutdownTabletServers); + List newIsr = + originIsr.stream() + .filter(replica -> !shutdownTabletServers.contains(replica)) + .collect(Collectors.toList()); + return Optional.of( + new ElectionResult( + new ArrayList<>(newAliveReplicas), + leaderAndIsr.newLeaderAndIsr(id, newIsr))); + } + } + return Optional.empty(); + } + } + + /** The reassignment replica leader election. */ + public static class ReassignmentLeaderElection extends ReplicaLeaderElection { + private final List newReplicas; + + public ReassignmentLeaderElection(List newReplicas) { + this.newReplicas = newReplicas; + } + + public Optional leaderElection( + List liveReplicas, LeaderAndIsr leaderAndIsr) { + // currently, we always use the first replica in targetReplicas, which also in + // liveReplicas and isr as the leader replica. For bucket reassignment, the first + // replica is the target leader replica. + List isr = leaderAndIsr.isr(); + for (int assignment : newReplicas) { + if (liveReplicas.contains(assignment) && isr.contains(assignment)) { + return Optional.of( + new ElectionResult( + liveReplicas, leaderAndIsr.newLeaderAndIsr(assignment, isr))); + } + } + + return Optional.empty(); + } + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaLeaderElectionAlgorithms.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaLeaderElectionAlgorithms.java deleted file mode 100644 index c7c1aa07a4..0000000000 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaLeaderElectionAlgorithms.java +++ /dev/null @@ -1,116 +0,0 @@ -/* - * 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.fluss.server.coordinator.statemachine; - -import org.apache.fluss.server.coordinator.statemachine.TableBucketStateMachine.ElectionResult; -import org.apache.fluss.server.zk.data.LeaderAndIsr; - -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Optional; -import java.util.Set; -import java.util.stream.Collectors; - -/** The algorithms to elect the replica leader. */ -public class ReplicaLeaderElectionAlgorithms { - - /** - * Init replica leader election when the bucket is new created. - * - * @param assignments the assignments - * @param aliveReplicas the alive replicas - * @param coordinatorEpoch the coordinator epoch - * @return the election result - */ - public static Optional initReplicaLeaderElection( - List assignments, List aliveReplicas, int coordinatorEpoch) { - // currently, we always use the first replica in assignment, which also in aliveReplicas and - // isr as the leader replica. - for (int assignment : assignments) { - if (aliveReplicas.contains(assignment)) { - return Optional.of( - new ElectionResult( - aliveReplicas, - new LeaderAndIsr( - assignment, 0, aliveReplicas, coordinatorEpoch, 0))); - } - } - - return Optional.empty(); - } - - /** - * Default replica leader election, like electing leader while leader offline. - * - * @param assignments the assignments - * @param aliveReplicas the alive replicas - * @param leaderAndIsr the original leaderAndIsr - * @return the election result - */ - public static Optional defaultReplicaLeaderElection( - List assignments, List aliveReplicas, LeaderAndIsr leaderAndIsr) { - // currently, we always use the first replica in assignment, which also in aliveReplicas and - // isr as the leader replica. - List isr = leaderAndIsr.isr(); - for (int assignment : assignments) { - if (aliveReplicas.contains(assignment) && isr.contains(assignment)) { - return Optional.of( - new ElectionResult( - aliveReplicas, leaderAndIsr.newLeaderAndIsr(assignment, isr))); - } - } - - return Optional.empty(); - } - - /** - * Controlled shutdown replica leader election. - * - * @param assignments the assignments - * @param aliveReplicas the alive replicas - * @param leaderAndIsr the original leaderAndIsr - * @param shutdownTabletServers the shutdown tabletServers - * @return the election result - */ - public static Optional controlledShutdownReplicaLeaderElection( - List assignments, - List aliveReplicas, - LeaderAndIsr leaderAndIsr, - Set shutdownTabletServers) { - List originIsr = leaderAndIsr.isr(); - Set isrSet = new HashSet<>(originIsr); - for (Integer id : assignments) { - if (aliveReplicas.contains(id) - && isrSet.contains(id) - && !shutdownTabletServers.contains(id)) { - Set newAliveReplicas = new HashSet<>(aliveReplicas); - newAliveReplicas.removeAll(shutdownTabletServers); - List newIsr = - originIsr.stream() - .filter(replica -> !shutdownTabletServers.contains(replica)) - .collect(Collectors.toList()); - return Optional.of( - new ElectionResult( - new ArrayList<>(newAliveReplicas), - leaderAndIsr.newLeaderAndIsr(id, newIsr))); - } - } - return Optional.empty(); - } -} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachine.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachine.java index 235f4151d8..e231fe41bd 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachine.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachine.java @@ -101,7 +101,7 @@ public void shutdown() { private Tuple2, Set> initializeReplicaState() { Set onlineReplicas = new HashSet<>(); Set offlineReplicas = new HashSet<>(); - Set allBuckets = coordinatorContext.allBuckets(); + Set allBuckets = coordinatorContext.getAllBuckets(); for (TableBucket tableBucket : allBuckets) { List replicas = coordinatorContext.getAssignment(tableBucket); for (Integer replica : replicas) { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachine.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachine.java index 33e278f443..c649e4895a 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachine.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachine.java @@ -17,10 +17,14 @@ package org.apache.fluss.server.coordinator.statemachine; +import org.apache.fluss.annotation.VisibleForTesting; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.server.coordinator.CoordinatorContext; import org.apache.fluss.server.coordinator.CoordinatorRequestBatch; +import org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElection.ControlledShutdownLeaderElection; +import org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElection.DefaultLeaderElection; +import org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElection.ReassignmentLeaderElection; import org.apache.fluss.server.entity.BatchRegisterLeadAndIsr; import org.apache.fluss.server.entity.RegisterTableBucketLeadAndIsrInfo; import org.apache.fluss.server.zk.ZooKeeperClient; @@ -40,12 +44,6 @@ import java.util.Set; import java.util.stream.Collectors; -import static org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElectionAlgorithms.controlledShutdownReplicaLeaderElection; -import static org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElectionAlgorithms.defaultReplicaLeaderElection; -import static org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElectionAlgorithms.initReplicaLeaderElection; -import static org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElectionStrategy.CONTROLLED_SHUTDOWN_ELECTION; -import static org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElectionStrategy.DEFAULT_ELECTION; - /* This file is based on source code of Apache Kafka Project (https://kafka.apache.org/), licensed by the Apache * Software Foundation (ASF) under the Apache License, Version 2.0. See the NOTICE file distributed with this work for * additional information regarding copyright ownership. */ @@ -83,7 +81,7 @@ public void startup() { * table buckets in zookeeper. */ private void initializeBucketState() { - Set tableBuckets = coordinatorContext.allBuckets(); + Set tableBuckets = coordinatorContext.getAllBuckets(); for (TableBucket tableBucket : tableBuckets) { BucketState bucketState = coordinatorContext @@ -121,13 +119,13 @@ public void shutdown() { } public void handleStateChange(Set tableBuckets, BucketState targetState) { - handleStateChange(tableBuckets, targetState, DEFAULT_ELECTION); + handleStateChange(tableBuckets, targetState, new DefaultLeaderElection()); } public void handleStateChange( Set tableBuckets, BucketState targetState, - ReplicaLeaderElectionStrategy replicaLeaderElectionStrategy) { + ReplicaLeaderElection replicaLeaderElection) { try { coordinatorRequestBatch.newBatch(); @@ -136,7 +134,7 @@ public void handleStateChange( batchHandleOnlineChangeAndInitLeader(tableBuckets); } else { for (TableBucket tableBucket : tableBuckets) { - doHandleStateChange(tableBucket, targetState, replicaLeaderElectionStrategy); + doHandleStateChange(tableBucket, targetState, replicaLeaderElection); } } coordinatorRequestBatch.sendRequestToTabletServers( @@ -188,12 +186,12 @@ public void handleStateChange( * * @param tableBucket The table bucket that is to do state change * @param targetState the target state that is to change to - * @param replicaLeaderElectionStrategy the strategy to choose a new leader + * @param replicaLeaderElection the strategy to choose a new leader */ private void doHandleStateChange( TableBucket tableBucket, BucketState targetState, - ReplicaLeaderElectionStrategy replicaLeaderElectionStrategy) { + ReplicaLeaderElection replicaLeaderElection) { coordinatorContext.putBucketStateIfNotExists(tableBucket, BucketState.NonExistentBucket); if (!checkValidTableBucketStateChange(tableBucket, targetState)) { return; @@ -241,8 +239,7 @@ private void doHandleStateChange( // current state is Online or Offline // not new bucket, we then need to update leader/epoch for the bucket Optional optionalElectionResult = - electNewLeaderForTableBuckets( - tableBucket, replicaLeaderElectionStrategy); + electNewLeaderForTableBuckets(tableBucket, replicaLeaderElection); if (!optionalElectionResult.isPresent()) { logFailedStateChange(tableBucket, currentState, targetState); } else { @@ -458,7 +455,7 @@ private List tryRegisterLeaderAndIsrOneByOne( } private Optional electNewLeaderForTableBuckets( - TableBucket tableBucket, ReplicaLeaderElectionStrategy electionStrategy) { + TableBucket tableBucket, ReplicaLeaderElection electionStrategy) { LeaderAndIsr leaderAndIsr; try { leaderAndIsr = zooKeeperClient.getLeaderAndIsr(tableBucket).get(); @@ -585,7 +582,7 @@ private String stringifyBucket(TableBucket tableBucket) { private Optional electLeader( TableBucket tableBucket, LeaderAndIsr leaderAndIsr, - ReplicaLeaderElectionStrategy electionStrategy) { + ReplicaLeaderElection electionStrategy) { List assignment = coordinatorContext.getAssignment(tableBucket); // filter out the live servers List liveReplicas = @@ -599,13 +596,23 @@ private Optional electLeader( } Optional resultOpt = Optional.empty(); - if (electionStrategy == DEFAULT_ELECTION) { - resultOpt = defaultReplicaLeaderElection(assignment, liveReplicas, leaderAndIsr); - } else if (electionStrategy == CONTROLLED_SHUTDOWN_ELECTION) { + if (electionStrategy instanceof DefaultLeaderElection) { + resultOpt = + ((DefaultLeaderElection) electionStrategy) + .leaderElection(assignment, liveReplicas, leaderAndIsr); + } else if (electionStrategy instanceof ControlledShutdownLeaderElection) { Set shuttingDownTabletServers = coordinatorContext.shuttingDownTabletServers(); resultOpt = - controlledShutdownReplicaLeaderElection( - assignment, liveReplicas, leaderAndIsr, shuttingDownTabletServers); + ((ControlledShutdownLeaderElection) electionStrategy) + .leaderElection( + assignment, + liveReplicas, + leaderAndIsr, + shuttingDownTabletServers); + } else if (electionStrategy instanceof ReassignmentLeaderElection) { + resultOpt = + ((ReassignmentLeaderElection) electionStrategy) + .leaderElection(liveReplicas, leaderAndIsr); } if (!resultOpt.isPresent()) { @@ -635,4 +642,30 @@ public LeaderAndIsr getLeaderAndIsr() { return leaderAndIsr; } } + + /** + * Init replica leader election when the bucket is new created. + * + * @param assignments the assignments + * @param aliveReplicas the alive replicas + * @param coordinatorEpoch the coordinator epoch + * @return the election result + */ + @VisibleForTesting + public static Optional initReplicaLeaderElection( + List assignments, List aliveReplicas, int coordinatorEpoch) { + // currently, we always use the first replica in assignment, which also in aliveReplicas and + // isr as the leader replica. + for (int assignment : assignments) { + if (aliveReplicas.contains(assignment)) { + return Optional.of( + new ElectionResult( + aliveReplicas, + new LeaderAndIsr( + assignment, 0, aliveReplicas, coordinatorEpoch, 0))); + } + } + + return Optional.empty(); + } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java index 8a5f856155..7c76bff0c8 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/ReplicaManager.java @@ -322,7 +322,8 @@ private void registerMetrics() { this::physicalStorageRemoteLogSize); } - private Stream onlineReplicas() { + @VisibleForTesting + public Stream onlineReplicas() { return allReplicas.values().stream() .map( t -> { @@ -348,6 +349,11 @@ private long atMinIsrCount() { return onlineReplicas().filter(Replica::isAtMinIsr).count(); } + @VisibleForTesting + public long leaderCount() { + return onlineReplicas().filter(Replica::isLeader).count(); + } + private int writerIdCount() { return onlineReplicas().map(Replica::writerIdCount).reduce(0, Integer::sum); } @@ -395,6 +401,11 @@ public void becomeLeaderOrFollower( List replicasToBeLeader = new ArrayList<>(); List replicasToBeFollower = new ArrayList<>(); for (NotifyLeaderAndIsrData data : notifyLeaderAndIsrDataList) { + LOG.info( + "Try to become leaderAndFollower for {} with isr {}, replicas: {}", + data.getTableBucket(), + data.getLeaderAndIsr(), + data.getReplicas()); TableBucket tb = data.getTableBucket(); try { boolean becomeLeader = validateAndGetIsBecomeLeader(data); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java b/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java index 78c1f587b1..6fcf54c77a 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/utils/ServerRpcMessageUtils.java @@ -20,6 +20,8 @@ import org.apache.fluss.cluster.Endpoint; import org.apache.fluss.cluster.ServerNode; import org.apache.fluss.cluster.ServerType; +import org.apache.fluss.cluster.rebalance.RebalancePlanForBucket; +import org.apache.fluss.cluster.rebalance.RebalanceResultForBucket; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.cluster.AlterConfigOpType; import org.apache.fluss.config.cluster.ConfigEntry; @@ -32,6 +34,7 @@ import org.apache.fluss.metadata.TableChange; import org.apache.fluss.metadata.TableDescriptor; import org.apache.fluss.metadata.TableInfo; +import org.apache.fluss.metadata.TablePartition; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.record.BytesViewLogRecords; import org.apache.fluss.record.DefaultKvRecordBatch; @@ -70,6 +73,7 @@ import org.apache.fluss.rpc.messages.ListOffsetsRequest; import org.apache.fluss.rpc.messages.ListOffsetsResponse; import org.apache.fluss.rpc.messages.ListPartitionInfosResponse; +import org.apache.fluss.rpc.messages.ListRebalanceProcessResponse; import org.apache.fluss.rpc.messages.LookupRequest; import org.apache.fluss.rpc.messages.LookupResponse; import org.apache.fluss.rpc.messages.MetadataResponse; @@ -113,6 +117,10 @@ import org.apache.fluss.rpc.messages.PbProduceLogRespForBucket; import org.apache.fluss.rpc.messages.PbPutKvReqForBucket; import org.apache.fluss.rpc.messages.PbPutKvRespForBucket; +import org.apache.fluss.rpc.messages.PbRebalancePlanForBucket; +import org.apache.fluss.rpc.messages.PbRebalancePlanForTable; +import org.apache.fluss.rpc.messages.PbRebalanceProcessForBucket; +import org.apache.fluss.rpc.messages.PbRebalanceProcessForTable; import org.apache.fluss.rpc.messages.PbRemoteLogSegment; import org.apache.fluss.rpc.messages.PbRemotePathAndLocalFile; import org.apache.fluss.rpc.messages.PbServerNode; @@ -129,6 +137,7 @@ import org.apache.fluss.rpc.messages.ProduceLogResponse; import org.apache.fluss.rpc.messages.PutKvRequest; import org.apache.fluss.rpc.messages.PutKvResponse; +import org.apache.fluss.rpc.messages.RebalanceResponse; import org.apache.fluss.rpc.messages.StopReplicaRequest; import org.apache.fluss.rpc.messages.StopReplicaResponse; import org.apache.fluss.rpc.messages.UpdateMetadataRequest; @@ -160,6 +169,7 @@ import org.apache.fluss.server.zk.data.BucketSnapshot; import org.apache.fluss.server.zk.data.LakeTableSnapshot; import org.apache.fluss.server.zk.data.LeaderAndIsr; +import org.apache.fluss.server.zk.data.RebalancePlan; import javax.annotation.Nullable; @@ -176,6 +186,7 @@ import java.util.Optional; import java.util.OptionalInt; import java.util.Set; +import java.util.function.BiConsumer; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -1692,6 +1703,142 @@ public static List toPbConfigEntries(List describ .collect(Collectors.toList()); } + public static RebalanceResponse makeRebalanceRespose(RebalancePlan rebalancePlan) { + RebalanceResponse response = new RebalanceResponse(); + List planForTables = new ArrayList<>(); + + // for none-partitioned tables. + for (Map.Entry> planForTable : + rebalancePlan.getPlanForBuckets().entrySet()) { + PbRebalancePlanForTable pbRebalancePlanForTable = + response.addPlanForTable().setTableId(planForTable.getKey()); + List planForBuckets = new ArrayList<>(); + planForTable + .getValue() + .forEach( + planForBucket -> + planForBuckets.add(toPbRebalancePlanForBucket(planForBucket))); + pbRebalancePlanForTable.addAllBucketsPlans(planForBuckets); + planForTables.add(pbRebalancePlanForTable); + } + response.addAllPlanForTables(planForTables); + + // for partitioned tables. + Map>> planForBucketsOfPartitionedTable = + new HashMap<>(); + for (Map.Entry> planForTable : + rebalancePlan.getPlanForBucketsOfPartitionedTable().entrySet()) { + Map> bucketsPlanForPartition = + planForBucketsOfPartitionedTable.computeIfAbsent( + planForTable.getKey().getTableId(), k -> new HashMap<>()); + bucketsPlanForPartition.put( + planForTable.getKey().getPartitionId(), + planForTable.getValue().stream() + .map(ServerRpcMessageUtils::toPbRebalancePlanForBucket) + .collect(Collectors.toList())); + } + + for (Map.Entry>> planForPartition : + planForBucketsOfPartitionedTable.entrySet()) { + PbRebalancePlanForTable pbRebalancePlanForTable = + response.addPlanForTable().setTableId(planForPartition.getKey()); + planForPartition + .getValue() + .forEach( + (partitionId, planForBuckets) -> + pbRebalancePlanForTable + .addPartitionsPlan() + .setPartitionId(partitionId) + .addAllBucketsPlans(planForBuckets)); + } + return response; + } + + private static PbRebalancePlanForBucket toPbRebalancePlanForBucket( + RebalancePlanForBucket planForBucket) { + PbRebalancePlanForBucket pbRebalancePlanForBucket = + new PbRebalancePlanForBucket() + .setBucketId(planForBucket.getBucketId()) + .setOriginalLeader(planForBucket.getOriginalLeader()) + .setNewLeader(planForBucket.getNewLeader()); + pbRebalancePlanForBucket + .setOriginalReplicas( + planForBucket.getOriginReplicas().stream() + .mapToInt(Integer::intValue) + .toArray()) + .setNewReplicas( + planForBucket.getNewReplicas().stream() + .mapToInt(Integer::intValue) + .toArray()); + return pbRebalancePlanForBucket; + } + + public static ListRebalanceProcessResponse makeListRebalanceProcessResponse( + Map ongoingRebalanceTasks, + Map finishedRebalanceTasks) { + ListRebalanceProcessResponse response = new ListRebalanceProcessResponse(); + + Map> processForTables = new HashMap<>(); + Map>> processForPartitions = + new HashMap<>(); + + BiConsumer collectProcessResult = + (tableBucket, rebalanceResultForBucket) -> { + if (tableBucket.getPartitionId() == null) { + processForTables + .computeIfAbsent(tableBucket.getTableId(), k -> new ArrayList<>()) + .add( + toPbRebalanceProcessForBucket( + tableBucket, rebalanceResultForBucket)); + } else { + processForPartitions + .computeIfAbsent(tableBucket.getTableId(), k -> new HashMap<>()) + .computeIfAbsent( + tableBucket.getPartitionId(), k -> new ArrayList<>()) + .add( + toPbRebalanceProcessForBucket( + tableBucket, rebalanceResultForBucket)); + } + }; + + ongoingRebalanceTasks.forEach(collectProcessResult); + finishedRebalanceTasks.forEach(collectProcessResult); + + processForTables.forEach( + (tableId, processForBuckets) -> + response.addProcessForTable() + .setTableId(tableId) + .addAllBucketsProcesses(processForBuckets)); + processForPartitions.forEach( + (tableId, processForPartition) -> { + PbRebalanceProcessForTable processForTable = + response.addProcessForTable().setTableId(tableId); + processForPartition.forEach( + (partitionId, processForBuckets) -> + processForTable + .addPartitionsProcess() + .setPartitionId(partitionId) + .addAllBucketsProcesses(processForBuckets)); + }); + + return response; + } + + private static PbRebalanceProcessForBucket toPbRebalanceProcessForBucket( + TableBucket tableBucket, RebalanceResultForBucket rebalanceResultForBucket) { + return new PbRebalanceProcessForBucket() + .setBucketId(tableBucket.getBucket()) + .setOriginalReplicas( + rebalanceResultForBucket.originReplicas().stream() + .mapToInt(Integer::intValue) + .toArray()) + .setNewReplicas( + rebalanceResultForBucket.newReplicas().stream() + .mapToInt(Integer::intValue) + .toArray()) + .setRebalanceStatus(rebalanceResultForBucket.status().getCode()); + } + private static Map mergeResponse( Map response, Map errors) { if (errors.isEmpty()) { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/utils/TableAssignmentUtils.java b/fluss-server/src/main/java/org/apache/fluss/server/utils/TableAssignmentUtils.java index 067edc35e6..eca96ddf5e 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/utils/TableAssignmentUtils.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/utils/TableAssignmentUtils.java @@ -224,31 +224,31 @@ private static TableAssignment generateRackAwareAssigment( replicas.add(leader); Set racksWithReplicas = new HashSet<>(); racksWithReplicas.add(serverRackMap.get(leader)); - Set brokersWithReplicas = new HashSet<>(); - brokersWithReplicas.add(leader); + Set tabletServersWithReplicas = new HashSet<>(); + tabletServersWithReplicas.add(leader); int k = 0; for (int j = 0; j < replicationFactor - 1; j++) { boolean done = false; while (!done) { - Integer broker = + Integer server = arrangedServerList.get( replicaIndex( firstReplicaIndex, nextReplicaShift * numRacks, k, arrangedServerList.size())); - String rack = serverRackMap.get(broker); + String rack = serverRackMap.get(server); // Skip this tabletServer if // 1. there is already a tabletServer in the same rack that has assigned a // replica AND there is one or more racks that do not have any replica, or // 2. the tabletServer has already assigned a replica AND there is one or more // tabletServers that do not have replica assigned if ((!racksWithReplicas.contains(rack) || racksWithReplicas.size() == numRacks) - && (!brokersWithReplicas.contains(broker) - || brokersWithReplicas.size() == numServers)) { - replicas.add(broker); + && (!tabletServersWithReplicas.contains(server) + || tabletServersWithReplicas.size() == numServers)) { + replicas.add(server); racksWithReplicas.add(rack); - brokersWithReplicas.add(broker); + tabletServersWithReplicas.add(server); done = true; } k += 1; diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java index 1c7401b8a7..932fbc2ced 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/ZooKeeperClient.java @@ -43,8 +43,10 @@ import org.apache.fluss.server.zk.data.LakeTableSnapshot; import org.apache.fluss.server.zk.data.LeaderAndIsr; import org.apache.fluss.server.zk.data.PartitionAssignment; +import org.apache.fluss.server.zk.data.RebalancePlan; import org.apache.fluss.server.zk.data.RemoteLogManifestHandle; import org.apache.fluss.server.zk.data.ResourceAcl; +import org.apache.fluss.server.zk.data.ServerTags; import org.apache.fluss.server.zk.data.TableAssignment; import org.apache.fluss.server.zk.data.TableRegistration; import org.apache.fluss.server.zk.data.TabletServerRegistration; @@ -64,11 +66,13 @@ import org.apache.fluss.server.zk.data.ZkData.PartitionSequenceIdZNode; import org.apache.fluss.server.zk.data.ZkData.PartitionZNode; import org.apache.fluss.server.zk.data.ZkData.PartitionsZNode; +import org.apache.fluss.server.zk.data.ZkData.RebalanceZNode; import org.apache.fluss.server.zk.data.ZkData.ResourceAclNode; import org.apache.fluss.server.zk.data.ZkData.SchemaZNode; import org.apache.fluss.server.zk.data.ZkData.SchemasZNode; import org.apache.fluss.server.zk.data.ZkData.ServerIdZNode; import org.apache.fluss.server.zk.data.ZkData.ServerIdsZNode; +import org.apache.fluss.server.zk.data.ZkData.ServerTagsZNode; import org.apache.fluss.server.zk.data.ZkData.TableIdZNode; import org.apache.fluss.server.zk.data.ZkData.TableSequenceIdZNode; import org.apache.fluss.server.zk.data.ZkData.TableZNode; @@ -289,7 +293,17 @@ public void updateTableAssignment(long tableId, TableAssignment tableAssignment) throws Exception { String path = TableIdZNode.path(tableId); zkClient.setData().forPath(path, TableIdZNode.encode(tableAssignment)); - LOG.info("Updated table assignment {} for table id {}.", tableAssignment, tableId); + LOG.debug("Updated table assignment {} for table id {}.", tableAssignment, tableId); + } + + public void updatePartitionAssignment(long partitionId, PartitionAssignment partitionAssignment) + throws Exception { + String path = PartitionIdZNode.path(partitionId); + zkClient.setData().forPath(path, PartitionIdZNode.encode(partitionAssignment)); + LOG.debug( + "Updated partition assignment {} for partition id {}.", + partitionAssignment, + partitionId); } public void deleteTableAssignment(long tableId) throws Exception { @@ -1165,6 +1179,49 @@ public void insertConfigChangeNotification() throws Exception { ZkData.ConfigEntityChangeNotificationSequenceZNode.encode()); } + // -------------------------------------------------------------------------------------------- + // Maintenance + // -------------------------------------------------------------------------------------------- + + public void registerServerTags(ServerTags newServerTags) throws Exception { + String path = ServerTagsZNode.path(); + if (getOrEmpty(path).isPresent()) { + zkClient.setData().forPath(path, ServerTagsZNode.encode(newServerTags)); + } else { + zkClient.create() + .creatingParentsIfNeeded() + .withMode(CreateMode.PERSISTENT) + .forPath(path, ServerTagsZNode.encode(newServerTags)); + } + } + + public Optional getServerTags() throws Exception { + String path = ServerTagsZNode.path(); + return getOrEmpty(path).map(ServerTagsZNode::decode); + } + + public void registerRebalancePlan(RebalancePlan rebalancePlan) throws Exception { + String path = RebalanceZNode.path(); + if (getOrEmpty(path).isPresent()) { + zkClient.setData().forPath(path, RebalanceZNode.encode(rebalancePlan)); + } else { + zkClient.create() + .creatingParentsIfNeeded() + .withMode(CreateMode.PERSISTENT) + .forPath(path, RebalanceZNode.encode(rebalancePlan)); + } + } + + public Optional getRebalancePlan() throws Exception { + String path = RebalanceZNode.path(); + return getOrEmpty(path).map(RebalanceZNode::decode); + } + + public void deleteRebalancePlan() throws Exception { + String path = RebalanceZNode.path(); + deletePath(path); + } + // -------------------------------------------------------------------------------------------- // Utils // -------------------------------------------------------------------------------------------- diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/LeaderAndIsr.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/LeaderAndIsr.java index 2512481389..2f37287ac8 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/LeaderAndIsr.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/LeaderAndIsr.java @@ -93,6 +93,10 @@ public LeaderAndIsr newLeaderAndIsr(List newIsr) { return new LeaderAndIsr(leader, leaderEpoch, newIsr, coordinatorEpoch, bucketEpoch + 1); } + public LeaderAndIsr newLeaderAndIsrWithNewLeaderEpoch() { + return new LeaderAndIsr(leader, leaderEpoch + 1, isr, coordinatorEpoch, bucketEpoch); + } + public int leader() { return leader; } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/RebalancePlan.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/RebalancePlan.java new file mode 100644 index 0000000000..b34dd573a1 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/RebalancePlan.java @@ -0,0 +1,127 @@ +/* + * 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.fluss.server.zk.data; + +import org.apache.fluss.cluster.rebalance.RebalancePlanForBucket; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TablePartition; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +/** + * The generated rebalance plan for this cluster. + * + *

The latest execution rebalance plan will be stored in {@link ZkData.RebalanceZNode}. + * + * @see RebalancePlanJsonSerde for json serialization and deserialization. + */ +public class RebalancePlan { + + /** A mapping from tableBucket to RebalancePlanForBuckets of none-partitioned table. */ + private final Map> planForBuckets; + + /** A mapping from tableBucket to RebalancePlanForBuckets of partitioned table. */ + private final Map> + planForBucketsOfPartitionedTable; + + public RebalancePlan(Map bucketPlan) { + this.planForBuckets = new HashMap<>(); + this.planForBucketsOfPartitionedTable = new HashMap<>(); + + for (Map.Entry entry : bucketPlan.entrySet()) { + TableBucket tableBucket = entry.getKey(); + RebalancePlanForBucket rebalancePlanForBucket = entry.getValue(); + if (tableBucket.getPartitionId() == null) { + planForBuckets + .computeIfAbsent(tableBucket.getTableId(), k -> new ArrayList<>()) + .add(rebalancePlanForBucket); + } else { + TablePartition tp = + new TablePartition(tableBucket.getTableId(), tableBucket.getPartitionId()); + planForBucketsOfPartitionedTable + .computeIfAbsent(tp, k -> new ArrayList<>()) + .add(rebalancePlanForBucket); + } + } + } + + public Map> getPlanForBuckets() { + return planForBuckets; + } + + public Map> getPlanForBucketsOfPartitionedTable() { + return planForBucketsOfPartitionedTable; + } + + public Map getExecutePlan() { + Map executePlan = new HashMap<>(); + planForBuckets.forEach( + (tableId, rebalancePlanForBuckets) -> + rebalancePlanForBuckets.forEach( + rebalancePlanForBucket -> + executePlan.put( + rebalancePlanForBucket.getTableBucket(), + rebalancePlanForBucket))); + + planForBucketsOfPartitionedTable.forEach( + (tablePartition, rebalancePlanForBuckets) -> + rebalancePlanForBuckets.forEach( + rebalancePlanForBucket -> + executePlan.put( + rebalancePlanForBucket.getTableBucket(), + rebalancePlanForBucket))); + return executePlan; + } + + @Override + public String toString() { + return "RebalancePlan{" + + "planForBuckets=" + + planForBuckets + + ", planForBucketsOfPartitionedTable=" + + planForBucketsOfPartitionedTable + + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + RebalancePlan that = (RebalancePlan) o; + + if (!Objects.equals(planForBuckets, that.planForBuckets)) { + return false; + } + return Objects.equals( + planForBucketsOfPartitionedTable, that.planForBucketsOfPartitionedTable); + } + + @Override + public int hashCode() { + return Objects.hash(planForBuckets, planForBucketsOfPartitionedTable); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/RebalancePlanJsonSerde.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/RebalancePlanJsonSerde.java new file mode 100644 index 0000000000..b79cd46e30 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/RebalancePlanJsonSerde.java @@ -0,0 +1,156 @@ +/* + * 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.fluss.server.zk.data; + +import org.apache.fluss.cluster.rebalance.RebalancePlanForBucket; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TablePartition; +import org.apache.fluss.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; +import org.apache.fluss.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.fluss.utils.json.JsonDeserializer; +import org.apache.fluss.utils.json.JsonSerializer; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** Json serializer and deserializer for {@link RebalancePlan}. */ +public class RebalancePlanJsonSerde + implements JsonSerializer, JsonDeserializer { + + public static final RebalancePlanJsonSerde INSTANCE = new RebalancePlanJsonSerde(); + + private static final String VERSION_KEY = "version"; + private static final String REBALANCE_PLAN = "rebalance_plan"; + + private static final String TABLE_ID = "table_id"; + private static final String PARTITION_ID = "partition_id"; + + private static final String BUCKETS = "buckets"; + private static final String BUCKET_ID = "bucket_id"; + private static final String ORIGINAL_LEADER = "original_leader"; + private static final String NEW_LEADER = "new_leader"; + private static final String ORIGIN_REPLICAS = "origin_replicas"; + private static final String NEW_REPLICAS = "new_replicas"; + + private static final int VERSION = 1; + + @Override + public void serialize(RebalancePlan rebalancePlan, JsonGenerator generator) throws IOException { + generator.writeStartObject(); + generator.writeNumberField(VERSION_KEY, VERSION); + + generator.writeArrayFieldStart(REBALANCE_PLAN); + // first to write none-partitioned tables. + for (Map.Entry> entry : + rebalancePlan.getPlanForBuckets().entrySet()) { + generator.writeStartObject(); + generator.writeNumberField(TABLE_ID, entry.getKey()); + generator.writeArrayFieldStart(BUCKETS); + for (RebalancePlanForBucket bucketPlan : entry.getValue()) { + serializeRebalancePlanForBucket(generator, bucketPlan); + } + generator.writeEndArray(); + generator.writeEndObject(); + } + + // then to write partitioned tables. + for (Map.Entry> entry : + rebalancePlan.getPlanForBucketsOfPartitionedTable().entrySet()) { + generator.writeStartObject(); + generator.writeNumberField(TABLE_ID, entry.getKey().getTableId()); + generator.writeNumberField(PARTITION_ID, entry.getKey().getPartitionId()); + generator.writeArrayFieldStart(BUCKETS); + for (RebalancePlanForBucket bucketPlan : entry.getValue()) { + serializeRebalancePlanForBucket(generator, bucketPlan); + } + generator.writeEndArray(); + generator.writeEndObject(); + } + + generator.writeEndArray(); + + generator.writeEndObject(); + } + + @Override + public RebalancePlan deserialize(JsonNode node) { + JsonNode rebalancePlanNode = node.get(REBALANCE_PLAN); + Map planForBuckets = new HashMap<>(); + + for (JsonNode tablePartitionPlanNode : rebalancePlanNode) { + long tableId = tablePartitionPlanNode.get(TABLE_ID).asLong(); + + Long partitionId = null; + if (tablePartitionPlanNode.has(PARTITION_ID)) { + partitionId = tablePartitionPlanNode.get(PARTITION_ID).asLong(); + } + + JsonNode bucketPlanNodes = tablePartitionPlanNode.get(BUCKETS); + for (JsonNode bucketPlanNode : bucketPlanNodes) { + int bucketId = bucketPlanNode.get(BUCKET_ID).asInt(); + TableBucket tableBucket = new TableBucket(tableId, partitionId, bucketId); + + int originLeader = bucketPlanNode.get(ORIGINAL_LEADER).asInt(); + + int newLeader = bucketPlanNode.get(NEW_LEADER).asInt(); + + List originReplicas = new ArrayList<>(); + Iterator elements = bucketPlanNode.get(ORIGIN_REPLICAS).elements(); + while (elements.hasNext()) { + originReplicas.add(elements.next().asInt()); + } + + List newReplicas = new ArrayList<>(); + elements = bucketPlanNode.get(NEW_REPLICAS).elements(); + while (elements.hasNext()) { + newReplicas.add(elements.next().asInt()); + } + + planForBuckets.put( + tableBucket, + new RebalancePlanForBucket( + tableBucket, originLeader, newLeader, originReplicas, newReplicas)); + } + } + + return new RebalancePlan(planForBuckets); + } + + private void serializeRebalancePlanForBucket( + JsonGenerator generator, RebalancePlanForBucket bucketPlan) throws IOException { + generator.writeStartObject(); + generator.writeNumberField(BUCKET_ID, bucketPlan.getBucketId()); + generator.writeNumberField(ORIGINAL_LEADER, bucketPlan.getOriginalLeader()); + generator.writeNumberField(NEW_LEADER, bucketPlan.getNewLeader()); + generator.writeArrayFieldStart(ORIGIN_REPLICAS); + for (Integer replica : bucketPlan.getOriginReplicas()) { + generator.writeNumber(replica); + } + generator.writeEndArray(); + generator.writeArrayFieldStart(NEW_REPLICAS); + for (Integer replica : bucketPlan.getNewReplicas()) { + generator.writeNumber(replica); + } + generator.writeEndArray(); + generator.writeEndObject(); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ServerTags.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ServerTags.java new file mode 100644 index 0000000000..edddcaaf75 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ServerTags.java @@ -0,0 +1,65 @@ +/* + * 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.fluss.server.zk.data; + +import org.apache.fluss.cluster.rebalance.ServerTag; + +import java.util.Map; +import java.util.Objects; + +/** + * The latest {@link ServerTags} of tabletServers in {@link ZkData.ServerTagsZNode}. It is used to + * store the serverTags information in zookeeper. + * + * @see ServerTagsJsonSerde for json serialization and deserialization. + */ +public class ServerTags { + + // a mapping from tabletServer id to serverTag. + private final Map serverTags; + + public ServerTags(Map serverTags) { + this.serverTags = serverTags; + } + + public Map getServerTags() { + return serverTags; + } + + @Override + public String toString() { + return "ServerTags{" + "serverTags=" + serverTags + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ServerTags that = (ServerTags) o; + return Objects.equals(serverTags, that.serverTags); + } + + @Override + public int hashCode() { + return Objects.hash(serverTags); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ServerTagsJsonSerde.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ServerTagsJsonSerde.java new file mode 100644 index 0000000000..7df94c74e4 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ServerTagsJsonSerde.java @@ -0,0 +1,67 @@ +/* + * 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.fluss.server.zk.data; + +import org.apache.fluss.cluster.rebalance.ServerTag; +import org.apache.fluss.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; +import org.apache.fluss.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.fluss.utils.json.JsonDeserializer; +import org.apache.fluss.utils.json.JsonSerializer; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; + +/** Json serializer and deserializer for {@link ServerTags}. */ +public class ServerTagsJsonSerde + implements JsonSerializer, JsonDeserializer { + + public static final ServerTagsJsonSerde INSTANCE = new ServerTagsJsonSerde(); + + private static final String VERSION_KEY = "version"; + private static final String SERVER_TAGS = "server_tags"; + private static final int VERSION = 1; + + @Override + public void serialize(ServerTags serverTags, JsonGenerator generator) throws IOException { + generator.writeStartObject(); + generator.writeNumberField(VERSION_KEY, VERSION); + generator.writeObjectFieldStart(SERVER_TAGS); + for (Map.Entry entry : serverTags.getServerTags().entrySet()) { + generator.writeNumberField(String.valueOf(entry.getKey()), entry.getValue().value); + } + generator.writeEndObject(); + + generator.writeEndObject(); + } + + @Override + public ServerTags deserialize(JsonNode node) { + JsonNode serverTagsNode = node.get(SERVER_TAGS); + Map serverTags = new HashMap<>(); + Iterator fieldNames = serverTagsNode.fieldNames(); + while (fieldNames.hasNext()) { + String serverId = fieldNames.next(); + serverTags.put( + Integer.valueOf(serverId), + ServerTag.valueOf(serverTagsNode.get(serverId).asInt())); + } + return new ServerTags(serverTags); + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java index 024707d8f0..e132123557 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/ZkData.java @@ -555,6 +555,25 @@ public static LakeTableSnapshot decode(byte[] json) { } } + /** + * The znode for server tags. The znode path is: + * + *

/tabletServers/server_tags + */ + public static final class ServerTagsZNode { + public static String path() { + return "/tabletservers/server_tags"; + } + + public static byte[] encode(ServerTags serverTag) { + return JsonSerdeUtils.writeValueAsBytes(serverTag, ServerTagsJsonSerde.INSTANCE); + } + + public static ServerTags decode(byte[] json) { + return JsonSerdeUtils.readValue(json, ServerTagsJsonSerde.INSTANCE); + } + } + // ------------------------------------------------------------------------------------------ // ZNodes for ACL(Access Control List). // ------------------------------------------------------------------------------------------ @@ -718,4 +737,27 @@ public static byte[] encode() { return new byte[0]; } } + + // ------------------------------------------------------------------------------------------ + // ZNodes under "/cluster/" + // ------------------------------------------------------------------------------------------ + + /** + * The znode for rebalance. The znode path is: + * + *

/cluster/rebalance + */ + public static final class RebalanceZNode { + public static String path() { + return "/cluster/rebalance"; + } + + public static byte[] encode(RebalancePlan rebalancePlan) { + return JsonSerdeUtils.writeValueAsBytes(rebalancePlan, RebalancePlanJsonSerde.INSTANCE); + } + + public static RebalancePlan decode(byte[] json) { + return JsonSerdeUtils.readValue(json, RebalancePlanJsonSerde.INSTANCE); + } + } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java index adbb147687..18ee1ba1af 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/CoordinatorEventProcessorTest.java @@ -19,6 +19,7 @@ import org.apache.fluss.cluster.Endpoint; import org.apache.fluss.cluster.TabletServerInfo; +import org.apache.fluss.cluster.rebalance.RebalancePlanForBucket; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; import org.apache.fluss.exception.FencedLeaderEpochException; @@ -41,6 +42,7 @@ import org.apache.fluss.server.coordinator.event.CommitKvSnapshotEvent; import org.apache.fluss.server.coordinator.event.CommitRemoteLogManifestEvent; import org.apache.fluss.server.coordinator.event.CoordinatorEventManager; +import org.apache.fluss.server.coordinator.event.ExecuteRebalanceTaskEvent; import org.apache.fluss.server.coordinator.statemachine.BucketState; import org.apache.fluss.server.coordinator.statemachine.ReplicaState; import org.apache.fluss.server.entity.AdjustIsrResultForBucket; @@ -374,11 +376,11 @@ void testServerBecomeOnlineAndOfflineLine() throws Exception { // should be offline verifyReplicaOnlineOrOffline( table1Id, table1Assignment, Collections.singleton(newlyServerId)); - verifyBucketIsr(table1Id, 0, new int[] {0, 2}); - verifyBucketIsr(table1Id, 1, new int[] {2, 0}); + verifyBucketIsr(new TableBucket(table1Id, 0), 0, new int[] {0, 2}); + verifyBucketIsr(new TableBucket(table1Id, 1), 2, new int[] {2, 0}); verifyReplicaOnlineOrOffline( table2Id, table2Assignment, Collections.singleton(newlyServerId)); - verifyBucketIsr(table2Id, 0, new int[] {3}); + verifyBucketIsr(new TableBucket(table2Id, 0), 3, new int[] {3}); // now, check bucket state TableBucket t1Bucket0 = new TableBucket(table1Id, 0); @@ -833,6 +835,184 @@ void testProcessAdjustIsr() throws Exception { assertThat(resultForBucketMap.values()).allMatch(AdjustIsrResultForBucket::succeeded); } + @Test + void testDoElectPreferredLeaders() throws Exception { + initCoordinatorChannel(); + TablePath t1 = TablePath.of(defaultDatabase, "test_preferred_leader_elect_table"); + final long t1Id = + createTable( + t1, + new TabletServerInfo[] { + new TabletServerInfo(0, "rack0"), + new TabletServerInfo(1, "rack1"), + new TabletServerInfo(2, "rack2") + }); + TableBucket tableBucket = new TableBucket(t1Id, 0); + LeaderAndIsr leaderAndIsr = + waitValue( + () -> fromCtx((ctx) -> ctx.getBucketLeaderAndIsr(tableBucket)), + Duration.ofMinutes(1), + "leader not elected"); + + // change leader, and add back the origin leader to isr set as follower. + int originLeader = leaderAndIsr.leader(); + int bucketLeaderEpoch = leaderAndIsr.leaderEpoch(); + int coordinatorEpoch = leaderAndIsr.coordinatorEpoch(); + List isr1 = leaderAndIsr.isr(); + + Integer follower1 = isr1.stream().filter(i -> i != originLeader).findFirst().get(); + Integer follower2 = + isr1.stream() + .filter(i -> i != originLeader && !i.equals(follower1)) + .findFirst() + .get(); + List isr2 = Arrays.asList(follower1, follower2, originLeader); + // change isr in coordinator context + // leader change from originLeader to follower1. + LeaderAndIsr newLeaderAndIsr = + new LeaderAndIsr( + follower1, + leaderAndIsr.leaderEpoch() + 1, + isr2, + coordinatorEpoch, + bucketLeaderEpoch + 1); + fromCtx( + ctx -> { + ctx.putBucketLeaderAndIsr(tableBucket, newLeaderAndIsr); + return null; + }); + // Also update zk. + zookeeperClient.updateLeaderAndIsr(tableBucket, newLeaderAndIsr); + + // trigger preferred leader election. + ExecuteRebalanceTaskEvent rebalanceTaskEvent = + new ExecuteRebalanceTaskEvent( + new RebalancePlanForBucket( + tableBucket, + newLeaderAndIsr.leader(), + originLeader, + leaderAndIsr.isr(), + newLeaderAndIsr.isr())); + eventProcessor.getCoordinatorEventManager().put(rebalanceTaskEvent); + + // verify preferred leader election + LeaderAndIsr newLeaderAndIsr2 = + waitValue( + () -> fromCtx((ctx) -> ctx.getBucketLeaderAndIsr(tableBucket)), + Duration.ofMinutes(1), + "leader not elected"); + LeaderAndIsr newLeaderAndIsr2OfZk = zookeeperClient.getLeaderAndIsr(tableBucket).get(); + assertThat(newLeaderAndIsr2.leader()).isEqualTo(originLeader); + assertThat(newLeaderAndIsr2OfZk.leader()).isEqualTo(originLeader); + assertThat(newLeaderAndIsr2OfZk.leaderEpoch()).isEqualTo(2); + assertThat(newLeaderAndIsr2OfZk.bucketEpoch()).isEqualTo(2); + } + + @Test + void testDoBucketReassignment() throws Exception { + zookeeperClient.registerTabletServer( + 3, + new TabletServerRegistration( + "rack3", + Collections.singletonList( + new Endpoint("host3", 1001, DEFAULT_LISTENER_NAME)), + System.currentTimeMillis())); + + initCoordinatorChannel(); + TablePath t1 = TablePath.of(defaultDatabase, "test_bucket_reassignment_table"); + // Mock un-balanced table assignment. + Map bucketAssignments = new HashMap<>(); + bucketAssignments.put(0, BucketAssignment.of(0, 1, 3)); + bucketAssignments.put(1, BucketAssignment.of(0, 1, 3)); + bucketAssignments.put(2, BucketAssignment.of(1, 2, 3)); + // For server-0: 2 leader, 2 replica + // For server-1: 1 leader, 3 replica + // For server-2: 0 leader, 1 replica + // For server-3: 0 leader, 3 replica + TableAssignment tableAssignment = new TableAssignment(bucketAssignments); + long t1Id = + metadataManager.createTable( + t1, CoordinatorEventProcessorTest.TEST_TABLE, tableAssignment, false); + TableBucket tb0 = new TableBucket(t1Id, 0); + TableBucket tb1 = new TableBucket(t1Id, 1); + TableBucket tb2 = new TableBucket(t1Id, 2); + verifyIsr(tb0, 0, Arrays.asList(0, 1, 3)); + verifyIsr(tb1, 0, Arrays.asList(0, 1, 3)); + verifyIsr(tb2, 1, Arrays.asList(1, 2, 3)); + + // trigger bucket reassignment for t1: + // bucket0 -> (0, 1, 2) + // bucket1 -> (1, 2, 3) + // bucket2 -> (2, 3, 0) + // For Server-0: 1 leader, 2 replica + // For Server-1: 1 leader, 2 replica + // For Server-2: 1 leader, 2 replica + // For Server-3: 0 leader, 2 replica + eventProcessor + .getCoordinatorEventManager() + .put( + new ExecuteRebalanceTaskEvent( + new RebalancePlanForBucket( + tb0, + 0, + 0, + Arrays.asList(0, 1, 3), + Arrays.asList(0, 1, 2)))); + eventProcessor + .getCoordinatorEventManager() + .put( + new ExecuteRebalanceTaskEvent( + new RebalancePlanForBucket( + tb1, + 0, + 0, + Arrays.asList(0, 1, 3), + Arrays.asList(1, 2, 3)))); + eventProcessor + .getCoordinatorEventManager() + .put( + new ExecuteRebalanceTaskEvent( + new RebalancePlanForBucket( + tb2, + 1, + 1, + Arrays.asList(1, 2, 3), + Arrays.asList(2, 3, 0)))); + + // Mock to finish rebalance tasks, in production case, this need to be trigged by receiving + // AdjustIsrRequest. + Map leaderAndIsrMap = new HashMap<>(); + CompletableFuture respCallback = new CompletableFuture<>(); + // This isr list equals originReplicas + addingReplicas. + leaderAndIsrMap.put(tb0, new LeaderAndIsr(0, 1, Arrays.asList(0, 1, 2, 3), 0, 0)); + leaderAndIsrMap.put(tb1, new LeaderAndIsr(0, 1, Arrays.asList(0, 1, 2, 3), 0, 0)); + leaderAndIsrMap.put(tb2, new LeaderAndIsr(1, 1, Arrays.asList(1, 2, 3, 0), 0, 0)); + eventProcessor + .getCoordinatorEventManager() + .put(new AdjustIsrReceivedEvent(leaderAndIsrMap, respCallback)); + respCallback.get(); + + verifyIsr(tb0, 0, Arrays.asList(0, 1, 2)); + verifyIsr(tb1, 1, Arrays.asList(1, 2, 3)); + verifyIsr(tb2, 2, Arrays.asList(2, 3, 0)); + } + + private void verifyIsr(TableBucket tb, int expectedLeader, List expectedIsr) + throws Exception { + LeaderAndIsr leaderAndIsr = + waitValue( + () -> fromCtx((ctx) -> ctx.getBucketLeaderAndIsr(tb)), + Duration.ofMinutes(1), + "leader not elected"); + LeaderAndIsr newLeaderAndIsrOfZk = zookeeperClient.getLeaderAndIsr(tb).get(); + assertThat(leaderAndIsr.leader()) + .isEqualTo(newLeaderAndIsrOfZk.leader()) + .isEqualTo(expectedLeader); + assertThat(leaderAndIsr.isr()) + .isEqualTo(newLeaderAndIsrOfZk.isr()) + .hasSameElementsAs(expectedIsr); + } + private CoordinatorEventProcessor buildCoordinatorEventProcessor() { return new CoordinatorEventProcessor( zookeeperClient, @@ -1037,13 +1217,13 @@ private void verifyReplicaOnlineOrOffline( }); } - private void verifyBucketIsr(long tableId, int bucket, int[] expectedIsr) { + private void verifyBucketIsr(TableBucket tableBucket, int leader, int[] expectedIsr) { retryVerifyContext( ctx -> { - TableBucket tableBucket = new TableBucket(tableId, bucket); // verify leaderAndIsr from coordinator context LeaderAndIsr leaderAndIsr = ctx.getBucketLeaderAndIsr(tableBucket).get(); assertThat(leaderAndIsr.isrArray()).isEqualTo(expectedIsr); + assertThat(leaderAndIsr.leader()).isEqualTo(leader); // verify leaderAndIsr from tablet server try { leaderAndIsr = zookeeperClient.getLeaderAndIsr(tableBucket).get(); @@ -1051,6 +1231,7 @@ private void verifyBucketIsr(long tableId, int bucket, int[] expectedIsr) { throw new RuntimeException("Fail to get leaderAndIsr of " + tableBucket); } assertThat(leaderAndIsr.isrArray()).isEqualTo(expectedIsr); + assertThat(leaderAndIsr.leader()).isEqualTo(leader); }); } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TestCoordinatorGateway.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TestCoordinatorGateway.java index ea20e62f99..d4faae57e8 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TestCoordinatorGateway.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TestCoordinatorGateway.java @@ -21,6 +21,8 @@ import org.apache.fluss.exception.IneligibleReplicaException; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.rpc.gateway.CoordinatorGateway; +import org.apache.fluss.rpc.messages.AddServerTagRequest; +import org.apache.fluss.rpc.messages.AddServerTagResponse; import org.apache.fluss.rpc.messages.AdjustIsrRequest; import org.apache.fluss.rpc.messages.AdjustIsrResponse; import org.apache.fluss.rpc.messages.AlterClusterConfigsRequest; @@ -29,6 +31,8 @@ import org.apache.fluss.rpc.messages.AlterTableResponse; import org.apache.fluss.rpc.messages.ApiVersionsRequest; import org.apache.fluss.rpc.messages.ApiVersionsResponse; +import org.apache.fluss.rpc.messages.CancelRebalanceRequest; +import org.apache.fluss.rpc.messages.CancelRebalanceResponse; import org.apache.fluss.rpc.messages.CommitKvSnapshotRequest; import org.apache.fluss.rpc.messages.CommitKvSnapshotResponse; import org.apache.fluss.rpc.messages.CommitLakeTableSnapshotRequest; @@ -79,10 +83,16 @@ import org.apache.fluss.rpc.messages.ListDatabasesResponse; import org.apache.fluss.rpc.messages.ListPartitionInfosRequest; import org.apache.fluss.rpc.messages.ListPartitionInfosResponse; +import org.apache.fluss.rpc.messages.ListRebalanceProcessRequest; +import org.apache.fluss.rpc.messages.ListRebalanceProcessResponse; import org.apache.fluss.rpc.messages.ListTablesRequest; import org.apache.fluss.rpc.messages.ListTablesResponse; import org.apache.fluss.rpc.messages.MetadataRequest; import org.apache.fluss.rpc.messages.MetadataResponse; +import org.apache.fluss.rpc.messages.RebalanceRequest; +import org.apache.fluss.rpc.messages.RebalanceResponse; +import org.apache.fluss.rpc.messages.RemoveServerTagRequest; +import org.apache.fluss.rpc.messages.RemoveServerTagResponse; import org.apache.fluss.rpc.messages.TableExistsRequest; import org.apache.fluss.rpc.messages.TableExistsResponse; import org.apache.fluss.rpc.protocol.ApiError; @@ -332,6 +342,34 @@ public CompletableFuture lakeTieringHeartbeat( throw new UnsupportedOperationException(); } + @Override + public CompletableFuture addServerTag(AddServerTagRequest request) { + throw new UnsupportedOperationException(); + } + + @Override + public CompletableFuture removeServerTag( + RemoveServerTagRequest request) { + throw new UnsupportedOperationException(); + } + + @Override + public CompletableFuture rebalance(RebalanceRequest request) { + throw new UnsupportedOperationException(); + } + + @Override + public CompletableFuture listRebalanceProcess( + ListRebalanceProcessRequest request) { + throw new UnsupportedOperationException(); + } + + @Override + public CompletableFuture cancelRebalance( + CancelRebalanceRequest request) { + throw new UnsupportedOperationException(); + } + @Override public CompletableFuture controlledShutdown( ControlledShutdownRequest request) { diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/RebalanceManagerITCase.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/RebalanceManagerITCase.java new file mode 100644 index 0000000000..88c3c59dec --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/RebalanceManagerITCase.java @@ -0,0 +1,112 @@ +/* + * 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.fluss.server.coordinator.rebalance; + +import org.apache.fluss.cluster.rebalance.ServerTag; +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.metadata.PartitionSpec; +import org.apache.fluss.metadata.TableDescriptor; +import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.rpc.messages.AddServerTagRequest; +import org.apache.fluss.server.coordinator.rebalance.model.ClusterModel; +import org.apache.fluss.server.testutils.FlussClusterExtension; +import org.apache.fluss.server.zk.ZooKeeperClient; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +import java.util.Collections; + +import static org.apache.fluss.record.TestData.DATA1_SCHEMA; +import static org.apache.fluss.record.TestData.DATA1_TABLE_DESCRIPTOR; +import static org.apache.fluss.record.TestData.DATA1_TABLE_PATH; +import static org.apache.fluss.server.testutils.RpcMessageTestUtils.createPartition; +import static org.apache.fluss.server.testutils.RpcMessageTestUtils.createTable; +import static org.assertj.core.api.Assertions.assertThat; + +/** IT test for {@link RebalanceManager}. */ +public class RebalanceManagerITCase { + + @RegisterExtension + public static final FlussClusterExtension FLUSS_CLUSTER_EXTENSION = + FlussClusterExtension.builder() + .setNumOfTabletServers(3) + .setClusterConf(initConfig()) + .build(); + + private ZooKeeperClient zkClient; + private RebalanceManager rebalanceManager; + + @BeforeEach + void beforeEach() { + zkClient = FLUSS_CLUSTER_EXTENSION.getZooKeeperClient(); + rebalanceManager = FLUSS_CLUSTER_EXTENSION.getRebalanceManager(); + } + + @Test + void testBuildClusterModel() throws Exception { + // one none-partitioned table. + long tableId1 = + createTable(FLUSS_CLUSTER_EXTENSION, DATA1_TABLE_PATH, DATA1_TABLE_DESCRIPTOR); + // one partitioned table. + TablePath partitionTablePath = TablePath.of("test_db_1", "test_partition_table_1"); + TableDescriptor partitionTableDescriptor = + TableDescriptor.builder() + .schema(DATA1_SCHEMA) + .distributedBy(3) + .partitionedBy("b") + .property(ConfigOptions.TABLE_AUTO_PARTITION_ENABLED, false) + .build(); + long tableId2 = + createTable(FLUSS_CLUSTER_EXTENSION, partitionTablePath, partitionTableDescriptor); + String partitionName1 = "b1"; + createPartition( + FLUSS_CLUSTER_EXTENSION, + partitionTablePath, + new PartitionSpec(Collections.singletonMap("b", partitionName1)), + false); + + ClusterModel clusterModel = rebalanceManager.getClusterModel(); + assertThat(clusterModel.servers().size()).isEqualTo(3); + assertThat(clusterModel.aliveServers().size()).isEqualTo(3); + assertThat(clusterModel.offlineServers().size()).isEqualTo(0); + assertThat(clusterModel.tables().size()).isEqualTo(2); + assertThat(clusterModel.tables()).contains(tableId1, tableId2); + + // offline one table. + AddServerTagRequest request = + new AddServerTagRequest().setServerTag(ServerTag.PERMANENT_OFFLINE.value); + request.addServerId(0); + FLUSS_CLUSTER_EXTENSION.newCoordinatorClient().addServerTag(request).get(); + + clusterModel = rebalanceManager.getClusterModel(); + assertThat(clusterModel.servers().size()).isEqualTo(3); + assertThat(clusterModel.aliveServers().size()).isEqualTo(2); + assertThat(clusterModel.offlineServers().size()).isEqualTo(1); + assertThat(clusterModel.tables().size()).isEqualTo(2); + assertThat(clusterModel.tables()).contains(tableId1, tableId2); + } + + private static Configuration initConfig() { + Configuration conf = new Configuration(); + conf.setInt(ConfigOptions.DEFAULT_REPLICATION_FACTOR, 3); + return conf; + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/RebalanceTestUtils.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/RebalanceTestUtils.java new file mode 100644 index 0000000000..5942043b02 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/RebalanceTestUtils.java @@ -0,0 +1,34 @@ +/* + * 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.fluss.server.coordinator.rebalance; + +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.server.coordinator.rebalance.model.ClusterModel; + +import java.util.List; + +/** A util class for rebalance test. */ +public class RebalanceTestUtils { + + public static void addBucket( + ClusterModel clusterModel, TableBucket tb, List replicas) { + for (int i = 0; i < replicas.size(); i++) { + clusterModel.createReplica(replicas.get(i), tb, i, i == 0); + } + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/goal/GoalOptimizerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/goal/GoalOptimizerTest.java new file mode 100644 index 0000000000..ef85d08071 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/goal/GoalOptimizerTest.java @@ -0,0 +1,48 @@ +/* + * 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.fluss.server.coordinator.rebalance.goal; + +import org.apache.fluss.server.coordinator.rebalance.model.ServerModel; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.SortedSet; +import java.util.TreeSet; + +/** Test for {@link GoalOptimizer}. */ +public class GoalOptimizerTest { + + private SortedSet servers; + + @BeforeEach + public void setup() { + servers = new TreeSet<>(); + ServerModel server0 = new ServerModel(0, "rack0", true); + ServerModel server1 = new ServerModel(1, "rack1", true); + ServerModel server2 = new ServerModel(2, "rack2", true); + ServerModel server3 = new ServerModel(3, "rack0", true); + servers.add(server0); + servers.add(server1); + servers.add(server2); + servers.add(server3); + } + + @Test + void testOptimize() {} +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/goal/GoalOptimizerUtilsTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/goal/GoalOptimizerUtilsTest.java new file mode 100644 index 0000000000..ebf9e70975 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/goal/GoalOptimizerUtilsTest.java @@ -0,0 +1,120 @@ +/* + * 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.fluss.server.coordinator.rebalance.goal; + +import org.apache.fluss.cluster.rebalance.RebalancePlanForBucket; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.server.coordinator.rebalance.model.ClusterModel; +import org.apache.fluss.server.coordinator.rebalance.model.ServerModel; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.SortedSet; +import java.util.TreeSet; + +import static org.apache.fluss.server.coordinator.rebalance.RebalanceTestUtils.addBucket; +import static org.apache.fluss.server.coordinator.rebalance.goal.GoalOptimizerUtils.getDiff; +import static org.apache.fluss.server.coordinator.rebalance.goal.GoalOptimizerUtils.hasDiff; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** A test class for {@link GoalOptimizerUtils}. */ +public class GoalOptimizerUtilsTest { + + private SortedSet servers; + + @BeforeEach + public void setup() { + servers = new TreeSet<>(); + ServerModel server0 = new ServerModel(0, "rack0", true); + ServerModel server1 = new ServerModel(1, "rack1", true); + ServerModel server2 = new ServerModel(2, "rack2", true); + ServerModel server3 = new ServerModel(3, "rack0", true); + servers.add(server0); + servers.add(server1); + servers.add(server2); + servers.add(server3); + } + + @Test + void testHasDiff() { + ClusterModel clusterModel = new ClusterModel(servers); + + // add buckets into clusterModel. + addBucket(clusterModel, new TableBucket(0, 0), Arrays.asList(0, 1, 2)); + addBucket(clusterModel, new TableBucket(1, 0), Arrays.asList(0, 1, 2)); + + Map> initialReplicaDistribution = + clusterModel.getReplicaDistribution(); + Map initialLeaderDistribution = clusterModel.getLeaderDistribution(); + assertThat(hasDiff(initialReplicaDistribution, initialLeaderDistribution, clusterModel)) + .isFalse(); + + clusterModel.relocateLeadership(new TableBucket(0, 0), 0, 1); + clusterModel.relocateReplica(new TableBucket(1, 0), 2, 3); + assertThat(hasDiff(initialReplicaDistribution, initialLeaderDistribution, clusterModel)) + .isTrue(); + + assertThatThrownBy(() -> hasDiff(new HashMap<>(), initialLeaderDistribution, clusterModel)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Initial and final replica distributions do not contain the same buckets."); + } + + @Test + void testGetDiff() { + ClusterModel clusterModel = new ClusterModel(servers); + + // add buckets into clusterModel. + addBucket(clusterModel, new TableBucket(0, 0), Arrays.asList(0, 1, 2)); + addBucket(clusterModel, new TableBucket(1, 0), Arrays.asList(0, 1, 2)); + + Map> initialReplicaDistribution = + clusterModel.getReplicaDistribution(); + Map initialLeaderDistribution = clusterModel.getLeaderDistribution(); + assertThat(hasDiff(initialReplicaDistribution, initialLeaderDistribution, clusterModel)) + .isFalse(); + + clusterModel.relocateLeadership(new TableBucket(0, 0), 0, 1); + clusterModel.relocateReplica(new TableBucket(1, 0), 2, 3); + assertThat(hasDiff(initialReplicaDistribution, initialLeaderDistribution, clusterModel)) + .isTrue(); + + List diffPlan = + getDiff(initialReplicaDistribution, initialLeaderDistribution, clusterModel); + assertThat(diffPlan) + .contains( + new RebalancePlanForBucket( + new TableBucket(0, 0), + 0, + 1, + Arrays.asList(0, 1, 2), + Arrays.asList(1, 0, 2)), + new RebalancePlanForBucket( + new TableBucket(1, 0), + 0, + 0, + Arrays.asList(0, 1, 2), + Arrays.asList(0, 1, 3))); + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/goal/LeaderReplicaDistributionGoalTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/goal/LeaderReplicaDistributionGoalTest.java new file mode 100644 index 0000000000..aa1f6916b5 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/goal/LeaderReplicaDistributionGoalTest.java @@ -0,0 +1,129 @@ +/* + * 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.fluss.server.coordinator.rebalance.goal; + +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.server.coordinator.rebalance.model.ClusterModel; +import org.apache.fluss.server.coordinator.rebalance.model.ServerModel; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.SortedSet; +import java.util.TreeSet; + +import static org.apache.fluss.server.coordinator.rebalance.RebalanceTestUtils.addBucket; +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link LeaderReplicaDistributionGoal}. */ +public class LeaderReplicaDistributionGoalTest { + private SortedSet servers; + + @BeforeEach + public void setup() { + servers = new TreeSet<>(); + ServerModel server0 = new ServerModel(0, "rack0", true); + ServerModel server1 = new ServerModel(1, "rack1", true); + ServerModel server2 = new ServerModel(2, "rack2", true); + ServerModel server3 = new ServerModel(3, "rack0", true); + servers.add(server0); + servers.add(server1); + servers.add(server2); + servers.add(server3); + } + + @Test + void testDoOptimize() { + LeaderReplicaDistributionGoal goal = new LeaderReplicaDistributionGoal(); + ClusterModel clusterModel = new ClusterModel(servers); + + // before optimize: + // for 18 buckets, the assignment: 0,1 + // for 18 buckets, the assignment: 1,0 + // the leader replica ratio of servers is 18:18:0:0, the avg buckets per server is 9 + for (int i = 0; i < 18; i++) { + addBucket(clusterModel, new TableBucket(0, i), Arrays.asList(0, 1)); + addBucket(clusterModel, new TableBucket(1, i), Arrays.asList(1, 0)); + } + + Map serverIdToLeaderReplicaNumber = getServerIdToLeaderReplicaNumber(); + assertThat(serverIdToLeaderReplicaNumber.get(0)).isEqualTo(18); + assertThat(serverIdToLeaderReplicaNumber.get(1)).isEqualTo(18); + assertThat(serverIdToLeaderReplicaNumber.get(2)).isEqualTo(0); + assertThat(serverIdToLeaderReplicaNumber.get(3)).isEqualTo(0); + + goal.optimize(clusterModel, new HashSet<>()); + + serverIdToLeaderReplicaNumber = getServerIdToLeaderReplicaNumber(); + assertThat(serverIdToLeaderReplicaNumber.get(0)).isEqualTo(10); + assertThat(serverIdToLeaderReplicaNumber.get(1)).isEqualTo(8); + assertThat(serverIdToLeaderReplicaNumber.get(2)).isEqualTo(10); + assertThat(serverIdToLeaderReplicaNumber.get(3)).isEqualTo(8); + } + + @Test + void testDoOptimizeWithOfflineServer() { + ServerModel server4 = new ServerModel(4, "rack0", false); + servers.add(server4); + + LeaderReplicaDistributionGoal goal = new LeaderReplicaDistributionGoal(); + ClusterModel clusterModel = new ClusterModel(servers); + + // before optimize: + // for 18 buckets, the assignment: 0,1 + // for 18 buckets, the assignment: 1,0 + // for 4 buckets, the assignment: 4,0,1 + // the leader replica ratio of servers is 18:18:0:0:4, the avg buckets per server is 8 + for (int i = 0; i < 18; i++) { + addBucket(clusterModel, new TableBucket(0, i), Arrays.asList(0, 1)); + addBucket(clusterModel, new TableBucket(1, i), Arrays.asList(1, 0)); + } + + for (int i = 0; i < 4; i++) { + addBucket(clusterModel, new TableBucket(2, i), Arrays.asList(4, 2, 1)); + } + + Map serverIdToLeaderReplicaNumber = getServerIdToLeaderReplicaNumber(); + assertThat(serverIdToLeaderReplicaNumber.get(0)).isEqualTo(18); + assertThat(serverIdToLeaderReplicaNumber.get(1)).isEqualTo(18); + assertThat(serverIdToLeaderReplicaNumber.get(2)).isEqualTo(0); + assertThat(serverIdToLeaderReplicaNumber.get(3)).isEqualTo(0); + assertThat(serverIdToLeaderReplicaNumber.get(4)).isEqualTo(4); + + goal.optimize(clusterModel, new HashSet<>()); + + serverIdToLeaderReplicaNumber = getServerIdToLeaderReplicaNumber(); + assertThat(serverIdToLeaderReplicaNumber.get(0)).isEqualTo(9); + assertThat(serverIdToLeaderReplicaNumber.get(1)).isEqualTo(11); + assertThat(serverIdToLeaderReplicaNumber.get(2)).isEqualTo(9); + assertThat(serverIdToLeaderReplicaNumber.get(3)).isEqualTo(11); + assertThat(serverIdToLeaderReplicaNumber.get(4)).isEqualTo(0); + } + + private Map getServerIdToLeaderReplicaNumber() { + Map idToLeaderReplicaNumber = new HashMap<>(); + for (ServerModel server : servers) { + idToLeaderReplicaNumber.put(server.id(), server.leaderReplicas().size()); + } + return idToLeaderReplicaNumber; + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/goal/PreferredLeaderElectionGoalTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/goal/PreferredLeaderElectionGoalTest.java new file mode 100644 index 0000000000..b3c6bb84dc --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/goal/PreferredLeaderElectionGoalTest.java @@ -0,0 +1,131 @@ +/* + * 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.fluss.server.coordinator.rebalance.goal; + +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.server.coordinator.rebalance.model.BucketModel; +import org.apache.fluss.server.coordinator.rebalance.model.ClusterModel; +import org.apache.fluss.server.coordinator.rebalance.model.ReplicaModel; +import org.apache.fluss.server.coordinator.rebalance.model.ServerModel; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.HashSet; +import java.util.SortedSet; +import java.util.TreeSet; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link PreferredLeaderElectionGoal}. */ +public class PreferredLeaderElectionGoalTest { + private SortedSet servers; + + @BeforeEach + public void setup() { + servers = new TreeSet<>(); + ServerModel server0 = new ServerModel(0, "rack0", true); + ServerModel server1 = new ServerModel(1, "rack1", true); + ServerModel server2 = new ServerModel(2, "rack2", true); + servers.add(server0); + servers.add(server1); + servers.add(server2); + } + + @Test + void testDoOptimize() { + PreferredLeaderElectionGoal goal = new PreferredLeaderElectionGoal(); + ClusterModel clusterModel = new ClusterModel(servers); + TableBucket t1b0 = new TableBucket(1, 0); + TableBucket t1b1 = new TableBucket(1, 1); + TableBucket t1b2 = new TableBucket(1, 2); + // init clusterModel with three buckets, and the leader of two buckets is not in preferred + // leader. + // before optimize: + // t1b0: assignment: 0, 1, 2, preferred leader: 0, current leader: 0 + // t1b1: assignment: 1, 0, 2, preferred leader: 1, current leader: 2 + // t1b2: assignment: 2, 0, 1, preferred leader: 2, current leader: 0 + clusterModel.createReplica(0, t1b0, 0, true); + clusterModel.createReplica(1, t1b0, 1, false); + clusterModel.createReplica(2, t1b0, 2, false); + + clusterModel.createReplica(1, t1b1, 0, false); + clusterModel.createReplica(0, t1b1, 1, false); + clusterModel.createReplica(2, t1b1, 2, true); + + clusterModel.createReplica(2, t1b2, 0, false); + clusterModel.createReplica(0, t1b2, 1, true); + clusterModel.createReplica(1, t1b2, 2, false); + + assertThat(clusterModel.bucket(t1b0)) + .isNotNull() + .extracting(BucketModel::leader) + .isNotNull() + .extracting(ReplicaModel::server) + .extracting(ServerModel::id) + .isEqualTo(0); + + assertThat(clusterModel.bucket(t1b1)) + .isNotNull() + .extracting(BucketModel::leader) + .isNotNull() + .extracting(ReplicaModel::server) + .extracting(ServerModel::id) + .isEqualTo(2); + + assertThat(clusterModel.bucket(t1b2)) + .isNotNull() + .extracting(BucketModel::leader) + .isNotNull() + .extracting(ReplicaModel::server) + .extracting(ServerModel::id) + .isEqualTo(0); + + // do optimize. + goal.optimize(clusterModel, new HashSet<>()); + + // check optimized result: + // after optimize: + // t1b0: assignment: 0, 1, 2, preferred leader: 0, current leader: 0 + // t1b1: assignment: 1, 0, 2, preferred leader: 1, current leader: 1 + // t1b2: assignment: 2, 0, 1, preferred leader: 2, current leader: 2 + assertThat(clusterModel.bucket(t1b0)) + .isNotNull() + .extracting(BucketModel::leader) + .isNotNull() + .extracting(ReplicaModel::server) + .extracting(ServerModel::id) + .isEqualTo(0); + + assertThat(clusterModel.bucket(t1b1)) + .isNotNull() + .extracting(BucketModel::leader) + .isNotNull() + .extracting(ReplicaModel::server) + .extracting(ServerModel::id) + .isEqualTo(1); + + assertThat(clusterModel.bucket(t1b2)) + .isNotNull() + .extracting(BucketModel::leader) + .isNotNull() + .extracting(ReplicaModel::server) + .extracting(ServerModel::id) + .isEqualTo(2); + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/goal/ReplicaDistributionGoalTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/goal/ReplicaDistributionGoalTest.java new file mode 100644 index 0000000000..bafdfd1684 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/goal/ReplicaDistributionGoalTest.java @@ -0,0 +1,141 @@ +/* + * 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.fluss.server.coordinator.rebalance.goal; + +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.server.coordinator.rebalance.model.ClusterModel; +import org.apache.fluss.server.coordinator.rebalance.model.ClusterModelStats; +import org.apache.fluss.server.coordinator.rebalance.model.ServerModel; +import org.apache.fluss.server.coordinator.rebalance.model.Statistic; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.SortedSet; +import java.util.TreeSet; + +import static org.apache.fluss.server.coordinator.rebalance.RebalanceTestUtils.addBucket; +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link ReplicaDistributionGoal}. */ +public class ReplicaDistributionGoalTest { + private SortedSet servers; + + @BeforeEach + public void setup() { + servers = new TreeSet<>(); + ServerModel server0 = new ServerModel(0, "rack0", true); + ServerModel server1 = new ServerModel(1, "rack1", true); + ServerModel server2 = new ServerModel(2, "rack2", true); + ServerModel server3 = new ServerModel(3, "rack0", true); + servers.add(server0); + servers.add(server1); + servers.add(server2); + servers.add(server3); + } + + @Test + void testDoOptimize() { + ReplicaDistributionGoal goal = new ReplicaDistributionGoal(); + ClusterModel clusterModel = new ClusterModel(servers); + TableBucket t1b0 = new TableBucket(1, 0); + TableBucket t1b1 = new TableBucket(1, 1); + + // before optimize: + // t1b0: assignment: 0, 1, 3 + // t1b1: assignment: 0, 1, 2 + // for other 11 buckets, the assignment: 0,1 + // the replica ratio of servers is 13:13:1:1, the avg buckets per server is 7 + addBucket(clusterModel, t1b0, Arrays.asList(0, 1, 3)); + addBucket(clusterModel, t1b1, Arrays.asList(0, 1, 2)); + for (int i = 0; i < 11; i++) { + addBucket(clusterModel, new TableBucket(2, i), Arrays.asList(0, 1)); + } + + ClusterModelStats clusterStats = clusterModel.getClusterStats(); + Map replicaStats = clusterStats.replicaStats(); + assertThat(replicaStats.get(Statistic.AVG)).isEqualTo(7.0); + assertThat(replicaStats.get(Statistic.MIN)).isEqualTo(1); + assertThat(replicaStats.get(Statistic.MAX)).isEqualTo(13); + + Map serverIdToReplicaNumber = getServerIdToReplicaNumber(clusterModel); + assertThat(serverIdToReplicaNumber.get(0)).isEqualTo(13); + assertThat(serverIdToReplicaNumber.get(1)).isEqualTo(13); + assertThat(serverIdToReplicaNumber.get(2)).isEqualTo(1); + assertThat(serverIdToReplicaNumber.get(3)).isEqualTo(1); + + goal.optimize(clusterModel, new HashSet<>()); + + serverIdToReplicaNumber = getServerIdToReplicaNumber(clusterModel); + assertThat(serverIdToReplicaNumber.get(0)).isEqualTo(8); + assertThat(serverIdToReplicaNumber.get(1)).isEqualTo(8); + assertThat(serverIdToReplicaNumber.get(2)).isEqualTo(6); + assertThat(serverIdToReplicaNumber.get(3)).isEqualTo(6); + } + + @Test + void testDoOptimizeWithOfflineServer() { + ServerModel server4 = new ServerModel(4, "rack0", false); + servers.add(server4); + + ReplicaDistributionGoal goal = new ReplicaDistributionGoal(); + ClusterModel clusterModel = new ClusterModel(servers); + TableBucket t1b0 = new TableBucket(1, 0); + TableBucket t1b1 = new TableBucket(1, 1); + + // All replicas in server4 need to be move out. + // before optimize: + // t1b0: assignment: 0, 1, 3 + // t1b1: assignment: 0, 1, 2 + // for other 13 buckets, the assignment: 0,1,4 + // the replica ratio of servers is 15:15:1:1:13, the avg buckets per server is 9 + addBucket(clusterModel, t1b0, Arrays.asList(0, 1, 3)); + addBucket(clusterModel, t1b1, Arrays.asList(0, 1, 2)); + for (int i = 0; i < 13; i++) { + addBucket(clusterModel, new TableBucket(2, i), Arrays.asList(0, 1, 4)); + } + + Map serverIdToReplicaNumber = getServerIdToReplicaNumber(clusterModel); + assertThat(serverIdToReplicaNumber.get(0)).isEqualTo(15); + assertThat(serverIdToReplicaNumber.get(1)).isEqualTo(15); + assertThat(serverIdToReplicaNumber.get(2)).isEqualTo(1); + assertThat(serverIdToReplicaNumber.get(3)).isEqualTo(1); + assertThat(serverIdToReplicaNumber.get(4)).isEqualTo(13); + + goal.optimize(clusterModel, new HashSet<>()); + + serverIdToReplicaNumber = getServerIdToReplicaNumber(clusterModel); + assertThat(serverIdToReplicaNumber.get(0)).isEqualTo(13); + assertThat(serverIdToReplicaNumber.get(1)).isEqualTo(10); + assertThat(serverIdToReplicaNumber.get(2)).isEqualTo(12); + assertThat(serverIdToReplicaNumber.get(3)).isEqualTo(10); + assertThat(serverIdToReplicaNumber.get(4)).isEqualTo(0); + } + + private Map getServerIdToReplicaNumber(ClusterModel clusterModel) { + Map idToReplicaNumber = new HashMap<>(); + for (ServerModel server : clusterModel.servers()) { + idToReplicaNumber.put(server.id(), server.replicas().size()); + } + return idToReplicaNumber; + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/model/BucketModelTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/model/BucketModelTest.java new file mode 100644 index 0000000000..052e6a385e --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/model/BucketModelTest.java @@ -0,0 +1,84 @@ +/* + * 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.fluss.server.coordinator.rebalance.model; + +import org.apache.fluss.metadata.TableBucket; + +import org.junit.jupiter.api.Test; + +import java.util.Collections; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Test for {@link BucketModel}. */ +public class BucketModelTest { + + @Test + void testBucketModel() { + BucketModel bucketModel = + new BucketModel( + new TableBucket(1L, 0), + Collections.singleton(new ServerModel(0, "rack0", true))); + assertThat(bucketModel.tableBucket()).isEqualTo(new TableBucket(1L, 0)); + assertThat(bucketModel.leader()).isNull(); + assertThat(bucketModel.bucketServers()).isEmpty(); + assertThat(bucketModel.replicas()).isEmpty(); + assertThat(bucketModel.canAssignReplicaToServer(new ServerModel(0, "rack0", true))) + .isFalse(); + + // add a leader replica. + ReplicaModel replicaModel1 = + new ReplicaModel(new TableBucket(1L, 0), new ServerModel(1, "rack1", true), true); + bucketModel.addLeader(replicaModel1, 0); + assertThat(bucketModel.leader()).isNotNull(); + assertThat(bucketModel.leader().tableBucket()).isEqualTo(new TableBucket(1L, 0)); + + // add a leader replica again will throw exception. + assertThatThrownBy( + () -> + bucketModel.addLeader( + new ReplicaModel( + new TableBucket(1L, 0), + new ServerModel(1, "rack1", false), + true), + 0)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Bucket TableBucket{tableId=1, bucket=0} already has a leader replica " + + "ReplicaModel[TableBucket=TableBucket{tableId=1, bucket=0},isLeader=true,rack=rack1,server=1,originalServer=1]. " + + "Cannot add a new leader replica ReplicaModel[TableBucket=TableBucket{tableId=1, bucket=0},isLeader=true,rack=rack1,server=1,originalServer=1]."); + + // add a follower replica. + ReplicaModel replicaModel2 = + new ReplicaModel(new TableBucket(1L, 0), new ServerModel(2, "rack2", true), false); + bucketModel.addFollower(replicaModel2, 1); + ReplicaModel replicaModel3 = + new ReplicaModel(new TableBucket(1L, 0), new ServerModel(3, "rack3", true), false); + bucketModel.addFollower(replicaModel3, 2); + + assertThat(bucketModel.replicas()).contains(replicaModel1, replicaModel2, replicaModel3); + assertThat(bucketModel.replica(1)).isEqualTo(replicaModel1); + assertThat(bucketModel.replica(2)).isEqualTo(replicaModel2); + assertThat(bucketModel.replica(3)).isEqualTo(replicaModel3); + + // change 2 to leader. + bucketModel.relocateLeadership(replicaModel2); + assertThat(bucketModel.leader()).isEqualTo(replicaModel2); + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/model/ClusterModelTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/model/ClusterModelTest.java new file mode 100644 index 0000000000..48ebd38711 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/model/ClusterModelTest.java @@ -0,0 +1,148 @@ +/* + * 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.fluss.server.coordinator.rebalance.model; + +import org.apache.fluss.metadata.TableBucket; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.List; +import java.util.Map; +import java.util.SortedSet; +import java.util.TreeSet; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +/** Test for {@link ClusterModel}. */ +public class ClusterModelTest { + private SortedSet servers; + private ServerModel server0; + private ServerModel server1; + private ServerModel server2; + private ServerModel server3; + + @BeforeEach + public void setup() { + servers = new TreeSet<>(); + server0 = new ServerModel(0, "rack0", true); + server1 = new ServerModel(1, "rack1", true); + server2 = new ServerModel(2, "rack2", true); + server3 = new ServerModel(3, "rack0", false); + servers.add(server0); + servers.add(server1); + servers.add(server2); + servers.add(server3); + } + + @Test + void testClusterModel() { + ClusterModel clusterModel = new ClusterModel(servers); + assertThat(clusterModel.aliveServers()).containsOnly(server0, server1, server2); + assertThat(clusterModel.offlineServers()).containsOnly(server3); + assertThat(clusterModel.servers()).containsOnly(server0, server1, server2, server3); + assertThat(clusterModel.bucket(new TableBucket(1, 0))).isNull(); + assertThat(clusterModel.numReplicas()).isEqualTo(0); + assertThat(clusterModel.numLeaderReplicas()).isEqualTo(0); + assertThat(clusterModel.rack("rack0").rack()).isEqualTo("rack0"); + assertThat(clusterModel.server(0)).isEqualTo(server0); + assertThat(clusterModel.server(5)).isNull(); + + // Test create replicas. + clusterModel.createReplica(0, new TableBucket(1, 0), 0, true); + clusterModel.createReplica(1, new TableBucket(1, 0), 1, false); + clusterModel.createReplica(2, new TableBucket(1, 0), 2, false); + clusterModel.createReplica(0, new TableBucket(2, 0L, 0), 0, true); + clusterModel.createReplica(1, new TableBucket(2, 0L, 0), 1, false); + clusterModel.createReplica(1, new TableBucket(2, 1L, 0), 0, true); + + assertThat(clusterModel.numReplicas()).isEqualTo(6); + assertThat(clusterModel.numLeaderReplicas()).isEqualTo(3); + assertThat(clusterModel.tables()).containsOnly(1L, 2L); + assertThat(clusterModel.getBucketsByTable()).hasSize(2); + + // test get replica distribution. + Map> replicaDistribution = clusterModel.getReplicaDistribution(); + assertThat(replicaDistribution).hasSize(3); + assertThat(replicaDistribution.get(new TableBucket(1, 0))).contains(0, 1, 2); + assertThat(replicaDistribution.get(new TableBucket(2, 0L, 0))).contains(0, 1); + assertThat(replicaDistribution.get(new TableBucket(2, 1L, 0))).contains(1); + + // test get leader distribution. + Map leaderDistribution = clusterModel.getLeaderDistribution(); + assertThat(leaderDistribution).hasSize(3); + assertThat(leaderDistribution.get(new TableBucket(1, 0))).isEqualTo(0); + assertThat(leaderDistribution.get(new TableBucket(2, 0L, 0))).isEqualTo(0); + assertThat(leaderDistribution.get(new TableBucket(2, 1L, 0))).isEqualTo(1); + } + + @Test + void testRelocateLeadership() { + TableBucket tb0 = new TableBucket(1, 0); + ClusterModel clusterModel = new ClusterModel(servers); + clusterModel.createReplica(0, tb0, 0, true); + clusterModel.createReplica(1, tb0, 1, false); + clusterModel.createReplica(2, tb0, 2, false); + + // try to relocate leadership from server 0 to server 1 + assertThat(clusterModel.relocateLeadership(tb0, 0, 1)).isTrue(); + ReplicaModel leaderReplica = clusterModel.bucket(tb0).leader(); + assertThat(leaderReplica).isNotNull(); + assertThat(leaderReplica.server().id()).isEqualTo(1); + + // try to relocate leadership from server 0 to server 2. As 0 is not leader, this operation + // will return false. + assertThat(clusterModel.relocateLeadership(tb0, 0, 2)).isFalse(); + + assertThatThrownBy(() -> clusterModel.relocateLeadership(tb0, 1, 1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Cannot relocate leadership of bucket TableBucket{tableId=1, bucket=0} " + + "from server 1 to server 1 because the destination replica is a leader."); + + assertThatThrownBy(() -> clusterModel.relocateLeadership(tb0, 1, 5)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Requested replica 5 is not a replica of bucket TableBucket{tableId=1, bucket=0}"); + } + + @Test + void testRelocateReplica() { + TableBucket tb0 = new TableBucket(1, 0); + ClusterModel clusterModel = new ClusterModel(servers); + clusterModel.createReplica(0, tb0, 0, true); + clusterModel.createReplica(1, tb0, 1, false); + + BucketModel bucket = clusterModel.bucket(tb0); + assertThat(bucket).isNotNull(); + assertThat(bucket.replica(0)).isNotNull(); + assertThat(bucket.replica(1)).isNotNull(); + assertThatThrownBy(() -> bucket.replica(2)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Requested replica 2 is not a replica of bucket TableBucket{tableId=1, bucket=0}"); + clusterModel.relocateReplica(tb0, 1, 2); + assertThat(bucket.replica(0)).isNotNull(); + assertThat(bucket.replica(2)).isNotNull(); + assertThatThrownBy(() -> bucket.replica(1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Requested replica 1 is not a replica of bucket TableBucket{tableId=1, bucket=0}"); + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/model/RackModelTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/model/RackModelTest.java new file mode 100644 index 0000000000..e345141e3c --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/model/RackModelTest.java @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.coordinator.rebalance.model; + +import org.apache.fluss.metadata.TableBucket; + +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link RackModel}. */ +public class RackModelTest { + + @Test + void testRackModel() { + RackModel rackModel = new RackModel("rack0"); + assertThat(rackModel.rack()).isEqualTo("rack0"); + assertThat(rackModel.server(0)).isNull(); + + ServerModel serverModel = new ServerModel(0, "rack0", true); + rackModel.addServer(serverModel); + assertThat(rackModel.server(0)).isEqualTo(serverModel); + + assertThat(rackModel.removeReplica(0, new TableBucket(1L, 0))).isNull(); + + ReplicaModel replicaModel = new ReplicaModel(new TableBucket(1L, 0), serverModel, false); + rackModel.addReplica(replicaModel); + assertThat(serverModel.replica(new TableBucket(1L, 0))).isEqualTo(replicaModel); + } + + @Test + void testToString() { + RackModel rackModel = new RackModel("rack0"); + assertThat(rackModel.toString()).isEqualTo("RackModel[rack=rack0,servers=0]"); + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/model/ReplicaModelTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/model/ReplicaModelTest.java new file mode 100644 index 0000000000..da07b8de12 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/model/ReplicaModelTest.java @@ -0,0 +1,81 @@ +/* + * 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.fluss.server.coordinator.rebalance.model; + +import org.apache.fluss.metadata.TableBucket; + +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link ReplicaModel}. */ +public class ReplicaModelTest { + + @Test + void testReplicaModel() { + ReplicaModel replicaModel = + new ReplicaModel(new TableBucket(1L, 0), new ServerModel(1, "rack1", true), false); + assertThat(replicaModel.tableBucket()).isEqualTo(new TableBucket(1L, 0)); + assertThat(replicaModel.isLeader()).isFalse(); + assertThat(replicaModel.server().id()).isEqualTo(1); + assertThat(replicaModel.originalServer().id()).isEqualTo(1); + + // make this replica as leader. + replicaModel.makeLeader(); + assertThat(replicaModel.isLeader()).isTrue(); + + // make as follower again. + replicaModel.makeFollower(); + assertThat(replicaModel.isLeader()).isFalse(); + + // set server. + replicaModel.setServer(new ServerModel(2, "rack2", true)); + assertThat(replicaModel.server().id()).isEqualTo(2); + assertThat(replicaModel.originalServer().id()).isEqualTo(1); + } + + @Test + void testToString() { + ReplicaModel replicaModel = + new ReplicaModel(new TableBucket(1L, 0), new ServerModel(1, "rack1", true), false); + assertThat(replicaModel.toString()) + .isEqualTo( + "ReplicaModel[TableBucket=TableBucket{tableId=1, bucket=0},isLeader=false,rack=rack1,server=1,originalServer=1]"); + + replicaModel.makeLeader(); + replicaModel.setServer(new ServerModel(2, "rack2", true)); + assertThat(replicaModel.toString()) + .isEqualTo( + "ReplicaModel[TableBucket=TableBucket{tableId=1, bucket=0},isLeader=true,rack=rack2,server=2,originalServer=1]"); + } + + @Test + void testEquals() { + ReplicaModel replicaModel1 = + new ReplicaModel(new TableBucket(1L, 0), new ServerModel(1, "rack1", true), false); + ReplicaModel replicaModel2 = + new ReplicaModel(new TableBucket(1L, 0), new ServerModel(1, "rack1", true), false); + assertThat(replicaModel1).isEqualTo(replicaModel2); + + replicaModel1.setServer(new ServerModel(2, "rack2", true)); + assertThat(replicaModel1).isEqualTo(replicaModel2); + + replicaModel1.setLeadership(true); + assertThat(replicaModel1).isEqualTo(replicaModel2); + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/model/ServerModelTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/model/ServerModelTest.java new file mode 100644 index 0000000000..66ec87d3a7 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/model/ServerModelTest.java @@ -0,0 +1,108 @@ +/* + * 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.fluss.server.coordinator.rebalance.model; + +import org.apache.fluss.metadata.TableBucket; + +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link ServerModel}. */ +public class ServerModelTest { + + @Test + void testServerModel() { + ServerModel serverModel = new ServerModel(0, "rack0", true); + assertThat(serverModel.id()).isEqualTo(0); + assertThat(serverModel.rack()).isEqualTo("rack0"); + assertThat(serverModel.isAlive()).isTrue(); + + // put some replicas. + TableBucket t1b0 = new TableBucket(1L, 0); + TableBucket t1b1 = new TableBucket(1L, 1); + TableBucket t1b2 = new TableBucket(1L, 2); + TableBucket t2b0 = new TableBucket(2L, 0); + TableBucket t3p0b0 = new TableBucket(3L, 0L, 0); + TableBucket t3p0b1 = new TableBucket(3L, 0L, 1); + TableBucket t3p1b0 = new TableBucket(3L, 1L, 0); + serverModel.putReplica(t1b0, new ReplicaModel(t1b0, serverModel, true)); + serverModel.putReplica(t1b1, new ReplicaModel(t1b1, serverModel, false)); + serverModel.putReplica(t1b2, new ReplicaModel(t1b2, serverModel, false)); + serverModel.putReplica(t2b0, new ReplicaModel(t2b0, serverModel, true)); + serverModel.putReplica(t3p0b0, new ReplicaModel(t3p0b0, serverModel, true)); + serverModel.putReplica(t3p0b1, new ReplicaModel(t3p0b1, serverModel, false)); + serverModel.putReplica(t3p1b0, new ReplicaModel(t3p1b0, serverModel, false)); + + assertThat(serverModel.replicas()).hasSize(7); + assertThat(serverModel.leaderReplicas()).hasSize(3); + assertThat(serverModel.tables()).containsExactly(1L, 2L, 3L); + + // make t1b0 as follower and make t1b1 as leader. + assertThat(serverModel.replica(t1b0).isLeader()).isTrue(); + assertThat(serverModel.replica(t1b1).isLeader()).isFalse(); + serverModel.makeFollower(t1b0); + serverModel.makeLeader(t1b1); + assertThat(serverModel.replica(t1b0).isLeader()).isFalse(); + assertThat(serverModel.replica(t1b1).isLeader()).isTrue(); + + // make t3p0b0 as follower and make t3p0b1 as leader. + assertThat(serverModel.replica(t3p0b0).isLeader()).isTrue(); + assertThat(serverModel.replica(t3p0b1).isLeader()).isFalse(); + serverModel.makeFollower(t3p0b0); + serverModel.makeLeader(t3p0b1); + assertThat(serverModel.replica(t3p0b0).isLeader()).isFalse(); + assertThat(serverModel.replica(t3p0b1).isLeader()).isTrue(); + + // remove replica t2b0 and t3p1b0. + serverModel.removeReplica(t2b0); + serverModel.removeReplica(t3p1b0); + assertThat(serverModel.replicas()).hasSize(5); + assertThat(serverModel.leaderReplicas()).hasSize(2); + assertThat(serverModel.tables()).containsExactly(1L, 3L); + } + + @Test + void testToString() { + ServerModel serverModel = new ServerModel(0, "rack0", true); + assertThat(serverModel.toString()) + .isEqualTo("ServerModel[id=0,rack=rack0,isAlive=true,replicaCount=0]"); + + serverModel.putReplica( + new TableBucket(1L, 0), + new ReplicaModel(new TableBucket(1L, 0), serverModel, false)); + assertThat(serverModel.toString()) + .isEqualTo("ServerModel[id=0,rack=rack0,isAlive=true,replicaCount=1]"); + } + + @Test + void testEquals() { + // equals by server Id. + ServerModel serverModel1 = new ServerModel(0, "rack0", true); + ServerModel serverModel2 = new ServerModel(0, "rack0", true); + assertThat(serverModel1).isEqualTo(serverModel2); + } + + @Test + void testCompareTo() { + // order by server Id. + ServerModel serverModel1 = new ServerModel(0, "rack0", true); + ServerModel serverModel2 = new ServerModel(1, "rack1", true); + assertThat(serverModel1.compareTo(serverModel2)).isEqualTo(-1); + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaLeaderElectionAlgorithmsTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaLeaderElectionTest.java similarity index 60% rename from fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaLeaderElectionAlgorithmsTest.java rename to fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaLeaderElectionTest.java index a530a19293..d1dc77d61a 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaLeaderElectionAlgorithmsTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaLeaderElectionTest.java @@ -17,6 +17,9 @@ package org.apache.fluss.server.coordinator.statemachine; +import org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElection.ControlledShutdownLeaderElection; +import org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElection.DefaultLeaderElection; +import org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElection.ReassignmentLeaderElection; import org.apache.fluss.server.coordinator.statemachine.TableBucketStateMachine.ElectionResult; import org.apache.fluss.server.zk.data.LeaderAndIsr; @@ -29,27 +32,10 @@ import java.util.Optional; import java.util.Set; -import static org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElectionAlgorithms.controlledShutdownReplicaLeaderElection; -import static org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElectionAlgorithms.defaultReplicaLeaderElection; -import static org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElectionAlgorithms.initReplicaLeaderElection; import static org.assertj.core.api.Assertions.assertThat; -/** Test for {@link ReplicaLeaderElectionAlgorithms}. */ -public class ReplicaLeaderElectionAlgorithmsTest { - - @Test - void testInitReplicaLeaderElection() { - List assignments = Arrays.asList(2, 4); - List liveReplicas = Collections.singletonList(4); - - Optional leaderElectionResultOpt = - initReplicaLeaderElection(assignments, liveReplicas, 0); - assertThat(leaderElectionResultOpt.isPresent()).isTrue(); - ElectionResult leaderElectionResult = leaderElectionResultOpt.get(); - assertThat(leaderElectionResult.getLiveReplicas()).containsExactlyInAnyOrder(4); - assertThat(leaderElectionResult.getLeaderAndIsr().leader()).isEqualTo(4); - assertThat(leaderElectionResult.getLeaderAndIsr().isr()).containsExactlyInAnyOrder(4); - } +/** Test for different implement of {@link ReplicaLeaderElection}. */ +public class ReplicaLeaderElectionTest { @Test void testDefaultReplicaLeaderElection() { @@ -57,8 +43,9 @@ void testDefaultReplicaLeaderElection() { List liveReplicas = Arrays.asList(2, 4); LeaderAndIsr originLeaderAndIsr = new LeaderAndIsr(4, 0, Arrays.asList(2, 4), 0, 0); + DefaultLeaderElection defaultLeaderElection = new DefaultLeaderElection(); Optional leaderElectionResultOpt = - defaultReplicaLeaderElection(assignments, liveReplicas, originLeaderAndIsr); + defaultLeaderElection.leaderElection(assignments, liveReplicas, originLeaderAndIsr); assertThat(leaderElectionResultOpt.isPresent()).isTrue(); ElectionResult leaderElectionResult = leaderElectionResultOpt.get(); assertThat(leaderElectionResult.getLiveReplicas()).containsExactlyInAnyOrder(2, 4); @@ -73,8 +60,10 @@ void testControlledShutdownReplicaLeaderElection() { LeaderAndIsr originLeaderAndIsr = new LeaderAndIsr(2, 0, Arrays.asList(2, 4), 0, 0); Set shutdownTabletServers = Collections.singleton(2); + ControlledShutdownLeaderElection controlledShutdownLeaderElection = + new ControlledShutdownLeaderElection(); Optional leaderElectionResultOpt = - controlledShutdownReplicaLeaderElection( + controlledShutdownLeaderElection.leaderElection( assignments, liveReplicas, originLeaderAndIsr, shutdownTabletServers); assertThat(leaderElectionResultOpt.isPresent()).isTrue(); ElectionResult leaderElectionResult = leaderElectionResultOpt.get(); @@ -91,8 +80,10 @@ void testControlledShutdownReplicaLeaderElectionLastIsrShuttingDown() { new LeaderAndIsr(2, 0, Collections.singletonList(2), 0, 0); Set shutdownTabletServers = Collections.singleton(2); + ControlledShutdownLeaderElection controlledShutdownLeaderElection = + new ControlledShutdownLeaderElection(); Optional leaderElectionResultOpt = - controlledShutdownReplicaLeaderElection( + controlledShutdownLeaderElection.leaderElection( assignments, liveReplicas, originLeaderAndIsr, shutdownTabletServers); assertThat(leaderElectionResultOpt).isEmpty(); } @@ -104,9 +95,39 @@ void testControlledShutdownPartitionLeaderElectionAllIsrSimultaneouslyShutdown() LeaderAndIsr originLeaderAndIsr = new LeaderAndIsr(2, 0, Arrays.asList(2, 4), 0, 0); Set shutdownTabletServers = new HashSet<>(Arrays.asList(2, 4)); + ControlledShutdownLeaderElection controlledShutdownLeaderElection = + new ControlledShutdownLeaderElection(); Optional leaderElectionResultOpt = - controlledShutdownReplicaLeaderElection( + controlledShutdownLeaderElection.leaderElection( assignments, liveReplicas, originLeaderAndIsr, shutdownTabletServers); assertThat(leaderElectionResultOpt).isEmpty(); } + + @Test + void testReassignBucketLeaderElection() { + List targetReplicas = Arrays.asList(1, 2, 3); + ReassignmentLeaderElection reassignmentLeaderElection = + new ReassignmentLeaderElection(targetReplicas); + List liveReplicas = Arrays.asList(1, 2, 3); + LeaderAndIsr leaderAndIsr = new LeaderAndIsr(1, 0, Arrays.asList(1, 2, 3), 0, 0); + Optional leaderOpt = + reassignmentLeaderElection.leaderElection(liveReplicas, leaderAndIsr); + assertThat(leaderOpt).isPresent(); + assertThat(leaderOpt.get().getLeaderAndIsr().leader()).isEqualTo(1); + + targetReplicas = Arrays.asList(1, 2, 3); + reassignmentLeaderElection = new ReassignmentLeaderElection(targetReplicas); + liveReplicas = Arrays.asList(2, 3); + leaderAndIsr = new LeaderAndIsr(1, 0, Arrays.asList(2, 3), 0, 0); + leaderOpt = reassignmentLeaderElection.leaderElection(liveReplicas, leaderAndIsr); + assertThat(leaderOpt).isPresent(); + assertThat(leaderOpt.get().getLeaderAndIsr().leader()).isEqualTo(2); + + targetReplicas = Arrays.asList(1, 2, 3); + reassignmentLeaderElection = new ReassignmentLeaderElection(targetReplicas); + liveReplicas = Arrays.asList(1, 2); + leaderAndIsr = new LeaderAndIsr(2, 1, Collections.emptyList(), 0, 1); + leaderOpt = reassignmentLeaderElection.leaderElection(liveReplicas, leaderAndIsr); + assertThat(leaderOpt).isNotPresent(); + } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachineTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachineTest.java index c57c9950e5..bafb477c54 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachineTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/TableBucketStateMachineTest.java @@ -35,6 +35,8 @@ import org.apache.fluss.server.coordinator.MetadataManager; import org.apache.fluss.server.coordinator.TestCoordinatorChannelManager; import org.apache.fluss.server.coordinator.event.CoordinatorEventManager; +import org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElection.ControlledShutdownLeaderElection; +import org.apache.fluss.server.coordinator.statemachine.TableBucketStateMachine.ElectionResult; import org.apache.fluss.server.metadata.CoordinatorMetadataCache; import org.apache.fluss.server.metrics.group.TestingMetricGroups; import org.apache.fluss.server.zk.NOPErrorHandler; @@ -66,7 +68,7 @@ import static org.apache.fluss.server.coordinator.statemachine.BucketState.NonExistentBucket; import static org.apache.fluss.server.coordinator.statemachine.BucketState.OfflineBucket; import static org.apache.fluss.server.coordinator.statemachine.BucketState.OnlineBucket; -import static org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElectionStrategy.CONTROLLED_SHUTDOWN_ELECTION; +import static org.apache.fluss.server.coordinator.statemachine.TableBucketStateMachine.initReplicaLeaderElection; import static org.apache.fluss.testutils.common.CommonTestUtils.retry; import static org.assertj.core.api.Assertions.assertThat; @@ -238,6 +240,9 @@ void testStateChangeToOnline() throws Exception { coordinatorContext.putBucketState(tableBucket, OfflineBucket); coordinatorContext.setLiveTabletServers(createServers(Collections.emptyList())); tableBucketStateMachine.handleStateChange(Collections.singleton(tableBucket), OnlineBucket); + coordinatorContext.setLiveTabletServers( + CoordinatorTestUtils.createServers(Collections.emptyList())); + tableBucketStateMachine.handleStateChange(Collections.singleton(tableBucket), OnlineBucket); // the state will still be offline assertThat(coordinatorContext.getBucketState(tableBucket)).isEqualTo(OfflineBucket); @@ -376,12 +381,26 @@ void testStateChangeForTabletServerControlledShutdown() { // handle state change for controlled shutdown. tableBucketStateMachine.handleStateChange( - Collections.singleton(tb), OnlineBucket, CONTROLLED_SHUTDOWN_ELECTION); + Collections.singleton(tb), OnlineBucket, new ControlledShutdownLeaderElection()); assertThat(coordinatorContext.getBucketState(tb)).isEqualTo(OnlineBucket); assertThat(coordinatorContext.getBucketLeaderAndIsr(tb).get().leader()) .isNotEqualTo(oldLeader); } + @Test + void testInitReplicaLeaderElection() { + List assignments = Arrays.asList(2, 4); + List liveReplicas = Collections.singletonList(4); + + Optional leaderElectionResultOpt = + initReplicaLeaderElection(assignments, liveReplicas, 0); + assertThat(leaderElectionResultOpt.isPresent()).isTrue(); + ElectionResult leaderElectionResult = leaderElectionResultOpt.get(); + assertThat(leaderElectionResult.getLiveReplicas()).containsExactlyInAnyOrder(4); + assertThat(leaderElectionResult.getLeaderAndIsr().leader()).isEqualTo(4); + assertThat(leaderElectionResult.getLeaderAndIsr().isr()).containsExactlyInAnyOrder(4); + } + private TableBucketStateMachine createTableBucketStateMachine() { return new TableBucketStateMachine( coordinatorContext, coordinatorRequestBatch, zookeeperClient); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java b/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java index bab31f10b2..b63138ea8c 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/testutils/FlussClusterExtension.java @@ -47,6 +47,7 @@ import org.apache.fluss.server.coordinator.CoordinatorServer; import org.apache.fluss.server.coordinator.LakeCatalogDynamicLoader; import org.apache.fluss.server.coordinator.MetadataManager; +import org.apache.fluss.server.coordinator.rebalance.RebalanceManager; import org.apache.fluss.server.entity.NotifyLeaderAndIsrData; import org.apache.fluss.server.kv.snapshot.CompletedSnapshot; import org.apache.fluss.server.kv.snapshot.CompletedSnapshotHandle; @@ -461,6 +462,10 @@ public ZooKeeperClient getZooKeeperClient() { return zooKeeperClient; } + public RebalanceManager getRebalanceManager() { + return coordinatorServer.getRebalanceManager(); + } + public RpcClient getRpcClient() { return rpcClient; } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java index 9c15121e34..38563d8417 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/ZooKeeperClientTest.java @@ -19,6 +19,8 @@ import org.apache.fluss.cluster.Endpoint; import org.apache.fluss.cluster.TabletServerInfo; +import org.apache.fluss.cluster.rebalance.RebalancePlanForBucket; +import org.apache.fluss.cluster.rebalance.ServerTag; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; import org.apache.fluss.metadata.Schema; @@ -33,6 +35,8 @@ import org.apache.fluss.server.zk.data.CoordinatorAddress; import org.apache.fluss.server.zk.data.LeaderAndIsr; import org.apache.fluss.server.zk.data.PartitionAssignment; +import org.apache.fluss.server.zk.data.RebalancePlan; +import org.apache.fluss.server.zk.data.ServerTags; import org.apache.fluss.server.zk.data.TableAssignment; import org.apache.fluss.server.zk.data.TableRegistration; import org.apache.fluss.server.zk.data.TabletServerRegistration; @@ -547,6 +551,80 @@ void testPartition() throws Exception { assertThat(partitions).containsExactly("p2"); } + @Test + void testServerTag() throws Exception { + Map serverTags = new HashMap<>(); + serverTags.put(0, ServerTag.PERMANENT_OFFLINE); + serverTags.put(1, ServerTag.TEMPORARY_OFFLINE); + + zookeeperClient.registerServerTags(new ServerTags(serverTags)); + assertThat(zookeeperClient.getServerTags()).hasValue(new ServerTags(serverTags)); + + // update server tags. + serverTags.put(0, ServerTag.TEMPORARY_OFFLINE); + serverTags.remove(1); + zookeeperClient.registerServerTags(new ServerTags(serverTags)); + assertThat(zookeeperClient.getServerTags()).hasValue(new ServerTags(serverTags)); + + zookeeperClient.registerServerTags(new ServerTags(Collections.emptyMap())); + assertThat(zookeeperClient.getServerTags()) + .hasValue(new ServerTags(Collections.emptyMap())); + } + + @Test + void testRebalancePlan() throws Exception { + Map bucketPlan = new HashMap<>(); + bucketPlan.put( + new TableBucket(0L, 0), + new RebalancePlanForBucket( + new TableBucket(0L, 0), + 0, + 3, + Arrays.asList(0, 1, 2), + Arrays.asList(3, 4, 5))); + bucketPlan.put( + new TableBucket(0L, 1), + new RebalancePlanForBucket( + new TableBucket(0L, 1), + 1, + 1, + Arrays.asList(0, 1, 2), + Arrays.asList(1, 2, 3))); + bucketPlan.put( + new TableBucket(1L, 1L, 0), + new RebalancePlanForBucket( + new TableBucket(1L, 1L, 0), + 1, + 1, + Arrays.asList(0, 1, 2), + Arrays.asList(1, 2, 3))); + bucketPlan.put( + new TableBucket(1L, 1L, 1), + new RebalancePlanForBucket( + new TableBucket(1L, 1L, 1), + 1, + 1, + Arrays.asList(0, 1, 2), + Arrays.asList(1, 2, 3))); + zookeeperClient.registerRebalancePlan(new RebalancePlan(bucketPlan)); + assertThat(zookeeperClient.getRebalancePlan()).hasValue(new RebalancePlan(bucketPlan)); + + bucketPlan = new HashMap<>(); + bucketPlan.put( + new TableBucket(0L, 0), + new RebalancePlanForBucket( + new TableBucket(0L, 0), + 0, + 3, + Arrays.asList(0, 1, 2), + Arrays.asList(3, 4, 5))); + zookeeperClient.registerRebalancePlan(new RebalancePlan(bucketPlan)); + assertThat(zookeeperClient.getRebalancePlan()).hasValue(new RebalancePlan(bucketPlan)); + + zookeeperClient.deleteRebalancePlan(); + assertThat(zookeeperClient.getRebalancePlan()).isEmpty(); + } + @Test void testZookeeperConfigPath() throws Exception { final Configuration config = new Configuration(); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/RebalancePlanJsonSerdeTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/RebalancePlanJsonSerdeTest.java new file mode 100644 index 0000000000..64b6da43e5 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/RebalancePlanJsonSerdeTest.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.fluss.server.zk.data; + +import org.apache.fluss.cluster.rebalance.RebalancePlanForBucket; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.utils.json.JsonSerdeTestBase; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +/** Test for {@link RebalancePlanJsonSerde}. */ +public class RebalancePlanJsonSerdeTest extends JsonSerdeTestBase { + + RebalancePlanJsonSerdeTest() { + super(RebalancePlanJsonSerde.INSTANCE); + } + + @Override + protected RebalancePlan[] createObjects() { + Map bucketPlan = new HashMap<>(); + bucketPlan.put( + new TableBucket(0L, 0), + new RebalancePlanForBucket( + new TableBucket(0L, 0), + 0, + 3, + Arrays.asList(0, 1, 2), + Arrays.asList(3, 4, 5))); + bucketPlan.put( + new TableBucket(0L, 1), + new RebalancePlanForBucket( + new TableBucket(0L, 1), + 1, + 1, + Arrays.asList(0, 1, 2), + Arrays.asList(1, 2, 3))); + + bucketPlan.put( + new TableBucket(1L, 0L, 0), + new RebalancePlanForBucket( + new TableBucket(1L, 0L, 0), + 0, + 3, + Arrays.asList(0, 1, 2), + Arrays.asList(3, 4, 5))); + bucketPlan.put( + new TableBucket(1L, 0L, 1), + new RebalancePlanForBucket( + new TableBucket(1L, 0L, 1), + 1, + 1, + Arrays.asList(0, 1, 2), + Arrays.asList(1, 2, 3))); + + bucketPlan.put( + new TableBucket(1L, 1L, 0), + new RebalancePlanForBucket( + new TableBucket(1L, 1L, 0), + 0, + 3, + Arrays.asList(0, 1, 2), + Arrays.asList(3, 4, 5))); + return new RebalancePlan[] {new RebalancePlan(bucketPlan)}; + } + + @Override + protected String[] expectedJsons() { + return new String[] { + "{\"version\":1,\"rebalance_plan\":" + + "[{\"table_id\":0,\"buckets\":" + + "[{\"bucket_id\":1,\"original_leader\":1,\"new_leader\":1,\"origin_replicas\":[0,1,2],\"new_replicas\":[1,2,3]}," + + "{\"bucket_id\":0,\"original_leader\":0,\"new_leader\":3,\"origin_replicas\":[0,1,2],\"new_replicas\":[3,4,5]}]}," + + "{\"table_id\":1,\"partition_id\":0,\"buckets\":[" + + "{\"bucket_id\":0,\"original_leader\":0,\"new_leader\":3,\"origin_replicas\":[0,1,2],\"new_replicas\":[3,4,5]}," + + "{\"bucket_id\":1,\"original_leader\":1,\"new_leader\":1,\"origin_replicas\":[0,1,2],\"new_replicas\":[1,2,3]}]}," + + "{\"table_id\":1,\"partition_id\":1,\"buckets\":[" + + "{\"bucket_id\":0,\"original_leader\":0,\"new_leader\":3,\"origin_replicas\":[0,1,2],\"new_replicas\":[3,4,5]}]}]}" + }; + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/ServerTagsJsonSerdeTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/ServerTagsJsonSerdeTest.java new file mode 100644 index 0000000000..8dd4e7f454 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/ServerTagsJsonSerdeTest.java @@ -0,0 +1,50 @@ +/* + * 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.fluss.server.zk.data; + +import org.apache.fluss.cluster.rebalance.ServerTag; +import org.apache.fluss.utils.json.JsonSerdeTestBase; + +import java.util.HashMap; +import java.util.Map; + +/** Test for {@link ServerTagsJsonSerde}. */ +public class ServerTagsJsonSerdeTest extends JsonSerdeTestBase { + + ServerTagsJsonSerdeTest() { + super(ServerTagsJsonSerde.INSTANCE); + } + + @Override + protected ServerTags[] createObjects() { + Map serverTags = new HashMap<>(); + serverTags.put(0, ServerTag.PERMANENT_OFFLINE); + serverTags.put(1, ServerTag.TEMPORARY_OFFLINE); + + Map serverTags2 = new HashMap<>(); + + return new ServerTags[] {new ServerTags(serverTags), new ServerTags(serverTags2)}; + } + + protected String[] expectedJsons() { + return new String[] { + "{\"version\":1,\"server_tags\":{\"0\":0,\"1\":1}}", + "{\"version\":1,\"server_tags\":{}}" + }; + } +}