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  @SuppressWarnings("deprecation")
77  public class TestSplitTransactionOnCluster {
78    private static final Log LOG =
79      LogFactory.getLog(TestSplitTransactionOnCluster.class);
80    private HBaseAdmin admin = null;
81    private MiniHBaseCluster cluster = null;
82    private static final int NB_SERVERS = 2;
83    private static CountDownLatch latch = new CountDownLatch(1);
84    private static volatile boolean secondSplit = false;
85    private static volatile boolean callRollBack = false;
86    private static volatile boolean firstSplitCompleted = false;
87    
88    private static final HBaseTestingUtility TESTING_UTIL =
89      new HBaseTestingUtility();
90  
91    @BeforeClass public static void before() throws Exception {
92      TESTING_UTIL.getConfiguration().setInt("hbase.balancer.period", 60000);
93      // Needed because some tests have splits happening on RS that are killed
94      // We don't want to wait 3min for the master to figure it out
95      TESTING_UTIL.getConfiguration().setInt(
96          "hbase.master.assignment.timeoutmonitor.timeout", 4000);
97      TESTING_UTIL.startMiniCluster(NB_SERVERS);
98    }
99  
100   @AfterClass public static void after() throws Exception {
101     TESTING_UTIL.shutdownMiniCluster();
102   }
103 
104   @Before public void setup() throws IOException {
105     TESTING_UTIL.ensureSomeNonStoppedRegionServersAvailable(NB_SERVERS);
106     this.admin = new HBaseAdmin(TESTING_UTIL.getConfiguration());
107     this.cluster = TESTING_UTIL.getMiniHBaseCluster();
108   }
109 
110   private HRegionInfo getAndCheckSingleTableRegion(final List<HRegion> regions) {
111     assertEquals(1, regions.size());
112     return regions.get(0).getRegionInfo();
113   }
114 
115   @Test(timeout = 2000000)
116   public void testShouldFailSplitIfZNodeDoesNotExistDueToPrevRollBack() throws Exception {
117     final byte[] tableName = Bytes
118         .toBytes("testShouldFailSplitIfZNodeDoesNotExistDueToPrevRollBack");
119     HBaseAdmin admin = new HBaseAdmin(TESTING_UTIL.getConfiguration());
120     try {
121       // Create table then get the single region for our new table.
122       HTable t = createTableAndWait(tableName, Bytes.toBytes("cf"));
123       final List<HRegion> regions = cluster.getRegions(tableName);
124       HRegionInfo hri = getAndCheckSingleTableRegion(regions);
125       int regionServerIndex = cluster.getServerWith(regions.get(0).getRegionName());
126       final HRegionServer regionServer = cluster.getRegionServer(regionServerIndex);
127       insertData(tableName, admin, t);
128       // Turn off balancer so it doesn't cut in and mess up our placements.
129       this.admin.setBalancerRunning(false, false);
130       // Turn off the meta scanner so it don't remove parent on us.
131       cluster.getMaster().setCatalogJanitorEnabled(false);
132 
133       new Thread() {
134         public void run() {
135           SplitTransaction st = null;
136           st = new MockedSplitTransaction(regions.get(0), Bytes.toBytes("row2"));
137           try {
138             st.prepare();
139             st.execute(regionServer, regionServer);
140           } catch (IOException e) {
141 
142           }
143         }
144       }.start();
145       for (int i = 0; !callRollBack && i < 100; i++) {
146         Thread.sleep(100);
147       }
148       assertTrue("Waited too long for rollback", callRollBack);
149       SplitTransaction st = null;
150       st = new MockedSplitTransaction(regions.get(0), Bytes.toBytes("row2"));
151       try {
152         secondSplit = true;
153         st.prepare();
154         st.execute(regionServer, regionServer);
155       } catch (IOException e) {
156         LOG.debug("Rollback started :"+ e.getMessage());
157         st.rollback(regionServer, regionServer);
158       }
159       for (int i=0; !firstSplitCompleted && i<100; i++) {
160         Thread.sleep(100);
161       }
162       assertTrue("fist split did not complete", firstSplitCompleted);
163       NavigableMap<String, RegionState> rit = cluster.getMaster().getAssignmentManager()
164           .getRegionsInTransition();
165       for (int i=0; rit.containsKey(hri.getTableNameAsString()) && i<100; i++) {
166         Thread.sleep(100);
167       }
168       assertFalse("region still in transition", rit.containsKey(hri.getTableNameAsString()));
169       List<HRegion> onlineRegions = regionServer.getOnlineRegions(tableName);
170       // Region server side split is successful.
171       assertEquals("The parent region should be splitted", 2, onlineRegions.size());
172       //Should be present in RIT
173       List<HRegionInfo> regionsOfTable = cluster.getMaster().getAssignmentManager().getRegionsOfTable(tableName);
174       // Master side should also reflect the same
175       assertEquals("No of regions in master", 2, regionsOfTable.size());
176     } finally {
177       admin.setBalancerRunning(true, false);
178       secondSplit = false;
179       firstSplitCompleted = false;
180       callRollBack = false;
181       cluster.getMaster().setCatalogJanitorEnabled(true);
182     }
183     if (admin.isTableAvailable(tableName) && admin.isTableEnabled(tableName)) {
184       admin.disableTable(tableName);
185       admin.deleteTable(tableName);
186       admin.close();
187     }
188   }
189 
190   /**
191    * A test that intentionally has master fail the processing of the split message.
192    * Tests that the regionserver split ephemeral node gets cleaned up if it
193    * crashes and that after we process server shutdown, the daughters are up on
194    * line.
195    * @throws IOException
196    * @throws InterruptedException
197    * @throws NodeExistsException
198    * @throws KeeperException
199    */
200   @Test (timeout = 300000) public void testRSSplitEphemeralsDisappearButDaughtersAreOnlinedAfterShutdownHandling()
201   throws IOException, InterruptedException, NodeExistsException, KeeperException {
202     final byte [] tableName =
203       Bytes.toBytes("ephemeral");
204 
205     // Create table then get the single region for our new table.
206     HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
207     List<HRegion> regions = cluster.getRegions(tableName);
208     HRegionInfo hri = getAndCheckSingleTableRegion(regions);
209 
210     int tableRegionIndex = ensureTableRegionNotOnSameServerAsMeta(admin, hri);
211 
212     // Turn off balancer so it doesn't cut in and mess up our placements.
213     this.admin.setBalancerRunning(false, true);
214     // Turn off the meta scanner so it don't remove parent on us.
215     cluster.getMaster().setCatalogJanitorEnabled(false);
216     try {
217       // Add a bit of load up into the table so splittable.
218       TESTING_UTIL.loadTable(t, HConstants.CATALOG_FAMILY);
219       // Get region pre-split.
220       HRegionServer server = cluster.getRegionServer(tableRegionIndex);
221       printOutRegions(server, "Initial regions: ");
222       int regionCount = server.getOnlineRegions().size();
223       // Now, before we split, set special flag in master, a flag that has
224       // it FAIL the processing of split.
225       SplitRegionHandler.TEST_SKIP = true;
226       // Now try splitting and it should work.
227       split(hri, server, regionCount);
228       // Get daughters
229       List<HRegion> daughters = checkAndGetDaughters(tableName);
230       // Assert the ephemeral node is up in zk.
231       String path = ZKAssign.getNodeName(t.getConnection().getZooKeeperWatcher(),
232         hri.getEncodedName());
233       Stat stats =
234         t.getConnection().getZooKeeperWatcher().getRecoverableZooKeeper().exists(path, false);
235       LOG.info("EPHEMERAL NODE BEFORE SERVER ABORT, path=" + path + ", stats=" + stats);
236       RegionTransitionData rtd =
237         ZKAssign.getData(t.getConnection().getZooKeeperWatcher(),
238           hri.getEncodedName());
239       // State could be SPLIT or SPLITTING.
240       assertTrue(rtd.getEventType().equals(EventType.RS_ZK_REGION_SPLIT) ||
241         rtd.getEventType().equals(EventType.RS_ZK_REGION_SPLITTING));
242       // Now crash the server
243       cluster.abortRegionServer(tableRegionIndex);
244       waitUntilRegionServerDead();
245       awaitDaughters(tableName, daughters.size());
246 
247       // Assert daughters are online.
248       regions = cluster.getRegions(tableName);
249       for (HRegion r: regions) {
250         assertTrue(daughters.contains(r));
251       }
252       // Finally assert that the ephemeral SPLIT znode was cleaned up.
253       for (int i=0; i<100; i++) {
254         // wait a bit (10s max) for the node to disappear
255         stats = t.getConnection().getZooKeeperWatcher().getRecoverableZooKeeper().exists(path, false);
256         if (stats == null) break;
257         Thread.sleep(100);
258       }
259       LOG.info("EPHEMERAL NODE AFTER SERVER ABORT, path=" + path + ", stats=" + stats);
260       assertTrue(stats == null);
261     } finally {
262       // Set this flag back.
263       SplitRegionHandler.TEST_SKIP = false;
264       admin.setBalancerRunning(true, false);
265       cluster.getMaster().setCatalogJanitorEnabled(true);
266       t.close();
267     }
268   }
269 
270   @Test (timeout = 300000) public void testExistingZnodeBlocksSplitAndWeRollback()
271   throws IOException, InterruptedException, NodeExistsException, KeeperException {
272     final byte [] tableName =
273       Bytes.toBytes("testExistingZnodeBlocksSplitAndWeRollback");
274 
275     // Create table then get the single region for our new table.
276     HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
277     List<HRegion> regions = cluster.getRegions(tableName);
278     HRegionInfo hri = getAndCheckSingleTableRegion(regions);
279 
280     int tableRegionIndex = ensureTableRegionNotOnSameServerAsMeta(admin, hri);
281 
282     // Turn off balancer so it doesn't cut in and mess up our placements.
283     this.admin.setBalancerRunning(false, true);
284     // Turn off the meta scanner so it don't remove parent on us.
285     cluster.getMaster().setCatalogJanitorEnabled(false);
286     try {
287       // Add a bit of load up into the table so splittable.
288       TESTING_UTIL.loadTable(t, HConstants.CATALOG_FAMILY);
289       // Get region pre-split.
290       HRegionServer server = cluster.getRegionServer(tableRegionIndex);
291       printOutRegions(server, "Initial regions: ");
292       int regionCount = server.getOnlineRegions().size();
293       // Insert into zk a blocking znode, a znode of same name as region
294       // so it gets in way of our splitting.
295       ZKAssign.createNodeClosing(t.getConnection().getZooKeeperWatcher(),
296         hri, new ServerName("any.old.server", 1234, -1));
297       // Now try splitting.... should fail.  And each should successfully
298       // rollback.
299       this.admin.split(hri.getRegionNameAsString());
300       this.admin.split(hri.getRegionNameAsString());
301       this.admin.split(hri.getRegionNameAsString());
302       // Wait around a while and assert count of regions remains constant.
303       for (int i = 0; i < 10; i++) {
304         Thread.sleep(100);
305         assertEquals(regionCount, server.getOnlineRegions().size());
306       }
307       // Now clear the zknode
308       ZKAssign.deleteClosingNode(t.getConnection().getZooKeeperWatcher(), hri);
309       // Now try splitting and it should work.
310       split(hri, server, regionCount);
311       // Get daughters
312       checkAndGetDaughters(tableName);
313       // OK, so split happened after we cleared the blocking node.
314     } finally {
315       admin.setBalancerRunning(true, false);
316       cluster.getMaster().setCatalogJanitorEnabled(true);
317       t.close();
318     }
319   }
320 
321   /**
322    * Messy test that simulates case where SplitTransactions fails to add one
323    * of the daughters up into the .META. table before crash.  We're testing
324    * fact that the shutdown handler will fixup the missing daughter region
325    * adding it back into .META.
326    * @throws IOException
327    * @throws InterruptedException
328    */
329   @Test (timeout = 300000) public void testShutdownSimpleFixup()
330   throws IOException, InterruptedException {
331     final byte [] tableName = Bytes.toBytes("testShutdownSimpleFixup");
332 
333     // Create table then get the single region for our new table.
334     HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
335     List<HRegion> regions = cluster.getRegions(tableName);
336     HRegionInfo hri = getAndCheckSingleTableRegion(regions);
337 
338     int tableRegionIndex = ensureTableRegionNotOnSameServerAsMeta(admin, hri);
339 
340     // Turn off balancer so it doesn't cut in and mess up our placements.
341     this.admin.setBalancerRunning(false, true);
342     // Turn off the meta scanner so it don't remove parent on us.
343     cluster.getMaster().setCatalogJanitorEnabled(false);
344     try {
345       // Add a bit of load up into the table so splittable.
346       TESTING_UTIL.loadTable(t, HConstants.CATALOG_FAMILY);
347       // Get region pre-split.
348       HRegionServer server = cluster.getRegionServer(tableRegionIndex);
349       printOutRegions(server, "Initial regions: ");
350       int regionCount = server.getOnlineRegions().size();
351       // Now split.
352       split(hri, server, regionCount);
353       // Get daughters
354       List<HRegion> daughters = checkAndGetDaughters(tableName);
355       // Remove one of the daughters from .META. to simulate failed insert of
356       // daughter region up into .META.
357       removeDaughterFromMeta(daughters.get(0).getRegionName());
358       // Now crash the server
359       cluster.abortRegionServer(tableRegionIndex);
360       waitUntilRegionServerDead();
361       awaitDaughters(tableName, daughters.size());
362       // Assert daughters are online.
363       regions = cluster.getRegions(tableName);
364       for (HRegion r: regions) {
365         assertTrue(daughters.contains(r));
366       }
367     } finally {
368       admin.setBalancerRunning(true, false);
369       cluster.getMaster().setCatalogJanitorEnabled(true);
370       t.close();
371     }
372   }
373 
374   /**
375    * Test that if daughter split on us, we won't do the shutdown handler fixup
376    * just because we can't find the immediate daughter of an offlined parent.
377    * @throws IOException
378    * @throws InterruptedException
379    */
380   @Test (timeout=300000) public void testShutdownFixupWhenDaughterHasSplit()
381   throws IOException, InterruptedException {
382     final byte [] tableName =
383       Bytes.toBytes("testShutdownFixupWhenDaughterHasSplit");
384 
385     // Create table then get the single region for our new table.
386     HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
387     List<HRegion> regions = cluster.getRegions(tableName);
388     HRegionInfo hri = getAndCheckSingleTableRegion(regions);
389 
390     int tableRegionIndex = ensureTableRegionNotOnSameServerAsMeta(admin, hri);
391 
392     // Turn off balancer so it doesn't cut in and mess up our placements.
393     this.admin.setBalancerRunning(false, true);
394     // Turn off the meta scanner so it don't remove parent on us.
395     cluster.getMaster().setCatalogJanitorEnabled(false);
396     try {
397       // Add a bit of load up into the table so splittable.
398       TESTING_UTIL.loadTable(t, HConstants.CATALOG_FAMILY);
399       // Get region pre-split.
400       HRegionServer server = cluster.getRegionServer(tableRegionIndex);
401       printOutRegions(server, "Initial regions: ");
402       int regionCount = server.getOnlineRegions().size();
403       // Now split.
404       split(hri, server, regionCount);
405       // Get daughters
406       List<HRegion> daughters = checkAndGetDaughters(tableName);
407       // Now split one of the daughters.
408       regionCount = server.getOnlineRegions().size();
409       HRegionInfo daughter = daughters.get(0).getRegionInfo();
410       // Compact first to ensure we have cleaned up references -- else the split
411       // will fail.
412       this.admin.compact(daughter.getRegionName());
413       daughters = cluster.getRegions(tableName);
414       HRegion daughterRegion = null;
415       for (HRegion r: daughters) {
416         if (r.getRegionInfo().equals(daughter)) daughterRegion = r;
417       }
418       assertTrue(daughterRegion != null);
419       for (int i=0; i<100; i++) {
420         if (!daughterRegion.hasReferences()) break;
421         Threads.sleep(100);
422       }
423       assertFalse("Waiting for refereces to be compacted", daughterRegion.hasReferences());
424       split(daughter, server, regionCount);
425       // Get list of daughters
426       daughters = cluster.getRegions(tableName);
427       // Now crash the server
428       cluster.abortRegionServer(tableRegionIndex);
429       waitUntilRegionServerDead();
430       awaitDaughters(tableName, daughters.size());
431       // Assert daughters are online and ONLY the original daughters -- that
432       // fixup didn't insert one during server shutdown recover.
433       regions = cluster.getRegions(tableName);
434       assertEquals(daughters.size(), regions.size());
435       for (HRegion r: regions) {
436         assertTrue(daughters.contains(r));
437       }
438     } finally {
439       admin.setBalancerRunning(true, false);
440       cluster.getMaster().setCatalogJanitorEnabled(true);
441       t.close();
442     }
443   }
444   
445   /**
446    * Verifies HBASE-5806.  When splitting is partially done and the master goes down
447    * when the SPLIT node is in either SPLIT or SPLITTING state.
448    * 
449    * @throws IOException
450    * @throws InterruptedException
451    * @throws NodeExistsException
452    * @throws KeeperException
453    */
454   @Test(timeout = 300000)
455   public void testMasterRestartWhenSplittingIsPartial()
456       throws IOException, InterruptedException, NodeExistsException,
457       KeeperException {
458     final byte[] tableName = Bytes.toBytes("testMasterRestartWhenSplittingIsPartial");
459 
460     // Create table then get the single region for our new table.
461     HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
462     List<HRegion> regions = cluster.getRegions(tableName);
463     HRegionInfo hri = getAndCheckSingleTableRegion(regions);
464 
465     int tableRegionIndex = ensureTableRegionNotOnSameServerAsMeta(admin, hri);
466 
467     // Turn off the meta scanner so it don't remove parent on us.
468     cluster.getMaster().setCatalogJanitorEnabled(false);
469     // Turn off balancer so it doesn't cut in and mess up our placements.
470     this.admin.setBalancerRunning(false, true);
471     
472     try {
473       // Add a bit of load up into the table so splittable.
474       TESTING_UTIL.loadTable(t, HConstants.CATALOG_FAMILY);
475       // Get region pre-split.
476       HRegionServer server = cluster.getRegionServer(tableRegionIndex);
477       printOutRegions(server, "Initial regions: ");
478       int regionCount = server.getOnlineRegions().size();
479       // Now, before we split, set special flag in master, a flag that has
480       // it FAIL the processing of split.
481       SplitRegionHandler.TEST_SKIP = true;
482       // Now try splitting and it should work.
483       split(hri, server, regionCount);
484       // Get daughters
485       checkAndGetDaughters(tableName);
486       // Assert the ephemeral node is up in zk.
487       String path = ZKAssign.getNodeName(t.getConnection()
488           .getZooKeeperWatcher(), hri.getEncodedName());
489       Stat stats = t.getConnection().getZooKeeperWatcher()
490           .getRecoverableZooKeeper().exists(path, false);
491       LOG.info("EPHEMERAL NODE BEFORE SERVER ABORT, path=" + path + ", stats="
492           + stats);
493       RegionTransitionData rtd = ZKAssign.getData(t.getConnection()
494           .getZooKeeperWatcher(), hri.getEncodedName());
495       // State could be SPLIT or SPLITTING.
496       assertTrue(rtd.getEventType().equals(EventType.RS_ZK_REGION_SPLIT)
497           || rtd.getEventType().equals(EventType.RS_ZK_REGION_SPLITTING));
498 
499 
500       // abort and wait for new master.
501       MockMasterWithoutCatalogJanitor master = abortAndWaitForMaster();
502 
503       this.admin = new HBaseAdmin(TESTING_UTIL.getConfiguration());
504 
505       // update the hri to be offlined and splitted. 
506       hri.setOffline(true);
507       hri.setSplit(true);
508       ServerName regionServerOfRegion = master.getAssignmentManager()
509           .getRegionServerOfRegion(hri);
510       assertTrue(regionServerOfRegion != null);
511 
512     } finally {
513       // Set this flag back.
514       SplitRegionHandler.TEST_SKIP = false;
515       admin.setBalancerRunning(true, false);
516       cluster.getMaster().setCatalogJanitorEnabled(true);
517       t.close();
518     }
519   }
520 
521 
522   /**
523    * Verifies HBASE-5806.  Here the case is that splitting is completed but before the
524    * CJ could remove the parent region the master is killed and restarted.
525    * @throws IOException
526    * @throws InterruptedException
527    * @throws NodeExistsException
528    * @throws KeeperException
529    */
530   @Test (timeout = 300000)
531   public void testMasterRestartAtRegionSplitPendingCatalogJanitor()
532       throws IOException, InterruptedException, NodeExistsException,
533       KeeperException {
534     final byte[] tableName = Bytes.toBytes("testMasterRestartAtRegionSplitPendingCatalogJanitor");
535 
536     // Create table then get the single region for our new table.
537     this.admin = new HBaseAdmin(TESTING_UTIL.getConfiguration());
538     HTable t = createTableAndWait(tableName, HConstants.CATALOG_FAMILY);
539     List<HRegion> regions = cluster.getRegions(tableName);
540     HRegionInfo hri = getAndCheckSingleTableRegion(regions);
541 
542     int tableRegionIndex = ensureTableRegionNotOnSameServerAsMeta(admin, hri);
543 
544     // Turn off balancer so it doesn't cut in and mess up our placements.
545     this.admin.setBalancerRunning(false, true);
546     // Turn off the meta scanner so it don't remove parent on us.
547     cluster.getMaster().setCatalogJanitorEnabled(false);
548     try {
549       // Add a bit of load up into the table so splittable.
550       TESTING_UTIL.loadTable(t, HConstants.CATALOG_FAMILY);
551       // Get region pre-split.
552       HRegionServer server = cluster.getRegionServer(tableRegionIndex);
553       printOutRegions(server, "Initial regions: ");
554       int regionCount = server.getOnlineRegions().size();
555       
556       split(hri, server, regionCount);
557       // Get daughters
558       checkAndGetDaughters(tableName);
559       // Assert the ephemeral node is up in zk.
560       String path = ZKAssign.getNodeName(t.getConnection()
561           .getZooKeeperWatcher(), hri.getEncodedName());
562       Stat stats = t.getConnection().getZooKeeperWatcher()
563           .getRecoverableZooKeeper().exists(path, false);
564       LOG.info("EPHEMERAL NODE BEFORE SERVER ABORT, path=" + path + ", stats="
565           + stats);
566       String node = ZKAssign.getNodeName(t.getConnection()
567           .getZooKeeperWatcher(), hri.getEncodedName());
568       Stat stat = new Stat();
569       byte[] data = ZKUtil.getDataNoWatch(t.getConnection()
570           .getZooKeeperWatcher(), node, stat);
571       // ZKUtil.create
572       for (int i=0; data != null && i<60; i++) {
573         Thread.sleep(1000);
574         data = ZKUtil.getDataNoWatch(t.getConnection().getZooKeeperWatcher(),
575             node, stat);
576 
577       }
578       assertNull("Waited too long for ZK node to be removed: "+node, data);
579       
580       MockMasterWithoutCatalogJanitor master = abortAndWaitForMaster();
581 
582       this.admin = new HBaseAdmin(TESTING_UTIL.getConfiguration());
583 
584       hri.setOffline(true);
585       hri.setSplit(true);
586       ServerName regionServerOfRegion = master.getAssignmentManager()
587           .getRegionServerOfRegion(hri);
588       assertTrue(regionServerOfRegion == null);
589     } finally {
590       // Set this flag back.
591       SplitRegionHandler.TEST_SKIP = false;
592       this.admin.setBalancerRunning(true, false);
593       cluster.getMaster().setCatalogJanitorEnabled(true);
594       t.close();
595     }
596   }
597 
598   /**
599    * While transitioning node from RS_ZK_REGION_SPLITTING to
600    * RS_ZK_REGION_SPLITTING during region split,if zookeper went down split always
601    * fails for the region. HBASE-6088 fixes this scenario.
602    * This test case is to test the znode is deleted(if created) or not in roll back.
603    * 
604    * @throws IOException
605    * @throws InterruptedException
606    * @throws KeeperException
607    */
608   @Test
609   public void testSplitBeforeSettingSplittingInZK() throws Exception,
610       InterruptedException, KeeperException {
611     testSplitBeforeSettingSplittingInZK(true);
612     testSplitBeforeSettingSplittingInZK(false);
613   }
614 
615   private void testSplitBeforeSettingSplittingInZK(boolean nodeCreated) throws Exception {
616     final byte[] tableName = Bytes.toBytes("testSplitBeforeSettingSplittingInZK");
617     
618     HBaseAdmin admin = new HBaseAdmin(TESTING_UTIL.getConfiguration());
619     // Create table then get the single region for our new table.
620     HTableDescriptor htd = new HTableDescriptor(tableName);
621     htd.addFamily(new HColumnDescriptor("cf"));
622     admin.createTable(htd);
623 
624     List<HRegion> regions = null;
625     for (int i=0; i<100; i++) {
626       regions = cluster.getRegions(tableName);
627       if (regions.size() > 0) break;
628       Thread.sleep(100);
629     }
630     int regionServerIndex = cluster.getServerWith(regions.get(0).getRegionName());
631     HRegionServer regionServer = cluster.getRegionServer(regionServerIndex);
632     SplitTransaction st = null;
633     if (nodeCreated) {
634       st = new MockedSplitTransaction(regions.get(0), null) {
635         @Override
636         int transitionNodeSplitting(ZooKeeperWatcher zkw, HRegionInfo parent,
637             ServerName serverName, int version) throws KeeperException, IOException {
638           throw new TransitionToSplittingFailedException();
639         }
640       };
641     } else {
642       st = new MockedSplitTransaction(regions.get(0), null) {
643         @Override
644         void createNodeSplitting(ZooKeeperWatcher zkw, HRegionInfo region, ServerName serverName)
645             throws KeeperException, IOException {
646           throw new SplittingNodeCreationFailedException ();
647         }
648       };
649     }
650     String node = ZKAssign.getNodeName(regionServer.getZooKeeper(), regions.get(0)
651         .getRegionInfo().getEncodedName());
652     // make sure the client is uptodate
653     regionServer.getZooKeeper().sync(node);
654     for (int i = 0; i < 100; i++) {
655       // We expect the znode to be deleted by this time. Here the znode could be in OPENED state and the 
656       // master has not yet deleted the znode.
657       if (ZKUtil.checkExists(regionServer.getZooKeeper(), node) != -1) {
658         Thread.sleep(100);
659       }
660     }
661     
662     try {
663       st.execute(regionServer, regionServer);
664     } catch (IOException e) {
665       // check for the specific instance in case the Split failed due to the existence of the znode in OPENED state.
666       // This will at least make the test to fail;
667       if (nodeCreated) {
668         assertTrue("Should be instance of TransitionToSplittingFailedException",
669             e instanceof TransitionToSplittingFailedException);
670       } else {
671         assertTrue("Should be instance of CreateSplittingNodeFailedException",
672             e instanceof SplittingNodeCreationFailedException );
673       }
674       node = ZKAssign.getNodeName(regionServer.getZooKeeper(), regions.get(0)
675           .getRegionInfo().getEncodedName());
676       // make sure the client is uptodate
677       regionServer.getZooKeeper().sync(node);
678       if (nodeCreated) {
679         assertFalse(ZKUtil.checkExists(regionServer.getZooKeeper(), node) == -1);
680       } else {
681         assertTrue(ZKUtil.checkExists(regionServer.getZooKeeper(), node) == -1);
682       }
683       assertTrue(st.rollback(regionServer, regionServer));
684       assertTrue(ZKUtil.checkExists(regionServer.getZooKeeper(), node) == -1);
685     }
686     if (admin.isTableAvailable(tableName) && admin.isTableEnabled(tableName)) {
687       admin.disableTable(tableName);
688       admin.deleteTable(tableName);
689     }
690   }
691   
692   @Test
693   public void testShouldClearRITWhenNodeFoundInSplittingState() throws Exception {
694     final byte[] tableName = Bytes.toBytes("testShouldClearRITWhenNodeFoundInSplittingState");
695     HBaseAdmin admin = new HBaseAdmin(TESTING_UTIL.getConfiguration());
696     // Create table then get the single region for our new table.
697     HTableDescriptor htd = new HTableDescriptor(tableName);
698     htd.addFamily(new HColumnDescriptor("cf"));
699     admin.createTable(htd);
700     for (int i = 0; cluster.getRegions(tableName).size() == 0 && i < 100; i++) {
701       Thread.sleep(100);
702     }
703     assertTrue("Table not online", cluster.getRegions(tableName).size() != 0);
704 
705     HRegion region = cluster.getRegions(tableName).get(0);
706     int regionServerIndex = cluster.getServerWith(region.getRegionName());
707     HRegionServer regionServer = cluster.getRegionServer(regionServerIndex);
708     SplitTransaction st = null;
709 
710     st = new MockedSplitTransaction(region, null) {
711       @Override
712       void createSplitDir(FileSystem fs, Path splitdir) throws IOException {
713         throw new IOException("");
714       }
715     };
716 
717     try {
718       st.execute(regionServer, regionServer);
719     } catch (IOException e) {
720       String node = ZKAssign.getNodeName(regionServer.getZooKeeper(), region
721           .getRegionInfo().getEncodedName());
722 
723       assertFalse(ZKUtil.checkExists(regionServer.getZooKeeper(), node) == -1);
724       AssignmentManager am = cluster.getMaster().getAssignmentManager();
725       for (int i = 0; !am.getRegionsInTransition().containsKey(
726           region.getRegionInfo().getEncodedName())
727           && i < 100; i++) {
728         Thread.sleep(200);
729       }
730       assertTrue("region is not in transition "+region,
731           am.getRegionsInTransition().containsKey(region.getRegionInfo().getEncodedName()));
732       RegionState regionState = am.getRegionsInTransition().get(region.getRegionInfo()
733           .getEncodedName());
734       assertTrue(regionState.getState() == RegionState.State.SPLITTING);
735       assertTrue(st.rollback(regionServer, regionServer));
736       assertTrue(ZKUtil.checkExists(regionServer.getZooKeeper(), node) == -1);
737       for (int i=0; am.getRegionsInTransition().containsKey(region.getRegionInfo().getEncodedName()) && i<100; i++) {
738         // Just in case the nodeDeleted event did not get executed.
739         Thread.sleep(200);
740       }
741       assertFalse("region is still in transition",
742           am.getRegionsInTransition().containsKey(region.getRegionInfo().getEncodedName()));
743     }
744     if (admin.isTableAvailable(tableName) && admin.isTableEnabled(tableName)) {
745       admin.disableTable(tableName);
746       admin.deleteTable(tableName);
747       admin.close();
748     }
749   }
750   
751   @Test(timeout = 60000)
752   public void testTableExistsIfTheSpecifiedTableRegionIsSplitParent() throws Exception {
753     ZooKeeperWatcher zkw = HBaseTestingUtility.getZooKeeperWatcher(TESTING_UTIL);
754     final byte[] tableName = 
755         Bytes.toBytes("testTableExistsIfTheSpecifiedTableRegionIsSplitParent");
756     HRegionServer regionServer = null;
757     List<HRegion> regions = null;
758     HBaseAdmin admin = new HBaseAdmin(TESTING_UTIL.getConfiguration());
759     try {
760       // Create table then get the single region for our new table.
761       HTable t = createTableAndWait(tableName, Bytes.toBytes("cf"));
762       regions = cluster.getRegions(tableName);
763       int regionServerIndex = cluster.getServerWith(regions.get(0).getRegionName());
764       regionServer = cluster.getRegionServer(regionServerIndex);
765       insertData(tableName, admin, t);
766       // Turn off balancer so it doesn't cut in and mess up our placements.
767       cluster.getMaster().setCatalogJanitorEnabled(false);
768       boolean tableExists = MetaReader.tableExists(regionServer.getCatalogTracker(),
769           Bytes.toString(tableName));
770       assertEquals("The specified table should present.", true, tableExists);
771       SplitTransaction st = new SplitTransaction(regions.get(0), Bytes.toBytes("row2"));
772       try {
773         st.prepare();
774         st.createDaughters(regionServer, regionServer);
775       } catch (IOException e) {
776 
777       }
778       tableExists = MetaReader.tableExists(regionServer.getCatalogTracker(),
779           Bytes.toString(tableName));
780       assertEquals("The specified table should present.", true, tableExists);
781     } finally {
782       if (regions != null) {
783         String node = ZKAssign.getNodeName(zkw, regions.get(0).getRegionInfo()
784             .getEncodedName());
785         ZKUtil.deleteNodeFailSilent(zkw, node);
786       }
787       cluster.getMaster().setCatalogJanitorEnabled(true);
788       admin.close();
789     }
790   }
791   
792   @Test(timeout = 180000)
793   public void testSplitShouldNotThrowNPEEvenARegionHasEmptySplitFiles() throws Exception {
794     Configuration conf = TESTING_UTIL.getConfiguration();
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     private static final long serialVersionUID = 7025885032995944524L;
1090 
1091     public TransitionToSplittingFailedException() {
1092       super();
1093     }
1094   }
1095 
1096   private static class SplittingNodeCreationFailedException  extends IOException {
1097     private static final long serialVersionUID = 1652404976265623004L;
1098 
1099     public SplittingNodeCreationFailedException () {
1100       super();
1101     }
1102   }
1103 }
1104