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.catalog.MetaEditor;
43  import org.apache.hadoop.hbase.io.Reference.Range;
44  import org.apache.hadoop.hbase.util.Bytes;
45  import org.apache.hadoop.hbase.util.CancelableProgressable;
46  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
47  import org.apache.hadoop.hbase.util.FSUtils;
48  import org.apache.hadoop.hbase.util.PairOfSameType;
49  import org.apache.zookeeper.KeeperException;
50  
51  import com.google.common.util.concurrent.ThreadFactoryBuilder;
52  
53  /**
54   * Executes region split as a "transaction".  Call {@link #prepare()} to setup
55   * the transaction, {@link #execute(OnlineRegions)} to run the transaction and
56   * {@link #rollback(OnlineRegions)} to cleanup if execute fails.
57   *
58   * <p>Here is an example of how you would use this class:
59   * <pre>
60   *  SplitTransaction st = new SplitTransaction(this.conf, parent, midKey)
61   *  if (!st.prepare()) return;
62   *  try {
63   *    st.execute(myOnlineRegions);
64   *  } catch (IOException ioe) {
65   *    try {
66   *      st.rollback(myOnlineRegions);
67   *      return;
68   *    } catch (RuntimeException e) {
69   *      myAbortable.abort("Failed split, abort");
70   *    }
71   *  }
72   * </Pre>
73   * <p>This class is not thread safe.  Caller needs ensure split is run by
74   * one thread only.
75   */
76  class SplitTransaction {
77    private static final Log LOG = LogFactory.getLog(SplitTransaction.class);
78    private static final String SPLITDIR = "splits";
79  
80    /*
81     * Region to split
82     */
83    private final HRegion parent;
84    private HRegionInfo hri_a;
85    private HRegionInfo hri_b;
86    private Path splitdir;
87    private long fileSplitTimeout = 30000;
88  
89    /*
90     * Row to split around
91     */
92    private final byte [] splitrow;
93  
94    /**
95     * Types to add to the transaction journal
96     */
97    enum JournalEntry {
98      /**
99       * We created the temporary split data directory.
100      */
101     CREATE_SPLIT_DIR,
102     /**
103      * Closed the parent region.
104      */
105     CLOSED_PARENT_REGION,
106     /**
107      * The parent has been taken out of the server's online regions list.
108      */
109     OFFLINED_PARENT,
110     /**
111      * Started in on creation of the first daughter region.
112      */
113     STARTED_REGION_A_CREATION,
114     /**
115      * Started in on the creation of the second daughter region.
116      */
117     STARTED_REGION_B_CREATION
118   }
119 
120   /*
121    * Journal of how far the split transaction has progressed.
122    */
123   private final List<JournalEntry> journal = new ArrayList<JournalEntry>();
124 
125   /**
126    * Constructor
127    * @param services So we can online new servces.  If null, we'll skip onlining
128    * (Useful testing).
129    * @param c Configuration to use running split
130    * @param r Region to split
131    * @param splitrow Row to split around
132    */
133   SplitTransaction(final HRegion r, final byte [] splitrow) {
134     this.parent = r;
135     this.splitrow = splitrow;
136     this.splitdir = getSplitDir(this.parent);
137   }
138 
139   /**
140    * Does checks on split inputs.
141    * @return <code>true</code> if the region is splittable else
142    * <code>false</code> if it is not (e.g. its already closed, etc.).
143    */
144   public boolean prepare() {
145     if (this.parent.isClosed() || this.parent.isClosing()) return false;
146     HRegionInfo hri = this.parent.getRegionInfo();
147     parent.prepareToSplit();
148     // Check splitrow.
149     byte [] startKey = hri.getStartKey();
150     byte [] endKey = hri.getEndKey();
151     if (Bytes.equals(startKey, splitrow) ||
152         !this.parent.getRegionInfo().containsRow(splitrow)) {
153       LOG.info("Split row is not inside region key range or is equal to " +
154           "startkey: " + Bytes.toString(this.splitrow));
155       return false;
156     }
157     long rid = getDaughterRegionIdTimestamp(hri);
158     this.hri_a = new HRegionInfo(hri.getTableDesc(), startKey, this.splitrow,
159       false, rid);
160     this.hri_b = new HRegionInfo(hri.getTableDesc(), this.splitrow, endKey,
161       false, rid);
162     return true;
163   }
164 
165   /**
166    * Calculate daughter regionid to use.
167    * @param hri Parent {@link HRegionInfo}
168    * @return Daughter region id (timestamp) to use.
169    */
170   private static long getDaughterRegionIdTimestamp(final HRegionInfo hri) {
171     long rid = EnvironmentEdgeManager.currentTimeMillis();
172     // Regionid is timestamp.  Can't be less than that of parent else will insert
173     // at wrong location in .META. (See HBASE-710).
174     if (rid < hri.getRegionId()) {
175       LOG.warn("Clock skew; parent regions id is " + hri.getRegionId() +
176         " but current time here is " + rid);
177       rid = hri.getRegionId() + 1;
178     }
179     return rid;
180   }
181 
182   /**
183    * Run the transaction.
184    * @param server Hosting server instance.
185    * @param services Used to online/offline regions.
186    * @throws IOException If thrown, transaction failed. Call {@link #rollback(OnlineRegions)}
187    * @return Regions created
188    * @see #rollback(OnlineRegions)
189    */
190   PairOfSameType<HRegion> execute(final Server server,
191       final RegionServerServices services)
192   throws IOException {
193     LOG.info("Starting split of region " + this.parent);
194     if ((server != null && server.isStopped()) ||
195         (services != null && services.isStopping())) {
196       throw new IOException("Server is stopped or stopping");
197     }
198     assert !this.parent.lock.writeLock().isHeldByCurrentThread() : "Unsafe to hold write lock while performing RPCs";
199 
200     // If true, no cluster to write meta edits into.
201     boolean testing = server == null? true:
202       server.getConfiguration().getBoolean("hbase.testing.nocluster", false);
203     this.fileSplitTimeout = testing ? this.fileSplitTimeout :
204         server.getConfiguration().getLong(
205             "hbase.regionserver.fileSplitTimeout", this.fileSplitTimeout);
206 
207     createSplitDir(this.parent.getFilesystem(), this.splitdir);
208     this.journal.add(JournalEntry.CREATE_SPLIT_DIR);
209 
210     List<StoreFile> hstoreFilesToSplit = this.parent.close(false);
211     if (hstoreFilesToSplit == null) {
212       // The region was closed by a concurrent thread.  We can't continue
213       // with the split, instead we must just abandon the split.  If we
214       // reopen or split this could cause problems because the region has
215       // probably already been moved to a different server, or is in the
216       // process of moving to a different server.
217       throw new IOException("Failed to close region: already closed by " +
218         "another thread");
219     }
220     this.journal.add(JournalEntry.CLOSED_PARENT_REGION);
221 
222     if (!testing) {
223       services.removeFromOnlineRegions(this.parent.getRegionInfo().getEncodedName());
224     }
225     this.journal.add(JournalEntry.OFFLINED_PARENT);
226 
227     // TODO: If the below were multithreaded would we complete steps in less
228     // elapsed time?  St.Ack 20100920
229 
230     splitStoreFiles(this.splitdir, hstoreFilesToSplit);
231     // splitStoreFiles creates daughter region dirs under the parent splits dir
232     // Nothing to unroll here if failure -- clean up of CREATE_SPLIT_DIR will
233     // clean this up.
234 
235     // Log to the journal that we are creating region A, the first daughter
236     // region.  We could fail halfway through.  If we do, we could have left
237     // stuff in fs that needs cleanup -- a storefile or two.  Thats why we
238     // add entry to journal BEFORE rather than AFTER the change.
239     this.journal.add(JournalEntry.STARTED_REGION_A_CREATION);
240     HRegion a = createDaughterRegion(this.hri_a, this.parent.flushRequester);
241 
242     // Ditto
243     this.journal.add(JournalEntry.STARTED_REGION_B_CREATION);
244     HRegion b = createDaughterRegion(this.hri_b, this.parent.flushRequester);
245 
246     // Edit parent in meta
247     if (!testing) {
248       MetaEditor.offlineParentInMeta(server.getCatalogTracker(),
249         this.parent.getRegionInfo(), a.getRegionInfo(), b.getRegionInfo());
250     }
251 
252     // The is the point of no return.  We are committed to the split now.  We
253     // have still the daughter regions to open but meta has been changed.
254     // If we fail from here on out, we can not rollback so, we'll just abort.
255     // The meta has been changed though so there will need to be a fixup run
256     // during processing of the crashed server by master (TODO: Verify this in place).
257 
258     // TODO: Could we be smarter about the sequence in which we do these steps?
259 
260     if (!testing) {
261       // Open daughters in parallel.
262       DaughterOpener aOpener = new DaughterOpener(server, services, a);
263       DaughterOpener bOpener = new DaughterOpener(server, services, b);
264       aOpener.start();
265       bOpener.start();
266       try {
267         aOpener.join();
268         bOpener.join();
269       } catch (InterruptedException e) {
270         server.abort("Exception running daughter opens", e);
271       }
272     }
273 
274     // Leaving here, the splitdir with its dross will be in place but since the
275     // split was successful, just leave it; it'll be cleaned when parent is
276     // deleted and cleaned up.
277     return new PairOfSameType<HRegion>(a, b);
278   }
279 
280   class DaughterOpener extends Thread {
281     private final RegionServerServices services;
282     private final Server server;
283     private final HRegion r;
284 
285     DaughterOpener(final Server s, final RegionServerServices services,
286         final HRegion r) {
287       super(s.getServerName() + "-daughterOpener=" + r.getRegionInfo().getEncodedName());
288       setDaemon(true);
289       this.services = services;
290       this.server = s;
291       this.r = r;
292     }
293 
294     @Override
295     public void run() {
296       try {
297         openDaughterRegion(this.server, this.services, r);
298       } catch (Throwable t) {
299         this.server.abort("Failed open of daughter " +
300           this.r.getRegionInfo().getRegionNameAsString(), t);
301       }
302     }
303   }
304 
305   /**
306    * Open daughter regions, add them to online list and update meta.
307    * @param server
308    * @param services
309    * @param daughter
310    * @throws IOException
311    * @throws KeeperException
312    */
313   void openDaughterRegion(final Server server,
314       final RegionServerServices services, final HRegion daughter)
315   throws IOException, KeeperException {
316     if (server.isStopped() || services.isStopping()) {
317       MetaEditor.addDaughter(server.getCatalogTracker(),
318         daughter.getRegionInfo(), null);
319       LOG.info("Not opening daughter " +
320         daughter.getRegionInfo().getRegionNameAsString() +
321         " because stopping=" + services.isStopping() + ", stopped=" +
322         server.isStopped());
323       return;
324     }
325     HRegionInfo hri = daughter.getRegionInfo();
326     LoggingProgressable reporter =
327       new LoggingProgressable(hri, server.getConfiguration());
328     HRegion r = daughter.openHRegion(reporter);
329     services.postOpenDeployTasks(r, server.getCatalogTracker(), true);
330   }
331 
332   static class LoggingProgressable implements CancelableProgressable {
333     private final HRegionInfo hri;
334     private long lastLog = -1;
335     private final long interval;
336 
337     LoggingProgressable(final HRegionInfo hri, final Configuration c) {
338       this.hri = hri;
339       this.interval = c.getLong("hbase.regionserver.split.daughter.open.log.interval",
340         10000);
341     }
342 
343     @Override
344     public boolean progress() {
345       long now = System.currentTimeMillis();
346       if (now - lastLog > this.interval) {
347         LOG.info("Opening " + this.hri.getRegionNameAsString());
348         this.lastLog = now;
349       }
350       return true;
351     }
352   }
353 
354   private static Path getSplitDir(final HRegion r) {
355     return new Path(r.getRegionDir(), SPLITDIR);
356   }
357 
358   /**
359    * @param fs Filesystem to use
360    * @param splitdir Directory to store temporary split data in
361    * @throws IOException If <code>splitdir</code> already exists or we fail
362    * to create it.
363    * @see #cleanupSplitDir(FileSystem, Path)
364    */
365   private static void createSplitDir(final FileSystem fs, final Path splitdir)
366   throws IOException {
367     if (fs.exists(splitdir)) throw new IOException("Splitdir already exits? " + splitdir);
368     if (!fs.mkdirs(splitdir)) throw new IOException("Failed create of " + splitdir);
369   }
370 
371   private static void cleanupSplitDir(final FileSystem fs, final Path splitdir)
372   throws IOException {
373     // Splitdir may have been cleaned up by reopen of the parent dir.
374     deleteDir(fs, splitdir, false);
375   }
376 
377   /**
378    * @param fs Filesystem to use
379    * @param dir Directory to delete
380    * @param mustPreExist If true, we'll throw exception if <code>dir</code>
381    * does not preexist, else we'll just pass.
382    * @throws IOException Thrown if we fail to delete passed <code>dir</code>
383    */
384   private static void deleteDir(final FileSystem fs, final Path dir,
385       final boolean mustPreExist)
386   throws IOException {
387     if (!fs.exists(dir)) {
388       if (mustPreExist) throw new IOException(dir.toString() + " does not exist!");
389     } else if (!fs.delete(dir, true)) {
390       throw new IOException("Failed delete of " + dir);
391     }
392   }
393 
394   private void splitStoreFiles(final Path splitdir,
395     final List<StoreFile> hstoreFilesToSplit)
396   throws IOException {
397     if (hstoreFilesToSplit == null) {
398       // Could be null because close didn't succeed -- for now consider it fatal
399       throw new IOException("Close returned empty list of StoreFiles");
400     }
401     // The following code sets up a thread pool executor with as many slots as
402     // there's files to split. It then fires up everything, waits for
403     // completion and finally checks for any exception
404     int nbFiles = hstoreFilesToSplit.size();
405     ThreadFactoryBuilder builder = new ThreadFactoryBuilder();
406     builder.setNameFormat("StoreFileSplitter-%1$d");
407     ThreadFactory factory = builder.build();
408     ThreadPoolExecutor threadPool =
409       (ThreadPoolExecutor) Executors.newFixedThreadPool(nbFiles, factory);
410     List<Future<Void>> futures = new ArrayList<Future<Void>>(nbFiles);
411 
412      // Split each store file.
413     for (StoreFile sf: hstoreFilesToSplit) {
414       //splitStoreFile(sf, splitdir);
415       StoreFileSplitter sfs = new StoreFileSplitter(sf, splitdir);
416       futures.add(threadPool.submit(sfs));
417     }
418     // Shutdown the pool
419     threadPool.shutdown();
420 
421     // Wait for all the tasks to finish
422     try {
423       boolean stillRunning = !threadPool.awaitTermination(
424           this.fileSplitTimeout, TimeUnit.MILLISECONDS);
425       if (stillRunning) {
426         threadPool.shutdownNow();
427         throw new IOException("Took too long to split the" +
428             " files and create the references, aborting split");
429       }
430     } catch (InterruptedException e) {
431       Thread.currentThread().interrupt();
432       throw new IOException("Interrupted while waiting for file splitters", e);
433     }
434 
435     // Look for any exception
436     for (Future future : futures) {
437       try {
438         future.get();
439       } catch (InterruptedException e) {
440         Thread.currentThread().interrupt();
441         throw new IOException(
442             "Interrupted while trying to get the results of file splitters", e);
443       } catch (ExecutionException e) {
444         throw new IOException(e);
445       }
446     }
447   }
448 
449   private void splitStoreFile(final StoreFile sf, final Path splitdir)
450   throws IOException {
451     FileSystem fs = this.parent.getFilesystem();
452     byte [] family = sf.getFamily();
453     String encoded = this.hri_a.getEncodedName();
454     Path storedir = Store.getStoreHomedir(splitdir, encoded, family);
455     StoreFile.split(fs, storedir, sf, this.splitrow, Range.bottom);
456     encoded = this.hri_b.getEncodedName();
457     storedir = Store.getStoreHomedir(splitdir, encoded, family);
458     StoreFile.split(fs, storedir, sf, this.splitrow, Range.top);
459   }
460 
461   /**
462    * Utility class used to do the file splitting / reference writing
463    * in parallel instead of sequentially.
464    */
465   class StoreFileSplitter implements Callable<Void> {
466 
467     private final StoreFile sf;
468     private final Path splitdir;
469 
470     /**
471      * Constructor that takes what it needs to split
472      * @param sf which file
473      * @param splitdir where the splitting is done
474      */
475     public StoreFileSplitter(final StoreFile sf, final Path splitdir) {
476       this.sf = sf;
477       this.splitdir = splitdir;
478     }
479 
480     public Void call() throws IOException {
481       splitStoreFile(sf, splitdir);
482       return null;
483     }
484   }
485 
486   /**
487    * @param hri Spec. for daughter region to open.
488    * @param flusher Flusher this region should use.
489    * @return Created daughter HRegion.
490    * @throws IOException
491    * @see #cleanupDaughterRegion(FileSystem, Path, HRegionInfo)
492    */
493   HRegion createDaughterRegion(final HRegionInfo hri,
494       final FlushRequester flusher)
495   throws IOException {
496     // Package private so unit tests have access.
497     FileSystem fs = this.parent.getFilesystem();
498     Path regionDir = getSplitDirForDaughter(this.parent.getFilesystem(),
499       this.splitdir, hri);
500     HRegion r = HRegion.newHRegion(this.parent.getTableDir(),
501       this.parent.getLog(), fs, this.parent.getConf(),
502       hri, flusher);
503     HRegion.moveInitialFilesIntoPlace(fs, regionDir, r.getRegionDir());
504     return r;
505   }
506 
507   private static void cleanupDaughterRegion(final FileSystem fs,
508     final Path tabledir, final String encodedName)
509   throws IOException {
510     Path regiondir = HRegion.getRegionDir(tabledir, encodedName);
511     // Dir may not preexist.
512     deleteDir(fs, regiondir, false);
513   }
514 
515   /*
516    * Get the daughter directories in the splits dir.  The splits dir is under
517    * the parent regions' directory.
518    * @param fs
519    * @param splitdir
520    * @param hri
521    * @return Path to daughter split dir.
522    * @throws IOException
523    */
524   private static Path getSplitDirForDaughter(final FileSystem fs,
525       final Path splitdir, final HRegionInfo hri)
526   throws IOException {
527     return new Path(splitdir, hri.getEncodedName());
528   }
529 
530   /**
531    * @param or Object that can online/offline parent region.  Can be passed null
532    * by unit tests.
533    * @return The region we were splitting
534    * @throws IOException If thrown, rollback failed.  Take drastic action.
535    */
536   public void rollback(final OnlineRegions or) throws IOException {
537     FileSystem fs = this.parent.getFilesystem();
538     ListIterator<JournalEntry> iterator =
539       this.journal.listIterator(this.journal.size());
540     while (iterator.hasPrevious()) {
541       JournalEntry je = iterator.previous();
542       switch(je) {
543       case CREATE_SPLIT_DIR:
544     	this.parent.writestate.writesEnabled = true;
545         cleanupSplitDir(fs, this.splitdir);
546         break;
547 
548       case CLOSED_PARENT_REGION:
549         // So, this returns a seqid but if we just closed and then reopened, we
550         // should be ok. On close, we flushed using sequenceid obtained from
551         // hosting regionserver so no need to propagate the sequenceid returned
552         // out of initialize below up into regionserver as we normally do.
553         // TODO: Verify.
554         this.parent.initialize();
555         break;
556 
557       case STARTED_REGION_A_CREATION:
558         cleanupDaughterRegion(fs, this.parent.getTableDir(),
559           this.hri_a.getEncodedName());
560         break;
561 
562       case STARTED_REGION_B_CREATION:
563         cleanupDaughterRegion(fs, this.parent.getTableDir(),
564           this.hri_b.getEncodedName());
565         break;
566 
567       case OFFLINED_PARENT:
568         if (or != null) or.addToOnlineRegions(this.parent);
569         break;
570 
571       default:
572         throw new RuntimeException("Unhandled journal entry: " + je);
573       }
574     }
575   }
576 
577   HRegionInfo getFirstDaughter() {
578     return hri_a;
579   }
580 
581   HRegionInfo getSecondDaughter() {
582     return hri_b;
583   }
584 
585   // For unit testing.
586   Path getSplitDir() {
587     return this.splitdir;
588   }
589 
590   /**
591    * Clean up any split detritus that may have been left around from previous
592    * split attempts.
593    * Call this method on initial region deploy.  Cleans up any mess
594    * left by previous deploys of passed <code>r</code> region.
595    * @param r
596    * @throws IOException
597    */
598   static void cleanupAnySplitDetritus(final HRegion r) throws IOException {
599     Path splitdir = getSplitDir(r);
600     FileSystem fs = r.getFilesystem();
601     if (!fs.exists(splitdir)) return;
602     // Look at the splitdir.  It could have the encoded names of the daughter
603     // regions we tried to make.  See if the daughter regions actually got made
604     // out under the tabledir.  If here under splitdir still, then the split did
605     // not complete.  Try and do cleanup.  This code WILL NOT catch the case
606     // where we successfully created daughter a but regionserver crashed during
607     // the creation of region b.  In this case, there'll be an orphan daughter
608     // dir in the filesystem.  TOOD: Fix.
609     FileStatus [] daughters = fs.listStatus(splitdir, new FSUtils.DirFilter(fs));
610     for (int i = 0; i < daughters.length; i++) {
611       cleanupDaughterRegion(fs, r.getTableDir(),
612         daughters[i].getPath().getName());
613     }
614     cleanupSplitDir(r.getFilesystem(), splitdir);
615     LOG.info("Cleaned up old failed split transaction detritus: " + splitdir);
616   }
617 }