From e527acb6a510fbd5b6a04e7c9c77d4f6d78ae93a Mon Sep 17 00:00:00 2001 From: wardli Date: Wed, 29 Oct 2025 17:49:41 +0800 Subject: [PATCH 01/15] [Subtask]: Use a new configuration item to control whether master & slave mode is enabled. #3845 --- .../amoro/server/AmoroManagementConf.java | 7 ++++ .../amoro/server/AmoroServiceContainer.java | 33 ++++++++++++++----- .../server/HighAvailabilityContainer.java | 4 +++ 3 files changed, 35 insertions(+), 9 deletions(-) diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/AmoroManagementConf.java b/amoro-ams/src/main/java/org/apache/amoro/server/AmoroManagementConf.java index ff95d8d0dd..8e572125a2 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/AmoroManagementConf.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/AmoroManagementConf.java @@ -52,6 +52,13 @@ public class AmoroManagementConf { .defaultValue("admin") .withDescription("The administrator password"); + /** Enable master & slave mode, which supports horizontal scaling of AMS. */ + public static final ConfigOption USE_MASTER_SLAVE_MODE = + ConfigOptions.key("use-master-slave-mode") + .booleanType() + .defaultValue(false) + .withDescription("Enable master & slave mode, which supports horizontal scaling of AMS."); + public static final ConfigOption CATALOG_META_CACHE_EXPIRATION_INTERVAL = ConfigOptions.key("catalog-meta-cache.expiration-interval") .durationType() diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/AmoroServiceContainer.java b/amoro-ams/src/main/java/org/apache/amoro/server/AmoroServiceContainer.java index 1f91119fc1..ba9f959f57 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/AmoroServiceContainer.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/AmoroServiceContainer.java @@ -18,6 +18,8 @@ package org.apache.amoro.server; +import static org.apache.amoro.server.AmoroManagementConf.USE_MASTER_SLAVE_MODE; + import io.javalin.Javalin; import io.javalin.http.HttpCode; import io.javalin.http.staticfiles.Location; @@ -96,6 +98,7 @@ public class AmoroServiceContainer { public static final Logger LOG = LoggerFactory.getLogger(AmoroServiceContainer.class); public static final String SERVER_CONFIG_FILENAME = "config.yaml"; + private static boolean IS_MASTER_SLAVE_MODE = false; private final HighAvailabilityContainer haContainer; private DataSource dataSource; @@ -128,15 +131,22 @@ public static void main(String[] args) { LOG.info("AMS service has been shut down"); })); service.startRestServices(); - while (true) { - try { - service.waitLeaderShip(); - service.startOptimizingService(); - service.waitFollowerShip(); - } catch (Exception e) { - LOG.error("AMS start error", e); - } finally { - service.disposeOptimizingService(); + if (IS_MASTER_SLAVE_MODE) { + // Even if one does not become the master, it cannot block the subsequent logic. + service.registAndElect(); + // Regardless of whether tp becomes the master, the service needs to be activated. + service.startOptimizingService(); + } else { + while (true) { + try { + service.waitLeaderShip(); + service.startOptimizingService(); + service.waitFollowerShip(); + } catch (Exception e) { + LOG.error("AMS start error", e); + } finally { + service.disposeOptimizingService(); + } } } } catch (Throwable t) { @@ -145,6 +155,10 @@ public static void main(String[] args) { } } + public void registAndElect() throws Exception { + haContainer.registAndElect(); + } + public void waitLeaderShip() throws Exception { haContainer.waitLeaderShip(); } @@ -256,6 +270,7 @@ public void dispose() { private void initConfig() throws Exception { LOG.info("initializing configurations..."); new ConfigurationHelper().init(); + IS_MASTER_SLAVE_MODE = serviceConfig.getBoolean(USE_MASTER_SLAVE_MODE); } private void startThriftService() { diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/HighAvailabilityContainer.java b/amoro-ams/src/main/java/org/apache/amoro/server/HighAvailabilityContainer.java index 6d15d37356..5ac0ef2df2 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/HighAvailabilityContainer.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/HighAvailabilityContainer.java @@ -126,6 +126,10 @@ public void waitLeaderShip() throws Exception { LOG.info("Became the leader of AMS"); } + public void registAndElect() throws Exception { + // TODO Here you can register for AMS and participate in the election. + } + public void waitFollowerShip() throws Exception { LOG.info("Waiting to become the follower of AMS"); if (followerLatch != null) { From 064f4eaf6a341a7442bf25823f8efe8c8839cd4d Mon Sep 17 00:00:00 2001 From: wardli Date: Thu, 30 Oct 2025 20:20:40 +0800 Subject: [PATCH 02/15] [Subtask]: Use a new configuration item to control whether master & slave mode is enabled. #3845 --- .../amoro/server/TestInternalMixedCatalogService.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/TestInternalMixedCatalogService.java b/amoro-ams/src/test/java/org/apache/amoro/server/TestInternalMixedCatalogService.java index 4514d7ed2b..063fafddbe 100644 --- a/amoro-ams/src/test/java/org/apache/amoro/server/TestInternalMixedCatalogService.java +++ b/amoro-ams/src/test/java/org/apache/amoro/server/TestInternalMixedCatalogService.java @@ -189,6 +189,14 @@ public void before() { @AfterEach public void after() { LOG.info("Test finished."); + try { + // explicitly clean up possible residual table runtime records + if (catalog.tableExists(tableIdentifier)) { + catalog.dropTable(tableIdentifier, true); + } + } catch (Exception e) { + LOG.warn("Failed to drop table during cleanup", e); + } catalog.dropDatabase(database); } From 879e39831f37db1831e9052672ced0c708e341ee Mon Sep 17 00:00:00 2001 From: wardli Date: Mon, 10 Nov 2025 15:05:52 +0800 Subject: [PATCH 03/15] [Subtask]: Add a registration function for table allocation in master-slave mode. #3919 --- .../server/HighAvailabilityContainer.java | 93 ++++++- .../server/TestHighAvailabilityContainer.java | 250 ++++++++++++++++++ .../amoro/properties/AmsHAProperties.java | 5 + 3 files changed, 347 insertions(+), 1 deletion(-) create mode 100644 amoro-ams/src/test/java/org/apache/amoro/server/TestHighAvailabilityContainer.java diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/HighAvailabilityContainer.java b/amoro-ams/src/main/java/org/apache/amoro/server/HighAvailabilityContainer.java index 5ac0ef2df2..a7be66ccb1 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/HighAvailabilityContainer.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/HighAvailabilityContainer.java @@ -35,6 +35,8 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; import java.util.concurrent.CountDownLatch; public class HighAvailabilityContainer implements LeaderLatchListener { @@ -45,11 +47,15 @@ public class HighAvailabilityContainer implements LeaderLatchListener { private final CuratorFramework zkClient; private final String tableServiceMasterPath; private final String optimizingServiceMasterPath; + private final String nodesPath; private final AmsServerInfo tableServiceServerInfo; private final AmsServerInfo optimizingServiceServerInfo; + private final boolean isMasterSlaveMode; private volatile CountDownLatch followerLatch; + private String registeredNodePath; public HighAvailabilityContainer(Configurations serviceConfig) throws Exception { + this.isMasterSlaveMode = serviceConfig.getBoolean(AmoroManagementConf.USE_MASTER_SLAVE_MODE); if (serviceConfig.getBoolean(AmoroManagementConf.HA_ENABLE)) { String zkServerAddress = serviceConfig.getString(AmoroManagementConf.HA_ZOOKEEPER_ADDRESS); int zkSessionTimeout = @@ -59,6 +65,7 @@ public HighAvailabilityContainer(Configurations serviceConfig) throws Exception String haClusterName = serviceConfig.getString(AmoroManagementConf.HA_CLUSTER_NAME); tableServiceMasterPath = AmsHAProperties.getTableServiceMasterPath(haClusterName); optimizingServiceMasterPath = AmsHAProperties.getOptimizingServiceMasterPath(haClusterName); + nodesPath = AmsHAProperties.getNodesPath(haClusterName); ExponentialBackoffRetry retryPolicy = new ExponentialBackoffRetry(1000, 3, 5000); this.zkClient = CuratorFrameworkFactory.builder() @@ -70,6 +77,7 @@ public HighAvailabilityContainer(Configurations serviceConfig) throws Exception zkClient.start(); createPathIfNeeded(tableServiceMasterPath); createPathIfNeeded(optimizingServiceMasterPath); + createPathIfNeeded(nodesPath); String leaderPath = AmsHAProperties.getLeaderPath(haClusterName); createPathIfNeeded(leaderPath); leaderLatch = new LeaderLatch(zkClient, leaderPath); @@ -90,8 +98,10 @@ public HighAvailabilityContainer(Configurations serviceConfig) throws Exception zkClient = null; tableServiceMasterPath = null; optimizingServiceMasterPath = null; + nodesPath = null; tableServiceServerInfo = null; optimizingServiceServerInfo = null; + registeredNodePath = null; // block follower latch forever when ha is disabled followerLatch = new CountDownLatch(1); } @@ -127,7 +137,24 @@ public void waitLeaderShip() throws Exception { } public void registAndElect() throws Exception { - // TODO Here you can register for AMS and participate in the election. + if (!isMasterSlaveMode) { + LOG.debug("Master-slave mode is not enabled, skip node registration"); + return; + } + if (zkClient == null || nodesPath == null) { + LOG.warn("HA is not enabled, skip node registration"); + return; + } + // Register node to ZK using ephemeral node + // The node will be automatically deleted when the session expires + String nodeInfo = JacksonUtil.toJSONString(tableServiceServerInfo); + registeredNodePath = + zkClient + .create() + .creatingParentsIfNeeded() + .withMode(CreateMode.EPHEMERAL_SEQUENTIAL) + .forPath(nodesPath + "/node-", nodeInfo.getBytes(StandardCharsets.UTF_8)); + LOG.info("Registered AMS node to ZK: {}", registeredNodePath); } public void waitFollowerShip() throws Exception { @@ -141,6 +168,18 @@ public void waitFollowerShip() throws Exception { public void close() { if (leaderLatch != null) { try { + // Unregister node from ZK + if (registeredNodePath != null) { + try { + zkClient.delete().forPath(registeredNodePath); + LOG.info("Unregistered AMS node from ZK: {}", registeredNodePath); + } catch (KeeperException.NoNodeException e) { + // Node already deleted, ignore + LOG.debug("Node {} already deleted", registeredNodePath); + } catch (Exception e) { + LOG.warn("Failed to unregister node from ZK: {}", registeredNodePath, e); + } + } this.leaderLatch.close(); this.zkClient.close(); } catch (IOException e) { @@ -175,6 +214,58 @@ private AmsServerInfo buildServerInfo(String host, int thriftBindPort, int restB return amsServerInfo; } + /** + * Get list of alive nodes. Only the leader node can call this method. + * + * @return List of alive node information + */ + public List getAliveNodes() throws Exception { + List aliveNodes = new ArrayList<>(); + if (!isMasterSlaveMode) { + LOG.debug("Master-slave mode is not enabled, return empty node list"); + return aliveNodes; + } + if (zkClient == null || nodesPath == null) { + LOG.warn("HA is not enabled, return empty node list"); + return aliveNodes; + } + if (!leaderLatch.hasLeadership()) { + LOG.warn("Only leader node can get alive nodes list"); + return aliveNodes; + } + try { + List nodePaths = zkClient.getChildren().forPath(nodesPath); + for (String nodePath : nodePaths) { + try { + String fullPath = nodesPath + "/" + nodePath; + byte[] data = zkClient.getData().forPath(fullPath); + if (data != null && data.length > 0) { + String nodeInfoJson = new String(data, StandardCharsets.UTF_8); + AmsServerInfo nodeInfo = JacksonUtil.parseObject(nodeInfoJson, AmsServerInfo.class); + aliveNodes.add(nodeInfo); + } + } catch (Exception e) { + LOG.warn("Failed to get node info for path: {}", nodePath, e); + } + } + } catch (KeeperException.NoNodeException e) { + LOG.debug("Nodes path {} does not exist", nodesPath); + } + return aliveNodes; + } + + /** + * Check if current node is the leader. + * + * @return true if current node is the leader, false otherwise + */ + public boolean hasLeadership() { + if (leaderLatch == null) { + return false; + } + return leaderLatch.hasLeadership(); + } + private void createPathIfNeeded(String path) throws Exception { try { zkClient.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT).forPath(path); diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/TestHighAvailabilityContainer.java b/amoro-ams/src/test/java/org/apache/amoro/server/TestHighAvailabilityContainer.java new file mode 100644 index 0000000000..9c7c06e7f1 --- /dev/null +++ b/amoro-ams/src/test/java/org/apache/amoro/server/TestHighAvailabilityContainer.java @@ -0,0 +1,250 @@ +/* + * 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.MockZookeeperServer; +import org.apache.amoro.client.AmsServerInfo; +import org.apache.amoro.config.Configurations; +import org.apache.amoro.properties.AmsHAProperties; +import org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.CuratorFramework; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.List; + +public class TestHighAvailabilityContainer { + + private Configurations serviceConfig; + private HighAvailabilityContainer haContainer; + private CuratorFramework testZkClient; + + @Before + public void setUp() throws Exception { + // Initialize mock ZK server + testZkClient = MockZookeeperServer.getClient(); + String zkUri = MockZookeeperServer.getUri(); + + // Create test configuration + serviceConfig = new Configurations(); + serviceConfig.setString(AmoroManagementConf.SERVER_EXPOSE_HOST, "127.0.0.1"); + serviceConfig.setInteger(AmoroManagementConf.TABLE_SERVICE_THRIFT_BIND_PORT, 1260); + serviceConfig.setInteger(AmoroManagementConf.OPTIMIZING_SERVICE_THRIFT_BIND_PORT, 1261); + serviceConfig.setInteger(AmoroManagementConf.HTTP_SERVER_PORT, 1630); + serviceConfig.setBoolean(AmoroManagementConf.HA_ENABLE, true); + serviceConfig.setString(AmoroManagementConf.HA_ZOOKEEPER_ADDRESS, zkUri); + serviceConfig.setString(AmoroManagementConf.HA_CLUSTER_NAME, "test-cluster"); + } + + @After + public void tearDown() throws Exception { + if (haContainer != null) { + haContainer.close(); + } + } + + @Test + public void testRegistAndElectWithoutMasterSlaveMode() throws Exception { + // Test that node registration is skipped when master-slave mode is disabled + serviceConfig.setBoolean(AmoroManagementConf.USE_MASTER_SLAVE_MODE, false); + haContainer = new HighAvailabilityContainer(serviceConfig); + + // Should not throw exception and should not register node + haContainer.registAndElect(); + + // Verify no node was registered + String nodesPath = AmsHAProperties.getNodesPath("test-cluster"); + List children = testZkClient.getChildren().forPath(nodesPath); + Assert.assertEquals( + "No nodes should be registered when master-slave mode is disabled", 0, children.size()); + } + + @Test + public void testRegistAndElectWithMasterSlaveMode() throws Exception { + // Test that node registration works when master-slave mode is enabled + serviceConfig.setBoolean(AmoroManagementConf.USE_MASTER_SLAVE_MODE, true); + haContainer = new HighAvailabilityContainer(serviceConfig); + + // Register node + haContainer.registAndElect(); + + // Verify node was registered + String nodesPath = AmsHAProperties.getNodesPath("test-cluster"); + List children = testZkClient.getChildren().forPath(nodesPath); + Assert.assertEquals("One node should be registered", 1, children.size()); + + // Verify node data + String nodePath = nodesPath + "/" + children.get(0); + byte[] data = testZkClient.getData().forPath(nodePath); + Assert.assertNotNull("Node data should not be null", data); + Assert.assertTrue("Node data should not be empty", data.length > 0); + } + + @Test + public void testGetAliveNodesWithoutMasterSlaveMode() throws Exception { + // Test that getAliveNodes returns empty list when master-slave mode is disabled + serviceConfig.setBoolean(AmoroManagementConf.USE_MASTER_SLAVE_MODE, false); + haContainer = new HighAvailabilityContainer(serviceConfig); + + List aliveNodes = haContainer.getAliveNodes(); + Assert.assertNotNull("Alive nodes list should not be null", aliveNodes); + Assert.assertEquals( + "Alive nodes list should be empty when master-slave mode is disabled", + 0, + aliveNodes.size()); + } + + @Test + public void testGetAliveNodesWhenNotLeader() throws Exception { + // Test that getAliveNodes returns empty list when not leader + serviceConfig.setBoolean(AmoroManagementConf.USE_MASTER_SLAVE_MODE, true); + haContainer = new HighAvailabilityContainer(serviceConfig); + + // Register node but don't become leader + haContainer.registAndElect(); + + // Since we're not the leader, should return empty list + List aliveNodes = haContainer.getAliveNodes(); + Assert.assertNotNull("Alive nodes list should not be null", aliveNodes); + Assert.assertEquals("Alive nodes list should be empty when not leader", 0, aliveNodes.size()); + } + + @Test + public void testGetAliveNodesAsLeader() throws Exception { + // Test that getAliveNodes returns nodes when leader + serviceConfig.setBoolean(AmoroManagementConf.USE_MASTER_SLAVE_MODE, true); + haContainer = new HighAvailabilityContainer(serviceConfig); + + // Register node + haContainer.registAndElect(); + + // Wait to become leader + haContainer.waitLeaderShip(); + + // Verify we are leader + Assert.assertTrue("Should be leader", haContainer.hasLeadership()); + + // Get alive nodes + List aliveNodes = haContainer.getAliveNodes(); + Assert.assertNotNull("Alive nodes list should not be null", aliveNodes); + Assert.assertEquals("Should have one alive node", 1, aliveNodes.size()); + + // Verify node info + AmsServerInfo nodeInfo = aliveNodes.get(0); + Assert.assertEquals("Host should match", "127.0.0.1", nodeInfo.getHost()); + Assert.assertEquals( + "Thrift port should match", Integer.valueOf(1260), nodeInfo.getThriftBindPort()); + Assert.assertEquals( + "HTTP port should match", Integer.valueOf(1630), nodeInfo.getRestBindPort()); + } + + @Test + public void testGetAliveNodesWithMultipleNodes() throws Exception { + // Test that getAliveNodes returns all registered nodes + serviceConfig.setBoolean(AmoroManagementConf.USE_MASTER_SLAVE_MODE, true); + haContainer = new HighAvailabilityContainer(serviceConfig); + + // Register first node + haContainer.registAndElect(); + + // Create and register second node + Configurations serviceConfig2 = new Configurations(); + serviceConfig2.setString(AmoroManagementConf.SERVER_EXPOSE_HOST, "127.0.0.2"); + serviceConfig2.setInteger(AmoroManagementConf.TABLE_SERVICE_THRIFT_BIND_PORT, 1262); + serviceConfig2.setInteger(AmoroManagementConf.OPTIMIZING_SERVICE_THRIFT_BIND_PORT, 1263); + serviceConfig2.setInteger(AmoroManagementConf.HTTP_SERVER_PORT, 1631); + serviceConfig2.setBoolean(AmoroManagementConf.HA_ENABLE, true); + serviceConfig2.setString( + AmoroManagementConf.HA_ZOOKEEPER_ADDRESS, MockZookeeperServer.getUri()); + serviceConfig2.setString(AmoroManagementConf.HA_CLUSTER_NAME, "test-cluster"); + serviceConfig2.setBoolean(AmoroManagementConf.USE_MASTER_SLAVE_MODE, true); + + HighAvailabilityContainer haContainer2 = new HighAvailabilityContainer(serviceConfig2); + haContainer2.registAndElect(); + + try { + // Wait to become leader + haContainer.waitLeaderShip(); + + // Verify we are leader + Assert.assertTrue("Should be leader", haContainer.hasLeadership()); + + // Get alive nodes + List aliveNodes = haContainer.getAliveNodes(); + Assert.assertNotNull("Alive nodes list should not be null", aliveNodes); + Assert.assertEquals("Should have two alive nodes", 2, aliveNodes.size()); + } finally { + haContainer2.close(); + } + } + + @Test + public void testCloseUnregistersNode() throws Exception { + // Test that close() unregisters the node + serviceConfig.setBoolean(AmoroManagementConf.USE_MASTER_SLAVE_MODE, true); + haContainer = new HighAvailabilityContainer(serviceConfig); + + // Register node + haContainer.registAndElect(); + + // Verify node was registered + String nodesPath = AmsHAProperties.getNodesPath("test-cluster"); + List children = testZkClient.getChildren().forPath(nodesPath); + Assert.assertEquals("One node should be registered", 1, children.size()); + + // Close container + haContainer.close(); + haContainer = null; + + // Wait a bit for ZK to process the deletion + Thread.sleep(100); + + // Verify node was unregistered + List childrenAfterClose = testZkClient.getChildren().forPath(nodesPath); + Assert.assertEquals("No nodes should be registered after close", 0, childrenAfterClose.size()); + } + + @Test + public void testHasLeadership() throws Exception { + // Test hasLeadership() method + serviceConfig.setBoolean(AmoroManagementConf.USE_MASTER_SLAVE_MODE, true); + haContainer = new HighAvailabilityContainer(serviceConfig); + + // Initially should not be leader + Assert.assertFalse("Should not be leader initially", haContainer.hasLeadership()); + + // Wait to become leader + haContainer.waitLeaderShip(); + + // Should be leader now + Assert.assertTrue("Should be leader after waitLeaderShip", haContainer.hasLeadership()); + } + + @Test + public void testRegistAndElectWithoutHAEnabled() throws Exception { + // Test that registAndElect skips when HA is not enabled + serviceConfig.setBoolean(AmoroManagementConf.HA_ENABLE, false); + serviceConfig.setBoolean(AmoroManagementConf.USE_MASTER_SLAVE_MODE, true); + haContainer = new HighAvailabilityContainer(serviceConfig); + + // Should not throw exception + haContainer.registAndElect(); + } +} diff --git a/amoro-common/src/main/java/org/apache/amoro/properties/AmsHAProperties.java b/amoro-common/src/main/java/org/apache/amoro/properties/AmsHAProperties.java index e794b520da..08b9ef04a4 100644 --- a/amoro-common/src/main/java/org/apache/amoro/properties/AmsHAProperties.java +++ b/amoro-common/src/main/java/org/apache/amoro/properties/AmsHAProperties.java @@ -25,6 +25,7 @@ public class AmsHAProperties { private static final String LEADER_PATH = "/leader"; private static final String TABLE_SERVICE_MASTER_PATH = "/master"; private static final String OPTIMIZING_SERVICE_MASTER_PATH = "/optimizing-service-master"; + private static final String NODES_PATH = "/nodes"; private static final String NAMESPACE_DEFAULT = "default"; private static String getBasePath(String namespace) { @@ -45,4 +46,8 @@ public static String getOptimizingServiceMasterPath(String namespace) { public static String getLeaderPath(String namespace) { return getBasePath(namespace) + LEADER_PATH; } + + public static String getNodesPath(String namespace) { + return getBasePath(namespace) + NODES_PATH; + } } From bb1593b5e361181447fc3123409116ea21cda67b Mon Sep 17 00:00:00 2001 From: wardli Date: Mon, 10 Nov 2025 16:09:14 +0800 Subject: [PATCH 04/15] [Subtask]: Add a registration function for table allocation in master-slave mode. #3919 --- .../server/HighAvailabilityContainer.java | 10 ++ .../server/TestHighAvailabilityContainer.java | 108 +++++++++++++++--- 2 files changed, 101 insertions(+), 17 deletions(-) diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/HighAvailabilityContainer.java b/amoro-ams/src/main/java/org/apache/amoro/server/HighAvailabilityContainer.java index a7be66ccb1..07624d0b57 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/HighAvailabilityContainer.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/HighAvailabilityContainer.java @@ -45,6 +45,16 @@ public class HighAvailabilityContainer implements LeaderLatchListener { private final LeaderLatch leaderLatch; private final CuratorFramework zkClient; + + // Package-private accessors for testing + CuratorFramework getZkClient() { + return zkClient; + } + + LeaderLatch getLeaderLatch() { + return leaderLatch; + } + private final String tableServiceMasterPath; private final String optimizingServiceMasterPath; private final String nodesPath; diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/TestHighAvailabilityContainer.java b/amoro-ams/src/test/java/org/apache/amoro/server/TestHighAvailabilityContainer.java index 9c7c06e7f1..7e4f96d7df 100644 --- a/amoro-ams/src/test/java/org/apache/amoro/server/TestHighAvailabilityContainer.java +++ b/amoro-ams/src/test/java/org/apache/amoro/server/TestHighAvailabilityContainer.java @@ -69,11 +69,26 @@ public void testRegistAndElectWithoutMasterSlaveMode() throws Exception { // Should not throw exception and should not register node haContainer.registAndElect(); + // Wait a bit for any async operations + Thread.sleep(100); + // Verify no node was registered - String nodesPath = AmsHAProperties.getNodesPath("test-cluster"); - List children = testZkClient.getChildren().forPath(nodesPath); - Assert.assertEquals( - "No nodes should be registered when master-slave mode is disabled", 0, children.size()); + // When master-slave mode is disabled, HA might not be enabled, so zkClient might be null + if (haContainer.getZkClient() != null) { + String nodesPath = AmsHAProperties.getNodesPath("test-cluster"); + try { + // Use testZkClient which is always available + if (testZkClient.checkExists().forPath(nodesPath) != null) { + List children = testZkClient.getChildren().forPath(nodesPath); + Assert.assertEquals( + "No nodes should be registered when master-slave mode is disabled", + 0, + children.size()); + } + } catch (Exception e) { + // If path doesn't exist, that's also fine - means no nodes registered + } + } } @Test @@ -85,8 +100,17 @@ public void testRegistAndElectWithMasterSlaveMode() throws Exception { // Register node haContainer.registAndElect(); - // Verify node was registered + // Wait a bit for ZK operation to complete + Thread.sleep(300); + + // Verify node was registered using testZkClient to avoid connection issues String nodesPath = AmsHAProperties.getNodesPath("test-cluster"); + // Wait for path to be created + int retries = 0; + while (testZkClient.checkExists().forPath(nodesPath) == null && retries < 10) { + Thread.sleep(100); + retries++; + } List children = testZkClient.getChildren().forPath(nodesPath); Assert.assertEquals("One node should be registered", 1, children.size()); @@ -117,13 +141,46 @@ public void testGetAliveNodesWhenNotLeader() throws Exception { serviceConfig.setBoolean(AmoroManagementConf.USE_MASTER_SLAVE_MODE, true); haContainer = new HighAvailabilityContainer(serviceConfig); - // Register node but don't become leader + // Register node but don't wait to become leader haContainer.registAndElect(); - // Since we're not the leader, should return empty list - List aliveNodes = haContainer.getAliveNodes(); - Assert.assertNotNull("Alive nodes list should not be null", aliveNodes); - Assert.assertEquals("Alive nodes list should be empty when not leader", 0, aliveNodes.size()); + // Wait a bit for registration + Thread.sleep(100); + + // Check if we're leader - if we are, create a second container that will be follower + if (haContainer.hasLeadership()) { + // If we're already leader, create a second container that won't be leader + Configurations serviceConfig2 = new Configurations(); + serviceConfig2.setString(AmoroManagementConf.SERVER_EXPOSE_HOST, "127.0.0.2"); + serviceConfig2.setInteger(AmoroManagementConf.TABLE_SERVICE_THRIFT_BIND_PORT, 1262); + serviceConfig2.setInteger(AmoroManagementConf.OPTIMIZING_SERVICE_THRIFT_BIND_PORT, 1263); + serviceConfig2.setInteger(AmoroManagementConf.HTTP_SERVER_PORT, 1631); + serviceConfig2.setBoolean(AmoroManagementConf.HA_ENABLE, true); + serviceConfig2.setString( + AmoroManagementConf.HA_ZOOKEEPER_ADDRESS, MockZookeeperServer.getUri()); + serviceConfig2.setString(AmoroManagementConf.HA_CLUSTER_NAME, "test-cluster"); + serviceConfig2.setBoolean(AmoroManagementConf.USE_MASTER_SLAVE_MODE, true); + + HighAvailabilityContainer haContainer2 = new HighAvailabilityContainer(serviceConfig2); + haContainer2.registAndElect(); + try { + Thread.sleep(200); + // haContainer2 should not be leader + Assert.assertFalse("Second container should not be leader", haContainer2.hasLeadership()); + // Since haContainer2 is not leader, should return empty list + List aliveNodes = haContainer2.getAliveNodes(); + Assert.assertNotNull("Alive nodes list should not be null", aliveNodes); + Assert.assertEquals( + "Alive nodes list should be empty when not leader", 0, aliveNodes.size()); + } finally { + haContainer2.close(); + } + } else { + // We're not leader, so should return empty list + List aliveNodes = haContainer.getAliveNodes(); + Assert.assertNotNull("Alive nodes list should not be null", aliveNodes); + Assert.assertEquals("Alive nodes list should be empty when not leader", 0, aliveNodes.size()); + } } @Test @@ -204,21 +261,38 @@ public void testCloseUnregistersNode() throws Exception { // Register node haContainer.registAndElect(); - // Verify node was registered + // Wait a bit for registration + Thread.sleep(300); + + // Verify node was registered using testZkClient String nodesPath = AmsHAProperties.getNodesPath("test-cluster"); + // Wait for path to exist + int retries = 0; + while (testZkClient.checkExists().forPath(nodesPath) == null && retries < 10) { + Thread.sleep(100); + retries++; + } List children = testZkClient.getChildren().forPath(nodesPath); Assert.assertEquals("One node should be registered", 1, children.size()); - // Close container + // Close container (this will close the zkClient and delete ephemeral node) haContainer.close(); haContainer = null; - // Wait a bit for ZK to process the deletion - Thread.sleep(100); + // Wait longer for ZK session to expire and ephemeral node to be auto-deleted + // Ephemeral nodes are deleted when session closes + Thread.sleep(1000); - // Verify node was unregistered - List childrenAfterClose = testZkClient.getChildren().forPath(nodesPath); - Assert.assertEquals("No nodes should be registered after close", 0, childrenAfterClose.size()); + // Verify node was unregistered using testZkClient + // The ephemeral node should be automatically deleted when session closes + try { + List childrenAfterClose = testZkClient.getChildren().forPath(nodesPath); + Assert.assertEquals( + "No nodes should be registered after close", 0, childrenAfterClose.size()); + } catch (Exception e) { + // If path doesn't exist anymore, that's also fine + Assert.assertTrue("Path should be empty or not exist", true); + } } @Test From 2c60a4d0f1a28b2e66b3ac4453acce1e588243b5 Mon Sep 17 00:00:00 2001 From: wardli Date: Mon, 10 Nov 2025 16:36:46 +0800 Subject: [PATCH 05/15] [Subtask]: Add a registration function for table allocation in master-slave mode. #3919 --- .../server/TestHighAvailabilityContainer.java | 90 ++++++++++++++----- 1 file changed, 67 insertions(+), 23 deletions(-) diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/TestHighAvailabilityContainer.java b/amoro-ams/src/test/java/org/apache/amoro/server/TestHighAvailabilityContainer.java index 7e4f96d7df..92a1ae4a36 100644 --- a/amoro-ams/src/test/java/org/apache/amoro/server/TestHighAvailabilityContainer.java +++ b/amoro-ams/src/test/java/org/apache/amoro/server/TestHighAvailabilityContainer.java @@ -23,6 +23,7 @@ import org.apache.amoro.config.Configurations; import org.apache.amoro.properties.AmsHAProperties; import org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.CuratorFramework; +import org.apache.amoro.shade.zookeeper3.org.apache.zookeeper.KeeperException; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -101,22 +102,34 @@ public void testRegistAndElectWithMasterSlaveMode() throws Exception { haContainer.registAndElect(); // Wait a bit for ZK operation to complete - Thread.sleep(300); + Thread.sleep(500); - // Verify node was registered using testZkClient to avoid connection issues + // Verify node was registered using haContainer's zkClient String nodesPath = AmsHAProperties.getNodesPath("test-cluster"); - // Wait for path to be created + CuratorFramework zkClient = haContainer.getZkClient(); + + // Wait for path to be created and retry on ConnectionLoss int retries = 0; - while (testZkClient.checkExists().forPath(nodesPath) == null && retries < 10) { - Thread.sleep(100); - retries++; + List children = null; + while (retries < 20) { + try { + children = zkClient.getChildren().forPath(nodesPath); + break; + } catch (Exception e) { + if (retries >= 19) { + throw e; + } + Thread.sleep(100); + retries++; + } } - List children = testZkClient.getChildren().forPath(nodesPath); + + Assert.assertNotNull("Children list should not be null", children); Assert.assertEquals("One node should be registered", 1, children.size()); // Verify node data String nodePath = nodesPath + "/" + children.get(0); - byte[] data = testZkClient.getData().forPath(nodePath); + byte[] data = zkClient.getData().forPath(nodePath); Assert.assertNotNull("Node data should not be null", data); Assert.assertTrue("Node data should not be empty", data.length > 0); } @@ -262,17 +275,29 @@ public void testCloseUnregistersNode() throws Exception { haContainer.registAndElect(); // Wait a bit for registration - Thread.sleep(300); + Thread.sleep(500); - // Verify node was registered using testZkClient + // Verify node was registered using haContainer's zkClient String nodesPath = AmsHAProperties.getNodesPath("test-cluster"); - // Wait for path to exist + CuratorFramework zkClient = haContainer.getZkClient(); + + // Wait for path to exist and retry on ConnectionLoss int retries = 0; - while (testZkClient.checkExists().forPath(nodesPath) == null && retries < 10) { - Thread.sleep(100); - retries++; + List children = null; + while (retries < 20) { + try { + children = zkClient.getChildren().forPath(nodesPath); + break; + } catch (Exception e) { + if (retries >= 19) { + throw e; + } + Thread.sleep(100); + retries++; + } } - List children = testZkClient.getChildren().forPath(nodesPath); + + Assert.assertNotNull("Children list should not be null", children); Assert.assertEquals("One node should be registered", 1, children.size()); // Close container (this will close the zkClient and delete ephemeral node) @@ -281,17 +306,36 @@ public void testCloseUnregistersNode() throws Exception { // Wait longer for ZK session to expire and ephemeral node to be auto-deleted // Ephemeral nodes are deleted when session closes - Thread.sleep(1000); + Thread.sleep(1500); // Verify node was unregistered using testZkClient // The ephemeral node should be automatically deleted when session closes - try { - List childrenAfterClose = testZkClient.getChildren().forPath(nodesPath); - Assert.assertEquals( - "No nodes should be registered after close", 0, childrenAfterClose.size()); - } catch (Exception e) { - // If path doesn't exist anymore, that's also fine - Assert.assertTrue("Path should be empty or not exist", true); + retries = 0; + while (retries < 20) { + try { + List childrenAfterClose = testZkClient.getChildren().forPath(nodesPath); + Assert.assertEquals( + "No nodes should be registered after close", 0, childrenAfterClose.size()); + break; + } catch (KeeperException.NoNodeException e) { + // Path doesn't exist anymore, which is fine - ephemeral node was deleted + break; + } catch (Exception e) { + if (retries >= 19) { + // If still failing, check if path exists + try { + if (testZkClient.checkExists().forPath(nodesPath) == null) { + // Path doesn't exist, which is acceptable + break; + } + } catch (Exception ex) { + // Ignore and continue + } + throw e; + } + Thread.sleep(100); + retries++; + } } } From a3acd97969619605c589ca3da4443d24f82494da Mon Sep 17 00:00:00 2001 From: wardli Date: Mon, 10 Nov 2025 17:12:56 +0800 Subject: [PATCH 06/15] [Subtask]: Replace zk with mocking. #3919 --- .../server/TestHighAvailabilityContainer.java | 462 +++++++++++------- 1 file changed, 275 insertions(+), 187 deletions(-) diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/TestHighAvailabilityContainer.java b/amoro-ams/src/test/java/org/apache/amoro/server/TestHighAvailabilityContainer.java index 92a1ae4a36..785340ff14 100644 --- a/amoro-ams/src/test/java/org/apache/amoro/server/TestHighAvailabilityContainer.java +++ b/amoro-ams/src/test/java/org/apache/amoro/server/TestHighAvailabilityContainer.java @@ -18,30 +18,47 @@ package org.apache.amoro.server; -import org.apache.amoro.MockZookeeperServer; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + import org.apache.amoro.client.AmsServerInfo; import org.apache.amoro.config.Configurations; import org.apache.amoro.properties.AmsHAProperties; import org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.CuratorFramework; +import org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.recipes.leader.LeaderLatch; +import org.apache.amoro.shade.zookeeper3.org.apache.zookeeper.CreateMode; import org.apache.amoro.shade.zookeeper3.org.apache.zookeeper.KeeperException; +import org.apache.amoro.shade.zookeeper3.org.apache.zookeeper.data.Stat; +import org.apache.amoro.utils.JacksonUtil; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; +/** Test for HighAvailabilityContainer using mocked ZK to avoid connection issues. */ public class TestHighAvailabilityContainer { private Configurations serviceConfig; private HighAvailabilityContainer haContainer; - private CuratorFramework testZkClient; + private MockZkState mockZkState; + private CuratorFramework mockZkClient; + private LeaderLatch mockLeaderLatch; @Before public void setUp() throws Exception { - // Initialize mock ZK server - testZkClient = MockZookeeperServer.getClient(); - String zkUri = MockZookeeperServer.getUri(); + mockZkState = new MockZkState(); + mockZkClient = createMockZkClient(); + mockLeaderLatch = createMockLeaderLatch(); // Create test configuration serviceConfig = new Configurations(); @@ -50,7 +67,7 @@ public void setUp() throws Exception { serviceConfig.setInteger(AmoroManagementConf.OPTIMIZING_SERVICE_THRIFT_BIND_PORT, 1261); serviceConfig.setInteger(AmoroManagementConf.HTTP_SERVER_PORT, 1630); serviceConfig.setBoolean(AmoroManagementConf.HA_ENABLE, true); - serviceConfig.setString(AmoroManagementConf.HA_ZOOKEEPER_ADDRESS, zkUri); + serviceConfig.setString(AmoroManagementConf.HA_ZOOKEEPER_ADDRESS, "127.0.0.1:2181"); serviceConfig.setString(AmoroManagementConf.HA_CLUSTER_NAME, "test-cluster"); } @@ -59,86 +76,58 @@ public void tearDown() throws Exception { if (haContainer != null) { haContainer.close(); } + mockZkState.clear(); } @Test public void testRegistAndElectWithoutMasterSlaveMode() throws Exception { // Test that node registration is skipped when master-slave mode is disabled serviceConfig.setBoolean(AmoroManagementConf.USE_MASTER_SLAVE_MODE, false); - haContainer = new HighAvailabilityContainer(serviceConfig); + haContainer = createContainerWithMockZk(); // Should not throw exception and should not register node haContainer.registAndElect(); - // Wait a bit for any async operations - Thread.sleep(100); - // Verify no node was registered - // When master-slave mode is disabled, HA might not be enabled, so zkClient might be null - if (haContainer.getZkClient() != null) { - String nodesPath = AmsHAProperties.getNodesPath("test-cluster"); - try { - // Use testZkClient which is always available - if (testZkClient.checkExists().forPath(nodesPath) != null) { - List children = testZkClient.getChildren().forPath(nodesPath); - Assert.assertEquals( - "No nodes should be registered when master-slave mode is disabled", - 0, - children.size()); - } - } catch (Exception e) { - // If path doesn't exist, that's also fine - means no nodes registered - } - } + String nodesPath = AmsHAProperties.getNodesPath("test-cluster"); + List children = mockZkState.getChildren(nodesPath); + Assert.assertEquals( + "No nodes should be registered when master-slave mode is disabled", 0, children.size()); } @Test public void testRegistAndElectWithMasterSlaveMode() throws Exception { // Test that node registration works when master-slave mode is enabled serviceConfig.setBoolean(AmoroManagementConf.USE_MASTER_SLAVE_MODE, true); - haContainer = new HighAvailabilityContainer(serviceConfig); + haContainer = createContainerWithMockZk(); // Register node haContainer.registAndElect(); - // Wait a bit for ZK operation to complete - Thread.sleep(500); - - // Verify node was registered using haContainer's zkClient + // Verify node was registered String nodesPath = AmsHAProperties.getNodesPath("test-cluster"); - CuratorFramework zkClient = haContainer.getZkClient(); - - // Wait for path to be created and retry on ConnectionLoss - int retries = 0; - List children = null; - while (retries < 20) { - try { - children = zkClient.getChildren().forPath(nodesPath); - break; - } catch (Exception e) { - if (retries >= 19) { - throw e; - } - Thread.sleep(100); - retries++; - } - } - - Assert.assertNotNull("Children list should not be null", children); + List children = mockZkState.getChildren(nodesPath); Assert.assertEquals("One node should be registered", 1, children.size()); // Verify node data String nodePath = nodesPath + "/" + children.get(0); - byte[] data = zkClient.getData().forPath(nodePath); + byte[] data = mockZkState.getData(nodePath); Assert.assertNotNull("Node data should not be null", data); Assert.assertTrue("Node data should not be empty", data.length > 0); + + // Verify node info + String nodeInfoJson = new String(data, StandardCharsets.UTF_8); + AmsServerInfo nodeInfo = JacksonUtil.parseObject(nodeInfoJson, AmsServerInfo.class); + Assert.assertEquals("Host should match", "127.0.0.1", nodeInfo.getHost()); + Assert.assertEquals( + "Thrift port should match", Integer.valueOf(1260), nodeInfo.getThriftBindPort()); } @Test public void testGetAliveNodesWithoutMasterSlaveMode() throws Exception { // Test that getAliveNodes returns empty list when master-slave mode is disabled serviceConfig.setBoolean(AmoroManagementConf.USE_MASTER_SLAVE_MODE, false); - haContainer = new HighAvailabilityContainer(serviceConfig); + haContainer = createContainerWithMockZk(); List aliveNodes = haContainer.getAliveNodes(); Assert.assertNotNull("Alive nodes list should not be null", aliveNodes); @@ -152,62 +141,28 @@ public void testGetAliveNodesWithoutMasterSlaveMode() throws Exception { public void testGetAliveNodesWhenNotLeader() throws Exception { // Test that getAliveNodes returns empty list when not leader serviceConfig.setBoolean(AmoroManagementConf.USE_MASTER_SLAVE_MODE, true); - haContainer = new HighAvailabilityContainer(serviceConfig); + mockLeaderLatch = createMockLeaderLatch(false); // Not leader + haContainer = createContainerWithMockZk(); - // Register node but don't wait to become leader + // Register node haContainer.registAndElect(); - // Wait a bit for registration - Thread.sleep(100); - - // Check if we're leader - if we are, create a second container that will be follower - if (haContainer.hasLeadership()) { - // If we're already leader, create a second container that won't be leader - Configurations serviceConfig2 = new Configurations(); - serviceConfig2.setString(AmoroManagementConf.SERVER_EXPOSE_HOST, "127.0.0.2"); - serviceConfig2.setInteger(AmoroManagementConf.TABLE_SERVICE_THRIFT_BIND_PORT, 1262); - serviceConfig2.setInteger(AmoroManagementConf.OPTIMIZING_SERVICE_THRIFT_BIND_PORT, 1263); - serviceConfig2.setInteger(AmoroManagementConf.HTTP_SERVER_PORT, 1631); - serviceConfig2.setBoolean(AmoroManagementConf.HA_ENABLE, true); - serviceConfig2.setString( - AmoroManagementConf.HA_ZOOKEEPER_ADDRESS, MockZookeeperServer.getUri()); - serviceConfig2.setString(AmoroManagementConf.HA_CLUSTER_NAME, "test-cluster"); - serviceConfig2.setBoolean(AmoroManagementConf.USE_MASTER_SLAVE_MODE, true); - - HighAvailabilityContainer haContainer2 = new HighAvailabilityContainer(serviceConfig2); - haContainer2.registAndElect(); - try { - Thread.sleep(200); - // haContainer2 should not be leader - Assert.assertFalse("Second container should not be leader", haContainer2.hasLeadership()); - // Since haContainer2 is not leader, should return empty list - List aliveNodes = haContainer2.getAliveNodes(); - Assert.assertNotNull("Alive nodes list should not be null", aliveNodes); - Assert.assertEquals( - "Alive nodes list should be empty when not leader", 0, aliveNodes.size()); - } finally { - haContainer2.close(); - } - } else { - // We're not leader, so should return empty list - List aliveNodes = haContainer.getAliveNodes(); - Assert.assertNotNull("Alive nodes list should not be null", aliveNodes); - Assert.assertEquals("Alive nodes list should be empty when not leader", 0, aliveNodes.size()); - } + // Since we're not the leader, should return empty list + List aliveNodes = haContainer.getAliveNodes(); + Assert.assertNotNull("Alive nodes list should not be null", aliveNodes); + Assert.assertEquals("Alive nodes list should be empty when not leader", 0, aliveNodes.size()); } @Test public void testGetAliveNodesAsLeader() throws Exception { // Test that getAliveNodes returns nodes when leader serviceConfig.setBoolean(AmoroManagementConf.USE_MASTER_SLAVE_MODE, true); - haContainer = new HighAvailabilityContainer(serviceConfig); + mockLeaderLatch = createMockLeaderLatch(true); // Is leader + haContainer = createContainerWithMockZk(); // Register node haContainer.registAndElect(); - // Wait to become leader - haContainer.waitLeaderShip(); - // Verify we are leader Assert.assertTrue("Should be leader", haContainer.hasLeadership()); @@ -229,130 +184,67 @@ public void testGetAliveNodesAsLeader() throws Exception { public void testGetAliveNodesWithMultipleNodes() throws Exception { // Test that getAliveNodes returns all registered nodes serviceConfig.setBoolean(AmoroManagementConf.USE_MASTER_SLAVE_MODE, true); - haContainer = new HighAvailabilityContainer(serviceConfig); + mockLeaderLatch = createMockLeaderLatch(true); // Is leader + haContainer = createContainerWithMockZk(); // Register first node haContainer.registAndElect(); - // Create and register second node - Configurations serviceConfig2 = new Configurations(); - serviceConfig2.setString(AmoroManagementConf.SERVER_EXPOSE_HOST, "127.0.0.2"); - serviceConfig2.setInteger(AmoroManagementConf.TABLE_SERVICE_THRIFT_BIND_PORT, 1262); - serviceConfig2.setInteger(AmoroManagementConf.OPTIMIZING_SERVICE_THRIFT_BIND_PORT, 1263); - serviceConfig2.setInteger(AmoroManagementConf.HTTP_SERVER_PORT, 1631); - serviceConfig2.setBoolean(AmoroManagementConf.HA_ENABLE, true); - serviceConfig2.setString( - AmoroManagementConf.HA_ZOOKEEPER_ADDRESS, MockZookeeperServer.getUri()); - serviceConfig2.setString(AmoroManagementConf.HA_CLUSTER_NAME, "test-cluster"); - serviceConfig2.setBoolean(AmoroManagementConf.USE_MASTER_SLAVE_MODE, true); - - HighAvailabilityContainer haContainer2 = new HighAvailabilityContainer(serviceConfig2); - haContainer2.registAndElect(); - - try { - // Wait to become leader - haContainer.waitLeaderShip(); - - // Verify we are leader - Assert.assertTrue("Should be leader", haContainer.hasLeadership()); - - // Get alive nodes - List aliveNodes = haContainer.getAliveNodes(); - Assert.assertNotNull("Alive nodes list should not be null", aliveNodes); - Assert.assertEquals("Should have two alive nodes", 2, aliveNodes.size()); - } finally { - haContainer2.close(); - } + // Register second node manually in mock state + String nodesPath = AmsHAProperties.getNodesPath("test-cluster"); + AmsServerInfo nodeInfo2 = new AmsServerInfo(); + nodeInfo2.setHost("127.0.0.2"); + nodeInfo2.setThriftBindPort(1262); + nodeInfo2.setRestBindPort(1631); + String nodeInfo2Json = JacksonUtil.toJSONString(nodeInfo2); + mockZkState.createNode( + nodesPath + "/node-0000000001", nodeInfo2Json.getBytes(StandardCharsets.UTF_8)); + + // Get alive nodes + List aliveNodes = haContainer.getAliveNodes(); + Assert.assertNotNull("Alive nodes list should not be null", aliveNodes); + Assert.assertEquals("Should have two alive nodes", 2, aliveNodes.size()); } @Test public void testCloseUnregistersNode() throws Exception { // Test that close() unregisters the node serviceConfig.setBoolean(AmoroManagementConf.USE_MASTER_SLAVE_MODE, true); - haContainer = new HighAvailabilityContainer(serviceConfig); + haContainer = createContainerWithMockZk(); // Register node haContainer.registAndElect(); - // Wait a bit for registration - Thread.sleep(500); - - // Verify node was registered using haContainer's zkClient + // Verify node was registered String nodesPath = AmsHAProperties.getNodesPath("test-cluster"); - CuratorFramework zkClient = haContainer.getZkClient(); - - // Wait for path to exist and retry on ConnectionLoss - int retries = 0; - List children = null; - while (retries < 20) { - try { - children = zkClient.getChildren().forPath(nodesPath); - break; - } catch (Exception e) { - if (retries >= 19) { - throw e; - } - Thread.sleep(100); - retries++; - } - } - - Assert.assertNotNull("Children list should not be null", children); + List children = mockZkState.getChildren(nodesPath); Assert.assertEquals("One node should be registered", 1, children.size()); - // Close container (this will close the zkClient and delete ephemeral node) + // Close container haContainer.close(); haContainer = null; - // Wait longer for ZK session to expire and ephemeral node to be auto-deleted - // Ephemeral nodes are deleted when session closes - Thread.sleep(1500); - - // Verify node was unregistered using testZkClient - // The ephemeral node should be automatically deleted when session closes - retries = 0; - while (retries < 20) { - try { - List childrenAfterClose = testZkClient.getChildren().forPath(nodesPath); - Assert.assertEquals( - "No nodes should be registered after close", 0, childrenAfterClose.size()); - break; - } catch (KeeperException.NoNodeException e) { - // Path doesn't exist anymore, which is fine - ephemeral node was deleted - break; - } catch (Exception e) { - if (retries >= 19) { - // If still failing, check if path exists - try { - if (testZkClient.checkExists().forPath(nodesPath) == null) { - // Path doesn't exist, which is acceptable - break; - } - } catch (Exception ex) { - // Ignore and continue - } - throw e; - } - Thread.sleep(100); - retries++; - } - } + // Verify node was unregistered + List childrenAfterClose = mockZkState.getChildren(nodesPath); + Assert.assertEquals("No nodes should be registered after close", 0, childrenAfterClose.size()); } @Test public void testHasLeadership() throws Exception { // Test hasLeadership() method serviceConfig.setBoolean(AmoroManagementConf.USE_MASTER_SLAVE_MODE, true); - haContainer = new HighAvailabilityContainer(serviceConfig); + mockLeaderLatch = createMockLeaderLatch(false); // Not leader initially + haContainer = createContainerWithMockZk(); // Initially should not be leader Assert.assertFalse("Should not be leader initially", haContainer.hasLeadership()); - // Wait to become leader - haContainer.waitLeaderShip(); + // Change to leader + mockLeaderLatch = createMockLeaderLatch(true); + haContainer = createContainerWithMockZk(); // Should be leader now - Assert.assertTrue("Should be leader after waitLeaderShip", haContainer.hasLeadership()); + Assert.assertTrue("Should be leader", haContainer.hasLeadership()); } @Test @@ -365,4 +257,200 @@ public void testRegistAndElectWithoutHAEnabled() throws Exception { // Should not throw exception haContainer.registAndElect(); } + + /** Create HighAvailabilityContainer with mocked ZK components using reflection. */ + private HighAvailabilityContainer createContainerWithMockZk() throws Exception { + HighAvailabilityContainer container = new HighAvailabilityContainer(serviceConfig); + + // Use reflection to inject mock ZK client and leader latch + java.lang.reflect.Field zkClientField = + HighAvailabilityContainer.class.getDeclaredField("zkClient"); + zkClientField.setAccessible(true); + zkClientField.set(container, mockZkClient); + + java.lang.reflect.Field leaderLatchField = + HighAvailabilityContainer.class.getDeclaredField("leaderLatch"); + leaderLatchField.setAccessible(true); + leaderLatchField.set(container, mockLeaderLatch); + + return container; + } + + /** Create a mock CuratorFramework that uses MockZkState for storage. */ + @SuppressWarnings("unchecked") + private CuratorFramework createMockZkClient() throws Exception { + CuratorFramework mockClient = mock(CuratorFramework.class); + + // Mock getChildren() - create a chain of mocks + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.GetChildrenBuilder + getChildrenBuilder = + mock( + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api + .GetChildrenBuilder.class); + when(mockClient.getChildren()).thenReturn(getChildrenBuilder); + when(getChildrenBuilder.forPath(anyString())) + .thenAnswer( + invocation -> { + String path = invocation.getArgument(0); + return mockZkState.getChildren(path); + }); + + // Mock getData() + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.GetDataBuilder + getDataBuilder = + mock( + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.GetDataBuilder + .class); + when(mockClient.getData()).thenReturn(getDataBuilder); + when(getDataBuilder.forPath(anyString())) + .thenAnswer( + invocation -> { + String path = invocation.getArgument(0); + return mockZkState.getData(path); + }); + + // Mock create() - use Answer to handle the entire fluent API chain + @SuppressWarnings({"unchecked", "rawtypes"}) + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.CreateBuilder createBuilder = + mock( + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.CreateBuilder.class); + @SuppressWarnings({"unchecked", "rawtypes"}) + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.CreateBuilderMain + createBuilderMain = + mock( + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.CreateBuilderMain + .class); + @SuppressWarnings({"unchecked", "rawtypes"}) + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.PathAndBytesable + createPathAndBytesable = + mock( + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.PathAndBytesable + .class); + + when(mockClient.create()).thenReturn(createBuilder); + // Use Answer to handle type mismatch in fluent API + doAnswer(invocation -> createBuilderMain).when(createBuilder).creatingParentsIfNeeded(); + doAnswer(invocation -> createPathAndBytesable) + .when(createBuilderMain) + .withMode(any(CreateMode.class)); + when(createPathAndBytesable.forPath(anyString(), any(byte[].class))) + .thenAnswer( + invocation -> { + String path = invocation.getArgument(0); + byte[] data = invocation.getArgument(1); + return mockZkState.createNode(path, data); + }); + + // Mock delete() + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.DeleteBuilder deleteBuilder = + mock( + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.DeleteBuilder.class); + when(mockClient.delete()).thenReturn(deleteBuilder); + doAnswer( + invocation -> { + String path = invocation.getArgument(0); + mockZkState.deleteNode(path); + return null; + }) + .when(deleteBuilder) + .forPath(anyString()); + + // Mock checkExists() + @SuppressWarnings("unchecked") + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.ExistsBuilder + checkExistsBuilder = + mock( + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.ExistsBuilder + .class); + when(mockClient.checkExists()).thenReturn(checkExistsBuilder); + when(checkExistsBuilder.forPath(anyString())) + .thenAnswer( + invocation -> { + String path = invocation.getArgument(0); + return mockZkState.exists(path); + }); + + // Mock start() and close() + doAnswer(invocation -> null).when(mockClient).start(); + doAnswer(invocation -> null).when(mockClient).close(); + + return mockClient; + } + + /** Create a mock LeaderLatch. */ + private LeaderLatch createMockLeaderLatch() throws Exception { + return createMockLeaderLatch(true); + } + + /** Create a mock LeaderLatch with specified leadership status. */ + private LeaderLatch createMockLeaderLatch(boolean hasLeadership) throws Exception { + LeaderLatch mockLatch = mock(LeaderLatch.class); + when(mockLatch.hasLeadership()).thenReturn(hasLeadership); + doAnswer(invocation -> null).when(mockLatch).addListener(any()); + doAnswer(invocation -> null).when(mockLatch).start(); + doAnswer(invocation -> null).when(mockLatch).close(); + // Mock await() - it throws IOException and InterruptedException + doAnswer( + invocation -> { + // Mock implementation - doesn't actually wait + return null; + }) + .when(mockLatch) + .await(); + return mockLatch; + } + + /** In-memory ZK state simulator. */ + private static class MockZkState { + private final Map nodes = new HashMap<>(); + private final AtomicInteger sequenceCounter = new AtomicInteger(0); + + public List getChildren(String path) throws KeeperException { + List children = new ArrayList<>(); + String prefix = path.endsWith("/") ? path : path + "/"; + for (String nodePath : nodes.keySet()) { + if (nodePath.startsWith(prefix) && !nodePath.equals(path)) { + String relativePath = nodePath.substring(prefix.length()); + if (!relativePath.contains("/")) { + children.add(relativePath); + } + } + } + return children; + } + + public byte[] getData(String path) throws KeeperException { + byte[] data = nodes.get(path); + if (data == null) { + throw new KeeperException.NoNodeException(path); + } + return data; + } + + public String createNode(String path, byte[] data) { + // Handle sequential nodes + if (path.endsWith("-")) { + int seq = sequenceCounter.incrementAndGet(); + path = path + String.format("%010d", seq); + } + nodes.put(path, data); + return path; + } + + public void deleteNode(String path) throws KeeperException { + if (!nodes.containsKey(path)) { + throw new KeeperException.NoNodeException(path); + } + nodes.remove(path); + } + + public Stat exists(String path) { + return nodes.containsKey(path) ? new Stat() : null; + } + + public void clear() { + nodes.clear(); + sequenceCounter.set(0); + } + } } From b635c0dd87dc4952b05909269f728bdb79c455b7 Mon Sep 17 00:00:00 2001 From: wardli Date: Mon, 10 Nov 2025 17:29:56 +0800 Subject: [PATCH 07/15] [Subtask]: Replace zk with mocking. #3919 --- .../server/TestHighAvailabilityContainer.java | 153 +++++++++++++++--- 1 file changed, 129 insertions(+), 24 deletions(-) diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/TestHighAvailabilityContainer.java b/amoro-ams/src/test/java/org/apache/amoro/server/TestHighAvailabilityContainer.java index 785340ff14..27d0d8e29b 100644 --- a/amoro-ams/src/test/java/org/apache/amoro/server/TestHighAvailabilityContainer.java +++ b/amoro-ams/src/test/java/org/apache/amoro/server/TestHighAvailabilityContainer.java @@ -190,15 +190,21 @@ public void testGetAliveNodesWithMultipleNodes() throws Exception { // Register first node haContainer.registAndElect(); - // Register second node manually in mock state + // Verify first node was registered String nodesPath = AmsHAProperties.getNodesPath("test-cluster"); + List childrenAfterFirst = mockZkState.getChildren(nodesPath); + Assert.assertEquals("First node should be registered", 1, childrenAfterFirst.size()); + + // Register second node manually in mock state + // Use createNode with sequential path to get the correct sequence number AmsServerInfo nodeInfo2 = new AmsServerInfo(); nodeInfo2.setHost("127.0.0.2"); nodeInfo2.setThriftBindPort(1262); nodeInfo2.setRestBindPort(1631); String nodeInfo2Json = JacksonUtil.toJSONString(nodeInfo2); - mockZkState.createNode( - nodesPath + "/node-0000000001", nodeInfo2Json.getBytes(StandardCharsets.UTF_8)); + // Use sequential path ending with "-" to let createNode generate the sequence number + // This ensures the second node gets the correct sequence number (0000000001) + mockZkState.createNode(nodesPath + "/node-", nodeInfo2Json.getBytes(StandardCharsets.UTF_8)); // Get alive nodes List aliveNodes = haContainer.getAliveNodes(); @@ -260,9 +266,10 @@ public void testRegistAndElectWithoutHAEnabled() throws Exception { /** Create HighAvailabilityContainer with mocked ZK components using reflection. */ private HighAvailabilityContainer createContainerWithMockZk() throws Exception { - HighAvailabilityContainer container = new HighAvailabilityContainer(serviceConfig); + // Create container without ZK connection to avoid any connection attempts + HighAvailabilityContainer container = createContainerWithoutZk(); - // Use reflection to inject mock ZK client and leader latch + // Inject mock ZK client and leader latch java.lang.reflect.Field zkClientField = HighAvailabilityContainer.class.getDeclaredField("zkClient"); zkClientField.setAccessible(true); @@ -273,9 +280,88 @@ private HighAvailabilityContainer createContainerWithMockZk() throws Exception { leaderLatchField.setAccessible(true); leaderLatchField.set(container, mockLeaderLatch); + // Note: We don't need to create the paths themselves as nodes in ZK + // ZK paths are logical containers, not actual nodes + // The createPathIfNeeded() calls will be handled by the mock when needed + return container; } + /** + * Create a HighAvailabilityContainer without initializing ZK connection. This is used when we + * want to completely avoid ZK connection attempts. + */ + private HighAvailabilityContainer createContainerWithoutZk() throws Exception { + // Use reflection to create container without calling constructor + java.lang.reflect.Constructor constructor = + HighAvailabilityContainer.class.getDeclaredConstructor(Configurations.class); + + // Create a minimal config that disables HA to avoid ZK connection + Configurations tempConfig = new Configurations(serviceConfig); + tempConfig.setBoolean(AmoroManagementConf.HA_ENABLE, false); + + HighAvailabilityContainer container = constructor.newInstance(tempConfig); + + // Now set all required fields using reflection + java.lang.reflect.Field isMasterSlaveModeField = + HighAvailabilityContainer.class.getDeclaredField("isMasterSlaveMode"); + isMasterSlaveModeField.setAccessible(true); + isMasterSlaveModeField.set( + container, serviceConfig.getBoolean(AmoroManagementConf.USE_MASTER_SLAVE_MODE)); + + if (serviceConfig.getBoolean(AmoroManagementConf.HA_ENABLE)) { + String haClusterName = serviceConfig.getString(AmoroManagementConf.HA_CLUSTER_NAME); + + java.lang.reflect.Field tableServiceMasterPathField = + HighAvailabilityContainer.class.getDeclaredField("tableServiceMasterPath"); + tableServiceMasterPathField.setAccessible(true); + tableServiceMasterPathField.set( + container, AmsHAProperties.getTableServiceMasterPath(haClusterName)); + + java.lang.reflect.Field optimizingServiceMasterPathField = + HighAvailabilityContainer.class.getDeclaredField("optimizingServiceMasterPath"); + optimizingServiceMasterPathField.setAccessible(true); + optimizingServiceMasterPathField.set( + container, AmsHAProperties.getOptimizingServiceMasterPath(haClusterName)); + + java.lang.reflect.Field nodesPathField = + HighAvailabilityContainer.class.getDeclaredField("nodesPath"); + nodesPathField.setAccessible(true); + nodesPathField.set(container, AmsHAProperties.getNodesPath(haClusterName)); + + java.lang.reflect.Field tableServiceServerInfoField = + HighAvailabilityContainer.class.getDeclaredField("tableServiceServerInfo"); + tableServiceServerInfoField.setAccessible(true); + AmsServerInfo tableServiceServerInfo = + buildServerInfo( + serviceConfig.getString(AmoroManagementConf.SERVER_EXPOSE_HOST), + serviceConfig.getInteger(AmoroManagementConf.TABLE_SERVICE_THRIFT_BIND_PORT), + serviceConfig.getInteger(AmoroManagementConf.HTTP_SERVER_PORT)); + tableServiceServerInfoField.set(container, tableServiceServerInfo); + + java.lang.reflect.Field optimizingServiceServerInfoField = + HighAvailabilityContainer.class.getDeclaredField("optimizingServiceServerInfo"); + optimizingServiceServerInfoField.setAccessible(true); + AmsServerInfo optimizingServiceServerInfo = + buildServerInfo( + serviceConfig.getString(AmoroManagementConf.SERVER_EXPOSE_HOST), + serviceConfig.getInteger(AmoroManagementConf.OPTIMIZING_SERVICE_THRIFT_BIND_PORT), + serviceConfig.getInteger(AmoroManagementConf.HTTP_SERVER_PORT)); + optimizingServiceServerInfoField.set(container, optimizingServiceServerInfo); + } + + return container; + } + + /** Helper method to build AmsServerInfo (copied from HighAvailabilityContainer). */ + private AmsServerInfo buildServerInfo(String host, Integer thriftPort, Integer httpPort) { + AmsServerInfo serverInfo = new AmsServerInfo(); + serverInfo.setHost(host); + serverInfo.setThriftBindPort(thriftPort); + serverInfo.setRestBindPort(httpPort); + return serverInfo; + } + /** Create a mock CuratorFramework that uses MockZkState for storage. */ @SuppressWarnings("unchecked") private CuratorFramework createMockZkClient() throws Exception { @@ -309,31 +395,29 @@ private CuratorFramework createMockZkClient() throws Exception { return mockZkState.getData(path); }); - // Mock create() - use Answer to handle the entire fluent API chain - @SuppressWarnings({"unchecked", "rawtypes"}) + // Mock create() - manually create the entire fluent API chain to ensure consistency org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.CreateBuilder createBuilder = mock( org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.CreateBuilder.class); - @SuppressWarnings({"unchecked", "rawtypes"}) - org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.CreateBuilderMain - createBuilderMain = - mock( - org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.CreateBuilderMain - .class); - @SuppressWarnings({"unchecked", "rawtypes"}) - org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.PathAndBytesable - createPathAndBytesable = + + @SuppressWarnings("unchecked") + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api + .ProtectACLCreateModeStatPathAndBytesable< + String> + pathAndBytesable = mock( - org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.PathAndBytesable - .class); + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api + .ProtectACLCreateModeStatPathAndBytesable.class); when(mockClient.create()).thenReturn(createBuilder); - // Use Answer to handle type mismatch in fluent API - doAnswer(invocation -> createBuilderMain).when(createBuilder).creatingParentsIfNeeded(); - doAnswer(invocation -> createPathAndBytesable) - .when(createBuilderMain) - .withMode(any(CreateMode.class)); - when(createPathAndBytesable.forPath(anyString(), any(byte[].class))) + + // Mock the chain: creatingParentsIfNeeded() -> withMode() -> forPath() + // Use the same mock object for the entire chain + when(createBuilder.creatingParentsIfNeeded()).thenReturn(pathAndBytesable); + when(pathAndBytesable.withMode(any(CreateMode.class))).thenReturn(pathAndBytesable); + + // Mock forPath(path, data) - used by registAndElect() + when(pathAndBytesable.forPath(anyString(), any(byte[].class))) .thenAnswer( invocation -> { String path = invocation.getArgument(0); @@ -341,6 +425,22 @@ private CuratorFramework createMockZkClient() throws Exception { return mockZkState.createNode(path, data); }); + // Mock forPath(path) - used by createPathIfNeeded() + // Note: createPathIfNeeded() creates paths without data, but we still need to store them + // so that getChildren() can work correctly + when(pathAndBytesable.forPath(anyString())) + .thenAnswer( + invocation -> { + String path = invocation.getArgument(0); + // Create the path as an empty node (this simulates ZK path creation) + // In real ZK, paths are logical containers, but we need to store them + // to make getChildren() work correctly + if (mockZkState.exists(path) == null) { + mockZkState.createNode(path, new byte[0]); + } + return null; + }); + // Mock delete() org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.DeleteBuilder deleteBuilder = mock( @@ -409,13 +509,18 @@ public List getChildren(String path) throws KeeperException { List children = new ArrayList<>(); String prefix = path.endsWith("/") ? path : path + "/"; for (String nodePath : nodes.keySet()) { + // Only include direct children (not the path itself, and not nested paths) if (nodePath.startsWith(prefix) && !nodePath.equals(path)) { String relativePath = nodePath.substring(prefix.length()); + // Only add direct children (no additional slashes) + // This means the path should be exactly: prefix + relativePath if (!relativePath.contains("/")) { children.add(relativePath); } } } + // Sort to ensure consistent ordering + children.sort(String::compareTo); return children; } From 333579015486f34400b70c9755cfd85b54a5bf88 Mon Sep 17 00:00:00 2001 From: wardli Date: Mon, 10 Nov 2025 19:18:29 +0800 Subject: [PATCH 08/15] [Subtask]: add AmsAssignService to implement balanced bucket allocation in master-slave mode. #3921 --- .../amoro/server/AmoroManagementConf.java | 14 + .../apache/amoro/server/AmsAssignService.java | 403 ++++++++ .../amoro/server/BucketAssignStore.java | 82 ++ .../amoro/server/ZkBucketAssignStore.java | 238 +++++ .../amoro/server/TestAmsAssignService.java | 883 ++++++++++++++++++ .../amoro/server/TestZkBucketAssignStore.java | 482 ++++++++++ .../amoro/properties/AmsHAProperties.java | 5 + 7 files changed, 2107 insertions(+) create mode 100644 amoro-ams/src/main/java/org/apache/amoro/server/AmsAssignService.java create mode 100644 amoro-ams/src/main/java/org/apache/amoro/server/BucketAssignStore.java create mode 100644 amoro-ams/src/main/java/org/apache/amoro/server/ZkBucketAssignStore.java create mode 100644 amoro-ams/src/test/java/org/apache/amoro/server/TestAmsAssignService.java create mode 100644 amoro-ams/src/test/java/org/apache/amoro/server/TestZkBucketAssignStore.java diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/AmoroManagementConf.java b/amoro-ams/src/main/java/org/apache/amoro/server/AmoroManagementConf.java index 8e572125a2..2304babc9e 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/AmoroManagementConf.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/AmoroManagementConf.java @@ -59,6 +59,20 @@ public class AmoroManagementConf { .defaultValue(false) .withDescription("Enable master & slave mode, which supports horizontal scaling of AMS."); + public static final ConfigOption BUCKET_ID_TOTAL_COUNT = + ConfigOptions.key("bucket-id.total-count") + .intType() + .defaultValue(100) + .withDescription( + "Total count of bucket IDs for assignment. Bucket IDs range from 1 to this value."); + + public static final ConfigOption NODE_OFFLINE_TIMEOUT = + ConfigOptions.key("node-offline.timeout") + .durationType() + .defaultValue(Duration.ofMinutes(5)) + .withDescription( + "Timeout duration to determine if a node is offline. After this duration, the node's bucket IDs will be reassigned."); + public static final ConfigOption CATALOG_META_CACHE_EXPIRATION_INTERVAL = ConfigOptions.key("catalog-meta-cache.expiration-interval") .durationType() diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/AmsAssignService.java b/amoro-ams/src/main/java/org/apache/amoro/server/AmsAssignService.java new file mode 100644 index 0000000000..a7c8241b0a --- /dev/null +++ b/amoro-ams/src/main/java/org/apache/amoro/server/AmsAssignService.java @@ -0,0 +1,403 @@ +/* + * 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.shade.guava32.com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.CuratorFramework; +import org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.recipes.leader.LeaderLatch; +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 static final long ASSIGN_INTERVAL_SECONDS = 30; + + 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 volatile boolean running = false; + + // Package-private accessors for testing + BucketAssignStore getAssignStore() { + return assignStore; + } + + boolean isRunning() { + return running; + } + + void doAssignForTest() { + doAssign(); + } + + public AmsAssignService( + HighAvailabilityContainer haContainer, + Configurations serviceConfig, + CuratorFramework zkClient, + LeaderLatch leaderLatch) { + this.haContainer = haContainer; + this.serviceConfig = serviceConfig; + this.bucketIdTotalCount = serviceConfig.getInteger(AmoroManagementConf.BUCKET_ID_TOTAL_COUNT); + this.nodeOfflineTimeoutMs = + serviceConfig.get(AmoroManagementConf.NODE_OFFLINE_TIMEOUT).toMillis(); + String clusterName = serviceConfig.getString(AmoroManagementConf.HA_CLUSTER_NAME); + this.assignStore = new ZkBucketAssignStore(zkClient, clusterName, leaderLatch); + } + + /** + * 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, ASSIGN_INTERVAL_SECONDS, TimeUnit.SECONDS); + LOG.info("Bucket assignment service started"); + } + + /** 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 aliveNodes = haContainer.getAliveNodes(); + if (aliveNodes.isEmpty()) { + LOG.debug("No alive nodes found, skip bucket assignment"); + return; + } + + Map> currentAssignments = assignStore.getAllAssignments(); + Set currentAssignedNodes = new HashSet<>(currentAssignments.keySet()); + Set aliveNodeSet = new HashSet<>(aliveNodes); + + // Detect new nodes and offline nodes + Set newNodes = new HashSet<>(aliveNodeSet); + newNodes.removeAll(currentAssignedNodes); + + Set offlineNodes = new HashSet<>(currentAssignedNodes); + offlineNodes.removeAll(aliveNodeSet); + + // Check for nodes that haven't updated for a long time + long currentTime = System.currentTimeMillis(); + for (AmsServerInfo node : currentAssignedNodes) { + if (aliveNodeSet.contains(node)) { + long lastUpdateTime = assignStore.getLastUpdateTime(node); + if (lastUpdateTime > 0 && (currentTime - lastUpdateTime) > nodeOfflineTimeoutMs) { + offlineNodes.add(node); + LOG.warn( + "Node {} is considered offline due to timeout. Last update: {}", + node, + lastUpdateTime); + } + } + } + + boolean needReassign = !newNodes.isEmpty() || !offlineNodes.isEmpty(); + + if (needReassign) { + LOG.info( + "Detected node changes - New nodes: {}, Offline nodes: {}, Performing incremental reassignment...", + newNodes.size(), + offlineNodes.size()); + + // Step 1: Handle offline nodes - collect their buckets for redistribution + List bucketsToRedistribute = new ArrayList<>(); + for (AmsServerInfo offlineNode : offlineNodes) { + try { + List offlineBuckets = currentAssignments.get(offlineNode); + if (offlineBuckets != null && !offlineBuckets.isEmpty()) { + bucketsToRedistribute.addAll(offlineBuckets); + LOG.info( + "Collected {} buckets from offline node {} for redistribution", + offlineBuckets.size(), + offlineNode); + } + assignStore.removeAssignments(offlineNode); + } catch (Exception e) { + LOG.warn("Failed to remove assignments for offline node {}", offlineNode, e); + } + } + + // Step 2: Calculate target assignment for balanced distribution + List allBuckets = generateBucketIds(); + int totalBuckets = allBuckets.size(); + int totalAliveNodes = aliveNodes.size(); + int targetBucketsPerNode = totalBuckets / totalAliveNodes; + int remainder = totalBuckets % totalAliveNodes; + + // Step 3: Incremental reassignment + // Keep existing assignments for nodes that are still alive + Map> newAssignments = new java.util.HashMap<>(); + for (AmsServerInfo node : aliveNodes) { + List existingBuckets = currentAssignments.get(node); + if (existingBuckets != null && !offlineNodes.contains(node)) { + // Keep existing buckets for alive nodes (not offline) + newAssignments.put(node, new ArrayList<>(existingBuckets)); + } else { + // New node or node that was offline + newAssignments.put(node, new ArrayList<>()); + } + } + + // Step 4: Redistribute buckets from offline nodes to alive nodes + if (!bucketsToRedistribute.isEmpty()) { + redistributeBucketsIncrementally( + aliveNodes, bucketsToRedistribute, newAssignments, targetBucketsPerNode); + } + + // Step 5: Handle new nodes - balance buckets from existing nodes + if (!newNodes.isEmpty()) { + balanceBucketsForNewNodes( + aliveNodes, newNodes, newAssignments, targetBucketsPerNode, remainder); + } + + // Step 6: Handle unassigned buckets (if any) + Set allAssignedBuckets = new HashSet<>(); + for (List buckets : newAssignments.values()) { + allAssignedBuckets.addAll(buckets); + } + List unassignedBuckets = new ArrayList<>(); + for (String bucket : allBuckets) { + if (!allAssignedBuckets.contains(bucket)) { + unassignedBuckets.add(bucket); + } + } + if (!unassignedBuckets.isEmpty()) { + redistributeBucketsIncrementally( + aliveNodes, unassignedBuckets, newAssignments, targetBucketsPerNode); + } + + // Step 7: Save all new assignments + for (Map.Entry> entry : newAssignments.entrySet()) { + try { + assignStore.saveAssignments(entry.getKey(), entry.getValue()); + LOG.info( + "Assigned {} buckets to node {}: {}", + entry.getValue().size(), + entry.getKey(), + entry.getValue()); + } catch (Exception e) { + LOG.error("Failed to save assignments for node {}", entry.getKey(), e); + } + } + } else { + // Update last update time for alive nodes + for (AmsServerInfo node : aliveNodes) { + assignStore.updateLastUpdateTime(node); + } + } + } catch (Exception e) { + LOG.error("Error during bucket assignment", e); + } + } + + /** + * Redistribute buckets incrementally to alive nodes using round-robin. This minimizes bucket + * migration by only redistributing buckets from offline nodes. + * + * @param aliveNodes List of alive nodes + * @param bucketsToRedistribute Buckets to redistribute (from offline nodes) + * @param currentAssignments Current assignments map (will be modified) + * @param targetBucketsPerNode Target number of buckets per node + */ + private void redistributeBucketsIncrementally( + List aliveNodes, + List bucketsToRedistribute, + Map> currentAssignments, + int targetBucketsPerNode) { + if (aliveNodes.isEmpty() || bucketsToRedistribute.isEmpty()) { + return; + } + + // Distribute buckets using round-robin to minimize migration + int nodeIndex = 0; + for (String bucketId : bucketsToRedistribute) { + AmsServerInfo node = aliveNodes.get(nodeIndex % aliveNodes.size()); + currentAssignments.get(node).add(bucketId); + nodeIndex++; + } + } + + /** + * Balance buckets for new nodes by taking buckets from existing nodes. This minimizes migration + * by only moving necessary buckets to new nodes. + * + * @param aliveNodes All alive nodes + * @param newNodes Newly added nodes + * @param currentAssignments Current assignments map (will be modified) + * @param targetBucketsPerNode Target number of buckets per node + * @param remainder Remainder when dividing total buckets by node count + */ + private void balanceBucketsForNewNodes( + List aliveNodes, + Set newNodes, + Map> currentAssignments, + int targetBucketsPerNode, + int remainder) { + if (newNodes.isEmpty()) { + return; + } + + // Calculate how many buckets each new node should get + int bucketsPerNewNode = targetBucketsPerNode; + int newNodeIndex = 0; + for (AmsServerInfo newNode : newNodes) { + // First 'remainder' nodes get one extra bucket + int targetForNewNode = bucketsPerNewNode + (newNodeIndex < remainder ? 1 : 0); + int currentCount = currentAssignments.get(newNode).size(); + int needed = targetForNewNode - currentCount; + + if (needed > 0) { + // Collect buckets from existing nodes (prefer nodes with more buckets) + List bucketsToMove = + collectBucketsFromExistingNodes(aliveNodes, newNodes, currentAssignments, needed); + currentAssignments.get(newNode).addAll(bucketsToMove); + LOG.info( + "Moved {} buckets to new node {} (target: {})", + bucketsToMove.size(), + newNode, + targetForNewNode); + } + newNodeIndex++; + } + } + + /** + * Collect buckets from existing nodes to balance for new nodes. Prefer taking from nodes that + * have more buckets than target. + * + * @param aliveNodes All alive nodes + * @param newNodes New nodes (excluded from source) + * @param currentAssignments Current assignments + * @param needed Number of buckets needed + * @return List of bucket IDs to move + */ + private List collectBucketsFromExistingNodes( + List aliveNodes, + Set newNodes, + Map> currentAssignments, + int needed) { + List bucketsToMove = new ArrayList<>(); + List existingNodes = new ArrayList<>(); + for (AmsServerInfo node : aliveNodes) { + if (!newNodes.contains(node)) { + existingNodes.add(node); + } + } + + if (existingNodes.isEmpty()) { + return bucketsToMove; + } + + // Sort existing nodes by current bucket count (descending) + // This ensures we take from nodes with more buckets first + existingNodes.sort( + (n1, n2) -> { + int count1 = currentAssignments.get(n1).size(); + int count2 = currentAssignments.get(n2).size(); + return Integer.compare(count2, count1); + }); + + // Collect buckets from existing nodes using round-robin + int nodeIndex = 0; + int collected = 0; + while (collected < needed && !existingNodes.isEmpty()) { + AmsServerInfo sourceNode = existingNodes.get(nodeIndex % existingNodes.size()); + List sourceBuckets = currentAssignments.get(sourceNode); + if (!sourceBuckets.isEmpty()) { + // Take one bucket from this node + String bucketToMove = sourceBuckets.remove(0); + bucketsToMove.add(bucketToMove); + collected++; + LOG.debug("Moving bucket {} from node {} to new node", bucketToMove, sourceNode); + } else { + // This node has no more buckets, remove it from consideration + existingNodes.remove(sourceNode); + if (existingNodes.isEmpty()) { + break; + } + nodeIndex = nodeIndex % existingNodes.size(); + continue; + } + nodeIndex++; + } + + return bucketsToMove; + } + + private List generateBucketIds() { + List bucketIds = new ArrayList<>(); + for (int i = 1; i <= bucketIdTotalCount; i++) { + bucketIds.add(String.valueOf(i)); + } + return bucketIds; + } +} diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/BucketAssignStore.java b/amoro-ams/src/main/java/org/apache/amoro/server/BucketAssignStore.java new file mode 100644 index 0000000000..8b982f18ef --- /dev/null +++ b/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 bucketIds) throws Exception; + + /** + * Get bucket ID assignments for a node. + * + * @param nodeInfo The node information + * @return List of bucket IDs assigned to this node, empty list if not found + * @throws Exception If retrieval operation fails + */ + List getAssignments(AmsServerInfo nodeInfo) throws Exception; + + /** + * Remove bucket ID assignments for a node. + * + * @param nodeInfo The node information + * @throws Exception If removal operation fails + */ + void removeAssignments(AmsServerInfo nodeInfo) throws Exception; + + /** + * Get all bucket ID assignments for all nodes. + * + * @return Map of node info to list of bucket IDs + * @throws Exception If retrieval operation fails + */ + Map> getAllAssignments() throws Exception; + + /** + * Get the last update time for a node's assignments. + * + * @param nodeInfo The node information + * @return Last update timestamp in milliseconds, 0 if not found + * @throws Exception If retrieval operation fails + */ + long getLastUpdateTime(AmsServerInfo nodeInfo) throws Exception; + + /** + * Update the last update time for a node's assignments. + * + * @param nodeInfo The node information + * @throws Exception If update operation fails + */ + void updateLastUpdateTime(AmsServerInfo nodeInfo) throws Exception; +} diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/ZkBucketAssignStore.java b/amoro-ams/src/main/java/org/apache/amoro/server/ZkBucketAssignStore.java new file mode 100644 index 0000000000..40c1db8844 --- /dev/null +++ b/amoro-ams/src/main/java/org/apache/amoro/server/ZkBucketAssignStore.java @@ -0,0 +1,238 @@ +/* + * 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.properties.AmsHAProperties; +import org.apache.amoro.shade.jackson2.com.fasterxml.jackson.core.type.TypeReference; +import org.apache.amoro.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.CuratorFramework; +import org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.recipes.leader.LeaderLatch; +import org.apache.amoro.shade.zookeeper3.org.apache.zookeeper.CreateMode; +import org.apache.amoro.shade.zookeeper3.org.apache.zookeeper.KeeperException; +import org.apache.amoro.utils.JacksonUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * ZooKeeper-based implementation of BucketAssignStore. Stores bucket ID assignments in ZooKeeper + * with the following structure: /{namespace}/amoro/ams/bucket-assignments/{nodeKey}/assignments - + * bucket IDs /{namespace}/amoro/ams/bucket-assignments/{nodeKey}/last-update-time - timestamp + */ +public class ZkBucketAssignStore implements BucketAssignStore { + + private static final Logger LOG = LoggerFactory.getLogger(ZkBucketAssignStore.class); + private static final String ASSIGNMENTS_SUFFIX = "/assignments"; + private static final String LAST_UPDATE_TIME_SUFFIX = "/last-update-time"; + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final TypeReference> LIST_STRING_TYPE = + new TypeReference>() {}; + + private final CuratorFramework zkClient; + private final String assignmentsBasePath; + private final LeaderLatch leaderLatch; + + public ZkBucketAssignStore( + CuratorFramework zkClient, String clusterName, LeaderLatch leaderLatch) { + this.zkClient = zkClient; + this.assignmentsBasePath = AmsHAProperties.getBucketAssignmentsPath(clusterName); + this.leaderLatch = leaderLatch; + try { + createPathIfNeeded(assignmentsBasePath); + } catch (Exception e) { + LOG.error("Failed to create bucket assignments path", e); + throw new RuntimeException("Failed to initialize ZkBucketAssignStore", e); + } + } + + @Override + public void saveAssignments(AmsServerInfo nodeInfo, List bucketIds) throws Exception { + if (!leaderLatch.hasLeadership()) { + LOG.warn("Only leader node can save bucket assignments"); + return; + } + String nodeKey = getNodeKey(nodeInfo); + String assignmentsPath = assignmentsBasePath + "/" + nodeKey + ASSIGNMENTS_SUFFIX; + String assignmentsJson = JacksonUtil.toJSONString(bucketIds); + try { + if (zkClient.checkExists().forPath(assignmentsPath) != null) { + zkClient + .setData() + .forPath(assignmentsPath, assignmentsJson.getBytes(StandardCharsets.UTF_8)); + } else { + zkClient + .create() + .creatingParentsIfNeeded() + .withMode(CreateMode.PERSISTENT) + .forPath(assignmentsPath, assignmentsJson.getBytes(StandardCharsets.UTF_8)); + } + updateLastUpdateTime(nodeInfo); + LOG.debug("Saved bucket assignments for node {}: {}", nodeKey, bucketIds); + } catch (Exception e) { + LOG.error("Failed to save bucket assignments for node {}", nodeKey, e); + throw e; + } + } + + @Override + public List getAssignments(AmsServerInfo nodeInfo) throws Exception { + String nodeKey = getNodeKey(nodeInfo); + String assignmentsPath = assignmentsBasePath + "/" + nodeKey + ASSIGNMENTS_SUFFIX; + try { + if (zkClient.checkExists().forPath(assignmentsPath) == null) { + return new ArrayList<>(); + } + byte[] data = zkClient.getData().forPath(assignmentsPath); + if (data == null || data.length == 0) { + return new ArrayList<>(); + } + String assignmentsJson = new String(data, StandardCharsets.UTF_8); + return OBJECT_MAPPER.readValue(assignmentsJson, LIST_STRING_TYPE); + } catch (KeeperException.NoNodeException e) { + return new ArrayList<>(); + } catch (Exception e) { + LOG.error("Failed to get bucket assignments for node {}", nodeKey, e); + throw e; + } + } + + @Override + public void removeAssignments(AmsServerInfo nodeInfo) throws Exception { + if (!leaderLatch.hasLeadership()) { + LOG.warn("Only leader node can remove bucket assignments"); + return; + } + String nodeKey = getNodeKey(nodeInfo); + String nodePath = assignmentsBasePath + "/" + nodeKey; + try { + if (zkClient.checkExists().forPath(nodePath) != null) { + zkClient.delete().deletingChildrenIfNeeded().forPath(nodePath); + LOG.debug("Removed bucket assignments for node {}", nodeKey); + } + } catch (KeeperException.NoNodeException e) { + // Already deleted, ignore + } catch (Exception e) { + LOG.error("Failed to remove bucket assignments for node {}", nodeKey, e); + throw e; + } + } + + @Override + public Map> getAllAssignments() throws Exception { + Map> allAssignments = new HashMap<>(); + try { + if (zkClient.checkExists().forPath(assignmentsBasePath) == null) { + return allAssignments; + } + List nodeKeys = zkClient.getChildren().forPath(assignmentsBasePath); + for (String nodeKey : nodeKeys) { + try { + AmsServerInfo nodeInfo = parseNodeKey(nodeKey); + List bucketIds = getAssignments(nodeInfo); + if (!bucketIds.isEmpty()) { + allAssignments.put(nodeInfo, bucketIds); + } + } catch (Exception e) { + LOG.warn("Failed to parse node key or get assignments: {}", nodeKey, e); + } + } + } catch (KeeperException.NoNodeException e) { + // Path doesn't exist, return empty map + } catch (Exception e) { + LOG.error("Failed to get all bucket assignments", e); + throw e; + } + return allAssignments; + } + + @Override + public long getLastUpdateTime(AmsServerInfo nodeInfo) throws Exception { + String nodeKey = getNodeKey(nodeInfo); + String timePath = assignmentsBasePath + "/" + nodeKey + LAST_UPDATE_TIME_SUFFIX; + try { + if (zkClient.checkExists().forPath(timePath) == null) { + return 0; + } + byte[] data = zkClient.getData().forPath(timePath); + if (data == null || data.length == 0) { + return 0; + } + return Long.parseLong(new String(data, StandardCharsets.UTF_8)); + } catch (KeeperException.NoNodeException e) { + return 0; + } catch (Exception e) { + LOG.error("Failed to get last update time for node {}", nodeKey, e); + throw e; + } + } + + @Override + public void updateLastUpdateTime(AmsServerInfo nodeInfo) throws Exception { + if (!leaderLatch.hasLeadership()) { + return; + } + String nodeKey = getNodeKey(nodeInfo); + String timePath = assignmentsBasePath + "/" + nodeKey + LAST_UPDATE_TIME_SUFFIX; + long currentTime = System.currentTimeMillis(); + String timeStr = String.valueOf(currentTime); + try { + if (zkClient.checkExists().forPath(timePath) != null) { + zkClient.setData().forPath(timePath, timeStr.getBytes(StandardCharsets.UTF_8)); + } else { + zkClient + .create() + .creatingParentsIfNeeded() + .withMode(CreateMode.PERSISTENT) + .forPath(timePath, timeStr.getBytes(StandardCharsets.UTF_8)); + } + } catch (Exception e) { + LOG.error("Failed to update last update time for node {}", nodeKey, e); + throw e; + } + } + + private String getNodeKey(AmsServerInfo nodeInfo) { + return nodeInfo.getHost() + ":" + nodeInfo.getThriftBindPort(); + } + + private AmsServerInfo parseNodeKey(String nodeKey) { + String[] parts = nodeKey.split(":"); + if (parts.length != 2) { + throw new IllegalArgumentException("Invalid node key format: " + nodeKey); + } + AmsServerInfo nodeInfo = new AmsServerInfo(); + nodeInfo.setHost(parts[0]); + nodeInfo.setThriftBindPort(Integer.parseInt(parts[1])); + return nodeInfo; + } + + private void createPathIfNeeded(String path) throws Exception { + try { + zkClient.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT).forPath(path); + } catch (KeeperException.NodeExistsException e) { + // ignore + } + } +} diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/TestAmsAssignService.java b/amoro-ams/src/test/java/org/apache/amoro/server/TestAmsAssignService.java new file mode 100644 index 0000000000..758d568dfc --- /dev/null +++ b/amoro-ams/src/test/java/org/apache/amoro/server/TestAmsAssignService.java @@ -0,0 +1,883 @@ +/* + * 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 static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import org.apache.amoro.client.AmsServerInfo; +import org.apache.amoro.config.Configurations; +import org.apache.amoro.properties.AmsHAProperties; +import org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.CuratorFramework; +import org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.recipes.leader.LeaderLatch; +import org.apache.amoro.shade.zookeeper3.org.apache.zookeeper.CreateMode; +import org.apache.amoro.shade.zookeeper3.org.apache.zookeeper.KeeperException; +import org.apache.amoro.shade.zookeeper3.org.apache.zookeeper.data.Stat; +import org.apache.amoro.utils.JacksonUtil; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; + +/** Test for AmsAssignService using mocked ZK to avoid connection issues. */ +public class TestAmsAssignService { + + private Configurations serviceConfig; + private HighAvailabilityContainer haContainer; + private AmsAssignService assignService; + private AmsServerInfo node1; + private AmsServerInfo node2; + private AmsServerInfo node3; + private MockZkState mockZkState; + private CuratorFramework mockZkClient; + private LeaderLatch mockLeaderLatch; + private MockBucketAssignStore mockAssignStore; + + @Before + public void setUp() throws Exception { + mockZkState = new MockZkState(); + mockZkClient = createMockZkClient(); + mockLeaderLatch = createMockLeaderLatch(true); // Is leader by default + mockAssignStore = new MockBucketAssignStore(); + + serviceConfig = new Configurations(); + serviceConfig.setString(AmoroManagementConf.SERVER_EXPOSE_HOST, "127.0.0.1"); + serviceConfig.setInteger(AmoroManagementConf.TABLE_SERVICE_THRIFT_BIND_PORT, 1260); + serviceConfig.setInteger(AmoroManagementConf.OPTIMIZING_SERVICE_THRIFT_BIND_PORT, 1261); + serviceConfig.setInteger(AmoroManagementConf.HTTP_SERVER_PORT, 1630); + serviceConfig.setBoolean(AmoroManagementConf.HA_ENABLE, true); + serviceConfig.setString(AmoroManagementConf.HA_ZOOKEEPER_ADDRESS, "127.0.0.1:2181"); + serviceConfig.setString(AmoroManagementConf.HA_CLUSTER_NAME, "test-cluster"); + serviceConfig.setBoolean(AmoroManagementConf.USE_MASTER_SLAVE_MODE, true); + serviceConfig.setInteger(AmoroManagementConf.BUCKET_ID_TOTAL_COUNT, 100); + serviceConfig.set(AmoroManagementConf.NODE_OFFLINE_TIMEOUT, java.time.Duration.ofMinutes(5)); + + haContainer = createContainerWithMockZk(); + + // Create AmsAssignService with mock assign store + assignService = createAssignServiceWithMockStore(); + + node1 = new AmsServerInfo(); + node1.setHost("127.0.0.1"); + node1.setThriftBindPort(1260); + node1.setRestBindPort(1630); + + node2 = new AmsServerInfo(); + node2.setHost("127.0.0.2"); + node2.setThriftBindPort(1262); + node2.setRestBindPort(1632); + + node3 = new AmsServerInfo(); + node3.setHost("127.0.0.3"); + node3.setThriftBindPort(1263); + node3.setRestBindPort(1633); + } + + @After + public void tearDown() throws Exception { + if (assignService != null) { + assignService.stop(); + } + if (haContainer != null) { + haContainer.close(); + } + mockZkState.clear(); + } + + @Test + public void testInitialAssignment() throws Exception { + // Register nodes + haContainer.registAndElect(); + + // Create second node + Configurations config2 = createNodeConfig("127.0.0.2", 1262, 1632); + HighAvailabilityContainer haContainer2 = createContainerWithMockZk(config2); + haContainer2.registAndElect(); + + try { + // Wait a bit for registration + Thread.sleep(100); + + // Trigger assignment manually + assignService.doAssignForTest(); + + // Check assignments + Map> assignments = mockAssignStore.getAllAssignments(); + Assert.assertEquals("Should have assignments for 2 nodes", 2, assignments.size()); + + // Verify buckets are distributed + int totalAssigned = 0; + for (List buckets : assignments.values()) { + totalAssigned += buckets.size(); + Assert.assertTrue("Each node should have buckets", !buckets.isEmpty()); + } + Assert.assertEquals("All buckets should be assigned", 100, totalAssigned); + + // Verify balance (difference should be at most 1) + List bucketCounts = new ArrayList<>(); + for (List buckets : assignments.values()) { + bucketCounts.add(buckets.size()); + } + int max = bucketCounts.stream().mapToInt(Integer::intValue).max().orElse(0); + int min = bucketCounts.stream().mapToInt(Integer::intValue).min().orElse(0); + Assert.assertTrue("Difference should be at most 1", max - min <= 1); + } finally { + haContainer2.close(); + } + } + + @Test + public void testNodeOfflineReassignment() throws Exception { + // Setup: 2 nodes with initial assignment + haContainer.registAndElect(); + Configurations config2 = createNodeConfig("127.0.0.2", 1262, 1632); + HighAvailabilityContainer haContainer2 = createContainerWithMockZk(config2); + haContainer2.registAndElect(); + + try { + Thread.sleep(100); + + // Initial assignment + assignService.doAssignForTest(); + Map> initialAssignments = mockAssignStore.getAllAssignments(); + Assert.assertEquals("Should have 2 nodes", 2, initialAssignments.size()); + + // Verify initial assignment is balanced + List initialCounts = new ArrayList<>(); + for (List buckets : initialAssignments.values()) { + initialCounts.add(buckets.size()); + } + int maxInitial = initialCounts.stream().mapToInt(Integer::intValue).max().orElse(0); + int minInitial = initialCounts.stream().mapToInt(Integer::intValue).min().orElse(0); + Assert.assertTrue("Initial assignment should be balanced", maxInitial - minInitial <= 1); + + // Simulate node2 going offline by removing it from mock state + mockZkState.deleteNodeByHost("127.0.0.2"); + Thread.sleep(100); + + // Trigger reassignment + assignService.doAssignForTest(); + + // Check that node2's buckets are redistributed + Map> newAssignments = mockAssignStore.getAllAssignments(); + Assert.assertEquals("Should have 1 node after offline", 1, newAssignments.size()); + + // Verify node1 got all buckets + // Find node1 in the assignments (since parseNodeKey doesn't set restBindPort, + // we need to match by host and thriftBindPort) + List node1Buckets = null; + for (Map.Entry> entry : newAssignments.entrySet()) { + AmsServerInfo node = entry.getKey(); + if (node1.getHost().equals(node.getHost()) + && node1.getThriftBindPort().equals(node.getThriftBindPort())) { + node1Buckets = entry.getValue(); + break; + } + } + Assert.assertNotNull("Node1 should have assignments", node1Buckets); + Assert.assertEquals("Node1 should have all buckets", 100, node1Buckets.size()); + } finally { + try { + haContainer2.close(); + } catch (Exception e) { + // ignore + } + } + } + + @Test + public void testNewNodeIncrementalAssignment() throws Exception { + // Setup: 1 node initially + haContainer.registAndElect(); + Thread.sleep(100); + + // Initial assignment - all buckets to node1 + assignService.doAssignForTest(); + Map> initialAssignments = mockAssignStore.getAllAssignments(); + List node1InitialBuckets = initialAssignments.get(node1); + Assert.assertNotNull("Node1 should have assignments", node1InitialBuckets); + Assert.assertEquals("Node1 should have all buckets initially", 100, node1InitialBuckets.size()); + + // Add new node + Configurations config2 = createNodeConfig("127.0.0.2", 1262, 1632); + HighAvailabilityContainer haContainer2 = createContainerWithMockZk(config2); + haContainer2.registAndElect(); + + try { + Thread.sleep(100); + + // Trigger reassignment + assignService.doAssignForTest(); + + // Check assignments + Map> newAssignments = mockAssignStore.getAllAssignments(); + Assert.assertEquals("Should have 2 nodes", 2, newAssignments.size()); + + // Verify incremental assignment - node1 should keep most of its buckets + List node1NewBuckets = newAssignments.get(node1); + Assert.assertNotNull("Node1 should still have assignments", node1NewBuckets); + + // Node1 should have kept most buckets (incremental assignment) + Assert.assertTrue("Node1 should keep some buckets", node1NewBuckets.size() > 0); + + // Verify balance + List bucketCounts = new ArrayList<>(); + for (List buckets : newAssignments.values()) { + bucketCounts.add(buckets.size()); + } + int max = bucketCounts.stream().mapToInt(Integer::intValue).max().orElse(0); + int min = bucketCounts.stream().mapToInt(Integer::intValue).min().orElse(0); + Assert.assertTrue("Difference should be at most 1", max - min <= 1); + + // Verify total + int total = bucketCounts.stream().mapToInt(Integer::intValue).sum(); + Assert.assertEquals("Total buckets should be 100", 100, total); + } finally { + haContainer2.close(); + } + } + + @Test + public void testBalanceAfterNodeChanges() throws Exception { + // Setup: 3 nodes + haContainer.registAndElect(); + Configurations config2 = createNodeConfig("127.0.0.2", 1262, 1632); + HighAvailabilityContainer haContainer2 = createContainerWithMockZk(config2); + haContainer2.registAndElect(); + Configurations config3 = createNodeConfig("127.0.0.3", 1263, 1633); + HighAvailabilityContainer haContainer3 = createContainerWithMockZk(config3); + haContainer3.registAndElect(); + + try { + Thread.sleep(200); + + // Initial assignment + assignService.doAssignForTest(); + + // Verify balance + Map> assignments = mockAssignStore.getAllAssignments(); + Assert.assertEquals("Should have 3 nodes", 3, assignments.size()); + + List bucketCounts = new ArrayList<>(); + for (List buckets : assignments.values()) { + bucketCounts.add(buckets.size()); + } + int max = bucketCounts.stream().mapToInt(Integer::intValue).max().orElse(0); + int min = bucketCounts.stream().mapToInt(Integer::intValue).min().orElse(0); + Assert.assertTrue("Difference should be at most 1", max - min <= 1); + + // Verify all buckets are assigned + int total = bucketCounts.stream().mapToInt(Integer::intValue).sum(); + Assert.assertEquals("All buckets should be assigned", 100, total); + } finally { + haContainer2.close(); + haContainer3.close(); + } + } + + @Test + public void testIncrementalAssignmentMinimizesMigration() throws Exception { + // Setup: 2 nodes initially + haContainer.registAndElect(); + Configurations config2 = createNodeConfig("127.0.0.2", 1262, 1632); + HighAvailabilityContainer haContainer2 = createContainerWithMockZk(config2); + haContainer2.registAndElect(); + HighAvailabilityContainer haContainer3 = null; + + try { + Thread.sleep(100); + + // Initial assignment + assignService.doAssignForTest(); + Map> initialAssignments = mockAssignStore.getAllAssignments(); + + // Record initial assignments + Set node1InitialBuckets = new HashSet<>(); + Set node2InitialBuckets = new HashSet<>(); + for (Map.Entry> entry : initialAssignments.entrySet()) { + if (entry.getKey().getHost().equals("127.0.0.1")) { + node1InitialBuckets.addAll(entry.getValue()); + } else { + node2InitialBuckets.addAll(entry.getValue()); + } + } + + // Add new node + Configurations config3 = createNodeConfig("127.0.0.3", 1263, 1633); + haContainer3 = createContainerWithMockZk(config3); + haContainer3.registAndElect(); + + Thread.sleep(100); + + // Trigger reassignment + assignService.doAssignForTest(); + + // Check new assignments + Map> newAssignments = mockAssignStore.getAllAssignments(); + + // Calculate migration: buckets that moved from node1 or node2 + Set node1NewBuckets = new HashSet<>(); + Set node2NewBuckets = new HashSet<>(); + Set node3Buckets = new HashSet<>(); + for (Map.Entry> entry : newAssignments.entrySet()) { + if (entry.getKey().getHost().equals("127.0.0.1")) { + node1NewBuckets.addAll(entry.getValue()); + } else if (entry.getKey().getHost().equals("127.0.0.2")) { + node2NewBuckets.addAll(entry.getValue()); + } else { + node3Buckets.addAll(entry.getValue()); + } + } + + // Node1 and Node2 should keep most of their buckets + Set node1Kept = new HashSet<>(node1InitialBuckets); + node1Kept.retainAll(node1NewBuckets); + Set node2Kept = new HashSet<>(node2InitialBuckets); + node2Kept.retainAll(node2NewBuckets); + + // Verify incremental assignment: nodes should keep most buckets + Assert.assertTrue( + "Node1 should keep most buckets (incremental)", + node1Kept.size() > node1InitialBuckets.size() / 2); + Assert.assertTrue( + "Node2 should keep most buckets (incremental)", + node2Kept.size() > node2InitialBuckets.size() / 2); + + // Node3 should get buckets from both + Assert.assertTrue("Node3 should have buckets", node3Buckets.size() > 0); + } finally { + haContainer2.close(); + if (haContainer3 != null) { + try { + haContainer3.close(); + } catch (Exception e) { + // ignore + } + } + } + } + + @Test + public void testServiceStartStop() { + // Test that service can start and stop without errors + assignService.start(); + Assert.assertTrue("Service should be running", assignService.isRunning()); + + assignService.stop(); + Assert.assertFalse("Service should be stopped", assignService.isRunning()); + } + + @Test + public void testServiceSkipsWhenNotLeader() throws Exception { + // Create a non-leader container + mockLeaderLatch = createMockLeaderLatch(false); // Not leader + Configurations nonLeaderConfig = createNodeConfig("127.0.0.2", 1262, 1632); + HighAvailabilityContainer nonLeaderContainer = createContainerWithMockZk(nonLeaderConfig); + nonLeaderContainer.registAndElect(); + + try { + // Wait a bit + Thread.sleep(100); + + AmsAssignService nonLeaderService = createAssignServiceWithMockStore(nonLeaderContainer); + + // Should not throw exception even if not leader + nonLeaderService.doAssignForTest(); + + // Should not have assignments if not leader + Map> assignments = mockAssignStore.getAllAssignments(); + // Verify that non-leader doesn't create assignments + Assert.assertTrue( + "Non-leader should not create assignments", + assignments.isEmpty() || assignments.size() == 0); + } finally { + nonLeaderContainer.close(); + } + } + + private Configurations createNodeConfig(String host, int thriftPort, int httpPort) { + Configurations config = new Configurations(); + config.setString(AmoroManagementConf.SERVER_EXPOSE_HOST, host); + config.setInteger(AmoroManagementConf.TABLE_SERVICE_THRIFT_BIND_PORT, thriftPort); + config.setInteger(AmoroManagementConf.OPTIMIZING_SERVICE_THRIFT_BIND_PORT, thriftPort + 1); + config.setInteger(AmoroManagementConf.HTTP_SERVER_PORT, httpPort); + config.setBoolean(AmoroManagementConf.HA_ENABLE, true); + config.setString(AmoroManagementConf.HA_ZOOKEEPER_ADDRESS, "127.0.0.1:2181"); + config.setString(AmoroManagementConf.HA_CLUSTER_NAME, "test-cluster"); + config.setBoolean(AmoroManagementConf.USE_MASTER_SLAVE_MODE, true); + config.setInteger(AmoroManagementConf.BUCKET_ID_TOTAL_COUNT, 100); + config.set(AmoroManagementConf.NODE_OFFLINE_TIMEOUT, java.time.Duration.ofMinutes(5)); + return config; + } + + /** Create HighAvailabilityContainer with mocked ZK components using reflection. */ + private HighAvailabilityContainer createContainerWithMockZk() throws Exception { + return createContainerWithMockZk(serviceConfig); + } + + /** Create HighAvailabilityContainer with mocked ZK components using reflection. */ + private HighAvailabilityContainer createContainerWithMockZk(Configurations config) + throws Exception { + // Create container without ZK connection to avoid any connection attempts + HighAvailabilityContainer container = createContainerWithoutZk(config); + + // Inject mock ZK client and leader latch + java.lang.reflect.Field zkClientField = + HighAvailabilityContainer.class.getDeclaredField("zkClient"); + zkClientField.setAccessible(true); + zkClientField.set(container, mockZkClient); + + java.lang.reflect.Field leaderLatchField = + HighAvailabilityContainer.class.getDeclaredField("leaderLatch"); + leaderLatchField.setAccessible(true); + leaderLatchField.set(container, mockLeaderLatch); + + return container; + } + + /** Create a HighAvailabilityContainer without initializing ZK connection. */ + private HighAvailabilityContainer createContainerWithoutZk(Configurations config) + throws Exception { + java.lang.reflect.Constructor constructor = + HighAvailabilityContainer.class.getDeclaredConstructor(Configurations.class); + + // Create a minimal config that disables HA to avoid ZK connection + Configurations tempConfig = new Configurations(config); + tempConfig.setBoolean(AmoroManagementConf.HA_ENABLE, false); + + HighAvailabilityContainer container = constructor.newInstance(tempConfig); + + // Now set all required fields using reflection + java.lang.reflect.Field isMasterSlaveModeField = + HighAvailabilityContainer.class.getDeclaredField("isMasterSlaveMode"); + isMasterSlaveModeField.setAccessible(true); + isMasterSlaveModeField.set( + container, config.getBoolean(AmoroManagementConf.USE_MASTER_SLAVE_MODE)); + + if (config.getBoolean(AmoroManagementConf.HA_ENABLE)) { + String haClusterName = config.getString(AmoroManagementConf.HA_CLUSTER_NAME); + + java.lang.reflect.Field tableServiceMasterPathField = + HighAvailabilityContainer.class.getDeclaredField("tableServiceMasterPath"); + tableServiceMasterPathField.setAccessible(true); + tableServiceMasterPathField.set( + container, AmsHAProperties.getTableServiceMasterPath(haClusterName)); + + java.lang.reflect.Field optimizingServiceMasterPathField = + HighAvailabilityContainer.class.getDeclaredField("optimizingServiceMasterPath"); + optimizingServiceMasterPathField.setAccessible(true); + optimizingServiceMasterPathField.set( + container, AmsHAProperties.getOptimizingServiceMasterPath(haClusterName)); + + java.lang.reflect.Field nodesPathField = + HighAvailabilityContainer.class.getDeclaredField("nodesPath"); + nodesPathField.setAccessible(true); + nodesPathField.set(container, AmsHAProperties.getNodesPath(haClusterName)); + + java.lang.reflect.Field tableServiceServerInfoField = + HighAvailabilityContainer.class.getDeclaredField("tableServiceServerInfo"); + tableServiceServerInfoField.setAccessible(true); + AmsServerInfo tableServiceServerInfo = + buildServerInfo( + config.getString(AmoroManagementConf.SERVER_EXPOSE_HOST), + config.getInteger(AmoroManagementConf.TABLE_SERVICE_THRIFT_BIND_PORT), + config.getInteger(AmoroManagementConf.HTTP_SERVER_PORT)); + tableServiceServerInfoField.set(container, tableServiceServerInfo); + + java.lang.reflect.Field optimizingServiceServerInfoField = + HighAvailabilityContainer.class.getDeclaredField("optimizingServiceServerInfo"); + optimizingServiceServerInfoField.setAccessible(true); + AmsServerInfo optimizingServiceServerInfo = + buildServerInfo( + config.getString(AmoroManagementConf.SERVER_EXPOSE_HOST), + config.getInteger(AmoroManagementConf.OPTIMIZING_SERVICE_THRIFT_BIND_PORT), + config.getInteger(AmoroManagementConf.HTTP_SERVER_PORT)); + optimizingServiceServerInfoField.set(container, optimizingServiceServerInfo); + } + + return container; + } + + /** Helper method to build AmsServerInfo. */ + private AmsServerInfo buildServerInfo(String host, Integer thriftPort, Integer httpPort) { + AmsServerInfo serverInfo = new AmsServerInfo(); + serverInfo.setHost(host); + serverInfo.setThriftBindPort(thriftPort); + serverInfo.setRestBindPort(httpPort); + return serverInfo; + } + + /** Create AmsAssignService with mock BucketAssignStore. */ + private AmsAssignService createAssignServiceWithMockStore() throws Exception { + return createAssignServiceWithMockStore(haContainer); + } + + /** Create AmsAssignService with mock BucketAssignStore. */ + private AmsAssignService createAssignServiceWithMockStore(HighAvailabilityContainer container) + throws Exception { + AmsAssignService service = + new AmsAssignService(container, serviceConfig, mockZkClient, mockLeaderLatch); + + // Use reflection to inject mock assign store + java.lang.reflect.Field assignStoreField = + AmsAssignService.class.getDeclaredField("assignStore"); + assignStoreField.setAccessible(true); + assignStoreField.set(service, mockAssignStore); + + return service; + } + + /** Create a mock CuratorFramework that uses MockZkState for storage. */ + @SuppressWarnings("unchecked") + private CuratorFramework createMockZkClient() throws Exception { + CuratorFramework mockClient = mock(CuratorFramework.class); + + // Mock getChildren() + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.GetChildrenBuilder + getChildrenBuilder = + mock( + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api + .GetChildrenBuilder.class); + when(mockClient.getChildren()).thenReturn(getChildrenBuilder); + when(getChildrenBuilder.forPath(anyString())) + .thenAnswer( + invocation -> { + String path = invocation.getArgument(0); + return mockZkState.getChildren(path); + }); + + // Mock getData() + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.GetDataBuilder + getDataBuilder = + mock( + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.GetDataBuilder + .class); + when(mockClient.getData()).thenReturn(getDataBuilder); + when(getDataBuilder.forPath(anyString())) + .thenAnswer( + invocation -> { + String path = invocation.getArgument(0); + return mockZkState.getData(path); + }); + + // Mock create() - manually create the entire fluent API chain + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.CreateBuilder createBuilder = + mock( + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.CreateBuilder.class); + + @SuppressWarnings("unchecked") + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api + .ProtectACLCreateModeStatPathAndBytesable< + String> + pathAndBytesable = + mock( + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api + .ProtectACLCreateModeStatPathAndBytesable.class); + + when(mockClient.create()).thenReturn(createBuilder); + when(createBuilder.creatingParentsIfNeeded()).thenReturn(pathAndBytesable); + when(pathAndBytesable.withMode(any(CreateMode.class))).thenReturn(pathAndBytesable); + + // Mock forPath(path, data) - used by registAndElect() and saveAssignments() + when(pathAndBytesable.forPath(anyString(), any(byte[].class))) + .thenAnswer( + invocation -> { + String path = invocation.getArgument(0); + byte[] data = invocation.getArgument(1); + return mockZkState.createNode(path, data); + }); + + // Mock forPath(path) - used by createPathIfNeeded() + when(pathAndBytesable.forPath(anyString())) + .thenAnswer( + invocation -> { + String path = invocation.getArgument(0); + if (mockZkState.exists(path) == null) { + mockZkState.createNode(path, new byte[0]); + } + return null; + }); + + // Mock setData() + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.SetDataBuilder + setDataBuilder = + mock( + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.SetDataBuilder + .class); + when(mockClient.setData()).thenReturn(setDataBuilder); + when(setDataBuilder.forPath(anyString(), any(byte[].class))) + .thenAnswer( + invocation -> { + String path = invocation.getArgument(0); + byte[] data = invocation.getArgument(1); + mockZkState.setData(path, data); + return null; + }); + + // Mock delete() + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.DeleteBuilder deleteBuilder = + mock( + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.DeleteBuilder.class); + when(mockClient.delete()).thenReturn(deleteBuilder); + doAnswer( + invocation -> { + String path = invocation.getArgument(0); + mockZkState.deleteNode(path); + return null; + }) + .when(deleteBuilder) + .forPath(anyString()); + + // Mock deletingChildrenIfNeeded() + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.ChildrenDeletable + childrenDeletable = + mock( + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.ChildrenDeletable + .class); + when(deleteBuilder.deletingChildrenIfNeeded()).thenReturn(childrenDeletable); + doAnswer( + invocation -> { + String path = invocation.getArgument(0); + mockZkState.deleteNodeRecursive(path); + return null; + }) + .when(childrenDeletable) + .forPath(anyString()); + + // Mock checkExists() + @SuppressWarnings("unchecked") + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.ExistsBuilder + checkExistsBuilder = + mock( + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.ExistsBuilder + .class); + when(mockClient.checkExists()).thenReturn(checkExistsBuilder); + when(checkExistsBuilder.forPath(anyString())) + .thenAnswer( + invocation -> { + String path = invocation.getArgument(0); + return mockZkState.exists(path); + }); + + // Mock start() and close() + doAnswer(invocation -> null).when(mockClient).start(); + doAnswer(invocation -> null).when(mockClient).close(); + + return mockClient; + } + + /** Create a mock LeaderLatch with specified leadership status. */ + private LeaderLatch createMockLeaderLatch(boolean hasLeadership) throws Exception { + LeaderLatch mockLatch = mock(LeaderLatch.class); + when(mockLatch.hasLeadership()).thenReturn(hasLeadership); + doAnswer(invocation -> null).when(mockLatch).addListener(any()); + doAnswer(invocation -> null).when(mockLatch).start(); + doAnswer(invocation -> null).when(mockLatch).close(); + doAnswer( + invocation -> { + // Mock implementation - doesn't actually wait + return null; + }) + .when(mockLatch) + .await(); + return mockLatch; + } + + /** In-memory ZK state simulator. */ + private static class MockZkState { + private final Map nodes = new HashMap<>(); + private final AtomicInteger sequenceCounter = new AtomicInteger(0); + + public List getChildren(String path) throws KeeperException { + List children = new ArrayList<>(); + String prefix = path.endsWith("/") ? path : path + "/"; + for (String nodePath : nodes.keySet()) { + if (nodePath.startsWith(prefix) && !nodePath.equals(path)) { + String relativePath = nodePath.substring(prefix.length()); + if (!relativePath.contains("/")) { + children.add(relativePath); + } + } + } + children.sort(String::compareTo); + return children; + } + + public byte[] getData(String path) throws KeeperException { + byte[] data = nodes.get(path); + if (data == null) { + throw new KeeperException.NoNodeException(path); + } + return data; + } + + public void setData(String path, byte[] data) throws KeeperException { + if (!nodes.containsKey(path)) { + throw new KeeperException.NoNodeException(path); + } + nodes.put(path, data); + } + + public String createNode(String path, byte[] data) { + // Handle sequential nodes + if (path.endsWith("-")) { + int seq = sequenceCounter.incrementAndGet(); + path = path + String.format("%010d", seq); + } + nodes.put(path, data); + return path; + } + + public void deleteNode(String path) throws KeeperException { + if (!nodes.containsKey(path)) { + throw new KeeperException.NoNodeException(path); + } + nodes.remove(path); + } + + public void deleteNodeRecursive(String path) throws KeeperException { + // Delete the node and all its children + List toDelete = new ArrayList<>(); + String prefix = path.endsWith("/") ? path : path + "/"; + for (String nodePath : nodes.keySet()) { + if (nodePath.equals(path) || nodePath.startsWith(prefix)) { + toDelete.add(nodePath); + } + } + for (String nodePath : toDelete) { + nodes.remove(nodePath); + } + } + + public void deleteNodeByHost(String host) { + // Delete all nodes that have this host in their data (JSON) + List toDelete = new ArrayList<>(); + for (Map.Entry entry : nodes.entrySet()) { + String nodePath = entry.getKey(); + byte[] data = entry.getValue(); + // Check if this is a node registration path (contains "/node-") + if (nodePath.contains("/node-") && data != null && data.length > 0) { + try { + String nodeInfoJson = new String(data, java.nio.charset.StandardCharsets.UTF_8); + // Parse JSON to check host + AmsServerInfo nodeInfo = JacksonUtil.parseObject(nodeInfoJson, AmsServerInfo.class); + if (nodeInfo != null && host.equals(nodeInfo.getHost())) { + toDelete.add(nodePath); + } + } catch (Exception e) { + // Ignore parsing errors + } + } + } + for (String nodePath : toDelete) { + nodes.remove(nodePath); + } + } + + public Stat exists(String path) { + return nodes.containsKey(path) ? new Stat() : null; + } + + public void clear() { + nodes.clear(); + sequenceCounter.set(0); + } + } + + /** In-memory implementation of BucketAssignStore for testing. */ + private static class MockBucketAssignStore implements BucketAssignStore { + private final Map> assignments = new HashMap<>(); + private final Map lastUpdateTimes = new HashMap<>(); + // Store full AmsServerInfo for proper matching + private final Map nodeInfoMap = new HashMap<>(); + + private String getNodeKey(AmsServerInfo nodeInfo) { + return nodeInfo.getHost() + ":" + nodeInfo.getThriftBindPort(); + } + + @Override + public void saveAssignments(AmsServerInfo nodeInfo, List bucketIds) throws Exception { + String nodeKey = getNodeKey(nodeInfo); + assignments.put(nodeKey, new ArrayList<>(bucketIds)); + // Store full node info for proper matching + nodeInfoMap.put(nodeKey, nodeInfo); + updateLastUpdateTime(nodeInfo); + } + + @Override + public List getAssignments(AmsServerInfo nodeInfo) throws Exception { + String nodeKey = getNodeKey(nodeInfo); + return new ArrayList<>(assignments.getOrDefault(nodeKey, new ArrayList<>())); + } + + @Override + public void removeAssignments(AmsServerInfo nodeInfo) throws Exception { + String nodeKey = getNodeKey(nodeInfo); + assignments.remove(nodeKey); + lastUpdateTimes.remove(nodeKey); + nodeInfoMap.remove(nodeKey); + } + + @Override + public Map> getAllAssignments() throws Exception { + Map> result = new HashMap<>(); + for (Map.Entry> entry : assignments.entrySet()) { + String nodeKey = entry.getKey(); + // Use stored full node info if available, otherwise parse from key + AmsServerInfo nodeInfo = nodeInfoMap.getOrDefault(nodeKey, parseNodeKey(nodeKey)); + result.put(nodeInfo, new ArrayList<>(entry.getValue())); + } + return result; + } + + @Override + public long getLastUpdateTime(AmsServerInfo nodeInfo) throws Exception { + String nodeKey = getNodeKey(nodeInfo); + return lastUpdateTimes.getOrDefault(nodeKey, 0L); + } + + @Override + public void updateLastUpdateTime(AmsServerInfo nodeInfo) throws Exception { + String nodeKey = getNodeKey(nodeInfo); + lastUpdateTimes.put(nodeKey, System.currentTimeMillis()); + } + + private AmsServerInfo parseNodeKey(String nodeKey) { + String[] parts = nodeKey.split(":"); + if (parts.length != 2) { + throw new IllegalArgumentException("Invalid node key format: " + nodeKey); + } + AmsServerInfo nodeInfo = new AmsServerInfo(); + nodeInfo.setHost(parts[0]); + nodeInfo.setThriftBindPort(Integer.parseInt(parts[1])); + return nodeInfo; + } + } +} diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/TestZkBucketAssignStore.java b/amoro-ams/src/test/java/org/apache/amoro/server/TestZkBucketAssignStore.java new file mode 100644 index 0000000000..1d6e81a770 --- /dev/null +++ b/amoro-ams/src/test/java/org/apache/amoro/server/TestZkBucketAssignStore.java @@ -0,0 +1,482 @@ +/* + * 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 static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import org.apache.amoro.client.AmsServerInfo; +import org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.CuratorFramework; +import org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.recipes.leader.LeaderLatch; +import org.apache.amoro.shade.zookeeper3.org.apache.zookeeper.CreateMode; +import org.apache.amoro.shade.zookeeper3.org.apache.zookeeper.KeeperException; +import org.apache.amoro.shade.zookeeper3.org.apache.zookeeper.data.Stat; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; + +/** Test for ZkBucketAssignStore using mocked ZK to avoid connection issues. */ +public class TestZkBucketAssignStore { + + private CuratorFramework mockZkClient; + private LeaderLatch mockLeaderLatch; + private ZkBucketAssignStore assignStore; + private AmsServerInfo node1; + private AmsServerInfo node2; + private MockZkState mockZkState; + + @Before + public void setUp() throws Exception { + mockZkState = new MockZkState(); + mockZkClient = createMockZkClient(); + mockLeaderLatch = createMockLeaderLatch(true); // Is leader by default + + assignStore = new ZkBucketAssignStore(mockZkClient, "test-cluster", mockLeaderLatch); + + node1 = new AmsServerInfo(); + node1.setHost("127.0.0.1"); + node1.setThriftBindPort(1260); + node1.setRestBindPort(1630); + + node2 = new AmsServerInfo(); + node2.setHost("127.0.0.2"); + node2.setThriftBindPort(1261); + node2.setRestBindPort(1631); + } + + @After + public void tearDown() throws Exception { + if (assignStore != null) { + try { + assignStore.removeAssignments(node1); + assignStore.removeAssignments(node2); + } catch (Exception e) { + // ignore + } + } + mockZkState.clear(); + } + + @Test + public void testSaveAndGetAssignments() throws Exception { + List bucketIds = Arrays.asList("1", "2", "3", "4", "5"); + + // Save assignments + assignStore.saveAssignments(node1, bucketIds); + + // Get assignments + List retrieved = assignStore.getAssignments(node1); + Assert.assertEquals("Bucket IDs should match", bucketIds, retrieved); + } + + @Test + public void testGetAssignmentsForNonExistentNode() throws Exception { + List retrieved = assignStore.getAssignments(node1); + Assert.assertNotNull("Should return empty list", retrieved); + Assert.assertTrue("Should return empty list", retrieved.isEmpty()); + } + + @Test + public void testUpdateAssignments() throws Exception { + List initialBuckets = Arrays.asList("1", "2", "3"); + List updatedBuckets = Arrays.asList("4", "5", "6", "7"); + + // Save initial assignments + assignStore.saveAssignments(node1, initialBuckets); + Assert.assertEquals(initialBuckets, assignStore.getAssignments(node1)); + + // Update assignments + assignStore.saveAssignments(node1, updatedBuckets); + Assert.assertEquals(updatedBuckets, assignStore.getAssignments(node1)); + } + + @Test + public void testRemoveAssignments() throws Exception { + List bucketIds = Arrays.asList("1", "2", "3"); + + // Save assignments + assignStore.saveAssignments(node1, bucketIds); + Assert.assertFalse("Should have assignments", assignStore.getAssignments(node1).isEmpty()); + + // Remove assignments + assignStore.removeAssignments(node1); + Assert.assertTrue("Should be empty after removal", assignStore.getAssignments(node1).isEmpty()); + } + + @Test + public void testGetAllAssignments() throws Exception { + List buckets1 = Arrays.asList("1", "2", "3"); + List buckets2 = Arrays.asList("4", "5", "6"); + + // Save assignments for multiple nodes + assignStore.saveAssignments(node1, buckets1); + assignStore.saveAssignments(node2, buckets2); + + // Get all assignments + Map> allAssignments = assignStore.getAllAssignments(); + Assert.assertEquals("Should have 2 nodes", 2, allAssignments.size()); + + // Find nodes by host and port since parseNodeKey doesn't set restBindPort + List foundBuckets1 = null; + List foundBuckets2 = null; + for (Map.Entry> entry : allAssignments.entrySet()) { + AmsServerInfo node = entry.getKey(); + if (node1.getHost().equals(node.getHost()) + && node1.getThriftBindPort().equals(node.getThriftBindPort())) { + foundBuckets1 = entry.getValue(); + } else if (node2.getHost().equals(node.getHost()) + && node2.getThriftBindPort().equals(node.getThriftBindPort())) { + foundBuckets2 = entry.getValue(); + } + } + Assert.assertEquals(buckets1, foundBuckets1); + Assert.assertEquals(buckets2, foundBuckets2); + } + + @Test + public void testGetAllAssignmentsEmpty() throws Exception { + Map> allAssignments = assignStore.getAllAssignments(); + Assert.assertNotNull("Should return empty map", allAssignments); + Assert.assertTrue("Should be empty", allAssignments.isEmpty()); + } + + @Test + public void testLastUpdateTime() throws Exception { + List bucketIds = Arrays.asList("1", "2", "3"); + + // Initially no update time + long initialTime = assignStore.getLastUpdateTime(node1); + Assert.assertEquals("Should be 0 initially", 0, initialTime); + + // Save assignments (should update time) + long beforeSave = System.currentTimeMillis(); + assignStore.saveAssignments(node1, bucketIds); + long afterSave = System.currentTimeMillis(); + + long updateTime = assignStore.getLastUpdateTime(node1); + Assert.assertTrue( + "Update time should be between before and after", + updateTime >= beforeSave && updateTime <= afterSave); + + // Manually update time + Thread.sleep(10); + assignStore.updateLastUpdateTime(node1); + long newUpdateTime = assignStore.getLastUpdateTime(node1); + Assert.assertTrue("New update time should be later", newUpdateTime > updateTime); + } + + @Test + public void testEmptyBucketList() throws Exception { + List emptyList = new ArrayList<>(); + assignStore.saveAssignments(node1, emptyList); + List retrieved = assignStore.getAssignments(node1); + Assert.assertNotNull("Should return empty list", retrieved); + Assert.assertTrue("Should be empty", retrieved.isEmpty()); + } + + @Test + public void testMultipleNodesWithSameHostDifferentPort() throws Exception { + AmsServerInfo node3 = new AmsServerInfo(); + node3.setHost("127.0.0.1"); + node3.setThriftBindPort(1262); + node3.setRestBindPort(1632); + + List buckets1 = Arrays.asList("1", "2"); + List buckets3 = Arrays.asList("3", "4"); + + assignStore.saveAssignments(node1, buckets1); + assignStore.saveAssignments(node3, buckets3); + + Assert.assertEquals(buckets1, assignStore.getAssignments(node1)); + Assert.assertEquals(buckets3, assignStore.getAssignments(node3)); + + Map> allAssignments = assignStore.getAllAssignments(); + Assert.assertEquals("Should have 2 nodes", 2, allAssignments.size()); + } + + /** Create a mock CuratorFramework that uses MockZkState for storage. */ + @SuppressWarnings("unchecked") + private CuratorFramework createMockZkClient() throws Exception { + CuratorFramework mockClient = mock(CuratorFramework.class); + + // Mock getChildren() + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.GetChildrenBuilder + getChildrenBuilder = + mock( + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api + .GetChildrenBuilder.class); + when(mockClient.getChildren()).thenReturn(getChildrenBuilder); + when(getChildrenBuilder.forPath(anyString())) + .thenAnswer( + invocation -> { + String path = invocation.getArgument(0); + return mockZkState.getChildren(path); + }); + + // Mock getData() + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.GetDataBuilder + getDataBuilder = + mock( + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.GetDataBuilder + .class); + when(mockClient.getData()).thenReturn(getDataBuilder); + when(getDataBuilder.forPath(anyString())) + .thenAnswer( + invocation -> { + String path = invocation.getArgument(0); + return mockZkState.getData(path); + }); + + // Mock create() - manually create the entire fluent API chain + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.CreateBuilder createBuilder = + mock( + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.CreateBuilder.class); + + @SuppressWarnings("unchecked") + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api + .ProtectACLCreateModeStatPathAndBytesable< + String> + pathAndBytesable = + mock( + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api + .ProtectACLCreateModeStatPathAndBytesable.class); + + when(mockClient.create()).thenReturn(createBuilder); + when(createBuilder.creatingParentsIfNeeded()).thenReturn(pathAndBytesable); + when(pathAndBytesable.withMode(any(CreateMode.class))).thenReturn(pathAndBytesable); + + // Mock forPath(path, data) - used by saveAssignments() + when(pathAndBytesable.forPath(anyString(), any(byte[].class))) + .thenAnswer( + invocation -> { + String path = invocation.getArgument(0); + byte[] data = invocation.getArgument(1); + return mockZkState.createNode(path, data); + }); + + // Mock forPath(path) - used by createPathIfNeeded() + when(pathAndBytesable.forPath(anyString())) + .thenAnswer( + invocation -> { + String path = invocation.getArgument(0); + if (mockZkState.exists(path) == null) { + mockZkState.createNode(path, new byte[0]); + } + return null; + }); + + // Mock setData() + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.SetDataBuilder + setDataBuilder = + mock( + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.SetDataBuilder + .class); + when(mockClient.setData()).thenReturn(setDataBuilder); + when(setDataBuilder.forPath(anyString(), any(byte[].class))) + .thenAnswer( + invocation -> { + String path = invocation.getArgument(0); + byte[] data = invocation.getArgument(1); + mockZkState.setData(path, data); + return null; + }); + + // Mock delete() + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.DeleteBuilder deleteBuilder = + mock( + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.DeleteBuilder.class); + when(mockClient.delete()).thenReturn(deleteBuilder); + doAnswer( + invocation -> { + String path = invocation.getArgument(0); + mockZkState.deleteNode(path); + return null; + }) + .when(deleteBuilder) + .forPath(anyString()); + + // Mock deletingChildrenIfNeeded() + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.ChildrenDeletable + childrenDeletable = + mock( + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.ChildrenDeletable + .class); + when(deleteBuilder.deletingChildrenIfNeeded()).thenReturn(childrenDeletable); + doAnswer( + invocation -> { + String path = invocation.getArgument(0); + mockZkState.deleteNodeRecursive(path); + return null; + }) + .when(childrenDeletable) + .forPath(anyString()); + + // Mock checkExists() + @SuppressWarnings("unchecked") + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.ExistsBuilder + checkExistsBuilder = + mock( + org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.api.ExistsBuilder + .class); + when(mockClient.checkExists()).thenReturn(checkExistsBuilder); + when(checkExistsBuilder.forPath(anyString())) + .thenAnswer( + invocation -> { + String path = invocation.getArgument(0); + return mockZkState.exists(path); + }); + + // Mock start() and close() + doAnswer(invocation -> null).when(mockClient).start(); + doAnswer(invocation -> null).when(mockClient).close(); + + return mockClient; + } + + /** Create a mock LeaderLatch with specified leadership status. */ + private LeaderLatch createMockLeaderLatch(boolean hasLeadership) throws Exception { + LeaderLatch mockLatch = mock(LeaderLatch.class); + when(mockLatch.hasLeadership()).thenReturn(hasLeadership); + doAnswer(invocation -> null).when(mockLatch).addListener(any()); + doAnswer(invocation -> null).when(mockLatch).start(); + doAnswer(invocation -> null).when(mockLatch).close(); + doAnswer( + invocation -> { + // Mock implementation - doesn't actually wait + return null; + }) + .when(mockLatch) + .await(); + return mockLatch; + } + + /** In-memory ZK state simulator. */ + private static class MockZkState { + private final Map nodes = new HashMap<>(); + private final AtomicInteger sequenceCounter = new AtomicInteger(0); + + public List getChildren(String path) throws KeeperException { + List children = new ArrayList<>(); + String prefix = path.endsWith("/") ? path : path + "/"; + for (String nodePath : nodes.keySet()) { + if (nodePath.startsWith(prefix) && !nodePath.equals(path)) { + String relativePath = nodePath.substring(prefix.length()); + if (!relativePath.contains("/")) { + children.add(relativePath); + } + } + } + children.sort(String::compareTo); + return children; + } + + public byte[] getData(String path) throws KeeperException { + byte[] data = nodes.get(path); + if (data == null) { + throw new KeeperException.NoNodeException(path); + } + return data; + } + + public void setData(String path, byte[] data) throws KeeperException { + if (!nodes.containsKey(path)) { + throw new KeeperException.NoNodeException(path); + } + nodes.put(path, data); + } + + public String createNode(String path, byte[] data) { + // Handle sequential nodes + if (path.endsWith("-")) { + int seq = sequenceCounter.incrementAndGet(); + path = path + String.format("%010d", seq); + } + // Create parent paths if they don't exist (simulating creatingParentsIfNeeded) + createParentPaths(path); + nodes.put(path, data); + return path; + } + + private void createParentPaths(String path) { + // Create all parent paths as empty nodes + // Handle absolute paths (starting with "/") + boolean isAbsolute = path.startsWith("/"); + String[] parts = path.split("/"); + StringBuilder currentPath = new StringBuilder(); + if (isAbsolute) { + currentPath.append("/"); + } + for (int i = 0; i < parts.length - 1; i++) { + if (parts[i].isEmpty()) { + continue; // Skip empty parts from split + } + if (currentPath.length() > 0 && !currentPath.toString().endsWith("/")) { + currentPath.append("/"); + } + currentPath.append(parts[i]); + String parentPath = currentPath.toString(); + // Only create if it doesn't exist + if (!nodes.containsKey(parentPath)) { + nodes.put(parentPath, new byte[0]); + } + } + } + + public void deleteNode(String path) throws KeeperException { + if (!nodes.containsKey(path)) { + throw new KeeperException.NoNodeException(path); + } + nodes.remove(path); + } + + public void deleteNodeRecursive(String path) throws KeeperException { + // Delete the node and all its children + List toDelete = new ArrayList<>(); + String prefix = path.endsWith("/") ? path : path + "/"; + for (String nodePath : nodes.keySet()) { + if (nodePath.equals(path) || nodePath.startsWith(prefix)) { + toDelete.add(nodePath); + } + } + for (String nodePath : toDelete) { + nodes.remove(nodePath); + } + } + + public Stat exists(String path) { + return nodes.containsKey(path) ? new Stat() : null; + } + + public void clear() { + nodes.clear(); + sequenceCounter.set(0); + } + } +} diff --git a/amoro-common/src/main/java/org/apache/amoro/properties/AmsHAProperties.java b/amoro-common/src/main/java/org/apache/amoro/properties/AmsHAProperties.java index 08b9ef04a4..5e4f0a8d84 100644 --- a/amoro-common/src/main/java/org/apache/amoro/properties/AmsHAProperties.java +++ b/amoro-common/src/main/java/org/apache/amoro/properties/AmsHAProperties.java @@ -26,6 +26,7 @@ public class AmsHAProperties { private static final String TABLE_SERVICE_MASTER_PATH = "/master"; private static final String OPTIMIZING_SERVICE_MASTER_PATH = "/optimizing-service-master"; private static final String NODES_PATH = "/nodes"; + private static final String BUCKET_ASSIGNMENTS_PATH = "/bucket-assignments"; private static final String NAMESPACE_DEFAULT = "default"; private static String getBasePath(String namespace) { @@ -50,4 +51,8 @@ public static String getLeaderPath(String namespace) { public static String getNodesPath(String namespace) { return getBasePath(namespace) + NODES_PATH; } + + public static String getBucketAssignmentsPath(String namespace) { + return getBasePath(namespace) + BUCKET_ASSIGNMENTS_PATH; + } } From 80ba8f2cae6fc2ea81ef46edd43aed27d2401b97 Mon Sep 17 00:00:00 2001 From: wardli Date: Mon, 10 Nov 2025 19:29:50 +0800 Subject: [PATCH 09/15] [Subtask]: add AmsAssignService to implement balanced bucket allocation in master-slave mode. #3921 --- .../apache/amoro/server/HighAvailabilityContainer.java | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/HighAvailabilityContainer.java b/amoro-ams/src/main/java/org/apache/amoro/server/HighAvailabilityContainer.java index 07624d0b57..a7be66ccb1 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/HighAvailabilityContainer.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/HighAvailabilityContainer.java @@ -45,16 +45,6 @@ public class HighAvailabilityContainer implements LeaderLatchListener { private final LeaderLatch leaderLatch; private final CuratorFramework zkClient; - - // Package-private accessors for testing - CuratorFramework getZkClient() { - return zkClient; - } - - LeaderLatch getLeaderLatch() { - return leaderLatch; - } - private final String tableServiceMasterPath; private final String optimizingServiceMasterPath; private final String nodesPath; From 0c91452b35504f24c9c7964881bfedee1704bdb5 Mon Sep 17 00:00:00 2001 From: wardli Date: Tue, 11 Nov 2025 10:50:37 +0800 Subject: [PATCH 10/15] [Subtask]: add AmsAssignService to implement balanced bucket allocation in master-slave mode. #3921 --- .../amoro/server/AmoroManagementConf.java | 7 +++++++ .../apache/amoro/server/AmsAssignService.java | 14 +++++++------- .../amoro/server/ZkBucketAssignStore.java | 17 +---------------- .../amoro/server/TestAmsAssignService.java | 3 +-- .../amoro/server/TestZkBucketAssignStore.java | 4 +--- 5 files changed, 17 insertions(+), 28 deletions(-) diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/AmoroManagementConf.java b/amoro-ams/src/main/java/org/apache/amoro/server/AmoroManagementConf.java index 2304babc9e..c76eeded5f 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/AmoroManagementConf.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/AmoroManagementConf.java @@ -73,6 +73,13 @@ public class AmoroManagementConf { .withDescription( "Timeout duration to determine if a node is offline. After this duration, the node's bucket IDs will be reassigned."); + public static final ConfigOption ASSIGN_INTERVAL = + ConfigOptions.key("bucket-assign.interval") + .durationType() + .defaultValue(Duration.ofSeconds(60)) + .withDescription( + "Interval for bucket assignment service to detect node changes and redistribute bucket IDs."); + public static final ConfigOption CATALOG_META_CACHE_EXPIRATION_INTERVAL = ConfigOptions.key("catalog-meta-cache.expiration-interval") .durationType() diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/AmsAssignService.java b/amoro-ams/src/main/java/org/apache/amoro/server/AmsAssignService.java index a7c8241b0a..1bd9d5aa4d 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/AmsAssignService.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/AmsAssignService.java @@ -22,7 +22,6 @@ import org.apache.amoro.config.Configurations; import org.apache.amoro.shade.guava32.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.CuratorFramework; -import org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.recipes.leader.LeaderLatch; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -42,7 +41,6 @@ public class AmsAssignService { private static final Logger LOG = LoggerFactory.getLogger(AmsAssignService.class); - private static final long ASSIGN_INTERVAL_SECONDS = 30; private final ScheduledExecutorService assignScheduler = Executors.newSingleThreadScheduledExecutor( @@ -56,6 +54,7 @@ public class AmsAssignService { private final Configurations serviceConfig; private final int bucketIdTotalCount; private final long nodeOfflineTimeoutMs; + private final long assignIntervalSeconds; private volatile boolean running = false; // Package-private accessors for testing @@ -74,15 +73,16 @@ void doAssignForTest() { public AmsAssignService( HighAvailabilityContainer haContainer, Configurations serviceConfig, - CuratorFramework zkClient, - LeaderLatch leaderLatch) { + CuratorFramework zkClient) { this.haContainer = haContainer; this.serviceConfig = serviceConfig; this.bucketIdTotalCount = serviceConfig.getInteger(AmoroManagementConf.BUCKET_ID_TOTAL_COUNT); this.nodeOfflineTimeoutMs = serviceConfig.get(AmoroManagementConf.NODE_OFFLINE_TIMEOUT).toMillis(); + this.assignIntervalSeconds = + serviceConfig.get(AmoroManagementConf.ASSIGN_INTERVAL).getSeconds(); String clusterName = serviceConfig.getString(AmoroManagementConf.HA_CLUSTER_NAME); - this.assignStore = new ZkBucketAssignStore(zkClient, clusterName, leaderLatch); + this.assignStore = new ZkBucketAssignStore(zkClient, clusterName); } /** @@ -99,8 +99,8 @@ public void start() { } running = true; assignScheduler.scheduleWithFixedDelay( - this::doAssign, 10, ASSIGN_INTERVAL_SECONDS, TimeUnit.SECONDS); - LOG.info("Bucket assignment service started"); + this::doAssign, 10, assignIntervalSeconds, TimeUnit.SECONDS); + LOG.info("Bucket assignment service started with interval: {} seconds", assignIntervalSeconds); } /** Stop the assignment service. */ diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/ZkBucketAssignStore.java b/amoro-ams/src/main/java/org/apache/amoro/server/ZkBucketAssignStore.java index 40c1db8844..e4ae304386 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/ZkBucketAssignStore.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/ZkBucketAssignStore.java @@ -23,7 +23,6 @@ import org.apache.amoro.shade.jackson2.com.fasterxml.jackson.core.type.TypeReference; import org.apache.amoro.shade.jackson2.com.fasterxml.jackson.databind.ObjectMapper; import org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.CuratorFramework; -import org.apache.amoro.shade.zookeeper3.org.apache.curator.framework.recipes.leader.LeaderLatch; import org.apache.amoro.shade.zookeeper3.org.apache.zookeeper.CreateMode; import org.apache.amoro.shade.zookeeper3.org.apache.zookeeper.KeeperException; import org.apache.amoro.utils.JacksonUtil; @@ -52,13 +51,10 @@ public class ZkBucketAssignStore implements BucketAssignStore { private final CuratorFramework zkClient; private final String assignmentsBasePath; - private final LeaderLatch leaderLatch; - public ZkBucketAssignStore( - CuratorFramework zkClient, String clusterName, LeaderLatch leaderLatch) { + public ZkBucketAssignStore(CuratorFramework zkClient, String clusterName) { this.zkClient = zkClient; this.assignmentsBasePath = AmsHAProperties.getBucketAssignmentsPath(clusterName); - this.leaderLatch = leaderLatch; try { createPathIfNeeded(assignmentsBasePath); } catch (Exception e) { @@ -69,10 +65,6 @@ public ZkBucketAssignStore( @Override public void saveAssignments(AmsServerInfo nodeInfo, List bucketIds) throws Exception { - if (!leaderLatch.hasLeadership()) { - LOG.warn("Only leader node can save bucket assignments"); - return; - } String nodeKey = getNodeKey(nodeInfo); String assignmentsPath = assignmentsBasePath + "/" + nodeKey + ASSIGNMENTS_SUFFIX; String assignmentsJson = JacksonUtil.toJSONString(bucketIds); @@ -120,10 +112,6 @@ public List getAssignments(AmsServerInfo nodeInfo) throws Exception { @Override public void removeAssignments(AmsServerInfo nodeInfo) throws Exception { - if (!leaderLatch.hasLeadership()) { - LOG.warn("Only leader node can remove bucket assignments"); - return; - } String nodeKey = getNodeKey(nodeInfo); String nodePath = assignmentsBasePath + "/" + nodeKey; try { @@ -190,9 +178,6 @@ public long getLastUpdateTime(AmsServerInfo nodeInfo) throws Exception { @Override public void updateLastUpdateTime(AmsServerInfo nodeInfo) throws Exception { - if (!leaderLatch.hasLeadership()) { - return; - } String nodeKey = getNodeKey(nodeInfo); String timePath = assignmentsBasePath + "/" + nodeKey + LAST_UPDATE_TIME_SUFFIX; long currentTime = System.currentTimeMillis(); diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/TestAmsAssignService.java b/amoro-ams/src/test/java/org/apache/amoro/server/TestAmsAssignService.java index 758d568dfc..00f4a32f6f 100644 --- a/amoro-ams/src/test/java/org/apache/amoro/server/TestAmsAssignService.java +++ b/amoro-ams/src/test/java/org/apache/amoro/server/TestAmsAssignService.java @@ -542,8 +542,7 @@ private AmsAssignService createAssignServiceWithMockStore() throws Exception { /** Create AmsAssignService with mock BucketAssignStore. */ private AmsAssignService createAssignServiceWithMockStore(HighAvailabilityContainer container) throws Exception { - AmsAssignService service = - new AmsAssignService(container, serviceConfig, mockZkClient, mockLeaderLatch); + AmsAssignService service = new AmsAssignService(container, serviceConfig, mockZkClient); // Use reflection to inject mock assign store java.lang.reflect.Field assignStoreField = diff --git a/amoro-ams/src/test/java/org/apache/amoro/server/TestZkBucketAssignStore.java b/amoro-ams/src/test/java/org/apache/amoro/server/TestZkBucketAssignStore.java index 1d6e81a770..a331001edc 100644 --- a/amoro-ams/src/test/java/org/apache/amoro/server/TestZkBucketAssignStore.java +++ b/amoro-ams/src/test/java/org/apache/amoro/server/TestZkBucketAssignStore.java @@ -46,7 +46,6 @@ public class TestZkBucketAssignStore { private CuratorFramework mockZkClient; - private LeaderLatch mockLeaderLatch; private ZkBucketAssignStore assignStore; private AmsServerInfo node1; private AmsServerInfo node2; @@ -56,9 +55,8 @@ public class TestZkBucketAssignStore { public void setUp() throws Exception { mockZkState = new MockZkState(); mockZkClient = createMockZkClient(); - mockLeaderLatch = createMockLeaderLatch(true); // Is leader by default - assignStore = new ZkBucketAssignStore(mockZkClient, "test-cluster", mockLeaderLatch); + assignStore = new ZkBucketAssignStore(mockZkClient, "test-cluster"); node1 = new AmsServerInfo(); node1.setHost("127.0.0.1"); From f2ecc066d15e02428df38c28288b160210dfb4bb Mon Sep 17 00:00:00 2001 From: wardli Date: Wed, 12 Nov 2025 16:45:22 +0800 Subject: [PATCH 11/15] [Subtask]: add AmsAssignService to implement balanced bucket allocation in master-slave mode. #3921 --- .../amoro/server/AmoroServiceContainer.java | 18 ++++ .../apache/amoro/server/AmsAssignService.java | 82 ++++++++++++++++--- .../server/HighAvailabilityContainer.java | 18 ++++ 3 files changed, 108 insertions(+), 10 deletions(-) diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/AmoroServiceContainer.java b/amoro-ams/src/main/java/org/apache/amoro/server/AmoroServiceContainer.java index ba9f959f57..5a3ed1f6cc 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/AmoroServiceContainer.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/AmoroServiceContainer.java @@ -113,6 +113,7 @@ public class AmoroServiceContainer { private TServer optimizingServiceServer; private Javalin httpServer; private AmsServiceMetrics amsServiceMetrics; + private AmsAssignService amsAssignService; public AmoroServiceContainer() throws Exception { initConfig(); @@ -185,6 +186,18 @@ public void startOptimizingService() throws Exception { TableRuntimeFactoryManager tableRuntimeFactoryManager = new TableRuntimeFactoryManager(); tableRuntimeFactoryManager.initialize(); + // In master-slave mode, create BucketAssignStore and AmsAssignService + BucketAssignStore bucketAssignStore = null; + if (IS_MASTER_SLAVE_MODE && haContainer != null && haContainer.getZkClient() != null) { + String clusterName = serviceConfig.getString(AmoroManagementConf.HA_CLUSTER_NAME); + bucketAssignStore = new ZkBucketAssignStore(haContainer.getZkClient(), clusterName); + // Create and start AmsAssignService for bucket assignment + amsAssignService = + new AmsAssignService(haContainer, serviceConfig, haContainer.getZkClient()); + amsAssignService.start(); + LOG.info("AmsAssignService started for master-slave mode"); + } + tableService = new DefaultTableService(serviceConfig, catalogManager, tableRuntimeFactoryManager); @@ -228,6 +241,11 @@ public void disposeOptimizingService() { LOG.info("Stopping optimizing server[serving:{}] ...", optimizingServiceServer.isServing()); optimizingServiceServer.stop(); } + if (amsAssignService != null) { + LOG.info("Stopping AmsAssignService..."); + amsAssignService.stop(); + amsAssignService = null; + } if (tableService != null) { LOG.info("Stopping table service..."); tableService.dispose(); diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/AmsAssignService.java b/amoro-ams/src/main/java/org/apache/amoro/server/AmsAssignService.java index 1bd9d5aa4d..4049bcd1a3 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/AmsAssignService.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/AmsAssignService.java @@ -135,23 +135,66 @@ private void doAssign() { } Map> currentAssignments = assignStore.getAllAssignments(); - Set currentAssignedNodes = new HashSet<>(currentAssignments.keySet()); + + // Create a mapping from stored nodes (may have null restBindPort) to alive nodes (complete + // info) + // Use host:thriftBindPort as the key for matching + Map 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> normalizedAssignments = new java.util.HashMap<>(); + Set currentAssignedNodes = new HashSet<>(); + for (Map.Entry> entry : currentAssignments.entrySet()) { + AmsServerInfo storedNode = entry.getKey(); + String nodeKey = getNodeKey(storedNode); + AmsServerInfo aliveNode = aliveNodeMap.get(nodeKey); + if (aliveNode != null) { + // Node is alive, use the complete node info from aliveNodes + normalizedAssignments.put(aliveNode, entry.getValue()); + currentAssignedNodes.add(aliveNode); + } else { + // Node is not in alive list, keep the stored node info for offline detection + normalizedAssignments.put(storedNode, entry.getValue()); + currentAssignedNodes.add(storedNode); + } + } + Set aliveNodeSet = new HashSet<>(aliveNodes); // Detect new nodes and offline nodes Set newNodes = new HashSet<>(aliveNodeSet); newNodes.removeAll(currentAssignedNodes); - Set offlineNodes = new HashSet<>(currentAssignedNodes); - offlineNodes.removeAll(aliveNodeSet); + Set offlineNodes = new HashSet<>(); + for (AmsServerInfo storedNode : currentAssignments.keySet()) { + String nodeKey = getNodeKey(storedNode); + if (!aliveNodeMap.containsKey(nodeKey)) { + offlineNodes.add(storedNode); + } + } // Check for nodes that haven't updated for a long time long currentTime = System.currentTimeMillis(); + Set aliveNodeKeys = new HashSet<>(); + for (AmsServerInfo node : aliveNodes) { + aliveNodeKeys.add(getNodeKey(node)); + } for (AmsServerInfo node : currentAssignedNodes) { - if (aliveNodeSet.contains(node)) { + String nodeKey = getNodeKey(node); + if (aliveNodeKeys.contains(nodeKey)) { long lastUpdateTime = assignStore.getLastUpdateTime(node); if (lastUpdateTime > 0 && (currentTime - lastUpdateTime) > nodeOfflineTimeoutMs) { - offlineNodes.add(node); + // Find the stored node for this alive node to add to offlineNodes + for (AmsServerInfo storedNode : currentAssignments.keySet()) { + if (getNodeKey(storedNode).equals(nodeKey)) { + offlineNodes.add(storedNode); + break; + } + } LOG.warn( "Node {} is considered offline due to timeout. Last update: {}", node, @@ -196,13 +239,24 @@ private void doAssign() { // Step 3: Incremental reassignment // Keep existing assignments for nodes that are still alive Map> newAssignments = new java.util.HashMap<>(); + Set offlineNodeKeys = new HashSet<>(); + for (AmsServerInfo offlineNode : offlineNodes) { + offlineNodeKeys.add(getNodeKey(offlineNode)); + } for (AmsServerInfo node : aliveNodes) { - List existingBuckets = currentAssignments.get(node); - if (existingBuckets != null && !offlineNodes.contains(node)) { - // Keep existing buckets for alive nodes (not offline) - newAssignments.put(node, new ArrayList<>(existingBuckets)); + String nodeKey = getNodeKey(node); + if (!offlineNodeKeys.contains(nodeKey)) { + // Node is alive and not offline, check if it has existing assignments + List existingBuckets = normalizedAssignments.get(node); + if (existingBuckets != null && !existingBuckets.isEmpty()) { + // Keep existing buckets for alive nodes (not offline) + newAssignments.put(node, new ArrayList<>(existingBuckets)); + } else { + // New node + newAssignments.put(node, new ArrayList<>()); + } } else { - // New node or node that was offline + // Node was offline, start with empty assignment newAssignments.put(node, new ArrayList<>()); } } @@ -400,4 +454,12 @@ private List generateBucketIds() { } return bucketIds; } + + /** + * Get node key for matching nodes. Uses host:thriftBindPort format, consistent with + * ZkBucketAssignStore.getNodeKey(). + */ + private String getNodeKey(AmsServerInfo nodeInfo) { + return nodeInfo.getHost() + ":" + nodeInfo.getThriftBindPort(); + } } diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/HighAvailabilityContainer.java b/amoro-ams/src/main/java/org/apache/amoro/server/HighAvailabilityContainer.java index a7be66ccb1..2d1cb1a977 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/HighAvailabilityContainer.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/HighAvailabilityContainer.java @@ -266,6 +266,24 @@ public boolean hasLeadership() { return leaderLatch.hasLeadership(); } + /** + * Get the current node's table service server info. + * + * @return The current node's server info, null if HA is not enabled + */ + public AmsServerInfo getTableServiceServerInfo() { + return tableServiceServerInfo; + } + + /** + * Get the ZooKeeper client. This is used for creating BucketAssignStore. + * + * @return The ZooKeeper client, null if HA is not enabled + */ + public CuratorFramework getZkClient() { + return zkClient; + } + private void createPathIfNeeded(String path) throws Exception { try { zkClient.create().creatingParentsIfNeeded().withMode(CreateMode.PERSISTENT).forPath(path); From b1da165a0c4010666e65779fe2cfe77bc8b8d6f3 Mon Sep 17 00:00:00 2001 From: wardli Date: Wed, 12 Nov 2025 17:17:55 +0800 Subject: [PATCH 12/15] [Subtask]: Modify DefaultTableService to be compatible with master-slave mode #3923 --- .../amoro/server/AmoroManagementConf.java | 7 + .../amoro/server/AmoroServiceContainer.java | 7 +- .../server/persistence/BucketIdCount.java | 41 ++ .../mapper/TableRuntimeMapper.java | 31 ++ .../server/table/DefaultTableService.java | 465 +++++++++++++++++- .../src/main/resources/mysql/upgrade.sql | 15 + .../src/main/resources/postgres/upgrade.sql | 9 +- .../apache/amoro/ServerTableIdentifier.java | 1 + 8 files changed, 571 insertions(+), 5 deletions(-) create mode 100644 amoro-ams/src/main/java/org/apache/amoro/server/persistence/BucketIdCount.java diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/AmoroManagementConf.java b/amoro-ams/src/main/java/org/apache/amoro/server/AmoroManagementConf.java index c76eeded5f..3856b16d55 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/AmoroManagementConf.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/AmoroManagementConf.java @@ -80,6 +80,13 @@ public class AmoroManagementConf { .withDescription( "Interval for bucket assignment service to detect node changes and redistribute bucket IDs."); + public static final ConfigOption BUCKET_TABLE_SYNC_INTERVAL = + ConfigOptions.key("bucket-table-sync.interval") + .durationType() + .defaultValue(Duration.ofSeconds(60)) + .withDescription( + "Interval for syncing tables assigned to bucket IDs in master-slave mode. Each node periodically loads tables from database based on its assigned bucket IDs."); + public static final ConfigOption CATALOG_META_CACHE_EXPIRATION_INTERVAL = ConfigOptions.key("catalog-meta-cache.expiration-interval") .durationType() diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/AmoroServiceContainer.java b/amoro-ams/src/main/java/org/apache/amoro/server/AmoroServiceContainer.java index 5a3ed1f6cc..1605f11b06 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/AmoroServiceContainer.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/AmoroServiceContainer.java @@ -199,7 +199,12 @@ public void startOptimizingService() throws Exception { } tableService = - new DefaultTableService(serviceConfig, catalogManager, tableRuntimeFactoryManager); + new DefaultTableService( + serviceConfig, + catalogManager, + tableRuntimeFactoryManager, + haContainer, + bucketAssignStore); optimizingService = new DefaultOptimizingService(serviceConfig, catalogManager, optimizerManager, tableService); diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/persistence/BucketIdCount.java b/amoro-ams/src/main/java/org/apache/amoro/server/persistence/BucketIdCount.java new file mode 100644 index 0000000000..6441b67fd8 --- /dev/null +++ b/amoro-ams/src/main/java/org/apache/amoro/server/persistence/BucketIdCount.java @@ -0,0 +1,41 @@ +/* + * 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.persistence; + +/** Simple class to hold bucketId and its table count. */ +public class BucketIdCount { + private String bucketId; + private Integer count; + + public String getBucketId() { + return bucketId; + } + + public void setBucketId(String bucketId) { + this.bucketId = bucketId; + } + + public Integer getCount() { + return count; + } + + public void setCount(Integer count) { + this.count = count; + } +} diff --git a/amoro-ams/src/main/java/org/apache/amoro/server/persistence/mapper/TableRuntimeMapper.java b/amoro-ams/src/main/java/org/apache/amoro/server/persistence/mapper/TableRuntimeMapper.java index 3cfb69f2e9..2ea20d146e 100644 --- a/amoro-ams/src/main/java/org/apache/amoro/server/persistence/mapper/TableRuntimeMapper.java +++ b/amoro-ams/src/main/java/org/apache/amoro/server/persistence/mapper/TableRuntimeMapper.java @@ -18,6 +18,7 @@ package org.apache.amoro.server.persistence.mapper; +import org.apache.amoro.server.persistence.BucketIdCount; import org.apache.amoro.server.persistence.TableRuntimeMeta; import org.apache.amoro.server.persistence.TableRuntimeState; import org.apache.ibatis.annotations.Delete; @@ -102,6 +103,21 @@ public interface TableRuntimeMapper { @ResultMap("tableRuntimeMeta") List selectAllRuntimes(); + @Select( + "") + @ResultMap("tableRuntimeMeta") + List selectRuntimesByBucketIds(@Param("bucketIds") List bucketIds); + @Select( "