View Javadoc

1   /**
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  package org.apache.hadoop.hbase.master;
19  
20  
21  import static org.junit.Assert.assertEquals;
22  import static org.junit.Assert.assertFalse;
23  import static org.junit.Assert.assertTrue;
24  
25  import java.io.IOException;
26  import java.net.InetAddress;
27  import java.net.UnknownHostException;
28  import java.util.ArrayList;
29  import java.util.List;
30  import java.util.Set;
31  
32  import org.apache.hadoop.conf.Configuration;
33  import org.apache.hadoop.hbase.Abortable;
34  import org.apache.hadoop.hbase.TableName;
35  import org.apache.hadoop.hbase.exceptions.DeserializationException;
36  import org.apache.hadoop.hbase.HBaseTestingUtility;
37  import org.apache.hadoop.hbase.HConstants;
38  import org.apache.hadoop.hbase.HRegionInfo;
39  import org.apache.hadoop.hbase.ServerLoad;
40  import org.apache.hadoop.hbase.Server;
41  import org.apache.hadoop.hbase.ServerName;
42  import org.apache.hadoop.hbase.ZooKeeperConnectionException;
43  import org.apache.hadoop.hbase.catalog.CatalogTracker;
44  import org.apache.hadoop.hbase.catalog.MetaMockingUtil;
45  import org.apache.hadoop.hbase.client.HConnection;
46  import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
47  import org.apache.hadoop.hbase.client.Result;
48  import org.apache.hadoop.hbase.master.RegionState.State;
49  import org.apache.hadoop.hbase.monitoring.MonitoredTask;
50  import org.apache.hadoop.hbase.regionserver.RegionOpeningState;
51  import org.apache.hadoop.hbase.util.FSUtils;
52  import org.apache.hadoop.hbase.util.Threads;
53  import org.apache.hadoop.hbase.zookeeper.MetaRegionTracker;
54  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
55  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
56  import org.apache.hadoop.hbase.testclassification.MediumTests;
57  import org.apache.zookeeper.KeeperException;
58  import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
59  import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.NameStringPair;
60  import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerReportRequest;
61  import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupRequest;
62  import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.RegionServerStartupResponse;
63  import com.google.protobuf.ServiceException;
64  import org.junit.After;
65  import org.junit.AfterClass;
66  import org.junit.BeforeClass;
67  import org.junit.Test;
68  import org.mockito.Mockito;
69  import org.junit.experimental.categories.Category;
70  import org.apache.commons.logging.Log;
71  import org.apache.commons.logging.LogFactory;
72  
73  /**
74   * Standup the master and fake it to test various aspects of master function.
75   * Does NOT spin up a mini hbase nor mini dfs cluster testing master (it does
76   * put up a zk cluster but this is usually pretty fast compared).  Also, should
77   * be possible to inject faults at points difficult to get at in cluster context.
78   * TODO: Speed up the zk connection by Master.  It pauses 5 seconds establishing
79   * session.
80   */
81  @Category(MediumTests.class)
82  public class TestMasterNoCluster {
83    private static final Log LOG = LogFactory.getLog(TestMasterNoCluster.class);
84    private static final HBaseTestingUtility TESTUTIL = new HBaseTestingUtility();
85  
86    @BeforeClass
87    public static void setUpBeforeClass() throws Exception {
88      Configuration c = TESTUTIL.getConfiguration();
89      // We use local filesystem.  Set it so it writes into the testdir.
90      FSUtils.setRootDir(c, TESTUTIL.getDataTestDir());
91      // Startup a mini zk cluster.
92      TESTUTIL.startMiniZKCluster();
93    }
94  
95    @AfterClass
96    public static void tearDownAfterClass() throws Exception {
97      TESTUTIL.shutdownMiniZKCluster();
98    }
99  
100   @After
101   public void tearDown()
102   throws KeeperException, ZooKeeperConnectionException, IOException {
103     // Make sure zk is clean before we run the next test.
104     ZooKeeperWatcher zkw = new ZooKeeperWatcher(TESTUTIL.getConfiguration(),
105         "@Before", new Abortable() {
106       @Override
107       public void abort(String why, Throwable e) {
108         throw new RuntimeException(why, e);
109       }
110 
111       @Override
112       public boolean isAborted() {
113         return false;
114       }
115     });
116     ZKUtil.deleteNodeRecursively(zkw, zkw.baseZNode);
117     zkw.close();
118   }
119 
120   /**
121    * Test starting master then stopping it before its fully up.
122    * @throws IOException
123    * @throws KeeperException
124    * @throws InterruptedException
125    */
126   @Test (timeout=30000)
127   public void testStopDuringStart()
128   throws IOException, KeeperException, InterruptedException {
129     HMaster master = new HMaster(TESTUTIL.getConfiguration());
130     master.start();
131     // Immediately have it stop.  We used hang in assigning meta.
132     master.stopMaster();
133     master.join();
134   }
135 
136   /**
137    * Test master failover.
138    * Start up three fake regionservers and a master.
139    * @throws IOException
140    * @throws KeeperException
141    * @throws InterruptedException
142    */
143   @Test (timeout=30000)
144   public void testFailover()
145   throws IOException, KeeperException, InterruptedException, ServiceException {
146     final long now = System.currentTimeMillis();
147     // Names for our three servers.  Make the port numbers match hostname.
148     // Will come in use down in the server when we need to figure how to respond.
149     final ServerName sn0 = ServerName.valueOf("0.example.org", 0, now);
150     final ServerName sn1 = ServerName.valueOf("1.example.org", 1, now);
151     final ServerName sn2 = ServerName.valueOf("2.example.org", 2, now);
152     final ServerName [] sns = new ServerName [] {sn0, sn1, sn2};
153     // Put up the mock servers
154     final Configuration conf = TESTUTIL.getConfiguration();
155     final MockRegionServer rs0 = new MockRegionServer(conf, sn0);
156     final MockRegionServer rs1 = new MockRegionServer(conf, sn1);
157     final MockRegionServer rs2 = new MockRegionServer(conf, sn2);
158     // Put some data into the servers.  Make it look like sn0 has the metaH
159     // Put data into sn2 so it looks like it has a few regions for a table named 't'.
160     MetaRegionTracker.setMetaLocation(rs0.getZooKeeper(), rs0.getServerName(), State.OPEN);
161     final TableName tableName = TableName.valueOf("t");
162     Result [] results = new Result [] {
163       MetaMockingUtil.getMetaTableRowResult(
164         new HRegionInfo(tableName, HConstants.EMPTY_START_ROW, HBaseTestingUtility.KEYS[1]),
165         rs2.getServerName()),
166       MetaMockingUtil.getMetaTableRowResult(
167         new HRegionInfo(tableName, HBaseTestingUtility.KEYS[1], HBaseTestingUtility.KEYS[2]),
168         rs2.getServerName()),
169       MetaMockingUtil.getMetaTableRowResult(new HRegionInfo(tableName, HBaseTestingUtility.KEYS[2],
170           HConstants.EMPTY_END_ROW),
171         rs2.getServerName())
172     };
173     rs1.setNextResults(HRegionInfo.FIRST_META_REGIONINFO.getRegionName(), results);
174 
175     // Create master.  Subclass to override a few methods so we can insert mocks
176     // and get notification on transitions.  We need to fake out any rpcs the
177     // master does opening/closing regions.  Also need to fake out the address
178     // of the 'remote' mocked up regionservers.
179     HMaster master = new HMaster(conf) {
180       InetAddress getRemoteInetAddress(final int port, final long serverStartCode)
181       throws UnknownHostException {
182         // Return different address dependent on port passed.
183         ServerName sn = sns[port];
184         return InetAddress.getByAddress(sn.getHostname(),
185           new byte [] {10, 0, 0, (byte)sn.getPort()});
186       }
187 
188       @Override
189       ServerManager createServerManager(Server master, MasterServices services)
190       throws IOException {
191         ServerManager sm = super.createServerManager(master, services);
192         // Spy on the created servermanager
193         ServerManager spy = Mockito.spy(sm);
194         // Fake a successful open.
195         Mockito.doReturn(RegionOpeningState.OPENED).when(spy).
196           sendRegionOpen((ServerName)Mockito.any(), (HRegionInfo)Mockito.any(),
197             Mockito.anyInt(), Mockito.anyListOf(ServerName.class));
198         return spy;
199       }
200 
201       @Override
202       CatalogTracker createCatalogTracker(ZooKeeperWatcher zk,
203           Configuration conf, Abortable abortable)
204       throws IOException {
205         // Insert a mock for the connection used by the CatalogTracker.  Any
206         // regionserver should do.  Use TESTUTIL.getConfiguration rather than
207         // the conf from the master; the conf will already have an HConnection
208         // associate so the below mocking of a connection will fail.
209         HConnection connection =
210           HConnectionTestingUtility.getMockedConnectionAndDecorate(TESTUTIL.getConfiguration(),
211             rs0, rs0, rs0.getServerName(), HRegionInfo.FIRST_META_REGIONINFO);
212         return new CatalogTracker(zk, conf, connection, abortable);
213       }
214 
215       @Override
216       void initNamespace() {
217       }
218     };
219     master.start();
220 
221     try {
222       // Wait till master is up ready for RPCs.
223       while (!master.isRpcServerOpen()) Threads.sleep(10);
224       // Fake master that there are regionservers out there.  Report in.
225       for (int i = 0; i < sns.length; i++) {
226         RegionServerReportRequest.Builder request = RegionServerReportRequest.newBuilder();;
227         ServerName sn = ServerName.parseVersionedServerName(sns[i].getVersionedBytes());
228         request.setServer(ProtobufUtil.toServerName(sn));
229         request.setLoad(ServerLoad.EMPTY_SERVERLOAD.obtainServerLoadPB());
230         master.regionServerReport(null, request.build());
231       }
232       // Master should now come up.
233       while (!master.isInitialized()) {Threads.sleep(10);}
234       assertTrue(master.isInitialized());
235     } finally {
236       rs0.stop("Test is done");
237       rs1.stop("Test is done");
238       rs2.stop("Test is done");
239       master.stopMaster();
240       master.join();
241     }
242   }
243 
244   /**
245    * Test starting master getting it up post initialized state using mocks.
246    * @throws IOException
247    * @throws KeeperException
248    * @throws InterruptedException
249    * @throws DeserializationException
250    * @throws ServiceException
251    */
252   @Test (timeout=60000)
253   public void testCatalogDeploys()
254       throws Exception {
255     final Configuration conf = TESTUTIL.getConfiguration();
256     conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MINTOSTART, 1);
257     conf.setInt(ServerManager.WAIT_ON_REGIONSERVERS_MAXTOSTART, 1);
258 
259     final long now = System.currentTimeMillis();
260     // Name for our single mocked up regionserver.
261     final ServerName sn = ServerName.valueOf("0.example.org", 0, now);
262     // Here is our mocked up regionserver.  Create it now.  Need it setting up
263     // master next.
264     final MockRegionServer rs0 = new MockRegionServer(conf, sn);
265 
266     // Create master.  Subclass to override a few methods so we can insert mocks
267     // and get notification on transitions.  We need to fake out any rpcs the
268     // master does opening/closing regions.  Also need to fake out the address
269     // of the 'remote' mocked up regionservers.
270     HMaster master = new HMaster(conf) {
271       InetAddress getRemoteInetAddress(final int port, final long serverStartCode)
272       throws UnknownHostException {
273         // Interject an unchecked, nonsense InetAddress; i.e. no resolve.
274         return InetAddress.getByAddress(rs0.getServerName().getHostname(),
275           new byte [] {10, 0, 0, 0});
276       }
277 
278       @Override
279       ServerManager createServerManager(Server master, MasterServices services)
280       throws IOException {
281         ServerManager sm = super.createServerManager(master, services);
282         // Spy on the created servermanager
283         ServerManager spy = Mockito.spy(sm);
284         // Fake a successful open.
285         Mockito.doReturn(RegionOpeningState.OPENED).when(spy).
286           sendRegionOpen((ServerName)Mockito.any(), (HRegionInfo)Mockito.any(),
287             Mockito.anyInt(), Mockito.anyListOf(ServerName.class));
288         return spy;
289       }
290 
291       @Override
292       CatalogTracker createCatalogTracker(ZooKeeperWatcher zk,
293           Configuration conf, Abortable abortable)
294       throws IOException {
295         // Insert a mock for the connection used by the CatalogTracker.   Use
296         // TESTUTIL.getConfiguration rather than the conf from the master; the
297         // conf will already have an HConnection associate so the below mocking
298         // of a connection will fail.
299         HConnection connection =
300           HConnectionTestingUtility.getMockedConnectionAndDecorate(TESTUTIL.getConfiguration(),
301             rs0, rs0, rs0.getServerName(), HRegionInfo.FIRST_META_REGIONINFO);
302         return new CatalogTracker(zk, conf, connection, abortable);
303       }
304 
305       @Override
306       void initNamespace() {
307       }
308     };
309     master.start();
310     LOG.info("Master has started");
311 
312     try {
313       // Wait till master is up ready for RPCs.
314       while (!master.isRpcServerOpen()) Threads.sleep(10);
315       LOG.info("RpcServerOpen has started");
316 
317       // Fake master that there is a regionserver out there.  Report in.
318       RegionServerStartupRequest.Builder request = RegionServerStartupRequest.newBuilder();
319       request.setPort(rs0.getServerName().getPort());
320       request.setServerStartCode(rs0.getServerName().getStartcode());
321       request.setServerCurrentTime(now);
322       RegionServerStartupResponse result =
323         master.regionServerStartup(null, request.build());
324       String rshostname = new String();
325       for (NameStringPair e : result.getMapEntriesList()) {
326         if (e.getName().toString().equals(HConstants.KEY_FOR_HOSTNAME_SEEN_BY_MASTER)) {
327           rshostname = e.getValue();
328         }
329       }
330       // Assert hostname is as expected.
331       assertEquals(rs0.getServerName().getHostname(), rshostname);
332       // Now master knows there is at least one regionserver checked in and so
333       // it'll wait a while to see if more and when none, will assign meta
334       // to this single server.  Will do an rpc open but we've
335       // mocked it above in our master override to return 'success'.  As part of
336       // region open, master will have set an unassigned znode for the region up
337       // into zk for the regionserver to transition.  Lets do that now to
338       // complete fake of a successful open.
339       Mocking.fakeRegionServerRegionOpenInZK(master, rs0.getZooKeeper(),
340         rs0.getServerName(), HRegionInfo.FIRST_META_REGIONINFO);
341       LOG.info("fakeRegionServerRegionOpenInZK has started");
342 
343       // Need to set meta location as r0.  Usually the regionserver does this
344       // when its figured it just opened the meta region by setting the meta
345       // location up into zk.  Since we're mocking regionserver, need to do this
346       // ourselves.
347       MetaRegionTracker.setMetaLocation(rs0.getZooKeeper(), rs0.getServerName(), State.OPEN);
348       // Master should now come up.
349       while (!master.isInitialized()) {Threads.sleep(10);}
350       assertTrue(master.isInitialized());
351     } finally {
352       rs0.stop("Test is done");
353       master.stopMaster();
354       master.join();
355     }
356   }
357 
358   @Test
359   public void testNotPullingDeadRegionServerFromZK()
360       throws IOException, KeeperException, InterruptedException {
361     final Configuration conf = TESTUTIL.getConfiguration();
362     final ServerName newServer = ServerName.valueOf("test.sample", 1, 101);
363     final ServerName deadServer = ServerName.valueOf("test.sample", 1, 100);
364     final MockRegionServer rs0 = new MockRegionServer(conf, newServer);
365 
366     HMaster master = new HMaster(conf) {
367       @Override
368       void assignMeta(MonitoredTask status, Set<ServerName> previouslyFailedMeatRSs) {
369       }
370 
371       @Override
372       void initializeZKBasedSystemTrackers() throws IOException,
373       InterruptedException, KeeperException {
374         super.initializeZKBasedSystemTrackers();
375         // Record a newer server in server manager at first
376         serverManager.recordNewServerWithLock(newServer, ServerLoad.EMPTY_SERVERLOAD);
377 
378         List<ServerName> onlineServers = new ArrayList<ServerName>();
379         onlineServers.add(deadServer);
380         onlineServers.add(newServer);
381         // Mock the region server tracker to pull the dead server from zk
382         regionServerTracker = Mockito.spy(regionServerTracker);
383         Mockito.doReturn(onlineServers).when(
384           regionServerTracker).getOnlineServers();
385       }
386 
387       @Override
388       CatalogTracker createCatalogTracker(ZooKeeperWatcher zk,
389           Configuration conf, Abortable abortable)
390       throws IOException {
391         // Insert a mock for the connection used by the CatalogTracker.  Any
392         // regionserver should do.  Use TESTUTIL.getConfiguration rather than
393         // the conf from the master; the conf will already have an HConnection
394         // associate so the below mocking of a connection will fail.
395         HConnection connection =
396           HConnectionTestingUtility.getMockedConnectionAndDecorate(TESTUTIL.getConfiguration(),
397             rs0, rs0, rs0.getServerName(), HRegionInfo.FIRST_META_REGIONINFO);
398         return new CatalogTracker(zk, conf, connection, abortable);
399       }
400 
401       @Override
402       void initNamespace() {
403       }
404     };
405     master.start();
406 
407     try {
408       // Wait till master is initialized.
409       while (!master.initialized) Threads.sleep(10);
410       LOG.info("Master is initialized");
411 
412       assertFalse("The dead server should not be pulled in",
413         master.serverManager.isServerOnline(deadServer));
414     } finally {
415       master.stopMaster();
416       master.join();
417     }
418   }
419 }