View Javadoc

1   /**
2    *
3    * Licensed to the Apache Software Foundation (ASF) under one
4    * or more contributor license agreements.  See the NOTICE file
5    * distributed with this work for additional information
6    * regarding copyright ownership.  The ASF licenses this file
7    * to you under the Apache License, Version 2.0 (the
8    * "License"); you may not use this file except in compliance
9    * with the License.  You may obtain a copy of the License at
10   *
11   *     http://www.apache.org/licenses/LICENSE-2.0
12   *
13   * Unless required by applicable law or agreed to in writing, software
14   * distributed under the License is distributed on an "AS IS" BASIS,
15   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
16   * See the License for the specific language governing permissions and
17   * limitations under the License.
18   */
19  
20  package org.apache.hadoop.hbase.regionserver;
21  
22  import java.io.FileNotFoundException;
23  import java.io.IOException;
24  import java.io.InterruptedIOException;
25  import java.util.ArrayList;
26  import java.util.Collection;
27  import java.util.List;
28  import java.util.Map;
29  import java.util.UUID;
30  
31  import org.apache.commons.logging.Log;
32  import org.apache.commons.logging.LogFactory;
33  import org.apache.hadoop.hbase.classification.InterfaceAudience;
34  import org.apache.hadoop.conf.Configuration;
35  import org.apache.hadoop.fs.FSDataInputStream;
36  import org.apache.hadoop.fs.FSDataOutputStream;
37  import org.apache.hadoop.fs.FileStatus;
38  import org.apache.hadoop.fs.FileSystem;
39  import org.apache.hadoop.fs.FileUtil;
40  import org.apache.hadoop.fs.Path;
41  import org.apache.hadoop.fs.permission.FsPermission;
42  import org.apache.hadoop.hbase.Cell;
43  import org.apache.hadoop.hbase.HColumnDescriptor;
44  import org.apache.hadoop.hbase.HConstants;
45  import org.apache.hadoop.hbase.HRegionInfo;
46  import org.apache.hadoop.hbase.HTableDescriptor;
47  import org.apache.hadoop.hbase.KeyValue;
48  import org.apache.hadoop.hbase.KeyValueUtil;
49  import org.apache.hadoop.hbase.backup.HFileArchiver;
50  import org.apache.hadoop.hbase.fs.HFileSystem;
51  import org.apache.hadoop.hbase.io.Reference;
52  import org.apache.hadoop.hbase.util.Bytes;
53  import org.apache.hadoop.hbase.util.FSHDFSUtils;
54  import org.apache.hadoop.hbase.util.FSUtils;
55  import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
56  
57  /**
58   * View to an on-disk Region.
59   * Provides the set of methods necessary to interact with the on-disk region data.
60   */
61  @InterfaceAudience.Private
62  public class HRegionFileSystem {
63    private static final Log LOG = LogFactory.getLog(HRegionFileSystem.class);
64  
65    /** Name of the region info file that resides just under the region directory. */
66    public final static String REGION_INFO_FILE = ".regioninfo";
67  
68    /** Temporary subdirectory of the region directory used for merges. */
69    public static final String REGION_MERGES_DIR = ".merges";
70  
71    /** Temporary subdirectory of the region directory used for splits. */
72    public static final String REGION_SPLITS_DIR = ".splits";
73  
74    /** Temporary subdirectory of the region directory used for compaction output. */
75    private static final String REGION_TEMP_DIR = ".tmp";
76  
77    private final HRegionInfo regionInfo;
78    //regionInfo for interacting with FS (getting encodedName, etc)
79    private final HRegionInfo regionInfoForFs;
80    private final Configuration conf;
81    private final Path tableDir;
82    private final FileSystem fs;
83  
84    /**
85     * In order to handle NN connectivity hiccups, one need to retry non-idempotent operation at the
86     * client level.
87     */
88    private final int hdfsClientRetriesNumber;
89    private final int baseSleepBeforeRetries;
90    private static final int DEFAULT_HDFS_CLIENT_RETRIES_NUMBER = 10;
91    private static final int DEFAULT_BASE_SLEEP_BEFORE_RETRIES = 1000;
92  
93    /**
94     * Create a view to the on-disk region
95     * @param conf the {@link Configuration} to use
96     * @param fs {@link FileSystem} that contains the region
97     * @param tableDir {@link Path} to where the table is being stored
98     * @param regionInfo {@link HRegionInfo} for region
99     */
100   HRegionFileSystem(final Configuration conf, final FileSystem fs, final Path tableDir,
101       final HRegionInfo regionInfo) {
102     this.fs = fs;
103     this.conf = conf;
104     this.tableDir = tableDir;
105     this.regionInfo = regionInfo;
106     this.regionInfoForFs = ServerRegionReplicaUtil.getRegionInfoForFs(regionInfo);
107     this.hdfsClientRetriesNumber = conf.getInt("hdfs.client.retries.number",
108       DEFAULT_HDFS_CLIENT_RETRIES_NUMBER);
109     this.baseSleepBeforeRetries = conf.getInt("hdfs.client.sleep.before.retries",
110       DEFAULT_BASE_SLEEP_BEFORE_RETRIES);
111  }
112 
113   /** @return the underlying {@link FileSystem} */
114   public FileSystem getFileSystem() {
115     return this.fs;
116   }
117 
118   /** @return the {@link HRegionInfo} that describe this on-disk region view */
119   public HRegionInfo getRegionInfo() {
120     return this.regionInfo;
121   }
122 
123   public HRegionInfo getRegionInfoForFS() {
124     return this.regionInfoForFs;
125   }
126 
127   /** @return {@link Path} to the region's root directory. */
128   public Path getTableDir() {
129     return this.tableDir;
130   }
131 
132   /** @return {@link Path} to the region directory. */
133   public Path getRegionDir() {
134     return new Path(this.tableDir, this.regionInfoForFs.getEncodedName());
135   }
136 
137   // ===========================================================================
138   //  Temp Helpers
139   // ===========================================================================
140   /** @return {@link Path} to the region's temp directory, used for file creations */
141   Path getTempDir() {
142     return new Path(getRegionDir(), REGION_TEMP_DIR);
143   }
144 
145   /**
146    * Clean up any temp detritus that may have been left around from previous operation attempts.
147    */
148   void cleanupTempDir() throws IOException {
149     deleteDir(getTempDir());
150   }
151 
152   // ===========================================================================
153   //  Store/StoreFile Helpers
154   // ===========================================================================
155   /**
156    * Returns the directory path of the specified family
157    * @param familyName Column Family Name
158    * @return {@link Path} to the directory of the specified family
159    */
160   public Path getStoreDir(final String familyName) {
161     return new Path(this.getRegionDir(), familyName);
162   }
163 
164   /**
165    * Create the store directory for the specified family name
166    * @param familyName Column Family Name
167    * @return {@link Path} to the directory of the specified family
168    * @throws IOException if the directory creation fails.
169    */
170   Path createStoreDir(final String familyName) throws IOException {
171     Path storeDir = getStoreDir(familyName);
172     if(!fs.exists(storeDir) && !createDir(storeDir))
173       throw new IOException("Failed creating "+storeDir);
174     return storeDir;
175   }
176 
177   /**
178    * Returns the store files available for the family.
179    * This methods performs the filtering based on the valid store files.
180    * @param familyName Column Family Name
181    * @return a set of {@link StoreFileInfo} for the specified family.
182    */
183   public Collection<StoreFileInfo> getStoreFiles(final byte[] familyName) throws IOException {
184     return getStoreFiles(Bytes.toString(familyName));
185   }
186 
187   public Collection<StoreFileInfo> getStoreFiles(final String familyName) throws IOException {
188     return getStoreFiles(familyName, true);
189   }
190 
191   /**
192    * Returns the store files available for the family.
193    * This methods performs the filtering based on the valid store files.
194    * @param familyName Column Family Name
195    * @return a set of {@link StoreFileInfo} for the specified family.
196    */
197   public Collection<StoreFileInfo> getStoreFiles(final String familyName, final boolean validate)
198       throws IOException {
199     Path familyDir = getStoreDir(familyName);
200     FileStatus[] files = FSUtils.listStatus(this.fs, familyDir);
201     if (files == null) {
202       LOG.debug("No StoreFiles for: " + familyDir);
203       return null;
204     }
205 
206     ArrayList<StoreFileInfo> storeFiles = new ArrayList<StoreFileInfo>(files.length);
207     for (FileStatus status: files) {
208       if (validate && !StoreFileInfo.isValid(status)) {
209         LOG.warn("Invalid StoreFile: " + status.getPath());
210         continue;
211       }
212       StoreFileInfo info = ServerRegionReplicaUtil.getStoreFileInfo(conf, fs, regionInfo,
213         regionInfoForFs, familyName, status.getPath());
214       storeFiles.add(info);
215 
216     }
217     return storeFiles;
218   }
219 
220   /**
221    * Return Qualified Path of the specified family/file
222    *
223    * @param familyName Column Family Name
224    * @param fileName File Name
225    * @return The qualified Path for the specified family/file
226    */
227   Path getStoreFilePath(final String familyName, final String fileName) {
228     Path familyDir = getStoreDir(familyName);
229     return new Path(familyDir, fileName).makeQualified(this.fs);
230   }
231 
232   /**
233    * Return the store file information of the specified family/file.
234    *
235    * @param familyName Column Family Name
236    * @param fileName File Name
237    * @return The {@link StoreFileInfo} for the specified family/file
238    */
239   StoreFileInfo getStoreFileInfo(final String familyName, final String fileName)
240       throws IOException {
241     Path familyDir = getStoreDir(familyName);
242     return ServerRegionReplicaUtil.getStoreFileInfo(conf, fs, regionInfo,
243       regionInfoForFs, familyName, new Path(familyDir, fileName));
244   }
245 
246   /**
247    * Returns true if the specified family has reference files
248    * @param familyName Column Family Name
249    * @return true if family contains reference files
250    * @throws IOException
251    */
252   public boolean hasReferences(final String familyName) throws IOException {
253     FileStatus[] files = FSUtils.listStatus(fs, getStoreDir(familyName),
254         new FSUtils.ReferenceFileFilter(fs));
255     return files != null && files.length > 0;
256   }
257 
258   /**
259    * Check whether region has Reference file
260    * @param htd table desciptor of the region
261    * @return true if region has reference file
262    * @throws IOException
263    */
264   public boolean hasReferences(final HTableDescriptor htd) throws IOException {
265     for (HColumnDescriptor family : htd.getFamilies()) {
266       if (hasReferences(family.getNameAsString())) {
267         return true;
268       }
269     }
270     return false;
271   }
272 
273   /**
274    * @return the set of families present on disk
275    * @throws IOException
276    */
277   public Collection<String> getFamilies() throws IOException {
278     FileStatus[] fds = FSUtils.listStatus(fs, getRegionDir(), new FSUtils.FamilyDirFilter(fs));
279     if (fds == null) return null;
280 
281     ArrayList<String> families = new ArrayList<String>(fds.length);
282     for (FileStatus status: fds) {
283       families.add(status.getPath().getName());
284     }
285 
286     return families;
287   }
288 
289   /**
290    * Remove the region family from disk, archiving the store files.
291    * @param familyName Column Family Name
292    * @throws IOException if an error occours during the archiving
293    */
294   public void deleteFamily(final String familyName) throws IOException {
295     // archive family store files
296     HFileArchiver.archiveFamily(fs, conf, regionInfoForFs, tableDir, Bytes.toBytes(familyName));
297 
298     // delete the family folder
299     Path familyDir = getStoreDir(familyName);
300     if(fs.exists(familyDir) && !deleteDir(familyDir))
301       throw new IOException("Could not delete family " + familyName
302           + " from FileSystem for region " + regionInfoForFs.getRegionNameAsString() + "("
303           + regionInfoForFs.getEncodedName() + ")");
304   }
305 
306   /**
307    * Generate a unique file name, used by createTempName() and commitStoreFile()
308    * @param suffix extra information to append to the generated name
309    * @return Unique file name
310    */
311   private static String generateUniqueName(final String suffix) {
312     String name = UUID.randomUUID().toString().replaceAll("-", "");
313     if (suffix != null) name += suffix;
314     return name;
315   }
316 
317   /**
318    * Generate a unique temporary Path. Used in conjuction with commitStoreFile()
319    * to get a safer file creation.
320    * <code>
321    * Path file = fs.createTempName();
322    * ...StoreFile.Writer(file)...
323    * fs.commitStoreFile("family", file);
324    * </code>
325    *
326    * @return Unique {@link Path} of the temporary file
327    */
328   public Path createTempName() {
329     return createTempName(null);
330   }
331 
332   /**
333    * Generate a unique temporary Path. Used in conjuction with commitStoreFile()
334    * to get a safer file creation.
335    * <code>
336    * Path file = fs.createTempName();
337    * ...StoreFile.Writer(file)...
338    * fs.commitStoreFile("family", file);
339    * </code>
340    *
341    * @param suffix extra information to append to the generated name
342    * @return Unique {@link Path} of the temporary file
343    */
344   public Path createTempName(final String suffix) {
345     return new Path(getTempDir(), generateUniqueName(suffix));
346   }
347 
348   /**
349    * Move the file from a build/temp location to the main family store directory.
350    * @param familyName Family that will gain the file
351    * @param buildPath {@link Path} to the file to commit.
352    * @return The new {@link Path} of the committed file
353    * @throws IOException
354    */
355   public Path commitStoreFile(final String familyName, final Path buildPath) throws IOException {
356     return commitStoreFile(familyName, buildPath, -1, false);
357   }
358 
359   /**
360    * Move the file from a build/temp location to the main family store directory.
361    * @param familyName Family that will gain the file
362    * @param buildPath {@link Path} to the file to commit.
363    * @param seqNum Sequence Number to append to the file name (less then 0 if no sequence number)
364    * @param generateNewName False if you want to keep the buildPath name
365    * @return The new {@link Path} of the committed file
366    * @throws IOException
367    */
368   private Path commitStoreFile(final String familyName, final Path buildPath,
369       final long seqNum, final boolean generateNewName) throws IOException {
370     Path storeDir = getStoreDir(familyName);
371     if(!fs.exists(storeDir) && !createDir(storeDir))
372       throw new IOException("Failed creating " + storeDir);
373 
374     String name = buildPath.getName();
375     if (generateNewName) {
376       name = generateUniqueName((seqNum < 0) ? null : "_SeqId_" + seqNum + "_");
377     }
378     Path dstPath = new Path(storeDir, name);
379     if (!fs.exists(buildPath)) {
380       throw new FileNotFoundException(buildPath.toString());
381     }
382     LOG.debug("Committing store file " + buildPath + " as " + dstPath);
383     // buildPath exists, therefore not doing an exists() check.
384     if (!rename(buildPath, dstPath)) {
385       throw new IOException("Failed rename of " + buildPath + " to " + dstPath);
386     }
387     return dstPath;
388   }
389 
390 
391   /**
392    * Moves multiple store files to the relative region's family store directory.
393    * @param storeFiles list of store files divided by family
394    * @throws IOException
395    */
396   void commitStoreFiles(final Map<byte[], List<StoreFile>> storeFiles) throws IOException {
397     for (Map.Entry<byte[], List<StoreFile>> es: storeFiles.entrySet()) {
398       String familyName = Bytes.toString(es.getKey());
399       for (StoreFile sf: es.getValue()) {
400         commitStoreFile(familyName, sf.getPath());
401       }
402     }
403   }
404 
405   /**
406    * Archives the specified store file from the specified family.
407    * @param familyName Family that contains the store files
408    * @param filePath {@link Path} to the store file to remove
409    * @throws IOException if the archiving fails
410    */
411   public void removeStoreFile(final String familyName, final Path filePath)
412       throws IOException {
413     HFileArchiver.archiveStoreFile(this.conf, this.fs, this.regionInfoForFs,
414         this.tableDir, Bytes.toBytes(familyName), filePath);
415   }
416 
417   /**
418    * Closes and archives the specified store files from the specified family.
419    * @param familyName Family that contains the store files
420    * @param storeFiles set of store files to remove
421    * @throws IOException if the archiving fails
422    */
423   public void removeStoreFiles(final String familyName, final Collection<StoreFile> storeFiles)
424       throws IOException {
425     HFileArchiver.archiveStoreFiles(this.conf, this.fs, this.regionInfoForFs,
426         this.tableDir, Bytes.toBytes(familyName), storeFiles);
427   }
428 
429   /**
430    * Bulk load: Add a specified store file to the specified family.
431    * If the source file is on the same different file-system is moved from the
432    * source location to the destination location, otherwise is copied over.
433    *
434    * @param familyName Family that will gain the file
435    * @param srcPath {@link Path} to the file to import
436    * @param seqNum Bulk Load sequence number
437    * @return The destination {@link Path} of the bulk loaded file
438    * @throws IOException
439    */
440   Path bulkLoadStoreFile(final String familyName, Path srcPath, long seqNum)
441       throws IOException {
442     // Copy the file if it's on another filesystem
443     FileSystem srcFs = srcPath.getFileSystem(conf);
444     FileSystem desFs = fs instanceof HFileSystem ? ((HFileSystem)fs).getBackingFs() : fs;
445 
446     // We can't compare FileSystem instances as equals() includes UGI instance
447     // as part of the comparison and won't work when doing SecureBulkLoad
448     // TODO deal with viewFS
449     if (!FSHDFSUtils.isSameHdfs(conf, srcFs, desFs)) {
450       LOG.info("Bulk-load file " + srcPath + " is on different filesystem than " +
451           "the destination store. Copying file over to destination filesystem.");
452       Path tmpPath = createTempName();
453       FileUtil.copy(srcFs, srcPath, fs, tmpPath, false, conf);
454       LOG.info("Copied " + srcPath + " to temporary path on destination filesystem: " + tmpPath);
455       srcPath = tmpPath;
456     }
457 
458     return commitStoreFile(familyName, srcPath, seqNum, true);
459   }
460 
461   // ===========================================================================
462   //  Splits Helpers
463   // ===========================================================================
464   /** @return {@link Path} to the temp directory used during split operations */
465   Path getSplitsDir() {
466     return new Path(getRegionDir(), REGION_SPLITS_DIR);
467   }
468 
469   Path getSplitsDir(final HRegionInfo hri) {
470     return new Path(getSplitsDir(), hri.getEncodedName());
471   }
472 
473   /**
474    * Clean up any split detritus that may have been left around from previous split attempts.
475    */
476   void cleanupSplitsDir() throws IOException {
477     deleteDir(getSplitsDir());
478   }
479 
480   /**
481    * Clean up any split detritus that may have been left around from previous
482    * split attempts.
483    * Call this method on initial region deploy.
484    * @throws IOException
485    */
486   void cleanupAnySplitDetritus() throws IOException {
487     Path splitdir = this.getSplitsDir();
488     if (!fs.exists(splitdir)) return;
489     // Look at the splitdir.  It could have the encoded names of the daughter
490     // regions we tried to make.  See if the daughter regions actually got made
491     // out under the tabledir.  If here under splitdir still, then the split did
492     // not complete.  Try and do cleanup.  This code WILL NOT catch the case
493     // where we successfully created daughter a but regionserver crashed during
494     // the creation of region b.  In this case, there'll be an orphan daughter
495     // dir in the filesystem.  TOOD: Fix.
496     FileStatus[] daughters = FSUtils.listStatus(fs, splitdir, new FSUtils.DirFilter(fs));
497     if (daughters != null) {
498       for (FileStatus daughter: daughters) {
499         Path daughterDir = new Path(getTableDir(), daughter.getPath().getName());
500         if (fs.exists(daughterDir) && !deleteDir(daughterDir)) {
501           throw new IOException("Failed delete of " + daughterDir);
502         }
503       }
504     }
505     cleanupSplitsDir();
506     LOG.info("Cleaned up old failed split transaction detritus: " + splitdir);
507   }
508 
509   /**
510    * Remove daughter region
511    * @param regionInfo daughter {@link HRegionInfo}
512    * @throws IOException
513    */
514   void cleanupDaughterRegion(final HRegionInfo regionInfo) throws IOException {
515     Path regionDir = new Path(this.tableDir, regionInfo.getEncodedName());
516     if (this.fs.exists(regionDir) && !deleteDir(regionDir)) {
517       throw new IOException("Failed delete of " + regionDir);
518     }
519   }
520 
521   /**
522    * Commit a daughter region, moving it from the split temporary directory
523    * to the proper location in the filesystem.
524    *
525    * @param regionInfo                 daughter {@link org.apache.hadoop.hbase.HRegionInfo}
526    * @throws IOException
527    */
528   Path commitDaughterRegion(final HRegionInfo regionInfo)
529       throws IOException {
530     Path regionDir = new Path(this.tableDir, regionInfo.getEncodedName());
531     Path daughterTmpDir = this.getSplitsDir(regionInfo);
532 
533     if (fs.exists(daughterTmpDir)) {
534 
535       // Write HRI to a file in case we need to recover hbase:meta
536       Path regionInfoFile = new Path(daughterTmpDir, REGION_INFO_FILE);
537       byte[] regionInfoContent = getRegionInfoFileContent(regionInfo);
538       writeRegionInfoFileContent(conf, fs, regionInfoFile, regionInfoContent);
539 
540       // Move the daughter temp dir to the table dir
541       if (!rename(daughterTmpDir, regionDir)) {
542         throw new IOException("Unable to rename " + daughterTmpDir + " to " + regionDir);
543       }
544     }
545 
546     return regionDir;
547   }
548 
549   /**
550    * Create the region splits directory.
551    */
552   void createSplitsDir() throws IOException {
553     Path splitdir = getSplitsDir();
554     if (fs.exists(splitdir)) {
555       LOG.info("The " + splitdir + " directory exists.  Hence deleting it to recreate it");
556       if (!deleteDir(splitdir)) {
557         throw new IOException("Failed deletion of " + splitdir
558             + " before creating them again.");
559       }
560     }
561     // splitDir doesn't exists now. No need to do an exists() call for it.
562     if (!createDir(splitdir)) {
563       throw new IOException("Failed create of " + splitdir);
564     }
565   }
566 
567   /**
568    * Write out a split reference. Package local so it doesnt leak out of
569    * regionserver.
570    * @param hri {@link HRegionInfo} of the destination
571    * @param familyName Column Family Name
572    * @param f File to split.
573    * @param splitRow Split Row
574    * @param top True if we are referring to the top half of the hfile.
575    * @param splitPolicy
576    * @return Path to created reference.
577    * @throws IOException
578    */
579   Path splitStoreFile(final HRegionInfo hri, final String familyName, final StoreFile f,
580       final byte[] splitRow, final boolean top, RegionSplitPolicy splitPolicy)
581           throws IOException {
582 
583     if (splitPolicy == null || !splitPolicy.skipStoreFileRangeCheck(familyName)) {
584       // Check whether the split row lies in the range of the store file
585       // If it is outside the range, return directly.
586       try {
587         if (top) {
588           //check if larger than last key.
589           KeyValue splitKey = KeyValueUtil.createFirstOnRow(splitRow);
590           Cell lastKey = f.getLastKey();
591           // If lastKey is null means storefile is empty.
592           if (lastKey == null) {
593             return null;
594           }
595           if (f.getComparator().compare(splitKey, lastKey) > 0) {
596             return null;
597           }
598         } else {
599           //check if smaller than first key
600           KeyValue splitKey = KeyValueUtil.createLastOnRow(splitRow);
601           Cell firstKey = f.getFirstKey();
602           // If firstKey is null means storefile is empty.
603           if (firstKey == null) {
604             return null;
605           }
606           if (f.getComparator().compare(splitKey, firstKey) < 0) {
607             return null;
608           }
609         }
610       } finally {
611         f.closeReader(true);
612       }
613     }
614 
615     Path splitDir = new Path(getSplitsDir(hri), familyName);
616     // A reference to the bottom half of the hsf store file.
617     Reference r =
618       top ? Reference.createTopReference(splitRow): Reference.createBottomReference(splitRow);
619     // Add the referred-to regions name as a dot separated suffix.
620     // See REF_NAME_REGEX regex above.  The referred-to regions name is
621     // up in the path of the passed in <code>f</code> -- parentdir is family,
622     // then the directory above is the region name.
623     String parentRegionName = regionInfoForFs.getEncodedName();
624     // Write reference with same file id only with the other region name as
625     // suffix and into the new region location (under same family).
626     Path p = new Path(splitDir, f.getPath().getName() + "." + parentRegionName);
627     return r.write(fs, p);
628   }
629 
630   // ===========================================================================
631   //  Merge Helpers
632   // ===========================================================================
633   /** @return {@link Path} to the temp directory used during merge operations */
634   Path getMergesDir() {
635     return new Path(getRegionDir(), REGION_MERGES_DIR);
636   }
637 
638   Path getMergesDir(final HRegionInfo hri) {
639     return new Path(getMergesDir(), hri.getEncodedName());
640   }
641 
642   /**
643    * Clean up any merge detritus that may have been left around from previous merge attempts.
644    */
645   void cleanupMergesDir() throws IOException {
646     deleteDir(getMergesDir());
647   }
648 
649   /**
650    * Remove merged region
651    * @param mergedRegion {@link HRegionInfo}
652    * @throws IOException
653    */
654   void cleanupMergedRegion(final HRegionInfo mergedRegion) throws IOException {
655     Path regionDir = new Path(this.tableDir, mergedRegion.getEncodedName());
656     if (this.fs.exists(regionDir) && !this.fs.delete(regionDir, true)) {
657       throw new IOException("Failed delete of " + regionDir);
658     }
659   }
660 
661   /**
662    * Create the region merges directory.
663    * @throws IOException If merges dir already exists or we fail to create it.
664    * @see HRegionFileSystem#cleanupMergesDir()
665    */
666   void createMergesDir() throws IOException {
667     Path mergesdir = getMergesDir();
668     if (fs.exists(mergesdir)) {
669       LOG.info("The " + mergesdir
670           + " directory exists.  Hence deleting it to recreate it");
671       if (!fs.delete(mergesdir, true)) {
672         throw new IOException("Failed deletion of " + mergesdir
673             + " before creating them again.");
674       }
675     }
676     if (!fs.mkdirs(mergesdir))
677       throw new IOException("Failed create of " + mergesdir);
678   }
679 
680   /**
681    * Write out a merge reference under the given merges directory. Package local
682    * so it doesnt leak out of regionserver.
683    * @param mergedRegion {@link HRegionInfo} of the merged region
684    * @param familyName Column Family Name
685    * @param f File to create reference.
686    * @param mergedDir
687    * @return Path to created reference.
688    * @throws IOException
689    */
690   Path mergeStoreFile(final HRegionInfo mergedRegion, final String familyName,
691       final StoreFile f, final Path mergedDir)
692       throws IOException {
693     Path referenceDir = new Path(new Path(mergedDir,
694         mergedRegion.getEncodedName()), familyName);
695     // A whole reference to the store file.
696     Reference r = Reference.createTopReference(regionInfoForFs.getStartKey());
697     // Add the referred-to regions name as a dot separated suffix.
698     // See REF_NAME_REGEX regex above. The referred-to regions name is
699     // up in the path of the passed in <code>f</code> -- parentdir is family,
700     // then the directory above is the region name.
701     String mergingRegionName = regionInfoForFs.getEncodedName();
702     // Write reference with same file id only with the other region name as
703     // suffix and into the new region location (under same family).
704     Path p = new Path(referenceDir, f.getPath().getName() + "."
705         + mergingRegionName);
706     return r.write(fs, p);
707   }
708 
709   /**
710    * Commit a merged region, moving it from the merges temporary directory to
711    * the proper location in the filesystem.
712    * @param mergedRegionInfo merged region {@link HRegionInfo}
713    * @throws IOException
714    */
715   void commitMergedRegion(final HRegionInfo mergedRegionInfo) throws IOException {
716     Path regionDir = new Path(this.tableDir, mergedRegionInfo.getEncodedName());
717     Path mergedRegionTmpDir = this.getMergesDir(mergedRegionInfo);
718     // Move the tmp dir in the expected location
719     if (mergedRegionTmpDir != null && fs.exists(mergedRegionTmpDir)) {
720       if (!fs.rename(mergedRegionTmpDir, regionDir)) {
721         throw new IOException("Unable to rename " + mergedRegionTmpDir + " to "
722             + regionDir);
723       }
724     }
725   }
726 
727   // ===========================================================================
728   //  Create/Open/Delete Helpers
729   // ===========================================================================
730   /**
731    * Log the current state of the region
732    * @param LOG log to output information
733    * @throws IOException if an unexpected exception occurs
734    */
735   void logFileSystemState(final Log LOG) throws IOException {
736     FSUtils.logFileSystemState(fs, this.getRegionDir(), LOG);
737   }
738 
739   /**
740    * @param hri
741    * @return Content of the file we write out to the filesystem under a region
742    * @throws IOException
743    */
744   private static byte[] getRegionInfoFileContent(final HRegionInfo hri) throws IOException {
745     return hri.toDelimitedByteArray();
746   }
747 
748   /**
749    * Create a {@link HRegionInfo} from the serialized version on-disk.
750    * @param fs {@link FileSystem} that contains the Region Info file
751    * @param regionDir {@link Path} to the Region Directory that contains the Info file
752    * @return An {@link HRegionInfo} instance gotten from the Region Info file.
753    * @throws IOException if an error occurred during file open/read operation.
754    */
755   public static HRegionInfo loadRegionInfoFileContent(final FileSystem fs, final Path regionDir)
756       throws IOException {
757     FSDataInputStream in = fs.open(new Path(regionDir, REGION_INFO_FILE));
758     try {
759       return HRegionInfo.parseFrom(in);
760     } finally {
761       in.close();
762     }
763   }
764 
765   /**
766    * Write the .regioninfo file on-disk.
767    */
768   private static void writeRegionInfoFileContent(final Configuration conf, final FileSystem fs,
769       final Path regionInfoFile, final byte[] content) throws IOException {
770     // First check to get the permissions
771     FsPermission perms = FSUtils.getFilePermissions(fs, conf, HConstants.DATA_FILE_UMASK_KEY);
772     // Write the RegionInfo file content
773     FSDataOutputStream out = FSUtils.create(conf, fs, regionInfoFile, perms, null);
774     try {
775       out.write(content);
776     } finally {
777       out.close();
778     }
779   }
780 
781   /**
782    * Write out an info file under the stored region directory. Useful recovering mangled regions.
783    * If the regionInfo already exists on-disk, then we fast exit.
784    */
785   void checkRegionInfoOnFilesystem() throws IOException {
786     // Compose the content of the file so we can compare to length in filesystem. If not same,
787     // rewrite it (it may have been written in the old format using Writables instead of pb). The
788     // pb version is much shorter -- we write now w/o the toString version -- so checking length
789     // only should be sufficient. I don't want to read the file every time to check if it pb
790     // serialized.
791     byte[] content = getRegionInfoFileContent(regionInfoForFs);
792     try {
793       Path regionInfoFile = new Path(getRegionDir(), REGION_INFO_FILE);
794 
795       FileStatus status = fs.getFileStatus(regionInfoFile);
796       if (status != null && status.getLen() == content.length) {
797         // Then assume the content good and move on.
798         // NOTE: that the length is not sufficient to define the the content matches.
799         return;
800       }
801 
802       LOG.info("Rewriting .regioninfo file at: " + regionInfoFile);
803       if (!fs.delete(regionInfoFile, false)) {
804         throw new IOException("Unable to remove existing " + regionInfoFile);
805       }
806     } catch (FileNotFoundException e) {
807       LOG.warn(REGION_INFO_FILE + " file not found for region: " + regionInfoForFs.getEncodedName() +
808           " on table " + regionInfo.getTable());
809     }
810 
811     // Write HRI to a file in case we need to recover hbase:meta
812     writeRegionInfoOnFilesystem(content, true);
813   }
814 
815   /**
816    * Write out an info file under the region directory. Useful recovering mangled regions.
817    * @param useTempDir indicate whether or not using the region .tmp dir for a safer file creation.
818    */
819   private void writeRegionInfoOnFilesystem(boolean useTempDir) throws IOException {
820     byte[] content = getRegionInfoFileContent(regionInfoForFs);
821     writeRegionInfoOnFilesystem(content, useTempDir);
822   }
823 
824   /**
825    * Write out an info file under the region directory. Useful recovering mangled regions.
826    * @param regionInfoContent serialized version of the {@link HRegionInfo}
827    * @param useTempDir indicate whether or not using the region .tmp dir for a safer file creation.
828    */
829   private void writeRegionInfoOnFilesystem(final byte[] regionInfoContent,
830       final boolean useTempDir) throws IOException {
831     Path regionInfoFile = new Path(getRegionDir(), REGION_INFO_FILE);
832     if (useTempDir) {
833       // Create in tmpDir and then move into place in case we crash after
834       // create but before close. If we don't successfully close the file,
835       // subsequent region reopens will fail the below because create is
836       // registered in NN.
837 
838       // And then create the file
839       Path tmpPath = new Path(getTempDir(), REGION_INFO_FILE);
840 
841       // If datanode crashes or if the RS goes down just before the close is called while trying to
842       // close the created regioninfo file in the .tmp directory then on next
843       // creation we will be getting AlreadyCreatedException.
844       // Hence delete and create the file if exists.
845       if (FSUtils.isExists(fs, tmpPath)) {
846         FSUtils.delete(fs, tmpPath, true);
847       }
848 
849       // Write HRI to a file in case we need to recover hbase:meta
850       writeRegionInfoFileContent(conf, fs, tmpPath, regionInfoContent);
851 
852       // Move the created file to the original path
853       if (fs.exists(tmpPath) &&  !rename(tmpPath, regionInfoFile)) {
854         throw new IOException("Unable to rename " + tmpPath + " to " + regionInfoFile);
855       }
856     } else {
857       // Write HRI to a file in case we need to recover hbase:meta
858       writeRegionInfoFileContent(conf, fs, regionInfoFile, regionInfoContent);
859     }
860   }
861 
862   /**
863    * Create a new Region on file-system.
864    * @param conf the {@link Configuration} to use
865    * @param fs {@link FileSystem} from which to add the region
866    * @param tableDir {@link Path} to where the table is being stored
867    * @param regionInfo {@link HRegionInfo} for region to be added
868    * @throws IOException if the region creation fails due to a FileSystem exception.
869    */
870   public static HRegionFileSystem createRegionOnFileSystem(final Configuration conf,
871       final FileSystem fs, final Path tableDir, final HRegionInfo regionInfo) throws IOException {
872     HRegionFileSystem regionFs = new HRegionFileSystem(conf, fs, tableDir, regionInfo);
873     Path regionDir = regionFs.getRegionDir();
874 
875     if (fs.exists(regionDir)) {
876       LOG.warn("Trying to create a region that already exists on disk: " + regionDir);
877       throw new IOException("The specified region already exists on disk: " + regionDir);
878     }
879 
880     // Create the region directory
881     if (!createDirOnFileSystem(fs, conf, regionDir)) {
882       LOG.warn("Unable to create the region directory: " + regionDir);
883       throw new IOException("Unable to create region directory: " + regionDir);
884     }
885 
886     // Write HRI to a file in case we need to recover hbase:meta
887     regionFs.writeRegionInfoOnFilesystem(false);
888     return regionFs;
889   }
890 
891   /**
892    * Open Region from file-system.
893    * @param conf the {@link Configuration} to use
894    * @param fs {@link FileSystem} from which to add the region
895    * @param tableDir {@link Path} to where the table is being stored
896    * @param regionInfo {@link HRegionInfo} for region to be added
897    * @param readOnly True if you don't want to edit the region data
898    * @throws IOException if the region creation fails due to a FileSystem exception.
899    */
900   public static HRegionFileSystem openRegionFromFileSystem(final Configuration conf,
901       final FileSystem fs, final Path tableDir, final HRegionInfo regionInfo, boolean readOnly)
902       throws IOException {
903     HRegionFileSystem regionFs = new HRegionFileSystem(conf, fs, tableDir, regionInfo);
904     Path regionDir = regionFs.getRegionDir();
905 
906     if (!fs.exists(regionDir)) {
907       LOG.warn("Trying to open a region that do not exists on disk: " + regionDir);
908       throw new IOException("The specified region do not exists on disk: " + regionDir);
909     }
910 
911     if (!readOnly) {
912       // Cleanup temporary directories
913       regionFs.cleanupTempDir();
914       regionFs.cleanupSplitsDir();
915       regionFs.cleanupMergesDir();
916 
917       // if it doesn't exists, Write HRI to a file, in case we need to recover hbase:meta
918       regionFs.checkRegionInfoOnFilesystem();
919     }
920 
921     return regionFs;
922   }
923 
924   /**
925    * Remove the region from the table directory, archiving the region's hfiles.
926    * @param conf the {@link Configuration} to use
927    * @param fs {@link FileSystem} from which to remove the region
928    * @param tableDir {@link Path} to where the table is being stored
929    * @param regionInfo {@link HRegionInfo} for region to be deleted
930    * @throws IOException if the request cannot be completed
931    */
932   public static void deleteRegionFromFileSystem(final Configuration conf,
933       final FileSystem fs, final Path tableDir, final HRegionInfo regionInfo) throws IOException {
934     HRegionFileSystem regionFs = new HRegionFileSystem(conf, fs, tableDir, regionInfo);
935     Path regionDir = regionFs.getRegionDir();
936 
937     if (!fs.exists(regionDir)) {
938       LOG.warn("Trying to delete a region that do not exists on disk: " + regionDir);
939       return;
940     }
941 
942     if (LOG.isDebugEnabled()) {
943       LOG.debug("DELETING region " + regionDir);
944     }
945 
946     // Archive region
947     Path rootDir = FSUtils.getRootDir(conf);
948     HFileArchiver.archiveRegion(fs, rootDir, tableDir, regionDir);
949 
950     // Delete empty region dir
951     if (!fs.delete(regionDir, true)) {
952       LOG.warn("Failed delete of " + regionDir);
953     }
954   }
955 
956   /**
957    * Creates a directory. Assumes the user has already checked for this directory existence.
958    * @param dir
959    * @return the result of fs.mkdirs(). In case underlying fs throws an IOException, it checks
960    *         whether the directory exists or not, and returns true if it exists.
961    * @throws IOException
962    */
963   boolean createDir(Path dir) throws IOException {
964     int i = 0;
965     IOException lastIOE = null;
966     do {
967       try {
968         return fs.mkdirs(dir);
969       } catch (IOException ioe) {
970         lastIOE = ioe;
971         if (fs.exists(dir)) return true; // directory is present
972         try {
973           sleepBeforeRetry("Create Directory", i+1);
974         } catch (InterruptedException e) {
975           throw (InterruptedIOException)new InterruptedIOException().initCause(e);
976         }
977       }
978     } while (++i <= hdfsClientRetriesNumber);
979     throw new IOException("Exception in createDir", lastIOE);
980   }
981 
982   /**
983    * Renames a directory. Assumes the user has already checked for this directory existence.
984    * @param srcpath
985    * @param dstPath
986    * @return true if rename is successful.
987    * @throws IOException
988    */
989   boolean rename(Path srcpath, Path dstPath) throws IOException {
990     IOException lastIOE = null;
991     int i = 0;
992     do {
993       try {
994         return fs.rename(srcpath, dstPath);
995       } catch (IOException ioe) {
996         lastIOE = ioe;
997         if (!fs.exists(srcpath) && fs.exists(dstPath)) return true; // successful move
998         // dir is not there, retry after some time.
999         try {
1000           sleepBeforeRetry("Rename Directory", i+1);
1001         } catch (InterruptedException e) {
1002           throw (InterruptedIOException)new InterruptedIOException().initCause(e);
1003         }
1004       }
1005     } while (++i <= hdfsClientRetriesNumber);
1006 
1007     throw new IOException("Exception in rename", lastIOE);
1008   }
1009 
1010   /**
1011    * Deletes a directory. Assumes the user has already checked for this directory existence.
1012    * @param dir
1013    * @return true if the directory is deleted.
1014    * @throws IOException
1015    */
1016   boolean deleteDir(Path dir) throws IOException {
1017     IOException lastIOE = null;
1018     int i = 0;
1019     do {
1020       try {
1021         return fs.delete(dir, true);
1022       } catch (IOException ioe) {
1023         lastIOE = ioe;
1024         if (!fs.exists(dir)) return true;
1025         // dir is there, retry deleting after some time.
1026         try {
1027           sleepBeforeRetry("Delete Directory", i+1);
1028         } catch (InterruptedException e) {
1029           throw (InterruptedIOException)new InterruptedIOException().initCause(e);
1030         }
1031       }
1032     } while (++i <= hdfsClientRetriesNumber);
1033 
1034     throw new IOException("Exception in DeleteDir", lastIOE);
1035   }
1036 
1037   /**
1038    * sleeping logic; handles the interrupt exception.
1039    */
1040   private void sleepBeforeRetry(String msg, int sleepMultiplier) throws InterruptedException {
1041     sleepBeforeRetry(msg, sleepMultiplier, baseSleepBeforeRetries, hdfsClientRetriesNumber);
1042   }
1043 
1044   /**
1045    * Creates a directory for a filesystem and configuration object. Assumes the user has already
1046    * checked for this directory existence.
1047    * @param fs
1048    * @param conf
1049    * @param dir
1050    * @return the result of fs.mkdirs(). In case underlying fs throws an IOException, it checks
1051    *         whether the directory exists or not, and returns true if it exists.
1052    * @throws IOException
1053    */
1054   private static boolean createDirOnFileSystem(FileSystem fs, Configuration conf, Path dir)
1055       throws IOException {
1056     int i = 0;
1057     IOException lastIOE = null;
1058     int hdfsClientRetriesNumber = conf.getInt("hdfs.client.retries.number",
1059       DEFAULT_HDFS_CLIENT_RETRIES_NUMBER);
1060     int baseSleepBeforeRetries = conf.getInt("hdfs.client.sleep.before.retries",
1061       DEFAULT_BASE_SLEEP_BEFORE_RETRIES);
1062     do {
1063       try {
1064         return fs.mkdirs(dir);
1065       } catch (IOException ioe) {
1066         lastIOE = ioe;
1067         if (fs.exists(dir)) return true; // directory is present
1068         try {
1069           sleepBeforeRetry("Create Directory", i+1, baseSleepBeforeRetries, hdfsClientRetriesNumber);
1070         } catch (InterruptedException e) {
1071           throw (InterruptedIOException)new InterruptedIOException().initCause(e);
1072         }
1073       }
1074     } while (++i <= hdfsClientRetriesNumber);
1075 
1076     throw new IOException("Exception in createDir", lastIOE);
1077   }
1078 
1079   /**
1080    * sleeping logic for static methods; handles the interrupt exception. Keeping a static version
1081    * for this to avoid re-looking for the integer values.
1082    */
1083   private static void sleepBeforeRetry(String msg, int sleepMultiplier, int baseSleepBeforeRetries,
1084       int hdfsClientRetriesNumber) throws InterruptedException {
1085     if (sleepMultiplier > hdfsClientRetriesNumber) {
1086       LOG.debug(msg + ", retries exhausted");
1087       return;
1088     }
1089     LOG.debug(msg + ", sleeping " + baseSleepBeforeRetries + " times " + sleepMultiplier);
1090     Thread.sleep((long)baseSleepBeforeRetries * sleepMultiplier);
1091   }
1092 }