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.catalog;
21  
22  import static org.junit.Assert.assertTrue;
23  
24  import java.io.IOException;
25  import java.net.ConnectException;
26  import java.util.ArrayList;
27  import java.util.List;
28  import java.util.concurrent.CountDownLatch;
29  import java.util.concurrent.atomic.AtomicBoolean;
30  import java.util.concurrent.atomic.AtomicInteger;
31  
32  import junit.framework.Assert;
33  
34  import org.apache.commons.logging.Log;
35  import org.apache.commons.logging.LogFactory;
36  import org.apache.hadoop.hbase.*;
37  import org.apache.hadoop.hbase.client.Get;
38  import org.apache.hadoop.hbase.client.HConnection;
39  import org.apache.hadoop.hbase.client.HConnectionManager;
40  import org.apache.hadoop.hbase.client.HConnectionTestingUtility;
41  import org.apache.hadoop.hbase.client.Result;
42  import org.apache.hadoop.hbase.client.RetriesExhaustedException;
43  import org.apache.hadoop.hbase.client.ServerCallable;
44  import org.apache.hadoop.hbase.ipc.HRegionInterface;
45  import org.apache.hadoop.hbase.util.Bytes;
46  import org.apache.hadoop.hbase.util.Threads;
47  import org.apache.hadoop.hbase.util.Writables;
48  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
49  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
50  import org.apache.hadoop.util.Progressable;
51  import org.apache.zookeeper.KeeperException;
52  import org.apache.hadoop.hbase.ipc.ServerNotRunningYetException;
53  import org.junit.After;
54  import org.junit.AfterClass;
55  import org.junit.Before;
56  import org.junit.BeforeClass;
57  import org.junit.Ignore;
58  import org.junit.Test;
59  import org.junit.experimental.categories.Category;
60  import org.mockito.Mockito;
61  
62  /**
63   * Test {@link CatalogTracker}
64   */
65  @Category(MediumTests.class)
66  public class TestCatalogTracker {
67    private static final Log LOG = LogFactory.getLog(TestCatalogTracker.class);
68    private static final HBaseTestingUtility UTIL = new HBaseTestingUtility();
69    private static final ServerName SN =
70      new ServerName("example.org", 1234, System.currentTimeMillis());
71    private ZooKeeperWatcher watcher;
72    private Abortable abortable;
73  
74    @BeforeClass public static void beforeClass() throws Exception {
75      // Set this down so tests run quicker
76      UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 3);
77      UTIL.startMiniZKCluster();
78    }
79  
80    @AfterClass public static void afterClass() throws IOException {
81      UTIL.getZkCluster().shutdown();
82    }
83  
84    @Before public void before() throws IOException {
85      this.abortable = new Abortable() {
86        @Override
87        public void abort(String why, Throwable e) {
88          LOG.info(why, e);
89        }
90        
91        @Override
92        public boolean isAborted()  {
93          return false;
94        }
95      };
96      this.watcher = new ZooKeeperWatcher(UTIL.getConfiguration(),
97        this.getClass().getSimpleName(), this.abortable, true);
98    }
99  
100   @After public void after() {
101     try {
102       // Clean out root location or later tests will be confused... they presume
103       // start fresh in zk.
104       RootLocationEditor.deleteRootLocation(this.watcher);
105     } catch (KeeperException e) {
106       LOG.warn("Unable to delete root location", e);
107     }
108 
109     // Clear out our doctored connection or could mess up subsequent tests.
110     HConnectionManager.deleteConnection(UTIL.getConfiguration());
111 
112     this.watcher.close();
113   }
114 
115   private CatalogTracker constructAndStartCatalogTracker(final HConnection c)
116   throws IOException, InterruptedException {
117     CatalogTracker ct = new CatalogTracker(this.watcher, UTIL.getConfiguration(),
118       c, this.abortable);
119     ct.start();
120     return ct;
121   }
122 
123   /**
124    * Test that we get notification if .META. moves.
125    * @throws IOException 
126    * @throws InterruptedException 
127    * @throws KeeperException 
128    */
129   @Test public void testThatIfMETAMovesWeAreNotified()
130   throws IOException, InterruptedException, KeeperException {
131     HConnection connection = Mockito.mock(HConnection.class);
132     constructAndStartCatalogTracker(connection);
133 
134     RootLocationEditor.setRootLocation(this.watcher,
135       new ServerName("example.com", 1234, System.currentTimeMillis()));
136   }
137 
138   /**
139    * Test interruptable while blocking wait on root and meta.
140    * @throws IOException
141    * @throws InterruptedException
142    */
143   @Test public void testInterruptWaitOnMetaAndRoot()
144   throws IOException, InterruptedException {
145     HRegionInterface implementation = Mockito.mock(HRegionInterface.class);
146     HConnection connection = mockConnection(implementation);
147 
148     final CatalogTracker ct = constructAndStartCatalogTracker(connection);
149     ServerName hsa = ct.getRootLocation();
150     Assert.assertNull(hsa);
151     ServerName meta = ct.getMetaLocation();
152     Assert.assertNull(meta);
153     Thread t = new Thread() {
154       @Override
155       public void run() {
156         try {
157           ct.waitForMeta();
158         } catch (InterruptedException e) {
159           throw new RuntimeException("Interrupted", e);
160         }
161       }
162     };
163     t.start();
164     while (!t.isAlive())
165       Threads.sleep(1);
166     Threads.sleep(1);
167     assertTrue(t.isAlive());
168     ct.stop();
169     // Join the thread... should exit shortly.
170     t.join();
171   }
172 
173   /**
174    * Test for HBASE-4288.  Throw an IOE when trying to verify meta region and
175    * prove it doesn't cause master shutdown.
176    * @see <a href="https://issues.apache.org/jira/browse/HBASE-4288">HBASE-4288</a>
177    * @throws IOException
178    * @throws InterruptedException
179    * @throws KeeperException
180    */
181   @Test
182   public void testServerNotRunningIOException()
183   throws IOException, InterruptedException, KeeperException {
184     // Mock an HRegionInterface.
185     final HRegionInterface implementation = Mockito.mock(HRegionInterface.class);
186     HConnection connection = mockConnection(implementation);
187 
188     // If a 'getRegionInfo' is called on mocked HRegionInterface, throw IOE
189     // the first time.  'Succeed' the second time we are called.
190     Mockito.when(implementation.getRegionInfo((byte[]) Mockito.any())).
191       thenThrow(new IOException("Server not running, aborting")).
192       thenReturn(new HRegionInfo());
193 
194     // After we encounter the above 'Server not running', we should catch the
195     // IOE and go into retrying for the meta mode.  We'll do gets on -ROOT- to
196     // get new meta location.  Return something so this 'get' succeeds
197     // (here we mock up getRegionServerWithRetries, the wrapper around
198     // the actual get).
199 
200     // TODO: Refactor.  This method has been moved out of HConnection.
201     // It works for now but has been deprecated.
202     Mockito.when(connection.getRegionServerWithRetries((ServerCallable<Result>)Mockito.any())).
203       thenReturn(getMetaTableRowResult());
204 
205     // Now start up the catalogtracker with our doctored Connection.
206     final CatalogTracker ct = constructAndStartCatalogTracker(connection);
207     try {
208       // Set a location for root and meta.
209       RootLocationEditor.setRootLocation(this.watcher, SN);
210       ct.setMetaLocation(SN);
211       // Call the method that HBASE-4288 calls.  It will try and verify the
212       // meta location and will fail on first attempt then go into a long wait.
213       // So, do this in a thread and then reset meta location to break it out
214       // of its wait after a bit of time.
215       final AtomicBoolean metaSet = new AtomicBoolean(false);
216       final CountDownLatch latch = new CountDownLatch(1);
217       Thread t = new Thread() {
218         @Override
219         public void run() {
220           try {
221             latch.countDown();
222             metaSet.set(ct.waitForMeta(100000) !=  null);
223           } catch (Exception e) {
224             throw new RuntimeException(e);
225           }
226         }
227       };
228       t.start();
229       latch.await();
230       Threads.sleep(1);
231       // Now reset the meta as though it were redeployed.
232       ct.setMetaLocation(SN);
233       t.join();
234       Assert.assertTrue(metaSet.get());
235     } finally {
236       // Clean out root and meta locations or later tests will be confused...
237       // they presume start fresh in zk.
238       ct.resetMetaLocation();
239     }
240   }
241 
242   private void testVerifyMetaRegionLocationWithException(Exception ex)
243   throws IOException, InterruptedException, KeeperException {
244     // Mock an HRegionInterface.
245     final HRegionInterface implementation = Mockito.mock(HRegionInterface.class);
246     HConnection connection = mockConnection(implementation);
247 
248     // If a 'get' is called on mocked interface, throw connection refused.
249     Mockito.when(implementation.get((byte[]) Mockito.any(), (Get) Mockito.any())).
250       thenThrow(ex);
251     // Now start up the catalogtracker with our doctored Connection.
252     final CatalogTracker ct = constructAndStartCatalogTracker(connection);
253     RootLocationEditor.setRootLocation(this.watcher, SN);
254     long timeout = UTIL.getConfiguration().
255       getLong("hbase.catalog.verification.timeout", 1000);
256     Assert.assertFalse(ct.verifyMetaRegionLocation(timeout));
257   }
258 
259   /**
260    * Test we survive a connection refused {@link ConnectException}
261    * @throws IOException
262    * @throws InterruptedException
263    * @throws KeeperException
264    */
265   @Test
266   public void testGetMetaServerConnectionFails()
267   throws IOException, InterruptedException, KeeperException {
268     testVerifyMetaRegionLocationWithException(new ConnectException("Connection refused"));
269   }
270 
271   /**
272    * Test that verifyMetaRegionLocation properly handles getting a
273    * ServerNotRunningException. See HBASE-4470.
274    * Note this doesn't check the exact exception thrown in the
275    * HBASE-4470 as there it is thrown from getHConnection() and
276    * here it is thrown from get() -- but those are both called
277    * from the same function anyway, and this way is less invasive than
278    * throwing from getHConnection would be.
279    *
280    * @throws IOException
281    * @throws InterruptedException
282    * @throws KeeperException
283    */
284   @Test
285   public void testVerifyMetaRegionServerNotRunning()
286   throws IOException, InterruptedException, KeeperException {
287     testVerifyMetaRegionLocationWithException(new ServerNotRunningYetException("mock"));
288   }
289 
290   /**
291    * Test get of root region fails properly if nothing to connect to.
292    * @throws IOException
293    * @throws InterruptedException
294    * @throws KeeperException
295    */
296   @Test
297   public void testVerifyRootRegionLocationFails()
298   throws IOException, InterruptedException, KeeperException {
299     HConnection connection = Mockito.mock(HConnection.class);
300     ConnectException connectException =
301       new ConnectException("Connection refused");
302     final HRegionInterface implementation =
303       Mockito.mock(HRegionInterface.class);
304     Mockito.when(implementation.getRegionInfo((byte [])Mockito.any())).
305       thenThrow(connectException);
306     Mockito.when(connection.getHRegionConnection(Mockito.anyString(),
307       Mockito.anyInt(), Mockito.anyBoolean())).
308       thenReturn(implementation);
309     final CatalogTracker ct = constructAndStartCatalogTracker(connection);
310     RootLocationEditor.setRootLocation(this.watcher,
311       new ServerName("example.com", 1234, System.currentTimeMillis()));
312     Assert.assertFalse(ct.verifyRootRegionLocation(100));
313   }
314 
315   @Test (expected = NotAllMetaRegionsOnlineException.class)
316   public void testTimeoutWaitForRoot()
317   throws IOException, InterruptedException {
318     HConnection connection = Mockito.mock(HConnection.class);
319     final CatalogTracker ct = constructAndStartCatalogTracker(connection);
320     ct.waitForRoot(100);
321   }
322 
323   @Test (expected = RetriesExhaustedException.class)
324   public void testTimeoutWaitForMeta()
325   throws IOException, InterruptedException {
326     HConnection connection =
327       HConnectionTestingUtility.getMockedConnection(UTIL.getConfiguration());
328     final CatalogTracker ct = constructAndStartCatalogTracker(connection);
329     ct.waitForMeta(100);
330   }
331 
332   /**
333    * Test waiting on root w/ no timeout specified.
334    * @throws IOException
335    * @throws InterruptedException
336    * @throws KeeperException
337    */
338   @Test public void testNoTimeoutWaitForRoot()
339   throws IOException, InterruptedException, KeeperException {
340     HConnection connection = Mockito.mock(HConnection.class);
341     final CatalogTracker ct = constructAndStartCatalogTracker(connection);
342     ServerName hsa = ct.getRootLocation();
343     Assert.assertNull(hsa);
344 
345     // Now test waiting on root location getting set.
346     Thread t = new WaitOnMetaThread(ct);
347     startWaitAliveThenWaitItLives(t, 1000);
348     // Set a root location.
349     hsa = setRootLocation();
350     // Join the thread... should exit shortly.
351     t.join();
352     // Now root is available.
353     Assert.assertTrue(ct.getRootLocation().equals(hsa));
354   }
355 
356   private ServerName setRootLocation() throws KeeperException {
357     RootLocationEditor.setRootLocation(this.watcher, SN);
358     return SN;
359   }
360 
361   /**
362    * Test waiting on meta w/ no timeout specified.
363    * @throws Exception 
364    */
365   @Ignore // Can't make it work reliably on all platforms; mockito gets confused
366   // Throwing: org.mockito.exceptions.misusing.WrongTypeOfReturnValue:
367   // Result cannot be returned by locateRegion()
368   // If you plug locateRegion, it then throws for incCounter, and if you plug
369   // that ... and so one.
370   @Test public void testNoTimeoutWaitForMeta()
371   throws Exception {
372     // Mock an HConnection and a HRegionInterface implementation.  Have the
373     // HConnection return the HRI.  Have the HRI return a few mocked up responses
374     // to make our test work.
375     // Mock an HRegionInterface.
376     final HRegionInterface implementation = Mockito.mock(HRegionInterface.class);
377     HConnection connection = mockConnection(implementation);
378 
379     // Now the ct is up... set into the mocks some answers that make it look
380     // like things have been getting assigned. Make it so we'll return a
381     // location (no matter what the Get is). Same for getHRegionInfo -- always
382     // just return the meta region.
383     final Result result = getMetaTableRowResult();
384 
385     // TODO: Refactor.  This method has been moved out of HConnection.
386     // It works for now but has been deprecated.
387     Mockito.when(connection.getRegionServerWithRetries((ServerCallable<Result>)Mockito.any())).
388       thenReturn(result);
389     Mockito.when(implementation.getRegionInfo((byte[]) Mockito.any())).
390       thenReturn(HRegionInfo.FIRST_META_REGIONINFO);
391     final CatalogTracker ct = constructAndStartCatalogTracker(connection);
392     ServerName hsa = ct.getMetaLocation();
393     Assert.assertNull(hsa);
394 
395     // Now test waiting on meta location getting set.
396     Thread t = new WaitOnMetaThread(ct) {
397       @Override
398       void doWaiting() throws InterruptedException {
399         this.ct.waitForMeta();
400       }
401     };
402     startWaitAliveThenWaitItLives(t, 1000);
403 
404     // This should trigger wake up of meta wait (Its the removal of the meta
405     // region unassigned node that triggers catalogtrackers that a meta has
406     // been assigned).
407     String node = ct.getMetaNodeTracker().getNode();
408     ZKUtil.createAndFailSilent(this.watcher, node);
409     MetaEditor.updateMetaLocation(ct, HRegionInfo.FIRST_META_REGIONINFO, SN);
410     ZKUtil.deleteNode(this.watcher, node);
411     // Go get the new meta location. waitForMeta gets and verifies meta.
412     Assert.assertTrue(ct.waitForMeta(10000).equals(SN));
413     // Join the thread... should exit shortly.
414     t.join();
415     // Now meta is available.
416     Assert.assertTrue(ct.waitForMeta(10000).equals(SN));
417   }
418 
419   /**
420    * @param implementation An {@link HRegionInterface} instance; you'll likely
421    * want to pass a mocked HRS; can be null.
422    * @return Mock up a connection that returns a {@link org.apache.hadoop.conf.Configuration} when
423    * {@link HConnection#getConfiguration()} is called, a 'location' when
424    * {@link HConnection#getRegionLocation(byte[], byte[], boolean)} is called,
425    * and that returns the passed {@link HRegionInterface} instance when
426    * {@link HConnection#getHRegionConnection(String, int)}
427    * is called (Be sure call
428    * {@link HConnectionManager#deleteConnection(org.apache.hadoop.conf.Configuration)}
429    * when done with this mocked Connection.
430    * @throws IOException
431    */
432   private HConnection mockConnection(final HRegionInterface implementation)
433   throws IOException {
434     HConnection connection =
435       HConnectionTestingUtility.getMockedConnection(UTIL.getConfiguration());
436     Mockito.doNothing().when(connection).close();
437     // Make it so we return any old location when asked.
438     final HRegionLocation anyLocation =
439       new HRegionLocation(HRegionInfo.FIRST_META_REGIONINFO, SN.getHostname(),
440         SN.getPort());
441     Mockito.when(connection.getRegionLocation((byte[]) Mockito.any(),
442         (byte[]) Mockito.any(), Mockito.anyBoolean())).
443       thenReturn(anyLocation);
444     Mockito.when(connection.locateRegion((byte[]) Mockito.any(),
445         (byte[]) Mockito.any())).
446       thenReturn(anyLocation);
447     if (implementation != null) {
448       // If a call to getHRegionConnection, return this implementation.
449       Mockito.when(connection.getHRegionConnection(Mockito.anyString(), Mockito.anyInt())).
450         thenReturn(implementation);
451     }
452     return connection;
453   }
454 
455   /**
456    * @return A mocked up Result that fakes a Get on a row in the
457    * <code>.META.</code> table.
458    * @throws IOException 
459    */
460   private Result getMetaTableRowResult() throws IOException {
461     List<KeyValue> kvs = new ArrayList<KeyValue>();
462     kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY,
463       HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
464       Writables.getBytes(HRegionInfo.FIRST_META_REGIONINFO)));
465     kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY,
466       HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
467       Bytes.toBytes(SN.getHostAndPort())));
468     kvs.add(new KeyValue(HConstants.EMPTY_BYTE_ARRAY,
469       HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER,
470       Bytes.toBytes(SN.getStartcode())));
471     return new Result(kvs);
472   }
473 
474   private void startWaitAliveThenWaitItLives(final Thread t, final int ms) {
475     t.start();
476     while(!t.isAlive()) {
477       // Wait
478     }
479     // Wait one second.
480     Threads.sleep(ms);
481     Assert.assertTrue("Assert " + t.getName() + " still waiting", t.isAlive());
482   }
483 
484   class CountingProgressable implements Progressable {
485     final AtomicInteger counter = new AtomicInteger(0);
486     @Override
487     public void progress() {
488       this.counter.incrementAndGet();
489     }
490   }
491 
492   /**
493    * Wait on META.
494    * Default is wait on -ROOT-.
495    */
496   class WaitOnMetaThread extends Thread {
497     final CatalogTracker ct;
498 
499     WaitOnMetaThread(final CatalogTracker ct) {
500       super("WaitOnMeta");
501       this.ct = ct;
502     }
503 
504     @Override
505     public void run() {
506       try {
507         doWaiting();
508       } catch (InterruptedException e) {
509         throw new RuntimeException("Failed wait", e);
510       }
511       LOG.info("Exiting " + getName());
512     }
513 
514     void doWaiting() throws InterruptedException {
515       this.ct.waitForRoot();
516     }
517   }
518 
519   @org.junit.Rule
520   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
521     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
522 }
523