1   /**
2    * Copyright 2010 The Apache Software Foundation
3    *
4    * Licensed to the Apache Software Foundation (ASF) under one
5    * or more contributor license agreements.  See the NOTICE file
6    * distributed with this work for additional information
7    * regarding copyright ownership.  The ASF licenses this file
8    * to you under the Apache License, Version 2.0 (the
9    * "License"); you may not use this file except in compliance
10   * with the License.  You may obtain a copy of the License at
11   *
12   *     http://www.apache.org/licenses/LICENSE-2.0
13   *
14   * Unless required by applicable law or agreed to in writing, software
15   * distributed under the License is distributed on an "AS IS" BASIS,
16   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17   * See the License for the specific language governing permissions and
18   * limitations under the License.
19   */
20  package org.apache.hadoop.hbase.master;
21  
22  import static org.junit.Assert.assertFalse;
23  import static org.junit.Assert.assertNotNull;
24  import static org.junit.Assert.assertTrue;
25  
26  import java.io.IOException;
27  import java.util.concurrent.Semaphore;
28  
29  import org.apache.commons.logging.Log;
30  import org.apache.commons.logging.LogFactory;
31  import org.apache.hadoop.conf.Configuration;
32  import org.apache.hadoop.hbase.HBaseTestingUtility;
33  import org.apache.hadoop.hbase.Server;
34  import org.apache.hadoop.hbase.ServerName;
35  import org.apache.hadoop.hbase.catalog.CatalogTracker;
36  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
37  import org.apache.hadoop.hbase.util.Bytes;
38  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
39  import org.apache.hadoop.hbase.zookeeper.ZooKeeperListener;
40  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
41  import org.apache.zookeeper.KeeperException;
42  import org.junit.AfterClass;
43  import org.junit.BeforeClass;
44  import org.junit.Test;
45  import org.mockito.Mockito;
46  
47  /**
48   * Test the {@link ActiveMasterManager}.
49   */
50  public class TestActiveMasterManager {
51    private final static Log LOG = LogFactory.getLog(TestActiveMasterManager.class);
52    private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
53  
54    @BeforeClass
55    public static void setUpBeforeClass() throws Exception {
56      TEST_UTIL.startMiniZKCluster();
57    }
58  
59    @AfterClass
60    public static void tearDownAfterClass() throws Exception {
61      TEST_UTIL.shutdownMiniZKCluster();
62    }
63  
64    @Test public void testRestartMaster() throws IOException, KeeperException {
65      ZooKeeperWatcher zk = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
66        "testActiveMasterManagerFromZK", null);
67      ZKUtil.createAndFailSilent(zk, zk.baseZNode);
68      try {
69        ZKUtil.deleteNode(zk, zk.masterAddressZNode);
70      } catch(KeeperException.NoNodeException nne) {}
71  
72      // Create the master node with a dummy address
73      ServerName master = new ServerName("localhost", 1, System.currentTimeMillis());
74      // Should not have a master yet
75      DummyMaster dummyMaster = new DummyMaster();
76      ActiveMasterManager activeMasterManager = new ActiveMasterManager(zk,
77        master, dummyMaster);
78      zk.registerListener(activeMasterManager);
79      assertFalse(activeMasterManager.clusterHasActiveMaster.get());
80  
81      // First test becoming the active master uninterrupted
82      MonitoredTask status = Mockito.mock(MonitoredTask.class);
83      activeMasterManager.blockUntilBecomingActiveMaster(status);
84      assertTrue(activeMasterManager.clusterHasActiveMaster.get());
85      assertMaster(zk, master);
86  
87      // Now pretend master restart
88      DummyMaster secondDummyMaster = new DummyMaster();
89      ActiveMasterManager secondActiveMasterManager = new ActiveMasterManager(zk,
90        master, secondDummyMaster);
91      zk.registerListener(secondActiveMasterManager);
92      assertFalse(secondActiveMasterManager.clusterHasActiveMaster.get());
93      activeMasterManager.blockUntilBecomingActiveMaster(status);
94      assertTrue(activeMasterManager.clusterHasActiveMaster.get());
95      assertMaster(zk, master);
96    }
97  
98    /**
99     * Unit tests that uses ZooKeeper but does not use the master-side methods
100    * but rather acts directly on ZK.
101    * @throws Exception
102    */
103   @Test
104   public void testActiveMasterManagerFromZK() throws Exception {
105     ZooKeeperWatcher zk = new ZooKeeperWatcher(TEST_UTIL.getConfiguration(),
106       "testActiveMasterManagerFromZK", null);
107     ZKUtil.createAndFailSilent(zk, zk.baseZNode);
108     try {
109       ZKUtil.deleteNode(zk, zk.masterAddressZNode);
110     } catch(KeeperException.NoNodeException nne) {}
111 
112     // Create the master node with a dummy address
113     ServerName firstMasterAddress =
114       new ServerName("localhost", 1, System.currentTimeMillis());
115     ServerName secondMasterAddress =
116       new ServerName("localhost", 2, System.currentTimeMillis());
117 
118     // Should not have a master yet
119     DummyMaster ms1 = new DummyMaster();
120     ActiveMasterManager activeMasterManager = new ActiveMasterManager(zk,
121       firstMasterAddress, ms1);
122     zk.registerListener(activeMasterManager);
123     assertFalse(activeMasterManager.clusterHasActiveMaster.get());
124 
125     // First test becoming the active master uninterrupted
126     activeMasterManager.blockUntilBecomingActiveMaster(
127         Mockito.mock(MonitoredTask.class));
128     assertTrue(activeMasterManager.clusterHasActiveMaster.get());
129     assertMaster(zk, firstMasterAddress);
130 
131     // New manager will now try to become the active master in another thread
132     WaitToBeMasterThread t = new WaitToBeMasterThread(zk, secondMasterAddress);
133     zk.registerListener(t.manager);
134     t.start();
135     // Wait for this guy to figure out there is another active master
136     // Wait for 1 second at most
137     int sleeps = 0;
138     while(!t.manager.clusterHasActiveMaster.get() && sleeps < 100) {
139       Thread.sleep(10);
140       sleeps++;
141     }
142 
143     // Both should see that there is an active master
144     assertTrue(activeMasterManager.clusterHasActiveMaster.get());
145     assertTrue(t.manager.clusterHasActiveMaster.get());
146     // But secondary one should not be the active master
147     assertFalse(t.isActiveMaster);
148 
149     // Close the first server and delete it's master node
150     ms1.stop("stopping first server");
151 
152     // Use a listener to capture when the node is actually deleted
153     NodeDeletionListener listener = new NodeDeletionListener(zk, zk.masterAddressZNode);
154     zk.registerListener(listener);
155 
156     LOG.info("Deleting master node");
157     ZKUtil.deleteNode(zk, zk.masterAddressZNode);
158 
159     // Wait for the node to be deleted
160     LOG.info("Waiting for active master manager to be notified");
161     listener.waitForDeletion();
162     LOG.info("Master node deleted");
163 
164     // Now we expect the secondary manager to have and be the active master
165     // Wait for 1 second at most
166     sleeps = 0;
167     while(!t.isActiveMaster && sleeps < 100) {
168       Thread.sleep(10);
169       sleeps++;
170     }
171     LOG.debug("Slept " + sleeps + " times");
172 
173     assertTrue(t.manager.clusterHasActiveMaster.get());
174     assertTrue(t.isActiveMaster);
175 
176     LOG.info("Deleting master node");
177     ZKUtil.deleteNode(zk, zk.masterAddressZNode);
178   }
179 
180   /**
181    * Assert there is an active master and that it has the specified address.
182    * @param zk
183    * @param thisMasterAddress
184    * @throws KeeperException
185    */
186   private void assertMaster(ZooKeeperWatcher zk,
187       ServerName expectedAddress)
188   throws KeeperException {
189     ServerName readAddress =
190       ServerName.parseVersionedServerName(ZKUtil.getData(zk, zk.masterAddressZNode));
191     assertNotNull(readAddress);
192     assertTrue(expectedAddress.equals(readAddress));
193   }
194 
195   public static class WaitToBeMasterThread extends Thread {
196 
197     ActiveMasterManager manager;
198     boolean isActiveMaster;
199 
200     public WaitToBeMasterThread(ZooKeeperWatcher zk, ServerName address) {
201       this.manager = new ActiveMasterManager(zk, address,
202           new DummyMaster());
203       isActiveMaster = false;
204     }
205 
206     @Override
207     public void run() {
208       manager.blockUntilBecomingActiveMaster(
209           Mockito.mock(MonitoredTask.class));
210       LOG.info("Second master has become the active master!");
211       isActiveMaster = true;
212     }
213   }
214 
215   public static class NodeDeletionListener extends ZooKeeperListener {
216     private static final Log LOG = LogFactory.getLog(NodeDeletionListener.class);
217 
218     private Semaphore lock;
219     private String node;
220 
221     public NodeDeletionListener(ZooKeeperWatcher watcher, String node) {
222       super(watcher);
223       lock = new Semaphore(0);
224       this.node = node;
225     }
226 
227     @Override
228     public void nodeDeleted(String path) {
229       if(path.equals(node)) {
230         LOG.debug("nodeDeleted(" + path + ")");
231         lock.release();
232       }
233     }
234 
235     public void waitForDeletion() throws InterruptedException {
236       lock.acquire();
237     }
238   }
239 
240   /**
241    * Dummy Master Implementation.
242    */
243   public static class DummyMaster implements Server {
244     private volatile boolean stopped;
245 
246     @Override
247     public void abort(final String msg, final Throwable t) {}
248     
249     @Override
250     public boolean isAborted() {
251       return false;
252     }
253 
254     @Override
255     public Configuration getConfiguration() {
256       return null;
257     }
258 
259     @Override
260     public ZooKeeperWatcher getZooKeeper() {
261       return null;
262     }
263 
264     @Override
265     public ServerName getServerName() {
266       return null;
267     }
268 
269     @Override
270     public boolean isStopped() {
271       return this.stopped;
272     }
273 
274     @Override
275     public void stop(String why) {
276       this.stopped = true;
277     }
278 
279     @Override
280     public CatalogTracker getCatalogTracker() {
281       return null;
282     }
283   }
284 }