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