001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.master;
019
020import static org.junit.Assert.assertEquals;
021import static org.junit.Assert.assertFalse;
022import static org.junit.Assert.assertNotNull;
023import static org.junit.Assert.assertTrue;
024import static org.mockito.ArgumentMatchers.any;
025import static org.mockito.Mockito.when;
026
027import java.io.IOException;
028import java.io.InterruptedIOException;
029import java.util.ArrayList;
030import java.util.List;
031import java.util.concurrent.Semaphore;
032import org.apache.hadoop.conf.Configuration;
033import org.apache.hadoop.hbase.HBaseClassTestRule;
034import org.apache.hadoop.hbase.HBaseTestingUtil;
035import org.apache.hadoop.hbase.ServerName;
036import org.apache.hadoop.hbase.keymeta.KeyManagementService;
037import org.apache.hadoop.hbase.monitoring.MonitoredTask;
038import org.apache.hadoop.hbase.monitoring.TaskGroup;
039import org.apache.hadoop.hbase.testclassification.MasterTests;
040import org.apache.hadoop.hbase.testclassification.MediumTests;
041import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
042import org.apache.hadoop.hbase.util.MockServer;
043import org.apache.hadoop.hbase.zookeeper.ClusterStatusTracker;
044import org.apache.hadoop.hbase.zookeeper.MasterAddressTracker;
045import org.apache.hadoop.hbase.zookeeper.ZKListener;
046import org.apache.hadoop.hbase.zookeeper.ZKUtil;
047import org.apache.hadoop.hbase.zookeeper.ZKWatcher;
048import org.apache.hadoop.hbase.zookeeper.ZNodePaths;
049import org.apache.zookeeper.KeeperException;
050import org.junit.AfterClass;
051import org.junit.BeforeClass;
052import org.junit.ClassRule;
053import org.junit.Test;
054import org.junit.experimental.categories.Category;
055import org.mockito.Mockito;
056import org.slf4j.Logger;
057import org.slf4j.LoggerFactory;
058
059/**
060 * Test the {@link ActiveMasterManager}.
061 */
062@Category({ MasterTests.class, MediumTests.class })
063public class TestActiveMasterManager {
064
065  @ClassRule
066  public static final HBaseClassTestRule CLASS_RULE =
067    HBaseClassTestRule.forClass(TestActiveMasterManager.class);
068
069  private final static Logger LOG = LoggerFactory.getLogger(TestActiveMasterManager.class);
070  private final static HBaseTestingUtil TEST_UTIL = new HBaseTestingUtil();
071
072  @BeforeClass
073  public static void setUpBeforeClass() throws Exception {
074    TEST_UTIL.startMiniZKCluster();
075  }
076
077  @AfterClass
078  public static void tearDownAfterClass() throws Exception {
079    TEST_UTIL.shutdownMiniZKCluster();
080  }
081
082  @Test
083  public void testRestartMaster() throws IOException, KeeperException {
084    try (ZKWatcher zk =
085      new ZKWatcher(TEST_UTIL.getConfiguration(), "testActiveMasterManagerFromZK", null, true)) {
086      try {
087        ZKUtil.deleteNode(zk, zk.getZNodePaths().masterAddressZNode);
088        ZKUtil.deleteNode(zk, zk.getZNodePaths().clusterStateZNode);
089      } catch (KeeperException.NoNodeException nne) {
090      }
091
092      // Create the master node with a dummy address
093      ServerName master = ServerName.valueOf("localhost", 1, EnvironmentEdgeManager.currentTime());
094      // Should not have a master yet
095      DummyMaster dummyMaster = new DummyMaster(zk, master);
096      ClusterStatusTracker clusterStatusTracker = dummyMaster.getClusterStatusTracker();
097      ActiveMasterManager activeMasterManager = dummyMaster.getActiveMasterManager();
098      assertFalse(activeMasterManager.clusterHasActiveMaster.get());
099      assertFalse(activeMasterManager.getActiveMasterServerName().isPresent());
100
101      // First test becoming the active master uninterrupted
102      TaskGroup status = mockTaskGroup();
103      clusterStatusTracker.setClusterUp();
104
105      activeMasterManager.blockUntilBecomingActiveMaster(100, status);
106      assertTrue(activeMasterManager.clusterHasActiveMaster.get());
107      assertMaster(zk, master);
108      assertMaster(zk, activeMasterManager.getActiveMasterServerName().get());
109
110      // Now pretend master restart
111      DummyMaster secondDummyMaster = new DummyMaster(zk, master);
112      ActiveMasterManager secondActiveMasterManager = secondDummyMaster.getActiveMasterManager();
113      assertFalse(secondActiveMasterManager.clusterHasActiveMaster.get());
114      activeMasterManager.blockUntilBecomingActiveMaster(100, status);
115      assertTrue(activeMasterManager.clusterHasActiveMaster.get());
116      assertMaster(zk, master);
117      assertMaster(zk, activeMasterManager.getActiveMasterServerName().get());
118      assertMaster(zk, secondActiveMasterManager.getActiveMasterServerName().get());
119    }
120  }
121
122  /**
123   * Unit tests that uses ZooKeeper but does not use the master-side methods but rather acts
124   * directly on ZK.
125   */
126  @Test
127  public void testActiveMasterManagerFromZK() throws Exception {
128    try (ZKWatcher zk =
129      new ZKWatcher(TEST_UTIL.getConfiguration(), "testActiveMasterManagerFromZK", null, true)) {
130      try {
131        ZKUtil.deleteNode(zk, zk.getZNodePaths().masterAddressZNode);
132        ZKUtil.deleteNode(zk, zk.getZNodePaths().clusterStateZNode);
133      } catch (KeeperException.NoNodeException nne) {
134      }
135
136      // Create the master node with a dummy address
137      ServerName firstMasterAddress =
138        ServerName.valueOf("localhost", 1, EnvironmentEdgeManager.currentTime());
139      ServerName secondMasterAddress =
140        ServerName.valueOf("localhost", 2, EnvironmentEdgeManager.currentTime());
141
142      // Should not have a master yet
143      DummyMaster ms1 = new DummyMaster(zk, firstMasterAddress);
144      ActiveMasterManager activeMasterManager = ms1.getActiveMasterManager();
145      assertFalse(activeMasterManager.clusterHasActiveMaster.get());
146      assertFalse(activeMasterManager.getActiveMasterServerName().isPresent());
147
148      // First test becoming the active master uninterrupted
149      ClusterStatusTracker clusterStatusTracker = ms1.getClusterStatusTracker();
150      clusterStatusTracker.setClusterUp();
151
152      activeMasterManager.blockUntilBecomingActiveMaster(100, mockTaskGroup());
153      assertTrue(activeMasterManager.clusterHasActiveMaster.get());
154      assertMaster(zk, firstMasterAddress);
155      assertMaster(zk, activeMasterManager.getActiveMasterServerName().get());
156
157      // New manager will now try to become the active master in another thread
158      WaitToBeMasterThread t = new WaitToBeMasterThread(zk, secondMasterAddress);
159      t.start();
160      // Wait for this guy to figure out there is another active master
161      // Wait for 1 second at most
162      int sleeps = 0;
163      while (!t.manager.clusterHasActiveMaster.get() && sleeps < 100) {
164        Thread.sleep(10);
165        sleeps++;
166      }
167
168      // Both should see that there is an active master
169      assertTrue(activeMasterManager.clusterHasActiveMaster.get());
170      assertTrue(t.manager.clusterHasActiveMaster.get());
171      // But secondary one should not be the active master
172      assertFalse(t.isActiveMaster);
173      // Verify the active master ServerName is populated in standby master.
174      assertEquals(firstMasterAddress, t.manager.getActiveMasterServerName().get());
175
176      // Close the first server and delete it's master node
177      ms1.stop("stopping first server");
178
179      // Use a listener to capture when the node is actually deleted
180      NodeDeletionListener listener =
181        new NodeDeletionListener(zk, zk.getZNodePaths().masterAddressZNode);
182      zk.registerListener(listener);
183
184      LOG.info("Deleting master node");
185      ZKUtil.deleteNode(zk, zk.getZNodePaths().masterAddressZNode);
186
187      // Wait for the node to be deleted
188      LOG.info("Waiting for active master manager to be notified");
189      listener.waitForDeletion();
190      LOG.info("Master node deleted");
191
192      // Now we expect the secondary manager to have and be the active master
193      // Wait for 1 second at most
194      sleeps = 0;
195      while (!t.isActiveMaster && sleeps < 100) {
196        Thread.sleep(10);
197        sleeps++;
198      }
199      LOG.debug("Slept " + sleeps + " times");
200
201      assertTrue(t.manager.clusterHasActiveMaster.get());
202      assertTrue(t.isActiveMaster);
203      assertEquals(secondMasterAddress, t.manager.getActiveMasterServerName().get());
204
205      LOG.info("Deleting master node");
206
207      ZKUtil.deleteNode(zk, zk.getZNodePaths().masterAddressZNode);
208    }
209  }
210
211  @Test
212  public void testBackupMasterUpdates() throws Exception {
213    Configuration conf = TEST_UTIL.getConfiguration();
214    try (ZKWatcher zk = new ZKWatcher(conf, "testBackupMasterUpdates", null, true)) {
215      ServerName sn1 = ServerName.valueOf("localhost", 1, -1);
216      DummyMaster master1 = new DummyMaster(zk, sn1);
217      ActiveMasterManager activeMasterManager = master1.getActiveMasterManager();
218      activeMasterManager.blockUntilBecomingActiveMaster(100, mockTaskGroup());
219      assertEquals(sn1, activeMasterManager.getActiveMasterServerName().get());
220      assertEquals(0, activeMasterManager.getBackupMasters().size());
221      // Add backup masters
222      List<String> backupZNodes = new ArrayList<>();
223      for (int i = 1; i <= 10; i++) {
224        ServerName backupSn = ServerName.valueOf("localhost", 1000 + i, -1);
225        String backupZn =
226          ZNodePaths.joinZNode(zk.getZNodePaths().backupMasterAddressesZNode, backupSn.toString());
227        backupZNodes.add(backupZn);
228        MasterAddressTracker.setMasterAddress(zk, backupZn, backupSn, 1234);
229        TEST_UTIL.waitFor(10000,
230          () -> activeMasterManager.getBackupMasters().size() == backupZNodes.size());
231      }
232      // Remove backup masters
233      int numBackups = backupZNodes.size();
234      for (String backupZNode : backupZNodes) {
235        ZKUtil.deleteNode(zk, backupZNode);
236        final int currentBackups = --numBackups;
237        TEST_UTIL.waitFor(10000,
238          () -> activeMasterManager.getBackupMasters().size() == currentBackups);
239      }
240    }
241  }
242
243  /**
244   * Assert there is an active master and that it has the specified address.
245   * @param zk              single Zookeeper watcher
246   * @param expectedAddress the expected address of the master
247   * @throws KeeperException unexpected Zookeeper exception
248   * @throws IOException     if an IO problem is encountered
249   */
250  private void assertMaster(ZKWatcher zk, ServerName expectedAddress)
251    throws KeeperException, IOException {
252    ServerName readAddress = MasterAddressTracker.getMasterAddress(zk);
253    assertNotNull(readAddress);
254    assertEquals(expectedAddress, readAddress);
255  }
256
257  public static class WaitToBeMasterThread extends Thread {
258
259    ActiveMasterManager manager;
260    DummyMaster dummyMaster;
261    boolean isActiveMaster;
262
263    public WaitToBeMasterThread(ZKWatcher zk, ServerName address) throws InterruptedIOException {
264      this.dummyMaster = new DummyMaster(zk, address);
265      this.manager = this.dummyMaster.getActiveMasterManager();
266      isActiveMaster = false;
267    }
268
269    @Override
270    public void run() {
271      manager.blockUntilBecomingActiveMaster(100, mockTaskGroup());
272      LOG.info("Second master has become the active master!");
273      isActiveMaster = true;
274    }
275  }
276
277  private static TaskGroup mockTaskGroup() {
278    TaskGroup taskGroup = Mockito.mock(TaskGroup.class);
279    MonitoredTask task = Mockito.mock(MonitoredTask.class);
280    when(taskGroup.addTask(any())).thenReturn(task);
281    return taskGroup;
282  }
283
284  public static class NodeDeletionListener extends ZKListener {
285    private static final Logger LOG = LoggerFactory.getLogger(NodeDeletionListener.class);
286
287    private Semaphore lock;
288    private String node;
289
290    public NodeDeletionListener(ZKWatcher watcher, String node) {
291      super(watcher);
292      lock = new Semaphore(0);
293      this.node = node;
294    }
295
296    @Override
297    public void nodeDeleted(String path) {
298      if (path.equals(node)) {
299        LOG.debug("nodeDeleted(" + path + ")");
300        lock.release();
301      }
302    }
303
304    public void waitForDeletion() throws InterruptedException {
305      lock.acquire();
306    }
307  }
308
309  /**
310   * Dummy Master Implementation.
311   */
312  public static class DummyMaster extends MockServer {
313    private ClusterStatusTracker clusterStatusTracker;
314    private ActiveMasterManager activeMasterManager;
315
316    public DummyMaster(ZKWatcher zk, ServerName master) throws InterruptedIOException {
317      this.clusterStatusTracker = new ClusterStatusTracker(zk, this);
318      clusterStatusTracker.start();
319
320      this.activeMasterManager = new ActiveMasterManager(zk, master, this);
321      zk.registerListener(activeMasterManager);
322    }
323
324    public ClusterStatusTracker getClusterStatusTracker() {
325      return clusterStatusTracker;
326    }
327
328    public ActiveMasterManager getActiveMasterManager() {
329      return activeMasterManager;
330    }
331
332    @Override
333    public KeyManagementService getKeyManagementService() {
334      return null;
335    }
336  }
337}