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.regionserver;
21  
22  import static org.junit.Assert.assertEquals;
23  import static org.junit.Assert.assertNotNull;
24  import static org.junit.Assert.assertNull;
25  import static org.junit.Assert.assertNotSame;
26  import static org.junit.Assert.assertTrue;
27  import static org.junit.Assert.assertFalse;
28  import static org.junit.Assert.fail;
29  
30  import java.io.IOException;
31  import java.util.List;
32  import java.util.NavigableMap;
33  import java.util.concurrent.CountDownLatch;
34  
35  import org.apache.commons.logging.Log;
36  import org.apache.commons.logging.LogFactory;
37  import org.apache.hadoop.conf.Configuration;
38  import org.apache.hadoop.fs.FileSystem;
39  import org.apache.hadoop.fs.Path;
40  import org.apache.hadoop.hbase.*;
41  import org.apache.hadoop.hbase.catalog.MetaReader;
42  import org.apache.hadoop.hbase.client.Delete;
43  import org.apache.hadoop.hbase.client.HBaseAdmin;
44  import org.apache.hadoop.hbase.client.HTable;
45  import org.apache.hadoop.hbase.client.Put;
46  import org.apache.hadoop.hbase.executor.EventHandler.EventType;
47  import org.apache.hadoop.hbase.executor.RegionTransitionData;
48  import org.apache.hadoop.hbase.master.AssignmentManager;
49  import org.apache.hadoop.hbase.master.AssignmentManager.RegionState;
50  import org.apache.hadoop.hbase.master.HMaster;
51  import org.apache.hadoop.hbase.master.handler.SplitRegionHandler;
52  import org.apache.hadoop.hbase.util.Bytes;
53  import org.apache.hadoop.hbase.util.JVMClusterUtil.RegionServerThread;
54  import org.apache.hadoop.hbase.util.Threads;
55  import org.apache.hadoop.hbase.zookeeper.ZKAssign;
56  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
57  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
58  import org.apache.zookeeper.KeeperException;
59  import org.apache.zookeeper.KeeperException.NodeExistsException;
60  import org.apache.zookeeper.data.Stat;
61  import org.junit.AfterClass;
62  import org.junit.Before;
63  import org.junit.BeforeClass;
64  import org.junit.Test;
65  import org.junit.experimental.categories.Category;
66  
67  /**
68   * Like {@link TestSplitTransaction} in that we're testing {@link SplitTransaction}
69   * only the below tests are against a running cluster where {@link TestSplitTransaction}
70   * is tests against a bare {@link HRegion}.
71   */
72  @Category(LargeTests.class)
73  public class TestSplitTransactionOnCluster {
74    private static final Log LOG =
75      LogFactory.getLog(TestSplitTransactionOnCluster.class);
76    private HBaseAdmin admin = null;
77    private MiniHBaseCluster cluster = null;
78    private static final int NB_SERVERS = 2;
79    private static CountDownLatch latch = new CountDownLatch(1);
80    private static volatile boolean secondSplit = false;
81    private static volatile boolean callRollBack = false;
82    private static volatile boolean firstSplitCompleted = false;
83    
84    private static final HBaseTestingUtility TESTING_UTIL =
85      new HBaseTestingUtility();
86  
87    @BeforeClass public static void before() throws Exception {
88      TESTING_UTIL.getConfiguration().setInt("hbase.balancer.period", 60000);
89      // Needed because some tests have splits happening on RS that are killed
90      // We don't want to wait 3min for the master to figure it out
91      TESTING_UTIL.getConfiguration().setInt(
92          "hbase.master.assignment.timeoutmonitor.timeout", 4000);
93      TESTING_UTIL.startMiniCluster(NB_SERVERS);
94    }
95  
96    @AfterClass public static void after() throws Exception {
97      TESTING_UTIL.shutdownMiniCluster();
98    }
99  
100   @Before public void setup() throws IOException {
101     TESTING_UTIL.ensureSomeNonStoppedRegionServersAvailable(NB_SERVERS);
102     this.admin = new HBaseAdmin(TESTING_UTIL.getConfiguration());
103     this.cluster = TESTING_UTIL.getMiniHBaseCluster();
104   }
105 
106   private HRegionInfo getAndCheckSingleTableRegion(final List<HRegion> regions) {
107     assertEquals(1, regions.size());
108     return regions.get(0).getRegionInfo();
109   }
110 
111   @Test(timeout = 2000000)
112   public void testShouldFailSplitIfZNodeDoesNotExistDueToPrevRollBack() throws Exception {
113     final byte[] tableName = Bytes
114         .toBytes("testShouldFailSplitIfZNodeDoesNotExistDueToPrevRollBack");
115     HBaseAdmin admin = new HBaseAdmin(TESTING_UTIL.getConfiguration());
116     try {
117       // Create table then get the single region for our new table.
118       HTable t = createTableAndWait(tableName, Bytes.toBytes("cf"));
119       final List<HRegion> regions = cluster.getRegions(tableName);
120       HRegionInfo hri = getAndCheckSingleTableRegion(regions);
121       int regionServerIndex = cluster.getServerWith(regions.get(0).getRegionName());
122       final HRegionServer regionServer = cluster.getRegionServer(regionServerIndex);
123       insertData(tableName, admin, t);
124       // Turn off balancer so it doesn't cut in and mess up our placements.
125       this.admin.setBalancerRunning(false, false);
126       // Turn off the meta scanner so it don't remove parent on us.
127       cluster.getMaster().setCatalogJanitorEnabled(false);
128 
129       new Thread() {
130         public void run() {
131           SplitTransaction st = null;
132           st = new MockedSplitTransaction(regions.get(0), Bytes.toBytes("row2"));
133           try {
134             st.prepare();
135             st.execute(regionServer, regionServer);
136           } catch (IOException e) {
137 
138           }
139         }
140       }.start();
141       for (int i = 0; !callRollBack && i < 100; i++) {
142         Thread.sleep(100);
143       }
144       assertTrue("Waited too long for rollback", callRollBack);
145       SplitTransaction st = null;
146       st = new MockedSplitTransaction(regions.get(0), Bytes.toBytes("row2"));
147       try {
148         secondSplit = true;
149         st.prepare();
150         st.execute(regionServer, regionServer);
151       } catch (IOException e) {
152         LOG.debug("Rollback started :"+ e.getMessage());
153         st.rollback(regionServer, regionServer);
154       }
155       for (int i=0; !firstSplitCompleted && i<100; i++) {
156         Thread.sleep(100);
157       }
158       assertTrue("fist split did not complete", firstSplitCompleted);
159       NavigableMap<String, RegionState> rit = cluster.getMaster().getAssignmentManager()
160           .getRegionsInTransition();
161       for (int i=0; rit.containsKey(hri.getTableNameAsString()) && i<100; i++) {
162         Thread.sleep(100);
163       }
164       assertFalse("region still in transition", rit.containsKey(hri.getTableNameAsString()));
165       List<HRegion> onlineRegions = regionServer.getOnlineRegions(tableName);
166       // Region server side split is successful.
167       assertEquals("The parent region should be splitted", 2, onlineRegions.size());
168       //Should be present in RIT
169       List<HRegionInfo> regionsOfTable = cluster.getMaster().getAssignmentManager().getRegionsOfTable(tableName);
170       // Master side should also reflect the same
171       assertEquals("No of regions in master", 2, regionsOfTable.size());
172     } finally {
173       admin.setBalancerRunning(true, false);
174       secondSplit = false;
175       firstSplitCompleted = false;
176       callRollBack = false;
177       cluster.getMaster().setCatalogJanitorEnabled(true);
178     }
179     if (admin.isTableAvailable(tableName) && admin.isTableEnabled(tableName)) {
180       admin.disableTable(tableName);
181       admin.deleteTable(tableName);
182       admin.close();
183     }
184   }
185 
186   /**
187    * A test that intentionally has master fail the processing of the split message.
188    * Tests that the regionserver split ephemeral node gets cleaned up if it
189    * crashes and that after we process server shutdown, the daughters are up on
190    * line.
191    * @throws IOException
192    * @throws InterruptedException
193    * @throws NodeExistsException
194    * @throws KeeperException
195    */
196   @Test (timeout = 300000) public void testRSSplitEphemeralsDisappearButDaughtersAreOnlinedAfterShutdownHandling()
197   throws IOException, InterruptedException, NodeExistsException, KeeperException {
198     final byte [] tableName =
199       Bytes.toBytes("ephemeral");
200 
201     // Create table then get the single region for our new table.
202     HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
203     List<HRegion> regions = cluster.getRegions(tableName);
204     HRegionInfo hri = getAndCheckSingleTableRegion(regions);
205 
206     int tableRegionIndex = ensureTableRegionNotOnSameServerAsMeta(admin, hri);
207 
208     // Turn off balancer so it doesn't cut in and mess up our placements.
209     this.admin.setBalancerRunning(false, true);
210     // Turn off the meta scanner so it don't remove parent on us.
211     cluster.getMaster().setCatalogJanitorEnabled(false);
212     try {
213       // Add a bit of load up into the table so splittable.
214       TESTING_UTIL.loadTable(t, HConstants.CATALOG_FAMILY);
215       // Get region pre-split.
216       HRegionServer server = cluster.getRegionServer(tableRegionIndex);
217       printOutRegions(server, "Initial regions: ");
218       int regionCount = server.getOnlineRegions().size();
219       // Now, before we split, set special flag in master, a flag that has
220       // it FAIL the processing of split.
221       SplitRegionHandler.TEST_SKIP = true;
222       // Now try splitting and it should work.
223       split(hri, server, regionCount);
224       // Get daughters
225       List<HRegion> daughters = checkAndGetDaughters(tableName);
226       // Assert the ephemeral node is up in zk.
227       String path = ZKAssign.getNodeName(t.getConnection().getZooKeeperWatcher(),
228         hri.getEncodedName());
229       Stat stats =
230         t.getConnection().getZooKeeperWatcher().getRecoverableZooKeeper().exists(path, false);
231       LOG.info("EPHEMERAL NODE BEFORE SERVER ABORT, path=" + path + ", stats=" + stats);
232       RegionTransitionData rtd =
233         ZKAssign.getData(t.getConnection().getZooKeeperWatcher(),
234           hri.getEncodedName());
235       // State could be SPLIT or SPLITTING.
236       assertTrue(rtd.getEventType().equals(EventType.RS_ZK_REGION_SPLIT) ||
237         rtd.getEventType().equals(EventType.RS_ZK_REGION_SPLITTING));
238       // Now crash the server
239       cluster.abortRegionServer(tableRegionIndex);
240       waitUntilRegionServerDead();
241       awaitDaughters(tableName, daughters.size());
242 
243       // Assert daughters are online.
244       regions = cluster.getRegions(tableName);
245       for (HRegion r: regions) {
246         assertTrue(daughters.contains(r));
247       }
248       // Finally assert that the ephemeral SPLIT znode was cleaned up.
249       for (int i=0; i<100; i++) {
250         // wait a bit (10s max) for the node to disappear
251         stats = t.getConnection().getZooKeeperWatcher().getRecoverableZooKeeper().exists(path, false);
252         if (stats == null) break;
253         Thread.sleep(100);
254       }
255       LOG.info("EPHEMERAL NODE AFTER SERVER ABORT, path=" + path + ", stats=" + stats);
256       assertTrue(stats == null);
257     } finally {
258       // Set this flag back.
259       SplitRegionHandler.TEST_SKIP = false;
260       admin.setBalancerRunning(true, false);
261       cluster.getMaster().setCatalogJanitorEnabled(true);
262       t.close();
263     }
264   }
265 
266   @Test (timeout = 300000) public void testExistingZnodeBlocksSplitAndWeRollback()
267   throws IOException, InterruptedException, NodeExistsException, KeeperException {
268     final byte [] tableName =
269       Bytes.toBytes("testExistingZnodeBlocksSplitAndWeRollback");
270 
271     // Create table then get the single region for our new table.
272     HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
273     List<HRegion> regions = cluster.getRegions(tableName);
274     HRegionInfo hri = getAndCheckSingleTableRegion(regions);
275 
276     int tableRegionIndex = ensureTableRegionNotOnSameServerAsMeta(admin, hri);
277 
278     // Turn off balancer so it doesn't cut in and mess up our placements.
279     this.admin.setBalancerRunning(false, true);
280     // Turn off the meta scanner so it don't remove parent on us.
281     cluster.getMaster().setCatalogJanitorEnabled(false);
282     try {
283       // Add a bit of load up into the table so splittable.
284       TESTING_UTIL.loadTable(t, HConstants.CATALOG_FAMILY);
285       // Get region pre-split.
286       HRegionServer server = cluster.getRegionServer(tableRegionIndex);
287       printOutRegions(server, "Initial regions: ");
288       int regionCount = server.getOnlineRegions().size();
289       // Insert into zk a blocking znode, a znode of same name as region
290       // so it gets in way of our splitting.
291       ZKAssign.createNodeClosing(t.getConnection().getZooKeeperWatcher(),
292         hri, new ServerName("any.old.server", 1234, -1));
293       // Now try splitting.... should fail.  And each should successfully
294       // rollback.
295       this.admin.split(hri.getRegionNameAsString());
296       this.admin.split(hri.getRegionNameAsString());
297       this.admin.split(hri.getRegionNameAsString());
298       // Wait around a while and assert count of regions remains constant.
299       for (int i = 0; i < 10; i++) {
300         Thread.sleep(100);
301         assertEquals(regionCount, server.getOnlineRegions().size());
302       }
303       // Now clear the zknode
304       ZKAssign.deleteClosingNode(t.getConnection().getZooKeeperWatcher(), hri);
305       // Now try splitting and it should work.
306       split(hri, server, regionCount);
307       // Get daughters
308       checkAndGetDaughters(tableName);
309       // OK, so split happened after we cleared the blocking node.
310     } finally {
311       admin.setBalancerRunning(true, false);
312       cluster.getMaster().setCatalogJanitorEnabled(true);
313       t.close();
314     }
315   }
316 
317   /**
318    * Messy test that simulates case where SplitTransactions fails to add one
319    * of the daughters up into the .META. table before crash.  We're testing
320    * fact that the shutdown handler will fixup the missing daughter region
321    * adding it back into .META.
322    * @throws IOException
323    * @throws InterruptedException
324    */
325   @Test (timeout = 300000) public void testShutdownSimpleFixup()
326   throws IOException, InterruptedException {
327     final byte [] tableName = Bytes.toBytes("testShutdownSimpleFixup");
328 
329     // Create table then get the single region for our new table.
330     HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
331     List<HRegion> regions = cluster.getRegions(tableName);
332     HRegionInfo hri = getAndCheckSingleTableRegion(regions);
333 
334     int tableRegionIndex = ensureTableRegionNotOnSameServerAsMeta(admin, hri);
335 
336     // Turn off balancer so it doesn't cut in and mess up our placements.
337     this.admin.setBalancerRunning(false, true);
338     // Turn off the meta scanner so it don't remove parent on us.
339     cluster.getMaster().setCatalogJanitorEnabled(false);
340     try {
341       // Add a bit of load up into the table so splittable.
342       TESTING_UTIL.loadTable(t, HConstants.CATALOG_FAMILY);
343       // Get region pre-split.
344       HRegionServer server = cluster.getRegionServer(tableRegionIndex);
345       printOutRegions(server, "Initial regions: ");
346       int regionCount = server.getOnlineRegions().size();
347       // Now split.
348       split(hri, server, regionCount);
349       // Get daughters
350       List<HRegion> daughters = checkAndGetDaughters(tableName);
351       // Remove one of the daughters from .META. to simulate failed insert of
352       // daughter region up into .META.
353       removeDaughterFromMeta(daughters.get(0).getRegionName());
354       // Now crash the server
355       cluster.abortRegionServer(tableRegionIndex);
356       waitUntilRegionServerDead();
357       awaitDaughters(tableName, daughters.size());
358       // Assert daughters are online.
359       regions = cluster.getRegions(tableName);
360       for (HRegion r: regions) {
361         assertTrue(daughters.contains(r));
362       }
363     } finally {
364       admin.setBalancerRunning(true, false);
365       cluster.getMaster().setCatalogJanitorEnabled(true);
366       t.close();
367     }
368   }
369 
370   /**
371    * Test that if daughter split on us, we won't do the shutdown handler fixup
372    * just because we can't find the immediate daughter of an offlined parent.
373    * @throws IOException
374    * @throws InterruptedException
375    */
376   @Test (timeout=300000) public void testShutdownFixupWhenDaughterHasSplit()
377   throws IOException, InterruptedException {
378     final byte [] tableName =
379       Bytes.toBytes("testShutdownFixupWhenDaughterHasSplit");
380 
381     // Create table then get the single region for our new table.
382     HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
383     List<HRegion> regions = cluster.getRegions(tableName);
384     HRegionInfo hri = getAndCheckSingleTableRegion(regions);
385 
386     int tableRegionIndex = ensureTableRegionNotOnSameServerAsMeta(admin, hri);
387 
388     // Turn off balancer so it doesn't cut in and mess up our placements.
389     this.admin.setBalancerRunning(false, true);
390     // Turn off the meta scanner so it don't remove parent on us.
391     cluster.getMaster().setCatalogJanitorEnabled(false);
392     try {
393       // Add a bit of load up into the table so splittable.
394       TESTING_UTIL.loadTable(t, HConstants.CATALOG_FAMILY);
395       // Get region pre-split.
396       HRegionServer server = cluster.getRegionServer(tableRegionIndex);
397       printOutRegions(server, "Initial regions: ");
398       int regionCount = server.getOnlineRegions().size();
399       // Now split.
400       split(hri, server, regionCount);
401       // Get daughters
402       List<HRegion> daughters = checkAndGetDaughters(tableName);
403       // Now split one of the daughters.
404       regionCount = server.getOnlineRegions().size();
405       HRegionInfo daughter = daughters.get(0).getRegionInfo();
406       // Compact first to ensure we have cleaned up references -- else the split
407       // will fail.
408       this.admin.compact(daughter.getRegionName());
409       daughters = cluster.getRegions(tableName);
410       HRegion daughterRegion = null;
411       for (HRegion r: daughters) {
412         if (r.getRegionInfo().equals(daughter)) daughterRegion = r;
413       }
414       assertTrue(daughterRegion != null);
415       for (int i=0; i<100; i++) {
416         if (!daughterRegion.hasReferences()) break;
417         Threads.sleep(100);
418       }
419       assertFalse("Waiting for refereces to be compacted", daughterRegion.hasReferences());
420       split(daughter, server, regionCount);
421       // Get list of daughters
422       daughters = cluster.getRegions(tableName);
423       // Now crash the server
424       cluster.abortRegionServer(tableRegionIndex);
425       waitUntilRegionServerDead();
426       awaitDaughters(tableName, daughters.size());
427       // Assert daughters are online and ONLY the original daughters -- that
428       // fixup didn't insert one during server shutdown recover.
429       regions = cluster.getRegions(tableName);
430       assertEquals(daughters.size(), regions.size());
431       for (HRegion r: regions) {
432         assertTrue(daughters.contains(r));
433       }
434     } finally {
435       admin.setBalancerRunning(true, false);
436       cluster.getMaster().setCatalogJanitorEnabled(true);
437       t.close();
438     }
439   }
440   
441   /**
442    * Verifies HBASE-5806.  When splitting is partially done and the master goes down
443    * when the SPLIT node is in either SPLIT or SPLITTING state.
444    * 
445    * @throws IOException
446    * @throws InterruptedException
447    * @throws NodeExistsException
448    * @throws KeeperException
449    */
450   @Test(timeout = 300000)
451   public void testMasterRestartWhenSplittingIsPartial()
452       throws IOException, InterruptedException, NodeExistsException,
453       KeeperException {
454     final byte[] tableName = Bytes.toBytes("testMasterRestartWhenSplittingIsPartial");
455 
456     // Create table then get the single region for our new table.
457     HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
458     List<HRegion> regions = cluster.getRegions(tableName);
459     HRegionInfo hri = getAndCheckSingleTableRegion(regions);
460 
461     int tableRegionIndex = ensureTableRegionNotOnSameServerAsMeta(admin, hri);
462 
463     // Turn off the meta scanner so it don't remove parent on us.
464     cluster.getMaster().setCatalogJanitorEnabled(false);
465     // Turn off balancer so it doesn't cut in and mess up our placements.
466     this.admin.setBalancerRunning(false, true);
467     
468     try {
469       // Add a bit of load up into the table so splittable.
470       TESTING_UTIL.loadTable(t, HConstants.CATALOG_FAMILY);
471       // Get region pre-split.
472       HRegionServer server = cluster.getRegionServer(tableRegionIndex);
473       printOutRegions(server, "Initial regions: ");
474       int regionCount = server.getOnlineRegions().size();
475       // Now, before we split, set special flag in master, a flag that has
476       // it FAIL the processing of split.
477       SplitRegionHandler.TEST_SKIP = true;
478       // Now try splitting and it should work.
479       split(hri, server, regionCount);
480       // Get daughters
481       checkAndGetDaughters(tableName);
482       // Assert the ephemeral node is up in zk.
483       String path = ZKAssign.getNodeName(t.getConnection()
484           .getZooKeeperWatcher(), hri.getEncodedName());
485       Stat stats = t.getConnection().getZooKeeperWatcher()
486           .getRecoverableZooKeeper().exists(path, false);
487       LOG.info("EPHEMERAL NODE BEFORE SERVER ABORT, path=" + path + ", stats="
488           + stats);
489       RegionTransitionData rtd = ZKAssign.getData(t.getConnection()
490           .getZooKeeperWatcher(), hri.getEncodedName());
491       // State could be SPLIT or SPLITTING.
492       assertTrue(rtd.getEventType().equals(EventType.RS_ZK_REGION_SPLIT)
493           || rtd.getEventType().equals(EventType.RS_ZK_REGION_SPLITTING));
494 
495 
496       // abort and wait for new master.
497       MockMasterWithoutCatalogJanitor master = abortAndWaitForMaster();
498 
499       this.admin = new HBaseAdmin(TESTING_UTIL.getConfiguration());
500 
501       // update the hri to be offlined and splitted. 
502       hri.setOffline(true);
503       hri.setSplit(true);
504       ServerName regionServerOfRegion = master.getAssignmentManager()
505           .getRegionServerOfRegion(hri);
506       assertTrue(regionServerOfRegion != null);
507 
508     } finally {
509       // Set this flag back.
510       SplitRegionHandler.TEST_SKIP = false;
511       admin.setBalancerRunning(true, false);
512       cluster.getMaster().setCatalogJanitorEnabled(true);
513       t.close();
514     }
515   }
516 
517 
518   /**
519    * Verifies HBASE-5806.  Here the case is that splitting is completed but before the
520    * CJ could remove the parent region the master is killed and restarted.
521    * @throws IOException
522    * @throws InterruptedException
523    * @throws NodeExistsException
524    * @throws KeeperException
525    */
526   @Test (timeout = 300000)
527   public void testMasterRestartAtRegionSplitPendingCatalogJanitor()
528       throws IOException, InterruptedException, NodeExistsException,
529       KeeperException {
530     final byte[] tableName = Bytes.toBytes("testMasterRestartAtRegionSplitPendingCatalogJanitor");
531 
532     // Create table then get the single region for our new table.
533     this.admin = new HBaseAdmin(TESTING_UTIL.getConfiguration());
534     HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
535     List<HRegion> regions = cluster.getRegions(tableName);
536     HRegionInfo hri = getAndCheckSingleTableRegion(regions);
537 
538     int tableRegionIndex = ensureTableRegionNotOnSameServerAsMeta(admin, hri);
539 
540     // Turn off balancer so it doesn't cut in and mess up our placements.
541     this.admin.setBalancerRunning(false, true);
542     // Turn off the meta scanner so it don't remove parent on us.
543     cluster.getMaster().setCatalogJanitorEnabled(false);
544     try {
545       // Add a bit of load up into the table so splittable.
546       TESTING_UTIL.loadTable(t, HConstants.CATALOG_FAMILY);
547       // Get region pre-split.
548       HRegionServer server = cluster.getRegionServer(tableRegionIndex);
549       printOutRegions(server, "Initial regions: ");
550       int regionCount = server.getOnlineRegions().size();
551       
552       split(hri, server, regionCount);
553       // Get daughters
554       checkAndGetDaughters(tableName);
555       // Assert the ephemeral node is up in zk.
556       String path = ZKAssign.getNodeName(t.getConnection()
557           .getZooKeeperWatcher(), hri.getEncodedName());
558       Stat stats = t.getConnection().getZooKeeperWatcher()
559           .getRecoverableZooKeeper().exists(path, false);
560       LOG.info("EPHEMERAL NODE BEFORE SERVER ABORT, path=" + path + ", stats="
561           + stats);
562       String node = ZKAssign.getNodeName(t.getConnection()
563           .getZooKeeperWatcher(), hri.getEncodedName());
564       Stat stat = new Stat();
565       byte[] data = ZKUtil.getDataNoWatch(t.getConnection()
566           .getZooKeeperWatcher(), node, stat);
567       // ZKUtil.create
568       for (int i=0; data != null && i<60; i++) {
569         Thread.sleep(1000);
570         data = ZKUtil.getDataNoWatch(t.getConnection().getZooKeeperWatcher(),
571             node, stat);
572 
573       }
574       assertNull("Waited too long for ZK node to be removed: "+node, data);
575       
576       MockMasterWithoutCatalogJanitor master = abortAndWaitForMaster();
577 
578       this.admin = new HBaseAdmin(TESTING_UTIL.getConfiguration());
579 
580       hri.setOffline(true);
581       hri.setSplit(true);
582       ServerName regionServerOfRegion = master.getAssignmentManager()
583           .getRegionServerOfRegion(hri);
584       assertTrue(regionServerOfRegion == null);
585     } finally {
586       // Set this flag back.
587       SplitRegionHandler.TEST_SKIP = false;
588       this.admin.setBalancerRunning(true, false);
589       cluster.getMaster().setCatalogJanitorEnabled(true);
590       t.close();
591     }
592   }
593 
594   /**
595    * While transitioning node from RS_ZK_REGION_SPLITTING to
596    * RS_ZK_REGION_SPLITTING during region split,if zookeper went down split always
597    * fails for the region. HBASE-6088 fixes this scenario.
598    * This test case is to test the znode is deleted(if created) or not in roll back.
599    * 
600    * @throws IOException
601    * @throws InterruptedException
602    * @throws KeeperException
603    */
604   @Test
605   public void testSplitBeforeSettingSplittingInZK() throws Exception,
606       InterruptedException, KeeperException {
607     testSplitBeforeSettingSplittingInZK(true);
608     testSplitBeforeSettingSplittingInZK(false);
609   }
610 
611   private void testSplitBeforeSettingSplittingInZK(boolean nodeCreated) throws Exception {
612     final byte[] tableName = Bytes.toBytes("testSplitBeforeSettingSplittingInZK");
613     
614     HBaseAdmin admin = new HBaseAdmin(TESTING_UTIL.getConfiguration());
615     // Create table then get the single region for our new table.
616     HTableDescriptor htd = new HTableDescriptor(tableName);
617     htd.addFamily(new HColumnDescriptor("cf"));
618     admin.createTable(htd);
619 
620     List<HRegion> regions = null;
621     for (int i=0; i<100; i++) {
622       regions = cluster.getRegions(tableName);
623       if (regions.size() > 0) break;
624       Thread.sleep(100);
625     }
626     int regionServerIndex = cluster.getServerWith(regions.get(0).getRegionName());
627     HRegionServer regionServer = cluster.getRegionServer(regionServerIndex);
628     SplitTransaction st = null;
629     if (nodeCreated) {
630       st = new MockedSplitTransaction(regions.get(0), null) {
631         @Override
632         int transitionNodeSplitting(ZooKeeperWatcher zkw, HRegionInfo parent,
633             ServerName serverName, int version) throws KeeperException, IOException {
634           throw new TransitionToSplittingFailedException();
635         }
636       };
637     } else {
638       st = new MockedSplitTransaction(regions.get(0), null) {
639         @Override
640         void createNodeSplitting(ZooKeeperWatcher zkw, HRegionInfo region, ServerName serverName)
641             throws KeeperException, IOException {
642           throw new SplittingNodeCreationFailedException ();
643         }
644       };
645     }
646     String node = ZKAssign.getNodeName(regionServer.getZooKeeper(), regions.get(0)
647         .getRegionInfo().getEncodedName());
648     // make sure the client is uptodate
649     regionServer.getZooKeeper().sync(node);
650     for (int i = 0; i < 100; i++) {
651       // We expect the znode to be deleted by this time. Here the znode could be in OPENED state and the 
652       // master has not yet deleted the znode.
653       if (ZKUtil.checkExists(regionServer.getZooKeeper(), node) != -1) {
654         Thread.sleep(100);
655       }
656     }
657     
658     try {
659       st.execute(regionServer, regionServer);
660     } catch (IOException e) {
661       // check for the specific instance in case the Split failed due to the existence of the znode in OPENED state.
662       // This will at least make the test to fail;
663       if (nodeCreated) {
664         assertTrue("Should be instance of TransitionToSplittingFailedException",
665             e instanceof TransitionToSplittingFailedException);
666       } else {
667         assertTrue("Should be instance of CreateSplittingNodeFailedException",
668             e instanceof SplittingNodeCreationFailedException );
669       }
670       node = ZKAssign.getNodeName(regionServer.getZooKeeper(), regions.get(0)
671           .getRegionInfo().getEncodedName());
672       // make sure the client is uptodate
673       regionServer.getZooKeeper().sync(node);
674       if (nodeCreated) {
675         assertFalse(ZKUtil.checkExists(regionServer.getZooKeeper(), node) == -1);
676       } else {
677         assertTrue(ZKUtil.checkExists(regionServer.getZooKeeper(), node) == -1);
678       }
679       assertTrue(st.rollback(regionServer, regionServer));
680       assertTrue(ZKUtil.checkExists(regionServer.getZooKeeper(), node) == -1);
681     }
682     if (admin.isTableAvailable(tableName) && admin.isTableEnabled(tableName)) {
683       admin.disableTable(tableName);
684       admin.deleteTable(tableName);
685     }
686   }
687   
688   @Test
689   public void testShouldClearRITWhenNodeFoundInSplittingState() throws Exception {
690     final byte[] tableName = Bytes.toBytes("testShouldClearRITWhenNodeFoundInSplittingState");
691     HBaseAdmin admin = new HBaseAdmin(TESTING_UTIL.getConfiguration());
692     // Create table then get the single region for our new table.
693     HTableDescriptor htd = new HTableDescriptor(tableName);
694     htd.addFamily(new HColumnDescriptor("cf"));
695     admin.createTable(htd);
696     for (int i = 0; cluster.getRegions(tableName).size() == 0 && i < 100; i++) {
697       Thread.sleep(100);
698     }
699     assertTrue("Table not online", cluster.getRegions(tableName).size() != 0);
700 
701     HRegion region = cluster.getRegions(tableName).get(0);
702     int regionServerIndex = cluster.getServerWith(region.getRegionName());
703     HRegionServer regionServer = cluster.getRegionServer(regionServerIndex);
704     SplitTransaction st = null;
705 
706     st = new MockedSplitTransaction(region, null) {
707       @Override
708       void createSplitDir(FileSystem fs, Path splitdir) throws IOException {
709         throw new IOException("");
710       }
711     };
712 
713     try {
714       st.execute(regionServer, regionServer);
715     } catch (IOException e) {
716       String node = ZKAssign.getNodeName(regionServer.getZooKeeper(), region
717           .getRegionInfo().getEncodedName());
718 
719       assertFalse(ZKUtil.checkExists(regionServer.getZooKeeper(), node) == -1);
720       AssignmentManager am = cluster.getMaster().getAssignmentManager();
721       for (int i = 0; !am.getRegionsInTransition().containsKey(
722           region.getRegionInfo().getEncodedName())
723           && i < 100; i++) {
724         Thread.sleep(200);
725       }
726       assertTrue("region is not in transition "+region,
727           am.getRegionsInTransition().containsKey(region.getRegionInfo().getEncodedName()));
728       RegionState regionState = am.getRegionsInTransition().get(region.getRegionInfo()
729           .getEncodedName());
730       assertTrue(regionState.getState() == RegionState.State.SPLITTING);
731       assertTrue(st.rollback(regionServer, regionServer));
732       assertTrue(ZKUtil.checkExists(regionServer.getZooKeeper(), node) == -1);
733       for (int i=0; am.getRegionsInTransition().containsKey(region.getRegionInfo().getEncodedName()) && i<100; i++) {
734         // Just in case the nodeDeleted event did not get executed.
735         Thread.sleep(200);
736       }
737       assertFalse("region is still in transition",
738           am.getRegionsInTransition().containsKey(region.getRegionInfo().getEncodedName()));
739     }
740     if (admin.isTableAvailable(tableName) && admin.isTableEnabled(tableName)) {
741       admin.disableTable(tableName);
742       admin.deleteTable(tableName);
743       admin.close();
744     }
745   }
746   
747   @Test(timeout = 60000)
748   public void testTableExistsIfTheSpecifiedTableRegionIsSplitParent() throws Exception {
749     final byte[] tableName = 
750         Bytes.toBytes("testTableExistsIfTheSpecifiedTableRegionIsSplitParent");
751     HRegionServer regionServer = null;
752     List<HRegion> regions = null;
753     HBaseAdmin admin = new HBaseAdmin(TESTING_UTIL.getConfiguration());
754     try {
755       // Create table then get the single region for our new table.
756       HTable t = createTableAndWait(tableName, Bytes.toBytes("cf"));
757       regions = cluster.getRegions(tableName);
758       int regionServerIndex = cluster.getServerWith(regions.get(0).getRegionName());
759       regionServer = cluster.getRegionServer(regionServerIndex);
760       insertData(tableName, admin, t);
761       // Turn off balancer so it doesn't cut in and mess up our placements.
762       cluster.getMaster().setCatalogJanitorEnabled(false);
763       boolean tableExists = MetaReader.tableExists(regionServer.getCatalogTracker(),
764           Bytes.toString(tableName));
765       assertEquals("The specified table should present.", true, tableExists);
766       SplitTransaction st = new SplitTransaction(regions.get(0), Bytes.toBytes("row2"));
767       try {
768         st.prepare();
769         st.createDaughters(regionServer, regionServer);
770       } catch (IOException e) {
771 
772       }
773       tableExists = MetaReader.tableExists(regionServer.getCatalogTracker(),
774           Bytes.toString(tableName));
775       assertEquals("The specified table should present.", true, tableExists);
776     } finally {
777       cluster.getMaster().setCatalogJanitorEnabled(true);
778       admin.close();
779     }
780   }
781   
782   private void insertData(final byte[] tableName, HBaseAdmin admin, HTable t) throws IOException,
783       InterruptedException {
784     Put p = new Put(Bytes.toBytes("row1"));
785     p.add(Bytes.toBytes("cf"), Bytes.toBytes("q1"), Bytes.toBytes("1"));
786     t.put(p);
787     p = new Put(Bytes.toBytes("row2"));
788     p.add(Bytes.toBytes("cf"), Bytes.toBytes("q1"), Bytes.toBytes("2"));
789     t.put(p);
790     p = new Put(Bytes.toBytes("row3"));
791     p.add(Bytes.toBytes("cf"), Bytes.toBytes("q1"), Bytes.toBytes("3"));
792     t.put(p);
793     p = new Put(Bytes.toBytes("row4"));
794     p.add(Bytes.toBytes("cf"), Bytes.toBytes("q1"), Bytes.toBytes("4"));
795     t.put(p);
796     admin.flush(tableName);
797   }
798 
799   public static class MockedSplitTransaction extends SplitTransaction {
800 
801     private HRegion currentRegion;
802     public MockedSplitTransaction(HRegion r, byte[] splitrow) {
803       super(r, splitrow);
804       this.currentRegion = r;
805     }
806     
807     @Override
808     void transitionZKNode(Server server, RegionServerServices services, HRegion a, HRegion b)
809         throws IOException {
810       if (this.currentRegion.getRegionInfo().getTableNameAsString()
811           .equals("testShouldFailSplitIfZNodeDoesNotExistDueToPrevRollBack")) {
812         try {
813           if (!secondSplit){
814             callRollBack = true;
815             latch.await();
816           }
817         } catch (InterruptedException e) {
818         }
819        
820       }
821       super.transitionZKNode(server, services, a, b);
822       if (this.currentRegion.getRegionInfo().getTableNameAsString()
823           .equals("testShouldFailSplitIfZNodeDoesNotExistDueToPrevRollBack")) {
824         firstSplitCompleted = true;
825       }
826     }
827     @Override
828     public boolean rollback(Server server, RegionServerServices services) throws IOException {
829       if (this.currentRegion.getRegionInfo().getTableNameAsString()
830           .equals("testShouldFailSplitIfZNodeDoesNotExistDueToPrevRollBack")) {
831         if(secondSplit){
832           super.rollback(server, services);
833           latch.countDown();
834           return true;
835         }
836       }
837       return super.rollback(server, services);
838     }
839 
840   }
841 
842   private List<HRegion> checkAndGetDaughters(byte[] tableName)
843       throws InterruptedException {    
844     List<HRegion> daughters = null;
845     // try up to 10s
846     for (int i=0; i<100; i++) {
847       daughters = cluster.getRegions(tableName);
848       if (daughters.size() >= 2) break;
849       Thread.sleep(100);
850     }
851     assertTrue(daughters.size() >= 2);
852     return daughters;
853   }
854 
855   private MockMasterWithoutCatalogJanitor abortAndWaitForMaster() 
856   throws IOException, InterruptedException {
857     cluster.abortMaster(0);
858     cluster.waitOnMaster(0);
859     cluster.getConfiguration().setClass(HConstants.MASTER_IMPL, 
860         MockMasterWithoutCatalogJanitor.class, HMaster.class);
861     MockMasterWithoutCatalogJanitor master = null;
862     master = (MockMasterWithoutCatalogJanitor) cluster.startMaster().getMaster();
863     cluster.waitForActiveAndReadyMaster();
864     return master;
865   }
866 
867   private void split(final HRegionInfo hri, final HRegionServer server,
868       final int regionCount)
869   throws IOException, InterruptedException {
870     this.admin.split(hri.getRegionNameAsString());
871     for (int i=0; server.getOnlineRegions().size() <= regionCount && i<100; i++) {
872       LOG.debug("Waiting on region to split");
873       Thread.sleep(100);
874     }
875     assertFalse("Waited too long for split", server.getOnlineRegions().size() <= regionCount);
876   }
877 
878   private void removeDaughterFromMeta(final byte [] regionName) throws IOException {
879     HTable metaTable =
880       new HTable(TESTING_UTIL.getConfiguration(), HConstants.META_TABLE_NAME);
881     Delete d = new Delete(regionName);
882     LOG.info("Deleted " + Bytes.toString(regionName));
883     metaTable.delete(d);
884   }
885 
886   /**
887    * Ensure single table region is not on same server as the single .META. table
888    * region.
889    * @param admin
890    * @param hri
891    * @return Index of the server hosting the single table region
892    * @throws UnknownRegionException
893    * @throws MasterNotRunningException
894    * @throws ZooKeeperConnectionException
895    * @throws InterruptedException
896    */
897   private int ensureTableRegionNotOnSameServerAsMeta(final HBaseAdmin admin,
898       final HRegionInfo hri)
899   throws UnknownRegionException, MasterNotRunningException,
900   ZooKeeperConnectionException, InterruptedException {
901     MiniHBaseCluster cluster = TESTING_UTIL.getMiniHBaseCluster();
902     // Now make sure that the table region is not on same server as that hosting
903     // .META.  We don't want .META. replay polluting our test when we later crash
904     // the table region serving server.
905     int metaServerIndex = cluster.getServerWithMeta();
906     assertTrue(metaServerIndex != -1);
907     HRegionServer metaRegionServer = cluster.getRegionServer(metaServerIndex);
908     int tableRegionIndex = cluster.getServerWith(hri.getRegionName());
909     assertTrue(tableRegionIndex != -1);
910     HRegionServer tableRegionServer = cluster.getRegionServer(tableRegionIndex);
911     if (metaRegionServer.getServerName().equals(tableRegionServer.getServerName())) {
912       HRegionServer hrs = getOtherRegionServer(cluster, metaRegionServer);
913       assertNotNull(hrs);
914       assertNotNull(hri);
915       LOG.
916         info("Moving " + hri.getRegionNameAsString() + " to " +
917         hrs.getServerName() + "; metaServerIndex=" + metaServerIndex);
918       for (int i = 0; cluster.getMaster().getAssignmentManager()
919           .getRegionServerOfRegion(hri) == null
920           && i < 100; i++) {
921         Thread.sleep(10);
922       }
923       admin.move(hri.getEncodedNameAsBytes(),
924         Bytes.toBytes(hrs.getServerName().toString()));
925     }
926     // Wait till table region is up on the server that is NOT carrying .META..
927     for (int i=0; i<100; i++) {
928       tableRegionIndex = cluster.getServerWith(hri.getRegionName());
929       if (tableRegionIndex != -1 && tableRegionIndex != metaServerIndex) break;
930       LOG.debug("Waiting on region move off the .META. server; current index " +
931         tableRegionIndex + " and metaServerIndex=" + metaServerIndex);
932       Thread.sleep(100);
933     }
934     assertTrue("Region not moved off .META. server", tableRegionIndex != -1
935         && tableRegionIndex != metaServerIndex);
936     // Verify for sure table region is not on same server as .META.
937     tableRegionIndex = cluster.getServerWith(hri.getRegionName());
938     assertTrue(tableRegionIndex != -1);
939     assertNotSame(metaServerIndex, tableRegionIndex);
940     return tableRegionIndex;
941   }
942 
943   /**
944    * Find regionserver other than the one passed.
945    * Can't rely on indexes into list of regionservers since crashed servers
946    * occupy an index.
947    * @param cluster
948    * @param notThisOne
949    * @return A regionserver that is not <code>notThisOne</code> or null if none
950    * found
951    */
952   private HRegionServer getOtherRegionServer(final MiniHBaseCluster cluster,
953       final HRegionServer notThisOne) {
954     for (RegionServerThread rst: cluster.getRegionServerThreads()) {
955       HRegionServer hrs = rst.getRegionServer();
956       if (hrs.getServerName().equals(notThisOne.getServerName())) continue;
957       if (hrs.isStopping() || hrs.isStopped()) continue;
958       return hrs;
959     }
960     return null;
961   }
962 
963   private void printOutRegions(final HRegionServer hrs, final String prefix)
964       throws IOException {
965     List<HRegionInfo> regions = hrs.getOnlineRegions();
966     for (HRegionInfo region: regions) {
967       LOG.info(prefix + region.getRegionNameAsString());
968     }
969   }
970 
971   private void waitUntilRegionServerDead() throws InterruptedException {
972     // Wait until the master processes the RS shutdown
973     for (int i=0; cluster.getMaster().getClusterStatus().
974         getServers().size() == NB_SERVERS && i<100; i++) {
975       LOG.info("Waiting on server to go down");
976       Thread.sleep(100);
977     }
978     assertFalse("Waited too long for RS to die", cluster.getMaster().getClusterStatus().
979     getServers().size() == NB_SERVERS);
980   }
981 
982   private void awaitDaughters(byte[] tableName, int numDaughters) throws InterruptedException {
983     // Wait till regions are back on line again.
984     for (int i=0; cluster.getRegions(tableName).size() < numDaughters && i<60; i++) {
985       LOG.info("Waiting for repair to happen");
986       Thread.sleep(1000);
987     }
988     if (cluster.getRegions(tableName).size() < numDaughters) {
989       fail("Waiting too long for daughter regions");
990     }
991   }
992 
993   private HTable createTableAndWait(byte[] tableName, byte[] cf) throws IOException,
994       InterruptedException {
995     HTable t = TESTING_UTIL.createTable(tableName, cf);
996     for (int i = 0; cluster.getRegions(tableName).size() == 0 && i < 100; i++) {
997       Thread.sleep(100);
998     }
999     assertTrue("Table not online: "+Bytes.toString(tableName), cluster.getRegions(tableName).size() != 0);
1000     return t;
1001   }
1002 
1003   @org.junit.Rule
1004   public org.apache.hadoop.hbase.ResourceCheckerJUnitRule cu =
1005     new org.apache.hadoop.hbase.ResourceCheckerJUnitRule();
1006   
1007   public static class MockMasterWithoutCatalogJanitor extends HMaster {
1008 
1009     public MockMasterWithoutCatalogJanitor(Configuration conf) throws IOException, KeeperException,
1010         InterruptedException {
1011       super(conf);
1012     }
1013 
1014     protected void startCatalogJanitorChore() {
1015       LOG.debug("Customised master executed.");
1016     }
1017   }
1018   
1019   private static class TransitionToSplittingFailedException extends IOException {
1020     public TransitionToSplittingFailedException() {
1021       super();
1022     }
1023   }
1024 
1025   private static class SplittingNodeCreationFailedException  extends IOException {
1026     public SplittingNodeCreationFailedException () {
1027       super();
1028     }
1029   }
1030 }
1031