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  
27  import org.apache.commons.logging.Log;
28  import org.apache.commons.logging.LogFactory;
29  import org.apache.hadoop.conf.Configuration;
30  import org.apache.hadoop.fs.FileStatus;
31  import org.apache.hadoop.fs.FileSystem;
32  import org.apache.hadoop.fs.Path;
33  import org.apache.hadoop.hbase.HConstants;
34  import org.apache.hadoop.hbase.HRegionInfo;
35  import org.apache.hadoop.hbase.client.HTable;
36  import org.apache.hadoop.hbase.client.Put;
37  import org.apache.hadoop.hbase.io.Reference.Range;
38  import org.apache.hadoop.hbase.util.Bytes;
39  import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
40  import org.apache.hadoop.hbase.util.FSUtils;
41  import org.apache.hadoop.hbase.util.PairOfSameType;
42  import org.apache.hadoop.hbase.util.Writables;
43  
44  /**
45   * Executes region split as a "transaction".  Call {@link #prepare()} to setup
46   * the transaction, {@link #execute(OnlineRegions)} to run the transaction and
47   * {@link #rollback(OnlineRegions)} to cleanup if execute fails.
48   *
49   * <p>Here is an example of how you would use this class:
50   * <pre>
51   *  SplitTransaction st = new SplitTransaction(this.conf, parent, midKey)
52   *  if (!st.prepare()) return;
53   *  try {
54   *    st.execute(myOnlineRegions);
55   *  } catch (IOException ioe) {
56   *    try {
57   *      st.rollback(myOnlineRegions);
58   *      return;
59   *    } catch (RuntimeException e) {
60   *      myAbortable.abort("Failed split, abort");
61   *    }
62   *  }
63   * </Pre>
64   * <p>This class is not thread safe.  Caller needs ensure split is run by
65   * one thread only.
66   */
67  class SplitTransaction {
68    private static final Log LOG = LogFactory.getLog(SplitTransaction.class);
69    private static final String SPLITDIR = "splits";
70  
71    /*
72     * Region to split
73     */
74    private final HRegion parent;
75    private HRegionInfo hri_a;
76    private HRegionInfo hri_b;
77    private Path splitdir;
78  
79    /*
80     * Row to split around
81     */
82    private final byte [] splitrow;
83  
84    /**
85     * Types to add to the transaction journal
86     */
87    enum JournalEntry {
88      /**
89       * We created the temporary split data directory.
90       */
91      CREATE_SPLIT_DIR,
92      /**
93       * Closed the parent region.
94       */
95      CLOSED_PARENT_REGION,
96      /**
97       * The parent has been taken out of the server's online regions list.
98       */
99      OFFLINED_PARENT,
100     /**
101      * Started in on creation of the first daughter region.
102      */
103     STARTED_REGION_A_CREATION,
104     /**
105      * Started in on the creation of the second daughter region.
106      */
107     STARTED_REGION_B_CREATION
108   }
109 
110   /*
111    * Journal of how far the split transaction has progressed.
112    */
113   private final List<JournalEntry> journal = new ArrayList<JournalEntry>();
114 
115   /**
116    * Constructor
117    * @param c Configuration to use running split
118    * @param r Region to split
119    * @param splitrow Row to split around
120    */
121   SplitTransaction(final HRegion r, final byte [] splitrow) {
122     this.parent = r;
123     this.splitrow = splitrow;
124     this.splitdir = getSplitDir(this.parent);
125   }
126 
127   /**
128    * Does checks on split inputs.
129    * @return <code>true</code> if the region is splittable else
130    * <code>false</code> if it is not (e.g. its already closed, etc.).
131    */
132   public boolean prepare() {
133     if (this.parent.isClosed() || this.parent.isClosing()) return false;
134     HRegionInfo hri = this.parent.getRegionInfo();
135     // Check splitrow.
136     byte [] startKey = hri.getStartKey();
137     byte [] endKey = hri.getEndKey();
138     if (Bytes.equals(startKey, splitrow) ||
139         !this.parent.getRegionInfo().containsRow(splitrow)) {
140       LOG.info("Split row is not inside region key range or is equal to " +
141           "startkey: " + Bytes.toString(this.splitrow));
142       return false;
143     }
144     long rid = getDaughterRegionIdTimestamp(hri);
145     this.hri_a = new HRegionInfo(hri.getTableDesc(), startKey, this.splitrow,
146       false, rid);
147     this.hri_b = new HRegionInfo(hri.getTableDesc(), this.splitrow, endKey,
148       false, rid);
149     return true;
150   }
151 
152   /**
153    * Calculate daughter regionid to use.
154    * @param hri Parent {@link HRegionInfo}
155    * @return Daughter region id (timestamp) to use.
156    */
157   private static long getDaughterRegionIdTimestamp(final HRegionInfo hri) {
158     long rid = EnvironmentEdgeManager.currentTimeMillis();
159     // Regionid is timestamp.  Can't be less than that of parent else will insert
160     // at wrong location in .META. (See HBASE-710).
161     if (rid < hri.getRegionId()) {
162       LOG.warn("Clock skew; parent regions id is " + hri.getRegionId() +
163         " but current time here is " + rid);
164       rid = hri.getRegionId() + 1;
165     }
166     return rid;
167   }
168 
169   /**
170    * Run the transaction.
171    * @param or Object that can online/offline parent region.
172    * @throws IOException If thrown, transaction failed. Call {@link #rollback(OnlineRegions)}
173    * @return Regions created
174    * @see #rollback(OnlineRegions)
175    */
176   public PairOfSameType<HRegion> execute(final OnlineRegions or) throws IOException {
177     return execute(or, or != null);
178   }
179 
180   /**
181    * Run the transaction.
182    * @param or Object that can online/offline parent region.  Can be null (Tests
183    * will pass null).
184    * @param If <code>true</code>, update meta (set to false when testing).
185    * @throws IOException If thrown, transaction failed. Call {@link #rollback(OnlineRegions)}
186    * @return Regions created
187    * @see #rollback(OnlineRegions)
188    */
189   PairOfSameType<HRegion> execute(final OnlineRegions or, final boolean updateMeta)
190   throws IOException {
191     LOG.info("Starting split of region " + this.parent);
192     assert !this.parent.lock.writeLock().isHeldByCurrentThread() : "Unsafe to hold write lock while performing RPCs";
193 
194     // We'll need one of these later but get it now because if we fail there
195     // is nothing to undo.
196     HTable t = null;
197     if (updateMeta) t = getTable(this.parent.getConf());
198 
199     createSplitDir(this.parent.getFilesystem(), this.splitdir);
200     this.journal.add(JournalEntry.CREATE_SPLIT_DIR);
201 
202     List<StoreFile> hstoreFilesToSplit = this.parent.close(false);
203     this.journal.add(JournalEntry.CLOSED_PARENT_REGION);
204 
205     if (or != null) or.removeFromOnlineRegions(this.parent.getRegionInfo());
206     this.journal.add(JournalEntry.OFFLINED_PARENT);
207 
208     splitStoreFiles(this.splitdir, hstoreFilesToSplit);
209     // splitStoreFiles creates daughter region dirs under the parent splits dir
210     // Nothing to unroll here if failure -- clean up of CREATE_SPLIT_DIR will
211     // clean this up.
212 
213     // Log to the journal that we are creating region A, the first daughter
214     // region.  We could fail halfway through.  If we do, we could have left
215     // stuff in fs that needs cleanup -- a storefile or two.  Thats why we
216     // add entry to journal BEFORE rather than AFTER the change.
217     this.journal.add(JournalEntry.STARTED_REGION_A_CREATION);
218     HRegion a = createDaughterRegion(this.hri_a);
219 
220     // Ditto
221     this.journal.add(JournalEntry.STARTED_REGION_B_CREATION);
222     HRegion b = createDaughterRegion(this.hri_b);
223 
224     Put editParentPut = createOfflineParentPut();
225     if (t != null) t.put(editParentPut);
226 
227     // The is the point of no return.  We are committed to the split now.  Up to
228     // a failure editing parent in meta or a crash of the hosting regionserver,
229     // we could rollback (or, if crash, we could cleanup on redeploy) but now
230     // meta has been changed, we can only go forward.  If the below last steps
231     // do not complete, repair has to be done by another agent.  For example,
232     // basescanner, at least up till master rewrite, would add daughter rows if
233     // missing from meta.  It could do this because the parent edit includes the
234     // daughter specs.  In Bigtable paper, they have another mechanism where
235     // some feedback to the master somehow flags it that split is incomplete and
236     // needs fixup.  Whatever the mechanism, its a TODO that we have some fixup.
237     
238     // I looked at writing the put of the parent edit above out to the WAL log
239     // before changing meta with the notion that should we fail, then on replay
240     // the offlining of the parent and addition of daughters up into meta could
241     // be reinserted.  The edits would have to be 'special' and given how our
242     // splits work, splitting by region, I think the replay would have to happen
243     // inside in the split code -- as soon as it saw one of these special edits,
244     // rather than write the edit out a file for the .META. region to replay or
245     // somehow, write it out to this regions edits file for it to handle on
246     // redeploy -- this'd be whacky, we'd be telling meta about a split during
247     // the deploy of the parent -- instead we'd have to play the edit inside
248     // in the split code somehow; this would involve a stop-the-splitting till
249     // meta had been edited which might hold up splitting a good while.
250 
251     // Finish up the meta edits.  If these fail, another agent needs to do fixup
252     HRegionInfo hri = this.hri_a;
253     try {
254       if (t != null) t.put(createDaughterPut(hri));
255       hri = this.hri_b;
256       if (t != null) t.put(createDaughterPut(hri));
257     } catch (IOException e) {
258       // Don't let this out or we'll run rollback.
259       LOG.warn("Failed adding daughter " + hri.toString());
260     }
261     // This should not fail because the HTable instance we are using is not
262     // running a buffer -- its immediately flushing its puts.
263     if (t != null) t.close();
264 
265     // Leaving here, the splitdir with its dross will be in place but since the
266     // split was successful, just leave it; it'll be cleaned when parent is
267     // deleted and cleaned up.
268     return new PairOfSameType<HRegion>(a, b);
269   }
270 
271   private static Path getSplitDir(final HRegion r) {
272     return new Path(r.getRegionDir(), SPLITDIR);
273   }
274 
275   /**
276    * @param fs Filesystem to use
277    * @param splitdir Directory to store temporary split data in
278    * @throws IOException If <code>splitdir</code> already exists or we fail
279    * to create it.
280    * @see #cleanupSplitDir(FileSystem, Path)
281    */
282   private static void createSplitDir(final FileSystem fs, final Path splitdir)
283   throws IOException {
284     if (fs.exists(splitdir)) throw new IOException("Splitdir already exits? " + splitdir);
285     if (!fs.mkdirs(splitdir)) throw new IOException("Failed create of " + splitdir);
286   }
287 
288   private static void cleanupSplitDir(final FileSystem fs, final Path splitdir)
289   throws IOException {
290     // Splitdir may have been cleaned up by reopen of the parent dir.
291     deleteDir(fs, splitdir, false);
292   }
293 
294   /**
295    * @param fs Filesystem to use
296    * @param dir Directory to delete
297    * @param mustPreExist If true, we'll throw exception if <code>dir</code>
298    * does not preexist, else we'll just pass.
299    * @throws IOException Thrown if we fail to delete passed <code>dir</code>
300    */
301   private static void deleteDir(final FileSystem fs, final Path dir,
302       final boolean mustPreExist)
303   throws IOException {
304     if (!fs.exists(dir)) {
305       if (mustPreExist) throw new IOException(dir.toString() + " does not exist!");
306     } else if (!fs.delete(dir, true)) {
307       throw new IOException("Failed delete of " + dir);
308     }
309   }
310 
311   private void splitStoreFiles(final Path splitdir,
312     final List<StoreFile> hstoreFilesToSplit)
313   throws IOException {
314     if (hstoreFilesToSplit == null) {
315       // Could be null because close didn't succeed -- for now consider it fatal
316       throw new IOException("Close returned empty list of StoreFiles");
317     }
318 
319      // Split each store file.
320      for (StoreFile sf: hstoreFilesToSplit) {
321        splitStoreFile(sf, splitdir);
322      }
323   }
324 
325   private void splitStoreFile(final StoreFile sf, final Path splitdir)
326   throws IOException {
327     FileSystem fs = this.parent.getFilesystem();
328     byte [] family = sf.getFamily();
329     String encoded = this.hri_a.getEncodedName();
330     Path storedir = Store.getStoreHomedir(splitdir, encoded, family);
331     StoreFile.split(fs, storedir, sf, this.splitrow, Range.bottom);
332     encoded = this.hri_b.getEncodedName();
333     storedir = Store.getStoreHomedir(splitdir, encoded, family);
334     StoreFile.split(fs, storedir, sf, this.splitrow, Range.top);
335   }
336 
337   /**
338    * @param hri
339    * @return Created daughter HRegion.
340    * @throws IOException
341    * @see #cleanupDaughterRegion(FileSystem, Path, HRegionInfo)
342    */
343   HRegion createDaughterRegion(final HRegionInfo hri)
344   throws IOException {
345     // Package private so unit tests have access.
346     FileSystem fs = this.parent.getFilesystem();
347     Path regionDir = getSplitDirForDaughter(this.parent.getFilesystem(),
348       this.splitdir, hri);
349     HRegion r = HRegion.newHRegion(this.parent.getTableDir(),
350       this.parent.getLog(), fs, this.parent.getConf(),
351       hri, null);
352     HRegion.moveInitialFilesIntoPlace(fs, regionDir, r.getRegionDir());
353     return r;
354   }
355 
356   private static void cleanupDaughterRegion(final FileSystem fs,
357     final Path tabledir, final String encodedName)
358   throws IOException {
359     Path regiondir = HRegion.getRegionDir(tabledir, encodedName);
360     // Dir may not preexist.
361     deleteDir(fs, regiondir, false);
362   }
363 
364   /*
365    * Get the daughter directories in the splits dir.  The splits dir is under
366    * the parent regions' directory.
367    * @param fs
368    * @param splitdir
369    * @param hri
370    * @return Path to daughter split dir.
371    * @throws IOException
372    */
373   private static Path getSplitDirForDaughter(final FileSystem fs,
374       final Path splitdir, final HRegionInfo hri)
375   throws IOException {
376     return new Path(splitdir, hri.getEncodedName());
377   }
378 
379   /*
380    * @param r Parent region we want to edit.
381    * @return An HTable instance against the meta table that holds passed
382    * <code>r</code>; it has autoFlush enabled so we immediately send puts (No
383    * buffering enabled).
384    * @throws IOException
385    */
386   private HTable getTable(final Configuration conf) throws IOException {
387     // When a region is split, the META table needs to updated if we're
388     // splitting a 'normal' region, and the ROOT table needs to be
389     // updated if we are splitting a META region.
390     HTable t = null;
391     if (this.parent.getRegionInfo().isMetaTable()) {
392       t = new HTable(conf, HConstants.ROOT_TABLE_NAME);
393     } else {
394       t = new HTable(conf, HConstants.META_TABLE_NAME);
395     }
396     // Flush puts as we send them -- no buffering.
397     t.setAutoFlush(true);
398     return t;
399   }
400 
401 
402   private Put createOfflineParentPut() throws IOException  {
403     HRegionInfo editedParentRegionInfo =
404       new HRegionInfo(this.parent.getRegionInfo());
405     editedParentRegionInfo.setOffline(true);
406     editedParentRegionInfo.setSplit(true);
407     Put put = new Put(editedParentRegionInfo.getRegionName());
408     put.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
409       Writables.getBytes(editedParentRegionInfo));
410     put.add(HConstants.CATALOG_FAMILY, HConstants.SERVER_QUALIFIER,
411         HConstants.EMPTY_BYTE_ARRAY);
412     put.add(HConstants.CATALOG_FAMILY, HConstants.STARTCODE_QUALIFIER,
413         HConstants.EMPTY_BYTE_ARRAY);
414     put.add(HConstants.CATALOG_FAMILY, HConstants.SPLITA_QUALIFIER,
415       Writables.getBytes(this.hri_a));
416     put.add(HConstants.CATALOG_FAMILY, HConstants.SPLITB_QUALIFIER,
417       Writables.getBytes(this.hri_b));
418     return put;
419   }
420 
421   private Put createDaughterPut(final HRegionInfo daughter)
422   throws IOException {
423     Put p = new Put(daughter.getRegionName());
424     p.add(HConstants.CATALOG_FAMILY, HConstants.REGIONINFO_QUALIFIER,
425       Writables.getBytes(daughter));
426     return p;
427   }
428 
429   /**
430    * @param or Object that can online/offline parent region.  Can be passed null
431    * by unit tests.
432    * @return The region we were splitting
433    * @throws IOException If thrown, rollback failed.  Take drastic action.
434    */
435   public void rollback(final OnlineRegions or) throws IOException {
436     FileSystem fs = this.parent.getFilesystem();
437     ListIterator<JournalEntry> iterator =
438       this.journal.listIterator(this.journal.size());
439     while (iterator.hasPrevious()) {
440       JournalEntry je = iterator.previous();
441       switch(je) {
442       case CREATE_SPLIT_DIR:
443         cleanupSplitDir(fs, this.splitdir);
444         break;
445 
446       case CLOSED_PARENT_REGION:
447         // So, this returns a seqid but if we just closed and then reopened, we
448         // should be ok. On close, we flushed using sequenceid obtained from
449         // hosting regionserver so no need to propagate the sequenceid returned
450         // out of initialize below up into regionserver as we normally do.
451         // TODO: Verify.
452         this.parent.initialize();
453         break;
454 
455       case STARTED_REGION_A_CREATION:
456         cleanupDaughterRegion(fs, this.parent.getTableDir(),
457           this.hri_a.getEncodedName());
458         break;
459 
460       case STARTED_REGION_B_CREATION:
461         cleanupDaughterRegion(fs, this.parent.getTableDir(),
462           this.hri_b.getEncodedName());
463         break;
464 
465       case OFFLINED_PARENT:
466         if (or != null) or.addToOnlineRegions(this.parent);
467         break;
468 
469       default:
470         throw new RuntimeException("Unhandled journal entry: " + je);
471       }
472     }
473   }
474 
475   HRegionInfo getFirstDaughter() {
476     return hri_a;
477   }
478 
479   HRegionInfo getSecondDaughter() {
480     return hri_b;
481   }
482 
483   // For unit testing.
484   Path getSplitDir() {
485     return this.splitdir;
486   }
487 
488   /**
489    * Clean up any split detritus that may have been left around from previous
490    * split attempts.
491    * Call this method on initial region deploy.  Cleans up any mess
492    * left by previous deploys of passed <code>r</code> region.
493    * @param r
494    * @throws IOException 
495    */
496   static void cleanupAnySplitDetritus(final HRegion r) throws IOException {
497     Path splitdir = getSplitDir(r);
498     FileSystem fs = r.getFilesystem();
499     if (!fs.exists(splitdir)) return;
500     // Look at the splitdir.  It could have the encoded names of the daughter
501     // regions we tried to make.  See if the daughter regions actually got made
502     // out under the tabledir.  If here under splitdir still, then the split did
503     // not complete.  Try and do cleanup.  This code WILL NOT catch the case
504     // where we successfully created daughter a but regionserver crashed during
505     // the creation of region b.  In this case, there'll be an orphan daughter
506     // dir in the filesystem.  TOOD: Fix.
507     FileStatus [] daughters = fs.listStatus(splitdir, new FSUtils.DirFilter(fs));
508     for (int i = 0; i < daughters.length; i++) {
509       cleanupDaughterRegion(fs, r.getTableDir(),
510         daughters[i].getPath().getName());
511     }
512     cleanupSplitDir(r.getFilesystem(), splitdir);
513     LOG.info("Cleaned up old failed split transaction detritus: " + splitdir);
514   }
515 }