From 60efaf5d17ec345c573c23d2b0e500bbdadcf154 Mon Sep 17 00:00:00 2001 From: yunhong <337361684@qq.com> Date: Mon, 21 Jul 2025 19:52:38 +0800 Subject: [PATCH 1/5] [server] Introduce new rebalance API --- .../org/apache/fluss/client/admin/Admin.java | 97 +++++++++++ .../apache/fluss/client/admin/FlussAdmin.java | 31 ++++ .../admin/RebalanceResultForBucket.java | 69 ++++++++ .../fluss/cluster/maintencance/GoalType.java | 51 ++++++ .../maintencance/RebalancePlanForBucket.java | 107 ++++++++++++ .../cluster/maintencance/RebalanceStatus.java | 51 ++++++ .../RebalanceStatusForBucket.java | 48 ++++++ .../fluss/cluster/maintencance/ServerTag.java | 52 ++++++ .../apache/fluss/config/ConfigOptions.java | 9 + .../NoRebalanceInProgressException.java | 34 ++++ .../exception/RebalanceFailureException.java | 34 ++++ .../exception/ServerNotExistException.java | 34 ++++ .../ServerTagAlreadyExistException.java | 34 ++++ .../exception/ServerTagNotExistException.java | 35 ++++ .../fluss/rpc/gateway/AdminGateway.java | 26 +++ .../apache/fluss/rpc/protocol/ApiKeys.java | 7 +- .../org/apache/fluss/rpc/protocol/Errors.java | 14 +- fluss-rpc/src/main/proto/FlussApi.proto | 75 ++++++++ .../coordinator/CoordinatorService.java | 38 ++++ .../fluss/server/zk/ZooKeeperClient.java | 47 +++++ .../fluss/server/zk/data/RebalancePlan.java | 121 +++++++++++++ .../zk/data/RebalancePlanJsonSerde.java | 162 ++++++++++++++++++ .../fluss/server/zk/data/ServerTags.java | 65 +++++++ .../server/zk/data/ServerTagsJsonSerde.java | 67 ++++++++ .../apache/fluss/server/zk/data/ZkData.java | 42 +++++ .../coordinator/TestCoordinatorGateway.java | 38 ++++ .../fluss/server/zk/ZooKeeperClientTest.java | 63 +++++++ .../zk/data/RebalancePlanJsonSerdeTest.java | 78 +++++++++ .../zk/data/ServerTagsJsonSerdeTest.java | 50 ++++++ 29 files changed, 1577 insertions(+), 2 deletions(-) create mode 100644 fluss-client/src/main/java/org/apache/fluss/client/admin/RebalanceResultForBucket.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/GoalType.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/RebalancePlanForBucket.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/RebalanceStatus.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/RebalanceStatusForBucket.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/ServerTag.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/exception/NoRebalanceInProgressException.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/exception/RebalanceFailureException.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/exception/ServerNotExistException.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/exception/ServerTagAlreadyExistException.java create mode 100644 fluss-common/src/main/java/org/apache/fluss/exception/ServerTagNotExistException.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/zk/data/RebalancePlan.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/zk/data/RebalancePlanJsonSerde.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/zk/data/ServerTags.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/zk/data/ServerTagsJsonSerde.java create mode 100644 fluss-server/src/test/java/org/apache/fluss/server/zk/data/RebalancePlanJsonSerdeTest.java create mode 100644 fluss-server/src/test/java/org/apache/fluss/server/zk/data/ServerTagsJsonSerdeTest.java 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 baaab3507b..238fd4bfcb 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,7 +22,11 @@ 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.maintencance.GoalType; +import org.apache.fluss.cluster.maintencance.RebalancePlanForBucket; +import org.apache.fluss.cluster.maintencance.ServerTag; import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.exception.AuthorizationException; import org.apache.fluss.exception.DatabaseAlreadyExistException; import org.apache.fluss.exception.DatabaseNotEmptyException; import org.apache.fluss.exception.DatabaseNotExistException; @@ -32,10 +36,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; @@ -53,9 +62,11 @@ import org.apache.fluss.metadata.TablePath; import org.apache.fluss.security.acl.AclBinding; import org.apache.fluss.security.acl.AclBindingFilter; +import org.apache.fluss.shaded.netty4.io.netty.util.concurrent.CompleteFuture; import java.util.Collection; import java.util.List; +import java.util.Map; import java.util.concurrent.CompletableFuture; /** @@ -452,4 +463,90 @@ ListOffsetsResult listOffsets( * @return A CompletableFuture indicating completion of the operation. */ DropAclsResult dropAcls(Collection filters); + + /** + * 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. + */ + CompleteFuture> 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. + */ + CompleteFuture> 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 0f9c007aa8..5a5e136f83 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,9 @@ import org.apache.fluss.client.utils.ClientRpcMessageUtils; import org.apache.fluss.cluster.Cluster; import org.apache.fluss.cluster.ServerNode; +import org.apache.fluss.cluster.maintencance.GoalType; +import org.apache.fluss.cluster.maintencance.RebalancePlanForBucket; +import org.apache.fluss.cluster.maintencance.ServerTag; import org.apache.fluss.exception.LeaderNotAvailableException; import org.apache.fluss.metadata.DatabaseDescriptor; import org.apache.fluss.metadata.DatabaseInfo; @@ -70,6 +73,7 @@ import org.apache.fluss.rpc.protocol.ApiError; import org.apache.fluss.security.acl.AclBinding; import org.apache.fluss.security.acl.AclBindingFilter; +import org.apache.fluss.shaded.netty4.io.netty.util.concurrent.CompleteFuture; import org.apache.fluss.utils.MapUtils; import javax.annotation.Nullable; @@ -464,6 +468,33 @@ public DropAclsResult dropAcls(Collection filters) { return result; } + @Override + public CompletableFuture addServerTag(List tabletServers, ServerTag serverTag) { + throw new UnsupportedOperationException("Support soon"); + } + + @Override + public CompletableFuture removeServerTag( + List tabletServers, ServerTag serverTag) { + throw new UnsupportedOperationException("Support soon"); + } + + @Override + public CompleteFuture> rebalance( + List priorityGoals, boolean dryRun) { + throw new UnsupportedOperationException("Support soon"); + } + + @Override + public CompleteFuture> listRebalanceProcess() { + throw new UnsupportedOperationException("Support soon"); + } + + @Override + public CompletableFuture cancelRebalance() { + throw new UnsupportedOperationException("Support soon"); + } + @Override public void close() { // nothing to do yet diff --git a/fluss-client/src/main/java/org/apache/fluss/client/admin/RebalanceResultForBucket.java b/fluss-client/src/main/java/org/apache/fluss/client/admin/RebalanceResultForBucket.java new file mode 100644 index 0000000000..172cb39dd3 --- /dev/null +++ b/fluss-client/src/main/java/org/apache/fluss/client/admin/RebalanceResultForBucket.java @@ -0,0 +1,69 @@ +/* + * 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.annotation.PublicEvolving; +import org.apache.fluss.cluster.maintencance.RebalanceStatusForBucket; + +import java.util.List; + +/** + * Result of rebalance process for a tabletBucket. + * + * @since 0.8 + */ +@PublicEvolving +public class RebalanceResultForBucket { + + private final List originReplicas; + private final List newReplicas; + private final RebalanceStatusForBucket rebalanceStatusForBucket; + + public RebalanceResultForBucket( + List originReplicas, + List newReplicas, + RebalanceStatusForBucket rebalanceStatusForBucket) { + this.originReplicas = originReplicas; + this.newReplicas = newReplicas; + this.rebalanceStatusForBucket = rebalanceStatusForBucket; + } + + public List replicas() { + return originReplicas; + } + + public List newReplicas() { + return newReplicas; + } + + public RebalanceStatusForBucket rebalanceStatus() { + return rebalanceStatusForBucket; + } + + @Override + public String toString() { + return "BucketReassignment{" + + "replicas=" + + originReplicas + + ", newReplicas=" + + newReplicas + + ", rebalanceStatus=" + + rebalanceStatusForBucket + + '}'; + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/GoalType.java b/fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/GoalType.java new file mode 100644 index 0000000000..4be91b31f6 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/GoalType.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.cluster.maintencance; + +import org.apache.fluss.annotation.PublicEvolving; + +import java.util.Arrays; + +/** + * The type of goal to optimize. + * + * @since 0.8 + */ +@PublicEvolving +public enum GoalType { + REPLICA_DISTRIBUTION_GOAL(0), + PREFERRED_LEADER_GOAL(1); + + 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 == 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/maintencance/RebalancePlanForBucket.java b/fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/RebalancePlanForBucket.java new file mode 100644 index 0000000000..85b16c9a97 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/RebalancePlanForBucket.java @@ -0,0 +1,107 @@ +/* + * 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.maintencance; + +import org.apache.fluss.annotation.PublicEvolving; + +import java.util.List; +import java.util.Objects; + +/** + * a Generated rebalance plan for a tableBucket. + * + * @since 0.8 + */ +@PublicEvolving +public class RebalancePlanForBucket { + private final int bucketId; + private final int originalLeader; + private final int newLeader; + private final List originReplicas; + private final List newReplicas; + + public RebalancePlanForBucket( + int bucketId, + int originalLeader, + int newLeader, + List originReplicas, + List newReplicas) { + this.bucketId = bucketId; + this.originalLeader = originalLeader; + this.newLeader = newLeader; + this.originReplicas = originReplicas; + this.newReplicas = newReplicas; + } + + public int getBucketId() { + return bucketId; + } + + public Integer getOriginalLeader() { + return originalLeader; + } + + public Integer getNewLeader() { + return newLeader; + } + + public List getOriginReplicas() { + return originReplicas; + } + + public List getNewReplicas() { + return newReplicas; + } + + @Override + public String toString() { + return "RebalancePlanForBucket{" + + "bucketId=" + + bucketId + + ", 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 bucketId == that.bucketId + && originalLeader == that.originalLeader + && newLeader == that.newLeader + && Objects.equals(originReplicas, that.originReplicas) + && Objects.equals(newReplicas, that.newReplicas); + } + + @Override + public int hashCode() { + return Objects.hash(bucketId, originalLeader, newLeader, originReplicas, newReplicas); + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/RebalanceStatus.java b/fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/RebalanceStatus.java new file mode 100644 index 0000000000..ba67043dce --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/RebalanceStatus.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.cluster.maintencance; + +import org.apache.fluss.annotation.PublicEvolving; + +/** + * Rebalance status for the rebalance task. + * + * @since 0.8 + */ +@PublicEvolving +public enum RebalanceStatus { + IN_PROGRESS(1), + COMPLETED(2), + FAILED(3); + + private final int code; + + RebalanceStatus(int code) { + this.code = code; + } + + public int code() { + return code; + } + + public static RebalanceStatus of(int code) { + for (RebalanceStatus status : RebalanceStatus.values()) { + if (status.code == code) { + return status; + } + } + return null; + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/RebalanceStatusForBucket.java b/fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/RebalanceStatusForBucket.java new file mode 100644 index 0000000000..cbbed5db27 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/RebalanceStatusForBucket.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.cluster.maintencance; + +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 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/maintencance/ServerTag.java b/fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/ServerTag.java new file mode 100644 index 0000000000..3a4cd3e934 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/ServerTag.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.maintencance; + +import org.apache.fluss.annotation.PublicEvolving; + +import java.util.Arrays; + +/** + * The tag of tabletServer. + * + * @since 0.8 + */ +@PublicEvolving +public enum ServerTag { + PERMANENT_OFFLINE(0), + 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 99e99405a7..ec051e8836 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 @@ -228,6 +228,15 @@ public class ConfigOptions { public static final ConfigOption> SERVER_SASL_ENABLED_MECHANISMS_CONFIG = key("security.sasl.enabled.mechanisms").stringType().asList().noDefaultValue(); + public static final ConfigOption AUTO_BALANCER_ENABLED = + key("auto-balancer.enabled") + .booleanType() + .defaultValue(false) + .withDescription( + "Specifies whether to enable the auto balancer feature. " + + "If enabled, the auto balancer will automatically balance the load of " + + "the tabletServers."); + // ------------------------------------------------------------------------ // ConfigOptions for Coordinator Server // ------------------------------------------------------------------------ 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..9607c3fd9f --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/exception/RebalanceFailureException.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.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); + } +} 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 b8cbd2512c..f2d80fc143 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,6 +17,10 @@ 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.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; @@ -33,6 +37,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; @@ -104,6 +114,22 @@ public interface AdminGateway extends AdminReadOnlyGateway { @RPC(api = ApiKeys.DROP_ACLS) CompletableFuture dropAcls(DropAclsRequest 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 0bc2d494a8..a808fad28d 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 @@ -70,7 +70,12 @@ public enum ApiKeys { CREATE_ACLS(1039, 0, 0, PUBLIC), LIST_ACLS(1040, 0, 0, PUBLIC), DROP_ACLS(1041, 0, 0, PUBLIC), - LAKE_TIERING_HEARTBEAT(1042, 0, 0, PRIVATE); + LAKE_TIERING_HEARTBEAT(1042, 0, 0, PRIVATE), + ADD_SERVER_TAG(1043, 0, 0, PUBLIC), + REMOVE_SERVER_TAG(1044, 0, 0, PUBLIC), + REBALANCE(1045, 0, 0, PUBLIC), + LIST_REBALANCE_PROCESS(1046, 0, 0, PUBLIC), + CANCEL_REBALANCE(1047, 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 5edb3ac833..4065fbec40 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 @@ -48,6 +48,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; @@ -56,11 +57,14 @@ 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.StorageException; import org.apache.fluss.exception.TableAlreadyExistException; import org.apache.fluss.exception.TableNotExistException; @@ -214,7 +218,15 @@ public enum Errors { INVALID_SERVER_RACK_INFO_EXCEPTION( 52, "The server rack info is invalid.", InvalidServerRackInfoException::new), LAKE_SNAPSHOT_NOT_EXIST( - 53, "The lake snapshot is not exist.", LakeTableSnapshotNotExistException::new); + 53, "The lake snapshot is not exist.", LakeTableSnapshotNotExistException::new), + SERVER_NOT_EXIST_EXCEPTION(54, "The server is not exist.", ServerNotExistException::new), + SEVER_TAG_ALREADY_EXIST_EXCEPTION( + 55, "The server tag already exist.", ServerTagAlreadyExistException::new), + SEVER_TAG_NOT_EXIST_EXCEPTION( + 56, "The server tag not exist.", ServerTagAlreadyExistException::new), + REBALANCE_FAILURE_EXCEPTION(57, "The rebalance task failure.", RebalanceFailureException::new), + NO_REBALANCE_IN_PROGRESS_EXCEPTION( + 58, "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 2ddaa44e8a..62afeaf453 100644 --- a/fluss-rpc/src/main/proto/FlussApi.proto +++ b/fluss-rpc/src/main/proto/FlussApi.proto @@ -530,6 +530,43 @@ message LakeTieringHeartbeatResponse { repeated PbHeartbeatRespForTable failed_table_resp = 5; } +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 ---------------- message PbApiVersion { @@ -867,4 +904,42 @@ message PbHeartbeatReqForTable { message PbHeartbeatRespForTable { required int64 table_id = 1; optional ErrorResponse error = 2; +} + +message PbRebalancePlanForTable { + required PbTablePath table_path = 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 string partition_name = 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 PbTablePath table_path = 1; + repeated PbRebalanceProcessForPartition partitions_process = 2; + repeated PbRebalanceProcessForBucket buckets_process = 3; +} + +message PbRebalanceProcessForPartition { + required string partition_name = 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/CoordinatorService.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorService.java index 7253ef9626..67c254b953 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 @@ -40,8 +40,12 @@ import org.apache.fluss.metadata.TablePartition; 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.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; @@ -66,10 +70,16 @@ 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.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; @@ -567,6 +577,34 @@ public CompletableFuture lakeTieringHeartbeat( return CompletableFuture.completedFuture(heartbeatResponse); } + @Override + public CompletableFuture addServerTag(AddServerTagRequest request) { + throw new UnsupportedOperationException("Support soon!"); + } + + @Override + public CompletableFuture removeServerTag( + RemoveServerTagRequest request) { + throw new UnsupportedOperationException("Support soon!"); + } + + @Override + public CompletableFuture rebalance(RebalanceRequest request) { + throw new UnsupportedOperationException("Support soon!"); + } + + @Override + public CompletableFuture listRebalanceProcess( + ListRebalanceProcessRequest request) { + throw new UnsupportedOperationException("Support soon!"); + } + + @Override + public CompletableFuture cancelRebalance( + CancelRebalanceRequest request) { + throw new UnsupportedOperationException("Support soon!"); + } + private void validateHeartbeatRequest( PbHeartbeatReqForTable heartbeatReqForTable, int currentEpoch) { if (heartbeatReqForTable.getCoordinatorEpoch() != currentEpoch) { 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 abdc5e1fdd..2542468628 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 @@ -35,8 +35,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; @@ -55,11 +57,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; @@ -902,6 +906,49 @@ public void insertAclChangeNotification(Resource resource) throws Exception { LOG.info("add acl change notification for resource {} ", resource); } + // -------------------------------------------------------------------------------------------- + // 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/RebalancePlan.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/RebalancePlan.java new file mode 100644 index 0000000000..6891aacf86 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/RebalancePlan.java @@ -0,0 +1,121 @@ +/* + * 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.maintencance.RebalancePlanForBucket; +import org.apache.fluss.cluster.maintencance.RebalanceStatus; +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 { + + /** The rebalance status of this rebalance plan. */ + private final RebalanceStatus rebalanceStatus; + + /** 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( + RebalanceStatus rebalanceStatus, 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); + } + } + + this.rebalanceStatus = rebalanceStatus; + } + + public Map> getPlanForBuckets() { + return planForBuckets; + } + + public Map> getPlanForBucketsOfPartitionedTable() { + return planForBucketsOfPartitionedTable; + } + + public RebalanceStatus getRebalanceStatus() { + return rebalanceStatus; + } + + @Override + public String toString() { + return "RebalancePlan{" + + "planForBuckets=" + + planForBuckets + + ", planForBucketsOfPartitionedTable=" + + planForBucketsOfPartitionedTable + + ", rebalanceStatus=" + + rebalanceStatus + + '}'; + } + + @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) + && rebalanceStatus == that.rebalanceStatus; + } + + @Override + public int hashCode() { + return Objects.hash(planForBuckets, planForBucketsOfPartitionedTable, rebalanceStatus); + } +} 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..b0183bd03d --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/RebalancePlanJsonSerde.java @@ -0,0 +1,162 @@ +/* + * 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.maintencance.RebalancePlanForBucket; +import org.apache.fluss.cluster.maintencance.RebalanceStatus; +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 REBALANCE_STATUS = "rebalance_status"; + + 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.writeNumberField(REBALANCE_STATUS, rebalancePlan.getRebalanceStatus().code()); + 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) { + + RebalanceStatus rebalanceStatus = RebalanceStatus.of(node.get(REBALANCE_STATUS).asInt()); + + 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( + bucketId, originLeader, newLeader, originReplicas, newReplicas)); + } + } + + return new RebalancePlan(rebalanceStatus, 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..c9ae5400a3 --- /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.maintencance.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..25257c8e23 --- /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.maintencance.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 9fa256d1cd..6d60675d85 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 @@ -554,6 +554,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). // ------------------------------------------------------------------------------------------ @@ -649,4 +668,27 @@ public static Resource decode(byte[] json) { } } } + + // ------------------------------------------------------------------------------------------ + // 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/TestCoordinatorGateway.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/TestCoordinatorGateway.java index 1f7509738a..a210a314cd 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 @@ -20,10 +20,14 @@ import org.apache.fluss.exception.FencedLeaderEpochException; 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.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; @@ -70,10 +74,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; @@ -296,6 +306,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 listAcls(ListAclsRequest request) { throw new UnsupportedOperationException(); 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 28d33f70d0..f511eb9ed5 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,9 @@ import org.apache.fluss.cluster.Endpoint; import org.apache.fluss.cluster.TabletServerInfo; +import org.apache.fluss.cluster.maintencance.RebalancePlanForBucket; +import org.apache.fluss.cluster.maintencance.RebalanceStatus; +import org.apache.fluss.cluster.maintencance.ServerTag; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; import org.apache.fluss.metadata.Schema; @@ -33,6 +36,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; @@ -503,6 +508,64 @@ 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( + 0, 0, 3, Arrays.asList(0, 1, 2), Arrays.asList(3, 4, 5))); + bucketPlan.put( + new TableBucket(0L, 1), + new RebalancePlanForBucket( + 1, 1, 1, Arrays.asList(0, 1, 2), Arrays.asList(1, 2, 3))); + bucketPlan.put( + new TableBucket(1L, 1L, 0), + new RebalancePlanForBucket( + 0, 1, 1, Arrays.asList(0, 1, 2), Arrays.asList(1, 2, 3))); + bucketPlan.put( + new TableBucket(1L, 1L, 1), + new RebalancePlanForBucket( + 1, 1, 1, Arrays.asList(0, 1, 2), Arrays.asList(1, 2, 3))); + zookeeperClient.registerRebalancePlan( + new RebalancePlan(RebalanceStatus.IN_PROGRESS, bucketPlan)); + assertThat(zookeeperClient.getRebalancePlan()) + .hasValue(new RebalancePlan(RebalanceStatus.IN_PROGRESS, bucketPlan)); + + bucketPlan = new HashMap<>(); + bucketPlan.put( + new TableBucket(0L, 0), + new RebalancePlanForBucket( + 0, 0, 3, Arrays.asList(0, 1, 2), Arrays.asList(3, 4, 5))); + zookeeperClient.registerRebalancePlan( + new RebalancePlan(RebalanceStatus.IN_PROGRESS, bucketPlan)); + assertThat(zookeeperClient.getRebalancePlan()) + .hasValue(new RebalancePlan(RebalanceStatus.IN_PROGRESS, 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..01e0db58a8 --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/RebalancePlanJsonSerdeTest.java @@ -0,0 +1,78 @@ +/* + * 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.maintencance.RebalancePlanForBucket; +import org.apache.fluss.cluster.maintencance.RebalanceStatus; +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( + 0, 0, 3, Arrays.asList(0, 1, 2), Arrays.asList(3, 4, 5))); + bucketPlan.put( + new TableBucket(0L, 1), + new RebalancePlanForBucket( + 1, 1, 1, Arrays.asList(0, 1, 2), Arrays.asList(1, 2, 3))); + + bucketPlan.put( + new TableBucket(1L, 0L, 0), + new RebalancePlanForBucket( + 0, 0, 3, Arrays.asList(0, 1, 2), Arrays.asList(3, 4, 5))); + bucketPlan.put( + new TableBucket(1L, 0L, 1), + new RebalancePlanForBucket( + 1, 1, 1, Arrays.asList(0, 1, 2), Arrays.asList(1, 2, 3))); + + bucketPlan.put( + new TableBucket(1L, 1L, 0), + new RebalancePlanForBucket( + 0, 0, 3, Arrays.asList(0, 1, 2), Arrays.asList(3, 4, 5))); + return new RebalancePlan[] {new RebalancePlan(RebalanceStatus.IN_PROGRESS, bucketPlan)}; + } + + @Override + protected String[] expectedJsons() { + return new String[] { + "{\"version\":1,\"rebalance_status\":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..aaec82be9a --- /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.maintencance.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\":{}}" + }; + } +} From 0788eb16a8f497f85a567f2b074b978db8586142 Mon Sep 17 00:00:00 2001 From: yunhong <337361684@qq.com> Date: Thu, 24 Jul 2025 10:42:11 +0800 Subject: [PATCH 2/5] [server] Support AddServerTag and RemoveServerTag --- .../apache/fluss/client/admin/FlussAdmin.java | 10 +- .../fluss/client/admin/FlussAdminITCase.java | 64 ++++++++++ .../org/apache/fluss/rpc/protocol/Errors.java | 4 +- .../coordinator/CoordinatorContext.java | 25 ++++ .../CoordinatorEventProcessor.java | 109 ++++++++++++++++++ .../coordinator/CoordinatorService.java | 29 ++++- .../coordinator/event/AddServerTagEvent.java | 52 +++++++++ .../event/RemoveServerTagEvent.java | 52 +++++++++ 8 files changed, 339 insertions(+), 6 deletions(-) create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/AddServerTagEvent.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/RemoveServerTagEvent.java 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 5a5e136f83..a2c221870d 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 @@ -44,6 +44,7 @@ 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.CreateAclsRequest; import org.apache.fluss.rpc.messages.CreateDatabaseRequest; import org.apache.fluss.rpc.messages.CreateTableRequest; @@ -68,6 +69,7 @@ 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.RemoveServerTagRequest; import org.apache.fluss.rpc.messages.TableExistsRequest; import org.apache.fluss.rpc.messages.TableExistsResponse; import org.apache.fluss.rpc.protocol.ApiError; @@ -470,13 +472,17 @@ public DropAclsResult dropAcls(Collection filters) { @Override public CompletableFuture addServerTag(List tabletServers, ServerTag serverTag) { - throw new UnsupportedOperationException("Support soon"); + 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) { - throw new UnsupportedOperationException("Support soon"); + RemoveServerTagRequest request = new RemoveServerTagRequest().setServerTag(serverTag.value); + tabletServers.forEach(request::addServerId); + return gateway.removeServerTag(request).thenApply(r -> null); } @Override 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 bf45d5974a..1d0f9b8b95 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.maintencance.ServerTag; import org.apache.fluss.config.AutoPartitionTimeUnit; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; @@ -39,6 +40,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; @@ -60,6 +64,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; @@ -1050,4 +1055,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-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 4065fbec40..0bb6715937 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 @@ -65,6 +65,7 @@ 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; @@ -222,8 +223,7 @@ public enum Errors { SERVER_NOT_EXIST_EXCEPTION(54, "The server is not exist.", ServerNotExistException::new), SEVER_TAG_ALREADY_EXIST_EXCEPTION( 55, "The server tag already exist.", ServerTagAlreadyExistException::new), - SEVER_TAG_NOT_EXIST_EXCEPTION( - 56, "The server tag not exist.", ServerTagAlreadyExistException::new), + SEVER_TAG_NOT_EXIST_EXCEPTION(56, "The server tag not exist.", ServerTagNotExistException::new), REBALANCE_FAILURE_EXCEPTION(57, "The rebalance task failure.", RebalanceFailureException::new), NO_REBALANCE_IN_PROGRESS_EXCEPTION( 58, "No rebalance task in progress.", NoRebalanceInProgressException::new); 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 aa372f56a4..0c14932d04 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.maintencance.ServerTag; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableBucketReplica; @@ -101,6 +102,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; @@ -616,6 +620,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(); @@ -636,5 +660,6 @@ public void resetContext() { clearTablesState(); // clear the live tablet servers liveTabletServers.clear(); + serverTags.clear(); } } 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 ffc6192b72..57382b9ba0 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,12 +21,17 @@ import org.apache.fluss.cluster.Endpoint; import org.apache.fluss.cluster.ServerNode; import org.apache.fluss.cluster.ServerType; +import org.apache.fluss.cluster.maintencance.ServerTag; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; import org.apache.fluss.exception.FencedLeaderEpochException; import org.apache.fluss.exception.FlussRuntimeException; 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.UnknownServerException; import org.apache.fluss.exception.UnknownTableOrBucketException; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.TableBucket; @@ -34,13 +39,16 @@ 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.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; @@ -57,6 +65,7 @@ import org.apache.fluss.server.coordinator.event.FencedCoordinatorEvent; import org.apache.fluss.server.coordinator.event.NewTabletServerEvent; 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.statemachine.ReplicaStateMachine; @@ -77,6 +86,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; @@ -298,6 +308,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 List autoPartitionTables = new ArrayList<>(); List> lakeTables = new ArrayList<>(); @@ -470,6 +485,16 @@ public void process(CoordinatorEvent event) { completeFromCallable( commitLakeTableSnapshotEvent.getRespCallback(), () -> tryProcessCommitLakeTableSnapshot(commitLakeTableSnapshotEvent)); + } 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 AccessContextEvent) { AccessContextEvent accessContextEvent = (AccessContextEvent) event; processAccessContext(accessContextEvent); @@ -827,6 +852,90 @@ 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)); + + 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); + + return removeServerTagResponse; + } + private List tryProcessAdjustIsr( Map leaderAndIsrList) { // TODO verify leader epoch. 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 67c254b953..8f717cb79e 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 @@ -19,6 +19,7 @@ import org.apache.fluss.cluster.ServerType; import org.apache.fluss.cluster.TabletServerInfo; +import org.apache.fluss.cluster.maintencance.ServerTag; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; import org.apache.fluss.exception.InvalidCoordinatorException; @@ -91,11 +92,13 @@ 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.EventManager; +import org.apache.fluss.server.coordinator.event.RemoveServerTagEvent; import org.apache.fluss.server.entity.CommitKvSnapshotData; import org.apache.fluss.server.entity.LakeTieringTableInfo; import org.apache.fluss.server.kv.snapshot.CompletedSnapshot; @@ -117,12 +120,14 @@ import java.io.UncheckedIOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.function.Supplier; +import java.util.stream.Collectors; import static org.apache.fluss.rpc.util.CommonRpcMessageUtils.toAclBindingFilters; import static org.apache.fluss.rpc.util.CommonRpcMessageUtils.toAclBindings; @@ -579,13 +584,33 @@ public CompletableFuture lakeTieringHeartbeat( @Override public CompletableFuture addServerTag(AddServerTagRequest request) { - throw new UnsupportedOperationException("Support soon!"); + 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) { - throw new UnsupportedOperationException("Support soon!"); + 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 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..6e76398a12 --- /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.maintencance.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/RemoveServerTagEvent.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/RemoveServerTagEvent.java new file mode 100644 index 0000000000..1e89d58a21 --- /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.maintencance.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; + } +} From 5c5f0dafe979c2dc785cc9f7c3c310a1412a6033 Mon Sep 17 00:00:00 2001 From: yunhong <337361684@qq.com> Date: Mon, 28 Jul 2025 19:55:56 +0800 Subject: [PATCH 3/5] [server] Support generate and execute reblance plan --- .../org/apache/fluss/client/admin/Admin.java | 12 +- .../apache/fluss/client/admin/FlussAdmin.java | 21 +- .../admin/RebalanceResultForBucket.java | 2 +- .../client/utils/ClientRpcMessageUtils.java | 47 ++ .../fluss/client/admin/FlussAdminITCase.java | 2 +- .../{maintencance => rebalance}/GoalType.java | 18 +- .../RebalancePlanForBucket.java | 29 +- .../rebalance/RebalanceResultForBucket.java | 134 ++++ .../RebalanceStatus.java | 10 +- .../RebalanceStatusForBucket.java | 2 +- .../ServerTag.java | 8 +- .../exception/RebalanceFailureException.java | 4 + fluss-rpc/src/main/proto/FlussApi.proto | 8 +- .../coordinator/CoordinatorContext.java | 39 +- .../CoordinatorEventProcessor.java | 614 +++++++++++++++++- .../server/coordinator/CoordinatorServer.java | 25 +- .../coordinator/CoordinatorService.java | 52 +- .../server/coordinator/TableManager.java | 22 +- .../coordinator/event/AddServerTagEvent.java | 2 +- .../event/CancalRebalanceEvent.java | 21 + .../event/ExecuteRebalanceTaskEvent.java | 46 ++ .../event/RemoveServerTagEvent.java | 2 +- .../rebalance/ActionAcceptance.java | 37 ++ .../coordinator/rebalance/ActionType.java | 30 + .../rebalance/ReBalancingAction.java | 101 +++ .../rebalance/RebalanceManager.java | 233 +++++++ .../executor/ActionExecutorService.java | 35 + .../RebalanceActionExecutorService.java | 116 ++++ .../rebalance/goal/AbstractGoal.java | 195 ++++++ .../coordinator/rebalance/goal/Goal.java | 119 ++++ .../rebalance/goal/GoalOptimizer.java | 80 +++ .../rebalance/goal/GoalOptimizerUtils.java | 172 +++++ .../coordinator/rebalance/goal/GoalUtils.java | 81 +++ .../goal/LeaderReplicaDistributionGoal.java | 334 ++++++++++ .../goal/PreferredLeaderElectionGoal.java | 101 +++ .../goal/ReplicaDistributionAbstractGoal.java | 161 +++++ .../goal/ReplicaDistributionGoal.java | 292 +++++++++ .../rebalance/model/BucketModel.java | 131 ++++ .../rebalance/model/ClusterModel.java | 276 ++++++++ .../rebalance/model/ClusterModelStats.java | 115 ++++ .../rebalance/model/RackModel.java | 84 +++ .../rebalance/model/ReplicaModel.java | 99 +++ .../rebalance/model/ServerModel.java | 200 ++++++ .../rebalance/model/Statistic.java | 26 + .../ReplicaLeaderElectionAlgorithms.java | 26 + .../ReplicaLeaderElectionStrategy.java | 25 + .../statemachine/ReplicaStateMachine.java | 2 +- .../statemachine/TableBucketStateMachine.java | 60 +- .../fluss/server/replica/ReplicaManager.java | 5 + .../server/utils/ServerRpcMessageUtils.java | 76 +++ .../server/utils/TableAssignmentUtils.java | 16 +- .../fluss/server/zk/ZooKeeperClient.java | 12 +- .../fluss/server/zk/data/LeaderAndIsr.java | 4 + .../fluss/server/zk/data/RebalancePlan.java | 38 +- .../zk/data/RebalancePlanJsonSerde.java | 12 +- .../fluss/server/zk/data/ServerTags.java | 2 +- .../server/zk/data/ServerTagsJsonSerde.java | 2 +- .../CoordinatorEventProcessorTest.java | 182 +++++- .../rebalance/RebalanceManagerITCase.java | 112 ++++ .../rebalance/RebalanceTestUtils.java | 34 + .../rebalance/goal/GoalOptimizerTest.java | 48 ++ .../goal/GoalOptimizerUtilsTest.java | 120 ++++ .../LeaderReplicaDistributionGoalTest.java | 129 ++++ .../goal/PreferredLeaderElectionGoalTest.java | 131 ++++ .../goal/ReplicaDistributionGoalTest.java | 141 ++++ .../rebalance/model/BucketModelTest.java | 84 +++ .../rebalance/model/ClusterModelTest.java | 148 +++++ .../rebalance/model/RackModelTest.java | 51 ++ .../rebalance/model/ReplicaModelTest.java | 81 +++ .../rebalance/model/ServerModelTest.java | 108 +++ .../ReplicaLeaderElectionAlgorithmsTest.java | 84 +++ .../TableBucketStateMachineTest.java | 31 +- .../testutils/FlussClusterExtension.java | 5 + .../fluss/server/zk/ZooKeeperClientTest.java | 47 +- .../zk/data/RebalancePlanJsonSerdeTest.java | 37 +- .../zk/data/ServerTagsJsonSerdeTest.java | 2 +- 76 files changed, 5841 insertions(+), 152 deletions(-) rename fluss-common/src/main/java/org/apache/fluss/cluster/{maintencance => rebalance}/GoalType.java (70%) rename fluss-common/src/main/java/org/apache/fluss/cluster/{maintencance => rebalance}/RebalancePlanForBucket.java (80%) create mode 100644 fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/RebalanceResultForBucket.java rename fluss-common/src/main/java/org/apache/fluss/cluster/{maintencance => rebalance}/RebalanceStatus.java (89%) rename fluss-common/src/main/java/org/apache/fluss/cluster/{maintencance => rebalance}/RebalanceStatusForBucket.java (96%) rename fluss-common/src/main/java/org/apache/fluss/cluster/{maintencance => rebalance}/ServerTag.java (84%) create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/CancalRebalanceEvent.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/ExecuteRebalanceTaskEvent.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/ActionAcceptance.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/ActionType.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/ReBalancingAction.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/RebalanceManager.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/executor/ActionExecutorService.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/executor/RebalanceActionExecutorService.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/AbstractGoal.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/Goal.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/GoalOptimizer.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/GoalOptimizerUtils.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/GoalUtils.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/LeaderReplicaDistributionGoal.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/PreferredLeaderElectionGoal.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/ReplicaDistributionAbstractGoal.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/goal/ReplicaDistributionGoal.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/model/BucketModel.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/model/ClusterModel.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/model/ClusterModelStats.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/model/RackModel.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/model/ReplicaModel.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/model/ServerModel.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/model/Statistic.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaLeaderElectionStrategy.java create mode 100644 fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/RebalanceManagerITCase.java create mode 100644 fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/RebalanceTestUtils.java create mode 100644 fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/goal/GoalOptimizerTest.java create mode 100644 fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/goal/GoalOptimizerUtilsTest.java create mode 100644 fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/goal/LeaderReplicaDistributionGoalTest.java create mode 100644 fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/goal/PreferredLeaderElectionGoalTest.java create mode 100644 fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/goal/ReplicaDistributionGoalTest.java create mode 100644 fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/model/BucketModelTest.java create mode 100644 fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/model/ClusterModelTest.java create mode 100644 fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/model/RackModelTest.java create mode 100644 fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/model/ReplicaModelTest.java create mode 100644 fluss-server/src/test/java/org/apache/fluss/server/coordinator/rebalance/model/ServerModelTest.java create mode 100644 fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaLeaderElectionAlgorithmsTest.java 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 238fd4bfcb..d78ccf8763 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,10 @@ 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.maintencance.GoalType; -import org.apache.fluss.cluster.maintencance.RebalancePlanForBucket; -import org.apache.fluss.cluster.maintencance.ServerTag; +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.exception.AuthorizationException; import org.apache.fluss.exception.DatabaseAlreadyExistException; @@ -62,7 +63,6 @@ import org.apache.fluss.metadata.TablePath; import org.apache.fluss.security.acl.AclBinding; import org.apache.fluss.security.acl.AclBindingFilter; -import org.apache.fluss.shaded.netty4.io.netty.util.concurrent.CompleteFuture; import java.util.Collection; import java.util.List; @@ -523,7 +523,7 @@ ListOffsetsResult listOffsets( * it. * @return the generated rebalance plan for all the tableBuckets which need to do rebalance. */ - CompleteFuture> rebalance( + CompletableFuture> rebalance( List priorityGoals, boolean dryRun); /** @@ -537,7 +537,7 @@ CompleteFuture> rebalance( * * @return the rebalance process for all the tableBuckets doing rebalance. */ - CompleteFuture> listRebalanceProcess(); + CompletableFuture> listRebalanceProcess(); /** * Cannel the rebalance task. 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 a2c221870d..5e0bc366d1 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,9 +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.maintencance.GoalType; -import org.apache.fluss.cluster.maintencance.RebalancePlanForBucket; -import org.apache.fluss.cluster.maintencance.ServerTag; +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.exception.LeaderNotAvailableException; import org.apache.fluss.metadata.DatabaseDescriptor; import org.apache.fluss.metadata.DatabaseInfo; @@ -45,6 +46,7 @@ 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.CancelRebalanceRequest; import org.apache.fluss.rpc.messages.CreateAclsRequest; import org.apache.fluss.rpc.messages.CreateDatabaseRequest; import org.apache.fluss.rpc.messages.CreateTableRequest; @@ -69,13 +71,13 @@ 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; import org.apache.fluss.security.acl.AclBinding; import org.apache.fluss.security.acl.AclBindingFilter; -import org.apache.fluss.shaded.netty4.io.netty.util.concurrent.CompleteFuture; import org.apache.fluss.utils.MapUtils; import javax.annotation.Nullable; @@ -486,19 +488,22 @@ public CompletableFuture removeServerTag( } @Override - public CompleteFuture> rebalance( + public CompletableFuture> rebalance( List priorityGoals, boolean dryRun) { - throw new UnsupportedOperationException("Support soon"); + RebalanceRequest request = new RebalanceRequest().setDryRun(dryRun); + priorityGoals.forEach(goal -> request.addGoal(goal.value)); + return gateway.rebalance(request).thenApply(ClientRpcMessageUtils::toRebalancePlan); } @Override - public CompleteFuture> listRebalanceProcess() { + public CompletableFuture> listRebalanceProcess() { throw new UnsupportedOperationException("Support soon"); } @Override public CompletableFuture cancelRebalance() { - throw new UnsupportedOperationException("Support soon"); + CancelRebalanceRequest request = new CancelRebalanceRequest(); + return gateway.cancelRebalance(request).thenApply(r -> null); } @Override diff --git a/fluss-client/src/main/java/org/apache/fluss/client/admin/RebalanceResultForBucket.java b/fluss-client/src/main/java/org/apache/fluss/client/admin/RebalanceResultForBucket.java index 172cb39dd3..5a0f5c321c 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/admin/RebalanceResultForBucket.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/admin/RebalanceResultForBucket.java @@ -18,7 +18,7 @@ package org.apache.fluss.client.admin; import org.apache.fluss.annotation.PublicEvolving; -import org.apache.fluss.cluster.maintencance.RebalanceStatusForBucket; +import org.apache.fluss.cluster.rebalance.RebalanceStatusForBucket; import java.util.List; 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 339b2c2fdd..c2591f5442 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,7 @@ 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.fs.FsPath; import org.apache.fluss.fs.FsPathAndFileName; import org.apache.fluss.fs.token.ObtainedSecurityToken; @@ -51,10 +52,14 @@ 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.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; @@ -323,6 +328,48 @@ 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 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 1d0f9b8b95..986d69ea30 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,7 +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.maintencance.ServerTag; +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; diff --git a/fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/GoalType.java b/fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/GoalType.java similarity index 70% rename from fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/GoalType.java rename to fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/GoalType.java index 4be91b31f6..130b75cf6b 100644 --- a/fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/GoalType.java +++ b/fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/GoalType.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.fluss.cluster.maintencance; +package org.apache.fluss.cluster.rebalance; import org.apache.fluss.annotation.PublicEvolving; @@ -28,8 +28,20 @@ */ @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), - PREFERRED_LEADER_GOAL(1); + + /** + * 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; @@ -40,6 +52,8 @@ public enum GoalType { 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 { diff --git a/fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/RebalancePlanForBucket.java b/fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/RebalancePlanForBucket.java similarity index 80% rename from fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/RebalancePlanForBucket.java rename to fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/RebalancePlanForBucket.java index 85b16c9a97..13ec496be5 100644 --- a/fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/RebalancePlanForBucket.java +++ b/fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/RebalancePlanForBucket.java @@ -15,41 +15,46 @@ * limitations under the License. */ -package org.apache.fluss.cluster.maintencance; +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. + * a generated rebalance plan for a tableBucket. * * @since 0.8 */ @PublicEvolving public class RebalancePlanForBucket { - private final int bucketId; + private final TableBucket tableBucket; private final int originalLeader; private final int newLeader; private final List originReplicas; private final List newReplicas; public RebalancePlanForBucket( - int bucketId, + TableBucket tableBucket, int originalLeader, int newLeader, List originReplicas, List newReplicas) { - this.bucketId = bucketId; + this.tableBucket = tableBucket; this.originalLeader = originalLeader; this.newLeader = newLeader; this.originReplicas = originReplicas; this.newReplicas = newReplicas; } + public TableBucket getTableBucket() { + return tableBucket; + } + public int getBucketId() { - return bucketId; + return tableBucket.getBucket(); } public Integer getOriginalLeader() { @@ -68,11 +73,15 @@ public List getNewReplicas() { return newReplicas; } + public boolean isLeaderAction() { + return originalLeader != newLeader; + } + @Override public String toString() { return "RebalancePlanForBucket{" - + "bucketId=" - + bucketId + + "tableBucket=" + + tableBucket + ", originalLeader=" + originalLeader + ", newLeader=" @@ -93,7 +102,7 @@ public boolean equals(Object o) { return false; } RebalancePlanForBucket that = (RebalancePlanForBucket) o; - return bucketId == that.bucketId + return Objects.equals(tableBucket, that.tableBucket) && originalLeader == that.originalLeader && newLeader == that.newLeader && Objects.equals(originReplicas, that.originReplicas) @@ -102,6 +111,6 @@ public boolean equals(Object o) { @Override public int hashCode() { - return Objects.hash(bucketId, originalLeader, newLeader, originReplicas, newReplicas); + 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..42be9aba5a --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/RebalanceResultForBucket.java @@ -0,0 +1,134 @@ +/* + * 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 javax.annotation.Nullable; + +import java.util.Collections; +import java.util.List; + +import static org.apache.fluss.cluster.rebalance.RebalanceStatusForBucket.FAILED; + +/** + * Result of rebalance process for a tabletBucket. + * + * @since 0.8 + */ +@PublicEvolving +public class RebalanceResultForBucket { + + private final @Nullable Integer originalLeader; + private final @Nullable Integer targetLeader; + private final List originReplicas; + private final List targetReplicas; + private RebalanceStatusForBucket rebalanceStatusForBucket; + + public RebalanceResultForBucket( + @Nullable Integer originalLeader, + @Nullable Integer targetLeader, + RebalanceStatusForBucket rebalanceStatusForBucket) { + this( + originalLeader, + targetLeader, + Collections.emptyList(), + Collections.emptyList(), + rebalanceStatusForBucket); + } + + public RebalanceResultForBucket( + List originReplicas, + List targetReplicas, + RebalanceStatusForBucket rebalanceStatusForBucket) { + this(null, null, originReplicas, targetReplicas, rebalanceStatusForBucket); + } + + public RebalanceResultForBucket( + @Nullable Integer originalLeader, + @Nullable Integer targetLeader, + List originReplicas, + List targetReplicas, + RebalanceStatusForBucket rebalanceStatusForBucket) { + this.originalLeader = originalLeader; + this.targetLeader = targetLeader; + this.originReplicas = originReplicas; + this.targetReplicas = targetReplicas; + this.rebalanceStatusForBucket = rebalanceStatusForBucket; + } + + public @Nullable Integer originalLeader() { + return originalLeader; + } + + public @Nullable Integer targetLeader() { + return targetLeader; + } + + public List replicas() { + return originReplicas; + } + + public List targetReplicas() { + return targetReplicas; + } + + public RebalanceResultForBucket markFailed() { + this.rebalanceStatusForBucket = FAILED; + return this; + } + + public RebalanceResultForBucket markCompleted() { + this.rebalanceStatusForBucket = RebalanceStatusForBucket.COMPLETED; + return this; + } + + public boolean isLeaderAction() { + return originalLeader != null && targetLeader != null; + } + + public RebalanceStatusForBucket rebalanceStatus() { + return rebalanceStatusForBucket; + } + + public static RebalanceResultForBucket of( + RebalancePlanForBucket planForBucket, RebalanceStatusForBucket status) { + return new RebalanceResultForBucket( + planForBucket.getOriginalLeader(), + planForBucket.getNewLeader(), + planForBucket.getOriginReplicas(), + planForBucket.getNewReplicas(), + status); + } + + @Override + public String toString() { + return "RebalanceResultForBucket{" + + "originalLeader=" + + originalLeader + + ", targetLeader=" + + targetLeader + + ", originReplicas=" + + originReplicas + + ", targetReplicas=" + + targetReplicas + + ", rebalanceStatusForBucket=" + + rebalanceStatusForBucket + + '}'; + } +} diff --git a/fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/RebalanceStatus.java b/fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/RebalanceStatus.java similarity index 89% rename from fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/RebalanceStatus.java rename to fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/RebalanceStatus.java index ba67043dce..33d1361227 100644 --- a/fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/RebalanceStatus.java +++ b/fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/RebalanceStatus.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.fluss.cluster.maintencance; +package org.apache.fluss.cluster.rebalance; import org.apache.fluss.annotation.PublicEvolving; @@ -26,9 +26,11 @@ */ @PublicEvolving public enum RebalanceStatus { - IN_PROGRESS(1), - COMPLETED(2), - FAILED(3); + NO_ONGOING_REBALANCE(0), + PLAN_GENERATING(1), + TASK_EXECUTING(2), + COMPLETED(3), + FAILED(4); private final int code; diff --git a/fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/RebalanceStatusForBucket.java b/fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/RebalanceStatusForBucket.java similarity index 96% rename from fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/RebalanceStatusForBucket.java rename to fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/RebalanceStatusForBucket.java index cbbed5db27..e8c0e46733 100644 --- a/fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/RebalanceStatusForBucket.java +++ b/fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/RebalanceStatusForBucket.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.fluss.cluster.maintencance; +package org.apache.fluss.cluster.rebalance; import org.apache.fluss.annotation.PublicEvolving; diff --git a/fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/ServerTag.java b/fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/ServerTag.java similarity index 84% rename from fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/ServerTag.java rename to fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/ServerTag.java index 3a4cd3e934..5e20b34f73 100644 --- a/fluss-common/src/main/java/org/apache/fluss/cluster/maintencance/ServerTag.java +++ b/fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/ServerTag.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.fluss.cluster.maintencance; +package org.apache.fluss.cluster.rebalance; import org.apache.fluss.annotation.PublicEvolving; @@ -28,7 +28,13 @@ */ @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; 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 index 9607c3fd9f..0dcf260b0c 100644 --- a/fluss-common/src/main/java/org/apache/fluss/exception/RebalanceFailureException.java +++ b/fluss-common/src/main/java/org/apache/fluss/exception/RebalanceFailureException.java @@ -31,4 +31,8 @@ public class RebalanceFailureException extends ApiException { public RebalanceFailureException(String message) { super(message); } + + public RebalanceFailureException(String message, Throwable cause) { + super(message, cause); + } } diff --git a/fluss-rpc/src/main/proto/FlussApi.proto b/fluss-rpc/src/main/proto/FlussApi.proto index 62afeaf453..439c89dcb8 100644 --- a/fluss-rpc/src/main/proto/FlussApi.proto +++ b/fluss-rpc/src/main/proto/FlussApi.proto @@ -907,14 +907,14 @@ message PbHeartbeatRespForTable { } message PbRebalancePlanForTable { - required PbTablePath table_path = 1; + 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 string partition_name = 1; + required int64 partition_id = 1; repeated PbRebalancePlanForBucket buckets_plan = 2; } @@ -927,13 +927,13 @@ message PbRebalancePlanForBucket { } message PbRebalanceProcessForTable { - required PbTablePath table_path = 1; + required int64 table_id = 1; repeated PbRebalanceProcessForPartition partitions_process = 2; repeated PbRebalanceProcessForBucket buckets_process = 3; } message PbRebalanceProcessForPartition { - required string partition_name = 1; + required int64 partition_id = 1; repeated PbRebalanceProcessForBucket buckets_process = 2; } 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 0c14932d04..705ef26de2 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,7 +18,8 @@ package org.apache.fluss.server.coordinator; import org.apache.fluss.annotation.VisibleForTesting; -import org.apache.fluss.cluster.maintencance.ServerTag; +import org.apache.fluss.cluster.rebalance.RebalanceResultForBucket; +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; @@ -105,6 +106,14 @@ public class CoordinatorContext { /** A mapping from tabletServers to server tag. */ private final Map serverTags = new HashMap<>(); + /** A mapping from table bucket to rebalance status of pending and running tasks. */ + private final Map ongoingRebalanceTasks = + new HashMap<>(); + + /** A mapping from table bucket to rebalance status of failed or completed tasks. */ + private final Map finishedRebalanceTasks = + new HashMap<>(); + private ServerInfo coordinatorServerInfo = null; private int coordinatorEpoch = INITIAL_COORDINATOR_EPOCH; @@ -165,7 +174,7 @@ public Map allTables() { return tablePathById; } - public Set allBuckets() { + public Set getAllBuckets() { Set allBuckets = new HashSet<>(); for (Map.Entry>> tableAssign : tableAssignments.entrySet()) { @@ -640,6 +649,30 @@ public void removeServerTag(int serverId) { serverTags.remove(serverId); } + public void putOngoingRebalanceTask(TableBucket tableBucket, RebalanceResultForBucket status) { + ongoingRebalanceTasks.put(tableBucket, status); + } + + public RebalanceResultForBucket removeOngoingRebalanceTask(TableBucket tableBucket) { + return ongoingRebalanceTasks.remove(tableBucket); + } + + public void putFinishedRebalanceTask(TableBucket tableBucket, RebalanceResultForBucket status) { + finishedRebalanceTasks.put(tableBucket, status); + } + + public RebalanceResultForBucket getOngoingRebalanceTask(TableBucket tableBucket) { + return ongoingRebalanceTasks.get(tableBucket); + } + + public Map getOngoingRebalanceTasks() { + return ongoingRebalanceTasks; + } + + public Map getFinishedRebalanceTasks() { + return finishedRebalanceTasks; + } + private void clearTablesState() { tableAssignments.clear(); partitionAssignments.clear(); @@ -661,5 +694,7 @@ public void resetContext() { // clear the live tablet servers liveTabletServers.clear(); serverTags.clear(); + ongoingRebalanceTasks.clear(); + finishedRebalanceTasks.clear(); } } 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 57382b9ba0..b063a39791 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,13 +21,17 @@ import org.apache.fluss.cluster.Endpoint; import org.apache.fluss.cluster.ServerNode; import org.apache.fluss.cluster.ServerType; -import org.apache.fluss.cluster.maintencance.ServerTag; +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.config.ConfigOptions; import org.apache.fluss.config.Configuration; import org.apache.fluss.exception.FencedLeaderEpochException; import org.apache.fluss.exception.FlussRuntimeException; import org.apache.fluss.exception.InvalidCoordinatorException; import org.apache.fluss.exception.InvalidUpdateVersionException; +import org.apache.fluss.exception.RebalanceFailureException; import org.apache.fluss.exception.ServerNotExistException; import org.apache.fluss.exception.ServerTagAlreadyExistException; import org.apache.fluss.exception.ServerTagNotExistException; @@ -45,11 +49,13 @@ import org.apache.fluss.rpc.messages.CommitLakeTableSnapshotResponse; import org.apache.fluss.rpc.messages.CommitRemoteLogManifestResponse; import org.apache.fluss.rpc.messages.PbCommitLakeTableSnapshotRespForTable; +import org.apache.fluss.rpc.messages.RebalanceResponse; 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.CancalRebalanceEvent; import org.apache.fluss.server.coordinator.event.CommitKvSnapshotEvent; import org.apache.fluss.server.coordinator.event.CommitLakeTableSnapshotEvent; import org.apache.fluss.server.coordinator.event.CommitRemoteLogManifestEvent; @@ -62,6 +68,7 @@ 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.NotifyLeaderAndIsrResponseReceivedEvent; @@ -85,6 +92,7 @@ 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.ServerTags; import org.apache.fluss.server.zk.data.TableAssignment; @@ -105,6 +113,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; @@ -113,11 +122,15 @@ 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.DEFAULT_ELECTION; +import static org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElectionStrategy.REASSIGN_BUCKET_LEADER_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; import static org.apache.fluss.server.coordinator.statemachine.ReplicaState.ReplicaDeletionSuccessful; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.makeAdjustIsrResponse; +import static org.apache.fluss.server.utils.ServerRpcMessageUtils.makeRebalanceRespose; import static org.apache.fluss.utils.concurrent.FutureUtils.completeFromCallable; /** An implementation for {@link EventProcessor}. */ @@ -140,7 +153,6 @@ 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 CompletedSnapshotStoreManager completedSnapshotStoreManager; @@ -198,7 +210,6 @@ public CoordinatorEventProcessor( zooKeeperClient); this.autoPartitionManager = autoPartitionManager; this.lakeTableTieringManager = lakeTableTieringManager; - this.coordinatorMetricGroup = coordinatorMetricGroup; this.internalListenerName = conf.getString(ConfigOptions.INTERNAL_LISTENER_NAME); } @@ -495,6 +506,10 @@ public void process(CoordinatorEvent event) { completeFromCallable( removeServerTagEvent.getRespCallback(), () -> processRemoveServerTag(removeServerTagEvent)); + } else if (event instanceof ExecuteRebalanceTaskEvent) { + tryToProcessRegisterRebalanceTask((ExecuteRebalanceTaskEvent) event); + } else if (event instanceof CancalRebalanceEvent) { + processCancelRebalanceEvent(); } else if (event instanceof AccessContextEvent) { AccessContextEvent accessContextEvent = (AccessContextEvent) event; processAccessContext(accessContextEvent); @@ -712,9 +727,11 @@ private void onReplicaBecomeOffline(Set offlineReplicas) { // for the bucket with offline leader, we set it to offline and // then try to transmit to Online // set it to offline as the leader replica fail - tableBucketStateMachine.handleStateChange(bucketWithOfflineLeader, OfflineBucket); + tableBucketStateMachine.handleStateChange( + bucketWithOfflineLeader, OfflineBucket, DEFAULT_ELECTION); // try to change it to online again, which may trigger re-election - tableBucketStateMachine.handleStateChange(bucketWithOfflineLeader, OnlineBucket); + tableBucketStateMachine.handleStateChange( + bucketWithOfflineLeader, OnlineBucket, DEFAULT_ELECTION); // for all the offline replicas, do nothing other than set it to offline currently like // kafka, todo: but we may need to select another tablet server to put @@ -830,7 +847,8 @@ private void processDeadTabletServer(DeadTabletServerEvent deadTabletServerEvent .collect(Collectors.toSet()); // trigger offline state for all the table buckets whose current leader // is the failed tablet server - tableBucketStateMachine.handleStateChange(bucketsWithOfflineLeader, OfflineBucket); + tableBucketStateMachine.handleStateChange( + bucketsWithOfflineLeader, OfflineBucket, DEFAULT_ELECTION); // trigger online state changes for offline or new buckets tableBucketStateMachine.triggerOnlineBucketStateChange(); @@ -890,6 +908,7 @@ private AddServerTagResponse processAddServerTag(AddServerTagEvent event) { // Then update coordinatorContext. serverIds.forEach(serverId -> coordinatorContext.putServerTag(serverId, serverTag)); + LOG.info("Server tag {} added for servers {}.", serverTag, serverIds); return addServerTagResponse; } @@ -932,10 +951,464 @@ private RemoveServerTagResponse processRemoveServerTag(RemoveServerTagEvent even // Then update coordinatorContext. serverIds.forEach(coordinatorContext::removeServerTag); + LOG.info("Server tag {} removed for servers {}.", serverTag, serverIds); return removeServerTagResponse; } + /** try to finish rebalance tasks after receive notify leader and isr response. */ + private void tryToFinishRebalanceTasks(TableBucket tableBucket) { + RebalanceResultForBucket rebalanceResultForBucket = + coordinatorContext.getOngoingRebalanceTask(tableBucket); + if (rebalanceResultForBucket != null) { + ReplicaReassignment reassignment = + ReplicaReassignment.build( + coordinatorContext.getAssignment(tableBucket), + rebalanceResultForBucket.targetReplicas()); + try { + 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 check reassignment complete for table bucket {}", + tableBucket, + e); + coordinatorContext.putFinishedRebalanceTask( + tableBucket, + coordinatorContext.removeOngoingRebalanceTask(tableBucket).markFailed()); + } + + // if (rebalanceResultForBucket.isLeaderAction()) { + // List assignedReplicas = + // coordinatorContext.getAssignment(tableBucket); + // int preferredReplica = assignedReplicas.get(0); + // int currentLeader = + // + // coordinatorContext.getBucketLeaderAndIsr(tableBucket).get().leader(); + // if (currentLeader == preferredReplica) { + // coordinatorContext.putFinishedRebalanceTask( + // tableBucket, + // coordinatorContext + // .removeOngoingRebalanceTask(tableBucket) + // .markCompleted()); + // } + // } else { + // } + } + + // judge whether the rebalance task is finished + LOG.info( + "Ongoing rebalance tasks: {}.", + coordinatorContext.getOngoingRebalanceTasks().size()); + LOG.info( + "Finished rebalance tasks: {}.", + coordinatorContext.getFinishedRebalanceTasks().size()); + if (coordinatorContext.getOngoingRebalanceTasks().isEmpty()) { + coordinatorContext.getFinishedRebalanceTasks().clear(); + // zk to remove rebalance task. + try { + zooKeeperClient.deleteRebalancePlan(); + } catch (Exception e) { + LOG.error("Error when delete rebalance plan from zookeeper.", e); + } + } + } + + /** + * 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 tryToProcessRegisterRebalanceTask(ExecuteRebalanceTaskEvent event) { + LOG.info("Register rebalance task."); + CompletableFuture respCallback = event.getRespCallback(); + if (!coordinatorContext.getOngoingRebalanceTasks().isEmpty() + || !coordinatorContext.getFinishedRebalanceTasks().isEmpty()) { + respCallback.completeExceptionally( + new RebalanceFailureException( + "Rebalance task already exists. Please wait for it to finish or cancel it first.")); + } + + // first register to zookeeper. + try { + zooKeeperClient.registerRebalancePlan(new RebalancePlan(event.getRebalancePlan())); + } catch (Exception e) { + LOG.error("Error when register rebalance task to zookeeper.", e); + respCallback.completeExceptionally( + new RebalanceFailureException( + "Error when register rebalance task to zookeeper.", e)); + } + + // buckets only need to change leader like preferred leader election and leader replica + // rebalance. + Set electableBuckets = new HashSet<>(); + // buckets to do reassignments. + Map reassignments = new HashMap<>(); + Set allBuckets = coordinatorContext.getAllBuckets(); + for (Map.Entry rebalancePlan : + event.getRebalancePlan().entrySet()) { + TableBucket tableBucket = rebalancePlan.getKey(); + RebalancePlanForBucket planForBucket = rebalancePlan.getValue(); + + if (!allBuckets.contains(tableBucket)) { + LOG.warn( + "Skipping rebalance task of tableBucket {} since it doesn't exist.", + tableBucket); + coordinatorContext.putFinishedRebalanceTask( + tableBucket, + RebalanceResultForBucket.of( + planForBucket, RebalanceStatusForBucket.FAILED)); + continue; + } + + if (coordinatorContext.isTableQueuedForDeletion(tableBucket.getTableId())) { + LOG.warn( + "Skipping rebalance task of tableBucket {} since the respective " + + "tables are being deleted.", + tableBucket); + coordinatorContext.putFinishedRebalanceTask( + tableBucket, + RebalanceResultForBucket.of( + planForBucket, RebalanceStatusForBucket.FAILED)); + continue; + } + + // if (planForBucket.isLeaderAction()) { + // List assignedReplicas = + // coordinatorContext.getAssignment(tableBucket); + // int preferredReplica = assignedReplicas.get(0); + // int currentLeader = + // + // coordinatorContext.getBucketLeaderAndIsr(tableBucket).get().leader(); + // if (currentLeader != preferredReplica) { + // electableBuckets.add(tableBucket); + // coordinatorContext.putOngoingRebalanceTask( + // tableBucket, + // new RebalanceResultForBucket( + // planForBucket.getOriginalLeader(), + // planForBucket.getNewLeader(), + // RebalanceStatusForBucket.PENDING)); + // } else { + // // already finished. + // coordinatorContext.putFinishedRebalanceTask( + // tableBucket, + // RebalanceResultForBucket.of( + // planForBucket, + // RebalanceStatusForBucket.COMPLETED)); + // } + // } else { + // } + + List newReplicas = planForBucket.getNewReplicas(); + ReplicaReassignment reassignment = + ReplicaReassignment.build( + coordinatorContext.getAssignment(tableBucket), newReplicas); + if (planForBucket.isLeaderAction() && !reassignment.isBeingReassigned()) { + LOG.info("trigger leader election for tableBucket {}.", tableBucket); + electableBuckets.add(tableBucket); + coordinatorContext.putOngoingRebalanceTask( + tableBucket, + new RebalanceResultForBucket( + planForBucket.getOriginReplicas(), + planForBucket.getNewReplicas(), + RebalanceStatusForBucket.PENDING)); + continue; + } + + reassignments.put(tableBucket, reassignment); + coordinatorContext.putOngoingRebalanceTask( + tableBucket, + new RebalanceResultForBucket( + planForBucket.getOriginReplicas(), + planForBucket.getNewReplicas(), + RebalanceStatusForBucket.PENDING)); + + // if (reassignment.isBeingReassigned()) { + // + // } else { + // // already finished. + // coordinatorContext.putFinishedRebalanceTask( + // tableBucket, + // RebalanceResultForBucket.of( + // planForBucket, RebalanceStatusForBucket.COMPLETED)); + // } + } + + // try to trigger leader election together. + tableBucketStateMachine.handleStateChange( + electableBuckets, OnlineBucket, REASSIGN_BUCKET_LEADER_ELECTION); + electableBuckets.forEach( + tableBucket -> + coordinatorContext.putFinishedRebalanceTask( + tableBucket, + coordinatorContext + .removeOngoingRebalanceTask(tableBucket) + .markCompleted())); + + // then try to trigger bucket reassignments. + reassignments.forEach( + (tableBucket, reassignment) -> { + 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); + RebalanceResultForBucket resultForBucket = + coordinatorContext.removeOngoingRebalanceTask(tableBucket); + if (resultForBucket != null) { + coordinatorContext.putFinishedRebalanceTask( + tableBucket, resultForBucket.markFailed()); + } + } + }); + + LOG.info("Register rebalance task success."); + event.getRespCallback() + .complete(makeRebalanceRespose(new RebalancePlan(event.getRebalancePlan()))); + } + + private void processCancelRebalanceEvent() { + LOG.info("Cancel rebalance task."); + + coordinatorContext.getFinishedRebalanceTasks().clear(); + coordinatorContext.getOngoingRebalanceTasks().clear(); + // zk to remove rebalance task. + try { + zooKeeperClient.deleteRebalancePlan(); + } catch (Exception e) { + LOG.error("Error when delete rebalance plan from zookeeper.", e); + } + + LOG.info("Cancel rebalance task success."); + } + + /** + * 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); + // B2. Set RS = TRS, AR = [], RR = [] in memory. + coordinatorContext.updateBucketReplicaAssignment( + tableBucket, reassignment.getTargetReplicas()); + // B6. Update ZK with RS = TRS, AR = [], RR = []. + updateReplicaAssignmentForBucket(tableBucket, targetReplicas); + // B7. 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)); + // B8. Mark the ongoing rebalance task to finish. + coordinatorContext.putFinishedRebalanceTask( + tableBucket, + coordinatorContext.removeOngoingRebalanceTask(tableBucket).markCompleted()); + } + } + + private boolean isReassignmentComplete( + TableBucket tableBucket, ReplicaReassignment reassignment) throws Exception { + // if (!reassignment.isBeingReassigned()) { + // return true; + // } + + 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) throws Exception { + int currentLeader = coordinatorContext.getBucketLeaderAndIsr(tableBucket).get().leader(); + if (!targetReplicas.contains(currentLeader)) { + LOG.info( + "Leader {} for tableBucket {} being reassigned, is not in the new list of replicas {}. Re-electing leader", + currentLeader, + tableBucket, + targetReplicas); + tableBucketStateMachine.handleStateChange( + Collections.singleton(tableBucket), + OnlineBucket, + REASSIGN_BUCKET_LEADER_ELECTION); + } else if (coordinatorContext.isReplicaAndServerOnline(currentLeader, tableBucket)) { + LOG.info( + "Leader {} for tableBucket {} being reassigned, is already in the new list of replicas {} and ia alive", + currentLeader, + tableBucket, + targetReplicas); + updateLeaderEpochAndSendRequest( + tableBucket, + new ReplicaReassignment( + targetReplicas, Collections.emptyList(), Collections.emptyList())); + } else { + LOG.info( + "Leader {} for tableBucket {} being reassigned, is already in the new list of replicas {} but is dead", + currentLeader, + tableBucket, + targetReplicas); + tableBucketStateMachine.handleStateChange( + Collections.singleton(tableBucket), + OnlineBucket, + REASSIGN_BUCKET_LEADER_ELECTION); + } + } + + 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); + // replicaStateMachine.handleStateChanges(replicasToBeDeleted, ReplicaDeletionSuccessful); + // replicaStateMachine.handleStateChanges(replicasToBeDeleted, NonExistentReplica); + } + + 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. @@ -1003,6 +1476,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::tryToFinishRebalanceTasks); + // TODO update metadata for all alive tablet servers. return result; @@ -1265,8 +1741,134 @@ private void updateTabletServerMetadataCache( coordinatorRequestBatch.sendUpdateMetadataRequest(); } + private void updateLeaderEpochAndSendRequest( + TableBucket tableBucket, ReplicaReassignment reassignment) throws Exception { + LeaderAndIsr leaderAndIsr = updateLeaderEpoch(tableBucket); + if (leaderAndIsr == null) { + return; + } + + 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()); + } + + /** Does not change leader or isr, but just increments the leader epoch. */ + private @Nullable LeaderAndIsr updateLeaderEpoch(TableBucket tableBucket) throws Exception { + LOG.debug("Updating leader epoch for bucket {}", tableBucket); + Optional leaderAndIsrOpt = zooKeeperClient.getLeaderAndIsr(tableBucket); + if (!leaderAndIsrOpt.isPresent()) { + return null; + } + LeaderAndIsr leaderAndIsr = leaderAndIsrOpt.get(); + // increment the leader epoch even if there are no leader or isr changes to allow the + // leader to cache the expanded assigned replica list. + // LeaderAndIsr newLeaderAndIsr = leaderAndIsr.newLeaderAndIsrWithNewLeaderEpoch(); + // zooKeeperClient.updateLeaderAndIsr(tableBucket, newLeaderAndIsr); + // update leader and isr + // coordinatorContext.putBucketLeaderAndIsr(tableBucket, newLeaderAndIsr); + // LOG.info( + // "Updated leader epoch for tableBucket {} from {} to {}", + // tableBucket, + // leaderAndIsr, + // newLeaderAndIsr); + return leaderAndIsr; + } + @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 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/CoordinatorServer.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/CoordinatorServer.java index ef393aeb5c..bdb5ab8673 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 @@ -37,6 +37,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; @@ -139,6 +140,9 @@ public class CoordinatorServer extends ServerBase { @Nullable private Authorizer authorizer; + @GuardedBy("lock") + private RebalanceManager rebalanceManager; + @GuardedBy("lock") private CoordinatorContext coordinatorContext; @@ -183,6 +187,11 @@ protected void startServices() throws Exception { this.lakeTableTieringManager = new LakeTableTieringManager(); + this.rebalanceManager = + new RebalanceManager( + () -> getCoordinatorEventProcessor().getCoordinatorEventManager(), + zkClient); + MetadataManager metadataManager = new MetadataManager(zkClient, conf); this.coordinatorService = new CoordinatorService( @@ -194,7 +203,8 @@ protected void startServices() throws Exception { metadataManager, authorizer, createLakeCatalog(), - lakeTableTieringManager); + lakeTableTieringManager, + rebalanceManager); this.rpcServer = RpcServer.create( @@ -448,6 +458,14 @@ CompletableFuture stopServices() { exception = ExceptionUtils.firstOrSuppressed(t, exception); } + try { + if (rebalanceManager != null) { + rebalanceManager.close(); + } + } catch (Throwable t) { + exception = ExceptionUtils.firstOrSuppressed(t, exception); + } + try { if (rpcClient != null) { rpcClient.close(); @@ -497,6 +515,11 @@ public ServerMetadataCache getMetadataCache() { return authorizer; } + @VisibleForTesting + public RebalanceManager getRebalanceManager() { + return rebalanceManager; + } + 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 8f717cb79e..f531e5dbfd 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 @@ -19,12 +19,14 @@ import org.apache.fluss.cluster.ServerType; import org.apache.fluss.cluster.TabletServerInfo; -import org.apache.fluss.cluster.maintencance.ServerTag; +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.exception.InvalidCoordinatorException; import org.apache.fluss.exception.InvalidDatabaseException; import org.apache.fluss.exception.InvalidTableException; +import org.apache.fluss.exception.RebalanceFailureException; import org.apache.fluss.exception.SecurityDisabledException; import org.apache.fluss.exception.TableAlreadyExistException; import org.apache.fluss.exception.TableNotPartitionedException; @@ -94,11 +96,15 @@ 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.CancalRebalanceEvent; 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.EventManager; +import org.apache.fluss.server.coordinator.event.ExecuteRebalanceTaskEvent; 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.kv.snapshot.CompletedSnapshot; @@ -111,11 +117,15 @@ import org.apache.fluss.server.zk.data.BucketAssignment; 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.TableAssignment; import org.apache.fluss.server.zk.data.TableRegistration; import org.apache.fluss.utils.IOUtils; import org.apache.fluss.utils.concurrent.FutureUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import javax.annotation.Nullable; import java.io.UncheckedIOException; @@ -131,6 +141,7 @@ 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; @@ -138,6 +149,7 @@ 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.makeRebalanceRespose; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.toTablePath; import static org.apache.fluss.server.utils.TableAssignmentUtils.generateAssignment; import static org.apache.fluss.utils.PartitionUtils.validatePartitionSpec; @@ -147,6 +159,8 @@ /** An RPC Gateway service for coordinator server. */ public final class CoordinatorService extends RpcServiceBase implements CoordinatorGateway { + private static final Logger LOG = LoggerFactory.getLogger(CoordinatorService.class); + private final int defaultBucketNumber; private final int defaultReplicationFactor; private final Supplier eventManagerSupplier; @@ -157,6 +171,7 @@ public final class CoordinatorService extends RpcServiceBase implements Coordina private final @Nullable DataLakeFormat dataLakeFormat; private final @Nullable LakeCatalog lakeCatalog; private final LakeTableTieringManager lakeTableTieringManager; + private final RebalanceManager rebalanceManager; public CoordinatorService( Configuration conf, @@ -167,7 +182,8 @@ public CoordinatorService( MetadataManager metadataManager, @Nullable Authorizer authorizer, @Nullable LakeCatalog lakeCatalog, - LakeTableTieringManager lakeTableTieringManager) { + LakeTableTieringManager lakeTableTieringManager, + RebalanceManager rebalanceManager) { super(remoteFileSystem, ServerType.COORDINATOR, zkClient, metadataManager, authorizer); this.defaultBucketNumber = conf.getInt(ConfigOptions.DEFAULT_BUCKET_NUMBER); this.defaultReplicationFactor = conf.getInt(ConfigOptions.DEFAULT_REPLICATION_FACTOR); @@ -178,6 +194,7 @@ public CoordinatorService( this.dataLakeFormat = conf.getOptional(ConfigOptions.DATALAKE_FORMAT).orElse(null); this.lakeCatalog = lakeCatalog; this.lakeTableTieringManager = lakeTableTieringManager; + this.rebalanceManager = rebalanceManager; this.metadataCache = metadataCache; checkState( (dataLakeFormat == null) == (lakeCatalog == null), @@ -615,7 +632,32 @@ public CompletableFuture removeServerTag( @Override public CompletableFuture rebalance(RebalanceRequest request) { - throw new UnsupportedOperationException("Support soon!"); + List goalsByPriority = new ArrayList<>(); + Arrays.stream(request.getGoals()) + .forEach(goal -> goalsByPriority.add(getGoalByType(GoalType.valueOf(goal)))); + boolean isDryRun = request.isDryRun(); + + // 1. generate rebalance plan. + RebalancePlan rebalancePlan; + try { + rebalancePlan = rebalanceManager.generateRebalancePlan(goalsByPriority); + } catch (Exception e) { + throw new RebalanceFailureException("Failed to generate rebalance plan.", e); + } + + if (!isDryRun) { + CompletableFuture response = new CompletableFuture<>(); + // 2. execute rebalance plan. + LOG.info("Trigger Executing rebalance task"); + ExecuteRebalanceTaskEvent executeRebalanceTaskEvent = + new ExecuteRebalanceTaskEvent(rebalancePlan.getExecutePlan(), response); + eventManagerSupplier.get().put(executeRebalanceTaskEvent); + return response; + + // return rebalanceManager.executeRebalancePlan(rebalancePlan); + } else { + return CompletableFuture.completedFuture(makeRebalanceRespose(rebalancePlan)); + } } @Override @@ -627,7 +669,9 @@ public CompletableFuture listRebalanceProcess( @Override public CompletableFuture cancelRebalance( CancelRebalanceRequest request) { - throw new UnsupportedOperationException("Support soon!"); + CancalRebalanceEvent cancalRebalanceEvent = new CancalRebalanceEvent(); + eventManagerSupplier.get().put(cancalRebalanceEvent); + return CompletableFuture.completedFuture(new CancelRebalanceResponse()); } private void validateHeartbeatRequest( 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 904f3b5a84..a5ceb0319f 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,11 +34,14 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.ArrayList; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; +import static org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElectionStrategy.DEFAULT_ELECTION; + /** A manager for tables. */ public class TableManager { private static final Logger LOG = LoggerFactory.getLogger(TableManager.class); @@ -148,13 +151,15 @@ private void onCreateNewTableBucket(long tableId, Set tableBuckets) tableBuckets, coordinatorContext.getTablePathById(tableId)); // first, we transmit it to state NewBucket - tableBucketStateMachine.handleStateChange(tableBuckets, BucketState.NewBucket); + tableBucketStateMachine.handleStateChange( + tableBuckets, BucketState.NewBucket, DEFAULT_ELECTION); // then get all the replicas of the all table buckets Set replicas = coordinatorContext.getBucketReplicas(tableBuckets); // transmit all the replicas to state NewReplica replicaStateMachine.handleStateChanges(replicas, ReplicaState.NewReplica); // transmit it to state Online - tableBucketStateMachine.handleStateChange(tableBuckets, BucketState.OnlineBucket); + tableBucketStateMachine.handleStateChange( + tableBuckets, BucketState.OnlineBucket, DEFAULT_ELECTION); // transmit all the replicas to state online replicaStateMachine.handleStateChanges(replicas, ReplicaState.OnlineReplica); } @@ -162,8 +167,10 @@ private void onCreateNewTableBucket(long tableId, Set tableBuckets) /** Invoked with a table to be deleted. */ public void onDeleteTable(long tableId) { Set tableBuckets = coordinatorContext.getAllBucketsForTable(tableId); - tableBucketStateMachine.handleStateChange(tableBuckets, BucketState.OfflineBucket); - tableBucketStateMachine.handleStateChange(tableBuckets, BucketState.NonExistentBucket); + tableBucketStateMachine.handleStateChange( + tableBuckets, BucketState.OfflineBucket, DEFAULT_ELECTION); + tableBucketStateMachine.handleStateChange( + tableBuckets, BucketState.NonExistentBucket, DEFAULT_ELECTION); onDeleteTableBucket(coordinatorContext.getAllReplicasForTable(tableId)); } @@ -171,8 +178,10 @@ public void onDeleteTable(long tableId) { public void onDeletePartition(long tableId, long partitionId) { Set deleteBuckets = coordinatorContext.getAllBucketsForPartition(tableId, partitionId); - tableBucketStateMachine.handleStateChange(deleteBuckets, BucketState.OfflineBucket); - tableBucketStateMachine.handleStateChange(deleteBuckets, BucketState.NonExistentBucket); + tableBucketStateMachine.handleStateChange( + deleteBuckets, BucketState.OfflineBucket, DEFAULT_ELECTION); + tableBucketStateMachine.handleStateChange( + deleteBuckets, BucketState.NonExistentBucket, DEFAULT_ELECTION); onDeleteTableBucket(coordinatorContext.getAllReplicasForPartition(tableId, partitionId)); } @@ -203,6 +212,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 index 6e76398a12..b6e7af8886 100644 --- 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 @@ -17,7 +17,7 @@ package org.apache.fluss.server.coordinator.event; -import org.apache.fluss.cluster.maintencance.ServerTag; +import org.apache.fluss.cluster.rebalance.ServerTag; import org.apache.fluss.rpc.messages.AddServerTagResponse; import java.util.List; diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/CancalRebalanceEvent.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/CancalRebalanceEvent.java new file mode 100644 index 0000000000..ccdf4fb378 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/CancalRebalanceEvent.java @@ -0,0 +1,21 @@ +/* + * 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; + +/** An event of cancelling rebalance task. */ +public class CancalRebalanceEvent implements CoordinatorEvent {} 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..a944888e6f --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/event/ExecuteRebalanceTaskEvent.java @@ -0,0 +1,46 @@ +/* + * 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; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.rpc.messages.RebalanceResponse; + +import java.util.Map; +import java.util.concurrent.CompletableFuture; + +/** An event of executing rebalance task. */ +public class ExecuteRebalanceTaskEvent implements CoordinatorEvent { + Map rebalancePlan; + private final CompletableFuture respCallback; + + public ExecuteRebalanceTaskEvent( + Map rebalancePlan, + CompletableFuture respCallback) { + this.rebalancePlan = rebalancePlan; + this.respCallback = respCallback; + } + + public Map getRebalancePlan() { + return rebalancePlan; + } + + public CompletableFuture getRespCallback() { + return respCallback; + } +} 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 index 1e89d58a21..ede6fdeb0c 100644 --- 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 @@ -17,7 +17,7 @@ package org.apache.fluss.server.coordinator.event; -import org.apache.fluss.cluster.maintencance.ServerTag; +import org.apache.fluss.cluster.rebalance.ServerTag; import org.apache.fluss.rpc.messages.RemoveServerTagResponse; import java.util.List; 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..1d0a4f72ae --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/RebalanceManager.java @@ -0,0 +1,233 @@ +/* + * 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.RebalanceStatus; +import org.apache.fluss.cluster.rebalance.ServerTag; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.rpc.messages.RebalanceResponse; +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.rebalance.executor.RebalanceActionExecutorService; +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.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; +import java.util.concurrent.CompletableFuture; +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.concurrent.LockUtils.inLock; + +/** A rebalance manager to generate rebalance plan, and execution rebalance plan. */ +public class RebalanceManager implements AutoCloseable { + + private static final Logger LOG = LoggerFactory.getLogger(RebalanceManager.class); + + private final AtomicBoolean isClosed = new AtomicBoolean(false); + private final Lock lock = new ReentrantLock(); + private final RebalanceActionExecutorService actionExecutorService; + private final ZooKeeperClient zkClient; + private final Supplier eventManagerSupplier; + + @GuardedBy("lock") + private final GoalOptimizer goalOptimizer; + + @GuardedBy("lock") + private RebalanceStatus rebalanceStatus; + + @GuardedBy("lock") + private @Nullable Throwable rebalanceFailureCause = null; + + public RebalanceManager(Supplier eventManagerSupplier, ZooKeeperClient zkClient) { + this.eventManagerSupplier = eventManagerSupplier; + this.actionExecutorService = new RebalanceActionExecutorService(eventManagerSupplier); + actionExecutorService.start(); + + this.zkClient = zkClient; + this.goalOptimizer = new GoalOptimizer(); + + registerRebalanceTasksFromZookeeper(); + } + + private void registerRebalanceTasksFromZookeeper() { + try { + Optional rebalancePlanOpt = zkClient.getRebalancePlan(); + if (rebalancePlanOpt.isPresent()) { + rebalanceStatus = RebalanceStatus.TASK_EXECUTING; + // TODO trigger to execution. + } else { + rebalanceStatus = RebalanceStatus.NO_ONGOING_REBALANCE; + } + } catch (Exception e) { + LOG.error( + "Failed to get rebalance plan from zookeeper, it will be treated as no " + + "ongoing rebalancing plan.", + e); + } + } + + public boolean hasOngoingRebalance() { + checkNotClosed(); + return inLock( + lock, + () -> + rebalanceStatus == RebalanceStatus.PLAN_GENERATING + || rebalanceStatus == RebalanceStatus.TASK_EXECUTING); + } + + public RebalancePlan generateRebalancePlan(List goalsByPriority) throws Exception { + checkNotClosed(); + return inLock( + lock, + () -> { + rebalanceStatus = RebalanceStatus.PLAN_GENERATING; + + 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); + rebalanceStatus = RebalanceStatus.FAILED; + throw e; + } + + rebalanceStatus = RebalanceStatus.NO_ONGOING_REBALANCE; + // group by tableId and partitionId to generate rebalance plan. + return buildRebalancePlan(rebalancePlanForBuckets); + }); + } + + public CompletableFuture executeRebalancePlan(RebalancePlan rebalancePlan) { + checkNotClosed(); + return inLock( + lock, + () -> { + rebalanceStatus = RebalanceStatus.TASK_EXECUTING; + return actionExecutorService + .execute(rebalancePlan.getExecutePlan()) + .whenComplete( + (unused, throwable) -> { + if (throwable == null) { + rebalanceStatus = RebalanceStatus.COMPLETED; + } else { + rebalanceStatus = RebalanceStatus.FAILED; + } + }); + }); + } + + @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."); + } + } + + @Override + public void close() throws Exception { + if (isClosed.compareAndSet(false, true)) { + actionExecutorService.shutdown(); + } + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/executor/ActionExecutorService.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/executor/ActionExecutorService.java new file mode 100644 index 0000000000..0a6a7e00a6 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/executor/ActionExecutorService.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.server.coordinator.rebalance.executor; + +import org.apache.fluss.cluster.rebalance.RebalancePlanForBucket; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.rpc.messages.RebalanceResponse; + +import java.util.Map; +import java.util.concurrent.CompletableFuture; + +/** An interface for action executor service. */ +public interface ActionExecutorService { + + void start(); + + void shutdown(); + + CompletableFuture execute(Map actions); +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/executor/RebalanceActionExecutorService.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/executor/RebalanceActionExecutorService.java new file mode 100644 index 0000000000..807bac6cee --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/executor/RebalanceActionExecutorService.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.server.coordinator.rebalance.executor; + +import org.apache.fluss.cluster.rebalance.RebalancePlanForBucket; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.rpc.messages.RebalanceResponse; +import org.apache.fluss.server.coordinator.event.EventManager; +import org.apache.fluss.server.coordinator.event.ExecuteRebalanceTaskEvent; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.function.Supplier; + +/** An executor service that executes the rebalance actions. */ +public class RebalanceActionExecutorService implements ActionExecutorService, Runnable { + + private static final Logger LOG = LoggerFactory.getLogger(RebalanceActionExecutorService.class); + + private final Supplier eventManagerSupplier; + private final BlockingQueue actionQueue = new ArrayBlockingQueue<>(5); + private volatile boolean shutdown; + + public RebalanceActionExecutorService(Supplier eventManagerSupplier) { + this.eventManagerSupplier = eventManagerSupplier; + } + + @Override + public void start() { + this.shutdown = false; + LOG.info("Starting rebalance action executor service."); + } + + @Override + public void shutdown() { + this.shutdown = true; + LOG.info("Shutting down rebalance action executor service."); + } + + @Override + public CompletableFuture execute( + Map actions) { + CompletableFuture cf = new CompletableFuture<>(); + if (!actionQueue.isEmpty()) { + cf.completeExceptionally( + new IllegalStateException( + "Rebalance action executor service is busy, Currently, we only support one task in progress.")); + } else { + try { + actionQueue.put(new Task(actions, cf)); + } catch (InterruptedException e) { + LOG.error("Failed to put rebalance action into action queue.", e); + cf.completeExceptionally(e); + } + } + return cf; + } + + @Override + public void run() { + while (!shutdown) { + try { + doReassign(actionQueue.take()); + } catch (InterruptedException e) { + // Do nothing. + } + } + } + + private void doReassign(Task task) { + LOG.info("Trigger Executing rebalance task"); + ExecuteRebalanceTaskEvent executeRebalanceTaskEvent = + new ExecuteRebalanceTaskEvent(task.getActions(), task.getFuture()); + eventManagerSupplier.get().put(executeRebalanceTaskEvent); + } + + private static class Task { + private final Map actions; + private final CompletableFuture future; + + public Task( + Map actions, + CompletableFuture future) { + this.actions = actions; + this.future = future; + } + + public Map getActions() { + return actions; + } + + public CompletableFuture getFuture() { + return future; + } + } +} 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/rebalance/model/Statistic.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/model/Statistic.java new file mode 100644 index 0000000000..bf12b8b281 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/rebalance/model/Statistic.java @@ -0,0 +1,26 @@ +/* + * 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; + +/** 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/ReplicaLeaderElectionAlgorithms.java b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaLeaderElectionAlgorithms.java index d0f6b1835d..669af1e4e5 100644 --- 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 @@ -34,4 +34,30 @@ public static Optional defaultReplicaLeaderElection( return Optional.empty(); } + + public static Optional reassignBucketLeaderElection( + List targetReplicas, List liveReplicas, List isr) { + // 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. + for (int assignment : targetReplicas) { + if (liveReplicas.contains(assignment) && isr.contains(assignment)) { + return Optional.of(assignment); + } + } + + return Optional.empty(); + } + + public static Optional preferredReplicaLeaderElection( + List assignment, List liveReplicas, List isr) { + if (!assignment.isEmpty()) { + // get the first one. + int preferredReplica = assignment.get(0); + if (liveReplicas.contains(preferredReplica) && isr.contains(preferredReplica)) { + return Optional.of(preferredReplica); + } + } + return Optional.empty(); + } } 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/statemachine/ReplicaLeaderElectionStrategy.java new file mode 100644 index 0000000000..9b58e59d40 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/coordinator/statemachine/ReplicaLeaderElectionStrategy.java @@ -0,0 +1,25 @@ +/* + * 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; + +/** The strategies to elect the replica leader. */ +public enum ReplicaLeaderElectionStrategy { + DEFAULT_ELECTION, + PREFERRED_ELECTION, + REASSIGN_BUCKET_LEADER_ELECTION +} 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 be77a43e51..2837d9a997 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 d1b9958cb1..41a19e9be6 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 @@ -40,6 +40,11 @@ import java.util.Set; import java.util.stream.Collectors; +import static org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElectionAlgorithms.defaultReplicaLeaderElection; +import static org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElectionAlgorithms.preferredReplicaLeaderElection; +import static org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElectionAlgorithms.reassignBucketLeaderElection; +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. */ @@ -77,7 +82,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 @@ -107,14 +112,17 @@ public void triggerOnlineBucketStateChange() { buckets.stream() .filter(tableBucket -> !coordinatorContext.isToBeDeleted(tableBucket)) .collect(Collectors.toSet()); - handleStateChange(buckets, BucketState.OnlineBucket); + handleStateChange(buckets, BucketState.OnlineBucket, DEFAULT_ELECTION); } public void shutdown() { LOG.info("Shutdown table bucket state machine."); } - public void handleStateChange(Set tableBuckets, BucketState targetState) { + public void handleStateChange( + Set tableBuckets, + BucketState targetState, + ReplicaLeaderElectionStrategy electStrategy) { try { coordinatorRequestBatch.newBatch(); @@ -123,7 +131,7 @@ public void handleStateChange(Set tableBuckets, BucketState targetS batchHandleOnlineChangeAndInitLeader(tableBuckets); } else { for (TableBucket tableBucket : tableBuckets) { - doHandleStateChange(tableBucket, targetState); + doHandleStateChange(tableBucket, targetState, electStrategy); } } coordinatorRequestBatch.sendRequestToTabletServers( @@ -175,8 +183,12 @@ public void handleStateChange(Set tableBuckets, BucketState targetS * * @param tableBucket The table bucket that is to do state change * @param targetState the target state that is to change to + * @param electStrategy the strategy to choose a new leader */ - private void doHandleStateChange(TableBucket tableBucket, BucketState targetState) { + private void doHandleStateChange( + TableBucket tableBucket, + BucketState targetState, + ReplicaLeaderElectionStrategy electStrategy) { coordinatorContext.putBucketStateIfNotExists(tableBucket, BucketState.NonExistentBucket); if (!checkValidTableBucketStateChange(tableBucket, targetState)) { return; @@ -224,7 +236,7 @@ private void doHandleStateChange(TableBucket tableBucket, BucketState targetStat // current state is Online or Offline // not new bucket, we then need to update leader/epoch for the bucket Optional optionalElectionResult = - electNewLeaderForTableBuckets(tableBucket); + electNewLeaderForTableBuckets(tableBucket, electStrategy); if (!optionalElectionResult.isPresent()) { logFailedStateChange(tableBucket, currentState, targetState); } else { @@ -415,8 +427,7 @@ private Optional doInitElectionForBucket( // servers as inSyncReplica set. List isr = liveServers; Optional leaderOpt = - ReplicaLeaderElectionAlgorithms.defaultReplicaLeaderElection( - assignedServers, liveServers, isr); + defaultReplicaLeaderElection(assignedServers, liveServers, isr); if (!leaderOpt.isPresent()) { LOG.error( "The leader election for table bucket {} is empty.", @@ -449,7 +460,8 @@ private List tryRegisterLeaderAndIsrOneByOne( return registerSuccessList; } - private Optional electNewLeaderForTableBuckets(TableBucket tableBucket) { + private Optional electNewLeaderForTableBuckets( + TableBucket tableBucket, ReplicaLeaderElectionStrategy electStrategy) { LeaderAndIsr leaderAndIsr; try { leaderAndIsr = zooKeeperClient.getLeaderAndIsr(tableBucket).get(); @@ -469,7 +481,7 @@ private Optional electNewLeaderForTableBuckets(TableBucket table } // re-election Optional optionalElectionResult = - leaderForOffline(tableBucket, leaderAndIsr); + leaderForOffline(tableBucket, leaderAndIsr, electStrategy); if (!optionalElectionResult.isPresent()) { LOG.error( "The result of elect leader for table bucket {} is empty.", @@ -568,7 +580,9 @@ private String stringifyBucket(TableBucket tableBucket) { * in isr set. */ private Optional leaderForOffline( - TableBucket tableBucket, LeaderAndIsr leaderAndIsr) { + TableBucket tableBucket, + LeaderAndIsr leaderAndIsr, + ReplicaLeaderElectionStrategy electionStrategy) { List assignment = coordinatorContext.getAssignment(tableBucket); // filter out the live servers List liveReplicas = @@ -584,9 +598,27 @@ private Optional leaderForOffline( return Optional.empty(); } - Optional leaderOpt = - ReplicaLeaderElectionAlgorithms.defaultReplicaLeaderElection( - assignment, liveReplicas, leaderAndIsr.isr()); + Optional leaderOpt; + switch (electionStrategy) { + case DEFAULT_ELECTION: + leaderOpt = + defaultReplicaLeaderElection(assignment, liveReplicas, leaderAndIsr.isr()); + break; + case PREFERRED_ELECTION: + leaderOpt = + preferredReplicaLeaderElection( + assignment, liveReplicas, leaderAndIsr.isr()); + break; + case REASSIGN_BUCKET_LEADER_ELECTION: + List targetReplicas = + coordinatorContext.getOngoingRebalanceTask(tableBucket).targetReplicas(); + leaderOpt = + reassignBucketLeaderElection( + targetReplicas, liveReplicas, leaderAndIsr.isr()); + break; + default: + leaderOpt = Optional.empty(); + } if (!leaderOpt.isPresent()) { LOG.error( "The leader election for table bucket {} is 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 26b31bdd59..7e9e4f3ae6 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 @@ -340,6 +340,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 f77cf378ed..2b8e845d66 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,7 @@ 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.config.ConfigOptions; import org.apache.fluss.fs.FsPath; import org.apache.fluss.fs.token.ObtainedSecurityToken; @@ -29,6 +30,7 @@ import org.apache.fluss.metadata.TableBucket; 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; @@ -108,6 +110,8 @@ 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.PbRemoteLogSegment; import org.apache.fluss.rpc.messages.PbRemotePathAndLocalFile; import org.apache.fluss.rpc.messages.PbServerNode; @@ -124,6 +128,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; @@ -155,6 +160,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; @@ -1624,6 +1630,76 @@ public static LakeTieringHeartbeatResponse makeLakeTieringHeartbeatResponse( return new LakeTieringHeartbeatResponse().setCoordinatorEpoch(coordinatorEpoch); } + 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; + } + 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 2542468628..4bffb7eeac 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 @@ -215,7 +215,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 { 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 6b6f8bc7a3..fdea4c0153 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 @@ -74,6 +74,10 @@ public LeaderAndIsr newLeaderAndIsr(int newLeader, List newIsr) { return new LeaderAndIsr(newLeader, 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 index 6891aacf86..b34dd573a1 100644 --- 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 @@ -17,8 +17,7 @@ package org.apache.fluss.server.zk.data; -import org.apache.fluss.cluster.maintencance.RebalancePlanForBucket; -import org.apache.fluss.cluster.maintencance.RebalanceStatus; +import org.apache.fluss.cluster.rebalance.RebalancePlanForBucket; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TablePartition; @@ -37,9 +36,6 @@ */ public class RebalancePlan { - /** The rebalance status of this rebalance plan. */ - private final RebalanceStatus rebalanceStatus; - /** A mapping from tableBucket to RebalancePlanForBuckets of none-partitioned table. */ private final Map> planForBuckets; @@ -47,8 +43,7 @@ public class RebalancePlan { private final Map> planForBucketsOfPartitionedTable; - public RebalancePlan( - RebalanceStatus rebalanceStatus, Map bucketPlan) { + public RebalancePlan(Map bucketPlan) { this.planForBuckets = new HashMap<>(); this.planForBucketsOfPartitionedTable = new HashMap<>(); @@ -67,8 +62,6 @@ public RebalancePlan( .add(rebalancePlanForBucket); } } - - this.rebalanceStatus = rebalanceStatus; } public Map> getPlanForBuckets() { @@ -79,8 +72,24 @@ public Map> getPlanForBucketsOfPart return planForBucketsOfPartitionedTable; } - public RebalanceStatus getRebalanceStatus() { - return rebalanceStatus; + 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 @@ -90,8 +99,6 @@ public String toString() { + planForBuckets + ", planForBucketsOfPartitionedTable=" + planForBucketsOfPartitionedTable - + ", rebalanceStatus=" - + rebalanceStatus + '}'; } @@ -110,12 +117,11 @@ public boolean equals(Object o) { return false; } return Objects.equals( - planForBucketsOfPartitionedTable, that.planForBucketsOfPartitionedTable) - && rebalanceStatus == that.rebalanceStatus; + planForBucketsOfPartitionedTable, that.planForBucketsOfPartitionedTable); } @Override public int hashCode() { - return Objects.hash(planForBuckets, planForBucketsOfPartitionedTable, rebalanceStatus); + 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 index b0183bd03d..b79cd46e30 100644 --- 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 @@ -17,8 +17,7 @@ package org.apache.fluss.server.zk.data; -import org.apache.fluss.cluster.maintencance.RebalancePlanForBucket; -import org.apache.fluss.cluster.maintencance.RebalanceStatus; +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; @@ -41,7 +40,6 @@ public class RebalancePlanJsonSerde private static final String VERSION_KEY = "version"; private static final String REBALANCE_PLAN = "rebalance_plan"; - private static final String REBALANCE_STATUS = "rebalance_status"; private static final String TABLE_ID = "table_id"; private static final String PARTITION_ID = "partition_id"; @@ -60,7 +58,6 @@ public void serialize(RebalancePlan rebalancePlan, JsonGenerator generator) thro generator.writeStartObject(); generator.writeNumberField(VERSION_KEY, VERSION); - generator.writeNumberField(REBALANCE_STATUS, rebalancePlan.getRebalanceStatus().code()); generator.writeArrayFieldStart(REBALANCE_PLAN); // first to write none-partitioned tables. for (Map.Entry> entry : @@ -96,9 +93,6 @@ public void serialize(RebalancePlan rebalancePlan, JsonGenerator generator) thro @Override public RebalancePlan deserialize(JsonNode node) { - - RebalanceStatus rebalanceStatus = RebalanceStatus.of(node.get(REBALANCE_STATUS).asInt()); - JsonNode rebalancePlanNode = node.get(REBALANCE_PLAN); Map planForBuckets = new HashMap<>(); @@ -134,11 +128,11 @@ public RebalancePlan deserialize(JsonNode node) { planForBuckets.put( tableBucket, new RebalancePlanForBucket( - bucketId, originLeader, newLeader, originReplicas, newReplicas)); + tableBucket, originLeader, newLeader, originReplicas, newReplicas)); } } - return new RebalancePlan(rebalanceStatus, planForBuckets); + return new RebalancePlan(planForBuckets); } private void serializeRebalancePlanForBucket( 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 index c9ae5400a3..edddcaaf75 100644 --- 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 @@ -17,7 +17,7 @@ package org.apache.fluss.server.zk.data; -import org.apache.fluss.cluster.maintencance.ServerTag; +import org.apache.fluss.cluster.rebalance.ServerTag; import java.util.Map; import java.util.Objects; 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 index 25257c8e23..7df94c74e4 100644 --- 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 @@ -17,7 +17,7 @@ package org.apache.fluss.server.zk.data; -import org.apache.fluss.cluster.maintencance.ServerTag; +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; 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 b448c41263..9ff610557d 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; @@ -36,11 +37,13 @@ import org.apache.fluss.rpc.messages.CommitRemoteLogManifestResponse; import org.apache.fluss.rpc.messages.NotifyKvSnapshotOffsetRequest; import org.apache.fluss.rpc.messages.NotifyRemoteLogOffsetsRequest; +import org.apache.fluss.rpc.messages.RebalanceResponse; import org.apache.fluss.server.coordinator.event.AccessContextEvent; import org.apache.fluss.server.coordinator.event.AdjustIsrReceivedEvent; 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; @@ -367,11 +370,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); @@ -819,6 +822,174 @@ 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 != 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. + Map rebalanceTaskMap = new HashMap<>(); + rebalanceTaskMap.put( + tableBucket, + new RebalancePlanForBucket( + tableBucket, + newLeaderAndIsr.leader(), + originLeader, + leaderAndIsr.isr(), + newLeaderAndIsr.isr())); + CompletableFuture future = new CompletableFuture<>(); + ExecuteRebalanceTaskEvent rebalanceTaskEvent = + new ExecuteRebalanceTaskEvent(rebalanceTaskMap, future); + eventProcessor.getCoordinatorEventManager().put(rebalanceTaskEvent); + future.get(); + + // 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 + Map rebalanceTaskMap = new HashMap<>(); + rebalanceTaskMap.put( + tb0, + new RebalancePlanForBucket( + tb0, 0, 0, Arrays.asList(0, 1, 3), Arrays.asList(0, 1, 2))); + rebalanceTaskMap.put( + tb1, + new RebalancePlanForBucket( + tb1, 0, 0, Arrays.asList(0, 1, 3), Arrays.asList(1, 2, 3))); + rebalanceTaskMap.put( + tb2, + new RebalancePlanForBucket( + tb2, 1, 1, Arrays.asList(1, 2, 3), Arrays.asList(2, 3, 0))); + CompletableFuture future = new CompletableFuture<>(); + ExecuteRebalanceTaskEvent rebalanceTaskEvent = + new ExecuteRebalanceTaskEvent(rebalanceTaskMap, future); + eventProcessor.getCoordinatorEventManager().put(rebalanceTaskEvent); + future.get(); + + // 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, @@ -1022,13 +1193,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(); @@ -1036,6 +1207,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/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/ReplicaLeaderElectionAlgorithmsTest.java new file mode 100644 index 0000000000..68678212ea --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaLeaderElectionAlgorithmsTest.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.statemachine; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Optional; + +import static org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElectionAlgorithms.defaultReplicaLeaderElection; +import static org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElectionAlgorithms.preferredReplicaLeaderElection; +import static org.apache.fluss.server.coordinator.statemachine.ReplicaLeaderElectionAlgorithms.reassignBucketLeaderElection; +import static org.assertj.core.api.Assertions.assertThat; + +/** Test for {@link ReplicaLeaderElectionAlgorithms}. */ +public class ReplicaLeaderElectionAlgorithmsTest { + + @Test + void testDefaultReplicaLeaderElection() { + List assignments = Arrays.asList(2, 4); + List liveReplicas = Collections.singletonList(4); + List isr = Arrays.asList(2, 4); + + Optional leaderOpt = defaultReplicaLeaderElection(assignments, liveReplicas, isr); + assertThat(leaderOpt).hasValue(4); + } + + @Test + void testReassignBucketLeaderElection() { + List targetReplicas = Arrays.asList(1, 2, 3); + List liveReplicas = Arrays.asList(1, 2, 3); + List isr = Arrays.asList(1, 2, 3); + Optional leaderOpt = + reassignBucketLeaderElection(targetReplicas, liveReplicas, isr); + assertThat(leaderOpt).isPresent(); + assertThat(leaderOpt.get()).isEqualTo(1); + + targetReplicas = Arrays.asList(1, 2, 3); + liveReplicas = Arrays.asList(2, 3); + isr = Arrays.asList(2, 3); + leaderOpt = reassignBucketLeaderElection(targetReplicas, liveReplicas, isr); + assertThat(leaderOpt).isPresent(); + assertThat(leaderOpt.get()).isEqualTo(2); + + targetReplicas = Arrays.asList(1, 2, 3); + liveReplicas = Arrays.asList(1, 2); + isr = Collections.emptyList(); + leaderOpt = reassignBucketLeaderElection(targetReplicas, liveReplicas, isr); + assertThat(leaderOpt).isNotPresent(); + } + + @Test + void testPreferredReplicaLeaderElection() { + List assignment = Arrays.asList(1, 2, 3); + List liveReplicas = Arrays.asList(1, 2, 3); + List isr = Arrays.asList(1, 2, 3); + Optional leaderOpt = preferredReplicaLeaderElection(assignment, liveReplicas, isr); + assertThat(leaderOpt).isPresent(); + assertThat(leaderOpt.get()).isEqualTo(1); + + assignment = Arrays.asList(1, 2, 3); + liveReplicas = Arrays.asList(2, 3); + isr = Arrays.asList(2, 3); + leaderOpt = preferredReplicaLeaderElection(assignment, liveReplicas, isr); + 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 9009765e4f..e0408e8678 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 @@ -61,6 +61,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.DEFAULT_ELECTION; import static org.apache.fluss.testutils.common.CommonTestUtils.retry; import static org.assertj.core.api.Assertions.assertThat; @@ -170,9 +171,9 @@ void testInvalidBucketStateChange() { // NonExistent to Online/Offline is invalid, shouldn't do state transmit tableBucketStateMachine.handleStateChange( - Collections.singleton(tableBucket0), OnlineBucket); + Collections.singleton(tableBucket0), OnlineBucket, DEFAULT_ELECTION); tableBucketStateMachine.handleStateChange( - Collections.singleton(tableBucket1), OfflineBucket); + Collections.singleton(tableBucket1), OfflineBucket, DEFAULT_ELECTION); // check it assertThat(coordinatorContext.getBucketState(tableBucket0)).isEqualTo(NonExistentBucket); @@ -198,7 +199,8 @@ void testStateChangeToOnline() throws Exception { coordinatorContext.updateBucketReplicaAssignment(tableBucket, Arrays.asList(0, 1, 2)); coordinatorContext.putBucketState(tableBucket, NewBucket); // case1: init a new leader for NewBucket to OnlineBucket - tableBucketStateMachine.handleStateChange(Collections.singleton(tableBucket), OnlineBucket); + tableBucketStateMachine.handleStateChange( + Collections.singleton(tableBucket), OnlineBucket, DEFAULT_ELECTION); // non any alive servers, the state change fail assertThat(coordinatorContext.getBucketState(tableBucket)).isEqualTo(NewBucket); @@ -209,7 +211,8 @@ void testStateChangeToOnline() throws Exception { coordinatorContext, testCoordinatorChannelManager); // change to online again - tableBucketStateMachine.handleStateChange(Collections.singleton(tableBucket), OnlineBucket); + tableBucketStateMachine.handleStateChange( + Collections.singleton(tableBucket), OnlineBucket, DEFAULT_ELECTION); assertThat(coordinatorContext.getBucketState(tableBucket)).isEqualTo(OnlineBucket); // check bucket LeaderAndIsr @@ -220,7 +223,8 @@ void testStateChangeToOnline() throws Exception { coordinatorContext.setLiveTabletServers( CoordinatorTestUtils.createServers(Arrays.asList(1, 2))); - tableBucketStateMachine.handleStateChange(Collections.singleton(tableBucket), OnlineBucket); + tableBucketStateMachine.handleStateChange( + Collections.singleton(tableBucket), OnlineBucket, DEFAULT_ELECTION); // check state is online assertThat(coordinatorContext.getBucketState(tableBucket)).isEqualTo(OnlineBucket); @@ -232,7 +236,12 @@ void testStateChangeToOnline() throws Exception { coordinatorContext.putBucketState(tableBucket, OfflineBucket); coordinatorContext.setLiveTabletServers( CoordinatorTestUtils.createServers(Collections.emptyList())); - tableBucketStateMachine.handleStateChange(Collections.singleton(tableBucket), OnlineBucket); + tableBucketStateMachine.handleStateChange( + Collections.singleton(tableBucket), OnlineBucket, DEFAULT_ELECTION); + coordinatorContext.setLiveTabletServers( + CoordinatorTestUtils.createServers(Collections.emptyList())); + tableBucketStateMachine.handleStateChange( + Collections.singleton(tableBucket), OnlineBucket, DEFAULT_ELECTION); // the state will still be offline assertThat(coordinatorContext.getBucketState(tableBucket)).isEqualTo(OfflineBucket); @@ -277,7 +286,7 @@ void testStateChangeToOnline() throws Exception { coordinatorContext.updateBucketReplicaAssignment(tableBucket1, Arrays.asList(0, 1, 2)); coordinatorContext.putBucketState(tableBucket1, NewBucket); tableBucketStateMachine.handleStateChange( - Collections.singleton(tableBucket1), OnlineBucket); + Collections.singleton(tableBucket1), OnlineBucket, DEFAULT_ELECTION); // retry util the leader has changed to 1 retry( Duration.ofMinutes(1), @@ -304,18 +313,18 @@ void testStateChangeForDropTable() { coordinatorContext.putBucketState(tableBucket1, OnlineBucket); tableBucketStateMachine.handleStateChange( - Collections.singleton(tableBucket0), OfflineBucket); + Collections.singleton(tableBucket0), OfflineBucket, DEFAULT_ELECTION); tableBucketStateMachine.handleStateChange( - Collections.singleton(tableBucket0), NonExistentBucket); + Collections.singleton(tableBucket0), NonExistentBucket, DEFAULT_ELECTION); // bucket 0 should be removed assertThat(coordinatorContext.getBucketState(tableBucket0)).isNull(); // bucket 1 should still exist assertThat(coordinatorContext.getBucketState(tableBucket1)).isNotNull(); tableBucketStateMachine.handleStateChange( - Collections.singleton(tableBucket1), OfflineBucket); + Collections.singleton(tableBucket1), OfflineBucket, DEFAULT_ELECTION); tableBucketStateMachine.handleStateChange( - Collections.singleton(tableBucket1), NonExistentBucket); + Collections.singleton(tableBucket1), NonExistentBucket, DEFAULT_ELECTION); assertThat(coordinatorContext.getBucketState(tableBucket0)).isNull(); } 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 05d85edf5b..86287d4fc1 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 @@ -46,6 +46,7 @@ import org.apache.fluss.server.authorizer.DefaultAuthorizer; import org.apache.fluss.server.coordinator.CoordinatorServer; 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; @@ -456,6 +457,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 f511eb9ed5..653d32a385 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,9 +19,8 @@ import org.apache.fluss.cluster.Endpoint; import org.apache.fluss.cluster.TabletServerInfo; -import org.apache.fluss.cluster.maintencance.RebalancePlanForBucket; -import org.apache.fluss.cluster.maintencance.RebalanceStatus; -import org.apache.fluss.cluster.maintencance.ServerTag; +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; @@ -534,33 +533,49 @@ void testRebalancePlan() throws Exception { bucketPlan.put( new TableBucket(0L, 0), new RebalancePlanForBucket( - 0, 0, 3, Arrays.asList(0, 1, 2), Arrays.asList(3, 4, 5))); + new TableBucket(0L, 0), + 0, + 3, + Arrays.asList(0, 1, 2), + Arrays.asList(3, 4, 5))); bucketPlan.put( new TableBucket(0L, 1), new RebalancePlanForBucket( - 1, 1, 1, Arrays.asList(0, 1, 2), Arrays.asList(1, 2, 3))); + 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( - 0, 1, 1, Arrays.asList(0, 1, 2), Arrays.asList(1, 2, 3))); + 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( - 1, 1, 1, Arrays.asList(0, 1, 2), Arrays.asList(1, 2, 3))); - zookeeperClient.registerRebalancePlan( - new RebalancePlan(RebalanceStatus.IN_PROGRESS, bucketPlan)); - assertThat(zookeeperClient.getRebalancePlan()) - .hasValue(new RebalancePlan(RebalanceStatus.IN_PROGRESS, bucketPlan)); + 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( - 0, 0, 3, Arrays.asList(0, 1, 2), Arrays.asList(3, 4, 5))); - zookeeperClient.registerRebalancePlan( - new RebalancePlan(RebalanceStatus.IN_PROGRESS, bucketPlan)); - assertThat(zookeeperClient.getRebalancePlan()) - .hasValue(new RebalancePlan(RebalanceStatus.IN_PROGRESS, bucketPlan)); + 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(); 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 index 01e0db58a8..64b6da43e5 100644 --- 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 @@ -17,8 +17,7 @@ package org.apache.fluss.server.zk.data; -import org.apache.fluss.cluster.maintencance.RebalancePlanForBucket; -import org.apache.fluss.cluster.maintencance.RebalanceStatus; +import org.apache.fluss.cluster.rebalance.RebalancePlanForBucket; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.utils.json.JsonSerdeTestBase; @@ -39,32 +38,52 @@ protected RebalancePlan[] createObjects() { bucketPlan.put( new TableBucket(0L, 0), new RebalancePlanForBucket( - 0, 0, 3, Arrays.asList(0, 1, 2), Arrays.asList(3, 4, 5))); + new TableBucket(0L, 0), + 0, + 3, + Arrays.asList(0, 1, 2), + Arrays.asList(3, 4, 5))); bucketPlan.put( new TableBucket(0L, 1), new RebalancePlanForBucket( - 1, 1, 1, Arrays.asList(0, 1, 2), Arrays.asList(1, 2, 3))); + 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( - 0, 0, 3, Arrays.asList(0, 1, 2), Arrays.asList(3, 4, 5))); + 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( - 1, 1, 1, Arrays.asList(0, 1, 2), Arrays.asList(1, 2, 3))); + 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( - 0, 0, 3, Arrays.asList(0, 1, 2), Arrays.asList(3, 4, 5))); - return new RebalancePlan[] {new RebalancePlan(RebalanceStatus.IN_PROGRESS, bucketPlan)}; + 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_status\":1,\"rebalance_plan\":" + "{\"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]}]}," 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 index aaec82be9a..8dd4e7f454 100644 --- 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 @@ -17,7 +17,7 @@ package org.apache.fluss.server.zk.data; -import org.apache.fluss.cluster.maintencance.ServerTag; +import org.apache.fluss.cluster.rebalance.ServerTag; import org.apache.fluss.utils.json.JsonSerdeTestBase; import java.util.HashMap; From 9bae99ee89e2d5251757687524b9ea4dde161245 Mon Sep 17 00:00:00 2001 From: yunhong <337361684@qq.com> Date: Fri, 22 Aug 2025 13:41:01 +0800 Subject: [PATCH 4/5] [server] Support rebalance for PrimaryKey Table --- .../apache/fluss/client/FlussConnection.java | 2 +- .../scanner/batch/KvSnapshotBatchScanner.java | 2 +- .../client/table/scanner/log/LogFetcher.java | 3 +- .../table/scanner/log/LogScannerImpl.java | 2 +- .../scanner/log/RemoteLogDownloader.java | 2 +- .../batch/KvSnapshotBatchScannerITCase.java | 2 +- .../table/scanner/log/LogFetcherTest.java | 2 +- .../scanner/log/RemoteLogDownloaderTest.java | 2 +- .../rebalance/RebalanceResultForBucket.java | 50 +-- .../fluss/remote/RemoteFileDownloader.java | 124 ++++++++ fluss-rpc/src/main/proto/FlussApi.proto | 7 + .../CoordinatorEventProcessor.java | 195 ++++++------ .../statemachine/TableBucketStateMachine.java | 19 +- .../entity/AdjustIsrResultForBucket.java | 2 +- .../fluss/server/entity/FetchReqInfo.java | 24 +- .../server/entity/NotifyLeaderAndIsrData.java | 16 + ...coverHelper.java => KvApplyLogHelper.java} | 182 +++++------ .../org/apache/fluss/server/kv/KvTablet.java | 61 ++++ .../apache/fluss/server/log/FetchParams.java | 12 +- .../apache/fluss/server/log/LogTablet.java | 18 +- .../server/log/remote/RemoteLogManager.java | 21 +- .../fluss/server/replica/FollowerReplica.java | 17 +- .../apache/fluss/server/replica/IsrState.java | 54 +++- .../apache/fluss/server/replica/Replica.java | 271 ++++++++++++++--- .../fluss/server/replica/ReplicaManager.java | 22 +- .../replica/fetcher/RemoteLeaderEndpoint.java | 25 ++ .../fetcher/ReplicaFetcherManager.java | 1 + .../replica/fetcher/ReplicaFetcherThread.java | 33 +- .../replica/standby/BecomeHotStandbyTask.java | 285 ++++++++++++++++++ .../replica/standby/KvStandbyManager.java | 163 ++++++++++ .../server/utils/ServerRpcMessageUtils.java | 63 ++-- .../fluss/server/zk/data/LeaderAndIsr.java | 137 +++++++-- .../server/zk/data/LeaderAndIsrJsonSerde.java | 42 ++- .../CoordinatorEventProcessorTest.java | 21 +- .../coordinator/TestCoordinatorGateway.java | 15 +- .../statemachine/ReplicaStateMachineTest.java | 38 ++- .../TableBucketStateMachineTest.java | 18 +- .../fluss/server/log/FetchParamsTest.java | 8 +- .../server/log/remote/RemoteLogTestBase.java | 8 +- .../fluss/server/replica/AdjustIsrITCase.java | 13 +- .../server/replica/AdjustIsrManagerTest.java | 40 ++- .../fluss/server/replica/AdjustIsrTest.java | 7 +- .../server/replica/FollowerReplicaTest.java | 7 +- .../replica/HighWatermarkPersistenceTest.java | 10 +- .../server/replica/ReplicaManagerTest.java | 31 +- .../fluss/server/replica/ReplicaTest.java | 38 +-- .../fluss/server/replica/ReplicaTestBase.java | 38 ++- .../replica/fetcher/ReplicaFetcherITCase.java | 13 +- .../fetcher/ReplicaFetcherManagerTest.java | 11 +- .../fetcher/ReplicaFetcherThreadTest.java | 22 +- .../fetcher/TestingLeaderEndpoint.java | 9 +- .../replica/standby/AdjustIssrTest.java | 139 +++++++++ .../standby/KvStandbyManagerITCase.java | 165 ++++++++++ .../server/tablet/TabletServiceITCase.java | 28 +- .../fluss/server/zk/ZooKeeperClientTest.java | 21 +- .../zk/data/LeaderAndIsrJsonSerdeTest.java | 64 +++- 56 files changed, 2116 insertions(+), 509 deletions(-) create mode 100644 fluss-common/src/main/java/org/apache/fluss/remote/RemoteFileDownloader.java rename fluss-server/src/main/java/org/apache/fluss/server/kv/{KvRecoverHelper.java => KvApplyLogHelper.java} (58%) create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/replica/standby/BecomeHotStandbyTask.java create mode 100644 fluss-server/src/main/java/org/apache/fluss/server/replica/standby/KvStandbyManager.java create mode 100644 fluss-server/src/test/java/org/apache/fluss/server/replica/standby/AdjustIssrTest.java create mode 100644 fluss-server/src/test/java/org/apache/fluss/server/replica/standby/KvStandbyManagerITCase.java diff --git a/fluss-client/src/main/java/org/apache/fluss/client/FlussConnection.java b/fluss-client/src/main/java/org/apache/fluss/client/FlussConnection.java index e6a31b9f3b..5b6ccae477 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/FlussConnection.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/FlussConnection.java @@ -23,7 +23,6 @@ import org.apache.fluss.client.metadata.MetadataUpdater; import org.apache.fluss.client.table.FlussTable; import org.apache.fluss.client.table.Table; -import org.apache.fluss.client.table.scanner.RemoteFileDownloader; import org.apache.fluss.client.token.DefaultSecurityTokenManager; import org.apache.fluss.client.token.DefaultSecurityTokenProvider; import org.apache.fluss.client.token.SecurityTokenManager; @@ -36,6 +35,7 @@ import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePath; import org.apache.fluss.metrics.registry.MetricRegistry; +import org.apache.fluss.remote.RemoteFileDownloader; import org.apache.fluss.rpc.GatewayClientProxy; import org.apache.fluss.rpc.RpcClient; import org.apache.fluss.rpc.gateway.AdminReadOnlyGateway; diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/batch/KvSnapshotBatchScanner.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/batch/KvSnapshotBatchScanner.java index 2c749c6628..91ad035807 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/batch/KvSnapshotBatchScanner.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/batch/KvSnapshotBatchScanner.java @@ -18,11 +18,11 @@ package org.apache.fluss.client.table.scanner.batch; import org.apache.fluss.annotation.Internal; -import org.apache.fluss.client.table.scanner.RemoteFileDownloader; import org.apache.fluss.exception.FlussRuntimeException; import org.apache.fluss.fs.FsPathAndFileName; import org.apache.fluss.metadata.KvFormat; import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.remote.RemoteFileDownloader; import org.apache.fluss.row.InternalRow; import org.apache.fluss.types.RowType; import org.apache.fluss.utils.CloseableIterator; diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetcher.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetcher.java index d195dc6f0a..b5fa27a250 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetcher.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogFetcher.java @@ -21,7 +21,6 @@ import org.apache.fluss.annotation.VisibleForTesting; import org.apache.fluss.client.metadata.MetadataUpdater; import org.apache.fluss.client.metrics.ScannerMetricGroup; -import org.apache.fluss.client.table.scanner.RemoteFileDownloader; import org.apache.fluss.client.table.scanner.ScanRecord; import org.apache.fluss.cluster.BucketLocation; import org.apache.fluss.config.ConfigOptions; @@ -37,6 +36,7 @@ import org.apache.fluss.record.LogRecordReadContext; import org.apache.fluss.record.LogRecords; import org.apache.fluss.record.MemoryLogRecords; +import org.apache.fluss.remote.RemoteFileDownloader; import org.apache.fluss.remote.RemoteLogFetchInfo; import org.apache.fluss.remote.RemoteLogSegment; import org.apache.fluss.rpc.entity.FetchLogResultForBucket; @@ -406,6 +406,7 @@ private Map prepareFetchLogRequests() { tb); // try to get the latest metadata info of this table because the leader for this // bucket is unknown. + // TODO tables to update once. metadataUpdater.updateTableOrPartitionMetadata(tablePath, tb.getPartitionId()); } else if (nodesWithPendingFetchRequests.contains(leader)) { LOG.trace( diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java index cef05808fb..6a74deef0c 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/LogScannerImpl.java @@ -20,13 +20,13 @@ import org.apache.fluss.annotation.PublicEvolving; import org.apache.fluss.client.metadata.MetadataUpdater; import org.apache.fluss.client.metrics.ScannerMetricGroup; -import org.apache.fluss.client.table.scanner.RemoteFileDownloader; import org.apache.fluss.client.table.scanner.ScanRecord; import org.apache.fluss.config.Configuration; import org.apache.fluss.exception.WakeupException; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableInfo; import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.remote.RemoteFileDownloader; import org.apache.fluss.rpc.metrics.ClientMetricGroup; import org.apache.fluss.types.RowType; import org.apache.fluss.utils.Projection; diff --git a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/RemoteLogDownloader.java b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/RemoteLogDownloader.java index 20b5ca5b84..80f477c76e 100644 --- a/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/RemoteLogDownloader.java +++ b/fluss-client/src/main/java/org/apache/fluss/client/table/scanner/log/RemoteLogDownloader.java @@ -20,12 +20,12 @@ import org.apache.fluss.annotation.Internal; import org.apache.fluss.annotation.VisibleForTesting; import org.apache.fluss.client.metrics.ScannerMetricGroup; -import org.apache.fluss.client.table.scanner.RemoteFileDownloader; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; import org.apache.fluss.fs.FsPath; import org.apache.fluss.fs.FsPathAndFileName; import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.remote.RemoteFileDownloader; import org.apache.fluss.remote.RemoteLogSegment; import org.apache.fluss.utils.ExceptionUtils; import org.apache.fluss.utils.FlussPaths; diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvSnapshotBatchScannerITCase.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvSnapshotBatchScannerITCase.java index b3f1eee8f7..4eab105a7a 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvSnapshotBatchScannerITCase.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/batch/KvSnapshotBatchScannerITCase.java @@ -21,7 +21,6 @@ import org.apache.fluss.client.admin.ClientToServerITCaseBase; import org.apache.fluss.client.metadata.KvSnapshots; import org.apache.fluss.client.table.Table; -import org.apache.fluss.client.table.scanner.RemoteFileDownloader; import org.apache.fluss.client.table.writer.UpsertWriter; import org.apache.fluss.client.write.HashBucketAssigner; import org.apache.fluss.metadata.DataLakeFormat; @@ -29,6 +28,7 @@ import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableDescriptor; import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.remote.RemoteFileDownloader; import org.apache.fluss.row.InternalRow; import org.apache.fluss.row.encode.CompactedKeyEncoder; import org.apache.fluss.row.encode.KeyEncoder; diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherTest.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherTest.java index 634bcb5ea0..fc52470276 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/LogFetcherTest.java @@ -20,12 +20,12 @@ import org.apache.fluss.client.admin.ClientToServerITCaseBase; import org.apache.fluss.client.metadata.MetadataUpdater; import org.apache.fluss.client.metrics.TestingScannerMetricGroup; -import org.apache.fluss.client.table.scanner.RemoteFileDownloader; import org.apache.fluss.client.table.scanner.ScanRecord; import org.apache.fluss.cluster.Cluster; import org.apache.fluss.cluster.ServerNode; import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.record.MemoryLogRecords; +import org.apache.fluss.remote.RemoteFileDownloader; import org.apache.fluss.rpc.RpcClient; import org.apache.fluss.rpc.gateway.TabletServerGateway; import org.apache.fluss.rpc.messages.PbProduceLogRespForBucket; diff --git a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteLogDownloaderTest.java b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteLogDownloaderTest.java index 407756a7eb..1e5cc9e824 100644 --- a/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteLogDownloaderTest.java +++ b/fluss-client/src/test/java/org/apache/fluss/client/table/scanner/log/RemoteLogDownloaderTest.java @@ -19,13 +19,13 @@ import org.apache.fluss.client.metrics.ScannerMetricGroup; import org.apache.fluss.client.metrics.TestingScannerMetricGroup; -import org.apache.fluss.client.table.scanner.RemoteFileDownloader; import org.apache.fluss.client.table.scanner.log.RemoteLogDownloader.RemoteLogDownloadRequest; import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; import org.apache.fluss.fs.FsPath; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.remote.RemoteFileDownloader; import org.apache.fluss.remote.RemoteLogSegment; import org.apache.fluss.utils.FileUtils; import org.apache.fluss.utils.IOUtils; 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 index 42be9aba5a..345d5b5d9c 100644 --- 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 @@ -19,9 +19,6 @@ import org.apache.fluss.annotation.PublicEvolving; -import javax.annotation.Nullable; - -import java.util.Collections; import java.util.List; import static org.apache.fluss.cluster.rebalance.RebalanceStatusForBucket.FAILED; @@ -34,50 +31,31 @@ @PublicEvolving public class RebalanceResultForBucket { - private final @Nullable Integer originalLeader; - private final @Nullable Integer targetLeader; + private final int originalLeader; + private final int newLeader; private final List originReplicas; private final List targetReplicas; private RebalanceStatusForBucket rebalanceStatusForBucket; public RebalanceResultForBucket( - @Nullable Integer originalLeader, - @Nullable Integer targetLeader, - RebalanceStatusForBucket rebalanceStatusForBucket) { - this( - originalLeader, - targetLeader, - Collections.emptyList(), - Collections.emptyList(), - rebalanceStatusForBucket); - } - - public RebalanceResultForBucket( - List originReplicas, - List targetReplicas, - RebalanceStatusForBucket rebalanceStatusForBucket) { - this(null, null, originReplicas, targetReplicas, rebalanceStatusForBucket); - } - - public RebalanceResultForBucket( - @Nullable Integer originalLeader, - @Nullable Integer targetLeader, + int originalLeader, + int newLeader, List originReplicas, List targetReplicas, RebalanceStatusForBucket rebalanceStatusForBucket) { this.originalLeader = originalLeader; - this.targetLeader = targetLeader; + this.newLeader = newLeader; this.originReplicas = originReplicas; this.targetReplicas = targetReplicas; this.rebalanceStatusForBucket = rebalanceStatusForBucket; } - public @Nullable Integer originalLeader() { + public int originalLeader() { return originalLeader; } - public @Nullable Integer targetLeader() { - return targetLeader; + public int targetLeader() { + return newLeader; } public List replicas() { @@ -98,14 +76,6 @@ public RebalanceResultForBucket markCompleted() { return this; } - public boolean isLeaderAction() { - return originalLeader != null && targetLeader != null; - } - - public RebalanceStatusForBucket rebalanceStatus() { - return rebalanceStatusForBucket; - } - public static RebalanceResultForBucket of( RebalancePlanForBucket planForBucket, RebalanceStatusForBucket status) { return new RebalanceResultForBucket( @@ -121,8 +91,8 @@ public String toString() { return "RebalanceResultForBucket{" + "originalLeader=" + originalLeader - + ", targetLeader=" - + targetLeader + + ", newLeader=" + + newLeader + ", originReplicas=" + originReplicas + ", targetReplicas=" diff --git a/fluss-common/src/main/java/org/apache/fluss/remote/RemoteFileDownloader.java b/fluss-common/src/main/java/org/apache/fluss/remote/RemoteFileDownloader.java new file mode 100644 index 0000000000..a0bad1c564 --- /dev/null +++ b/fluss-common/src/main/java/org/apache/fluss/remote/RemoteFileDownloader.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.remote; + +import org.apache.fluss.fs.FSDataInputStream; +import org.apache.fluss.fs.FileSystem; +import org.apache.fluss.fs.FsPath; +import org.apache.fluss.fs.FsPathAndFileName; +import org.apache.fluss.fs.utils.FileDownloadSpec; +import org.apache.fluss.fs.utils.FileDownloadUtils; +import org.apache.fluss.utils.CloseableRegistry; +import org.apache.fluss.utils.IOUtils; +import org.apache.fluss.utils.concurrent.ExecutorThreadFactory; + +import java.io.Closeable; +import java.io.IOException; +import java.io.OutputStream; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +/** + * The downloader that has a IO thread pool to download the remote files (like kv snapshots files, + * log segment files). + */ +public class RemoteFileDownloader implements Closeable { + + protected final ExecutorService downloadThreadPool; + + public RemoteFileDownloader(int threadNum) { + downloadThreadPool = + Executors.newFixedThreadPool( + threadNum, + new ExecutorThreadFactory( + "fluss-client-remote-file-downloader", + // use the current classloader of the current thread as the given + // classloader of the thread created by the ExecutorThreadFactory + // to avoid use weird classloader provided by + // CompletableFuture.runAsync of method #initReaderAsynchronously + Thread.currentThread().getContextClassLoader())); + } + + /** + * Downloads the file from the given remote file path to the target directory asynchronously, + * returns a Future object of the number of downloaded bytes. The Future will fail if the + * download fails after retrying for RETRY_COUNT times. + */ + public CompletableFuture downloadFileAsync( + FsPathAndFileName fsPathAndFileName, Path targetDirectory) { + CompletableFuture future = new CompletableFuture<>(); + downloadThreadPool.submit( + () -> { + try { + Path targetFilePath = + targetDirectory.resolve(fsPathAndFileName.getFileName()); + FsPath remoteFilePath = fsPathAndFileName.getPath(); + long downloadBytes = downloadFile(targetFilePath, remoteFilePath); + future.complete(downloadBytes); + } catch (Exception e) { + future.completeExceptionally(e); + } + }); + return future; + } + + /** + * Copies the file from a remote file path to the given target file path, returns the number of + * downloaded bytes. + */ + protected long downloadFile(Path targetFilePath, FsPath remoteFilePath) throws IOException { + List closeableRegistry = new ArrayList<>(2); + try { + FileSystem fileSystem = remoteFilePath.getFileSystem(); + FSDataInputStream inputStream = fileSystem.open(remoteFilePath); + closeableRegistry.add(inputStream); + + Files.createDirectories(targetFilePath.getParent()); + OutputStream outputStream = Files.newOutputStream(targetFilePath); + closeableRegistry.add(outputStream); + + return IOUtils.copyBytes(inputStream, outputStream, false); + } catch (Exception ex) { + throw new IOException(ex); + } finally { + closeableRegistry.forEach(IOUtils::closeQuietly); + } + } + + @Override + public void close() throws IOException { + downloadThreadPool.shutdownNow(); + } + + public void transferAllToDirectory( + List fsPathAndFileNames, + Path targetDirectory, + CloseableRegistry closeableRegistry) + throws IOException { + FileDownloadSpec fileDownloadSpec = + new FileDownloadSpec(fsPathAndFileNames, targetDirectory); + FileDownloadUtils.transferAllDataToDirectory( + Collections.singleton(fileDownloadSpec), closeableRegistry, downloadThreadPool); + } +} diff --git a/fluss-rpc/src/main/proto/FlussApi.proto b/fluss-rpc/src/main/proto/FlussApi.proto index 439c89dcb8..408fd07218 100644 --- a/fluss-rpc/src/main/proto/FlussApi.proto +++ b/fluss-rpc/src/main/proto/FlussApi.proto @@ -656,6 +656,7 @@ message PbFetchLogReqForBucket { // TODO leader epoch required int64 fetch_offset = 3; required int32 max_fetch_bytes = 4; + optional int64 kv_applied_offset = 5; } message PbFetchLogRespForTable { @@ -742,6 +743,8 @@ message PbAdjustIsrReqForBucket { repeated int32 new_isr = 4 [packed = true]; required int32 coordinator_epoch = 5; required int32 bucket_epoch = 6; + // inSyncStandbyReplicas + repeated int32 new_issr = 7 [packed = true]; } message PbAdjustIsrRespForTable { @@ -759,6 +762,8 @@ message PbAdjustIsrRespForBucket { repeated int32 isr = 7 [packed = true]; optional int32 bucket_epoch = 8; optional int32 coordinator_epoch = 9; + repeated int32 hot_standby_replicas = 10 [packed = true]; + repeated int32 issr = 11 [packed = true]; } message PbListOffsetsRespForBucket { @@ -779,6 +784,8 @@ message PbNotifyLeaderAndIsrReqForBucket { repeated int32 replicas = 5 [packed = true]; repeated int32 isr = 6 [packed = true]; required int32 bucket_epoch = 7; + repeated int32 hot_standby_replicas = 8 [packed = true]; + repeated int32 issr = 9 [packed = true]; } message PbNotifyLeaderAndIsrRespForBucket { 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 b063a39791..38010ff0e0 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 @@ -963,6 +963,8 @@ private void tryToFinishRebalanceTasks(TableBucket tableBucket) { if (rebalanceResultForBucket != null) { ReplicaReassignment reassignment = ReplicaReassignment.build( + rebalanceResultForBucket.originalLeader(), + rebalanceResultForBucket.targetLeader(), coordinatorContext.getAssignment(tableBucket), rebalanceResultForBucket.targetReplicas()); try { @@ -983,23 +985,6 @@ private void tryToFinishRebalanceTasks(TableBucket tableBucket) { tableBucket, coordinatorContext.removeOngoingRebalanceTask(tableBucket).markFailed()); } - - // if (rebalanceResultForBucket.isLeaderAction()) { - // List assignedReplicas = - // coordinatorContext.getAssignment(tableBucket); - // int preferredReplica = assignedReplicas.get(0); - // int currentLeader = - // - // coordinatorContext.getBucketLeaderAndIsr(tableBucket).get().leader(); - // if (currentLeader == preferredReplica) { - // coordinatorContext.putFinishedRebalanceTask( - // tableBucket, - // coordinatorContext - // .removeOngoingRebalanceTask(tableBucket) - // .markCompleted()); - // } - // } else { - // } } // judge whether the rebalance task is finished @@ -1050,7 +1035,7 @@ private void tryToProcessRegisterRebalanceTask(ExecuteRebalanceTaskEvent event) } // buckets only need to change leader like preferred leader election and leader replica - // rebalance. + // rebalance. (Excluding for PrimaryKey Table) Set electableBuckets = new HashSet<>(); // buckets to do reassignments. Map reassignments = new HashMap<>(); @@ -1083,42 +1068,25 @@ private void tryToProcessRegisterRebalanceTask(ExecuteRebalanceTaskEvent event) continue; } - // if (planForBucket.isLeaderAction()) { - // List assignedReplicas = - // coordinatorContext.getAssignment(tableBucket); - // int preferredReplica = assignedReplicas.get(0); - // int currentLeader = - // - // coordinatorContext.getBucketLeaderAndIsr(tableBucket).get().leader(); - // if (currentLeader != preferredReplica) { - // electableBuckets.add(tableBucket); - // coordinatorContext.putOngoingRebalanceTask( - // tableBucket, - // new RebalanceResultForBucket( - // planForBucket.getOriginalLeader(), - // planForBucket.getNewLeader(), - // RebalanceStatusForBucket.PENDING)); - // } else { - // // already finished. - // coordinatorContext.putFinishedRebalanceTask( - // tableBucket, - // RebalanceResultForBucket.of( - // planForBucket, - // RebalanceStatusForBucket.COMPLETED)); - // } - // } else { - // } - List newReplicas = planForBucket.getNewReplicas(); ReplicaReassignment reassignment = ReplicaReassignment.build( - coordinatorContext.getAssignment(tableBucket), newReplicas); - if (planForBucket.isLeaderAction() && !reassignment.isBeingReassigned()) { + planForBucket.getOriginalLeader(), + planForBucket.getNewLeader(), + coordinatorContext.getAssignment(tableBucket), + newReplicas); + if (planForBucket.isLeaderAction() + && !reassignment.isBeingReassigned() + && !coordinatorContext + .getTableInfoById(tableBucket.getTableId()) + .hasPrimaryKey()) { LOG.info("trigger leader election for tableBucket {}.", tableBucket); electableBuckets.add(tableBucket); coordinatorContext.putOngoingRebalanceTask( tableBucket, new RebalanceResultForBucket( + planForBucket.getOriginalLeader(), + planForBucket.getNewLeader(), planForBucket.getOriginReplicas(), planForBucket.getNewReplicas(), RebalanceStatusForBucket.PENDING)); @@ -1129,19 +1097,11 @@ private void tryToProcessRegisterRebalanceTask(ExecuteRebalanceTaskEvent event) coordinatorContext.putOngoingRebalanceTask( tableBucket, new RebalanceResultForBucket( + planForBucket.getOriginalLeader(), + planForBucket.getNewLeader(), planForBucket.getOriginReplicas(), planForBucket.getNewReplicas(), RebalanceStatusForBucket.PENDING)); - - // if (reassignment.isBeingReassigned()) { - // - // } else { - // // already finished. - // coordinatorContext.putFinishedRebalanceTask( - // tableBucket, - // RebalanceResultForBucket.of( - // planForBucket, RebalanceStatusForBucket.COMPLETED)); - // } } // try to trigger leader election together. @@ -1299,7 +1259,11 @@ private void onBucketReassignment(TableBucket tableBucket, ReplicaReassignment r 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); + maybeReassignedBucketLeaderIfRequired( + tableBucket, + reassignment.originalLeader, + reassignment.newLeader, + 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); @@ -1324,18 +1288,26 @@ private void onBucketReassignment(TableBucket tableBucket, ReplicaReassignment r private boolean isReassignmentComplete( TableBucket tableBucket, ReplicaReassignment reassignment) throws Exception { - // if (!reassignment.isBeingReassigned()) { - // return true; - // } - LeaderAndIsr leaderAndIsr = zooKeeperClient.getLeaderAndIsr(tableBucket).get(); List isr = leaderAndIsr.isr(); List targetReplicas = reassignment.getTargetReplicas(); - return targetReplicas.isEmpty() || new HashSet<>(isr).containsAll(targetReplicas); + + boolean isPrimaryKeyTable = + coordinatorContext.getTableInfoById(tableBucket.getTableId()).hasPrimaryKey(); + if (isPrimaryKeyTable) { + return new HashSet<>(leaderAndIsr.issr()).contains(reassignment.newLeader) + && (targetReplicas.isEmpty() || new HashSet<>(isr).containsAll(targetReplicas)); + } else { + return targetReplicas.isEmpty() || new HashSet<>(isr).containsAll(targetReplicas); + } } private void maybeReassignedBucketLeaderIfRequired( - TableBucket tableBucket, List targetReplicas) throws Exception { + TableBucket tableBucket, + int originalLeader, + int newLeader, + List targetReplicas) + throws Exception { int currentLeader = coordinatorContext.getBucketLeaderAndIsr(tableBucket).get().leader(); if (!targetReplicas.contains(currentLeader)) { LOG.info( @@ -1356,7 +1328,11 @@ private void maybeReassignedBucketLeaderIfRequired( updateLeaderEpochAndSendRequest( tableBucket, new ReplicaReassignment( - targetReplicas, Collections.emptyList(), Collections.emptyList())); + targetReplicas, + originalLeader, + newLeader, + Collections.emptyList(), + Collections.emptyList())); } else { LOG.info( "Leader {} for tableBucket {} being reassigned, is already in the new list of replicas {} but is dead", @@ -1435,18 +1411,19 @@ private List tryProcessAdjustIsr( new FlussRuntimeException( "Leader not found for table bucket " + tableBucket)); - LeaderAndIsr newLeaderAndIsr = - new LeaderAndIsr( - // the leaderEpoch in request has been validated to be equal to current - // leaderEpoch, which means the leader is still the same, so we use - // leader and leaderEpoch in currentLeaderAndIsr. - currentLeaderAndIsr.leader(), - currentLeaderAndIsr.leaderEpoch(), - // TODO: reject the request if there is a replica in ISR is not online, - // see KIP-841. - tryAdjustLeaderAndIsr.isr(), - coordinatorContext.getCoordinatorEpoch(), - currentLeaderAndIsr.bucketEpoch() + 1); + + LeaderAndIsr.Builder builder = new LeaderAndIsr.Builder(); + // the leaderEpoch in request has been validated to be equal to current + // leaderEpoch, which means the leader is still the same, so we use + // leader and leaderEpoch in currentLeaderAndIsr. + builder.leader(currentLeaderAndIsr.leader()) + .leaderEpoch(currentLeaderAndIsr.leaderEpoch()); + // TODO: reject the request if there is a replica in ISR is not online, + // see KIP-841. + builder.isr(tryAdjustLeaderAndIsr.isr()) + .coordinatorEpoch(currentLeaderAndIsr.coordinatorEpoch()) + .bucketEpoch(currentLeaderAndIsr.bucketEpoch() + 1); + LeaderAndIsr newLeaderAndIsr = builder.build(); newLeaderAndIsrList.put(tableBucket, newLeaderAndIsr); } @@ -1743,11 +1720,13 @@ private void updateTabletServerMetadataCache( private void updateLeaderEpochAndSendRequest( TableBucket tableBucket, ReplicaReassignment reassignment) throws Exception { - LeaderAndIsr leaderAndIsr = updateLeaderEpoch(tableBucket); - if (leaderAndIsr == null) { + Optional leaderAndIsrOpt = + coordinatorContext.getBucketLeaderAndIsr(tableBucket); + if (!leaderAndIsrOpt.isPresent()) { return; } + LeaderAndIsr leaderAndIsr = leaderAndIsrOpt.get(); String partitionName = null; if (tableBucket.getPartitionId() != null) { partitionName = coordinatorContext.getPartitionName(tableBucket.getPartitionId()); @@ -1757,6 +1736,16 @@ private void updateLeaderEpochAndSendRequest( } } + // For kv table, we need to add hot standby replica. + if (coordinatorContext.getTableInfoById(tableBucket.getTableId()).hasPrimaryKey() + && reassignment.isLeaderAction()) { + int newLeader = reassignment.getNewLeader(); + // Set issr to empty list. + leaderAndIsr = + leaderAndIsr.newLeaderAndIsr( + Collections.singletonList(newLeader), Collections.emptyList()); + } + coordinatorRequestBatch.newBatch(); coordinatorRequestBatch.addNotifyLeaderRequestForTabletServers( new HashSet<>(reassignment.replicas), @@ -1778,17 +1767,6 @@ private void updateLeaderEpochAndSendRequest( return null; } LeaderAndIsr leaderAndIsr = leaderAndIsrOpt.get(); - // increment the leader epoch even if there are no leader or isr changes to allow the - // leader to cache the expanded assigned replica list. - // LeaderAndIsr newLeaderAndIsr = leaderAndIsr.newLeaderAndIsrWithNewLeaderEpoch(); - // zooKeeperClient.updateLeaderAndIsr(tableBucket, newLeaderAndIsr); - // update leader and isr - // coordinatorContext.putBucketLeaderAndIsr(tableBucket, newLeaderAndIsr); - // LOG.info( - // "Updated leader epoch for tableBucket {} from {} to {}", - // tableBucket, - // leaderAndIsr, - // newLeaderAndIsr); return leaderAndIsr; } @@ -1799,20 +1777,29 @@ CompletedSnapshotStoreManager completedSnapshotStoreManager() { private static final class ReplicaReassignment { private final List replicas; + private final int originalLeader; + private final int newLeader; private final List addingReplicas; private final List removingReplicas; private ReplicaReassignment( List replicas, + int originalLeader, + int newLeader, List addingReplicas, List removingReplicas) { this.replicas = Collections.unmodifiableList(replicas); + this.originalLeader = originalLeader; + this.newLeader = newLeader; this.addingReplicas = Collections.unmodifiableList(addingReplicas); this.removingReplicas = Collections.unmodifiableList(removingReplicas); } private static ReplicaReassignment build( - List originReplicas, List targetReplicas) { + int originalLeader, + int newLeader, + List originReplicas, + List targetReplicas) { // targetReplicas behind originReplicas in full set. List fullReplicaSet = new ArrayList<>(targetReplicas); fullReplicaSet.addAll(originReplicas); @@ -1824,7 +1811,16 @@ private static ReplicaReassignment build( List newRemovingReplicas = new ArrayList<>(originReplicas); newRemovingReplicas.removeAll(targetReplicas); - return new ReplicaReassignment(fullReplicaSet, newAddingReplicas, newRemovingReplicas); + return new ReplicaReassignment( + fullReplicaSet, + originalLeader, + newLeader, + newAddingReplicas, + newRemovingReplicas); + } + + private int getNewLeader() { + return newLeader; } private List getTargetReplicas() { @@ -1833,10 +1829,8 @@ private List getTargetReplicas() { return Collections.unmodifiableList(computed); } - private List getOriginReplicas() { - List computed = new ArrayList<>(replicas); - computed.removeAll(addingReplicas); - return Collections.unmodifiableList(computed); + private boolean isLeaderAction() { + return originalLeader != newLeader; } private boolean isBeingReassigned() { @@ -1846,8 +1840,8 @@ private boolean isBeingReassigned() { @Override public String toString() { return String.format( - "ReplicaAssignment(replicas=%s, addingReplicas=%s, removingReplicas=%s)", - replicas, addingReplicas, removingReplicas); + "ReplicaAssignment(replicas=%s, originalLeader=%s, newLeader=%s, addingReplicas=%s, removingReplicas=%s)", + replicas, originalLeader, newLeader, addingReplicas, removingReplicas); } @Override @@ -1863,12 +1857,15 @@ public boolean equals(Object o) { ReplicaReassignment that = (ReplicaReassignment) o; return Objects.equals(replicas, that.replicas) && Objects.equals(addingReplicas, that.addingReplicas) - && Objects.equals(removingReplicas, that.removingReplicas); + && Objects.equals(removingReplicas, that.removingReplicas) + && Objects.equals(originalLeader, that.originalLeader) + && Objects.equals(newLeader, that.newLeader); } @Override public int hashCode() { - return Objects.hash(replicas, addingReplicas, removingReplicas); + return Objects.hash( + replicas, originalLeader, newLeader, addingReplicas, removingReplicas); } } } 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 41a19e9be6..b3654d37fb 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 @@ -438,7 +438,11 @@ private Optional doInitElectionForBucket( // Register the initial leader and isr. LeaderAndIsr leaderAndIsr = - new LeaderAndIsr(leader, 0, isr, coordinatorContext.getCoordinatorEpoch(), 0); + new LeaderAndIsr.Builder() + .leader(leader) + .isr(isr) + .coordinatorEpoch(coordinatorContext.getCoordinatorEpoch()) + .build(); return Optional.of(new ElectionResult(liveServers, leaderAndIsr)); } @@ -628,12 +632,13 @@ private Optional leaderForOffline( // get the updated leader and isr LeaderAndIsr newLeaderAndIsr = - new LeaderAndIsr( - leaderOpt.get(), - leaderAndIsr.leaderEpoch() + 1, - leaderAndIsr.isr(), - coordinatorContext.getCoordinatorEpoch(), - leaderAndIsr.bucketEpoch() + 1); + new LeaderAndIsr.Builder() + .leader(leaderOpt.get()) + .leaderEpoch(leaderAndIsr.leaderEpoch() + 1) + .isr(leaderAndIsr.isr()) + .coordinatorEpoch(coordinatorContext.getCoordinatorEpoch()) + .bucketEpoch(leaderAndIsr.bucketEpoch() + 1) + .build(); return Optional.of(new ElectionResult(liveReplicas, newLeaderAndIsr)); } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/entity/AdjustIsrResultForBucket.java b/fluss-server/src/main/java/org/apache/fluss/server/entity/AdjustIsrResultForBucket.java index 9e4594cdb1..4464056a46 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/entity/AdjustIsrResultForBucket.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/entity/AdjustIsrResultForBucket.java @@ -32,7 +32,7 @@ public AdjustIsrResultForBucket(TableBucket tableBucket, LeaderAndIsr leaderAndI } public AdjustIsrResultForBucket(TableBucket tableBucket, ApiError error) { - this(tableBucket, new LeaderAndIsr(-1, 0), error); + this(tableBucket, new LeaderAndIsr.Builder().leader(-1).leaderEpoch(0).build(), error); } private AdjustIsrResultForBucket( diff --git a/fluss-server/src/main/java/org/apache/fluss/server/entity/FetchReqInfo.java b/fluss-server/src/main/java/org/apache/fluss/server/entity/FetchReqInfo.java index 0f8df68002..68f24255e2 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/entity/FetchReqInfo.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/entity/FetchReqInfo.java @@ -30,19 +30,25 @@ public final class FetchReqInfo { private final long tableId; private final long fetchOffset; @Nullable private final int[] projectFields; + @Nullable private final Long kvAppliedOffset; private int maxBytes; public FetchReqInfo(long tableId, long fetchOffset, int maxBytes) { - this(tableId, fetchOffset, maxBytes, null); + this(tableId, fetchOffset, maxBytes, null, null); } public FetchReqInfo( - long tableId, long fetchOffset, int maxBytes, @Nullable int[] projectFields) { + long tableId, + long fetchOffset, + int maxBytes, + @Nullable int[] projectFields, + @Nullable Long kvAppliedOffset) { this.tableId = tableId; this.fetchOffset = fetchOffset; this.maxBytes = maxBytes; this.projectFields = projectFields; + this.kvAppliedOffset = kvAppliedOffset; } public long getTableId() { @@ -66,6 +72,11 @@ public int[] getProjectFields() { return projectFields; } + @Nullable + public Long getKvAppliedOffset() { + return kvAppliedOffset; + } + @Override public String toString() { return "FetchData{" @@ -77,6 +88,8 @@ public String toString() { + maxBytes + ", projectionFields=" + Arrays.toString(projectFields) + + ", kvAppliedOffset=" + + kvAppliedOffset + '}'; } @@ -97,11 +110,14 @@ public boolean equals(Object o) { return false; } - return fetchOffset == fetchReqInfo.fetchOffset && maxBytes == fetchReqInfo.maxBytes; + return fetchOffset == fetchReqInfo.fetchOffset + && maxBytes == fetchReqInfo.maxBytes + && Objects.equals(kvAppliedOffset, fetchReqInfo.kvAppliedOffset); } @Override public int hashCode() { - return Objects.hash(tableId, fetchOffset, maxBytes, Arrays.hashCode(projectFields)); + return Objects.hash( + tableId, fetchOffset, maxBytes, Arrays.hashCode(projectFields), kvAppliedOffset); } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/entity/NotifyLeaderAndIsrData.java b/fluss-server/src/main/java/org/apache/fluss/server/entity/NotifyLeaderAndIsrData.java index 8ba0002084..4e3303c268 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/entity/NotifyLeaderAndIsrData.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/entity/NotifyLeaderAndIsrData.java @@ -78,10 +78,26 @@ public List getIsr() { return leaderAndIsr.isr(); } + public List getHotStandbyReplicas() { + return leaderAndIsr.standbyList(); + } + public int[] getIsrArray() { return leaderAndIsr.isr().stream().mapToInt(Integer::intValue).toArray(); } + public List getIssr() { + return leaderAndIsr.issr(); + } + + public int[] getStandbyArray() { + return leaderAndIsr.standbyList().stream().mapToInt(Integer::intValue).toArray(); + } + + public int[] getIssrArray() { + return leaderAndIsr.issr().stream().mapToInt(Integer::intValue).toArray(); + } + public LeaderAndIsr getLeaderAndIsr() { return leaderAndIsr; } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/KvRecoverHelper.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvApplyLogHelper.java similarity index 58% rename from fluss-server/src/main/java/org/apache/fluss/server/kv/KvRecoverHelper.java rename to fluss-server/src/main/java/org/apache/fluss/server/kv/KvApplyLogHelper.java index 9057598a19..a8d061e8b4 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/KvRecoverHelper.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvApplyLogHelper.java @@ -20,9 +20,7 @@ import org.apache.fluss.exception.KvStorageException; import org.apache.fluss.metadata.DataLakeFormat; import org.apache.fluss.metadata.KvFormat; -import org.apache.fluss.metadata.TableBucket; import org.apache.fluss.metadata.TableInfo; -import org.apache.fluss.metadata.TablePath; import org.apache.fluss.record.ChangeType; import org.apache.fluss.record.LogRecord; import org.apache.fluss.record.LogRecordBatch; @@ -36,7 +34,7 @@ import org.apache.fluss.row.encode.ValueEncoder; import org.apache.fluss.row.indexed.IndexedRow; import org.apache.fluss.server.log.FetchIsolation; -import org.apache.fluss.server.log.LogTablet; +import org.apache.fluss.server.replica.Replica; import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.types.DataType; import org.apache.fluss.types.RowType; @@ -46,15 +44,13 @@ import javax.annotation.Nullable; import static org.apache.fluss.server.TabletManagerBase.getTableInfo; +import static org.apache.fluss.utils.Preconditions.checkNotNull; -/** A helper for recovering Kv from log. */ -public class KvRecoverHelper { +/** A helper for kv storage to apply log. */ +public class KvApplyLogHelper { - private final KvTablet kvTablet; - private final LogTablet logTablet; - private final long recoverPointOffset; - private final KvRecoverContext recoverContext; - private final KvFormat kvFormat; + private final ZooKeeperClient zkClient; + private final Replica replica; // will be initialized when first encounter a log record during recovering from log private Integer currentSchemaId; @@ -65,20 +61,12 @@ public class KvRecoverHelper { private InternalRow.FieldGetter[] currentFieldGetters; - public KvRecoverHelper( - KvTablet kvTablet, - LogTablet logTablet, - long recoverPointOffset, - KvRecoverContext recoverContext, - KvFormat kvFormat) { - this.kvTablet = kvTablet; - this.logTablet = logTablet; - this.recoverPointOffset = recoverPointOffset; - this.recoverContext = recoverContext; - this.kvFormat = kvFormat; + public KvApplyLogHelper(Replica replica, ZooKeeperClient zkClient) { + this.replica = replica; + this.zkClient = zkClient; } - public void recover() throws Exception { + public void recover(long recoverPointOffset, int maxFetchLogSize) throws Exception { // first step: read to high watermark and apply them to kv directly; that // 's for the data acked @@ -91,6 +79,8 @@ public void recover() throws Exception { long nextLogOffset = recoverPointOffset; // read to high watermark + KvTablet kvTablet = replica.getKvTablet(); + checkNotNull(kvTablet); try (KvBatchWriter kvBatchWriter = kvTablet.createKvBatchWriter()) { ThrowingConsumer resumeRecordApplier = (resumeRecord) -> { @@ -103,7 +93,10 @@ public void recover() throws Exception { nextLogOffset = readLogRecordsAndApply( - nextLogOffset, FetchIsolation.HIGH_WATERMARK, resumeRecordApplier); + nextLogOffset, + FetchIsolation.HIGH_WATERMARK, + resumeRecordApplier, + maxFetchLogSize); } // the all data up to nextLogOffset has been flush into kv @@ -114,65 +107,99 @@ public void recover() throws Exception { (resumeRecord) -> kvTablet.putToPreWriteBuffer( resumeRecord.key, resumeRecord.value, resumeRecord.logOffset); - readLogRecordsAndApply(nextLogOffset, FetchIsolation.LOG_END, resumeRecordApplier); + readLogRecordsAndApply( + nextLogOffset, FetchIsolation.LOG_END, resumeRecordApplier, maxFetchLogSize); } private long readLogRecordsAndApply( long startFetchOffset, FetchIsolation fetchIsolation, - ThrowingConsumer resumeRecordConsumer) + ThrowingConsumer resumeRecordConsumer, + int maxFetchLogSize) throws Exception { long nextFetchOffset = startFetchOffset; while (true) { LogRecords logRecords = - logTablet - .read( - nextFetchOffset, - recoverContext.maxFetchLogSizeInRecoverKv, - fetchIsolation, - true, - null) + replica.getLogTablet() + .read(nextFetchOffset, maxFetchLogSize, fetchIsolation, true, null) .getRecords(); if (logRecords == MemoryLogRecords.EMPTY) { break; } - for (LogRecordBatch logRecordBatch : logRecords.batches()) { - short schemaId = logRecordBatch.schemaId(); - if (currentSchemaId == null) { - initSchema(schemaId); - } else if (currentSchemaId != schemaId) { - throw new KvStorageException( - String.format( - "Can't recover kv tablet for table bucket from log %s since the schema changes from schema id %d to schema id %d. " - + "Currently, schema change is not supported.", - recoverContext.tableBucket, currentSchemaId, schemaId)); - } + nextFetchOffset = applyLogRecords(logRecords, nextFetchOffset, resumeRecordConsumer); + } + + return nextFetchOffset; + } - try (LogRecordReadContext readContext = - LogRecordReadContext.createArrowReadContext( - currentRowType, currentSchemaId); - CloseableIterator logRecordIter = - logRecordBatch.records(readContext)) { - while (logRecordIter.hasNext()) { - LogRecord logRecord = logRecordIter.next(); - if (logRecord.getChangeType() != ChangeType.UPDATE_BEFORE) { - InternalRow logRow = logRecord.getRow(); - byte[] key = keyEncoder.encodeKey(logRow); - byte[] value = null; - if (logRecord.getChangeType() != ChangeType.DELETE) { - // the log row format may not compatible with kv row format, - // e.g, arrow vs. compacted, thus needs a conversion here. - BinaryRow row = toKvRow(logRecord.getRow()); - value = ValueEncoder.encodeValue(schemaId, row); - } - resumeRecordConsumer.accept( - new KeyValueAndLogOffset(key, value, logRecord.logOffset())); + /** + * Apply log records to kv. + * + * @param logRecords log records to apply. + * @param startFetchOffset start fetch offset. + * @return next fetch offset. + */ + public long applyLogRecords(LogRecords logRecords, long startFetchOffset) throws Exception { + ThrowingConsumer resumeRecordApplier = + (resumeRecord) -> { + KvTablet kvTablet = replica.getKvTablet(); + checkNotNull(kvTablet); + kvTablet.putToPreWriteBuffer( + resumeRecord.key, resumeRecord.value, resumeRecord.logOffset); + }; + return applyLogRecords(logRecords, startFetchOffset, resumeRecordApplier); + } + + /** + * Apply log records to kv. + * + * @param logRecords log records to apply. + * @param startFetchOffset start fetch offset. + * @param resumeRecordConsumer resume record consumer. + * @return next fetch offset. + */ + private long applyLogRecords( + LogRecords logRecords, + long startFetchOffset, + ThrowingConsumer resumeRecordConsumer) + throws Exception { + long nextFetchOffset = startFetchOffset; + for (LogRecordBatch logRecordBatch : logRecords.batches()) { + short schemaId = logRecordBatch.schemaId(); + if (currentSchemaId == null) { + initSchema(schemaId); + } else if (currentSchemaId != schemaId) { + throw new KvStorageException( + String.format( + "Can't recover kv tablet for table bucket from log %s since the schema changes from schema id %d to schema id %d. " + + "Currently, schema change is not supported.", + replica.getTableBucket(), currentSchemaId, schemaId)); + } + + try (LogRecordReadContext readContext = + LogRecordReadContext.createArrowReadContext( + currentRowType, currentSchemaId); + CloseableIterator logRecordIter = + logRecordBatch.records(readContext)) { + while (logRecordIter.hasNext()) { + LogRecord logRecord = logRecordIter.next(); + if (logRecord.getChangeType() != ChangeType.UPDATE_BEFORE) { + InternalRow logRow = logRecord.getRow(); + byte[] key = keyEncoder.encodeKey(logRow); + byte[] value = null; + if (logRecord.getChangeType() != ChangeType.DELETE) { + // the log row format may not compatible with kv row format, + // e.g, arrow vs. compacted, thus needs a conversion here. + BinaryRow row = toKvRow(logRecord.getRow()); + value = ValueEncoder.encodeValue(schemaId, row); } + resumeRecordConsumer.accept( + new KeyValueAndLogOffset(key, value, logRecord.logOffset())); } } - nextFetchOffset = logRecordBatch.nextLogOffset(); } + nextFetchOffset = logRecordBatch.nextLogOffset(); } return nextFetchOffset; } @@ -180,7 +207,7 @@ private long readLogRecordsAndApply( // TODO: this is very in-efficient, because the conversion is CPU heavy. Should be optimized in // the future. private BinaryRow toKvRow(InternalRow originalRow) { - if (kvFormat == KvFormat.INDEXED) { + if (replica.getKvFormat() == KvFormat.INDEXED) { // if the row is in indexed row format, just return the original row directly if (originalRow instanceof IndexedRow) { return (IndexedRow) originalRow; @@ -198,7 +225,7 @@ private BinaryRow toKvRow(InternalRow originalRow) { private void initSchema(int schemaId) throws Exception { // todo, may need a cache, // but now, we get the schema from zk - TableInfo tableInfo = getTableInfo(recoverContext.zkClient, recoverContext.tablePath); + TableInfo tableInfo = getTableInfo(zkClient, replica.getTablePath()); // todo: we need to check the schema's table id is equal to the // kv tablet's table id or not. If not equal, it means other table with same // table path has been created, so the kv tablet's table is consider to be @@ -209,7 +236,7 @@ private void initSchema(int schemaId) throws Exception { DataLakeFormat lakeFormat = tableInfo.getTableConfig().getDataLakeFormat().orElse(null); keyEncoder = KeyEncoder.of(currentRowType, tableInfo.getPhysicalPrimaryKeys(), lakeFormat); - rowEncoder = RowEncoder.create(kvFormat, dataTypes); + rowEncoder = RowEncoder.create(replica.getKvFormat(), dataTypes); currentFieldGetters = new InternalRow.FieldGetter[currentRowType.getFieldCount()]; for (int i = 0; i < currentRowType.getFieldCount(); i++) { currentFieldGetters[i] = InternalRow.createFieldGetter(currentRowType.getTypeAt(i), i); @@ -221,31 +248,10 @@ private static final class KeyValueAndLogOffset { private final @Nullable byte[] value; private final long logOffset; - public KeyValueAndLogOffset(byte[] key, byte[] value, long logOffset) { + public KeyValueAndLogOffset(byte[] key, @Nullable byte[] value, long logOffset) { this.key = key; this.value = value; this.logOffset = logOffset; } } - - /** A context to provide necessary objects for kv recovering. */ - public static class KvRecoverContext { - - private final TablePath tablePath; - private final TableBucket tableBucket; - - private final ZooKeeperClient zkClient; - private final int maxFetchLogSizeInRecoverKv; - - public KvRecoverContext( - TablePath tablePath, - TableBucket tableBucket, - ZooKeeperClient zkClient, - int maxFetchLogSizeInRecoverKv) { - this.tablePath = tablePath; - this.tableBucket = tableBucket; - this.zkClient = zkClient; - this.maxFetchLogSizeInRecoverKv = maxFetchLogSizeInRecoverKv; - } - } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/kv/KvTablet.java b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvTablet.java index 4c3fde6976..1cead3779b 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/kv/KvTablet.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/kv/KvTablet.java @@ -22,6 +22,7 @@ import org.apache.fluss.config.ConfigOptions; import org.apache.fluss.config.Configuration; import org.apache.fluss.exception.KvStorageException; +import org.apache.fluss.exception.OutOfOrderSequenceException; import org.apache.fluss.memory.MemorySegmentPool; import org.apache.fluss.metadata.KvFormat; import org.apache.fluss.metadata.LogFormat; @@ -36,6 +37,7 @@ import org.apache.fluss.record.KvRecord; import org.apache.fluss.record.KvRecordBatch; import org.apache.fluss.record.KvRecordReadContext; +import org.apache.fluss.record.MemoryLogRecords; import org.apache.fluss.row.BinaryRow; import org.apache.fluss.row.arrow.ArrowWriterPool; import org.apache.fluss.row.arrow.ArrowWriterProvider; @@ -57,6 +59,7 @@ import org.apache.fluss.server.log.LogTablet; import org.apache.fluss.server.metrics.group.BucketMetricGroup; import org.apache.fluss.server.utils.FatalErrorHandler; +import org.apache.fluss.server.zk.data.LeaderAndIsr; import org.apache.fluss.shaded.arrow.org.apache.arrow.memory.BufferAllocator; import org.apache.fluss.types.DataType; import org.apache.fluss.types.RowType; @@ -116,6 +119,14 @@ public final class KvTablet { */ private volatile long flushedLogOffset = 0; + /** + * This is specifically used for standby replicas. It refers to the current log apply offset of + * the standby replica. If this offset catches up to the log's highWatermark, it indicates that + * the replica has already been included in the {@link LeaderAndIsr}'s ISSR (In-Sync Standby + * Replica) set. + */ + private volatile long kvAppliedOffset = -1L; + @GuardedBy("kvLock") private volatile boolean isClosed = false; @@ -243,6 +254,14 @@ public long getFlushedLogOffset() { return flushedLogOffset; } + public long getKvAppliedOffset() { + return kvAppliedOffset; + } + + public void setKvAppliedOffset(long kvAppliedOffset) { + this.kvAppliedOffset = kvAppliedOffset; + } + public void registerMetrics(BucketMetricGroup bucketMetricGroup) { MetricGroup metricGroup = bucketMetricGroup.addGroup("kv"); @@ -393,6 +412,48 @@ public LogAppendInfo putAsLeader(KvRecordBatch kvRecords, @Nullable int[] target }); } + /** + * Put the MemoryLogRecords into the kv storage for hotStandbyReplica, and apply the log records + * to log storage. + * + * @param logRecords the log records to put into + * @param kvApplyLogHelper the helper to apply the log records to kv storage + */ + public LogAppendInfo putAsHotStandby( + MemoryLogRecords logRecords, KvApplyLogHelper kvApplyLogHelper) throws Exception { + return inWriteLock( + kvLock, + () -> { + long nextApplyOffset; + LogAppendInfo logAppendInfo; + try { + rocksDBKv.checkIfRocksDBClosed(); + + // 1. apply kv to private buffer. + nextApplyOffset = + kvApplyLogHelper.applyLogRecords(logRecords, kvAppliedOffset); + + // 2. apply log to follower. + logAppendInfo = logTablet.appendAsFollower(logRecords); + // if the batch is duplicated, we should truncate the kvPreWriteBuffer + // already written. + if (logAppendInfo.duplicated()) { + kvPreWriteBuffer.truncateTo(kvAppliedOffset, TruncateReason.DUPLICATED); + throw new OutOfOrderSequenceException( + "The log records are duplicated."); + } + } catch (Throwable t) { + kvPreWriteBuffer.truncateTo(kvAppliedOffset, TruncateReason.ERROR); + throw t; + } + + // increment the kv applied offset and flush the pre-write buffer + kvAppliedOffset = nextApplyOffset; + kvPreWriteBuffer.flush(kvAppliedOffset); + return logAppendInfo; + }); + } + private WalBuilder createWalBuilder(int schemaId, RowType rowType) throws Exception { switch (logFormat) { case INDEXED: diff --git a/fluss-server/src/main/java/org/apache/fluss/server/log/FetchParams.java b/fluss-server/src/main/java/org/apache/fluss/server/log/FetchParams.java index 2af972ae7c..7f9ab64e03 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/log/FetchParams.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/log/FetchParams.java @@ -66,6 +66,10 @@ public final class FetchParams { private final int minFetchBytes; private final long maxWaitMs; + + // This param only used for hotStandbyReplica to fetch log from leader. + private long kvAppliedOffset = -1L; + // TODO: add more params like epoch etc. public FetchParams(int replicaId, int maxFetchBytes) { @@ -99,7 +103,8 @@ public void setCurrentFetch( int maxFetchBytes, RowType schema, ArrowCompressionInfo compressionInfo, - @Nullable int[] projectedFields) { + @Nullable int[] projectedFields, + long kvAppliedOffset) { this.fetchOffset = fetchOffset; this.maxFetchBytes = maxFetchBytes; if (projectedFields != null) { @@ -112,6 +117,7 @@ public void setCurrentFetch( } else { projectionEnabled = false; } + this.kvAppliedOffset = kvAppliedOffset; } /** @@ -175,6 +181,10 @@ public long fetchOffset() { return fetchOffset; } + public long kvAppliedOffset() { + return kvAppliedOffset; + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/log/LogTablet.java b/fluss-server/src/main/java/org/apache/fluss/server/log/LogTablet.java index 689fc122b6..5c90527d4f 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/log/LogTablet.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/log/LogTablet.java @@ -113,6 +113,11 @@ public final class LogTablet { // The minimum offset that should be retained in the local log. This is used to ensure that, // the offset of kv snapshot should be retained, otherwise, kv recovery will fail. private volatile long minRetainOffset; + + // The minimum offset that should be retained in the local log for kv recovery for hotStandby + // replica. + private volatile long minKvAppliedOffset; + // tracking the log start offset in remote storage private volatile long remoteLogStartOffset = Long.MAX_VALUE; // tracking the log end offset in remote storage @@ -163,6 +168,7 @@ private LogTablet( // updating this value in time. Default value to Long.MAX_VALUE for normal log table, // as we don't need to retain logs for kv recovery. this.minRetainOffset = isChangelog ? 0L : Long.MAX_VALUE; + this.minKvAppliedOffset = Long.MAX_VALUE; } public PhysicalTablePath getPhysicalTablePath() { @@ -491,6 +497,15 @@ public void updateMinRetainOffset(long minRetainOffset) { } } + public void updateMinKvAppliedOffset(long minKvAppliedOffset) { + if (minKvAppliedOffset > this.minKvAppliedOffset) { + this.minKvAppliedOffset = minKvAppliedOffset; + } + + // try to delete the old segments that are not needed. + deleteSegmentsAlreadyExistsInRemote(); + } + public void updateLakeTableSnapshotId(long snapshotId) { if (snapshotId > this.lakeTableSnapshotId) { this.lakeTableSnapshotId = snapshotId; @@ -558,7 +573,8 @@ private void deleteSegments(long cleanUpToOffset) { try { // shouldn't clean up segments that will be used by kv recovery. - long cleanupToOffset = Math.min(minRetainOffset, cleanUpToOffset); + long cleanupToOffset = + Math.min(Math.min(minRetainOffset, minKvAppliedOffset), cleanUpToOffset); deleteOldSegments(cleanupToOffset, SegmentDeletionReason.LOG_MOVE_TO_REMOTE); } catch (IOException e) { LOG.error( diff --git a/fluss-server/src/main/java/org/apache/fluss/server/log/remote/RemoteLogManager.java b/fluss-server/src/main/java/org/apache/fluss/server/log/remote/RemoteLogManager.java index cc23be0be5..4fcee1796b 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/log/remote/RemoteLogManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/log/remote/RemoteLogManager.java @@ -133,6 +133,23 @@ public void startLogTiering(Replica replica) throws Exception { if (remoteDisabled()) { return; } + TableBucket tableBucket = replica.getTableBucket(); + // first register the remote log. + RemoteLogTablet remoteLog = addRemoteLog(replica); + + doHandleLeaderReplica(replica, remoteLog, tableBucket); + LOG.debug("Added the remote log tiering task for replica {}", tableBucket); + } + + public boolean hasRemoteLog(TableBucket tableBucket) { + return remoteLogs.containsKey(tableBucket); + } + + public void removeRemoteLog(TableBucket tableBucket) { + remoteLogs.remove(tableBucket); + } + + public RemoteLogTablet addRemoteLog(Replica replica) throws Exception { TableBucket tableBucket = replica.getTableBucket(); PhysicalTablePath physicalTablePath = replica.getPhysicalTablePath(); LogTablet log = replica.getLogTablet(); @@ -153,9 +170,7 @@ public void startLogTiering(Replica replica) throws Exception { // leader needs to register the remote log metrics remoteLog.registerMetrics(replica.bucketMetrics()); remoteLogs.put(tableBucket, remoteLog); - - doHandleLeaderReplica(replica, remoteLog, tableBucket); - LOG.debug("Added the remote log tiering task for replica {}", tableBucket); + return remoteLog; } /** Stop the log tiering task for the given replica. */ diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/FollowerReplica.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/FollowerReplica.java index fbf78266fe..d88a66be5f 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/replica/FollowerReplica.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/FollowerReplica.java @@ -68,6 +68,7 @@ FollowerReplicaState stateSnapshot() { */ void updateFetchState( LogOffsetMetadata followerFetchOffsetMetadata, + long kvAppliedOffset, long followerFetchTimeMs, long leaderEndOffset) { followerReplicaState.updateAndGet( @@ -87,6 +88,7 @@ void updateFetchState( } return new FollowerReplicaState( followerFetchOffsetMetadata, + kvAppliedOffset, Math.max( leaderEndOffset, currentReplicaState.getLastFetchLeaderLogEndOffset()), @@ -115,11 +117,13 @@ void resetFollowerReplicaState( return new FollowerReplicaState( LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, -1L, + -1L, 0L, lastCaughtUpTimeMs); } else { return new FollowerReplicaState( currentReplicaState.getLogEndOffsetMetadata(), + currentReplicaState.getKvAppliedOffset(), leaderEndOffset, isFollowerInSync ? currentTimeMs : 0L, lastCaughtUpTimeMs); @@ -149,6 +153,8 @@ static class FollowerReplicaState { */ private final LogOffsetMetadata logEndOffsetMetadata; + private final long kvAppliedOffset; + /** * The log end offset value at the time the leader received the last FetchRequest from this * follower. This is used to determine the lastCaughtUpTimeMs of the follower. It is reset @@ -171,14 +177,17 @@ static class FollowerReplicaState { private final long lastCaughtUpTimeMs; static final FollowerReplicaState EMPTY = - new FollowerReplicaState(LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, 0L, 0L, 0L); + new FollowerReplicaState( + LogOffsetMetadata.UNKNOWN_OFFSET_METADATA, -1L, 0L, 0L, 0L); FollowerReplicaState( LogOffsetMetadata logEndOffsetMetadata, + long kvAppliedOffset, long lastFetchLeaderLogEndOffset, long lastFetchTimeMs, long lastCaughtUpTimeMs) { this.logEndOffsetMetadata = logEndOffsetMetadata; + this.kvAppliedOffset = kvAppliedOffset; this.lastFetchLeaderLogEndOffset = lastFetchLeaderLogEndOffset; this.lastFetchTimeMs = lastFetchTimeMs; this.lastCaughtUpTimeMs = lastCaughtUpTimeMs; @@ -193,6 +202,10 @@ long getLogEndOffset() { return logEndOffsetMetadata.getMessageOffset(); } + long getKvAppliedOffset() { + return kvAppliedOffset; + } + long getLastCaughtUpTimeMs() { return lastCaughtUpTimeMs; } @@ -221,6 +234,8 @@ public String toString() { return "FollowerReplicaState{" + "logEndOffsetMetadata=" + logEndOffsetMetadata + + ", kvAppliedOffset=" + + kvAppliedOffset + ", lastFetchLeaderLogEndOffset=" + lastFetchLeaderLogEndOffset + ", lastFetchTimeMs=" diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/IsrState.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/IsrState.java index aac5df385d..7c64a89942 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/replica/IsrState.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/IsrState.java @@ -43,12 +43,21 @@ public interface IsrState { /** Indicates if we have an AdjustIsr request inflight. */ boolean isInflight(); + List standbyReplicas(); + + List issr(); + /** Class to represent the committed isr state of a {@link TableBucket}. */ class CommittedIsrState implements IsrState { private final List isr; + private final List standbyReplicas; + private final List issr; - public CommittedIsrState(List isr) { + public CommittedIsrState( + List isr, List standbyReplicas, List issr) { this.isr = isr; + this.standbyReplicas = standbyReplicas; + this.issr = issr; } @Override @@ -66,6 +75,16 @@ public boolean isInflight() { return false; } + @Override + public List standbyReplicas() { + return standbyReplicas; + } + + @Override + public List issr() { + return issr; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -75,12 +94,21 @@ public boolean equals(Object o) { return false; } CommittedIsrState that = (CommittedIsrState) o; - return isr.equals(that.isr); + return isr.equals(that.isr) + && standbyReplicas.equals(that.standbyReplicas) + && issr.equals(that.issr); } @Override public String toString() { - return "CommittedIsrState{" + "isr=" + isr + '}'; + return "CommittedIsrState{" + + "isr=" + + isr + + ", standbyReplicas=" + + standbyReplicas + + ", issr=" + + issr + + '}'; } } @@ -147,6 +175,16 @@ public boolean isInflight() { return true; } + @Override + public List standbyReplicas() { + return lastCommittedState.standbyReplicas; + } + + @Override + public List issr() { + return lastCommittedState.issr; + } + @Override public boolean equals(Object o) { if (this == o) { @@ -215,6 +253,16 @@ public boolean isInflight() { return true; } + @Override + public List standbyReplicas() { + return lastCommittedState.standbyReplicas; + } + + @Override + public List issr() { + return lastCommittedState.issr; + } + @Override public boolean equals(Object o) { if (this == o) { diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java index ddae557c0c..fd757fbb11 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/Replica.java @@ -31,6 +31,7 @@ import org.apache.fluss.exception.NotEnoughReplicasException; import org.apache.fluss.exception.NotLeaderOrFollowerException; import org.apache.fluss.fs.FsPath; +import org.apache.fluss.metadata.KvFormat; import org.apache.fluss.metadata.LogFormat; import org.apache.fluss.metadata.PhysicalTablePath; import org.apache.fluss.metadata.Schema; @@ -49,8 +50,8 @@ import org.apache.fluss.server.SequenceIDCounter; import org.apache.fluss.server.coordinator.CoordinatorContext; import org.apache.fluss.server.entity.NotifyLeaderAndIsrData; +import org.apache.fluss.server.kv.KvApplyLogHelper; import org.apache.fluss.server.kv.KvManager; -import org.apache.fluss.server.kv.KvRecoverHelper; import org.apache.fluss.server.kv.KvTablet; import org.apache.fluss.server.kv.rocksdb.RocksDBKvBuilder; import org.apache.fluss.server.kv.snapshot.CompletedKvSnapshotCommitter; @@ -81,6 +82,7 @@ import org.apache.fluss.server.replica.delay.DelayedOperationManager; import org.apache.fluss.server.replica.delay.DelayedTableBucketKey; import org.apache.fluss.server.replica.delay.DelayedWrite; +import org.apache.fluss.server.replica.standby.KvStandbyManager; import org.apache.fluss.server.utils.FatalErrorHandler; import org.apache.fluss.server.zk.ZkSequenceIDCounter; import org.apache.fluss.server.zk.ZooKeeperClient; @@ -175,6 +177,7 @@ public final class Replica { private final AtomicReference leaderReplicaIdOpt = new AtomicReference<>(); private final ReadWriteLock leaderIsrUpdateLock = new ReentrantReadWriteLock(); private final Clock clock; + private final @Nullable KvStandbyManager kvStandbyManager; /** * storing the remote follower replicas' state, used to update leader's highWatermark and @@ -185,7 +188,9 @@ public final class Replica { private final Map followerReplicasMap = MapUtils.newConcurrentHashMap(); - private volatile IsrState isrState = new IsrState.CommittedIsrState(Collections.emptyList()); + private volatile IsrState isrState = + new IsrState.CommittedIsrState( + Collections.emptyList(), Collections.emptyList(), Collections.emptyList()); private volatile int leaderEpoch = LeaderAndIsr.INITIAL_LEADER_EPOCH - 1; private volatile int bucketEpoch = LeaderAndIsr.INITIAL_BUCKET_EPOCH; private volatile int coordinatorEpoch = CoordinatorContext.INITIAL_COORDINATOR_EPOCH; @@ -193,6 +198,7 @@ public final class Replica { // null if table without pk or haven't become leader private volatile @Nullable KvTablet kvTablet; private volatile @Nullable CloseableRegistry closeableRegistryForKv; + private volatile boolean isStandbyReplica = false; // ------- metrics private Counter isrShrinks; @@ -216,7 +222,8 @@ public Replica( FatalErrorHandler fatalErrorHandler, BucketMetricGroup bucketMetricGroup, TableInfo tableInfo, - Clock clock) + Clock clock, + @Nullable KvStandbyManager kvStandbyManager) throws Exception { this.physicalPath = physicalPath; this.tableBucket = tableBucket; @@ -241,6 +248,7 @@ public Replica( this.logTablet = createLog(lazyHighWatermarkCheckpoint); this.clock = clock; + this.kvStandbyManager = kvStandbyManager; registerMetrics(); } @@ -264,6 +272,19 @@ public boolean isKvTable() { return kvManager != null; } + public boolean isStandbyReplica() { + return isKvTable() && isStandbyReplica; + } + + public boolean isHotStandbyReplica() { + if (!isStandbyReplica) { + return false; + } + + long kvAppliedOffset = kvTablet.getKvAppliedOffset(); + return kvAppliedOffset == getLocalLogEndOffset(); + } + public RowType getRowType() { return schema.getRowType(); } @@ -324,6 +345,10 @@ public int writerIdCount() { return logTablet.getWriterIdCount(); } + public KvFormat getKvFormat() { + return tableConfig.getKvFormat(); + } + public Path getTabletParentDir() { return logManager.getTabletParentDir(physicalPath, tableBucket); } @@ -365,7 +390,12 @@ public void makeLeader(NotifyLeaderAndIsrData data) throws IOException { long currentTimeMs = clock.milliseconds(); // Updating the assignment and ISR state is safe if the bucket epoch is // larger or equal to the current bucket epoch. - updateAssignmentAndIsr(data.getReplicas(), true, data.getIsr()); + updateAssignmentAndIsr( + data.getReplicas(), + true, + data.getIsr(), + data.getHotStandbyReplicas(), + data.getIssr()); int requestLeaderEpoch = data.getLeaderEpoch(); if (requestLeaderEpoch > leaderEpoch) { @@ -413,23 +443,43 @@ public boolean makeFollower(NotifyLeaderAndIsrData data) { coordinatorEpoch = data.getCoordinatorEpoch(); - updateAssignmentAndIsr(Collections.emptyList(), false, Collections.emptyList()); + updateAssignmentAndIsr( + Collections.emptyList(), + false, + Collections.emptyList(), + Collections.emptyList(), + Collections.emptyList()); + boolean toBeHotStandby = toBeNewHotStandby(data.getHotStandbyReplicas()); int requestLeaderEpoch = data.getLeaderEpoch(); boolean isNewLeaderEpoch = requestLeaderEpoch > leaderEpoch; if (isNewLeaderEpoch) { LOG.info( - "Follower {} starts at leader epoch {} from end offset {}", + "Follower{} {} starts at leader epoch {} from end offset {}", + toBeHotStandby ? " (hotStandby)" : "", tableBucket, requestLeaderEpoch, logTablet.localLogEndOffset()); - onBecomeNewFollower(); + if (toBeHotStandby) { + onBecomeNewHotStandby(); + } else { + onBecomeNewFollower(); + } } else if (requestLeaderEpoch == leaderEpoch) { - LOG.info( - "Skipped the become-follower state change for bucket {} since " - + "it's already the follower with leader epoch {}", - tableBucket, - leaderEpoch); + if (toBeHotStandby) { + LOG.info( + "Follower (hotStandby) {} starts at leader epoch {} from end offset {}", + tableBucket, + requestLeaderEpoch, + logTablet.localLogEndOffset()); + onBecomeNewHotStandby(); + } else { + LOG.info( + "Skipped the become-follower state change for bucket {} since " + + "it's already the follower with leader epoch {}", + tableBucket, + leaderEpoch); + } } else { String errorMessage = String.format( @@ -511,12 +561,19 @@ private void onBecomeNewLeader() { updateLeaderEndOffsetSnapshot(); if (isKvTable()) { - // if it's become new leader, we must - // first destroy the old kv tablet - // if exist. Otherwise, it'll use still the old kv tablet which will cause data loss - dropKv(); - // now, we can create a new kv tablet - createKv(); + if (isStandbyReplica()) { + checkNotNull(kvStandbyManager, "kv standby manager should not be null."); + kvStandbyManager.stopStandby(tableBucket); + isStandbyReplica = false; + checkNotNull(kvTablet, "kv tablet should not be null."); + kvTablet.setKvAppliedOffset(-1L); + } else { + // if it's become new leader, we must first destroy the old kv tablet if exists. + // Otherwise, it'll use still the old kv tablet which will cause data loss + dropKv(); + // now, we can create a new kv tablet + createKv(); + } } } @@ -527,6 +584,13 @@ private void onBecomeNewFollower() { } } + private void onBecomeNewHotStandby() { + this.isStandbyReplica = true; + + checkNotNull(kvStandbyManager, "kv standby manager should not be null."); + kvStandbyManager.startStandby(this); + } + @VisibleForTesting public void updateLeaderEndOffsetSnapshot() { logTablet.updateLeaderEndOffsetSnapshot(); @@ -543,12 +607,12 @@ private void createKv() { } // init kv tablet and get the snapshot it uses to init if have any - Optional snapshotUsed = initKvTablet(); + Optional snapshotUsed = initKvTablet(true); // start periodic kv snapshot startPeriodicKvSnapshot(snapshotUsed.orElse(null)); } - private void dropKv() { + public void dropKv() { // close any closeable registry for kv if (closeableRegistry.unregisterCloseable(closeableRegistryForKv)) { IOUtils.closeQuietly(closeableRegistryForKv); @@ -561,7 +625,7 @@ private void dropKv() { } } - private void mayFlushKv(long newHighWatermark) { + public void mayFlushKv(long newHighWatermark) { KvTablet kvTablet = this.kvTablet; if (kvTablet != null) { kvTablet.flush(newHighWatermark, fatalErrorHandler); @@ -573,7 +637,7 @@ private void mayFlushKv(long newHighWatermark) { * * @return the snapshot used to init kv tablet, empty if no any snapshot. */ - private Optional initKvTablet() { + public Optional initKvTablet(boolean needRecover) { checkNotNull(kvManager); long startTime = clock.milliseconds(); LOG.info("Start to init kv tablet for {} of table {}.", tableBucket, physicalPath); @@ -631,7 +695,10 @@ private Optional initKvTablet() { kvTablet.registerMetrics(bucketMetricGroup); logTablet.updateMinRetainOffset(restoreStartOffset); - recoverKvTablet(restoreStartOffset); + + if (needRecover) { + recoverKvTablet(restoreStartOffset); + } } catch (Exception e) { throw new KvStorageException( String.format( @@ -688,20 +755,10 @@ private void recoverKvTablet(long startRecoverLogOffset) { long start = clock.milliseconds(); checkNotNull(kvTablet, "kv tablet should not be null."); try { - KvRecoverHelper.KvRecoverContext recoverContext = - new KvRecoverHelper.KvRecoverContext( - getTablePath(), - tableBucket, - snapshotContext.getZooKeeperClient(), - snapshotContext.maxFetchLogSizeInRecoverKv()); - KvRecoverHelper kvRecoverHelper = - new KvRecoverHelper( - kvTablet, - logTablet, - startRecoverLogOffset, - recoverContext, - tableConfig.getKvFormat()); - kvRecoverHelper.recover(); + KvApplyLogHelper kvApplyLogHelper = + new KvApplyLogHelper(this, snapshotContext.getZooKeeperClient()); + kvApplyLogHelper.recover( + startRecoverLogOffset, snapshotContext.maxFetchLogSizeInRecoverKv()); } catch (Exception e) { throw new KvStorageException( String.format( @@ -872,6 +929,24 @@ public LogAppendInfo putRecordsToLeader( }); } + public LogAppendInfo putRecordsToHotStandby( + MemoryLogRecords logRecords, KvApplyLogHelper kvApplyLogHelper) { + KvTablet kv = this.kvTablet; + checkNotNull( + kv, + "KvTablet for the replica to put kv records to hotStandbyReplica shouldn't be null."); + LogAppendInfo logAppendInfo; + try { + logAppendInfo = kv.putAsHotStandby(logRecords, kvApplyLogHelper); + } catch (Exception e) { + LOG.error("Error while putting records to hotStandbyReplica for {}", tableBucket, e); + fatalErrorHandler.onFatalError(e); + throw new KvStorageException( + "Error while putting records to hotStandbyReplica for " + tableBucket, e); + } + return logAppendInfo; + } + public LogReadInfo fetchRecords(FetchParams fetchParams) throws IOException { if (fetchParams.projection() != null && logFormat != LogFormat.ARROW) { throw new InvalidColumnProjectionException( @@ -894,7 +969,8 @@ public LogReadInfo fetchRecords(FetchParams fetchParams) throws IOException { followerReplica, logReadInfo.getFetchedData().getFetchOffsetMetadata(), followerFetchTimeMs, - logReadInfo.getLogEndOffset()); + logReadInfo.getLogEndOffset(), + fetchParams.kvAppliedOffset()); return logReadInfo; } else { return inReadLock( @@ -980,7 +1056,11 @@ private boolean shouldWaitForReplicaToJoinIsr( } private void updateAssignmentAndIsr( - List replicas, boolean isLeader, List isr) { + List replicas, + boolean isLeader, + List isr, + List standbyReplicas, + List issr) { if (isLeader) { List followers = replicas.stream() @@ -1003,14 +1083,15 @@ private void updateAssignmentAndIsr( } // update isr info. - isrState = new IsrState.CommittedIsrState(isr); + isrState = new IsrState.CommittedIsrState(isr, standbyReplicas, issr); } private void updateFollowerFetchState( FollowerReplica followerReplica, LogOffsetMetadata followerFetchOffsetMetadata, long followerFetchTimeMs, - long leaderLogEndOffset) + long leaderLogEndOffset, + long kvAppliedOffset) throws IOException { long prevFollowerEndOffset = followerReplica.stateSnapshot().getLogEndOffset(); @@ -1021,11 +1102,13 @@ private void updateFollowerFetchState( () -> followerReplica.updateFetchState( followerFetchOffsetMetadata, + kvAppliedOffset, followerFetchTimeMs, leaderLogEndOffset)); // Check if this in-sync replica needs to be added to the ISR. - maybeExpandISr(followerReplica); + maybeExpandIsr(followerReplica); + maybeExpandIssr(followerReplica); // check if the HW of the replica can now be incremented since the replica may already be in // the ISR and its LEO has just incremented @@ -1363,7 +1446,7 @@ private void validateBucketEpoch(int requestBucketEpoch) { * *

This function can be triggered when a replica's LEO has incremented. */ - private void maybeExpandISr(FollowerReplica followerReplica) { + private void maybeExpandIsr(FollowerReplica followerReplica) { IsrState currentIsrState = isrState; boolean needsIsrUpdate = !currentIsrState.isInflight() @@ -1395,6 +1478,37 @@ private void maybeExpandISr(FollowerReplica followerReplica) { } } + private void maybeExpandIssr(FollowerReplica followerReplica) { + IsrState currentIsrState = isrState; + boolean needsIssrUpdate = + !currentIsrState.isInflight() + && inReadLock(leaderIsrUpdateLock, () -> needsExpandIssr(followerReplica)); + if (needsIssrUpdate) { + Optional adjustIsrUpdateOpt = + inWriteLock( + leaderIsrUpdateLock, + () -> { + // check if this replica needs to be added to the ISR. + if (currentIsrState instanceof IsrState.CommittedIsrState) { + if (needsExpandIssr(followerReplica)) { + return Optional.of( + prepareIssrExpand( + (IsrState.CommittedIsrState) + currentIsrState, + followerReplica.getFollowerId())); + } + } + + return Optional.empty(); + }); + + // Send adjust isr request outside the leaderIsrUpdateLock since the completion + // logic may increment the high watermark (and consequently complete delayed + // operations). + adjustIsrUpdateOpt.map(this::submitAdjustIsr); + } + } + void maybeShrinkIsr() { IsrState currentIstState = isrState; boolean needsIsrUpdate = @@ -1454,8 +1568,40 @@ private IsrState.PendingExpandIsrState prepareIsrExpand( // TODO add server epoch to isr. LeaderAndIsr newLeaderAndIsr = - new LeaderAndIsr( - localTabletServerId, leaderEpoch, isrToSend, coordinatorEpoch, bucketEpoch); + new LeaderAndIsr.Builder() + .leader(localTabletServerId) + .leaderEpoch(leaderEpoch) + .isr(isrToSend) + .coordinatorEpoch(coordinatorEpoch) + .bucketEpoch(bucketEpoch) + .standbyReplicas(isrState.standbyReplicas()) + .issr(isrState.issr()) + .build(); + + IsrState.PendingExpandIsrState updatedState = + new IsrState.PendingExpandIsrState( + newInSyncReplicaId, newLeaderAndIsr, currentState); + isrState = updatedState; + return updatedState; + } + + private IsrState.PendingExpandIsrState prepareIssrExpand( + IsrState.CommittedIsrState currentState, int newInSyncReplicaId) { + List issrToSend = new ArrayList<>(isrState.issr()); + issrToSend.add(newInSyncReplicaId); + + // TODO add server epoch to isr. + + LeaderAndIsr newLeaderAndIsr = + new LeaderAndIsr.Builder() + .leader(localTabletServerId) + .leaderEpoch(leaderEpoch) + .isr(isrState.isr()) + .coordinatorEpoch(coordinatorEpoch) + .bucketEpoch(bucketEpoch) + .standbyReplicas(isrState.standbyReplicas()) + .issr(issrToSend) + .build(); IsrState.PendingExpandIsrState updatedState = new IsrState.PendingExpandIsrState( @@ -1476,8 +1622,13 @@ IsrState.PendingShrinkIsrState prepareIsrShrink( // TODO add server epoch to isr. LeaderAndIsr newLeaderAndIsr = - new LeaderAndIsr( - localTabletServerId, leaderEpoch, isrToSend, coordinatorEpoch, bucketEpoch); + new LeaderAndIsr.Builder() + .leader(localTabletServerId) + .leaderEpoch(leaderEpoch) + .isr(isrToSend) + .coordinatorEpoch(coordinatorEpoch) + .bucketEpoch(bucketEpoch) + .build(); IsrState.PendingShrinkIsrState updatedState = new IsrState.PendingShrinkIsrState( outOfSyncFollowerReplicas, newLeaderAndIsr, currentState); @@ -1570,7 +1721,9 @@ private boolean handleAdjustIsrUpdate( // proposed and actual state are the same. // In both cases, we want to move from Pending to Committed state to ensure new updates // are processed. - isrState = new IsrState.CommittedIsrState(leaderAndIsr.isr()); + isrState = + new IsrState.CommittedIsrState( + leaderAndIsr.isr(), leaderAndIsr.standbyList(), leaderAndIsr.issr()); bucketEpoch = leaderAndIsr.bucketEpoch(); LOG.info( "ISR updated to {} and bucket epoch updated to {} for bucket {}", @@ -1662,6 +1815,14 @@ private boolean needsExpandIsr(FollowerReplica followerReplica) { && isFollowerInSync(followerReplica); } + private boolean needsExpandIssr(FollowerReplica followerReplica) { + if (!isrState.standbyReplicas().contains(followerReplica.getFollowerId())) { + return false; + } + long kvAppliedOffset = followerReplica.stateSnapshot().getKvAppliedOffset(); + return kvAppliedOffset >= logTablet.getHighWatermark(); + } + private boolean needsShrinkIsr() { return isLeader() && !getOutOfSyncFollowerReplicas(replicaMaxLagTime).isEmpty(); } @@ -1830,6 +1991,10 @@ private void traceAckInfo(List curMaximalIsr, long requiredOffset) { .collect(Collectors.toList())); } + private boolean toBeNewHotStandby(List hotStandbyList) { + return isKvTable() && !isStandbyReplica() && hotStandbyList.contains(localTabletServerId); + } + @VisibleForTesting public int getBucketEpoch() { return bucketEpoch; @@ -1839,4 +2004,14 @@ public int getBucketEpoch() { public List getIsr() { return isrState.isr(); } + + @VisibleForTesting + public List getStandbyReplicas() { + return isrState.standbyReplicas(); + } + + @VisibleForTesting + public List getIssr() { + return isrState.issr(); + } } 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 7e9e4f3ae6..95ca9ccacf 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 @@ -90,6 +90,7 @@ import org.apache.fluss.server.replica.delay.DelayedWrite; import org.apache.fluss.server.replica.fetcher.InitialFetchStatus; import org.apache.fluss.server.replica.fetcher.ReplicaFetcherManager; +import org.apache.fluss.server.replica.standby.KvStandbyManager; import org.apache.fluss.server.utils.FatalErrorHandler; import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.server.zk.data.LakeTableSnapshot; @@ -137,6 +138,7 @@ public class ReplicaManager { private final Scheduler scheduler; private final LogManager logManager; private final KvManager kvManager; + private final KvStandbyManager kvStandbyManager; private final ZooKeeperClient zkClient; protected final int serverId; private final AtomicBoolean highWatermarkCheckPointThreadStarted = new AtomicBoolean(false); @@ -237,6 +239,13 @@ public ReplicaManager( this.scheduler = scheduler; this.logManager = logManager; this.kvManager = kvManager; + this.kvStandbyManager = + new KvStandbyManager( + conf, + scheduler, + zkClient, + logManager.getDataDir().getAbsolutePath(), + remoteLogManager); this.serverId = serverId; this.metadataCache = metadataCache; @@ -293,6 +302,10 @@ public RemoteLogManager getRemoteLogManager() { return remoteLogManager; } + public KvStandbyManager getKvStandbyManager() { + return kvStandbyManager; + } + private void registerMetrics() { serverMetricGroup.gauge( MetricNames.REPLICA_LEADER_COUNT, @@ -1013,7 +1026,10 @@ public Map readFromLog( adjustedMaxBytes, replica.getRowType(), replica.getArrowCompressionInfo(), - fetchReqInfo.getProjectFields()); + fetchReqInfo.getProjectFields(), + fetchReqInfo.getKvAppliedOffset() == null + ? -1L + : fetchReqInfo.getKvAppliedOffset()); LogReadInfo readInfo = replica.fetchRecords(fetchParams); // Once we read from a non-empty bucket, we stop ignoring request and bucket @@ -1485,7 +1501,8 @@ protected Optional maybeCreateReplica(NotifyLeaderAndIsrData data) { fatalErrorHandler, bucketMetricGroup, tableInfo, - clock); + clock, + isKvTable ? kvStandbyManager : null); allReplicas.put(tb, new OnlineReplica(replica)); replicaOpt = Optional.of(replica); } else if (hostedReplica instanceof OnlineReplica) { @@ -1578,6 +1595,7 @@ public void shutdown() throws InterruptedException { replicaFetcherManager.shutdown(); delayedWriteManager.shutdown(); delayedFetchLogManager.shutdown(); + kvStandbyManager.shutdown(); // Checkpoint highWatermark. checkpointHighWatermarks(); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/fetcher/RemoteLeaderEndpoint.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/fetcher/RemoteLeaderEndpoint.java index 2cd62ec5e1..4254e8b19c 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/replica/fetcher/RemoteLeaderEndpoint.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/fetcher/RemoteLeaderEndpoint.java @@ -29,7 +29,10 @@ import org.apache.fluss.rpc.messages.PbFetchLogRespForTable; import org.apache.fluss.rpc.messages.PbListOffsetsRespForBucket; import org.apache.fluss.rpc.protocol.Errors; +import org.apache.fluss.server.kv.KvTablet; import org.apache.fluss.server.log.ListOffsetsParam; +import org.apache.fluss.server.replica.Replica; +import org.apache.fluss.server.replica.ReplicaManager; import java.util.ArrayList; import java.util.HashMap; @@ -40,9 +43,11 @@ import static org.apache.fluss.rpc.util.CommonRpcMessageUtils.getFetchLogResultForBucket; import static org.apache.fluss.server.utils.ServerRpcMessageUtils.makeListOffsetsRequest; +import static org.apache.fluss.utils.Preconditions.checkNotNull; /** Facilitates fetches from a remote replica leader in one tablet server. */ final class RemoteLeaderEndpoint implements LeaderEndpoint { + private final ReplicaManager replicaManager; private final int followerServerId; private final int remoteServerId; private final TabletServerGateway tabletServerGateway; @@ -55,10 +60,12 @@ final class RemoteLeaderEndpoint implements LeaderEndpoint { private final int maxFetchWaitMs; RemoteLeaderEndpoint( + ReplicaManager replicaManager, Configuration conf, int followerServerId, int remoteServerId, TabletServerGateway tabletServerGateway) { + this.replicaManager = replicaManager; this.followerServerId = followerServerId; this.remoteServerId = remoteServerId; this.maxFetchSize = (int) conf.get(ConfigOptions.LOG_REPLICA_FETCH_MAX_BYTES).getBytes(); @@ -128,8 +135,20 @@ public CompletableFuture fetchLog(FetchLogContext fetchLogContext) { @Override public Optional buildFetchLogContext( Map replicas) { + Map kvAppliedOffsets = new HashMap<>(); + for (Map.Entry entry : replicas.entrySet()) { + TableBucket tb = entry.getKey(); + Replica replica = replicaManager.getReplicaOrException(tb); + if (replica.isStandbyReplica()) { + KvTablet kvTablet = replica.getKvTablet(); + checkNotNull(kvTablet); + kvAppliedOffsets.put(tb, kvTablet.getKvAppliedOffset()); + } + } + return buildFetchLogContext( replicas, + kvAppliedOffsets, followerServerId, maxFetchSize, maxFetchSizeForBucket, @@ -144,6 +163,7 @@ public void close() { static Optional buildFetchLogContext( Map replicas, + Map kvAppliedOffsets, int followerServerId, int maxFetchSize, int maxFetchSizeForBucket, @@ -170,6 +190,11 @@ static Optional buildFetchLogContext( if (tb.getPartitionId() != null) { fetchLogReqForBucket.setPartitionId(tb.getPartitionId()); } + + if (kvAppliedOffsets.containsKey(tb)) { + fetchLogReqForBucket.setKvAppliedOffset(kvAppliedOffsets.get(tb)); + } + fetchLogReqForBuckets .computeIfAbsent(tb.getTableId(), key -> new ArrayList<>()) .add(fetchLogReqForBucket); diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherManager.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherManager.java index 9e4479b3eb..2b1a31fd0c 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherManager.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherManager.java @@ -193,6 +193,7 @@ ReplicaFetcherThread createFetcherThread(int fetcherId, int leaderId) { @VisibleForTesting public RemoteLeaderEndpoint buildRemoteLogEndpoint(int leaderId) { return new RemoteLeaderEndpoint( + replicaManager, conf, serverId, leaderId, diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherThread.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherThread.java index 398bfb6ffd..cdd2afb8c6 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherThread.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherThread.java @@ -39,6 +39,7 @@ import org.apache.fluss.server.replica.Replica; import org.apache.fluss.server.replica.ReplicaManager; import org.apache.fluss.server.replica.fetcher.LeaderEndpoint.FetchData; +import org.apache.fluss.server.replica.standby.KvStandbyManager; import org.apache.fluss.shaded.netty4.io.netty.buffer.ByteBuf; import org.apache.fluss.utils.FileUtils; import org.apache.fluss.utils.FlussPaths; @@ -537,12 +538,32 @@ private LogAppendInfo processFetchResultFromLocalStorage( replicaData.getHighWatermark()); // Append the messages to the follower log tablet. - LogAppendInfo logAppendInfo = replica.appendRecordsToFollower(records); - LOG.trace( - "Follower has replica log end offset {} after appending {} bytes of messages for replica {}", - logTablet.localLogEndOffset(), - records.sizeInBytes(), - tableBucket); + LogAppendInfo logAppendInfo; + if (replica.isHotStandbyReplica()) { + KvStandbyManager kvStandbyManager = replicaManager.getKvStandbyManager(); + // first, cancel standby task. + if (kvStandbyManager.hasBecomeHotStandbyTask(tableBucket)) { + kvStandbyManager.cancelBecomeHotStandbyTask(tableBucket); + } + + // apply log to kv. + logAppendInfo = + replica.putRecordsToHotStandby( + records, kvStandbyManager.getApplyLogToKvHelper(tableBucket)); + LOG.trace( + "Follower(HotStandbyReplica) has replica log end offset {} after appending {} bytes of messages for replica {}", + logTablet.localLogEndOffset(), + records.sizeInBytes(), + tableBucket); + + } else { + logAppendInfo = replica.appendRecordsToFollower(records); + LOG.trace( + "Follower has replica log end offset {} after appending {} bytes of messages for replica {}", + logTablet.localLogEndOffset(), + records.sizeInBytes(), + tableBucket); + } // For the follower replica, we do not need to keep its segment base offset and physical // position. These values will be computed upon becoming leader or handling a preferred read diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/standby/BecomeHotStandbyTask.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/standby/BecomeHotStandbyTask.java new file mode 100644 index 0000000000..35cdea3be5 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/standby/BecomeHotStandbyTask.java @@ -0,0 +1,285 @@ +/* + * 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.replica.standby; + +import org.apache.fluss.exception.FlussRuntimeException; +import org.apache.fluss.fs.FsPath; +import org.apache.fluss.fs.FsPathAndFileName; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.record.FileLogRecords; +import org.apache.fluss.record.LogRecords; +import org.apache.fluss.record.MemoryLogRecords; +import org.apache.fluss.remote.RemoteFileDownloader; +import org.apache.fluss.remote.RemoteLogSegment; +import org.apache.fluss.server.kv.KvApplyLogHelper; +import org.apache.fluss.server.kv.KvTablet; +import org.apache.fluss.server.kv.snapshot.CompletedSnapshot; +import org.apache.fluss.server.log.FetchIsolation; +import org.apache.fluss.server.log.LogTablet; +import org.apache.fluss.server.log.remote.RemoteLogManager; +import org.apache.fluss.server.replica.Replica; +import org.apache.fluss.utils.FlussPaths; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.ExecutionException; + +import static org.apache.fluss.utils.FlussPaths.LOG_FILE_SUFFIX; +import static org.apache.fluss.utils.FlussPaths.remoteLogSegmentDir; +import static org.apache.fluss.utils.FlussPaths.remoteLogSegmentFile; +import static org.apache.fluss.utils.Preconditions.checkNotNull; + +/** A task interface to become hotStandbyReplica. */ +public class BecomeHotStandbyTask implements Runnable { + + private static final Logger LOG = LoggerFactory.getLogger(BecomeHotStandbyTask.class); + + private final Replica replica; + private final RemoteFileDownloader remoteFileDownloader; + private final Path remoteLogTempDir; + private final RemoteLogManager remoteLogManager; + private final FsPath remoteLogTabletDir; + private final KvApplyLogHelper kvApplyLogHelper; + private final int maxFetchLogSizeWhenApplying; + + private volatile boolean cancelled = false; + + public BecomeHotStandbyTask( + Replica replica, + RemoteFileDownloader remoteFileDownloader, + RemoteLogManager remoteLogManager, + Path remoteLogTempDir, + KvApplyLogHelper kvApplyLogHelper, + int maxFetchLogSizeWhenApplying) { + this.kvApplyLogHelper = kvApplyLogHelper; + this.replica = replica; + this.remoteFileDownloader = remoteFileDownloader; + this.remoteLogManager = remoteLogManager; + this.remoteLogTempDir = remoteLogTempDir; + this.remoteLogTabletDir = + FlussPaths.remoteLogTabletDir( + remoteLogManager.remoteLogDir(), + replica.getPhysicalTablePath(), + replica.getTableBucket()); + this.maxFetchLogSizeWhenApplying = maxFetchLogSizeWhenApplying; + } + + @Override + public void run() { + if (isCancelled()) { + return; + } + + // 1. try to drop kv if it exists. + replica.dropKv(); + + // update minRetainOffset to 0 means all logs are retained from now on. + replica.getLogTablet().updateMinKvAppliedOffset(0L); + + // 2. apply snapshot to kvTablet. + Optional completedSnapshotOpt = replica.initKvTablet(false); + TableBucket tableBucket = replica.getTableBucket(); + if (completedSnapshotOpt.isPresent()) { + CompletedSnapshot snapshot = completedSnapshotOpt.get(); + long logOffset = snapshot.getLogOffset(); + LOG.info( + "Init kv tablet for {} of {} finish, logOffset: {}", + replica.getPhysicalTablePath(), + tableBucket, + logOffset); + KvTablet kvTablet = replica.getKvTablet(); + checkNotNull(kvTablet, "kvTablet cannot be null"); + // update kvAppliedOffset. + kvTablet.setKvAppliedOffset(logOffset); + replica.getLogTablet().updateMinKvAppliedOffset(logOffset); + } else { + LOG.warn("snapshot is not present"); + } + + // 3. apply remote log to kvTablet. + if (isCancelled()) { + return; + } + + KvTablet kvTablet = replica.getKvTablet(); + checkNotNull(kvTablet, "kvTablet cannot be null"); + long currentAppliedOffset = kvTablet.getKvAppliedOffset(); + + if (!remoteLogManager.hasRemoteLog(tableBucket)) { + try { + remoteLogManager.addRemoteLog(replica); + } catch (Exception e) { + LOG.warn("No remote log segment to apply"); + // TODO handle the exception + } + } + List remoteLogSegments = + remoteLogManager.relevantRemoteLogSegments(tableBucket, currentAppliedOffset); + LOG.info( + "Try to apply remote log for {}, currentAppliedOffset: {}.", + tableBucket, + currentAppliedOffset); + if (remoteLogSegments.isEmpty()) { + LOG.warn("No remote log segment to apply"); + } else { + int startPos; + for (int i = 0; i < remoteLogSegments.size(); i++) { + RemoteLogSegment remoteLogSegment = remoteLogSegments.get(i); + if (i == 0) { + startPos = + remoteLogManager.lookupPositionForOffset( + remoteLogSegment, currentAppliedOffset); + } else { + startPos = 0; + } + + long remoteLogEndOffset = remoteLogSegment.remoteLogEndOffset(); + if (remoteLogEndOffset <= replica.getLocalLogStartOffset()) { + long nextFetchOffset = 0; + try { + nextFetchOffset = applyRemoteLogSegment(remoteLogSegment, startPos); + } catch (Exception e) { + LOG.warn("Failed to apply remote log segment: {}", remoteLogSegment, e); + // TODO handle the exception + } + kvTablet.setKvAppliedOffset(nextFetchOffset); + // flush kv data from kvPreWriteBuffer to rocksdb. + replica.mayFlushKv(nextFetchOffset); + replica.getLogTablet().updateMinKvAppliedOffset(nextFetchOffset); + } else { + // When remoteLogEndOffset > localLogStartOffset, it indicates that log segments + // already exist locally. In this case, we prioritize consuming the local logs + // to accelerate the recovery process. + break; + } + } + } + remoteLogManager.removeRemoteLog(tableBucket); + + // 4. apply local log to kvTablet. + LogTablet logTablet = replica.getLogTablet(); + while (true) { + if (isCancelled()) { + return; + } + + long nextFetchOffset = kvTablet.getKvAppliedOffset(); + try { + LogRecords logRecords = + logTablet + .read( + nextFetchOffset, + maxFetchLogSizeWhenApplying, + FetchIsolation.LOG_END, + true, + null) + .getRecords(); + if (logRecords == MemoryLogRecords.EMPTY) { + Thread.sleep(100); + continue; + } + + nextFetchOffset = kvApplyLogHelper.applyLogRecords(logRecords, nextFetchOffset); + kvTablet.setKvAppliedOffset(nextFetchOffset); + // flush kv data from kvPreWriteBuffer to rocksdb. + replica.mayFlushKv(nextFetchOffset); + replica.getLogTablet().updateMinKvAppliedOffset(nextFetchOffset); + } catch (Exception e) { + LOG.warn("read log failed", e); + // TODO handle the error. + replica.truncateTo(nextFetchOffset); + } + } + } + + public void cancel() { + cancelled = true; + } + + public boolean isCancelled() { + return cancelled; + } + + private long applyRemoteLogSegment(RemoteLogSegment remoteLogSegment, int startPos) + throws Exception { + LOG.info("Apply remote log segment: {}", remoteLogSegment); + FsPathAndFileName fsPathAndFileName = + getFsPathAndFileName(remoteLogTabletDir, remoteLogSegment); + try { + remoteFileDownloader.downloadFileAsync(fsPathAndFileName, remoteLogTempDir).get(); + } catch (InterruptedException | ExecutionException e) { + // TODO handle the exception + } + + // local log file to consume. + File localFile = new File(remoteLogTempDir.toFile(), fsPathAndFileName.getFileName()); + FileLogRecords fileLogRecords = getFileLogRecords(localFile, startPos); + + // consume and write to kv. + long nextFetchOffset = kvApplyLogHelper.applyLogRecords(fileLogRecords, startPos); + + // delete local file after consuming. + localFile.delete(); + + return nextFetchOffset; + } + + private static FsPathAndFileName getFsPathAndFileName( + FsPath remoteLogTabletDir, RemoteLogSegment segment) { + FsPath remotePath = + remoteLogSegmentFile( + remoteLogSegmentDir(remoteLogTabletDir, segment.remoteLogSegmentId()), + segment.remoteLogStartOffset()); + return new FsPathAndFileName(remotePath, getLocalFileNameOfRemoteSegment(segment)); + } + + /** + * Get the local file name of the remote log segment. + * + *

The file name is in pattern: + * + *

+     *     ${remote_segment_id}_${offset_prefix}.log
+     * 
+ */ + private static String getLocalFileNameOfRemoteSegment(RemoteLogSegment segment) { + return segment.remoteLogSegmentId() + + "_" + + FlussPaths.filenamePrefixFromOffset(segment.remoteLogStartOffset()) + + LOG_FILE_SUFFIX; + } + + private FileLogRecords getFileLogRecords(File localFile, int startPosition) { + try { + FileLogRecords fileLogRecords = FileLogRecords.open(localFile, false); + if (startPosition > 0) { + return fileLogRecords.slice(startPosition, Integer.MAX_VALUE); + } else { + return fileLogRecords; + } + } catch (IOException e) { + throw new FlussRuntimeException(e); + } + } +} diff --git a/fluss-server/src/main/java/org/apache/fluss/server/replica/standby/KvStandbyManager.java b/fluss-server/src/main/java/org/apache/fluss/server/replica/standby/KvStandbyManager.java new file mode 100644 index 0000000000..1db8d01e80 --- /dev/null +++ b/fluss-server/src/main/java/org/apache/fluss/server/replica/standby/KvStandbyManager.java @@ -0,0 +1,163 @@ +/* + * 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.replica.standby; + +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.remote.RemoteFileDownloader; +import org.apache.fluss.server.kv.KvApplyLogHelper; +import org.apache.fluss.server.log.remote.RemoteLogManager; +import org.apache.fluss.server.replica.Replica; +import org.apache.fluss.server.zk.ZooKeeperClient; +import org.apache.fluss.utils.MapUtils; +import org.apache.fluss.utils.concurrent.Scheduler; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Comparator; +import java.util.Map; +import java.util.concurrent.Future; +import java.util.concurrent.ScheduledFuture; + +/** The manager to manage the kv standby replicas. */ +public class KvStandbyManager { + + private static final Logger LOG = LoggerFactory.getLogger(KvStandbyManager.class); + + private final Scheduler scheduler; + private final ZooKeeperClient zkClient; + private final RemoteFileDownloader remoteFileDownloader; + private final RemoteLogManager remoteLogManager; + private final Path remoteLogTempDir; + private final int maxFetchLogSizeWhenApplying; + + private final Map standbyTasks = MapUtils.newConcurrentHashMap(); + private final Map applyLogToKvHelpers = + MapUtils.newConcurrentHashMap(); + + public KvStandbyManager( + Configuration conf, + Scheduler scheduler, + ZooKeeperClient zkClient, + String dataDir, + RemoteLogManager remoteLogManager) + throws IOException { + this.scheduler = scheduler; + this.zkClient = zkClient; + this.maxFetchLogSizeWhenApplying = + (int) conf.get(ConfigOptions.KV_RECOVER_LOG_RECORD_BATCH_MAX_SIZE).getBytes(); + this.remoteLogManager = remoteLogManager; + this.remoteFileDownloader = new RemoteFileDownloader(1); + this.remoteLogTempDir = Paths.get(dataDir, "temp-remote-logs"); + prepareRemoteLogTempDir(); + } + + private void prepareRemoteLogTempDir() throws IOException { + if (!Files.exists(remoteLogTempDir)) { + Files.createDirectory(remoteLogTempDir); + } else { + // remove all temp file. + Files.walk(remoteLogTempDir) + .sorted(Comparator.reverseOrder()) + .map(Path::toFile) + .forEach(File::delete); + } + } + + public void shutdown() { + // do nothing now. + } + + public void startStandby(Replica replica) { + TableBucket tableBucket = replica.getTableBucket(); + + KvApplyLogHelper kvApplyLogHelper = + applyLogToKvHelpers.computeIfAbsent( + tableBucket, tb -> new KvApplyLogHelper(replica, zkClient)); + + standbyTasks.compute( + tableBucket, + (tb, prevTask) -> { + if (prevTask != null) { + LOG.info("Cancelling the standby task for table-bucket: {}", tableBucket); + prevTask.cancel(); + } + BecomeHotStandbyTask task = + new BecomeHotStandbyTask( + replica, + remoteFileDownloader, + remoteLogManager, + remoteLogTempDir, + kvApplyLogHelper, + maxFetchLogSizeWhenApplying); + LOG.info("Created a new standby task: {} and getting scheduled one time", task); + ScheduledFuture future = + scheduler.scheduleOnce("become-hot-standby-task", task); + return new TaskWithFuture(task, future); + }); + } + + public KvApplyLogHelper getApplyLogToKvHelper(TableBucket tableBucket) { + return applyLogToKvHelpers.get(tableBucket); + } + + public boolean hasBecomeHotStandbyTask(TableBucket tableBucket) { + return standbyTasks.containsKey(tableBucket); + } + + public void cancelBecomeHotStandbyTask(TableBucket tableBucket) { + standbyTasks.computeIfPresent( + tableBucket, + (tb, prevTask) -> { + prevTask.cancel(); + return prevTask; + }); + } + + public void stopStandby(TableBucket tableBucket) { + standbyTasks.remove(tableBucket); + applyLogToKvHelpers.remove(tableBucket); + } + + static class TaskWithFuture { + + private final BecomeHotStandbyTask task; + private final Future future; + + TaskWithFuture(BecomeHotStandbyTask task, Future future) { + this.task = task; + this.future = future; + } + + public void cancel() { + task.cancel(); + try { + future.cancel(true); + } catch (Exception ex) { + LOG.error("Error occurred while canceling the task: {}", task, ex); + } + } + } +} 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 2b8e845d66..0576fa995b 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 @@ -504,7 +504,9 @@ public static PbNotifyLeaderAndIsrReqForBucket makeNotifyBucketLeaderAndIsr( reqForBucket .setPhysicalTablePath(fromPhysicalTablePath(physicalTablePath)) .setReplicas(notifyLeaderAndIsrData.getReplicasArray()) - .setIsrs(notifyLeaderAndIsrData.getIsrArray()); + .setIsrs(notifyLeaderAndIsrData.getIsrArray()) + .setHotStandbyReplicas(notifyLeaderAndIsrData.getStandbyArray()) + .setIssrs(notifyLeaderAndIsrData.getIssrArray()); return reqForBucket; } @@ -525,17 +527,26 @@ public static List getNotifyLeaderAndIsrRequestData( } PbTableBucket pbTableBucket = reqForBucket.getTableBucket(); + LeaderAndIsr.Builder builder = + new LeaderAndIsr.Builder() + .leader(reqForBucket.getLeader()) + .leaderEpoch(reqForBucket.getLeaderEpoch()) + .isr(isr) + .coordinatorEpoch(request.getCoordinatorEpoch()) + .bucketEpoch(reqForBucket.getBucketEpoch()); + + builder.standbyReplicas( + Arrays.stream(reqForBucket.getHotStandbyReplicas()) + .boxed() + .collect(Collectors.toList())); + builder.issr( + Arrays.stream(reqForBucket.getIssrs()).boxed().collect(Collectors.toList())); notifyLeaderAndIsrDataList.add( new NotifyLeaderAndIsrData( toPhysicalTablePath(reqForBucket.getPhysicalTablePath()), toTableBucket(pbTableBucket), replicas, - new LeaderAndIsr( - reqForBucket.getLeader(), - reqForBucket.getLeaderEpoch(), - isr, - request.getCoordinatorEpoch(), - reqForBucket.getBucketEpoch()))); + builder.build())); } return notifyLeaderAndIsrDataList; } @@ -708,7 +719,10 @@ public static Map getFetchLogData(FetchLogRequest req tableId, fetchLogReqForBucket.getFetchOffset(), fetchLogReqForBucket.getMaxFetchBytes(), - projectionFields)); + projectionFields, + fetchLogReqForBucket.hasKvAppliedOffset() + ? fetchLogReqForBucket.getKvAppliedOffset() + : null)); } } @@ -1042,7 +1056,10 @@ public static AdjustIsrRequest makeAdjustIsrRequest( if (tb.getPartitionId() != null) { reqForBucket.setPartitionId(tb.getPartitionId()); } + leaderAndIsr.isr().forEach(reqForBucket::addNewIsr); + leaderAndIsr.issr().forEach(reqForBucket::addNewIssr); + if (reqForBucketByTableId.containsKey(tb.getTableId())) { reqForBucketByTableId.get(tb.getTableId()).add(reqForBucket); } else { @@ -1086,12 +1103,13 @@ public static Map getAdjustIsrData(AdjustIsrRequest r } leaderAndIsrMap.put( tb, - new LeaderAndIsr( - leaderId, - reqForBucket.getLeaderEpoch(), - newIsr, - reqForBucket.getCoordinatorEpoch(), - reqForBucket.getBucketEpoch())); + new LeaderAndIsr.Builder() + .leader(leaderId) + .leaderEpoch(reqForBucket.getLeaderEpoch()) + .isr(newIsr) + .coordinatorEpoch(reqForBucket.getCoordinatorEpoch()) + .bucketEpoch(reqForBucket.getBucketEpoch()) + .build()); } } return leaderAndIsrMap; @@ -1116,7 +1134,9 @@ public static AdjustIsrResponse makeAdjustIsrResponse( .setLeaderEpoch(leaderAndIsr.leaderEpoch()) .setCoordinatorEpoch(leaderAndIsr.coordinatorEpoch()) .setBucketEpoch(leaderAndIsr.bucketEpoch()) - .setIsrs(leaderAndIsr.isrArray()); + .setIsrs(leaderAndIsr.isrArray()) + .setHotStandbyReplicas(leaderAndIsr.standbyArray()) + .setIssrs(leaderAndIsr.issrArray()); } if (respMap.containsKey(tb.getTableId())) { @@ -1169,12 +1189,13 @@ public static Map getAdjustIsrResponseDat tb, new AdjustIsrResultForBucket( tb, - new LeaderAndIsr( - respForBucket.getLeaderId(), - respForBucket.getLeaderEpoch(), - isr, - respForBucket.getCoordinatorEpoch(), - respForBucket.getBucketEpoch()))); + new LeaderAndIsr.Builder() + .leader(respForBucket.getLeaderId()) + .leaderEpoch(respForBucket.getLeaderEpoch()) + .isr(isr) + .coordinatorEpoch(respForBucket.getCoordinatorEpoch()) + .bucketEpoch(respForBucket.getBucketEpoch()) + .build())); } } return adjustIsrResult; 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 fdea4c0153..6bab29da02 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 @@ -18,9 +18,11 @@ package org.apache.fluss.server.zk.data; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Objects; +import static org.apache.fluss.server.coordinator.CoordinatorContext.INITIAL_COORDINATOR_EPOCH; import static org.apache.fluss.utils.Preconditions.checkNotNull; /** @@ -46,36 +48,49 @@ public class LeaderAndIsr { /** The coordinator epoch. */ private final int coordinatorEpoch; + /** The latest standbyReplica collection. */ + private final List standbyReplicas; + + /** The latest inSyncStandby replica collection. * */ + private final List issr; + /** * The epoch of the state of the bucket (i.e., the leader and isr information). The epoch is a * monotonically increasing value which is incremented after every leaderAndIsr change. */ private final int bucketEpoch; - public LeaderAndIsr(int leader, int coordinatorEpoch) { - this( - leader, - INITIAL_LEADER_EPOCH, - new ArrayList<>(), - coordinatorEpoch, - INITIAL_BUCKET_EPOCH); - } - - public LeaderAndIsr( - int leader, int leaderEpoch, List isr, int coordinatorEpoch, int bucketEpoch) { + private LeaderAndIsr( + int leader, + int leaderEpoch, + List isr, + int coordinatorEpoch, + int bucketEpoch, + List standbyReplicas, + List issr) { this.leader = leader; this.leaderEpoch = leaderEpoch; this.isr = checkNotNull(isr); this.coordinatorEpoch = coordinatorEpoch; this.bucketEpoch = bucketEpoch; + this.standbyReplicas = checkNotNull(standbyReplicas); + this.issr = checkNotNull(issr); } public LeaderAndIsr newLeaderAndIsr(int newLeader, List newIsr) { - return new LeaderAndIsr(newLeader, leaderEpoch, newIsr, coordinatorEpoch, bucketEpoch + 1); + return new LeaderAndIsr( + newLeader, + leaderEpoch, + newIsr, + coordinatorEpoch, + bucketEpoch + 1, + Collections.emptyList(), + Collections.emptyList()); } - public LeaderAndIsr newLeaderAndIsrWithNewLeaderEpoch() { - return new LeaderAndIsr(leader, leaderEpoch + 1, isr, coordinatorEpoch, bucketEpoch); + public LeaderAndIsr newLeaderAndIsr(List standbyReplicas, List issr) { + return new LeaderAndIsr( + leader, leaderEpoch, isr, coordinatorEpoch, bucketEpoch + 1, standbyReplicas, issr); } public int leader() { @@ -102,12 +117,20 @@ public int bucketEpoch() { return bucketEpoch; } - public boolean equalsAllowStalePartitionEpoch(LeaderAndIsr other) { - return leader == other.leader - && leaderEpoch == other.leaderEpoch - && coordinatorEpoch == other.coordinatorEpoch - && isr.equals(other.isr) - && bucketEpoch <= other.bucketEpoch; + public List standbyList() { + return standbyReplicas; + } + + public List issr() { + return issr; + } + + public int[] standbyArray() { + return standbyReplicas.stream().mapToInt(Integer::intValue).toArray(); + } + + public int[] issrArray() { + return issr.stream().mapToInt(Integer::intValue).toArray(); } @Override @@ -123,12 +146,15 @@ public boolean equals(Object o) { && leaderEpoch == that.leaderEpoch && coordinatorEpoch == that.coordinatorEpoch && bucketEpoch == that.bucketEpoch - && Objects.equals(isr, that.isr); + && Objects.equals(isr, that.isr) + && Objects.equals(standbyReplicas, that.standbyReplicas) + && Objects.equals(issr, that.issr); } @Override public int hashCode() { - return Objects.hash(leader, leaderEpoch, isr, coordinatorEpoch, bucketEpoch); + return Objects.hash( + leader, leaderEpoch, isr, coordinatorEpoch, bucketEpoch, standbyReplicas, issr); } @Override @@ -144,6 +170,73 @@ public String toString() { + coordinatorEpoch + ", bucketEpoch=" + bucketEpoch + + ", standbyReplicas=" + + standbyReplicas + + ", issr=" + + issr + '}'; } + + // --------------------------------------------------------------------------------------------- + + /** Builder for {@link LeaderAndIsr}. */ + public static class Builder { + private int leader; + private int leaderEpoch; + private List isr; + private int coordinatorEpoch; + private int bucketEpoch; + private List standbyReplicas; + private List issr; + + public Builder() { + this.leader = NO_LEADER; + this.leaderEpoch = INITIAL_LEADER_EPOCH; + this.isr = new ArrayList<>(); + this.coordinatorEpoch = INITIAL_COORDINATOR_EPOCH; + this.bucketEpoch = INITIAL_BUCKET_EPOCH; + this.standbyReplicas = new ArrayList<>(); + this.issr = new ArrayList<>(); + } + + public Builder leader(int leader) { + this.leader = leader; + return this; + } + + public Builder leaderEpoch(int leaderEpoch) { + this.leaderEpoch = leaderEpoch; + return this; + } + + public Builder isr(List isr) { + this.isr = isr; + return this; + } + + public Builder coordinatorEpoch(int coordinatorEpoch) { + this.coordinatorEpoch = coordinatorEpoch; + return this; + } + + public Builder bucketEpoch(int bucketEpoch) { + this.bucketEpoch = bucketEpoch; + return this; + } + + public Builder standbyReplicas(List hotStandbyReplicas) { + this.standbyReplicas = hotStandbyReplicas; + return this; + } + + public Builder issr(List issr) { + this.issr = issr; + return this; + } + + public LeaderAndIsr build() { + return new LeaderAndIsr( + leader, leaderEpoch, isr, coordinatorEpoch, bucketEpoch, standbyReplicas, issr); + } + } } diff --git a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/LeaderAndIsrJsonSerde.java b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/LeaderAndIsrJsonSerde.java index 9d76330f20..b31e8024b8 100644 --- a/fluss-server/src/main/java/org/apache/fluss/server/zk/data/LeaderAndIsrJsonSerde.java +++ b/fluss-server/src/main/java/org/apache/fluss/server/zk/data/LeaderAndIsrJsonSerde.java @@ -35,13 +35,15 @@ public class LeaderAndIsrJsonSerde public static final LeaderAndIsrJsonSerde INSTANCE = new LeaderAndIsrJsonSerde(); private static final String VERSION_KEY = "version"; - private static final int VERSION = 1; + private static final int VERSION = 2; private static final String LEADER = "leader"; private static final String LEADER_EPOCH = "leader_epoch"; private static final String ISR = "isr"; private static final String COORDINATOR_EPOCH = "coordinator_epoch"; private static final String BUCKET_EPOCH = "bucket_epoch"; + private static final String HOT_STANDBY_REPLICAS = "hot_standby_replicas"; + private static final String ISSR = "issr"; @Override public void serialize(LeaderAndIsr leaderAndIsr, JsonGenerator generator) throws IOException { @@ -57,11 +59,24 @@ public void serialize(LeaderAndIsr leaderAndIsr, JsonGenerator generator) throws generator.writeNumberField(COORDINATOR_EPOCH, leaderAndIsr.coordinatorEpoch()); generator.writeNumberField(BUCKET_EPOCH, leaderAndIsr.bucketEpoch()); + generator.writeArrayFieldStart(HOT_STANDBY_REPLICAS); + for (Integer replica : leaderAndIsr.standbyList()) { + generator.writeNumber(replica); + } + generator.writeEndArray(); + + generator.writeArrayFieldStart(ISSR); + for (Integer replica : leaderAndIsr.issr()) { + generator.writeNumber(replica); + } + generator.writeEndArray(); + generator.writeEndObject(); } @Override public LeaderAndIsr deserialize(JsonNode node) { + int version = node.get(VERSION_KEY).asInt(); int leader = node.get(LEADER).asInt(); int leaderEpoch = node.get(LEADER_EPOCH).asInt(); int coordinatorEpoch = node.get(COORDINATOR_EPOCH).asInt(); @@ -71,6 +86,29 @@ public LeaderAndIsr deserialize(JsonNode node) { while (isrNodes.hasNext()) { isr.add(isrNodes.next().asInt()); } - return new LeaderAndIsr(leader, leaderEpoch, isr, coordinatorEpoch, bucketEpoch); + + List hotStandbyList = new ArrayList<>(); + List iss = new ArrayList<>(); + if (version > 1) { + Iterator hotStandbyListNodes = node.get(HOT_STANDBY_REPLICAS).elements(); + while (hotStandbyListNodes.hasNext()) { + hotStandbyList.add(hotStandbyListNodes.next().asInt()); + } + + Iterator issNodes = node.get(ISSR).elements(); + while (issNodes.hasNext()) { + iss.add(issNodes.next().asInt()); + } + } + + return new LeaderAndIsr.Builder() + .leader(leader) + .leaderEpoch(leaderEpoch) + .isr(isr) + .coordinatorEpoch(coordinatorEpoch) + .bucketEpoch(bucketEpoch) + .standbyReplicas(hotStandbyList) + .issr(iss) + .build(); } } 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 9ff610557d..e25bc6b1f6 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 @@ -727,7 +727,7 @@ void testNotifyOffsetsWithShrinkISR(@TempDir Path tempDir) throws Exception { ctx -> { ctx.putBucketLeaderAndIsr( tableBucket, - new LeaderAndIsr( + leaderAndIsr( leader, leaderAndIsr.leaderEpoch(), newIsr, @@ -854,7 +854,7 @@ void testDoElectPreferredLeaders() throws Exception { // change isr in coordinator context // leader change from originLeader to follower1. LeaderAndIsr newLeaderAndIsr = - new LeaderAndIsr( + leaderAndIsr( follower1, leaderAndIsr.leaderEpoch() + 1, isr2, @@ -961,9 +961,9 @@ void testDoBucketReassignment() throws Exception { 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)); + leaderAndIsrMap.put(tb0, leaderAndIsr(0, 1, Arrays.asList(0, 1, 2, 3), 0, 0)); + leaderAndIsrMap.put(tb1, leaderAndIsr(0, 1, Arrays.asList(0, 1, 2, 3), 0, 0)); + leaderAndIsrMap.put(tb2, leaderAndIsr(1, 1, Arrays.asList(1, 2, 3, 0), 0, 0)); eventProcessor .getCoordinatorEventManager() .put(new AdjustIsrReceivedEvent(leaderAndIsrMap, respCallback)); @@ -1331,4 +1331,15 @@ private PartitionIdName(long partitionId, String partitionName) { this.partitionName = partitionName; } } + + private LeaderAndIsr leaderAndIsr( + int leader, int leaderEpoch, List isr, int coordinatorEpoch, int bucketEpoch) { + return new LeaderAndIsr.Builder() + .leader(leader) + .leaderEpoch(leaderEpoch) + .isr(isr) + .coordinatorEpoch(coordinatorEpoch) + .bucketEpoch(bucketEpoch) + .build(); + } } 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 a210a314cd..30fa19723e 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 @@ -251,12 +251,15 @@ public CompletableFuture adjustIsr(AdjustIsrRequest request) adjustIsrResultForBucket = new AdjustIsrResultForBucket( tb, - new LeaderAndIsr( - leaderAndIsr.leader(), - currentLeaderEpoch, - leaderAndIsr.isr(), - leaderAndIsr.coordinatorEpoch(), - leaderAndIsr.bucketEpoch() + 1)); + new LeaderAndIsr.Builder() + .leader(leaderAndIsr.leader()) + .leaderEpoch(currentLeaderEpoch) + .isr(leaderAndIsr.isr()) + .coordinatorEpoch(leaderAndIsr.coordinatorEpoch()) + .bucketEpoch(leaderAndIsr.bucketEpoch() + 1) + .standbyReplicas(leaderAndIsr.standbyList()) + .issr(leaderAndIsr.issr()) + .build()); } resultForBuckets.add(adjustIsrResultForBucket); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachineTest.java b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachineTest.java index 4c3cf1ab4e..bb237b9b4b 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachineTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/coordinator/statemachine/ReplicaStateMachineTest.java @@ -155,8 +155,8 @@ void testDeleteReplicaStateChange() { TableBucketReplica b2Replica1 = new TableBucketReplica(tableBucket2, 1); List replicas = Arrays.asList(b1Replica0, b1Replica1, b2Replica0, b2Replica1); - coordinatorContext.putBucketLeaderAndIsr(tableBucket1, new LeaderAndIsr(0, 0)); - coordinatorContext.putBucketLeaderAndIsr(tableBucket2, new LeaderAndIsr(0, 0)); + coordinatorContext.putBucketLeaderAndIsr(tableBucket1, leaderAndIsr(0, 0)); + coordinatorContext.putBucketLeaderAndIsr(tableBucket2, leaderAndIsr(0, 0)); toReplicaDeletionStartedState(replicaStateMachine, replicas); for (TableBucketReplica replica : isReplicaDeleteSuccess.keySet()) { @@ -167,8 +167,8 @@ void testDeleteReplicaStateChange() { coordinatorContext = new CoordinatorContext(); coordinatorContext.setLiveTabletServers( CoordinatorTestUtils.createServers(Arrays.asList(0, 1))); - coordinatorContext.putBucketLeaderAndIsr(tableBucket1, new LeaderAndIsr(0, 0)); - coordinatorContext.putBucketLeaderAndIsr(tableBucket2, new LeaderAndIsr(0, 0)); + coordinatorContext.putBucketLeaderAndIsr(tableBucket1, leaderAndIsr(0, 0)); + coordinatorContext.putBucketLeaderAndIsr(tableBucket2, leaderAndIsr(0, 0)); // delete replica will fail for some gateway will return exception CoordinatorTestUtils.makeSendLeaderAndStopRequestFailContext( @@ -213,7 +213,7 @@ void testOfflineReplicasShouldBeRemovedFromIsr() throws Exception { replicas.add(replica); } // put leader and isr - LeaderAndIsr leaderAndIsr = new LeaderAndIsr(0, 0, Arrays.asList(0, 1, 2), 0, 0); + LeaderAndIsr leaderAndIsr = leaderAndIsr(0, 0, Arrays.asList(0, 1, 2), 0, 0); zookeeperClient.registerLeaderAndIsr(tableBucket, leaderAndIsr); coordinatorContext.updateBucketReplicaAssignment(tableBucket, Arrays.asList(0, 1, 2)); coordinatorContext.putBucketLeaderAndIsr(tableBucket, leaderAndIsr); @@ -223,7 +223,9 @@ void testOfflineReplicasShouldBeRemovedFromIsr() throws Exception { replicaStateMachine.handleStateChanges(replicas, OfflineReplica); leaderAndIsr = coordinatorContext.getBucketLeaderAndIsr(tableBucket).get(); assertThat(leaderAndIsr) - .isEqualTo(new LeaderAndIsr(LeaderAndIsr.NO_LEADER, 0, Arrays.asList(2), 0, 3)); + .isEqualTo( + leaderAndIsr( + LeaderAndIsr.NO_LEADER, 0, Collections.singletonList(2), 0, 3)); } @Test @@ -249,7 +251,7 @@ void testOfflineReplicaShouldBeRemovedFromIsr() throws Exception { coordinatorContext.putReplicaState(replica, OnlineReplica); } // put leader and isr - LeaderAndIsr leaderAndIsr = new LeaderAndIsr(0, 0, Arrays.asList(0, 1, 2), 0, 0); + LeaderAndIsr leaderAndIsr = leaderAndIsr(0, 0, Arrays.asList(0, 1, 2), 0, 0); zookeeperClient.registerLeaderAndIsr(tableBucket, leaderAndIsr); coordinatorContext.updateBucketReplicaAssignment(tableBucket, Arrays.asList(0, 1, 2)); coordinatorContext.putBucketLeaderAndIsr(tableBucket, leaderAndIsr); @@ -258,14 +260,13 @@ void testOfflineReplicaShouldBeRemovedFromIsr() throws Exception { replicaStateMachine.handleStateChanges( Collections.singleton(new TableBucketReplica(tableBucket, 1)), OfflineReplica); leaderAndIsr = coordinatorContext.getBucketLeaderAndIsr(tableBucket).get(); - assertThat(leaderAndIsr).isEqualTo(new LeaderAndIsr(0, 0, Arrays.asList(0, 2), 0, 1)); + assertThat(leaderAndIsr).isEqualTo(leaderAndIsr(0, 0, Arrays.asList(0, 2), 0, 1)); // set replica 2 to offline replicaStateMachine.handleStateChanges( Collections.singleton(new TableBucketReplica(tableBucket, 2)), OfflineReplica); leaderAndIsr = coordinatorContext.getBucketLeaderAndIsr(tableBucket).get(); - assertThat(leaderAndIsr) - .isEqualTo(new LeaderAndIsr(0, 0, Collections.singletonList(0), 0, 2)); + assertThat(leaderAndIsr).isEqualTo(leaderAndIsr(0, 0, Collections.singletonList(0), 0, 2)); // set replica 0 to offline, isr shouldn't be empty, leader should be NO_LEADER replicaStateMachine.handleStateChanges( @@ -273,7 +274,7 @@ void testOfflineReplicaShouldBeRemovedFromIsr() throws Exception { leaderAndIsr = coordinatorContext.getBucketLeaderAndIsr(tableBucket).get(); assertThat(leaderAndIsr) .isEqualTo( - new LeaderAndIsr( + leaderAndIsr( LeaderAndIsr.NO_LEADER, 0, Collections.singletonList(0), 0, 3)); } @@ -343,4 +344,19 @@ private List createServers(int[] serverIds) { } return servers; } + + private LeaderAndIsr leaderAndIsr(int leader, int coordinatorEpoch) { + return new LeaderAndIsr.Builder().leader(leader).coordinatorEpoch(coordinatorEpoch).build(); + } + + private LeaderAndIsr leaderAndIsr( + int leader, int leaderEpoch, List isr, int coordinatorEpoch, int bucketEpoch) { + return new LeaderAndIsr.Builder() + .leader(leader) + .leaderEpoch(leaderEpoch) + .isr(isr) + .coordinatorEpoch(coordinatorEpoch) + .bucketEpoch(bucketEpoch) + .build(); + } } 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 e0408e8678..0691bd6245 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 @@ -132,10 +132,24 @@ void testStartup() throws Exception { coordinatorContext.updateBucketReplicaAssignment(t2b0, Arrays.asList(1, 2)); // create LeaderAndIsr for t10/t11 info in zk, + LeaderAndIsr.Builder builder = new LeaderAndIsr.Builder(); zookeeperClient.registerLeaderAndIsr( - new TableBucket(t1Id, 0), new LeaderAndIsr(0, 0, Arrays.asList(0, 1), 0, 0)); + new TableBucket(t1Id, 0), + builder.leader(0) + .leaderEpoch(0) + .isr(Arrays.asList(0, 1)) + .coordinatorEpoch(0) + .bucketEpoch(0) + .build()); + builder = new LeaderAndIsr.Builder(); zookeeperClient.registerLeaderAndIsr( - new TableBucket(t1Id, 1), new LeaderAndIsr(2, 0, Arrays.asList(2, 3), 0, 0)); + new TableBucket(t1Id, 1), + builder.leader(2) + .leaderEpoch(0) + .isr(Arrays.asList(2, 3)) + .coordinatorEpoch(0) + .bucketEpoch(0) + .build()); // update the LeaderAndIsr to context coordinatorContext.putBucketLeaderAndIsr( t1b0, zookeeperClient.getLeaderAndIsr(new TableBucket(t1Id, 0)).get()); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/log/FetchParamsTest.java b/fluss-server/src/test/java/org/apache/fluss/server/log/FetchParamsTest.java index d6762b3997..64a3e9731b 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/log/FetchParamsTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/log/FetchParamsTest.java @@ -32,13 +32,13 @@ class FetchParamsTest { void testSetCurrentFetch() { FetchParams fetchParams = new FetchParams(1, 100); fetchParams.setCurrentFetch( - 1L, 20L, 1024, TestData.DATA1_ROW_TYPE, DEFAULT_COMPRESSION, null); + 1L, 20L, 1024, TestData.DATA1_ROW_TYPE, DEFAULT_COMPRESSION, null, -1L); assertThat(fetchParams.fetchOffset()).isEqualTo(20L); assertThat(fetchParams.maxFetchBytes()).isEqualTo(1024); assertThat(fetchParams.projection()).isNull(); fetchParams.setCurrentFetch( - 2L, 30L, 512, TestData.DATA2_ROW_TYPE, DEFAULT_COMPRESSION, new int[] {0, 2}); + 2L, 30L, 512, TestData.DATA2_ROW_TYPE, DEFAULT_COMPRESSION, new int[] {0, 2}, -1L); assertThat(fetchParams.fetchOffset()).isEqualTo(30L); assertThat(fetchParams.maxFetchBytes()).isEqualTo(512); assertThat(fetchParams.projection()).isNotNull(); @@ -46,11 +46,11 @@ void testSetCurrentFetch() { FileLogProjection prevProjection = fetchParams.projection(); fetchParams.setCurrentFetch( - 1L, 40L, 256, TestData.DATA1_ROW_TYPE, DEFAULT_COMPRESSION, null); + 1L, 40L, 256, TestData.DATA1_ROW_TYPE, DEFAULT_COMPRESSION, null, -1L); assertThat(fetchParams.projection()).isNull(); fetchParams.setCurrentFetch( - 2L, 30L, 512, TestData.DATA2_ROW_TYPE, DEFAULT_COMPRESSION, new int[] {0, 2}); + 2L, 30L, 512, TestData.DATA2_ROW_TYPE, DEFAULT_COMPRESSION, new int[] {0, 2}, -1L); // the FileLogProjection should be cached assertThat(fetchParams.projection()).isNotNull().isSameAs(prevProjection); } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/log/remote/RemoteLogTestBase.java b/fluss-server/src/test/java/org/apache/fluss/server/log/remote/RemoteLogTestBase.java index 7a0a8a1257..4416d8db36 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/log/remote/RemoteLogTestBase.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/log/remote/RemoteLogTestBase.java @@ -86,7 +86,13 @@ private Replica makeReplicaAndAddSegments( physicalTablePath, tb, Collections.singletonList(0), - new LeaderAndIsr(0, 0, Collections.singletonList(0), 0, 0))); + new LeaderAndIsr.Builder() + .leader(0) + .leaderEpoch(0) + .isr(Collections.singletonList(0)) + .coordinatorEpoch(0) + .bucketEpoch(0) + .build())); addMultiSegmentsToLogTablet(replica.getLogTablet(), segmentSize); return replica; } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/AdjustIsrITCase.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/AdjustIsrITCase.java index 933abd231d..c74c6460b9 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/AdjustIsrITCase.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/AdjustIsrITCase.java @@ -131,12 +131,13 @@ void testIsrShrinkAndExpand() throws Exception { currentLeaderAndIsr = zkClient.getLeaderAndIsr(tb).get(); LeaderAndIsr newLeaderAndIsr = - new LeaderAndIsr( - currentLeaderAndIsr.leader(), - currentLeaderAndIsr.leaderEpoch() + 1, - isr, - currentLeaderAndIsr.coordinatorEpoch(), - currentLeaderAndIsr.bucketEpoch()); + new LeaderAndIsr.Builder() + .leader(currentLeaderAndIsr.leader()) + .leaderEpoch(currentLeaderAndIsr.leaderEpoch() + 1) + .isr(isr) + .coordinatorEpoch(currentLeaderAndIsr.coordinatorEpoch()) + .bucketEpoch(currentLeaderAndIsr.bucketEpoch()) + .build(); isr.add(stopFollower); FLUSS_CLUSTER_EXTENSION.notifyLeaderAndIsr( stopFollower, DATA1_TABLE_PATH, tb, newLeaderAndIsr, isr); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/AdjustIsrManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/AdjustIsrManagerTest.java index 3f8623569c..0c66356099 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/AdjustIsrManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/AdjustIsrManagerTest.java @@ -41,19 +41,51 @@ void testSubmitShrinkIsr() throws Exception { new AdjustIsrManager( new FlussScheduler(1), new TestCoordinatorGateway(), tabletServerId); + LeaderAndIsr.Builder builder = new LeaderAndIsr.Builder(); + // shrink isr TableBucket tb = new TableBucket(150001L, 0); List currentIsr = Arrays.asList(1, 2); - LeaderAndIsr adjustIsr = new LeaderAndIsr(tabletServerId, 0, currentIsr, 0, 0); + + LeaderAndIsr adjustIsr = + builder.leader(tabletServerId) + .leaderEpoch(0) + .isr(currentIsr) + .coordinatorEpoch(0) + .bucketEpoch(0) + .build(); LeaderAndIsr result = adjustIsrManager.submit(tb, adjustIsr).get(); + + builder = new LeaderAndIsr.Builder(); assertThat(result) - .isEqualTo(new LeaderAndIsr(tabletServerId, 0, Arrays.asList(1, 2), 0, 1)); + .isEqualTo( + builder.leader(tabletServerId) + .leaderEpoch(0) + .isr(Arrays.asList(1, 2)) + .coordinatorEpoch(0) + .bucketEpoch(1) + .build()); // expand isr currentIsr = Arrays.asList(1, 2, 3); - adjustIsr = new LeaderAndIsr(tabletServerId, 0, currentIsr, 0, 1); + builder = new LeaderAndIsr.Builder(); + adjustIsr = + builder.leader(tabletServerId) + .leaderEpoch(0) + .isr(currentIsr) + .coordinatorEpoch(0) + .bucketEpoch(1) + .build(); result = adjustIsrManager.submit(tb, adjustIsr).get(); + + builder = new LeaderAndIsr.Builder(); assertThat(result) - .isEqualTo(new LeaderAndIsr(tabletServerId, 0, Arrays.asList(1, 2, 3), 0, 2)); + .isEqualTo( + builder.leader(tabletServerId) + .leaderEpoch(0) + .isr(Arrays.asList(1, 2, 3)) + .coordinatorEpoch(0) + .bucketEpoch(2) + .build()); } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/AdjustIsrTest.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/AdjustIsrTest.java index f90561485f..f668fc944c 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/AdjustIsrTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/AdjustIsrTest.java @@ -41,7 +41,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -/** UT test for adjust isr for tablet server. */ +/** UT test for adjust isr for TabletServer. */ public class AdjustIsrTest extends ReplicaTestBase { @Override @@ -129,7 +129,10 @@ void testSubmitShrinkIsrAsLeaderFenced() throws Exception { // To mock we prepare an isr shrink in Replica#maybeShrinkIsr(); IsrState.PendingShrinkIsrState pendingShrinkIsrState = replica.prepareIsrShrink( - new IsrState.CommittedIsrState(Arrays.asList(1, 2, 3)), + new IsrState.CommittedIsrState( + Arrays.asList(1, 2, 3), + Collections.emptyList(), + Collections.emptyList()), Arrays.asList(1, 2), Collections.singletonList(3)); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/FollowerReplicaTest.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/FollowerReplicaTest.java index 2cf8c1f501..9a535cc727 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/FollowerReplicaTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/FollowerReplicaTest.java @@ -122,14 +122,17 @@ private void assertFollowerReplicaState( private long updateFetchState(long followerFetchOffset, long leaderEndOffset) { long currentTimeMillis = System.currentTimeMillis(); followerReplica.updateFetchState( - new LogOffsetMetadata(followerFetchOffset), currentTimeMillis, leaderEndOffset); + new LogOffsetMetadata(followerFetchOffset), + -1L, + currentTimeMillis, + leaderEndOffset); return currentTimeMillis; } private void updateFetchState( long currentTimeMs, long followerFetchOffset, long leaderEndOffset) { followerReplica.updateFetchState( - new LogOffsetMetadata(followerFetchOffset), currentTimeMs, leaderEndOffset); + new LogOffsetMetadata(followerFetchOffset), -1L, currentTimeMs, leaderEndOffset); } private long resetReplicaState( diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/HighWatermarkPersistenceTest.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/HighWatermarkPersistenceTest.java index b093787f02..88ee7c2729 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/HighWatermarkPersistenceTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/HighWatermarkPersistenceTest.java @@ -101,12 +101,10 @@ void testHighWatermarkPersistenceMultipleReplicas() throws Exception { PhysicalTablePath.of(DATA2_TABLE_PATH), tableBucket1, Collections.singletonList(TABLET_SERVER_ID), - new LeaderAndIsr( - TABLET_SERVER_ID, - LeaderAndIsr.INITIAL_LEADER_EPOCH, - Collections.singletonList(TABLET_SERVER_ID), - INITIAL_COORDINATOR_EPOCH, - LeaderAndIsr.INITIAL_BUCKET_EPOCH))), + new LeaderAndIsr.Builder() + .leader(TABLET_SERVER_ID) + .isr(Collections.singletonList(TABLET_SERVER_ID)) + .build())), result -> {}); replicaManager.checkpointHighWatermarks(); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java index 0d90a9a252..242a682825 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaManagerTest.java @@ -1080,12 +1080,7 @@ void becomeLeaderOrFollower() throws Exception { PhysicalTablePath.of(DATA1_TABLE_PATH), tb, Arrays.asList(1, 2, 3), - new LeaderAndIsr( - TABLET_SERVER_ID, - 1, - Arrays.asList(1, 2, 3), - INITIAL_COORDINATOR_EPOCH, - INITIAL_BUCKET_EPOCH))), + leaderAndIsr(TABLET_SERVER_ID, 1, Arrays.asList(1, 2, 3)))), future::complete); assertThat(future.get()).containsOnly(new NotifyLeaderAndIsrResultForBucket(tb)); assertReplicaEpochEquals( @@ -1100,12 +1095,10 @@ void becomeLeaderOrFollower() throws Exception { PhysicalTablePath.of(DATA1_TABLE_PATH), tb, Arrays.asList(1, 2, 3), - new LeaderAndIsr( + leaderAndIsr( TABLET_SERVER_ID, INITIAL_LEADER_EPOCH, - Arrays.asList(1, 2, 3), - INITIAL_COORDINATOR_EPOCH, - INITIAL_BUCKET_EPOCH))), + Arrays.asList(1, 2, 3)))), future::complete); assertThat(future.get()) .containsOnly( @@ -1134,12 +1127,7 @@ void testStopReplica() throws Exception { PhysicalTablePath.of(DATA1_TABLE_PATH), tb, Arrays.asList(1, 2, 3), - new LeaderAndIsr( - TABLET_SERVER_ID, - 1, - Arrays.asList(1, 2, 3), - INITIAL_COORDINATOR_EPOCH, - INITIAL_BUCKET_EPOCH))), + leaderAndIsr(TABLET_SERVER_ID, 1, Arrays.asList(1, 2, 3)))), future::complete); assertThat(future.get()).containsOnly(new NotifyLeaderAndIsrResultForBucket(tb)); assertReplicaEpochEquals( @@ -1165,12 +1153,7 @@ void testStopReplica() throws Exception { PhysicalTablePath.of(DATA1_TABLE_PATH), tb, Arrays.asList(1, 2, 3), - new LeaderAndIsr( - TABLET_SERVER_ID, - 2, - Arrays.asList(1, 2, 3), - INITIAL_COORDINATOR_EPOCH, - INITIAL_BUCKET_EPOCH))), + leaderAndIsr(TABLET_SERVER_ID, 2, Arrays.asList(1, 2, 3)))), future::complete); assertThat(future.get()).containsOnly(new NotifyLeaderAndIsrResultForBucket(tb)); assertReplicaEpochEquals( @@ -1882,4 +1865,8 @@ private void assertUpdateMetadataEquals( } }); } + + private LeaderAndIsr leaderAndIsr(int leader, int leaderEpoch, List isr) { + return new LeaderAndIsr.Builder().leader(leader).leaderEpoch(leaderEpoch).isr(isr).build(); + } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTest.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTest.java index b0b7d02a5b..f972d409d8 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTest.java @@ -68,7 +68,6 @@ import static org.apache.fluss.record.TestData.DATA1_TABLE_PATH; import static org.apache.fluss.record.TestData.DATA1_TABLE_PATH_PK; import static org.apache.fluss.record.TestData.DEFAULT_SCHEMA_ID; -import static org.apache.fluss.server.coordinator.CoordinatorContext.INITIAL_COORDINATOR_EPOCH; import static org.apache.fluss.server.zk.data.LeaderAndIsr.INITIAL_LEADER_EPOCH; import static org.apache.fluss.testutils.DataTestUtils.assertLogRecordsEquals; import static org.apache.fluss.testutils.DataTestUtils.createBasicMemoryLogRecords; @@ -124,7 +123,13 @@ void testAppendRecordsToLeader() throws Exception { conf.get(ConfigOptions.CLIENT_SCANNER_LOG_FETCH_MAX_BYTES) .getBytes()); fetchParams.setCurrentFetch( - DATA1_TABLE_ID, 0, Integer.MAX_VALUE, DATA1_ROW_TYPE, DEFAULT_COMPRESSION, null); + DATA1_TABLE_ID, + 0, + Integer.MAX_VALUE, + DATA1_ROW_TYPE, + DEFAULT_COMPRESSION, + null, + -1L); LogReadInfo logReadInfo = logReplica.fetchRecords(fetchParams); assertLogRecordsEquals(DATA1_ROW_TYPE, logReadInfo.getFetchedData().getRecords(), DATA1); } @@ -521,13 +526,12 @@ private void makeKvReplicaAsFollower(Replica replica, int leaderEpoch) { PhysicalTablePath.of(DATA1_TABLE_PATH_PK), new TableBucket(DATA1_TABLE_ID_PK, 1), Collections.singletonList(TABLET_SERVER_ID), - new LeaderAndIsr( - TABLET_SERVER_ID, - leaderEpoch, - Collections.singletonList(TABLET_SERVER_ID), - INITIAL_COORDINATOR_EPOCH, - // we also use the leader epoch as bucket epoch - leaderEpoch))); + new LeaderAndIsr.Builder() + .leader(TABLET_SERVER_ID) + .leaderEpoch(leaderEpoch) + .isr(Collections.singletonList(TABLET_SERVER_ID)) + .bucketEpoch(leaderEpoch) + .build())); } private void makeLeaderReplica( @@ -538,13 +542,12 @@ private void makeLeaderReplica( PhysicalTablePath.of(tablePath), tableBucket, Collections.singletonList(TABLET_SERVER_ID), - new LeaderAndIsr( - TABLET_SERVER_ID, - leaderEpoch, - Collections.singletonList(TABLET_SERVER_ID), - INITIAL_COORDINATOR_EPOCH, - // we also use the leader epoch as bucket epoch - leaderEpoch))); + new LeaderAndIsr.Builder() + .leader(TABLET_SERVER_ID) + .leaderEpoch(leaderEpoch) + .isr(Collections.singletonList(TABLET_SERVER_ID)) + .bucketEpoch(leaderEpoch) + .build())); } private static LogRecords fetchRecords(Replica replica) throws IOException { @@ -559,7 +562,8 @@ private static LogRecords fetchRecords(Replica replica, long offset) throws IOEx Integer.MAX_VALUE, replica.getRowType(), DEFAULT_COMPRESSION, - null); + null, + -1L); LogReadInfo logReadInfo = replica.fetchRecords(fetchParams); return logReadInfo.getFetchedData().getRecords(); } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTestBase.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTestBase.java index 55df33f6c9..ec100f24b1 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTestBase.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/ReplicaTestBase.java @@ -52,6 +52,7 @@ import org.apache.fluss.server.metadata.TabletServerMetadataCache; import org.apache.fluss.server.metrics.group.BucketMetricGroup; import org.apache.fluss.server.metrics.group.TestingMetricGroups; +import org.apache.fluss.server.replica.standby.KvStandbyManager; import org.apache.fluss.server.zk.NOPErrorHandler; import org.apache.fluss.server.zk.ZooKeeperClient; import org.apache.fluss.server.zk.ZooKeeperExtension; @@ -107,7 +108,6 @@ import static org.apache.fluss.record.TestData.DATA2_TABLE_PATH; import static org.apache.fluss.server.coordinator.CoordinatorContext.INITIAL_COORDINATOR_EPOCH; import static org.apache.fluss.server.replica.ReplicaManager.HIGH_WATERMARK_CHECKPOINT_FILE_NAME; -import static org.apache.fluss.server.zk.data.LeaderAndIsr.INITIAL_BUCKET_EPOCH; import static org.apache.fluss.server.zk.data.LeaderAndIsr.INITIAL_LEADER_EPOCH; import static org.apache.fluss.testutils.DataTestUtils.genMemoryLogRecordsWithWriterId; import static org.apache.fluss.utils.FlussPaths.remoteLogDir; @@ -133,6 +133,7 @@ public class ReplicaTestBase { protected ManualClock manualClock; protected LogManager logManager; protected KvManager kvManager; + protected KvStandbyManager kvStandbyManager; protected ReplicaManager replicaManager; protected RpcClient rpcClient; protected Configuration conf; @@ -184,6 +185,14 @@ public void setup() throws Exception { kvManager = KvManager.create(conf, zkClient, logManager); kvManager.startup(); + kvStandbyManager = + new KvStandbyManager( + conf, + scheduler, + zkClient, + logManager.getDataDir().getAbsolutePath(), + remoteLogManager); + serverMetadataCache = new TabletServerMetadataCache(new MetadataManager(zkClient, conf), zkClient); initMetadataCache(serverMetadataCache); @@ -351,12 +360,10 @@ protected void makeLogTableAsLeader( : DATA1_PHYSICAL_TABLE_PATH, tb, replicas, - new LeaderAndIsr( - TABLET_SERVER_ID, - INITIAL_LEADER_EPOCH, - isr, - INITIAL_COORDINATOR_EPOCH, - INITIAL_BUCKET_EPOCH)))); + new LeaderAndIsr.Builder() + .leader(TABLET_SERVER_ID) + .isr(isr) + .build()))); } // TODO this is only for single tablet server unit test. @@ -392,13 +399,13 @@ protected void makeKvTableAsLeader( : PhysicalTablePath.of(tablePath), tb, replicas, - new LeaderAndIsr( - TABLET_SERVER_ID, - leaderEpoch, - isr, - INITIAL_COORDINATOR_EPOCH, - // use leader epoch as bucket epoch - leaderEpoch)))); + new LeaderAndIsr.Builder() + .leader(TABLET_SERVER_ID) + .leaderEpoch(leaderEpoch) + .isr(isr) + .coordinatorEpoch(INITIAL_COORDINATOR_EPOCH) + .bucketEpoch(leaderEpoch) + .build()))); } protected void makeLeaderAndFollower(List notifyLeaderAndIsrDataList) { @@ -459,7 +466,8 @@ private Replica makeReplica( NOPErrorHandler.INSTANCE, metricGroup, DATA1_TABLE_INFO, - manualClock); + manualClock, + isPkTable ? kvStandbyManager : null); } private void initRemoteLogEnv() throws Exception { diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherITCase.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherITCase.java index 0b149abee0..cdf383a59a 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherITCase.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherITCase.java @@ -309,12 +309,13 @@ void testFlushForPutKvNeedAck() throws Exception { // then the kv should be flushed finally LeaderAndIsr currentLeaderAndIsr = zkClient.getLeaderAndIsr(tb).get(); LeaderAndIsr newLeaderAndIsr = - new LeaderAndIsr( - currentLeaderAndIsr.leader(), - currentLeaderAndIsr.leaderEpoch() + 1, - currentLeaderAndIsr.isr(), - currentLeaderAndIsr.coordinatorEpoch(), - currentLeaderAndIsr.bucketEpoch()); + new LeaderAndIsr.Builder() + .leader(currentLeaderAndIsr.leader()) + .leaderEpoch(currentLeaderAndIsr.leaderEpoch() + 1) + .isr(currentLeaderAndIsr.isr()) + .coordinatorEpoch(currentLeaderAndIsr.coordinatorEpoch()) + .bucketEpoch(currentLeaderAndIsr.bucketEpoch()) + .build(); FLUSS_CLUSTER_EXTENSION.notifyLeaderAndIsr( followerToStop, DATA1_TABLE_PATH, tb, newLeaderAndIsr, Arrays.asList(0, 1, 2)); diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherManagerTest.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherManagerTest.java index 1b3d60fa63..1da11b67b0 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherManagerTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherManagerTest.java @@ -60,6 +60,7 @@ public void setup() throws Exception { "test-fetcher-thread", replicaManager, new RemoteLeaderEndpoint( + replicaManager, conf, TABLET_SERVER_ID, leader.id(), @@ -87,12 +88,10 @@ void testAddAndRemoveBucket() { PhysicalTablePath.of(DATA1_TABLE_PATH), tb, Arrays.asList(leader.id(), TABLET_SERVER_ID), - new LeaderAndIsr( - leader.id(), - LeaderAndIsr.INITIAL_LEADER_EPOCH, - Arrays.asList(leader.id(), TABLET_SERVER_ID), - INITIAL_COORDINATOR_EPOCH, - LeaderAndIsr.INITIAL_BUCKET_EPOCH))), + new LeaderAndIsr.Builder() + .leader(leader.id()) + .isr(Arrays.asList(leader.id(), TABLET_SERVER_ID)) + .build())), result -> {}); InitialFetchStatus initialFetchStatus = diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherThreadTest.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherThreadTest.java index 777999f7ee..738c47072e 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherThreadTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/ReplicaFetcherThreadTest.java @@ -72,8 +72,6 @@ import static org.apache.fluss.record.TestData.DATA1_TABLE_ID; import static org.apache.fluss.record.TestData.DATA1_TABLE_PATH; import static org.apache.fluss.server.coordinator.CoordinatorContext.INITIAL_COORDINATOR_EPOCH; -import static org.apache.fluss.server.zk.data.LeaderAndIsr.INITIAL_BUCKET_EPOCH; -import static org.apache.fluss.server.zk.data.LeaderAndIsr.INITIAL_LEADER_EPOCH; import static org.apache.fluss.testutils.DataTestUtils.genMemoryLogRecordsByObject; import static org.apache.fluss.testutils.DataTestUtils.genMemoryLogRecordsWithWriterId; import static org.apache.fluss.testutils.common.CommonTestUtils.retry; @@ -293,12 +291,10 @@ private void makeLeaderAndFollower() { PhysicalTablePath.of(DATA1_TABLE_PATH), tb, Arrays.asList(leaderServerId, followerServerId), - new LeaderAndIsr( - leaderServerId, - INITIAL_LEADER_EPOCH, - Arrays.asList(leaderServerId, followerServerId), - INITIAL_COORDINATOR_EPOCH, - INITIAL_BUCKET_EPOCH))), + new LeaderAndIsr.Builder() + .leader(leaderServerId) + .isr(Arrays.asList(leaderServerId, followerServerId)) + .build())), result -> {}); followerRM.becomeLeaderOrFollower( INITIAL_COORDINATOR_EPOCH, @@ -307,12 +303,10 @@ private void makeLeaderAndFollower() { PhysicalTablePath.of(DATA1_TABLE_PATH), tb, Arrays.asList(leaderServerId, followerServerId), - new LeaderAndIsr( - leaderServerId, - INITIAL_LEADER_EPOCH, - Arrays.asList(leaderServerId, followerServerId), - INITIAL_COORDINATOR_EPOCH, - INITIAL_BUCKET_EPOCH))), + new LeaderAndIsr.Builder() + .leader(leaderServerId) + .isr(Arrays.asList(leaderServerId, followerServerId)) + .build())), result -> {}); } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/TestingLeaderEndpoint.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/TestingLeaderEndpoint.java index ecae06d5d5..3f9ee56cad 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/TestingLeaderEndpoint.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/fetcher/TestingLeaderEndpoint.java @@ -33,6 +33,7 @@ import org.apache.fluss.server.replica.ReplicaManager; import java.nio.ByteBuffer; +import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.Optional; @@ -102,7 +103,13 @@ public CompletableFuture fetchLog(FetchLogContext fetchLogContext) { public Optional buildFetchLogContext( Map replicas) { return RemoteLeaderEndpoint.buildFetchLogContext( - replicas, localNode.id(), maxFetchSize, maxFetchSizeForBucket, -1, -1); + replicas, + Collections.emptyMap(), + localNode.id(), + maxFetchSize, + maxFetchSizeForBucket, + -1, + -1); } @Override diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/standby/AdjustIssrTest.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/standby/AdjustIssrTest.java new file mode 100644 index 0000000000..9f78d73dcc --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/standby/AdjustIssrTest.java @@ -0,0 +1,139 @@ +/* + * 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.replica.standby; + +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.rpc.entity.PutKvResultForBucket; +import org.apache.fluss.server.entity.FetchReqInfo; +import org.apache.fluss.server.entity.NotifyLeaderAndIsrData; +import org.apache.fluss.server.log.FetchParams; +import org.apache.fluss.server.replica.Replica; +import org.apache.fluss.server.replica.ReplicaTestBase; +import org.apache.fluss.server.zk.data.LeaderAndIsr; +import org.apache.fluss.utils.types.Tuple2; + +import org.junit.jupiter.api.Test; + +import java.time.Duration; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CompletableFuture; + +import static org.apache.fluss.record.TestData.DATA1_KEY_TYPE; +import static org.apache.fluss.record.TestData.DATA1_PHYSICAL_TABLE_PATH_PK; +import static org.apache.fluss.record.TestData.DATA1_ROW_TYPE; +import static org.apache.fluss.record.TestData.DATA1_TABLE_ID_PK; +import static org.apache.fluss.testutils.DataTestUtils.genKvRecordBatchWithWriterId; +import static org.apache.fluss.testutils.common.CommonTestUtils.retry; +import static org.assertj.core.api.Assertions.assertThat; + +/** UT Test for adjust isrr(InsyncStandbyReplica). */ +public class AdjustIssrTest extends ReplicaTestBase { + @Override + public Configuration getServerConf() { + Configuration conf = new Configuration(); + conf.set(ConfigOptions.LOG_REPLICA_MAX_LAG_TIME, Duration.ofSeconds(3)); + return conf; + } + + @Test + void testExpandIssr() throws Exception { + // replica set is 1,2,3 , isr set is 1. standbySet is 2. + TableBucket tb = new TableBucket(DATA1_TABLE_ID_PK, 0); + makeKvTableAsLeader( + tb, + Arrays.asList(1, 2, 3), + Collections.singletonList(1), + Collections.singletonList(2)); + + Replica replica = replicaManager.getReplicaOrException(tb); + assertThat(replica.getIsr()).containsExactlyInAnyOrder(1); + assertThat(replica.getStandbyReplicas()).containsExactlyInAnyOrder(2); + assertThat(replica.getIssr()).isEmpty(); + + // 1. put records to leader. + List> data1 = + Arrays.asList( + Tuple2.of(new Object[] {1}, new Object[] {1, "a"}), + Tuple2.of(new Object[] {2}, new Object[] {2, "b"}), + Tuple2.of(new Object[] {3}, new Object[] {3, "c"}), + Tuple2.of(new Object[] {1}, new Object[] {1, "a1"})); + CompletableFuture> future = new CompletableFuture<>(); + replicaManager.putRecordsToKv( + 20000, + 1, + Collections.singletonMap( + tb, + genKvRecordBatchWithWriterId( + data1, DATA1_KEY_TYPE, DATA1_ROW_TYPE, 100L, 0)), + null, + future::complete); + assertThat(future.get()).containsOnly(new PutKvResultForBucket(tb, 5)); + assertThat(replicaManager.getReplicaOrException(tb).getLocalLogEndOffset()).isEqualTo(5L); + + // mock follower 3 (not standby replica) to fetch data from leader. fetch offset is 5 + // (which indicate the follower catch up the leader, it will be added into isr list). + replicaManager.fetchLogRecords( + new FetchParams( + 3, (int) conf.get(ConfigOptions.LOG_REPLICA_FETCH_MAX_BYTES).getBytes()), + Collections.singletonMap( + tb, new FetchReqInfo(tb.getTableId(), 5L, Integer.MAX_VALUE)), + result -> {}); + retry( + Duration.ofSeconds(20), + () -> { + Replica replica1 = replicaManager.getReplicaOrException(tb); + assertThat(replica1.getIsr()).containsExactlyInAnyOrder(1, 3); + }); + + // mock follower 2 (standby replica) to fetch data from leader. fetch offset is 5, and + // kvAppliedOffset is 5. (which indicate the follower catch up the leader, it will be added + // into isr list and issr list). + replicaManager.fetchLogRecords( + new FetchParams( + 2, (int) conf.get(ConfigOptions.LOG_REPLICA_FETCH_MAX_BYTES).getBytes()), + Collections.singletonMap( + tb, new FetchReqInfo(tb.getTableId(), 5L, Integer.MAX_VALUE, null, 5L)), + result -> {}); + retry( + Duration.ofSeconds(20), + () -> { + Replica replica1 = replicaManager.getReplicaOrException(tb); + assertThat(replica1.getIsr()).containsExactlyInAnyOrder(1, 2, 3); + assertThat(replica1.getIssr()).containsExactlyInAnyOrder(2); + }); + } + + private void makeKvTableAsLeader( + TableBucket tb, List replicas, List isr, List standbyList) { + makeLeaderAndFollower( + Collections.singletonList( + new NotifyLeaderAndIsrData( + DATA1_PHYSICAL_TABLE_PATH_PK, + tb, + replicas, + new LeaderAndIsr.Builder() + .leader(TABLET_SERVER_ID) + .isr(isr) + .standbyReplicas(standbyList) + .build()))); + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/replica/standby/KvStandbyManagerITCase.java b/fluss-server/src/test/java/org/apache/fluss/server/replica/standby/KvStandbyManagerITCase.java new file mode 100644 index 0000000000..d344020b7f --- /dev/null +++ b/fluss-server/src/test/java/org/apache/fluss/server/replica/standby/KvStandbyManagerITCase.java @@ -0,0 +1,165 @@ +/* + * 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.replica.standby; + +import org.apache.fluss.config.ConfigOptions; +import org.apache.fluss.config.Configuration; +import org.apache.fluss.config.MemorySize; +import org.apache.fluss.metadata.TableBucket; +import org.apache.fluss.metadata.TableDescriptor; +import org.apache.fluss.metadata.TablePath; +import org.apache.fluss.record.KvRecordBatch; +import org.apache.fluss.rpc.gateway.TabletServerGateway; +import org.apache.fluss.rpc.messages.PutKvRequest; +import org.apache.fluss.server.kv.KvTablet; +import org.apache.fluss.server.kv.snapshot.ZooKeeperCompletedSnapshotHandleStore; +import org.apache.fluss.server.replica.Replica; +import org.apache.fluss.server.replica.ReplicaManager; +import org.apache.fluss.server.tablet.TabletServer; +import org.apache.fluss.server.testutils.FlussClusterExtension; +import org.apache.fluss.server.zk.ZooKeeperClient; +import org.apache.fluss.utils.types.Tuple2; + +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.List; + +import static org.apache.fluss.record.TestData.DATA1_SCHEMA_PK; +import static org.apache.fluss.server.testutils.RpcMessageTestUtils.createTable; +import static org.apache.fluss.server.testutils.RpcMessageTestUtils.newPutKvRequest; +import static org.apache.fluss.testutils.DataTestUtils.genKvRecordBatch; +import static org.apache.fluss.testutils.common.CommonTestUtils.retry; +import static org.apache.fluss.testutils.common.CommonTestUtils.waitValue; +import static org.assertj.core.api.Assertions.assertThat; + +/** ITCase for {@link KvStandbyManager}. */ +public class KvStandbyManagerITCase { + @RegisterExtension + public static final FlussClusterExtension FLUSS_CLUSTER_EXTENSION = + FlussClusterExtension.builder() + .setNumOfTabletServers(3) + .setClusterConf(initConfig()) + .build(); + + private ZooKeeperCompletedSnapshotHandleStore completedSnapshotHandleStore; + private ZooKeeperClient zkClient; + + @BeforeEach + void beforeEach() { + completedSnapshotHandleStore = + new ZooKeeperCompletedSnapshotHandleStore( + FLUSS_CLUSTER_EXTENSION.getZooKeeperClient()); + zkClient = FLUSS_CLUSTER_EXTENSION.getZooKeeperClient(); + } + + @Test + void testStartStandby() throws Exception { + int bucketId = 0; + TablePath tablePath = TablePath.of("test_db", "test_table_standby"); + TableDescriptor descriptor = + TableDescriptor.builder().schema(DATA1_SCHEMA_PK).distributedBy(1, "a").build(); + long tableId = createTable(FLUSS_CLUSTER_EXTENSION, tablePath, descriptor); + TableBucket tb = new TableBucket(tableId, bucketId); + + int leaderId = FLUSS_CLUSTER_EXTENSION.waitAndGetLeader(tb); + TabletServerGateway leaderGateway = + FLUSS_CLUSTER_EXTENSION.newTabletServerClientForNode(leaderId); + + produceKvRecordsAndWaitRemoteLogCopy(0, leaderGateway, tb, true); + final long snapshot1Id = 0; + waitValue( + () -> completedSnapshotHandleStore.get(tb, snapshot1Id), + Duration.ofMinutes(2), + "Fail to wait for the snapshot 0 for bucket " + tb) + .retrieveCompleteSnapshot(); + + produceKvRecordsAndWaitRemoteLogCopy(1, leaderGateway, tb, false); + + // get a follower. + FLUSS_CLUSTER_EXTENSION.waitUntilTableReady(tableId); + List replicas = + zkClient.getTableAssignment(tableId).get().getBucketAssignment(0).getReplicas(); + Integer follower = replicas.stream().filter(i -> i != leaderId).findFirst().get(); + TabletServer tabletServer = FLUSS_CLUSTER_EXTENSION.getTabletServerById(follower); + ReplicaManager replicaManager = tabletServer.getReplicaManager(); + + Replica replica = replicaManager.getReplicaOrException(tb); + assertThat(replica.getKvTablet()).isNull(); + + // try to start standby. + KvStandbyManager kvStandbyManager = replicaManager.getKvStandbyManager(); + kvStandbyManager.startStandby(replicaManager.getReplicaOrException(tb)); + + // wait until kv become hotStandby. + retry( + Duration.ofMinutes(1), + () -> { + KvTablet kvTablet = replica.getKvTablet(); + assertThat(kvTablet).isNotNull(); + assertThat(kvTablet.getKvAppliedOffset()) + .isEqualTo(replica.getLogTablet().localLogEndOffset()); + assertThat(kvTablet.getFlushedLogOffset()) + .isEqualTo(replica.getLogTablet().localLogEndOffset()); + }); + + kvStandbyManager.cancelBecomeHotStandbyTask(tb); + kvStandbyManager.stopStandby(tb); + } + + private void produceKvRecordsAndWaitRemoteLogCopy( + int keyStart, + TabletServerGateway leaderGateway, + TableBucket tb, + boolean waitRemoteLogCopy) + throws Exception { + for (int i = keyStart * 10; i < keyStart * 10 + 10; i++) { + KvRecordBatch kvRecordBatch = + genKvRecordBatch( + Tuple2.of("k1-" + i, new Object[] {1, "k1-" + i}), + Tuple2.of("k2-" + i, new Object[] {2, "k2-" + i}), + Tuple2.of("k3-" + i, new Object[] {3, "k3-" + i})); + PutKvRequest putKvRequest = + newPutKvRequest(tb.getTableId(), tb.getBucket(), 1, kvRecordBatch); + leaderGateway.putKv(putKvRequest).get(); + } + + if (waitRemoteLogCopy) { + FLUSS_CLUSTER_EXTENSION.waitUntilSomeLogSegmentsCopyToRemote(tb); + } + } + + private static Configuration initConfig() { + Configuration conf = new Configuration(); + conf.setInt(ConfigOptions.DEFAULT_BUCKET_NUMBER, 1); + conf.setInt(ConfigOptions.DEFAULT_REPLICATION_FACTOR, 3); + // set a shorter interval for testing purpose + conf.set(ConfigOptions.REMOTE_LOG_TASK_INTERVAL_DURATION, Duration.ofSeconds(1)); + conf.set(ConfigOptions.LOG_SEGMENT_FILE_SIZE, MemorySize.parse("1kb")); + + // set a shorter interval for test + conf.set(ConfigOptions.KV_SNAPSHOT_INTERVAL, Duration.ofSeconds(10)); + + // set a shorter max log time to allow replica shrink from isr. Don't be too low, otherwise + // normal follower synchronization will also be affected + conf.set(ConfigOptions.LOG_REPLICA_MAX_LAG_TIME, Duration.ofSeconds(5)); + return conf; + } +} diff --git a/fluss-server/src/test/java/org/apache/fluss/server/tablet/TabletServiceITCase.java b/fluss-server/src/test/java/org/apache/fluss/server/tablet/TabletServiceITCase.java index 04f8f3ce38..a452377775 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/tablet/TabletServiceITCase.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/tablet/TabletServiceITCase.java @@ -836,12 +836,14 @@ void testBecomeLeaderOrFollowerWithOneTabletServerOffline() throws Exception { makeNotifyLeaderAndIsrRequest( DATA1_PHYSICAL_TABLE_PATH, tb, - new LeaderAndIsr( - leader, - 1, - originLeaderAndIsr.isr(), - originLeaderAndIsr.coordinatorEpoch(), - originLeaderAndIsr.bucketEpoch()))) + new LeaderAndIsr.Builder() + .leader(leader) + .leaderEpoch(1) + .isr(originLeaderAndIsr.isr()) + .coordinatorEpoch( + originLeaderAndIsr.coordinatorEpoch()) + .bucketEpoch(originLeaderAndIsr.bucketEpoch()) + .build())) .get(); List result = getNotifyLeaderAndIsrResponseData(notifyLeaderAndIsrResponse); @@ -878,12 +880,14 @@ void testBecomeLeaderOrFollowerWithOneTabletServerOffline() throws Exception { makeNotifyLeaderAndIsrRequest( DATA1_PHYSICAL_TABLE_PATH, tb, - new LeaderAndIsr( - leader, - 2, - originLeaderAndIsr.isr(), - originLeaderAndIsr.coordinatorEpoch(), - originLeaderAndIsr.bucketEpoch()))) + new LeaderAndIsr.Builder() + .leader(leader) + .leaderEpoch(2) + .isr(originLeaderAndIsr.isr()) + .coordinatorEpoch( + originLeaderAndIsr.coordinatorEpoch()) + .bucketEpoch(originLeaderAndIsr.bucketEpoch()) + .build())) .get(); result = getNotifyLeaderAndIsrResponseData(notifyLeaderAndIsrResponse); assertThat(result.size()).isEqualTo(1); 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 653d32a385..20d9d1ab07 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 @@ -169,12 +169,12 @@ void testLeaderAndIsr() throws Exception { assertThat(zookeeperClient.getLeaderAndIsr(tableBucket)).isEmpty(); // try to register bucket leaderAndIsr - LeaderAndIsr leaderAndIsr = new LeaderAndIsr(1, 10, Arrays.asList(1, 2, 3), 100, 1000); + LeaderAndIsr leaderAndIsr = leaderAndIsr(1, 10, Arrays.asList(1, 2, 3), 100, 1000); zookeeperClient.registerLeaderAndIsr(tableBucket, leaderAndIsr); assertThat(zookeeperClient.getLeaderAndIsr(tableBucket)).hasValue(leaderAndIsr); // test update - leaderAndIsr = new LeaderAndIsr(2, 20, Collections.emptyList(), 200, 2000); + leaderAndIsr = leaderAndIsr(2, 20, Collections.emptyList(), 200, 2000); zookeeperClient.updateLeaderAndIsr(tableBucket, leaderAndIsr); assertThat(zookeeperClient.getLeaderAndIsr(tableBucket)).hasValue(leaderAndIsr); @@ -192,7 +192,7 @@ void testBatchCreateAndUpdateLeaderAndIsr(boolean isPartitionTable) throws Excep TableBucket tableBucket = isPartitionTable ? new TableBucket(1, 2L, i) : new TableBucket(1, i); LeaderAndIsr leaderAndIsr = - new LeaderAndIsr(i, 10, Arrays.asList(i + 1, i + 2, i + 3), 100, 1000); + leaderAndIsr(i, 10, Arrays.asList(i + 1, i + 2, i + 3), 100, 1000); leaderAndIsrList.add(leaderAndIsr); RegisterTableBucketLeadAndIsrInfo info = isPartitionTable @@ -252,7 +252,7 @@ void testBatchUpdateLeaderAndIsr() throws Exception { for (int i = 0; i < totalCount; i++) { TableBucket tableBucket = new TableBucket(1, i); LeaderAndIsr leaderAndIsr = - new LeaderAndIsr(i, 10, Arrays.asList(i + 1, i + 2, i + 3), 100, 1000); + leaderAndIsr(i, 10, Arrays.asList(i + 1, i + 2, i + 3), 100, 1000); leaderAndIsrList.put(tableBucket, leaderAndIsr); zookeeperClient.registerLeaderAndIsr(tableBucket, leaderAndIsr); } @@ -265,7 +265,7 @@ void testBatchUpdateLeaderAndIsr() throws Exception { Map.Entry::getKey, entry -> { LeaderAndIsr old = entry.getValue(); - return new LeaderAndIsr( + return leaderAndIsr( old.leader() + 1, old.leaderEpoch() + 1, old.isr(), @@ -610,4 +610,15 @@ void testZookeeperConfigPath() throws Exception { .isEqualTo("zookeeper2"); } } + + private LeaderAndIsr leaderAndIsr( + int leader, int leaderEpoch, List isr, int coordinatorEpoch, int bucketEpoch) { + return new LeaderAndIsr.Builder() + .leader(leader) + .leaderEpoch(leaderEpoch) + .isr(isr) + .coordinatorEpoch(coordinatorEpoch) + .bucketEpoch(bucketEpoch) + .build(); + } } diff --git a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/LeaderAndIsrJsonSerdeTest.java b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/LeaderAndIsrJsonSerdeTest.java index 128b14a182..6b2bf5f840 100644 --- a/fluss-server/src/test/java/org/apache/fluss/server/zk/data/LeaderAndIsrJsonSerdeTest.java +++ b/fluss-server/src/test/java/org/apache/fluss/server/zk/data/LeaderAndIsrJsonSerdeTest.java @@ -17,8 +17,14 @@ package org.apache.fluss.server.zk.data; +import org.apache.fluss.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode; +import org.apache.fluss.shaded.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.fluss.utils.json.JsonSerdeTestBase; +import org.junit.jupiter.api.Test; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.Collections; @@ -31,16 +37,64 @@ public class LeaderAndIsrJsonSerdeTest extends JsonSerdeTestBase { @Override protected LeaderAndIsr[] createObjects() { - LeaderAndIsr leaderAndIsr1 = new LeaderAndIsr(1, 10, Arrays.asList(1, 2, 3), 100, 1000); - LeaderAndIsr leaderAndIsr2 = new LeaderAndIsr(2, 20, Collections.emptyList(), 200, 2000); - return new LeaderAndIsr[] {leaderAndIsr1, leaderAndIsr2}; + + LeaderAndIsr leaderAndIsr1 = + new LeaderAndIsr.Builder() + .leader(1) + .leaderEpoch(10) + .isr(Arrays.asList(1, 2, 3)) + .coordinatorEpoch(100) + .bucketEpoch(1000) + .build(); + LeaderAndIsr leaderAndIsr2 = + new LeaderAndIsr.Builder() + .leader(2) + .leaderEpoch(20) + .coordinatorEpoch(200) + .bucketEpoch(2000) + .build(); + LeaderAndIsr leaderAndIsr3 = + new LeaderAndIsr.Builder() + .leader(1) + .leaderEpoch(10) + .isr(Arrays.asList(1, 2, 3)) + .coordinatorEpoch(10) + .bucketEpoch(10) + .standbyReplicas(Collections.singletonList(2)) + .issr(Collections.singletonList(2)) + .build(); + return new LeaderAndIsr[] {leaderAndIsr1, leaderAndIsr2, leaderAndIsr3}; } @Override protected String[] expectedJsons() { return new String[] { - "{\"version\":1,\"leader\":1,\"leader_epoch\":10,\"isr\":[1,2,3],\"coordinator_epoch\":100,\"bucket_epoch\":1000}", - "{\"version\":1,\"leader\":2,\"leader_epoch\":20,\"isr\":[],\"coordinator_epoch\":200,\"bucket_epoch\":2000}" + "{\"version\":2,\"leader\":1,\"leader_epoch\":10,\"isr\":[1,2,3],\"coordinator_epoch\":100,\"bucket_epoch\":1000," + + "\"hot_standby_replicas\":[],\"issr\":[]}", + "{\"version\":2,\"leader\":2,\"leader_epoch\":20,\"isr\":[],\"coordinator_epoch\":200,\"bucket_epoch\":2000," + + "\"hot_standby_replicas\":[],\"issr\":[]}", + "{\"version\":2,\"leader\":1,\"leader_epoch\":10,\"isr\":[1,2,3],\"coordinator_epoch\":10,\"bucket_epoch\":10," + + "\"hot_standby_replicas\":[2],\"issr\":[2]}", }; } + + @Test + void testCompatibility() throws IOException { + // compatibility with version 1 + JsonNode jsonInVersion1 = + new ObjectMapper() + .readTree( + "{\"version\":1,\"leader\":1,\"leader_epoch\":10,\"isr\":[1,2,3],\"coordinator_epoch\":100,\"bucket_epoch\":1000}" + .getBytes(StandardCharsets.UTF_8)); + LeaderAndIsr leaderAndIsr = LeaderAndIsrJsonSerde.INSTANCE.deserialize(jsonInVersion1); + LeaderAndIsr expectedLeaderAndIsr = + new LeaderAndIsr.Builder() + .leader(1) + .leaderEpoch(10) + .isr(Arrays.asList(1, 2, 3)) + .coordinatorEpoch(100) + .bucketEpoch(1000) + .build(); + assertEquals(leaderAndIsr, expectedLeaderAndIsr); + } } From bacdb710b5236d2390dc9e73d670279a1ce32588 Mon Sep 17 00:00:00 2001 From: Liebing Date: Thu, 11 Sep 2025 20:16:55 +0800 Subject: [PATCH 5/5] WIP --- .../apache/fluss/client/admin/FlussAdmin.java | 4 +- .../client/utils/ClientRpcMessageUtils.java | 53 ++++++++++++++ .../rebalance/RebalanceResultForBucket.java | 4 ++ .../rebalance/RebalanceStatusForBucket.java | 4 ++ .../coordinator/CoordinatorService.java | 13 +++- .../server/utils/ServerRpcMessageUtils.java | 71 +++++++++++++++++++ 6 files changed, 147 insertions(+), 2 deletions(-) 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 5e0bc366d1..dc577a848d 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 @@ -66,6 +66,7 @@ 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.PbListOffsetsRespForBucket; @@ -497,7 +498,8 @@ public CompletableFuture> rebalance( @Override public CompletableFuture> listRebalanceProcess() { - throw new UnsupportedOperationException("Support soon"); + return gateway.listRebalanceProcess(new ListRebalanceProcessRequest()) + .thenApply(ClientRpcMessageUtils::toRebalanceProcess); } @Override 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 c2591f5442..033be0dcd7 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 @@ -26,6 +26,8 @@ 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.fs.FsPath; import org.apache.fluss.fs.FsPathAndFileName; import org.apache.fluss.fs.token.ObtainedSecurityToken; @@ -42,6 +44,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.PbKeyValue; @@ -55,6 +58,9 @@ 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; @@ -370,6 +376,53 @@ private static RebalancePlanForBucket toRebalancePlanForBucket( 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(); + rebalanceProcess.put( + new TableBucket(tableId, partitionId, bucketId), + toRebalanceResultForBucket(pbRebalanceProcessForBucket)); + } + } + + for (PbRebalanceProcessForBucket pbRebalanceProcessForBucket : + pbRebalanceProcessForTable.getBucketsProcessesList()) { + int bucketId = pbRebalanceProcessForBucket.getBucketId(); + rebalanceProcess.put( + new TableBucket(tableId, null, bucketId), + toRebalanceResultForBucket(pbRebalanceProcessForBucket)); + } + } + + return rebalanceProcess; + } + + private static RebalanceResultForBucket toRebalanceResultForBucket( + PbRebalanceProcessForBucket pbRebalanceProcessForBucket) { + return new RebalanceResultForBucket( + 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-common/src/main/java/org/apache/fluss/cluster/rebalance/RebalanceResultForBucket.java b/fluss-common/src/main/java/org/apache/fluss/cluster/rebalance/RebalanceResultForBucket.java index 345d5b5d9c..c0b4a07bd5 100644 --- 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 @@ -66,6 +66,10 @@ public List targetReplicas() { return targetReplicas; } + public RebalanceStatusForBucket status() { + return rebalanceStatusForBucket; + } + public RebalanceResultForBucket markFailed() { this.rebalanceStatusForBucket = FAILED; return this; 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 index e8c0e46733..35f833bfd8 100644 --- 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 @@ -37,6 +37,10 @@ public enum RebalanceStatusForBucket { this.code = code; } + public int getCode() { + return code; + } + public static RebalanceStatusForBucket of(int code) { for (RebalanceStatusForBucket status : RebalanceStatusForBucket.values()) { if (status.code == code) { 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 f531e5dbfd..df6701da70 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 @@ -149,6 +149,7 @@ 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.toTablePath; import static org.apache.fluss.server.utils.TableAssignmentUtils.generateAssignment; @@ -663,7 +664,17 @@ public CompletableFuture rebalance(RebalanceRequest request) @Override public CompletableFuture listRebalanceProcess( ListRebalanceProcessRequest request) { - throw new UnsupportedOperationException("Support soon!"); + if (authorizer != null) { + authorizer.authorize(currentSession(), OperationType.DESCRIBE, Resource.cluster()); + } + + AccessContextEvent accessContextEvent = + new AccessContextEvent<>( + ctx -> + makeListRebalanceProcessResponse( + ctx.getOngoingRebalanceTasks(), + ctx.getFinishedRebalanceTasks())); + return accessContextEvent.getResultFuture(); } @Override 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 0576fa995b..fa5165ac99 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 @@ -21,6 +21,7 @@ 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.fs.FsPath; import org.apache.fluss.fs.token.ObtainedSecurityToken; @@ -69,6 +70,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; @@ -112,6 +114,8 @@ 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; @@ -176,6 +180,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; @@ -1721,6 +1726,72 @@ private static PbRebalancePlanForBucket toPbRebalancePlanForBucket( 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.replicas().stream() + .mapToInt(Integer::intValue) + .toArray()) + .setNewReplicas( + rebalanceResultForBucket.targetReplicas().stream() + .mapToInt(Integer::intValue) + .toArray()) + .setRebalanceStatus(rebalanceResultForBucket.status().getCode()); + } + private static Map mergeResponse( Map response, Map errors) { if (errors.isEmpty()) {