View Javadoc

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 java.io.IOException;
23  import java.util.ArrayList;
24  import java.util.List;
25  import java.util.ListIterator;
26  import java.util.concurrent.Callable;
27  import java.util.concurrent.ExecutionException;
28  import java.util.concurrent.Executors;
29  import java.util.concurrent.Future;
30  import java.util.concurrent.ThreadFactory;
31  import java.util.concurrent.ThreadPoolExecutor;
32  import java.util.concurrent.TimeUnit;
33  
34  import org.apache.commons.logging.Log;
35  import org.apache.commons.logging.LogFactory;
36  import org.apache.hadoop.conf.Configuration;
37  import org.apache.hadoop.fs.FileStatus;
38  import org.apache.hadoop.fs.FileSystem;
39  import org.apache.hadoop.fs.Path;
40  import org.apache.hadoop.hbase.HRegionInfo;
41  import org.apache.hadoop.hbase.Server;
42  import org.apache.hadoop.hbase.ServerName;
43  import org.apache.hadoop.hbase.catalog.MetaEditor;
44  import org.apache.hadoop.hbase.executor.EventHandler.EventType;
45  import org.apache.hadoop.hbase.executor.RegionTransitionData;
46  import org.apache.hadoop.hbase.io.Reference.Range;
47  import org.apache.hadoop.hbase.util.Bytes;
48  import org.apache.hadoop.hbase.util.CancelableProgressable;
49  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
50  import org.apache.hadoop.hbase.util.FSUtils;
51  import org.apache.hadoop.hbase.util.HasThread;
52  import org.apache.hadoop.hbase.util.PairOfSameType;
53  import org.apache.hadoop.hbase.util.Writables;
54  import org.apache.hadoop.hbase.zookeeper.ZKAssign;
55  import org.apache.hadoop.hbase.zookeeper.ZKUtil;
56  import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher;
57  import org.apache.zookeeper.KeeperException;
58  import org.apache.zookeeper.KeeperException.NodeExistsException;
59  
60  import com.google.common.util.concurrent.ThreadFactoryBuilder;
61  
62  /**
63   * Executes region split as a "transaction".  Call {@link #prepare()} to setup
64   * the transaction, {@link #execute(Server, RegionServerServices)} to run the
65   * transaction and {@link #rollback(Server, RegionServerServices)} to cleanup if execute fails.
66   *
67   * <p>Here is an example of how you would use this class:
68   * <pre>
69   *  SplitTransaction st = new SplitTransaction(this.conf, parent, midKey)
70   *  if (!st.prepare()) return;
71   *  try {
72   *    st.execute(server, services);
73   *  } catch (IOException ioe) {
74   *    try {
75   *      st.rollback(server, services);
76   *      return;
77   *    } catch (RuntimeException e) {
78   *      myAbortable.abort("Failed split, abort");
79   *    }
80   *  }
81   * </Pre>
82   * <p>This class is not thread safe.  Caller needs ensure split is run by
83   * one thread only.
84   */
85  public class SplitTransaction {
86    private static final Log LOG = LogFactory.getLog(SplitTransaction.class);
87    private static final String SPLITDIR = ".splits";
88  
89    /*
90     * Region to split
91     */
92    private final HRegion parent;
93    private HRegionInfo hri_a;
94    private HRegionInfo hri_b;
95    private Path splitdir;
96    private long fileSplitTimeout = 30000;
97    private int znodeVersion = -1;
98  
99    /*
100    * Row to split around
101    */
102   private final byte [] splitrow;
103 
104   /**
105    * Types to add to the transaction journal.
106    * Each enum is a step in the split transaction. Used to figure how much
107    * we need to rollback.
108    */
109   enum JournalEntry {
110     /**
111      * Set region as in transition, set it into SPLITTING state.
112      */
113     SET_SPLITTING_IN_ZK,
114     /**
115      * We created the temporary split data directory.
116      */
117     CREATE_SPLIT_DIR,
118     /**
119      * Closed the parent region.
120      */
121     CLOSED_PARENT_REGION,
122     /**
123      * The parent has been taken out of the server's online regions list.
124      */
125     OFFLINED_PARENT,
126     /**
127      * Started in on creation of the first daughter region.
128      */
129     STARTED_REGION_A_CREATION,
130     /**
131      * Started in on the creation of the second daughter region.
132      */
133     STARTED_REGION_B_CREATION,
134     /**
135      * Point of no return.
136      * If we got here, then transaction is not recoverable other than by
137      * crashing out the regionserver.
138      */
139     PONR
140   }
141 
142   /*
143    * Journal of how far the split transaction has progressed.
144    */
145   private final List<JournalEntry> journal = new ArrayList<JournalEntry>();
146 
147   /**
148    * Constructor
149    * @param r Region to split
150    * @param splitrow Row to split around
151    */
152   public SplitTransaction(final HRegion r, final byte [] splitrow) {
153     this.parent = r;
154     this.splitrow = splitrow;
155     this.splitdir = getSplitDir(this.parent);
156   }
157 
158   /**
159    * Does checks on split inputs.
160    * @return <code>true</code> if the region is splittable else
161    * <code>false</code> if it is not (e.g. its already closed, etc.).
162    */
163   public boolean prepare() {
164     if (!this.parent.isSplittable()) return false;
165     // Split key can be null if this region is unsplittable; i.e. has refs.
166     if (this.splitrow == null) return false;
167     HRegionInfo hri = this.parent.getRegionInfo();
168     parent.prepareToSplit();
169     // Check splitrow.
170     byte [] startKey = hri.getStartKey();
171     byte [] endKey = hri.getEndKey();
172     if (Bytes.equals(startKey, splitrow) ||
173         !this.parent.getRegionInfo().containsRow(splitrow)) {
174       LOG.info("Split row is not inside region key range or is equal to " +
175           "startkey: " + Bytes.toStringBinary(this.splitrow));
176       return false;
177     }
178     long rid = getDaughterRegionIdTimestamp(hri);
179     this.hri_a = new HRegionInfo(hri.getTableName(), startKey, this.splitrow,
180       false, rid);
181     this.hri_b = new HRegionInfo(hri.getTableName(), this.splitrow, endKey,
182       false, rid);
183     return true;
184   }
185 
186   /**
187    * Calculate daughter regionid to use.
188    * @param hri Parent {@link HRegionInfo}
189    * @return Daughter region id (timestamp) to use.
190    */
191   private static long getDaughterRegionIdTimestamp(final HRegionInfo hri) {
192     long rid = EnvironmentEdgeManager.currentTimeMillis();
193     // Regionid is timestamp.  Can't be less than that of parent else will insert
194     // at wrong location in .META. (See HBASE-710).
195     if (rid < hri.getRegionId()) {
196       LOG.warn("Clock skew; parent regions id is " + hri.getRegionId() +
197         " but current time here is " + rid);
198       rid = hri.getRegionId() + 1;
199     }
200     return rid;
201   }
202 
203   private static IOException closedByOtherException = new IOException(
204       "Failed to close region: already closed by another thread");
205   
206   /**
207    * Prepare the regions and region files.
208    * @param server Hosting server instance.  Can be null when testing (won't try
209    * and update in zk if a null server)
210    * @param services Used to online/offline regions.
211    * @throws IOException If thrown, transaction failed. Call {@link #rollback(Server, RegionServerServices)}
212    * @return Regions created
213    */
214   /* package */PairOfSameType<HRegion> createDaughters(final Server server,
215       final RegionServerServices services) throws IOException {
216     LOG.info("Starting split of region " + this.parent);
217     if ((server != null && server.isStopped()) ||
218         (services != null && services.isStopping())) {
219       throw new IOException("Server is stopped or stopping");
220     }
221     assert !this.parent.lock.writeLock().isHeldByCurrentThread(): "Unsafe to hold write lock while performing RPCs";
222 
223     // Coprocessor callback
224     if (this.parent.getCoprocessorHost() != null) {
225       this.parent.getCoprocessorHost().preSplit();
226     }
227 
228     // If true, no cluster to write meta edits to or to update znodes in.
229     boolean testing = server == null? true:
230       server.getConfiguration().getBoolean("hbase.testing.nocluster", false);
231     this.fileSplitTimeout = testing ? this.fileSplitTimeout :
232       server.getConfiguration().getLong("hbase.regionserver.fileSplitTimeout",
233           this.fileSplitTimeout);
234 
235     // Set ephemeral SPLITTING znode up in zk.  Mocked servers sometimes don't
236     // have zookeeper so don't do zk stuff if server or zookeeper is null
237     if (server != null && server.getZooKeeper() != null) {
238       try {
239         createNodeSplitting(server.getZooKeeper(),
240           this.parent.getRegionInfo(), server.getServerName());
241       } catch (KeeperException e) {
242         throw new IOException("Failed creating SPLITTING znode on " +
243           this.parent.getRegionNameAsString(), e);
244       }
245     }
246     this.journal.add(JournalEntry.SET_SPLITTING_IN_ZK);
247     if (server != null && server.getZooKeeper() != null) {
248       try {
249         // Transition node from SPLITTING to SPLITTING after creating the split node.
250         // Master will get the callback for node change only if the transition is successful.
251         // Note that if the transition fails then the rollback will delete the created znode
252         // TODO : May be we can add some new state to znode and handle the new state incase of success/failure
253         this.znodeVersion = transitionNodeSplitting(server.getZooKeeper(),
254             this.parent.getRegionInfo(), server.getServerName(), -1);
255       } catch (KeeperException e) {
256         throw new IOException("Failed setting SPLITTING znode on "
257             + this.parent.getRegionNameAsString(), e);
258       }
259     }
260     createSplitDir(this.parent.getFilesystem(), this.splitdir);
261     this.journal.add(JournalEntry.CREATE_SPLIT_DIR);
262  
263     List<StoreFile> hstoreFilesToSplit = null;
264     Exception exceptionToThrow = null;
265     try{
266       hstoreFilesToSplit = this.parent.close(false);
267     } catch (Exception e) {
268       exceptionToThrow = e;
269     }
270     if (exceptionToThrow == null && hstoreFilesToSplit == null) {
271       // The region was closed by a concurrent thread.  We can't continue
272       // with the split, instead we must just abandon the split.  If we
273       // reopen or split this could cause problems because the region has
274       // probably already been moved to a different server, or is in the
275       // process of moving to a different server.
276       exceptionToThrow = closedByOtherException;
277     }
278     if (exceptionToThrow != closedByOtherException) {
279       this.journal.add(JournalEntry.CLOSED_PARENT_REGION);
280     }
281     if (exceptionToThrow != null) {
282       if (exceptionToThrow instanceof IOException) throw (IOException)exceptionToThrow;
283       throw new IOException(exceptionToThrow);
284     }
285 
286     if (!testing) {
287       services.removeFromOnlineRegions(this.parent.getRegionInfo().getEncodedName());
288     }
289     this.journal.add(JournalEntry.OFFLINED_PARENT);
290 
291     // TODO: If splitStoreFiles were multithreaded would we complete steps in
292     // less elapsed time?  St.Ack 20100920
293     //
294     // splitStoreFiles creates daughter region dirs under the parent splits dir
295     // Nothing to unroll here if failure -- clean up of CREATE_SPLIT_DIR will
296     // clean this up.
297     splitStoreFiles(this.splitdir, hstoreFilesToSplit);
298 
299     // Log to the journal that we are creating region A, the first daughter
300     // region.  We could fail halfway through.  If we do, we could have left
301     // stuff in fs that needs cleanup -- a storefile or two.  Thats why we
302     // add entry to journal BEFORE rather than AFTER the change.
303     this.journal.add(JournalEntry.STARTED_REGION_A_CREATION);
304     HRegion a = createDaughterRegion(this.hri_a, this.parent.rsServices);
305 
306     // Ditto
307     this.journal.add(JournalEntry.STARTED_REGION_B_CREATION);
308     HRegion b = createDaughterRegion(this.hri_b, this.parent.rsServices);
309 
310     // This is the point of no return.  Adding subsequent edits to .META. as we
311     // do below when we do the daughter opens adding each to .META. can fail in
312     // various interesting ways the most interesting of which is a timeout
313     // BUT the edits all go through (See HBASE-3872).  IF we reach the PONR
314     // then subsequent failures need to crash out this regionserver; the
315     // server shutdown processing should be able to fix-up the incomplete split.
316     // The offlined parent will have the daughters as extra columns.  If
317     // we leave the daughter regions in place and do not remove them when we
318     // crash out, then they will have their references to the parent in place
319     // still and the server shutdown fixup of .META. will point to these
320     // regions.
321     // We should add PONR JournalEntry before offlineParentInMeta,so even if
322     // OfflineParentInMeta timeout,this will cause regionserver exit,and then
323     // master ServerShutdownHandler will fix daughter & avoid data loss. (See 
324     // HBase-4562).
325     this.journal.add(JournalEntry.PONR);
326 
327     // Edit parent in meta.  Offlines parent region and adds splita and splitb.
328     if (!testing) {
329       MetaEditor.offlineParentInMeta(server.getCatalogTracker(),
330         this.parent.getRegionInfo(), a.getRegionInfo(), b.getRegionInfo());
331     }
332     return new PairOfSameType<HRegion>(a, b);
333   }
334 
335   /**
336    * Perform time consuming opening of the daughter regions.
337    * @param server Hosting server instance.  Can be null when testing (won't try
338    * and update in zk if a null server)
339    * @param services Used to online/offline regions.
340    * @param a first daughter region
341    * @param a second daughter region
342    * @throws IOException If thrown, transaction failed. Call {@link #rollback(Server, RegionServerServices)}
343    */
344   /* package */void openDaughters(final Server server,
345       final RegionServerServices services, HRegion a, HRegion b)
346       throws IOException {
347     boolean stopped = server != null && server.isStopped();
348     boolean stopping = services != null && services.isStopping();
349     // TODO: Is this check needed here?
350     if (stopped || stopping) {
351       LOG.info("Not opening daughters " +
352           b.getRegionInfo().getRegionNameAsString() +
353           " and " +
354           a.getRegionInfo().getRegionNameAsString() +
355           " because stopping=" + stopping + ", stopped=" + stopped);
356     } else {
357       // Open daughters in parallel.
358       DaughterOpener aOpener = new DaughterOpener(server, a);
359       DaughterOpener bOpener = new DaughterOpener(server, b);
360       aOpener.start();
361       bOpener.start();
362       try {
363         aOpener.join();
364         bOpener.join();
365       } catch (InterruptedException e) {
366         Thread.currentThread().interrupt();
367         throw new IOException("Interrupted " + e.getMessage());
368       }
369       if (aOpener.getException() != null) {
370         throw new IOException("Failed " +
371           aOpener.getName(), aOpener.getException());
372       }
373       if (bOpener.getException() != null) {
374         throw new IOException("Failed " +
375           bOpener.getName(), bOpener.getException());
376       }
377       if (services != null) {
378         try {
379           // add 2nd daughter first (see HBASE-4335)
380           services.postOpenDeployTasks(b, server.getCatalogTracker(), true);
381           // Should add it to OnlineRegions
382           services.addToOnlineRegions(b);
383           services.postOpenDeployTasks(a, server.getCatalogTracker(), true);
384           services.addToOnlineRegions(a);
385         } catch (KeeperException ke) {
386           throw new IOException(ke);
387         }
388       }
389     }
390   }
391 
392   /**
393    * Finish off split transaction, transition the zknode
394    * @param server Hosting server instance.  Can be null when testing (won't try
395    * and update in zk if a null server)
396    * @param services Used to online/offline regions.
397    * @param a first daughter region
398    * @param a second daughter region
399    * @throws IOException If thrown, transaction failed. Call {@link #rollback(Server, RegionServerServices)}
400    */
401   /* package */void transitionZKNode(final Server server,
402       final RegionServerServices services, HRegion a, HRegion b)
403       throws IOException {
404     // Tell master about split by updating zk.  If we fail, abort.
405     if (server != null && server.getZooKeeper() != null) {
406       try {
407         this.znodeVersion = transitionNodeSplit(server.getZooKeeper(),
408           parent.getRegionInfo(), a.getRegionInfo(), b.getRegionInfo(),
409           server.getServerName(), this.znodeVersion);
410 
411         int spins = 0;
412         // Now wait for the master to process the split. We know it's done
413         // when the znode is deleted. The reason we keep tickling the znode is
414         // that it's possible for the master to miss an event.
415         do {
416           if (spins % 10 == 0) {
417             LOG.debug("Still waiting on the master to process the split for " +
418                 this.parent.getRegionInfo().getEncodedName());
419           }
420           Thread.sleep(100);
421           // When this returns -1 it means the znode doesn't exist
422           this.znodeVersion = tickleNodeSplit(server.getZooKeeper(),
423             parent.getRegionInfo(), a.getRegionInfo(), b.getRegionInfo(),
424             server.getServerName(), this.znodeVersion);
425           spins++;
426         } while (this.znodeVersion != -1 && !server.isStopped()
427             && !services.isStopping());
428       } catch (Exception e) {
429         if (e instanceof InterruptedException) {
430           Thread.currentThread().interrupt();
431         }
432         throw new IOException("Failed telling master about split", e);
433       }
434     }
435 
436     // Coprocessor callback
437     if (this.parent.getCoprocessorHost() != null) {
438       this.parent.getCoprocessorHost().postSplit(a,b);
439     }
440 
441     // Leaving here, the splitdir with its dross will be in place but since the
442     // split was successful, just leave it; it'll be cleaned when parent is
443     // deleted and cleaned up.
444   }
445 
446   /**
447    * Run the transaction.
448    * @param server Hosting server instance.  Can be null when testing (won't try
449    * and update in zk if a null server)
450    * @param services Used to online/offline regions.
451    * @throws IOException If thrown, transaction failed. Call {@link #rollback(Server, RegionServerServices)}
452    * @return Regions created
453    * @throws IOException
454    * @see #rollback(Server, RegionServerServices)
455    */
456   public PairOfSameType<HRegion> execute(final Server server,
457       final RegionServerServices services)
458   throws IOException {
459     PairOfSameType<HRegion> regions = createDaughters(server, services);
460     openDaughters(server, services, regions.getFirst(), regions.getSecond());
461     transitionZKNode(server, services, regions.getFirst(), regions.getSecond());
462     return regions;
463   }
464 
465   /*
466    * Open daughter region in its own thread.
467    * If we fail, abort this hosting server.
468    */
469   class DaughterOpener extends HasThread {
470     private final Server server;
471     private final HRegion r;
472     private Throwable t = null;
473 
474     DaughterOpener(final Server s, final HRegion r) {
475       super((s == null? "null-services": s.getServerName()) +
476         "-daughterOpener=" + r.getRegionInfo().getEncodedName());
477       setDaemon(true);
478       this.server = s;
479       this.r = r;
480     }
481 
482     /**
483      * @return Null if open succeeded else exception that causes us fail open.
484      * Call it after this thread exits else you may get wrong view on result.
485      */
486     Throwable getException() {
487       return this.t;
488     }
489 
490     @Override
491     public void run() {
492       try {
493         openDaughterRegion(this.server, r);
494       } catch (Throwable t) {
495         this.t = t;
496       }
497     }
498   }
499 
500   /**
501    * Open daughter regions, add them to online list and update meta.
502    * @param server
503    * @param services Can be null when testing.
504    * @param daughter
505    * @throws IOException
506    * @throws KeeperException
507    */
508   void openDaughterRegion(final Server server, final HRegion daughter)
509   throws IOException, KeeperException {
510     HRegionInfo hri = daughter.getRegionInfo();
511     LoggingProgressable reporter = server == null? null:
512       new LoggingProgressable(hri, server.getConfiguration());
513     daughter.openHRegion(reporter);
514   }
515 
516   static class LoggingProgressable implements CancelableProgressable {
517     private final HRegionInfo hri;
518     private long lastLog = -1;
519     private final long interval;
520 
521     LoggingProgressable(final HRegionInfo hri, final Configuration c) {
522       this.hri = hri;
523       this.interval = c.getLong("hbase.regionserver.split.daughter.open.log.interval",
524         10000);
525     }
526 
527     @Override
528     public boolean progress() {
529       long now = System.currentTimeMillis();
530       if (now - lastLog > this.interval) {
531         LOG.info("Opening " + this.hri.getRegionNameAsString());
532         this.lastLog = now;
533       }
534       return true;
535     }
536   }
537 
538   private static Path getSplitDir(final HRegion r) {
539     return new Path(r.getRegionDir(), SPLITDIR);
540   }
541 
542   /**
543    * @param fs Filesystem to use
544    * @param splitdir Directory to store temporary split data in
545    * @throws IOException If <code>splitdir</code> already exists or we fail
546    * to create it.
547    * @see #cleanupSplitDir(FileSystem, Path)
548    */
549   void createSplitDir(final FileSystem fs, final Path splitdir)
550   throws IOException {
551     if (fs.exists(splitdir)) {
552       LOG.info("The " + splitdir
553           + " directory exists.  Hence deleting it to recreate it");
554       if (!fs.delete(splitdir, true)) {
555         throw new IOException("Failed deletion of " + splitdir
556             + " before creating them again.");
557       }
558     }
559     if (!fs.mkdirs(splitdir)) throw new IOException("Failed create of " + splitdir);
560   }
561 
562   private static void cleanupSplitDir(final FileSystem fs, final Path splitdir)
563   throws IOException {
564     // Splitdir may have been cleaned up by reopen of the parent dir.
565     deleteDir(fs, splitdir, false);
566   }
567 
568   /**
569    * @param fs Filesystem to use
570    * @param dir Directory to delete
571    * @param mustPreExist If true, we'll throw exception if <code>dir</code>
572    * does not preexist, else we'll just pass.
573    * @throws IOException Thrown if we fail to delete passed <code>dir</code>
574    */
575   private static void deleteDir(final FileSystem fs, final Path dir,
576       final boolean mustPreExist)
577   throws IOException {
578     if (!fs.exists(dir)) {
579       if (mustPreExist) throw new IOException(dir.toString() + " does not exist!");
580     } else if (!fs.delete(dir, true)) {
581       throw new IOException("Failed delete of " + dir);
582     }
583   }
584 
585   private void splitStoreFiles(final Path splitdir,
586     final List<StoreFile> hstoreFilesToSplit)
587   throws IOException {
588     if (hstoreFilesToSplit == null) {
589       // Could be null because close didn't succeed -- for now consider it fatal
590       throw new IOException("Close returned empty list of StoreFiles");
591     }
592     // The following code sets up a thread pool executor with as many slots as
593     // there's files to split. It then fires up everything, waits for
594     // completion and finally checks for any exception
595     int nbFiles = hstoreFilesToSplit.size();
596     if (nbFiles == 0) {
597       // no file needs to be splitted.
598       return;
599     }
600     ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
601     builder.setNameFormat("StoreFileSplitter-%1$d");
602     ThreadFactory factory = builder.build();
603     ThreadPoolExecutor threadPool =
604       (ThreadPoolExecutor) Executors.newFixedThreadPool(nbFiles, factory);
605     List<Future<Void>> futures = new ArrayList<Future<Void>>(nbFiles);
606 
607      // Split each store file.
608     for (StoreFile sf: hstoreFilesToSplit) {
609       //splitStoreFile(sf, splitdir);
610       StoreFileSplitter sfs = new StoreFileSplitter(sf, splitdir);
611       futures.add(threadPool.submit(sfs));
612     }
613     // Shutdown the pool
614     threadPool.shutdown();
615 
616     // Wait for all the tasks to finish
617     try {
618       boolean stillRunning = !threadPool.awaitTermination(
619           this.fileSplitTimeout, TimeUnit.MILLISECONDS);
620       if (stillRunning) {
621         threadPool.shutdownNow();
622         // wait for the thread to shutdown completely.
623         while (!threadPool.isTerminated()) {
624           Thread.sleep(50);
625         }
626         throw new IOException("Took too long to split the" +
627             " files and create the references, aborting split");
628       }
629     } catch (InterruptedException e) {
630       Thread.currentThread().interrupt();
631       throw new IOException("Interrupted while waiting for file splitters", e);
632     }
633 
634     // Look for any exception
635     for (Future<Void> future: futures) {
636       try {
637         future.get();
638       } catch (InterruptedException e) {
639         Thread.currentThread().interrupt();
640         throw new IOException(
641             "Interrupted while trying to get the results of file splitters", e);
642       } catch (ExecutionException e) {
643         throw new IOException(e);
644       }
645     }
646   }
647 
648   private void splitStoreFile(final StoreFile sf, final Path splitdir)
649   throws IOException {
650     FileSystem fs = this.parent.getFilesystem();
651     byte [] family = sf.getFamily();
652     String encoded = this.hri_a.getEncodedName();
653     Path storedir = Store.getStoreHomedir(splitdir, encoded, family);
654     StoreFile.split(fs, storedir, sf, this.splitrow, Range.bottom);
655     encoded = this.hri_b.getEncodedName();
656     storedir = Store.getStoreHomedir(splitdir, encoded, family);
657     StoreFile.split(fs, storedir, sf, this.splitrow, Range.top);
658   }
659 
660   /**
661    * Utility class used to do the file splitting / reference writing
662    * in parallel instead of sequentially.
663    */
664   class StoreFileSplitter implements Callable<Void> {
665 
666     private final StoreFile sf;
667     private final Path splitdir;
668 
669     /**
670      * Constructor that takes what it needs to split
671      * @param sf which file
672      * @param splitdir where the splitting is done
673      */
674     public StoreFileSplitter(final StoreFile sf, final Path splitdir) {
675       this.sf = sf;
676       this.splitdir = splitdir;
677     }
678 
679     public Void call() throws IOException {
680       splitStoreFile(sf, splitdir);
681       return null;
682     }
683   }
684 
685   /**
686    * @param hri Spec. for daughter region to open.
687    * @param flusher Flusher this region should use.
688    * @return Created daughter HRegion.
689    * @throws IOException
690    * @see #cleanupDaughterRegion(FileSystem, Path, HRegionInfo)
691    */
692   HRegion createDaughterRegion(final HRegionInfo hri,
693       final RegionServerServices rsServices)
694   throws IOException {
695     // Package private so unit tests have access.
696     FileSystem fs = this.parent.getFilesystem();
697     Path regionDir = getSplitDirForDaughter(this.parent.getFilesystem(),
698       this.splitdir, hri);
699     HRegion r = HRegion.newHRegion(this.parent.getTableDir(),
700       this.parent.getLog(), fs, this.parent.getConf(),
701       hri, this.parent.getTableDesc(), rsServices);
702     long halfParentReadRequestCount = this.parent.getReadRequestsCount() / 2;
703     r.readRequestsCount.set(halfParentReadRequestCount);
704     r.setOpMetricsReadRequestCount(halfParentReadRequestCount);
705     long halfParentWriteRequest = this.parent.getWriteRequestsCount() / 2;
706     r.writeRequestsCount.set(halfParentWriteRequest);
707     r.setOpMetricsWriteRequestCount(halfParentWriteRequest);    
708     HRegion.moveInitialFilesIntoPlace(fs, regionDir, r.getRegionDir());
709     return r;
710   }
711 
712   private static void cleanupDaughterRegion(final FileSystem fs,
713     final Path tabledir, final String encodedName)
714   throws IOException {
715     Path regiondir = HRegion.getRegionDir(tabledir, encodedName);
716     // Dir may not preexist.
717     deleteDir(fs, regiondir, false);
718   }
719 
720   /*
721    * Get the daughter directories in the splits dir.  The splits dir is under
722    * the parent regions' directory.
723    * @param fs
724    * @param splitdir
725    * @param hri
726    * @return Path to daughter split dir.
727    * @throws IOException
728    */
729   private static Path getSplitDirForDaughter(final FileSystem fs,
730       final Path splitdir, final HRegionInfo hri)
731   throws IOException {
732     return new Path(splitdir, hri.getEncodedName());
733   }
734 
735   /**
736    * @param server Hosting server instance (May be null when testing).
737    * @param services
738    * @throws IOException If thrown, rollback failed.  Take drastic action.
739    * @return True if we successfully rolled back, false if we got to the point
740    * of no return and so now need to abort the server to minimize damage.
741    */
742   public boolean rollback(final Server server, final RegionServerServices services)
743   throws IOException {
744     boolean result = true;
745     FileSystem fs = this.parent.getFilesystem();
746     ListIterator<JournalEntry> iterator =
747       this.journal.listIterator(this.journal.size());
748     // Iterate in reverse.
749     while (iterator.hasPrevious()) {
750       JournalEntry je = iterator.previous();
751       switch(je) {
752       
753       case SET_SPLITTING_IN_ZK:
754         if (server != null && server.getZooKeeper() != null) {
755           cleanZK(server, this.parent.getRegionInfo());
756         }
757         break;
758 
759       case CREATE_SPLIT_DIR:
760     	this.parent.writestate.writesEnabled = true;
761         cleanupSplitDir(fs, this.splitdir);
762         break;
763 
764       case CLOSED_PARENT_REGION:
765         try {
766           // So, this returns a seqid but if we just closed and then reopened, we
767           // should be ok. On close, we flushed using sequenceid obtained from
768           // hosting regionserver so no need to propagate the sequenceid returned
769           // out of initialize below up into regionserver as we normally do.
770           // TODO: Verify.
771           this.parent.initialize();
772         } catch (IOException e) {
773           LOG.error("Failed rollbacking CLOSED_PARENT_REGION of region " +
774             this.parent.getRegionNameAsString(), e);
775           throw new RuntimeException(e);
776         }
777         break;
778 
779       case STARTED_REGION_A_CREATION:
780         cleanupDaughterRegion(fs, this.parent.getTableDir(),
781           this.hri_a.getEncodedName());
782         break;
783 
784       case STARTED_REGION_B_CREATION:
785         cleanupDaughterRegion(fs, this.parent.getTableDir(),
786           this.hri_b.getEncodedName());
787         break;
788 
789       case OFFLINED_PARENT:
790         if (services != null) services.addToOnlineRegions(this.parent);
791         break;
792 
793       case PONR:
794         // We got to the point-of-no-return so we need to just abort. Return
795         // immediately.  Do not clean up created daughter regions.  They need
796         // to be in place so we don't delete the parent region mistakenly.
797         // See HBASE-3872.
798         return false;
799 
800       default:
801         throw new RuntimeException("Unhandled journal entry: " + je);
802       }
803     }
804     return result;
805   }
806 
807   HRegionInfo getFirstDaughter() {
808     return hri_a;
809   }
810 
811   HRegionInfo getSecondDaughter() {
812     return hri_b;
813   }
814 
815   // For unit testing.
816   Path getSplitDir() {
817     return this.splitdir;
818   }
819 
820   /**
821    * Clean up any split detritus that may have been left around from previous
822    * split attempts.
823    * Call this method on initial region deploy.  Cleans up any mess
824    * left by previous deploys of passed <code>r</code> region.
825    * @param r
826    * @throws IOException
827    */
828   static void cleanupAnySplitDetritus(final HRegion r) throws IOException {
829     Path splitdir = getSplitDir(r);
830     FileSystem fs = r.getFilesystem();
831     if (!fs.exists(splitdir)) return;
832     // Look at the splitdir.  It could have the encoded names of the daughter
833     // regions we tried to make.  See if the daughter regions actually got made
834     // out under the tabledir.  If here under splitdir still, then the split did
835     // not complete.  Try and do cleanup.  This code WILL NOT catch the case
836     // where we successfully created daughter a but regionserver crashed during
837     // the creation of region b.  In this case, there'll be an orphan daughter
838     // dir in the filesystem.  TOOD: Fix.
839     FileStatus [] daughters = fs.listStatus(splitdir, new FSUtils.DirFilter(fs));
840     for (int i = 0; i < daughters.length; i++) {
841       cleanupDaughterRegion(fs, r.getTableDir(),
842         daughters[i].getPath().getName());
843     }
844     cleanupSplitDir(r.getFilesystem(), splitdir);
845     LOG.info("Cleaned up old failed split transaction detritus: " + splitdir);
846   }
847 
848   private static void cleanZK(final Server server, final HRegionInfo hri) {
849     try {
850       // Only delete if its in expected state; could have been hijacked.
851       ZKAssign.deleteNode(server.getZooKeeper(), hri.getEncodedName(),
852         EventType.RS_ZK_REGION_SPLITTING);
853     } catch (KeeperException e) {
854       server.abort("Failed cleanup of " + hri.getRegionNameAsString(), e);
855     }
856   }
857 
858   /**
859    * Creates a new ephemeral node in the SPLITTING state for the specified region.
860    * Create it ephemeral in case regionserver dies mid-split.
861    *
862    * <p>Does not transition nodes from other states.  If a node already exists
863    * for this region, a {@link NodeExistsException} will be thrown.
864    *
865    * @param zkw zk reference
866    * @param region region to be created as offline
867    * @param serverName server event originates from
868    * @return Version of znode created.
869    * @throws KeeperException 
870    * @throws IOException 
871    */
872   void createNodeSplitting(final ZooKeeperWatcher zkw, final HRegionInfo region,
873       final ServerName serverName) throws KeeperException, IOException {
874     LOG.debug(zkw.prefix("Creating ephemeral node for " +
875       region.getEncodedName() + " in SPLITTING state"));
876     RegionTransitionData data =
877       new RegionTransitionData(EventType.RS_ZK_REGION_SPLITTING,
878         region.getRegionName(), serverName);
879 
880     String node = ZKAssign.getNodeName(zkw, region.getEncodedName());
881     if (!ZKUtil.createEphemeralNodeAndWatch(zkw, node, data.getBytes())) {
882       throw new IOException("Failed create of ephemeral " + node);
883     }
884   }
885 
886   /**
887    * Transitions an existing node for the specified region which is
888    * currently in the SPLITTING state to be in the SPLIT state.  Converts the
889    * ephemeral SPLITTING znode to an ephemeral SPLIT node.  Master cleans up
890    * SPLIT znode when it reads it (or if we crash, zk will clean it up).
891    *
892    * <p>Does not transition nodes from other states.  If for some reason the
893    * node could not be transitioned, the method returns -1.  If the transition
894    * is successful, the version of the node after transition is returned.
895    *
896    * <p>This method can fail and return false for three different reasons:
897    * <ul><li>Node for this region does not exist</li>
898    * <li>Node for this region is not in SPLITTING state</li>
899    * <li>After verifying SPLITTING state, update fails because of wrong version
900    * (this should never actually happen since an RS only does this transition
901    * following a transition to SPLITTING.  if two RS are conflicting, one would
902    * fail the original transition to SPLITTING and not this transition)</li>
903    * </ul>
904    *
905    * <p>Does not set any watches.
906    *
907    * <p>This method should only be used by a RegionServer when completing the
908    * open of a region.
909    *
910    * @param zkw zk reference
911    * @param parent region to be transitioned to opened
912    * @param a Daughter a of split
913    * @param b Daughter b of split
914    * @param serverName server event originates from
915    * @return version of node after transition, -1 if unsuccessful transition
916    * @throws KeeperException if unexpected zookeeper exception
917    * @throws IOException 
918    */
919   private static int transitionNodeSplit(ZooKeeperWatcher zkw,
920       HRegionInfo parent, HRegionInfo a, HRegionInfo b, ServerName serverName,
921       final int znodeVersion)
922   throws KeeperException, IOException {
923     byte [] payload = Writables.getBytes(a, b);
924     return ZKAssign.transitionNode(zkw, parent, serverName,
925       EventType.RS_ZK_REGION_SPLITTING, EventType.RS_ZK_REGION_SPLIT,
926       znodeVersion, payload);
927   }
928 
929   /**
930    * 
931    * @param zkw zk reference
932    * @param parent region to be transitioned to splitting
933    * @param serverName server event originates from
934    * @param version znode version
935    * @return version of node after transition, -1 if unsuccessful transition
936    * @throws KeeperException
937    * @throws IOException
938    */
939   int transitionNodeSplitting(final ZooKeeperWatcher zkw, final HRegionInfo parent,
940       final ServerName serverName, final int version) throws KeeperException, IOException {
941     return ZKAssign.transitionNode(zkw, parent, serverName,
942       EventType.RS_ZK_REGION_SPLITTING, EventType.RS_ZK_REGION_SPLITTING, version);
943   }
944 
945   private static int tickleNodeSplit(ZooKeeperWatcher zkw,
946       HRegionInfo parent, HRegionInfo a, HRegionInfo b, ServerName serverName,
947       final int znodeVersion)
948   throws KeeperException, IOException {
949     byte [] payload = Writables.getBytes(a, b);
950     return ZKAssign.transitionNode(zkw, parent, serverName,
951       EventType.RS_ZK_REGION_SPLIT, EventType.RS_ZK_REGION_SPLIT,
952       znodeVersion, payload);
953   }
954 }