001/**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.backup;
019
020import java.io.FileNotFoundException;
021import java.io.IOException;
022import java.io.InterruptedIOException;
023import java.util.ArrayList;
024import java.util.Collection;
025import java.util.Collections;
026import java.util.List;
027import java.util.concurrent.ExecutionException;
028import java.util.concurrent.Future;
029import java.util.concurrent.ThreadFactory;
030import java.util.concurrent.ThreadPoolExecutor;
031import java.util.concurrent.TimeUnit;
032import java.util.concurrent.atomic.AtomicInteger;
033import java.util.function.Function;
034import java.util.stream.Collectors;
035import java.util.stream.Stream;
036import org.apache.hadoop.conf.Configuration;
037import org.apache.hadoop.fs.FileStatus;
038import org.apache.hadoop.fs.FileSystem;
039import org.apache.hadoop.fs.Path;
040import org.apache.hadoop.fs.PathFilter;
041import org.apache.hadoop.hbase.HConstants;
042import org.apache.hadoop.hbase.client.RegionInfo;
043import org.apache.hadoop.hbase.regionserver.HStoreFile;
044import org.apache.hadoop.hbase.util.Bytes;
045import org.apache.hadoop.hbase.util.CommonFSUtils;
046import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
047import org.apache.hadoop.hbase.util.FSUtils;
048import org.apache.hadoop.hbase.util.HFileArchiveUtil;
049import org.apache.hadoop.hbase.util.Threads;
050import org.apache.hadoop.io.MultipleIOException;
051import org.apache.yetus.audience.InterfaceAudience;
052import org.slf4j.Logger;
053import org.slf4j.LoggerFactory;
054
055import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
056
057/**
058 * Utility class to handle the removal of HFiles (or the respective {@link HStoreFile StoreFiles})
059 * for a HRegion from the {@link FileSystem}. The hfiles will be archived or deleted, depending on
060 * the state of the system.
061 */
062@InterfaceAudience.Private
063public class HFileArchiver {
064  private static final Logger LOG = LoggerFactory.getLogger(HFileArchiver.class);
065  private static final String SEPARATOR = ".";
066
067  /** Number of retries in case of fs operation failure */
068  private static final int DEFAULT_RETRIES_NUMBER = 3;
069
070  private static final Function<File, Path> FUNC_FILE_TO_PATH =
071      new Function<File, Path>() {
072        @Override
073        public Path apply(File file) {
074          return file == null ? null : file.getPath();
075        }
076      };
077
078  private static ThreadPoolExecutor archiveExecutor;
079
080  private HFileArchiver() {
081    // hidden ctor since this is just a util
082  }
083
084  /**
085   * @return True if the Region exits in the filesystem.
086   */
087  public static boolean exists(Configuration conf, FileSystem fs, RegionInfo info)
088      throws IOException {
089    Path rootDir = CommonFSUtils.getRootDir(conf);
090    Path regionDir = FSUtils.getRegionDirFromRootDir(rootDir, info);
091    return fs.exists(regionDir);
092  }
093
094  /**
095   * Cleans up all the files for a HRegion by archiving the HFiles to the archive directory
096   * @param conf the configuration to use
097   * @param fs the file system object
098   * @param info RegionInfo for region to be deleted
099   */
100  public static void archiveRegion(Configuration conf, FileSystem fs, RegionInfo info)
101      throws IOException {
102    Path rootDir = CommonFSUtils.getRootDir(conf);
103    archiveRegion(fs, rootDir, CommonFSUtils.getTableDir(rootDir, info.getTable()),
104      FSUtils.getRegionDirFromRootDir(rootDir, info));
105  }
106
107  /**
108   * Remove an entire region from the table directory via archiving the region's hfiles.
109   * @param fs {@link FileSystem} from which to remove the region
110   * @param rootdir {@link Path} to the root directory where hbase files are stored (for building
111   *          the archive path)
112   * @param tableDir {@link Path} to where the table is being stored (for building the archive path)
113   * @param regionDir {@link Path} to where a region is being stored (for building the archive path)
114   * @return <tt>true</tt> if the region was successfully deleted. <tt>false</tt> if the filesystem
115   *         operations could not complete.
116   * @throws IOException if the request cannot be completed
117   */
118  public static boolean archiveRegion(FileSystem fs, Path rootdir, Path tableDir, Path regionDir)
119      throws IOException {
120    // otherwise, we archive the files
121    // make sure we can archive
122    if (tableDir == null || regionDir == null) {
123      LOG.error("No archive directory could be found because tabledir (" + tableDir
124          + ") or regiondir (" + regionDir + "was null. Deleting files instead.");
125      if (regionDir != null) {
126        deleteRegionWithoutArchiving(fs, regionDir);
127      }
128      // we should have archived, but failed to. Doesn't matter if we deleted
129      // the archived files correctly or not.
130      return false;
131    }
132
133    LOG.debug("ARCHIVING {}", regionDir);
134
135    // make sure the regiondir lives under the tabledir
136    Preconditions.checkArgument(regionDir.toString().startsWith(tableDir.toString()));
137    Path regionArchiveDir = HFileArchiveUtil.getRegionArchiveDir(rootdir,
138      CommonFSUtils.getTableName(tableDir), regionDir.getName());
139
140    FileStatusConverter getAsFile = new FileStatusConverter(fs);
141    // otherwise, we attempt to archive the store files
142
143    // build collection of just the store directories to archive
144    Collection<File> toArchive = new ArrayList<>();
145    final PathFilter dirFilter = new FSUtils.DirFilter(fs);
146    PathFilter nonHidden = new PathFilter() {
147      @Override
148      public boolean accept(Path file) {
149        return dirFilter.accept(file) && !file.getName().startsWith(".");
150      }
151    };
152    FileStatus[] storeDirs = CommonFSUtils.listStatus(fs, regionDir, nonHidden);
153    // if there no files, we can just delete the directory and return;
154    if (storeDirs == null) {
155      LOG.debug("Directory {} empty.", regionDir);
156      return deleteRegionWithoutArchiving(fs, regionDir);
157    }
158
159    // convert the files in the region to a File
160    Stream.of(storeDirs).map(getAsFile).forEachOrdered(toArchive::add);
161    LOG.debug("Archiving " + toArchive);
162    List<File> failedArchive = resolveAndArchive(fs, regionArchiveDir, toArchive,
163        EnvironmentEdgeManager.currentTime());
164    if (!failedArchive.isEmpty()) {
165      throw new FailedArchiveException(
166        "Failed to archive/delete all the files for region:" + regionDir.getName() + " into " +
167          regionArchiveDir + ". Something is probably awry on the filesystem.",
168        failedArchive.stream().map(FUNC_FILE_TO_PATH).collect(Collectors.toList()));
169    }
170    // if that was successful, then we delete the region
171    return deleteRegionWithoutArchiving(fs, regionDir);
172  }
173
174  /**
175   * Archive the specified regions in parallel.
176   * @param conf the configuration to use
177   * @param fs {@link FileSystem} from which to remove the region
178   * @param rootDir {@link Path} to the root directory where hbase files are stored (for building
179   *                            the archive path)
180   * @param tableDir {@link Path} to where the table is being stored (for building the archive
181   *                             path)
182   * @param regionDirList {@link Path} to where regions are being stored (for building the archive
183   *                                  path)
184   * @throws IOException if the request cannot be completed
185   */
186  public static void archiveRegions(Configuration conf, FileSystem fs, Path rootDir, Path tableDir,
187    List<Path> regionDirList) throws IOException {
188    List<Future<Void>> futures = new ArrayList<>(regionDirList.size());
189    for (Path regionDir: regionDirList) {
190      Future<Void> future = getArchiveExecutor(conf).submit(() -> {
191        archiveRegion(fs, rootDir, tableDir, regionDir);
192        return null;
193      });
194      futures.add(future);
195    }
196    try {
197      for (Future<Void> future: futures) {
198        future.get();
199      }
200    } catch (InterruptedException e) {
201      throw new InterruptedIOException(e.getMessage());
202    } catch (ExecutionException e) {
203      throw new IOException(e.getCause());
204    }
205  }
206
207  private static synchronized ThreadPoolExecutor getArchiveExecutor(final Configuration conf) {
208    if (archiveExecutor == null) {
209      int maxThreads = conf.getInt("hbase.hfilearchiver.thread.pool.max", 8);
210      archiveExecutor = Threads.getBoundedCachedThreadPool(maxThreads, 30L, TimeUnit.SECONDS,
211        getThreadFactory());
212
213      // Shutdown this ThreadPool in a shutdown hook
214      Runtime.getRuntime().addShutdownHook(new Thread(() -> archiveExecutor.shutdown()));
215    }
216    return archiveExecutor;
217  }
218
219  // We need this method instead of Threads.getNamedThreadFactory() to pass some tests.
220  // The difference from Threads.getNamedThreadFactory() is that it doesn't fix ThreadGroup for
221  // new threads. If we use Threads.getNamedThreadFactory(), we will face ThreadGroup related
222  // issues in some tests.
223  private static ThreadFactory getThreadFactory() {
224    return new ThreadFactory() {
225      final AtomicInteger threadNumber = new AtomicInteger(1);
226
227      @Override
228      public Thread newThread(Runnable r) {
229        final String name = "HFileArchiver-" + threadNumber.getAndIncrement();
230        Thread t = new Thread(r, name);
231        t.setDaemon(true);
232        return t;
233      }
234    };
235  }
236
237  /**
238   * Remove from the specified region the store files of the specified column family,
239   * either by archiving them or outright deletion
240   * @param fs the filesystem where the store files live
241   * @param conf {@link Configuration} to examine to determine the archive directory
242   * @param parent Parent region hosting the store files
243   * @param tableDir {@link Path} to where the table is being stored (for building the archive path)
244   * @param family the family hosting the store files
245   * @throws IOException if the files could not be correctly disposed.
246   */
247  public static void archiveFamily(FileSystem fs, Configuration conf,
248      RegionInfo parent, Path tableDir, byte[] family) throws IOException {
249    Path familyDir = new Path(tableDir, new Path(parent.getEncodedName(), Bytes.toString(family)));
250    archiveFamilyByFamilyDir(fs, conf, parent, familyDir, family);
251  }
252
253  /**
254   * Removes from the specified region the store files of the specified column family,
255   * either by archiving them or outright deletion
256   * @param fs the filesystem where the store files live
257   * @param conf {@link Configuration} to examine to determine the archive directory
258   * @param parent Parent region hosting the store files
259   * @param familyDir {@link Path} to where the family is being stored
260   * @param family the family hosting the store files
261   * @throws IOException if the files could not be correctly disposed.
262   */
263  public static void archiveFamilyByFamilyDir(FileSystem fs, Configuration conf,
264      RegionInfo parent, Path familyDir, byte[] family) throws IOException {
265    FileStatus[] storeFiles = CommonFSUtils.listStatus(fs, familyDir);
266    if (storeFiles == null) {
267      LOG.debug("No files to dispose of in {}, family={}", parent.getRegionNameAsString(),
268          Bytes.toString(family));
269      return;
270    }
271
272    FileStatusConverter getAsFile = new FileStatusConverter(fs);
273    Collection<File> toArchive = Stream.of(storeFiles).map(getAsFile).collect(Collectors.toList());
274    Path storeArchiveDir = HFileArchiveUtil.getStoreArchivePath(conf, parent, family);
275
276    // do the actual archive
277    List<File> failedArchive = resolveAndArchive(fs, storeArchiveDir, toArchive,
278        EnvironmentEdgeManager.currentTime());
279    if (!failedArchive.isEmpty()){
280      throw new FailedArchiveException("Failed to archive/delete all the files for region:"
281          + Bytes.toString(parent.getRegionName()) + ", family:" + Bytes.toString(family)
282          + " into " + storeArchiveDir + ". Something is probably awry on the filesystem.",
283          failedArchive.stream().map(FUNC_FILE_TO_PATH).collect(Collectors.toList()));
284    }
285  }
286
287  /**
288   * Remove the store files, either by archiving them or outright deletion
289   * @param conf {@link Configuration} to examine to determine the archive directory
290   * @param fs the filesystem where the store files live
291   * @param regionInfo {@link RegionInfo} of the region hosting the store files
292   * @param family the family hosting the store files
293   * @param compactedFiles files to be disposed of. No further reading of these files should be
294   *          attempted; otherwise likely to cause an {@link IOException}
295   * @throws IOException if the files could not be correctly disposed.
296   */
297  public static void archiveStoreFiles(Configuration conf, FileSystem fs, RegionInfo regionInfo,
298      Path tableDir, byte[] family, Collection<HStoreFile> compactedFiles)
299      throws IOException {
300    Path storeArchiveDir = HFileArchiveUtil.getStoreArchivePath(conf, regionInfo, tableDir, family);
301    archive(fs, regionInfo, family, compactedFiles, storeArchiveDir);
302  }
303
304  /**
305   * Archive recovered edits using existing logic for archiving store files. This is currently only
306   * relevant when <b>hbase.region.archive.recovered.edits</b> is true, as recovered edits shouldn't
307   * be kept after replay. In theory, we could use very same method available for archiving
308   * store files, but supporting WAL dir and store files on different FileSystems added the need for
309   * extra validation of the passed FileSystem instance and the path where the archiving edits
310   * should be placed.
311   * @param conf {@link Configuration} to determine the archive directory.
312   * @param fs the filesystem used for storing WAL files.
313   * @param regionInfo {@link RegionInfo} a pseudo region representation for the archiving logic.
314   * @param family a pseudo familiy representation for the archiving logic.
315   * @param replayedEdits the recovered edits to be archived.
316   * @throws IOException if files can't be achived due to some internal error.
317   */
318  public static void archiveRecoveredEdits(Configuration conf, FileSystem fs, RegionInfo regionInfo,
319    byte[] family, Collection<HStoreFile> replayedEdits)
320    throws IOException {
321    String workingDir = conf.get(CommonFSUtils.HBASE_WAL_DIR, conf.get(HConstants.HBASE_DIR));
322    //extra sanity checks for the right FS
323    Path path = new Path(workingDir);
324    if(path.isAbsoluteAndSchemeAuthorityNull()){
325      //no schema specified on wal dir value, so it's on same FS as StoreFiles
326      path = new Path(conf.get(HConstants.HBASE_DIR));
327    }
328    if(path.toUri().getScheme()!=null && !path.toUri().getScheme().equals(fs.getScheme())){
329      throw new IOException("Wrong file system! Should be " + path.toUri().getScheme() +
330        ", but got " +  fs.getScheme());
331    }
332    path = HFileArchiveUtil.getStoreArchivePathForRootDir(path, regionInfo, family);
333    archive(fs, regionInfo, family, replayedEdits, path);
334  }
335
336  private static void archive(FileSystem fs, RegionInfo regionInfo, byte[] family,
337    Collection<HStoreFile> compactedFiles, Path storeArchiveDir) throws IOException {
338    // sometimes in testing, we don't have rss, so we need to check for that
339    if (fs == null) {
340      LOG.warn("Passed filesystem is null, so just deleting files without archiving for {}," +
341              "family={}", Bytes.toString(regionInfo.getRegionName()), Bytes.toString(family));
342      deleteStoreFilesWithoutArchiving(compactedFiles);
343      return;
344    }
345
346    // short circuit if we don't have any files to delete
347    if (compactedFiles.isEmpty()) {
348      LOG.debug("No files to dispose of, done!");
349      return;
350    }
351
352    // build the archive path
353    if (regionInfo == null || family == null) throw new IOException(
354        "Need to have a region and a family to archive from.");
355    // make sure we don't archive if we can't and that the archive dir exists
356    if (!fs.mkdirs(storeArchiveDir)) {
357      throw new IOException("Could not make archive directory (" + storeArchiveDir + ") for store:"
358          + Bytes.toString(family) + ", deleting compacted files instead.");
359    }
360
361    // otherwise we attempt to archive the store files
362    LOG.debug("Archiving compacted files.");
363
364    // Wrap the storefile into a File
365    StoreToFile getStorePath = new StoreToFile(fs);
366    Collection<File> storeFiles =
367      compactedFiles.stream().map(getStorePath).collect(Collectors.toList());
368
369    // do the actual archive
370    List<File> failedArchive =
371      resolveAndArchive(fs, storeArchiveDir, storeFiles, EnvironmentEdgeManager.currentTime());
372
373    if (!failedArchive.isEmpty()){
374      throw new FailedArchiveException("Failed to archive/delete all the files for region:"
375          + Bytes.toString(regionInfo.getRegionName()) + ", family:" + Bytes.toString(family)
376          + " into " + storeArchiveDir + ". Something is probably awry on the filesystem.",
377          failedArchive.stream().map(FUNC_FILE_TO_PATH).collect(Collectors.toList()));
378    }
379  }
380
381  /**
382   * Archive the store file
383   * @param fs the filesystem where the store files live
384   * @param regionInfo region hosting the store files
385   * @param conf {@link Configuration} to examine to determine the archive directory
386   * @param tableDir {@link Path} to where the table is being stored (for building the archive path)
387   * @param family the family hosting the store files
388   * @param storeFile file to be archived
389   * @throws IOException if the files could not be correctly disposed.
390   */
391  public static void archiveStoreFile(Configuration conf, FileSystem fs, RegionInfo regionInfo,
392      Path tableDir, byte[] family, Path storeFile) throws IOException {
393    Path storeArchiveDir = HFileArchiveUtil.getStoreArchivePath(conf, regionInfo, tableDir, family);
394    // make sure we don't archive if we can't and that the archive dir exists
395    if (!fs.mkdirs(storeArchiveDir)) {
396      throw new IOException("Could not make archive directory (" + storeArchiveDir + ") for store:"
397          + Bytes.toString(family) + ", deleting compacted files instead.");
398    }
399
400    // do the actual archive
401    long start = EnvironmentEdgeManager.currentTime();
402    File file = new FileablePath(fs, storeFile);
403    if (!resolveAndArchiveFile(storeArchiveDir, file, Long.toString(start))) {
404      throw new IOException("Failed to archive/delete the file for region:"
405          + regionInfo.getRegionNameAsString() + ", family:" + Bytes.toString(family)
406          + " into " + storeArchiveDir + ". Something is probably awry on the filesystem.");
407    }
408  }
409
410  /**
411   * Resolve any conflict with an existing archive file via timestamp-append
412   * renaming of the existing file and then archive the passed in files.
413   * @param fs {@link FileSystem} on which to archive the files
414   * @param baseArchiveDir base archive directory to store the files. If any of
415   *          the files to archive are directories, will append the name of the
416   *          directory to the base archive directory name, creating a parallel
417   *          structure.
418   * @param toArchive files/directories that need to be archvied
419   * @param start time the archiving started - used for resolving archive
420   *          conflicts.
421   * @return the list of failed to archive files.
422   * @throws IOException if an unexpected file operation exception occurred
423   */
424  private static List<File> resolveAndArchive(FileSystem fs, Path baseArchiveDir,
425      Collection<File> toArchive, long start) throws IOException {
426    // short circuit if no files to move
427    if (toArchive.isEmpty()) {
428      return Collections.emptyList();
429    }
430
431    LOG.trace("Moving files to the archive directory {}", baseArchiveDir);
432
433    // make sure the archive directory exists
434    if (!fs.exists(baseArchiveDir)) {
435      if (!fs.mkdirs(baseArchiveDir)) {
436        throw new IOException("Failed to create the archive directory:" + baseArchiveDir
437            + ", quitting archive attempt.");
438      }
439      LOG.trace("Created archive directory {}", baseArchiveDir);
440    }
441
442    List<File> failures = new ArrayList<>();
443    String startTime = Long.toString(start);
444    for (File file : toArchive) {
445      // if its a file archive it
446      try {
447        LOG.trace("Archiving {}", file);
448        if (file.isFile()) {
449          // attempt to archive the file
450          if (!resolveAndArchiveFile(baseArchiveDir, file, startTime)) {
451            LOG.warn("Couldn't archive " + file + " into backup directory: " + baseArchiveDir);
452            failures.add(file);
453          }
454        } else {
455          // otherwise its a directory and we need to archive all files
456          LOG.trace("{} is a directory, archiving children files", file);
457          // so we add the directory name to the one base archive
458          Path parentArchiveDir = new Path(baseArchiveDir, file.getName());
459          // and then get all the files from that directory and attempt to
460          // archive those too
461          Collection<File> children = file.getChildren();
462          failures.addAll(resolveAndArchive(fs, parentArchiveDir, children, start));
463        }
464      } catch (IOException e) {
465        LOG.warn("Failed to archive {}", file, e);
466        failures.add(file);
467      }
468    }
469    return failures;
470  }
471
472  /**
473   * Attempt to archive the passed in file to the archive directory.
474   * <p>
475   * If the same file already exists in the archive, it is moved to a timestamped directory under
476   * the archive directory and the new file is put in its place.
477   * @param archiveDir {@link Path} to the directory that stores the archives of the hfiles
478   * @param currentFile {@link Path} to the original HFile that will be archived
479   * @param archiveStartTime time the archiving started, to resolve naming conflicts
480   * @return <tt>true</tt> if the file is successfully archived. <tt>false</tt> if there was a
481   *         problem, but the operation still completed.
482   * @throws IOException on failure to complete {@link FileSystem} operations.
483   */
484  private static boolean resolveAndArchiveFile(Path archiveDir, File currentFile,
485      String archiveStartTime) throws IOException {
486    // build path as it should be in the archive
487    String filename = currentFile.getName();
488    Path archiveFile = new Path(archiveDir, filename);
489    FileSystem fs = currentFile.getFileSystem();
490
491    // if the file already exists in the archive, move that one to a timestamped backup. This is a
492    // really, really unlikely situtation, where we get the same name for the existing file, but
493    // is included just for that 1 in trillion chance.
494    if (fs.exists(archiveFile)) {
495      LOG.debug("{} already exists in archive, moving to timestamped backup and " +
496          "overwriting current.", archiveFile);
497
498      // move the archive file to the stamped backup
499      Path backedupArchiveFile = new Path(archiveDir, filename + SEPARATOR + archiveStartTime);
500      if (!fs.rename(archiveFile, backedupArchiveFile)) {
501        LOG.error("Could not rename archive file to backup: " + backedupArchiveFile
502            + ", deleting existing file in favor of newer.");
503        // try to delete the exisiting file, if we can't rename it
504        if (!fs.delete(archiveFile, false)) {
505          throw new IOException("Couldn't delete existing archive file (" + archiveFile
506              + ") or rename it to the backup file (" + backedupArchiveFile
507              + ") to make room for similarly named file.");
508        }
509      }
510      LOG.debug("Backed up archive file from " + archiveFile);
511    }
512
513    LOG.trace("No existing file in archive for {}, free to archive original file.", archiveFile);
514
515    // at this point, we should have a free spot for the archive file
516    boolean success = false;
517    for (int i = 0; !success && i < DEFAULT_RETRIES_NUMBER; ++i) {
518      if (i > 0) {
519        // Ensure that the archive directory exists.
520        // The previous "move to archive" operation has failed probably because
521        // the cleaner has removed our archive directory (HBASE-7643).
522        // (we're in a retry loop, so don't worry too much about the exception)
523        try {
524          if (!fs.exists(archiveDir)) {
525            if (fs.mkdirs(archiveDir)) {
526              LOG.debug("Created archive directory {}", archiveDir);
527            }
528          }
529        } catch (IOException e) {
530          LOG.warn("Failed to create directory {}", archiveDir, e);
531        }
532      }
533
534      try {
535        success = currentFile.moveAndClose(archiveFile);
536      } catch (FileNotFoundException fnfe) {
537        LOG.warn("Failed to archive " + currentFile +
538            " because it does not exist! Skipping and continuing on.", fnfe);
539        success = true;
540      } catch (IOException e) {
541        LOG.warn("Failed to archive " + currentFile + " on try #" + i, e);
542        success = false;
543      }
544    }
545
546    if (!success) {
547      LOG.error("Failed to archive " + currentFile);
548      return false;
549    }
550
551    LOG.debug("Archived from {} to {}", currentFile, archiveFile);
552    return true;
553  }
554
555  /**
556   * Without regard for backup, delete a region. Should be used with caution.
557   * @param regionDir {@link Path} to the region to be deleted.
558   * @param fs FileSystem from which to delete the region
559   * @return <tt>true</tt> on successful deletion, <tt>false</tt> otherwise
560   * @throws IOException on filesystem operation failure
561   */
562  private static boolean deleteRegionWithoutArchiving(FileSystem fs, Path regionDir)
563      throws IOException {
564    if (fs.delete(regionDir, true)) {
565      LOG.debug("Deleted {}", regionDir);
566      return true;
567    }
568    LOG.debug("Failed to delete directory {}", regionDir);
569    return false;
570  }
571
572  /**
573   * Just do a simple delete of the given store files
574   * <p>
575   * A best effort is made to delete each of the files, rather than bailing on the first failure.
576   * <p>
577   * @param compactedFiles store files to delete from the file system.
578   * @throws IOException if a file cannot be deleted. All files will be attempted to deleted before
579   *           throwing the exception, rather than failing at the first file.
580   */
581  private static void deleteStoreFilesWithoutArchiving(Collection<HStoreFile> compactedFiles)
582      throws IOException {
583    LOG.debug("Deleting files without archiving.");
584    List<IOException> errors = new ArrayList<>(0);
585    for (HStoreFile hsf : compactedFiles) {
586      try {
587        hsf.deleteStoreFile();
588      } catch (IOException e) {
589        LOG.error("Failed to delete {}", hsf.getPath());
590        errors.add(e);
591      }
592    }
593    if (errors.size() > 0) {
594      throw MultipleIOException.createIOException(errors);
595    }
596  }
597
598  /**
599   * Adapt a type to match the {@link File} interface, which is used internally for handling
600   * archival/removal of files
601   * @param <T> type to adapt to the {@link File} interface
602   */
603  private static abstract class FileConverter<T> implements Function<T, File> {
604    protected final FileSystem fs;
605
606    public FileConverter(FileSystem fs) {
607      this.fs = fs;
608    }
609  }
610
611  /**
612   * Convert a FileStatus to something we can manage in the archiving
613   */
614  private static class FileStatusConverter extends FileConverter<FileStatus> {
615    public FileStatusConverter(FileSystem fs) {
616      super(fs);
617    }
618
619    @Override
620    public File apply(FileStatus input) {
621      return new FileablePath(fs, input.getPath());
622    }
623  }
624
625  /**
626   * Convert the {@link HStoreFile} into something we can manage in the archive
627   * methods
628   */
629  private static class StoreToFile extends FileConverter<HStoreFile> {
630    public StoreToFile(FileSystem fs) {
631      super(fs);
632    }
633
634    @Override
635    public File apply(HStoreFile input) {
636      return new FileableStoreFile(fs, input);
637    }
638  }
639
640  /**
641   * Wrapper to handle file operations uniformly
642   */
643  private static abstract class File {
644    protected final FileSystem fs;
645
646    public File(FileSystem fs) {
647      this.fs = fs;
648    }
649
650    /**
651     * Delete the file
652     * @throws IOException on failure
653     */
654    abstract void delete() throws IOException;
655
656    /**
657     * Check to see if this is a file or a directory
658     * @return <tt>true</tt> if it is a file, <tt>false</tt> otherwise
659     * @throws IOException on {@link FileSystem} connection error
660     */
661    abstract boolean isFile() throws IOException;
662
663    /**
664     * @return if this is a directory, returns all the children in the
665     *         directory, otherwise returns an empty list
666     * @throws IOException
667     */
668    abstract Collection<File> getChildren() throws IOException;
669
670    /**
671     * close any outside readers of the file
672     * @throws IOException
673     */
674    abstract void close() throws IOException;
675
676    /**
677     * @return the name of the file (not the full fs path, just the individual
678     *         file name)
679     */
680    abstract String getName();
681
682    /**
683     * @return the path to this file
684     */
685    abstract Path getPath();
686
687    /**
688     * Move the file to the given destination
689     * @param dest
690     * @return <tt>true</tt> on success
691     * @throws IOException
692     */
693    public boolean moveAndClose(Path dest) throws IOException {
694      this.close();
695      Path p = this.getPath();
696      return CommonFSUtils.renameAndSetModifyTime(fs, p, dest);
697    }
698
699    /**
700     * @return the {@link FileSystem} on which this file resides
701     */
702    public FileSystem getFileSystem() {
703      return this.fs;
704    }
705
706    @Override
707    public String toString() {
708      return this.getClass().getSimpleName() + ", " + getPath().toString();
709    }
710  }
711
712  /**
713   * A {@link File} that wraps a simple {@link Path} on a {@link FileSystem}.
714   */
715  private static class FileablePath extends File {
716    private final Path file;
717    private final FileStatusConverter getAsFile;
718
719    public FileablePath(FileSystem fs, Path file) {
720      super(fs);
721      this.file = file;
722      this.getAsFile = new FileStatusConverter(fs);
723    }
724
725    @Override
726    public void delete() throws IOException {
727      if (!fs.delete(file, true)) throw new IOException("Failed to delete:" + this.file);
728    }
729
730    @Override
731    public String getName() {
732      return file.getName();
733    }
734
735    @Override
736    public Collection<File> getChildren() throws IOException {
737      if (fs.isFile(file)) {
738        return Collections.emptyList();
739      }
740      return Stream.of(fs.listStatus(file)).map(getAsFile).collect(Collectors.toList());
741    }
742
743    @Override
744    public boolean isFile() throws IOException {
745      return fs.isFile(file);
746    }
747
748    @Override
749    public void close() throws IOException {
750      // NOOP - files are implicitly closed on removal
751    }
752
753    @Override
754    Path getPath() {
755      return file;
756    }
757  }
758
759  /**
760   * {@link File} adapter for a {@link HStoreFile} living on a {@link FileSystem}
761   * .
762   */
763  private static class FileableStoreFile extends File {
764    HStoreFile file;
765
766    public FileableStoreFile(FileSystem fs, HStoreFile store) {
767      super(fs);
768      this.file = store;
769    }
770
771    @Override
772    public void delete() throws IOException {
773      file.deleteStoreFile();
774    }
775
776    @Override
777    public String getName() {
778      return file.getPath().getName();
779    }
780
781    @Override
782    public boolean isFile() {
783      return true;
784    }
785
786    @Override
787    public Collection<File> getChildren() throws IOException {
788      // storefiles don't have children
789      return Collections.emptyList();
790    }
791
792    @Override
793    public void close() throws IOException {
794      file.closeStoreFile(true);
795    }
796
797    @Override
798    Path getPath() {
799      return file.getPath();
800    }
801  }
802}