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