View Javadoc

1   /*
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  package org.apache.hadoop.hbase.client;
20  
21  import static org.junit.Assert.assertEquals;
22  import static org.junit.Assert.assertFalse;
23  import static org.junit.Assert.assertNotNull;
24  import static org.junit.Assert.assertNull;
25  import static org.junit.Assert.assertTrue;
26  import static org.junit.Assert.fail;
27  
28  import java.io.IOException;
29  import java.lang.reflect.Field;
30  import java.lang.reflect.Modifier;
31  import java.net.SocketTimeoutException;
32  import java.util.ArrayList;
33  import java.util.HashMap;
34  import java.util.List;
35  import java.util.Map;
36  import java.util.Random;
37  import java.util.concurrent.ExecutorService;
38  import java.util.concurrent.SynchronousQueue;
39  import java.util.concurrent.ThreadPoolExecutor;
40  import java.util.concurrent.TimeUnit;
41  import java.util.concurrent.atomic.AtomicBoolean;
42  import java.util.concurrent.atomic.AtomicInteger;
43  import java.util.concurrent.atomic.AtomicLong;
44  import java.util.concurrent.atomic.AtomicReference;
45  
46  import org.apache.commons.logging.Log;
47  import org.apache.commons.logging.LogFactory;
48  import org.apache.hadoop.conf.Configuration;
49  import org.apache.hadoop.hbase.Cell;
50  import org.apache.hadoop.hbase.HBaseConfiguration;
51  import org.apache.hadoop.hbase.HBaseTestingUtility;
52  import org.apache.hadoop.hbase.HConstants;
53  import org.apache.hadoop.hbase.HRegionLocation;
54  import org.apache.hadoop.hbase.HTableDescriptor;
55  import org.apache.hadoop.hbase.RegionLocations;
56  import org.apache.hadoop.hbase.ServerName;
57  import org.apache.hadoop.hbase.TableName;
58  import org.apache.hadoop.hbase.Waiter;
59  import org.apache.hadoop.hbase.client.ConnectionManager.HConnectionImplementation;
60  import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver;
61  import org.apache.hadoop.hbase.coprocessor.ObserverContext;
62  import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
63  import org.apache.hadoop.hbase.exceptions.DeserializationException;
64  import org.apache.hadoop.hbase.exceptions.RegionMovedException;
65  import org.apache.hadoop.hbase.filter.Filter;
66  import org.apache.hadoop.hbase.filter.FilterBase;
67  import org.apache.hadoop.hbase.ipc.RpcClient;
68  import org.apache.hadoop.hbase.master.HMaster;
69  import org.apache.hadoop.hbase.regionserver.HRegionServer;
70  import org.apache.hadoop.hbase.regionserver.Region;
71  import org.apache.hadoop.hbase.regionserver.RegionServerStoppedException;
72  import org.apache.hadoop.hbase.testclassification.MediumTests;
73  import org.apache.hadoop.hbase.util.Bytes;
74  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
75  import org.apache.hadoop.hbase.util.JVMClusterUtil;
76  import org.apache.hadoop.hbase.util.ManualEnvironmentEdge;
77  import org.apache.hadoop.hbase.util.Threads;
78  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
79  import org.junit.AfterClass;
80  import org.junit.Assert;
81  import org.junit.BeforeClass;
82  import org.junit.Ignore;
83  import org.junit.Test;
84  import org.junit.experimental.categories.Category;
85  
86  import com.google.common.collect.Lists;
87  
88  /**
89   * This class is for testing HBaseConnectionManager features
90   */
91  @Category(MediumTests.class)
92  public class TestHCM {
93    private static final Log LOG = LogFactory.getLog(TestHCM.class);
94    private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
95    private static final TableName TABLE_NAME =
96        TableName.valueOf("test");
97    private static final TableName TABLE_NAME1 =
98        TableName.valueOf("test1");
99    private static final TableName TABLE_NAME2 =
100       TableName.valueOf("test2");
101   private static final TableName TABLE_NAME3 =
102       TableName.valueOf("test3");
103   private static final TableName TABLE_NAME4 =
104       TableName.valueOf("test4");
105   private static final byte[] FAM_NAM = Bytes.toBytes("f");
106   private static final byte[] ROW = Bytes.toBytes("bbb");
107   private static final byte[] ROW_X = Bytes.toBytes("xxx");
108   private static Random _randy = new Random();
109 
110 /**
111 * This copro sleeps 20 second. The first call it fails. The second time, it works.
112 */
113   public static class SleepAndFailFirstTime extends BaseRegionObserver {
114     static final AtomicLong ct = new AtomicLong(0);
115 
116     public SleepAndFailFirstTime() {
117     }
118 
119     @Override
120     public void preGetOp(final ObserverContext<RegionCoprocessorEnvironment> e,
121               final Get get, final List<Cell> results) throws IOException {
122       Threads.sleep(20000);
123       if (ct.incrementAndGet() == 1){
124         throw new IOException("first call I fail");
125       }
126     }
127   }
128 
129   @BeforeClass
130   public static void setUpBeforeClass() throws Exception {
131     TEST_UTIL.getConfiguration().setBoolean(HConstants.STATUS_PUBLISHED, true);
132     TEST_UTIL.startMiniCluster(2);
133   }
134 
135   @AfterClass public static void tearDownAfterClass() throws Exception {
136     TEST_UTIL.shutdownMiniCluster();
137   }
138 
139 
140   private static int getHConnectionManagerCacheSize(){
141     return HConnectionTestingUtility.getConnectionCount();
142   }
143 
144   @Test
145   public void testClusterConnection() throws IOException {
146     ThreadPoolExecutor otherPool = new ThreadPoolExecutor(1, 1,
147         5, TimeUnit.SECONDS,
148         new SynchronousQueue<Runnable>(),
149         Threads.newDaemonThreadFactory("test-hcm"));
150 
151     HConnection con1 = HConnectionManager.createConnection(TEST_UTIL.getConfiguration());
152     HConnection con2 = HConnectionManager.createConnection(TEST_UTIL.getConfiguration(), otherPool);
153     // make sure the internally created ExecutorService is the one passed
154     assertTrue(otherPool == ((HConnectionImplementation)con2).getCurrentBatchPool());
155 
156     String tableName = "testClusterConnection";
157     TEST_UTIL.createTable(tableName.getBytes(), FAM_NAM).close();
158     HTable t = (HTable)con1.getTable(tableName, otherPool);
159     // make sure passing a pool to the getTable does not trigger creation of an internal pool
160     assertNull("Internal Thread pool should be null", ((HConnectionImplementation)con1).getCurrentBatchPool());
161     // table should use the pool passed
162     assertTrue(otherPool == t.getPool());
163     t.close();
164 
165     t = (HTable)con2.getTable(tableName);
166     // table should use the connectin's internal pool
167     assertTrue(otherPool == t.getPool());
168     t.close();
169 
170     t = (HTable)con2.getTable(Bytes.toBytes(tableName));
171     // try other API too
172     assertTrue(otherPool == t.getPool());
173     t.close();
174 
175     t = (HTable)con2.getTable(TableName.valueOf(tableName));
176     // try other API too
177     assertTrue(otherPool == t.getPool());
178     t.close();
179 
180     t = (HTable)con1.getTable(tableName);
181     ExecutorService pool = ((HConnectionImplementation)con1).getCurrentBatchPool();
182     // make sure an internal pool was created
183     assertNotNull("An internal Thread pool should have been created", pool);
184     // and that the table is using it
185     assertTrue(t.getPool() == pool);
186     t.close();
187 
188     t = (HTable)con1.getTable(tableName);
189     // still using the *same* internal pool
190     assertTrue(t.getPool() == pool);
191     t.close();
192 
193     con1.close();
194     // if the pool was created on demand it should be closed upon connection close
195     assertTrue(pool.isShutdown());
196 
197     con2.close();
198     // if the pool is passed, it is not closed
199     assertFalse(otherPool.isShutdown());
200     otherPool.shutdownNow();
201   }
202 
203   /**
204    * Naive test to check that HConnection#getAdmin returns a properly constructed HBaseAdmin object
205    * @throws IOException Unable to construct admin
206    */
207   @Test
208   public void testAdminFactory() throws IOException {
209     Connection con1 = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
210     Admin admin = con1.getAdmin();
211     assertTrue(admin.getConnection() == con1);
212     assertTrue(admin.getConfiguration() == TEST_UTIL.getConfiguration());
213     con1.close();
214   }
215 
216   // Fails too often!  Needs work.  HBASE-12558
217   @Ignore @Test(expected = RegionServerStoppedException.class)
218   public void testClusterStatus() throws Exception {
219 
220     TableName tn =
221         TableName.valueOf("testClusterStatus");
222     byte[] cf = "cf".getBytes();
223     byte[] rk = "rk1".getBytes();
224 
225     JVMClusterUtil.RegionServerThread rs = TEST_UTIL.getHBaseCluster().startRegionServer();
226     rs.waitForServerOnline();
227     final ServerName sn = rs.getRegionServer().getServerName();
228 
229     HTable t = TEST_UTIL.createTable(tn, cf);
230     TEST_UTIL.waitTableAvailable(tn);
231 
232     while(TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().
233         getRegionStates().isRegionsInTransition()){
234       Thread.sleep(1);
235     }
236     final HConnectionImplementation hci =  (HConnectionImplementation)t.getConnection();
237     while (t.getRegionLocation(rk).getPort() != sn.getPort()){
238       TEST_UTIL.getHBaseAdmin().move(t.getRegionLocation(rk).getRegionInfo().
239           getEncodedNameAsBytes(), Bytes.toBytes(sn.toString()));
240       while(TEST_UTIL.getHBaseCluster().getMaster().getAssignmentManager().
241           getRegionStates().isRegionsInTransition()){
242         Thread.sleep(1);
243       }
244       hci.clearRegionCache(tn);
245     }
246     Assert.assertNotNull(hci.clusterStatusListener);
247     TEST_UTIL.assertRegionOnServer(t.getRegionLocation(rk).getRegionInfo(), sn, 20000);
248 
249     Put p1 = new Put(rk);
250     p1.add(cf, "qual".getBytes(), "val".getBytes());
251     t.put(p1);
252 
253     rs.getRegionServer().abort("I'm dead");
254 
255     // We want the status to be updated. That's a least 10 second
256     TEST_UTIL.waitFor(40000, 1000, true, new Waiter.Predicate<Exception>() {
257       @Override
258       public boolean evaluate() throws Exception {
259         return TEST_UTIL.getHBaseCluster().getMaster().getServerManager().
260             getDeadServers().isDeadServer(sn);
261       }
262     });
263 
264     TEST_UTIL.waitFor(40000, 1000, true, new Waiter.Predicate<Exception>() {
265       @Override
266       public boolean evaluate() throws Exception {
267         return hci.clusterStatusListener.isDeadServer(sn);
268       }
269     });
270 
271     t.close();
272     hci.getClient(sn);  // will throw an exception: RegionServerStoppedException
273   }
274 
275   /**
276    * Test that we can handle connection close: it will trigger a retry, but the calls will
277    *  finish.
278    */
279   @Test
280   public void testConnectionCloseAllowsInterrupt() throws Exception {
281     testConnectionClose(true);
282   }
283 
284   @Test
285   public void testConnectionNotAllowsInterrupt() throws Exception {
286     testConnectionClose(false);
287   }
288 
289   /**
290    * Test that an operation can fail if we read the global operation timeout, even if the
291    * individual timeout is fine. We do that with:
292    * - client side: an operation timeout of 30 seconds
293    * - server side: we sleep 20 second at each attempt. The first work fails, the second one
294    * succeeds. But the client won't wait that much, because 20 + 20 > 30, so the client
295    * timeouted when the server answers.
296    */
297   @Test
298   public void testOperationTimeout() throws Exception {
299     HTableDescriptor hdt = TEST_UTIL.createTableDescriptor("HCM-testOperationTimeout");
300     hdt.addCoprocessor(SleepAndFailFirstTime.class.getName());
301     HTable table = TEST_UTIL.createTable(hdt, new byte[][]{FAM_NAM}, TEST_UTIL.getConfiguration());
302 
303     // Check that it works if the timeout is big enough
304     table.setOperationTimeout(120 * 1000);
305     table.get(new Get(FAM_NAM));
306 
307     // Resetting and retrying. Will fail this time, not enough time for the second try
308     SleepAndFailFirstTime.ct.set(0);
309     try {
310       table.setOperationTimeout(30 * 1000);
311       table.get(new Get(FAM_NAM));
312       Assert.fail("We expect an exception here");
313     } catch (SocketTimeoutException e) {
314       // The client has a CallTimeout class, but it's not shared.We're not very clean today,
315       //  in the general case you can expect the call to stop, but the exception may vary.
316       // In this test however, we're sure that it will be a socket timeout.
317       LOG.info("We received an exception, as expected ", e);
318     } catch (IOException e) {
319       Assert.fail("Wrong exception:" + e.getMessage());
320     } finally {
321       table.close();
322     }
323   }
324 
325 
326   private void testConnectionClose(boolean allowsInterrupt) throws Exception {
327     TableName tableName = TableName.valueOf("HCM-testConnectionClose" + allowsInterrupt);
328     TEST_UTIL.createTable(tableName, FAM_NAM).close();
329 
330     boolean previousBalance = TEST_UTIL.getHBaseAdmin().setBalancerRunning(false, true);
331 
332     Configuration c2 = new Configuration(TEST_UTIL.getConfiguration());
333     // We want to work on a separate connection.
334     c2.set(HConstants.HBASE_CLIENT_INSTANCE_ID, String.valueOf(-1));
335     c2.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 100); // retry a lot
336     c2.setInt(HConstants.HBASE_CLIENT_PAUSE, 0); // don't wait between retries.
337     c2.setInt(RpcClient.FAILED_SERVER_EXPIRY_KEY, 0); // Server do not really expire
338     c2.setBoolean(RpcClient.SPECIFIC_WRITE_THREAD, allowsInterrupt);
339 
340     final HTable table = new HTable(c2, tableName);
341 
342     Put put = new Put(ROW);
343     put.add(FAM_NAM, ROW, ROW);
344     table.put(put);
345 
346     // 4 steps: ready=0; doGets=1; mustStop=2; stopped=3
347     final AtomicInteger step = new AtomicInteger(0);
348 
349     final AtomicReference<Throwable> failed = new AtomicReference<Throwable>(null);
350     Thread t = new Thread("testConnectionCloseThread") {
351       @Override
352       public void run() {
353         int done = 0;
354         try {
355           step.set(1);
356           while (step.get() == 1) {
357             Get get = new Get(ROW);
358             table.get(get);
359             done++;
360             if (done % 100 == 0)
361               LOG.info("done=" + done);
362           }
363         } catch (Throwable t) {
364           failed.set(t);
365           LOG.error(t);
366         }
367         step.set(3);
368       }
369     };
370     t.start();
371     TEST_UTIL.waitFor(20000, new Waiter.Predicate<Exception>() {
372       @Override
373       public boolean evaluate() throws Exception {
374         return step.get() == 1;
375       }
376     });
377 
378     ServerName sn = table.getRegionLocation(ROW).getServerName();
379     ConnectionManager.HConnectionImplementation conn =
380         (ConnectionManager.HConnectionImplementation) table.getConnection();
381     RpcClient rpcClient = conn.getRpcClient();
382 
383     LOG.info("Going to cancel connections. connection=" + conn.toString() + ", sn=" + sn);
384     for (int i = 0; i < 5000; i++) {
385       rpcClient.cancelConnections(sn);
386       Thread.sleep(5);
387     }
388 
389     step.compareAndSet(1, 2);
390     // The test may fail here if the thread doing the gets is stuck. The way to find
391     //  out what's happening is to look for the thread named 'testConnectionCloseThread'
392     TEST_UTIL.waitFor(40000, new Waiter.Predicate<Exception>() {
393       @Override
394       public boolean evaluate() throws Exception {
395         return step.get() == 3;
396       }
397     });
398     table.close();
399     Assert.assertTrue("Unexpected exception is " + failed.get(), failed.get() == null);
400     TEST_UTIL.getHBaseAdmin().setBalancerRunning(previousBalance, true);
401   }
402 
403   /**
404    * Test that connection can become idle without breaking everything.
405    */
406   @Test
407   public void testConnectionIdle() throws Exception {
408     TableName tableName = TableName.valueOf("HCM-testConnectionIdle");
409     TEST_UTIL.createTable(tableName, FAM_NAM).close();
410     int idleTime =  20000;
411     boolean previousBalance = TEST_UTIL.getHBaseAdmin().setBalancerRunning(false, true);
412 
413     Configuration c2 = new Configuration(TEST_UTIL.getConfiguration());
414     // We want to work on a separate connection.
415     c2.set(HConstants.HBASE_CLIENT_INSTANCE_ID, String.valueOf(-1));
416     c2.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); // Don't retry: retry = test failed
417     c2.setInt(RpcClient.IDLE_TIME, idleTime);
418 
419     final Table table = new HTable(c2, tableName);
420 
421     Put put = new Put(ROW);
422     put.add(FAM_NAM, ROW, ROW);
423     table.put(put);
424 
425     ManualEnvironmentEdge mee = new ManualEnvironmentEdge();
426     mee.setValue(System.currentTimeMillis());
427     EnvironmentEdgeManager.injectEdge(mee);
428     LOG.info("first get");
429     table.get(new Get(ROW));
430 
431     LOG.info("first get - changing the time & sleeping");
432     mee.incValue(idleTime + 1000);
433     Thread.sleep(1500); // we need to wait a little for the connection to be seen as idle.
434                         // 1500 = sleep time in RpcClient#waitForWork + a margin
435 
436     LOG.info("second get - connection has been marked idle in the middle");
437     // To check that the connection actually became idle would need to read some private
438     //  fields of RpcClient.
439     table.get(new Get(ROW));
440     mee.incValue(idleTime + 1000);
441 
442     LOG.info("third get - connection is idle, but the reader doesn't know yet");
443     // We're testing here a special case:
444     //  time limit reached BUT connection not yet reclaimed AND a new call.
445     //  in this situation, we don't close the connection, instead we use it immediately.
446     // If we're very unlucky we can have a race condition in the test: the connection is already
447     //  under closing when we do the get, so we have an exception, and we don't retry as the
448     //  retry number is 1. The probability is very very low, and seems acceptable for now. It's
449     //  a test issue only.
450     table.get(new Get(ROW));
451 
452     LOG.info("we're done - time will change back");
453 
454     table.close();
455     EnvironmentEdgeManager.reset();
456     TEST_UTIL.getHBaseAdmin().setBalancerRunning(previousBalance, true);
457   }
458 
459     /**
460      * Test that the connection to the dead server is cut immediately when we receive the
461      *  notification.
462      * @throws Exception
463      */
464   @Test
465   public void testConnectionCut() throws Exception {
466 
467     TableName tableName = TableName.valueOf("HCM-testConnectionCut");
468 
469     TEST_UTIL.createTable(tableName, FAM_NAM).close();
470     boolean previousBalance = TEST_UTIL.getHBaseAdmin().setBalancerRunning(false, true);
471 
472     Configuration c2 = new Configuration(TEST_UTIL.getConfiguration());
473     // We want to work on a separate connection.
474     c2.set(HConstants.HBASE_CLIENT_INSTANCE_ID, String.valueOf(-1));
475     c2.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
476     c2.setInt(HConstants.HBASE_RPC_TIMEOUT_KEY, 30 * 1000);
477 
478     HTable table = new HTable(c2, tableName);
479 
480     Put p = new Put(FAM_NAM);
481     p.add(FAM_NAM, FAM_NAM, FAM_NAM);
482     table.put(p);
483 
484     final HConnectionImplementation hci =  (HConnectionImplementation)table.getConnection();
485     final HRegionLocation loc = table.getRegionLocation(FAM_NAM);
486 
487     Get get = new Get(FAM_NAM);
488     Assert.assertNotNull(table.get(get));
489 
490     get = new Get(FAM_NAM);
491     get.setFilter(new BlockingFilter());
492 
493     // This thread will mark the server as dead while we're waiting during a get.
494     Thread t = new Thread() {
495       @Override
496       public void run() {
497         synchronized (syncBlockingFilter) {
498           try {
499             syncBlockingFilter.wait();
500           } catch (InterruptedException e) {
501             throw new RuntimeException(e);
502           }
503         }
504         hci.clusterStatusListener.deadServerHandler.newDead(loc.getServerName());
505       }
506     };
507 
508     t.start();
509     try {
510       table.get(get);
511       Assert.fail();
512     } catch (IOException expected) {
513       LOG.debug("Received: " + expected);
514       Assert.assertFalse(expected instanceof SocketTimeoutException);
515       Assert.assertFalse(syncBlockingFilter.get());
516     } finally {
517       syncBlockingFilter.set(true);
518       t.join();
519       HConnectionManager.getConnection(c2).close();
520       TEST_UTIL.getHBaseAdmin().setBalancerRunning(previousBalance, true);
521     }
522 
523     table.close();
524   }
525 
526   protected static final AtomicBoolean syncBlockingFilter = new AtomicBoolean(false);
527 
528   public static class BlockingFilter extends FilterBase {
529     @Override
530     public boolean filterRowKey(byte[] buffer, int offset, int length) throws IOException {
531       int i = 0;
532       while (i++ < 1000 && !syncBlockingFilter.get()) {
533         synchronized (syncBlockingFilter) {
534           syncBlockingFilter.notifyAll();
535         }
536         Threads.sleep(100);
537       }
538       syncBlockingFilter.set(true);
539       return false;
540     }
541     @Override
542     public ReturnCode filterKeyValue(Cell ignored) throws IOException {
543       return ReturnCode.INCLUDE;
544     }
545 
546     public static Filter parseFrom(final byte [] pbBytes) throws DeserializationException{
547       return new BlockingFilter();
548     }
549   }
550 
551   @Test
552   public void abortingHConnectionRemovesItselfFromHCM() throws Exception {
553     // Save off current HConnections
554     Map<HConnectionKey, HConnectionImplementation> oldHBaseInstances =
555         new HashMap<HConnectionKey, HConnectionImplementation>();
556     oldHBaseInstances.putAll(ConnectionManager.CONNECTION_INSTANCES);
557 
558     ConnectionManager.CONNECTION_INSTANCES.clear();
559 
560     try {
561       HConnection connection = HConnectionManager.getConnection(TEST_UTIL.getConfiguration());
562       connection.abort("test abortingHConnectionRemovesItselfFromHCM", new Exception(
563           "test abortingHConnectionRemovesItselfFromHCM"));
564       Assert.assertNotSame(connection,
565         HConnectionManager.getConnection(TEST_UTIL.getConfiguration()));
566     } finally {
567       // Put original HConnections back
568       ConnectionManager.CONNECTION_INSTANCES.clear();
569       ConnectionManager.CONNECTION_INSTANCES.putAll(oldHBaseInstances);
570     }
571   }
572 
573   /**
574    * Test that when we delete a location using the first row of a region
575    * that we really delete it.
576    * @throws Exception
577    */
578   @Test
579   public void testRegionCaching() throws Exception{
580     TEST_UTIL.createMultiRegionTable(TABLE_NAME, FAM_NAM).close();
581     Configuration conf =  new Configuration(TEST_UTIL.getConfiguration());
582     conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
583     HTable table = new HTable(conf, TABLE_NAME);
584 
585     TEST_UTIL.waitUntilAllRegionsAssigned(table.getName());
586     Put put = new Put(ROW);
587     put.add(FAM_NAM, ROW, ROW);
588     table.put(put);
589     ConnectionManager.HConnectionImplementation conn =
590       (ConnectionManager.HConnectionImplementation)table.getConnection();
591 
592     assertNotNull(conn.getCachedLocation(TABLE_NAME, ROW));
593 
594     final int nextPort = conn.getCachedLocation(TABLE_NAME, ROW).getRegionLocation().getPort() + 1;
595     HRegionLocation loc = conn.getCachedLocation(TABLE_NAME, ROW).getRegionLocation();
596     conn.updateCachedLocation(loc.getRegionInfo(), loc.getServerName(),
597         ServerName.valueOf("127.0.0.1", nextPort,
598         HConstants.LATEST_TIMESTAMP), HConstants.LATEST_TIMESTAMP);
599     Assert.assertEquals(conn.getCachedLocation(TABLE_NAME, ROW)
600       .getRegionLocation().getPort(), nextPort);
601 
602     conn.clearRegionCache(TABLE_NAME, ROW.clone());
603     RegionLocations rl = conn.getCachedLocation(TABLE_NAME, ROW);
604     assertNull("What is this location?? " + rl, rl);
605 
606     // We're now going to move the region and check that it works for the client
607     // First a new put to add the location in the cache
608     conn.clearRegionCache(TABLE_NAME);
609     Assert.assertEquals(0, conn.getNumberOfCachedRegionLocations(TABLE_NAME));
610     Put put2 = new Put(ROW);
611     put2.add(FAM_NAM, ROW, ROW);
612     table.put(put2);
613     assertNotNull(conn.getCachedLocation(TABLE_NAME, ROW));
614     assertNotNull(conn.getCachedLocation(TableName.valueOf(TABLE_NAME.getName()), ROW.clone()));
615 
616     TEST_UTIL.getHBaseAdmin().setBalancerRunning(false, false);
617     HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
618 
619     // We can wait for all regions to be online, that makes log reading easier when debugging
620     while (master.getAssignmentManager().getRegionStates().isRegionsInTransition()) {
621       Thread.sleep(1);
622     }
623 
624     // Now moving the region to the second server
625     HRegionLocation toMove = conn.getCachedLocation(TABLE_NAME, ROW).getRegionLocation();
626     byte[] regionName = toMove.getRegionInfo().getRegionName();
627     byte[] encodedRegionNameBytes = toMove.getRegionInfo().getEncodedNameAsBytes();
628 
629     // Choose the other server.
630     int curServerId = TEST_UTIL.getHBaseCluster().getServerWith(regionName);
631     int destServerId = (curServerId == 0 ? 1 : 0);
632 
633     HRegionServer curServer = TEST_UTIL.getHBaseCluster().getRegionServer(curServerId);
634     HRegionServer destServer = TEST_UTIL.getHBaseCluster().getRegionServer(destServerId);
635 
636     ServerName destServerName = destServer.getServerName();
637 
638     // Check that we are in the expected state
639     Assert.assertTrue(curServer != destServer);
640     Assert.assertFalse(curServer.getServerName().equals(destServer.getServerName()));
641     Assert.assertFalse( toMove.getPort() == destServerName.getPort());
642     Assert.assertNotNull(curServer.getOnlineRegion(regionName));
643     Assert.assertNull(destServer.getOnlineRegion(regionName));
644     Assert.assertFalse(TEST_UTIL.getMiniHBaseCluster().getMaster().
645         getAssignmentManager().getRegionStates().isRegionsInTransition());
646 
647     // Moving. It's possible that we don't have all the regions online at this point, so
648     //  the test must depends only on the region we're looking at.
649     LOG.info("Move starting region="+toMove.getRegionInfo().getRegionNameAsString());
650     TEST_UTIL.getHBaseAdmin().move(
651       toMove.getRegionInfo().getEncodedNameAsBytes(),
652       destServerName.getServerName().getBytes()
653     );
654 
655     while (destServer.getOnlineRegion(regionName) == null ||
656         destServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes) ||
657         curServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes) ||
658         master.getAssignmentManager().getRegionStates().isRegionsInTransition()) {
659       // wait for the move to be finished
660       Thread.sleep(1);
661     }
662 
663     LOG.info("Move finished for region="+toMove.getRegionInfo().getRegionNameAsString());
664 
665     // Check our new state.
666     Assert.assertNull(curServer.getOnlineRegion(regionName));
667     Assert.assertNotNull(destServer.getOnlineRegion(regionName));
668     Assert.assertFalse(destServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes));
669     Assert.assertFalse(curServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes));
670 
671 
672     // Cache was NOT updated and points to the wrong server
673     Assert.assertFalse(
674         conn.getCachedLocation(TABLE_NAME, ROW).getRegionLocation()
675           .getPort() == destServerName.getPort());
676 
677     // This part relies on a number of tries equals to 1.
678     // We do a put and expect the cache to be updated, even if we don't retry
679     LOG.info("Put starting");
680     Put put3 = new Put(ROW);
681     put3.add(FAM_NAM, ROW, ROW);
682     try {
683       table.put(put3);
684       Assert.fail("Unreachable point");
685     } catch (RetriesExhaustedWithDetailsException e){
686       LOG.info("Put done, exception caught: " + e.getClass());
687       Assert.assertEquals(1, e.getNumExceptions());
688       Assert.assertEquals(1, e.getCauses().size());
689       Assert.assertArrayEquals(e.getRow(0).getRow(), ROW);
690 
691       // Check that we unserialized the exception as expected
692       Throwable cause = ConnectionManager.findException(e.getCause(0));
693       Assert.assertNotNull(cause);
694       Assert.assertTrue(cause instanceof RegionMovedException);
695     }
696     Assert.assertNotNull("Cached connection is null", conn.getCachedLocation(TABLE_NAME, ROW));
697     Assert.assertEquals(
698         "Previous server was " + curServer.getServerName().getHostAndPort(),
699         destServerName.getPort(),
700         conn.getCachedLocation(TABLE_NAME, ROW).getRegionLocation().getPort());
701 
702     Assert.assertFalse(destServer.getRegionsInTransitionInRS()
703       .containsKey(encodedRegionNameBytes));
704     Assert.assertFalse(curServer.getRegionsInTransitionInRS()
705       .containsKey(encodedRegionNameBytes));
706 
707     // We move it back to do another test with a scan
708     LOG.info("Move starting region=" + toMove.getRegionInfo().getRegionNameAsString());
709     TEST_UTIL.getHBaseAdmin().move(
710       toMove.getRegionInfo().getEncodedNameAsBytes(),
711       curServer.getServerName().getServerName().getBytes()
712     );
713 
714     while (curServer.getOnlineRegion(regionName) == null ||
715         destServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes) ||
716         curServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes) ||
717         master.getAssignmentManager().getRegionStates().isRegionsInTransition()) {
718       // wait for the move to be finished
719       Thread.sleep(1);
720     }
721 
722     // Check our new state.
723     Assert.assertNotNull(curServer.getOnlineRegion(regionName));
724     Assert.assertNull(destServer.getOnlineRegion(regionName));
725     LOG.info("Move finished for region=" + toMove.getRegionInfo().getRegionNameAsString());
726 
727     // Cache was NOT updated and points to the wrong server
728     Assert.assertFalse(conn.getCachedLocation(TABLE_NAME, ROW).getRegionLocation().getPort() ==
729       curServer.getServerName().getPort());
730 
731     Scan sc = new Scan();
732     sc.setStopRow(ROW);
733     sc.setStartRow(ROW);
734 
735     // The scanner takes the max retries from the connection configuration, not the table as
736     // the put.
737     TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
738 
739     try {
740       ResultScanner rs = table.getScanner(sc);
741       while (rs.next() != null) {
742       }
743       Assert.fail("Unreachable point");
744     } catch (RetriesExhaustedException e) {
745       LOG.info("Scan done, expected exception caught: " + e.getClass());
746     }
747 
748     // Cache is updated with the right value.
749     Assert.assertNotNull(conn.getCachedLocation(TABLE_NAME, ROW));
750     Assert.assertEquals(
751       "Previous server was "+destServer.getServerName().getHostAndPort(),
752       curServer.getServerName().getPort(),
753       conn.getCachedLocation(TABLE_NAME, ROW).getRegionLocation().getPort());
754 
755     TEST_UTIL.getConfiguration().setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER,
756         HConstants.DEFAULT_HBASE_CLIENT_RETRIES_NUMBER);
757     table.close();
758   }
759 
760   /**
761    * Test that Connection or Pool are not closed when managed externally
762    * @throws Exception
763    */
764   @Test
765   public void testConnectionManagement() throws Exception{
766     Table table0 = TEST_UTIL.createTable(TABLE_NAME1, FAM_NAM);
767     Connection conn = ConnectionFactory.createConnection(TEST_UTIL.getConfiguration());
768     HTable table = (HTable) conn.getTable(TABLE_NAME1);
769     table.close();
770     assertFalse(conn.isClosed());
771     assertFalse(table.getPool().isShutdown());
772     table = (HTable) conn.getTable(TABLE_NAME1);
773     table.close();
774     assertFalse(table.getPool().isShutdown());
775     conn.close();
776     assertTrue(table.getPool().isShutdown());
777     table0.close();
778   }
779 
780   /**
781    * Test that stale cache updates don't override newer cached values.
782    */
783   @Test(timeout = 60000)
784   public void testCacheSeqNums() throws Exception{
785     HTable table = TEST_UTIL.createMultiRegionTable(TABLE_NAME2, FAM_NAM);
786     Put put = new Put(ROW);
787     put.add(FAM_NAM, ROW, ROW);
788     table.put(put);
789     ConnectionManager.HConnectionImplementation conn =
790       (ConnectionManager.HConnectionImplementation)table.getConnection();
791 
792     HRegionLocation location = conn.getCachedLocation(TABLE_NAME2, ROW).getRegionLocation();
793     assertNotNull(location);
794 
795     ServerName anySource = ServerName.valueOf(location.getHostname(), location.getPort() - 1, 0L);
796 
797     // Same server as already in cache reporting - overwrites any value despite seqNum.
798     int nextPort = location.getPort() + 1;
799     conn.updateCachedLocation(location.getRegionInfo(), location.getServerName(),
800         ServerName.valueOf("127.0.0.1", nextPort, 0), location.getSeqNum() - 1);
801     location = conn.getCachedLocation(TABLE_NAME2, ROW).getRegionLocation();
802     Assert.assertEquals(nextPort, location.getPort());
803 
804     // No source specified - same.
805     nextPort = location.getPort() + 1;
806     conn.updateCachedLocation(location.getRegionInfo(), location.getServerName(),
807         ServerName.valueOf("127.0.0.1", nextPort, 0), location.getSeqNum() - 1);
808     location = conn.getCachedLocation(TABLE_NAME2, ROW).getRegionLocation();
809     Assert.assertEquals(nextPort, location.getPort());
810 
811     // Higher seqNum - overwrites lower seqNum.
812     nextPort = location.getPort() + 1;
813     conn.updateCachedLocation(location.getRegionInfo(), anySource,
814         ServerName.valueOf("127.0.0.1", nextPort, 0), location.getSeqNum() + 1);
815     location = conn.getCachedLocation(TABLE_NAME2, ROW).getRegionLocation();
816     Assert.assertEquals(nextPort, location.getPort());
817 
818     // Lower seqNum - does not overwrite higher seqNum.
819     nextPort = location.getPort() + 1;
820     conn.updateCachedLocation(location.getRegionInfo(), anySource,
821         ServerName.valueOf("127.0.0.1", nextPort, 0), location.getSeqNum() - 1);
822     location = conn.getCachedLocation(TABLE_NAME2, ROW).getRegionLocation();
823     Assert.assertEquals(nextPort - 1, location.getPort());
824     table.close();
825   }
826 
827   /**
828    * Make sure that {@link Configuration} instances that are essentially the
829    * same map to the same {@link HConnection} instance.
830    */
831   @Test
832   public void testConnectionSameness() throws Exception {
833     Connection previousConnection = null;
834     for (int i = 0; i < 2; i++) {
835       // set random key to differentiate the connection from previous ones
836       Configuration configuration = TEST_UTIL.getConfiguration();
837       configuration.set("some_key", String.valueOf(_randy.nextInt()));
838       LOG.info("The hash code of the current configuration is: "
839           + configuration.hashCode());
840       Connection currentConnection = HConnectionManager
841           .getConnection(configuration);
842       if (previousConnection != null) {
843         assertTrue(
844             "Did not get the same connection even though its key didn't change",
845             previousConnection == currentConnection);
846       }
847       previousConnection = currentConnection;
848       // change the configuration, so that it is no longer reachable from the
849       // client's perspective. However, since its part of the LRU doubly linked
850       // list, it will eventually get thrown out, at which time it should also
851       // close the corresponding {@link HConnection}.
852       configuration.set("other_key", String.valueOf(_randy.nextInt()));
853     }
854   }
855 
856   /**
857    * Makes sure that there is no leaking of
858    * {@link ConnectionManager.HConnectionImplementation} in the {@link HConnectionManager}
859    * class.
860    * @deprecated Tests deprecated functionality.  Remove in 1.0.
861    */
862   @Deprecated
863   @Test
864   public void testConnectionUniqueness() throws Exception {
865     int zkmaxconnections = TEST_UTIL.getConfiguration().
866       getInt(HConstants.ZOOKEEPER_MAX_CLIENT_CNXNS,
867           HConstants.DEFAULT_ZOOKEPER_MAX_CLIENT_CNXNS);
868     // Test up to a max that is < the maximum number of zk connections.  If we
869     // go above zk connections, we just fall into cycle where we are failing
870     // to set up a session and test runs for a long time.
871     int maxConnections = Math.min(zkmaxconnections - 1, 20);
872     List<HConnection> connections = new ArrayList<HConnection>(maxConnections);
873     Connection previousConnection = null;
874     try {
875       for (int i = 0; i < maxConnections; i++) {
876         // set random key to differentiate the connection from previous ones
877         Configuration configuration = new Configuration(TEST_UTIL.getConfiguration());
878         configuration.set("some_key", String.valueOf(_randy.nextInt()));
879         configuration.set(HConstants.HBASE_CLIENT_INSTANCE_ID,
880             String.valueOf(_randy.nextInt()));
881         LOG.info("The hash code of the current configuration is: "
882             + configuration.hashCode());
883         HConnection currentConnection =
884           HConnectionManager.getConnection(configuration);
885         if (previousConnection != null) {
886           assertTrue("Got the same connection even though its key changed!",
887               previousConnection != currentConnection);
888         }
889         // change the configuration, so that it is no longer reachable from the
890         // client's perspective. However, since its part of the LRU doubly linked
891         // list, it will eventually get thrown out, at which time it should also
892         // close the corresponding {@link HConnection}.
893         configuration.set("other_key", String.valueOf(_randy.nextInt()));
894 
895         previousConnection = currentConnection;
896         LOG.info("The current HConnectionManager#HBASE_INSTANCES cache size is: "
897             + getHConnectionManagerCacheSize());
898         Thread.sleep(50);
899         connections.add(currentConnection);
900       }
901     } finally {
902       for (Connection c: connections) {
903         // Clean up connections made so we don't interfere w/ subsequent tests.
904         HConnectionManager.deleteConnection(c.getConfiguration());
905       }
906     }
907   }
908 
909   @Test
910   public void testClosing() throws Exception {
911     Configuration configuration =
912       new Configuration(TEST_UTIL.getConfiguration());
913     configuration.set(HConstants.HBASE_CLIENT_INSTANCE_ID,
914         String.valueOf(_randy.nextInt()));
915 
916     Connection c1 = ConnectionFactory.createConnection(configuration);
917     // We create two connections with the same key.
918     Connection c2 = ConnectionFactory.createConnection(configuration);
919 
920     Connection c3 = HConnectionManager.getConnection(configuration);
921     Connection c4 = HConnectionManager.getConnection(configuration);
922     assertTrue(c3 == c4);
923 
924     c1.close();
925     assertTrue(c1.isClosed());
926     assertFalse(c2.isClosed());
927     assertFalse(c3.isClosed());
928 
929     c3.close();
930     // still a reference left
931     assertFalse(c3.isClosed());
932     c3.close();
933     assertTrue(c3.isClosed());
934     // c3 was removed from the cache
935     Connection c5 = HConnectionManager.getConnection(configuration);
936     assertTrue(c5 != c3);
937 
938     assertFalse(c2.isClosed());
939     c2.close();
940     assertTrue(c2.isClosed());
941     c5.close();
942     assertTrue(c5.isClosed());
943   }
944 
945   /**
946    * Trivial test to verify that nobody messes with
947    * {@link HConnectionManager#createConnection(Configuration)}
948    */
949   @Test
950   public void testCreateConnection() throws Exception {
951     Configuration configuration = TEST_UTIL.getConfiguration();
952     Connection c1 = ConnectionFactory.createConnection(configuration);
953     Connection c2 = ConnectionFactory.createConnection(configuration);
954     // created from the same configuration, yet they are different
955     assertTrue(c1 != c2);
956     assertTrue(c1.getConfiguration() == c2.getConfiguration());
957     // make sure these were not cached
958     Connection c3 = HConnectionManager.getConnection(configuration);
959     assertTrue(c1 != c3);
960     assertTrue(c2 != c3);
961   }
962 
963 
964   /**
965    * This test checks that one can connect to the cluster with only the
966    *  ZooKeeper quorum set. Other stuff like master address will be read
967    *  from ZK by the client.
968    */
969   @Test(timeout = 60000)
970   public void testConnection() throws Exception{
971     // We create an empty config and add the ZK address.
972     Configuration c = new Configuration();
973     c.set(HConstants.ZOOKEEPER_QUORUM,
974       TEST_UTIL.getConfiguration().get(HConstants.ZOOKEEPER_QUORUM));
975     c.set(HConstants.ZOOKEEPER_CLIENT_PORT ,
976       TEST_UTIL.getConfiguration().get(HConstants.ZOOKEEPER_CLIENT_PORT));
977 
978     // This should be enough to connect
979     HConnection conn = HConnectionManager.getConnection(c);
980     assertTrue( conn.isMasterRunning() );
981     conn.close();
982   }
983 
984   private int setNumTries(HConnectionImplementation hci, int newVal) throws Exception {
985     Field numTries = hci.getClass().getDeclaredField("numTries");
986     numTries.setAccessible(true);
987     Field modifiersField = Field.class.getDeclaredField("modifiers");
988     modifiersField.setAccessible(true);
989     modifiersField.setInt(numTries, numTries.getModifiers() & ~Modifier.FINAL);
990     final int prevNumRetriesVal = (Integer)numTries.get(hci);
991     numTries.set(hci, newVal);
992 
993     return prevNumRetriesVal;
994   }
995 
996   @Test (timeout=30000)
997   public void testMulti() throws Exception {
998     HTable table = TEST_UTIL.createMultiRegionTable(TABLE_NAME3, FAM_NAM);
999      try {
1000        ConnectionManager.HConnectionImplementation conn =
1001            ( ConnectionManager.HConnectionImplementation)table.getConnection();
1002 
1003        // We're now going to move the region and check that it works for the client
1004        // First a new put to add the location in the cache
1005        conn.clearRegionCache(TABLE_NAME3);
1006        Assert.assertEquals(0, conn.getNumberOfCachedRegionLocations(TABLE_NAME3));
1007 
1008        TEST_UTIL.getHBaseAdmin().setBalancerRunning(false, false);
1009        HMaster master = TEST_UTIL.getMiniHBaseCluster().getMaster();
1010 
1011        // We can wait for all regions to be online, that makes log reading easier when debugging
1012        while (master.getAssignmentManager().getRegionStates().isRegionsInTransition()) {
1013          Thread.sleep(1);
1014        }
1015 
1016        Put put = new Put(ROW_X);
1017        put.add(FAM_NAM, ROW_X, ROW_X);
1018        table.put(put);
1019 
1020        // Now moving the region to the second server
1021        HRegionLocation toMove = conn.getCachedLocation(TABLE_NAME3, ROW_X).getRegionLocation();
1022        byte[] regionName = toMove.getRegionInfo().getRegionName();
1023        byte[] encodedRegionNameBytes = toMove.getRegionInfo().getEncodedNameAsBytes();
1024 
1025        // Choose the other server.
1026        int curServerId = TEST_UTIL.getHBaseCluster().getServerWith(regionName);
1027        int destServerId = (curServerId == 0 ? 1 : 0);
1028 
1029        HRegionServer curServer = TEST_UTIL.getHBaseCluster().getRegionServer(curServerId);
1030        HRegionServer destServer = TEST_UTIL.getHBaseCluster().getRegionServer(destServerId);
1031 
1032        ServerName destServerName = destServer.getServerName();
1033 
1034        //find another row in the cur server that is less than ROW_X
1035        List<Region> regions = curServer.getOnlineRegions(TABLE_NAME3);
1036        byte[] otherRow = null;
1037        for (Region region : regions) {
1038          if (!region.getRegionInfo().getEncodedName().equals(toMove.getRegionInfo().getEncodedName())
1039              && Bytes.BYTES_COMPARATOR.compare(region.getRegionInfo().getStartKey(), ROW_X) < 0) {
1040            otherRow = region.getRegionInfo().getStartKey();
1041            break;
1042          }
1043        }
1044        assertNotNull(otherRow);
1045        // If empty row, set it to first row.-f
1046        if (otherRow.length <= 0) otherRow = Bytes.toBytes("aaa");
1047        Put put2 = new Put(otherRow);
1048        put2.add(FAM_NAM, otherRow, otherRow);
1049        table.put(put2); //cache put2's location
1050 
1051        // Check that we are in the expected state
1052        Assert.assertTrue(curServer != destServer);
1053        Assert.assertNotEquals(curServer.getServerName(), destServer.getServerName());
1054        Assert.assertNotEquals(toMove.getPort(), destServerName.getPort());
1055        Assert.assertNotNull(curServer.getOnlineRegion(regionName));
1056        Assert.assertNull(destServer.getOnlineRegion(regionName));
1057        Assert.assertFalse(TEST_UTIL.getMiniHBaseCluster().getMaster().
1058            getAssignmentManager().getRegionStates().isRegionsInTransition());
1059 
1060        // Moving. It's possible that we don't have all the regions online at this point, so
1061        //  the test must depends only on the region we're looking at.
1062        LOG.info("Move starting region="+toMove.getRegionInfo().getRegionNameAsString());
1063        TEST_UTIL.getHBaseAdmin().move(
1064            toMove.getRegionInfo().getEncodedNameAsBytes(),
1065            destServerName.getServerName().getBytes()
1066            );
1067 
1068        while (destServer.getOnlineRegion(regionName) == null ||
1069            destServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes) ||
1070            curServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes) ||
1071            master.getAssignmentManager().getRegionStates().isRegionsInTransition()) {
1072          // wait for the move to be finished
1073          Thread.sleep(1);
1074         }
1075 
1076        LOG.info("Move finished for region="+toMove.getRegionInfo().getRegionNameAsString());
1077 
1078        // Check our new state.
1079        Assert.assertNull(curServer.getOnlineRegion(regionName));
1080        Assert.assertNotNull(destServer.getOnlineRegion(regionName));
1081        Assert.assertFalse(destServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes));
1082        Assert.assertFalse(curServer.getRegionsInTransitionInRS().containsKey(encodedRegionNameBytes));
1083 
1084 
1085        // Cache was NOT updated and points to the wrong server
1086        Assert.assertFalse(
1087            conn.getCachedLocation(TABLE_NAME3, ROW_X).getRegionLocation()
1088             .getPort() == destServerName.getPort());
1089 
1090        // Hijack the number of retry to fail after 2 tries
1091        final int prevNumRetriesVal = setNumTries(conn, 2);
1092 
1093        Put put3 = new Put(ROW_X);
1094        put3.add(FAM_NAM, ROW_X, ROW_X);
1095        Put put4 = new Put(otherRow);
1096        put4.add(FAM_NAM, otherRow, otherRow);
1097 
1098        // do multi
1099        table.batch(Lists.newArrayList(put4, put3)); // first should be a valid row,
1100        // second we get RegionMovedException.
1101 
1102        setNumTries(conn, prevNumRetriesVal);
1103      } finally {
1104        table.close();
1105      }
1106   }
1107 
1108   @Ignore ("Test presumes RETRY_BACKOFF will never change; it has") @Test
1109   public void testErrorBackoffTimeCalculation() throws Exception {
1110     // TODO: This test would seem to presume hardcoded RETRY_BACKOFF which it should not.
1111     final long ANY_PAUSE = 100;
1112     ServerName location = ServerName.valueOf("127.0.0.1", 1, 0);
1113     ServerName diffLocation = ServerName.valueOf("127.0.0.1", 2, 0);
1114 
1115     ManualEnvironmentEdge timeMachine = new ManualEnvironmentEdge();
1116     EnvironmentEdgeManager.injectEdge(timeMachine);
1117     try {
1118       long timeBase = timeMachine.currentTime();
1119       long largeAmountOfTime = ANY_PAUSE * 1000;
1120       ConnectionManager.ServerErrorTracker tracker =
1121           new ConnectionManager.ServerErrorTracker(largeAmountOfTime, 100);
1122 
1123       // The default backoff is 0.
1124       assertEquals(0, tracker.calculateBackoffTime(location, ANY_PAUSE));
1125 
1126       // Check some backoff values from HConstants sequence.
1127       tracker.reportServerError(location);
1128       assertEqualsWithJitter(ANY_PAUSE, tracker.calculateBackoffTime(location, ANY_PAUSE));
1129       tracker.reportServerError(location);
1130       tracker.reportServerError(location);
1131       tracker.reportServerError(location);
1132       assertEqualsWithJitter(ANY_PAUSE * 5, tracker.calculateBackoffTime(location, ANY_PAUSE));
1133 
1134       // All of this shouldn't affect backoff for different location.
1135       assertEquals(0, tracker.calculateBackoffTime(diffLocation, ANY_PAUSE));
1136       tracker.reportServerError(diffLocation);
1137       assertEqualsWithJitter(ANY_PAUSE, tracker.calculateBackoffTime(diffLocation, ANY_PAUSE));
1138 
1139       // Check with different base.
1140       assertEqualsWithJitter(ANY_PAUSE * 10,
1141           tracker.calculateBackoffTime(location, ANY_PAUSE * 2));
1142 
1143       // See that time from last error is taken into account. Time shift is applied after jitter,
1144       // so pass the original expected backoff as the base for jitter.
1145       long timeShift = (long)(ANY_PAUSE * 0.5);
1146       timeMachine.setValue(timeBase + timeShift);
1147       assertEqualsWithJitter((ANY_PAUSE * 5) - timeShift,
1148         tracker.calculateBackoffTime(location, ANY_PAUSE), ANY_PAUSE * 2);
1149 
1150       // However we should not go into negative.
1151       timeMachine.setValue(timeBase + ANY_PAUSE * 100);
1152       assertEquals(0, tracker.calculateBackoffTime(location, ANY_PAUSE));
1153 
1154       // We also should not go over the boundary; last retry would be on it.
1155       long timeLeft = (long)(ANY_PAUSE * 0.5);
1156       timeMachine.setValue(timeBase + largeAmountOfTime - timeLeft);
1157       assertTrue(tracker.canRetryMore(1));
1158       tracker.reportServerError(location);
1159       assertEquals(timeLeft, tracker.calculateBackoffTime(location, ANY_PAUSE));
1160       timeMachine.setValue(timeBase + largeAmountOfTime);
1161       assertFalse(tracker.canRetryMore(1));
1162     } finally {
1163       EnvironmentEdgeManager.reset();
1164     }
1165   }
1166 
1167   private static void assertEqualsWithJitter(long expected, long actual) {
1168     assertEqualsWithJitter(expected, actual, expected);
1169   }
1170 
1171   private static void assertEqualsWithJitter(long expected, long actual, long jitterBase) {
1172     assertTrue("Value not within jitter: " + expected + " vs " + actual,
1173         Math.abs(actual - expected) <= (0.01f * jitterBase));
1174   }
1175 
1176   /**
1177    * Tests that a destroyed connection does not have a live zookeeper.
1178    * Below is timing based.  We put up a connection to a table and then close the connection while
1179    * having a background thread running that is forcing close of the connection to try and
1180    * provoke a close catastrophe; we are hoping for a car crash so we can see if we are leaking
1181    * zk connections.
1182    * @throws Exception
1183    */
1184   @Ignore ("Flakey test: See HBASE-8996")@Test
1185   public void testDeleteForZKConnLeak() throws Exception {
1186     TEST_UTIL.createTable(TABLE_NAME4, FAM_NAM);
1187     final Configuration config = HBaseConfiguration.create(TEST_UTIL.getConfiguration());
1188     config.setInt("zookeeper.recovery.retry", 1);
1189     config.setInt("zookeeper.recovery.retry.intervalmill", 1000);
1190     config.setInt("hbase.rpc.timeout", 2000);
1191     config.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1);
1192 
1193     ThreadPoolExecutor pool = new ThreadPoolExecutor(1, 10,
1194       5, TimeUnit.SECONDS,
1195       new SynchronousQueue<Runnable>(),
1196       Threads.newDaemonThreadFactory("test-hcm-delete"));
1197 
1198     pool.submit(new Runnable() {
1199       @Override
1200       public void run() {
1201         while (!Thread.interrupted()) {
1202           try {
1203             HConnection conn = HConnectionManager.getConnection(config);
1204             LOG.info("Connection " + conn);
1205             HConnectionManager.deleteStaleConnection(conn);
1206             LOG.info("Connection closed " + conn);
1207             // TODO: This sleep time should be less than the time that it takes to open and close
1208             // a table.  Ideally we would do a few runs first to measure.  For now this is
1209             // timing based; hopefully we hit the bad condition.
1210             Threads.sleep(10);
1211           } catch (Exception e) {
1212           }
1213         }
1214       }
1215     });
1216 
1217     // Use connection multiple times.
1218     for (int i = 0; i < 30; i++) {
1219       Connection c1 = null;
1220       try {
1221         c1 = ConnectionManager.getConnectionInternal(config);
1222         LOG.info("HTable connection " + i + " " + c1);
1223         Table table = new HTable(config, TABLE_NAME4, pool);
1224         table.close();
1225         LOG.info("HTable connection " + i + " closed " + c1);
1226       } catch (Exception e) {
1227         LOG.info("We actually want this to happen!!!!  So we can see if we are leaking zk", e);
1228       } finally {
1229         if (c1 != null) {
1230           if (c1.isClosed()) {
1231             // cannot use getZooKeeper as method instantiates watcher if null
1232             Field zkwField = c1.getClass().getDeclaredField("keepAliveZookeeper");
1233             zkwField.setAccessible(true);
1234             Object watcher = zkwField.get(c1);
1235 
1236             if (watcher != null) {
1237               if (((ZooKeeperWatcher)watcher).getRecoverableZooKeeper().getState().isAlive()) {
1238                 // non-synchronized access to watcher; sleep and check again in case zk connection
1239                 // hasn't been cleaned up yet.
1240                 Thread.sleep(1000);
1241                 if (((ZooKeeperWatcher) watcher).getRecoverableZooKeeper().getState().isAlive()) {
1242                   pool.shutdownNow();
1243                   fail("Live zookeeper in closed connection");
1244                 }
1245               }
1246             }
1247           }
1248           c1.close();
1249         }
1250       }
1251     }
1252     pool.shutdownNow();
1253   }
1254 
1255   @Test(timeout = 60000)
1256   public void testConnectionRideOverClusterRestart() throws IOException, InterruptedException {
1257     Configuration config = new Configuration(TEST_UTIL.getConfiguration());
1258 
1259     TableName tableName = TableName.valueOf("testConnectionRideOverClusterRestart");
1260     TEST_UTIL.createTable(tableName.getName(), new byte[][] {FAM_NAM}, config).close();
1261 
1262     Connection connection = ConnectionFactory.createConnection(config);
1263     Table table = connection.getTable(tableName);
1264 
1265     // this will cache the meta location and table's region location
1266     table.get(new Get(Bytes.toBytes("foo")));
1267 
1268     // restart HBase
1269     TEST_UTIL.shutdownMiniHBaseCluster();
1270     TEST_UTIL.restartHBaseCluster(2);
1271     // this should be able to discover new locations for meta and table's region
1272     table.get(new Get(Bytes.toBytes("foo")));
1273     TEST_UTIL.deleteTable(tableName);
1274     table.close();
1275     connection.close();
1276   }
1277 }
1278