xxubai commented on code in PR #3922:
URL: https://github.com/apache/amoro/pull/3922#discussion_r2917204175


##########
amoro-ams/src/main/java/org/apache/amoro/server/AmsAssignService.java:
##########
@@ -0,0 +1,453 @@
+/*
+ * 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.amoro.server;
+
+import org.apache.amoro.client.AmsServerInfo;
+import org.apache.amoro.config.Configurations;
+import org.apache.amoro.server.ha.HighAvailabilityContainer;
+import 
org.apache.amoro.shade.guava32.com.google.common.util.concurrent.ThreadFactoryBuilder;
+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 java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Service for assigning bucket IDs to AMS nodes in master-slave mode. 
Periodically detects node
+ * changes and redistributes bucket IDs evenly.
+ */
+public class AmsAssignService {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(AmsAssignService.class);
+
+  private final ScheduledExecutorService assignScheduler =
+      Executors.newSingleThreadScheduledExecutor(
+          new ThreadFactoryBuilder()
+              .setNameFormat("ams-assign-scheduler-%d")
+              .setDaemon(true)
+              .build());
+
+  private final HighAvailabilityContainer haContainer;
+  private final BucketAssignStore assignStore;
+  private final Configurations serviceConfig;
+  private final int bucketIdTotalCount;
+  private final long nodeOfflineTimeoutMs;
+  private final long assignIntervalSeconds;
+  private volatile boolean running = false;
+
+  boolean isRunning() {
+    return running;
+  }
+
+  void doAssignForTest() {
+    doAssign();
+  }
+
+  public AmsAssignService(HighAvailabilityContainer haContainer, 
Configurations serviceConfig) {
+    this.haContainer = haContainer;
+    this.serviceConfig = serviceConfig;
+    this.bucketIdTotalCount =
+        serviceConfig.getInteger(AmoroManagementConf.HA_BUCKET_ID_TOTAL_COUNT);
+    this.nodeOfflineTimeoutMs =
+        serviceConfig.get(AmoroManagementConf.NODE_OFFLINE_TIMEOUT).toMillis();
+    this.assignIntervalSeconds =
+        serviceConfig.get(AmoroManagementConf.ASSIGN_INTERVAL).getSeconds();
+    this.assignStore = BucketAssignStoreFactory.create(haContainer, 
serviceConfig);
+  }
+
+  /**
+   * Start the assignment service. Only works in master-slave mode and when 
current node is leader.
+   */
+  public void start() {
+    if (!serviceConfig.getBoolean(AmoroManagementConf.USE_MASTER_SLAVE_MODE)) {
+      LOG.info("Master-slave mode is not enabled, skip starting bucket 
assignment service");
+      return;
+    }
+    if (running) {
+      LOG.warn("Bucket assignment service is already running");
+      return;
+    }
+    running = true;
+    assignScheduler.scheduleWithFixedDelay(
+        this::doAssign, 10, assignIntervalSeconds, TimeUnit.SECONDS);
+    LOG.info("Bucket assignment service started with interval: {} seconds", 
assignIntervalSeconds);
+  }
+
+  /** Stop the assignment service. */
+  public void stop() {
+    if (!running) {
+      return;
+    }
+    running = false;
+    assignScheduler.shutdown();
+    try {
+      if (!assignScheduler.awaitTermination(5, TimeUnit.SECONDS)) {
+        assignScheduler.shutdownNow();
+      }
+    } catch (InterruptedException e) {
+      assignScheduler.shutdownNow();
+      Thread.currentThread().interrupt();
+    }
+    LOG.info("Bucket assignment service stopped");
+  }
+
+  private void doAssign() {

Review Comment:
   The doAssign() method is ~120 lines with 7 inline steps (node discovery, 
offline detection, reassignment, unassigned bucket handling, persistence). 
Consider extracting well-named private methods like detectNodeChanges(), 
handleOfflineNodes(), rebalance() to improve readability and testability.



##########
amoro-ams/src/main/java/org/apache/amoro/server/AmsAssignService.java:
##########
@@ -0,0 +1,453 @@
+/*
+ * 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.amoro.server;
+
+import org.apache.amoro.client.AmsServerInfo;
+import org.apache.amoro.config.Configurations;
+import org.apache.amoro.server.ha.HighAvailabilityContainer;
+import 
org.apache.amoro.shade.guava32.com.google.common.util.concurrent.ThreadFactoryBuilder;
+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 java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Service for assigning bucket IDs to AMS nodes in master-slave mode. 
Periodically detects node
+ * changes and redistributes bucket IDs evenly.
+ */
+public class AmsAssignService {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(AmsAssignService.class);
+
+  private final ScheduledExecutorService assignScheduler =
+      Executors.newSingleThreadScheduledExecutor(
+          new ThreadFactoryBuilder()
+              .setNameFormat("ams-assign-scheduler-%d")
+              .setDaemon(true)
+              .build());
+
+  private final HighAvailabilityContainer haContainer;
+  private final BucketAssignStore assignStore;
+  private final Configurations serviceConfig;
+  private final int bucketIdTotalCount;
+  private final long nodeOfflineTimeoutMs;
+  private final long assignIntervalSeconds;
+  private volatile boolean running = false;
+
+  boolean isRunning() {
+    return running;
+  }
+
+  void doAssignForTest() {
+    doAssign();
+  }
+
+  public AmsAssignService(HighAvailabilityContainer haContainer, 
Configurations serviceConfig) {
+    this.haContainer = haContainer;
+    this.serviceConfig = serviceConfig;
+    this.bucketIdTotalCount =
+        serviceConfig.getInteger(AmoroManagementConf.HA_BUCKET_ID_TOTAL_COUNT);
+    this.nodeOfflineTimeoutMs =
+        serviceConfig.get(AmoroManagementConf.NODE_OFFLINE_TIMEOUT).toMillis();
+    this.assignIntervalSeconds =
+        serviceConfig.get(AmoroManagementConf.ASSIGN_INTERVAL).getSeconds();
+    this.assignStore = BucketAssignStoreFactory.create(haContainer, 
serviceConfig);
+  }
+
+  /**
+   * Start the assignment service. Only works in master-slave mode and when 
current node is leader.
+   */
+  public void start() {
+    if (!serviceConfig.getBoolean(AmoroManagementConf.USE_MASTER_SLAVE_MODE)) {
+      LOG.info("Master-slave mode is not enabled, skip starting bucket 
assignment service");
+      return;
+    }
+    if (running) {
+      LOG.warn("Bucket assignment service is already running");
+      return;
+    }
+    running = true;
+    assignScheduler.scheduleWithFixedDelay(
+        this::doAssign, 10, assignIntervalSeconds, TimeUnit.SECONDS);
+    LOG.info("Bucket assignment service started with interval: {} seconds", 
assignIntervalSeconds);
+  }
+
+  /** Stop the assignment service. */
+  public void stop() {
+    if (!running) {
+      return;
+    }
+    running = false;
+    assignScheduler.shutdown();
+    try {
+      if (!assignScheduler.awaitTermination(5, TimeUnit.SECONDS)) {
+        assignScheduler.shutdownNow();
+      }
+    } catch (InterruptedException e) {
+      assignScheduler.shutdownNow();
+      Thread.currentThread().interrupt();
+    }
+    LOG.info("Bucket assignment service stopped");
+  }
+
+  private void doAssign() {
+    try {
+      if (!haContainer.hasLeadership()) {
+        LOG.debug("Current node is not leader, skip bucket assignment");
+        return;
+      }
+
+      List<AmsServerInfo> aliveNodes = haContainer.getAliveNodes();
+      if (aliveNodes.isEmpty()) {
+        LOG.debug("No alive nodes found, skip bucket assignment");
+        return;
+      }
+
+      Map<AmsServerInfo, List<String>> currentAssignments = 
assignStore.getAllAssignments();
+
+      // Create a mapping from stored nodes (may have null restBindPort) to 
alive nodes (complete
+      // info)
+      // Use host:thriftBindPort as the key for matching
+      Map<String, AmsServerInfo> aliveNodeMap = new java.util.HashMap<>();
+      for (AmsServerInfo node : aliveNodes) {
+        String key = getNodeKey(node);
+        aliveNodeMap.put(key, node);
+      }
+
+      // Normalize current assignments: map stored nodes to their 
corresponding alive nodes
+      Map<AmsServerInfo, List<String>> normalizedAssignments = new 
java.util.HashMap<>();

Review Comment:
   Multiple occurrences of new java.util.HashMap<>() appear even though 
java.util.* classes are already imported. Use short names consistently (e.g., 
new HashMap<>()).



##########
amoro-ams/src/main/java/org/apache/amoro/server/BucketAssignStore.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.amoro.server;
+
+import org.apache.amoro.client.AmsServerInfo;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Interface for storing and retrieving bucket ID assignments to AMS nodes. 
Different
+ * implementations can use different storage backends (e.g., ZooKeeper, 
database).
+ */
+public interface BucketAssignStore {
+
+  /**
+   * Save bucket ID assignments for a node.
+   *
+   * @param nodeInfo The node information
+   * @param bucketIds List of bucket IDs assigned to this node
+   * @throws Exception If save operation fails
+   */
+  void saveAssignments(AmsServerInfo nodeInfo, List<String> bucketIds) throws 
Exception;

Review Comment:
   All methods declare throws Exception. This forces callers into overly broad 
catches and makes it hard to distinguish recoverable vs. fatal errors. Consider 
a custom `BucketAssignStoreException` or at least IOException.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to