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